diff --git a/.github/actions/java-test/action.yaml b/.github/actions/java-test/action.yaml index 57a30851cf..be88778368 100644 --- a/.github/actions/java-test/action.yaml +++ b/.github/actions/java-test/action.yaml @@ -32,7 +32,7 @@ inputs: scan_impl: description: 'The default Parquet scan implementation' required: false - default: 'native_comet' + default: 'auto' upload-test-reports: description: 'Whether to upload test results including coverage to GitHub' required: false diff --git a/.github/workflows/iceberg_spark_test.yml b/.github/workflows/iceberg_spark_test.yml index 3452170832..d171956166 100644 --- a/.github/workflows/iceberg_spark_test.yml +++ b/.github/workflows/iceberg_spark_test.yml @@ -69,14 +69,16 @@ jobs: ~/.cargo/registry ~/.cargo/git native/target - key: ${{ runner.os }}-cargo-ci-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }} + key: ${{ runner.os }}-cargo-ci-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }}-${{ hashFiles('native/**/*.rs') }} restore-keys: | - ${{ runner.os }}-cargo-ci- + ${{ runner.os }}-cargo-ci-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }}- - name: Build native library # Use CI profile for faster builds (no LTO) and to share cache with pr_build_linux.yml. run: | cd native && cargo build --profile ci + env: + RUSTFLAGS: "-Ctarget-cpu=x86-64-v3" - name: Save Cargo cache uses: actions/cache/save@v5 @@ -86,7 +88,7 @@ jobs: ~/.cargo/registry ~/.cargo/git native/target - key: ${{ runner.os }}-cargo-ci-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }} + key: ${{ runner.os }}-cargo-ci-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }}-${{ hashFiles('native/**/*.rs') }} - name: Upload native library uses: actions/upload-artifact@v6 diff --git a/.github/workflows/pr_build_linux.yml b/.github/workflows/pr_build_linux.yml index 8376afbc6e..ae433b7db7 100644 --- a/.github/workflows/pr_build_linux.yml +++ b/.github/workflows/pr_build_linux.yml @@ -84,9 +84,9 @@ jobs: ~/.cargo/registry ~/.cargo/git native/target - key: ${{ runner.os }}-cargo-ci-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }} + key: ${{ runner.os }}-cargo-ci-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }}-${{ hashFiles('native/**/*.rs') }} restore-keys: | - ${{ runner.os }}-cargo-ci- + ${{ runner.os }}-cargo-ci-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }}- - name: Build native library (CI profile) run: | @@ -94,6 +94,8 @@ jobs: # CI profile: same overflow behavior as release, but faster compilation # (no LTO, parallel codegen) cargo build --profile ci + env: + RUSTFLAGS: "-Ctarget-cpu=x86-64-v3" - name: Upload native library uses: actions/upload-artifact@v6 @@ -110,7 +112,7 @@ jobs: ~/.cargo/registry ~/.cargo/git native/target - key: ${{ runner.os }}-cargo-ci-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }} + key: ${{ runner.os }}-cargo-ci-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }}-${{ hashFiles('native/**/*.rs') }} # Run Rust tests (runs in parallel with build-native, uses debug builds) linux-test-rust: @@ -136,9 +138,9 @@ jobs: ~/.cargo/git native/target # Note: Java version intentionally excluded - Rust target is JDK-independent - key: ${{ runner.os }}-cargo-debug-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }} + key: ${{ runner.os }}-cargo-debug-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }}-${{ hashFiles('native/**/*.rs') }} restore-keys: | - ${{ runner.os }}-cargo-debug- + ${{ runner.os }}-cargo-debug-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }}- - name: Rust test steps uses: ./.github/actions/rust-test @@ -151,7 +153,7 @@ jobs: ~/.cargo/registry ~/.cargo/git native/target - key: ${{ runner.os }}-cargo-debug-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }} + key: ${{ runner.os }}-cargo-debug-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }}-${{ hashFiles('native/**/*.rs') }} linux-test: needs: build-native @@ -164,7 +166,7 @@ jobs: - name: "Spark 3.4, JDK 11, Scala 2.12" java_version: "11" maven_opts: "-Pspark-3.4 -Pscala-2.12" - scan_impl: "native_comet" + scan_impl: "auto" - name: "Spark 3.5.5, JDK 17, Scala 2.13" java_version: "17" @@ -174,7 +176,7 @@ jobs: - name: "Spark 3.5.6, JDK 17, Scala 2.13" java_version: "17" maven_opts: "-Pspark-3.5 -Dspark.version=3.5.6 -Pscala-2.13" - scan_impl: "native_comet" + scan_impl: "auto" - name: "Spark 3.5, JDK 17, Scala 2.12" java_version: "17" @@ -260,6 +262,7 @@ jobs: org.apache.comet.CometStringExpressionSuite org.apache.comet.CometBitwiseExpressionSuite org.apache.comet.CometMapExpressionSuite + org.apache.comet.CometCsvExpressionSuite org.apache.comet.CometJsonExpressionSuite org.apache.comet.expressions.conditional.CometIfSuite org.apache.comet.expressions.conditional.CometCoalesceSuite diff --git a/.github/workflows/pr_build_macos.yml b/.github/workflows/pr_build_macos.yml index 3a64c0051f..9c4064f5d3 100644 --- a/.github/workflows/pr_build_macos.yml +++ b/.github/workflows/pr_build_macos.yml @@ -84,9 +84,9 @@ jobs: ~/.cargo/registry ~/.cargo/git native/target - key: ${{ runner.os }}-cargo-ci-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }} + key: ${{ runner.os }}-cargo-ci-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }}-${{ hashFiles('native/**/*.rs') }} restore-keys: | - ${{ runner.os }}-cargo-ci- + ${{ runner.os }}-cargo-ci-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }}- - name: Build native library (CI profile) run: | @@ -94,6 +94,8 @@ jobs: # CI profile: same overflow behavior as release, but faster compilation # (no LTO, parallel codegen) cargo build --profile ci + env: + RUSTFLAGS: "-Ctarget-cpu=apple-m1" - name: Upload native library uses: actions/upload-artifact@v6 @@ -110,7 +112,7 @@ jobs: ~/.cargo/registry ~/.cargo/git native/target - key: ${{ runner.os }}-cargo-ci-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }} + key: ${{ runner.os }}-cargo-ci-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }}-${{ hashFiles('native/**/*.rs') }} macos-aarch64-test: needs: build-native @@ -204,6 +206,7 @@ jobs: org.apache.comet.CometBitwiseExpressionSuite org.apache.comet.CometMapExpressionSuite org.apache.comet.CometJsonExpressionSuite + org.apache.comet.CometCsvExpressionSuite org.apache.comet.expressions.conditional.CometIfSuite org.apache.comet.expressions.conditional.CometCoalesceSuite org.apache.comet.expressions.conditional.CometCaseWhenSuite diff --git a/.github/workflows/spark_sql_test.yml b/.github/workflows/spark_sql_test.yml index 2e4b6926c2..1332704701 100644 --- a/.github/workflows/spark_sql_test.yml +++ b/.github/workflows/spark_sql_test.yml @@ -75,14 +75,16 @@ jobs: ~/.cargo/registry ~/.cargo/git native/target - key: ${{ runner.os }}-cargo-ci-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }} + key: ${{ runner.os }}-cargo-ci-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }}-${{ hashFiles('native/**/*.rs') }} restore-keys: | - ${{ runner.os }}-cargo-ci- + ${{ runner.os }}-cargo-ci-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }}- - name: Build native library (CI profile) run: | cd native cargo build --profile ci + env: + RUSTFLAGS: "-Ctarget-cpu=x86-64-v3" - name: Upload native library uses: actions/upload-artifact@v6 @@ -99,7 +101,7 @@ jobs: ~/.cargo/registry ~/.cargo/git native/target - key: ${{ runner.os }}-cargo-ci-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }} + key: ${{ runner.os }}-cargo-ci-${{ hashFiles('native/**/Cargo.lock', 'native/**/Cargo.toml') }}-${{ hashFiles('native/**/*.rs') }} spark-sql-test: needs: build-native @@ -116,18 +118,15 @@ jobs: - {name: "sql_hive-3", args1: "", args2: "hive/testOnly * -- -n org.apache.spark.tags.SlowHiveTest"} # Test combinations: # - auto scan: all Spark versions (3.4, 3.5, 4.0) - # - native_comet: Spark 3.4, 3.5 # - native_iceberg_compat: Spark 3.5 only config: - - {spark-short: '3.4', spark-full: '3.4.3', java: 11, scan-impl: 'auto', scan-env: ''} - - {spark-short: '3.5', spark-full: '3.5.8', java: 11, scan-impl: 'auto', scan-env: ''} - - {spark-short: '4.0', spark-full: '4.0.1', java: 17, scan-impl: 'auto', scan-env: ''} - - {spark-short: '3.4', spark-full: '3.4.3', java: 11, scan-impl: 'native_comet', scan-env: 'COMET_PARQUET_SCAN_IMPL=native_comet'} - - {spark-short: '3.5', spark-full: '3.5.8', java: 11, scan-impl: 'native_comet', scan-env: 'COMET_PARQUET_SCAN_IMPL=native_comet'} - - {spark-short: '3.5', spark-full: '3.5.8', java: 11, scan-impl: 'native_iceberg_compat', scan-env: 'COMET_PARQUET_SCAN_IMPL=native_iceberg_compat'} + - {spark-short: '3.4', spark-full: '3.4.3', java: 11, scan-impl: 'auto'} + - {spark-short: '3.5', spark-full: '3.5.8', java: 11, scan-impl: 'auto'} + - {spark-short: '3.5', spark-full: '3.5.8', java: 11, scan-impl: 'native_datafusion'} + - {spark-short: '4.0', spark-full: '4.0.1', java: 17, scan-impl: 'auto'} # Skip sql_hive-1 for Spark 4.0 due to https://github.com/apache/datafusion-comet/issues/2946 exclude: - - config: {spark-short: '4.0', spark-full: '4.0.1', java: 17, scan-impl: 'auto', scan-env: ''} + - config: {spark-short: '4.0', spark-full: '4.0.1', java: 17, scan-impl: 'auto'} module: {name: "sql_hive-1", args1: "", args2: "hive/testOnly * -- -l org.apache.spark.tags.ExtendedHiveTest -l org.apache.spark.tags.SlowHiveTest"} fail-fast: false name: spark-sql-${{ matrix.config.scan-impl }}-${{ matrix.module.name }}/spark-${{ matrix.config.spark-full }} @@ -156,7 +155,7 @@ jobs: run: | cd apache-spark rm -rf /root/.m2/repository/org/apache/parquet # somehow parquet cache requires cleanups - ENABLE_COMET=true ENABLE_COMET_ONHEAP=true ${{ matrix.config.scan-env }} ENABLE_COMET_LOG_FALLBACK_REASONS=${{ github.event.inputs.collect-fallback-logs || 'false' }} \ + NOLINT_ON_COMPILE=true ENABLE_COMET=true ENABLE_COMET_ONHEAP=true COMET_PARQUET_SCAN_IMPL=${{ matrix.config.scan-impl }} ENABLE_COMET_LOG_FALLBACK_REASONS=${{ github.event.inputs.collect-fallback-logs || 'false' }} \ build/sbt -Dsbt.log.noformat=true ${{ matrix.module.args1 }} "${{ matrix.module.args2 }}" if [ "${{ github.event.inputs.collect-fallback-logs }}" = "true" ]; then find . -type f -name "unit-tests.log" -print0 | xargs -0 grep -h "Comet cannot accelerate" | sed 's/.*Comet cannot accelerate/Comet cannot accelerate/' | sort -u > fallback.log diff --git a/Makefile b/Makefile index a96fdab6ee..60ea51bf0c 100644 --- a/Makefile +++ b/Makefile @@ -51,7 +51,7 @@ format: # build native libs for amd64 architecture Linux/MacOS on a Linux/amd64 machine/container core-amd64-libs: - cd native && cargo build -j 2 --release $(FEATURES_ARG) + cd native && RUSTFLAGS="-Ctarget-cpu=x86-64-v3" cargo build -j 2 --release $(FEATURES_ARG) ifdef HAS_OSXCROSS rustup target add x86_64-apple-darwin cd native && cargo build -j 2 --target x86_64-apple-darwin --release $(FEATURES_ARG) @@ -59,7 +59,7 @@ endif # build native libs for arm64 architecture Linux/MacOS on a Linux/arm64 machine/container core-arm64-libs: - cd native && cargo build -j 2 --release $(FEATURES_ARG) + cd native && RUSTFLAGS="-Ctarget-cpu=neoverse-n1" cargo build -j 2 --release $(FEATURES_ARG) ifdef HAS_OSXCROSS rustup target add aarch64-apple-darwin cd native && cargo build -j 2 --target aarch64-apple-darwin --release $(FEATURES_ARG) @@ -67,10 +67,10 @@ endif core-amd64: rustup target add x86_64-apple-darwin - 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 $(FEATURES_ARG) + cd native && RUSTFLAGS="-Ctarget-cpu=skylake" CC=o64-clang CXX=o64-clang++ cargo build --target x86_64-apple-darwin --release $(FEATURES_ARG) mkdir -p common/target/classes/org/apache/comet/darwin/x86_64 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 $(FEATURES_ARG) + cd native && RUSTFLAGS="-Ctarget-cpu=x86-64-v3" cargo build --release $(FEATURES_ARG) mkdir -p 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 \ @@ -83,7 +83,7 @@ core-arm64: 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 $(FEATURES_ARG) mkdir -p common/target/classes/org/apache/comet/darwin/aarch64 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 $(FEATURES_ARG) + cd native && RUSTFLAGS="-Ctarget-cpu=neoverse-n1" cargo build --release $(FEATURES_ARG) mkdir -p 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 \ @@ -94,8 +94,8 @@ core-arm64: release-linux: clean rustup target add aarch64-apple-darwin x86_64-apple-darwin 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 $(FEATURES_ARG) - 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 $(FEATURES_ARG) - cd native && RUSTFLAGS="-Ctarget-cpu=native -Ctarget-feature=-prefer-256-bit" cargo build --release $(FEATURES_ARG) + cd native && RUSTFLAGS="-Ctarget-cpu=skylake" CC=o64-clang CXX=o64-clang++ cargo build --target x86_64-apple-darwin --release $(FEATURES_ARG) + cd native && RUSTFLAGS="-Ctarget-cpu=native" cargo build --release $(FEATURES_ARG) ./mvnw install -Prelease -DskipTests $(PROFILES) release: cd native && RUSTFLAGS="$(RUSTFLAGS) -Ctarget-cpu=native" cargo build --release $(FEATURES_ARG) diff --git a/common/src/main/java/org/apache/comet/parquet/ArrowConstantColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ArrowConstantColumnReader.java new file mode 100644 index 0000000000..521eb4aa5b --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/ArrowConstantColumnReader.java @@ -0,0 +1,255 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet.parquet; + +import java.math.BigDecimal; + +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.*; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns; +import org.apache.spark.sql.types.*; +import org.apache.spark.unsafe.types.UTF8String; + +import org.apache.comet.vector.CometPlainVector; +import org.apache.comet.vector.CometVector; + +/** + * A column reader that returns constant vectors using Arrow Java vectors directly (no native + * mutable buffers). Used for partition columns and missing columns in the native_iceberg_compat + * scan path. + * + *

The vector is filled with the constant value repeated for every row in the batch. This is + * necessary because the underlying Arrow vector's buffers must be large enough to match the + * reported value count — otherwise variable-width types (strings, binary) would have undersized + * offset buffers, causing out-of-bounds reads on the native side. + */ +public class ArrowConstantColumnReader extends AbstractColumnReader { + private final BufferAllocator allocator = new RootAllocator(); + + private boolean isNull; + private Object value; + private FieldVector fieldVector; + private CometPlainVector vector; + private int currentSize; + + /** Constructor for missing columns (default values from schema). */ + ArrowConstantColumnReader(StructField field, int batchSize, boolean useDecimal128) { + super(field.dataType(), TypeUtil.convertToParquet(field), useDecimal128, false); + this.batchSize = batchSize; + this.value = + ResolveDefaultColumns.getExistenceDefaultValues(new StructType(new StructField[] {field}))[ + 0]; + initVector(value, batchSize); + } + + /** Constructor for partition columns with values from a row. */ + ArrowConstantColumnReader( + StructField field, int batchSize, InternalRow values, int index, boolean useDecimal128) { + super(field.dataType(), TypeUtil.convertToParquet(field), useDecimal128, false); + this.batchSize = batchSize; + Object v = values.get(index, field.dataType()); + this.value = v; + initVector(v, batchSize); + } + + @Override + public void setBatchSize(int batchSize) { + close(); + this.batchSize = batchSize; + initVector(value, batchSize); + } + + @Override + public void readBatch(int total) { + if (total != currentSize) { + close(); + initVector(value, total); + } + } + + @Override + public CometVector currentBatch() { + return vector; + } + + @Override + public void close() { + if (vector != null) { + vector.close(); + vector = null; + } + if (fieldVector != null) { + fieldVector.close(); + fieldVector = null; + } + } + + private void initVector(Object value, int count) { + currentSize = count; + if (value == null) { + isNull = true; + fieldVector = createNullVector(count); + } else { + isNull = false; + fieldVector = createFilledVector(value, count); + } + vector = new CometPlainVector(fieldVector, useDecimal128, false, true); + } + + /** Creates a vector of the correct type with {@code count} null values. */ + private FieldVector createNullVector(int count) { + String name = "constant"; + FieldVector v; + if (type == DataTypes.BooleanType) { + v = new BitVector(name, allocator); + } else if (type == DataTypes.ByteType) { + v = new TinyIntVector(name, allocator); + } else if (type == DataTypes.ShortType) { + v = new SmallIntVector(name, allocator); + } else if (type == DataTypes.IntegerType || type == DataTypes.DateType) { + v = new IntVector(name, allocator); + } else if (type == DataTypes.LongType + || type == DataTypes.TimestampType + || type == TimestampNTZType$.MODULE$) { + v = new BigIntVector(name, allocator); + } else if (type == DataTypes.FloatType) { + v = new Float4Vector(name, allocator); + } else if (type == DataTypes.DoubleType) { + v = new Float8Vector(name, allocator); + } else if (type == DataTypes.BinaryType) { + v = new VarBinaryVector(name, allocator); + } else if (type == DataTypes.StringType) { + v = new VarCharVector(name, allocator); + } else if (type instanceof DecimalType) { + DecimalType dt = (DecimalType) type; + if (!useDecimal128 && dt.precision() <= Decimal.MAX_INT_DIGITS()) { + v = new IntVector(name, allocator); + } else if (!useDecimal128 && dt.precision() <= Decimal.MAX_LONG_DIGITS()) { + v = new BigIntVector(name, allocator); + } else { + v = new DecimalVector(name, allocator, dt.precision(), dt.scale()); + } + } else { + throw new UnsupportedOperationException("Unsupported Spark type: " + type); + } + v.setValueCount(count); + return v; + } + + /** Creates a vector filled with {@code count} copies of the given value. */ + private FieldVector createFilledVector(Object value, int count) { + String name = "constant"; + if (type == DataTypes.BooleanType) { + BitVector v = new BitVector(name, allocator); + v.allocateNew(count); + int bit = (boolean) value ? 1 : 0; + for (int i = 0; i < count; i++) v.setSafe(i, bit); + v.setValueCount(count); + return v; + } else if (type == DataTypes.ByteType) { + TinyIntVector v = new TinyIntVector(name, allocator); + v.allocateNew(count); + byte val = (byte) value; + for (int i = 0; i < count; i++) v.setSafe(i, val); + v.setValueCount(count); + return v; + } else if (type == DataTypes.ShortType) { + SmallIntVector v = new SmallIntVector(name, allocator); + v.allocateNew(count); + short val = (short) value; + for (int i = 0; i < count; i++) v.setSafe(i, val); + v.setValueCount(count); + return v; + } else if (type == DataTypes.IntegerType || type == DataTypes.DateType) { + IntVector v = new IntVector(name, allocator); + v.allocateNew(count); + int val = (int) value; + for (int i = 0; i < count; i++) v.setSafe(i, val); + v.setValueCount(count); + return v; + } else if (type == DataTypes.LongType + || type == DataTypes.TimestampType + || type == TimestampNTZType$.MODULE$) { + BigIntVector v = new BigIntVector(name, allocator); + v.allocateNew(count); + long val = (long) value; + for (int i = 0; i < count; i++) v.setSafe(i, val); + v.setValueCount(count); + return v; + } else if (type == DataTypes.FloatType) { + Float4Vector v = new Float4Vector(name, allocator); + v.allocateNew(count); + float val = (float) value; + for (int i = 0; i < count; i++) v.setSafe(i, val); + v.setValueCount(count); + return v; + } else if (type == DataTypes.DoubleType) { + Float8Vector v = new Float8Vector(name, allocator); + v.allocateNew(count); + double val = (double) value; + for (int i = 0; i < count; i++) v.setSafe(i, val); + v.setValueCount(count); + return v; + } else if (type == DataTypes.BinaryType) { + VarBinaryVector v = new VarBinaryVector(name, allocator); + v.allocateNew(count); + byte[] bytes = (byte[]) value; + for (int i = 0; i < count; i++) v.setSafe(i, bytes, 0, bytes.length); + v.setValueCount(count); + return v; + } else if (type == DataTypes.StringType) { + VarCharVector v = new VarCharVector(name, allocator); + v.allocateNew(count); + byte[] bytes = ((UTF8String) value).getBytes(); + for (int i = 0; i < count; i++) v.setSafe(i, bytes, 0, bytes.length); + v.setValueCount(count); + return v; + } else if (type instanceof DecimalType) { + DecimalType dt = (DecimalType) type; + Decimal d = (Decimal) value; + if (!useDecimal128 && dt.precision() <= Decimal.MAX_INT_DIGITS()) { + IntVector v = new IntVector(name, allocator); + v.allocateNew(count); + int val = (int) d.toUnscaledLong(); + for (int i = 0; i < count; i++) v.setSafe(i, val); + v.setValueCount(count); + return v; + } else if (!useDecimal128 && dt.precision() <= Decimal.MAX_LONG_DIGITS()) { + BigIntVector v = new BigIntVector(name, allocator); + v.allocateNew(count); + long val = d.toUnscaledLong(); + for (int i = 0; i < count; i++) v.setSafe(i, val); + v.setValueCount(count); + return v; + } else { + DecimalVector v = new DecimalVector(name, allocator, dt.precision(), dt.scale()); + v.allocateNew(count); + BigDecimal bd = d.toJavaBigDecimal(); + for (int i = 0; i < count; i++) v.setSafe(i, bd); + v.setValueCount(count); + return v; + } + } else { + throw new UnsupportedOperationException("Unsupported Spark type: " + type); + } + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/ArrowRowIndexColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ArrowRowIndexColumnReader.java new file mode 100644 index 0000000000..7d17e551df --- /dev/null +++ b/common/src/main/java/org/apache/comet/parquet/ArrowRowIndexColumnReader.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet.parquet; + +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.BigIntVector; +import org.apache.spark.sql.types.*; + +import org.apache.comet.vector.CometPlainVector; +import org.apache.comet.vector.CometVector; + +/** + * A column reader that computes row indices in Java and creates Arrow BigIntVectors directly (no + * native mutable buffers). Used for the row index metadata column in the native_iceberg_compat scan + * path. + * + *

The {@code indices} array contains alternating pairs of (start_index, count) representing + * ranges of sequential row indices within each row group. + */ +public class ArrowRowIndexColumnReader extends AbstractColumnReader { + private final BufferAllocator allocator = new RootAllocator(); + + /** Alternating (start_index, count) pairs from row groups. */ + private final long[] indices; + + /** Number of row indices consumed so far across batches. */ + private long offset; + + private BigIntVector fieldVector; + private CometPlainVector vector; + + public ArrowRowIndexColumnReader(StructField field, int batchSize, long[] indices) { + super(field.dataType(), TypeUtil.convertToParquet(field), false, false); + this.indices = indices; + this.batchSize = batchSize; + } + + @Override + public void setBatchSize(int batchSize) { + close(); + this.batchSize = batchSize; + } + + @Override + public void readBatch(int total) { + close(); + + fieldVector = new BigIntVector("row_index", allocator); + fieldVector.allocateNew(total); + + // Port of Rust set_indices: iterate (start, count) pairs, skip offset rows, fill up to total. + long skipped = 0; + int filled = 0; + for (int i = 0; i < indices.length && filled < total; i += 2) { + long index = indices[i]; + long count = indices[i + 1]; + long skip = Math.min(count, offset - skipped); + skipped += skip; + if (count == skip) { + continue; + } + long remaining = Math.min(count - skip, total - filled); + for (long j = 0; j < remaining; j++) { + fieldVector.setSafe(filled, index + skip + j); + filled++; + } + } + offset += filled; + + fieldVector.setValueCount(filled); + vector = new CometPlainVector(fieldVector, false, false, false); + vector.setNumValues(filled); + } + + @Override + public CometVector currentBatch() { + return vector; + } + + @Override + public void close() { + if (vector != null) { + vector.close(); + vector = null; + } + if (fieldVector != null) { + fieldVector.close(); + fieldVector = null; + } + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/NativeBatchReader.java b/common/src/main/java/org/apache/comet/parquet/NativeBatchReader.java index d10a8932be..32edcb2640 100644 --- a/common/src/main/java/org/apache/comet/parquet/NativeBatchReader.java +++ b/common/src/main/java/org/apache/comet/parquet/NativeBatchReader.java @@ -448,7 +448,8 @@ public void init() throws Throwable { // TODO(SPARK-40059): Allow users to include columns named // FileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME in their schemas. long[] rowIndices = FileReader.getRowIndices(blocks); - columnReaders[i] = new RowIndexColumnReader(nonPartitionFields[i], capacity, rowIndices); + columnReaders[i] = + new ArrowRowIndexColumnReader(nonPartitionFields[i], capacity, rowIndices); hasRowIndexColumn = true; missingColumns[i] = true; } else if (optFileField.isPresent()) { @@ -473,8 +474,8 @@ public void init() throws Throwable { + filePath); } if (field.isPrimitive()) { - ConstantColumnReader reader = - new ConstantColumnReader(nonPartitionFields[i], capacity, useDecimal128); + ArrowConstantColumnReader reader = + new ArrowConstantColumnReader(nonPartitionFields[i], capacity, useDecimal128); columnReaders[i] = reader; missingColumns[i] = true; } else { @@ -492,8 +493,9 @@ public void init() throws Throwable { for (int i = fields.size(); i < columnReaders.length; i++) { int fieldIndex = i - fields.size(); StructField field = partitionFields[fieldIndex]; - ConstantColumnReader reader = - new ConstantColumnReader(field, capacity, partitionValues, fieldIndex, useDecimal128); + ArrowConstantColumnReader reader = + new ArrowConstantColumnReader( + field, capacity, partitionValues, fieldIndex, useDecimal128); columnReaders[i] = reader; } } diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 6504c0294b..522ccbc94c 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -124,11 +124,8 @@ object CometConf extends ShimCometConf { val COMET_NATIVE_SCAN_IMPL: ConfigEntry[String] = conf("spark.comet.scan.impl") .category(CATEGORY_SCAN) .doc( - s"The implementation of Comet Native Scan to use. Available modes are `$SCAN_NATIVE_COMET`," + + "The implementation of Comet Native Scan to use. Available modes are " + s"`$SCAN_NATIVE_DATAFUSION`, and `$SCAN_NATIVE_ICEBERG_COMPAT`. " + - s"`$SCAN_NATIVE_COMET` (DEPRECATED - will be removed in a future release) is for the " + - "original Comet native scan which uses a jvm based parquet file reader and native " + - "column decoding. Supports simple types only. " + s"`$SCAN_NATIVE_DATAFUSION` is a fully native implementation of scan based on " + "DataFusion. " + s"`$SCAN_NATIVE_ICEBERG_COMPAT` is the recommended native implementation that " + @@ -137,8 +134,7 @@ object CometConf extends ShimCometConf { .internal() .stringConf .transform(_.toLowerCase(Locale.ROOT)) - .checkValues( - Set(SCAN_NATIVE_COMET, SCAN_NATIVE_DATAFUSION, SCAN_NATIVE_ICEBERG_COMPAT, SCAN_AUTO)) + .checkValues(Set(SCAN_NATIVE_DATAFUSION, SCAN_NATIVE_ICEBERG_COMPAT, SCAN_AUTO)) .createWithEnvVarOrDefault("COMET_PARQUET_SCAN_IMPL", SCAN_AUTO) val COMET_ICEBERG_NATIVE_ENABLED: ConfigEntry[Boolean] = diff --git a/dev/benchmarks/README.md b/dev/benchmarks/README.md index 2ef7a9a260..b3ea674199 100644 --- a/dev/benchmarks/README.md +++ b/dev/benchmarks/README.md @@ -73,3 +73,79 @@ Generating charts: ```shell python3 generate-comparison.py --benchmark tpch --labels "Spark 3.5.3" "Comet 0.9.0" "Gluten 1.4.0" --title "TPC-H @ 100 GB (single executor, 8 cores, local Parquet files)" spark-tpch-1752338506381.json comet-tpch-1752337818039.json gluten-tpch-1752337474344.json ``` + +## Iceberg Benchmarking + +Comet includes native Iceberg support via iceberg-rust integration. This enables benchmarking TPC-H queries +against Iceberg tables with native scan acceleration. + +### Prerequisites + +Download the Iceberg Spark runtime JAR (required for running the benchmark): + +```shell +wget https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-spark-runtime-3.5_2.12/1.8.1/iceberg-spark-runtime-3.5_2.12-1.8.1.jar +export ICEBERG_JAR=/path/to/iceberg-spark-runtime-3.5_2.12-1.8.1.jar +``` + +Note: Table creation uses `--packages` which auto-downloads the dependency. + +### Create Iceberg TPC-H tables + +Convert existing Parquet TPC-H data to Iceberg format: + +```shell +export ICEBERG_WAREHOUSE=/mnt/bigdata/iceberg-warehouse +export ICEBERG_CATALOG=${ICEBERG_CATALOG:-local} + +$SPARK_HOME/bin/spark-submit \ + --master $SPARK_MASTER \ + --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.8.1 \ + --conf spark.driver.memory=8G \ + --conf spark.executor.instances=1 \ + --conf spark.executor.cores=8 \ + --conf spark.cores.max=8 \ + --conf spark.executor.memory=16g \ + --conf spark.sql.catalog.${ICEBERG_CATALOG}=org.apache.iceberg.spark.SparkCatalog \ + --conf spark.sql.catalog.${ICEBERG_CATALOG}.type=hadoop \ + --conf spark.sql.catalog.${ICEBERG_CATALOG}.warehouse=$ICEBERG_WAREHOUSE \ + create-iceberg-tpch.py \ + --parquet-path $TPCH_DATA \ + --catalog $ICEBERG_CATALOG \ + --database tpch +``` + +### Run Iceberg benchmark + +```shell +export JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64 +export COMET_JAR=/opt/comet/comet-spark-spark3.5_2.12-0.10.0.jar +export ICEBERG_JAR=/path/to/iceberg-spark-runtime-3.5_2.12-1.8.1.jar +export ICEBERG_WAREHOUSE=/mnt/bigdata/iceberg-warehouse +export TPCH_QUERIES=/mnt/bigdata/tpch/queries/ +sudo ./drop-caches.sh +./comet-tpch-iceberg.sh +``` + +The benchmark uses `spark.comet.scan.icebergNative.enabled=true` to enable Comet's native iceberg-rust +integration. Verify native scanning is active by checking for `CometIcebergNativeScanExec` in the +physical plan output. + +### Iceberg-specific options + +| Environment Variable | Default | Description | +| -------------------- | ---------- | ----------------------------------- | +| `ICEBERG_CATALOG` | `local` | Iceberg catalog name | +| `ICEBERG_DATABASE` | `tpch` | Database containing TPC-H tables | +| `ICEBERG_WAREHOUSE` | (required) | Path to Iceberg warehouse directory | + +### Comparing Parquet vs Iceberg performance + +Run both benchmarks and compare: + +```shell +python3 generate-comparison.py --benchmark tpch \ + --labels "Comet (Parquet)" "Comet (Iceberg)" \ + --title "TPC-H @ 100 GB: Parquet vs Iceberg" \ + comet-tpch-*.json comet-iceberg-tpch-*.json +``` diff --git a/dev/benchmarks/comet-tpcds.sh b/dev/benchmarks/comet-tpcds.sh index 86cc12b02c..b55b27188c 100755 --- a/dev/benchmarks/comet-tpcds.sh +++ b/dev/benchmarks/comet-tpcds.sh @@ -40,6 +40,7 @@ $SPARK_HOME/bin/spark-submit \ --conf spark.executor.extraClassPath=$COMET_JAR \ --conf spark.plugins=org.apache.spark.CometPlugin \ --conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \ + --conf spark.comet.scan.impl=native_datafusion \ --conf spark.comet.expression.Cast.allowIncompatible=true \ --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ diff --git a/dev/benchmarks/comet-tpch-iceberg.sh b/dev/benchmarks/comet-tpch-iceberg.sh new file mode 100755 index 0000000000..7907125c82 --- /dev/null +++ b/dev/benchmarks/comet-tpch-iceberg.sh @@ -0,0 +1,114 @@ +#!/bin/bash +# +# 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. +# + +# TPC-H benchmark using Iceberg tables with Comet's native iceberg-rust integration. +# +# Required environment variables: +# SPARK_HOME - Path to Spark installation +# SPARK_MASTER - Spark master URL (e.g., spark://localhost:7077) +# COMET_JAR - Path to Comet JAR +# ICEBERG_JAR - Path to Iceberg Spark runtime JAR +# ICEBERG_WAREHOUSE - Path to Iceberg warehouse directory +# TPCH_QUERIES - Path to TPC-H query files +# +# Optional: +# ICEBERG_CATALOG - Catalog name (default: local) +# ICEBERG_DATABASE - Database name (default: tpch) +# +# Setup (run once to create Iceberg tables from Parquet): +# $SPARK_HOME/bin/spark-submit \ +# --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.8.1 \ +# --conf spark.sql.catalog.local=org.apache.iceberg.spark.SparkCatalog \ +# --conf spark.sql.catalog.local.type=hadoop \ +# --conf spark.sql.catalog.local.warehouse=$ICEBERG_WAREHOUSE \ +# create-iceberg-tpch.py \ +# --parquet-path $TPCH_DATA \ +# --catalog local \ +# --database tpch + +set -e + +# Defaults +ICEBERG_CATALOG=${ICEBERG_CATALOG:-local} +ICEBERG_DATABASE=${ICEBERG_DATABASE:-tpch} + +# Validate required variables +if [ -z "$SPARK_HOME" ]; then + echo "Error: SPARK_HOME is not set" + exit 1 +fi +if [ -z "$COMET_JAR" ]; then + echo "Error: COMET_JAR is not set" + exit 1 +fi +if [ -z "$ICEBERG_JAR" ]; then + echo "Error: ICEBERG_JAR is not set" + echo "Download from: https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-spark-runtime-3.5_2.12/1.8.1/" + exit 1 +fi +if [ -z "$ICEBERG_WAREHOUSE" ]; then + echo "Error: ICEBERG_WAREHOUSE is not set" + exit 1 +fi +if [ -z "$TPCH_QUERIES" ]; then + echo "Error: TPCH_QUERIES is not set" + exit 1 +fi + +$SPARK_HOME/sbin/stop-master.sh 2>/dev/null || true +$SPARK_HOME/sbin/stop-worker.sh 2>/dev/null || true + +$SPARK_HOME/sbin/start-master.sh +$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER + +$SPARK_HOME/bin/spark-submit \ + --master $SPARK_MASTER \ + --jars $COMET_JAR,$ICEBERG_JAR \ + --driver-class-path $COMET_JAR:$ICEBERG_JAR \ + --conf spark.driver.memory=8G \ + --conf spark.executor.instances=1 \ + --conf spark.executor.cores=8 \ + --conf spark.cores.max=8 \ + --conf spark.executor.memory=16g \ + --conf spark.memory.offHeap.enabled=true \ + --conf spark.memory.offHeap.size=16g \ + --conf spark.eventLog.enabled=true \ + --conf spark.driver.extraClassPath=$COMET_JAR:$ICEBERG_JAR \ + --conf spark.executor.extraClassPath=$COMET_JAR:$ICEBERG_JAR \ + --conf spark.plugins=org.apache.spark.CometPlugin \ + --conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \ + --conf spark.comet.exec.replaceSortMergeJoin=true \ + --conf spark.comet.expression.Cast.allowIncompatible=true \ + --conf spark.comet.enabled=true \ + --conf spark.comet.exec.enabled=true \ + --conf spark.comet.scan.icebergNative.enabled=true \ + --conf spark.comet.explainFallback.enabled=true \ + --conf spark.sql.catalog.${ICEBERG_CATALOG}=org.apache.iceberg.spark.SparkCatalog \ + --conf spark.sql.catalog.${ICEBERG_CATALOG}.type=hadoop \ + --conf spark.sql.catalog.${ICEBERG_CATALOG}.warehouse=$ICEBERG_WAREHOUSE \ + --conf spark.sql.defaultCatalog=${ICEBERG_CATALOG} \ + tpcbench.py \ + --name comet-iceberg \ + --benchmark tpch \ + --catalog $ICEBERG_CATALOG \ + --database $ICEBERG_DATABASE \ + --queries $TPCH_QUERIES \ + --output . \ + --iterations 1 diff --git a/dev/benchmarks/comet-tpch.sh b/dev/benchmarks/comet-tpch.sh index f0709b7ef0..a748a02319 100755 --- a/dev/benchmarks/comet-tpch.sh +++ b/dev/benchmarks/comet-tpch.sh @@ -40,6 +40,7 @@ $SPARK_HOME/bin/spark-submit \ --conf spark.executor.extraClassPath=$COMET_JAR \ --conf spark.plugins=org.apache.spark.CometPlugin \ --conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \ + --conf spark.comet.scan.impl=native_datafusion \ --conf spark.comet.exec.replaceSortMergeJoin=true \ --conf spark.comet.expression.Cast.allowIncompatible=true \ --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ diff --git a/dev/benchmarks/create-iceberg-tpch.py b/dev/benchmarks/create-iceberg-tpch.py new file mode 100644 index 0000000000..44f0f63a2e --- /dev/null +++ b/dev/benchmarks/create-iceberg-tpch.py @@ -0,0 +1,88 @@ +# 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. + +""" +Convert TPC-H Parquet data to Iceberg tables. + +Usage: + spark-submit \ + --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.8.1 \ + --conf spark.sql.catalog.local=org.apache.iceberg.spark.SparkCatalog \ + --conf spark.sql.catalog.local.type=hadoop \ + --conf spark.sql.catalog.local.warehouse=/path/to/iceberg-warehouse \ + create-iceberg-tpch.py \ + --parquet-path /path/to/tpch/parquet \ + --catalog local \ + --database tpch +""" + +import argparse +from pyspark.sql import SparkSession +import time + + +def main(parquet_path: str, catalog: str, database: str): + spark = SparkSession.builder \ + .appName("Create Iceberg TPC-H Tables") \ + .getOrCreate() + + table_names = [ + "customer", + "lineitem", + "nation", + "orders", + "part", + "partsupp", + "region", + "supplier" + ] + + # Create database if it doesn't exist + spark.sql(f"CREATE DATABASE IF NOT EXISTS {catalog}.{database}") + + for table in table_names: + parquet_table_path = f"{parquet_path}/{table}.parquet" + iceberg_table = f"{catalog}.{database}.{table}" + + print(f"Converting {parquet_table_path} -> {iceberg_table}") + start_time = time.time() + + # Drop table if exists to allow re-running + spark.sql(f"DROP TABLE IF EXISTS {iceberg_table}") + + # Read parquet and write as Iceberg + df = spark.read.parquet(parquet_table_path) + df.writeTo(iceberg_table).using("iceberg").create() + + row_count = spark.table(iceberg_table).count() + elapsed = time.time() - start_time + print(f" Created {iceberg_table} with {row_count} rows in {elapsed:.2f}s") + + print("\nAll TPC-H tables created successfully!") + print(f"Tables available at: {catalog}.{database}.*") + + spark.stop() + + +if __name__ == "__main__": + parser = argparse.ArgumentParser(description="Convert TPC-H Parquet data to Iceberg tables") + parser.add_argument("--parquet-path", required=True, help="Path to TPC-H Parquet data directory") + parser.add_argument("--catalog", required=True, help="Iceberg catalog name (e.g., 'local')") + parser.add_argument("--database", default="tpch", help="Database name to create tables in") + args = parser.parse_args() + + main(args.parquet_path, args.catalog, args.database) diff --git a/dev/benchmarks/tpcbench.py b/dev/benchmarks/tpcbench.py index 130db7a628..400ccd175a 100644 --- a/dev/benchmarks/tpcbench.py +++ b/dev/benchmarks/tpcbench.py @@ -15,6 +15,14 @@ # specific language governing permissions and limitations # under the License. +""" +TPC-H / TPC-DS benchmark runner. + +Supports two data sources: + - Files: use --data with --format (parquet, csv, json) and optional --options + - Iceberg tables: use --catalog and --database to specify the catalog location +""" + import argparse from datetime import datetime import json @@ -22,11 +30,9 @@ import time from typing import Dict -# rename same columns aliases -# a, a, b, b -> a, a_1, b, b_1 -# -# Important for writing data where column name uniqueness is required + def dedup_columns(df): + """Rename duplicate column aliases: a, a, b, b -> a, a_1, b, b_1""" counts = {} new_cols = [] for c in df.columns: @@ -38,30 +44,59 @@ def dedup_columns(df): new_cols.append(f"{c}_{counts[c]}") return df.toDF(*new_cols) -def main(benchmark: str, data_path: str, query_path: str, iterations: int, output: str, name: str, format: str, query_num: int = None, write_path: str = None, options: Dict[str, str] = None): - # Initialize a SparkSession +def main( + benchmark: str, + data_path: str, + catalog: str, + database: str, + query_path: str, + iterations: int, + output: str, + name: str, + format: str, + query_num: int = None, + write_path: str = None, + options: Dict[str, str] = None +): + if options is None: + options = {} + spark = SparkSession.builder \ .appName(f"{name} benchmark derived from {benchmark}") \ .getOrCreate() - # Register the tables + # Define tables for each benchmark if benchmark == "tpch": num_queries = 22 - table_names = ["customer", "lineitem", "nation", "orders", "part", "partsupp", "region", "supplier"] + table_names = [ + "customer", "lineitem", "nation", "orders", + "part", "partsupp", "region", "supplier" + ] elif benchmark == "tpcds": num_queries = 99 - table_names = ["call_center", "catalog_page", "catalog_returns", "catalog_sales", "customer", - "customer_address", "customer_demographics", "date_dim", "time_dim", "household_demographics", - "income_band", "inventory", "item", "promotion", "reason", "ship_mode", "store", "store_returns", - "store_sales", "warehouse", "web_page", "web_returns", "web_sales", "web_site"] + table_names = [ + "call_center", "catalog_page", "catalog_returns", "catalog_sales", + "customer", "customer_address", "customer_demographics", "date_dim", + "time_dim", "household_demographics", "income_band", "inventory", + "item", "promotion", "reason", "ship_mode", "store", "store_returns", + "store_sales", "warehouse", "web_page", "web_returns", "web_sales", + "web_site" + ] else: - raise "invalid benchmark" + raise ValueError(f"Invalid benchmark: {benchmark}") + # Register tables from either files or Iceberg catalog + using_iceberg = catalog is not None for table in table_names: - path = f"{data_path}/{table}.{format}" - print(f"Registering table {table} using path {path}") - df = spark.read.format(format).options(**options).load(path) + if using_iceberg: + source = f"{catalog}.{database}.{table}" + print(f"Registering table {table} from {source}") + df = spark.table(source) + else: + source = f"{data_path}/{table}.{format}" + print(f"Registering table {table} from {source}") + df = spark.read.format(format).options(**options).load(source) df.createOrReplaceTempView(table) conf_dict = {k: v for k, v in spark.sparkContext.getConf().getAll()} @@ -69,95 +104,154 @@ def main(benchmark: str, data_path: str, query_path: str, iterations: int, outpu results = { 'engine': 'datafusion-comet', 'benchmark': benchmark, - 'data_path': data_path, 'query_path': query_path, 'spark_conf': conf_dict, } + if using_iceberg: + results['catalog'] = catalog + results['database'] = database + else: + results['data_path'] = data_path - for iteration in range(0, iterations): - print(f"Starting iteration {iteration} of {iterations}") + for iteration in range(iterations): + print(f"\n{'='*60}") + print(f"Starting iteration {iteration + 1} of {iterations}") + print(f"{'='*60}") iter_start_time = time.time() # Determine which queries to run if query_num is not None: - # Validate query number if query_num < 1 or query_num > num_queries: - raise ValueError(f"Query number {query_num} is out of range. Valid range is 1-{num_queries} for {benchmark}") + raise ValueError( + f"Query number {query_num} out of range. " + f"Valid: 1-{num_queries} for {benchmark}" + ) queries_to_run = [query_num] else: - queries_to_run = range(1, num_queries+1) + queries_to_run = range(1, num_queries + 1) for query in queries_to_run: spark.sparkContext.setJobDescription(f"{benchmark} q{query}") - # read text file path = f"{query_path}/q{query}.sql" + print(f"\nRunning query {query} from {path}") - print(f"Reading query {query} using path {path}") with open(path, "r") as f: text = f.read() - # each file can contain multiple queries queries = text.split(";") start_time = time.time() for sql in queries: sql = sql.strip().replace("create view", "create temp view") if len(sql) > 0: - print(f"Executing: {sql}") + print(f"Executing: {sql[:100]}...") df = spark.sql(sql) df.explain("formatted") if write_path is not None: - # skip results with empty schema - # coming across for running DDL stmt if len(df.columns) > 0: output_path = f"{write_path}/q{query}" - # rename same column names for output - # a, a, b, b => a, a_1, b, b_1 - # output doesn't allow non unique column names deduped = dedup_columns(df) - # sort by all columns to have predictable output dataset for comparison deduped.orderBy(*deduped.columns).coalesce(1).write.mode("overwrite").parquet(output_path) - print(f"Query {query} results written to {output_path}") - else: - print(f"Skipping write: DataFrame has no schema for {output_path}") + print(f"Results written to {output_path}") else: rows = df.collect() print(f"Query {query} returned {len(rows)} rows") end_time = time.time() - print(f"Query {query} took {end_time - start_time} seconds") + elapsed = end_time - start_time + print(f"Query {query} took {elapsed:.2f} seconds") - # store timings in list and later add option to run > 1 iterations query_timings = results.setdefault(query, []) - query_timings.append(end_time - start_time) + query_timings.append(elapsed) iter_end_time = time.time() - print(f"Iteration {iteration} took {round(iter_end_time - iter_start_time,2)} seconds") + print(f"\nIteration {iteration + 1} took {iter_end_time - iter_start_time:.2f} seconds") - str = json.dumps(results, indent=4) + # Write results + result_str = json.dumps(results, indent=4) current_time_millis = int(datetime.now().timestamp() * 1000) results_path = f"{output}/{name}-{benchmark}-{current_time_millis}.json" - print(f"Writing results to {results_path}") + print(f"\nWriting results to {results_path}") with open(results_path, "w") as f: - f.write(str) + f.write(result_str) - # Stop the SparkSession spark.stop() + if __name__ == "__main__": - parser = argparse.ArgumentParser(description="DataFusion benchmark derived from TPC-H / TPC-DS") - parser.add_argument("--benchmark", required=True, default="tpch", help="Benchmark to run (tpch or tpcds)") - parser.add_argument("--data", required=True, help="Path to data files") - parser.add_argument("--queries", required=True, help="Path to query files") - parser.add_argument("--iterations", required=False, default="1", help="How many iterations to run") - parser.add_argument("--output", required=True, help="Path to write output") - parser.add_argument("--name", required=True, help="Prefix for result file e.g. spark/comet/gluten") - parser.add_argument("--query", required=False, type=int, help="Specific query number to run (1-based). If not specified, all queries will be run.") - parser.add_argument("--write", required=False, help="Path to save query results to, in Parquet format.") - parser.add_argument("--format", required=True, default="parquet", help="Input file format (parquet, csv, json)") - parser.add_argument("--options", type=json.loads, required=False, default={}, help='Spark options as JSON string, e.g., \'{"header": "true", "delimiter": ","}\'') - args = parser.parse_args() + parser = argparse.ArgumentParser( + description="TPC-H/TPC-DS benchmark runner for files or Iceberg tables" + ) + parser.add_argument( + "--benchmark", required=True, + help="Benchmark to run (tpch or tpcds)" + ) + + # Data source - mutually exclusive: either file path or Iceberg catalog + source_group = parser.add_mutually_exclusive_group(required=True) + source_group.add_argument( + "--data", + help="Path to data files" + ) + source_group.add_argument( + "--catalog", + help="Iceberg catalog name" + ) - main(args.benchmark, args.data, args.queries, int(args.iterations), args.output, args.name, args.format, args.query, args.write, args.options) + # Options for file-based reading + parser.add_argument( + "--format", default="parquet", + help="Input file format: parquet, csv, json (only used with --data)" + ) + parser.add_argument( + "--options", type=json.loads, default={}, + help='Spark reader options as JSON string, e.g., \'{"header": "true"}\' (only used with --data)' + ) + + # Options for Iceberg + parser.add_argument( + "--database", default="tpch", + help="Database containing TPC tables (only used with --catalog)" + ) + + parser.add_argument( + "--queries", required=True, + help="Path to query SQL files" + ) + parser.add_argument( + "--iterations", type=int, default=1, + help="Number of iterations" + ) + parser.add_argument( + "--output", required=True, + help="Path to write results JSON" + ) + parser.add_argument( + "--name", required=True, + help="Prefix for result file" + ) + parser.add_argument( + "--query", type=int, + help="Specific query number (1-based). If omitted, run all." + ) + parser.add_argument( + "--write", + help="Path to save query results as Parquet" + ) + args = parser.parse_args() + main( + args.benchmark, + args.data, + args.catalog, + args.database, + args.queries, + args.iterations, + args.output, + args.name, + args.format, + args.query, + args.write, + args.options + ) diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index c023f4149a..0205888433 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -1150,14 +1150,14 @@ index 02990a7a40d..bddf5e1ccc2 100644 } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala -index cfc8b2cc845..c6fcfd7bd08 100644 +index cfc8b2cc845..c4be7eb3731 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala @@ -21,6 +21,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkConf import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -+import org.apache.spark.sql.comet.CometScanExec ++import org.apache.spark.sql.comet.{CometNativeScanExec, CometScanExec} import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, Table, TableCapability} import org.apache.spark.sql.connector.read.ScanBuilder import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} @@ -1167,7 +1167,7 @@ index cfc8b2cc845..c6fcfd7bd08 100644 assert( - df.queryExecution.executedPlan.exists(_.isInstanceOf[FileSourceScanExec])) + df.queryExecution.executedPlan.exists { -+ case _: FileSourceScanExec | _: CometScanExec => true ++ case _: FileSourceScanExec | _: CometScanExec | _: CometNativeScanExec => true + case _ => false + } + ) @@ -2385,7 +2385,7 @@ index d083cac48ff..3c11bcde807 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -index 266bb343526..e58a2f49eb9 100644 +index 266bb343526..f8ad838e2b2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -19,15 +19,18 @@ package org.apache.spark.sql.sources @@ -2409,7 +2409,7 @@ index 266bb343526..e58a2f49eb9 100644 import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -@@ -101,12 +104,20 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -101,12 +104,22 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } } @@ -2419,6 +2419,7 @@ index 266bb343526..e58a2f49eb9 100644 + val fileScan = collect(plan) { + case f: FileSourceScanExec => f + case f: CometScanExec => f ++ case f: CometNativeScanExec => f + } assert(fileScan.nonEmpty, plan) fileScan.head @@ -2427,12 +2428,13 @@ index 266bb343526..e58a2f49eb9 100644 + private def getBucketScan(plan: SparkPlan): Boolean = getFileScan(plan) match { + case fs: FileSourceScanExec => fs.bucketedScan + case bs: CometScanExec => bs.bucketedScan ++ case ns: CometNativeScanExec => ns.bucketedScan + } + // To verify if the bucket pruning works, this function checks two conditions: // 1) Check if the pruned buckets (before filtering) are empty. // 2) Verify the final result is the same as the expected one -@@ -155,7 +166,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -155,7 +168,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val planWithoutBucketedScan = bucketedDataFrame.filter(filterCondition) .queryExecution.executedPlan val fileScan = getFileScan(planWithoutBucketedScan) @@ -2442,7 +2444,7 @@ index 266bb343526..e58a2f49eb9 100644 val bucketColumnType = bucketedDataFrame.schema.apply(bucketColumnIndex).dataType val rowsWithInvalidBuckets = fileScan.execute().filter(row => { -@@ -451,28 +463,54 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -451,28 +465,54 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val joinOperator = if (joined.sqlContext.conf.adaptiveExecutionEnabled) { val executedPlan = joined.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan @@ -2505,7 +2507,7 @@ index 266bb343526..e58a2f49eb9 100644 s"expected sort in the right child to be $sortRight but found\n${joinOperator.right}") // check the output partitioning -@@ -835,11 +873,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -835,11 +875,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") val scanDF = spark.table("bucketed_table").select("j") @@ -2519,7 +2521,7 @@ index 266bb343526..e58a2f49eb9 100644 checkAnswer(aggDF, df1.groupBy("j").agg(max("k"))) } } -@@ -894,7 +932,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -894,7 +934,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } test("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") { @@ -2530,7 +2532,7 @@ index 266bb343526..e58a2f49eb9 100644 SQLConf.SHUFFLE_PARTITIONS.key -> "5", SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "7") { val bucketSpec = Some(BucketSpec(6, Seq("i", "j"), Nil)) -@@ -913,7 +954,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -913,7 +956,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } test("SPARK-32767 Bucket join should work if SHUFFLE_PARTITIONS larger than bucket number") { @@ -2541,7 +2543,7 @@ index 266bb343526..e58a2f49eb9 100644 SQLConf.SHUFFLE_PARTITIONS.key -> "9", SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "10") { -@@ -943,7 +987,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -943,7 +989,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } test("bucket coalescing eliminates shuffle") { @@ -2552,7 +2554,7 @@ index 266bb343526..e58a2f49eb9 100644 SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. -@@ -1026,15 +1073,23 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -1026,15 +1075,26 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti expectedNumShuffles: Int, expectedCoalescedNumBuckets: Option[Int]): Unit = { val plan = sql(query).queryExecution.executedPlan @@ -2565,6 +2567,7 @@ index 266bb343526..e58a2f49eb9 100644 val scans = plan.collect { case f: FileSourceScanExec if f.optionalNumCoalescedBuckets.isDefined => f + case b: CometScanExec if b.optionalNumCoalescedBuckets.isDefined => b ++ case b: CometNativeScanExec if b.optionalNumCoalescedBuckets.isDefined => b } if (expectedCoalescedNumBuckets.isDefined) { assert(scans.length == 1) @@ -2574,6 +2577,8 @@ index 266bb343526..e58a2f49eb9 100644 + assert(f.optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) + case b: CometScanExec => + assert(b.optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) ++ case b: CometNativeScanExec => ++ assert(b.optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) + } } else { assert(scans.isEmpty) @@ -2604,18 +2609,18 @@ index b5f6d2f9f68..277784a92af 100644 protected override lazy val sql = spark.sql _ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala -index 1f55742cd67..42377f7cf26 100644 +index 1f55742cd67..f20129d9dd8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.sources import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning -+import org.apache.spark.sql.comet.CometScanExec ++import org.apache.spark.sql.comet.{CometNativeScanExec, CometScanExec} import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec -@@ -71,7 +72,10 @@ abstract class DisableUnnecessaryBucketedScanSuite +@@ -71,7 +72,11 @@ abstract class DisableUnnecessaryBucketedScanSuite def checkNumBucketedScan(query: String, expectedNumBucketedScan: Int): Unit = { val plan = sql(query).queryExecution.executedPlan @@ -2623,6 +2628,7 @@ index 1f55742cd67..42377f7cf26 100644 + val bucketedScan = collect(plan) { + case s: FileSourceScanExec if s.bucketedScan => s + case s: CometScanExec if s.bucketedScan => s ++ case s: CometNativeScanExec if s.bucketedScan => s + } assert(bucketedScan.length == expectedNumBucketedScan) } diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index ad820001f3..beef445490 100644 --- a/dev/diffs/3.5.8.diff +++ b/dev/diffs/3.5.8.diff @@ -1,5 +1,5 @@ diff --git a/pom.xml b/pom.xml -index a0e25ce4d8d..b95fba458f2 100644 +index edd2ad57880..77a975ea48f 100644 --- a/pom.xml +++ b/pom.xml @@ -152,6 +152,8 @@ @@ -38,7 +38,7 @@ index a0e25ce4d8d..b95fba458f2 100644 diff --git a/sql/core/pom.xml b/sql/core/pom.xml -index e3d324c8edb..22342150522 100644 +index bc00c448b80..82068d7a2eb 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -77,6 +77,10 @@ @@ -238,6 +238,20 @@ index e5494726695..00937f025c2 100644 } test("A cached table preserves the partitioning and ordering of its cached SparkPlan") { +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +index 9e8d77c53f3..855e3ada7d1 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +@@ -790,7 +790,8 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { + } + } + +- test("input_file_name, input_file_block_start, input_file_block_length - FileScanRDD") { ++ test("input_file_name, input_file_block_start, input_file_block_length - FileScanRDD", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3312")) { + withTempPath { dir => + val data = sparkContext.parallelize(0 to 10).toDF("id") + data.write.parquet(dir.getCanonicalPath) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 6f3090d8908..c08a60fb0c2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -374,7 +388,7 @@ index a1d5d579338..c201d39cc78 100644 } case _ => false diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala -index f32b32ffc5a..447d7c6416e 100644 +index c4fb4fa943c..a04b23870a8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi} @@ -396,7 +410,7 @@ index f32b32ffc5a..447d7c6416e 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 f33432ddb6f..0e1499a24ca 100644 +index f33432ddb6f..42eb9fd1cb7 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 @@ -447,7 +461,17 @@ index f33432ddb6f..0e1499a24ca 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ WITH v as ( -@@ -1729,6 +1736,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1698,7 +1705,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat + * Check the static scan metrics with and without DPP + */ + test("static scan metrics", +- DisableAdaptiveExecution("DPP in AQE must reuse broadcast")) { ++ DisableAdaptiveExecution("DPP in AQE must reuse broadcast"), ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3313")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { +@@ -1729,6 +1737,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat case s: BatchScanExec => // we use f1 col for v2 tables due to schema pruning s.output.exists(_.exists(_.argString(maxFields = 100).contains("f1"))) @@ -457,10 +481,20 @@ index f33432ddb6f..0e1499a24ca 100644 } assert(scanOption.isDefined) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala -index a206e97c353..fea1149b67d 100644 +index a206e97c353..79813d8e259 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala -@@ -467,7 +467,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -280,7 +280,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite + } + } + +- test("explain formatted - check presence of subquery in case of DPP") { ++ test("explain formatted - check presence of subquery in case of DPP", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3313")) { + withTable("df1", "df2") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", +@@ -467,7 +468,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } @@ -470,7 +504,7 @@ index a206e97c353..fea1149b67d 100644 withTempDir { dir => Seq("parquet", "orc", "csv", "json").foreach { fmt => val basePath = dir.getCanonicalPath + "/" + fmt -@@ -545,7 +546,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -545,7 +547,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } @@ -482,7 +516,7 @@ index a206e97c353..fea1149b67d 100644 test("SPARK-35884: Explain Formatted") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala -index 93275487f29..33b2e7ad3b1 100644 +index 93275487f29..510e3087e0f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -23,6 +23,7 @@ import java.nio.file.{Files, StandardOpenOption} @@ -510,7 +544,17 @@ index 93275487f29..33b2e7ad3b1 100644 checkErrorMatchPVals( exception = intercept[SparkException] { testIgnoreMissingFiles(options) -@@ -955,6 +959,7 @@ class FileBasedDataSourceSuite extends QueryTest +@@ -639,7 +643,8 @@ class FileBasedDataSourceSuite extends QueryTest + } + + Seq("parquet", "orc").foreach { format => +- test(s"Spark native readers should respect spark.sql.caseSensitive - ${format}") { ++ test(s"Spark native readers should respect spark.sql.caseSensitive - ${format}", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { + withTempDir { dir => + val tableName = s"spark_25132_${format}_native" + val tableDir = dir.getCanonicalPath + s"/$tableName" +@@ -955,6 +960,7 @@ class FileBasedDataSourceSuite extends QueryTest assert(bJoinExec.isEmpty) val smJoinExec = collect(joinedDF.queryExecution.executedPlan) { case smJoin: SortMergeJoinExec => smJoin @@ -518,7 +562,7 @@ index 93275487f29..33b2e7ad3b1 100644 } assert(smJoinExec.nonEmpty) } -@@ -1015,6 +1020,7 @@ class FileBasedDataSourceSuite extends QueryTest +@@ -1015,6 +1021,7 @@ class FileBasedDataSourceSuite extends QueryTest val fileScan = df.queryExecution.executedPlan collectFirst { case BatchScanExec(_, f: FileScan, _, _, _, _) => f @@ -526,7 +570,7 @@ index 93275487f29..33b2e7ad3b1 100644 } assert(fileScan.nonEmpty) assert(fileScan.get.partitionFilters.nonEmpty) -@@ -1056,6 +1062,7 @@ class FileBasedDataSourceSuite extends QueryTest +@@ -1056,6 +1063,7 @@ class FileBasedDataSourceSuite extends QueryTest val fileScan = df.queryExecution.executedPlan collectFirst { case BatchScanExec(_, f: FileScan, _, _, _, _) => f @@ -534,7 +578,7 @@ index 93275487f29..33b2e7ad3b1 100644 } assert(fileScan.nonEmpty) assert(fileScan.get.partitionFilters.isEmpty) -@@ -1240,6 +1247,9 @@ class FileBasedDataSourceSuite extends QueryTest +@@ -1240,6 +1248,9 @@ class FileBasedDataSourceSuite extends QueryTest val filters = df.queryExecution.executedPlan.collect { case f: FileSourceScanLike => f.dataFilters case b: BatchScanExec => b.scan.asInstanceOf[FileScan].dataFilters @@ -546,7 +590,7 @@ index 93275487f29..33b2e7ad3b1 100644 } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala new file mode 100644 -index 00000000000..5691536c114 +index 00000000000..1ee842b6f62 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala @@ -0,0 +1,45 @@ @@ -586,8 +630,8 @@ index 00000000000..5691536c114 + * Helper trait that disables Comet for all tests regardless of default config values. + */ +trait IgnoreCometSuite extends SQLTestUtils { -+ override protected def test(testName: String, testTags: Tag*)(testFun: => Any) -+ (implicit pos: Position): Unit = { ++ override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit ++ pos: Position): Unit = { + if (isCometEnabled) { + ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) + } else { @@ -1040,6 +1084,20 @@ index 04702201f82..5ee11f83ecf 100644 } assert(exchanges.size === 1) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +index 9f8e979e3fb..3bc9dab8023 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +@@ -87,7 +87,8 @@ class UDFSuite extends QueryTest with SharedSparkSession { + spark.catalog.dropTempView("tmp_table") + } + +- test("SPARK-8005 input_file_name") { ++ test("SPARK-8005 input_file_name", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3312")) { + withTempPath { dir => + val data = sparkContext.parallelize(0 to 10, 2).toDF("id") + data.write.parquet(dir.getCanonicalPath) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index d269290e616..13726a31e07 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala @@ -1104,24 +1162,37 @@ index d269290e616..13726a31e07 100644 } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala -index cfc8b2cc845..c6fcfd7bd08 100644 +index cfc8b2cc845..b7c234e1437 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala -@@ -21,6 +21,7 @@ import scala.collection.mutable.ArrayBuffer +@@ -19,8 +19,9 @@ package org.apache.spark.sql.connector + import scala.collection.mutable.ArrayBuffer + import org.apache.spark.SparkConf - import org.apache.spark.sql.{AnalysisException, QueryTest} +-import org.apache.spark.sql.{AnalysisException, QueryTest} ++import org.apache.spark.sql.{AnalysisException, IgnoreCometNativeDataFusion, QueryTest} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -+import org.apache.spark.sql.comet.CometScanExec ++import org.apache.spark.sql.comet.{CometNativeScanExec, CometScanExec} import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, Table, TableCapability} import org.apache.spark.sql.connector.read.ScanBuilder import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} -@@ -184,7 +185,11 @@ class FileDataSourceV2FallBackSuite extends QueryTest with SharedSparkSession { +@@ -152,7 +153,8 @@ class FileDataSourceV2FallBackSuite extends QueryTest with SharedSparkSession { + } + } + +- test("Fallback Parquet V2 to V1") { ++ test("Fallback Parquet V2 to V1", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3315")) { + Seq("parquet", classOf[ParquetDataSourceV2].getCanonicalName).foreach { format => + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> format) { + val commands = ArrayBuffer.empty[(String, LogicalPlan)] +@@ -184,7 +186,11 @@ class FileDataSourceV2FallBackSuite extends QueryTest with SharedSparkSession { val df = spark.read.format(format).load(path.getCanonicalPath) checkAnswer(df, inputData.toDF()) assert( - df.queryExecution.executedPlan.exists(_.isInstanceOf[FileSourceScanExec])) + df.queryExecution.executedPlan.exists { -+ case _: FileSourceScanExec | _: CometScanExec => true ++ case _: FileSourceScanExec | _: CometScanExec | _: CometNativeScanExec => true + case _ => false + } + ) @@ -1379,6 +1450,28 @@ index 47679ed7865..9ffbaecb98e 100644 }.length == hashAggCount) assert(collectWithSubqueries(plan) { case s: SortAggregateExec => s }.length == sortAggCount) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala +index a1147c16cc8..c7a29496328 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala +@@ -19,6 +19,7 @@ package org.apache.spark.sql.execution + + import org.apache.spark.{SparkArithmeticException, SparkException, SparkFileNotFoundException} + import org.apache.spark.sql._ ++import org.apache.spark.sql.IgnoreCometNativeDataFusion + import org.apache.spark.sql.catalyst.TableIdentifier + import org.apache.spark.sql.catalyst.expressions.{Add, Alias, Divide} + import org.apache.spark.sql.catalyst.parser.ParseException +@@ -968,7 +969,8 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { + } + } + +- test("alter temporary view should follow current storeAnalyzedPlanForView config") { ++ test("alter temporary view should follow current storeAnalyzedPlanForView config", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3314")) { + withTable("t") { + Seq(2, 3, 1).toDF("c1").write.format("parquet").saveAsTable("t") + withView("v1") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala index eec396b2e39..bf3f1c769d6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala @@ -1877,7 +1970,7 @@ index 2f8e401e743..a4f94417dcc 100644 assert(o1.semanticEquals(o2), "Different output column order after AQE optimization") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala -index bf496d6db21..9bb57a9b4c6 100644 +index fd52d038ca6..154c800be67 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.Concat @@ -1888,7 +1981,7 @@ index bf496d6db21..9bb57a9b4c6 100644 import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.functions._ -@@ -868,6 +869,8 @@ abstract class SchemaPruningSuite +@@ -884,6 +885,8 @@ abstract class SchemaPruningSuite val fileSourceScanSchemata = collect(df.queryExecution.executedPlan) { case scan: FileSourceScanExec => scan.requiredSchema @@ -1898,7 +1991,7 @@ index bf496d6db21..9bb57a9b4c6 100644 assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala -index ce43edb79c1..4dbb5942bc3 100644 +index 5fd27410dcb..468abb1543a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources @@ -1909,7 +2002,7 @@ index ce43edb79c1..4dbb5942bc3 100644 import org.apache.spark.sql.execution.{QueryExecution, SortExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.internal.SQLConf -@@ -225,6 +226,7 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write +@@ -243,6 +244,7 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write // assert the outer most sort in the executed plan assert(plan.collectFirst { case s: SortExec => s @@ -1917,7 +2010,7 @@ index ce43edb79c1..4dbb5942bc3 100644 }.exists { case SortExec(Seq( SortOrder(AttributeReference("key", IntegerType, _, _), Ascending, NullsFirst, _), -@@ -272,6 +274,7 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write +@@ -290,6 +292,7 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write // assert the outer most sort in the executed plan assert(plan.collectFirst { case s: SortExec => s @@ -1973,7 +2066,7 @@ index 07e2849ce6f..3e73645b638 100644 ParquetOutputFormat.WRITER_VERSION -> ParquetProperties.WriterVersion.PARQUET_2_0.toString ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala -index 8e88049f51e..8f3cf8a0f80 100644 +index 8e88049f51e..49f2001dc6b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -1095,7 +1095,11 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared @@ -1999,7 +2092,17 @@ index 8e88049f51e..8f3cf8a0f80 100644 import testImplicits._ withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true", -@@ -1580,7 +1585,11 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared +@@ -1548,7 +1553,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared + } + } + +- test("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") { ++ test("SPARK-31026: Parquet predicate pushdown for fields having dots in the names", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3320")) { + import testImplicits._ + + withAllParquetReaders { +@@ -1580,13 +1586,18 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared // than the total length but should not be a single record. // Note that, if record level filtering is enabled, it should be a single record. // If no filter is pushed down to Parquet, it should be the total length of data. @@ -2012,7 +2115,15 @@ index 8e88049f51e..8f3cf8a0f80 100644 } } } -@@ -1607,7 +1616,11 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared + } + +- test("Filters should be pushed down for Parquet readers at row group level") { ++ test("Filters should be pushed down for Parquet readers at row group level", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3320")) { + import testImplicits._ + + withSQLConf( +@@ -1607,7 +1618,11 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared // than the total length but should not be a single record. // Note that, if record level filtering is enabled, it should be a single record. // If no filter is pushed down to Parquet, it should be the total length of data. @@ -2025,7 +2136,7 @@ index 8e88049f51e..8f3cf8a0f80 100644 } } } -@@ -1699,7 +1712,7 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared +@@ -1699,7 +1714,7 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared (attr, value) => sources.StringContains(attr, value)) } @@ -2034,7 +2145,7 @@ index 8e88049f51e..8f3cf8a0f80 100644 import testImplicits._ // keep() should take effect on StartsWith/EndsWith/Contains Seq( -@@ -1743,7 +1756,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared +@@ -1743,7 +1758,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } } @@ -2044,7 +2155,17 @@ index 8e88049f51e..8f3cf8a0f80 100644 val schema = StructType(Seq( StructField("a", IntegerType, nullable = false) )) -@@ -1984,7 +1998,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared +@@ -1933,7 +1949,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared + } + } + +- test("SPARK-25207: exception when duplicate fields in case-insensitive mode") { ++ test("SPARK-25207: exception when duplicate fields in case-insensitive mode", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { + withTempPath { dir => + val count = 10 + val tableName = "spark_25207" +@@ -1984,7 +2001,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } } @@ -2054,7 +2175,7 @@ index 8e88049f51e..8f3cf8a0f80 100644 // block 1: // null count min max // page-0 0 0 99 -@@ -2044,7 +2059,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared +@@ -2044,7 +2062,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } } @@ -2064,7 +2185,7 @@ index 8e88049f51e..8f3cf8a0f80 100644 withTempPath { dir => val path = dir.getCanonicalPath spark.range(100).selectExpr("id * 2 AS id") -@@ -2276,7 +2292,11 @@ class ParquetV1FilterSuite extends ParquetFilterSuite { +@@ -2276,7 +2295,11 @@ class ParquetV1FilterSuite extends ParquetFilterSuite { assert(pushedParquetFilters.exists(_.getClass === filterClass), s"${pushedParquetFilters.map(_.getClass).toList} did not contain ${filterClass}.") @@ -2077,7 +2198,7 @@ index 8e88049f51e..8f3cf8a0f80 100644 } else { assert(selectedFilters.isEmpty, "There is filter pushed down") } -@@ -2336,7 +2356,11 @@ class ParquetV2FilterSuite extends ParquetFilterSuite { +@@ -2336,7 +2359,11 @@ class ParquetV2FilterSuite extends ParquetFilterSuite { assert(pushedParquetFilters.exists(_.getClass === filterClass), s"${pushedParquetFilters.map(_.getClass).toList} did not contain ${filterClass}.") @@ -2091,10 +2212,30 @@ index 8e88049f51e..8f3cf8a0f80 100644 case _ => throw new AnalysisException("Can not match ParquetTable in the query.") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala -index 8ed9ef1630e..eed2a6f5ad5 100644 +index 8ed9ef1630e..f312174b182 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala -@@ -1345,7 +1345,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession +@@ -1064,7 +1064,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession + } + } + +- test("SPARK-35640: read binary as timestamp should throw schema incompatible error") { ++ test("SPARK-35640: read binary as timestamp should throw schema incompatible error", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { + val data = (1 to 4).map(i => Tuple1(i.toString)) + val readSchema = StructType(Seq(StructField("_1", DataTypes.TimestampType))) + +@@ -1075,7 +1076,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession + } + } + +- test("SPARK-35640: int as long should throw schema incompatible error") { ++ test("SPARK-35640: int as long should throw schema incompatible error", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { + val data = (1 to 4).map(i => Tuple1(i)) + val readSchema = StructType(Seq(StructField("_1", DataTypes.LongType))) + +@@ -1345,7 +1347,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession } } @@ -2105,10 +2246,20 @@ index 8ed9ef1630e..eed2a6f5ad5 100644 checkAnswer( // "fruit" column in this file is encoded using DELTA_LENGTH_BYTE_ARRAY. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala -index f6472ba3d9d..7a8f5317ed7 100644 +index f6472ba3d9d..ce39ebb52e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala -@@ -998,7 +998,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS +@@ -185,7 +185,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS + } + } + +- test("SPARK-36182: can't read TimestampLTZ as TimestampNTZ") { ++ test("SPARK-36182: can't read TimestampLTZ as TimestampNTZ", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { + val data = (1 to 1000).map { i => + val ts = new java.sql.Timestamp(i) + Row(ts) +@@ -998,7 +999,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS } } @@ -2118,7 +2269,17 @@ index f6472ba3d9d..7a8f5317ed7 100644 withAllParquetReaders { withTempPath { path => // Repeated values for dictionary encoding. -@@ -1067,7 +1068,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS +@@ -1051,7 +1053,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS + testMigration(fromTsType = "TIMESTAMP_MICROS", toTsType = "INT96") + } + +- test("SPARK-34212 Parquet should read decimals correctly") { ++ test("SPARK-34212 Parquet should read decimals correctly", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { + def readParquet(schema: String, path: File): DataFrame = { + spark.read.schema(schema).parquet(path.toString) + } +@@ -1067,7 +1070,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS checkAnswer(readParquet(schema, path), df) } @@ -2128,7 +2289,7 @@ index f6472ba3d9d..7a8f5317ed7 100644 val schema1 = "a DECIMAL(3, 2), b DECIMAL(18, 3), c DECIMAL(37, 3)" checkAnswer(readParquet(schema1, path), df) val schema2 = "a DECIMAL(3, 0), b DECIMAL(18, 1), c DECIMAL(37, 1)" -@@ -1089,7 +1091,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS +@@ -1089,7 +1093,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS val df = sql(s"SELECT 1 a, 123456 b, ${Int.MaxValue.toLong * 10} c, CAST('1.2' AS BINARY) d") df.write.parquet(path.toString) @@ -2138,7 +2299,17 @@ index f6472ba3d9d..7a8f5317ed7 100644 checkAnswer(readParquet("a DECIMAL(3, 2)", path), sql("SELECT 1.00")) checkAnswer(readParquet("b DECIMAL(3, 2)", path), Row(null)) checkAnswer(readParquet("b DECIMAL(11, 1)", path), sql("SELECT 123456.0")) -@@ -1148,7 +1151,7 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS +@@ -1133,7 +1138,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS + } + } + +- test("row group skipping doesn't overflow when reading into larger type") { ++ test("row group skipping doesn't overflow when reading into larger type", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { + withTempPath { path => + Seq(0).toDF("a").write.parquet(path.toString) + // The vectorized and non-vectorized readers will produce different exceptions, we don't need +@@ -1148,7 +1154,7 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS .where(s"a < ${Long.MaxValue}") .collect() } @@ -2243,14 +2414,14 @@ index 5c0b7def039..151184bc98c 100644 assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala -index 3f47c5e506f..bc1ee1ec0ba 100644 +index 3f47c5e506f..92a5eafec84 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -27,6 +27,7 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName import org.apache.parquet.schema.Type._ import org.apache.spark.SparkException -+import org.apache.spark.sql.IgnoreComet ++import org.apache.spark.sql.{IgnoreComet, IgnoreCometNativeDataFusion} import org.apache.spark.sql.catalyst.expressions.Cast.toSQLType import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException import org.apache.spark.sql.functions.desc @@ -2264,6 +2435,26 @@ index 3f47c5e506f..bc1ee1ec0ba 100644 withTempPath { dir => val e = testSchemaMismatch(dir.getCanonicalPath, vectorizedReaderEnabled = false) val expectedMessage = "Encountered error while reading file" +@@ -1046,7 +1048,8 @@ class ParquetSchemaSuite extends ParquetSchemaTest { + } + } + +- test("schema mismatch failure error message for parquet vectorized reader") { ++ test("schema mismatch failure error message for parquet vectorized reader", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { + withTempPath { dir => + val e = testSchemaMismatch(dir.getCanonicalPath, vectorizedReaderEnabled = true) + assert(e.getCause.isInstanceOf[SparkException]) +@@ -1087,7 +1090,8 @@ class ParquetSchemaSuite extends ParquetSchemaTest { + } + } + +- test("SPARK-45604: schema mismatch failure error on timestamp_ntz to array") { ++ test("SPARK-45604: schema mismatch failure error on timestamp_ntz to array", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { + import testImplicits._ + + withTempPath { dir => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala index b8f3ea3c6f3..bbd44221288 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala @@ -2313,18 +2504,30 @@ index 5cdbdc27b32..307fba16578 100644 spark.range(10).selectExpr("id", "id % 3 as p") .write.partitionBy("p").saveAsTable("testDataForScan") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala -index 0ab8691801d..d9125f658ad 100644 +index 0ab8691801d..7b81f3a8f6d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala -@@ -18,6 +18,7 @@ +@@ -17,7 +17,9 @@ + package org.apache.spark.sql.execution.python ++import org.apache.spark.sql.IgnoreCometNativeDataFusion import org.apache.spark.sql.catalyst.plans.logical.{ArrowEvalPython, BatchEvalPython, Limit, LocalLimit} +import org.apache.spark.sql.comet._ import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan, SparkPlanTest} import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan -@@ -108,6 +109,7 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { +@@ -93,7 +95,8 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { + assert(arrowEvalNodes.size == 2) + } + +- test("Python UDF should not break column pruning/filter pushdown -- Parquet V1") { ++ test("Python UDF should not break column pruning/filter pushdown -- Parquet V1", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3312")) { + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "parquet") { + withTempPath { f => + spark.range(10).select($"id".as("a"), $"id".as("b")) +@@ -108,6 +111,7 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { val scanNodes = query.queryExecution.executedPlan.collect { case scan: FileSourceScanExec => scan @@ -2332,7 +2535,7 @@ index 0ab8691801d..d9125f658ad 100644 } assert(scanNodes.length == 1) assert(scanNodes.head.output.map(_.name) == Seq("a")) -@@ -120,11 +122,16 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { +@@ -120,11 +124,16 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { val scanNodes = query.queryExecution.executedPlan.collect { case scan: FileSourceScanExec => scan @@ -2351,7 +2554,7 @@ index 0ab8691801d..d9125f658ad 100644 } } } -@@ -145,6 +152,7 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { +@@ -145,6 +154,7 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { val scanNodes = query.queryExecution.executedPlan.collect { case scan: BatchScanExec => scan @@ -2359,7 +2562,7 @@ index 0ab8691801d..d9125f658ad 100644 } assert(scanNodes.length == 1) assert(scanNodes.head.output.map(_.name) == Seq("a")) -@@ -157,6 +165,7 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { +@@ -157,6 +167,7 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { val scanNodes = query.queryExecution.executedPlan.collect { case scan: BatchScanExec => scan @@ -2384,7 +2587,7 @@ index d083cac48ff..3c11bcde807 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -index 746f289c393..a773971d3c1 100644 +index 746f289c393..7a6a88a9fce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -19,16 +19,19 @@ package org.apache.spark.sql.sources @@ -2409,7 +2612,7 @@ index 746f289c393..a773971d3c1 100644 import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -@@ -102,12 +105,20 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -102,12 +105,22 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } } @@ -2419,6 +2622,7 @@ index 746f289c393..a773971d3c1 100644 + val fileScan = collect(plan) { + case f: FileSourceScanExec => f + case f: CometScanExec => f ++ case f: CometNativeScanExec => f + } assert(fileScan.nonEmpty, plan) fileScan.head @@ -2427,12 +2631,13 @@ index 746f289c393..a773971d3c1 100644 + private def getBucketScan(plan: SparkPlan): Boolean = getFileScan(plan) match { + case fs: FileSourceScanExec => fs.bucketedScan + case bs: CometScanExec => bs.bucketedScan ++ case ns: CometNativeScanExec => ns.bucketedScan + } + // To verify if the bucket pruning works, this function checks two conditions: // 1) Check if the pruned buckets (before filtering) are empty. // 2) Verify the final result is the same as the expected one -@@ -156,7 +167,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -156,7 +169,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val planWithoutBucketedScan = bucketedDataFrame.filter(filterCondition) .queryExecution.executedPlan val fileScan = getFileScan(planWithoutBucketedScan) @@ -2442,7 +2647,7 @@ index 746f289c393..a773971d3c1 100644 val bucketColumnType = bucketedDataFrame.schema.apply(bucketColumnIndex).dataType val rowsWithInvalidBuckets = fileScan.execute().filter(row => { -@@ -452,28 +464,54 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -452,28 +466,54 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val joinOperator = if (joined.sqlContext.conf.adaptiveExecutionEnabled) { val executedPlan = joined.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan @@ -2505,7 +2710,7 @@ index 746f289c393..a773971d3c1 100644 s"expected sort in the right child to be $sortRight but found\n${joinOperator.right}") // check the output partitioning -@@ -836,11 +874,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -836,11 +876,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") val scanDF = spark.table("bucketed_table").select("j") @@ -2519,7 +2724,7 @@ index 746f289c393..a773971d3c1 100644 checkAnswer(aggDF, df1.groupBy("j").agg(max("k"))) } } -@@ -895,7 +933,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -895,7 +935,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } test("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") { @@ -2530,7 +2735,7 @@ index 746f289c393..a773971d3c1 100644 SQLConf.SHUFFLE_PARTITIONS.key -> "5", SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "7") { val bucketSpec = Some(BucketSpec(6, Seq("i", "j"), Nil)) -@@ -914,7 +955,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -914,7 +957,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } test("SPARK-32767 Bucket join should work if SHUFFLE_PARTITIONS larger than bucket number") { @@ -2541,7 +2746,7 @@ index 746f289c393..a773971d3c1 100644 SQLConf.SHUFFLE_PARTITIONS.key -> "9", SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "10") { -@@ -944,7 +988,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -944,7 +990,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } test("bucket coalescing eliminates shuffle") { @@ -2552,7 +2757,7 @@ index 746f289c393..a773971d3c1 100644 SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. -@@ -1029,15 +1076,21 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -1029,15 +1078,24 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti Seq(true, false).foreach { aqeEnabled => withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled.toString) { val plan = sql(query).queryExecution.executedPlan @@ -2563,6 +2768,7 @@ index 746f289c393..a773971d3c1 100644 val scans = collect(plan) { case f: FileSourceScanExec if f.optionalNumCoalescedBuckets.isDefined => f + case b: CometScanExec if b.optionalNumCoalescedBuckets.isDefined => b ++ case b: CometNativeScanExec if b.optionalNumCoalescedBuckets.isDefined => b } if (expectedCoalescedNumBuckets.isDefined) { assert(scans.length == 1) @@ -2572,6 +2778,8 @@ index 746f289c393..a773971d3c1 100644 + assert(f.optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) + case b: CometScanExec => + assert(b.optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) ++ case b: CometNativeScanExec => ++ assert(b.optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) + } } else { assert(scans.isEmpty) @@ -2601,18 +2809,18 @@ index 6f897a9c0b7..b0723634f68 100644 protected override lazy val sql = spark.sql _ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala -index d675503a8ba..659fa686fb7 100644 +index d675503a8ba..f220892396e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.sources import org.apache.spark.sql.QueryTest -+import org.apache.spark.sql.comet.CometScanExec ++import org.apache.spark.sql.comet.{CometNativeScanExec, CometScanExec} import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} import org.apache.spark.sql.internal.SQLConf -@@ -68,7 +69,10 @@ abstract class DisableUnnecessaryBucketedScanSuite +@@ -68,7 +69,11 @@ abstract class DisableUnnecessaryBucketedScanSuite def checkNumBucketedScan(query: String, expectedNumBucketedScan: Int): Unit = { val plan = sql(query).queryExecution.executedPlan @@ -2620,15 +2828,16 @@ index d675503a8ba..659fa686fb7 100644 + val bucketedScan = collect(plan) { + case s: FileSourceScanExec if s.bucketedScan => s + case s: CometScanExec if s.bucketedScan => s ++ case s: CometNativeScanExec if s.bucketedScan => s + } assert(bucketedScan.length == expectedNumBucketedScan) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala -index 1954cce7fdc..73d1464780e 100644 +index 7f6fa2a123e..c778b4e2c48 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala -@@ -34,6 +34,7 @@ import org.apache.spark.paths.SparkPath +@@ -35,6 +35,7 @@ import org.apache.spark.paths.SparkPath import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} import org.apache.spark.sql.{AnalysisException, DataFrame} import org.apache.spark.sql.catalyst.util.stringToFile @@ -2636,7 +2845,7 @@ index 1954cce7fdc..73d1464780e 100644 import org.apache.spark.sql.execution.DataSourceScanExec import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation, FileScan, FileTable} -@@ -761,6 +762,8 @@ class FileStreamSinkV2Suite extends FileStreamSinkSuite { +@@ -777,6 +778,8 @@ class FileStreamSinkV2Suite extends FileStreamSinkSuite { val fileScan = df.queryExecution.executedPlan.collect { case batch: BatchScanExec if batch.scan.isInstanceOf[FileScan] => batch.scan.asInstanceOf[FileScan] @@ -2758,6 +2967,72 @@ index aad91601758..201083bd621 100644 }) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +index b5cf13a9c12..ac17603fb7f 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +@@ -36,7 +36,7 @@ import org.scalatestplus.mockito.MockitoSugar + + import org.apache.spark.{SparkException, TestUtils} + import org.apache.spark.internal.Logging +-import org.apache.spark.sql.{AnalysisException, Column, DataFrame, Dataset, Row, SaveMode} ++import org.apache.spark.sql.{AnalysisException, Column, DataFrame, Dataset, IgnoreCometNativeDataFusion, Row, SaveMode} + import org.apache.spark.sql.catalyst.InternalRow + import org.apache.spark.sql.catalyst.expressions.{Literal, Rand, Randn, Shuffle, Uuid} + import org.apache.spark.sql.catalyst.plans.logical.{CTERelationDef, CTERelationRef, LocalRelation} +@@ -660,7 +660,8 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi + ) + } + +- test("SPARK-41198: input row calculation with CTE") { ++ test("SPARK-41198: input row calculation with CTE", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3315")) { + withTable("parquet_tbl", "parquet_streaming_tbl") { + spark.range(0, 10).selectExpr("id AS col1", "id AS col2") + .write.format("parquet").saveAsTable("parquet_tbl") +@@ -712,7 +713,8 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi + } + } + +- test("SPARK-41199: input row calculation with mixed-up of DSv1 and DSv2 streaming sources") { ++ test("SPARK-41199: input row calculation with mixed-up of DSv1 and DSv2 streaming sources", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3315")) { + withTable("parquet_streaming_tbl") { + val streamInput = MemoryStream[Int] + val streamDf = streamInput.toDF().selectExpr("value AS key", "value AS value_stream") +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSelfUnionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSelfUnionSuite.scala +index 8f099c31e6b..ce4b7ad25b3 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSelfUnionSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSelfUnionSuite.scala +@@ -20,7 +20,7 @@ package org.apache.spark.sql.streaming + import org.scalatest.BeforeAndAfter + import org.scalatest.concurrent.PatienceConfiguration.Timeout + +-import org.apache.spark.sql.SaveMode ++import org.apache.spark.sql.{IgnoreCometNativeDataFusion, SaveMode} + import org.apache.spark.sql.connector.catalog.Identifier + import org.apache.spark.sql.execution.streaming.MemoryStream + import org.apache.spark.sql.streaming.test.{InMemoryStreamTable, InMemoryStreamTableCatalog} +@@ -42,7 +42,8 @@ class StreamingSelfUnionSuite extends StreamTest with BeforeAndAfter { + sqlContext.streams.active.foreach(_.stop()) + } + +- test("self-union, DSv1, read via DataStreamReader API") { ++ test("self-union, DSv1, read via DataStreamReader API", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3401")) { + withTempPath { dir => + val dataLocation = dir.getAbsolutePath + spark.range(1, 4).write.format("parquet").save(dataLocation) +@@ -66,7 +67,8 @@ class StreamingSelfUnionSuite extends StreamTest with BeforeAndAfter { + } + } + +- test("self-union, DSv1, read via table API") { ++ test("self-union, DSv1, read via table API", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3401")) { + withTable("parquet_streaming_tbl") { + spark.sql("CREATE TABLE parquet_streaming_tbl (key integer) USING parquet") + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala index abe606ad9c1..2d930b64cca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala @@ -2782,7 +3057,7 @@ index abe606ad9c1..2d930b64cca 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 e937173a590..ca06132102d 100644 +index e937173a590..7d20538bc68 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 @@ -27,6 +27,7 @@ import scala.concurrent.duration._ @@ -2801,37 +3076,42 @@ index e937173a590..ca06132102d 100644 import org.apache.spark.sql.execution.FilterExec import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecution import org.apache.spark.sql.execution.datasources.DataSourceUtils -@@ -126,7 +128,28 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with - } - } - } else { -- super.test(testName, testTags: _*)(testFun) -+ if (isCometEnabled && testTags.exists(_.isInstanceOf[IgnoreComet])) { -+ ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) -+ } else { -+ val cometScanImpl = CometConf.COMET_NATIVE_SCAN_IMPL.get(conf) -+ val isNativeIcebergCompat = cometScanImpl == CometConf.SCAN_NATIVE_ICEBERG_COMPAT || -+ cometScanImpl == CometConf.SCAN_AUTO -+ val isNativeDataFusion = cometScanImpl == CometConf.SCAN_NATIVE_DATAFUSION || -+ cometScanImpl == CometConf.SCAN_AUTO -+ if (isCometEnabled && isNativeIcebergCompat && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeIcebergCompat])) { -+ ignore(testName + " (disabled for NATIVE_ICEBERG_COMPAT)", testTags: _*)(testFun) -+ } else if (isCometEnabled && isNativeDataFusion && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeDataFusion])) { -+ ignore(testName + " (disabled for NATIVE_DATAFUSION)", testTags: _*)(testFun) -+ } else if (isCometEnabled && (isNativeDataFusion || isNativeIcebergCompat) && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeScan])) { -+ ignore(testName + " (disabled for NATIVE_DATAFUSION and NATIVE_ICEBERG_COMPAT)", -+ testTags: _*)(testFun) -+ } else { -+ super.test(testName, testTags: _*)(testFun) -+ } -+ } - } - } +@@ -119,6 +121,34 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with -@@ -242,6 +265,29 @@ private[sql] trait SQLTestUtilsBase + override protected def test(testName: String, testTags: Tag*)(testFun: => Any) + (implicit pos: Position): Unit = { ++ // Check Comet skip tags first, before DisableAdaptiveExecution handling ++ if (isCometEnabled && testTags.exists(_.isInstanceOf[IgnoreComet])) { ++ ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) ++ return ++ } ++ if (isCometEnabled) { ++ val cometScanImpl = CometConf.COMET_NATIVE_SCAN_IMPL.get(conf) ++ val isNativeIcebergCompat = cometScanImpl == CometConf.SCAN_NATIVE_ICEBERG_COMPAT || ++ cometScanImpl == CometConf.SCAN_AUTO ++ val isNativeDataFusion = cometScanImpl == CometConf.SCAN_NATIVE_DATAFUSION || ++ cometScanImpl == CometConf.SCAN_AUTO ++ if (isNativeIcebergCompat && ++ testTags.exists(_.isInstanceOf[IgnoreCometNativeIcebergCompat])) { ++ ignore(testName + " (disabled for NATIVE_ICEBERG_COMPAT)", testTags: _*)(testFun) ++ return ++ } ++ if (isNativeDataFusion && ++ testTags.exists(_.isInstanceOf[IgnoreCometNativeDataFusion])) { ++ ignore(testName + " (disabled for NATIVE_DATAFUSION)", testTags: _*)(testFun) ++ return ++ } ++ if ((isNativeDataFusion || isNativeIcebergCompat) && ++ testTags.exists(_.isInstanceOf[IgnoreCometNativeScan])) { ++ ignore(testName + " (disabled for NATIVE_DATAFUSION and NATIVE_ICEBERG_COMPAT)", ++ testTags: _*)(testFun) ++ return ++ } ++ } + if (testTags.exists(_.isInstanceOf[DisableAdaptiveExecution])) { + super.test(testName, testTags: _*) { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { +@@ -242,6 +272,29 @@ private[sql] trait SQLTestUtilsBase protected override def _sqlContext: SQLContext = self.spark.sqlContext } @@ -2861,7 +3141,7 @@ index e937173a590..ca06132102d 100644 protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { SparkSession.setActiveSession(spark) super.withSQLConf(pairs: _*)(f) -@@ -435,6 +481,8 @@ private[sql] trait SQLTestUtilsBase +@@ -435,6 +488,8 @@ private[sql] trait SQLTestUtilsBase val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child @@ -2963,6 +3243,29 @@ index de3b1ffccf0..2a76d127093 100644 override def beforeEach(): Unit = { super.beforeEach() +diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala +index f3be79f9022..b4b1ea8dbc4 100644 +--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala ++++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala +@@ -34,7 +34,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn + import org.apache.hadoop.io.{LongWritable, Writable} + + import org.apache.spark.{SparkException, SparkFiles, TestUtils} +-import org.apache.spark.sql.{AnalysisException, QueryTest, Row} ++import org.apache.spark.sql.{AnalysisException, IgnoreCometNativeDataFusion, QueryTest, Row} + import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode + import org.apache.spark.sql.catalyst.plans.logical.Project + import org.apache.spark.sql.execution.WholeStageCodegenExec +@@ -448,7 +448,8 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { + } + } + +- test("SPARK-11522 select input_file_name from non-parquet table") { ++ test("SPARK-11522 select input_file_name from non-parquet table", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3312")) { + + withTempDir { tempDir => + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 6160c3e5f6c..0956d7d9edc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala diff --git a/dev/diffs/4.0.1.diff b/dev/diffs/4.0.1.diff index 8c6c0dd523..d6694e827f 100644 --- a/dev/diffs/4.0.1.diff +++ b/dev/diffs/4.0.1.diff @@ -1436,14 +1436,14 @@ index 3eeed2e4175..9f21d547c1c 100644 } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala -index 2a0ab21ddb0..e8a5a891105 100644 +index 2a0ab21ddb0..6030e7c2b9b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala @@ -21,6 +21,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -+import org.apache.spark.sql.comet.CometScanExec ++import org.apache.spark.sql.comet.{CometNativeScanExec, CometScanExec} import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, Table, TableCapability} import org.apache.spark.sql.connector.read.ScanBuilder import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} @@ -1453,7 +1453,7 @@ index 2a0ab21ddb0..e8a5a891105 100644 assert( - df.queryExecution.executedPlan.exists(_.isInstanceOf[FileSourceScanExec])) + df.queryExecution.executedPlan.exists { -+ case _: FileSourceScanExec | _: CometScanExec => true ++ case _: FileSourceScanExec | _: CometScanExec | _: CometNativeScanExec => true + case _ => false + } + ) @@ -3080,7 +3080,7 @@ index 7838e62013d..8fa09652921 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -index c4b09c4b289..dd5763e8405 100644 +index c4b09c4b289..75c3437788e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -26,10 +26,11 @@ import org.apache.spark.sql.catalyst.expressions @@ -3097,7 +3097,7 @@ index c4b09c4b289..dd5763e8405 100644 import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -@@ -103,12 +104,20 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -103,12 +104,22 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } } @@ -3107,6 +3107,7 @@ index c4b09c4b289..dd5763e8405 100644 + val fileScan = collect(plan) { + case f: FileSourceScanExec => f + case f: CometScanExec => f ++ case f: CometNativeScanExec => f + } assert(fileScan.nonEmpty, plan) fileScan.head @@ -3115,12 +3116,13 @@ index c4b09c4b289..dd5763e8405 100644 + private def getBucketScan(plan: SparkPlan): Boolean = getFileScan(plan) match { + case fs: FileSourceScanExec => fs.bucketedScan + case bs: CometScanExec => bs.bucketedScan ++ case ns: CometNativeScanExec => ns.bucketedScan + } + // To verify if the bucket pruning works, this function checks two conditions: // 1) Check if the pruned buckets (before filtering) are empty. // 2) Verify the final result is the same as the expected one -@@ -157,7 +166,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -157,7 +168,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val planWithoutBucketedScan = bucketedDataFrame.filter(filterCondition) .queryExecution.executedPlan val fileScan = getFileScan(planWithoutBucketedScan) @@ -3130,7 +3132,7 @@ index c4b09c4b289..dd5763e8405 100644 val bucketColumnType = bucketedDataFrame.schema.apply(bucketColumnIndex).dataType val rowsWithInvalidBuckets = fileScan.execute().filter(row => { -@@ -454,28 +464,54 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -454,28 +466,54 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val joinOperator = if (joined.sparkSession.sessionState.conf.adaptiveExecutionEnabled) { val executedPlan = joined.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan @@ -3193,7 +3195,7 @@ index c4b09c4b289..dd5763e8405 100644 s"expected sort in the right child to be $sortRight but found\n${joinOperator.right}") // check the output partitioning -@@ -838,11 +874,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -838,11 +876,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") val scanDF = spark.table("bucketed_table").select("j") @@ -3207,7 +3209,7 @@ index c4b09c4b289..dd5763e8405 100644 checkAnswer(aggDF, df1.groupBy("j").agg(max("k"))) } } -@@ -1031,15 +1067,21 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -1031,15 +1069,24 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti Seq(true, false).foreach { aqeEnabled => withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled.toString) { val plan = sql(query).queryExecution.executedPlan @@ -3218,6 +3220,7 @@ index c4b09c4b289..dd5763e8405 100644 val scans = collect(plan) { case f: FileSourceScanExec if f.optionalNumCoalescedBuckets.isDefined => f + case b: CometScanExec if b.optionalNumCoalescedBuckets.isDefined => b ++ case b: CometNativeScanExec if b.optionalNumCoalescedBuckets.isDefined => b } if (expectedCoalescedNumBuckets.isDefined) { assert(scans.length == 1) @@ -3227,6 +3230,8 @@ index c4b09c4b289..dd5763e8405 100644 + assert(f.optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) + case b: CometScanExec => + assert(b.optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) ++ case b: CometNativeScanExec => ++ assert(b.optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) + } } else { assert(scans.isEmpty) @@ -3256,18 +3261,18 @@ index 95c2fcbd7b5..e2d4a20c5d9 100644 protected override lazy val sql = spark.sql _ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala -index c5c56f081d8..197cd241f48 100644 +index c5c56f081d8..6cc51f93b4f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.sources import org.apache.spark.sql.QueryTest -+import org.apache.spark.sql.comet.CometScanExec ++import org.apache.spark.sql.comet.{CometNativeScanExec, CometScanExec} import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} import org.apache.spark.sql.internal.SQLConf -@@ -68,7 +69,10 @@ abstract class DisableUnnecessaryBucketedScanSuite +@@ -68,7 +69,11 @@ abstract class DisableUnnecessaryBucketedScanSuite def checkNumBucketedScan(query: String, expectedNumBucketedScan: Int): Unit = { val plan = sql(query).queryExecution.executedPlan @@ -3275,6 +3280,7 @@ index c5c56f081d8..197cd241f48 100644 + val bucketedScan = collect(plan) { + case s: FileSourceScanExec if s.bucketedScan => s + case s: CometScanExec if s.bucketedScan => s ++ case s: CometNativeScanExec if s.bucketedScan => s + } assert(bucketedScan.length == expectedNumBucketedScan) } diff --git a/dev/regenerate-golden-files.sh b/dev/regenerate-golden-files.sh index 56bfb24565..4fe2e283f7 100755 --- a/dev/regenerate-golden-files.sh +++ b/dev/regenerate-golden-files.sh @@ -74,16 +74,6 @@ build_native() { cd native && cargo build && cd .. } -# Install Comet for a specific Spark version -install_for_spark_version() { - local spark_version=$1 - echo "" - echo "==============================================" - echo "[INFO] Installing Comet for Spark $spark_version" - echo "==============================================" - ./mvnw install -DskipTests -Pspark-$spark_version -} - # Regenerate golden files for a specific Spark version regenerate_golden_files() { local spark_version=$1 @@ -94,12 +84,12 @@ regenerate_golden_files() { echo "==============================================" echo "[INFO] Running CometTPCDSV1_4_PlanStabilitySuite..." - SPARK_GENERATE_GOLDEN_FILES=1 ./mvnw -pl spark \ + SPARK_GENERATE_GOLDEN_FILES=1 ./mvnw \ -Dsuites="org.apache.spark.sql.comet.CometTPCDSV1_4_PlanStabilitySuite" \ -Pspark-$spark_version -nsu test echo "[INFO] Running CometTPCDSV2_7_PlanStabilitySuite..." - SPARK_GENERATE_GOLDEN_FILES=1 ./mvnw -pl spark \ + SPARK_GENERATE_GOLDEN_FILES=1 ./mvnw \ -Dsuites="org.apache.spark.sql.comet.CometTPCDSV2_7_PlanStabilitySuite" \ -Pspark-$spark_version -nsu test } @@ -158,9 +148,8 @@ main() { versions=("3.4" "3.5" "4.0") fi - # Install and regenerate for each version + # Regenerate for each version for version in "${versions[@]}"; do - install_for_spark_version "$version" regenerate_golden_files "$version" done diff --git a/dev/release/release-tarball.sh b/dev/release/release-tarball.sh index 19eb4b449d..268614cd19 100755 --- a/dev/release/release-tarball.sh +++ b/dev/release/release-tarball.sh @@ -43,6 +43,13 @@ fi version=$1 rc=$2 +read -r -p "Proceed to release tarball for ${version}-rc${rc}? [y/N]: " answer +answer=${answer:-no} +if [ "${answer}" != "y" ]; then + echo "Cancelled tarball release!" + exit 1 +fi + tmp_dir=tmp-apache-datafusion-comet-dist echo "Recreate temporary directory: ${tmp_dir}" diff --git a/docs/source/contributor-guide/development.md b/docs/source/contributor-guide/development.md index 52519e7c60..14a67ff848 100644 --- a/docs/source/contributor-guide/development.md +++ b/docs/source/contributor-guide/development.md @@ -191,33 +191,24 @@ Spark version, and runs the plan stability tests with `SPARK_GENERATE_GOLDEN_FIL Alternatively, you can run the tests manually using the following commands. -First, Comet needs to be installed for each Spark version to be tested: - -```sh -./mvnw install -DskipTests -Pspark-3.4 -./mvnw install -DskipTests -Pspark-3.5 -# note that Spark 4.0 requires JDK 17 or later -./mvnw install -DskipTests -Pspark-4.0 -``` - Note that the output files get written to `$SPARK_HOME`. The tests can be run with: ```sh export SPARK_HOME=`pwd` -./mvnw -pl spark -Dsuites="org.apache.spark.sql.comet.CometTPCDSV1_4_PlanStabilitySuite" -Pspark-3.4 -nsu test -./mvnw -pl spark -Dsuites="org.apache.spark.sql.comet.CometTPCDSV1_4_PlanStabilitySuite" -Pspark-3.5 -nsu test -./mvnw -pl spark -Dsuites="org.apache.spark.sql.comet.CometTPCDSV1_4_PlanStabilitySuite" -Pspark-4.0 -nsu test +./mvnw -Dsuites="org.apache.spark.sql.comet.CometTPCDSV1_4_PlanStabilitySuite" -Pspark-3.4 -nsu test +./mvnw -Dsuites="org.apache.spark.sql.comet.CometTPCDSV1_4_PlanStabilitySuite" -Pspark-3.5 -nsu test +./mvnw -Dsuites="org.apache.spark.sql.comet.CometTPCDSV1_4_PlanStabilitySuite" -Pspark-4.0 -nsu test ``` and ```sh export SPARK_HOME=`pwd` -./mvnw -pl spark -Dsuites="org.apache.spark.sql.comet.CometTPCDSV2_7_PlanStabilitySuite" -Pspark-3.4 -nsu test -./mvnw -pl spark -Dsuites="org.apache.spark.sql.comet.CometTPCDSV2_7_PlanStabilitySuite" -Pspark-3.5 -nsu test -./mvnw -pl spark -Dsuites="org.apache.spark.sql.comet.CometTPCDSV2_7_PlanStabilitySuite" -Pspark-4.0 -nsu test +./mvnw -Dsuites="org.apache.spark.sql.comet.CometTPCDSV2_7_PlanStabilitySuite" -Pspark-3.4 -nsu test +./mvnw -Dsuites="org.apache.spark.sql.comet.CometTPCDSV2_7_PlanStabilitySuite" -Pspark-3.5 -nsu test +./mvnw -Dsuites="org.apache.spark.sql.comet.CometTPCDSV2_7_PlanStabilitySuite" -Pspark-4.0 -nsu test ``` If your pull request changes the query plans generated by Comet, you should regenerate the golden files. @@ -225,18 +216,18 @@ To regenerate the golden files, you can run the following commands. ```sh export SPARK_HOME=`pwd` -SPARK_GENERATE_GOLDEN_FILES=1 ./mvnw -pl spark -Dsuites="org.apache.spark.sql.comet.CometTPCDSV1_4_PlanStabilitySuite" -Pspark-3.4 -nsu test -SPARK_GENERATE_GOLDEN_FILES=1 ./mvnw -pl spark -Dsuites="org.apache.spark.sql.comet.CometTPCDSV1_4_PlanStabilitySuite" -Pspark-3.5 -nsu test -SPARK_GENERATE_GOLDEN_FILES=1 ./mvnw -pl spark -Dsuites="org.apache.spark.sql.comet.CometTPCDSV1_4_PlanStabilitySuite" -Pspark-4.0 -nsu test +SPARK_GENERATE_GOLDEN_FILES=1 ./mvnw -Dsuites="org.apache.spark.sql.comet.CometTPCDSV1_4_PlanStabilitySuite" -Pspark-3.4 -nsu test +SPARK_GENERATE_GOLDEN_FILES=1 ./mvnw -Dsuites="org.apache.spark.sql.comet.CometTPCDSV1_4_PlanStabilitySuite" -Pspark-3.5 -nsu test +SPARK_GENERATE_GOLDEN_FILES=1 ./mvnw -Dsuites="org.apache.spark.sql.comet.CometTPCDSV1_4_PlanStabilitySuite" -Pspark-4.0 -nsu test ``` and ```sh export SPARK_HOME=`pwd` -SPARK_GENERATE_GOLDEN_FILES=1 ./mvnw -pl spark -Dsuites="org.apache.spark.sql.comet.CometTPCDSV2_7_PlanStabilitySuite" -Pspark-3.4 -nsu test -SPARK_GENERATE_GOLDEN_FILES=1 ./mvnw -pl spark -Dsuites="org.apache.spark.sql.comet.CometTPCDSV2_7_PlanStabilitySuite" -Pspark-3.5 -nsu test -SPARK_GENERATE_GOLDEN_FILES=1 ./mvnw -pl spark -Dsuites="org.apache.spark.sql.comet.CometTPCDSV2_7_PlanStabilitySuite" -Pspark-4.0 -nsu test +SPARK_GENERATE_GOLDEN_FILES=1 ./mvnw -Dsuites="org.apache.spark.sql.comet.CometTPCDSV2_7_PlanStabilitySuite" -Pspark-3.4 -nsu test +SPARK_GENERATE_GOLDEN_FILES=1 ./mvnw -Dsuites="org.apache.spark.sql.comet.CometTPCDSV2_7_PlanStabilitySuite" -Pspark-3.5 -nsu test +SPARK_GENERATE_GOLDEN_FILES=1 ./mvnw -Dsuites="org.apache.spark.sql.comet.CometTPCDSV2_7_PlanStabilitySuite" -Pspark-4.0 -nsu test ``` ## Benchmark diff --git a/docs/source/contributor-guide/sql-file-tests.md b/docs/source/contributor-guide/sql-file-tests.md index f521cb4adf..1a55323b84 100644 --- a/docs/source/contributor-guide/sql-file-tests.md +++ b/docs/source/contributor-guide/sql-file-tests.md @@ -25,10 +25,21 @@ way to add expression and operator test coverage without writing Scala test code ## Running the tests +Run all SQL file tests: + +```shell +./mvnw test -Dsuites="org.apache.comet.CometSqlFileTestSuite" -Dtest=none +``` + +Run a single test file by adding the file name (without `.sql` extension) after the suite name: + ```shell -mvn test -pl spark -Dsuites="org.apache.comet.CometSqlFileTestSuite" -Dtest=none +./mvnw test -Dsuites="org.apache.comet.CometSqlFileTestSuite create_named_struct" -Dtest=none ``` +This uses ScalaTest's substring matching, so the argument must match part of the test name. +Test names follow the pattern `sql-file: expressions//.sql []`. + ## Test file location SQL test files live under: @@ -183,6 +194,27 @@ query ignore(https://github.com/apache/datafusion-comet/issues/3326) SELECT space(n) FROM test_space WHERE n < 0 ``` +#### `query expect_error()` + +Asserts that both Spark and Comet throw an exception containing the given pattern. Use this +for ANSI mode tests where invalid operations should throw errors. + +```sql +-- Config: spark.sql.ansi.enabled=true + +-- integer overflow should throw in ANSI mode +query expect_error(ARITHMETIC_OVERFLOW) +SELECT 2147483647 + 1 + +-- division by zero should throw in ANSI mode +query expect_error(DIVIDE_BY_ZERO) +SELECT 1 / 0 + +-- array out of bounds should throw in ANSI mode +query expect_error(INVALID_ARRAY_INDEX) +SELECT array(1, 2, 3)[10] +``` + ## Adding a new test 1. Create a `.sql` file under the appropriate subdirectory in @@ -208,7 +240,7 @@ SELECT space(n) FROM test_space WHERE n < 0 6. Run the tests to verify: ```shell - mvn test -pl spark -Dsuites="org.apache.comet.CometSqlFileTestSuite" -Dtest=none + ./mvnw test -Dsuites="org.apache.comet.CometSqlFileTestSuite" -Dtest=none ``` ### Tips for writing thorough tests diff --git a/docs/source/user-guide/latest/installation.md b/docs/source/user-guide/latest/installation.md index ee583931d1..f04e6359ba 100644 --- a/docs/source/user-guide/latest/installation.md +++ b/docs/source/user-guide/latest/installation.md @@ -60,6 +60,12 @@ Cloud Service Providers. Comet jar files are available in [Maven Central](https://central.sonatype.com/namespace/org.apache.datafusion) for amd64 and arm64 architectures for Linux. For Apple macOS, it is currently necessary to build from source. +For performance reasons, published Comet jar files target baseline CPUs available in modern data centers. For example, +the amd64 build uses the `x86-64-v3` target that adds CPU instructions (_e.g._, AVX2) common after 2013. Similarly, the +arm64 build uses the `neoverse-n1` target, which is a common baseline for ARM cores found in AWS (Graviton2+), GCP, and +Azure after 2019. If the Comet library fails for SIGILL (illegal instruction), please open an issue on the GitHub +repository describing your environment, and [build from source] for your target architecture. + Here are the direct links for downloading the Comet $COMET_VERSION jar file. Note that these links are not valid if you are viewing the documentation for the latest `SNAPSHOT` release, or for the latest release while it is still going through the release vote. Release candidate jars can be found [here](https://repository.apache.org/#nexus-search;quick~org.apache.datafusion). @@ -72,10 +78,10 @@ through the release vote. Release candidate jars can be found [here](https://rep ## Building from source -Refer to the [Building from Source] guide for instructions from building Comet from source, either from official +Refer to the [Building from source] guide for instructions from building Comet from source, either from official source releases, or from the latest code in the GitHub repository. -[Building from Source]: source.md +[Building from source]: source.md ## Deploying to Kubernetes diff --git a/docs/spark_expressions_support.md b/docs/spark_expressions_support.md index 27b6ad3b59..2c18cbd08d 100644 --- a/docs/spark_expressions_support.md +++ b/docs/spark_expressions_support.md @@ -272,11 +272,11 @@ - [ ] element_at - [ ] map - [ ] map_concat -- [ ] map_contains_key +- [x] map_contains_key - [ ] map_entries - [ ] map_from_arrays - [ ] map_from_entries -- [ ] map_keys +- [x] map_keys - [ ] map_values - [ ] str_to_map - [ ] try_element_at diff --git a/native/Cargo.lock b/native/Cargo.lock index 4adf9ed06e..c1224c2a06 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -162,9 +162,9 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2a2b10dcb159faf30d3f81f6d56c1211a5bea2ca424eabe477648a44b993320e" +checksum = "e4754a624e5ae42081f464514be454b39711daae0458906dacde5f4c632f33a8" dependencies = [ "arrow-arith", "arrow-array", @@ -183,9 +183,9 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "288015089e7931843c80ed4032c5274f02b37bcb720c4a42096d50b390e70372" +checksum = "f7b3141e0ec5145a22d8694ea8b6d6f69305971c4fa1c1a13ef0195aef2d678b" dependencies = [ "arrow-array", "arrow-buffer", @@ -197,9 +197,9 @@ dependencies = [ [[package]] name = "arrow-array" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "65ca404ea6191e06bf30956394173337fa9c35f445bd447fe6c21ab944e1a23c" +checksum = "4c8955af33b25f3b175ee10af580577280b4bd01f7e823d94c7cdef7cf8c9aef" dependencies = [ "ahash 0.8.12", "arrow-buffer", @@ -216,9 +216,9 @@ dependencies = [ [[package]] name = "arrow-buffer" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "36356383099be0151dacc4245309895f16ba7917d79bdb71a7148659c9206c56" +checksum = "c697ddca96183182f35b3a18e50b9110b11e916d7b7799cbfd4d34662f2c56c2" dependencies = [ "bytes", "half", @@ -228,9 +228,9 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c8e372ed52bd4ee88cc1e6c3859aa7ecea204158ac640b10e187936e7e87074" +checksum = "646bbb821e86fd57189c10b4fcdaa941deaf4181924917b0daa92735baa6ada5" dependencies = [ "arrow-array", "arrow-buffer", @@ -250,9 +250,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e4100b729fe656f2e4fb32bc5884f14acf9118d4ad532b7b33c1132e4dce896" +checksum = "8da746f4180004e3ce7b83c977daf6394d768332349d3d913998b10a120b790a" dependencies = [ "arrow-array", "arrow-cast", @@ -265,9 +265,9 @@ dependencies = [ [[package]] name = "arrow-data" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bf87f4ff5fc13290aa47e499a8b669a82c5977c6a1fedce22c7f542c1fd5a597" +checksum = "1fdd994a9d28e6365aa78e15da3f3950c0fdcea6b963a12fa1c391afb637b304" dependencies = [ "arrow-buffer", "arrow-schema", @@ -278,9 +278,9 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb3ca63edd2073fcb42ba112f8ae165df1de935627ead6e203d07c99445f2081" +checksum = "abf7df950701ab528bf7c0cf7eeadc0445d03ef5d6ffc151eaae6b38a58feff1" dependencies = [ "arrow-array", "arrow-buffer", @@ -293,9 +293,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a36b2332559d3310ebe3e173f75b29989b4412df4029a26a30cc3f7da0869297" +checksum = "0ff8357658bedc49792b13e2e862b80df908171275f8e6e075c460da5ee4bf86" dependencies = [ "arrow-array", "arrow-buffer", @@ -317,9 +317,9 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13c4e0530272ca755d6814218dffd04425c5b7854b87fa741d5ff848bf50aa39" +checksum = "f7d8f1870e03d4cbed632959498bcc84083b5a24bded52905ae1695bd29da45b" dependencies = [ "arrow-array", "arrow-buffer", @@ -330,9 +330,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b07f52788744cc71c4628567ad834cadbaeb9f09026ff1d7a4120f69edf7abd3" +checksum = "18228633bad92bff92a95746bbeb16e5fc318e8382b75619dec26db79e4de4c0" dependencies = [ "arrow-array", "arrow-buffer", @@ -343,9 +343,9 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6bb63203e8e0e54b288d0d8043ca8fa1013820822a27692ef1b78a977d879f2c" +checksum = "8c872d36b7bf2a6a6a2b40de9156265f0242910791db366a2c17476ba8330d68" dependencies = [ "bitflags 2.10.0", "serde_core", @@ -354,9 +354,9 @@ dependencies = [ [[package]] name = "arrow-select" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c96d8a1c180b44ecf2e66c9a2f2bbcb8b1b6f14e165ce46ac8bde211a363411b" +checksum = "68bf3e3efbd1278f770d67e5dc410257300b161b93baedb3aae836144edcaf4b" dependencies = [ "ahash 0.8.12", "arrow-array", @@ -368,9 +368,9 @@ dependencies = [ [[package]] name = "arrow-string" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8ad6a81add9d3ea30bf8374ee8329992c7fd246ffd8b7e2f48a3cea5aa0cc9a" +checksum = "85e968097061b3c0e9fe3079cf2e703e487890700546b5b0647f60fca1b5a8d8" dependencies = [ "arrow-array", "arrow-buffer", @@ -559,9 +559,9 @@ checksum = "c08606f8c3cbf4ce6ec8e28fb0014a2c086708fe954eaa885384a6165172e7e8" [[package]] name = "aws-config" -version = "1.8.12" +version = "1.8.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96571e6996817bf3d58f6b569e4b9fd2e9d2fcf9f7424eed07b2ce9bb87535e5" +checksum = "c456581cb3c77fafcc8c67204a70680d40b61112d6da78c77bd31d945b65f1b5" dependencies = [ "aws-credential-types", "aws-runtime", @@ -623,9 +623,9 @@ dependencies = [ [[package]] name = "aws-runtime" -version = "1.5.18" +version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "959dab27ce613e6c9658eb3621064d0e2027e5f2acb65bc526a43577facea557" +checksum = "c635c2dc792cb4a11ce1a4f392a925340d1bdf499289b5ec1ec6810954eb43f5" dependencies = [ "aws-credential-types", "aws-sigv4", @@ -637,8 +637,8 @@ dependencies = [ "aws-types", "bytes", "fastrand", - "http 0.2.12", - "http-body 0.4.6", + "http 1.4.0", + "http-body 1.0.1", "percent-encoding", "pin-project-lite", "tracing", @@ -647,9 +647,9 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "1.92.0" +version = "1.93.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b7d63bd2bdeeb49aa3f9b00c15e18583503b778b2e792fc06284d54e7d5b6566" +checksum = "9dcb38bb33fc0a11f1ffc3e3e85669e0a11a37690b86f77e75306d8f369146a0" dependencies = [ "aws-credential-types", "aws-runtime", @@ -664,15 +664,16 @@ dependencies = [ "bytes", "fastrand", "http 0.2.12", + "http 1.4.0", "regex-lite", "tracing", ] [[package]] name = "aws-sdk-ssooidc" -version = "1.94.0" +version = "1.95.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "532d93574bf731f311bafb761366f9ece345a0416dbcc273d81d6d1a1205239b" +checksum = "2ada8ffbea7bd1be1f53df1dadb0f8fdb04badb13185b3321b929d1ee3caad09" dependencies = [ "aws-credential-types", "aws-runtime", @@ -687,15 +688,16 @@ dependencies = [ "bytes", "fastrand", "http 0.2.12", + "http 1.4.0", "regex-lite", "tracing", ] [[package]] name = "aws-sdk-sts" -version = "1.96.0" +version = "1.97.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "357e9a029c7524db6a0099cd77fbd5da165540339e7296cca603531bc783b56c" +checksum = "e6443ccadc777095d5ed13e21f5c364878c9f5bad4e35187a6cdbd863b0afcad" dependencies = [ "aws-credential-types", "aws-runtime", @@ -711,15 +713,16 @@ dependencies = [ "aws-types", "fastrand", "http 0.2.12", + "http 1.4.0", "regex-lite", "tracing", ] [[package]] name = "aws-sigv4" -version = "1.3.7" +version = "1.3.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "69e523e1c4e8e7e8ff219d732988e22bfeae8a1cafdbe6d9eca1546fa080be7c" +checksum = "efa49f3c607b92daae0c078d48a4571f599f966dce3caee5f1ea55c4d9073f99" dependencies = [ "aws-credential-types", "aws-smithy-http", @@ -739,9 +742,9 @@ dependencies = [ [[package]] name = "aws-smithy-async" -version = "1.2.7" +version = "1.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ee19095c7c4dda59f1697d028ce704c24b2d33c6718790c7f1d5a3015b4107c" +checksum = "52eec3db979d18cb807fc1070961cc51d87d069abe9ab57917769687368a8c6c" dependencies = [ "futures-util", "pin-project-lite", @@ -750,9 +753,9 @@ dependencies = [ [[package]] name = "aws-smithy-http" -version = "0.62.6" +version = "0.63.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "826141069295752372f8203c17f28e30c464d22899a43a0c9fd9c458d469c88b" +checksum = "630e67f2a31094ffa51b210ae030855cb8f3b7ee1329bdd8d085aaf61e8b97fc" dependencies = [ "aws-smithy-runtime-api", "aws-smithy-types", @@ -760,9 +763,9 @@ dependencies = [ "bytes-utils", "futures-core", "futures-util", - "http 0.2.12", "http 1.4.0", - "http-body 0.4.6", + "http-body 1.0.1", + "http-body-util", "percent-encoding", "pin-project-lite", "pin-utils", @@ -771,9 +774,9 @@ dependencies = [ [[package]] name = "aws-smithy-http-client" -version = "1.1.5" +version = "1.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59e62db736db19c488966c8d787f52e6270be565727236fd5579eaa301e7bc4a" +checksum = "12fb0abf49ff0cab20fd31ac1215ed7ce0ea92286ba09e2854b42ba5cabe7525" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", @@ -795,27 +798,27 @@ dependencies = [ [[package]] name = "aws-smithy-json" -version = "0.61.9" +version = "0.62.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49fa1213db31ac95288d981476f78d05d9cbb0353d22cdf3472cc05bb02f6551" +checksum = "3cb96aa208d62ee94104645f7b2ecaf77bf27edf161590b6224bfbac2832f979" dependencies = [ "aws-smithy-types", ] [[package]] name = "aws-smithy-observability" -version = "0.2.0" +version = "0.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ef1fcbefc7ece1d70dcce29e490f269695dfca2d2bacdeaf9e5c3f799e4e6a42" +checksum = "c0a46543fbc94621080b3cf553eb4cbbdc41dd9780a30c4756400f0139440a1d" dependencies = [ "aws-smithy-runtime-api", ] [[package]] name = "aws-smithy-query" -version = "0.60.9" +version = "0.60.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ae5d689cf437eae90460e944a58b5668530d433b4ff85789e69d2f2a556e057d" +checksum = "0cebbddb6f3a5bd81553643e9c7daf3cc3dc5b0b5f398ac668630e8a84e6fff0" dependencies = [ "aws-smithy-types", "urlencoding", @@ -823,9 +826,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime" -version = "1.9.8" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb5b6167fcdf47399024e81ac08e795180c576a20e4d4ce67949f9a88ae37dc1" +checksum = "f3df87c14f0127a0d77eb261c3bc45d5b4833e2a1f63583ebfb728e4852134ee" dependencies = [ "aws-smithy-async", "aws-smithy-http", @@ -839,6 +842,7 @@ dependencies = [ "http 1.4.0", "http-body 0.4.6", "http-body 1.0.1", + "http-body-util", "pin-project-lite", "pin-utils", "tokio", @@ -847,9 +851,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime-api" -version = "1.10.0" +version = "1.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "efce7aaaf59ad53c5412f14fc19b2d5c6ab2c3ec688d272fd31f76ec12f44fb0" +checksum = "49952c52f7eebb72ce2a754d3866cc0f87b97d2a46146b79f80f3a93fb2b3716" dependencies = [ "aws-smithy-async", "aws-smithy-types", @@ -864,9 +868,9 @@ dependencies = [ [[package]] name = "aws-smithy-types" -version = "1.3.6" +version = "1.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "65f172bcb02424eb94425db8aed1b6d583b5104d4d5ddddf22402c661a320048" +checksum = "3b3a26048eeab0ddeba4b4f9d51654c79af8c3b32357dc5f336cee85ab331c33" dependencies = [ "base64-simd", "bytes", @@ -1056,7 +1060,7 @@ dependencies = [ "cc", "cfg-if", "constant_time_eq", - "cpufeatures", + "cpufeatures 0.2.17", ] [[package]] @@ -1192,9 +1196,9 @@ checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" -version = "1.11.0" +version = "1.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b35204fbdc0b3f4446b89fc1ac2cf84a8a68971995d0bf2e925ec7cd960f9cb3" +checksum = "1e748733b7cbc798e1434b6ac524f0c1ff2ab456fe201501e6497c8417a4fc33" [[package]] name = "bytes-utils" @@ -1242,9 +1246,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.2.54" +version = "1.2.55" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6354c81bbfd62d9cfa9cb3c773c2b7b2a3a482d569de977fd0e961f6e7c00583" +checksum = "47b26a0954ae34af09b50f0de26458fa95369a0d478d8236d3f93082b219bd29" dependencies = [ "find-msvc-tools", "jobserver", @@ -1279,6 +1283,17 @@ version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "613afe47fcd5fac7ccf1db93babcb082c5994d996f20b8b159f2ad1658eb5724" +[[package]] +name = "chacha20" +version = "0.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6f8d983286843e49675a4b7a2d174efe136dc93a18d69130dd18198a6c167601" +dependencies = [ + "cfg-if", + "cpufeatures 0.3.0", + "rand_core 0.10.0", +] + [[package]] name = "chrono" version = "0.4.43" @@ -1470,6 +1485,15 @@ dependencies = [ "libc", ] +[[package]] +name = "cpufeatures" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b2a41393f66f16b0823bb79094d54ac5fbd34ab292ddafb9a0456ac9f87d201" +dependencies = [ + "libc", +] + [[package]] name = "crc32c" version = "0.6.8" @@ -1855,7 +1879,7 @@ dependencies = [ "pprof", "procfs", "prost", - "rand 0.9.2", + "rand 0.10.0", "regex", "reqwest", "serde_json", @@ -1921,7 +1945,7 @@ dependencies = [ "futures", "hex", "num", - "rand 0.9.2", + "rand 0.10.0", "regex", "serde_json", "thiserror 2.0.18", @@ -2705,9 +2729,9 @@ checksum = "37909eebbb50d72f9059c3b6d82c0463f2ff062c9e95845c43a6c9c0355411be" [[package]] name = "find-msvc-tools" -version = "0.1.8" +version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8591b0bcc8a98a64310a2fae1bb3e9b8564dd10e381e6e28010fde8e8e8568db" +checksum = "5baebc0774151f905a1a2cc41989300b1e6fbb29aff0ceffa1064fdd3088d582" [[package]] name = "findshlibs" @@ -2940,6 +2964,20 @@ dependencies = [ "wasm-bindgen", ] +[[package]] +name = "getrandom" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "139ef39800118c7683f2fd3c98c1b23c09ae076556b435f8e9064ae108aaeeec" +dependencies = [ + "cfg-if", + "libc", + "r-efi", + "rand_core 0.10.0", + "wasip2", + "wasip3", +] + [[package]] name = "gimli" version = "0.32.3" @@ -3380,6 +3418,12 @@ dependencies = [ "zerovec", ] +[[package]] +name = "id-arena" +version = "2.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3d3067d79b975e8844ca9eb072e16b31c3c1c36928edf9c6789548c524d0d954" + [[package]] name = "ident_case" version = "1.0.1" @@ -3620,6 +3664,12 @@ version = "1.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "830d08ce1d1d941e6b30645f1a0eb5643013d835ce3779a5fc208261dbe10f55" +[[package]] +name = "leb128fmt" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09edd9e8b54e49e587e4f6295a7d29c3ea94d469cb40ab8ca70b288248a81db2" + [[package]] name = "lexical-core" version = "1.0.6" @@ -3979,9 +4029,9 @@ dependencies = [ [[package]] name = "num-conv" -version = "0.1.0" +version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "51d515d32fb182ee37cda2ccdcb92950d6a3c2893aa280e540671c2cd0f3b1d9" +checksum = "cf97ec579c3c42f953ef76dbf8d55ac91fb219dde70e49aa4a6b7d74e9919050" [[package]] name = "num-format" @@ -4748,6 +4798,17 @@ dependencies = [ "rand_core 0.9.5", ] +[[package]] +name = "rand" +version = "0.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bc266eb313df6c5c09c1c7b1fbe2510961e5bcd3add930c1e31f7ed9da0feff8" +dependencies = [ + "chacha20", + "getrandom 0.4.1", + "rand_core 0.10.0", +] + [[package]] name = "rand_chacha" version = "0.3.1" @@ -4786,6 +4847,12 @@ dependencies = [ "getrandom 0.3.4", ] +[[package]] +name = "rand_core" +version = "0.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c8d0fd677905edcbeedbf2edb6494d676f0e98d54d5cf9bda0b061cb8fb8aba" + [[package]] name = "rayon" version = "1.11.0" @@ -4837,9 +4904,9 @@ dependencies = [ [[package]] name = "regex" -version = "1.12.2" +version = "1.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "843bc0191f75f3e22651ae5f1e72939ab2f72a4bc30fa80a066bd66edefc24d4" +checksum = "e10754a14b9137dd7b1e3e5b0493cc9171fdd105e0ab477f51b72e7f3ac0e276" dependencies = [ "aho-corasick", "memchr", @@ -5423,7 +5490,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e3bf829a2d51ab4a5ddf1352d8470c140cadc8301b2ae1789db023f01cedd6ba" dependencies = [ "cfg-if", - "cpufeatures", + "cpufeatures 0.2.17", "digest", ] @@ -5434,7 +5501,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a7507d819769d01a365ab707794a4084392c824f54a7a6a7862f8c3d0892b283" dependencies = [ "cfg-if", - "cpufeatures", + "cpufeatures 0.2.17", "digest", ] @@ -5759,9 +5826,9 @@ dependencies = [ [[package]] name = "time" -version = "0.3.45" +version = "0.3.47" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f9e442fc33d7fdb45aa9bfeb312c095964abdf596f7567261062b2a7107aaabd" +checksum = "743bd48c283afc0388f9b8827b976905fb217ad9e647fae3a379a9283c4def2c" dependencies = [ "deranged", "itoa", @@ -5774,15 +5841,15 @@ dependencies = [ [[package]] name = "time-core" -version = "0.1.7" +version = "0.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8b36ee98fd31ec7426d599183e8fe26932a8dc1fb76ddb6214d05493377d34ca" +checksum = "7694e1cfe791f8d31026952abf09c69ca6f6fa4e1a1229e18988f06a04a12dca" [[package]] name = "time-macros" -version = "0.2.25" +version = "0.2.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "71e552d1249bf61ac2a52db88179fd0673def1e1ad8243a00d9ec9ed71fee3dd" +checksum = "2e70e4c5a0e0a8a4823ad65dfe1a6930e4f4d756dcd9dd7939022b5e8c501215" dependencies = [ "num-conv", "time-core", @@ -6172,6 +6239,15 @@ dependencies = [ "wit-bindgen", ] +[[package]] +name = "wasip3" +version = "0.4.0+wasi-0.3.0-rc-2026-01-06" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5428f8bf88ea5ddc08faddef2ac4a67e390b88186c703ce6dbd955e1c145aca5" +dependencies = [ + "wit-bindgen", +] + [[package]] name = "wasm-bindgen" version = "0.2.108" @@ -6231,6 +6307,28 @@ dependencies = [ "unicode-ident", ] +[[package]] +name = "wasm-encoder" +version = "0.244.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "990065f2fe63003fe337b932cfb5e3b80e0b4d0f5ff650e6985b1048f62c8319" +dependencies = [ + "leb128fmt", + "wasmparser", +] + +[[package]] +name = "wasm-metadata" +version = "0.244.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bb0e353e6a2fbdc176932bbaab493762eb1255a7900fe0fea1a2f96c296cc909" +dependencies = [ + "anyhow", + "indexmap 2.13.0", + "wasm-encoder", + "wasmparser", +] + [[package]] name = "wasm-streams" version = "0.4.2" @@ -6244,6 +6342,18 @@ dependencies = [ "web-sys", ] +[[package]] +name = "wasmparser" +version = "0.244.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "47b807c72e1bac69382b3a6fb3dbe8ea4c0ed87ff5629b8685ae6b9a611028fe" +dependencies = [ + "bitflags 2.10.0", + "hashbrown 0.15.5", + "indexmap 2.13.0", + "semver", +] + [[package]] name = "web-sys" version = "0.3.85" @@ -6620,6 +6730,88 @@ name = "wit-bindgen" version = "0.51.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d7249219f66ced02969388cf2bb044a09756a083d0fab1e566056b04d9fbcaa5" +dependencies = [ + "wit-bindgen-rust-macro", +] + +[[package]] +name = "wit-bindgen-core" +version = "0.51.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ea61de684c3ea68cb082b7a88508a8b27fcc8b797d738bfc99a82facf1d752dc" +dependencies = [ + "anyhow", + "heck", + "wit-parser", +] + +[[package]] +name = "wit-bindgen-rust" +version = "0.51.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b7c566e0f4b284dd6561c786d9cb0142da491f46a9fbed79ea69cdad5db17f21" +dependencies = [ + "anyhow", + "heck", + "indexmap 2.13.0", + "prettyplease", + "syn 2.0.114", + "wasm-metadata", + "wit-bindgen-core", + "wit-component", +] + +[[package]] +name = "wit-bindgen-rust-macro" +version = "0.51.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c0f9bfd77e6a48eccf51359e3ae77140a7f50b1e2ebfe62422d8afdaffab17a" +dependencies = [ + "anyhow", + "prettyplease", + "proc-macro2", + "quote", + "syn 2.0.114", + "wit-bindgen-core", + "wit-bindgen-rust", +] + +[[package]] +name = "wit-component" +version = "0.244.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9d66ea20e9553b30172b5e831994e35fbde2d165325bec84fc43dbf6f4eb9cb2" +dependencies = [ + "anyhow", + "bitflags 2.10.0", + "indexmap 2.13.0", + "log", + "serde", + "serde_derive", + "serde_json", + "wasm-encoder", + "wasm-metadata", + "wasmparser", + "wit-parser", +] + +[[package]] +name = "wit-parser" +version = "0.244.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ecc8ac4bc1dc3381b7f59c34f00b67e18f910c2c0f50015669dde7def656a736" +dependencies = [ + "anyhow", + "id-arena", + "indexmap 2.13.0", + "log", + "semver", + "serde", + "serde_derive", + "serde_json", + "unicode-xid", + "wasmparser", +] [[package]] name = "writeable" diff --git a/native/Cargo.toml b/native/Cargo.toml index 216057f9bd..03e7f6e915 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -34,9 +34,9 @@ edition = "2021" rust-version = "1.88" [workspace.dependencies] -arrow = { version = "57.2.0", features = ["prettyprint", "ffi", "chrono-tz"] } +arrow = { version = "57.3.0", features = ["prettyprint", "ffi", "chrono-tz"] } async-trait = { version = "0.1" } -bytes = { version = "1.10.0" } +bytes = { version = "1.11.1" } parquet = { version = "57.2.0", default-features = false, features = ["experimental"] } datafusion = { version = "51.0.0", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] } datafusion-datasource = { version = "51.0.0" } @@ -47,12 +47,12 @@ chrono = { version = "0.4", default-features = false, features = ["clock"] } chrono-tz = { version = "0.10" } futures = "0.3.28" num = "0.4" -rand = "0.9" -regex = "1.11.3" +rand = "0.10" +regex = "1.12.3" thiserror = "2" object_store = { version = "0.12.3", features = ["gcp", "azure", "aws", "http"] } url = "2.2" -aws-config = "1.8.10" +aws-config = "1.8.13" aws-credential-types = "1.2.9" iceberg = { git = "https://github.com/apache/iceberg-rust", rev = "ee21563" } diff --git a/native/core/benches/bit_util.rs b/native/core/benches/bit_util.rs index 8a250e27c1..92ae497a56 100644 --- a/native/core/benches/bit_util.rs +++ b/native/core/benches/bit_util.rs @@ -17,7 +17,7 @@ use std::{mem::size_of, time::Duration}; -use rand::{rng, Rng}; +use rand::{rng, RngExt}; use arrow::buffer::Buffer; use comet::common::bit::{ diff --git a/native/core/benches/common.rs b/native/core/benches/common.rs index 06f90cc5c3..6eb0319645 100644 --- a/native/core/benches/common.rs +++ b/native/core/benches/common.rs @@ -23,7 +23,7 @@ use arrow::{ use rand::{ distr::{Distribution, StandardUniform}, rngs::StdRng, - Rng, SeedableRng, + RngExt, SeedableRng, }; use std::sync::Arc; diff --git a/native/core/benches/parquet_read.rs b/native/core/benches/parquet_read.rs index 659a895e7a..a56c748e0f 100644 --- a/native/core/benches/parquet_read.rs +++ b/native/core/benches/parquet_read.rs @@ -36,7 +36,7 @@ use comet::parquet::util::test_common::page_util::{ }; use perf::FlamegraphProfiler; -use rand::{prelude::StdRng, Rng, SeedableRng}; +use rand::{prelude::StdRng, RngExt, SeedableRng}; use zstd::zstd_safe::WriteBuf; fn bench(c: &mut Criterion) { diff --git a/native/core/src/common/bit.rs b/native/core/src/common/bit.rs index 15055e9d62..459c38a52b 100644 --- a/native/core/src/common/bit.rs +++ b/native/core/src/common/bit.rs @@ -997,7 +997,7 @@ mod tests { use rand::{ distr::{Distribution, StandardUniform}, - Rng, + RngExt, }; use std::fmt::Debug; diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index 2f639e9f70..bc20592e90 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -44,6 +44,7 @@ use crate::parquet::parquet_support::SparkParquetOptions; use crate::parquet::schema_adapter::SparkSchemaAdapterFactory; use datafusion::datasource::schema_adapter::{SchemaAdapterFactory, SchemaMapper}; use datafusion_comet_spark_expr::EvalMode; +use iceberg::scan::FileScanTask; /// Iceberg table scan operator that uses iceberg-rust to read Iceberg tables. /// @@ -58,8 +59,8 @@ pub struct IcebergScanExec { plan_properties: PlanProperties, /// Catalog-specific configuration for FileIO catalog_properties: HashMap, - /// Pre-planned file scan tasks, grouped by partition - file_task_groups: Vec>, + /// Pre-planned file scan tasks + tasks: Vec, /// Metrics metrics: ExecutionPlanMetricsSet, } @@ -69,11 +70,10 @@ impl IcebergScanExec { metadata_location: String, schema: SchemaRef, catalog_properties: HashMap, - file_task_groups: Vec>, + tasks: Vec, ) -> Result { let output_schema = schema; - let num_partitions = file_task_groups.len(); - let plan_properties = Self::compute_properties(Arc::clone(&output_schema), num_partitions); + let plan_properties = Self::compute_properties(Arc::clone(&output_schema), 1); let metrics = ExecutionPlanMetricsSet::new(); @@ -82,7 +82,7 @@ impl IcebergScanExec { output_schema, plan_properties, catalog_properties, - file_task_groups, + tasks, metrics, }) } @@ -127,19 +127,10 @@ impl ExecutionPlan for IcebergScanExec { fn execute( &self, - partition: usize, + _partition: usize, context: Arc, ) -> DFResult { - if partition < self.file_task_groups.len() { - let tasks = &self.file_task_groups[partition]; - self.execute_with_tasks(tasks.clone(), partition, context) - } else { - Err(DataFusionError::Execution(format!( - "IcebergScanExec: Partition index {} out of range (only {} task groups available)", - partition, - self.file_task_groups.len() - ))) - } + self.execute_with_tasks(self.tasks.clone(), context) } fn metrics(&self) -> Option { @@ -152,15 +143,14 @@ impl IcebergScanExec { /// deletes via iceberg-rust's ArrowReader. fn execute_with_tasks( &self, - tasks: Vec, - partition: usize, + tasks: Vec, context: Arc, ) -> DFResult { let output_schema = Arc::clone(&self.output_schema); let file_io = Self::load_file_io(&self.catalog_properties, &self.metadata_location)?; let batch_size = context.session_config().batch_size(); - let metrics = IcebergScanMetrics::new(&self.metrics, partition); + let metrics = IcebergScanMetrics::new(&self.metrics); let num_tasks = tasks.len(); metrics.num_splits.add(num_tasks); @@ -221,10 +211,10 @@ struct IcebergScanMetrics { } impl IcebergScanMetrics { - fn new(metrics: &ExecutionPlanMetricsSet, partition: usize) -> Self { + fn new(metrics: &ExecutionPlanMetricsSet) -> Self { Self { - baseline: BaselineMetrics::new(metrics, partition), - num_splits: MetricBuilder::new(metrics).counter("num_splits", partition), + baseline: BaselineMetrics::new(metrics, 0), + num_splits: MetricBuilder::new(metrics).counter("num_splits", 0), } } } @@ -311,11 +301,11 @@ where impl DisplayAs for IcebergScanExec { fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { - let num_tasks: usize = self.file_task_groups.iter().map(|g| g.len()).sum(); write!( f, "IcebergScanExec: metadata_location={}, num_tasks={}", - self.metadata_location, num_tasks + self.metadata_location, + self.tasks.len() ) } } diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 44ff20a44f..2c3d00a23b 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -71,7 +71,7 @@ use datafusion::{ }; use datafusion_comet_spark_expr::{ create_comet_physical_fun, create_comet_physical_fun_with_eval_mode, BinaryOutputStyle, - BloomFilterAgg, BloomFilterMightContain, EvalMode, SumInteger, + BloomFilterAgg, BloomFilterMightContain, CsvWriteOptions, EvalMode, SumInteger, ToCsv, }; use iceberg::expr::Bind; @@ -585,6 +585,25 @@ impl PhysicalPlanner { ExprStruct::MonotonicallyIncreasingId(_) => Ok(Arc::new( MonotonicallyIncreasingId::from_partition_id(self.partition), )), + ExprStruct::ToCsv(expr) => { + let csv_struct_expr = + self.create_expr(expr.child.as_ref().unwrap(), Arc::clone(&input_schema))?; + let options = expr.options.clone().unwrap(); + let csv_write_options = CsvWriteOptions::new( + options.delimiter, + options.quote, + options.escape, + options.null_value, + options.quote_all, + options.ignore_leading_white_space, + options.ignore_trailing_white_space, + ); + Ok(Arc::new(ToCsv::new( + csv_struct_expr, + &options.timezone, + csv_write_options, + ))) + } expr => Err(GeneralError(format!("Not implemented: {expr:?}"))), } } @@ -1132,33 +1151,28 @@ impl PhysicalPlanner { )) } OpStruct::IcebergScan(scan) => { - let required_schema: SchemaRef = - convert_spark_types_to_arrow_schema(scan.required_schema.as_slice()); + // Extract common data and single partition's file tasks + // Per-partition injection happens in Scala before sending to native + let common = scan + .common + .as_ref() + .ok_or_else(|| GeneralError("IcebergScan missing common data".into()))?; - let catalog_properties: HashMap = scan + let required_schema = + convert_spark_types_to_arrow_schema(common.required_schema.as_slice()); + let catalog_properties: HashMap = common .catalog_properties .iter() .map(|(k, v)| (k.clone(), v.clone())) .collect(); - - let metadata_location = scan.metadata_location.clone(); - - debug_assert!( - !scan.file_partitions.is_empty(), - "IcebergScan must have at least one file partition. This indicates a bug in Scala serialization." - ); - - let tasks = parse_file_scan_tasks( - scan, - &scan.file_partitions[self.partition as usize].file_scan_tasks, - )?; - let file_task_groups = vec![tasks]; + let metadata_location = common.metadata_location.clone(); + let tasks = parse_file_scan_tasks_from_common(common, &scan.file_scan_tasks)?; let iceberg_scan = IcebergScanExec::new( metadata_location, required_schema, catalog_properties, - file_task_groups, + tasks, )?; Ok(( @@ -2743,15 +2757,14 @@ fn partition_data_to_struct( /// Each task contains a residual predicate that is used for row-group level filtering /// during Parquet scanning. /// -/// This function uses deduplication pools from the IcebergScan to avoid redundant parsing -/// of schemas, partition specs, partition types, name mappings, and other repeated data. -fn parse_file_scan_tasks( - proto_scan: &spark_operator::IcebergScan, +/// This function uses deduplication pools from the IcebergScanCommon to avoid redundant +/// parsing of schemas, partition specs, partition types, name mappings, and other repeated data. +fn parse_file_scan_tasks_from_common( + proto_common: &spark_operator::IcebergScanCommon, proto_tasks: &[spark_operator::IcebergFileScanTask], ) -> Result, ExecutionError> { - // Build caches upfront: for 10K tasks with 1 schema, this parses the schema - // once instead of 10K times, eliminating redundant JSON deserialization - let schema_cache: Vec> = proto_scan + // Parse each unique schema once, not once per task + let schema_cache: Vec> = proto_common .schema_pool .iter() .map(|json| { @@ -2764,7 +2777,7 @@ fn parse_file_scan_tasks( }) .collect::, _>>()?; - let partition_spec_cache: Vec>> = proto_scan + let partition_spec_cache: Vec>> = proto_common .partition_spec_pool .iter() .map(|json| { @@ -2774,7 +2787,7 @@ fn parse_file_scan_tasks( }) .collect(); - let name_mapping_cache: Vec>> = proto_scan + let name_mapping_cache: Vec>> = proto_common .name_mapping_pool .iter() .map(|json| { @@ -2784,7 +2797,7 @@ fn parse_file_scan_tasks( }) .collect(); - let delete_files_cache: Vec> = proto_scan + let delete_files_cache: Vec> = proto_common .delete_files_pool .iter() .map(|list| { @@ -2796,7 +2809,7 @@ fn parse_file_scan_tasks( "EQUALITY_DELETES" => iceberg::spec::DataContentType::EqualityDeletes, other => { return Err(GeneralError(format!( - "Invalid delete content type '{}'. This indicates a bug in Scala serialization.", + "Invalid delete content type '{}'", other ))) } @@ -2817,7 +2830,6 @@ fn parse_file_scan_tasks( }) .collect::, _>>()?; - // Partition data pool is in protobuf messages let results: Result, _> = proto_tasks .iter() .map(|proto_task| { @@ -2851,7 +2863,7 @@ fn parse_file_scan_tasks( }; let bound_predicate = if let Some(idx) = proto_task.residual_idx { - proto_scan + proto_common .residual_pool .get(idx as usize) .and_then(convert_spark_expr_to_predicate) @@ -2871,24 +2883,22 @@ fn parse_file_scan_tasks( }; let partition = if let Some(partition_data_idx) = proto_task.partition_data_idx { - // Get partition data from protobuf pool - let partition_data_proto = proto_scan + let partition_data_proto = proto_common .partition_data_pool .get(partition_data_idx as usize) .ok_or_else(|| { ExecutionError::GeneralError(format!( "Invalid partition_data_idx: {} (pool size: {})", partition_data_idx, - proto_scan.partition_data_pool.len() + proto_common.partition_data_pool.len() )) })?; - // Convert protobuf PartitionData to iceberg Struct match partition_data_to_struct(partition_data_proto) { Ok(s) => Some(s), Err(e) => { return Err(ExecutionError::GeneralError(format!( - "Failed to deserialize partition data from protobuf: {}", + "Failed to deserialize partition data: {}", e ))) } @@ -2907,14 +2917,14 @@ fn parse_file_scan_tasks( .and_then(|idx| name_mapping_cache.get(idx as usize)) .and_then(|opt| opt.clone()); - let project_field_ids = proto_scan + let project_field_ids = proto_common .project_field_ids_pool .get(proto_task.project_field_ids_idx as usize) .ok_or_else(|| { ExecutionError::GeneralError(format!( "Invalid project_field_ids_idx: {} (pool size: {})", proto_task.project_field_ids_idx, - proto_scan.project_field_ids_pool.len() + proto_common.project_field_ids_pool.len() )) })? .field_ids diff --git a/native/core/src/execution/shuffle/comet_partitioning.rs b/native/core/src/execution/shuffle/comet_partitioning.rs index b7ad158790..b8d68cd21e 100644 --- a/native/core/src/execution/shuffle/comet_partitioning.rs +++ b/native/core/src/execution/shuffle/comet_partitioning.rs @@ -46,3 +46,26 @@ impl CometPartitioning { } } } + +pub(super) fn pmod(hash: u32, n: usize) -> usize { + let hash = hash as i32; + let n = n as i32; + let r = hash % n; + let result = if r < 0 { (r + n) % n } else { r }; + result as usize +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_pmod() { + let i: Vec = vec![0x99f0149d, 0x9c67b85d, 0xc8008529, 0xa05b5d7b, 0xcd1e64fb]; + let result = i.into_iter().map(|i| pmod(i, 200)).collect::>(); + + // expected partition from Spark with n=200 + let expected = vec![69, 5, 193, 171, 115]; + assert_eq!(result, expected); + } +} diff --git a/native/core/src/execution/shuffle/mod.rs b/native/core/src/execution/shuffle/mod.rs index a72258322a..6018cff50f 100644 --- a/native/core/src/execution/shuffle/mod.rs +++ b/native/core/src/execution/shuffle/mod.rs @@ -18,8 +18,10 @@ pub(crate) mod codec; mod comet_partitioning; mod metrics; +mod partitioners; mod shuffle_writer; pub mod spark_unsafe; +mod writers; pub use codec::{read_ipc_compressed, CompressionCodec, ShuffleBlockWriter}; pub use comet_partitioning::CometPartitioning; diff --git a/native/core/src/execution/shuffle/partitioners/mod.rs b/native/core/src/execution/shuffle/partitioners/mod.rs new file mode 100644 index 0000000000..b9058f66f4 --- /dev/null +++ b/native/core/src/execution/shuffle/partitioners/mod.rs @@ -0,0 +1,35 @@ +// 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. + +mod multi_partition; +mod partitioned_batch_iterator; +mod single_partition; + +use arrow::record_batch::RecordBatch; +use datafusion::common::Result; + +pub(super) use multi_partition::MultiPartitionShuffleRepartitioner; +pub(super) use partitioned_batch_iterator::PartitionedBatchIterator; +pub(super) use single_partition::SinglePartitionShufflePartitioner; + +#[async_trait::async_trait] +pub(super) trait ShufflePartitioner: Send + Sync { + /// Insert a batch into the partitioner + async fn insert_batch(&mut self, batch: RecordBatch) -> Result<()>; + /// Write shuffle data and shuffle index file to disk + fn shuffle_write(&mut self) -> Result<()>; +} diff --git a/native/core/src/execution/shuffle/partitioners/multi_partition.rs b/native/core/src/execution/shuffle/partitioners/multi_partition.rs new file mode 100644 index 0000000000..9c366ad462 --- /dev/null +++ b/native/core/src/execution/shuffle/partitioners/multi_partition.rs @@ -0,0 +1,642 @@ +// 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 crate::execution::shuffle::metrics::ShufflePartitionerMetrics; +use crate::execution::shuffle::partitioners::partitioned_batch_iterator::{ + PartitionedBatchIterator, PartitionedBatchesProducer, +}; +use crate::execution::shuffle::partitioners::ShufflePartitioner; +use crate::execution::shuffle::writers::{BufBatchWriter, PartitionWriter}; +use crate::execution::shuffle::{ + comet_partitioning, CometPartitioning, CompressionCodec, ShuffleBlockWriter, +}; +use crate::execution::tracing::{with_trace, with_trace_async}; +use arrow::array::{ArrayRef, RecordBatch}; +use arrow::datatypes::SchemaRef; +use datafusion::common::utils::proxy::VecAllocExt; +use datafusion::common::DataFusionError; +use datafusion::execution::memory_pool::{MemoryConsumer, MemoryReservation}; +use datafusion::execution::runtime_env::RuntimeEnv; +use datafusion::physical_plan::metrics::Time; +use datafusion_comet_spark_expr::murmur3::create_murmur3_hashes; +use itertools::Itertools; +use std::fmt; +use std::fmt::{Debug, Formatter}; +use std::fs::{File, OpenOptions}; +use std::io::{BufReader, BufWriter, Seek, Write}; +use std::sync::Arc; +use tokio::time::Instant; + +#[derive(Default)] +struct ScratchSpace { + /// Hashes for each row in the current batch. + hashes_buf: Vec, + /// Partition ids for each row in the current batch. + partition_ids: Vec, + /// The row indices of the rows in each partition. This array is conceptually divided into + /// partitions, where each partition contains the row indices of the rows in that partition. + /// The length of this array is the same as the number of rows in the batch. + partition_row_indices: Vec, + /// The start indices of partitions in partition_row_indices. partition_starts[K] and + /// partition_starts[K + 1] are the start and end indices of partition K in partition_row_indices. + /// The length of this array is 1 + the number of partitions. + partition_starts: Vec, +} + +impl ScratchSpace { + fn map_partition_ids_to_starts_and_indices( + &mut self, + num_output_partitions: usize, + num_rows: usize, + ) { + let partition_ids = &mut self.partition_ids[..num_rows]; + + // count each partition size, while leaving the last extra element as 0 + let partition_counters = &mut self.partition_starts; + partition_counters.resize(num_output_partitions + 1, 0); + partition_counters.fill(0); + partition_ids + .iter() + .for_each(|partition_id| partition_counters[*partition_id as usize] += 1); + + // accumulate partition counters into partition ends + // e.g. partition counter: [1, 3, 2, 1, 0] => [1, 4, 6, 7, 7] + let partition_ends = partition_counters; + let mut accum = 0; + partition_ends.iter_mut().for_each(|v| { + *v += accum; + accum = *v; + }); + + // calculate partition row indices and partition starts + // e.g. partition ids: [3, 1, 1, 1, 2, 2, 0] will produce the following partition_row_indices + // and partition_starts arrays: + // + // partition_row_indices: [6, 1, 2, 3, 4, 5, 0] + // partition_starts: [0, 1, 4, 6, 7] + // + // partition_starts conceptually splits partition_row_indices into smaller slices. + // Each slice partition_row_indices[partition_starts[K]..partition_starts[K + 1]] contains the + // row indices of the input batch that are partitioned into partition K. For example, + // first partition 0 has one row index [6], partition 1 has row indices [1, 2, 3], etc. + let partition_row_indices = &mut self.partition_row_indices; + partition_row_indices.resize(num_rows, 0); + for (index, partition_id) in partition_ids.iter().enumerate().rev() { + partition_ends[*partition_id as usize] -= 1; + let end = partition_ends[*partition_id as usize]; + partition_row_indices[end as usize] = index as u32; + } + + // after calculating, partition ends become partition starts + } +} + +/// A partitioner that uses a hash function to partition data into multiple partitions +pub(crate) struct MultiPartitionShuffleRepartitioner { + output_data_file: String, + output_index_file: String, + buffered_batches: Vec, + partition_indices: Vec>, + partition_writers: Vec, + shuffle_block_writer: ShuffleBlockWriter, + /// Partitioning scheme to use + partitioning: CometPartitioning, + runtime: Arc, + metrics: ShufflePartitionerMetrics, + /// Reused scratch space for computing partition indices + scratch: ScratchSpace, + /// The configured batch size + batch_size: usize, + /// Reservation for repartitioning + reservation: MemoryReservation, + tracing_enabled: bool, + /// Size of the write buffer in bytes + write_buffer_size: usize, +} + +impl MultiPartitionShuffleRepartitioner { + #[allow(clippy::too_many_arguments)] + pub(crate) fn try_new( + partition: usize, + output_data_file: String, + output_index_file: String, + schema: SchemaRef, + partitioning: CometPartitioning, + metrics: ShufflePartitionerMetrics, + runtime: Arc, + batch_size: usize, + codec: CompressionCodec, + tracing_enabled: bool, + write_buffer_size: usize, + ) -> datafusion::common::Result { + let num_output_partitions = partitioning.partition_count(); + assert_ne!( + num_output_partitions, 1, + "Use SinglePartitionShufflePartitioner for 1 output partition." + ); + + // Vectors in the scratch space will be filled with valid values before being used, this + // initialization code is simply initializing the vectors to the desired size. + // The initial values are not used. + let scratch = ScratchSpace { + hashes_buf: match partitioning { + // Allocate hashes_buf for hash and round robin partitioning. + // Round robin hashes all columns to achieve even, deterministic distribution. + CometPartitioning::Hash(_, _) | CometPartitioning::RoundRobin(_, _) => { + vec![0; batch_size] + } + _ => vec![], + }, + partition_ids: vec![0; batch_size], + partition_row_indices: vec![0; batch_size], + partition_starts: vec![0; num_output_partitions + 1], + }; + + let shuffle_block_writer = ShuffleBlockWriter::try_new(schema.as_ref(), codec.clone())?; + + let partition_writers = (0..num_output_partitions) + .map(|_| PartitionWriter::try_new(shuffle_block_writer.clone())) + .collect::>>()?; + + let reservation = MemoryConsumer::new(format!("ShuffleRepartitioner[{partition}]")) + .with_can_spill(true) + .register(&runtime.memory_pool); + + Ok(Self { + output_data_file, + output_index_file, + buffered_batches: vec![], + partition_indices: vec![vec![]; num_output_partitions], + partition_writers, + shuffle_block_writer, + partitioning, + runtime, + metrics, + scratch, + batch_size, + reservation, + tracing_enabled, + write_buffer_size, + }) + } + + /// Shuffles rows in input batch into corresponding partition buffer. + /// This function first calculates hashes for rows and then takes rows in same + /// partition as a record batch which is appended into partition buffer. + /// This should not be called directly. Use `insert_batch` instead. + async fn partitioning_batch(&mut self, input: RecordBatch) -> datafusion::common::Result<()> { + if input.num_rows() == 0 { + // skip empty batch + return Ok(()); + } + + if input.num_rows() > self.batch_size { + return Err(DataFusionError::Internal( + "Input batch size exceeds configured batch size. Call `insert_batch` instead." + .to_string(), + )); + } + + // Update data size metric + self.metrics.data_size.add(input.get_array_memory_size()); + + // NOTE: in shuffle writer exec, the output_rows metrics represents the + // number of rows those are written to output data file. + self.metrics.baseline.record_output(input.num_rows()); + + match &self.partitioning { + CometPartitioning::Hash(exprs, num_output_partitions) => { + let mut scratch = std::mem::take(&mut self.scratch); + let (partition_starts, partition_row_indices): (&Vec, &Vec) = { + let mut timer = self.metrics.repart_time.timer(); + + // Evaluate partition expressions to get rows to apply partitioning scheme. + let arrays = exprs + .iter() + .map(|expr| expr.evaluate(&input)?.into_array(input.num_rows())) + .collect::>>()?; + + let num_rows = arrays[0].len(); + + // Use identical seed as Spark hash partitioning. + let hashes_buf = &mut scratch.hashes_buf[..num_rows]; + hashes_buf.fill(42_u32); + + // Generate partition ids for every row. + { + // Hash arrays and compute partition ids based on number of partitions. + let partition_ids = &mut scratch.partition_ids[..num_rows]; + create_murmur3_hashes(&arrays, hashes_buf)? + .iter() + .enumerate() + .for_each(|(idx, hash)| { + partition_ids[idx] = + comet_partitioning::pmod(*hash, *num_output_partitions) as u32; + }); + } + + // We now have partition ids for every input row, map that to partition starts + // and partition indices to eventually right these rows to partition buffers. + scratch + .map_partition_ids_to_starts_and_indices(*num_output_partitions, num_rows); + + timer.stop(); + Ok::<(&Vec, &Vec), DataFusionError>(( + &scratch.partition_starts, + &scratch.partition_row_indices, + )) + }?; + + self.buffer_partitioned_batch_may_spill( + input, + partition_row_indices, + partition_starts, + ) + .await?; + self.scratch = scratch; + } + CometPartitioning::RangePartitioning( + lex_ordering, + num_output_partitions, + row_converter, + bounds, + ) => { + let mut scratch = std::mem::take(&mut self.scratch); + let (partition_starts, partition_row_indices): (&Vec, &Vec) = { + let mut timer = self.metrics.repart_time.timer(); + + // Evaluate partition expressions for values to apply partitioning scheme on. + let arrays = lex_ordering + .iter() + .map(|expr| expr.expr.evaluate(&input)?.into_array(input.num_rows())) + .collect::>>()?; + + let num_rows = arrays[0].len(); + + // Generate partition ids for every row, first by converting the partition + // arrays to Rows, and then doing binary search for each Row against the + // bounds Rows. + { + let row_batch = row_converter.convert_columns(arrays.as_slice())?; + let partition_ids = &mut scratch.partition_ids[..num_rows]; + + row_batch.iter().enumerate().for_each(|(row_idx, row)| { + partition_ids[row_idx] = bounds + .as_slice() + .partition_point(|bound| bound.row() <= row) + as u32 + }); + } + + // We now have partition ids for every input row, map that to partition starts + // and partition indices to eventually right these rows to partition buffers. + scratch + .map_partition_ids_to_starts_and_indices(*num_output_partitions, num_rows); + + timer.stop(); + Ok::<(&Vec, &Vec), DataFusionError>(( + &scratch.partition_starts, + &scratch.partition_row_indices, + )) + }?; + + self.buffer_partitioned_batch_may_spill( + input, + partition_row_indices, + partition_starts, + ) + .await?; + self.scratch = scratch; + } + CometPartitioning::RoundRobin(num_output_partitions, max_hash_columns) => { + // Comet implements "round robin" as hash partitioning on columns. + // This achieves the same goal as Spark's round robin (even distribution + // without semantic grouping) while being deterministic for fault tolerance. + // + // Note: This produces different partition assignments than Spark's round robin, + // which sorts by UnsafeRow binary representation before assigning partitions. + // However, both approaches provide even distribution and determinism. + let mut scratch = std::mem::take(&mut self.scratch); + let (partition_starts, partition_row_indices): (&Vec, &Vec) = { + let mut timer = self.metrics.repart_time.timer(); + + let num_rows = input.num_rows(); + + // Collect columns for hashing, respecting max_hash_columns limit + // max_hash_columns of 0 means no limit (hash all columns) + // Negative values are normalized to 0 in the planner + let num_columns_to_hash = if *max_hash_columns == 0 { + input.num_columns() + } else { + (*max_hash_columns).min(input.num_columns()) + }; + let columns_to_hash: Vec = (0..num_columns_to_hash) + .map(|i| Arc::clone(input.column(i))) + .collect(); + + // Use identical seed as Spark hash partitioning. + let hashes_buf = &mut scratch.hashes_buf[..num_rows]; + hashes_buf.fill(42_u32); + + // Compute hash for selected columns + create_murmur3_hashes(&columns_to_hash, hashes_buf)?; + + // Assign partition IDs based on hash (same as hash partitioning) + let partition_ids = &mut scratch.partition_ids[..num_rows]; + hashes_buf.iter().enumerate().for_each(|(idx, hash)| { + partition_ids[idx] = + comet_partitioning::pmod(*hash, *num_output_partitions) as u32; + }); + + // We now have partition ids for every input row, map that to partition starts + // and partition indices to eventually write these rows to partition buffers. + scratch + .map_partition_ids_to_starts_and_indices(*num_output_partitions, num_rows); + + timer.stop(); + Ok::<(&Vec, &Vec), DataFusionError>(( + &scratch.partition_starts, + &scratch.partition_row_indices, + )) + }?; + + self.buffer_partitioned_batch_may_spill( + input, + partition_row_indices, + partition_starts, + ) + .await?; + self.scratch = scratch; + } + other => { + // this should be unreachable as long as the validation logic + // in the constructor is kept up-to-date + return Err(DataFusionError::NotImplemented(format!( + "Unsupported shuffle partitioning scheme {other:?}" + ))); + } + } + Ok(()) + } + + async fn buffer_partitioned_batch_may_spill( + &mut self, + input: RecordBatch, + partition_row_indices: &[u32], + partition_starts: &[u32], + ) -> datafusion::common::Result<()> { + let mut mem_growth: usize = input.get_array_memory_size(); + let buffered_partition_idx = self.buffered_batches.len() as u32; + self.buffered_batches.push(input); + + // partition_starts conceptually slices partition_row_indices into smaller slices, + // each slice contains the indices of rows in input that will go into the corresponding + // partition. The following loop iterates over the slices and put the row indices into + // the indices array of the corresponding partition. + for (partition_id, (&start, &end)) in partition_starts + .iter() + .tuple_windows() + .enumerate() + .filter(|(_, (start, end))| start < end) + { + let row_indices = &partition_row_indices[start as usize..end as usize]; + + // Put row indices for the current partition into the indices array of that partition. + // This indices array will be used for calling interleave_record_batch to produce + // shuffled batches. + let indices = &mut self.partition_indices[partition_id]; + let before_size = indices.allocated_size(); + indices.reserve(row_indices.len()); + for row_idx in row_indices { + indices.push((buffered_partition_idx, *row_idx)); + } + let after_size = indices.allocated_size(); + mem_growth += after_size.saturating_sub(before_size); + } + + if self.reservation.try_grow(mem_growth).is_err() { + self.spill()?; + } + + Ok(()) + } + + fn shuffle_write_partition( + partition_iter: &mut PartitionedBatchIterator, + shuffle_block_writer: &mut ShuffleBlockWriter, + output_data: &mut BufWriter, + encode_time: &Time, + write_time: &Time, + write_buffer_size: usize, + batch_size: usize, + ) -> datafusion::common::Result<()> { + let mut buf_batch_writer = BufBatchWriter::new( + shuffle_block_writer, + output_data, + write_buffer_size, + batch_size, + ); + for batch in partition_iter { + let batch = batch?; + buf_batch_writer.write(&batch, encode_time, write_time)?; + } + buf_batch_writer.flush(encode_time, write_time)?; + Ok(()) + } + + fn used(&self) -> usize { + self.reservation.size() + } + + fn spilled_bytes(&self) -> usize { + self.metrics.spilled_bytes.value() + } + + fn spill_count(&self) -> usize { + self.metrics.spill_count.value() + } + + fn data_size(&self) -> usize { + self.metrics.data_size.value() + } + + /// This function transfers the ownership of the buffered batches and partition indices from the + /// ShuffleRepartitioner to a new PartitionedBatches struct. The returned PartitionedBatches struct + /// can be used to produce shuffled batches. + fn partitioned_batches(&mut self) -> PartitionedBatchesProducer { + let num_output_partitions = self.partition_indices.len(); + let buffered_batches = std::mem::take(&mut self.buffered_batches); + // let indices = std::mem::take(&mut self.partition_indices); + let indices = std::mem::replace( + &mut self.partition_indices, + vec![vec![]; num_output_partitions], + ); + PartitionedBatchesProducer::new(buffered_batches, indices, self.batch_size) + } + + pub(crate) fn spill(&mut self) -> datafusion::common::Result<()> { + log::info!( + "ShuffleRepartitioner spilling shuffle data of {} to disk while inserting ({} time(s) so far)", + self.used(), + self.spill_count() + ); + + // we could always get a chance to free some memory as long as we are holding some + if self.buffered_batches.is_empty() { + return Ok(()); + } + + with_trace("shuffle_spill", self.tracing_enabled, || { + let num_output_partitions = self.partition_writers.len(); + let mut partitioned_batches = self.partitioned_batches(); + let mut spilled_bytes = 0; + + for partition_id in 0..num_output_partitions { + let partition_writer = &mut self.partition_writers[partition_id]; + let mut iter = partitioned_batches.produce(partition_id); + spilled_bytes += partition_writer.spill( + &mut iter, + &self.runtime, + &self.metrics, + self.write_buffer_size, + self.batch_size, + )?; + } + + self.reservation.free(); + self.metrics.spill_count.add(1); + self.metrics.spilled_bytes.add(spilled_bytes); + Ok(()) + }) + } + + #[cfg(test)] + pub(crate) fn partition_writers(&self) -> &[PartitionWriter] { + &self.partition_writers + } +} + +#[async_trait::async_trait] +impl ShufflePartitioner for MultiPartitionShuffleRepartitioner { + /// Shuffles rows in input batch into corresponding partition buffer. + /// This function will slice input batch according to configured batch size and then + /// shuffle rows into corresponding partition buffer. + async fn insert_batch(&mut self, batch: RecordBatch) -> datafusion::common::Result<()> { + with_trace_async("shuffle_insert_batch", self.tracing_enabled, || async { + let start_time = Instant::now(); + let mut start = 0; + while start < batch.num_rows() { + let end = (start + self.batch_size).min(batch.num_rows()); + let batch = batch.slice(start, end - start); + self.partitioning_batch(batch).await?; + start = end; + } + self.metrics.input_batches.add(1); + self.metrics + .baseline + .elapsed_compute() + .add_duration(start_time.elapsed()); + Ok(()) + }) + .await + } + + /// Writes buffered shuffled record batches into Arrow IPC bytes. + fn shuffle_write(&mut self) -> datafusion::common::Result<()> { + with_trace("shuffle_write", self.tracing_enabled, || { + let start_time = Instant::now(); + + let mut partitioned_batches = self.partitioned_batches(); + let num_output_partitions = self.partition_indices.len(); + let mut offsets = vec![0; num_output_partitions + 1]; + + let data_file = self.output_data_file.clone(); + let index_file = self.output_index_file.clone(); + + let output_data = OpenOptions::new() + .write(true) + .create(true) + .truncate(true) + .open(data_file) + .map_err(|e| DataFusionError::Execution(format!("shuffle write error: {e:?}")))?; + + let mut output_data = BufWriter::new(output_data); + + #[allow(clippy::needless_range_loop)] + for i in 0..num_output_partitions { + offsets[i] = output_data.stream_position()?; + + // if we wrote a spill file for this partition then copy the + // contents into the shuffle file + if let Some(spill_path) = self.partition_writers[i].path() { + let mut spill_file = BufReader::new(File::open(spill_path)?); + let mut write_timer = self.metrics.write_time.timer(); + std::io::copy(&mut spill_file, &mut output_data)?; + write_timer.stop(); + } + + // Write in memory batches to output data file + let mut partition_iter = partitioned_batches.produce(i); + Self::shuffle_write_partition( + &mut partition_iter, + &mut self.shuffle_block_writer, + &mut output_data, + &self.metrics.encode_time, + &self.metrics.write_time, + self.write_buffer_size, + self.batch_size, + )?; + } + + let mut write_timer = self.metrics.write_time.timer(); + output_data.flush()?; + write_timer.stop(); + + // add one extra offset at last to ease partition length computation + offsets[num_output_partitions] = output_data.stream_position()?; + + let mut write_timer = self.metrics.write_time.timer(); + let mut output_index = + BufWriter::new(File::create(index_file).map_err(|e| { + DataFusionError::Execution(format!("shuffle write error: {e:?}")) + })?); + for offset in offsets { + output_index.write_all(&(offset as i64).to_le_bytes()[..])?; + } + output_index.flush()?; + write_timer.stop(); + + self.metrics + .baseline + .elapsed_compute() + .add_duration(start_time.elapsed()); + + Ok(()) + }) + } +} + +impl Debug for MultiPartitionShuffleRepartitioner { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + f.debug_struct("ShuffleRepartitioner") + .field("memory_used", &self.used()) + .field("spilled_bytes", &self.spilled_bytes()) + .field("spilled_count", &self.spill_count()) + .field("data_size", &self.data_size()) + .finish() + } +} diff --git a/native/core/src/execution/shuffle/partitioners/partitioned_batch_iterator.rs b/native/core/src/execution/shuffle/partitioners/partitioned_batch_iterator.rs new file mode 100644 index 0000000000..77010938cd --- /dev/null +++ b/native/core/src/execution/shuffle/partitioners/partitioned_batch_iterator.rs @@ -0,0 +1,110 @@ +// 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 arrow::array::RecordBatch; +use arrow::compute::interleave_record_batch; +use datafusion::common::DataFusionError; + +/// A helper struct to produce shuffled batches. +/// This struct takes ownership of the buffered batches and partition indices from the +/// ShuffleRepartitioner, and provides an iterator over the batches in the specified partitions. +pub(super) struct PartitionedBatchesProducer { + buffered_batches: Vec, + partition_indices: Vec>, + batch_size: usize, +} + +impl PartitionedBatchesProducer { + pub(super) fn new( + buffered_batches: Vec, + indices: Vec>, + batch_size: usize, + ) -> Self { + Self { + partition_indices: indices, + buffered_batches, + batch_size, + } + } + + pub(super) fn produce(&mut self, partition_id: usize) -> PartitionedBatchIterator<'_> { + PartitionedBatchIterator::new( + &self.partition_indices[partition_id], + &self.buffered_batches, + self.batch_size, + ) + } +} + +pub(crate) struct PartitionedBatchIterator<'a> { + record_batches: Vec<&'a RecordBatch>, + batch_size: usize, + indices: Vec<(usize, usize)>, + pos: usize, +} + +impl<'a> PartitionedBatchIterator<'a> { + fn new( + indices: &'a [(u32, u32)], + buffered_batches: &'a [RecordBatch], + batch_size: usize, + ) -> Self { + if indices.is_empty() { + // Avoid unnecessary allocations when the partition is empty + return Self { + record_batches: vec![], + batch_size, + indices: vec![], + pos: 0, + }; + } + let record_batches = buffered_batches.iter().collect::>(); + let current_indices = indices + .iter() + .map(|(i_batch, i_row)| (*i_batch as usize, *i_row as usize)) + .collect::>(); + Self { + record_batches, + batch_size, + indices: current_indices, + pos: 0, + } + } +} + +impl Iterator for PartitionedBatchIterator<'_> { + type Item = datafusion::common::Result; + + fn next(&mut self) -> Option { + if self.pos >= self.indices.len() { + return None; + } + + let indices_end = std::cmp::min(self.pos + self.batch_size, self.indices.len()); + let indices = &self.indices[self.pos..indices_end]; + match interleave_record_batch(&self.record_batches, indices) { + Ok(batch) => { + self.pos = indices_end; + Some(Ok(batch)) + } + Err(e) => Some(Err(DataFusionError::ArrowError( + Box::from(e), + Some(DataFusionError::get_back_trace()), + ))), + } + } +} diff --git a/native/core/src/execution/shuffle/partitioners/single_partition.rs b/native/core/src/execution/shuffle/partitioners/single_partition.rs new file mode 100644 index 0000000000..eeca4458cc --- /dev/null +++ b/native/core/src/execution/shuffle/partitioners/single_partition.rs @@ -0,0 +1,192 @@ +// 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 crate::execution::shuffle::metrics::ShufflePartitionerMetrics; +use crate::execution::shuffle::partitioners::ShufflePartitioner; +use crate::execution::shuffle::writers::BufBatchWriter; +use crate::execution::shuffle::{CompressionCodec, ShuffleBlockWriter}; +use arrow::array::RecordBatch; +use arrow::datatypes::SchemaRef; +use datafusion::common::DataFusionError; +use std::fs::{File, OpenOptions}; +use std::io::{BufWriter, Write}; +use tokio::time::Instant; + +/// A partitioner that writes all shuffle data to a single file and a single index file +pub(crate) struct SinglePartitionShufflePartitioner { + // output_data_file: File, + output_data_writer: BufBatchWriter, + output_index_path: String, + /// Batches that are smaller than the batch size and to be concatenated + buffered_batches: Vec, + /// Number of rows in the concatenating batches + num_buffered_rows: usize, + /// Metrics for the repartitioner + metrics: ShufflePartitionerMetrics, + /// The configured batch size + batch_size: usize, +} + +impl SinglePartitionShufflePartitioner { + pub(crate) fn try_new( + output_data_path: String, + output_index_path: String, + schema: SchemaRef, + metrics: ShufflePartitionerMetrics, + batch_size: usize, + codec: CompressionCodec, + write_buffer_size: usize, + ) -> datafusion::common::Result { + let shuffle_block_writer = ShuffleBlockWriter::try_new(schema.as_ref(), codec.clone())?; + + let output_data_file = OpenOptions::new() + .write(true) + .create(true) + .truncate(true) + .open(output_data_path)?; + + let output_data_writer = BufBatchWriter::new( + shuffle_block_writer, + output_data_file, + write_buffer_size, + batch_size, + ); + + Ok(Self { + output_data_writer, + output_index_path, + buffered_batches: vec![], + num_buffered_rows: 0, + metrics, + batch_size, + }) + } + + /// Add a batch to the buffer of the partitioner, these buffered batches will be concatenated + /// and written to the output data file when the number of rows in the buffer reaches the batch size. + fn add_buffered_batch(&mut self, batch: RecordBatch) { + self.num_buffered_rows += batch.num_rows(); + self.buffered_batches.push(batch); + } + + /// Consumes buffered batches and return a concatenated batch if successful + fn concat_buffered_batches(&mut self) -> datafusion::common::Result> { + if self.buffered_batches.is_empty() { + Ok(None) + } else if self.buffered_batches.len() == 1 { + let batch = self.buffered_batches.remove(0); + self.num_buffered_rows = 0; + Ok(Some(batch)) + } else { + let schema = &self.buffered_batches[0].schema(); + match arrow::compute::concat_batches(schema, self.buffered_batches.iter()) { + Ok(concatenated) => { + self.buffered_batches.clear(); + self.num_buffered_rows = 0; + Ok(Some(concatenated)) + } + Err(e) => Err(DataFusionError::ArrowError( + Box::from(e), + Some(DataFusionError::get_back_trace()), + )), + } + } + } +} + +#[async_trait::async_trait] +impl ShufflePartitioner for SinglePartitionShufflePartitioner { + async fn insert_batch(&mut self, batch: RecordBatch) -> datafusion::common::Result<()> { + let start_time = Instant::now(); + let num_rows = batch.num_rows(); + + if num_rows > 0 { + self.metrics.data_size.add(batch.get_array_memory_size()); + self.metrics.baseline.record_output(num_rows); + + if num_rows >= self.batch_size || num_rows + self.num_buffered_rows > self.batch_size { + let concatenated_batch = self.concat_buffered_batches()?; + + // Write the concatenated buffered batch + if let Some(batch) = concatenated_batch { + self.output_data_writer.write( + &batch, + &self.metrics.encode_time, + &self.metrics.write_time, + )?; + } + + if num_rows >= self.batch_size { + // Write the new batch + self.output_data_writer.write( + &batch, + &self.metrics.encode_time, + &self.metrics.write_time, + )?; + } else { + // Add the new batch to the buffer + self.add_buffered_batch(batch); + } + } else { + self.add_buffered_batch(batch); + } + } + + self.metrics.input_batches.add(1); + self.metrics + .baseline + .elapsed_compute() + .add_duration(start_time.elapsed()); + Ok(()) + } + + fn shuffle_write(&mut self) -> datafusion::common::Result<()> { + let start_time = Instant::now(); + let concatenated_batch = self.concat_buffered_batches()?; + + // Write the concatenated buffered batch + if let Some(batch) = concatenated_batch { + self.output_data_writer.write( + &batch, + &self.metrics.encode_time, + &self.metrics.write_time, + )?; + } + self.output_data_writer + .flush(&self.metrics.encode_time, &self.metrics.write_time)?; + + // Write index file. It should only contain 2 entries: 0 and the total number of bytes written + let index_file = OpenOptions::new() + .write(true) + .create(true) + .truncate(true) + .open(self.output_index_path.clone()) + .map_err(|e| DataFusionError::Execution(format!("shuffle write error: {e:?}")))?; + let mut index_buf_writer = BufWriter::new(index_file); + let data_file_length = self.output_data_writer.writer_stream_position()?; + for offset in [0, data_file_length] { + index_buf_writer.write_all(&(offset as i64).to_le_bytes()[..])?; + } + index_buf_writer.flush()?; + + self.metrics + .baseline + .elapsed_compute() + .add_duration(start_time.elapsed()); + Ok(()) + } +} diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 5c68940b98..fe1bf0fccf 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -18,44 +18,34 @@ //! Defines the External shuffle repartition plan. use crate::execution::shuffle::metrics::ShufflePartitionerMetrics; -use crate::execution::shuffle::{CometPartitioning, CompressionCodec, ShuffleBlockWriter}; -use crate::execution::tracing::{with_trace, with_trace_async}; -use arrow::compute::interleave_record_batch; +use crate::execution::shuffle::partitioners::{ + MultiPartitionShuffleRepartitioner, ShufflePartitioner, SinglePartitionShufflePartitioner, +}; +use crate::execution::shuffle::{CometPartitioning, CompressionCodec}; +use crate::execution::tracing::with_trace_async; use async_trait::async_trait; -use datafusion::common::utils::proxy::VecAllocExt; +use datafusion::common::exec_datafusion_err; use datafusion::physical_expr::{EquivalenceProperties, Partitioning}; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::physical_plan::EmptyRecordBatchStream; use datafusion::{ - arrow::{array::*, datatypes::SchemaRef, error::ArrowError, record_batch::RecordBatch}, - error::{DataFusionError, Result}, - execution::{ - context::TaskContext, - disk_manager::RefCountedTempFile, - memory_pool::{MemoryConsumer, MemoryReservation}, - runtime_env::RuntimeEnv, - }, + arrow::{datatypes::SchemaRef, error::ArrowError}, + error::Result, + execution::context::TaskContext, physical_plan::{ - metrics::{ExecutionPlanMetricsSet, MetricsSet, Time}, + metrics::{ExecutionPlanMetricsSet, MetricsSet}, stream::RecordBatchStreamAdapter, DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, SendableRecordBatchStream, Statistics, }, }; -use datafusion_comet_spark_expr::hash_funcs::murmur3::create_murmur3_hashes; use futures::{StreamExt, TryFutureExt, TryStreamExt}; -use itertools::Itertools; -use std::borrow::Borrow; -use std::io::{Cursor, Error, SeekFrom}; use std::{ any::Any, fmt, fmt::{Debug, Formatter}, - fs::{File, OpenOptions}, - io::{BufReader, BufWriter, Seek, Write}, sync::Arc, }; -use tokio::time::Instant; /// The shuffle writer operator maps each input partition to M output partitions based on a /// partitioning scheme. No guarantees are made about the order of the resulting partitions. @@ -256,10 +246,15 @@ async fn external_shuffle( // into the corresponding partition buffer. // Otherwise, pull the next batch from the input stream might overwrite the // current batch in the repartitioner. - repartitioner.insert_batch(batch?).await?; + repartitioner + .insert_batch(batch?) + .await + .map_err(|err| exec_datafusion_err!("Error inserting batch: {err}"))?; } - repartitioner.shuffle_write()?; + repartitioner + .shuffle_write() + .map_err(|err| exec_datafusion_err!("Error in shuffle write: {err}"))?; // shuffle writer always has empty output Ok(Box::pin(EmptyRecordBatchStream::new(Arc::clone(&schema))) as SendableRecordBatchStream) @@ -267,1021 +262,24 @@ async fn external_shuffle( .await } -#[async_trait::async_trait] -trait ShufflePartitioner: Send + Sync { - /// Insert a batch into the partitioner - async fn insert_batch(&mut self, batch: RecordBatch) -> Result<()>; - /// Write shuffle data and shuffle index file to disk - fn shuffle_write(&mut self) -> Result<()>; -} - -/// A partitioner that uses a hash function to partition data into multiple partitions -struct MultiPartitionShuffleRepartitioner { - output_data_file: String, - output_index_file: String, - buffered_batches: Vec, - partition_indices: Vec>, - partition_writers: Vec, - shuffle_block_writer: ShuffleBlockWriter, - /// Partitioning scheme to use - partitioning: CometPartitioning, - runtime: Arc, - metrics: ShufflePartitionerMetrics, - /// Reused scratch space for computing partition indices - scratch: ScratchSpace, - /// The configured batch size - batch_size: usize, - /// Reservation for repartitioning - reservation: MemoryReservation, - tracing_enabled: bool, - /// Size of the write buffer in bytes - write_buffer_size: usize, -} - -#[derive(Default)] -struct ScratchSpace { - /// Hashes for each row in the current batch. - hashes_buf: Vec, - /// Partition ids for each row in the current batch. - partition_ids: Vec, - /// The row indices of the rows in each partition. This array is conceptually divided into - /// partitions, where each partition contains the row indices of the rows in that partition. - /// The length of this array is the same as the number of rows in the batch. - partition_row_indices: Vec, - /// The start indices of partitions in partition_row_indices. partition_starts[K] and - /// partition_starts[K + 1] are the start and end indices of partition K in partition_row_indices. - /// The length of this array is 1 + the number of partitions. - partition_starts: Vec, -} - -impl ScratchSpace { - fn map_partition_ids_to_starts_and_indices( - &mut self, - num_output_partitions: usize, - num_rows: usize, - ) { - let partition_ids = &mut self.partition_ids[..num_rows]; - - // count each partition size, while leaving the last extra element as 0 - let partition_counters = &mut self.partition_starts; - partition_counters.resize(num_output_partitions + 1, 0); - partition_counters.fill(0); - partition_ids - .iter() - .for_each(|partition_id| partition_counters[*partition_id as usize] += 1); - - // accumulate partition counters into partition ends - // e.g. partition counter: [1, 3, 2, 1, 0] => [1, 4, 6, 7, 7] - let partition_ends = partition_counters; - let mut accum = 0; - partition_ends.iter_mut().for_each(|v| { - *v += accum; - accum = *v; - }); - - // calculate partition row indices and partition starts - // e.g. partition ids: [3, 1, 1, 1, 2, 2, 0] will produce the following partition_row_indices - // and partition_starts arrays: - // - // partition_row_indices: [6, 1, 2, 3, 4, 5, 0] - // partition_starts: [0, 1, 4, 6, 7] - // - // partition_starts conceptually splits partition_row_indices into smaller slices. - // Each slice partition_row_indices[partition_starts[K]..partition_starts[K + 1]] contains the - // row indices of the input batch that are partitioned into partition K. For example, - // first partition 0 has one row index [6], partition 1 has row indices [1, 2, 3], etc. - let partition_row_indices = &mut self.partition_row_indices; - partition_row_indices.resize(num_rows, 0); - for (index, partition_id) in partition_ids.iter().enumerate().rev() { - partition_ends[*partition_id as usize] -= 1; - let end = partition_ends[*partition_id as usize]; - partition_row_indices[end as usize] = index as u32; - } - - // after calculating, partition ends become partition starts - } -} - -impl MultiPartitionShuffleRepartitioner { - #[allow(clippy::too_many_arguments)] - pub fn try_new( - partition: usize, - output_data_file: String, - output_index_file: String, - schema: SchemaRef, - partitioning: CometPartitioning, - metrics: ShufflePartitionerMetrics, - runtime: Arc, - batch_size: usize, - codec: CompressionCodec, - tracing_enabled: bool, - write_buffer_size: usize, - ) -> Result { - let num_output_partitions = partitioning.partition_count(); - assert_ne!( - num_output_partitions, 1, - "Use SinglePartitionShufflePartitioner for 1 output partition." - ); - - // Vectors in the scratch space will be filled with valid values before being used, this - // initialization code is simply initializing the vectors to the desired size. - // The initial values are not used. - let scratch = ScratchSpace { - hashes_buf: match partitioning { - // Allocate hashes_buf for hash and round robin partitioning. - // Round robin hashes all columns to achieve even, deterministic distribution. - CometPartitioning::Hash(_, _) | CometPartitioning::RoundRobin(_, _) => { - vec![0; batch_size] - } - _ => vec![], - }, - partition_ids: vec![0; batch_size], - partition_row_indices: vec![0; batch_size], - partition_starts: vec![0; num_output_partitions + 1], - }; - - let shuffle_block_writer = ShuffleBlockWriter::try_new(schema.as_ref(), codec.clone())?; - - let partition_writers = (0..num_output_partitions) - .map(|_| PartitionWriter::try_new(shuffle_block_writer.clone())) - .collect::>>()?; - - let reservation = MemoryConsumer::new(format!("ShuffleRepartitioner[{partition}]")) - .with_can_spill(true) - .register(&runtime.memory_pool); - - Ok(Self { - output_data_file, - output_index_file, - buffered_batches: vec![], - partition_indices: vec![vec![]; num_output_partitions], - partition_writers, - shuffle_block_writer, - partitioning, - runtime, - metrics, - scratch, - batch_size, - reservation, - tracing_enabled, - write_buffer_size, - }) - } - - /// Shuffles rows in input batch into corresponding partition buffer. - /// This function first calculates hashes for rows and then takes rows in same - /// partition as a record batch which is appended into partition buffer. - /// This should not be called directly. Use `insert_batch` instead. - async fn partitioning_batch(&mut self, input: RecordBatch) -> Result<()> { - if input.num_rows() == 0 { - // skip empty batch - return Ok(()); - } - - if input.num_rows() > self.batch_size { - return Err(DataFusionError::Internal( - "Input batch size exceeds configured batch size. Call `insert_batch` instead." - .to_string(), - )); - } - - // Update data size metric - self.metrics.data_size.add(input.get_array_memory_size()); - - // NOTE: in shuffle writer exec, the output_rows metrics represents the - // number of rows those are written to output data file. - self.metrics.baseline.record_output(input.num_rows()); - - match &self.partitioning { - CometPartitioning::Hash(exprs, num_output_partitions) => { - let mut scratch = std::mem::take(&mut self.scratch); - let (partition_starts, partition_row_indices): (&Vec, &Vec) = { - let mut timer = self.metrics.repart_time.timer(); - - // Evaluate partition expressions to get rows to apply partitioning scheme. - let arrays = exprs - .iter() - .map(|expr| expr.evaluate(&input)?.into_array(input.num_rows())) - .collect::>>()?; - - let num_rows = arrays[0].len(); - - // Use identical seed as Spark hash partitioning. - let hashes_buf = &mut scratch.hashes_buf[..num_rows]; - hashes_buf.fill(42_u32); - - // Generate partition ids for every row. - { - // Hash arrays and compute partition ids based on number of partitions. - let partition_ids = &mut scratch.partition_ids[..num_rows]; - create_murmur3_hashes(&arrays, hashes_buf)? - .iter() - .enumerate() - .for_each(|(idx, hash)| { - partition_ids[idx] = pmod(*hash, *num_output_partitions) as u32; - }); - } - - // We now have partition ids for every input row, map that to partition starts - // and partition indices to eventually right these rows to partition buffers. - scratch - .map_partition_ids_to_starts_and_indices(*num_output_partitions, num_rows); - - timer.stop(); - Ok::<(&Vec, &Vec), DataFusionError>(( - &scratch.partition_starts, - &scratch.partition_row_indices, - )) - }?; - - self.buffer_partitioned_batch_may_spill( - input, - partition_row_indices, - partition_starts, - ) - .await?; - self.scratch = scratch; - } - CometPartitioning::RangePartitioning( - lex_ordering, - num_output_partitions, - row_converter, - bounds, - ) => { - let mut scratch = std::mem::take(&mut self.scratch); - let (partition_starts, partition_row_indices): (&Vec, &Vec) = { - let mut timer = self.metrics.repart_time.timer(); - - // Evaluate partition expressions for values to apply partitioning scheme on. - let arrays = lex_ordering - .iter() - .map(|expr| expr.expr.evaluate(&input)?.into_array(input.num_rows())) - .collect::>>()?; - - let num_rows = arrays[0].len(); - - // Generate partition ids for every row, first by converting the partition - // arrays to Rows, and then doing binary search for each Row against the - // bounds Rows. - { - let row_batch = row_converter.convert_columns(arrays.as_slice())?; - let partition_ids = &mut scratch.partition_ids[..num_rows]; - - row_batch.iter().enumerate().for_each(|(row_idx, row)| { - partition_ids[row_idx] = bounds - .as_slice() - .partition_point(|bound| bound.row() <= row) - as u32 - }); - } - - // We now have partition ids for every input row, map that to partition starts - // and partition indices to eventually right these rows to partition buffers. - scratch - .map_partition_ids_to_starts_and_indices(*num_output_partitions, num_rows); - - timer.stop(); - Ok::<(&Vec, &Vec), DataFusionError>(( - &scratch.partition_starts, - &scratch.partition_row_indices, - )) - }?; - - self.buffer_partitioned_batch_may_spill( - input, - partition_row_indices, - partition_starts, - ) - .await?; - self.scratch = scratch; - } - CometPartitioning::RoundRobin(num_output_partitions, max_hash_columns) => { - // Comet implements "round robin" as hash partitioning on columns. - // This achieves the same goal as Spark's round robin (even distribution - // without semantic grouping) while being deterministic for fault tolerance. - // - // Note: This produces different partition assignments than Spark's round robin, - // which sorts by UnsafeRow binary representation before assigning partitions. - // However, both approaches provide even distribution and determinism. - let mut scratch = std::mem::take(&mut self.scratch); - let (partition_starts, partition_row_indices): (&Vec, &Vec) = { - let mut timer = self.metrics.repart_time.timer(); - - let num_rows = input.num_rows(); - - // Collect columns for hashing, respecting max_hash_columns limit - // max_hash_columns of 0 means no limit (hash all columns) - // Negative values are normalized to 0 in the planner - let num_columns_to_hash = if *max_hash_columns == 0 { - input.num_columns() - } else { - (*max_hash_columns).min(input.num_columns()) - }; - let columns_to_hash: Vec = (0..num_columns_to_hash) - .map(|i| Arc::clone(input.column(i))) - .collect(); - - // Use identical seed as Spark hash partitioning. - let hashes_buf = &mut scratch.hashes_buf[..num_rows]; - hashes_buf.fill(42_u32); - - // Compute hash for selected columns - create_murmur3_hashes(&columns_to_hash, hashes_buf)?; - - // Assign partition IDs based on hash (same as hash partitioning) - let partition_ids = &mut scratch.partition_ids[..num_rows]; - hashes_buf.iter().enumerate().for_each(|(idx, hash)| { - partition_ids[idx] = pmod(*hash, *num_output_partitions) as u32; - }); - - // We now have partition ids for every input row, map that to partition starts - // and partition indices to eventually write these rows to partition buffers. - scratch - .map_partition_ids_to_starts_and_indices(*num_output_partitions, num_rows); - - timer.stop(); - Ok::<(&Vec, &Vec), DataFusionError>(( - &scratch.partition_starts, - &scratch.partition_row_indices, - )) - }?; - - self.buffer_partitioned_batch_may_spill( - input, - partition_row_indices, - partition_starts, - ) - .await?; - self.scratch = scratch; - } - other => { - // this should be unreachable as long as the validation logic - // in the constructor is kept up-to-date - return Err(DataFusionError::NotImplemented(format!( - "Unsupported shuffle partitioning scheme {other:?}" - ))); - } - } - Ok(()) - } - - async fn buffer_partitioned_batch_may_spill( - &mut self, - input: RecordBatch, - partition_row_indices: &[u32], - partition_starts: &[u32], - ) -> Result<()> { - let mut mem_growth: usize = input.get_array_memory_size(); - let buffered_partition_idx = self.buffered_batches.len() as u32; - self.buffered_batches.push(input); - - // partition_starts conceptually slices partition_row_indices into smaller slices, - // each slice contains the indices of rows in input that will go into the corresponding - // partition. The following loop iterates over the slices and put the row indices into - // the indices array of the corresponding partition. - for (partition_id, (&start, &end)) in partition_starts - .iter() - .tuple_windows() - .enumerate() - .filter(|(_, (start, end))| start < end) - { - let row_indices = &partition_row_indices[start as usize..end as usize]; - - // Put row indices for the current partition into the indices array of that partition. - // This indices array will be used for calling interleave_record_batch to produce - // shuffled batches. - let indices = &mut self.partition_indices[partition_id]; - let before_size = indices.allocated_size(); - indices.reserve(row_indices.len()); - for row_idx in row_indices { - indices.push((buffered_partition_idx, *row_idx)); - } - let after_size = indices.allocated_size(); - mem_growth += after_size.saturating_sub(before_size); - } - - if self.reservation.try_grow(mem_growth).is_err() { - self.spill()?; - } - - Ok(()) - } - - fn shuffle_write_partition( - partition_iter: &mut PartitionedBatchIterator, - shuffle_block_writer: &mut ShuffleBlockWriter, - output_data: &mut BufWriter, - encode_time: &Time, - write_time: &Time, - write_buffer_size: usize, - ) -> Result<()> { - let mut buf_batch_writer = - BufBatchWriter::new(shuffle_block_writer, output_data, write_buffer_size); - for batch in partition_iter { - let batch = batch?; - buf_batch_writer.write(&batch, encode_time, write_time)?; - } - buf_batch_writer.flush(write_time)?; - Ok(()) - } - - fn used(&self) -> usize { - self.reservation.size() - } - - fn spilled_bytes(&self) -> usize { - self.metrics.spilled_bytes.value() - } - - fn spill_count(&self) -> usize { - self.metrics.spill_count.value() - } - - fn data_size(&self) -> usize { - self.metrics.data_size.value() - } - - /// This function transfers the ownership of the buffered batches and partition indices from the - /// ShuffleRepartitioner to a new PartitionedBatches struct. The returned PartitionedBatches struct - /// can be used to produce shuffled batches. - fn partitioned_batches(&mut self) -> PartitionedBatchesProducer { - let num_output_partitions = self.partition_indices.len(); - let buffered_batches = std::mem::take(&mut self.buffered_batches); - // let indices = std::mem::take(&mut self.partition_indices); - let indices = std::mem::replace( - &mut self.partition_indices, - vec![vec![]; num_output_partitions], - ); - PartitionedBatchesProducer::new(buffered_batches, indices, self.batch_size) - } - - fn spill(&mut self) -> Result<()> { - log::info!( - "ShuffleRepartitioner spilling shuffle data of {} to disk while inserting ({} time(s) so far)", - self.used(), - self.spill_count() - ); - - // we could always get a chance to free some memory as long as we are holding some - if self.buffered_batches.is_empty() { - return Ok(()); - } - - with_trace("shuffle_spill", self.tracing_enabled, || { - let num_output_partitions = self.partition_writers.len(); - let mut partitioned_batches = self.partitioned_batches(); - let mut spilled_bytes = 0; - - for partition_id in 0..num_output_partitions { - let partition_writer = &mut self.partition_writers[partition_id]; - let mut iter = partitioned_batches.produce(partition_id); - spilled_bytes += partition_writer.spill( - &mut iter, - &self.runtime, - &self.metrics, - self.write_buffer_size, - )?; - } - - self.reservation.free(); - self.metrics.spill_count.add(1); - self.metrics.spilled_bytes.add(spilled_bytes); - Ok(()) - }) - } -} - -#[async_trait::async_trait] -impl ShufflePartitioner for MultiPartitionShuffleRepartitioner { - /// Shuffles rows in input batch into corresponding partition buffer. - /// This function will slice input batch according to configured batch size and then - /// shuffle rows into corresponding partition buffer. - async fn insert_batch(&mut self, batch: RecordBatch) -> Result<()> { - with_trace_async("shuffle_insert_batch", self.tracing_enabled, || async { - let start_time = Instant::now(); - let mut start = 0; - while start < batch.num_rows() { - let end = (start + self.batch_size).min(batch.num_rows()); - let batch = batch.slice(start, end - start); - self.partitioning_batch(batch).await?; - start = end; - } - self.metrics.input_batches.add(1); - self.metrics - .baseline - .elapsed_compute() - .add_duration(start_time.elapsed()); - Ok(()) - }) - .await - } - - /// Writes buffered shuffled record batches into Arrow IPC bytes. - fn shuffle_write(&mut self) -> Result<()> { - with_trace("shuffle_write", self.tracing_enabled, || { - let start_time = Instant::now(); - - let mut partitioned_batches = self.partitioned_batches(); - let num_output_partitions = self.partition_indices.len(); - let mut offsets = vec![0; num_output_partitions + 1]; - - let data_file = self.output_data_file.clone(); - let index_file = self.output_index_file.clone(); - - let output_data = OpenOptions::new() - .write(true) - .create(true) - .truncate(true) - .open(data_file) - .map_err(|e| DataFusionError::Execution(format!("shuffle write error: {e:?}")))?; - - let mut output_data = BufWriter::new(output_data); - - #[allow(clippy::needless_range_loop)] - for i in 0..num_output_partitions { - offsets[i] = output_data.stream_position()?; - - // if we wrote a spill file for this partition then copy the - // contents into the shuffle file - if let Some(spill_data) = self.partition_writers[i].spill_file.as_ref() { - let mut spill_file = - BufReader::new(File::open(spill_data.temp_file.path()).map_err(to_df_err)?); - let mut write_timer = self.metrics.write_time.timer(); - std::io::copy(&mut spill_file, &mut output_data).map_err(to_df_err)?; - write_timer.stop(); - } - - // Write in memory batches to output data file - let mut partition_iter = partitioned_batches.produce(i); - Self::shuffle_write_partition( - &mut partition_iter, - &mut self.shuffle_block_writer, - &mut output_data, - &self.metrics.encode_time, - &self.metrics.write_time, - self.write_buffer_size, - )?; - } - - let mut write_timer = self.metrics.write_time.timer(); - output_data.flush()?; - write_timer.stop(); - - // add one extra offset at last to ease partition length computation - offsets[num_output_partitions] = output_data.stream_position().map_err(to_df_err)?; - - let mut write_timer = self.metrics.write_time.timer(); - let mut output_index = - BufWriter::new(File::create(index_file).map_err(|e| { - DataFusionError::Execution(format!("shuffle write error: {e:?}")) - })?); - for offset in offsets { - output_index - .write_all(&(offset as i64).to_le_bytes()[..]) - .map_err(to_df_err)?; - } - output_index.flush()?; - write_timer.stop(); - - self.metrics - .baseline - .elapsed_compute() - .add_duration(start_time.elapsed()); - - Ok(()) - }) - } -} - -impl Debug for MultiPartitionShuffleRepartitioner { - fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { - f.debug_struct("ShuffleRepartitioner") - .field("memory_used", &self.used()) - .field("spilled_bytes", &self.spilled_bytes()) - .field("spilled_count", &self.spill_count()) - .field("data_size", &self.data_size()) - .finish() - } -} - -/// A partitioner that writes all shuffle data to a single file and a single index file -struct SinglePartitionShufflePartitioner { - // output_data_file: File, - output_data_writer: BufBatchWriter, - output_index_path: String, - /// Batches that are smaller than the batch size and to be concatenated - buffered_batches: Vec, - /// Number of rows in the concatenating batches - num_buffered_rows: usize, - /// Metrics for the repartitioner - metrics: ShufflePartitionerMetrics, - /// The configured batch size - batch_size: usize, -} - -impl SinglePartitionShufflePartitioner { - fn try_new( - output_data_path: String, - output_index_path: String, - schema: SchemaRef, - metrics: ShufflePartitionerMetrics, - batch_size: usize, - codec: CompressionCodec, - write_buffer_size: usize, - ) -> Result { - let shuffle_block_writer = ShuffleBlockWriter::try_new(schema.as_ref(), codec.clone())?; - - let output_data_file = OpenOptions::new() - .write(true) - .create(true) - .truncate(true) - .open(output_data_path) - .map_err(to_df_err)?; - - let output_data_writer = - BufBatchWriter::new(shuffle_block_writer, output_data_file, write_buffer_size); - - Ok(Self { - output_data_writer, - output_index_path, - buffered_batches: vec![], - num_buffered_rows: 0, - metrics, - batch_size, - }) - } - - /// Add a batch to the buffer of the partitioner, these buffered batches will be concatenated - /// and written to the output data file when the number of rows in the buffer reaches the batch size. - fn add_buffered_batch(&mut self, batch: RecordBatch) { - self.num_buffered_rows += batch.num_rows(); - self.buffered_batches.push(batch); - } - - /// Consumes buffered batches and return a concatenated batch if successful - fn concat_buffered_batches(&mut self) -> Result> { - if self.buffered_batches.is_empty() { - Ok(None) - } else if self.buffered_batches.len() == 1 { - let batch = self.buffered_batches.remove(0); - self.num_buffered_rows = 0; - Ok(Some(batch)) - } else { - let schema = &self.buffered_batches[0].schema(); - match arrow::compute::concat_batches(schema, self.buffered_batches.iter()) { - Ok(concatenated) => { - self.buffered_batches.clear(); - self.num_buffered_rows = 0; - Ok(Some(concatenated)) - } - Err(e) => Err(DataFusionError::ArrowError( - Box::from(e), - Some(DataFusionError::get_back_trace()), - )), - } - } - } -} - -#[async_trait::async_trait] -impl ShufflePartitioner for SinglePartitionShufflePartitioner { - async fn insert_batch(&mut self, batch: RecordBatch) -> Result<()> { - let start_time = Instant::now(); - let num_rows = batch.num_rows(); - - if num_rows > 0 { - self.metrics.data_size.add(batch.get_array_memory_size()); - self.metrics.baseline.record_output(num_rows); - - if num_rows >= self.batch_size || num_rows + self.num_buffered_rows > self.batch_size { - let concatenated_batch = self.concat_buffered_batches()?; - - // Write the concatenated buffered batch - if let Some(batch) = concatenated_batch { - self.output_data_writer.write( - &batch, - &self.metrics.encode_time, - &self.metrics.write_time, - )?; - } - - if num_rows >= self.batch_size { - // Write the new batch - self.output_data_writer.write( - &batch, - &self.metrics.encode_time, - &self.metrics.write_time, - )?; - } else { - // Add the new batch to the buffer - self.add_buffered_batch(batch); - } - } else { - self.add_buffered_batch(batch); - } - } - - self.metrics.input_batches.add(1); - self.metrics - .baseline - .elapsed_compute() - .add_duration(start_time.elapsed()); - Ok(()) - } - - fn shuffle_write(&mut self) -> Result<()> { - let start_time = Instant::now(); - let concatenated_batch = self.concat_buffered_batches()?; - - // Write the concatenated buffered batch - if let Some(batch) = concatenated_batch { - self.output_data_writer.write( - &batch, - &self.metrics.encode_time, - &self.metrics.write_time, - )?; - } - self.output_data_writer.flush(&self.metrics.write_time)?; - - // Write index file. It should only contain 2 entries: 0 and the total number of bytes written - let index_file = OpenOptions::new() - .write(true) - .create(true) - .truncate(true) - .open(self.output_index_path.clone()) - .map_err(|e| DataFusionError::Execution(format!("shuffle write error: {e:?}")))?; - let mut index_buf_writer = BufWriter::new(index_file); - let data_file_length = self - .output_data_writer - .writer - .stream_position() - .map_err(to_df_err)?; - for offset in [0, data_file_length] { - index_buf_writer - .write_all(&(offset as i64).to_le_bytes()[..]) - .map_err(to_df_err)?; - } - index_buf_writer.flush()?; - - self.metrics - .baseline - .elapsed_compute() - .add_duration(start_time.elapsed()); - Ok(()) - } -} - -fn to_df_err(e: Error) -> DataFusionError { - DataFusionError::Execution(format!("shuffle write error: {e:?}")) -} - -/// A helper struct to produce shuffled batches. -/// This struct takes ownership of the buffered batches and partition indices from the -/// ShuffleRepartitioner, and provides an iterator over the batches in the specified partitions. -struct PartitionedBatchesProducer { - buffered_batches: Vec, - partition_indices: Vec>, - batch_size: usize, -} - -impl PartitionedBatchesProducer { - fn new( - buffered_batches: Vec, - indices: Vec>, - batch_size: usize, - ) -> Self { - Self { - partition_indices: indices, - buffered_batches, - batch_size, - } - } - - fn produce(&mut self, partition_id: usize) -> PartitionedBatchIterator<'_> { - PartitionedBatchIterator::new( - &self.partition_indices[partition_id], - &self.buffered_batches, - self.batch_size, - ) - } -} - -struct PartitionedBatchIterator<'a> { - record_batches: Vec<&'a RecordBatch>, - batch_size: usize, - indices: Vec<(usize, usize)>, - pos: usize, -} - -impl<'a> PartitionedBatchIterator<'a> { - fn new( - indices: &'a [(u32, u32)], - buffered_batches: &'a [RecordBatch], - batch_size: usize, - ) -> Self { - if indices.is_empty() { - // Avoid unnecessary allocations when the partition is empty - return Self { - record_batches: vec![], - batch_size, - indices: vec![], - pos: 0, - }; - } - let record_batches = buffered_batches.iter().collect::>(); - let current_indices = indices - .iter() - .map(|(i_batch, i_row)| (*i_batch as usize, *i_row as usize)) - .collect::>(); - Self { - record_batches, - batch_size, - indices: current_indices, - pos: 0, - } - } -} - -impl Iterator for PartitionedBatchIterator<'_> { - type Item = Result; - - fn next(&mut self) -> Option { - if self.pos >= self.indices.len() { - return None; - } - - let indices_end = std::cmp::min(self.pos + self.batch_size, self.indices.len()); - let indices = &self.indices[self.pos..indices_end]; - match interleave_record_batch(&self.record_batches, indices) { - Ok(batch) => { - self.pos = indices_end; - Some(Ok(batch)) - } - Err(e) => Some(Err(DataFusionError::ArrowError( - Box::from(e), - Some(DataFusionError::get_back_trace()), - ))), - } - } -} - -struct PartitionWriter { - /// Spill file for intermediate shuffle output for this partition. Each spill event - /// will append to this file and the contents will be copied to the shuffle file at - /// the end of processing. - spill_file: Option, - /// Writer that performs encoding and compression - shuffle_block_writer: ShuffleBlockWriter, -} - -struct SpillFile { - temp_file: RefCountedTempFile, - file: File, -} - -impl PartitionWriter { - fn try_new(shuffle_block_writer: ShuffleBlockWriter) -> Result { - Ok(Self { - spill_file: None, - shuffle_block_writer, - }) - } - - fn spill( - &mut self, - iter: &mut PartitionedBatchIterator, - runtime: &RuntimeEnv, - metrics: &ShufflePartitionerMetrics, - write_buffer_size: usize, - ) -> Result { - if let Some(batch) = iter.next() { - self.ensure_spill_file_created(runtime)?; - - let total_bytes_written = { - let mut buf_batch_writer = BufBatchWriter::new( - &mut self.shuffle_block_writer, - &mut self.spill_file.as_mut().unwrap().file, - write_buffer_size, - ); - let mut bytes_written = - buf_batch_writer.write(&batch?, &metrics.encode_time, &metrics.write_time)?; - for batch in iter { - let batch = batch?; - bytes_written += buf_batch_writer.write( - &batch, - &metrics.encode_time, - &metrics.write_time, - )?; - } - buf_batch_writer.flush(&metrics.write_time)?; - bytes_written - }; - - Ok(total_bytes_written) - } else { - Ok(0) - } - } - - fn ensure_spill_file_created(&mut self, runtime: &RuntimeEnv) -> Result<()> { - if self.spill_file.is_none() { - // Spill file is not yet created, create it - let spill_file = runtime - .disk_manager - .create_tmp_file("shuffle writer spill")?; - let spill_data = OpenOptions::new() - .write(true) - .create(true) - .truncate(true) - .open(spill_file.path()) - .map_err(|e| { - DataFusionError::Execution(format!("Error occurred while spilling {e}")) - })?; - self.spill_file = Some(SpillFile { - temp_file: spill_file, - file: spill_data, - }); - } - Ok(()) - } -} - -/// Write batches to writer while using a buffer to avoid frequent system calls. -/// The record batches were first written by ShuffleBlockWriter into an internal buffer. -/// Once the buffer exceeds the max size, the buffer will be flushed to the writer. -struct BufBatchWriter, W: Write> { - shuffle_block_writer: S, - writer: W, - buffer: Vec, - buffer_max_size: usize, -} - -impl, W: Write> BufBatchWriter { - fn new(shuffle_block_writer: S, writer: W, buffer_max_size: usize) -> Self { - Self { - shuffle_block_writer, - writer, - buffer: vec![], - buffer_max_size, - } - } - - fn write( - &mut self, - batch: &RecordBatch, - encode_time: &Time, - write_time: &Time, - ) -> Result { - let mut cursor = Cursor::new(&mut self.buffer); - cursor.seek(SeekFrom::End(0))?; - let bytes_written = - self.shuffle_block_writer - .borrow() - .write_batch(batch, &mut cursor, encode_time)?; - let pos = cursor.position(); - if pos >= self.buffer_max_size as u64 { - let mut write_timer = write_time.timer(); - self.writer.write_all(&self.buffer)?; - write_timer.stop(); - self.buffer.clear(); - } - Ok(bytes_written) - } - - fn flush(&mut self, write_time: &Time) -> Result<()> { - let mut write_timer = write_time.timer(); - if !self.buffer.is_empty() { - self.writer.write_all(&self.buffer)?; - } - self.writer.flush()?; - write_timer.stop(); - self.buffer.clear(); - Ok(()) - } -} - -fn pmod(hash: u32, n: usize) -> usize { - let hash = hash as i32; - let n = n as i32; - let r = hash % n; - let result = if r < 0 { (r + n) % n } else { r }; - result as usize -} - #[cfg(test)] mod test { use super::*; - use crate::execution::shuffle::read_ipc_compressed; + use crate::execution::shuffle::{read_ipc_compressed, ShuffleBlockWriter}; + use arrow::array::{Array, StringArray, StringBuilder}; use arrow::datatypes::{DataType, Field, Schema}; + use arrow::record_batch::RecordBatch; use arrow::row::{RowConverter, SortField}; use datafusion::datasource::memory::MemorySourceConfig; use datafusion::datasource::source::DataSourceExec; use datafusion::execution::config::SessionConfig; - use datafusion::execution::runtime_env::RuntimeEnvBuilder; + use datafusion::execution::runtime_env::{RuntimeEnv, RuntimeEnvBuilder}; use datafusion::physical_expr::expressions::{col, Column}; use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr}; use datafusion::physical_plan::common::collect; + use datafusion::physical_plan::metrics::Time; use datafusion::prelude::SessionContext; + use itertools::Itertools; use std::io::Cursor; use tokio::runtime::Runtime; @@ -1369,16 +367,22 @@ mod test { repartitioner.insert_batch(batch.clone()).await.unwrap(); - assert_eq!(2, repartitioner.partition_writers.len()); + { + let partition_writers = repartitioner.partition_writers(); + assert_eq!(partition_writers.len(), 2); - assert!(repartitioner.partition_writers[0].spill_file.is_none()); - assert!(repartitioner.partition_writers[1].spill_file.is_none()); + assert!(!partition_writers[0].has_spill_file()); + assert!(!partition_writers[1].has_spill_file()); + } repartitioner.spill().unwrap(); // after spill, there should be spill files - assert!(repartitioner.partition_writers[0].spill_file.is_some()); - assert!(repartitioner.partition_writers[1].spill_file.is_some()); + { + let partition_writers = repartitioner.partition_writers(); + assert!(partition_writers[0].has_spill_file()); + assert!(partition_writers[1].has_spill_file()); + } // insert another batch after spilling repartitioner.insert_batch(batch.clone()).await.unwrap(); @@ -1491,16 +495,6 @@ mod test { RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(array)]).unwrap() } - #[test] - fn test_pmod() { - let i: Vec = vec![0x99f0149d, 0x9c67b85d, 0xc8008529, 0xa05b5d7b, 0xcd1e64fb]; - let result = i.into_iter().map(|i| pmod(i, 200)).collect::>(); - - // expected partition from Spark with n=200 - let expected = vec![69, 5, 193, 171, 115]; - assert_eq!(result, expected); - } - #[test] #[cfg_attr(miri, ignore)] fn test_round_robin_deterministic() { @@ -1591,4 +585,112 @@ mod test { let _ = fs::remove_file("/tmp/rr_data_1.out"); let _ = fs::remove_file("/tmp/rr_index_1.out"); } + + /// Test that batch coalescing in BufBatchWriter reduces output size by + /// writing fewer, larger IPC blocks instead of many small ones. + #[test] + #[cfg_attr(miri, ignore)] + fn test_batch_coalescing_reduces_size() { + use crate::execution::shuffle::writers::BufBatchWriter; + use arrow::array::Int32Array; + + // Create a wide schema to amplify per-block schema overhead + let fields: Vec = (0..20) + .map(|i| Field::new(format!("col_{i}"), DataType::Int32, false)) + .collect(); + let schema = Arc::new(Schema::new(fields)); + + // Create many small batches (50 rows each) + let small_batches: Vec = (0..100) + .map(|batch_idx| { + let columns: Vec> = (0..20) + .map(|col_idx| { + let values: Vec = (0..50) + .map(|row| batch_idx * 50 + row + col_idx * 1000) + .collect(); + Arc::new(Int32Array::from(values)) as Arc + }) + .collect(); + RecordBatch::try_new(Arc::clone(&schema), columns).unwrap() + }) + .collect(); + + let codec = CompressionCodec::Lz4Frame; + let encode_time = Time::default(); + let write_time = Time::default(); + + // Write with coalescing (batch_size=8192) + let mut coalesced_output = Vec::new(); + { + let mut writer = ShuffleBlockWriter::try_new(schema.as_ref(), codec.clone()).unwrap(); + let mut buf_writer = BufBatchWriter::new( + &mut writer, + Cursor::new(&mut coalesced_output), + 1024 * 1024, + 8192, + ); + for batch in &small_batches { + buf_writer.write(batch, &encode_time, &write_time).unwrap(); + } + buf_writer.flush(&encode_time, &write_time).unwrap(); + } + + // Write without coalescing (batch_size=1) + let mut uncoalesced_output = Vec::new(); + { + let mut writer = ShuffleBlockWriter::try_new(schema.as_ref(), codec.clone()).unwrap(); + let mut buf_writer = BufBatchWriter::new( + &mut writer, + Cursor::new(&mut uncoalesced_output), + 1024 * 1024, + 1, + ); + for batch in &small_batches { + buf_writer.write(batch, &encode_time, &write_time).unwrap(); + } + buf_writer.flush(&encode_time, &write_time).unwrap(); + } + + // Coalesced output should be smaller due to fewer IPC schema blocks + assert!( + coalesced_output.len() < uncoalesced_output.len(), + "Coalesced output ({} bytes) should be smaller than uncoalesced ({} bytes)", + coalesced_output.len(), + uncoalesced_output.len() + ); + + // Verify both roundtrip correctly by reading all IPC blocks + let coalesced_rows = read_all_ipc_blocks(&coalesced_output); + let uncoalesced_rows = read_all_ipc_blocks(&uncoalesced_output); + assert_eq!( + coalesced_rows, 5000, + "Coalesced should contain all 5000 rows" + ); + assert_eq!( + uncoalesced_rows, 5000, + "Uncoalesced should contain all 5000 rows" + ); + } + + /// Read all IPC blocks from a byte buffer written by BufBatchWriter/ShuffleBlockWriter, + /// returning the total number of rows. + fn read_all_ipc_blocks(data: &[u8]) -> usize { + let mut offset = 0; + let mut total_rows = 0; + while offset < data.len() { + // First 8 bytes are the IPC length (little-endian u64) + let ipc_length = + u64::from_le_bytes(data[offset..offset + 8].try_into().unwrap()) as usize; + // Skip the 8-byte length prefix; the next 8 bytes are field_count + codec header + let block_start = offset + 8; + let block_end = block_start + ipc_length; + // read_ipc_compressed expects data starting after the 16-byte header + // (i.e., after length + field_count), at the codec tag + let ipc_data = &data[block_start + 8..block_end]; + let batch = read_ipc_compressed(ipc_data).unwrap(); + total_rows += batch.num_rows(); + offset = block_end; + } + total_rows + } } diff --git a/native/core/src/execution/shuffle/writers/buf_batch_writer.rs b/native/core/src/execution/shuffle/writers/buf_batch_writer.rs new file mode 100644 index 0000000000..8d056d7bb0 --- /dev/null +++ b/native/core/src/execution/shuffle/writers/buf_batch_writer.rs @@ -0,0 +1,142 @@ +// 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 crate::execution::shuffle::ShuffleBlockWriter; +use arrow::array::RecordBatch; +use arrow::compute::kernels::coalesce::BatchCoalescer; +use datafusion::physical_plan::metrics::Time; +use std::borrow::Borrow; +use std::io::{Cursor, Seek, SeekFrom, Write}; + +/// Write batches to writer while using a buffer to avoid frequent system calls. +/// The record batches were first written by ShuffleBlockWriter into an internal buffer. +/// Once the buffer exceeds the max size, the buffer will be flushed to the writer. +/// +/// Small batches are coalesced using Arrow's [`BatchCoalescer`] before serialization, +/// producing exactly `batch_size`-row output batches to reduce per-block IPC schema overhead. +/// The coalescer is lazily initialized on the first write. +pub(crate) struct BufBatchWriter, W: Write> { + shuffle_block_writer: S, + writer: W, + buffer: Vec, + buffer_max_size: usize, + /// Coalesces small batches into target_batch_size before serialization. + /// Lazily initialized on first write to capture the schema. + coalescer: Option, + /// Target batch size for coalescing + batch_size: usize, +} + +impl, W: Write> BufBatchWriter { + pub(crate) fn new( + shuffle_block_writer: S, + writer: W, + buffer_max_size: usize, + batch_size: usize, + ) -> Self { + Self { + shuffle_block_writer, + writer, + buffer: vec![], + buffer_max_size, + coalescer: None, + batch_size, + } + } + + pub(crate) fn write( + &mut self, + batch: &RecordBatch, + encode_time: &Time, + write_time: &Time, + ) -> datafusion::common::Result { + let coalescer = self + .coalescer + .get_or_insert_with(|| BatchCoalescer::new(batch.schema(), self.batch_size)); + coalescer.push_batch(batch.clone())?; + + // Drain completed batches into a local vec so the coalescer borrow ends + // before we call write_batch_to_buffer (which borrows &mut self). + let mut completed = Vec::new(); + while let Some(batch) = coalescer.next_completed_batch() { + completed.push(batch); + } + + let mut bytes_written = 0; + for batch in &completed { + bytes_written += self.write_batch_to_buffer(batch, encode_time, write_time)?; + } + Ok(bytes_written) + } + + /// Serialize a single batch into the byte buffer, flushing to the writer if needed. + fn write_batch_to_buffer( + &mut self, + batch: &RecordBatch, + encode_time: &Time, + write_time: &Time, + ) -> datafusion::common::Result { + let mut cursor = Cursor::new(&mut self.buffer); + cursor.seek(SeekFrom::End(0))?; + let bytes_written = + self.shuffle_block_writer + .borrow() + .write_batch(batch, &mut cursor, encode_time)?; + let pos = cursor.position(); + if pos >= self.buffer_max_size as u64 { + let mut write_timer = write_time.timer(); + self.writer.write_all(&self.buffer)?; + write_timer.stop(); + self.buffer.clear(); + } + Ok(bytes_written) + } + + pub(crate) fn flush( + &mut self, + encode_time: &Time, + write_time: &Time, + ) -> datafusion::common::Result<()> { + // Finish any remaining buffered rows in the coalescer + let mut remaining = Vec::new(); + if let Some(coalescer) = &mut self.coalescer { + coalescer.finish_buffered_batch()?; + while let Some(batch) = coalescer.next_completed_batch() { + remaining.push(batch); + } + } + for batch in &remaining { + self.write_batch_to_buffer(batch, encode_time, write_time)?; + } + + // Flush the byte buffer to the underlying writer + let mut write_timer = write_time.timer(); + if !self.buffer.is_empty() { + self.writer.write_all(&self.buffer)?; + } + self.writer.flush()?; + write_timer.stop(); + self.buffer.clear(); + Ok(()) + } +} + +impl, W: Write + Seek> BufBatchWriter { + pub(crate) fn writer_stream_position(&mut self) -> datafusion::common::Result { + self.writer.stream_position().map_err(Into::into) + } +} diff --git a/native/core/src/execution/shuffle/writers/mod.rs b/native/core/src/execution/shuffle/writers/mod.rs new file mode 100644 index 0000000000..d41363b7fb --- /dev/null +++ b/native/core/src/execution/shuffle/writers/mod.rs @@ -0,0 +1,22 @@ +// 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. + +mod buf_batch_writer; +mod partition_writer; + +pub(super) use buf_batch_writer::BufBatchWriter; +pub(super) use partition_writer::PartitionWriter; diff --git a/native/core/src/execution/shuffle/writers/partition_writer.rs b/native/core/src/execution/shuffle/writers/partition_writer.rs new file mode 100644 index 0000000000..7c2dbe0444 --- /dev/null +++ b/native/core/src/execution/shuffle/writers/partition_writer.rs @@ -0,0 +1,124 @@ +// 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 crate::execution::shuffle::metrics::ShufflePartitionerMetrics; +use crate::execution::shuffle::partitioners::PartitionedBatchIterator; +use crate::execution::shuffle::writers::buf_batch_writer::BufBatchWriter; +use crate::execution::shuffle::ShuffleBlockWriter; +use datafusion::common::DataFusionError; +use datafusion::execution::disk_manager::RefCountedTempFile; +use datafusion::execution::runtime_env::RuntimeEnv; +use std::fs::{File, OpenOptions}; + +struct SpillFile { + temp_file: RefCountedTempFile, + file: File, +} + +pub(crate) struct PartitionWriter { + /// Spill file for intermediate shuffle output for this partition. Each spill event + /// will append to this file and the contents will be copied to the shuffle file at + /// the end of processing. + spill_file: Option, + /// Writer that performs encoding and compression + shuffle_block_writer: ShuffleBlockWriter, +} + +impl PartitionWriter { + pub(crate) fn try_new( + shuffle_block_writer: ShuffleBlockWriter, + ) -> datafusion::common::Result { + Ok(Self { + spill_file: None, + shuffle_block_writer, + }) + } + + fn ensure_spill_file_created( + &mut self, + runtime: &RuntimeEnv, + ) -> datafusion::common::Result<()> { + if self.spill_file.is_none() { + // Spill file is not yet created, create it + let spill_file = runtime + .disk_manager + .create_tmp_file("shuffle writer spill")?; + let spill_data = OpenOptions::new() + .write(true) + .create(true) + .truncate(true) + .open(spill_file.path()) + .map_err(|e| { + DataFusionError::Execution(format!("Error occurred while spilling {e}")) + })?; + self.spill_file = Some(SpillFile { + temp_file: spill_file, + file: spill_data, + }); + } + Ok(()) + } + + pub(crate) fn spill( + &mut self, + iter: &mut PartitionedBatchIterator, + runtime: &RuntimeEnv, + metrics: &ShufflePartitionerMetrics, + write_buffer_size: usize, + batch_size: usize, + ) -> datafusion::common::Result { + if let Some(batch) = iter.next() { + self.ensure_spill_file_created(runtime)?; + + let total_bytes_written = { + let mut buf_batch_writer = BufBatchWriter::new( + &mut self.shuffle_block_writer, + &mut self.spill_file.as_mut().unwrap().file, + write_buffer_size, + batch_size, + ); + let mut bytes_written = + buf_batch_writer.write(&batch?, &metrics.encode_time, &metrics.write_time)?; + for batch in iter { + let batch = batch?; + bytes_written += buf_batch_writer.write( + &batch, + &metrics.encode_time, + &metrics.write_time, + )?; + } + buf_batch_writer.flush(&metrics.encode_time, &metrics.write_time)?; + bytes_written + }; + + Ok(total_bytes_written) + } else { + Ok(0) + } + } + + pub(crate) fn path(&self) -> Option<&std::path::Path> { + self.spill_file + .as_ref() + .map(|spill_file| spill_file.temp_file.path()) + } + + #[cfg(test)] + pub(crate) fn has_spill_file(&self) -> bool { + self.spill_file.is_some() + } +} diff --git a/native/core/src/parquet/util/test_common/rand_gen.rs b/native/core/src/parquet/util/test_common/rand_gen.rs index 0a1ad07748..0116b2c8c8 100644 --- a/native/core/src/parquet/util/test_common/rand_gen.rs +++ b/native/core/src/parquet/util/test_common/rand_gen.rs @@ -17,7 +17,7 @@ use rand::{ distr::{uniform::SampleUniform, Distribution, StandardUniform}, - rng, Rng, + rng, RngExt, }; pub fn random_bytes(n: usize) -> Vec { diff --git a/native/proto/src/lib.rs b/native/proto/src/lib.rs index 6dfe546ac8..a55657b7af 100644 --- a/native/proto/src/lib.rs +++ b/native/proto/src/lib.rs @@ -34,6 +34,7 @@ pub mod spark_partitioning { // Include generated modules from .proto files. #[allow(missing_docs)] +#[allow(clippy::large_enum_variant)] pub mod spark_operator { include!(concat!("generated", "/spark.spark_operator.rs")); } diff --git a/native/proto/src/proto/expr.proto b/native/proto/src/proto/expr.proto index e2d0ca1c0f..944505ba1c 100644 --- a/native/proto/src/proto/expr.proto +++ b/native/proto/src/proto/expr.proto @@ -87,6 +87,7 @@ message Expr { EmptyExpr monotonically_increasing_id = 64; UnixTimestamp unix_timestamp = 65; FromJson from_json = 66; + ToCsv to_csv = 67; } } @@ -276,6 +277,22 @@ message FromJson { string timezone = 3; } +message ToCsv { + Expr child = 1; + CsvWriteOptions options = 2; +} + +message CsvWriteOptions { + string delimiter = 1; + string quote = 2; + string escape = 3; + string null_value = 4; + bool quote_all = 5; + bool ignore_leading_white_space = 6; + bool ignore_trailing_white_space = 7; + string timezone = 8; +} + enum BinaryOutputStyle { UTF8 = 0; BASIC = 1; diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 73c087cf36..78f118e6db 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -156,28 +156,34 @@ message PartitionData { repeated PartitionValue values = 1; } -message IcebergScan { - // Schema to read - repeated SparkStructField required_schema = 1; - +// Common data shared by all partitions in split mode (sent once, captured in closure) +message IcebergScanCommon { // Catalog-specific configuration for FileIO (credentials, S3/GCS config, etc.) - map catalog_properties = 2; - - // Pre-planned file scan tasks grouped by Spark partition - repeated IcebergFilePartition file_partitions = 3; + map catalog_properties = 1; // Table metadata file path for FileIO initialization - string metadata_location = 4; + string metadata_location = 2; + + // Schema to read + repeated SparkStructField required_schema = 3; - // Deduplication pools - shared data referenced by index from tasks - repeated string schema_pool = 5; - repeated string partition_type_pool = 6; - repeated string partition_spec_pool = 7; - repeated string name_mapping_pool = 8; - repeated ProjectFieldIdList project_field_ids_pool = 9; - repeated PartitionData partition_data_pool = 10; - repeated DeleteFileList delete_files_pool = 11; - repeated spark.spark_expression.Expr residual_pool = 12; + // Deduplication pools (must contain all entries for cross-partition deduplication) + repeated string schema_pool = 4; + repeated string partition_type_pool = 5; + repeated string partition_spec_pool = 6; + repeated string name_mapping_pool = 7; + repeated ProjectFieldIdList project_field_ids_pool = 8; + repeated PartitionData partition_data_pool = 9; + repeated DeleteFileList delete_files_pool = 10; + repeated spark.spark_expression.Expr residual_pool = 11; +} + +message IcebergScan { + // Common data shared across partitions (pools, metadata, catalog props) + IcebergScanCommon common = 1; + + // Single partition's file scan tasks + repeated IcebergFileScanTask file_scan_tasks = 2; } // Helper message for deduplicating field ID lists @@ -190,11 +196,6 @@ message DeleteFileList { repeated IcebergDeleteFile delete_files = 1; } -// Groups FileScanTasks for a single Spark partition -message IcebergFilePartition { - repeated IcebergFileScanTask file_scan_tasks = 1; -} - // Iceberg FileScanTask containing data file, delete files, and residual filter message IcebergFileScanTask { // Data file path (e.g., s3://bucket/warehouse/db/table/data/00000-0-abc.parquet) diff --git a/native/spark-expr/Cargo.toml b/native/spark-expr/Cargo.toml index 94653d8864..fd0a211b29 100644 --- a/native/spark-expr/Cargo.toml +++ b/native/spark-expr/Cargo.toml @@ -88,6 +88,10 @@ harness = false name = "normalize_nan" harness = false +[[bench]] +name = "to_csv" +harness = false + [[test]] name = "test_udf_registration" path = "tests/spark_expr_reg.rs" diff --git a/native/spark-expr/benches/to_csv.rs b/native/spark-expr/benches/to_csv.rs new file mode 100644 index 0000000000..8620dd0f16 --- /dev/null +++ b/native/spark-expr/benches/to_csv.rs @@ -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. + +use arrow::array::{ + BooleanBuilder, Int16Builder, Int32Builder, Int64Builder, Int8Builder, StringBuilder, + StructArray, StructBuilder, +}; +use arrow::datatypes::{DataType, Field}; +use criterion::{criterion_group, criterion_main, Criterion}; +use datafusion_comet_spark_expr::{to_csv_inner, CsvWriteOptions, EvalMode, SparkCastOptions}; +use std::hint::black_box; + +fn create_struct_array(array_size: usize) -> StructArray { + let fields = vec![ + Field::new("f1", DataType::Boolean, true), + Field::new("f2", DataType::Int8, true), + Field::new("f3", DataType::Int16, true), + Field::new("f4", DataType::Int32, true), + Field::new("f5", DataType::Int64, true), + Field::new("f6", DataType::Utf8, true), + ]; + let mut struct_builder = StructBuilder::from_fields(fields, array_size); + for i in 0..array_size { + struct_builder + .field_builder::(0) + .unwrap() + .append_option(if i % 10 == 0 { None } else { Some(i % 2 == 0) }); + + struct_builder + .field_builder::(1) + .unwrap() + .append_option(if i % 10 == 0 { + None + } else { + Some((i % 128) as i8) + }); + + struct_builder + .field_builder::(2) + .unwrap() + .append_option(if i % 10 == 0 { None } else { Some(i as i16) }); + + struct_builder + .field_builder::(3) + .unwrap() + .append_option(if i % 10 == 0 { None } else { Some(i as i32) }); + + struct_builder + .field_builder::(4) + .unwrap() + .append_option(if i % 10 == 0 { None } else { Some(i as i64) }); + + struct_builder + .field_builder::(5) + .unwrap() + .append_option(if i % 10 == 0 { + None + } else { + Some(format!("string_{}", i)) + }); + + struct_builder.append(true); + } + struct_builder.finish() +} + +fn criterion_benchmark(c: &mut Criterion) { + let array_size = 8192; + let timezone = "UTC"; + let struct_array = create_struct_array(array_size); + let default_delimiter = ","; + let default_null_value = ""; + let default_quote = "\""; + let default_escape = "\\"; + let mut cast_options = SparkCastOptions::new(EvalMode::Legacy, timezone, false); + cast_options.null_string = default_null_value.to_string(); + let csv_write_options = CsvWriteOptions::new( + default_delimiter.to_string(), + default_quote.to_string(), + default_escape.to_string(), + default_null_value.to_string(), + false, + true, + true, + ); + c.bench_function("to_csv", |b| { + b.iter(|| { + black_box(to_csv_inner(&struct_array, &cast_options, &csv_write_options).unwrap()) + }) + }); +} + +criterion_group!(benches, criterion_benchmark); +criterion_main!(benches); diff --git a/native/spark-expr/src/comet_scalar_funcs.rs b/native/spark-expr/src/comet_scalar_funcs.rs index 760dc3570f..98a6da485e 100644 --- a/native/spark-expr/src/comet_scalar_funcs.rs +++ b/native/spark-expr/src/comet_scalar_funcs.rs @@ -22,8 +22,8 @@ use crate::math_funcs::modulo_expr::spark_modulo; use crate::{ spark_array_repeat, spark_ceil, spark_decimal_div, spark_decimal_integral_div, spark_floor, spark_isnan, spark_lpad, spark_make_decimal, spark_read_side_padding, spark_round, spark_rpad, - spark_unhex, spark_unscaled_value, EvalMode, SparkBitwiseCount, SparkDateDiff, SparkDateTrunc, - SparkSizeFunc, SparkStringSpace, + spark_unhex, spark_unscaled_value, EvalMode, SparkBitwiseCount, SparkContains, SparkDateDiff, + SparkDateTrunc, SparkMakeDate, SparkSizeFunc, SparkStringSpace, }; use arrow::datatypes::DataType; use datafusion::common::{DataFusionError, Result as DataFusionResult}; @@ -192,8 +192,10 @@ pub fn create_comet_physical_fun_with_eval_mode( fn all_scalar_functions() -> Vec> { vec![ Arc::new(ScalarUDF::new_from_impl(SparkBitwiseCount::default())), + Arc::new(ScalarUDF::new_from_impl(SparkContains::default())), Arc::new(ScalarUDF::new_from_impl(SparkDateDiff::default())), Arc::new(ScalarUDF::new_from_impl(SparkDateTrunc::default())), + Arc::new(ScalarUDF::new_from_impl(SparkMakeDate::default())), Arc::new(ScalarUDF::new_from_impl(SparkStringSpace::default())), Arc::new(ScalarUDF::new_from_impl(SparkSizeFunc::default())), ] diff --git a/native/spark-expr/src/conversion_funcs/cast.rs b/native/spark-expr/src/conversion_funcs/cast.rs index 186a10c9a7..5c65336183 100644 --- a/native/spark-expr/src/conversion_funcs/cast.rs +++ b/native/spark-expr/src/conversion_funcs/cast.rs @@ -21,7 +21,7 @@ use crate::{EvalMode, SparkError, SparkResult}; use arrow::array::builder::StringBuilder; use arrow::array::{ BooleanBuilder, Decimal128Builder, DictionaryArray, GenericByteArray, ListArray, - PrimitiveBuilder, StringArray, StructArray, + PrimitiveBuilder, StringArray, StructArray, TimestampMicrosecondBuilder, }; use arrow::compute::can_cast_types; use arrow::datatypes::{ @@ -1100,6 +1100,7 @@ fn cast_array( Ok(cast_with_options(&array, to_type, &CAST_OPTIONS)?) } (Binary, Utf8) => Ok(cast_binary_to_string::(&array, cast_options)?), + (Date32, Timestamp(_, tz)) => Ok(cast_date_to_timestamp(&array, cast_options, tz)?), _ if cast_options.is_adapting_schema || is_datafusion_spark_compatible(from_type, to_type) => { @@ -1118,6 +1119,50 @@ fn cast_array( Ok(spark_cast_postprocess(cast_result?, from_type, to_type)) } +fn cast_date_to_timestamp( + array_ref: &ArrayRef, + cast_options: &SparkCastOptions, + target_tz: &Option>, +) -> SparkResult { + let tz_str = if cast_options.timezone.is_empty() { + "UTC" + } else { + cast_options.timezone.as_str() + }; + // safe to unwrap since we are falling back to UTC above + let tz = timezone::Tz::from_str(tz_str)?; + let epoch = NaiveDate::from_ymd_opt(1970, 1, 1).unwrap(); + let date_array = array_ref.as_primitive::(); + + let mut builder = TimestampMicrosecondBuilder::with_capacity(date_array.len()); + + for date in date_array.iter() { + match date { + Some(date) => { + // safe to unwrap since chrono's range ( 262,143 yrs) is higher than + // number of years possible with days as i32 (~ 6 mil yrs) + // convert date in session timezone to timestamp in UTC + let naive_date = epoch + chrono::Duration::days(date as i64); + let local_midnight = naive_date.and_hms_opt(0, 0, 0).unwrap(); + let local_midnight_in_microsec = tz + .from_local_datetime(&local_midnight) + // return earliest possible time (edge case with spring / fall DST changes) + .earliest() + .map(|dt| dt.timestamp_micros()) + // in case there is an issue with DST and returns None , we fall back to UTC + .unwrap_or((date as i64) * 86_400 * 1_000_000); + builder.append_value(local_midnight_in_microsec); + } + None => { + builder.append_null(); + } + } + } + Ok(Arc::new( + builder.finish().with_timezone_opt(target_tz.clone()), + )) +} + fn cast_string_to_float( array: &ArrayRef, to_type: &DataType, @@ -3408,6 +3453,64 @@ mod tests { assert!(result.is_err()) } + #[test] + fn test_cast_date_to_timestamp() { + use arrow::array::Date32Array; + + // verifying epoch , DST change dates (US) and a null value (comprehensive tests on spark side) + let dates: ArrayRef = Arc::new(Date32Array::from(vec![ + Some(0), + Some(19723), + Some(19793), + None, + ])); + + let non_dst_date = 1704067200000000i64; + let dst_date = 1710115200000000i64; + let seven_hours_ts = 25200000000i64; + let eight_hours_ts = 28800000000i64; + + // validate UTC + let result = cast_array( + Arc::clone(&dates), + &DataType::Timestamp(TimeUnit::Microsecond, Some("UTC".into())), + &SparkCastOptions::new(EvalMode::Legacy, "UTC", false), + ) + .unwrap(); + let ts = result.as_primitive::(); + assert_eq!(ts.value(0), 0); + assert_eq!(ts.value(1), non_dst_date); + assert_eq!(ts.value(2), dst_date); + assert!(ts.is_null(3)); + + // validate LA timezone (follows Daylight savings) + let result = cast_array( + Arc::clone(&dates), + &DataType::Timestamp(TimeUnit::Microsecond, Some("UTC".into())), + &SparkCastOptions::new(EvalMode::Legacy, "America/Los_Angeles", false), + ) + .unwrap(); + let ts = result.as_primitive::(); + assert_eq!(ts.value(0), eight_hours_ts); + assert_eq!(ts.value(1), non_dst_date + eight_hours_ts); + // should adjust for DST + assert_eq!(ts.value(2), dst_date + seven_hours_ts); + assert!(ts.is_null(3)); + + // Phoenix timezone (does not follow Daylight savings) + let result = cast_array( + Arc::clone(&dates), + &DataType::Timestamp(TimeUnit::Microsecond, Some("UTC".into())), + &SparkCastOptions::new(EvalMode::Legacy, "America/Phoenix", false), + ) + .unwrap(); + let ts = result.as_primitive::(); + assert_eq!(ts.value(0), seven_hours_ts); + assert_eq!(ts.value(1), non_dst_date + seven_hours_ts); + assert_eq!(ts.value(2), dst_date + seven_hours_ts); + assert!(ts.is_null(3)); + } + #[test] fn test_cast_struct_to_utf8() { let a: ArrayRef = Arc::new(Int32Array::from(vec![ diff --git a/native/spark-expr/src/csv_funcs/csv_write_options.rs b/native/spark-expr/src/csv_funcs/csv_write_options.rs new file mode 100644 index 0000000000..ee312e0f45 --- /dev/null +++ b/native/spark-expr/src/csv_funcs/csv_write_options.rs @@ -0,0 +1,61 @@ +// 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::fmt::{Display, Formatter}; + +#[derive(Debug, Clone, Hash, PartialEq, Eq)] +pub struct CsvWriteOptions { + pub delimiter: String, + pub quote: String, + pub escape: String, + pub null_value: String, + pub quote_all: bool, + pub ignore_leading_white_space: bool, + pub ignore_trailing_white_space: bool, +} + +impl Display for CsvWriteOptions { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "csv_write_options(delimiter={}, quote={}, escape={}, null_value={}, quote_all={}, ignore_leading_white_space={}, ignore_trailing_white_space={})", + self.delimiter, self.quote, self.escape, self.null_value, self.quote_all, self.ignore_leading_white_space, self.ignore_trailing_white_space + ) + } +} + +impl CsvWriteOptions { + pub fn new( + delimiter: String, + quote: String, + escape: String, + null_value: String, + quote_all: bool, + ignore_leading_white_space: bool, + ignore_trailing_white_space: bool, + ) -> Self { + Self { + delimiter, + quote, + escape, + null_value, + quote_all, + ignore_leading_white_space, + ignore_trailing_white_space, + } + } +} diff --git a/native/spark-expr/src/csv_funcs/mod.rs b/native/spark-expr/src/csv_funcs/mod.rs new file mode 100644 index 0000000000..9c417f9ebb --- /dev/null +++ b/native/spark-expr/src/csv_funcs/mod.rs @@ -0,0 +1,22 @@ +// 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. + +mod csv_write_options; +mod to_csv; + +pub use csv_write_options::CsvWriteOptions; +pub use to_csv::{to_csv_inner, ToCsv}; diff --git a/native/spark-expr/src/csv_funcs/to_csv.rs b/native/spark-expr/src/csv_funcs/to_csv.rs new file mode 100644 index 0000000000..5816f79993 --- /dev/null +++ b/native/spark-expr/src/csv_funcs/to_csv.rs @@ -0,0 +1,223 @@ +// 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 crate::csv_funcs::csv_write_options::CsvWriteOptions; +use crate::{spark_cast, EvalMode, SparkCastOptions}; +use arrow::array::{as_string_array, as_struct_array, Array, ArrayRef, StringArray, StringBuilder}; +use arrow::array::{RecordBatch, StructArray}; +use arrow::datatypes::{DataType, Schema}; +use datafusion::common::Result; +use datafusion::logical_expr::ColumnarValue; +use datafusion::physical_expr::PhysicalExpr; +use std::any::Any; +use std::fmt::{Display, Formatter}; +use std::hash::Hash; +use std::sync::Arc; + +/// to_csv spark function +#[derive(Debug, Eq)] +pub struct ToCsv { + expr: Arc, + timezone: String, + csv_write_options: CsvWriteOptions, +} + +impl Hash for ToCsv { + fn hash(&self, state: &mut H) { + self.expr.hash(state); + self.timezone.hash(state); + self.csv_write_options.hash(state); + } +} + +impl PartialEq for ToCsv { + fn eq(&self, other: &Self) -> bool { + self.expr.eq(&other.expr) + && self.timezone.eq(&other.timezone) + && self.csv_write_options.eq(&other.csv_write_options) + } +} + +impl ToCsv { + pub fn new( + expr: Arc, + timezone: &str, + csv_write_options: CsvWriteOptions, + ) -> Self { + Self { + expr, + timezone: timezone.to_owned(), + csv_write_options, + } + } +} + +impl Display for ToCsv { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "to_csv({}, timezone={}, csv_write_options={})", + self.expr, self.timezone, self.csv_write_options + ) + } +} + +impl PhysicalExpr for ToCsv { + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, _: &Schema) -> Result { + Ok(DataType::Utf8) + } + + fn nullable(&self, input_schema: &Schema) -> Result { + self.expr.nullable(input_schema) + } + + fn evaluate(&self, batch: &RecordBatch) -> Result { + let input_array = self.expr.evaluate(batch)?.into_array(batch.num_rows())?; + let mut cast_options = SparkCastOptions::new(EvalMode::Legacy, &self.timezone, false); + cast_options.null_string = self.csv_write_options.null_value.clone(); + let struct_array = as_struct_array(&input_array); + + let csv_array = to_csv_inner(struct_array, &cast_options, &self.csv_write_options)?; + + Ok(ColumnarValue::Array(csv_array)) + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.expr] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> Result> { + Ok(Arc::new(Self::new( + Arc::clone(&children[0]), + &self.timezone, + self.csv_write_options.clone(), + ))) + } + + fn fmt_sql(&self, _: &mut Formatter<'_>) -> std::fmt::Result { + unimplemented!() + } +} + +pub fn to_csv_inner( + array: &StructArray, + cast_options: &SparkCastOptions, + write_options: &CsvWriteOptions, +) -> Result { + let string_arrays: Vec = as_struct_array(&array) + .columns() + .iter() + .map(|array| { + spark_cast( + ColumnarValue::Array(Arc::clone(array)), + &DataType::Utf8, + cast_options, + )? + .into_array(array.len()) + }) + .collect::>>()?; + let string_arrays: Vec<&StringArray> = string_arrays + .iter() + .map(|array| as_string_array(array)) + .collect(); + let is_string: Vec = array + .fields() + .iter() + .map(|f| matches!(f.data_type(), DataType::Utf8 | DataType::LargeUtf8)) + .collect(); + + let mut builder = StringBuilder::with_capacity(array.len(), array.len() * 16); + let mut csv_string = String::with_capacity(array.len() * 16); + + let quote_char = write_options.quote.chars().next().unwrap_or('"'); + let escape_char = write_options.escape.chars().next().unwrap_or('\\'); + for row_idx in 0..array.len() { + if array.is_null(row_idx) { + builder.append_null(); + } else { + csv_string.clear(); + for (col_idx, column) in string_arrays.iter().enumerate() { + if col_idx > 0 { + csv_string.push_str(&write_options.delimiter); + } + if column.is_null(row_idx) { + if write_options.quote_all { + csv_string.push(quote_char); + } + csv_string.push_str(&write_options.null_value); + if write_options.quote_all { + csv_string.push(quote_char); + } + } else { + let mut value = column.value(row_idx); + let is_string_field = is_string[col_idx]; + + if is_string_field { + if write_options.ignore_leading_white_space { + value = value.trim_start(); + } + if write_options.ignore_trailing_white_space { + value = value.trim_end(); + } + } + + let needs_quoting = write_options.quote_all + || (is_string_field + && (value.contains(&write_options.delimiter) + || value.contains(quote_char) + || value.contains('\n') + || value.contains('\r')) + || value.is_empty()); + + let needs_escaping = needs_quoting + && (value.contains(quote_char) || value.contains(escape_char)); + + if needs_quoting { + csv_string.push(quote_char); + } + if needs_escaping { + escape_value(value, quote_char, escape_char, &mut csv_string); + } else { + csv_string.push_str(value); + } + if needs_quoting { + csv_string.push(quote_char); + } + } + } + builder.append_value(&csv_string); + } + } + Ok(Arc::new(builder.finish())) +} + +#[inline] +fn escape_value(value: &str, quote_char: char, escape_char: char, output: &mut String) { + for ch in value.chars() { + if ch == quote_char || ch == escape_char { + output.push(escape_char); + } + output.push(ch); + } +} diff --git a/native/spark-expr/src/datetime_funcs/date_diff.rs b/native/spark-expr/src/datetime_funcs/date_diff.rs index 6a593f0f87..ca148c103a 100644 --- a/native/spark-expr/src/datetime_funcs/date_diff.rs +++ b/native/spark-expr/src/datetime_funcs/date_diff.rs @@ -71,9 +71,18 @@ impl ScalarUDFImpl for SparkDateDiff { fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { let [end_date, start_date] = take_function_args(self.name(), args.args)?; - // Convert scalars to arrays for uniform processing - let end_arr = end_date.into_array(1)?; - let start_arr = start_date.into_array(1)?; + // Determine the batch size from array arguments (scalars have no inherent size) + let num_rows = [&end_date, &start_date] + .iter() + .find_map(|arg| match arg { + ColumnarValue::Array(array) => Some(array.len()), + ColumnarValue::Scalar(_) => None, + }) + .unwrap_or(1); + + // Convert scalars to arrays for uniform processing, using the correct batch size + let end_arr = end_date.into_array(num_rows)?; + let start_arr = start_date.into_array(num_rows)?; let end_date_array = end_arr .as_any() diff --git a/native/spark-expr/src/datetime_funcs/make_date.rs b/native/spark-expr/src/datetime_funcs/make_date.rs new file mode 100644 index 0000000000..58e4108580 --- /dev/null +++ b/native/spark-expr/src/datetime_funcs/make_date.rs @@ -0,0 +1,236 @@ +// 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 arrow::array::{Array, Date32Array, Int32Array}; +use arrow::compute::cast; +use arrow::datatypes::DataType; +use chrono::NaiveDate; +use datafusion::common::{utils::take_function_args, DataFusionError, Result}; +use datafusion::logical_expr::{ + ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, +}; +use std::any::Any; +use std::sync::Arc; + +/// Spark-compatible make_date function. +/// Creates a date from year, month, and day columns. +/// Returns NULL for invalid dates (e.g., Feb 30, month 13, etc.) instead of throwing an error. +#[derive(Debug, PartialEq, Eq, Hash)] +pub struct SparkMakeDate { + signature: Signature, +} + +impl SparkMakeDate { + pub fn new() -> Self { + Self { + // Accept any numeric type - we'll cast to Int32 internally + signature: Signature::any(3, Volatility::Immutable), + } + } +} + +impl Default for SparkMakeDate { + fn default() -> Self { + Self::new() + } +} + +/// Cast an array to Int32Array if it's not already Int32. +fn cast_to_int32(arr: &Arc) -> Result> { + if arr.data_type() == &DataType::Int32 { + Ok(Arc::clone(arr)) + } else { + cast(arr.as_ref(), &DataType::Int32) + .map_err(|e| DataFusionError::Execution(format!("Failed to cast to Int32: {e}"))) + } +} + +/// Convert year, month, day to days since Unix epoch (1970-01-01). +/// Returns None if the date is invalid. +fn make_date(year: i32, month: i32, day: i32) -> Option { + // Validate month and day ranges first + if !(1..=12).contains(&month) || !(1..=31).contains(&day) { + return None; + } + + // Try to create a valid date + NaiveDate::from_ymd_opt(year, month as u32, day as u32).map(|date| { + date.signed_duration_since(NaiveDate::from_ymd_opt(1970, 1, 1).unwrap()) + .num_days() as i32 + }) +} + +impl ScalarUDFImpl for SparkMakeDate { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "make_date" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _: &[DataType]) -> Result { + Ok(DataType::Date32) + } + + fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { + let [year, month, day] = take_function_args(self.name(), args.args)?; + + // Determine the batch size from array arguments (scalars have no inherent size) + let num_rows = [&year, &month, &day] + .iter() + .find_map(|arg| match arg { + ColumnarValue::Array(array) => Some(array.len()), + ColumnarValue::Scalar(_) => None, + }) + .unwrap_or(1); + + // Convert scalars to arrays for uniform processing, using the correct batch size + let year_arr = year.into_array(num_rows)?; + let month_arr = month.into_array(num_rows)?; + let day_arr = day.into_array(num_rows)?; + + // Cast to Int32 if needed (handles Int64 literals from SQL) + let year_arr = cast_to_int32(&year_arr)?; + let month_arr = cast_to_int32(&month_arr)?; + let day_arr = cast_to_int32(&day_arr)?; + + let year_array = year_arr + .as_any() + .downcast_ref::() + .ok_or_else(|| { + DataFusionError::Execution("make_date: failed to cast year to Int32".to_string()) + })?; + + let month_array = month_arr + .as_any() + .downcast_ref::() + .ok_or_else(|| { + DataFusionError::Execution("make_date: failed to cast month to Int32".to_string()) + })?; + + let day_array = day_arr + .as_any() + .downcast_ref::() + .ok_or_else(|| { + DataFusionError::Execution("make_date: failed to cast day to Int32".to_string()) + })?; + + let len = year_array.len(); + let mut builder = Date32Array::builder(len); + + for i in 0..len { + if year_array.is_null(i) || month_array.is_null(i) || day_array.is_null(i) { + builder.append_null(); + } else { + let y = year_array.value(i); + let m = month_array.value(i); + let d = day_array.value(i); + + match make_date(y, m, d) { + Some(days) => builder.append_value(days), + None => builder.append_null(), + } + } + } + + Ok(ColumnarValue::Array(Arc::new(builder.finish()))) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_make_date_valid() { + // Unix epoch + assert_eq!(make_date(1970, 1, 1), Some(0)); + // Day after epoch + assert_eq!(make_date(1970, 1, 2), Some(1)); + // Day before epoch + assert_eq!(make_date(1969, 12, 31), Some(-1)); + // Leap years - just verify they return Some (valid dates) + assert!(make_date(2000, 2, 29).is_some()); // 2000 is a leap year + assert!(make_date(2004, 2, 29).is_some()); // 2004 is a leap year + // Regular date + assert!(make_date(2023, 6, 15).is_some()); + } + + #[test] + fn test_make_date_invalid_month() { + assert_eq!(make_date(2023, 0, 15), None); + assert_eq!(make_date(2023, 13, 15), None); + assert_eq!(make_date(2023, -1, 15), None); + } + + #[test] + fn test_make_date_invalid_day() { + assert_eq!(make_date(2023, 6, 0), None); + assert_eq!(make_date(2023, 6, 32), None); + assert_eq!(make_date(2023, 6, -1), None); + } + + #[test] + fn test_make_date_invalid_dates() { + // Feb 30 never exists + assert_eq!(make_date(2023, 2, 30), None); + // Feb 29 on non-leap year + assert_eq!(make_date(2023, 2, 29), None); + // 1900 is not a leap year (divisible by 100 but not 400) + assert_eq!(make_date(1900, 2, 29), None); + // 2100 will not be a leap year + assert_eq!(make_date(2100, 2, 29), None); + // April has 30 days + assert_eq!(make_date(2023, 4, 31), None); + } + + #[test] + fn test_make_date_extreme_years() { + // Spark supports dates from 0001-01-01 to 9999-12-31 (Proleptic Gregorian calendar) + + // Minimum valid date in Spark: 0001-01-01 + assert!(make_date(1, 1, 1).is_some(), "Year 1 should be valid"); + + // Maximum valid date in Spark: 9999-12-31 + assert!( + make_date(9999, 12, 31).is_some(), + "Year 9999 should be valid" + ); + + // Year 0 - In Proleptic Gregorian calendar, year 0 = 1 BCE + // Spark returns NULL for year 0 in make_date + // chrono supports year 0, but we should match Spark's behavior + // For now, chrono allows it - this may need adjustment for full Spark compatibility + let year_0_result = make_date(0, 1, 1); + // chrono allows year 0 (1 BCE in proleptic Gregorian) + assert!(year_0_result.is_some(), "chrono allows year 0"); + + // Negative years - Spark returns NULL for negative years + // chrono supports negative years (BCE dates) + let negative_year_result = make_date(-1, 1, 1); + // chrono allows negative years + assert!( + negative_year_result.is_some(), + "chrono allows negative years" + ); + } +} diff --git a/native/spark-expr/src/datetime_funcs/mod.rs b/native/spark-expr/src/datetime_funcs/mod.rs index 1832711479..5bafc1d287 100644 --- a/native/spark-expr/src/datetime_funcs/mod.rs +++ b/native/spark-expr/src/datetime_funcs/mod.rs @@ -18,6 +18,7 @@ mod date_diff; mod date_trunc; mod extract_date_part; +mod make_date; mod timestamp_trunc; mod unix_timestamp; @@ -26,5 +27,6 @@ pub use date_trunc::SparkDateTrunc; pub use extract_date_part::SparkHour; pub use extract_date_part::SparkMinute; pub use extract_date_part::SparkSecond; +pub use make_date::SparkMakeDate; pub use timestamp_trunc::TimestampTruncExpr; pub use unix_timestamp::SparkUnixTimestamp; diff --git a/native/spark-expr/src/lib.rs b/native/spark-expr/src/lib.rs index 086c097304..40eb180ab8 100644 --- a/native/spark-expr/src/lib.rs +++ b/native/spark-expr/src/lib.rs @@ -29,6 +29,7 @@ pub use static_invoke::*; mod struct_funcs; pub use struct_funcs::{CreateNamedStruct, GetStructField}; +mod csv_funcs; mod json_funcs; pub mod test_common; pub mod timezone; @@ -69,9 +70,10 @@ pub use comet_scalar_funcs::{ create_comet_physical_fun, create_comet_physical_fun_with_eval_mode, register_all_comet_functions, }; +pub use csv_funcs::*; pub use datetime_funcs::{ - SparkDateDiff, SparkDateTrunc, SparkHour, SparkMinute, SparkSecond, SparkUnixTimestamp, - TimestampTruncExpr, + SparkDateDiff, SparkDateTrunc, SparkHour, SparkMakeDate, SparkMinute, SparkSecond, + SparkUnixTimestamp, TimestampTruncExpr, }; pub use error::{SparkError, SparkResult}; pub use hash_funcs::*; diff --git a/native/spark-expr/src/string_funcs/contains.rs b/native/spark-expr/src/string_funcs/contains.rs new file mode 100644 index 0000000000..bc34ce9cba --- /dev/null +++ b/native/spark-expr/src/string_funcs/contains.rs @@ -0,0 +1,246 @@ +// 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. + +//! Optimized `contains` string function for Spark compatibility. +//! +//! Optimized for scalar pattern case by passing scalar directly to arrow_contains +//! instead of expanding to arrays like DataFusion's built-in contains. + +use arrow::array::{Array, ArrayRef, BooleanArray, StringArray}; +use arrow::compute::kernels::comparison::contains as arrow_contains; +use arrow::datatypes::DataType; +use datafusion::common::{exec_err, Result, ScalarValue}; +use datafusion::logical_expr::{ + ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, +}; +use std::any::Any; +use std::sync::Arc; + +/// Spark-optimized contains function. +/// Returns true if the first string argument contains the second string argument. +/// Optimized for scalar pattern constants. +#[derive(Debug, PartialEq, Eq, Hash)] +pub struct SparkContains { + signature: Signature, +} + +impl Default for SparkContains { + fn default() -> Self { + Self::new() + } +} + +impl SparkContains { + pub fn new() -> Self { + Self { + signature: Signature::variadic_any(Volatility::Immutable), + } + } +} + +impl ScalarUDFImpl for SparkContains { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "contains" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(DataType::Boolean) + } + + fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { + if args.args.len() != 2 { + return exec_err!("contains function requires exactly 2 arguments"); + } + spark_contains(&args.args[0], &args.args[1]) + } +} + +/// Execute contains function with optimized scalar pattern handling. +fn spark_contains(haystack: &ColumnarValue, needle: &ColumnarValue) -> Result { + match (haystack, needle) { + // Both arrays - use arrow's contains directly + (ColumnarValue::Array(haystack_array), ColumnarValue::Array(needle_array)) => { + let result = arrow_contains(haystack_array, needle_array)?; + Ok(ColumnarValue::Array(Arc::new(result))) + } + + // Array haystack, scalar needle - OPTIMIZED PATH + (ColumnarValue::Array(haystack_array), ColumnarValue::Scalar(needle_scalar)) => { + let result = contains_with_arrow_scalar(haystack_array, needle_scalar)?; + Ok(ColumnarValue::Array(result)) + } + + // Scalar haystack, array needle - less common + (ColumnarValue::Scalar(haystack_scalar), ColumnarValue::Array(needle_array)) => { + let haystack_array = haystack_scalar.to_array_of_size(needle_array.len())?; + let result = arrow_contains(&haystack_array, needle_array)?; + Ok(ColumnarValue::Array(Arc::new(result))) + } + + // Both scalars - compute single result + (ColumnarValue::Scalar(haystack_scalar), ColumnarValue::Scalar(needle_scalar)) => { + let result = contains_scalar_scalar(haystack_scalar, needle_scalar)?; + Ok(ColumnarValue::Scalar(result)) + } + } +} + +/// Optimized contains for array haystack with scalar needle. +/// Uses Arrow's native scalar handling for better performance. +fn contains_with_arrow_scalar( + haystack_array: &ArrayRef, + needle_scalar: &ScalarValue, +) -> Result { + // Handle null needle + if needle_scalar.is_null() { + return Ok(Arc::new(BooleanArray::new_null(haystack_array.len()))); + } + + // Extract the needle string + let needle_str = match needle_scalar { + ScalarValue::Utf8(Some(s)) + | ScalarValue::LargeUtf8(Some(s)) + | ScalarValue::Utf8View(Some(s)) => s.clone(), + _ => { + return exec_err!( + "contains function requires string type for needle, got {:?}", + needle_scalar.data_type() + ) + } + }; + + // Create scalar array for needle - tells Arrow to use optimized paths + let needle_scalar_array = StringArray::new_scalar(needle_str); + + // Use Arrow's contains which detects scalar case and uses optimized paths + let result = arrow_contains(haystack_array, &needle_scalar_array)?; + Ok(Arc::new(result)) +} + +/// Contains for two scalar values. +fn contains_scalar_scalar( + haystack_scalar: &ScalarValue, + needle_scalar: &ScalarValue, +) -> Result { + // Handle nulls + if haystack_scalar.is_null() || needle_scalar.is_null() { + return Ok(ScalarValue::Boolean(None)); + } + + let haystack_str = match haystack_scalar { + ScalarValue::Utf8(Some(s)) + | ScalarValue::LargeUtf8(Some(s)) + | ScalarValue::Utf8View(Some(s)) => s.as_str(), + _ => { + return exec_err!( + "contains function requires string type for haystack, got {:?}", + haystack_scalar.data_type() + ) + } + }; + + let needle_str = match needle_scalar { + ScalarValue::Utf8(Some(s)) + | ScalarValue::LargeUtf8(Some(s)) + | ScalarValue::Utf8View(Some(s)) => s.as_str(), + _ => { + return exec_err!( + "contains function requires string type for needle, got {:?}", + needle_scalar.data_type() + ) + } + }; + + Ok(ScalarValue::Boolean(Some( + haystack_str.contains(needle_str), + ))) +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow::array::StringArray; + + #[test] + fn test_contains_array_scalar() { + let haystack = Arc::new(StringArray::from(vec![ + Some("hello world"), + Some("foo bar"), + Some("testing"), + None, + ])) as ArrayRef; + let needle = ScalarValue::Utf8(Some("world".to_string())); + + let result = contains_with_arrow_scalar(&haystack, &needle).unwrap(); + let bool_array = result.as_any().downcast_ref::().unwrap(); + + assert!(bool_array.value(0)); // "hello world" contains "world" + assert!(!bool_array.value(1)); // "foo bar" does not contain "world" + assert!(!bool_array.value(2)); // "testing" does not contain "world" + assert!(bool_array.is_null(3)); // null input => null output + } + + #[test] + fn test_contains_scalar_scalar() { + let haystack = ScalarValue::Utf8(Some("hello world".to_string())); + let needle = ScalarValue::Utf8(Some("world".to_string())); + + let result = contains_scalar_scalar(&haystack, &needle).unwrap(); + assert_eq!(result, ScalarValue::Boolean(Some(true))); + + let needle_not_found = ScalarValue::Utf8(Some("xyz".to_string())); + let result = contains_scalar_scalar(&haystack, &needle_not_found).unwrap(); + assert_eq!(result, ScalarValue::Boolean(Some(false))); + } + + #[test] + fn test_contains_null_needle() { + let haystack = Arc::new(StringArray::from(vec![ + Some("hello world"), + Some("foo bar"), + ])) as ArrayRef; + let needle = ScalarValue::Utf8(None); + + let result = contains_with_arrow_scalar(&haystack, &needle).unwrap(); + let bool_array = result.as_any().downcast_ref::().unwrap(); + + // Null needle should produce null results + assert!(bool_array.is_null(0)); + assert!(bool_array.is_null(1)); + } + + #[test] + fn test_contains_empty_needle() { + let haystack = Arc::new(StringArray::from(vec![Some("hello world"), Some("")])) as ArrayRef; + let needle = ScalarValue::Utf8(Some("".to_string())); + + let result = contains_with_arrow_scalar(&haystack, &needle).unwrap(); + let bool_array = result.as_any().downcast_ref::().unwrap(); + + // Empty string is contained in any string + assert!(bool_array.value(0)); + assert!(bool_array.value(1)); + } +} diff --git a/native/spark-expr/src/string_funcs/mod.rs b/native/spark-expr/src/string_funcs/mod.rs index aac8204e29..abdd0cc89b 100644 --- a/native/spark-expr/src/string_funcs/mod.rs +++ b/native/spark-expr/src/string_funcs/mod.rs @@ -15,8 +15,10 @@ // specific language governing permissions and limitations // under the License. +mod contains; mod string_space; mod substring; +pub use contains::SparkContains; pub use string_space::SparkStringSpace; pub use substring::SubstringExpr; diff --git a/native/spark-expr/src/string_funcs/string_space.rs b/native/spark-expr/src/string_funcs/string_space.rs index 4ab5362793..78d94208d8 100644 --- a/native/spark-expr/src/string_funcs/string_space.rs +++ b/native/spark-expr/src/string_funcs/string_space.rs @@ -21,7 +21,7 @@ use arrow::array::{ }; use arrow::buffer::MutableBuffer; use arrow::datatypes::{DataType, Int32Type}; -use datafusion::common::{exec_err, internal_datafusion_err, DataFusionError, Result}; +use datafusion::common::{exec_err, internal_datafusion_err, DataFusionError, Result, ScalarValue}; use datafusion::logical_expr::{ ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, }; @@ -86,15 +86,17 @@ impl ScalarUDFImpl for SparkStringSpace { pub fn spark_string_space(args: &[ColumnarValue; 1]) -> Result { match args { [ColumnarValue::Array(array)] => { - let result = string_space(&array)?; - + let result = string_space_array(&array)?; Ok(ColumnarValue::Array(result)) } - _ => exec_err!("StringSpace(scalar) should be fold in Spark JVM side."), + [ColumnarValue::Scalar(scalar)] => { + let result = string_space_scalar(scalar)?; + Ok(ColumnarValue::Scalar(result)) + } } } -fn string_space(length: &dyn Array) -> std::result::Result { +fn string_space_array(length: &dyn Array) -> std::result::Result { match length.data_type() { DataType::Int32 => { let array = length.as_any().downcast_ref::().unwrap(); @@ -102,7 +104,7 @@ fn string_space(length: &dyn Array) -> std::result::Result { let dict = as_dictionary_array::(length); - let values = string_space(dict.values())?; + let values = string_space_array(dict.values())?; let result = DictionaryArray::try_new(dict.keys().clone(), values)?; Ok(Arc::new(result)) } @@ -110,6 +112,24 @@ fn string_space(length: &dyn Array) -> std::result::Result Result { + match scalar { + ScalarValue::Int32(value) => { + let result = value.map(|v| { + if v <= 0 { + String::new() + } else { + " ".repeat(v as usize) + } + }); + Ok(ScalarValue::Utf8(result)) + } + other => { + exec_err!("Unsupported data type {other:?} for function `space`") + } + } +} + fn generic_string_space(length: &Int32Array) -> ArrayRef { let array_len = length.len(); let mut offsets = MutableBuffer::new((array_len + 1) * std::mem::size_of::()); diff --git a/native/spark-expr/tests/spark_expr_reg.rs b/native/spark-expr/tests/spark_expr_reg.rs index 88e34ae2ba..633b226068 100644 --- a/native/spark-expr/tests/spark_expr_reg.rs +++ b/native/spark-expr/tests/spark_expr_reg.rs @@ -23,6 +23,7 @@ mod tests { use datafusion::execution::FunctionRegistry; use datafusion::prelude::SessionContext; use datafusion_comet_spark_expr::create_comet_physical_fun; + use datafusion_comet_spark_expr::register_all_comet_functions; #[tokio::test] async fn test_udf_registration() -> Result<()> { @@ -48,4 +49,46 @@ mod tests { Ok(()) } + + #[tokio::test] + async fn test_make_date_returns_null_for_invalid_input() -> Result<()> { + // Setup session with all Comet functions registered + let mut ctx = SessionContext::new(); + register_all_comet_functions(&mut ctx)?; + + // Test that make_date returns NULL for invalid month (0) + // DataFusion's built-in make_date would throw an error + let df = ctx.sql("SELECT make_date(2023, 0, 15)").await?; + let results = df.collect().await?; + + // Should return one row with NULL + assert_eq!(results.len(), 1); + assert_eq!(results[0].num_rows(), 1); + + // The result should be NULL for invalid input + let column = results[0].column(0); + assert!(column.is_null(0), "Expected NULL for invalid month"); + + Ok(()) + } + + #[tokio::test] + async fn test_make_date_valid_input() -> Result<()> { + // Setup session with all Comet functions registered + let mut ctx = SessionContext::new(); + register_all_comet_functions(&mut ctx)?; + + // Test that make_date works for valid input + let df = ctx.sql("SELECT make_date(1970, 1, 1)").await?; + let results = df.collect().await?; + + assert_eq!(results.len(), 1); + assert_eq!(results[0].num_rows(), 1); + + // Should return epoch date (1970-01-01 = day 0) + let column = results[0].column(0); + assert!(!column.is_null(0), "Expected valid date for epoch"); + + Ok(()) + } } diff --git a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala index 9fc4b3afdf..f42a5d8d8e 100644 --- a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala +++ b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala @@ -168,6 +168,7 @@ object CometCast extends CometExpressionSerde[Cast] with CometExprShim { } } Compatible() + case (DataTypes.DateType, toType) => canCastFromDate(toType) case _ => unsupported(fromType, toType) } } @@ -344,6 +345,12 @@ object CometCast extends CometExpressionSerde[Cast] with CometExprShim { case _ => Unsupported(Some(s"Cast from DecimalType to $toType is not supported")) } + private def canCastFromDate(toType: DataType): SupportLevel = toType match { + case DataTypes.TimestampType => + Compatible() + case _ => Unsupported(Some(s"Cast from DateType to $toType is not supported")) + } + private def unsupported(fromType: DataType, toType: DataType): Unsupported = { Unsupported(Some(s"Cast from $fromType to $toType is not supported")) } diff --git a/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala b/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala index 2d772063e4..7642749ad8 100644 --- a/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala +++ b/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala @@ -333,32 +333,6 @@ object IcebergReflection extends Logging { } } - /** - * Gets delete files from scan tasks. - * - * @param tasks - * List of Iceberg FileScanTask objects - * @return - * List of all delete files across all tasks - * @throws Exception - * if reflection fails (callers must handle appropriately based on context) - */ - def getDeleteFiles(tasks: java.util.List[_]): java.util.List[_] = { - import scala.jdk.CollectionConverters._ - val allDeletes = new java.util.ArrayList[Any]() - - // scalastyle:off classforname - val fileScanTaskClass = Class.forName(ClassNames.FILE_SCAN_TASK) - // scalastyle:on classforname - - tasks.asScala.foreach { task => - val deletes = getDeleteFilesFromTask(task, fileScanTaskClass) - allDeletes.addAll(deletes) - } - - allDeletes - } - /** * Gets delete files from a single FileScanTask. * @@ -495,91 +469,6 @@ object IcebergReflection extends Logging { } } - /** - * Validates file formats and filesystem schemes for Iceberg tasks. - * - * Checks that all data files and delete files are Parquet format and use filesystem schemes - * supported by iceberg-rust (file, s3, s3a, gs, gcs, oss, abfss, abfs, wasbs, wasb). - * - * @param tasks - * List of Iceberg FileScanTask objects - * @return - * (allParquet, unsupportedSchemes) where: - allParquet: true if all files are Parquet format - * \- unsupportedSchemes: Set of unsupported filesystem schemes found (empty if all supported) - */ - def validateFileFormatsAndSchemes(tasks: java.util.List[_]): (Boolean, Set[String]) = { - import scala.jdk.CollectionConverters._ - - // scalastyle:off classforname - val contentScanTaskClass = Class.forName(ClassNames.CONTENT_SCAN_TASK) - val contentFileClass = Class.forName(ClassNames.CONTENT_FILE) - // scalastyle:on classforname - - val fileMethod = contentScanTaskClass.getMethod("file") - val formatMethod = contentFileClass.getMethod("format") - val pathMethod = contentFileClass.getMethod("path") - - // Filesystem schemes supported by iceberg-rust - // See: iceberg-rust/crates/iceberg/src/io/storage.rs parse_scheme() - val supportedSchemes = - Set("file", "s3", "s3a", "gs", "gcs", "oss", "abfss", "abfs", "wasbs", "wasb") - - var allParquet = true - val unsupportedSchemes = scala.collection.mutable.Set[String]() - - tasks.asScala.foreach { task => - val dataFile = fileMethod.invoke(task) - val fileFormat = formatMethod.invoke(dataFile).toString - - // Check file format - if (fileFormat != FileFormats.PARQUET) { - allParquet = false - } else { - // Only check filesystem schemes for Parquet files we'll actually process - try { - val filePath = pathMethod.invoke(dataFile).toString - val uri = new java.net.URI(filePath) - val scheme = uri.getScheme - - if (scheme != null && !supportedSchemes.contains(scheme)) { - unsupportedSchemes += scheme - } - } catch { - case _: java.net.URISyntaxException => - // Ignore URI parsing errors - file paths may contain special characters - // If the path is invalid, we'll fail later during actual file access - } - - // Check delete files if they exist - try { - val deletesMethod = task.getClass.getMethod("deletes") - val deleteFiles = deletesMethod.invoke(task).asInstanceOf[java.util.List[_]] - - deleteFiles.asScala.foreach { deleteFile => - extractFileLocation(contentFileClass, deleteFile).foreach { deletePath => - try { - val deleteUri = new java.net.URI(deletePath) - val deleteScheme = deleteUri.getScheme - - if (deleteScheme != null && !supportedSchemes.contains(deleteScheme)) { - unsupportedSchemes += deleteScheme - } - } catch { - case _: java.net.URISyntaxException => - // Ignore URI parsing errors for delete files too - } - } - } - } catch { - case _: Exception => - // Ignore errors accessing delete files - they may not be supported - } - } - } - - (allParquet, unsupportedSchemes.toSet) - } - /** * Validates partition column types for compatibility with iceberg-rust. * @@ -643,68 +532,6 @@ object IcebergReflection extends Logging { unsupportedTypes.toList } - - /** - * Checks if tasks have non-identity transforms in their residual expressions. - * - * Residual expressions are filters that must be evaluated after reading data from Parquet. - * iceberg-rust can only handle simple column references in residuals, not transformed columns. - * Transform functions like truncate, bucket, year, month, day, hour require evaluation by - * Spark. - * - * @param tasks - * List of Iceberg FileScanTask objects - * @return - * Some(transformType) if an unsupported transform is found (e.g., "truncate[4]"), None if all - * transforms are identity or no transforms are present - * @throws Exception - * if reflection fails - caller must handle appropriately (fallback in planning, fatal in - * serialization) - */ - def findNonIdentityTransformInResiduals(tasks: java.util.List[_]): Option[String] = { - import scala.jdk.CollectionConverters._ - - // scalastyle:off classforname - val fileScanTaskClass = Class.forName(ClassNames.FILE_SCAN_TASK) - val contentScanTaskClass = Class.forName(ClassNames.CONTENT_SCAN_TASK) - val unboundPredicateClass = Class.forName(ClassNames.UNBOUND_PREDICATE) - // scalastyle:on classforname - - tasks.asScala.foreach { task => - if (fileScanTaskClass.isInstance(task)) { - try { - val residualMethod = contentScanTaskClass.getMethod("residual") - val residual = residualMethod.invoke(task) - - // Check if residual is an UnboundPredicate with a transform - if (unboundPredicateClass.isInstance(residual)) { - val termMethod = unboundPredicateClass.getMethod("term") - val term = termMethod.invoke(residual) - - // Check if term has a transform - try { - val transformMethod = term.getClass.getMethod("transform") - transformMethod.setAccessible(true) - val transform = transformMethod.invoke(term) - val transformStr = transform.toString - - // Only identity transform is supported in residuals - if (transformStr != Transforms.IDENTITY) { - return Some(transformStr) - } - } catch { - case _: NoSuchMethodException => - // No transform method means it's a simple reference - OK - } - } - } catch { - case _: Exception => - // Skip tasks where we can't get residual - they may not have one - } - } - } - None - } } /** @@ -734,7 +561,7 @@ case class CometIcebergNativeScanMetadata( table: Any, metadataLocation: String, nameMapping: Option[String], - tasks: java.util.List[_], + @transient tasks: java.util.List[_], scanSchema: Any, tableSchema: Any, globalFieldIdMapping: Map[String, Int], @@ -783,10 +610,8 @@ object CometIcebergNativeScanMetadata extends Logging { val globalFieldIdMapping = buildFieldIdMapping(scanSchema) // File format is always PARQUET, - // validated in CometScanRule.validateFileFormatsAndSchemes() + // validated in CometScanRule.validateIcebergFileScanTasks() // Hardcoded here for extensibility (future ORC/Avro support would add logic here) - val fileFormat = FileFormats.PARQUET - CometIcebergNativeScanMetadata( table = table, metadataLocation = metadataLocation, @@ -796,7 +621,7 @@ object CometIcebergNativeScanMetadata extends Logging { tableSchema = tableSchema, globalFieldIdMapping = globalFieldIdMapping, catalogProperties = catalogProperties, - fileFormat = fileFormat) + fileFormat = FileFormats.PARQUET) } } } diff --git a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala index 45faa4d940..50abb26087 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -28,16 +28,16 @@ import scala.jdk.CollectionConverters._ import org.apache.hadoop.conf.Configuration import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, GenericInternalRow, PlanExpression} +import org.apache.spark.sql.catalyst.expressions.{Attribute, DynamicPruningExpression, Expression, GenericInternalRow, PlanExpression} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.{sideBySide, ArrayBasedMapData, GenericArrayData, MetadataColumnHelper} import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.getExistenceDefaultValues import org.apache.spark.sql.comet.{CometBatchScanExec, CometScanExec} -import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} +import org.apache.spark.sql.execution.{FileSourceScanExec, InSubqueryExec, SparkPlan, SubqueryAdaptiveBroadcastExec} import org.apache.spark.sql.execution.datasources.HadoopFsRelation +import org.apache.spark.sql.execution.datasources.parquet.ParquetUtils import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan -import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -47,17 +47,18 @@ import org.apache.comet.CometSparkSessionExtensions.{isCometLoaded, withInfo, wi import org.apache.comet.DataTypeSupport.isComplexType import org.apache.comet.iceberg.{CometIcebergNativeScanMetadata, IcebergReflection} import org.apache.comet.objectstore.NativeConfig -import org.apache.comet.parquet.{CometParquetScan, Native, SupportsComet} +import org.apache.comet.parquet.{Native, SupportsComet} import org.apache.comet.parquet.CometParquetUtils.{encryptionEnabled, isEncryptionConfigSupported} import org.apache.comet.serde.operator.CometNativeScan -import org.apache.comet.shims.CometTypeShim +import org.apache.comet.shims.{CometTypeShim, ShimFileFormat, ShimSubqueryBroadcast} /** * Spark physical optimizer rule for replacing Spark scans with Comet scans. */ -case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with CometTypeShim { - - import CometScanRule._ +case class CometScanRule(session: SparkSession) + extends Rule[SparkPlan] + with CometTypeShim + with ShimSubqueryBroadcast { private lazy val showTransformations = CometConf.COMET_EXPLAIN_TRANSFORMATIONS.get() @@ -172,8 +173,6 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com nativeDataFusionScan(session, scanExec, r, hadoopConf).getOrElse(scanExec) case SCAN_NATIVE_ICEBERG_COMPAT => nativeIcebergCompatScan(session, scanExec, r, hadoopConf).getOrElse(scanExec) - case SCAN_NATIVE_COMET => - nativeCometScan(session, scanExec, r, hadoopConf).getOrElse(scanExec) } case _ => @@ -193,6 +192,19 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com withInfo(scanExec, s"$SCAN_NATIVE_DATAFUSION does not support encryption") return None } + if (scanExec.fileConstantMetadataColumns.nonEmpty) { + withInfo(scanExec, "Native DataFusion scan does not support metadata columns") + return None + } + if (ShimFileFormat.findRowIndexColumnIndexInSchema(scanExec.requiredSchema) >= 0) { + withInfo(scanExec, "Native DataFusion scan does not support row index generation") + return None + } + if (session.sessionState.conf.getConf(SQLConf.PARQUET_FIELD_ID_READ_ENABLED) && + ParquetUtils.hasFieldIds(scanExec.requiredSchema)) { + withInfo(scanExec, "Native DataFusion scan does not support Parquet field ID matching") + return None + } if (!isSchemaSupported(scanExec, SCAN_NATIVE_DATAFUSION, r)) { return None } @@ -214,47 +226,9 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com Some(CometScanExec(scanExec, session, SCAN_NATIVE_ICEBERG_COMPAT)) } - private def nativeCometScan( - session: SparkSession, - scanExec: FileSourceScanExec, - r: HadoopFsRelation, - hadoopConf: Configuration): Option[SparkPlan] = { - if (!isSchemaSupported(scanExec, SCAN_NATIVE_COMET, r)) { - return None - } - Some(CometScanExec(scanExec, session, SCAN_NATIVE_COMET)) - } - private def transformV2Scan(scanExec: BatchScanExec): SparkPlan = { scanExec.scan match { - case scan: ParquetScan if COMET_NATIVE_SCAN_IMPL.get() == SCAN_NATIVE_COMET => - val fallbackReasons = new ListBuffer[String]() - val schemaSupported = - CometBatchScanExec.isSchemaSupported(scan.readDataSchema, fallbackReasons) - if (!schemaSupported) { - fallbackReasons += s"Schema ${scan.readDataSchema} is not supported" - } - - val partitionSchemaSupported = - CometBatchScanExec.isSchemaSupported(scan.readPartitionSchema, fallbackReasons) - if (!partitionSchemaSupported) { - fallbackReasons += s"Partition schema ${scan.readPartitionSchema} is not supported" - } - - if (scan.pushedAggregate.nonEmpty) { - fallbackReasons += "Comet does not support pushed aggregate" - } - - if (schemaSupported && partitionSchemaSupported && scan.pushedAggregate.isEmpty) { - val cometScan = CometParquetScan(session, scanExec.scan.asInstanceOf[ParquetScan]) - CometBatchScanExec( - scanExec.copy(scan = cometScan), - runtimeFilters = scanExec.runtimeFilters) - } else { - withInfos(scanExec, fallbackReasons.toSet) - } - case scan: CSVScan if COMET_CSV_V2_NATIVE_ENABLED.get() => val fallbackReasons = new ListBuffer[String]() val schemaSupported = @@ -327,10 +301,6 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com case _ if scanExec.scan.getClass.getName == "org.apache.iceberg.spark.source.SparkBatchQueryScan" => - if (scanExec.runtimeFilters.exists(isDynamicPruningFilter)) { - return withInfo(scanExec, "Dynamic Partition Pruning is not supported") - } - val fallbackReasons = new ListBuffer[String]() // Native Iceberg scan requires both configs to be enabled @@ -454,20 +424,28 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com false } - // Check if all files are Parquet format and use supported filesystem schemes - val (allParquetFiles, unsupportedSchemes) = - IcebergReflection.validateFileFormatsAndSchemes(metadata.tasks) + // Single-pass validation of all FileScanTasks + val taskValidation = + try { + CometScanRule.validateIcebergFileScanTasks(metadata.tasks) + } catch { + case e: Exception => + fallbackReasons += "Iceberg reflection failure: Could not validate " + + s"FileScanTasks: ${e.getMessage}" + return withInfos(scanExec, fallbackReasons.toSet) + } - val allSupportedFilesystems = if (unsupportedSchemes.isEmpty) { + // Check if all files are Parquet format and use supported filesystem schemes + val allSupportedFilesystems = if (taskValidation.unsupportedSchemes.isEmpty) { true } else { fallbackReasons += "Iceberg scan contains files with unsupported filesystem " + - s"schemes: ${unsupportedSchemes.mkString(", ")}. " + + s"schemes: ${taskValidation.unsupportedSchemes.mkString(", ")}. " + "Comet only supports: file, s3, s3a, gs, gcs, oss, abfss, abfs, wasbs, wasb" false } - if (!allParquetFiles) { + if (!taskValidation.allParquet) { fallbackReasons += "Iceberg scan contains non-Parquet files (ORC or Avro). " + "Comet only supports Parquet files in Iceberg tables" } @@ -555,37 +533,24 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com // Check for unsupported transform functions in residual expressions // iceberg-rust can only handle identity transforms in residuals; all other transforms // (truncate, bucket, year, month, day, hour) must fall back to Spark - val transformFunctionsSupported = - try { - IcebergReflection.findNonIdentityTransformInResiduals(metadata.tasks) match { - case Some(transformType) => - // Found unsupported transform - fallbackReasons += - s"Iceberg transform function '$transformType' in residual expression " + - "is not yet supported by iceberg-rust. " + - "Only identity transforms are supported." - false - case None => - // No unsupported transforms found - safe to use native execution - true - } - } catch { - case e: Exception => - // Reflection failure - cannot verify safety, must fall back - fallbackReasons += "Iceberg reflection failure: Could not check for " + - s"transform functions in residuals: ${e.getMessage}" - false - } + val transformFunctionsSupported = taskValidation.nonIdentityTransform match { + case Some(transformType) => + fallbackReasons += + s"Iceberg transform function '$transformType' in residual expression " + + "is not yet supported by iceberg-rust. " + + "Only identity transforms are supported." + false + case None => + true + } // Check for unsupported struct types in delete files val deleteFileTypesSupported = { var hasUnsupportedDeletes = false try { - val deleteFiles = IcebergReflection.getDeleteFiles(metadata.tasks) - - if (!deleteFiles.isEmpty) { - deleteFiles.asScala.foreach { deleteFile => + if (!taskValidation.deleteFiles.isEmpty) { + taskValidation.deleteFiles.asScala.foreach { deleteFile => val equalityFieldIds = IcebergReflection.getEqualityFieldIds(deleteFile) if (!equalityFieldIds.isEmpty) { @@ -621,10 +586,47 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com !hasUnsupportedDeletes } - if (schemaSupported && fileIOCompatible && formatVersionSupported && allParquetFiles && - allSupportedFilesystems && partitionTypesSupported && + // Check that all DPP subqueries use InSubqueryExec which we know how to handle. + // Future Spark versions might introduce new subquery types we haven't tested. + val dppSubqueriesSupported = { + val unsupportedSubqueries = scanExec.runtimeFilters.collect { + case DynamicPruningExpression(e) if !e.isInstanceOf[InSubqueryExec] => + e.getClass.getSimpleName + } + // Check for multi-index DPP which we don't support yet. + // SPARK-46946 changed SubqueryAdaptiveBroadcastExec from index: Int to indices: Seq[Int] + // as a preparatory refactor for future features (Null Safe Equality DPP, multiple + // equality predicates). Currently indices always has one element, but future Spark + // versions might use multiple indices. + val multiIndexDpp = scanExec.runtimeFilters.exists { + case DynamicPruningExpression(e: InSubqueryExec) => + e.plan match { + case sab: SubqueryAdaptiveBroadcastExec => + getSubqueryBroadcastIndices(sab).length > 1 + case _ => false + } + case _ => false + } + if (unsupportedSubqueries.nonEmpty) { + fallbackReasons += + s"Unsupported DPP subquery types: ${unsupportedSubqueries.mkString(", ")}. " + + "CometIcebergNativeScanExec only supports InSubqueryExec for DPP" + false + } else if (multiIndexDpp) { + // See SPARK-46946 for context on multi-index DPP + fallbackReasons += + "Multi-index DPP (indices.length > 1) is not yet supported. " + + "See SPARK-46946 for context." + false + } else { + true + } + } + + if (schemaSupported && fileIOCompatible && formatVersionSupported && + taskValidation.allParquet && allSupportedFilesystems && partitionTypesSupported && complexTypePredicatesSupported && transformFunctionsSupported && - deleteFileTypesSupported) { + deleteFileTypesSupported && dppSubqueriesSupported) { CometBatchScanExec( scanExec.clone().asInstanceOf[BatchScanExec], runtimeFilters = scanExec.runtimeFilters, @@ -641,48 +643,6 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com } } - private def selectScan( - scanExec: FileSourceScanExec, - partitionSchema: StructType, - hadoopConf: Configuration): String = { - - val fallbackReasons = new ListBuffer[String]() - - // native_iceberg_compat only supports local filesystem and S3 - if (scanExec.relation.inputFiles - .forall(path => path.startsWith("file://") || path.startsWith("s3a://"))) { - - val filePath = scanExec.relation.inputFiles.headOption - if (filePath.exists(_.startsWith("s3a://"))) { - validateObjectStoreConfig(filePath.get, hadoopConf, fallbackReasons) - } - } else { - fallbackReasons += s"$SCAN_NATIVE_ICEBERG_COMPAT only supports local filesystem and S3" - } - - val typeChecker = CometScanTypeChecker(SCAN_NATIVE_ICEBERG_COMPAT) - val schemaSupported = - typeChecker.isSchemaSupported(scanExec.requiredSchema, fallbackReasons) - val partitionSchemaSupported = - typeChecker.isSchemaSupported(partitionSchema, fallbackReasons) - - val cometExecEnabled = COMET_EXEC_ENABLED.get() - if (!cometExecEnabled) { - fallbackReasons += s"$SCAN_NATIVE_ICEBERG_COMPAT requires ${COMET_EXEC_ENABLED.key}=true" - } - - if (cometExecEnabled && schemaSupported && partitionSchemaSupported && - fallbackReasons.isEmpty) { - logInfo(s"Auto scan mode selecting $SCAN_NATIVE_ICEBERG_COMPAT") - SCAN_NATIVE_ICEBERG_COMPAT - } else { - logInfo( - s"Auto scan mode falling back to $SCAN_NATIVE_COMET due to " + - s"${fallbackReasons.mkString(", ")}") - SCAN_NATIVE_COMET - } - } - private def isDynamicPruningFilter(e: Expression): Boolean = e.exists(_.isInstanceOf[PlanExpression[_]]) @@ -725,16 +685,12 @@ case class CometScanTypeChecker(scanImpl: String) extends DataTypeSupport with C name: String, fallbackReasons: ListBuffer[String]): Boolean = { dt match { - case ShortType - if scanImpl != CometConf.SCAN_NATIVE_COMET && - CometConf.COMET_PARQUET_UNSIGNED_SMALL_INT_CHECK.get() => + case ShortType if CometConf.COMET_PARQUET_UNSIGNED_SMALL_INT_CHECK.get() => fallbackReasons += s"$scanImpl scan may not handle unsigned UINT_8 correctly for $dt. " + s"Set ${CometConf.COMET_PARQUET_UNSIGNED_SMALL_INT_CHECK.key}=false to allow " + "native execution if your data does not contain unsigned small integers. " + CometConf.COMPAT_GUIDE false - case _: StructType | _: ArrayType | _: MapType if scanImpl == CometConf.SCAN_NATIVE_COMET => - false case dt if isStringCollationType(dt) => // we don't need specific support for collation in scans, but this // is a convenient place to force the whole query to fall back to Spark for now @@ -804,4 +760,120 @@ object CometScanRule extends Logging { } } + + /** + * Single-pass validation of Iceberg FileScanTasks. + * + * Consolidates file format, filesystem scheme, residual transform, and delete file checks into + * one iteration for better performance with large tables. + */ + def validateIcebergFileScanTasks(tasks: java.util.List[_]): IcebergTaskValidationResult = { + // scalastyle:off classforname + val contentScanTaskClass = Class.forName(IcebergReflection.ClassNames.CONTENT_SCAN_TASK) + val contentFileClass = Class.forName(IcebergReflection.ClassNames.CONTENT_FILE) + val fileScanTaskClass = Class.forName(IcebergReflection.ClassNames.FILE_SCAN_TASK) + val unboundPredicateClass = Class.forName(IcebergReflection.ClassNames.UNBOUND_PREDICATE) + // scalastyle:on classforname + + // Cache all method lookups outside the loop + val fileMethod = contentScanTaskClass.getMethod("file") + val formatMethod = contentFileClass.getMethod("format") + val pathMethod = contentFileClass.getMethod("path") + val residualMethod = contentScanTaskClass.getMethod("residual") + val deletesMethod = fileScanTaskClass.getMethod("deletes") + val termMethod = unboundPredicateClass.getMethod("term") + + val supportedSchemes = + Set("file", "s3", "s3a", "gs", "gcs", "oss", "abfss", "abfs", "wasbs", "wasb") + + var allParquet = true + val unsupportedSchemes = mutable.Set[String]() + var nonIdentityTransform: Option[String] = None + val deleteFiles = new java.util.ArrayList[Any]() + + tasks.asScala.foreach { task => + val dataFile = fileMethod.invoke(task) + + // File format check + val fileFormat = formatMethod.invoke(dataFile).toString + if (fileFormat != IcebergReflection.FileFormats.PARQUET) { + allParquet = false + } + + // Filesystem scheme check for data file + try { + val filePath = pathMethod.invoke(dataFile).toString + val uri = new URI(filePath) + val scheme = uri.getScheme + if (scheme != null && !supportedSchemes.contains(scheme)) { + unsupportedSchemes += scheme + } + } catch { + case _: java.net.URISyntaxException => // ignore + } + + // Residual transform check (short-circuit if already found unsupported) + if (nonIdentityTransform.isEmpty && fileScanTaskClass.isInstance(task)) { + try { + val residual = residualMethod.invoke(task) + if (unboundPredicateClass.isInstance(residual)) { + val term = termMethod.invoke(residual) + try { + val transformMethod = term.getClass.getMethod("transform") + transformMethod.setAccessible(true) + val transform = transformMethod.invoke(term) + val transformStr = transform.toString + if (transformStr != IcebergReflection.Transforms.IDENTITY) { + nonIdentityTransform = Some(transformStr) + } + } catch { + case _: NoSuchMethodException => // No transform = simple reference, OK + } + } + } catch { + case _: Exception => // Skip tasks where we can't get residual + } + } + + // Collect delete files and check their schemes + if (fileScanTaskClass.isInstance(task)) { + try { + val deletes = deletesMethod.invoke(task).asInstanceOf[java.util.List[_]] + deleteFiles.addAll(deletes) + + deletes.asScala.foreach { deleteFile => + IcebergReflection.extractFileLocation(contentFileClass, deleteFile).foreach { + deletePath => + try { + val deleteUri = new URI(deletePath) + val deleteScheme = deleteUri.getScheme + if (deleteScheme != null && !supportedSchemes.contains(deleteScheme)) { + unsupportedSchemes += deleteScheme + } + } catch { + case _: java.net.URISyntaxException => // ignore + } + } + } + } catch { + case _: Exception => // ignore errors accessing delete files + } + } + } + + IcebergTaskValidationResult( + allParquet, + unsupportedSchemes.toSet, + nonIdentityTransform, + deleteFiles) + } } + +/** + * Result of single-pass validation over Iceberg FileScanTasks. + */ +case class IcebergTaskValidationResult( + allParquet: Boolean, + unsupportedSchemes: Set[String], + nonIdentityTransform: Option[String], + deleteFiles: java.util.List[_]) diff --git a/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala b/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala index d1c3b07677..d56c9b6333 100644 --- a/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala +++ b/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala @@ -22,7 +22,7 @@ package org.apache.comet.rules import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.sideBySide -import org.apache.spark.sql.comet.{CometBatchScanExec, CometCollectLimitExec, CometColumnarToRowExec, CometNativeColumnarToRowExec, CometNativeWriteExec, CometPlan, CometScanExec, CometSparkToColumnarExec} +import org.apache.spark.sql.comet.{CometBatchScanExec, CometCollectLimitExec, CometColumnarToRowExec, CometNativeColumnarToRowExec, CometNativeWriteExec, CometPlan, CometSparkToColumnarExec} import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometShuffleExchangeExec} import org.apache.spark.sql.execution.{ColumnarToRowExec, RowToColumnarExec, SparkPlan} import org.apache.spark.sql.execution.adaptive.QueryStageExec @@ -155,9 +155,6 @@ case class EliminateRedundantTransitions(session: SparkSession) extends Rule[Spa * with such scans because the buffers may be modified after C2R reads them. * * This includes: - * - CometScanExec with native_comet scan implementation (V1 path) - uses BatchReader - * - CometScanExec with native_iceberg_compat and partition columns - uses - * ConstantColumnReader * - CometBatchScanExec with CometParquetScan (V2 Parquet path) - uses BatchReader */ private def hasScanUsingMutableBuffers(op: SparkPlan): Boolean = { @@ -166,10 +163,6 @@ case class EliminateRedundantTransitions(session: SparkSession) extends Rule[Spa case c: ReusedExchangeExec => hasScanUsingMutableBuffers(c.child) case _ => op.exists { - case scan: CometScanExec => - scan.scanImpl == CometConf.SCAN_NATIVE_COMET || - (scan.scanImpl == CometConf.SCAN_NATIVE_ICEBERG_COMPAT && - scan.relation.partitionSchema.nonEmpty) case scan: CometBatchScanExec => scan.scan.isInstanceOf[CometParquetScan] case _ => false } 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 e25d7fb4eb..60c0dbdc0c 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -127,6 +127,7 @@ object QueryPlanSerde extends Logging with CometExprShim { classOf[MapEntries] -> CometMapEntries, classOf[MapValues] -> CometMapValues, classOf[MapFromArrays] -> CometMapFromArrays, + classOf[MapContainsKey] -> CometMapContainsKey, classOf[MapFromEntries] -> CometMapFromEntries) private val structExpressions: Map[Class[_ <: Expression], CometExpressionSerde[_]] = Map( @@ -134,7 +135,8 @@ object QueryPlanSerde extends Logging with CometExprShim { classOf[GetArrayStructFields] -> CometGetArrayStructFields, classOf[GetStructField] -> CometGetStructField, classOf[JsonToStructs] -> CometJsonToStructs, - classOf[StructsToJson] -> CometStructsToJson) + classOf[StructsToJson] -> CometStructsToJson, + classOf[StructsToCsv] -> CometStructsToCsv) private val hashExpressions: Map[Class[_ <: Expression], CometExpressionSerde[_]] = Map( classOf[Md5] -> CometScalarFunction("md5"), @@ -172,6 +174,7 @@ object QueryPlanSerde extends Logging with CometExprShim { classOf[StringTrimLeft] -> CometScalarFunction("ltrim"), classOf[StringTrimRight] -> CometScalarFunction("rtrim"), classOf[Left] -> CometLeft, + classOf[Right] -> CometRight, classOf[Substring] -> CometSubstring, classOf[Upper] -> CometUpper) @@ -194,6 +197,7 @@ object QueryPlanSerde extends Logging with CometExprShim { classOf[FromUnixTime] -> CometFromUnixTime, classOf[LastDay] -> CometLastDay, classOf[Hour] -> CometHour, + classOf[MakeDate] -> CometMakeDate, classOf[Minute] -> CometMinute, classOf[Second] -> CometSecond, classOf[TruncDate] -> CometTruncDate, diff --git a/spark/src/main/scala/org/apache/comet/serde/datetime.scala b/spark/src/main/scala/org/apache/comet/serde/datetime.scala index a623146916..c2ddb0317d 100644 --- a/spark/src/main/scala/org/apache/comet/serde/datetime.scala +++ b/spark/src/main/scala/org/apache/comet/serde/datetime.scala @@ -21,7 +21,7 @@ package org.apache.comet.serde import java.util.Locale -import org.apache.spark.sql.catalyst.expressions.{Attribute, DateAdd, DateDiff, DateFormatClass, DateSub, DayOfMonth, DayOfWeek, DayOfYear, GetDateField, Hour, LastDay, Literal, Minute, Month, Quarter, Second, TruncDate, TruncTimestamp, UnixDate, UnixTimestamp, WeekDay, WeekOfYear, Year} +import org.apache.spark.sql.catalyst.expressions.{Attribute, DateAdd, DateDiff, DateFormatClass, DateSub, DayOfMonth, DayOfWeek, DayOfYear, GetDateField, Hour, LastDay, Literal, MakeDate, Minute, Month, Quarter, Second, TruncDate, TruncTimestamp, UnixDate, UnixTimestamp, WeekDay, WeekOfYear, Year} import org.apache.spark.sql.types.{DateType, IntegerType, StringType, TimestampType} import org.apache.spark.unsafe.types.UTF8String @@ -310,6 +310,8 @@ object CometDateAdd extends CometScalarFunction[DateAdd]("date_add") object CometDateSub extends CometScalarFunction[DateSub]("date_sub") +object CometMakeDate extends CometScalarFunction[MakeDate]("make_date") + object CometLastDay extends CometScalarFunction[LastDay]("last_day") object CometDateDiff extends CometScalarFunction[DateDiff]("date_diff") diff --git a/spark/src/main/scala/org/apache/comet/serde/maps.scala b/spark/src/main/scala/org/apache/comet/serde/maps.scala index 78b2180756..34e76215f3 100644 --- a/spark/src/main/scala/org/apache/comet/serde/maps.scala +++ b/spark/src/main/scala/org/apache/comet/serde/maps.scala @@ -90,6 +90,23 @@ object CometMapFromArrays extends CometExpressionSerde[MapFromArrays] { } } +object CometMapContainsKey extends CometExpressionSerde[MapContainsKey] { + + override def convert( + expr: MapContainsKey, + inputs: Seq[Attribute], + binding: Boolean): Option[ExprOuterClass.Expr] = { + // Replace with array_has(map_keys(map), key) + val mapExpr = exprToProtoInternal(expr.left, inputs, binding) + val keyExpr = exprToProtoInternal(expr.right, inputs, binding) + + val mapKeysExpr = scalarFunctionExprToProto("map_keys", mapExpr) + + val mapContainsKeyExpr = scalarFunctionExprToProto("array_has", mapKeysExpr, keyExpr) + optExprWithInfo(mapContainsKeyExpr, expr, expr.children: _*) + } +} + object CometMapFromEntries extends CometScalarFunction[MapFromEntries]("map_from_entries") { val keyUnsupportedReason = "Using BinaryType as Map keys is not allowed in map_from_entries" val valueUnsupportedReason = "Using BinaryType as Map values is not allowed in map_from_entries" diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometIcebergNativeScan.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometIcebergNativeScan.scala index 0ad82af8f8..957f621032 100644 --- a/spark/src/main/scala/org/apache/comet/serde/operator/CometIcebergNativeScan.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometIcebergNativeScan.scala @@ -28,10 +28,11 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.comet.{CometBatchScanExec, CometNativeExec} +import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceRDD, DataSourceRDDPartition} import org.apache.spark.sql.types._ import org.apache.comet.ConfigEntry -import org.apache.comet.iceberg.IcebergReflection +import org.apache.comet.iceberg.{CometIcebergNativeScanMetadata, IcebergReflection} import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} import org.apache.comet.serde.ExprOuterClass.Expr import org.apache.comet.serde.OperatorOuterClass.{Operator, SparkStructField} @@ -309,7 +310,7 @@ object CometIcebergNativeScan extends CometOperatorSerde[CometBatchScanExec] wit contentScanTaskClass: Class[_], fileScanTaskClass: Class[_], taskBuilder: OperatorOuterClass.IcebergFileScanTask.Builder, - icebergScanBuilder: OperatorOuterClass.IcebergScan.Builder, + commonBuilder: OperatorOuterClass.IcebergScanCommon.Builder, partitionTypeToPoolIndex: mutable.HashMap[String, Int], partitionSpecToPoolIndex: mutable.HashMap[String, Int], partitionDataToPoolIndex: mutable.HashMap[String, Int]): Unit = { @@ -334,7 +335,7 @@ object CometIcebergNativeScan extends CometOperatorSerde[CometBatchScanExec] wit val specIdx = partitionSpecToPoolIndex.getOrElseUpdate( partitionSpecJson, { val idx = partitionSpecToPoolIndex.size - icebergScanBuilder.addPartitionSpecPool(partitionSpecJson) + commonBuilder.addPartitionSpecPool(partitionSpecJson) idx }) taskBuilder.setPartitionSpecIdx(specIdx) @@ -415,7 +416,7 @@ object CometIcebergNativeScan extends CometOperatorSerde[CometBatchScanExec] wit val typeIdx = partitionTypeToPoolIndex.getOrElseUpdate( partitionTypeJson, { val idx = partitionTypeToPoolIndex.size - icebergScanBuilder.addPartitionTypePool(partitionTypeJson) + commonBuilder.addPartitionTypePool(partitionTypeJson) idx }) taskBuilder.setPartitionTypeIdx(typeIdx) @@ -470,7 +471,7 @@ object CometIcebergNativeScan extends CometOperatorSerde[CometBatchScanExec] wit val partitionDataIdx = partitionDataToPoolIndex.getOrElseUpdate( partitionDataKey, { val idx = partitionDataToPoolIndex.size - icebergScanBuilder.addPartitionDataPool(partitionDataProto) + commonBuilder.addPartitionDataPool(partitionDataProto) idx }) taskBuilder.setPartitionDataIdx(partitionDataIdx) @@ -671,17 +672,59 @@ object CometIcebergNativeScan extends CometOperatorSerde[CometBatchScanExec] wit } /** - * Serializes a CometBatchScanExec wrapping an Iceberg SparkBatchQueryScan to protobuf. + * Converts a CometBatchScanExec to a minimal placeholder IcebergScan operator. * - * Uses pre-extracted metadata from CometScanRule to avoid redundant reflection operations. All - * reflection and validation was done during planning, so serialization failures here would - * indicate a programming error rather than an expected fallback condition. + * Returns a placeholder operator with only metadata_location for matching during partition + * injection. All other fields (catalog properties, required schema, pools, partition data) are + * set by serializePartitions() at execution time after DPP resolves. */ override def convert( scan: CometBatchScanExec, builder: Operator.Builder, childOp: Operator*): Option[OperatorOuterClass.Operator] = { + + val metadata = scan.nativeIcebergScanMetadata.getOrElse { + throw new IllegalStateException( + "Programming error: CometBatchScanExec.nativeIcebergScanMetadata is None. " + + "Metadata should have been extracted in CometScanRule.") + } + val icebergScanBuilder = OperatorOuterClass.IcebergScan.newBuilder() + val commonBuilder = OperatorOuterClass.IcebergScanCommon.newBuilder() + + // Only set metadata_location - used for matching in PlanDataInjector. + // All other fields (catalog_properties, required_schema, pools) are set by + // serializePartitions() at execution time, so setting them here would be wasted work. + commonBuilder.setMetadataLocation(metadata.metadataLocation) + + icebergScanBuilder.setCommon(commonBuilder.build()) + // partition field intentionally empty - will be populated at execution time + + builder.clearChildren() + Some(builder.setIcebergScan(icebergScanBuilder).build()) + } + + /** + * Serializes partitions from inputRDD at execution time. + * + * Called after doPrepare() has resolved DPP subqueries. Builds pools and per-partition data in + * one pass from the DPP-filtered partitions. + * + * @param scanExec + * The BatchScanExec whose inputRDD contains the DPP-filtered partitions + * @param output + * The output attributes for the scan + * @param metadata + * Pre-extracted Iceberg metadata from CometScanRule + * @return + * Tuple of (commonBytes, perPartitionBytes) for native execution + */ + def serializePartitions( + scanExec: BatchScanExec, + output: Seq[Attribute], + metadata: CometIcebergNativeScanMetadata): (Array[Byte], Array[Array[Byte]]) = { + + val commonBuilder = OperatorOuterClass.IcebergScanCommon.newBuilder() // Deduplication structures - map unique values to pool indices val schemaToPoolIndex = mutable.HashMap[AnyRef, Int]() @@ -689,300 +732,225 @@ object CometIcebergNativeScan extends CometOperatorSerde[CometBatchScanExec] wit val partitionSpecToPoolIndex = mutable.HashMap[String, Int]() val nameMappingToPoolIndex = mutable.HashMap[String, Int]() val projectFieldIdsToPoolIndex = mutable.HashMap[Seq[Int], Int]() - val partitionDataToPoolIndex = mutable.HashMap[String, Int]() // Base64 bytes -> pool index + val partitionDataToPoolIndex = mutable.HashMap[String, Int]() val deleteFilesToPoolIndex = mutable.HashMap[Seq[OperatorOuterClass.IcebergDeleteFile], Int]() val residualToPoolIndex = mutable.HashMap[Option[Expr], Int]() - var totalTasks = 0 + val perPartitionBuilders = mutable.ArrayBuffer[OperatorOuterClass.IcebergScan]() - // Get pre-extracted metadata from planning phase - // If metadata is None, this is a programming error - metadata should have been extracted - // in CometScanRule before creating CometBatchScanExec - val metadata = scan.nativeIcebergScanMetadata.getOrElse { - throw new IllegalStateException( - "Programming error: CometBatchScanExec.nativeIcebergScanMetadata is None. " + - "Metadata should have been extracted in CometScanRule.") - } - - // Use pre-extracted metadata (no reflection needed) - icebergScanBuilder.setMetadataLocation(metadata.metadataLocation) + var totalTasks = 0 + commonBuilder.setMetadataLocation(metadata.metadataLocation) metadata.catalogProperties.foreach { case (key, value) => - icebergScanBuilder.putCatalogProperties(key, value) + commonBuilder.putCatalogProperties(key, value) } - // Set required_schema from output - scan.output.foreach { attr => + output.foreach { attr => val field = SparkStructField .newBuilder() .setName(attr.name) .setNullable(attr.nullable) serializeDataType(attr.dataType).foreach(field.setDataType) - icebergScanBuilder.addRequiredSchema(field.build()) + commonBuilder.addRequiredSchema(field.build()) } - // Extract FileScanTasks from the InputPartitions in the RDD - try { - scan.wrapped.inputRDD match { - case rdd: org.apache.spark.sql.execution.datasources.v2.DataSourceRDD => - val partitions = rdd.partitions - partitions.foreach { partition => - val partitionBuilder = OperatorOuterClass.IcebergFilePartition.newBuilder() + // Load Iceberg classes once (avoid repeated class loading in loop) + // scalastyle:off classforname + val contentScanTaskClass = Class.forName(IcebergReflection.ClassNames.CONTENT_SCAN_TASK) + val fileScanTaskClass = Class.forName(IcebergReflection.ClassNames.FILE_SCAN_TASK) + val contentFileClass = Class.forName(IcebergReflection.ClassNames.CONTENT_FILE) + val schemaParserClass = Class.forName(IcebergReflection.ClassNames.SCHEMA_PARSER) + val schemaClass = Class.forName(IcebergReflection.ClassNames.SCHEMA) + // scalastyle:on classforname - val inputPartitions = partition - .asInstanceOf[org.apache.spark.sql.execution.datasources.v2.DataSourceRDDPartition] - .inputPartitions + // Cache method lookups (avoid repeated getMethod in loop) + val fileMethod = contentScanTaskClass.getMethod("file") + val startMethod = contentScanTaskClass.getMethod("start") + val lengthMethod = contentScanTaskClass.getMethod("length") + val residualMethod = contentScanTaskClass.getMethod("residual") + val taskSchemaMethod = fileScanTaskClass.getMethod("schema") + val toJsonMethod = schemaParserClass.getMethod("toJson", schemaClass) + toJsonMethod.setAccessible(true) + + // Access inputRDD - safe now, DPP is resolved + scanExec.inputRDD match { + case rdd: DataSourceRDD => + val partitions = rdd.partitions + partitions.foreach { partition => + val partitionBuilder = OperatorOuterClass.IcebergScan.newBuilder() + + val inputPartitions = partition + .asInstanceOf[DataSourceRDDPartition] + .inputPartitions + + inputPartitions.foreach { inputPartition => + val inputPartClass = inputPartition.getClass - inputPartitions.foreach { inputPartition => - val inputPartClass = inputPartition.getClass + try { + val taskGroupMethod = inputPartClass.getDeclaredMethod("taskGroup") + taskGroupMethod.setAccessible(true) + val taskGroup = taskGroupMethod.invoke(inputPartition) - try { - val taskGroupMethod = inputPartClass.getDeclaredMethod("taskGroup") - taskGroupMethod.setAccessible(true) - val taskGroup = taskGroupMethod.invoke(inputPartition) + val taskGroupClass = taskGroup.getClass + val tasksMethod = taskGroupClass.getMethod("tasks") + val tasksCollection = + tasksMethod.invoke(taskGroup).asInstanceOf[java.util.Collection[_]] - val taskGroupClass = taskGroup.getClass - val tasksMethod = taskGroupClass.getMethod("tasks") - val tasksCollection = - tasksMethod.invoke(taskGroup).asInstanceOf[java.util.Collection[_]] + tasksCollection.asScala.foreach { task => + totalTasks += 1 - tasksCollection.asScala.foreach { task => - totalTasks += 1 + val taskBuilder = OperatorOuterClass.IcebergFileScanTask.newBuilder() - try { - val taskBuilder = OperatorOuterClass.IcebergFileScanTask.newBuilder() - - // scalastyle:off classforname - val contentScanTaskClass = - Class.forName(IcebergReflection.ClassNames.CONTENT_SCAN_TASK) - val fileScanTaskClass = - Class.forName(IcebergReflection.ClassNames.FILE_SCAN_TASK) - val contentFileClass = - Class.forName(IcebergReflection.ClassNames.CONTENT_FILE) - // scalastyle:on classforname - - val fileMethod = contentScanTaskClass.getMethod("file") - val dataFile = fileMethod.invoke(task) - - val filePathOpt = - IcebergReflection.extractFileLocation(contentFileClass, dataFile) - - filePathOpt match { - case Some(filePath) => - taskBuilder.setDataFilePath(filePath) - case None => - val msg = - "Iceberg reflection failure: Cannot extract file path from data file" - logError(msg) - throw new RuntimeException(msg) - } + val dataFile = fileMethod.invoke(task) - val startMethod = contentScanTaskClass.getMethod("start") - val start = startMethod.invoke(task).asInstanceOf[Long] - taskBuilder.setStart(start) - - val lengthMethod = contentScanTaskClass.getMethod("length") - val length = lengthMethod.invoke(task).asInstanceOf[Long] - taskBuilder.setLength(length) - - try { - // Equality deletes require the full table schema to resolve field IDs, - // even for columns not in the projection. Schema evolution requires - // using the snapshot's schema to correctly read old data files. - // These requirements conflict, so we choose based on delete presence. - - val taskSchemaMethod = fileScanTaskClass.getMethod("schema") - val taskSchema = taskSchemaMethod.invoke(task) - - val deletes = - IcebergReflection.getDeleteFilesFromTask(task, fileScanTaskClass) - val hasDeletes = !deletes.isEmpty - - // Schema to pass to iceberg-rust's FileScanTask. - // This is used by RecordBatchTransformer for field type lookups (e.g., in - // constants_map) and default value generation. The actual projection is - // controlled by project_field_ids. - // - // Schema selection logic: - // 1. If hasDeletes=true: Use taskSchema (file-specific schema) because - // delete files reference specific schema versions and we need exact schema - // matching for MOR. - // 2. Else if scanSchema contains columns not in tableSchema: Use scanSchema - // because this is a VERSION AS OF query reading a historical snapshot with - // different schema (e.g., after column drop, scanSchema has old columns - // that tableSchema doesn't) - // 3. Else: Use tableSchema because scanSchema is the query OUTPUT schema - // (e.g., for aggregates like "SELECT count(*)", scanSchema only has - // aggregate fields and doesn't contain partition columns needed by - // constants_map) - val schema: AnyRef = - if (hasDeletes) { - taskSchema - } else { - // Check if scanSchema has columns that tableSchema doesn't have - // (VERSION AS OF case) - val scanSchemaFieldIds = IcebergReflection - .buildFieldIdMapping(metadata.scanSchema) - .values - .toSet - val tableSchemaFieldIds = IcebergReflection - .buildFieldIdMapping(metadata.tableSchema) - .values - .toSet - val hasHistoricalColumns = - scanSchemaFieldIds.exists(id => !tableSchemaFieldIds.contains(id)) - - if (hasHistoricalColumns) { - // VERSION AS OF: scanSchema has columns that current table doesn't have - metadata.scanSchema.asInstanceOf[AnyRef] - } else { - // Regular query: use tableSchema for partition field lookups - metadata.tableSchema.asInstanceOf[AnyRef] - } - } - - // scalastyle:off classforname - val schemaParserClass = - Class.forName(IcebergReflection.ClassNames.SCHEMA_PARSER) - val schemaClass = Class.forName(IcebergReflection.ClassNames.SCHEMA) - // scalastyle:on classforname - val toJsonMethod = schemaParserClass.getMethod("toJson", schemaClass) - toJsonMethod.setAccessible(true) - - // Use object identity for deduplication: Iceberg Schema objects are immutable - // and reused across tasks, making identity-based deduplication safe - val schemaIdx = schemaToPoolIndex.getOrElseUpdate( - schema, { - val idx = schemaToPoolIndex.size - val schemaJson = toJsonMethod.invoke(null, schema).asInstanceOf[String] - icebergScanBuilder.addSchemaPool(schemaJson) - idx - }) - taskBuilder.setSchemaIdx(schemaIdx) - - // Build field ID mapping from the schema we're using - val nameToFieldId = IcebergReflection.buildFieldIdMapping(schema) - - // Extract project_field_ids for scan.output columns. - // For schema evolution: try task schema first, then fall back to - // global scan schema (pre-extracted in metadata). - val projectFieldIds = scan.output.flatMap { attr => - nameToFieldId - .get(attr.name) - .orElse(metadata.globalFieldIdMapping.get(attr.name)) - .orElse { - logWarning( - s"Column '${attr.name}' not found in task or scan schema," + - "skipping projection") - None - } - } - - // Deduplicate project field IDs - val projectFieldIdsIdx = projectFieldIdsToPoolIndex.getOrElseUpdate( - projectFieldIds, { - val idx = projectFieldIdsToPoolIndex.size - val listBuilder = OperatorOuterClass.ProjectFieldIdList.newBuilder() - projectFieldIds.foreach(id => listBuilder.addFieldIds(id)) - icebergScanBuilder.addProjectFieldIdsPool(listBuilder.build()) - idx - }) - taskBuilder.setProjectFieldIdsIdx(projectFieldIdsIdx) - } catch { - case e: Exception => - val msg = - "Iceberg reflection failure: " + - "Failed to extract schema from FileScanTask: " + - s"${e.getMessage}" - logError(msg) - throw new RuntimeException(msg, e) - } + val filePathOpt = + IcebergReflection.extractFileLocation(contentFileClass, dataFile) - // Deduplicate delete files - val deleteFilesList = - extractDeleteFilesList(task, contentFileClass, fileScanTaskClass) - if (deleteFilesList.nonEmpty) { - val deleteFilesIdx = deleteFilesToPoolIndex.getOrElseUpdate( - deleteFilesList, { - val idx = deleteFilesToPoolIndex.size - val listBuilder = OperatorOuterClass.DeleteFileList.newBuilder() - deleteFilesList.foreach(df => listBuilder.addDeleteFiles(df)) - icebergScanBuilder.addDeleteFilesPool(listBuilder.build()) - idx - }) - taskBuilder.setDeleteFilesIdx(deleteFilesIdx) - } + filePathOpt match { + case Some(filePath) => + taskBuilder.setDataFilePath(filePath) + case None => + val msg = + "Iceberg reflection failure: Cannot extract file path from data file" + logError(msg) + throw new RuntimeException(msg) + } - // Extract and deduplicate residual expression - val residualExprOpt = - try { - val residualMethod = contentScanTaskClass.getMethod("residual") - val residualExpr = residualMethod.invoke(task) - - val catalystExpr = convertIcebergExpression(residualExpr, scan.output) - - catalystExpr.flatMap { expr => - exprToProto(expr, scan.output, binding = false) - } - } catch { - case e: Exception => - logWarning( - "Failed to extract residual expression from FileScanTask: " + - s"${e.getMessage}") - None - } - - residualExprOpt.foreach { residualExpr => - val residualIdx = residualToPoolIndex.getOrElseUpdate( - Some(residualExpr), { - val idx = residualToPoolIndex.size - icebergScanBuilder.addResidualPool(residualExpr) - idx - }) - taskBuilder.setResidualIdx(residualIdx) + val start = startMethod.invoke(task).asInstanceOf[Long] + taskBuilder.setStart(start) + + val length = lengthMethod.invoke(task).asInstanceOf[Long] + taskBuilder.setLength(length) + + val taskSchema = taskSchemaMethod.invoke(task) + + val deletes = + IcebergReflection.getDeleteFilesFromTask(task, fileScanTaskClass) + val hasDeletes = !deletes.isEmpty + + val schema: AnyRef = + if (hasDeletes) { + taskSchema + } else { + val scanSchemaFieldIds = IcebergReflection + .buildFieldIdMapping(metadata.scanSchema) + .values + .toSet + val tableSchemaFieldIds = IcebergReflection + .buildFieldIdMapping(metadata.tableSchema) + .values + .toSet + val hasHistoricalColumns = + scanSchemaFieldIds.exists(id => !tableSchemaFieldIds.contains(id)) + + if (hasHistoricalColumns) { + metadata.scanSchema.asInstanceOf[AnyRef] + } else { + metadata.tableSchema.asInstanceOf[AnyRef] } + } - // Serialize partition spec and data (field definitions, transforms, values) - serializePartitionData( - task, - contentScanTaskClass, - fileScanTaskClass, - taskBuilder, - icebergScanBuilder, - partitionTypeToPoolIndex, - partitionSpecToPoolIndex, - partitionDataToPoolIndex) - - // Deduplicate name mapping - metadata.nameMapping.foreach { nm => - val nmIdx = nameMappingToPoolIndex.getOrElseUpdate( - nm, { - val idx = nameMappingToPoolIndex.size - icebergScanBuilder.addNameMappingPool(nm) - idx - }) - taskBuilder.setNameMappingIdx(nmIdx) + val schemaIdx = schemaToPoolIndex.getOrElseUpdate( + schema, { + val idx = schemaToPoolIndex.size + val schemaJson = toJsonMethod.invoke(null, schema).asInstanceOf[String] + commonBuilder.addSchemaPool(schemaJson) + idx + }) + taskBuilder.setSchemaIdx(schemaIdx) + + val nameToFieldId = IcebergReflection.buildFieldIdMapping(schema) + + val projectFieldIds = output.flatMap { attr => + nameToFieldId + .get(attr.name) + .orElse(metadata.globalFieldIdMapping.get(attr.name)) + .orElse { + logWarning(s"Column '${attr.name}' not found in task or scan schema, " + + "skipping projection") + None } + } - partitionBuilder.addFileScanTasks(taskBuilder.build()) + val projectFieldIdsIdx = projectFieldIdsToPoolIndex.getOrElseUpdate( + projectFieldIds, { + val idx = projectFieldIdsToPoolIndex.size + val listBuilder = OperatorOuterClass.ProjectFieldIdList.newBuilder() + projectFieldIds.foreach(id => listBuilder.addFieldIds(id)) + commonBuilder.addProjectFieldIdsPool(listBuilder.build()) + idx + }) + taskBuilder.setProjectFieldIdsIdx(projectFieldIdsIdx) + + val deleteFilesList = + extractDeleteFilesList(task, contentFileClass, fileScanTaskClass) + if (deleteFilesList.nonEmpty) { + val deleteFilesIdx = deleteFilesToPoolIndex.getOrElseUpdate( + deleteFilesList, { + val idx = deleteFilesToPoolIndex.size + val listBuilder = OperatorOuterClass.DeleteFileList.newBuilder() + deleteFilesList.foreach(df => listBuilder.addDeleteFiles(df)) + commonBuilder.addDeleteFilesPool(listBuilder.build()) + idx + }) + taskBuilder.setDeleteFilesIdx(deleteFilesIdx) + } + + val residualExprOpt = + try { + val residualExpr = residualMethod.invoke(task) + val catalystExpr = convertIcebergExpression(residualExpr, output) + catalystExpr.flatMap { expr => + exprToProto(expr, output, binding = false) + } + } catch { + case e: Exception => + logWarning( + "Failed to extract residual expression from FileScanTask: " + + s"${e.getMessage}") + None } + + residualExprOpt.foreach { residualExpr => + val residualIdx = residualToPoolIndex.getOrElseUpdate( + Some(residualExpr), { + val idx = residualToPoolIndex.size + commonBuilder.addResidualPool(residualExpr) + idx + }) + taskBuilder.setResidualIdx(residualIdx) + } + + serializePartitionData( + task, + contentScanTaskClass, + fileScanTaskClass, + taskBuilder, + commonBuilder, + partitionTypeToPoolIndex, + partitionSpecToPoolIndex, + partitionDataToPoolIndex) + + metadata.nameMapping.foreach { nm => + val nmIdx = nameMappingToPoolIndex.getOrElseUpdate( + nm, { + val idx = nameMappingToPoolIndex.size + commonBuilder.addNameMappingPool(nm) + idx + }) + taskBuilder.setNameMappingIdx(nmIdx) } + + partitionBuilder.addFileScanTasks(taskBuilder.build()) } } - - val builtPartition = partitionBuilder.build() - icebergScanBuilder.addFilePartitions(builtPartition) } - case _ => - } - } catch { - case e: Exception => - // CometScanRule already validated this scan should use native execution. - // Failure here is a programming error, not a graceful fallback scenario. - throw new IllegalStateException( - s"Native Iceberg scan serialization failed unexpectedly: ${e.getMessage}", - e) + + perPartitionBuilders += partitionBuilder.build() + } + case _ => + throw new IllegalStateException("Expected DataSourceRDD from BatchScanExec") } // Log deduplication summary @@ -999,7 +967,6 @@ object CometIcebergNativeScan extends CometOperatorSerde[CometBatchScanExec] wit val avgDedup = if (totalTasks == 0) { "0.0" } else { - // Filter out empty pools - they shouldn't count as 100% dedup val nonEmptyPools = allPoolSizes.filter(_ > 0) if (nonEmptyPools.isEmpty) { "0.0" @@ -1009,8 +976,7 @@ object CometIcebergNativeScan extends CometOperatorSerde[CometBatchScanExec] wit } } - // Calculate partition data pool size in bytes (protobuf format) - val partitionDataPoolBytes = icebergScanBuilder.getPartitionDataPoolList.asScala + val partitionDataPoolBytes = commonBuilder.getPartitionDataPoolList.asScala .map(_.getSerializedSize) .sum @@ -1021,8 +987,10 @@ object CometIcebergNativeScan extends CometOperatorSerde[CometBatchScanExec] wit s"$partitionDataPoolBytes bytes (protobuf)") } - builder.clearChildren() - Some(builder.setIcebergScan(icebergScanBuilder).build()) + val commonBytes = commonBuilder.build().toByteArray + val perPartitionBytes = perPartitionBuilders.map(_.toByteArray).toArray + + (commonBytes, perPartitionBytes) } override def createExec(nativeOp: Operator, op: CometBatchScanExec): CometNativeExec = { @@ -1035,10 +1003,11 @@ object CometIcebergNativeScan extends CometOperatorSerde[CometBatchScanExec] wit "Metadata should have been extracted in CometScanRule.") } - // Extract metadataLocation from the native operator - val metadataLocation = nativeOp.getIcebergScan.getMetadataLocation + // Extract metadataLocation from the native operator's common data + val metadataLocation = nativeOp.getIcebergScan.getCommon.getMetadataLocation - // Create the CometIcebergNativeScanExec using the companion object's apply method + // Pass BatchScanExec reference for deferred serialization (DPP support) + // Serialization happens at execution time after doPrepare() resolves DPP subqueries CometIcebergNativeScanExec(nativeOp, op.wrapped, op.session, metadataLocation, metadata) } } diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometSink.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometSink.scala index ca9dbdad7c..4f45f1ce00 100644 --- a/spark/src/main/scala/org/apache/comet/serde/operator/CometSink.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometSink.scala @@ -37,7 +37,7 @@ import org.apache.comet.serde.QueryPlanSerde.{serializeDataType, supportedDataTy abstract class CometSink[T <: SparkPlan] extends CometOperatorSerde[T] { /** Whether the data produced by the Comet operator is FFI safe */ - def isFfiSafe: Boolean = false + def isFfiSafe(op: T): Boolean = false override def enabledConfig: Option[ConfigEntry[Boolean]] = None @@ -61,7 +61,7 @@ abstract class CometSink[T <: SparkPlan] extends CometOperatorSerde[T] { } else { scanBuilder.setSource(source) } - scanBuilder.setArrowFfiSafe(isFfiSafe) + scanBuilder.setArrowFfiSafe(isFfiSafe(op)) val scanTypes = op.output.flatten { attr => serializeDataType(attr.dataType) @@ -93,7 +93,7 @@ object CometExchangeSink extends CometSink[SparkPlan] { * * Source of shuffle exchange batches is NativeBatchDecoderIterator. */ - override def isFfiSafe: Boolean = true + override def isFfiSafe(op: SparkPlan): Boolean = true override def createExec(nativeOp: Operator, op: SparkPlan): CometNativeExec = CometSinkPlaceHolder(nativeOp, op, op) diff --git a/spark/src/main/scala/org/apache/comet/serde/strings.scala b/spark/src/main/scala/org/apache/comet/serde/strings.scala index ea42b245aa..5bf15fd6a3 100644 --- a/spark/src/main/scala/org/apache/comet/serde/strings.scala +++ b/spark/src/main/scala/org/apache/comet/serde/strings.scala @@ -21,8 +21,9 @@ package org.apache.comet.serde import java.util.Locale -import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Concat, Expression, InitCap, Left, Length, Like, Literal, Lower, RegExpReplace, RLike, StringLPad, StringRepeat, StringRPad, Substring, Upper} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Concat, Expression, If, InitCap, IsNull, Left, Length, Like, Literal, Lower, RegExpReplace, Right, RLike, StringLPad, StringRepeat, StringRPad, Substring, Upper} import org.apache.spark.sql.types.{BinaryType, DataTypes, LongType, StringType} +import org.apache.spark.unsafe.types.UTF8String import org.apache.comet.CometConf import org.apache.comet.CometSparkSessionExtensions.withInfo @@ -143,6 +144,49 @@ object CometLeft extends CometExpressionSerde[Left] { } } +object CometRight extends CometExpressionSerde[Right] { + + override def convert(expr: Right, inputs: Seq[Attribute], binding: Boolean): Option[Expr] = { + expr.len match { + case Literal(lenValue, _) => + val lenInt = lenValue.asInstanceOf[Int] + if (lenInt <= 0) { + // Match Spark's behavior: If(IsNull(str), NULL, "") + // This ensures NULL propagation: RIGHT(NULL, 0) -> NULL, RIGHT("hello", 0) -> "" + val isNullExpr = IsNull(expr.str) + val nullLiteral = Literal.create(null, StringType) + val emptyStringLiteral = Literal(UTF8String.EMPTY_UTF8, StringType) + val ifExpr = If(isNullExpr, nullLiteral, emptyStringLiteral) + + // Serialize the If expression using existing infrastructure + exprToProtoInternal(ifExpr, inputs, binding) + } else { + exprToProtoInternal(expr.str, inputs, binding) match { + case Some(strExpr) => + val builder = ExprOuterClass.Substring.newBuilder() + builder.setChild(strExpr) + builder.setStart(-lenInt) + builder.setLen(lenInt) + Some(ExprOuterClass.Expr.newBuilder().setSubstring(builder).build()) + case None => + withInfo(expr, expr.str) + None + } + } + case _ => + withInfo(expr, "RIGHT len must be a literal") + None + } + } + + override def getSupportLevel(expr: Right): SupportLevel = { + expr.str.dataType match { + case _: StringType => Compatible() + case _ => Unsupported(Some(s"RIGHT does not support ${expr.str.dataType}")) + } + } +} + object CometConcat extends CometScalarFunction[Concat]("concat") { val unsupportedReason = "CONCAT supports only string input parameters" diff --git a/spark/src/main/scala/org/apache/comet/serde/structs.scala b/spark/src/main/scala/org/apache/comet/serde/structs.scala index b76c64bac9..d9d83f6594 100644 --- a/spark/src/main/scala/org/apache/comet/serde/structs.scala +++ b/spark/src/main/scala/org/apache/comet/serde/structs.scala @@ -20,11 +20,13 @@ package org.apache.comet.serde import scala.jdk.CollectionConverters._ +import scala.util.Try -import org.apache.spark.sql.catalyst.expressions.{Attribute, CreateNamedStruct, GetArrayStructFields, GetStructField, JsonToStructs, StructsToJson} -import org.apache.spark.sql.types.{ArrayType, DataType, DataTypes, MapType, StructType} +import org.apache.spark.sql.catalyst.expressions.{Attribute, CreateNamedStruct, GetArrayStructFields, GetStructField, JsonToStructs, StructsToCsv, StructsToJson} +import org.apache.spark.sql.types._ import org.apache.comet.CometSparkSessionExtensions.withInfo +import org.apache.comet.DataTypeSupport import org.apache.comet.serde.QueryPlanSerde.{exprToProtoInternal, serializeDataType} object CometCreateNamedStruct extends CometExpressionSerde[CreateNamedStruct] { @@ -230,3 +232,69 @@ object CometJsonToStructs extends CometExpressionSerde[JsonToStructs] { } } } + +object CometStructsToCsv extends CometExpressionSerde[StructsToCsv] { + + private val incompatibleDataTypes = Seq(DateType, TimestampType, TimestampNTZType, BinaryType) + + override def getSupportLevel(expr: StructsToCsv): SupportLevel = { + val dataTypes = expr.inputSchema.fields.map(_.dataType) + val containsComplexType = dataTypes.exists(DataTypeSupport.isComplexType) + if (containsComplexType) { + return Unsupported( + Some( + s"The schema ${expr.inputSchema} is not supported because it includes a complex type")) + } + val containsIncompatibleDataTypes = dataTypes.exists(incompatibleDataTypes.contains) + if (containsIncompatibleDataTypes) { + return Incompatible( + Some( + s"The schema ${expr.inputSchema} is not supported because " + + s"it includes a incompatible data types: $incompatibleDataTypes")) + } + // https://github.com/apache/datafusion-comet/issues/3232 + Incompatible() + } + + override def convert( + expr: StructsToCsv, + inputs: Seq[Attribute], + binding: Boolean): Option[ExprOuterClass.Expr] = { + for { + childProto <- exprToProtoInternal(expr.child, inputs, binding) + } yield { + val optionsProto = options2Proto(expr.options, expr.timeZoneId) + val toCsv = ExprOuterClass.ToCsv + .newBuilder() + .setChild(childProto) + .setOptions(optionsProto) + .build() + ExprOuterClass.Expr.newBuilder().setToCsv(toCsv).build() + } + } + + private def options2Proto( + options: Map[String, String], + timeZoneId: Option[String]): ExprOuterClass.CsvWriteOptions = { + ExprOuterClass.CsvWriteOptions + .newBuilder() + .setDelimiter(options.getOrElse("delimiter", ",")) + .setQuote(options.getOrElse("quote", "\"")) + .setEscape(options.getOrElse("escape", "\\")) + .setNullValue(options.getOrElse("nullValue", "")) + .setTimezone(timeZoneId.getOrElse("UTC")) + .setIgnoreLeadingWhiteSpace(options + .get("ignoreLeadingWhiteSpace") + .flatMap(ignoreLeadingWhiteSpace => Try(ignoreLeadingWhiteSpace.toBoolean).toOption) + .getOrElse(true)) + .setIgnoreTrailingWhiteSpace(options + .get("ignoreTrailingWhiteSpace") + .flatMap(ignoreTrailingWhiteSpace => Try(ignoreTrailingWhiteSpace.toBoolean).toOption) + .getOrElse(true)) + .setQuoteAll(options + .get("quoteAll") + .flatMap(quoteAll => Try(quoteAll.toBoolean).toOption) + .getOrElse(false)) + .build() + } +} 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 f40e05ea0c..3bc5aaef4d 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 @@ -271,7 +271,7 @@ object CometBroadcastExchangeExec extends CometSink[BroadcastExchangeExec] { * * Source of broadcast exchange batches is ArrowStreamReader. */ - override def isFfiSafe: Boolean = true + override def isFfiSafe(op: BroadcastExchangeExec): Boolean = true override def enabledConfig: Option[ConfigEntry[Boolean]] = Some( CometConf.COMET_EXEC_BROADCAST_EXCHANGE_ENABLED) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometExecRDD.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometExecRDD.scala index 2fd7f12c24..ad0c4f2afe 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometExecRDD.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometExecRDD.scala @@ -19,39 +19,168 @@ package org.apache.spark.sql.comet -import org.apache.spark.{Partition, SparkContext, TaskContext} -import org.apache.spark.rdd.{RDD, RDDOperationScope} +import org.apache.spark._ +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.execution.ScalarSubquery import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.SerializableConfiguration + +import org.apache.comet.CometExecIterator +import org.apache.comet.serde.OperatorOuterClass + +/** + * Partition that carries per-partition planning data, avoiding closure capture of all partitions. + */ +private[spark] class CometExecPartition( + override val index: Int, + val inputPartitions: Array[Partition], + val planDataByKey: Map[String, Array[Byte]]) + extends Partition /** - * A RDD that executes Spark SQL query in Comet native execution to generate ColumnarBatch. + * Unified RDD for Comet native execution. + * + * Solves the closure capture problem: instead of capturing all partitions' data in the closure + * (which gets serialized to every task), each Partition object carries only its own data. + * + * Handles three cases: + * - With inputs + per-partition data: injects planning data into operator tree + * - With inputs + no per-partition data: just zips inputs (no injection overhead) + * - No inputs: uses numPartitions to create partitions + * + * NOTE: This RDD does not handle DPP (InSubqueryExec), which is resolved in + * CometIcebergNativeScanExec.serializedPartitionData before this RDD is created. It also handles + * ScalarSubquery expressions by registering them with CometScalarSubquery before execution. */ private[spark] class CometExecRDD( sc: SparkContext, - partitionNum: Int, - var f: (Seq[Iterator[ColumnarBatch]], Int, Int) => Iterator[ColumnarBatch]) - extends RDD[ColumnarBatch](sc, Nil) { + var inputRDDs: Seq[RDD[ColumnarBatch]], + commonByKey: Map[String, Array[Byte]], + @transient perPartitionByKey: Map[String, Array[Array[Byte]]], + serializedPlan: Array[Byte], + defaultNumPartitions: Int, + numOutputCols: Int, + nativeMetrics: CometMetricNode, + subqueries: Seq[ScalarSubquery], + broadcastedHadoopConfForEncryption: Option[Broadcast[SerializableConfiguration]] = None, + encryptedFilePaths: Seq[String] = Seq.empty) + extends RDD[ColumnarBatch](sc, inputRDDs.map(rdd => new OneToOneDependency(rdd))) { - override def compute(s: Partition, context: TaskContext): Iterator[ColumnarBatch] = { - f(Seq.empty, partitionNum, s.index) + // Determine partition count: from inputs if available, otherwise from parameter + private val numPartitions: Int = if (inputRDDs.nonEmpty) { + inputRDDs.head.partitions.length + } else if (perPartitionByKey.nonEmpty) { + perPartitionByKey.values.head.length + } else { + defaultNumPartitions } + // Validate all per-partition arrays have the same length to prevent + // ArrayIndexOutOfBoundsException in getPartitions (e.g., from broadcast scans with + // different partition counts after DPP filtering) + require( + perPartitionByKey.values.forall(_.length == numPartitions), + s"All per-partition arrays must have length $numPartitions, but found: " + + perPartitionByKey.map { case (key, arr) => s"$key -> ${arr.length}" }.mkString(", ")) + override protected def getPartitions: Array[Partition] = { - Array.tabulate(partitionNum)(i => - new Partition { - override def index: Int = i - }) + (0 until numPartitions).map { idx => + val inputParts = inputRDDs.map(_.partitions(idx)).toArray + val planData = perPartitionByKey.map { case (key, arr) => key -> arr(idx) } + new CometExecPartition(idx, inputParts, planData) + }.toArray + } + + override def compute(split: Partition, context: TaskContext): Iterator[ColumnarBatch] = { + val partition = split.asInstanceOf[CometExecPartition] + + val inputs = inputRDDs.zip(partition.inputPartitions).map { case (rdd, part) => + rdd.iterator(part, context) + } + + // Only inject if we have per-partition planning data + val actualPlan = if (commonByKey.nonEmpty) { + val basePlan = OperatorOuterClass.Operator.parseFrom(serializedPlan) + val injected = + PlanDataInjector.injectPlanData(basePlan, commonByKey, partition.planDataByKey) + PlanDataInjector.serializeOperator(injected) + } else { + serializedPlan + } + + val it = new CometExecIterator( + CometExec.newIterId, + inputs, + numOutputCols, + actualPlan, + nativeMetrics, + numPartitions, + partition.index, + broadcastedHadoopConfForEncryption, + encryptedFilePaths) + + // Register ScalarSubqueries so native code can look them up + subqueries.foreach(sub => CometScalarSubquery.setSubquery(it.id, sub)) + + Option(context).foreach { ctx => + ctx.addTaskCompletionListener[Unit] { _ => + it.close() + subqueries.foreach(sub => CometScalarSubquery.removeSubquery(it.id, sub)) + } + } + + it + } + + // Duplicates logic from Spark's ZippedPartitionsBaseRDD.getPreferredLocations + override def getPreferredLocations(split: Partition): Seq[String] = { + if (inputRDDs == null || inputRDDs.isEmpty) return Nil + + val idx = split.index + val prefs = inputRDDs.map(rdd => rdd.preferredLocations(rdd.partitions(idx))) + // Prefer nodes where all inputs are local; fall back to any input's preferred location + val intersection = prefs.reduce((a, b) => a.intersect(b)) + if (intersection.nonEmpty) intersection else prefs.flatten.distinct + } + + override def clearDependencies(): Unit = { + super.clearDependencies() + inputRDDs = null } } object CometExecRDD { - def apply(sc: SparkContext, partitionNum: Int)( - f: (Seq[Iterator[ColumnarBatch]], Int, Int) => Iterator[ColumnarBatch]) - : RDD[ColumnarBatch] = - withScope(sc) { - new CometExecRDD(sc, partitionNum, f) - } - private[spark] def withScope[U](sc: SparkContext)(body: => U): U = - RDDOperationScope.withScope[U](sc)(body) + /** + * Creates an RDD for native execution with optional per-partition planning data. + */ + // scalastyle:off + def apply( + sc: SparkContext, + inputRDDs: Seq[RDD[ColumnarBatch]], + commonByKey: Map[String, Array[Byte]], + perPartitionByKey: Map[String, Array[Array[Byte]]], + serializedPlan: Array[Byte], + numPartitions: Int, + numOutputCols: Int, + nativeMetrics: CometMetricNode, + subqueries: Seq[ScalarSubquery], + broadcastedHadoopConfForEncryption: Option[Broadcast[SerializableConfiguration]] = None, + encryptedFilePaths: Seq[String] = Seq.empty): CometExecRDD = { + // scalastyle:on + + new CometExecRDD( + sc, + inputRDDs, + commonByKey, + perPartitionByKey, + serializedPlan, + numPartitions, + numOutputCols, + nativeMetrics, + subqueries, + broadcastedHadoopConfForEncryption, + encryptedFilePaths) + } } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala index 223ae4fbb7..36085b6329 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometIcebergNativeScanExec.scala @@ -21,18 +21,23 @@ package org.apache.spark.sql.comet import scala.jdk.CollectionConverters._ +import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, DynamicPruningExpression, SortOrder} import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.physical.{KeyGroupedPartitioning, Partitioning, UnknownPartitioning} +import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} +import org.apache.spark.sql.execution.{InSubqueryExec, SubqueryAdaptiveBroadcastExec} import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.AccumulatorV2 import com.google.common.base.Objects import org.apache.comet.iceberg.CometIcebergNativeScanMetadata import org.apache.comet.serde.OperatorOuterClass.Operator +import org.apache.comet.serde.operator.CometIcebergNativeScan +import org.apache.comet.shims.ShimSubqueryBroadcast /** * Native Iceberg scan operator that delegates file reading to iceberg-rust. @@ -41,6 +46,10 @@ import org.apache.comet.serde.OperatorOuterClass.Operator * execution. Iceberg's catalog and planning run in Spark to produce FileScanTasks, which are * serialized to protobuf for the native side to execute using iceberg-rust's FileIO and * ArrowReader. This provides better performance than reading through Spark's abstraction layers. + * + * Supports Dynamic Partition Pruning (DPP) by deferring partition serialization to execution + * time. The doPrepare() method waits for DPP subqueries to resolve, then lazy + * serializedPartitionData serializes the DPP-filtered partitions from inputRDD. */ case class CometIcebergNativeScanExec( override val nativeOp: Operator, @@ -48,16 +57,136 @@ case class CometIcebergNativeScanExec( @transient override val originalPlan: BatchScanExec, override val serializedPlanOpt: SerializedPlan, metadataLocation: String, - numPartitions: Int, @transient nativeIcebergScanMetadata: CometIcebergNativeScanMetadata) - extends CometLeafExec { + extends CometLeafExec + with ShimSubqueryBroadcast { override val supportsColumnar: Boolean = true override val nodeName: String = "CometIcebergNativeScan" - override lazy val outputPartitioning: Partitioning = - UnknownPartitioning(numPartitions) + /** + * Prepare DPP subquery plans. Called by Spark's prepare() before doExecuteColumnar(). + * + * This follows Spark's convention of preparing subqueries in doPrepare() rather than + * doExecuteColumnar(). While the actual waiting for DPP results happens later in + * serializedPartitionData, calling prepare() here ensures subquery plans are set up before + * execution begins. + */ + override protected def doPrepare(): Unit = { + originalPlan.runtimeFilters.foreach { + case DynamicPruningExpression(e: InSubqueryExec) => + e.plan.prepare() + case _ => + } + super.doPrepare() + } + + /** + * Lazy partition serialization - deferred until execution time for DPP support. + * + * Entry points: This lazy val may be triggered from either doExecuteColumnar() (via + * commonData/perPartitionData) or capturedMetricValues (for Iceberg metrics). Lazy val + * semantics ensure single evaluation regardless of entry point. + * + * DPP (Dynamic Partition Pruning) Flow: + * + * {{{ + * Planning time: + * CometIcebergNativeScanExec created + * - serializedPartitionData not evaluated (lazy) + * - No partition serialization yet + * + * Execution time: + * 1. Spark calls prepare() on the plan tree + * - doPrepare() calls e.plan.prepare() for each DPP filter + * - Subquery plans are set up (but not yet executed) + * + * 2. Spark calls doExecuteColumnar() (or metrics are accessed) + * - Accesses perPartitionData (or capturedMetricValues) + * - Forces serializedPartitionData evaluation (here) + * - Waits for DPP values (updateResult or reflection) + * - Calls serializePartitions with DPP-filtered inputRDD + * - Only matching partitions are serialized + * }}} + */ + @transient private lazy val serializedPartitionData: (Array[Byte], Array[Array[Byte]]) = { + // Ensure DPP subqueries are resolved before accessing inputRDD. + originalPlan.runtimeFilters.foreach { + case DynamicPruningExpression(e: InSubqueryExec) if e.values().isEmpty => + e.plan match { + case sab: SubqueryAdaptiveBroadcastExec => + // SubqueryAdaptiveBroadcastExec.executeCollect() throws, so we call + // child.executeCollect() directly. We use the index from SAB to find the + // right buildKey, then locate that key's column in child.output. + val rows = sab.child.executeCollect() + val indices = getSubqueryBroadcastIndices(sab) + + // SPARK-46946 changed index: Int to indices: Seq[Int] as a preparatory refactor + // for future features (Null Safe Equality DPP, multiple equality predicates). + // Currently indices always has one element. CometScanRule checks for multi-index + // DPP and falls back, so this assertion should never fail. + assert( + indices.length == 1, + s"Multi-index DPP not supported: indices=$indices. See SPARK-46946.") + val buildKeyIndex = indices.head + val buildKey = sab.buildKeys(buildKeyIndex) + + // Find column index in child.output by matching buildKey's exprId + val colIndex = buildKey match { + case attr: Attribute => + sab.child.output.indexWhere(_.exprId == attr.exprId) + // DPP may cast partition column to match join key type + case Cast(attr: Attribute, _, _, _) => + sab.child.output.indexWhere(_.exprId == attr.exprId) + case _ => buildKeyIndex + } + if (colIndex < 0) { + throw new IllegalStateException( + s"DPP build key '$buildKey' not found in ${sab.child.output.map(_.name)}") + } + + setInSubqueryResult(e, rows.map(_.get(colIndex, e.child.dataType))) + case _ => + e.updateResult() + } + case _ => + } + + CometIcebergNativeScan.serializePartitions(originalPlan, output, nativeIcebergScanMetadata) + } + + /** + * Sets InSubqueryExec's private result field via reflection. + * + * Reflection is required because: + * - SubqueryAdaptiveBroadcastExec.executeCollect() throws UnsupportedOperationException + * - InSubqueryExec has no public setter for result, only updateResult() which calls + * executeCollect() + * - We can't replace e.plan since it's a val + */ + private def setInSubqueryResult(e: InSubqueryExec, result: Array[_]): Unit = { + val fields = e.getClass.getDeclaredFields + // Field name is mangled by Scala compiler, e.g. "org$apache$...$InSubqueryExec$$result" + val resultField = fields + .find(f => f.getName.endsWith("$result") && !f.getName.contains("Broadcast")) + .getOrElse { + throw new IllegalStateException( + s"Cannot find 'result' field in ${e.getClass.getName}. " + + "Spark version may be incompatible with Comet's DPP implementation.") + } + resultField.setAccessible(true) + resultField.set(e, result) + } + + def commonData: Array[Byte] = serializedPartitionData._1 + def perPartitionData: Array[Array[Byte]] = serializedPartitionData._2 + + // numPartitions for execution - derived from actual DPP-filtered partitions + // Only accessed during execution, not planning + def numPartitions: Int = perPartitionData.length + + override lazy val outputPartitioning: Partitioning = UnknownPartitioning(numPartitions) override lazy val outputOrdering: Seq[SortOrder] = Nil @@ -95,17 +224,34 @@ case class CometIcebergNativeScanExec( } } - private val capturedMetricValues: Seq[MetricValue] = { - originalPlan.metrics - .filterNot { case (name, _) => - // Filter out metrics that are now runtime metrics incremented on the native side - name == "numOutputRows" || name == "numDeletes" || name == "numSplits" - } - .map { case (name, metric) => - val mappedType = mapMetricType(name, metric.metricType) - MetricValue(name, metric.value, mappedType) - } - .toSeq + /** + * Captures Iceberg planning metrics for display in Spark UI. + * + * This lazy val intentionally triggers serializedPartitionData evaluation because Iceberg + * populates metrics during planning (when inputRDD is accessed). Both this and + * doExecuteColumnar() may trigger serializedPartitionData, but lazy val semantics ensure it's + * evaluated only once. + */ + @transient private lazy val capturedMetricValues: Seq[MetricValue] = { + // Guard against null originalPlan (from doCanonicalize) + if (originalPlan == null) { + Seq.empty + } else { + // Trigger serializedPartitionData to ensure Iceberg planning has run and + // metrics are populated + val _ = serializedPartitionData + + originalPlan.metrics + .filterNot { case (name, _) => + // Filter out metrics that are now runtime metrics incremented on the native side + name == "numOutputRows" || name == "numDeletes" || name == "numSplits" + } + .map { case (name, metric) => + val mappedType = mapMetricType(name, metric.metricType) + MetricValue(name, metric.value, mappedType) + } + .toSeq + } } /** @@ -146,62 +292,88 @@ case class CometIcebergNativeScanExec( baseMetrics ++ icebergMetrics + ("num_splits" -> numSplitsMetric) } + /** Executes using CometExecRDD - planning data is computed lazily on first access. */ + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + val nativeMetrics = CometMetricNode.fromCometPlan(this) + val serializedPlan = CometExec.serializeNativePlan(nativeOp) + CometExecRDD( + sparkContext, + inputRDDs = Seq.empty, + commonByKey = Map(metadataLocation -> commonData), + perPartitionByKey = Map(metadataLocation -> perPartitionData), + serializedPlan = serializedPlan, + numPartitions = perPartitionData.length, + numOutputCols = output.length, + nativeMetrics = nativeMetrics, + subqueries = Seq.empty) + } + + /** + * Override convertBlock to preserve @transient fields. The parent implementation uses + * makeCopy() which loses transient fields. + */ + override def convertBlock(): CometIcebergNativeScanExec = { + // Serialize the native plan if not already done + val newSerializedPlan = if (serializedPlanOpt.isEmpty) { + val bytes = CometExec.serializeNativePlan(nativeOp) + SerializedPlan(Some(bytes)) + } else { + serializedPlanOpt + } + + // Create new instance preserving transient fields + CometIcebergNativeScanExec( + nativeOp, + output, + originalPlan, + newSerializedPlan, + metadataLocation, + nativeIcebergScanMetadata) + } + override protected def doCanonicalize(): CometIcebergNativeScanExec = { CometIcebergNativeScanExec( nativeOp, output.map(QueryPlan.normalizeExpressions(_, output)), - originalPlan.doCanonicalize(), + null, // Don't need originalPlan for canonicalization SerializedPlan(None), metadataLocation, - numPartitions, - nativeIcebergScanMetadata) + null + ) // Don't need metadata for canonicalization } - override def stringArgs: Iterator[Any] = - Iterator(output, s"$metadataLocation, ${originalPlan.scan.description()}", numPartitions) + override def stringArgs: Iterator[Any] = { + // Use metadata task count to avoid triggering serializedPartitionData during planning + val hasMeta = nativeIcebergScanMetadata != null && nativeIcebergScanMetadata.tasks != null + val taskCount = if (hasMeta) nativeIcebergScanMetadata.tasks.size() else 0 + val scanDesc = if (originalPlan != null) originalPlan.scan.description() else "canonicalized" + // Include runtime filters (DPP) in string representation + val runtimeFiltersStr = if (originalPlan != null && originalPlan.runtimeFilters.nonEmpty) { + s", runtimeFilters=${originalPlan.runtimeFilters.mkString("[", ", ", "]")}" + } else { + "" + } + Iterator(output, s"$metadataLocation, $scanDesc$runtimeFiltersStr", taskCount) + } override def equals(obj: Any): Boolean = { obj match { case other: CometIcebergNativeScanExec => this.metadataLocation == other.metadataLocation && this.output == other.output && - this.serializedPlanOpt == other.serializedPlanOpt && - this.numPartitions == other.numPartitions + this.serializedPlanOpt == other.serializedPlanOpt case _ => false } } override def hashCode(): Int = - Objects.hashCode( - metadataLocation, - output.asJava, - serializedPlanOpt, - numPartitions: java.lang.Integer) + Objects.hashCode(metadataLocation, output.asJava, serializedPlanOpt) } object CometIcebergNativeScanExec { - /** - * Creates a CometIcebergNativeScanExec from a Spark BatchScanExec. - * - * Determines the number of partitions from Iceberg's output partitioning: - * - KeyGroupedPartitioning: Use Iceberg's partition count - * - Other cases: Use the number of InputPartitions from Iceberg's planning - * - * @param nativeOp - * The serialized native operator - * @param scanExec - * The original Spark BatchScanExec - * @param session - * The SparkSession - * @param metadataLocation - * Path to table metadata file - * @param nativeIcebergScanMetadata - * Pre-extracted Iceberg metadata from planning phase - * @return - * A new CometIcebergNativeScanExec - */ + /** Creates a CometIcebergNativeScanExec with deferred partition serialization. */ def apply( nativeOp: Operator, scanExec: BatchScanExec, @@ -209,21 +381,12 @@ object CometIcebergNativeScanExec { metadataLocation: String, nativeIcebergScanMetadata: CometIcebergNativeScanMetadata): CometIcebergNativeScanExec = { - // Determine number of partitions from Iceberg's output partitioning - val numParts = scanExec.outputPartitioning match { - case p: KeyGroupedPartitioning => - p.numPartitions - case _ => - scanExec.inputRDD.getNumPartitions - } - val exec = CometIcebergNativeScanExec( nativeOp, scanExec.output, scanExec, SerializedPlan(None), metadataLocation, - numParts, nativeIcebergScanMetadata) scanExec.logicalLink.foreach(exec.setLogicalLink) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala index 6f0718d0ed..841bc21aa2 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala @@ -65,8 +65,16 @@ case class CometNativeScanExec( override val nodeName: String = s"CometNativeScan $relation ${tableIdentifier.map(_.unquotedString).getOrElse("")}" - override lazy val outputPartitioning: Partitioning = - UnknownPartitioning(originalPlan.inputRDD.getNumPartitions) + // exposed for testing + lazy val bucketedScan: Boolean = originalPlan.bucketedScan && !disableBucketedScan + + override lazy val outputPartitioning: Partitioning = { + if (bucketedScan) { + originalPlan.outputPartitioning + } else { + UnknownPartitioning(originalPlan.inputRDD.getNumPartitions) + } + } override lazy val outputOrdering: Seq[SortOrder] = originalPlan.outputOrdering diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/ZippedPartitionsRDD.scala b/spark/src/main/scala/org/apache/spark/sql/comet/ZippedPartitionsRDD.scala deleted file mode 100644 index fdf8bf393d..0000000000 --- a/spark/src/main/scala/org/apache/spark/sql/comet/ZippedPartitionsRDD.scala +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.spark.sql.comet - -import org.apache.spark.{Partition, SparkContext, TaskContext} -import org.apache.spark.rdd.{RDD, RDDOperationScope, ZippedPartitionsBaseRDD, ZippedPartitionsPartition} -import org.apache.spark.sql.vectorized.ColumnarBatch - -/** - * Similar to Spark `ZippedPartitionsRDD[1-4]` classes, this class is used to zip partitions of - * the multiple RDDs into a single RDD. Spark `ZippedPartitionsRDD[1-4]` classes only support at - * most 4 RDDs. This class is used to support more than 4 RDDs. This ZipPartitionsRDD is used to - * zip the input sources of the Comet physical plan. So it only zips partitions of ColumnarBatch. - */ -private[spark] class ZippedPartitionsRDD( - sc: SparkContext, - var f: (Seq[Iterator[ColumnarBatch]], Int, Int) => Iterator[ColumnarBatch], - var zipRdds: Seq[RDD[ColumnarBatch]], - preservesPartitioning: Boolean = false) - extends ZippedPartitionsBaseRDD[ColumnarBatch](sc, zipRdds, preservesPartitioning) { - - // We need to get the number of partitions in `compute` but `getNumPartitions` is not available - // on the executors. So we need to capture it here. - private val numParts: Int = this.getNumPartitions - - override def compute(s: Partition, context: TaskContext): Iterator[ColumnarBatch] = { - val partitions = s.asInstanceOf[ZippedPartitionsPartition].partitions - val iterators = - zipRdds.zipWithIndex.map(pair => pair._1.iterator(partitions(pair._2), context)) - f(iterators, numParts, s.index) - } - - override def clearDependencies(): Unit = { - super.clearDependencies() - zipRdds = null - f = null - } -} - -object ZippedPartitionsRDD { - def apply(sc: SparkContext, rdds: Seq[RDD[ColumnarBatch]])( - f: (Seq[Iterator[ColumnarBatch]], Int, Int) => Iterator[ColumnarBatch]) - : RDD[ColumnarBatch] = - withScope(sc) { - new ZippedPartitionsRDD(sc, f, rdds) - } - - private[spark] def withScope[U](sc: SparkContext)(body: => U): U = - RDDOperationScope.withScope[U](sc)(body) -} 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 6f33467efe..3c6941f687 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 @@ -25,7 +25,6 @@ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.jdk.CollectionConverters._ -import org.apache.spark.TaskContext import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -59,6 +58,139 @@ import org.apache.comet.serde.OperatorOuterClass.{AggregateMode => CometAggregat import org.apache.comet.serde.QueryPlanSerde.{aggExprToProto, exprToProto, supportedSortType} import org.apache.comet.serde.operator.CometSink +/** + * Trait for injecting per-partition planning data into operator nodes. + * + * Implementations handle specific operator types (e.g., Iceberg scans, Delta scans). + */ +private[comet] trait PlanDataInjector { + + /** Check if this injector can handle the given operator. */ + def canInject(op: Operator): Boolean + + /** Extract the key used to look up planning data for this operator. */ + def getKey(op: Operator): Option[String] + + /** Inject common + partition data into the operator node. */ + def inject(op: Operator, commonBytes: Array[Byte], partitionBytes: Array[Byte]): Operator +} + +/** + * Registry and utilities for injecting per-partition planning data into operator trees. + */ +private[comet] object PlanDataInjector { + + // Registry of injectors for different operator types + private val injectors: Seq[PlanDataInjector] = Seq( + IcebergPlanDataInjector + // Future: DeltaPlanDataInjector, HudiPlanDataInjector, etc. + ) + + /** + * Injects planning data into an Operator tree by finding nodes that need injection and applying + * the appropriate injector. + * + * Supports joins over multiple tables by matching each operator with its corresponding data + * based on a key (e.g., metadata_location for Iceberg). + */ + def injectPlanData( + op: Operator, + commonByKey: Map[String, Array[Byte]], + partitionByKey: Map[String, Array[Byte]]): Operator = { + val builder = op.toBuilder + + // Try each injector to see if it can handle this operator + for (injector <- injectors if injector.canInject(op)) { + injector.getKey(op) match { + case Some(key) => + (commonByKey.get(key), partitionByKey.get(key)) match { + case (Some(commonBytes), Some(partitionBytes)) => + val injectedOp = injector.inject(op, commonBytes, partitionBytes) + // Copy the injected operator's fields to our builder + builder.clear() + builder.mergeFrom(injectedOp) + case _ => + throw new CometRuntimeException(s"Missing planning data for key: $key") + } + case None => // No key, skip injection + } + } + + // Recursively process children + builder.clearChildren() + op.getChildrenList.asScala.foreach { child => + builder.addChildren(injectPlanData(child, commonByKey, partitionByKey)) + } + + builder.build() + } + + def serializeOperator(op: Operator): Array[Byte] = { + val size = op.getSerializedSize + val bytes = new Array[Byte](size) + val codedOutput = CodedOutputStream.newInstance(bytes) + op.writeTo(codedOutput) + codedOutput.checkNoSpaceLeft() + bytes + } +} + +/** + * Injector for Iceberg scan operators. + */ +private[comet] object IcebergPlanDataInjector extends PlanDataInjector { + import java.nio.ByteBuffer + import java.util.{LinkedHashMap, Map => JMap} + + private final val maxCacheEntries = 16 + + // Cache parsed IcebergScanCommon to avoid reparsing for Iceberg tables with large numbers of + // partitions (thousands or more) that may repeatedly parse the same commonBytes. + // IcebergPlanDataInjector is a singleton, so we use an LRU cache to eventually evict old + // IcebergScanCommon objects. 16 seems like a reasonable starting point since these objects + // are not large. Thread-safe LinkedHashMap with accessOrder=true provides LRU ordering. + private val commonCache = java.util.Collections.synchronizedMap( + new LinkedHashMap[ByteBuffer, OperatorOuterClass.IcebergScanCommon](4, 0.75f, true) { + override def removeEldestEntry( + eldest: JMap.Entry[ByteBuffer, OperatorOuterClass.IcebergScanCommon]): Boolean = { + size() > maxCacheEntries + } + }) + + override def canInject(op: Operator): Boolean = + op.hasIcebergScan && + op.getIcebergScan.getFileScanTasksCount == 0 && + op.getIcebergScan.hasCommon + + override def getKey(op: Operator): Option[String] = + Some(op.getIcebergScan.getCommon.getMetadataLocation) + + override def inject( + op: Operator, + commonBytes: Array[Byte], + partitionBytes: Array[Byte]): Operator = { + val scan = op.getIcebergScan + + // Cache the parsed common data to avoid deserializing on every partition + val cacheKey = ByteBuffer.wrap(commonBytes) + val common = commonCache.synchronized { + Option(commonCache.get(cacheKey)).getOrElse { + val parsed = OperatorOuterClass.IcebergScanCommon.parseFrom(commonBytes) + commonCache.put(cacheKey, parsed) + parsed + } + } + + val tasksOnly = OperatorOuterClass.IcebergScan.parseFrom(partitionBytes) + + val scanBuilder = scan.toBuilder + scanBuilder.setCommon(common) + scanBuilder.addAllFileScanTasks(tasksOnly.getFileScanTasksList) + + op.toBuilder.setIcebergScan(scanBuilder).build() + } +} + /** * A Comet physical operator */ @@ -105,6 +237,15 @@ abstract class CometExec extends CometPlan { } } } + + /** Collects all ScalarSubquery expressions from a plan tree. */ + protected def collectSubqueries(sparkPlan: SparkPlan): Seq[ScalarSubquery] = { + val childSubqueries = sparkPlan.children.flatMap(collectSubqueries) + val planSubqueries = sparkPlan.expressions.flatMap { + _.collect { case sub: ScalarSubquery => sub } + } + childSubqueries ++ planSubqueries + } } object CometExec { @@ -290,32 +431,8 @@ abstract class CometNativeExec extends CometExec { case None => (None, Seq.empty) } - def createCometExecIter( - inputs: Seq[Iterator[ColumnarBatch]], - numParts: Int, - partitionIndex: Int): CometExecIterator = { - val it = new CometExecIterator( - CometExec.newIterId, - inputs, - output.length, - serializedPlanCopy, - nativeMetrics, - numParts, - partitionIndex, - broadcastedHadoopConfForEncryption, - encryptedFilePaths) - - setSubqueries(it.id, this) - - Option(TaskContext.get()).foreach { context => - context.addTaskCompletionListener[Unit] { _ => - it.close() - cleanSubqueries(it.id, this) - } - } - - it - } + // Find planning data within this stage (stops at shuffle boundaries). + val (commonByKey, perPartitionByKey) = findAllPlanData(this) // Collect the input ColumnarBatches from the child operators and create a CometExecIterator // to execute the native plan. @@ -395,12 +512,20 @@ abstract class CometNativeExec extends CometExec { throw new CometRuntimeException(s"No input for CometNativeExec:\n $this") } - if (inputs.nonEmpty) { - ZippedPartitionsRDD(sparkContext, inputs.toSeq)(createCometExecIter) - } else { - val partitionNum = firstNonBroadcastPlanNumPartitions - CometExecRDD(sparkContext, partitionNum)(createCometExecIter) - } + // Unified RDD creation - CometExecRDD handles all cases + val subqueries = collectSubqueries(this) + CometExecRDD( + sparkContext, + inputs.toSeq, + commonByKey, + perPartitionByKey, + serializedPlanCopy, + firstNonBroadcastPlanNumPartitions, + output.length, + nativeMetrics, + subqueries, + broadcastedHadoopConfForEncryption, + encryptedFilePaths) } } @@ -440,6 +565,49 @@ abstract class CometNativeExec extends CometExec { } } + /** + * Find all plan nodes with per-partition planning data in the plan tree. Returns two maps keyed + * by a unique identifier: one for common data (shared across partitions) and one for + * per-partition data. + * + * Currently supports Iceberg scans (keyed by metadata_location). Additional scan types can be + * added by extending this method. + * + * Stops at stage boundaries (shuffle exchanges, etc.) because partition indices are only valid + * within the same stage. + * + * @return + * (commonByKey, perPartitionByKey) - common data is shared, per-partition varies + */ + private def findAllPlanData( + plan: SparkPlan): (Map[String, Array[Byte]], Map[String, Array[Array[Byte]]]) = { + plan match { + // Found an Iceberg scan with planning data + case iceberg: CometIcebergNativeScanExec + if iceberg.commonData.nonEmpty && iceberg.perPartitionData.nonEmpty => + ( + Map(iceberg.metadataLocation -> iceberg.commonData), + Map(iceberg.metadataLocation -> iceberg.perPartitionData)) + + // Broadcast stages are boundaries - don't collect per-partition data from inside them. + // After DPP filtering, broadcast scans may have different partition counts than the + // probe side, causing ArrayIndexOutOfBoundsException in CometExecRDD.getPartitions. + case _: BroadcastQueryStageExec | _: CometBroadcastExchangeExec => + (Map.empty, Map.empty) + + // Stage boundaries - stop searching (partition indices won't align after these) + case _: ShuffleQueryStageExec | _: AQEShuffleReadExec | _: CometShuffleExchangeExec | + _: CometUnionExec | _: CometTakeOrderedAndProjectExec | _: CometCoalesceExec | + _: ReusedExchangeExec | _: CometSparkToColumnarExec => + (Map.empty, Map.empty) + + // Continue searching through other operators, combining results from all children + case _ => + val results = plan.children.map(findAllPlanData) + (results.flatMap(_._1).toMap, results.flatMap(_._2).toMap) + } + } + /** * Converts this native Comet operator and its children into a native block which can be * executed as a whole (i.e., in a single JNI call) from the native side. @@ -1904,6 +2072,13 @@ case class CometSortMergeJoinExec( } object CometScanWrapper extends CometSink[SparkPlan] { + override def isFfiSafe(op: SparkPlan): Boolean = op match { + // CometScanExec (native_iceberg_compat) uses immutable Arrow readers + case _: CometScanExec => true + // CometBatchScanExec (Iceberg Java integration) still uses mutable buffers + case _ => false + } + override def createExec(nativeOp: Operator, op: SparkPlan): CometNativeExec = { CometScanWrapper(nativeOp, op) } diff --git a/spark/src/main/spark-3.4/org/apache/comet/shims/ShimSubqueryBroadcast.scala b/spark/src/main/spark-3.4/org/apache/comet/shims/ShimSubqueryBroadcast.scala new file mode 100644 index 0000000000..1ff0935041 --- /dev/null +++ b/spark/src/main/spark-3.4/org/apache/comet/shims/ShimSubqueryBroadcast.scala @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet.shims + +import org.apache.spark.sql.execution.SubqueryAdaptiveBroadcastExec + +trait ShimSubqueryBroadcast { + + /** + * Gets the build key indices from SubqueryAdaptiveBroadcastExec. Spark 3.x has `index: Int`, + * Spark 4.x has `indices: Seq[Int]`. + */ + def getSubqueryBroadcastIndices(sab: SubqueryAdaptiveBroadcastExec): Seq[Int] = { + Seq(sab.index) + } +} diff --git a/spark/src/main/spark-3.5/org/apache/comet/shims/ShimSubqueryBroadcast.scala b/spark/src/main/spark-3.5/org/apache/comet/shims/ShimSubqueryBroadcast.scala new file mode 100644 index 0000000000..1ff0935041 --- /dev/null +++ b/spark/src/main/spark-3.5/org/apache/comet/shims/ShimSubqueryBroadcast.scala @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet.shims + +import org.apache.spark.sql.execution.SubqueryAdaptiveBroadcastExec + +trait ShimSubqueryBroadcast { + + /** + * Gets the build key indices from SubqueryAdaptiveBroadcastExec. Spark 3.x has `index: Int`, + * Spark 4.x has `indices: Seq[Int]`. + */ + def getSubqueryBroadcastIndices(sab: SubqueryAdaptiveBroadcastExec): Seq[Int] = { + Seq(sab.index) + } +} diff --git a/spark/src/main/spark-4.0/org/apache/comet/shims/ShimSubqueryBroadcast.scala b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimSubqueryBroadcast.scala new file mode 100644 index 0000000000..417dfd46b7 --- /dev/null +++ b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimSubqueryBroadcast.scala @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet.shims + +import org.apache.spark.sql.execution.SubqueryAdaptiveBroadcastExec + +trait ShimSubqueryBroadcast { + + /** + * Gets the build key indices from SubqueryAdaptiveBroadcastExec. Spark 3.x has `index: Int`, + * Spark 4.x has `indices: Seq[Int]`. + */ + def getSubqueryBroadcastIndices(sab: SubqueryAdaptiveBroadcastExec): Seq[Int] = { + sab.indices + } +} diff --git a/spark/src/test/resources/sql-tests/expressions/array/element_at_ansi.sql b/spark/src/test/resources/sql-tests/expressions/array/element_at_ansi.sql new file mode 100644 index 0000000000..0a3f418907 --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/array/element_at_ansi.sql @@ -0,0 +1,74 @@ +-- 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. + +-- ANSI mode element_at tests +-- Tests that element_at throws exceptions for out-of-bounds access in ANSI mode +-- Note: element_at uses 1-based indexing + +-- Config: spark.sql.ansi.enabled=true + +-- ============================================================================ +-- Test data setup +-- ============================================================================ + +statement +CREATE TABLE ansi_element_at_oob(arr array) USING parquet + +statement +INSERT INTO ansi_element_at_oob VALUES (array(1, 2, 3)) + +-- ============================================================================ +-- element_at index out of bounds (positive index) +-- Spark throws: [INVALID_ARRAY_INDEX_IN_ELEMENT_AT] ... +-- Comet throws: Index out of bounds for array +-- See https://github.com/apache/datafusion-comet/issues/3375 +-- ============================================================================ + +-- index beyond array length should throw (1-based indexing) +query ignore(https://github.com/apache/datafusion-comet/issues/3375) +SELECT element_at(arr, 10) FROM ansi_element_at_oob + +-- literal array with out of bounds access +query ignore(https://github.com/apache/datafusion-comet/issues/3375) +SELECT element_at(array(1, 2, 3), 5) + +-- ============================================================================ +-- element_at with index 0 (invalid) +-- Spark throws: [INVALID_INDEX_OF_ZERO] The index 0 is invalid +-- Comet throws: different error message +-- See https://github.com/apache/datafusion-comet/issues/3375 +-- ============================================================================ + +-- index 0 is not valid for element_at (1-based indexing) +query ignore(https://github.com/apache/datafusion-comet/issues/3375) +SELECT element_at(arr, 0) FROM ansi_element_at_oob + +-- literal with index 0 +query ignore(https://github.com/apache/datafusion-comet/issues/3375) +SELECT element_at(array(1, 2, 3), 0) + +-- ============================================================================ +-- element_at index out of bounds (negative index beyond array) +-- ============================================================================ + +-- negative index beyond array size should throw +query ignore(https://github.com/apache/datafusion-comet/issues/3375) +SELECT element_at(arr, -10) FROM ansi_element_at_oob + +-- literal with negative out of bounds +query ignore(https://github.com/apache/datafusion-comet/issues/3375) +SELECT element_at(array(1, 2, 3), -5) diff --git a/spark/src/test/resources/sql-tests/expressions/array/get_array_item_ansi.sql b/spark/src/test/resources/sql-tests/expressions/array/get_array_item_ansi.sql new file mode 100644 index 0000000000..19d0cb2a63 --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/array/get_array_item_ansi.sql @@ -0,0 +1,58 @@ +-- 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. + +-- ANSI mode array index access tests +-- Tests that array[index] throws exceptions for out-of-bounds access in ANSI mode + +-- Config: spark.sql.ansi.enabled=true + +-- ============================================================================ +-- Test data setup +-- ============================================================================ + +statement +CREATE TABLE ansi_array_oob(arr array) USING parquet + +statement +INSERT INTO ansi_array_oob VALUES (array(1, 2, 3)) + +-- ============================================================================ +-- Array index out of bounds (positive index) +-- Spark throws: [INVALID_ARRAY_INDEX] The index X is out of bounds +-- Comet throws: Index out of bounds for array +-- See https://github.com/apache/datafusion-comet/issues/3375 +-- ============================================================================ + +-- index beyond array length should throw (0-based indexing) +query ignore(https://github.com/apache/datafusion-comet/issues/3375) +SELECT arr[10] FROM ansi_array_oob + +-- literal array with out of bounds access +query ignore(https://github.com/apache/datafusion-comet/issues/3375) +SELECT array(1, 2, 3)[5] + +-- ============================================================================ +-- Array index out of bounds (negative index) +-- ============================================================================ + +-- negative index should throw +query ignore(https://github.com/apache/datafusion-comet/issues/3375) +SELECT arr[-1] FROM ansi_array_oob + +-- literal with negative index +query ignore(https://github.com/apache/datafusion-comet/issues/3375) +SELECT array(1, 2, 3)[-1] diff --git a/spark/src/test/resources/sql-tests/expressions/datetime/make_date.sql b/spark/src/test/resources/sql-tests/expressions/datetime/make_date.sql new file mode 100644 index 0000000000..544f3cc960 --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/datetime/make_date.sql @@ -0,0 +1,178 @@ +-- 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. + +-- ConfigMatrix: parquet.enable.dictionary=false,true + +statement +CREATE TABLE test_make_date(year int, month int, day int) USING parquet + +statement +INSERT INTO test_make_date VALUES + (2023, 12, 25), + (1970, 1, 1), + (2000, 6, 15), + (1999, 12, 31), + (2024, 2, 29), + (NULL, 1, 1), + (2023, NULL, 1), + (2023, 1, NULL), + (NULL, NULL, NULL) + +-- column arguments +query +SELECT year, month, day, make_date(year, month, day) FROM test_make_date ORDER BY year, month, day + +-- literal year, column month and day +query +SELECT make_date(2023, month, day) FROM test_make_date ORDER BY month, day + +-- column year, literal month and day +query +SELECT make_date(year, 6, 15) FROM test_make_date ORDER BY year + +-- column year and month, literal day +query +SELECT make_date(year, month, 1) FROM test_make_date ORDER BY year, month + +-- literal values +query +SELECT make_date(2023, 12, 25) + +query +SELECT make_date(1970, 1, 1) + +-- null handling with literals +query +SELECT make_date(NULL, 1, 1) + +query +SELECT make_date(2023, NULL, 1) + +query +SELECT make_date(2023, 1, NULL) + +-- leap year edge cases +-- 2000 WAS a leap year (divisible by 400) +query +SELECT make_date(2000, 2, 29) + +-- 2004 was a leap year (divisible by 4, not by 100) +query +SELECT make_date(2004, 2, 29) + +-- 2023 is NOT a leap year - Feb 29 should return NULL +query +SELECT make_date(2023, 2, 29) + +-- 1900 was NOT a leap year (divisible by 100 but not 400) - Feb 29 should return NULL +query +SELECT make_date(1900, 2, 29) + +-- 2100 will NOT be a leap year (divisible by 100 but not 400) +query +SELECT make_date(2100, 2, 29) + +-- invalid date handling - should return NULL +query +SELECT make_date(2023, 2, 30) + +query +SELECT make_date(2023, 2, 31) + +query +SELECT make_date(2023, 4, 31) + +query +SELECT make_date(2023, 6, 31) + +query +SELECT make_date(2023, 9, 31) + +query +SELECT make_date(2023, 11, 31) + +-- boundary values - invalid month/day values should return NULL +query +SELECT make_date(2023, 0, 15) + +query +SELECT make_date(2023, 13, 15) + +query +SELECT make_date(2023, -1, 15) + +query +SELECT make_date(2023, 6, 0) + +query +SELECT make_date(2023, 6, 32) + +query +SELECT make_date(2023, 6, -1) + +-- extreme years +query +SELECT make_date(1, 1, 1) + +query +SELECT make_date(9999, 12, 31) + +query +SELECT make_date(0, 1, 1) + +query +SELECT make_date(-1, 1, 1) + +-- month boundaries - last day of each month +query +SELECT make_date(2023, 1, 31) + +query +SELECT make_date(2023, 3, 31) + +query +SELECT make_date(2023, 4, 30) + +query +SELECT make_date(2023, 5, 31) + +query +SELECT make_date(2023, 6, 30) + +query +SELECT make_date(2023, 7, 31) + +query +SELECT make_date(2023, 8, 31) + +query +SELECT make_date(2023, 9, 30) + +query +SELECT make_date(2023, 10, 31) + +query +SELECT make_date(2023, 11, 30) + +query +SELECT make_date(2023, 12, 31) + +query +SELECT make_date(2024, 2, 29) + +query +SELECT make_date(2023, 2, 28) diff --git a/spark/src/test/resources/sql-tests/expressions/map/map_contains_key.sql b/spark/src/test/resources/sql-tests/expressions/map/map_contains_key.sql new file mode 100644 index 0000000000..7dc3ce436d --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/map/map_contains_key.sql @@ -0,0 +1,75 @@ +-- 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. + +-- ConfigMatrix: parquet.enable.dictionary=false,true + +-- TODO: replace map_from_arrays with map whenever map is supported in Comet + +-- Basic integer key tests with map literals +query +select map_contains_key(map_from_arrays(array(1, 2), array('a', 'b')), 5) + +query +select map_contains_key(map_from_arrays(array(1, 2), array('a', 'b')), 1) + +-- Decimal type coercion tests +-- TODO: requires map cast to be supported in Comet +query spark_answer_only +select map_contains_key(map_from_arrays(array(1, 2), array('a', 'b')), 5.0) + +query spark_answer_only +select map_contains_key(map_from_arrays(array(1, 2), array('a', 'b')), 1.0) + +query spark_answer_only +select map_contains_key(map_from_arrays(array(1.0, 2), array('a', 'b')), 5) + +query spark_answer_only +select map_contains_key(map_from_arrays(array(1.0, 2), array('a', 'b')), 1) + +-- Empty map tests +-- TODO: requires casting from NullType to be supported in Comet +query spark_answer_only +select map_contains_key(map_from_arrays(array(), array()), 0) + +-- Test with table data +statement +CREATE TABLE test_map_contains_key(m map) USING parquet + +statement +INSERT INTO test_map_contains_key VALUES (map_from_arrays(array('a', 'b', 'c'), array(1, 2, 3))), (map_from_arrays(array('x'), array(10))), (map_from_arrays(array(), array())), (NULL) + +query +SELECT map_contains_key(m, 'a') FROM test_map_contains_key + +query +SELECT map_contains_key(m, 'x') FROM test_map_contains_key + +query +SELECT map_contains_key(m, 'missing') FROM test_map_contains_key + +-- Test with integer key map +statement +CREATE TABLE test_map_int_key(m map) USING parquet + +statement +INSERT INTO test_map_int_key VALUES (map_from_arrays(array(1, 2), array('a', 'b'))), (map_from_arrays(array(), array())), (NULL) + +query +SELECT map_contains_key(m, 1) FROM test_map_int_key + +query +SELECT map_contains_key(m, 5) FROM test_map_int_key diff --git a/spark/src/test/resources/sql-tests/expressions/map/map_from_entries.sql b/spark/src/test/resources/sql-tests/expressions/map/map_from_entries.sql new file mode 100644 index 0000000000..0d5f81f6d5 --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/map/map_from_entries.sql @@ -0,0 +1,37 @@ +-- Licensed to the Apache Software Foundation (ASF) under one +-- or more contributor license agreements. See the NOTICE file +-- distributed with this work for additional information +-- regarding copyright ownership. The ASF licenses this file +-- to you under the Apache License, Version 2.0 (the +-- "License"); you may not use this file except in compliance +-- with the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, +-- software distributed under the License is distributed on an +-- "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +-- KIND, either express or implied. See the License for the +-- specific language governing permissions and limitations +-- under the License. + +-- ConfigMatrix: parquet.enable.dictionary=false,true + +statement +CREATE TABLE test_map_from_entries(entries array>) USING parquet + +statement +INSERT INTO test_map_from_entries VALUES (array(struct('a', 1), struct('b', 2), struct('c', 3))), (array()), (NULL) + +query +SELECT map_from_entries(entries) FROM test_map_from_entries + +query expect_fallback(Using BinaryType as Map keys is not allowed in map_from_entries) +SELECT map_from_entries(array(struct(cast('x' as binary), 10))) + +query expect_fallback(Using BinaryType as Map values is not allowed in map_from_entries) +SELECT map_from_entries(array(struct(10, cast('x' as binary)))) + +-- literal arguments +query spark_answer_only +SELECT map_from_entries(array(struct('x', 10), struct('y', 20), struct('z', 30))) diff --git a/spark/src/test/resources/sql-tests/expressions/math/abs_ansi.sql b/spark/src/test/resources/sql-tests/expressions/math/abs_ansi.sql new file mode 100644 index 0000000000..c89a2958c1 --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/math/abs_ansi.sql @@ -0,0 +1,97 @@ +-- 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. + +-- ANSI mode abs function tests +-- Tests that abs throws exceptions for overflow on minimum integer values + +-- Config: spark.sql.ansi.enabled=true + +-- ============================================================================ +-- Test data setup +-- ============================================================================ + +statement +CREATE TABLE ansi_test_abs_int(v int) USING parquet + +statement +INSERT INTO ansi_test_abs_int VALUES (-2147483648) + +statement +CREATE TABLE ansi_test_abs_long(v long) USING parquet + +statement +INSERT INTO ansi_test_abs_long VALUES (-9223372036854775808) + +statement +CREATE TABLE ansi_test_abs_short(v short) USING parquet + +statement +INSERT INTO ansi_test_abs_short VALUES (-32768) + +statement +CREATE TABLE ansi_test_abs_byte(v tinyint) USING parquet + +statement +INSERT INTO ansi_test_abs_byte VALUES (-128) + +-- ============================================================================ +-- abs(INT_MIN) overflow +-- ============================================================================ + +-- abs(-2147483648) cannot be represented as int (since INT_MAX = 2147483647) +query expect_error(overflow) +SELECT abs(v) FROM ansi_test_abs_int + +-- literal +query expect_error(overflow) +SELECT abs(-2147483648) + +-- ============================================================================ +-- abs(LONG_MIN) overflow +-- ============================================================================ + +-- abs(-9223372036854775808) cannot be represented as long +query expect_error(overflow) +SELECT abs(v) FROM ansi_test_abs_long + +-- literal +query expect_error(overflow) +SELECT abs(-9223372036854775808L) + +-- ============================================================================ +-- abs(SHORT_MIN) overflow +-- ============================================================================ + +-- abs(-32768) cannot be represented as short +query expect_error(overflow) +SELECT abs(v) FROM ansi_test_abs_short + +-- literal +query expect_error(overflow) +SELECT abs(cast(-32768 as short)) + +-- ============================================================================ +-- abs(BYTE_MIN) overflow +-- ============================================================================ + +-- abs(-128) cannot be represented as tinyint +query expect_error(overflow) +SELECT abs(v) FROM ansi_test_abs_byte + +-- literal +query expect_error(overflow) +SELECT abs(cast(-128 as tinyint)) diff --git a/spark/src/test/resources/sql-tests/expressions/math/arithmetic_ansi.sql b/spark/src/test/resources/sql-tests/expressions/math/arithmetic_ansi.sql new file mode 100644 index 0000000000..093e5a3f30 --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/math/arithmetic_ansi.sql @@ -0,0 +1,167 @@ +-- 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. + +-- ANSI mode arithmetic tests +-- Tests that ANSI mode throws exceptions for overflow and division by zero + +-- Config: spark.sql.ansi.enabled=true + +-- ============================================================================ +-- Test data setup for integer overflow +-- ============================================================================ + +statement +CREATE TABLE ansi_int_overflow(a int, b int) USING parquet + +statement +INSERT INTO ansi_int_overflow VALUES (2147483647, 1), (-2147483648, 1), (-2147483648, -1) + +statement +CREATE TABLE ansi_long_overflow(a long, b long) USING parquet + +statement +INSERT INTO ansi_long_overflow VALUES (9223372036854775807, 1), (-9223372036854775808, 1), (-9223372036854775808, -1) + +statement +CREATE TABLE ansi_div_zero(a int, b int, c long, d long) USING parquet + +statement +INSERT INTO ansi_div_zero VALUES (1, 0, 1, 0) + +-- ============================================================================ +-- Integer addition overflow +-- ============================================================================ + +-- INT_MAX + 1 should overflow +query expect_error(ARITHMETIC_OVERFLOW) +SELECT a + b FROM ansi_int_overflow WHERE a = 2147483647 + +-- literal overflow +query expect_error(ARITHMETIC_OVERFLOW) +SELECT 2147483647 + 1 + +-- ============================================================================ +-- Integer subtraction overflow +-- ============================================================================ + +-- INT_MIN - 1 should overflow +query expect_error(ARITHMETIC_OVERFLOW) +SELECT a - b FROM ansi_int_overflow WHERE a = -2147483648 + +-- literal overflow +query expect_error(ARITHMETIC_OVERFLOW) +SELECT -2147483648 - 1 + +-- ============================================================================ +-- Integer multiplication overflow +-- ============================================================================ + +-- INT_MAX * 2 should overflow +query expect_error(ARITHMETIC_OVERFLOW) +SELECT a * 2 FROM ansi_int_overflow WHERE a = 2147483647 + +-- literal overflow +query expect_error(ARITHMETIC_OVERFLOW) +SELECT 2147483647 * 2 + +-- ============================================================================ +-- Long addition overflow +-- ============================================================================ + +-- LONG_MAX + 1 should overflow +query expect_error(ARITHMETIC_OVERFLOW) +SELECT a + b FROM ansi_long_overflow WHERE a = 9223372036854775807 + +-- ============================================================================ +-- Long subtraction overflow +-- ============================================================================ + +-- LONG_MIN - 1 should overflow +query expect_error(ARITHMETIC_OVERFLOW) +SELECT a - b FROM ansi_long_overflow WHERE a = -9223372036854775808 + +-- ============================================================================ +-- Long multiplication overflow +-- ============================================================================ + +-- LONG_MAX * 2 should overflow +query expect_error(ARITHMETIC_OVERFLOW) +SELECT a * 2 FROM ansi_long_overflow WHERE a = 9223372036854775807 + +-- ============================================================================ +-- Integer division by zero +-- ============================================================================ + +-- column / 0 should throw +query expect_error(DIVIDE_BY_ZERO) +SELECT a / b FROM ansi_div_zero + +-- column div 0 (integral division) should throw +query expect_error(DIVIDE_BY_ZERO) +SELECT a div b FROM ansi_div_zero + +-- column % 0 (remainder) should throw +query expect_error(DIVIDE_BY_ZERO) +SELECT a % b FROM ansi_div_zero + +-- literal / 0 should throw +query expect_error(DIVIDE_BY_ZERO) +SELECT 1 / 0 + +-- literal div 0 should throw +query expect_error(DIVIDE_BY_ZERO) +SELECT 1 div 0 + +-- literal % 0 should throw +query expect_error(DIVIDE_BY_ZERO) +SELECT 1 % 0 + +-- ============================================================================ +-- Long division by zero +-- ============================================================================ + +-- long column / 0 should throw +query expect_error(DIVIDE_BY_ZERO) +SELECT c / d FROM ansi_div_zero + +-- long column div 0 should throw +query expect_error(DIVIDE_BY_ZERO) +SELECT c div d FROM ansi_div_zero + +-- long column % 0 should throw +query expect_error(DIVIDE_BY_ZERO) +SELECT c % d FROM ansi_div_zero + +-- ============================================================================ +-- Unary minus overflow +-- ============================================================================ + +-- negating INT_MIN should overflow (since INT_MAX is 2147483647, -(-2147483648) cannot fit) +query expect_error(ARITHMETIC_OVERFLOW) +SELECT -a FROM ansi_int_overflow WHERE a = -2147483648 + +-- negating LONG_MIN should overflow +query expect_error(ARITHMETIC_OVERFLOW) +SELECT -a FROM ansi_long_overflow WHERE a = -9223372036854775808 + +-- literal negation overflow +query expect_error(ARITHMETIC_OVERFLOW) +SELECT -(-2147483648) + +-- literal long negation overflow +query expect_error(ARITHMETIC_OVERFLOW) +SELECT -(-9223372036854775808L) diff --git a/spark/src/test/resources/sql-tests/expressions/string/string_space.sql b/spark/src/test/resources/sql-tests/expressions/string/string_space.sql index ec24bfb97e..4b826fbe3d 100644 --- a/spark/src/test/resources/sql-tests/expressions/string/string_space.sql +++ b/spark/src/test/resources/sql-tests/expressions/string/string_space.sql @@ -26,11 +26,9 @@ INSERT INTO test_space VALUES (0), (1), (5), (NULL), (-1) query SELECT concat('[', space(n), ']') FROM test_space WHERE n >= 0 OR n IS NULL --- Comet bug: space(-1) causes native crash "failed to round upto multiple of 64" --- https://github.com/apache/datafusion-comet/issues/3326 -query ignore(https://github.com/apache/datafusion-comet/issues/3326) +query SELECT concat('[', space(n), ']') FROM test_space WHERE n < 0 -- literal arguments -query ignore(https://github.com/apache/datafusion-comet/issues/3337) +query SELECT concat('[', space(5), ']'), concat('[', space(0), ']'), space(-1), space(NULL) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/explain.txt deleted file mode 100644 index ba9b1ca1f6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/explain.txt +++ /dev/null @@ -1,291 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (44) -+- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Filter (11) - : : : +- * HashAggregate (10) - : : : +- * CometColumnarToRow (9) - : : : +- CometColumnarExchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_returns (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (27) - : : +- * Filter (26) - : : +- * HashAggregate (25) - : : +- * CometColumnarToRow (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * HashAggregate (21) - : : +- * CometColumnarToRow (20) - : : +- CometColumnarExchange (19) - : : +- * HashAggregate (18) - : : +- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Filter (14) - : : : +- * ColumnarToRow (13) - : : : +- Scan parquet spark_catalog.default.store_returns (12) - : : +- ReusedExchange (15) - : +- BroadcastExchange (34) - : +- * CometColumnarToRow (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometNativeScan parquet spark_catalog.default.store (30) - +- BroadcastExchange (41) - +- * CometColumnarToRow (40) - +- CometProject (39) - +- CometFilter (38) - +- CometNativeScan parquet spark_catalog.default.customer (37) - - -(1) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk), IsNotNull(sr_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 2] -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] - -(3) Filter [codegen id : 2] -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [sr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 2] -Output [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] -Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] - -(7) HashAggregate [codegen id : 2] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] -Aggregate Attributes [1]: [sum#7] -Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] - -(8) CometColumnarExchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(9) CometColumnarToRow [codegen id : 9] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] - -(10) HashAggregate [codegen id : 9] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#9] -Results [3]: [sr_customer_sk#1 AS ctr_customer_sk#10, sr_store_sk#2 AS ctr_store_sk#11, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#9,17,2) AS ctr_total_return#12] - -(11) Filter [codegen id : 9] -Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12] -Condition : isnotnull(ctr_total_return#12) - -(12) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#16), dynamicpruningexpression(sr_returned_date_sk#16 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(13) ColumnarToRow [codegen id : 4] -Input [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] - -(14) Filter [codegen id : 4] -Input [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] -Condition : isnotnull(sr_store_sk#14) - -(15) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#17] - -(16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sr_returned_date_sk#16] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 4] -Output [3]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15] -Input [5]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16, d_date_sk#17] - -(18) HashAggregate [codegen id : 4] -Input [3]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15] -Keys [2]: [sr_customer_sk#13, sr_store_sk#14] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#15))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] - -(19) CometColumnarExchange -Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] -Arguments: hashpartitioning(sr_customer_sk#13, sr_store_sk#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(20) CometColumnarToRow [codegen id : 5] -Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] - -(21) HashAggregate [codegen id : 5] -Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] -Keys [2]: [sr_customer_sk#13, sr_store_sk#14] -Functions [1]: [sum(UnscaledValue(sr_return_amt#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#15))#9] -Results [2]: [sr_store_sk#14 AS ctr_store_sk#20, MakeDecimal(sum(UnscaledValue(sr_return_amt#15))#9,17,2) AS ctr_total_return#21] - -(22) HashAggregate [codegen id : 5] -Input [2]: [ctr_store_sk#20, ctr_total_return#21] -Keys [1]: [ctr_store_sk#20] -Functions [1]: [partial_avg(ctr_total_return#21)] -Aggregate Attributes [2]: [sum#22, count#23] -Results [3]: [ctr_store_sk#20, sum#24, count#25] - -(23) CometColumnarExchange -Input [3]: [ctr_store_sk#20, sum#24, count#25] -Arguments: hashpartitioning(ctr_store_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometColumnarToRow [codegen id : 6] -Input [3]: [ctr_store_sk#20, sum#24, count#25] - -(25) HashAggregate [codegen id : 6] -Input [3]: [ctr_store_sk#20, sum#24, count#25] -Keys [1]: [ctr_store_sk#20] -Functions [1]: [avg(ctr_total_return#21)] -Aggregate Attributes [1]: [avg(ctr_total_return#21)#26] -Results [2]: [(avg(ctr_total_return#21)#26 * 1.2) AS (avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] - -(26) Filter [codegen id : 6] -Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#27) - -(27) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=4] - -(28) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ctr_store_sk#11] -Right keys [1]: [ctr_store_sk#20] -Join type: Inner -Join condition: (cast(ctr_total_return#12 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#27) - -(29) Project [codegen id : 9] -Output [2]: [ctr_customer_sk#10, ctr_store_sk#11] -Input [5]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12, (avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] - -(30) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#28, s_state#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [s_store_sk#28, s_state#29] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#29, 2, true, false, true) = TN) AND isnotnull(s_store_sk#28)) - -(32) CometProject -Input [2]: [s_store_sk#28, s_state#29] -Arguments: [s_store_sk#28], [s_store_sk#28] - -(33) CometColumnarToRow [codegen id : 7] -Input [1]: [s_store_sk#28] - -(34) BroadcastExchange -Input [1]: [s_store_sk#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ctr_store_sk#11] -Right keys [1]: [s_store_sk#28] -Join type: Inner -Join condition: None - -(36) Project [codegen id : 9] -Output [1]: [ctr_customer_sk#10] -Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, s_store_sk#28] - -(37) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#30, c_customer_id#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(38) CometFilter -Input [2]: [c_customer_sk#30, c_customer_id#31] -Condition : isnotnull(c_customer_sk#30) - -(39) CometProject -Input [2]: [c_customer_sk#30, c_customer_id#31] -Arguments: [c_customer_sk#30, c_customer_id#32], [c_customer_sk#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#31, 16, true, false, true) AS c_customer_id#32] - -(40) CometColumnarToRow [codegen id : 8] -Input [2]: [c_customer_sk#30, c_customer_id#32] - -(41) BroadcastExchange -Input [2]: [c_customer_sk#30, c_customer_id#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(42) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ctr_customer_sk#10] -Right keys [1]: [c_customer_sk#30] -Join type: Inner -Join condition: None - -(43) Project [codegen id : 9] -Output [1]: [c_customer_id#32] -Input [3]: [ctr_customer_sk#10, c_customer_sk#30, c_customer_id#32] - -(44) TakeOrderedAndProject -Input [1]: [c_customer_id#32] -Arguments: 100, [c_customer_id#32 ASC NULLS FIRST], [c_customer_id#32] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometNativeScan parquet spark_catalog.default.date_dim (45) - - -(45) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [d_date_sk#6, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 2000)) AND isnotnull(d_date_sk#6)) - -(47) CometProject -Input [2]: [d_date_sk#6, d_year#33] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(49) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 12 Hosting Expression = sr_returned_date_sk#16 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/simplified.txt deleted file mode 100644 index ed85c142aa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/simplified.txt +++ /dev/null @@ -1,72 +0,0 @@ -TakeOrderedAndProject [c_customer_id] - WholeStageCodegen (9) - Project [c_customer_id] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk] - BroadcastHashJoin [ctr_store_sk,s_store_sk] - Project [ctr_customer_sk,ctr_store_sk] - BroadcastHashJoin [ctr_store_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [sr_customer_sk,sr_store_sk] #1 - WholeStageCodegen (2) - HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] - Project [sr_customer_sk,sr_store_sk,sr_return_amt] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_store_sk,sr_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (6) - Filter [(avg(ctr_total_return) * 1.2)] - HashAggregate [ctr_store_sk,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ctr_store_sk] #4 - WholeStageCodegen (5) - HashAggregate [ctr_store_sk,ctr_total_return] [sum,count,sum,count] - HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_store_sk,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [sr_customer_sk,sr_store_sk] #5 - WholeStageCodegen (4) - HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] - Project [sr_customer_sk,sr_store_sk,sr_return_amt] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id] [c_customer_sk,c_customer_id] - CometFilter [c_customer_sk,c_customer_id] - CometNativeScan 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/q1.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/explain.txt deleted file mode 100644 index e66401996f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,269 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometTakeOrderedAndProject (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometFilter (12) - : : : +- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (25) - : : +- CometFilter (24) - : : +- CometHashAggregate (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometHashAggregate (20) - : : +- CometExchange (19) - : : +- CometHashAggregate (18) - : : +- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (13) - : : +- ReusedExchange (15) - : +- CometBroadcastExchange (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (28) - +- CometBroadcastExchange (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (34) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk), IsNotNull(sr_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [sr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] -Arguments: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3], [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] - -(9) CometHashAggregate -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] - -(10) CometExchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] - -(12) CometFilter -Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] -Condition : isnotnull(ctr_total_return#11) - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#16)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#13) - -(15) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#17] - -(16) CometBroadcastHashJoin -Left output [4]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15] -Right output [1]: [d_date_sk#17] -Arguments: [sr_returned_date_sk#15], [d_date_sk#17], Inner, BuildRight - -(17) CometProject -Input [5]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15, d_date_sk#17] -Arguments: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14], [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14] - -(18) CometHashAggregate -Input [3]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14] -Keys [2]: [sr_customer_sk#12, sr_store_sk#13] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#14))] - -(19) CometExchange -Input [3]: [sr_customer_sk#12, sr_store_sk#13, sum#18] -Arguments: hashpartitioning(sr_customer_sk#12, sr_store_sk#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometHashAggregate -Input [3]: [sr_customer_sk#12, sr_store_sk#13, sum#18] -Keys [2]: [sr_customer_sk#12, sr_store_sk#13] -Functions [1]: [sum(UnscaledValue(sr_return_amt#14))] - -(21) CometHashAggregate -Input [2]: [ctr_store_sk#19, ctr_total_return#20] -Keys [1]: [ctr_store_sk#19] -Functions [1]: [partial_avg(ctr_total_return#20)] - -(22) CometExchange -Input [3]: [ctr_store_sk#19, sum#21, count#22] -Arguments: hashpartitioning(ctr_store_sk#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(23) CometHashAggregate -Input [3]: [ctr_store_sk#19, sum#21, count#22] -Keys [1]: [ctr_store_sk#19] -Functions [1]: [avg(ctr_total_return#20)] - -(24) CometFilter -Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) - -(25) CometBroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] -Arguments: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] - -(26) CometBroadcastHashJoin -Left output [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] -Right output [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] -Arguments: [ctr_store_sk#10], [ctr_store_sk#19], Inner, (cast(ctr_total_return#11 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#23), BuildRight - -(27) CometProject -Input [5]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11, (avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] -Arguments: [ctr_customer_sk#9, ctr_store_sk#10], [ctr_customer_sk#9, ctr_store_sk#10] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_state#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(29) CometFilter -Input [2]: [s_store_sk#24, s_state#25] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#25, 2, true, false, true) = TN) AND isnotnull(s_store_sk#24)) - -(30) CometProject -Input [2]: [s_store_sk#24, s_state#25] -Arguments: [s_store_sk#24], [s_store_sk#24] - -(31) CometBroadcastExchange -Input [1]: [s_store_sk#24] -Arguments: [s_store_sk#24] - -(32) CometBroadcastHashJoin -Left output [2]: [ctr_customer_sk#9, ctr_store_sk#10] -Right output [1]: [s_store_sk#24] -Arguments: [ctr_store_sk#10], [s_store_sk#24], Inner, BuildRight - -(33) CometProject -Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, s_store_sk#24] -Arguments: [ctr_customer_sk#9], [ctr_customer_sk#9] - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#26, c_customer_id#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [c_customer_sk#26, c_customer_id#27] -Condition : isnotnull(c_customer_sk#26) - -(36) CometProject -Input [2]: [c_customer_sk#26, c_customer_id#27] -Arguments: [c_customer_sk#26, c_customer_id#28], [c_customer_sk#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#27, 16, true, false, true) AS c_customer_id#28] - -(37) CometBroadcastExchange -Input [2]: [c_customer_sk#26, c_customer_id#28] -Arguments: [c_customer_sk#26, c_customer_id#28] - -(38) CometBroadcastHashJoin -Left output [1]: [ctr_customer_sk#9] -Right output [2]: [c_customer_sk#26, c_customer_id#28] -Arguments: [ctr_customer_sk#9], [c_customer_sk#26], Inner, BuildRight - -(39) CometProject -Input [3]: [ctr_customer_sk#9, c_customer_sk#26, c_customer_id#28] -Arguments: [c_customer_id#28], [c_customer_id#28] - -(40) CometTakeOrderedAndProject -Input [1]: [c_customer_id#28] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#28 ASC NULLS FIRST], output=[c_customer_id#28]), [c_customer_id#28], 100, 0, [c_customer_id#28 ASC NULLS FIRST], [c_customer_id#28] - -(41) CometColumnarToRow [codegen id : 1] -Input [1]: [c_customer_id#28] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) - - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(44) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(45) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(46) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 13 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/simplified.txt deleted file mode 100644 index 3e1d6243f8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id] - CometProject [c_customer_id] - CometBroadcastHashJoin [ctr_customer_sk,c_customer_sk,c_customer_id] - CometProject [ctr_customer_sk] - CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,s_store_sk] - CometProject [ctr_customer_sk,ctr_store_sk] - CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_store_sk] - CometFilter [ctr_customer_sk,ctr_store_sk,ctr_total_return] - CometHashAggregate [sum] [ctr_customer_sk,ctr_store_sk,ctr_total_return,sr_customer_sk,sr_store_sk,sum(UnscaledValue(sr_return_amt))] - CometExchange [sr_customer_sk,sr_store_sk] #1 - CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] - CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_store_sk] #4 - CometFilter [(avg(ctr_total_return) * 1.2),ctr_store_sk] - CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_store_sk,avg(ctr_total_return)] - CometExchange [ctr_store_sk] #5 - CometHashAggregate [ctr_total_return] [ctr_store_sk,sum,count] - CometHashAggregate [sr_customer_sk,sum] [ctr_store_sk,ctr_total_return,sr_store_sk,sum(UnscaledValue(sr_return_amt))] - CometExchange [sr_customer_sk,sr_store_sk] #6 - CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] - CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - 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 [native_iceberg_compat] 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 - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [c_customer_sk,c_customer_id] #8 - CometProject [c_customer_id] [c_customer_sk,c_customer_id] - CometFilter [c_customer_sk,c_customer_id] - CometScan [native_iceberg_compat] 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/q1/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/explain.txt deleted file mode 100644 index e66401996f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/explain.txt +++ /dev/null @@ -1,269 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometTakeOrderedAndProject (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometFilter (12) - : : : +- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (25) - : : +- CometFilter (24) - : : +- CometHashAggregate (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometHashAggregate (20) - : : +- CometExchange (19) - : : +- CometHashAggregate (18) - : : +- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (13) - : : +- ReusedExchange (15) - : +- CometBroadcastExchange (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (28) - +- CometBroadcastExchange (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (34) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk), IsNotNull(sr_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [sr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] -Arguments: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3], [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] - -(9) CometHashAggregate -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] - -(10) CometExchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] - -(12) CometFilter -Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] -Condition : isnotnull(ctr_total_return#11) - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#16)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#13) - -(15) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#17] - -(16) CometBroadcastHashJoin -Left output [4]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15] -Right output [1]: [d_date_sk#17] -Arguments: [sr_returned_date_sk#15], [d_date_sk#17], Inner, BuildRight - -(17) CometProject -Input [5]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15, d_date_sk#17] -Arguments: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14], [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14] - -(18) CometHashAggregate -Input [3]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14] -Keys [2]: [sr_customer_sk#12, sr_store_sk#13] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#14))] - -(19) CometExchange -Input [3]: [sr_customer_sk#12, sr_store_sk#13, sum#18] -Arguments: hashpartitioning(sr_customer_sk#12, sr_store_sk#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometHashAggregate -Input [3]: [sr_customer_sk#12, sr_store_sk#13, sum#18] -Keys [2]: [sr_customer_sk#12, sr_store_sk#13] -Functions [1]: [sum(UnscaledValue(sr_return_amt#14))] - -(21) CometHashAggregate -Input [2]: [ctr_store_sk#19, ctr_total_return#20] -Keys [1]: [ctr_store_sk#19] -Functions [1]: [partial_avg(ctr_total_return#20)] - -(22) CometExchange -Input [3]: [ctr_store_sk#19, sum#21, count#22] -Arguments: hashpartitioning(ctr_store_sk#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(23) CometHashAggregate -Input [3]: [ctr_store_sk#19, sum#21, count#22] -Keys [1]: [ctr_store_sk#19] -Functions [1]: [avg(ctr_total_return#20)] - -(24) CometFilter -Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) - -(25) CometBroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] -Arguments: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] - -(26) CometBroadcastHashJoin -Left output [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] -Right output [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] -Arguments: [ctr_store_sk#10], [ctr_store_sk#19], Inner, (cast(ctr_total_return#11 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#23), BuildRight - -(27) CometProject -Input [5]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11, (avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] -Arguments: [ctr_customer_sk#9, ctr_store_sk#10], [ctr_customer_sk#9, ctr_store_sk#10] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_state#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(29) CometFilter -Input [2]: [s_store_sk#24, s_state#25] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#25, 2, true, false, true) = TN) AND isnotnull(s_store_sk#24)) - -(30) CometProject -Input [2]: [s_store_sk#24, s_state#25] -Arguments: [s_store_sk#24], [s_store_sk#24] - -(31) CometBroadcastExchange -Input [1]: [s_store_sk#24] -Arguments: [s_store_sk#24] - -(32) CometBroadcastHashJoin -Left output [2]: [ctr_customer_sk#9, ctr_store_sk#10] -Right output [1]: [s_store_sk#24] -Arguments: [ctr_store_sk#10], [s_store_sk#24], Inner, BuildRight - -(33) CometProject -Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, s_store_sk#24] -Arguments: [ctr_customer_sk#9], [ctr_customer_sk#9] - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#26, c_customer_id#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [c_customer_sk#26, c_customer_id#27] -Condition : isnotnull(c_customer_sk#26) - -(36) CometProject -Input [2]: [c_customer_sk#26, c_customer_id#27] -Arguments: [c_customer_sk#26, c_customer_id#28], [c_customer_sk#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#27, 16, true, false, true) AS c_customer_id#28] - -(37) CometBroadcastExchange -Input [2]: [c_customer_sk#26, c_customer_id#28] -Arguments: [c_customer_sk#26, c_customer_id#28] - -(38) CometBroadcastHashJoin -Left output [1]: [ctr_customer_sk#9] -Right output [2]: [c_customer_sk#26, c_customer_id#28] -Arguments: [ctr_customer_sk#9], [c_customer_sk#26], Inner, BuildRight - -(39) CometProject -Input [3]: [ctr_customer_sk#9, c_customer_sk#26, c_customer_id#28] -Arguments: [c_customer_id#28], [c_customer_id#28] - -(40) CometTakeOrderedAndProject -Input [1]: [c_customer_id#28] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#28 ASC NULLS FIRST], output=[c_customer_id#28]), [c_customer_id#28], 100, 0, [c_customer_id#28 ASC NULLS FIRST], [c_customer_id#28] - -(41) CometColumnarToRow [codegen id : 1] -Input [1]: [c_customer_id#28] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) - - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(44) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(45) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(46) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 13 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/extended.txt deleted file mode 100644 index 8aa14c43dd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/extended.txt +++ /dev/null @@ -1,53 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 46 out of 49 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 3e1d6243f8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id] - CometProject [c_customer_id] - CometBroadcastHashJoin [ctr_customer_sk,c_customer_sk,c_customer_id] - CometProject [ctr_customer_sk] - CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,s_store_sk] - CometProject [ctr_customer_sk,ctr_store_sk] - CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_store_sk] - CometFilter [ctr_customer_sk,ctr_store_sk,ctr_total_return] - CometHashAggregate [sum] [ctr_customer_sk,ctr_store_sk,ctr_total_return,sr_customer_sk,sr_store_sk,sum(UnscaledValue(sr_return_amt))] - CometExchange [sr_customer_sk,sr_store_sk] #1 - CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] - CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_store_sk] #4 - CometFilter [(avg(ctr_total_return) * 1.2),ctr_store_sk] - CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_store_sk,avg(ctr_total_return)] - CometExchange [ctr_store_sk] #5 - CometHashAggregate [ctr_total_return] [ctr_store_sk,sum,count] - CometHashAggregate [sr_customer_sk,sum] [ctr_store_sk,ctr_total_return,sr_store_sk,sum(UnscaledValue(sr_return_amt))] - CometExchange [sr_customer_sk,sr_store_sk] #6 - CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] - CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - 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 [native_iceberg_compat] 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 - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [c_customer_sk,c_customer_id] #8 - CometProject [c_customer_id] [c_customer_sk,c_customer_id] - CometFilter [c_customer_sk,c_customer_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/explain.txt deleted file mode 100644 index 29034bd457..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/explain.txt +++ /dev/null @@ -1,295 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (45) -+- * HashAggregate (44) - +- * CometColumnarToRow (43) - +- CometColumnarExchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometNativeScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (38) - +- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometNativeScan parquet spark_catalog.default.customer_demographics (34) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(4) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] - -(6) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#9] - -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None - -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#6] -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] - -(9) BroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#6] -Join type: LeftSemi -Join condition: None - -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] - -(13) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#12] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#10] -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] - -(16) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(17) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#10] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] - -(20) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#15] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#13] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] - -(23) BroadcastExchange -Input [1]: [cs_ship_customer_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#13] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(25) Filter [codegen id : 9] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(26) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(27) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_county#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [ca_address_sk#16, ca_county#17] -Condition : (ca_county#17 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#16)) - -(29) CometProject -Input [2]: [ca_address_sk#16, ca_county#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(30) CometColumnarToRow [codegen id : 7] -Input [1]: [ca_address_sk#16] - -(31) BroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#16] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 9] -Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#16] - -(34) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(35) CometFilter -Input [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Condition : isnotnull(cd_demo_sk#18) - -(36) CometProject -Input [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#19, 1, true, false, true) AS cd_gender#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) AS cd_marital_status#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) AS cd_education_status#29, cd_purchase_estimate#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#23, 10, true, false, true) AS cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(37) CometColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(38) BroadcastExchange -Input [9]: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 9] -Output [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(41) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#31] -Results [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] - -(42) CometColumnarExchange -Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] -Arguments: hashpartitioning(cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) CometColumnarToRow [codegen id : 10] -Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] - -(44) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] -Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#33] -Results [14]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, count(1)#33 AS cnt1#34, cd_purchase_estimate#22, count(1)#33 AS cnt2#35, cd_credit_rating#30, count(1)#33 AS cnt3#36, cd_dep_count#24, count(1)#33 AS cnt4#37, cd_dep_employed_count#25, count(1)#33 AS cnt5#38, cd_dep_college_count#26, count(1)#33 AS cnt6#39] - -(45) TakeOrderedAndProject -Input [14]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cnt1#34, cd_purchase_estimate#22, cnt2#35, cd_credit_rating#30, cnt3#36, cd_dep_count#24, cnt4#37, cd_dep_employed_count#25, cnt5#38, cd_dep_college_count#26, cnt6#39] -Arguments: 100, [cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_education_status#29 ASC NULLS FIRST, cd_purchase_estimate#22 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [cd_gender#27, cd_marital_status#28, cd_education_status#29, cnt1#34, cd_purchase_estimate#22, cnt2#35, cd_credit_rating#30, cnt3#36, cd_dep_count#24, cnt4#37, cd_dep_employed_count#25, cnt5#38, cd_dep_college_count#26, cnt6#39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (50) -+- * CometColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometNativeScan parquet spark_catalog.default.date_dim (46) - - -(46) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#40, d_moy#41] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_date_sk#9, d_year#40, d_moy#41] -Condition : (((((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2002)) AND (d_moy#41 >= 1)) AND (d_moy#41 <= 4)) AND isnotnull(d_date_sk#9)) - -(48) CometProject -Input [3]: [d_date_sk#9, d_year#40, d_moy#41] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(49) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(50) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/simplified.txt deleted file mode 100644 index 0cc108b0b0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/simplified.txt +++ /dev/null @@ -1,77 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (10) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [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] - 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] - CometNativeScan 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/q10.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/explain.txt deleted file mode 100644 index 68bd670d8e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * Filter (27) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2002)) AND (d_moy#11 >= 1)) AND (d_moy#11 <= 4)) AND isnotnull(d_date_sk#9)) - -(6) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: [d_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#9] -Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -Arguments: [ss_customer_sk#6], [ss_customer_sk#6] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: [ss_customer_sk#6] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Right output [1]: [ss_customer_sk#6] -Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#15] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Right output [1]: [d_date_sk#15] -Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] -Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#12] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#12] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#19] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#16] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#16] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(27) Filter [codegen id : 5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(28) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_county#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [ca_address_sk#20, ca_county#21] -Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#20)) - -(31) CometProject -Input [2]: [ca_address_sk#20, ca_county#21] -Arguments: [ca_address_sk#20], [ca_address_sk#20] - -(32) CometColumnarToRow [codegen id : 3] -Input [1]: [ca_address_sk#20] - -(33) BroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(34) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 5] -Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(37) CometFilter -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Condition : isnotnull(cd_demo_sk#22) - -(38) CometProject -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_demo_sk#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#23, 1, true, false, true) AS cd_gender#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#24, 1, true, false, true) AS cd_marital_status#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#25, 20, true, false, true) AS cd_education_status#33, cd_purchase_estimate#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#27, 10, true, false, true) AS cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(39) CometColumnarToRow [codegen id : 4] -Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(40) BroadcastExchange -Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(41) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#22] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 5] -Output [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(43) HashAggregate [codegen id : 5] -Input [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#35] -Results [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] - -(44) CometColumnarExchange -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] -Arguments: hashpartitioning(cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 6] -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] - -(46) HashAggregate [codegen id : 6] -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] -Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#37] -Results [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, count(1)#37 AS cnt1#38, cd_purchase_estimate#26, count(1)#37 AS cnt2#39, cd_credit_rating#34, count(1)#37 AS cnt3#40, cd_dep_count#28, count(1)#37 AS cnt4#41, cd_dep_employed_count#29, count(1)#37 AS cnt5#42, cd_dep_college_count#30, count(1)#37 AS cnt6#43] - -(47) TakeOrderedAndProject -Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#38, cd_purchase_estimate#26, cnt2#39, cd_credit_rating#34, cnt3#40, cd_dep_count#28, cnt4#41, cd_dep_employed_count#29, cnt5#42, cd_dep_college_count#30, cnt6#43] -Arguments: 100, [cd_gender#31 ASC NULLS FIRST, cd_marital_status#32 ASC NULLS FIRST, cd_education_status#33 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#34 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#38, cd_purchase_estimate#26, cnt2#39, cd_credit_rating#34, cnt3#40, cd_dep_count#28, cnt4#41, cd_dep_employed_count#29, cnt5#42, cd_dep_college_count#30, cnt6#43] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2002)) AND (d_moy#11 >= 1)) AND (d_moy#11 <= 4)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(52) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/simplified.txt deleted file mode 100644 index e7193f87e1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (6) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [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] - 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 [native_iceberg_compat] 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/q10/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/explain.txt deleted file mode 100644 index 68bd670d8e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * Filter (27) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2002)) AND (d_moy#11 >= 1)) AND (d_moy#11 <= 4)) AND isnotnull(d_date_sk#9)) - -(6) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: [d_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#9] -Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -Arguments: [ss_customer_sk#6], [ss_customer_sk#6] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: [ss_customer_sk#6] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Right output [1]: [ss_customer_sk#6] -Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#15] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Right output [1]: [d_date_sk#15] -Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] -Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#12] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#12] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#19] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#16] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#16] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(27) Filter [codegen id : 5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(28) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_county#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [ca_address_sk#20, ca_county#21] -Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#20)) - -(31) CometProject -Input [2]: [ca_address_sk#20, ca_county#21] -Arguments: [ca_address_sk#20], [ca_address_sk#20] - -(32) CometColumnarToRow [codegen id : 3] -Input [1]: [ca_address_sk#20] - -(33) BroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(34) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 5] -Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(37) CometFilter -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Condition : isnotnull(cd_demo_sk#22) - -(38) CometProject -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_demo_sk#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#23, 1, true, false, true) AS cd_gender#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#24, 1, true, false, true) AS cd_marital_status#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#25, 20, true, false, true) AS cd_education_status#33, cd_purchase_estimate#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#27, 10, true, false, true) AS cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(39) CometColumnarToRow [codegen id : 4] -Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(40) BroadcastExchange -Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(41) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#22] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 5] -Output [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(43) HashAggregate [codegen id : 5] -Input [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#35] -Results [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] - -(44) CometColumnarExchange -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] -Arguments: hashpartitioning(cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 6] -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] - -(46) HashAggregate [codegen id : 6] -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] -Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#37] -Results [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, count(1)#37 AS cnt1#38, cd_purchase_estimate#26, count(1)#37 AS cnt2#39, cd_credit_rating#34, count(1)#37 AS cnt3#40, cd_dep_count#28, count(1)#37 AS cnt4#41, cd_dep_employed_count#29, count(1)#37 AS cnt5#42, cd_dep_college_count#30, count(1)#37 AS cnt6#43] - -(47) TakeOrderedAndProject -Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#38, cd_purchase_estimate#26, cnt2#39, cd_credit_rating#34, cnt3#40, cd_dep_count#28, cnt4#41, cd_dep_employed_count#29, cnt5#42, cd_dep_college_count#30, cnt6#43] -Arguments: 100, [cd_gender#31 ASC NULLS FIRST, cd_marital_status#32 ASC NULLS FIRST, cd_education_status#33 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#34 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#38, cd_purchase_estimate#26, cnt2#39, cd_credit_rating#34, cnt3#40, cd_dep_count#28, cnt4#41, cd_dep_employed_count#29, cnt5#42, cd_dep_college_count#30, cnt6#43] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2002)) AND (d_moy#11 >= 1)) AND (d_moy#11 <= 4)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(52) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/extended.txt deleted file mode 100644 index a6f33d6f7e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/extended.txt +++ /dev/null @@ -1,63 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index e7193f87e1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (6) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [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] - 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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/explain.txt deleted file mode 100644 index d697d27f18..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/explain.txt +++ /dev/null @@ -1,518 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (80) -+- * Project (79) - +- * BroadcastHashJoin Inner BuildRight (78) - :- * Project (59) - : +- * BroadcastHashJoin Inner BuildRight (58) - : :- * Project (38) - : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- * HashAggregate (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * Project (28) - : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (26) - : : : +- * Filter (25) - : : : +- * ColumnarToRow (24) - : : : +- Scan parquet spark_catalog.default.store_sales (23) - : : +- ReusedExchange (29) - : +- BroadcastExchange (57) - : +- * Filter (56) - : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) - : +- CometColumnarExchange (53) - : +- * HashAggregate (52) - : +- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Project (48) - : : +- * BroadcastHashJoin Inner BuildRight (47) - : : :- * CometColumnarToRow (42) - : : : +- CometProject (41) - : : : +- CometFilter (40) - : : : +- CometNativeScan parquet spark_catalog.default.customer (39) - : : +- BroadcastExchange (46) - : : +- * Filter (45) - : : +- * ColumnarToRow (44) - : : +- Scan parquet spark_catalog.default.web_sales (43) - : +- ReusedExchange (49) - +- BroadcastExchange (77) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- CometColumnarExchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * Project (69) - : +- * BroadcastHashJoin Inner BuildRight (68) - : :- * CometColumnarToRow (63) - : : +- CometProject (62) - : : +- CometFilter (61) - : : +- CometNativeScan parquet spark_catalog.default.customer (60) - : +- BroadcastExchange (67) - : +- * Filter (66) - : +- * ColumnarToRow (65) - : +- Scan parquet spark_catalog.default.web_sales (64) - +- ReusedExchange (70) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] - -(4) CometColumnarToRow [codegen id : 3] -Input [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] - -(5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(7) Filter [codegen id : 1] -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#15) - -(8) BroadcastExchange -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#15] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(11) ReusedExchange [Reuses operator id: 84] -Output [2]: [d_date_sk#20, d_year#21] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#18] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] - -(14) HashAggregate [codegen id : 3] -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] -Aggregate Attributes [1]: [sum#22] -Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] - -(15) CometColumnarExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 16] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] - -(17) HashAggregate [codegen id : 16] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24] -Results [2]: [c_customer_id#9 AS customer_id#25, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24,18,2) AS year_total#26] - -(18) Filter [codegen id : 16] -Input [2]: [customer_id#25, year_total#26] -Condition : (isnotnull(year_total#26) AND (year_total#26 > 0.00)) - -(19) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(20) CometFilter -Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] -Condition : (isnotnull(c_customer_sk#27) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#28, 16, true, false, true))) - -(21) CometProject -Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] -Arguments: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14], [c_customer_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#28, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#29, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#30, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#31, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#33, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#34, 50, true, false, true) AS c_email_address#14] - -(22) CometColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] - -(23) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_sold_date_sk#38 IN dynamicpruning#39)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] - -(25) Filter [codegen id : 4] -Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Condition : isnotnull(ss_customer_sk#35) - -(26) BroadcastExchange -Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#27] -Right keys [1]: [ss_customer_sk#35] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Input [12]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] - -(29) ReusedExchange [Reuses operator id: 88] -Output [2]: [d_date_sk#40, d_year#41] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38, d_date_sk#40, d_year#41] - -(32) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] -Aggregate Attributes [1]: [sum#42] -Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] - -(33) CometColumnarExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 7] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] - -(35) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))#24] -Results [3]: [c_customer_id#9 AS customer_id#44, c_preferred_cust_flag#12 AS customer_preferred_cust_flag#45, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))#24,18,2) AS year_total#46] - -(36) BroadcastExchange -Input [3]: [customer_id#44, customer_preferred_cust_flag#45, year_total#46] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(37) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#44] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 16] -Output [4]: [customer_id#25, year_total#26, customer_preferred_cust_flag#45, year_total#46] -Input [5]: [customer_id#25, year_total#26, customer_id#44, customer_preferred_cust_flag#45, year_total#46] - -(39) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [8]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54] -Condition : (isnotnull(c_customer_sk#47) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#48, 16, true, false, true))) - -(41) CometProject -Input [8]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54] -Arguments: [c_customer_sk#47, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60], [c_customer_sk#47, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#48, 16, true, false, true) AS c_customer_id#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#49, 20, true, false, true) AS c_first_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#50, 30, true, false, true) AS c_last_name#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#51, 1, true, false, true) AS c_preferred_cust_flag#58, c_birth_country#52, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#53, 13, true, false, true) AS c_login#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#54, 50, true, false, true) AS c_email_address#60] - -(42) CometColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#47, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60] - -(43) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#64), dynamicpruningexpression(ws_sold_date_sk#64 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(44) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64] - -(45) Filter [codegen id : 8] -Input [4]: [ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64] -Condition : isnotnull(ws_bill_customer_sk#61) - -(46) BroadcastExchange -Input [4]: [ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -(47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#47] -Right keys [1]: [ws_bill_customer_sk#61] -Join type: Inner -Join condition: None - -(48) Project [codegen id : 10] -Output [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64] -Input [12]: [c_customer_sk#47, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64] - -(49) ReusedExchange [Reuses operator id: 84] -Output [2]: [d_date_sk#65, d_year#66] - -(50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#64] -Right keys [1]: [d_date_sk#65] -Join type: Inner -Join condition: None - -(51) Project [codegen id : 10] -Output [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, d_year#66] -Input [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64, d_date_sk#65, d_year#66] - -(52) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, d_year#66] -Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#63 - ws_ext_discount_amt#62)))] -Aggregate Attributes [1]: [sum#67] -Results [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66, sum#68] - -(53) CometColumnarExchange -Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66, sum#68] -Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(54) CometColumnarToRow [codegen id : 11] -Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66, sum#68] - -(55) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66, sum#68] -Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#63 - ws_ext_discount_amt#62)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#63 - ws_ext_discount_amt#62)))#69] -Results [2]: [c_customer_id#55 AS customer_id#70, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#63 - ws_ext_discount_amt#62)))#69,18,2) AS year_total#71] - -(56) Filter [codegen id : 11] -Input [2]: [customer_id#70, year_total#71] -Condition : (isnotnull(year_total#71) AND (year_total#71 > 0.00)) - -(57) BroadcastExchange -Input [2]: [customer_id#70, year_total#71] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(58) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#70] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 16] -Output [5]: [customer_id#25, year_total#26, customer_preferred_cust_flag#45, year_total#46, year_total#71] -Input [6]: [customer_id#25, year_total#26, customer_preferred_cust_flag#45, year_total#46, customer_id#70, year_total#71] - -(60) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(61) CometFilter -Input [8]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79] -Condition : (isnotnull(c_customer_sk#72) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#73, 16, true, false, true))) - -(62) CometProject -Input [8]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79] -Arguments: [c_customer_sk#72, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60], [c_customer_sk#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#73, 16, true, false, true) AS c_customer_id#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#74, 20, true, false, true) AS c_first_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#75, 30, true, false, true) AS c_last_name#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#76, 1, true, false, true) AS c_preferred_cust_flag#58, c_birth_country#77, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#78, 13, true, false, true) AS c_login#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#79, 50, true, false, true) AS c_email_address#60] - -(63) CometColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#72, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60] - -(64) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#83), dynamicpruningexpression(ws_sold_date_sk#83 IN dynamicpruning#39)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(65) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] - -(66) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] -Condition : isnotnull(ws_bill_customer_sk#80) - -(67) BroadcastExchange -Input [4]: [ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(68) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#72] -Right keys [1]: [ws_bill_customer_sk#80] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 14] -Output [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] -Input [12]: [c_customer_sk#72, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] - -(70) ReusedExchange [Reuses operator id: 88] -Output [2]: [d_date_sk#84, d_year#85] - -(71) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#83] -Right keys [1]: [d_date_sk#84] -Join type: Inner -Join condition: None - -(72) Project [codegen id : 14] -Output [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, ws_ext_discount_amt#81, ws_ext_list_price#82, d_year#85] -Input [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83, d_date_sk#84, d_year#85] - -(73) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, ws_ext_discount_amt#81, ws_ext_list_price#82, d_year#85] -Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#82 - ws_ext_discount_amt#81)))] -Aggregate Attributes [1]: [sum#86] -Results [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85, sum#87] - -(74) CometColumnarExchange -Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85, sum#87] -Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(75) CometColumnarToRow [codegen id : 15] -Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85, sum#87] - -(76) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85, sum#87] -Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#82 - ws_ext_discount_amt#81)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#82 - ws_ext_discount_amt#81)))#69] -Results [2]: [c_customer_id#55 AS customer_id#88, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#82 - ws_ext_discount_amt#81)))#69,18,2) AS year_total#89] - -(77) BroadcastExchange -Input [2]: [customer_id#88, year_total#89] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] - -(78) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#88] -Join type: Inner -Join condition: (CASE WHEN (year_total#71 > 0.00) THEN (year_total#89 / year_total#71) END > CASE WHEN (year_total#26 > 0.00) THEN (year_total#46 / year_total#26) END) - -(79) Project [codegen id : 16] -Output [1]: [customer_preferred_cust_flag#45] -Input [7]: [customer_id#25, year_total#26, customer_preferred_cust_flag#45, year_total#46, year_total#71, customer_id#88, year_total#89] - -(80) TakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#45] -Arguments: 100, [customer_preferred_cust_flag#45 ASC NULLS FIRST], [customer_preferred_cust_flag#45] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (84) -+- * CometColumnarToRow (83) - +- CometFilter (82) - +- CometNativeScan parquet spark_catalog.default.date_dim (81) - - -(81) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(82) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(83) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#20, d_year#21] - -(84) BroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 -BroadcastExchange (88) -+- * CometColumnarToRow (87) - +- CometFilter (86) - +- CometNativeScan parquet spark_catalog.default.date_dim (85) - - -(85) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#40, d_year#41] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(86) CometFilter -Input [2]: [d_date_sk#40, d_year#41] -Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#40)) - -(87) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#40, d_year#41] - -(88) BroadcastExchange -Input [2]: [d_date_sk#40, d_year#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] - -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#64 IN dynamicpruning#19 - -Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#83 IN dynamicpruning#39 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/simplified.txt deleted file mode 100644 index 6c9e276c01..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/simplified.txt +++ /dev/null @@ -1,131 +0,0 @@ -TakeOrderedAndProject [customer_preferred_cust_flag] - WholeStageCodegen (16) - Project [customer_preferred_cust_flag] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - Project [customer_id,year_total,customer_preferred_cust_flag,year_total] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_preferred_cust_flag,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - WholeStageCodegen (6) - HashAggregate [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] [sum,sum] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) - HashAggregate [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] [sum,sum] - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) - HashAggregate [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] [sum,sum] - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/explain.txt deleted file mode 100644 index 091ad62567..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,475 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (73) -+- CometTakeOrderedAndProject (72) - +- CometProject (71) - +- CometBroadcastHashJoin (70) - :- CometProject (54) - : +- CometBroadcastHashJoin (53) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (52) - : +- CometFilter (51) - : +- CometHashAggregate (50) - : +- CometExchange (49) - : +- CometHashAggregate (48) - : +- CometProject (47) - : +- CometBroadcastHashJoin (46) - : :- CometProject (44) - : : +- CometBroadcastHashJoin (43) - : : :- CometProject (39) - : : : +- CometFilter (38) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) - : : +- CometBroadcastExchange (42) - : : +- CometFilter (41) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) - : +- ReusedExchange (45) - +- CometBroadcastExchange (69) - +- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometProject (65) - +- CometBroadcastHashJoin (64) - :- CometProject (62) - : +- CometBroadcastHashJoin (61) - : :- CometProject (57) - : : +- CometFilter (56) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (55) - : +- CometBroadcastExchange (60) - : +- CometFilter (59) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (58) - +- ReusedExchange (63) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#15) - -(6) CometBroadcastExchange -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(7) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_sk#1], [ss_customer_sk#15], Inner, BuildRight - -(8) CometProject -Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: [d_date_sk#20, d_year#21] - -(12) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Right output [2]: [d_date_sk#20, d_year#21] -Arguments: [ss_sold_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(13) CometProject -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] - -(14) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(15) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(17) CometFilter -Input [2]: [customer_id#23, year_total#24] -Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true))) - -(20) CometProject -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Arguments: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14], [c_customer_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#29, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#31, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#32, 50, true, false, true) AS c_email_address#14] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#36), dynamicpruningexpression(ss_sold_date_sk#36 IN dynamicpruning#37)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Condition : isnotnull(ss_customer_sk#33) - -(23) CometBroadcastExchange -Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] - -(24) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [c_customer_sk#25], [ss_customer_sk#33], Inner, BuildRight - -(25) CometProject -Input [12]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: [d_date_sk#38, d_year#39] - -(29) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Right output [2]: [d_date_sk#38, d_year#39] -Arguments: [ss_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight - -(30) CometProject -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36, d_date_sk#38, d_year#39] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] - -(31) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] - -(32) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] - -(34) CometBroadcastExchange -Input [3]: [customer_id#41, customer_preferred_cust_flag#42, year_total#43] -Arguments: [customer_id#41, customer_preferred_cust_flag#42, year_total#43] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#23, year_total#24] -Right output [3]: [customer_id#41, customer_preferred_cust_flag#42, year_total#43] -Arguments: [customer_id#23], [customer_id#41], Inner, BuildRight - -(36) CometProject -Input [5]: [customer_id#23, year_total#24, customer_id#41, customer_preferred_cust_flag#42, year_total#43] -Arguments: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43], [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47, c_preferred_cust_flag#48, c_birth_country#49, c_login#50, c_email_address#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(38) CometFilter -Input [8]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47, c_preferred_cust_flag#48, c_birth_country#49, c_login#50, c_email_address#51] -Condition : (isnotnull(c_customer_sk#44) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#45, 16, true, false, true))) - -(39) CometProject -Input [8]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47, c_preferred_cust_flag#48, c_birth_country#49, c_login#50, c_email_address#51] -Arguments: [c_customer_sk#44, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57], [c_customer_sk#44, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#45, 16, true, false, true) AS c_customer_id#52, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#46, 20, true, false, true) AS c_first_name#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#47, 30, true, false, true) AS c_last_name#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#48, 1, true, false, true) AS c_preferred_cust_flag#55, c_birth_country#49, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#50, 13, true, false, true) AS c_login#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#51, 50, true, false, true) AS c_email_address#57] - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#61), dynamicpruningexpression(ws_sold_date_sk#61 IN dynamicpruning#62)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(41) CometFilter -Input [4]: [ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Condition : isnotnull(ws_bill_customer_sk#58) - -(42) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Arguments: [ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] - -(43) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#44, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57] -Right output [4]: [ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Arguments: [c_customer_sk#44], [ws_bill_customer_sk#58], Inner, BuildRight - -(44) CometProject -Input [12]: [c_customer_sk#44, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Arguments: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61], [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] - -(45) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#63, d_year#64] - -(46) CometBroadcastHashJoin -Left output [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Right output [2]: [d_date_sk#63, d_year#64] -Arguments: [ws_sold_date_sk#61], [d_date_sk#63], Inner, BuildRight - -(47) CometProject -Input [12]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61, d_date_sk#63, d_year#64] -Arguments: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, d_year#64], [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, d_year#64] - -(48) CometHashAggregate -Input [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, d_year#64] -Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, d_year#64] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#60 - ws_ext_discount_amt#59)))] - -(49) CometExchange -Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, d_year#64, sum#65] -Arguments: hashpartitioning(c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, d_year#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(50) CometHashAggregate -Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, d_year#64, sum#65] -Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, d_year#64] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#60 - ws_ext_discount_amt#59)))] - -(51) CometFilter -Input [2]: [customer_id#66, year_total#67] -Condition : (isnotnull(year_total#67) AND (year_total#67 > 0.00)) - -(52) CometBroadcastExchange -Input [2]: [customer_id#66, year_total#67] -Arguments: [customer_id#66, year_total#67] - -(53) CometBroadcastHashJoin -Left output [4]: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43] -Right output [2]: [customer_id#66, year_total#67] -Arguments: [customer_id#23], [customer_id#66], Inner, BuildRight - -(54) CometProject -Input [6]: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43, customer_id#66, year_total#67] -Arguments: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43, year_total#67], [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43, year_total#67] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#68, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(56) CometFilter -Input [8]: [c_customer_sk#68, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75] -Condition : (isnotnull(c_customer_sk#68) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#69, 16, true, false, true))) - -(57) CometProject -Input [8]: [c_customer_sk#68, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75] -Arguments: [c_customer_sk#68, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57], [c_customer_sk#68, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#69, 16, true, false, true) AS c_customer_id#52, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#70, 20, true, false, true) AS c_first_name#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#71, 30, true, false, true) AS c_last_name#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#72, 1, true, false, true) AS c_preferred_cust_flag#55, c_birth_country#73, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#74, 13, true, false, true) AS c_login#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#75, 50, true, false, true) AS c_email_address#57] - -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#80)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(59) CometFilter -Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Condition : isnotnull(ws_bill_customer_sk#76) - -(60) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Arguments: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] - -(61) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#68, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57] -Right output [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Arguments: [c_customer_sk#68], [ws_bill_customer_sk#76], Inner, BuildRight - -(62) CometProject -Input [12]: [c_customer_sk#68, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Arguments: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79], [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] - -(63) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#81, d_year#82] - -(64) CometBroadcastHashJoin -Left output [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Right output [2]: [d_date_sk#81, d_year#82] -Arguments: [ws_sold_date_sk#79], [d_date_sk#81], Inner, BuildRight - -(65) CometProject -Input [12]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79, d_date_sk#81, d_year#82] -Arguments: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#82], [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#82] - -(66) CometHashAggregate -Input [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#82] -Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, d_year#82] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#78 - ws_ext_discount_amt#77)))] - -(67) CometExchange -Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, d_year#82, sum#83] -Arguments: hashpartitioning(c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, d_year#82, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(68) CometHashAggregate -Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, d_year#82, sum#83] -Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, d_year#82] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#78 - ws_ext_discount_amt#77)))] - -(69) CometBroadcastExchange -Input [2]: [customer_id#84, year_total#85] -Arguments: [customer_id#84, year_total#85] - -(70) CometBroadcastHashJoin -Left output [5]: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43, year_total#67] -Right output [2]: [customer_id#84, year_total#85] -Arguments: [customer_id#23], [customer_id#84], Inner, (CASE WHEN (year_total#67 > 0.00) THEN (year_total#85 / year_total#67) END > CASE WHEN (year_total#24 > 0.00) THEN (year_total#43 / year_total#24) END), BuildRight - -(71) CometProject -Input [7]: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43, year_total#67, customer_id#84, year_total#85] -Arguments: [customer_preferred_cust_flag#42], [customer_preferred_cust_flag#42] - -(72) CometTakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#42] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_preferred_cust_flag#42 ASC NULLS FIRST], output=[customer_preferred_cust_flag#42]), [customer_preferred_cust_flag#42], 100, 0, [customer_preferred_cust_flag#42 ASC NULLS FIRST], [customer_preferred_cust_flag#42] - -(73) CometColumnarToRow [codegen id : 1] -Input [1]: [customer_preferred_cust_flag#42] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (77) -+- * CometColumnarToRow (76) - +- CometFilter (75) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (74) - - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(75) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(76) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#20, d_year#21] - -(77) BroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#36 IN dynamicpruning#37 -BroadcastExchange (81) -+- * CometColumnarToRow (80) - +- CometFilter (79) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (78) - - -(78) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(79) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) - -(80) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#38, d_year#39] - -(81) BroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#61 IN dynamicpruning#19 - -Subquery:4 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#37 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/simplified.txt deleted file mode 100644 index 4599ecf539..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,91 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_preferred_cust_flag] - CometProject [customer_preferred_cust_flag] - CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_preferred_cust_flag,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_preferred_cust_flag,year_total] #5 - CometHashAggregate [c_first_name,c_last_name,d_year,c_birth_country,c_login,c_email_address,sum] [customer_id,customer_preferred_cust_flag,year_total,c_customer_id,c_preferred_cust_flag,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/explain.txt deleted file mode 100644 index 091ad62567..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/explain.txt +++ /dev/null @@ -1,475 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (73) -+- CometTakeOrderedAndProject (72) - +- CometProject (71) - +- CometBroadcastHashJoin (70) - :- CometProject (54) - : +- CometBroadcastHashJoin (53) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (52) - : +- CometFilter (51) - : +- CometHashAggregate (50) - : +- CometExchange (49) - : +- CometHashAggregate (48) - : +- CometProject (47) - : +- CometBroadcastHashJoin (46) - : :- CometProject (44) - : : +- CometBroadcastHashJoin (43) - : : :- CometProject (39) - : : : +- CometFilter (38) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) - : : +- CometBroadcastExchange (42) - : : +- CometFilter (41) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) - : +- ReusedExchange (45) - +- CometBroadcastExchange (69) - +- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometProject (65) - +- CometBroadcastHashJoin (64) - :- CometProject (62) - : +- CometBroadcastHashJoin (61) - : :- CometProject (57) - : : +- CometFilter (56) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (55) - : +- CometBroadcastExchange (60) - : +- CometFilter (59) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (58) - +- ReusedExchange (63) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#15) - -(6) CometBroadcastExchange -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(7) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_sk#1], [ss_customer_sk#15], Inner, BuildRight - -(8) CometProject -Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: [d_date_sk#20, d_year#21] - -(12) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Right output [2]: [d_date_sk#20, d_year#21] -Arguments: [ss_sold_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(13) CometProject -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] - -(14) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(15) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(17) CometFilter -Input [2]: [customer_id#23, year_total#24] -Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true))) - -(20) CometProject -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Arguments: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14], [c_customer_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#29, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#31, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#32, 50, true, false, true) AS c_email_address#14] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#36), dynamicpruningexpression(ss_sold_date_sk#36 IN dynamicpruning#37)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Condition : isnotnull(ss_customer_sk#33) - -(23) CometBroadcastExchange -Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] - -(24) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [c_customer_sk#25], [ss_customer_sk#33], Inner, BuildRight - -(25) CometProject -Input [12]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: [d_date_sk#38, d_year#39] - -(29) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Right output [2]: [d_date_sk#38, d_year#39] -Arguments: [ss_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight - -(30) CometProject -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36, d_date_sk#38, d_year#39] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] - -(31) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] - -(32) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] - -(34) CometBroadcastExchange -Input [3]: [customer_id#41, customer_preferred_cust_flag#42, year_total#43] -Arguments: [customer_id#41, customer_preferred_cust_flag#42, year_total#43] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#23, year_total#24] -Right output [3]: [customer_id#41, customer_preferred_cust_flag#42, year_total#43] -Arguments: [customer_id#23], [customer_id#41], Inner, BuildRight - -(36) CometProject -Input [5]: [customer_id#23, year_total#24, customer_id#41, customer_preferred_cust_flag#42, year_total#43] -Arguments: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43], [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47, c_preferred_cust_flag#48, c_birth_country#49, c_login#50, c_email_address#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(38) CometFilter -Input [8]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47, c_preferred_cust_flag#48, c_birth_country#49, c_login#50, c_email_address#51] -Condition : (isnotnull(c_customer_sk#44) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#45, 16, true, false, true))) - -(39) CometProject -Input [8]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47, c_preferred_cust_flag#48, c_birth_country#49, c_login#50, c_email_address#51] -Arguments: [c_customer_sk#44, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57], [c_customer_sk#44, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#45, 16, true, false, true) AS c_customer_id#52, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#46, 20, true, false, true) AS c_first_name#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#47, 30, true, false, true) AS c_last_name#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#48, 1, true, false, true) AS c_preferred_cust_flag#55, c_birth_country#49, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#50, 13, true, false, true) AS c_login#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#51, 50, true, false, true) AS c_email_address#57] - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#61), dynamicpruningexpression(ws_sold_date_sk#61 IN dynamicpruning#62)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(41) CometFilter -Input [4]: [ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Condition : isnotnull(ws_bill_customer_sk#58) - -(42) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Arguments: [ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] - -(43) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#44, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57] -Right output [4]: [ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Arguments: [c_customer_sk#44], [ws_bill_customer_sk#58], Inner, BuildRight - -(44) CometProject -Input [12]: [c_customer_sk#44, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Arguments: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61], [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] - -(45) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#63, d_year#64] - -(46) CometBroadcastHashJoin -Left output [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Right output [2]: [d_date_sk#63, d_year#64] -Arguments: [ws_sold_date_sk#61], [d_date_sk#63], Inner, BuildRight - -(47) CometProject -Input [12]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61, d_date_sk#63, d_year#64] -Arguments: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, d_year#64], [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, d_year#64] - -(48) CometHashAggregate -Input [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, d_year#64] -Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, d_year#64] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#60 - ws_ext_discount_amt#59)))] - -(49) CometExchange -Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, d_year#64, sum#65] -Arguments: hashpartitioning(c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, d_year#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(50) CometHashAggregate -Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, d_year#64, sum#65] -Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, d_year#64] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#60 - ws_ext_discount_amt#59)))] - -(51) CometFilter -Input [2]: [customer_id#66, year_total#67] -Condition : (isnotnull(year_total#67) AND (year_total#67 > 0.00)) - -(52) CometBroadcastExchange -Input [2]: [customer_id#66, year_total#67] -Arguments: [customer_id#66, year_total#67] - -(53) CometBroadcastHashJoin -Left output [4]: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43] -Right output [2]: [customer_id#66, year_total#67] -Arguments: [customer_id#23], [customer_id#66], Inner, BuildRight - -(54) CometProject -Input [6]: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43, customer_id#66, year_total#67] -Arguments: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43, year_total#67], [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43, year_total#67] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#68, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(56) CometFilter -Input [8]: [c_customer_sk#68, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75] -Condition : (isnotnull(c_customer_sk#68) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#69, 16, true, false, true))) - -(57) CometProject -Input [8]: [c_customer_sk#68, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75] -Arguments: [c_customer_sk#68, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57], [c_customer_sk#68, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#69, 16, true, false, true) AS c_customer_id#52, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#70, 20, true, false, true) AS c_first_name#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#71, 30, true, false, true) AS c_last_name#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#72, 1, true, false, true) AS c_preferred_cust_flag#55, c_birth_country#73, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#74, 13, true, false, true) AS c_login#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#75, 50, true, false, true) AS c_email_address#57] - -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#80)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(59) CometFilter -Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Condition : isnotnull(ws_bill_customer_sk#76) - -(60) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Arguments: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] - -(61) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#68, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57] -Right output [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Arguments: [c_customer_sk#68], [ws_bill_customer_sk#76], Inner, BuildRight - -(62) CometProject -Input [12]: [c_customer_sk#68, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Arguments: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79], [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] - -(63) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#81, d_year#82] - -(64) CometBroadcastHashJoin -Left output [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Right output [2]: [d_date_sk#81, d_year#82] -Arguments: [ws_sold_date_sk#79], [d_date_sk#81], Inner, BuildRight - -(65) CometProject -Input [12]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79, d_date_sk#81, d_year#82] -Arguments: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#82], [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#82] - -(66) CometHashAggregate -Input [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#82] -Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, d_year#82] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#78 - ws_ext_discount_amt#77)))] - -(67) CometExchange -Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, d_year#82, sum#83] -Arguments: hashpartitioning(c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, d_year#82, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(68) CometHashAggregate -Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, d_year#82, sum#83] -Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, d_year#82] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#78 - ws_ext_discount_amt#77)))] - -(69) CometBroadcastExchange -Input [2]: [customer_id#84, year_total#85] -Arguments: [customer_id#84, year_total#85] - -(70) CometBroadcastHashJoin -Left output [5]: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43, year_total#67] -Right output [2]: [customer_id#84, year_total#85] -Arguments: [customer_id#23], [customer_id#84], Inner, (CASE WHEN (year_total#67 > 0.00) THEN (year_total#85 / year_total#67) END > CASE WHEN (year_total#24 > 0.00) THEN (year_total#43 / year_total#24) END), BuildRight - -(71) CometProject -Input [7]: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43, year_total#67, customer_id#84, year_total#85] -Arguments: [customer_preferred_cust_flag#42], [customer_preferred_cust_flag#42] - -(72) CometTakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#42] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_preferred_cust_flag#42 ASC NULLS FIRST], output=[customer_preferred_cust_flag#42]), [customer_preferred_cust_flag#42], 100, 0, [customer_preferred_cust_flag#42 ASC NULLS FIRST], [customer_preferred_cust_flag#42] - -(73) CometColumnarToRow [codegen id : 1] -Input [1]: [customer_preferred_cust_flag#42] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (77) -+- * CometColumnarToRow (76) - +- CometFilter (75) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (74) - - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(75) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(76) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#20, d_year#21] - -(77) BroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#36 IN dynamicpruning#37 -BroadcastExchange (81) -+- * CometColumnarToRow (80) - +- CometFilter (79) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (78) - - -(78) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(79) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) - -(80) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#38, d_year#39] - -(81) BroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#61 IN dynamicpruning#19 - -Subquery:4 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#37 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/extended.txt deleted file mode 100644 index 781eae9054..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/extended.txt +++ /dev/null @@ -1,91 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 80 out of 86 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 4599ecf539..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11/simplified.txt +++ /dev/null @@ -1,91 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_preferred_cust_flag] - CometProject [customer_preferred_cust_flag] - CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_preferred_cust_flag,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_preferred_cust_flag,year_total] #5 - CometHashAggregate [c_first_name,c_last_name,d_year,c_birth_country,c_login,c_email_address,sum] [customer_id,customer_preferred_cust_flag,year_total,c_customer_id,c_preferred_cust_flag,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/explain.txt deleted file mode 100644 index 19d16c378b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/explain.txt +++ /dev/null @@ -1,163 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.web_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.item (4) - +- ReusedExchange (11) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(7) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(8) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(11) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] - -(14) HashAggregate [codegen id : 3] -Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(15) CometColumnarExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(17) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#17] -Results [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#17,17,2) AS _w0#19, i_item_id#11] - -(18) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(22) Project [codegen id : 6] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] - -(23) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometNativeScan parquet spark_catalog.default.date_dim (24) - - -(24) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#22] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/simplified.txt deleted file mode 100644 index b7b0a89774..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/explain.txt deleted file mode 100644 index 93308784a9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ws_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/simplified.txt deleted file mode 100644 index fb83fd2f9a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/explain.txt deleted file mode 100644 index 93308784a9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ws_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/extended.txt deleted file mode 100644 index 3f41c97ff5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index fb83fd2f9a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/explain.txt deleted file mode 100644 index dbacf525ef..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/explain.txt +++ /dev/null @@ -1,241 +0,0 @@ -== Physical Plan == -* HashAggregate (36) -+- * CometColumnarToRow (35) - +- CometColumnarExchange (34) - +- * HashAggregate (33) - +- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * CometColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.store (4) - : : : +- BroadcastExchange (14) - : : : +- * CometColumnarToRow (13) - : : : +- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (10) - : : +- ReusedExchange (17) - : +- BroadcastExchange (24) - : +- * CometColumnarToRow (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometNativeScan parquet spark_catalog.default.customer_demographics (20) - +- BroadcastExchange (30) - +- * CometColumnarToRow (29) - +- CometFilter (28) - +- CometNativeScan parquet spark_catalog.default.household_demographics (27) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00))), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00)))] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] - -(3) Filter [codegen id : 6] -Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Condition : (((((isnotnull(ss_store_sk#4) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_hdemo_sk#2)) AND ((((ss_net_profit#9 >= 100.00) AND (ss_net_profit#9 <= 200.00)) OR ((ss_net_profit#9 >= 150.00) AND (ss_net_profit#9 <= 300.00))) OR ((ss_net_profit#9 >= 50.00) AND (ss_net_profit#9 <= 250.00)))) AND ((((ss_sales_price#6 >= 100.00) AND (ss_sales_price#6 <= 150.00)) OR ((ss_sales_price#6 >= 50.00) AND (ss_sales_price#6 <= 100.00))) OR ((ss_sales_price#6 >= 150.00) AND (ss_sales_price#6 <= 200.00)))) - -(4) CometNativeScan parquet spark_catalog.default.store -Output [1]: [s_store_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [1]: [s_store_sk#12] -Condition : isnotnull(s_store_sk#12) - -(6) CometColumnarToRow [codegen id : 1] -Input [1]: [s_store_sk#12] - -(7) BroadcastExchange -Input [1]: [s_store_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#12] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 6] -Output [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, s_store_sk#12] - -(10) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Condition : (((isnotnull(ca_country#15) AND (ca_country#15 = United States)) AND isnotnull(ca_address_sk#13)) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) IN (TX,OH) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) IN (OR,NM,KY)) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) IN (VA,TX,MS))) - -(12) CometProject -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Arguments: [ca_address_sk#13, ca_state#16], [ca_address_sk#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) AS ca_state#16] - -(13) CometColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#13, ca_state#16] - -(14) BroadcastExchange -Input [2]: [ca_address_sk#13, ca_state#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#13] -Join type: Inner -Join condition: ((((ca_state#16 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#16 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#16 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))) - -(16) Project [codegen id : 6] -Output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] -Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#13, ca_state#16] - -(17) ReusedExchange [Reuses operator id: 41] -Output [1]: [d_date_sk#17] - -(18) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#10] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 6] -Output [6]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10, d_date_sk#17] - -(20) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] -Condition : (isnotnull(cd_demo_sk#18) AND ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#20, 20, true, false, true) = Advanced Degree )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = S) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#20, 20, true, false, true) = College ))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = W) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#20, 20, true, false, true) = 2 yr Degree )))) - -(22) CometProject -Input [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] -Arguments: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22], [cd_demo_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) AS cd_marital_status#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#20, 20, true, false, true) AS cd_education_status#22] - -(23) CometColumnarToRow [codegen id : 4] -Input [3]: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] - -(24) BroadcastExchange -Input [3]: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_cdemo_sk#1] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: ((((((cd_marital_status#21 = M) AND (cd_education_status#22 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#21 = S) AND (cd_education_status#22 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#21 = W) AND (cd_education_status#22 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) - -(26) Project [codegen id : 6] -Output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#21, cd_education_status#22] -Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] - -(27) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#23, hd_dep_count#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), Or(EqualTo(hd_dep_count,3),EqualTo(hd_dep_count,1))] -ReadSchema: struct - -(28) CometFilter -Input [2]: [hd_demo_sk#23, hd_dep_count#24] -Condition : (isnotnull(hd_demo_sk#23) AND ((hd_dep_count#24 = 3) OR (hd_dep_count#24 = 1))) - -(29) CometColumnarToRow [codegen id : 5] -Input [2]: [hd_demo_sk#23, hd_dep_count#24] - -(30) BroadcastExchange -Input [2]: [hd_demo_sk#23, hd_dep_count#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#23] -Join type: Inner -Join condition: (((((((cd_marital_status#21 = M) AND (cd_education_status#22 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#24 = 3)) OR (((((cd_marital_status#21 = S) AND (cd_education_status#22 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#24 = 1))) OR (((((cd_marital_status#21 = W) AND (cd_education_status#22 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#24 = 1))) - -(32) Project [codegen id : 6] -Output [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#21, cd_education_status#22, hd_demo_sk#23, hd_dep_count#24] - -(33) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Keys: [] -Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sales_price#7)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#8)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#8))] -Aggregate Attributes [7]: [sum#25, count#26, sum#27, count#28, sum#29, count#30, sum#31] -Results [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] - -(34) CometColumnarExchange -Input [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(35) CometColumnarToRow [codegen id : 7] -Input [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] - -(36) HashAggregate [codegen id : 7] -Input [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] -Keys: [] -Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] -Aggregate Attributes [4]: [avg(ss_quantity#5)#39, avg(UnscaledValue(ss_ext_sales_price#7))#40, avg(UnscaledValue(ss_ext_wholesale_cost#8))#41, sum(UnscaledValue(ss_ext_wholesale_cost#8))#42] -Results [4]: [avg(ss_quantity#5)#39 AS avg(ss_quantity)#43, cast((avg(UnscaledValue(ss_ext_sales_price#7))#40 / 100.0) as decimal(11,6)) AS avg(ss_ext_sales_price)#44, cast((avg(UnscaledValue(ss_ext_wholesale_cost#8))#41 / 100.0) as decimal(11,6)) AS avg(ss_ext_wholesale_cost)#45, MakeDecimal(sum(UnscaledValue(ss_ext_wholesale_cost#8))#42,17,2) AS sum(ss_ext_wholesale_cost)#46] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (41) -+- * CometColumnarToRow (40) - +- CometProject (39) - +- CometFilter (38) - +- CometNativeScan parquet spark_catalog.default.date_dim (37) - - -(37) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_year#47] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(38) CometFilter -Input [2]: [d_date_sk#17, d_year#47] -Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2001)) AND isnotnull(d_date_sk#17)) - -(39) CometProject -Input [2]: [d_date_sk#17, d_year#47] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(40) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] - -(41) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/simplified.txt deleted file mode 100644 index a33ae5a161..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (7) - HashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost)),avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (6) - HashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum,sum,count,sum,count,sum,count,sum] - Project [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] - Project [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] - Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] - Project [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] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometFilter [hd_demo_sk,hd_dep_count] - CometNativeScan 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/q13.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/explain.txt deleted file mode 100644 index 74da8ba883..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,231 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (34) -+- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (7) - : : : : +- CometBroadcastHashJoin (6) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (20) - +- CometBroadcastExchange (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00))), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00)))] -ReadSchema: struct - -(2) CometFilter -Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Condition : (((((isnotnull(ss_store_sk#4) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_hdemo_sk#2)) AND ((((ss_net_profit#9 >= 100.00) AND (ss_net_profit#9 <= 200.00)) OR ((ss_net_profit#9 >= 150.00) AND (ss_net_profit#9 <= 300.00))) OR ((ss_net_profit#9 >= 50.00) AND (ss_net_profit#9 <= 250.00)))) AND ((((ss_sales_price#6 >= 100.00) AND (ss_sales_price#6 <= 150.00)) OR ((ss_sales_price#6 >= 50.00) AND (ss_sales_price#6 <= 100.00))) OR ((ss_sales_price#6 >= 150.00) AND (ss_sales_price#6 <= 200.00)))) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [s_store_sk#12] -Condition : isnotnull(s_store_sk#12) - -(5) CometBroadcastExchange -Input [1]: [s_store_sk#12] -Arguments: [s_store_sk#12] - -(6) CometBroadcastHashJoin -Left output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Right output [1]: [s_store_sk#12] -Arguments: [ss_store_sk#4], [s_store_sk#12], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, s_store_sk#12] -Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Condition : (((isnotnull(ca_country#15) AND (ca_country#15 = United States)) AND isnotnull(ca_address_sk#13)) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) IN (TX,OH) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) IN (OR,NM,KY)) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) IN (VA,TX,MS))) - -(10) CometProject -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Arguments: [ca_address_sk#13, ca_state#16], [ca_address_sk#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) AS ca_state#16] - -(11) CometBroadcastExchange -Input [2]: [ca_address_sk#13, ca_state#16] -Arguments: [ca_address_sk#13, ca_state#16] - -(12) CometBroadcastHashJoin -Left output [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Right output [2]: [ca_address_sk#13, ca_state#16] -Arguments: [ss_addr_sk#3], [ca_address_sk#13], Inner, ((((ca_state#16 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#16 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#16 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))), BuildRight - -(13) CometProject -Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#13, ca_state#16] -Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_year#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#17, d_year#18] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(16) CometProject -Input [2]: [d_date_sk#17, d_year#18] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: [d_date_sk#17] - -(18) CometBroadcastHashJoin -Left output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] -Right output [1]: [d_date_sk#17] -Arguments: [ss_sold_date_sk#10], [d_date_sk#17], Inner, BuildRight - -(19) CometProject -Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10, d_date_sk#17] -Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Condition : (isnotnull(cd_demo_sk#19) AND ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = Advanced Degree )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = S) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = College ))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = W) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = 2 yr Degree )))) - -(22) CometProject -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) AS cd_marital_status#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) AS cd_education_status#23] - -(23) CometBroadcastExchange -Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] - -(24) CometBroadcastHashJoin -Left output [6]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Right output [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [ss_cdemo_sk#1], [cd_demo_sk#19], Inner, ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))), BuildRight - -(25) CometProject -Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23], [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#24, hd_dep_count#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), Or(EqualTo(hd_dep_count,3),EqualTo(hd_dep_count,1))] -ReadSchema: struct - -(27) CometFilter -Input [2]: [hd_demo_sk#24, hd_dep_count#25] -Condition : (isnotnull(hd_demo_sk#24) AND ((hd_dep_count#25 = 3) OR (hd_dep_count#25 = 1))) - -(28) CometBroadcastExchange -Input [2]: [hd_demo_sk#24, hd_dep_count#25] -Arguments: [hd_demo_sk#24, hd_dep_count#25] - -(29) CometBroadcastHashJoin -Left output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23] -Right output [2]: [hd_demo_sk#24, hd_dep_count#25] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#24], Inner, (((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#25 = 3)) OR (((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#25 = 1))) OR (((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#25 = 1))), BuildRight - -(30) CometProject -Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23, hd_demo_sk#24, hd_dep_count#25] -Arguments: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] - -(31) CometHashAggregate -Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Keys: [] -Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sales_price#7)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#8)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#8))] - -(32) CometExchange -Input [7]: [sum#26, count#27, sum#28, count#29, sum#30, count#31, sum#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(33) CometHashAggregate -Input [7]: [sum#26, count#27, sum#28, count#29, sum#30, count#31, sum#32] -Keys: [] -Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] - -(34) CometColumnarToRow [codegen id : 1] -Input [4]: [avg(ss_quantity)#33, avg(ss_ext_sales_price)#34, avg(ss_ext_wholesale_cost)#35, sum(ss_ext_wholesale_cost)#36] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (39) -+- * CometColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (35) - - -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_year#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(36) CometFilter -Input [2]: [d_date_sk#17, d_year#18] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(37) CometProject -Input [2]: [d_date_sk#17, d_year#18] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(38) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] - -(39) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/simplified.txt deleted file mode 100644 index 8ef882a435..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,44 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost))] - CometExchange #1 - CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum] - CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - 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,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_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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk] #3 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [hd_demo_sk,hd_dep_count] #7 - CometFilter [hd_demo_sk,hd_dep_count] - CometScan [native_iceberg_compat] 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/q13/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/explain.txt deleted file mode 100644 index 74da8ba883..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/explain.txt +++ /dev/null @@ -1,231 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (34) -+- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (7) - : : : : +- CometBroadcastHashJoin (6) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (20) - +- CometBroadcastExchange (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00))), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00)))] -ReadSchema: struct - -(2) CometFilter -Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Condition : (((((isnotnull(ss_store_sk#4) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_hdemo_sk#2)) AND ((((ss_net_profit#9 >= 100.00) AND (ss_net_profit#9 <= 200.00)) OR ((ss_net_profit#9 >= 150.00) AND (ss_net_profit#9 <= 300.00))) OR ((ss_net_profit#9 >= 50.00) AND (ss_net_profit#9 <= 250.00)))) AND ((((ss_sales_price#6 >= 100.00) AND (ss_sales_price#6 <= 150.00)) OR ((ss_sales_price#6 >= 50.00) AND (ss_sales_price#6 <= 100.00))) OR ((ss_sales_price#6 >= 150.00) AND (ss_sales_price#6 <= 200.00)))) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [s_store_sk#12] -Condition : isnotnull(s_store_sk#12) - -(5) CometBroadcastExchange -Input [1]: [s_store_sk#12] -Arguments: [s_store_sk#12] - -(6) CometBroadcastHashJoin -Left output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Right output [1]: [s_store_sk#12] -Arguments: [ss_store_sk#4], [s_store_sk#12], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, s_store_sk#12] -Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Condition : (((isnotnull(ca_country#15) AND (ca_country#15 = United States)) AND isnotnull(ca_address_sk#13)) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) IN (TX,OH) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) IN (OR,NM,KY)) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) IN (VA,TX,MS))) - -(10) CometProject -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Arguments: [ca_address_sk#13, ca_state#16], [ca_address_sk#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) AS ca_state#16] - -(11) CometBroadcastExchange -Input [2]: [ca_address_sk#13, ca_state#16] -Arguments: [ca_address_sk#13, ca_state#16] - -(12) CometBroadcastHashJoin -Left output [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Right output [2]: [ca_address_sk#13, ca_state#16] -Arguments: [ss_addr_sk#3], [ca_address_sk#13], Inner, ((((ca_state#16 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#16 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#16 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))), BuildRight - -(13) CometProject -Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#13, ca_state#16] -Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_year#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#17, d_year#18] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(16) CometProject -Input [2]: [d_date_sk#17, d_year#18] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: [d_date_sk#17] - -(18) CometBroadcastHashJoin -Left output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] -Right output [1]: [d_date_sk#17] -Arguments: [ss_sold_date_sk#10], [d_date_sk#17], Inner, BuildRight - -(19) CometProject -Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10, d_date_sk#17] -Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Condition : (isnotnull(cd_demo_sk#19) AND ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = Advanced Degree )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = S) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = College ))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = W) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = 2 yr Degree )))) - -(22) CometProject -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) AS cd_marital_status#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) AS cd_education_status#23] - -(23) CometBroadcastExchange -Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] - -(24) CometBroadcastHashJoin -Left output [6]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Right output [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [ss_cdemo_sk#1], [cd_demo_sk#19], Inner, ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))), BuildRight - -(25) CometProject -Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23], [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#24, hd_dep_count#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), Or(EqualTo(hd_dep_count,3),EqualTo(hd_dep_count,1))] -ReadSchema: struct - -(27) CometFilter -Input [2]: [hd_demo_sk#24, hd_dep_count#25] -Condition : (isnotnull(hd_demo_sk#24) AND ((hd_dep_count#25 = 3) OR (hd_dep_count#25 = 1))) - -(28) CometBroadcastExchange -Input [2]: [hd_demo_sk#24, hd_dep_count#25] -Arguments: [hd_demo_sk#24, hd_dep_count#25] - -(29) CometBroadcastHashJoin -Left output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23] -Right output [2]: [hd_demo_sk#24, hd_dep_count#25] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#24], Inner, (((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#25 = 3)) OR (((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#25 = 1))) OR (((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#25 = 1))), BuildRight - -(30) CometProject -Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23, hd_demo_sk#24, hd_dep_count#25] -Arguments: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] - -(31) CometHashAggregate -Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Keys: [] -Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sales_price#7)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#8)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#8))] - -(32) CometExchange -Input [7]: [sum#26, count#27, sum#28, count#29, sum#30, count#31, sum#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(33) CometHashAggregate -Input [7]: [sum#26, count#27, sum#28, count#29, sum#30, count#31, sum#32] -Keys: [] -Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] - -(34) CometColumnarToRow [codegen id : 1] -Input [4]: [avg(ss_quantity)#33, avg(ss_ext_sales_price)#34, avg(ss_ext_wholesale_cost)#35, sum(ss_ext_wholesale_cost)#36] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (39) -+- * CometColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (35) - - -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_year#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(36) CometFilter -Input [2]: [d_date_sk#17, d_year#18] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(37) CometProject -Input [2]: [d_date_sk#17, d_year#18] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(38) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] - -(39) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/extended.txt deleted file mode 100644 index 08e9beb692..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - -Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 8ef882a435..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13/simplified.txt +++ /dev/null @@ -1,44 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost))] - CometExchange #1 - CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum] - CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - 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,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_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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk] #3 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [hd_demo_sk,hd_dep_count] #7 - CometFilter [hd_demo_sk,hd_dep_count] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/explain.txt deleted file mode 100644 index f7377ccf8a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/explain.txt +++ /dev/null @@ -1,822 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (110) -+- * HashAggregate (109) - +- * CometColumnarToRow (108) - +- CometColumnarExchange (107) - +- * HashAggregate (106) - +- * Expand (105) - +- Union (104) - :- * Project (69) - : +- * Filter (68) - : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) - : +- CometColumnarExchange (65) - : +- * HashAggregate (64) - : +- * Project (63) - : +- * BroadcastHashJoin Inner BuildRight (62) - : :- * Project (60) - : : +- * BroadcastHashJoin Inner BuildRight (59) - : : :- * BroadcastHashJoin LeftSemi BuildRight (52) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * CometColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (48) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * CometColumnarToRow (36) - : : : : +- CometHashAggregate (35) - : : : : +- CometColumnarExchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * CometColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * CometColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * Filter (39) - : : : : : +- * ColumnarToRow (38) - : : : : : +- Scan parquet spark_catalog.default.web_sales (37) - : : : : +- ReusedExchange (40) - : : : +- ReusedExchange (43) - : : +- BroadcastExchange (58) - : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : :- * CometColumnarToRow (55) - : : : +- CometFilter (54) - : : : +- CometNativeScan parquet spark_catalog.default.item (53) - : : +- ReusedExchange (56) - : +- ReusedExchange (61) - :- * Project (86) - : +- * Filter (85) - : +- * HashAggregate (84) - : +- * CometColumnarToRow (83) - : +- CometColumnarExchange (82) - : +- * HashAggregate (81) - : +- * Project (80) - : +- * BroadcastHashJoin Inner BuildRight (79) - : :- * Project (77) - : : +- * BroadcastHashJoin Inner BuildRight (76) - : : :- * BroadcastHashJoin LeftSemi BuildRight (74) - : : : :- * Filter (72) - : : : : +- * ColumnarToRow (71) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (70) - : : : +- ReusedExchange (73) - : : +- ReusedExchange (75) - : +- ReusedExchange (78) - +- * Project (103) - +- * Filter (102) - +- * HashAggregate (101) - +- * CometColumnarToRow (100) - +- CometColumnarExchange (99) - +- * HashAggregate (98) - +- * Project (97) - +- * BroadcastHashJoin Inner BuildRight (96) - :- * Project (94) - : +- * BroadcastHashJoin Inner BuildRight (93) - : :- * BroadcastHashJoin LeftSemi BuildRight (91) - : : :- * Filter (89) - : : : +- * ColumnarToRow (88) - : : : +- Scan parquet spark_catalog.default.web_sales (87) - : : +- ReusedExchange (90) - : +- ReusedExchange (92) - +- ReusedExchange (95) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(6) CometColumnarToRow [codegen id : 11] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(10) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(11) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(12) CometColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(16) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) - -(18) CometColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(19) BroadcastExchange -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#19] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(22) ReusedExchange [Reuses operator id: 140] -Output [1]: [d_date_sk#23] - -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#23] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#23] - -(25) BroadcastExchange -Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] -Join type: LeftSemi -Join condition: None - -(27) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None - -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(30) ReusedExchange [Reuses operator id: 140] -Output [1]: [d_date_sk#24] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#25, i_class_id#15 AS class_id#26, i_category_id#16 AS category_id#27] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#24] - -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#25, class_id#26, category_id#27] - -(34) CometColumnarExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(35) CometHashAggregate -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] - -(36) CometColumnarToRow [codegen id : 10] -Input [3]: [brand_id#25, class_id#26, category_id#27] - -(37) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] - -(39) Filter [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Condition : isnotnull(ws_item_sk#28) - -(40) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#28] -Right keys [1]: [i_item_sk#30] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] -Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(43) ReusedExchange [Reuses operator id: 140] -Output [1]: [d_date_sk#34] - -(44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#29] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 9] -Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] - -(46) BroadcastExchange -Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] - -(47) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] -Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] -Join type: LeftSemi -Join condition: None - -(48) BroadcastExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] - -(49) BroadcastHashJoin [codegen id : 11] -Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#25, class_id#26, category_id#27] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#35] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] - -(51) BroadcastExchange -Input [1]: [ss_item_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(52) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(53) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(54) CometFilter -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Condition : isnotnull(i_item_sk#36) - -(55) CometColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(56) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#35] - -(57) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#36] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(58) BroadcastExchange -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(59) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#36] -Join type: Inner -Join condition: None - -(60) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(61) ReusedExchange [Reuses operator id: 135] -Output [1]: [d_date_sk#40] - -(62) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] - -(64) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] -Results [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(65) CometColumnarExchange -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Arguments: hashpartitioning(i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(66) CometColumnarToRow [codegen id : 26] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(67) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] -Results [5]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#49, count(1)#48 AS number_sales#50] - -(68) Filter [codegen id : 26] -Input [5]: [i_brand_id#37, i_class_id#38, i_category_id#39, sales#49, number_sales#50] -Condition : (isnotnull(sales#49) AND (cast(sales#49 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(69) Project [codegen id : 26] -Output [6]: [sales#49, number_sales#50, store AS channel#53, i_brand_id#37 AS i_brand_id#54, i_class_id#38 AS i_class_id#55, i_category_id#39 AS i_category_id#56] -Input [5]: [i_brand_id#37, i_class_id#38, i_category_id#39, sales#49, number_sales#50] - -(70) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(71) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] - -(72) Filter [codegen id : 51] -Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Condition : isnotnull(cs_item_sk#57) - -(73) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#61] - -(74) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#57] -Right keys [1]: [ss_item_sk#61] -Join type: LeftSemi -Join condition: None - -(75) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] - -(76) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#57] -Right keys [1]: [i_item_sk#62] -Join type: Inner -Join condition: None - -(77) Project [codegen id : 51] -Output [6]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65] -Input [8]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] - -(78) ReusedExchange [Reuses operator id: 135] -Output [1]: [d_date_sk#66] - -(79) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#60] -Right keys [1]: [d_date_sk#66] -Join type: Inner -Join condition: None - -(80) Project [codegen id : 51] -Output [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] -Input [7]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] - -(81) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [partial_sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), partial_count(1)] -Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] -Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] - -(82) CometColumnarExchange -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(83) CometColumnarToRow [codegen id : 52] -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] - -(84) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), count(1)] -Aggregate Attributes [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#73, count(1)#74] -Results [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#73 AS sales#75, count(1)#74 AS number_sales#76] - -(85) Filter [codegen id : 52] -Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#75, number_sales#76] -Condition : (isnotnull(sales#75) AND (cast(sales#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(86) Project [codegen id : 52] -Output [6]: [sales#75, number_sales#76, catalog AS channel#77, i_brand_id#63, i_class_id#64, i_category_id#65] -Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#75, number_sales#76] - -(87) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(88) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] - -(89) Filter [codegen id : 77] -Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_item_sk#78) - -(90) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#82] - -(91) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#78] -Right keys [1]: [ss_item_sk#82] -Join type: LeftSemi -Join condition: None - -(92) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#83, i_brand_id#84, i_class_id#85, i_category_id#86] - -(93) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#78] -Right keys [1]: [i_item_sk#83] -Join type: Inner -Join condition: None - -(94) Project [codegen id : 77] -Output [6]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#84, i_class_id#85, i_category_id#86] -Input [8]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_item_sk#83, i_brand_id#84, i_class_id#85, i_category_id#86] - -(95) ReusedExchange [Reuses operator id: 135] -Output [1]: [d_date_sk#87] - -(96) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#81] -Right keys [1]: [d_date_sk#87] -Join type: Inner -Join condition: None - -(97) Project [codegen id : 77] -Output [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#84, i_class_id#85, i_category_id#86] -Input [7]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#84, i_class_id#85, i_category_id#86, d_date_sk#87] - -(98) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#84, i_class_id#85, i_category_id#86] -Keys [3]: [i_brand_id#84, i_class_id#85, i_category_id#86] -Functions [2]: [partial_sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), partial_count(1)] -Aggregate Attributes [3]: [sum#88, isEmpty#89, count#90] -Results [6]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum#91, isEmpty#92, count#93] - -(99) CometColumnarExchange -Input [6]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum#91, isEmpty#92, count#93] -Arguments: hashpartitioning(i_brand_id#84, i_class_id#85, i_category_id#86, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(100) CometColumnarToRow [codegen id : 78] -Input [6]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum#91, isEmpty#92, count#93] - -(101) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum#91, isEmpty#92, count#93] -Keys [3]: [i_brand_id#84, i_class_id#85, i_category_id#86] -Functions [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), count(1)] -Aggregate Attributes [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#94, count(1)#95] -Results [5]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#94 AS sales#96, count(1)#95 AS number_sales#97] - -(102) Filter [codegen id : 78] -Input [5]: [i_brand_id#84, i_class_id#85, i_category_id#86, sales#96, number_sales#97] -Condition : (isnotnull(sales#96) AND (cast(sales#96 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(103) Project [codegen id : 78] -Output [6]: [sales#96, number_sales#97, web AS channel#98, i_brand_id#84, i_class_id#85, i_category_id#86] -Input [5]: [i_brand_id#84, i_class_id#85, i_category_id#86, sales#96, number_sales#97] - -(104) Union - -(105) Expand [codegen id : 79] -Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: [[sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, null, 1], [sales#49, number_sales#50, channel#53, i_brand_id#54, null, null, 3], [sales#49, number_sales#50, channel#53, null, null, null, 7], [sales#49, number_sales#50, null, null, null, null, 15]], [sales#49, number_sales#50, channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103] - -(106) HashAggregate [codegen id : 79] -Input [7]: [sales#49, number_sales#50, channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103] -Keys [5]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103] -Functions [2]: [partial_sum(sales#49), partial_sum(number_sales#50)] -Aggregate Attributes [3]: [sum#104, isEmpty#105, sum#106] -Results [8]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103, sum#107, isEmpty#108, sum#109] - -(107) CometColumnarExchange -Input [8]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103, sum#107, isEmpty#108, sum#109] -Arguments: hashpartitioning(channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(108) CometColumnarToRow [codegen id : 80] -Input [8]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103, sum#107, isEmpty#108, sum#109] - -(109) HashAggregate [codegen id : 80] -Input [8]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103, sum#107, isEmpty#108, sum#109] -Keys [5]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103] -Functions [2]: [sum(sales#49), sum(number_sales#50)] -Aggregate Attributes [2]: [sum(sales#49)#110, sum(number_sales#50)#111] -Results [6]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, sum(sales#49)#110 AS sum(sales)#112, sum(number_sales#50)#111 AS sum(number_sales)#113] - -(110) TakeOrderedAndProject -Input [6]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, sum(sales)#112, sum(number_sales)#113] -Arguments: 100, [channel#99 ASC NULLS FIRST, i_brand_id#100 ASC NULLS FIRST, i_class_id#101 ASC NULLS FIRST, i_category_id#102 ASC NULLS FIRST], [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, sum(sales)#112, sum(number_sales)#113] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* HashAggregate (130) -+- * CometColumnarToRow (129) - +- CometColumnarExchange (128) - +- * HashAggregate (127) - +- Union (126) - :- * Project (115) - : +- * BroadcastHashJoin Inner BuildRight (114) - : :- * ColumnarToRow (112) - : : +- Scan parquet spark_catalog.default.store_sales (111) - : +- ReusedExchange (113) - :- * Project (120) - : +- * BroadcastHashJoin Inner BuildRight (119) - : :- * ColumnarToRow (117) - : : +- Scan parquet spark_catalog.default.catalog_sales (116) - : +- ReusedExchange (118) - +- * Project (125) - +- * BroadcastHashJoin Inner BuildRight (124) - :- * ColumnarToRow (122) - : +- Scan parquet spark_catalog.default.web_sales (121) - +- ReusedExchange (123) - - -(111) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#114, ss_list_price#115, ss_sold_date_sk#116] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#116), dynamicpruningexpression(ss_sold_date_sk#116 IN dynamicpruning#12)] -ReadSchema: struct - -(112) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#114, ss_list_price#115, ss_sold_date_sk#116] - -(113) ReusedExchange [Reuses operator id: 140] -Output [1]: [d_date_sk#117] - -(114) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#116] -Right keys [1]: [d_date_sk#117] -Join type: Inner -Join condition: None - -(115) Project [codegen id : 2] -Output [2]: [ss_quantity#114 AS quantity#118, ss_list_price#115 AS list_price#119] -Input [4]: [ss_quantity#114, ss_list_price#115, ss_sold_date_sk#116, d_date_sk#117] - -(116) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#120, cs_list_price#121, cs_sold_date_sk#122] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#122), dynamicpruningexpression(cs_sold_date_sk#122 IN dynamicpruning#12)] -ReadSchema: struct - -(117) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#120, cs_list_price#121, cs_sold_date_sk#122] - -(118) ReusedExchange [Reuses operator id: 140] -Output [1]: [d_date_sk#123] - -(119) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#122] -Right keys [1]: [d_date_sk#123] -Join type: Inner -Join condition: None - -(120) Project [codegen id : 4] -Output [2]: [cs_quantity#120 AS quantity#124, cs_list_price#121 AS list_price#125] -Input [4]: [cs_quantity#120, cs_list_price#121, cs_sold_date_sk#122, d_date_sk#123] - -(121) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#126, ws_list_price#127, ws_sold_date_sk#128] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#128), dynamicpruningexpression(ws_sold_date_sk#128 IN dynamicpruning#12)] -ReadSchema: struct - -(122) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#126, ws_list_price#127, ws_sold_date_sk#128] - -(123) ReusedExchange [Reuses operator id: 140] -Output [1]: [d_date_sk#129] - -(124) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#128] -Right keys [1]: [d_date_sk#129] -Join type: Inner -Join condition: None - -(125) Project [codegen id : 6] -Output [2]: [ws_quantity#126 AS quantity#130, ws_list_price#127 AS list_price#131] -Input [4]: [ws_quantity#126, ws_list_price#127, ws_sold_date_sk#128, d_date_sk#129] - -(126) Union - -(127) HashAggregate [codegen id : 7] -Input [2]: [quantity#118, list_price#119] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#118 as decimal(10,0)) * list_price#119))] -Aggregate Attributes [2]: [sum#132, count#133] -Results [2]: [sum#134, count#135] - -(128) CometColumnarExchange -Input [2]: [sum#134, count#135] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(129) CometColumnarToRow [codegen id : 8] -Input [2]: [sum#134, count#135] - -(130) HashAggregate [codegen id : 8] -Input [2]: [sum#134, count#135] -Keys: [] -Functions [1]: [avg((cast(quantity#118 as decimal(10,0)) * list_price#119))] -Aggregate Attributes [1]: [avg((cast(quantity#118 as decimal(10,0)) * list_price#119))#136] -Results [1]: [avg((cast(quantity#118 as decimal(10,0)) * list_price#119))#136 AS average_sales#137] - -Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#116 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 116 Hosting Expression = cs_sold_date_sk#122 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 121 Hosting Expression = ws_sold_date_sk#128 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (135) -+- * CometColumnarToRow (134) - +- CometProject (133) - +- CometFilter (132) - +- CometNativeScan parquet spark_catalog.default.date_dim (131) - - -(131) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#40, d_year#138, d_moy#139] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(132) CometFilter -Input [3]: [d_date_sk#40, d_year#138, d_moy#139] -Condition : ((((isnotnull(d_year#138) AND isnotnull(d_moy#139)) AND (d_year#138 = 2001)) AND (d_moy#139 = 11)) AND isnotnull(d_date_sk#40)) - -(133) CometProject -Input [3]: [d_date_sk#40, d_year#138, d_moy#139] -Arguments: [d_date_sk#40], [d_date_sk#40] - -(134) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#40] - -(135) BroadcastExchange -Input [1]: [d_date_sk#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (140) -+- * CometColumnarToRow (139) - +- CometProject (138) - +- CometFilter (137) - +- CometNativeScan parquet spark_catalog.default.date_dim (136) - - -(136) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#140] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(137) CometFilter -Input [2]: [d_date_sk#24, d_year#140] -Condition : (((isnotnull(d_year#140) AND (d_year#140 >= 1999)) AND (d_year#140 <= 2001)) AND isnotnull(d_date_sk#24)) - -(138) CometProject -Input [2]: [d_date_sk#24, d_year#140] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(139) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#24] - -(140) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] - -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:8 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 85 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] - -Subquery:10 Hosting operator id = 70 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 - -Subquery:11 Hosting operator id = 102 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] - -Subquery:12 Hosting operator id = 87 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/simplified.txt deleted file mode 100644 index c989fe9a81..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/simplified.txt +++ /dev/null @@ -1,220 +0,0 @@ -TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - WholeStageCodegen (80) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - WholeStageCodegen (79) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - InputAdapter - Union - WholeStageCodegen (26) - Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] - Filter [sales] - Subquery #3 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #13 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #7 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #7 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #7 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (52) - Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (51) - HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (78) - Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (77) - HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/explain.txt deleted file mode 100644 index 22f1896b63..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,754 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (102) -+- CometTakeOrderedAndProject (101) - +- CometHashAggregate (100) - +- CometExchange (99) - +- CometHashAggregate (98) - +- CometExpand (97) - +- CometUnion (96) - :- CometProject (65) - : +- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometBroadcastHashJoin (47) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (46) - : : : +- CometProject (45) - : : : +- CometBroadcastHashJoin (44) - : : : :- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (43) - : : : +- CometBroadcastHashJoin (42) - : : : :- CometHashAggregate (32) - : : : : +- CometExchange (31) - : : : : +- CometHashAggregate (30) - : : : : +- CometProject (29) - : : : : +- CometBroadcastHashJoin (28) - : : : : :- CometProject (26) - : : : : : +- CometBroadcastHashJoin (25) - : : : : : :- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : +- CometBroadcastExchange (24) - : : : : : +- CometBroadcastHashJoin (23) - : : : : : :- CometFilter (8) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : +- CometBroadcastExchange (22) - : : : : : +- CometProject (21) - : : : : : +- CometBroadcastHashJoin (20) - : : : : : :- CometProject (15) - : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : :- CometFilter (10) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : +- CometBroadcastExchange (13) - : : : : : : +- CometFilter (12) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : +- CometBroadcastExchange (19) - : : : : : +- CometProject (18) - : : : : : +- CometFilter (17) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : +- ReusedExchange (27) - : : : +- CometBroadcastExchange (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (34) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometFilter (49) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : +- ReusedExchange (50) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - :- CometProject (80) - : +- CometFilter (79) - : +- CometHashAggregate (78) - : +- CometExchange (77) - : +- CometHashAggregate (76) - : +- CometProject (75) - : +- CometBroadcastHashJoin (74) - : :- CometProject (72) - : : +- CometBroadcastHashJoin (71) - : : :- CometBroadcastHashJoin (69) - : : : :- CometFilter (67) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (66) - : : : +- ReusedExchange (68) - : : +- ReusedExchange (70) - : +- ReusedExchange (73) - +- CometProject (95) - +- CometFilter (94) - +- CometHashAggregate (93) - +- CometExchange (92) - +- CometHashAggregate (91) - +- CometProject (90) - +- CometBroadcastHashJoin (89) - :- CometProject (87) - : +- CometBroadcastHashJoin (86) - : :- CometBroadcastHashJoin (84) - : : :- CometFilter (82) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (81) - : : +- ReusedExchange (83) - : +- ReusedExchange (85) - +- ReusedExchange (88) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : isnotnull(i_item_sk#39) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2001)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#49, number_sales#50] -Condition : (isnotnull(sales#49) AND (cast(sales#49 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(65) CometProject -Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#49, number_sales#50] -Arguments: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56], [sales#49, number_sales#50, store AS channel#53, i_brand_id#40 AS i_brand_id#54, i_class_id#41 AS i_class_id#55, i_category_id#42 AS i_category_id#56] - -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(67) CometFilter -Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Condition : isnotnull(cs_item_sk#57) - -(68) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#62] - -(69) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Right output [1]: [ss_item_sk#62] -Arguments: [cs_item_sk#57], [ss_item_sk#62], LeftSemi, BuildRight - -(70) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66] - -(71) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Right output [4]: [i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66] -Arguments: [cs_item_sk#57], [i_item_sk#63], Inner, BuildRight - -(72) CometProject -Input [8]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66] -Arguments: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66], [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66] - -(73) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#67] - -(74) CometBroadcastHashJoin -Left output [6]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66] -Right output [1]: [d_date_sk#67] -Arguments: [cs_sold_date_sk#60], [d_date_sk#67], Inner, BuildRight - -(75) CometProject -Input [7]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66, d_date_sk#67] -Arguments: [cs_quantity#58, cs_list_price#59, i_brand_id#64, i_class_id#65, i_category_id#66], [cs_quantity#58, cs_list_price#59, i_brand_id#64, i_class_id#65, i_category_id#66] - -(76) CometHashAggregate -Input [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#64, i_class_id#65, i_category_id#66] -Keys [3]: [i_brand_id#64, i_class_id#65, i_category_id#66] -Functions [2]: [partial_sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), partial_count(1)] - -(77) CometExchange -Input [6]: [i_brand_id#64, i_class_id#65, i_category_id#66, sum#68, isEmpty#69, count#70] -Arguments: hashpartitioning(i_brand_id#64, i_class_id#65, i_category_id#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(78) CometHashAggregate -Input [6]: [i_brand_id#64, i_class_id#65, i_category_id#66, sum#68, isEmpty#69, count#70] -Keys [3]: [i_brand_id#64, i_class_id#65, i_category_id#66] -Functions [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), count(1)] - -(79) CometFilter -Input [5]: [i_brand_id#64, i_class_id#65, i_category_id#66, sales#71, number_sales#72] -Condition : (isnotnull(sales#71) AND (cast(sales#71 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(80) CometProject -Input [5]: [i_brand_id#64, i_class_id#65, i_category_id#66, sales#71, number_sales#72] -Arguments: [sales#71, number_sales#72, channel#73, i_brand_id#64, i_class_id#65, i_category_id#66], [sales#71, number_sales#72, catalog AS channel#73, i_brand_id#64, i_class_id#65, i_category_id#66] - -(81) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#77), dynamicpruningexpression(ws_sold_date_sk#77 IN dynamicpruning#78)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(82) CometFilter -Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] -Condition : isnotnull(ws_item_sk#74) - -(83) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#79] - -(84) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] -Right output [1]: [ss_item_sk#79] -Arguments: [ws_item_sk#74], [ss_item_sk#79], LeftSemi, BuildRight - -(85) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] - -(86) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] -Right output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] -Arguments: [ws_item_sk#74], [i_item_sk#80], Inner, BuildRight - -(87) CometProject -Input [8]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] -Arguments: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#81, i_class_id#82, i_category_id#83], [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#81, i_class_id#82, i_category_id#83] - -(88) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#84] - -(89) CometBroadcastHashJoin -Left output [6]: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#81, i_class_id#82, i_category_id#83] -Right output [1]: [d_date_sk#84] -Arguments: [ws_sold_date_sk#77], [d_date_sk#84], Inner, BuildRight - -(90) CometProject -Input [7]: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#81, i_class_id#82, i_category_id#83, d_date_sk#84] -Arguments: [ws_quantity#75, ws_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83], [ws_quantity#75, ws_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] - -(91) CometHashAggregate -Input [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [partial_sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), partial_count(1)] - -(92) CometExchange -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#85, isEmpty#86, count#87] -Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(93) CometHashAggregate -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#85, isEmpty#86, count#87] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), count(1)] - -(94) CometFilter -Input [5]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#88, number_sales#89] -Condition : (isnotnull(sales#88) AND (cast(sales#88 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(95) CometProject -Input [5]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#88, number_sales#89] -Arguments: [sales#88, number_sales#89, channel#90, i_brand_id#81, i_class_id#82, i_category_id#83], [sales#88, number_sales#89, web AS channel#90, i_brand_id#81, i_class_id#82, i_category_id#83] - -(96) CometUnion -Child 0 Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Child 1 Input [6]: [sales#71, number_sales#72, channel#73, i_brand_id#64, i_class_id#65, i_category_id#66] -Child 2 Input [6]: [sales#88, number_sales#89, channel#90, i_brand_id#81, i_class_id#82, i_category_id#83] - -(97) CometExpand -Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: [[sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, null, 1], [sales#49, number_sales#50, channel#53, i_brand_id#54, null, null, 3], [sales#49, number_sales#50, channel#53, null, null, null, 7], [sales#49, number_sales#50, null, null, null, null, 15]], [sales#49, number_sales#50, channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95] - -(98) CometHashAggregate -Input [7]: [sales#49, number_sales#50, channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95] -Keys [5]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95] -Functions [2]: [partial_sum(sales#49), partial_sum(number_sales#50)] - -(99) CometExchange -Input [8]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95, sum#96, isEmpty#97, sum#98] -Arguments: hashpartitioning(channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(100) CometHashAggregate -Input [8]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95, sum#96, isEmpty#97, sum#98] -Keys [5]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95] -Functions [2]: [sum(sales#49), sum(number_sales#50)] - -(101) CometTakeOrderedAndProject -Input [6]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#91 ASC NULLS FIRST,i_brand_id#92 ASC NULLS FIRST,i_class_id#93 ASC NULLS FIRST,i_category_id#94 ASC NULLS FIRST], output=[channel#91,i_brand_id#92,i_class_id#93,i_category_id#94,sum(sales)#99,sum(number_sales)#100]), [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100], 100, 0, [channel#91 ASC NULLS FIRST, i_brand_id#92 ASC NULLS FIRST, i_class_id#93 ASC NULLS FIRST, i_category_id#94 ASC NULLS FIRST], [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100] - -(102) CometColumnarToRow [codegen id : 1] -Input [6]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* CometColumnarToRow (119) -+- CometHashAggregate (118) - +- CometExchange (117) - +- CometHashAggregate (116) - +- CometUnion (115) - :- CometProject (106) - : +- CometBroadcastHashJoin (105) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (103) - : +- ReusedExchange (104) - :- CometProject (110) - : +- CometBroadcastHashJoin (109) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (107) - : +- ReusedExchange (108) - +- CometProject (114) - +- CometBroadcastHashJoin (113) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (111) - +- ReusedExchange (112) - - -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#101, ss_list_price#102, ss_sold_date_sk#103] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#103), dynamicpruningexpression(ss_sold_date_sk#103 IN dynamicpruning#104)] -ReadSchema: struct - -(104) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#105] - -(105) CometBroadcastHashJoin -Left output [3]: [ss_quantity#101, ss_list_price#102, ss_sold_date_sk#103] -Right output [1]: [d_date_sk#105] -Arguments: [ss_sold_date_sk#103], [d_date_sk#105], Inner, BuildRight - -(106) CometProject -Input [4]: [ss_quantity#101, ss_list_price#102, ss_sold_date_sk#103, d_date_sk#105] -Arguments: [quantity#106, list_price#107], [ss_quantity#101 AS quantity#106, ss_list_price#102 AS list_price#107] - -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#108, cs_list_price#109, cs_sold_date_sk#110] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#110), dynamicpruningexpression(cs_sold_date_sk#110 IN dynamicpruning#111)] -ReadSchema: struct - -(108) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#112] - -(109) CometBroadcastHashJoin -Left output [3]: [cs_quantity#108, cs_list_price#109, cs_sold_date_sk#110] -Right output [1]: [d_date_sk#112] -Arguments: [cs_sold_date_sk#110], [d_date_sk#112], Inner, BuildRight - -(110) CometProject -Input [4]: [cs_quantity#108, cs_list_price#109, cs_sold_date_sk#110, d_date_sk#112] -Arguments: [quantity#113, list_price#114], [cs_quantity#108 AS quantity#113, cs_list_price#109 AS list_price#114] - -(111) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#115, ws_list_price#116, ws_sold_date_sk#117] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#117), dynamicpruningexpression(ws_sold_date_sk#117 IN dynamicpruning#118)] -ReadSchema: struct - -(112) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#119] - -(113) CometBroadcastHashJoin -Left output [3]: [ws_quantity#115, ws_list_price#116, ws_sold_date_sk#117] -Right output [1]: [d_date_sk#119] -Arguments: [ws_sold_date_sk#117], [d_date_sk#119], Inner, BuildRight - -(114) CometProject -Input [4]: [ws_quantity#115, ws_list_price#116, ws_sold_date_sk#117, d_date_sk#119] -Arguments: [quantity#120, list_price#121], [ws_quantity#115 AS quantity#120, ws_list_price#116 AS list_price#121] - -(115) CometUnion -Child 0 Input [2]: [quantity#106, list_price#107] -Child 1 Input [2]: [quantity#113, list_price#114] -Child 2 Input [2]: [quantity#120, list_price#121] - -(116) CometHashAggregate -Input [2]: [quantity#106, list_price#107] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#106 as decimal(10,0)) * list_price#107))] - -(117) CometExchange -Input [2]: [sum#122, count#123] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(118) CometHashAggregate -Input [2]: [sum#122, count#123] -Keys: [] -Functions [1]: [avg((cast(quantity#106 as decimal(10,0)) * list_price#107))] - -(119) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#124] - -Subquery:2 Hosting operator id = 103 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 107 Hosting Expression = cs_sold_date_sk#110 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 111 Hosting Expression = ws_sold_date_sk#117 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (124) -+- * CometColumnarToRow (123) - +- CometProject (122) - +- CometFilter (121) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (120) - - -(120) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(121) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2001)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(122) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(123) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(124) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (129) -+- * CometColumnarToRow (128) - +- CometProject (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) - - -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#125] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#26, d_year#125] -Condition : (((isnotnull(d_year#125) AND (d_year#125 >= 1999)) AND (d_year#125 <= 2001)) AND isnotnull(d_date_sk#26)) - -(127) CometProject -Input [2]: [d_date_sk#26, d_year#125] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(128) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(129) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 79 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] - -Subquery:10 Hosting operator id = 66 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 - -Subquery:11 Hosting operator id = 94 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] - -Subquery:12 Hosting operator id = 81 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 79c782f2ca..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,149 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales),sum(sales),sum(number_salesL)] - CometExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - CometHashAggregate [sales,number_sales] [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] - CometExpand [channel,i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] - CometUnion [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometProject [i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #15 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #2 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk] #4 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #5 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #6 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #11 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #11 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #12 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - ReusedExchange [d_date_sk] #11 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #4 - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [cs_quantity,cs_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk] - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #4 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - ReusedExchange [d_date_sk] #14 - CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #17 - CometHashAggregate [ws_quantity,ws_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk] - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #4 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - ReusedExchange [d_date_sk] #14 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/explain.txt deleted file mode 100644 index 22f1896b63..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/explain.txt +++ /dev/null @@ -1,754 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (102) -+- CometTakeOrderedAndProject (101) - +- CometHashAggregate (100) - +- CometExchange (99) - +- CometHashAggregate (98) - +- CometExpand (97) - +- CometUnion (96) - :- CometProject (65) - : +- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometBroadcastHashJoin (47) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (46) - : : : +- CometProject (45) - : : : +- CometBroadcastHashJoin (44) - : : : :- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (43) - : : : +- CometBroadcastHashJoin (42) - : : : :- CometHashAggregate (32) - : : : : +- CometExchange (31) - : : : : +- CometHashAggregate (30) - : : : : +- CometProject (29) - : : : : +- CometBroadcastHashJoin (28) - : : : : :- CometProject (26) - : : : : : +- CometBroadcastHashJoin (25) - : : : : : :- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : +- CometBroadcastExchange (24) - : : : : : +- CometBroadcastHashJoin (23) - : : : : : :- CometFilter (8) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : +- CometBroadcastExchange (22) - : : : : : +- CometProject (21) - : : : : : +- CometBroadcastHashJoin (20) - : : : : : :- CometProject (15) - : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : :- CometFilter (10) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : +- CometBroadcastExchange (13) - : : : : : : +- CometFilter (12) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : +- CometBroadcastExchange (19) - : : : : : +- CometProject (18) - : : : : : +- CometFilter (17) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : +- ReusedExchange (27) - : : : +- CometBroadcastExchange (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (34) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometFilter (49) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : +- ReusedExchange (50) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - :- CometProject (80) - : +- CometFilter (79) - : +- CometHashAggregate (78) - : +- CometExchange (77) - : +- CometHashAggregate (76) - : +- CometProject (75) - : +- CometBroadcastHashJoin (74) - : :- CometProject (72) - : : +- CometBroadcastHashJoin (71) - : : :- CometBroadcastHashJoin (69) - : : : :- CometFilter (67) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (66) - : : : +- ReusedExchange (68) - : : +- ReusedExchange (70) - : +- ReusedExchange (73) - +- CometProject (95) - +- CometFilter (94) - +- CometHashAggregate (93) - +- CometExchange (92) - +- CometHashAggregate (91) - +- CometProject (90) - +- CometBroadcastHashJoin (89) - :- CometProject (87) - : +- CometBroadcastHashJoin (86) - : :- CometBroadcastHashJoin (84) - : : :- CometFilter (82) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (81) - : : +- ReusedExchange (83) - : +- ReusedExchange (85) - +- ReusedExchange (88) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : isnotnull(i_item_sk#39) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2001)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#49, number_sales#50] -Condition : (isnotnull(sales#49) AND (cast(sales#49 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(65) CometProject -Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#49, number_sales#50] -Arguments: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56], [sales#49, number_sales#50, store AS channel#53, i_brand_id#40 AS i_brand_id#54, i_class_id#41 AS i_class_id#55, i_category_id#42 AS i_category_id#56] - -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(67) CometFilter -Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Condition : isnotnull(cs_item_sk#57) - -(68) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#62] - -(69) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Right output [1]: [ss_item_sk#62] -Arguments: [cs_item_sk#57], [ss_item_sk#62], LeftSemi, BuildRight - -(70) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66] - -(71) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Right output [4]: [i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66] -Arguments: [cs_item_sk#57], [i_item_sk#63], Inner, BuildRight - -(72) CometProject -Input [8]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66] -Arguments: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66], [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66] - -(73) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#67] - -(74) CometBroadcastHashJoin -Left output [6]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66] -Right output [1]: [d_date_sk#67] -Arguments: [cs_sold_date_sk#60], [d_date_sk#67], Inner, BuildRight - -(75) CometProject -Input [7]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66, d_date_sk#67] -Arguments: [cs_quantity#58, cs_list_price#59, i_brand_id#64, i_class_id#65, i_category_id#66], [cs_quantity#58, cs_list_price#59, i_brand_id#64, i_class_id#65, i_category_id#66] - -(76) CometHashAggregate -Input [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#64, i_class_id#65, i_category_id#66] -Keys [3]: [i_brand_id#64, i_class_id#65, i_category_id#66] -Functions [2]: [partial_sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), partial_count(1)] - -(77) CometExchange -Input [6]: [i_brand_id#64, i_class_id#65, i_category_id#66, sum#68, isEmpty#69, count#70] -Arguments: hashpartitioning(i_brand_id#64, i_class_id#65, i_category_id#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(78) CometHashAggregate -Input [6]: [i_brand_id#64, i_class_id#65, i_category_id#66, sum#68, isEmpty#69, count#70] -Keys [3]: [i_brand_id#64, i_class_id#65, i_category_id#66] -Functions [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), count(1)] - -(79) CometFilter -Input [5]: [i_brand_id#64, i_class_id#65, i_category_id#66, sales#71, number_sales#72] -Condition : (isnotnull(sales#71) AND (cast(sales#71 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(80) CometProject -Input [5]: [i_brand_id#64, i_class_id#65, i_category_id#66, sales#71, number_sales#72] -Arguments: [sales#71, number_sales#72, channel#73, i_brand_id#64, i_class_id#65, i_category_id#66], [sales#71, number_sales#72, catalog AS channel#73, i_brand_id#64, i_class_id#65, i_category_id#66] - -(81) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#77), dynamicpruningexpression(ws_sold_date_sk#77 IN dynamicpruning#78)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(82) CometFilter -Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] -Condition : isnotnull(ws_item_sk#74) - -(83) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#79] - -(84) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] -Right output [1]: [ss_item_sk#79] -Arguments: [ws_item_sk#74], [ss_item_sk#79], LeftSemi, BuildRight - -(85) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] - -(86) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] -Right output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] -Arguments: [ws_item_sk#74], [i_item_sk#80], Inner, BuildRight - -(87) CometProject -Input [8]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] -Arguments: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#81, i_class_id#82, i_category_id#83], [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#81, i_class_id#82, i_category_id#83] - -(88) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#84] - -(89) CometBroadcastHashJoin -Left output [6]: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#81, i_class_id#82, i_category_id#83] -Right output [1]: [d_date_sk#84] -Arguments: [ws_sold_date_sk#77], [d_date_sk#84], Inner, BuildRight - -(90) CometProject -Input [7]: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#81, i_class_id#82, i_category_id#83, d_date_sk#84] -Arguments: [ws_quantity#75, ws_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83], [ws_quantity#75, ws_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] - -(91) CometHashAggregate -Input [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [partial_sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), partial_count(1)] - -(92) CometExchange -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#85, isEmpty#86, count#87] -Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(93) CometHashAggregate -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#85, isEmpty#86, count#87] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), count(1)] - -(94) CometFilter -Input [5]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#88, number_sales#89] -Condition : (isnotnull(sales#88) AND (cast(sales#88 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(95) CometProject -Input [5]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#88, number_sales#89] -Arguments: [sales#88, number_sales#89, channel#90, i_brand_id#81, i_class_id#82, i_category_id#83], [sales#88, number_sales#89, web AS channel#90, i_brand_id#81, i_class_id#82, i_category_id#83] - -(96) CometUnion -Child 0 Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Child 1 Input [6]: [sales#71, number_sales#72, channel#73, i_brand_id#64, i_class_id#65, i_category_id#66] -Child 2 Input [6]: [sales#88, number_sales#89, channel#90, i_brand_id#81, i_class_id#82, i_category_id#83] - -(97) CometExpand -Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: [[sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, null, 1], [sales#49, number_sales#50, channel#53, i_brand_id#54, null, null, 3], [sales#49, number_sales#50, channel#53, null, null, null, 7], [sales#49, number_sales#50, null, null, null, null, 15]], [sales#49, number_sales#50, channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95] - -(98) CometHashAggregate -Input [7]: [sales#49, number_sales#50, channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95] -Keys [5]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95] -Functions [2]: [partial_sum(sales#49), partial_sum(number_sales#50)] - -(99) CometExchange -Input [8]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95, sum#96, isEmpty#97, sum#98] -Arguments: hashpartitioning(channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(100) CometHashAggregate -Input [8]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95, sum#96, isEmpty#97, sum#98] -Keys [5]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95] -Functions [2]: [sum(sales#49), sum(number_sales#50)] - -(101) CometTakeOrderedAndProject -Input [6]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#91 ASC NULLS FIRST,i_brand_id#92 ASC NULLS FIRST,i_class_id#93 ASC NULLS FIRST,i_category_id#94 ASC NULLS FIRST], output=[channel#91,i_brand_id#92,i_class_id#93,i_category_id#94,sum(sales)#99,sum(number_sales)#100]), [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100], 100, 0, [channel#91 ASC NULLS FIRST, i_brand_id#92 ASC NULLS FIRST, i_class_id#93 ASC NULLS FIRST, i_category_id#94 ASC NULLS FIRST], [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100] - -(102) CometColumnarToRow [codegen id : 1] -Input [6]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* CometColumnarToRow (119) -+- CometHashAggregate (118) - +- CometExchange (117) - +- CometHashAggregate (116) - +- CometUnion (115) - :- CometProject (106) - : +- CometBroadcastHashJoin (105) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (103) - : +- ReusedExchange (104) - :- CometProject (110) - : +- CometBroadcastHashJoin (109) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (107) - : +- ReusedExchange (108) - +- CometProject (114) - +- CometBroadcastHashJoin (113) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (111) - +- ReusedExchange (112) - - -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#101, ss_list_price#102, ss_sold_date_sk#103] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#103), dynamicpruningexpression(ss_sold_date_sk#103 IN dynamicpruning#104)] -ReadSchema: struct - -(104) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#105] - -(105) CometBroadcastHashJoin -Left output [3]: [ss_quantity#101, ss_list_price#102, ss_sold_date_sk#103] -Right output [1]: [d_date_sk#105] -Arguments: [ss_sold_date_sk#103], [d_date_sk#105], Inner, BuildRight - -(106) CometProject -Input [4]: [ss_quantity#101, ss_list_price#102, ss_sold_date_sk#103, d_date_sk#105] -Arguments: [quantity#106, list_price#107], [ss_quantity#101 AS quantity#106, ss_list_price#102 AS list_price#107] - -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#108, cs_list_price#109, cs_sold_date_sk#110] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#110), dynamicpruningexpression(cs_sold_date_sk#110 IN dynamicpruning#111)] -ReadSchema: struct - -(108) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#112] - -(109) CometBroadcastHashJoin -Left output [3]: [cs_quantity#108, cs_list_price#109, cs_sold_date_sk#110] -Right output [1]: [d_date_sk#112] -Arguments: [cs_sold_date_sk#110], [d_date_sk#112], Inner, BuildRight - -(110) CometProject -Input [4]: [cs_quantity#108, cs_list_price#109, cs_sold_date_sk#110, d_date_sk#112] -Arguments: [quantity#113, list_price#114], [cs_quantity#108 AS quantity#113, cs_list_price#109 AS list_price#114] - -(111) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#115, ws_list_price#116, ws_sold_date_sk#117] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#117), dynamicpruningexpression(ws_sold_date_sk#117 IN dynamicpruning#118)] -ReadSchema: struct - -(112) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#119] - -(113) CometBroadcastHashJoin -Left output [3]: [ws_quantity#115, ws_list_price#116, ws_sold_date_sk#117] -Right output [1]: [d_date_sk#119] -Arguments: [ws_sold_date_sk#117], [d_date_sk#119], Inner, BuildRight - -(114) CometProject -Input [4]: [ws_quantity#115, ws_list_price#116, ws_sold_date_sk#117, d_date_sk#119] -Arguments: [quantity#120, list_price#121], [ws_quantity#115 AS quantity#120, ws_list_price#116 AS list_price#121] - -(115) CometUnion -Child 0 Input [2]: [quantity#106, list_price#107] -Child 1 Input [2]: [quantity#113, list_price#114] -Child 2 Input [2]: [quantity#120, list_price#121] - -(116) CometHashAggregate -Input [2]: [quantity#106, list_price#107] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#106 as decimal(10,0)) * list_price#107))] - -(117) CometExchange -Input [2]: [sum#122, count#123] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(118) CometHashAggregate -Input [2]: [sum#122, count#123] -Keys: [] -Functions [1]: [avg((cast(quantity#106 as decimal(10,0)) * list_price#107))] - -(119) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#124] - -Subquery:2 Hosting operator id = 103 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 107 Hosting Expression = cs_sold_date_sk#110 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 111 Hosting Expression = ws_sold_date_sk#117 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (124) -+- * CometColumnarToRow (123) - +- CometProject (122) - +- CometFilter (121) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (120) - - -(120) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(121) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2001)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(122) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(123) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(124) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (129) -+- * CometColumnarToRow (128) - +- CometProject (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) - - -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#125] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#26, d_year#125] -Condition : (((isnotnull(d_year#125) AND (d_year#125 >= 1999)) AND (d_year#125 <= 2001)) AND isnotnull(d_date_sk#26)) - -(127) CometProject -Input [2]: [d_date_sk#26, d_year#125] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(128) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(129) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 79 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] - -Subquery:10 Hosting operator id = 66 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 - -Subquery:11 Hosting operator id = 94 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] - -Subquery:12 Hosting operator id = 81 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/extended.txt deleted file mode 100644 index 4af04a7846..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/extended.txt +++ /dev/null @@ -1,469 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 424 out of 458 eligible operators (92%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 79c782f2ca..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a/simplified.txt +++ /dev/null @@ -1,149 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales),sum(sales),sum(number_salesL)] - CometExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - CometHashAggregate [sales,number_sales] [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] - CometExpand [channel,i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] - CometUnion [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometProject [i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #15 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #2 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk] #4 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #5 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #6 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #11 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #11 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #12 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - ReusedExchange [d_date_sk] #11 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #4 - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [cs_quantity,cs_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk] - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #4 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - ReusedExchange [d_date_sk] #14 - CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #17 - CometHashAggregate [ws_quantity,ws_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk] - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #4 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - ReusedExchange [d_date_sk] #14 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/explain.txt deleted file mode 100644 index 563fa0a20b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/explain.txt +++ /dev/null @@ -1,769 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (87) -+- * BroadcastHashJoin Inner BuildRight (86) - :- * Filter (68) - : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) - : +- CometColumnarExchange (65) - : +- * HashAggregate (64) - : +- * Project (63) - : +- * BroadcastHashJoin Inner BuildRight (62) - : :- * Project (60) - : : +- * BroadcastHashJoin Inner BuildRight (59) - : : :- * BroadcastHashJoin LeftSemi BuildRight (52) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * CometColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (48) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * CometColumnarToRow (36) - : : : : +- CometHashAggregate (35) - : : : : +- CometColumnarExchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * CometColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * CometColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * Filter (39) - : : : : : +- * ColumnarToRow (38) - : : : : : +- Scan parquet spark_catalog.default.web_sales (37) - : : : : +- ReusedExchange (40) - : : : +- ReusedExchange (43) - : : +- BroadcastExchange (58) - : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : :- * CometColumnarToRow (55) - : : : +- CometFilter (54) - : : : +- CometNativeScan parquet spark_catalog.default.item (53) - : : +- ReusedExchange (56) - : +- ReusedExchange (61) - +- BroadcastExchange (85) - +- * Filter (84) - +- * HashAggregate (83) - +- * CometColumnarToRow (82) - +- CometColumnarExchange (81) - +- * HashAggregate (80) - +- * Project (79) - +- * BroadcastHashJoin Inner BuildRight (78) - :- * Project (76) - : +- * BroadcastHashJoin Inner BuildRight (75) - : :- * BroadcastHashJoin LeftSemi BuildRight (73) - : : :- * Filter (71) - : : : +- * ColumnarToRow (70) - : : : +- Scan parquet spark_catalog.default.store_sales (69) - : : +- ReusedExchange (72) - : +- ReusedExchange (74) - +- ReusedExchange (77) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(6) CometColumnarToRow [codegen id : 11] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(10) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(11) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(12) CometColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(16) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) - -(18) CometColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(19) BroadcastExchange -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#19] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(22) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#23] - -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#23] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#23] - -(25) BroadcastExchange -Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] -Join type: LeftSemi -Join condition: None - -(27) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None - -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(30) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#24] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#25, i_class_id#15 AS class_id#26, i_category_id#16 AS category_id#27] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#24] - -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#25, class_id#26, category_id#27] - -(34) CometColumnarExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(35) CometHashAggregate -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] - -(36) CometColumnarToRow [codegen id : 10] -Input [3]: [brand_id#25, class_id#26, category_id#27] - -(37) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] - -(39) Filter [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Condition : isnotnull(ws_item_sk#28) - -(40) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#28] -Right keys [1]: [i_item_sk#30] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] -Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(43) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#34] - -(44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#29] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 9] -Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] - -(46) BroadcastExchange -Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] - -(47) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] -Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] -Join type: LeftSemi -Join condition: None - -(48) BroadcastExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] - -(49) BroadcastHashJoin [codegen id : 11] -Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#25, class_id#26, category_id#27] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#35] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] - -(51) BroadcastExchange -Input [1]: [ss_item_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(52) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(53) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(54) CometFilter -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Condition : (((isnotnull(i_item_sk#36) AND isnotnull(i_brand_id#37)) AND isnotnull(i_class_id#38)) AND isnotnull(i_category_id#39)) - -(55) CometColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(56) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#35] - -(57) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#36] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(58) BroadcastExchange -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(59) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#36] -Join type: Inner -Join condition: None - -(60) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(61) ReusedExchange [Reuses operator id: 112] -Output [1]: [d_date_sk#40] - -(62) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] - -(64) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] -Results [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(65) CometColumnarExchange -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Arguments: hashpartitioning(i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(66) CometColumnarToRow [codegen id : 52] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(67) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] -Results [6]: [store AS channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51] - -(68) Filter [codegen id : 52] -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51] -Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(69) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_sold_date_sk#57 IN dynamicpruning#58)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(70) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] - -(71) Filter [codegen id : 50] -Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -Condition : isnotnull(ss_item_sk#54) - -(72) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#59] - -(73) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [ss_item_sk#59] -Join type: LeftSemi -Join condition: None - -(74) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] - -(75) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [i_item_sk#60] -Join type: Inner -Join condition: None - -(76) Project [codegen id : 50] -Output [6]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] -Input [8]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] - -(77) ReusedExchange [Reuses operator id: 126] -Output [1]: [d_date_sk#64] - -(78) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#57] -Right keys [1]: [d_date_sk#64] -Join type: Inner -Join condition: None - -(79) Project [codegen id : 50] -Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] -Input [7]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] - -(80) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)] -Aggregate Attributes [3]: [sum#65, isEmpty#66, count#67] -Results [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] - -(81) CometColumnarExchange -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] -Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(82) CometColumnarToRow [codegen id : 51] -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] - -(83) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#71, count(1)#72] -Results [6]: [store AS channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#71 AS sales#74, count(1)#72 AS number_sales#75] - -(84) Filter [codegen id : 51] -Input [6]: [channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] -Condition : (isnotnull(sales#74) AND (cast(sales#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(85) BroadcastExchange -Input [6]: [channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] - -(86) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Right keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Join type: Inner -Join condition: None - -(87) TakeOrderedAndProject -Input [12]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51, channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] -Arguments: 100, [i_brand_id#37 ASC NULLS FIRST, i_class_id#38 ASC NULLS FIRST, i_category_id#39 ASC NULLS FIRST], [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51, channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* HashAggregate (107) -+- * CometColumnarToRow (106) - +- CometColumnarExchange (105) - +- * HashAggregate (104) - +- Union (103) - :- * Project (92) - : +- * BroadcastHashJoin Inner BuildRight (91) - : :- * ColumnarToRow (89) - : : +- Scan parquet spark_catalog.default.store_sales (88) - : +- ReusedExchange (90) - :- * Project (97) - : +- * BroadcastHashJoin Inner BuildRight (96) - : :- * ColumnarToRow (94) - : : +- Scan parquet spark_catalog.default.catalog_sales (93) - : +- ReusedExchange (95) - +- * Project (102) - +- * BroadcastHashJoin Inner BuildRight (101) - :- * ColumnarToRow (99) - : +- Scan parquet spark_catalog.default.web_sales (98) - +- ReusedExchange (100) - - -(88) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#12)] -ReadSchema: struct - -(89) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] - -(90) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#79] - -(91) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#78] -Right keys [1]: [d_date_sk#79] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 2] -Output [2]: [ss_quantity#76 AS quantity#80, ss_list_price#77 AS list_price#81] -Input [4]: [ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78, d_date_sk#79] - -(93) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#12)] -ReadSchema: struct - -(94) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] - -(95) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#85] - -(96) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#84] -Right keys [1]: [d_date_sk#85] -Join type: Inner -Join condition: None - -(97) Project [codegen id : 4] -Output [2]: [cs_quantity#82 AS quantity#86, cs_list_price#83 AS list_price#87] -Input [4]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84, d_date_sk#85] - -(98) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#90), dynamicpruningexpression(ws_sold_date_sk#90 IN dynamicpruning#12)] -ReadSchema: struct - -(99) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] - -(100) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#91] - -(101) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#90] -Right keys [1]: [d_date_sk#91] -Join type: Inner -Join condition: None - -(102) Project [codegen id : 6] -Output [2]: [ws_quantity#88 AS quantity#92, ws_list_price#89 AS list_price#93] -Input [4]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90, d_date_sk#91] - -(103) Union - -(104) HashAggregate [codegen id : 7] -Input [2]: [quantity#80, list_price#81] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] -Aggregate Attributes [2]: [sum#94, count#95] -Results [2]: [sum#96, count#97] - -(105) CometColumnarExchange -Input [2]: [sum#96, count#97] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(106) CometColumnarToRow [codegen id : 8] -Input [2]: [sum#96, count#97] - -(107) HashAggregate [codegen id : 8] -Input [2]: [sum#96, count#97] -Keys: [] -Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] -Aggregate Attributes [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))#98] -Results [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))#98 AS average_sales#99] - -Subquery:2 Hosting operator id = 88 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 93 Hosting Expression = cs_sold_date_sk#84 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 98 Hosting Expression = ws_sold_date_sk#90 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (112) -+- * CometColumnarToRow (111) - +- CometProject (110) - +- CometFilter (109) - +- CometNativeScan parquet spark_catalog.default.date_dim (108) - - -(108) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#40, d_week_seq#100] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(109) CometFilter -Input [2]: [d_date_sk#40, d_week_seq#100] -Condition : ((isnotnull(d_week_seq#100) AND (d_week_seq#100 = Subquery scalar-subquery#101, [id=#102])) AND isnotnull(d_date_sk#40)) - -(110) CometProject -Input [2]: [d_date_sk#40, d_week_seq#100] -Arguments: [d_date_sk#40], [d_date_sk#40] - -(111) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#40] - -(112) BroadcastExchange -Input [1]: [d_date_sk#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:6 Hosting operator id = 109 Hosting Expression = Subquery scalar-subquery#101, [id=#102] -* CometColumnarToRow (116) -+- CometProject (115) - +- CometFilter (114) - +- CometNativeScan parquet spark_catalog.default.date_dim (113) - - -(113) CometNativeScan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] -ReadSchema: struct - -(114) CometFilter -Input [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] -Condition : (((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND isnotnull(d_dom#106)) AND (d_year#104 = 2000)) AND (d_moy#105 = 12)) AND (d_dom#106 = 11)) - -(115) CometProject -Input [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] -Arguments: [d_week_seq#103], [d_week_seq#103] - -(116) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#103] - -Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometNativeScan parquet spark_catalog.default.date_dim (117) - - -(117) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#107] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#24, d_year#107] -Condition : (((isnotnull(d_year#107) AND (d_year#107 >= 1999)) AND (d_year#107 <= 2001)) AND isnotnull(d_date_sk#24)) - -(119) CometProject -Input [2]: [d_date_sk#24, d_year#107] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#24] - -(121) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:8 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 - -Subquery:10 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:11 Hosting operator id = 69 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 -BroadcastExchange (126) -+- * CometColumnarToRow (125) - +- CometProject (124) - +- CometFilter (123) - +- CometNativeScan parquet spark_catalog.default.date_dim (122) - - -(122) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#64, d_week_seq#108] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(123) CometFilter -Input [2]: [d_date_sk#64, d_week_seq#108] -Condition : ((isnotnull(d_week_seq#108) AND (d_week_seq#108 = Subquery scalar-subquery#109, [id=#110])) AND isnotnull(d_date_sk#64)) - -(124) CometProject -Input [2]: [d_date_sk#64, d_week_seq#108] -Arguments: [d_date_sk#64], [d_date_sk#64] - -(125) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#64] - -(126) BroadcastExchange -Input [1]: [d_date_sk#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] - -Subquery:12 Hosting operator id = 123 Hosting Expression = Subquery scalar-subquery#109, [id=#110] -* CometColumnarToRow (130) -+- CometProject (129) - +- CometFilter (128) - +- CometNativeScan parquet spark_catalog.default.date_dim (127) - - -(127) CometNativeScan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#111, d_year#112, d_moy#113, d_dom#114] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] -ReadSchema: struct - -(128) CometFilter -Input [4]: [d_week_seq#111, d_year#112, d_moy#113, d_dom#114] -Condition : (((((isnotnull(d_year#112) AND isnotnull(d_moy#113)) AND isnotnull(d_dom#114)) AND (d_year#112 = 1999)) AND (d_moy#113 = 12)) AND (d_dom#114 = 11)) - -(129) CometProject -Input [4]: [d_week_seq#111, d_year#112, d_moy#113, d_dom#114] -Arguments: [d_week_seq#111], [d_week_seq#111] - -(130) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#111] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/simplified.txt deleted file mode 100644 index b0eae963c3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/simplified.txt +++ /dev/null @@ -1,206 +0,0 @@ -TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (52) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Filter [sales] - Subquery #4 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #12 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (51) - Filter [sales] - ReusedSubquery [average_sales] #4 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (50) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #15 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/explain.txt deleted file mode 100644 index 879213d892..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,743 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (85) -+- CometTakeOrderedAndProject (84) - +- CometBroadcastHashJoin (83) - :- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometBroadcastHashJoin (47) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (46) - : : : +- CometProject (45) - : : : +- CometBroadcastHashJoin (44) - : : : :- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (43) - : : : +- CometBroadcastHashJoin (42) - : : : :- CometHashAggregate (32) - : : : : +- CometExchange (31) - : : : : +- CometHashAggregate (30) - : : : : +- CometProject (29) - : : : : +- CometBroadcastHashJoin (28) - : : : : :- CometProject (26) - : : : : : +- CometBroadcastHashJoin (25) - : : : : : :- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : +- CometBroadcastExchange (24) - : : : : : +- CometBroadcastHashJoin (23) - : : : : : :- CometFilter (8) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : +- CometBroadcastExchange (22) - : : : : : +- CometProject (21) - : : : : : +- CometBroadcastHashJoin (20) - : : : : : :- CometProject (15) - : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : :- CometFilter (10) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : +- CometBroadcastExchange (13) - : : : : : : +- CometFilter (12) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : +- CometBroadcastExchange (19) - : : : : : +- CometProject (18) - : : : : : +- CometFilter (17) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : +- ReusedExchange (27) - : : : +- CometBroadcastExchange (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (34) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometFilter (49) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : +- ReusedExchange (50) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometBroadcastExchange (82) - +- CometFilter (81) - +- CometHashAggregate (80) - +- CometExchange (79) - +- CometHashAggregate (78) - +- CometProject (77) - +- CometBroadcastHashJoin (76) - :- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometBroadcastHashJoin (68) - : : :- CometFilter (66) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (65) - : : +- ReusedExchange (67) - : +- ReusedExchange (69) - +- CometBroadcastExchange (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42)) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Condition : (isnotnull(sales#51) AND (cast(sales#51 as decimal(32,6)) > cast(Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Condition : isnotnull(ss_item_sk#55) - -(67) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#60] - -(68) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [1]: [ss_item_sk#60] -Arguments: [ss_item_sk#55], [ss_item_sk#60], LeftSemi, BuildRight - -(69) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] - -(70) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -Arguments: [ss_item_sk#55], [i_item_sk#61], Inner, BuildRight - -(71) CometProject -Input [8]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -Arguments: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#66] -Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = ReusedSubquery Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) - -(74) CometProject -Input [2]: [d_date_sk#65, d_week_seq#66] -Arguments: [d_date_sk#65], [d_date_sk#65] - -(75) CometBroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: [d_date_sk#65] - -(76) CometBroadcastHashJoin -Left output [6]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(77) CometProject -Input [7]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] -Arguments: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] - -(78) CometHashAggregate -Input [5]: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [partial_sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), partial_count(1)] - -(79) CometExchange -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(80) CometHashAggregate -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), count(1)] - -(81) CometFilter -Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(82) CometBroadcastExchange -Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -(83) CometBroadcastHashJoin -Left output [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Right output [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#62, i_class_id#63, i_category_id#64], Inner, BuildRight - -(84) CometTakeOrderedAndProject -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#72,i_brand_id#62,i_class_id#63,i_category_id#64,sales#73,number_sales#74]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74], 100, 0, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -(85) CometColumnarToRow [codegen id : 1] -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* CometColumnarToRow (102) -+- CometHashAggregate (101) - +- CometExchange (100) - +- CometHashAggregate (99) - +- CometUnion (98) - :- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (86) - : +- ReusedExchange (87) - :- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (90) - : +- ReusedExchange (91) - +- CometProject (97) - +- CometBroadcastHashJoin (96) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (94) - +- ReusedExchange (95) - - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#78)] -ReadSchema: struct - -(87) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#79] - -(88) CometBroadcastHashJoin -Left output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -Right output [1]: [d_date_sk#79] -Arguments: [ss_sold_date_sk#77], [d_date_sk#79], Inner, BuildRight - -(89) CometProject -Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#79] -Arguments: [quantity#80, list_price#81], [ss_quantity#75 AS quantity#80, ss_list_price#76 AS list_price#81] - -(90) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#85)] -ReadSchema: struct - -(91) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#86] - -(92) CometBroadcastHashJoin -Left output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] -Right output [1]: [d_date_sk#86] -Arguments: [cs_sold_date_sk#84], [d_date_sk#86], Inner, BuildRight - -(93) CometProject -Input [4]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84, d_date_sk#86] -Arguments: [quantity#87, list_price#88], [cs_quantity#82 AS quantity#87, cs_list_price#83 AS list_price#88] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#91), dynamicpruningexpression(ws_sold_date_sk#91 IN dynamicpruning#92)] -ReadSchema: struct - -(95) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#93] - -(96) CometBroadcastHashJoin -Left output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] -Right output [1]: [d_date_sk#93] -Arguments: [ws_sold_date_sk#91], [d_date_sk#93], Inner, BuildRight - -(97) CometProject -Input [4]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91, d_date_sk#93] -Arguments: [quantity#94, list_price#95], [ws_quantity#89 AS quantity#94, ws_list_price#90 AS list_price#95] - -(98) CometUnion -Child 0 Input [2]: [quantity#80, list_price#81] -Child 1 Input [2]: [quantity#87, list_price#88] -Child 2 Input [2]: [quantity#94, list_price#95] - -(99) CometHashAggregate -Input [2]: [quantity#80, list_price#81] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] - -(100) CometExchange -Input [2]: [sum#96, count#97] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(101) CometHashAggregate -Input [2]: [sum#96, count#97] -Keys: [] -Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] - -(102) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#98] - -Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#84 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (107) -+- * CometColumnarToRow (106) - +- CometProject (105) - +- CometFilter (104) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) - - -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(104) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(105) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(106) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(107) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) -+- CometProject (110) - +- CometFilter (109) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) - - -(108) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] -ReadSchema: struct - -(109) CometFilter -Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d_dom#102)) AND (d_year#100 = 2000)) AND (d_moy#101 = 12)) AND (d_dom#102 = 11)) - -(110) CometProject -Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Arguments: [d_week_seq#99], [d_week_seq#99] - -(111) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#99] - -Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (116) -+- * CometColumnarToRow (115) - +- CometProject (114) - +- CometFilter (113) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) - - -(112) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#103] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(113) CometFilter -Input [2]: [d_date_sk#26, d_year#103] -Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1999)) AND (d_year#103 <= 2001)) AND isnotnull(d_date_sk#26)) - -(114) CometProject -Input [2]: [d_date_sk#26, d_year#103] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(115) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(116) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:8 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:10 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] - -Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] - -Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#66] -Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) - -(119) CometProject -Input [2]: [d_date_sk#65, d_week_seq#66] -Arguments: [d_date_sk#65], [d_date_sk#65] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#65] - -(121) BroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* CometColumnarToRow (125) -+- CometProject (124) - +- CometFilter (123) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) - - -(122) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] -ReadSchema: struct - -(123) CometFilter -Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d_dom#107)) AND (d_year#105 = 1999)) AND (d_moy#106 = 12)) AND (d_dom#107 = 11)) - -(124) CometProject -Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Arguments: [d_week_seq#104], [d_week_seq#104] - -(125) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#104] - -Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/simplified.txt deleted file mode 100644 index fb9abae378..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,153 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #14 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #1 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [ss_item_sk] #3 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #4 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #5 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #10 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #11 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #3 - CometBroadcastExchange [d_date_sk] #13 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #4 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - ReusedExchange [ss_item_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastExchange [d_date_sk] #18 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/explain.txt deleted file mode 100644 index 879213d892..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/explain.txt +++ /dev/null @@ -1,743 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (85) -+- CometTakeOrderedAndProject (84) - +- CometBroadcastHashJoin (83) - :- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometBroadcastHashJoin (47) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (46) - : : : +- CometProject (45) - : : : +- CometBroadcastHashJoin (44) - : : : :- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (43) - : : : +- CometBroadcastHashJoin (42) - : : : :- CometHashAggregate (32) - : : : : +- CometExchange (31) - : : : : +- CometHashAggregate (30) - : : : : +- CometProject (29) - : : : : +- CometBroadcastHashJoin (28) - : : : : :- CometProject (26) - : : : : : +- CometBroadcastHashJoin (25) - : : : : : :- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : +- CometBroadcastExchange (24) - : : : : : +- CometBroadcastHashJoin (23) - : : : : : :- CometFilter (8) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : +- CometBroadcastExchange (22) - : : : : : +- CometProject (21) - : : : : : +- CometBroadcastHashJoin (20) - : : : : : :- CometProject (15) - : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : :- CometFilter (10) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : +- CometBroadcastExchange (13) - : : : : : : +- CometFilter (12) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : +- CometBroadcastExchange (19) - : : : : : +- CometProject (18) - : : : : : +- CometFilter (17) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : +- ReusedExchange (27) - : : : +- CometBroadcastExchange (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (34) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometFilter (49) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : +- ReusedExchange (50) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometBroadcastExchange (82) - +- CometFilter (81) - +- CometHashAggregate (80) - +- CometExchange (79) - +- CometHashAggregate (78) - +- CometProject (77) - +- CometBroadcastHashJoin (76) - :- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometBroadcastHashJoin (68) - : : :- CometFilter (66) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (65) - : : +- ReusedExchange (67) - : +- ReusedExchange (69) - +- CometBroadcastExchange (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42)) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Condition : (isnotnull(sales#51) AND (cast(sales#51 as decimal(32,6)) > cast(Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Condition : isnotnull(ss_item_sk#55) - -(67) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#60] - -(68) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [1]: [ss_item_sk#60] -Arguments: [ss_item_sk#55], [ss_item_sk#60], LeftSemi, BuildRight - -(69) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] - -(70) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -Arguments: [ss_item_sk#55], [i_item_sk#61], Inner, BuildRight - -(71) CometProject -Input [8]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -Arguments: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#66] -Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = ReusedSubquery Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) - -(74) CometProject -Input [2]: [d_date_sk#65, d_week_seq#66] -Arguments: [d_date_sk#65], [d_date_sk#65] - -(75) CometBroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: [d_date_sk#65] - -(76) CometBroadcastHashJoin -Left output [6]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(77) CometProject -Input [7]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] -Arguments: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] - -(78) CometHashAggregate -Input [5]: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [partial_sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), partial_count(1)] - -(79) CometExchange -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(80) CometHashAggregate -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), count(1)] - -(81) CometFilter -Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(82) CometBroadcastExchange -Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -(83) CometBroadcastHashJoin -Left output [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Right output [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#62, i_class_id#63, i_category_id#64], Inner, BuildRight - -(84) CometTakeOrderedAndProject -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#72,i_brand_id#62,i_class_id#63,i_category_id#64,sales#73,number_sales#74]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74], 100, 0, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -(85) CometColumnarToRow [codegen id : 1] -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* CometColumnarToRow (102) -+- CometHashAggregate (101) - +- CometExchange (100) - +- CometHashAggregate (99) - +- CometUnion (98) - :- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (86) - : +- ReusedExchange (87) - :- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (90) - : +- ReusedExchange (91) - +- CometProject (97) - +- CometBroadcastHashJoin (96) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (94) - +- ReusedExchange (95) - - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#78)] -ReadSchema: struct - -(87) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#79] - -(88) CometBroadcastHashJoin -Left output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -Right output [1]: [d_date_sk#79] -Arguments: [ss_sold_date_sk#77], [d_date_sk#79], Inner, BuildRight - -(89) CometProject -Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#79] -Arguments: [quantity#80, list_price#81], [ss_quantity#75 AS quantity#80, ss_list_price#76 AS list_price#81] - -(90) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#85)] -ReadSchema: struct - -(91) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#86] - -(92) CometBroadcastHashJoin -Left output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] -Right output [1]: [d_date_sk#86] -Arguments: [cs_sold_date_sk#84], [d_date_sk#86], Inner, BuildRight - -(93) CometProject -Input [4]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84, d_date_sk#86] -Arguments: [quantity#87, list_price#88], [cs_quantity#82 AS quantity#87, cs_list_price#83 AS list_price#88] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#91), dynamicpruningexpression(ws_sold_date_sk#91 IN dynamicpruning#92)] -ReadSchema: struct - -(95) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#93] - -(96) CometBroadcastHashJoin -Left output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] -Right output [1]: [d_date_sk#93] -Arguments: [ws_sold_date_sk#91], [d_date_sk#93], Inner, BuildRight - -(97) CometProject -Input [4]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91, d_date_sk#93] -Arguments: [quantity#94, list_price#95], [ws_quantity#89 AS quantity#94, ws_list_price#90 AS list_price#95] - -(98) CometUnion -Child 0 Input [2]: [quantity#80, list_price#81] -Child 1 Input [2]: [quantity#87, list_price#88] -Child 2 Input [2]: [quantity#94, list_price#95] - -(99) CometHashAggregate -Input [2]: [quantity#80, list_price#81] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] - -(100) CometExchange -Input [2]: [sum#96, count#97] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(101) CometHashAggregate -Input [2]: [sum#96, count#97] -Keys: [] -Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] - -(102) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#98] - -Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#84 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (107) -+- * CometColumnarToRow (106) - +- CometProject (105) - +- CometFilter (104) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) - - -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(104) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(105) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(106) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(107) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) -+- CometProject (110) - +- CometFilter (109) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) - - -(108) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] -ReadSchema: struct - -(109) CometFilter -Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d_dom#102)) AND (d_year#100 = 2000)) AND (d_moy#101 = 12)) AND (d_dom#102 = 11)) - -(110) CometProject -Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Arguments: [d_week_seq#99], [d_week_seq#99] - -(111) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#99] - -Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (116) -+- * CometColumnarToRow (115) - +- CometProject (114) - +- CometFilter (113) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) - - -(112) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#103] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(113) CometFilter -Input [2]: [d_date_sk#26, d_year#103] -Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1999)) AND (d_year#103 <= 2001)) AND isnotnull(d_date_sk#26)) - -(114) CometProject -Input [2]: [d_date_sk#26, d_year#103] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(115) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(116) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:8 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:10 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] - -Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] - -Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#66] -Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) - -(119) CometProject -Input [2]: [d_date_sk#65, d_week_seq#66] -Arguments: [d_date_sk#65], [d_date_sk#65] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#65] - -(121) BroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* CometColumnarToRow (125) -+- CometProject (124) - +- CometFilter (123) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) - - -(122) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] -ReadSchema: struct - -(123) CometFilter -Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d_dom#107)) AND (d_year#105 = 1999)) AND (d_moy#106 = 12)) AND (d_dom#107 = 11)) - -(124) CometProject -Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Arguments: [d_week_seq#104], [d_week_seq#104] - -(125) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#104] - -Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/extended.txt deleted file mode 100644 index f56d229b68..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/extended.txt +++ /dev/null @@ -1,339 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometBroadcastHashJoin - :- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 298 out of 327 eligible operators (91%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index fb9abae378..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b/simplified.txt +++ /dev/null @@ -1,153 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #14 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #1 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [ss_item_sk] #3 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #4 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #5 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #10 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #11 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #3 - CometBroadcastExchange [d_date_sk] #13 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #4 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - ReusedExchange [ss_item_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastExchange [d_date_sk] #18 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/explain.txt deleted file mode 100644 index 594939616e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/explain.txt +++ /dev/null @@ -1,173 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (24) -+- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometColumnarExchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : +- BroadcastExchange (7) - : : +- * CometColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometNativeScan parquet spark_catalog.default.customer (4) - : +- BroadcastExchange (14) - : +- * CometColumnarToRow (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometNativeScan parquet spark_catalog.default.customer_address (10) - +- ReusedExchange (17) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] - -(3) Filter [codegen id : 4] -Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_bill_customer_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#5, c_current_addr_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] -Condition : (isnotnull(c_customer_sk#5) AND isnotnull(c_current_addr_sk#6)) - -(6) CometColumnarToRow [codegen id : 1] -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] - -(7) BroadcastExchange -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#5] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 4] -Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] -Input [5]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3, c_customer_sk#5, c_current_addr_sk#6] - -(10) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Condition : isnotnull(ca_address_sk#7) - -(12) CometProject -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Arguments: [ca_address_sk#7, ca_state#10, ca_zip#11], [ca_address_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#8, 2, true, false, true) AS ca_state#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#9, 10, true, false, true) AS ca_zip#11] - -(13) CometColumnarToRow [codegen id : 2] -Input [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] - -(14) BroadcastExchange -Input [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [c_current_addr_sk#6] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: ((substr(ca_zip#11, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#10 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)) - -(16) Project [codegen id : 4] -Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11] -Input [6]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, ca_address_sk#7, ca_state#10, ca_zip#11] - -(17) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#12] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [2]: [cs_sales_price#2, ca_zip#11] -Input [4]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11, d_date_sk#12] - -(20) HashAggregate [codegen id : 4] -Input [2]: [cs_sales_price#2, ca_zip#11] -Keys [1]: [ca_zip#11] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum#13] -Results [2]: [ca_zip#11, sum#14] - -(21) CometColumnarExchange -Input [2]: [ca_zip#11, sum#14] -Arguments: hashpartitioning(ca_zip#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [2]: [ca_zip#11, sum#14] - -(23) HashAggregate [codegen id : 5] -Input [2]: [ca_zip#11, sum#14] -Keys [1]: [ca_zip#11] -Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#15] -Results [2]: [ca_zip#11, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#15,17,2) AS sum(cs_sales_price)#16] - -(24) TakeOrderedAndProject -Input [2]: [ca_zip#11, sum(cs_sales_price)#16] -Arguments: 100, [ca_zip#11 ASC NULLS FIRST], [ca_zip#11, sum(cs_sales_price)#16] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometNativeScan parquet spark_catalog.default.date_dim (25) - - -(25) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#12, d_year#17, d_qoy#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [3]: [d_date_sk#12, d_year#17, d_qoy#18] -Condition : ((((isnotnull(d_qoy#18) AND isnotnull(d_year#17)) AND (d_qoy#18 = 2)) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#12)) - -(27) CometProject -Input [3]: [d_date_sk#12, d_year#17, d_qoy#18] -Arguments: [d_date_sk#12], [d_date_sk#12] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#12] - -(29) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/simplified.txt deleted file mode 100644 index f31442dcfe..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - WholeStageCodegen (5) - HashAggregate [ca_zip,sum] [sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price),sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_zip] #1 - WholeStageCodegen (4) - HashAggregate [ca_zip,cs_sales_price] [sum,sum] - Project [cs_sales_price,ca_zip] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sales_price,cs_sold_date_sk,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] - Project [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip] - CometFilter [ca_address_sk,ca_state,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - InputAdapter - ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/explain.txt deleted file mode 100644 index af9b2efbd1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,173 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (13) - : +- CometBroadcastHashJoin (12) - : :- CometProject (7) - : : +- CometBroadcastHashJoin (6) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : +- CometBroadcastExchange (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (3) - : +- CometBroadcastExchange (11) - : +- CometProject (10) - : +- CometFilter (9) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - +- CometBroadcastExchange (17) - +- CometProject (16) - +- CometFilter (15) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_bill_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#5, c_current_addr_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] -Condition : (isnotnull(c_customer_sk#5) AND isnotnull(c_current_addr_sk#6)) - -(5) CometBroadcastExchange -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] -Arguments: [c_customer_sk#5, c_current_addr_sk#6] - -(6) CometBroadcastHashJoin -Left output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Right output [2]: [c_customer_sk#5, c_current_addr_sk#6] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#5], Inner, BuildRight - -(7) CometProject -Input [5]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3, c_customer_sk#5, c_current_addr_sk#6] -Arguments: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6], [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Condition : isnotnull(ca_address_sk#7) - -(10) CometProject -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Arguments: [ca_address_sk#7, ca_state#10, ca_zip#11], [ca_address_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#8, 2, true, false, true) AS ca_state#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#9, 10, true, false, true) AS ca_zip#11] - -(11) CometBroadcastExchange -Input [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] -Arguments: [ca_address_sk#7, ca_state#10, ca_zip#11] - -(12) CometBroadcastHashJoin -Left output [3]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] -Right output [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] -Arguments: [c_current_addr_sk#6], [ca_address_sk#7], Inner, ((substr(ca_zip#11, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#10 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)), BuildRight - -(13) CometProject -Input [6]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, ca_address_sk#7, ca_state#10, ca_zip#11] -Arguments: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11], [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2)) AND (d_year#13 = 2001)) AND isnotnull(d_date_sk#12)) - -(16) CometProject -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Arguments: [d_date_sk#12], [d_date_sk#12] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: [d_date_sk#12] - -(18) CometBroadcastHashJoin -Left output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11] -Right output [1]: [d_date_sk#12] -Arguments: [cs_sold_date_sk#3], [d_date_sk#12], Inner, BuildRight - -(19) CometProject -Input [4]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11, d_date_sk#12] -Arguments: [cs_sales_price#2, ca_zip#11], [cs_sales_price#2, ca_zip#11] - -(20) CometHashAggregate -Input [2]: [cs_sales_price#2, ca_zip#11] -Keys [1]: [ca_zip#11] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] - -(21) CometExchange -Input [2]: [ca_zip#11, sum#15] -Arguments: hashpartitioning(ca_zip#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [2]: [ca_zip#11, sum#15] -Keys [1]: [ca_zip#11] -Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] - -(23) CometTakeOrderedAndProject -Input [2]: [ca_zip#11, sum(cs_sales_price)#16] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_zip#11 ASC NULLS FIRST], output=[ca_zip#11,sum(cs_sales_price)#16]), [ca_zip#11, sum(cs_sales_price)#16], 100, 0, [ca_zip#11 ASC NULLS FIRST], [ca_zip#11, sum(cs_sales_price)#16] - -(24) CometColumnarToRow [codegen id : 1] -Input [2]: [ca_zip#11, sum(cs_sales_price)#16] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2)) AND (d_year#13 = 2001)) AND isnotnull(d_date_sk#12)) - -(27) CometProject -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Arguments: [d_date_sk#12], [d_date_sk#12] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#12] - -(29) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/simplified.txt deleted file mode 100644 index c39b96efe3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,34 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - CometHashAggregate [sum] [ca_zip,sum(cs_sales_price),sum(UnscaledValue(cs_sales_price))] - CometExchange [ca_zip] #1 - CometHashAggregate [cs_sales_price] [ca_zip,sum] - CometProject [cs_sales_price,ca_zip] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] - CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip] - CometFilter [ca_address_sk,ca_state,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] 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/q15/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/explain.txt deleted file mode 100644 index af9b2efbd1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/explain.txt +++ /dev/null @@ -1,173 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (13) - : +- CometBroadcastHashJoin (12) - : :- CometProject (7) - : : +- CometBroadcastHashJoin (6) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : +- CometBroadcastExchange (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (3) - : +- CometBroadcastExchange (11) - : +- CometProject (10) - : +- CometFilter (9) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - +- CometBroadcastExchange (17) - +- CometProject (16) - +- CometFilter (15) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_bill_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#5, c_current_addr_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] -Condition : (isnotnull(c_customer_sk#5) AND isnotnull(c_current_addr_sk#6)) - -(5) CometBroadcastExchange -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] -Arguments: [c_customer_sk#5, c_current_addr_sk#6] - -(6) CometBroadcastHashJoin -Left output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Right output [2]: [c_customer_sk#5, c_current_addr_sk#6] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#5], Inner, BuildRight - -(7) CometProject -Input [5]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3, c_customer_sk#5, c_current_addr_sk#6] -Arguments: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6], [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Condition : isnotnull(ca_address_sk#7) - -(10) CometProject -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Arguments: [ca_address_sk#7, ca_state#10, ca_zip#11], [ca_address_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#8, 2, true, false, true) AS ca_state#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#9, 10, true, false, true) AS ca_zip#11] - -(11) CometBroadcastExchange -Input [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] -Arguments: [ca_address_sk#7, ca_state#10, ca_zip#11] - -(12) CometBroadcastHashJoin -Left output [3]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] -Right output [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] -Arguments: [c_current_addr_sk#6], [ca_address_sk#7], Inner, ((substr(ca_zip#11, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#10 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)), BuildRight - -(13) CometProject -Input [6]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, ca_address_sk#7, ca_state#10, ca_zip#11] -Arguments: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11], [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2)) AND (d_year#13 = 2001)) AND isnotnull(d_date_sk#12)) - -(16) CometProject -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Arguments: [d_date_sk#12], [d_date_sk#12] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: [d_date_sk#12] - -(18) CometBroadcastHashJoin -Left output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11] -Right output [1]: [d_date_sk#12] -Arguments: [cs_sold_date_sk#3], [d_date_sk#12], Inner, BuildRight - -(19) CometProject -Input [4]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11, d_date_sk#12] -Arguments: [cs_sales_price#2, ca_zip#11], [cs_sales_price#2, ca_zip#11] - -(20) CometHashAggregate -Input [2]: [cs_sales_price#2, ca_zip#11] -Keys [1]: [ca_zip#11] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] - -(21) CometExchange -Input [2]: [ca_zip#11, sum#15] -Arguments: hashpartitioning(ca_zip#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [2]: [ca_zip#11, sum#15] -Keys [1]: [ca_zip#11] -Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] - -(23) CometTakeOrderedAndProject -Input [2]: [ca_zip#11, sum(cs_sales_price)#16] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_zip#11 ASC NULLS FIRST], output=[ca_zip#11,sum(cs_sales_price)#16]), [ca_zip#11, sum(cs_sales_price)#16], 100, 0, [ca_zip#11 ASC NULLS FIRST], [ca_zip#11, sum(cs_sales_price)#16] - -(24) CometColumnarToRow [codegen id : 1] -Input [2]: [ca_zip#11, sum(cs_sales_price)#16] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2)) AND (d_year#13 = 2001)) AND isnotnull(d_date_sk#12)) - -(27) CometProject -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Arguments: [d_date_sk#12], [d_date_sk#12] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#12] - -(29) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/extended.txt deleted file mode 100644 index 6de0c64850..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/extended.txt +++ /dev/null @@ -1,32 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index c39b96efe3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15/simplified.txt +++ /dev/null @@ -1,34 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - CometHashAggregate [sum] [ca_zip,sum(cs_sales_price),sum(UnscaledValue(cs_sales_price))] - CometExchange [ca_zip] #1 - CometHashAggregate [cs_sales_price] [ca_zip,sum] - CometProject [cs_sales_price,ca_zip] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] - CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip] - CometFilter [ca_address_sk,ca_state,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_datafusion/explain.txt deleted file mode 100644 index 2a3c8932ec..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_datafusion/explain.txt +++ /dev/null @@ -1,235 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometColumnarExchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * CometColumnarToRow (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometNativeScan parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometNativeScan parquet spark_catalog.default.call_center (29) - - -(1) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) - -(3) CometProject -Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(4) CometExchange -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -ReadSchema: struct - -(7) CometProject -Input [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] -Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_warehouse_sk#9, cs_order_number#10] - -(8) CometExchange -Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_order_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_order_number#5], [cs_order_number#10], LeftSemi, NOT (cs_warehouse_sk#4 = cs_warehouse_sk#9) - -(11) CometProject -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(12) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [2]: [cr_order_number#12, cr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -ReadSchema: struct - -(13) CometProject -Input [2]: [cr_order_number#12, cr_returned_date_sk#13] -Arguments: [cr_order_number#12], [cr_order_number#12] - -(14) CometExchange -Input [1]: [cr_order_number#12] -Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(15) CometSort -Input [1]: [cr_order_number#12] -Arguments: [cr_order_number#12], [cr_order_number#12 ASC NULLS FIRST] - -(16) CometSortMergeJoin -Left output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [cr_order_number#12] -Arguments: [cs_order_number#5], [cr_order_number#12], LeftAnti - -(17) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2002-02-01)) AND (d_date#15 <= 2002-04-02)) AND isnotnull(d_date_sk#14)) - -(19) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(20) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(21) CometBroadcastHashJoin -Left output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [d_date_sk#14] -Arguments: [cs_ship_date_sk#1], [d_date_sk#14], Inner, BuildRight - -(22) CometProject -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#14] -Arguments: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(23) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#17, 2, true, false, true) = GA) AND isnotnull(ca_address_sk#16)) - -(25) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(26) CometBroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: [ca_address_sk#16] - -(27) CometBroadcastHashJoin -Left output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [ca_address_sk#16] -Arguments: [cs_ship_addr_sk#2], [ca_address_sk#16], Inner, BuildRight - -(28) CometProject -Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] -Arguments: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(29) CometNativeScan parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#18, cc_county#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [cc_call_center_sk#18, cc_county#19] -Condition : ((isnotnull(cc_county#19) AND (cc_county#19 = Williamson County)) AND isnotnull(cc_call_center_sk#18)) - -(31) CometProject -Input [2]: [cc_call_center_sk#18, cc_county#19] -Arguments: [cc_call_center_sk#18], [cc_call_center_sk#18] - -(32) CometBroadcastExchange -Input [1]: [cc_call_center_sk#18] -Arguments: [cc_call_center_sk#18] - -(33) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [cc_call_center_sk#18] -Arguments: [cs_call_center_sk#3], [cc_call_center_sk#18], Inner, BuildRight - -(34) CometProject -Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#18] -Arguments: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(35) CometHashAggregate -Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Keys [1]: [cs_order_number#5] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] - -(36) CometColumnarToRow [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] - -(37) HashAggregate [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] -Keys [1]: [cs_order_number#5] -Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23] -Results [3]: [cs_order_number#5, sum#20, sum#21] - -(38) HashAggregate [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23, count(cs_order_number#5)#24] -Results [3]: [sum#20, sum#21, count#25] - -(39) CometColumnarExchange -Input [3]: [sum#20, sum#21, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(40) CometHashAggregate -Input [3]: [sum#20, sum#21, count#25] -Keys: [] -Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] - -(41) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #26, total shipping cost #27, total net profit #28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_datafusion/simplified.txt deleted file mode 100644 index 429d83d08c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_datafusion/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(cs_order_number),sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [cs_ext_ship_cost,cs_net_profit] [cs_order_number,sum,sum] - CometProject [cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cc_call_center_sk] - CometProject [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,ca_address_sk] - CometProject [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,d_date_sk] - CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cr_order_number] - CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometSortMergeJoin [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_order_number,cs_warehouse_sk] - CometSort [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] - CometExchange [cs_order_number] #2 - 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,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - CometNativeScan 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] - CometSort [cs_warehouse_sk,cs_order_number] - CometExchange [cs_order_number] #3 - CometProject [cs_warehouse_sk,cs_order_number] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] - CometSort [cr_order_number] - CometExchange [cr_order_number] #4 - CometProject [cr_order_number] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cc_call_center_sk] #7 - CometProject [cc_call_center_sk] - CometFilter [cc_call_center_sk,cc_county] - CometNativeScan 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/q16.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_iceberg_compat/explain.txt deleted file mode 100644 index f2473f1f01..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,235 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometColumnarExchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * CometColumnarToRow (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (29) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) - -(3) CometProject -Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(4) CometExchange -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -ReadSchema: struct - -(7) CometProject -Input [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] -Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_warehouse_sk#9, cs_order_number#10] - -(8) CometExchange -Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_order_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_order_number#5], [cs_order_number#10], LeftSemi, NOT (cs_warehouse_sk#4 = cs_warehouse_sk#9) - -(11) CometProject -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [2]: [cr_order_number#12, cr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -ReadSchema: struct - -(13) CometProject -Input [2]: [cr_order_number#12, cr_returned_date_sk#13] -Arguments: [cr_order_number#12], [cr_order_number#12] - -(14) CometExchange -Input [1]: [cr_order_number#12] -Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(15) CometSort -Input [1]: [cr_order_number#12] -Arguments: [cr_order_number#12], [cr_order_number#12 ASC NULLS FIRST] - -(16) CometSortMergeJoin -Left output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [cr_order_number#12] -Arguments: [cs_order_number#5], [cr_order_number#12], LeftAnti - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2002-02-01)) AND (d_date#15 <= 2002-04-02)) AND isnotnull(d_date_sk#14)) - -(19) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(20) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(21) CometBroadcastHashJoin -Left output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [d_date_sk#14] -Arguments: [cs_ship_date_sk#1], [d_date_sk#14], Inner, BuildRight - -(22) CometProject -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#14] -Arguments: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#17, 2, true, false, true) = GA) AND isnotnull(ca_address_sk#16)) - -(25) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(26) CometBroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: [ca_address_sk#16] - -(27) CometBroadcastHashJoin -Left output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [ca_address_sk#16] -Arguments: [cs_ship_addr_sk#2], [ca_address_sk#16], Inner, BuildRight - -(28) CometProject -Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] -Arguments: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#18, cc_county#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [cc_call_center_sk#18, cc_county#19] -Condition : ((isnotnull(cc_county#19) AND (cc_county#19 = Williamson County)) AND isnotnull(cc_call_center_sk#18)) - -(31) CometProject -Input [2]: [cc_call_center_sk#18, cc_county#19] -Arguments: [cc_call_center_sk#18], [cc_call_center_sk#18] - -(32) CometBroadcastExchange -Input [1]: [cc_call_center_sk#18] -Arguments: [cc_call_center_sk#18] - -(33) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [cc_call_center_sk#18] -Arguments: [cs_call_center_sk#3], [cc_call_center_sk#18], Inner, BuildRight - -(34) CometProject -Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#18] -Arguments: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(35) CometHashAggregate -Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Keys [1]: [cs_order_number#5] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] - -(36) CometColumnarToRow [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] - -(37) HashAggregate [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] -Keys [1]: [cs_order_number#5] -Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23] -Results [3]: [cs_order_number#5, sum#20, sum#21] - -(38) HashAggregate [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23, count(cs_order_number#5)#24] -Results [3]: [sum#20, sum#21, count#25] - -(39) CometColumnarExchange -Input [3]: [sum#20, sum#21, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(40) CometHashAggregate -Input [3]: [sum#20, sum#21, count#25] -Keys: [] -Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] - -(41) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #26, total shipping cost #27, total net profit #28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_iceberg_compat/simplified.txt deleted file mode 100644 index 8427aa49a9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(cs_order_number),sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [cs_ext_ship_cost,cs_net_profit] [cs_order_number,sum,sum] - CometProject [cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cc_call_center_sk] - CometProject [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,ca_address_sk] - CometProject [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,d_date_sk] - CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cr_order_number] - CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometSortMergeJoin [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_order_number,cs_warehouse_sk] - CometSort [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] - CometExchange [cs_order_number] #2 - 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,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [cs_warehouse_sk,cs_order_number] - CometExchange [cs_order_number] #3 - CometProject [cs_warehouse_sk,cs_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] - CometSort [cr_order_number] - CometExchange [cr_order_number] #4 - CometProject [cr_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cc_call_center_sk] #7 - CometProject [cc_call_center_sk] - CometFilter [cc_call_center_sk,cc_county] - CometScan [native_iceberg_compat] 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/q16/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/explain.txt deleted file mode 100644 index f2473f1f01..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/explain.txt +++ /dev/null @@ -1,235 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometColumnarExchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * CometColumnarToRow (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (29) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) - -(3) CometProject -Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(4) CometExchange -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -ReadSchema: struct - -(7) CometProject -Input [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] -Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_warehouse_sk#9, cs_order_number#10] - -(8) CometExchange -Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_order_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_order_number#5], [cs_order_number#10], LeftSemi, NOT (cs_warehouse_sk#4 = cs_warehouse_sk#9) - -(11) CometProject -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [2]: [cr_order_number#12, cr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -ReadSchema: struct - -(13) CometProject -Input [2]: [cr_order_number#12, cr_returned_date_sk#13] -Arguments: [cr_order_number#12], [cr_order_number#12] - -(14) CometExchange -Input [1]: [cr_order_number#12] -Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(15) CometSort -Input [1]: [cr_order_number#12] -Arguments: [cr_order_number#12], [cr_order_number#12 ASC NULLS FIRST] - -(16) CometSortMergeJoin -Left output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [cr_order_number#12] -Arguments: [cs_order_number#5], [cr_order_number#12], LeftAnti - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2002-02-01)) AND (d_date#15 <= 2002-04-02)) AND isnotnull(d_date_sk#14)) - -(19) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(20) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(21) CometBroadcastHashJoin -Left output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [d_date_sk#14] -Arguments: [cs_ship_date_sk#1], [d_date_sk#14], Inner, BuildRight - -(22) CometProject -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#14] -Arguments: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#17, 2, true, false, true) = GA) AND isnotnull(ca_address_sk#16)) - -(25) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(26) CometBroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: [ca_address_sk#16] - -(27) CometBroadcastHashJoin -Left output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [ca_address_sk#16] -Arguments: [cs_ship_addr_sk#2], [ca_address_sk#16], Inner, BuildRight - -(28) CometProject -Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] -Arguments: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#18, cc_county#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [cc_call_center_sk#18, cc_county#19] -Condition : ((isnotnull(cc_county#19) AND (cc_county#19 = Williamson County)) AND isnotnull(cc_call_center_sk#18)) - -(31) CometProject -Input [2]: [cc_call_center_sk#18, cc_county#19] -Arguments: [cc_call_center_sk#18], [cc_call_center_sk#18] - -(32) CometBroadcastExchange -Input [1]: [cc_call_center_sk#18] -Arguments: [cc_call_center_sk#18] - -(33) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [cc_call_center_sk#18] -Arguments: [cs_call_center_sk#3], [cc_call_center_sk#18], Inner, BuildRight - -(34) CometProject -Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#18] -Arguments: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(35) CometHashAggregate -Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Keys [1]: [cs_order_number#5] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] - -(36) CometColumnarToRow [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] - -(37) HashAggregate [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] -Keys [1]: [cs_order_number#5] -Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23] -Results [3]: [cs_order_number#5, sum#20, sum#21] - -(38) HashAggregate [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23, count(cs_order_number#5)#24] -Results [3]: [sum#20, sum#21, count#25] - -(39) CometColumnarExchange -Input [3]: [sum#20, sum#21, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(40) CometHashAggregate -Input [3]: [sum#20, sum#21, count#25] -Keys: [] -Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] - -(41) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #26, total shipping cost #27, total net profit #28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/extended.txt deleted file mode 100644 index 2ad029e444..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/extended.txt +++ /dev/null @@ -1,43 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - -Comet accelerated 37 out of 39 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 8427aa49a9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q16/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(cs_order_number),sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [cs_ext_ship_cost,cs_net_profit] [cs_order_number,sum,sum] - CometProject [cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cc_call_center_sk] - CometProject [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,ca_address_sk] - CometProject [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,d_date_sk] - CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cr_order_number] - CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometSortMergeJoin [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_order_number,cs_warehouse_sk] - CometSort [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] - CometExchange [cs_order_number] #2 - 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,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [cs_warehouse_sk,cs_order_number] - CometExchange [cs_order_number] #3 - CometProject [cs_warehouse_sk,cs_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] - CometSort [cr_order_number] - CometExchange [cr_order_number] #4 - CometProject [cr_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cc_call_center_sk] #7 - CometProject [cc_call_center_sk] - CometFilter [cc_call_center_sk,cc_county] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/explain.txt deleted file mode 100644 index 5005b676fc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/explain.txt +++ /dev/null @@ -1,312 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (18) - : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet spark_catalog.default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * Filter (12) - : : : : : +- * ColumnarToRow (11) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (10) - : : : : +- ReusedExchange (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.store (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.item (32) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 8] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] - -(3) Filter [codegen id : 8] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(4) Scan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(6) Filter [codegen id : 1] -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(7) BroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 8] -Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] -Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(10) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(12) Filter [codegen id : 2] -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(13) BroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] -Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(16) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#18] - -(17) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#18] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 8] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#18] - -(19) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#19] - -(20) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [sr_returned_date_sk#12] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] - -(22) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#20] - -(23) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 8] -Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#20] - -(25) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#21, s_state#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [s_store_sk#21, s_state#22] -Condition : isnotnull(s_store_sk#21) - -(27) CometProject -Input [2]: [s_store_sk#21, s_state#22] -Arguments: [s_store_sk#21, s_state#23], [s_store_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#22, 2, true, false, true) AS s_state#23] - -(28) CometColumnarToRow [codegen id : 6] -Input [2]: [s_store_sk#21, s_state#23] - -(29) BroadcastExchange -Input [2]: [s_store_sk#21, s_state#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(30) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#21] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 8] -Output [5]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23] -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#21, s_state#23] - -(32) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] -Condition : isnotnull(i_item_sk#24) - -(34) CometProject -Input [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] -Arguments: [i_item_sk#24, i_item_id#27, i_item_desc#26], [i_item_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#25, 16, true, false, true) AS i_item_id#27, i_item_desc#26] - -(35) CometColumnarToRow [codegen id : 7] -Input [3]: [i_item_sk#24, i_item_id#27, i_item_desc#26] - -(36) BroadcastExchange -Input [3]: [i_item_sk#24, i_item_id#27, i_item_desc#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(37) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#24] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 8] -Output [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_id#27, i_item_desc#26] -Input [8]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_sk#24, i_item_id#27, i_item_desc#26] - -(39) HashAggregate [codegen id : 8] -Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_id#27, i_item_desc#26] -Keys [3]: [i_item_id#27, i_item_desc#26, s_state#23] -Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partial_stddev_samp(cast(ss_quantity#5 as double)), partial_count(sr_return_quantity#11), partial_avg(sr_return_quantity#11), partial_stddev_samp(cast(sr_return_quantity#11 as double)), partial_count(cs_quantity#16), partial_avg(cs_quantity#16), partial_stddev_samp(cast(cs_quantity#16 as double))] -Aggregate Attributes [18]: [count#28, sum#29, count#30, n#31, avg#32, m2#33, count#34, sum#35, count#36, n#37, avg#38, m2#39, count#40, sum#41, count#42, n#43, avg#44, m2#45] -Results [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count#48, n#49, avg#50, m2#51, count#52, sum#53, count#54, n#55, avg#56, m2#57, count#58, sum#59, count#60, n#61, avg#62, m2#63] - -(40) CometColumnarExchange -Input [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count#48, n#49, avg#50, m2#51, count#52, sum#53, count#54, n#55, avg#56, m2#57, count#58, sum#59, count#60, n#61, avg#62, m2#63] -Arguments: hashpartitioning(i_item_id#27, i_item_desc#26, s_state#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(41) CometColumnarToRow [codegen id : 9] -Input [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count#48, n#49, avg#50, m2#51, count#52, sum#53, count#54, n#55, avg#56, m2#57, count#58, sum#59, count#60, n#61, avg#62, m2#63] - -(42) HashAggregate [codegen id : 9] -Input [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count#48, n#49, avg#50, m2#51, count#52, sum#53, count#54, n#55, avg#56, m2#57, count#58, sum#59, count#60, n#61, avg#62, m2#63] -Keys [3]: [i_item_id#27, i_item_desc#26, s_state#23] -Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#11), avg(sr_return_quantity#11), stddev_samp(cast(sr_return_quantity#11 as double)), count(cs_quantity#16), avg(cs_quantity#16), stddev_samp(cast(cs_quantity#16 as double))] -Aggregate Attributes [9]: [count(ss_quantity#5)#64, avg(ss_quantity#5)#65, stddev_samp(cast(ss_quantity#5 as double))#66, count(sr_return_quantity#11)#67, avg(sr_return_quantity#11)#68, stddev_samp(cast(sr_return_quantity#11 as double))#69, count(cs_quantity#16)#70, avg(cs_quantity#16)#71, stddev_samp(cast(cs_quantity#16 as double))#72] -Results [15]: [i_item_id#27, i_item_desc#26, s_state#23, count(ss_quantity#5)#64 AS store_sales_quantitycount#73, avg(ss_quantity#5)#65 AS store_sales_quantityave#74, stddev_samp(cast(ss_quantity#5 as double))#66 AS store_sales_quantitystdev#75, (stddev_samp(cast(ss_quantity#5 as double))#66 / avg(ss_quantity#5)#65) AS store_sales_quantitycov#76, count(sr_return_quantity#11)#67 AS as_store_returns_quantitycount#77, avg(sr_return_quantity#11)#68 AS as_store_returns_quantityave#78, stddev_samp(cast(sr_return_quantity#11 as double))#69 AS as_store_returns_quantitystdev#79, (stddev_samp(cast(sr_return_quantity#11 as double))#69 / avg(sr_return_quantity#11)#68) AS store_returns_quantitycov#80, count(cs_quantity#16)#70 AS catalog_sales_quantitycount#81, avg(cs_quantity#16)#71 AS catalog_sales_quantityave#82, (stddev_samp(cast(cs_quantity#16 as double))#72 / avg(cs_quantity#16)#71) AS catalog_sales_quantitystdev#83, (stddev_samp(cast(cs_quantity#16 as double))#72 / avg(cs_quantity#16)#71) AS catalog_sales_quantitycov#84] - -(43) TakeOrderedAndProject -Input [15]: [i_item_id#27, i_item_desc#26, s_state#23, store_sales_quantitycount#73, store_sales_quantityave#74, store_sales_quantitystdev#75, store_sales_quantitycov#76, as_store_returns_quantitycount#77, as_store_returns_quantityave#78, as_store_returns_quantitystdev#79, store_returns_quantitycov#80, catalog_sales_quantitycount#81, catalog_sales_quantityave#82, catalog_sales_quantitystdev#83, catalog_sales_quantitycov#84] -Arguments: 100, [i_item_id#27 ASC NULLS FIRST, i_item_desc#26 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#27, i_item_desc#26, s_state#23, store_sales_quantitycount#73, store_sales_quantityave#74, store_sales_quantitystdev#75, store_sales_quantitycov#76, as_store_returns_quantitycount#77, as_store_returns_quantityave#78, as_store_returns_quantitystdev#79, store_returns_quantitycov#80, catalog_sales_quantitycount#81, catalog_sales_quantityave#82, catalog_sales_quantitystdev#83, catalog_sales_quantitycov#84] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometNativeScan parquet spark_catalog.default.date_dim (44) - - -(44) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_quarter_name#85] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [2]: [d_date_sk#18, d_quarter_name#85] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_quarter_name#85, 6, true, false, true) = 2001Q1) AND isnotnull(d_date_sk#18)) - -(46) CometProject -Input [2]: [d_date_sk#18, d_quarter_name#85] -Arguments: [d_date_sk#18], [d_date_sk#18] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#18] - -(48) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometNativeScan parquet spark_catalog.default.date_dim (49) - - -(49) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_quarter_name#86] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [2]: [d_date_sk#19, d_quarter_name#86] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_quarter_name#86, 6, true, false, true) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#19)) - -(51) CometProject -Input [2]: [d_date_sk#19, d_quarter_name#86] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(53) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/simplified.txt deleted file mode 100644 index c63dd716a1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/simplified.txt +++ /dev/null @@ -1,79 +0,0 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] - WholeStageCodegen (9) - HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(ss_quantity),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(sr_return_quantity),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cs_quantity),stddev_samp(cast(cs_quantity as double)),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,s_state] #1 - WholeStageCodegen (8) - HashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] - Project [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [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] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Filter [cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometNativeScan 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/q17.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/explain.txt deleted file mode 100644 index f502a5cd8d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,319 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometProject (18) - : : : : +- CometBroadcastHashJoin (17) - : : : : :- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (8) - : : : : +- CometBroadcastExchange (16) - : : : : +- CometProject (15) - : : : : +- CometFilter (14) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : : : +- CometBroadcastExchange (22) - : : : +- CometProject (21) - : : : +- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - : : +- ReusedExchange (25) - : +- CometBroadcastExchange (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (28) - +- CometBroadcastExchange (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (34) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(5) CometBroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(6) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(10) CometBroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(11) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] -Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight - -(12) CometProject -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_quarter_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#19, d_quarter_name#20] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_quarter_name#20, 6, true, false, true) = 2001Q1) AND isnotnull(d_date_sk#19)) - -(15) CometProject -Input [2]: [d_date_sk#19, d_quarter_name#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#21, d_quarter_name#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#21, d_quarter_name#22] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_quarter_name#22, 6, true, false, true) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#21)) - -(21) CometProject -Input [2]: [d_date_sk#21, d_quarter_name#22] -Arguments: [d_date_sk#21], [d_date_sk#21] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: [d_date_sk#21] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#21] -Arguments: [sr_returned_date_sk#12], [d_date_sk#21], Inner, BuildRight - -(24) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#21] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] - -(25) ReusedExchange [Reuses operator id: 22] -Output [1]: [d_date_sk#23] - -(26) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#23] -Arguments: [cs_sold_date_sk#17], [d_date_sk#23], Inner, BuildRight - -(27) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#23] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_state#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(29) CometFilter -Input [2]: [s_store_sk#24, s_state#25] -Condition : isnotnull(s_store_sk#24) - -(30) CometProject -Input [2]: [s_store_sk#24, s_state#25] -Arguments: [s_store_sk#24, s_state#26], [s_store_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#25, 2, true, false, true) AS s_state#26] - -(31) CometBroadcastExchange -Input [2]: [s_store_sk#24, s_state#26] -Arguments: [s_store_sk#24, s_state#26] - -(32) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Right output [2]: [s_store_sk#24, s_state#26] -Arguments: [ss_store_sk#3], [s_store_sk#24], Inner, BuildRight - -(33) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#24, s_state#26] -Arguments: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26], [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26] - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#27, i_item_id#28, i_item_desc#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [i_item_sk#27, i_item_id#28, i_item_desc#29] -Condition : isnotnull(i_item_sk#27) - -(36) CometProject -Input [3]: [i_item_sk#27, i_item_id#28, i_item_desc#29] -Arguments: [i_item_sk#27, i_item_id#30, i_item_desc#29], [i_item_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#28, 16, true, false, true) AS i_item_id#30, i_item_desc#29] - -(37) CometBroadcastExchange -Input [3]: [i_item_sk#27, i_item_id#30, i_item_desc#29] -Arguments: [i_item_sk#27, i_item_id#30, i_item_desc#29] - -(38) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26] -Right output [3]: [i_item_sk#27, i_item_id#30, i_item_desc#29] -Arguments: [ss_item_sk#1], [i_item_sk#27], Inner, BuildRight - -(39) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_sk#27, i_item_id#30, i_item_desc#29] -Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#30, i_item_desc#29], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#30, i_item_desc#29] - -(40) CometHashAggregate -Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#30, i_item_desc#29] -Keys [3]: [i_item_id#30, i_item_desc#29, s_state#26] -Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partial_stddev_samp(cast(ss_quantity#5 as double)), partial_count(sr_return_quantity#11), partial_avg(sr_return_quantity#11), partial_stddev_samp(cast(sr_return_quantity#11 as double)), partial_count(cs_quantity#16), partial_avg(cs_quantity#16), partial_stddev_samp(cast(cs_quantity#16 as double))] - -(41) CometExchange -Input [21]: [i_item_id#30, i_item_desc#29, s_state#26, count#31, sum#32, count#33, n#34, avg#35, m2#36, count#37, sum#38, count#39, n#40, avg#41, m2#42, count#43, sum#44, count#45, n#46, avg#47, m2#48] -Arguments: hashpartitioning(i_item_id#30, i_item_desc#29, s_state#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(42) CometHashAggregate -Input [21]: [i_item_id#30, i_item_desc#29, s_state#26, count#31, sum#32, count#33, n#34, avg#35, m2#36, count#37, sum#38, count#39, n#40, avg#41, m2#42, count#43, sum#44, count#45, n#46, avg#47, m2#48] -Keys [3]: [i_item_id#30, i_item_desc#29, s_state#26] -Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#11), avg(sr_return_quantity#11), stddev_samp(cast(sr_return_quantity#11 as double)), count(cs_quantity#16), avg(cs_quantity#16), stddev_samp(cast(cs_quantity#16 as double))] - -(43) CometTakeOrderedAndProject -Input [15]: [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#30 ASC NULLS FIRST,i_item_desc#29 ASC NULLS FIRST,s_state#26 ASC NULLS FIRST], output=[i_item_id#30,i_item_desc#29,s_state#26,store_sales_quantitycount#49,store_sales_quantityave#50,store_sales_quantitystdev#51,store_sales_quantitycov#52,as_store_returns_quantitycount#53,as_store_returns_quantityave#54,as_store_returns_quantitystdev#55,store_returns_quantitycov#56,catalog_sales_quantitycount#57,catalog_sales_quantityave#58,catalog_sales_quantitystdev#59,catalog_sales_quantitycov#60]), [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60], 100, 0, [i_item_id#30 ASC NULLS FIRST, i_item_desc#29 ASC NULLS FIRST, s_state#26 ASC NULLS FIRST], [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60] - -(44) CometColumnarToRow [codegen id : 1] -Input [15]: [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_quarter_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [d_date_sk#19, d_quarter_name#20] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_quarter_name#20, 6, true, false, true) = 2001Q1) AND isnotnull(d_date_sk#19)) - -(47) CometProject -Input [2]: [d_date_sk#19, d_quarter_name#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(49) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) - - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#21, d_quarter_name#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [2]: [d_date_sk#21, d_quarter_name#22] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_quarter_name#22, 6, true, false, true) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#21)) - -(52) CometProject -Input [2]: [d_date_sk#21, d_quarter_name#22] -Arguments: [d_date_sk#21], [d_date_sk#21] - -(53) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#21] - -(54) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/simplified.txt deleted file mode 100644 index 1d48f96b09..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,63 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] - CometHashAggregate [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count(ss_quantity),avg(ss_quantity),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(sr_return_quantity),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cs_quantity),stddev_samp(cast(cs_quantity as double))] - CometExchange [i_item_id,i_item_desc,s_state] #1 - CometHashAggregate [ss_quantity,sr_return_quantity,cs_quantity] [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] - CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - 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_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 [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 [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_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 [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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - 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 [native_iceberg_compat] 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 [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan [native_iceberg_compat] 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/q17/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/explain.txt deleted file mode 100644 index f502a5cd8d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/explain.txt +++ /dev/null @@ -1,319 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometProject (18) - : : : : +- CometBroadcastHashJoin (17) - : : : : :- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (8) - : : : : +- CometBroadcastExchange (16) - : : : : +- CometProject (15) - : : : : +- CometFilter (14) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : : : +- CometBroadcastExchange (22) - : : : +- CometProject (21) - : : : +- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - : : +- ReusedExchange (25) - : +- CometBroadcastExchange (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (28) - +- CometBroadcastExchange (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (34) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(5) CometBroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(6) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(10) CometBroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(11) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] -Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight - -(12) CometProject -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_quarter_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#19, d_quarter_name#20] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_quarter_name#20, 6, true, false, true) = 2001Q1) AND isnotnull(d_date_sk#19)) - -(15) CometProject -Input [2]: [d_date_sk#19, d_quarter_name#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#21, d_quarter_name#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#21, d_quarter_name#22] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_quarter_name#22, 6, true, false, true) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#21)) - -(21) CometProject -Input [2]: [d_date_sk#21, d_quarter_name#22] -Arguments: [d_date_sk#21], [d_date_sk#21] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: [d_date_sk#21] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#21] -Arguments: [sr_returned_date_sk#12], [d_date_sk#21], Inner, BuildRight - -(24) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#21] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] - -(25) ReusedExchange [Reuses operator id: 22] -Output [1]: [d_date_sk#23] - -(26) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#23] -Arguments: [cs_sold_date_sk#17], [d_date_sk#23], Inner, BuildRight - -(27) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#23] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_state#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(29) CometFilter -Input [2]: [s_store_sk#24, s_state#25] -Condition : isnotnull(s_store_sk#24) - -(30) CometProject -Input [2]: [s_store_sk#24, s_state#25] -Arguments: [s_store_sk#24, s_state#26], [s_store_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#25, 2, true, false, true) AS s_state#26] - -(31) CometBroadcastExchange -Input [2]: [s_store_sk#24, s_state#26] -Arguments: [s_store_sk#24, s_state#26] - -(32) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Right output [2]: [s_store_sk#24, s_state#26] -Arguments: [ss_store_sk#3], [s_store_sk#24], Inner, BuildRight - -(33) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#24, s_state#26] -Arguments: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26], [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26] - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#27, i_item_id#28, i_item_desc#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [i_item_sk#27, i_item_id#28, i_item_desc#29] -Condition : isnotnull(i_item_sk#27) - -(36) CometProject -Input [3]: [i_item_sk#27, i_item_id#28, i_item_desc#29] -Arguments: [i_item_sk#27, i_item_id#30, i_item_desc#29], [i_item_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#28, 16, true, false, true) AS i_item_id#30, i_item_desc#29] - -(37) CometBroadcastExchange -Input [3]: [i_item_sk#27, i_item_id#30, i_item_desc#29] -Arguments: [i_item_sk#27, i_item_id#30, i_item_desc#29] - -(38) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26] -Right output [3]: [i_item_sk#27, i_item_id#30, i_item_desc#29] -Arguments: [ss_item_sk#1], [i_item_sk#27], Inner, BuildRight - -(39) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_sk#27, i_item_id#30, i_item_desc#29] -Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#30, i_item_desc#29], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#30, i_item_desc#29] - -(40) CometHashAggregate -Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#30, i_item_desc#29] -Keys [3]: [i_item_id#30, i_item_desc#29, s_state#26] -Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partial_stddev_samp(cast(ss_quantity#5 as double)), partial_count(sr_return_quantity#11), partial_avg(sr_return_quantity#11), partial_stddev_samp(cast(sr_return_quantity#11 as double)), partial_count(cs_quantity#16), partial_avg(cs_quantity#16), partial_stddev_samp(cast(cs_quantity#16 as double))] - -(41) CometExchange -Input [21]: [i_item_id#30, i_item_desc#29, s_state#26, count#31, sum#32, count#33, n#34, avg#35, m2#36, count#37, sum#38, count#39, n#40, avg#41, m2#42, count#43, sum#44, count#45, n#46, avg#47, m2#48] -Arguments: hashpartitioning(i_item_id#30, i_item_desc#29, s_state#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(42) CometHashAggregate -Input [21]: [i_item_id#30, i_item_desc#29, s_state#26, count#31, sum#32, count#33, n#34, avg#35, m2#36, count#37, sum#38, count#39, n#40, avg#41, m2#42, count#43, sum#44, count#45, n#46, avg#47, m2#48] -Keys [3]: [i_item_id#30, i_item_desc#29, s_state#26] -Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#11), avg(sr_return_quantity#11), stddev_samp(cast(sr_return_quantity#11 as double)), count(cs_quantity#16), avg(cs_quantity#16), stddev_samp(cast(cs_quantity#16 as double))] - -(43) CometTakeOrderedAndProject -Input [15]: [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#30 ASC NULLS FIRST,i_item_desc#29 ASC NULLS FIRST,s_state#26 ASC NULLS FIRST], output=[i_item_id#30,i_item_desc#29,s_state#26,store_sales_quantitycount#49,store_sales_quantityave#50,store_sales_quantitystdev#51,store_sales_quantitycov#52,as_store_returns_quantitycount#53,as_store_returns_quantityave#54,as_store_returns_quantitystdev#55,store_returns_quantitycov#56,catalog_sales_quantitycount#57,catalog_sales_quantityave#58,catalog_sales_quantitystdev#59,catalog_sales_quantitycov#60]), [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60], 100, 0, [i_item_id#30 ASC NULLS FIRST, i_item_desc#29 ASC NULLS FIRST, s_state#26 ASC NULLS FIRST], [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60] - -(44) CometColumnarToRow [codegen id : 1] -Input [15]: [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_quarter_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [d_date_sk#19, d_quarter_name#20] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_quarter_name#20, 6, true, false, true) = 2001Q1) AND isnotnull(d_date_sk#19)) - -(47) CometProject -Input [2]: [d_date_sk#19, d_quarter_name#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(49) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) - - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#21, d_quarter_name#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [2]: [d_date_sk#21, d_quarter_name#22] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_quarter_name#22, 6, true, false, true) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#21)) - -(52) CometProject -Input [2]: [d_date_sk#21, d_quarter_name#22] -Arguments: [d_date_sk#21], [d_date_sk#21] - -(53) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#21] - -(54) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/extended.txt deleted file mode 100644 index 4e1a4f3ec0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/extended.txt +++ /dev/null @@ -1,62 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 1d48f96b09..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17/simplified.txt +++ /dev/null @@ -1,63 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] - CometHashAggregate [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count(ss_quantity),avg(ss_quantity),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(sr_return_quantity),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cs_quantity),stddev_samp(cast(cs_quantity as double))] - CometExchange [i_item_id,i_item_desc,s_state] #1 - CometHashAggregate [ss_quantity,sr_return_quantity,cs_quantity] [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] - CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - 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_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 [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 [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_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 [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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - 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 [native_iceberg_compat] 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 [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/explain.txt deleted file mode 100644 index 0503116eb3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/explain.txt +++ /dev/null @@ -1,295 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (46) -+- * HashAggregate (45) - +- * CometColumnarToRow (44) - +- CometColumnarExchange (43) - +- * HashAggregate (42) - +- * Expand (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * Project (23) - : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : :- * Project (17) - : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : :- * Project (10) - : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : :- * Filter (3) - : : : : : : +- * ColumnarToRow (2) - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : : +- BroadcastExchange (8) - : : : : : +- * CometColumnarToRow (7) - : : : : : +- CometProject (6) - : : : : : +- CometFilter (5) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) - : : : : +- BroadcastExchange (15) - : : : : +- * CometColumnarToRow (14) - : : : : +- CometProject (13) - : : : : +- CometFilter (12) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (11) - : : : +- BroadcastExchange (21) - : : : +- * CometColumnarToRow (20) - : : : +- CometFilter (19) - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (18) - : : +- BroadcastExchange (28) - : : +- * CometColumnarToRow (27) - : : +- CometProject (26) - : : +- CometFilter (25) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (24) - : +- ReusedExchange (31) - +- BroadcastExchange (38) - +- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometNativeScan parquet spark_catalog.default.item (34) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 7] -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] - -(3) Filter [codegen id : 7] -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) - -(4) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#12, 1, true, false, true) = F) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = Unknown )) AND isnotnull(cd_demo_sk#11)) - -(6) CometProject -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] - -(7) CometColumnarToRow [codegen id : 1] -Input [2]: [cd_demo_sk#11, cd_dep_count#14] - -(8) BroadcastExchange -Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 7] -Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] - -(11) CometNativeScan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [In(c_birth_month, [1,12,2,6,8,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(12) CometFilter -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Condition : (((c_birth_month#18 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) - -(13) CometProject -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(14) CometColumnarToRow [codegen id : 2] -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(15) BroadcastExchange -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(18) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(19) CometFilter -Input [1]: [cd_demo_sk#20] -Condition : isnotnull(cd_demo_sk#20) - -(20) CometColumnarToRow [codegen id : 3] -Input [1]: [cd_demo_sk#20] - -(21) BroadcastExchange -Input [1]: [cd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#16] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None - -(23) Project [codegen id : 7] -Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] - -(24) CometNativeScan parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#21)) - -(26) CometProject -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) AS ca_state#25, ca_country#24] - -(27) CometColumnarToRow [codegen id : 4] -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(28) BroadcastExchange -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(29) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#17] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None - -(30) Project [codegen id : 7] -Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(31) ReusedExchange [Reuses operator id: 51] -Output [1]: [d_date_sk#26] - -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#26] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] - -(34) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#27, i_item_id#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [i_item_sk#27, i_item_id#28] -Condition : isnotnull(i_item_sk#27) - -(36) CometProject -Input [2]: [i_item_sk#27, i_item_id#28] -Arguments: [i_item_sk#27, i_item_id#29], [i_item_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#28, 16, true, false, true) AS i_item_id#29] - -(37) CometColumnarToRow [codegen id : 6] -Input [2]: [i_item_sk#27, i_item_id#29] - -(38) BroadcastExchange -Input [2]: [i_item_sk#27, i_item_id#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#27] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 7] -Output [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, ca_county#22] -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#27, i_item_id#29] - -(41) Expand [codegen id : 7] -Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, ca_county#22] -Arguments: [[cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, ca_county#22, 0], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, null, 1], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, null, null, 3], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, null, null, null, 7], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, null, null, null, null, 15]], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] - -(42) HashAggregate [codegen id : 7] -Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] -Keys [5]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] -Functions [7]: [partial_avg(cast(cs_quantity#4 as decimal(12,2))), partial_avg(cast(cs_list_price#5 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#7 as decimal(12,2))), partial_avg(cast(cs_sales_price#6 as decimal(12,2))), partial_avg(cast(cs_net_profit#8 as decimal(12,2))), partial_avg(cast(c_birth_year#19 as decimal(12,2))), partial_avg(cast(cd_dep_count#14 as decimal(12,2)))] -Aggregate Attributes [14]: [sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42, sum#43, count#44, sum#45, count#46, sum#47, count#48] -Results [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] - -(43) CometColumnarExchange -Input [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Arguments: hashpartitioning(i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(44) CometColumnarToRow [codegen id : 8] -Input [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] - -(45) HashAggregate [codegen id : 8] -Input [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Keys [5]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] -Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_price#5 as decimal(12,2))), avg(cast(cs_coupon_amt#7 as decimal(12,2))), avg(cast(cs_sales_price#6 as decimal(12,2))), avg(cast(cs_net_profit#8 as decimal(12,2))), avg(cast(c_birth_year#19 as decimal(12,2))), avg(cast(cd_dep_count#14 as decimal(12,2)))] -Aggregate Attributes [7]: [avg(cast(cs_quantity#4 as decimal(12,2)))#63, avg(cast(cs_list_price#5 as decimal(12,2)))#64, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#65, avg(cast(cs_sales_price#6 as decimal(12,2)))#66, avg(cast(cs_net_profit#8 as decimal(12,2)))#67, avg(cast(c_birth_year#19 as decimal(12,2)))#68, avg(cast(cd_dep_count#14 as decimal(12,2)))#69] -Results [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, avg(cast(cs_quantity#4 as decimal(12,2)))#63 AS agg1#70, avg(cast(cs_list_price#5 as decimal(12,2)))#64 AS agg2#71, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#65 AS agg3#72, avg(cast(cs_sales_price#6 as decimal(12,2)))#66 AS agg4#73, avg(cast(cs_net_profit#8 as decimal(12,2)))#67 AS agg5#74, avg(cast(c_birth_year#19 as decimal(12,2)))#68 AS agg6#75, avg(cast(cd_dep_count#14 as decimal(12,2)))#69 AS agg7#76] - -(46) TakeOrderedAndProject -Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#70, agg2#71, agg3#72, agg4#73, agg5#74, agg6#75, agg7#76] -Arguments: 100, [ca_country#31 ASC NULLS FIRST, ca_state#32 ASC NULLS FIRST, ca_county#33 ASC NULLS FIRST, i_item_id#30 ASC NULLS FIRST], [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#70, agg2#71, agg3#72, agg4#73, agg5#74, agg6#75, agg7#76] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (51) -+- * CometColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometNativeScan parquet spark_catalog.default.date_dim (47) - - -(47) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#77] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(48) CometFilter -Input [2]: [d_date_sk#26, d_year#77] -Condition : ((isnotnull(d_year#77) AND (d_year#77 = 1998)) AND isnotnull(d_date_sk#26)) - -(49) CometProject -Input [2]: [d_date_sk#26, d_year#77] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(50) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(51) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/simplified.txt deleted file mode 100644 index 77a45c46cd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - WholeStageCodegen (8) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2))),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 - WholeStageCodegen (7) - HashAggregate [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] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Expand [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] - Project [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] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [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] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan 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/q18.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/explain.txt deleted file mode 100644 index 5fefd21bfa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,280 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (43) -+- CometTakeOrderedAndProject (42) - +- CometHashAggregate (41) - +- CometExchange (40) - +- CometHashAggregate (39) - +- CometExpand (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (19) - : : : +- CometBroadcastHashJoin (18) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometProject (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : : : +- CometBroadcastExchange (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (15) - : : +- CometBroadcastExchange (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#12, 1, true, false, true) = F) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = Unknown )) AND isnotnull(cd_demo_sk#11)) - -(5) CometProject -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] - -(6) CometBroadcastExchange -Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14] - -(7) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Right output [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight - -(8) CometProject -Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [In(c_birth_month, [1,12,2,6,8,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Condition : (((c_birth_month#18 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) - -(11) CometProject -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(12) CometBroadcastExchange -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(13) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight - -(14) CometProject -Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [cd_demo_sk#20] -Condition : isnotnull(cd_demo_sk#20) - -(17) CometBroadcastExchange -Input [1]: [cd_demo_sk#20] -Arguments: [cd_demo_sk#20] - -(18) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Right output [1]: [cd_demo_sk#20] -Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight - -(19) CometProject -Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#21)) - -(22) CometProject -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) AS ca_state#25, ca_country#24] - -(23) CometBroadcastExchange -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(24) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -Right output [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight - -(25) CometProject -Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 1998)) AND isnotnull(d_date_sk#26)) - -(28) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(29) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] - -(30) CometBroadcastHashJoin -Left output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [1]: [d_date_sk#26] -Arguments: [cs_sold_date_sk#9], [d_date_sk#26], Inner, BuildRight - -(31) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#28, i_item_id#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [i_item_sk#28, i_item_id#29] -Condition : isnotnull(i_item_sk#28) - -(34) CometProject -Input [2]: [i_item_sk#28, i_item_id#29] -Arguments: [i_item_sk#28, i_item_id#30], [i_item_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#29, 16, true, false, true) AS i_item_id#30] - -(35) CometBroadcastExchange -Input [2]: [i_item_sk#28, i_item_id#30] -Arguments: [i_item_sk#28, i_item_id#30] - -(36) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [2]: [i_item_sk#28, i_item_id#30] -Arguments: [cs_item_sk#3], [i_item_sk#28], Inner, BuildRight - -(37) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#28, i_item_id#30] -Arguments: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22] - -(38) CometExpand -Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22] -Arguments: [[cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22, 0], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, null, 1], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, null, null, 3], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, null, null, null, 7], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, null, null, null, null, 15]], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] - -(39) CometHashAggregate -Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] -Keys [5]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] -Functions [7]: [partial_avg(cast(cs_quantity#4 as decimal(12,2))), partial_avg(cast(cs_list_price#5 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#7 as decimal(12,2))), partial_avg(cast(cs_sales_price#6 as decimal(12,2))), partial_avg(cast(cs_net_profit#8 as decimal(12,2))), partial_avg(cast(c_birth_year#19 as decimal(12,2))), partial_avg(cast(cd_dep_count#14 as decimal(12,2)))] - -(40) CometExchange -Input [19]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35, sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49] -Arguments: hashpartitioning(i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(41) CometHashAggregate -Input [19]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35, sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49] -Keys [5]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] -Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_price#5 as decimal(12,2))), avg(cast(cs_coupon_amt#7 as decimal(12,2))), avg(cast(cs_sales_price#6 as decimal(12,2))), avg(cast(cs_net_profit#8 as decimal(12,2))), avg(cast(c_birth_year#19 as decimal(12,2))), avg(cast(cd_dep_count#14 as decimal(12,2)))] - -(42) CometTakeOrderedAndProject -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#32 ASC NULLS FIRST,ca_state#33 ASC NULLS FIRST,ca_county#34 ASC NULLS FIRST,i_item_id#31 ASC NULLS FIRST], output=[i_item_id#31,ca_country#32,ca_state#33,ca_county#34,agg1#50,agg2#51,agg3#52,agg4#53,agg5#54,agg6#55,agg7#56]), [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56], 100, 0, [ca_country#32 ASC NULLS FIRST, ca_state#33 ASC NULLS FIRST, ca_county#34 ASC NULLS FIRST, i_item_id#31 ASC NULLS FIRST], [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56] - -(43) CometColumnarToRow [codegen id : 1] -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) - - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 1998)) AND isnotnull(d_date_sk#26)) - -(46) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(48) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/simplified.txt deleted file mode 100644 index 6c2b8b2e4f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,53 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometHashAggregate [spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2)))] - CometExchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 - CometHashAggregate [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,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,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,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_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 [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 [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,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_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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - 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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange [cd_demo_sk] #5 - CometFilter [cd_demo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #8 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] 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/q18/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/explain.txt deleted file mode 100644 index 5fefd21bfa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/explain.txt +++ /dev/null @@ -1,280 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (43) -+- CometTakeOrderedAndProject (42) - +- CometHashAggregate (41) - +- CometExchange (40) - +- CometHashAggregate (39) - +- CometExpand (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (19) - : : : +- CometBroadcastHashJoin (18) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometProject (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : : : +- CometBroadcastExchange (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (15) - : : +- CometBroadcastExchange (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#12, 1, true, false, true) = F) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = Unknown )) AND isnotnull(cd_demo_sk#11)) - -(5) CometProject -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] - -(6) CometBroadcastExchange -Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14] - -(7) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Right output [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight - -(8) CometProject -Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [In(c_birth_month, [1,12,2,6,8,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Condition : (((c_birth_month#18 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) - -(11) CometProject -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(12) CometBroadcastExchange -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(13) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight - -(14) CometProject -Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [cd_demo_sk#20] -Condition : isnotnull(cd_demo_sk#20) - -(17) CometBroadcastExchange -Input [1]: [cd_demo_sk#20] -Arguments: [cd_demo_sk#20] - -(18) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Right output [1]: [cd_demo_sk#20] -Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight - -(19) CometProject -Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#21)) - -(22) CometProject -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) AS ca_state#25, ca_country#24] - -(23) CometBroadcastExchange -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(24) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -Right output [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight - -(25) CometProject -Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 1998)) AND isnotnull(d_date_sk#26)) - -(28) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(29) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] - -(30) CometBroadcastHashJoin -Left output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [1]: [d_date_sk#26] -Arguments: [cs_sold_date_sk#9], [d_date_sk#26], Inner, BuildRight - -(31) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#28, i_item_id#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [i_item_sk#28, i_item_id#29] -Condition : isnotnull(i_item_sk#28) - -(34) CometProject -Input [2]: [i_item_sk#28, i_item_id#29] -Arguments: [i_item_sk#28, i_item_id#30], [i_item_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#29, 16, true, false, true) AS i_item_id#30] - -(35) CometBroadcastExchange -Input [2]: [i_item_sk#28, i_item_id#30] -Arguments: [i_item_sk#28, i_item_id#30] - -(36) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [2]: [i_item_sk#28, i_item_id#30] -Arguments: [cs_item_sk#3], [i_item_sk#28], Inner, BuildRight - -(37) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#28, i_item_id#30] -Arguments: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22] - -(38) CometExpand -Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22] -Arguments: [[cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22, 0], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, null, 1], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, null, null, 3], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, null, null, null, 7], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, null, null, null, null, 15]], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] - -(39) CometHashAggregate -Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] -Keys [5]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] -Functions [7]: [partial_avg(cast(cs_quantity#4 as decimal(12,2))), partial_avg(cast(cs_list_price#5 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#7 as decimal(12,2))), partial_avg(cast(cs_sales_price#6 as decimal(12,2))), partial_avg(cast(cs_net_profit#8 as decimal(12,2))), partial_avg(cast(c_birth_year#19 as decimal(12,2))), partial_avg(cast(cd_dep_count#14 as decimal(12,2)))] - -(40) CometExchange -Input [19]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35, sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49] -Arguments: hashpartitioning(i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(41) CometHashAggregate -Input [19]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35, sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49] -Keys [5]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] -Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_price#5 as decimal(12,2))), avg(cast(cs_coupon_amt#7 as decimal(12,2))), avg(cast(cs_sales_price#6 as decimal(12,2))), avg(cast(cs_net_profit#8 as decimal(12,2))), avg(cast(c_birth_year#19 as decimal(12,2))), avg(cast(cd_dep_count#14 as decimal(12,2)))] - -(42) CometTakeOrderedAndProject -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#32 ASC NULLS FIRST,ca_state#33 ASC NULLS FIRST,ca_county#34 ASC NULLS FIRST,i_item_id#31 ASC NULLS FIRST], output=[i_item_id#31,ca_country#32,ca_state#33,ca_county#34,agg1#50,agg2#51,agg3#52,agg4#53,agg5#54,agg6#55,agg7#56]), [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56], 100, 0, [ca_country#32 ASC NULLS FIRST, ca_state#33 ASC NULLS FIRST, ca_county#34 ASC NULLS FIRST, i_item_id#31 ASC NULLS FIRST], [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56] - -(43) CometColumnarToRow [codegen id : 1] -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) - - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 1998)) AND isnotnull(d_date_sk#26)) - -(46) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(48) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/extended.txt deleted file mode 100644 index 0b554c7e7c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/extended.txt +++ /dev/null @@ -1,51 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 6c2b8b2e4f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18/simplified.txt +++ /dev/null @@ -1,53 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometHashAggregate [spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2)))] - CometExchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 - CometHashAggregate [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,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,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,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_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 [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 [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,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_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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - 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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange [cd_demo_sk] #5 - CometFilter [cd_demo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #8 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/explain.txt deleted file mode 100644 index 539af26bc5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (36) -+- CometTakeOrderedAndProject (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometNativeScan parquet spark_catalog.default.item (9) - : : +- CometBroadcastExchange (17) - : : +- CometFilter (16) - : : +- CometNativeScan parquet spark_catalog.default.customer (15) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometNativeScan parquet spark_catalog.default.customer_address (20) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometFilter (27) - +- CometNativeScan parquet spark_catalog.default.store (26) - - -(1) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1998)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1], [d_date_sk#1] - -(4) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_item_sk#4) AND isnotnull(ss_customer_sk#5)) AND isnotnull(ss_store_sk#6)) - -(6) CometBroadcastExchange -Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] - -(7) CometBroadcastHashJoin -Left output [1]: [d_date_sk#1] -Right output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [d_date_sk#1], [ss_sold_date_sk#8], Inner, BuildRight - -(8) CometProject -Input [6]: [d_date_sk#1, ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7], [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] - -(9) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Condition : ((isnotnull(i_manager_id#14) AND (i_manager_id#14 = 8)) AND isnotnull(i_item_sk#9)) - -(11) CometProject -Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Arguments: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [i_item_sk#9, i_brand_id#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#11, 50, true, false, true) AS i_brand#15, i_manufact_id#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#13, 50, true, false, true) AS i_manufact#16] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] -Right output [5]: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [ss_item_sk#4], [i_item_sk#9], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(15) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(c_current_addr_sk#18)) - -(17) CometBroadcastExchange -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [c_customer_sk#17, c_current_addr_sk#18] - -(18) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Right output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_customer_sk#5], [c_customer_sk#17], Inner, BuildRight - -(19) CometProject -Input [9]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18] - -(20) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_zip#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [ca_address_sk#19, ca_zip#20] -Condition : (isnotnull(ca_address_sk#19) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#20, 10, true, false, true))) - -(22) CometProject -Input [2]: [ca_address_sk#19, ca_zip#20] -Arguments: [ca_address_sk#19, ca_zip#21], [ca_address_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#20, 10, true, false, true) AS ca_zip#21] - -(23) CometBroadcastExchange -Input [2]: [ca_address_sk#19, ca_zip#21] -Arguments: [ca_address_sk#19, ca_zip#21] - -(24) CometBroadcastHashJoin -Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18] -Right output [2]: [ca_address_sk#19, ca_zip#21] -Arguments: [c_current_addr_sk#18], [ca_address_sk#19], Inner, BuildRight - -(25) CometProject -Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18, ca_address_sk#19, ca_zip#21] -Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21] - -(26) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#22, s_zip#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [s_store_sk#22, s_zip#23] -Condition : (isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#23, 10, true, false, true)) AND isnotnull(s_store_sk#22)) - -(28) CometProject -Input [2]: [s_store_sk#22, s_zip#23] -Arguments: [s_store_sk#22, s_zip#24], [s_store_sk#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#23, 10, true, false, true) AS s_zip#24] - -(29) CometBroadcastExchange -Input [2]: [s_store_sk#22, s_zip#24] -Arguments: [s_store_sk#22, s_zip#24] - -(30) CometBroadcastHashJoin -Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21] -Right output [2]: [s_store_sk#22, s_zip#24] -Arguments: [ss_store_sk#6], [s_store_sk#22], Inner, NOT (substr(ca_zip#21, 1, 5) = substr(s_zip#24, 1, 5)), BuildRight - -(31) CometProject -Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21, s_store_sk#22, s_zip#24] -Arguments: [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(32) CometHashAggregate -Input [5]: [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Keys [4]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] - -(33) CometExchange -Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] -Arguments: hashpartitioning(i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(34) CometHashAggregate -Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] -Keys [4]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] - -(35) CometTakeOrderedAndProject -Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#28 DESC NULLS LAST,brand#27 ASC NULLS FIRST,brand_id#26 ASC NULLS FIRST,i_manufact_id#12 ASC NULLS FIRST,i_manufact#16 ASC NULLS FIRST], output=[brand_id#26,brand#27,i_manufact_id#12,i_manufact#16,ext_price#28]), [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28], 100, 0, [ext_price#28 DESC NULLS LAST, brand#27 ASC NULLS FIRST, brand_id#26 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#16 ASC NULLS FIRST], [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] - -(36) CometColumnarToRow [codegen id : 1] -Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/simplified.txt deleted file mode 100644 index 675500cd0b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_datafusion/simplified.txt +++ /dev/null @@ -1,38 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,i_manufact_id,i_manufact,ext_price] - CometHashAggregate [sum] [brand_id,brand,i_manufact_id,i_manufact,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 - CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] - CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - 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 [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,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,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_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 - CometProject [i_brand,i_manufact] [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_zip] #5 - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [s_store_sk,s_zip] #6 - CometProject [s_zip] [s_store_sk,s_zip] - CometFilter [s_store_sk,s_zip] - CometNativeScan 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/q19.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_iceberg_compat/explain.txt deleted file mode 100644 index 0724450064..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (36) -+- CometTakeOrderedAndProject (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - : : +- CometBroadcastExchange (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (15) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1998)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1], [d_date_sk#1] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_item_sk#4) AND isnotnull(ss_customer_sk#5)) AND isnotnull(ss_store_sk#6)) - -(6) CometBroadcastExchange -Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] - -(7) CometBroadcastHashJoin -Left output [1]: [d_date_sk#1] -Right output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [d_date_sk#1], [ss_sold_date_sk#8], Inner, BuildRight - -(8) CometProject -Input [6]: [d_date_sk#1, ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7], [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Condition : ((isnotnull(i_manager_id#14) AND (i_manager_id#14 = 8)) AND isnotnull(i_item_sk#9)) - -(11) CometProject -Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Arguments: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [i_item_sk#9, i_brand_id#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#11, 50, true, false, true) AS i_brand#15, i_manufact_id#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#13, 50, true, false, true) AS i_manufact#16] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] -Right output [5]: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [ss_item_sk#4], [i_item_sk#9], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(c_current_addr_sk#18)) - -(17) CometBroadcastExchange -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [c_customer_sk#17, c_current_addr_sk#18] - -(18) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Right output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_customer_sk#5], [c_customer_sk#17], Inner, BuildRight - -(19) CometProject -Input [9]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_zip#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [ca_address_sk#19, ca_zip#20] -Condition : (isnotnull(ca_address_sk#19) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#20, 10, true, false, true))) - -(22) CometProject -Input [2]: [ca_address_sk#19, ca_zip#20] -Arguments: [ca_address_sk#19, ca_zip#21], [ca_address_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#20, 10, true, false, true) AS ca_zip#21] - -(23) CometBroadcastExchange -Input [2]: [ca_address_sk#19, ca_zip#21] -Arguments: [ca_address_sk#19, ca_zip#21] - -(24) CometBroadcastHashJoin -Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18] -Right output [2]: [ca_address_sk#19, ca_zip#21] -Arguments: [c_current_addr_sk#18], [ca_address_sk#19], Inner, BuildRight - -(25) CometProject -Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18, ca_address_sk#19, ca_zip#21] -Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#22, s_zip#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [s_store_sk#22, s_zip#23] -Condition : (isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#23, 10, true, false, true)) AND isnotnull(s_store_sk#22)) - -(28) CometProject -Input [2]: [s_store_sk#22, s_zip#23] -Arguments: [s_store_sk#22, s_zip#24], [s_store_sk#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#23, 10, true, false, true) AS s_zip#24] - -(29) CometBroadcastExchange -Input [2]: [s_store_sk#22, s_zip#24] -Arguments: [s_store_sk#22, s_zip#24] - -(30) CometBroadcastHashJoin -Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21] -Right output [2]: [s_store_sk#22, s_zip#24] -Arguments: [ss_store_sk#6], [s_store_sk#22], Inner, NOT (substr(ca_zip#21, 1, 5) = substr(s_zip#24, 1, 5)), BuildRight - -(31) CometProject -Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21, s_store_sk#22, s_zip#24] -Arguments: [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(32) CometHashAggregate -Input [5]: [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Keys [4]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] - -(33) CometExchange -Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] -Arguments: hashpartitioning(i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(34) CometHashAggregate -Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] -Keys [4]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] - -(35) CometTakeOrderedAndProject -Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#28 DESC NULLS LAST,brand#27 ASC NULLS FIRST,brand_id#26 ASC NULLS FIRST,i_manufact_id#12 ASC NULLS FIRST,i_manufact#16 ASC NULLS FIRST], output=[brand_id#26,brand#27,i_manufact_id#12,i_manufact#16,ext_price#28]), [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28], 100, 0, [ext_price#28 DESC NULLS LAST, brand#27 ASC NULLS FIRST, brand_id#26 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#16 ASC NULLS FIRST], [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] - -(36) CometColumnarToRow [codegen id : 1] -Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_iceberg_compat/simplified.txt deleted file mode 100644 index 93ab89c142..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,38 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,i_manufact_id,i_manufact,ext_price] - CometHashAggregate [sum] [brand_id,brand,i_manufact_id,i_manufact,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 - CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] - CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - 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 [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,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,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_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 - CometProject [i_brand,i_manufact] [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_zip] #5 - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [s_store_sk,s_zip] #6 - CometProject [s_zip] [s_store_sk,s_zip] - CometFilter [s_store_sk,s_zip] - CometScan [native_iceberg_compat] 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/q19/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/explain.txt deleted file mode 100644 index 0724450064..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (36) -+- CometTakeOrderedAndProject (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - : : +- CometBroadcastExchange (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (15) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1998)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1], [d_date_sk#1] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_item_sk#4) AND isnotnull(ss_customer_sk#5)) AND isnotnull(ss_store_sk#6)) - -(6) CometBroadcastExchange -Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] - -(7) CometBroadcastHashJoin -Left output [1]: [d_date_sk#1] -Right output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [d_date_sk#1], [ss_sold_date_sk#8], Inner, BuildRight - -(8) CometProject -Input [6]: [d_date_sk#1, ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7], [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Condition : ((isnotnull(i_manager_id#14) AND (i_manager_id#14 = 8)) AND isnotnull(i_item_sk#9)) - -(11) CometProject -Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Arguments: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [i_item_sk#9, i_brand_id#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#11, 50, true, false, true) AS i_brand#15, i_manufact_id#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#13, 50, true, false, true) AS i_manufact#16] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] -Right output [5]: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [ss_item_sk#4], [i_item_sk#9], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(c_current_addr_sk#18)) - -(17) CometBroadcastExchange -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [c_customer_sk#17, c_current_addr_sk#18] - -(18) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Right output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_customer_sk#5], [c_customer_sk#17], Inner, BuildRight - -(19) CometProject -Input [9]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_zip#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [ca_address_sk#19, ca_zip#20] -Condition : (isnotnull(ca_address_sk#19) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#20, 10, true, false, true))) - -(22) CometProject -Input [2]: [ca_address_sk#19, ca_zip#20] -Arguments: [ca_address_sk#19, ca_zip#21], [ca_address_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#20, 10, true, false, true) AS ca_zip#21] - -(23) CometBroadcastExchange -Input [2]: [ca_address_sk#19, ca_zip#21] -Arguments: [ca_address_sk#19, ca_zip#21] - -(24) CometBroadcastHashJoin -Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18] -Right output [2]: [ca_address_sk#19, ca_zip#21] -Arguments: [c_current_addr_sk#18], [ca_address_sk#19], Inner, BuildRight - -(25) CometProject -Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18, ca_address_sk#19, ca_zip#21] -Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#22, s_zip#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [s_store_sk#22, s_zip#23] -Condition : (isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#23, 10, true, false, true)) AND isnotnull(s_store_sk#22)) - -(28) CometProject -Input [2]: [s_store_sk#22, s_zip#23] -Arguments: [s_store_sk#22, s_zip#24], [s_store_sk#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#23, 10, true, false, true) AS s_zip#24] - -(29) CometBroadcastExchange -Input [2]: [s_store_sk#22, s_zip#24] -Arguments: [s_store_sk#22, s_zip#24] - -(30) CometBroadcastHashJoin -Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21] -Right output [2]: [s_store_sk#22, s_zip#24] -Arguments: [ss_store_sk#6], [s_store_sk#22], Inner, NOT (substr(ca_zip#21, 1, 5) = substr(s_zip#24, 1, 5)), BuildRight - -(31) CometProject -Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21, s_store_sk#22, s_zip#24] -Arguments: [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(32) CometHashAggregate -Input [5]: [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Keys [4]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] - -(33) CometExchange -Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] -Arguments: hashpartitioning(i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(34) CometHashAggregate -Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] -Keys [4]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] - -(35) CometTakeOrderedAndProject -Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#28 DESC NULLS LAST,brand#27 ASC NULLS FIRST,brand_id#26 ASC NULLS FIRST,i_manufact_id#12 ASC NULLS FIRST,i_manufact#16 ASC NULLS FIRST], output=[brand_id#26,brand#27,i_manufact_id#12,i_manufact#16,ext_price#28]), [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28], 100, 0, [ext_price#28 DESC NULLS LAST, brand#27 ASC NULLS FIRST, brand_id#26 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#16 ASC NULLS FIRST], [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] - -(36) CometColumnarToRow [codegen id : 1] -Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/extended.txt deleted file mode 100644 index a8cfde2d31..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/extended.txt +++ /dev/null @@ -1,38 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 35 out of 35 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 93ab89c142..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q19/simplified.txt +++ /dev/null @@ -1,38 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,i_manufact_id,i_manufact,ext_price] - CometHashAggregate [sum] [brand_id,brand,i_manufact_id,i_manufact,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 - CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] - CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - 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 [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,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,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_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 - CometProject [i_brand,i_manufact] [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_zip] #5 - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [s_store_sk,s_zip] #6 - CometProject [s_zip] [s_store_sk,s_zip] - CometFilter [s_store_sk,s_zip] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_datafusion/explain.txt deleted file mode 100644 index 40e213818e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_datafusion/explain.txt +++ /dev/null @@ -1,193 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (34) -+- CometSort (33) - +- CometExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometHashAggregate (14) - : : +- CometExchange (13) - : : +- CometHashAggregate (12) - : : +- CometProject (11) - : : +- CometBroadcastHashJoin (10) - : : :- CometUnion (5) - : : : :- CometProject (2) - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales (1) - : : : +- CometProject (4) - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales (3) - : : +- CometBroadcastExchange (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (6) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometNativeScan parquet spark_catalog.default.date_dim (15) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometBroadcastHashJoin (27) - :- CometHashAggregate (22) - : +- ReusedExchange (21) - +- CometBroadcastExchange (26) - +- CometProject (25) - +- CometFilter (24) - +- CometNativeScan parquet spark_catalog.default.date_dim (23) - - -(1) CometNativeScan parquet spark_catalog.default.web_sales -Output [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ws_sold_date_sk#2)] -ReadSchema: struct - -(2) CometProject -Input [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] -Arguments: [sold_date_sk#3, sales_price#4], [ws_sold_date_sk#2 AS sold_date_sk#3, ws_ext_sales_price#1 AS sales_price#4] - -(3) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(cs_sold_date_sk#6)] -ReadSchema: struct - -(4) CometProject -Input [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] -Arguments: [sold_date_sk#7, sales_price#8], [cs_sold_date_sk#6 AS sold_date_sk#7, cs_ext_sales_price#5 AS sales_price#8] - -(5) CometUnion -Child 0 Input [2]: [sold_date_sk#3, sales_price#4] -Child 1 Input [2]: [sold_date_sk#7, sales_price#8] - -(6) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Condition : (isnotnull(d_date_sk#9) AND isnotnull(d_week_seq#10)) - -(8) CometProject -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Arguments: [d_date_sk#9, d_week_seq#10, d_day_name#12], [d_date_sk#9, d_week_seq#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_day_name#11, 9, true, false, true) AS d_day_name#12] - -(9) CometBroadcastExchange -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [d_date_sk#9, d_week_seq#10, d_day_name#12] - -(10) CometBroadcastHashJoin -Left output [2]: [sold_date_sk#3, sales_price#4] -Right output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [sold_date_sk#3], [d_date_sk#9], Inner, BuildRight - -(11) CometProject -Input [5]: [sold_date_sk#3, sales_price#4, d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [sales_price#4, d_week_seq#10, d_day_name#12], [sales_price#4, d_week_seq#10, d_day_name#12] - -(12) CometHashAggregate -Input [3]: [sales_price#4, d_week_seq#10, d_day_name#12] -Keys [1]: [d_week_seq#10] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(13) CometExchange -Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(14) CometHashAggregate -Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Keys [1]: [d_week_seq#10] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(15) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_week_seq)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [d_week_seq#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_week_seq#20)) - -(17) CometProject -Input [2]: [d_week_seq#20, d_year#21] -Arguments: [d_week_seq#20], [d_week_seq#20] - -(18) CometBroadcastExchange -Input [1]: [d_week_seq#20] -Arguments: [d_week_seq#20] - -(19) CometBroadcastHashJoin -Left output [8]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28] -Right output [1]: [d_week_seq#20] -Arguments: [d_week_seq#10], [d_week_seq#20], Inner, BuildRight - -(20) CometProject -Input [9]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28, d_week_seq#20] -Arguments: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36], [d_week_seq#10 AS d_week_seq1#29, sun_sales#22 AS sun_sales1#30, mon_sales#23 AS mon_sales1#31, tue_sales#24 AS tue_sales1#32, wed_sales#25 AS wed_sales1#33, thu_sales#26 AS thu_sales1#34, fri_sales#27 AS fri_sales1#35, sat_sales#28 AS sat_sales1#36] - -(21) ReusedExchange [Reuses operator id: 13] -Output [8]: [d_week_seq#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] - -(22) CometHashAggregate -Input [8]: [d_week_seq#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] -Keys [1]: [d_week_seq#37] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(23) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#45, d_year#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [d_week_seq#45, d_year#46] -Condition : ((isnotnull(d_year#46) AND (d_year#46 = 2002)) AND isnotnull(d_week_seq#45)) - -(25) CometProject -Input [2]: [d_week_seq#45, d_year#46] -Arguments: [d_week_seq#45], [d_week_seq#45] - -(26) CometBroadcastExchange -Input [1]: [d_week_seq#45] -Arguments: [d_week_seq#45] - -(27) CometBroadcastHashJoin -Left output [8]: [d_week_seq#37, sun_sales#47, mon_sales#48, tue_sales#49, wed_sales#50, thu_sales#51, fri_sales#52, sat_sales#53] -Right output [1]: [d_week_seq#45] -Arguments: [d_week_seq#37], [d_week_seq#45], Inner, BuildRight - -(28) CometProject -Input [9]: [d_week_seq#37, sun_sales#47, mon_sales#48, tue_sales#49, wed_sales#50, thu_sales#51, fri_sales#52, sat_sales#53, d_week_seq#45] -Arguments: [d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61], [d_week_seq#37 AS d_week_seq2#54, sun_sales#47 AS sun_sales2#55, mon_sales#48 AS mon_sales2#56, tue_sales#49 AS tue_sales2#57, wed_sales#50 AS wed_sales2#58, thu_sales#51 AS thu_sales2#59, fri_sales#52 AS fri_sales2#60, sat_sales#53 AS sat_sales2#61] - -(29) CometBroadcastExchange -Input [8]: [d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61] -Arguments: [d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61] - -(30) CometBroadcastHashJoin -Left output [8]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36] -Right output [8]: [d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61] -Arguments: [d_week_seq1#29], [(d_week_seq2#54 - 53)], Inner, BuildRight - -(31) CometProject -Input [16]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36, d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61] -Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68], [d_week_seq1#29, round((sun_sales1#30 / sun_sales2#55), 2) AS round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1#31 / mon_sales2#56), 2) AS round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1#32 / tue_sales2#57), 2) AS round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1#33 / wed_sales2#58), 2) AS round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1#34 / thu_sales2#59), 2) AS round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1#35 / fri_sales2#60), 2) AS round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1#36 / sat_sales2#61), 2) AS round((sat_sales1 / sat_sales2), 2)#68] - -(32) CometExchange -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68] -Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometSort -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68] -Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68], [d_week_seq1#29 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 1] -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_datafusion/simplified.txt deleted file mode 100644 index e4b6e81639..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_datafusion/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] - CometExchange [d_week_seq1] #1 - CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] - CometBroadcastHashJoin [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] - CometExchange [d_week_seq] #2 - CometHashAggregate [d_day_name,sales_price] [d_week_seq,sum,sum,sum,sum,sum,sum,sum] - CometProject [sales_price,d_week_seq,d_day_name] - 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] - CometNativeScan 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] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] - CometBroadcastExchange [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] - ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 - CometBroadcastExchange [d_week_seq] #6 - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometNativeScan 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/q2.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_iceberg_compat/explain.txt deleted file mode 100644 index 58e7d31f47..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,193 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (34) -+- CometSort (33) - +- CometExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometHashAggregate (14) - : : +- CometExchange (13) - : : +- CometHashAggregate (12) - : : +- CometProject (11) - : : +- CometBroadcastHashJoin (10) - : : :- CometUnion (5) - : : : :- CometProject (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometProject (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (3) - : : +- CometBroadcastExchange (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (6) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (15) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometBroadcastHashJoin (27) - :- CometHashAggregate (22) - : +- ReusedExchange (21) - +- CometBroadcastExchange (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (23) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#2)] -ReadSchema: struct - -(2) CometProject -Input [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] -Arguments: [sold_date_sk#3, sales_price#4], [ws_sold_date_sk#2 AS sold_date_sk#3, ws_ext_sales_price#1 AS sales_price#4] - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#6)] -ReadSchema: struct - -(4) CometProject -Input [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] -Arguments: [sold_date_sk#7, sales_price#8], [cs_sold_date_sk#6 AS sold_date_sk#7, cs_ext_sales_price#5 AS sales_price#8] - -(5) CometUnion -Child 0 Input [2]: [sold_date_sk#3, sales_price#4] -Child 1 Input [2]: [sold_date_sk#7, sales_price#8] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Condition : (isnotnull(d_date_sk#9) AND isnotnull(d_week_seq#10)) - -(8) CometProject -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Arguments: [d_date_sk#9, d_week_seq#10, d_day_name#12], [d_date_sk#9, d_week_seq#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_day_name#11, 9, true, false, true) AS d_day_name#12] - -(9) CometBroadcastExchange -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [d_date_sk#9, d_week_seq#10, d_day_name#12] - -(10) CometBroadcastHashJoin -Left output [2]: [sold_date_sk#3, sales_price#4] -Right output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [sold_date_sk#3], [d_date_sk#9], Inner, BuildRight - -(11) CometProject -Input [5]: [sold_date_sk#3, sales_price#4, d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [sales_price#4, d_week_seq#10, d_day_name#12], [sales_price#4, d_week_seq#10, d_day_name#12] - -(12) CometHashAggregate -Input [3]: [sales_price#4, d_week_seq#10, d_day_name#12] -Keys [1]: [d_week_seq#10] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(13) CometExchange -Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(14) CometHashAggregate -Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Keys [1]: [d_week_seq#10] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_week_seq)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [d_week_seq#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_week_seq#20)) - -(17) CometProject -Input [2]: [d_week_seq#20, d_year#21] -Arguments: [d_week_seq#20], [d_week_seq#20] - -(18) CometBroadcastExchange -Input [1]: [d_week_seq#20] -Arguments: [d_week_seq#20] - -(19) CometBroadcastHashJoin -Left output [8]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28] -Right output [1]: [d_week_seq#20] -Arguments: [d_week_seq#10], [d_week_seq#20], Inner, BuildRight - -(20) CometProject -Input [9]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28, d_week_seq#20] -Arguments: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36], [d_week_seq#10 AS d_week_seq1#29, sun_sales#22 AS sun_sales1#30, mon_sales#23 AS mon_sales1#31, tue_sales#24 AS tue_sales1#32, wed_sales#25 AS wed_sales1#33, thu_sales#26 AS thu_sales1#34, fri_sales#27 AS fri_sales1#35, sat_sales#28 AS sat_sales1#36] - -(21) ReusedExchange [Reuses operator id: 13] -Output [8]: [d_week_seq#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] - -(22) CometHashAggregate -Input [8]: [d_week_seq#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] -Keys [1]: [d_week_seq#37] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#45, d_year#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [d_week_seq#45, d_year#46] -Condition : ((isnotnull(d_year#46) AND (d_year#46 = 2002)) AND isnotnull(d_week_seq#45)) - -(25) CometProject -Input [2]: [d_week_seq#45, d_year#46] -Arguments: [d_week_seq#45], [d_week_seq#45] - -(26) CometBroadcastExchange -Input [1]: [d_week_seq#45] -Arguments: [d_week_seq#45] - -(27) CometBroadcastHashJoin -Left output [8]: [d_week_seq#37, sun_sales#47, mon_sales#48, tue_sales#49, wed_sales#50, thu_sales#51, fri_sales#52, sat_sales#53] -Right output [1]: [d_week_seq#45] -Arguments: [d_week_seq#37], [d_week_seq#45], Inner, BuildRight - -(28) CometProject -Input [9]: [d_week_seq#37, sun_sales#47, mon_sales#48, tue_sales#49, wed_sales#50, thu_sales#51, fri_sales#52, sat_sales#53, d_week_seq#45] -Arguments: [d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61], [d_week_seq#37 AS d_week_seq2#54, sun_sales#47 AS sun_sales2#55, mon_sales#48 AS mon_sales2#56, tue_sales#49 AS tue_sales2#57, wed_sales#50 AS wed_sales2#58, thu_sales#51 AS thu_sales2#59, fri_sales#52 AS fri_sales2#60, sat_sales#53 AS sat_sales2#61] - -(29) CometBroadcastExchange -Input [8]: [d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61] -Arguments: [d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61] - -(30) CometBroadcastHashJoin -Left output [8]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36] -Right output [8]: [d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61] -Arguments: [d_week_seq1#29], [(d_week_seq2#54 - 53)], Inner, BuildRight - -(31) CometProject -Input [16]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36, d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61] -Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68], [d_week_seq1#29, round((sun_sales1#30 / sun_sales2#55), 2) AS round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1#31 / mon_sales2#56), 2) AS round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1#32 / tue_sales2#57), 2) AS round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1#33 / wed_sales2#58), 2) AS round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1#34 / thu_sales2#59), 2) AS round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1#35 / fri_sales2#60), 2) AS round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1#36 / sat_sales2#61), 2) AS round((sat_sales1 / sat_sales2), 2)#68] - -(32) CometExchange -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68] -Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometSort -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68] -Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68], [d_week_seq1#29 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 1] -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_iceberg_compat/simplified.txt deleted file mode 100644 index 852c5fca0f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] - CometExchange [d_week_seq1] #1 - CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] - CometBroadcastHashJoin [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] - CometExchange [d_week_seq] #2 - CometHashAggregate [d_day_name,sales_price] [d_week_seq,sum,sum,sum,sum,sum,sum,sum] - CometProject [sales_price,d_week_seq,d_day_name] - 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 [native_iceberg_compat] 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year] - CometBroadcastExchange [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] - ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 - CometBroadcastExchange [d_week_seq] #6 - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometScan [native_iceberg_compat] 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/q2/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/explain.txt deleted file mode 100644 index 58e7d31f47..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/explain.txt +++ /dev/null @@ -1,193 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (34) -+- CometSort (33) - +- CometExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometHashAggregate (14) - : : +- CometExchange (13) - : : +- CometHashAggregate (12) - : : +- CometProject (11) - : : +- CometBroadcastHashJoin (10) - : : :- CometUnion (5) - : : : :- CometProject (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometProject (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (3) - : : +- CometBroadcastExchange (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (6) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (15) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometBroadcastHashJoin (27) - :- CometHashAggregate (22) - : +- ReusedExchange (21) - +- CometBroadcastExchange (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (23) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#2)] -ReadSchema: struct - -(2) CometProject -Input [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] -Arguments: [sold_date_sk#3, sales_price#4], [ws_sold_date_sk#2 AS sold_date_sk#3, ws_ext_sales_price#1 AS sales_price#4] - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#6)] -ReadSchema: struct - -(4) CometProject -Input [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] -Arguments: [sold_date_sk#7, sales_price#8], [cs_sold_date_sk#6 AS sold_date_sk#7, cs_ext_sales_price#5 AS sales_price#8] - -(5) CometUnion -Child 0 Input [2]: [sold_date_sk#3, sales_price#4] -Child 1 Input [2]: [sold_date_sk#7, sales_price#8] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Condition : (isnotnull(d_date_sk#9) AND isnotnull(d_week_seq#10)) - -(8) CometProject -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Arguments: [d_date_sk#9, d_week_seq#10, d_day_name#12], [d_date_sk#9, d_week_seq#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_day_name#11, 9, true, false, true) AS d_day_name#12] - -(9) CometBroadcastExchange -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [d_date_sk#9, d_week_seq#10, d_day_name#12] - -(10) CometBroadcastHashJoin -Left output [2]: [sold_date_sk#3, sales_price#4] -Right output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [sold_date_sk#3], [d_date_sk#9], Inner, BuildRight - -(11) CometProject -Input [5]: [sold_date_sk#3, sales_price#4, d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [sales_price#4, d_week_seq#10, d_day_name#12], [sales_price#4, d_week_seq#10, d_day_name#12] - -(12) CometHashAggregate -Input [3]: [sales_price#4, d_week_seq#10, d_day_name#12] -Keys [1]: [d_week_seq#10] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(13) CometExchange -Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(14) CometHashAggregate -Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Keys [1]: [d_week_seq#10] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_week_seq)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [d_week_seq#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_week_seq#20)) - -(17) CometProject -Input [2]: [d_week_seq#20, d_year#21] -Arguments: [d_week_seq#20], [d_week_seq#20] - -(18) CometBroadcastExchange -Input [1]: [d_week_seq#20] -Arguments: [d_week_seq#20] - -(19) CometBroadcastHashJoin -Left output [8]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28] -Right output [1]: [d_week_seq#20] -Arguments: [d_week_seq#10], [d_week_seq#20], Inner, BuildRight - -(20) CometProject -Input [9]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28, d_week_seq#20] -Arguments: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36], [d_week_seq#10 AS d_week_seq1#29, sun_sales#22 AS sun_sales1#30, mon_sales#23 AS mon_sales1#31, tue_sales#24 AS tue_sales1#32, wed_sales#25 AS wed_sales1#33, thu_sales#26 AS thu_sales1#34, fri_sales#27 AS fri_sales1#35, sat_sales#28 AS sat_sales1#36] - -(21) ReusedExchange [Reuses operator id: 13] -Output [8]: [d_week_seq#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] - -(22) CometHashAggregate -Input [8]: [d_week_seq#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] -Keys [1]: [d_week_seq#37] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#45, d_year#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [d_week_seq#45, d_year#46] -Condition : ((isnotnull(d_year#46) AND (d_year#46 = 2002)) AND isnotnull(d_week_seq#45)) - -(25) CometProject -Input [2]: [d_week_seq#45, d_year#46] -Arguments: [d_week_seq#45], [d_week_seq#45] - -(26) CometBroadcastExchange -Input [1]: [d_week_seq#45] -Arguments: [d_week_seq#45] - -(27) CometBroadcastHashJoin -Left output [8]: [d_week_seq#37, sun_sales#47, mon_sales#48, tue_sales#49, wed_sales#50, thu_sales#51, fri_sales#52, sat_sales#53] -Right output [1]: [d_week_seq#45] -Arguments: [d_week_seq#37], [d_week_seq#45], Inner, BuildRight - -(28) CometProject -Input [9]: [d_week_seq#37, sun_sales#47, mon_sales#48, tue_sales#49, wed_sales#50, thu_sales#51, fri_sales#52, sat_sales#53, d_week_seq#45] -Arguments: [d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61], [d_week_seq#37 AS d_week_seq2#54, sun_sales#47 AS sun_sales2#55, mon_sales#48 AS mon_sales2#56, tue_sales#49 AS tue_sales2#57, wed_sales#50 AS wed_sales2#58, thu_sales#51 AS thu_sales2#59, fri_sales#52 AS fri_sales2#60, sat_sales#53 AS sat_sales2#61] - -(29) CometBroadcastExchange -Input [8]: [d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61] -Arguments: [d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61] - -(30) CometBroadcastHashJoin -Left output [8]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36] -Right output [8]: [d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61] -Arguments: [d_week_seq1#29], [(d_week_seq2#54 - 53)], Inner, BuildRight - -(31) CometProject -Input [16]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36, d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61] -Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68], [d_week_seq1#29, round((sun_sales1#30 / sun_sales2#55), 2) AS round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1#31 / mon_sales2#56), 2) AS round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1#32 / tue_sales2#57), 2) AS round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1#33 / wed_sales2#58), 2) AS round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1#34 / thu_sales2#59), 2) AS round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1#35 / fri_sales2#60), 2) AS round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1#36 / sat_sales2#61), 2) AS round((sat_sales1 / sat_sales2), 2)#68] - -(32) CometExchange -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68] -Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometSort -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68] -Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68], [d_week_seq1#29 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 1] -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/extended.txt deleted file mode 100644 index 15f5db847d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/extended.txt +++ /dev/null @@ -1,48 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometProject - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 45 out of 45 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 852c5fca0f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] - CometExchange [d_week_seq1] #1 - CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] - CometBroadcastHashJoin [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] - CometExchange [d_week_seq] #2 - CometHashAggregate [d_day_name,sales_price] [d_week_seq,sum,sum,sum,sum,sum,sum,sum] - CometProject [sales_price,d_week_seq,d_day_name] - 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 [native_iceberg_compat] 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year] - CometBroadcastExchange [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] - ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 - CometBroadcastExchange [d_week_seq] #6 - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/explain.txt deleted file mode 100644 index c699bf18de..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/explain.txt +++ /dev/null @@ -1,163 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.item (4) - +- ReusedExchange (11) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(7) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(8) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(11) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] - -(14) HashAggregate [codegen id : 3] -Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(15) CometColumnarExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(17) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#17] -Results [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#17,17,2) AS _w0#19, i_item_id#11] - -(18) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(22) Project [codegen id : 6] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] - -(23) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometNativeScan parquet spark_catalog.default.date_dim (24) - - -(24) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#22] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/simplified.txt deleted file mode 100644 index fb0ed62abe..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/explain.txt deleted file mode 100644 index c23383e09a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [cs_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/simplified.txt deleted file mode 100644 index 2958d060fe..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/explain.txt deleted file mode 100644 index c23383e09a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [cs_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/extended.txt deleted file mode 100644 index cd52b2cd12..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 2958d060fe..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/explain.txt deleted file mode 100644 index 65cbe8c435..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/explain.txt +++ /dev/null @@ -1,173 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (25) -+- * Filter (24) - +- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometColumnarExchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.inventory (1) - : : +- BroadcastExchange (7) - : : +- * CometColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometNativeScan parquet spark_catalog.default.warehouse (4) - : +- BroadcastExchange (14) - : +- * CometColumnarToRow (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometNativeScan parquet spark_catalog.default.item (10) - +- ReusedExchange (17) - - -(1) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_warehouse_sk), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_warehouse_sk#2) AND isnotnull(inv_item_sk#1)) - -(4) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(6) CometColumnarToRow [codegen id : 1] -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] - -(7) BroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#6] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 4] -Output [4]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] -Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_sk#6, w_warehouse_name#7] - -(10) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Condition : (((isnotnull(i_current_price#10) AND (i_current_price#10 >= 0.99)) AND (i_current_price#10 <= 1.49)) AND isnotnull(i_item_sk#8)) - -(12) CometProject -Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Arguments: [i_item_sk#8, i_item_id#11], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#9, 16, true, false, true) AS i_item_id#11] - -(13) CometColumnarToRow [codegen id : 2] -Input [2]: [i_item_sk#8, i_item_id#11] - -(14) BroadcastExchange -Input [2]: [i_item_sk#8, i_item_id#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#8] -Join type: Inner -Join condition: None - -(16) Project [codegen id : 4] -Output [4]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11] -Input [6]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_sk#8, i_item_id#11] - -(17) ReusedExchange [Reuses operator id: 29] -Output [2]: [d_date_sk#12, d_date#13] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] -Input [6]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11, d_date_sk#12, d_date#13] - -(20) HashAggregate [codegen id : 4] -Input [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] -Keys [2]: [w_warehouse_name#7, i_item_id#11] -Functions [2]: [partial_sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), partial_sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] -Aggregate Attributes [2]: [sum#14, sum#15] -Results [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] - -(21) CometColumnarExchange -Input [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] -Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] - -(23) HashAggregate [codegen id : 5] -Input [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] -Keys [2]: [w_warehouse_name#7, i_item_id#11] -Functions [2]: [sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] -Aggregate Attributes [2]: [sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18, sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#19] -Results [4]: [w_warehouse_name#7, i_item_id#11, sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18 AS inv_before#20, sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#19 AS inv_after#21] - -(24) Filter [codegen id : 5] -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#20, inv_after#21] -Condition : (CASE WHEN (inv_before#20 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#21 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#20 as double)))))) >= 0.666667) END AND CASE WHEN (inv_before#20 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#21 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#20 as double)))))) <= 1.5) END) - -(25) TakeOrderedAndProject -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#20, inv_after#21] -Arguments: 100, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#11, inv_before#20, inv_after#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometFilter (27) - +- CometNativeScan parquet spark_catalog.default.date_dim (26) - - -(26) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_date#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) - -(28) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_date#13] - -(29) BroadcastExchange -Input [2]: [d_date_sk#12, d_date#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/simplified.txt deleted file mode 100644 index 94925f8911..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] - WholeStageCodegen (5) - Filter [inv_before,inv_after] - HashAggregate [w_warehouse_name,i_item_id,sum,sum] [sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),inv_before,inv_after,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,i_item_id] #1 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] - Project [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Filter [inv_warehouse_sk,inv_item_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - InputAdapter - ReusedExchange [d_date_sk,d_date] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/explain.txt deleted file mode 100644 index 5656916223..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometFilter (22) - +- CometHashAggregate (21) - +- CometExchange (20) - +- CometHashAggregate (19) - +- CometProject (18) - +- CometBroadcastHashJoin (17) - :- CometProject (13) - : +- CometBroadcastHashJoin (12) - : :- CometProject (7) - : : +- CometBroadcastHashJoin (6) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : +- CometBroadcastExchange (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : +- CometBroadcastExchange (11) - : +- CometProject (10) - : +- CometFilter (9) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (8) - +- CometBroadcastExchange (16) - +- CometFilter (15) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_warehouse_sk), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_warehouse_sk#2) AND isnotnull(inv_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7], [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Condition : (((isnotnull(i_current_price#10) AND (i_current_price#10 >= 0.99)) AND (i_current_price#10 <= 1.49)) AND isnotnull(i_item_sk#8)) - -(10) CometProject -Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Arguments: [i_item_sk#8, i_item_id#11], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#9, 16, true, false, true) AS i_item_id#11] - -(11) CometBroadcastExchange -Input [2]: [i_item_sk#8, i_item_id#11] -Arguments: [i_item_sk#8, i_item_id#11] - -(12) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] -Right output [2]: [i_item_sk#8, i_item_id#11] -Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight - -(13) CometProject -Input [6]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_sk#8, i_item_id#11] -Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11], [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_date#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_date#13] -Arguments: [d_date_sk#12, d_date#13] - -(17) CometBroadcastHashJoin -Left output [4]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11] -Right output [2]: [d_date_sk#12, d_date#13] -Arguments: [inv_date_sk#4], [d_date_sk#12], Inner, BuildRight - -(18) CometProject -Input [6]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11, d_date_sk#12, d_date#13] -Arguments: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13], [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] - -(19) CometHashAggregate -Input [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] -Keys [2]: [w_warehouse_name#7, i_item_id#11] -Functions [2]: [partial_sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), partial_sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] - -(20) CometExchange -Input [4]: [w_warehouse_name#7, i_item_id#11, sum#14, sum#15] -Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [4]: [w_warehouse_name#7, i_item_id#11, sum#14, sum#15] -Keys [2]: [w_warehouse_name#7, i_item_id#11] -Functions [2]: [sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] - -(22) CometFilter -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] -Condition : (CASE WHEN (inv_before#16 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#17 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#16 as double)))))) >= 0.666667) END AND CASE WHEN (inv_before#16 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#17 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#16 as double)))))) <= 1.5) END) - -(23) CometTakeOrderedAndProject -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#7 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST], output=[w_warehouse_name#7,i_item_id#11,inv_before#16,inv_after#17]), [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17], 100, 0, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] - -(24) CometColumnarToRow [codegen id : 1] -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_date#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) - -(27) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_date#13] - -(28) BroadcastExchange -Input [2]: [d_date_sk#12, d_date#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/simplified.txt deleted file mode 100644 index 1c2e80c991..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,33 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] - CometFilter [w_warehouse_name,i_item_id,inv_before,inv_after] - CometHashAggregate [sum,sum] [w_warehouse_name,i_item_id,inv_before,inv_after,sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END)] - CometExchange [w_warehouse_name,i_item_id] #1 - CometHashAggregate [d_date,inv_quantity_on_hand] [w_warehouse_name,i_item_id,sum,sum] - CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - 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,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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_id] #4 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange [d_date_sk,d_date] #5 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/explain.txt deleted file mode 100644 index 5656916223..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometFilter (22) - +- CometHashAggregate (21) - +- CometExchange (20) - +- CometHashAggregate (19) - +- CometProject (18) - +- CometBroadcastHashJoin (17) - :- CometProject (13) - : +- CometBroadcastHashJoin (12) - : :- CometProject (7) - : : +- CometBroadcastHashJoin (6) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : +- CometBroadcastExchange (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : +- CometBroadcastExchange (11) - : +- CometProject (10) - : +- CometFilter (9) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (8) - +- CometBroadcastExchange (16) - +- CometFilter (15) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_warehouse_sk), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_warehouse_sk#2) AND isnotnull(inv_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7], [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Condition : (((isnotnull(i_current_price#10) AND (i_current_price#10 >= 0.99)) AND (i_current_price#10 <= 1.49)) AND isnotnull(i_item_sk#8)) - -(10) CometProject -Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Arguments: [i_item_sk#8, i_item_id#11], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#9, 16, true, false, true) AS i_item_id#11] - -(11) CometBroadcastExchange -Input [2]: [i_item_sk#8, i_item_id#11] -Arguments: [i_item_sk#8, i_item_id#11] - -(12) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] -Right output [2]: [i_item_sk#8, i_item_id#11] -Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight - -(13) CometProject -Input [6]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_sk#8, i_item_id#11] -Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11], [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_date#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_date#13] -Arguments: [d_date_sk#12, d_date#13] - -(17) CometBroadcastHashJoin -Left output [4]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11] -Right output [2]: [d_date_sk#12, d_date#13] -Arguments: [inv_date_sk#4], [d_date_sk#12], Inner, BuildRight - -(18) CometProject -Input [6]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11, d_date_sk#12, d_date#13] -Arguments: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13], [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] - -(19) CometHashAggregate -Input [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] -Keys [2]: [w_warehouse_name#7, i_item_id#11] -Functions [2]: [partial_sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), partial_sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] - -(20) CometExchange -Input [4]: [w_warehouse_name#7, i_item_id#11, sum#14, sum#15] -Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [4]: [w_warehouse_name#7, i_item_id#11, sum#14, sum#15] -Keys [2]: [w_warehouse_name#7, i_item_id#11] -Functions [2]: [sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] - -(22) CometFilter -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] -Condition : (CASE WHEN (inv_before#16 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#17 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#16 as double)))))) >= 0.666667) END AND CASE WHEN (inv_before#16 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#17 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#16 as double)))))) <= 1.5) END) - -(23) CometTakeOrderedAndProject -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#7 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST], output=[w_warehouse_name#7,i_item_id#11,inv_before#16,inv_after#17]), [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17], 100, 0, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] - -(24) CometColumnarToRow [codegen id : 1] -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_date#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) - -(27) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_date#13] - -(28) BroadcastExchange -Input [2]: [d_date_sk#12, d_date#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/extended.txt deleted file mode 100644 index ef02334507..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 1c2e80c991..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21/simplified.txt +++ /dev/null @@ -1,33 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] - CometFilter [w_warehouse_name,i_item_id,inv_before,inv_after] - CometHashAggregate [sum,sum] [w_warehouse_name,i_item_id,inv_before,inv_after,sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END)] - CometExchange [w_warehouse_name,i_item_id] #1 - CometHashAggregate [d_date,inv_quantity_on_hand] [w_warehouse_name,i_item_id,sum,sum] - CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - 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,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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_id] #4 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange [d_date_sk,d_date] #5 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/explain.txt deleted file mode 100644 index f25f305679..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/explain.txt +++ /dev/null @@ -1,178 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (25) -+- * HashAggregate (24) - +- * CometColumnarToRow (23) - +- CometColumnarExchange (22) - +- * HashAggregate (21) - +- * Expand (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.inventory (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) - : +- CometProject (9) - : +- CometFilter (8) - : +- CometNativeScan parquet spark_catalog.default.item (7) - +- BroadcastExchange (17) - +- * CometColumnarToRow (16) - +- CometFilter (15) - +- CometNativeScan parquet spark_catalog.default.warehouse (14) - - -(1) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(4) ReusedExchange [Reuses operator id: 30] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(8) CometFilter -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Condition : isnotnull(i_item_sk#7) - -(9) CometProject -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#8, 50, true, false, true) AS i_brand#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#11, 50, true, false, true) AS i_product_name#15] - -(10) CometColumnarToRow [codegen id : 2] -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(11) BroadcastExchange -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(14) CometNativeScan parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(15) CometFilter -Input [1]: [w_warehouse_sk#16] -Condition : isnotnull(w_warehouse_sk#16) - -(16) CometColumnarToRow [codegen id : 3] -Input [1]: [w_warehouse_sk#16] - -(17) BroadcastExchange -Input [1]: [w_warehouse_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#16] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, i_category#14] -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15, w_warehouse_sk#16] - -(20) Expand [codegen id : 4] -Input [5]: [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, i_category#14] -Arguments: [[inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, i_category#14, 0], [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, null, 1], [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, null, null, 3], [inv_quantity_on_hand#3, i_product_name#15, null, null, null, 7], [inv_quantity_on_hand#3, null, null, null, null, 15]], [inv_quantity_on_hand#3, i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] - -(21) HashAggregate [codegen id : 4] -Input [6]: [inv_quantity_on_hand#3, i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] -Keys [5]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] -Functions [1]: [partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [sum#22, count#23] -Results [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, sum#24, count#25] - -(22) CometColumnarExchange -Input [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, sum#24, count#25] -Arguments: hashpartitioning(i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 5] -Input [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, sum#24, count#25] - -(24) HashAggregate [codegen id : 5] -Input [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, sum#24, count#25] -Keys [5]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#26] -Results [5]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, avg(inv_quantity_on_hand#3)#26 AS qoh#27] - -(25) TakeOrderedAndProject -Input [5]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, qoh#27] -Arguments: 100, [qoh#27 ASC NULLS FIRST, i_product_name#17 ASC NULLS FIRST, i_brand#18 ASC NULLS FIRST, i_class#19 ASC NULLS FIRST, i_category#20 ASC NULLS FIRST], [i_product_name#17, i_brand#18, i_class#19, i_category#20, qoh#27] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometNativeScan parquet spark_catalog.default.date_dim (26) - - -(26) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#28] -Condition : (((isnotnull(d_month_seq#28) AND (d_month_seq#28 >= 1200)) AND (d_month_seq#28 <= 1211)) AND isnotnull(d_date_sk#6)) - -(28) CometProject -Input [2]: [d_date_sk#6, d_month_seq#28] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(30) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/simplified.txt deleted file mode 100644 index 092e187177..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/simplified.txt +++ /dev/null @@ -1,44 +0,0 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (5) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/explain.txt deleted file mode 100644 index 02cfd90b87..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,178 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometTakeOrderedAndProject (24) - +- CometHashAggregate (23) - +- CometExchange (22) - +- CometHashAggregate (21) - +- CometExpand (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [inv_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] -Arguments: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3], [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Condition : isnotnull(i_item_sk#8) - -(11) CometProject -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#12, 50, true, false, true) AS i_product_name#16] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(13) CometBroadcastHashJoin -Left output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -Right output [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight - -(14) CometProject -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [w_warehouse_sk#17] -Condition : isnotnull(w_warehouse_sk#17) - -(17) CometBroadcastExchange -Input [1]: [w_warehouse_sk#17] -Arguments: [w_warehouse_sk#17] - -(18) CometBroadcastHashJoin -Left output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Right output [1]: [w_warehouse_sk#17] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#17], Inner, BuildRight - -(19) CometProject -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16, w_warehouse_sk#17] -Arguments: [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15], [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15] - -(20) CometExpand -Input [5]: [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15] -Arguments: [[inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15, 0], [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, null, 1], [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, null, null, 3], [inv_quantity_on_hand#3, i_product_name#16, null, null, null, 7], [inv_quantity_on_hand#3, null, null, null, null, 15]], [inv_quantity_on_hand#3, i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] - -(21) CometHashAggregate -Input [6]: [inv_quantity_on_hand#3, i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] -Keys [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] -Functions [1]: [partial_avg(inv_quantity_on_hand#3)] - -(22) CometExchange -Input [7]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22, sum#23, count#24] -Arguments: hashpartitioning(i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [7]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22, sum#23, count#24] -Keys [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] -Functions [1]: [avg(inv_quantity_on_hand#3)] - -(24) CometTakeOrderedAndProject -Input [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#25 ASC NULLS FIRST,i_product_name#18 ASC NULLS FIRST,i_brand#19 ASC NULLS FIRST,i_class#20 ASC NULLS FIRST,i_category#21 ASC NULLS FIRST], output=[i_product_name#18,i_brand#19,i_class#20,i_category#21,qoh#25]), [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25], 100, 0, [qoh#25 ASC NULLS FIRST, i_product_name#18 ASC NULLS FIRST, i_brand#19 ASC NULLS FIRST, i_class#20 ASC NULLS FIRST, i_category#21 ASC NULLS FIRST], [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] - -(25) CometColumnarToRow [codegen id : 1] -Input [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6)) - -(28) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(30) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/simplified.txt deleted file mode 100644 index 9119ee749d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] - CometHashAggregate [spark_grouping_id,sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(inv_quantity_on_hand)] - CometExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - CometHashAggregate [inv_quantity_on_hand] [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] - 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,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,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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange [w_warehouse_sk] #5 - CometFilter [w_warehouse_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/explain.txt deleted file mode 100644 index 02cfd90b87..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/explain.txt +++ /dev/null @@ -1,178 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometTakeOrderedAndProject (24) - +- CometHashAggregate (23) - +- CometExchange (22) - +- CometHashAggregate (21) - +- CometExpand (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [inv_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] -Arguments: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3], [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Condition : isnotnull(i_item_sk#8) - -(11) CometProject -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#12, 50, true, false, true) AS i_product_name#16] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(13) CometBroadcastHashJoin -Left output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -Right output [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight - -(14) CometProject -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [w_warehouse_sk#17] -Condition : isnotnull(w_warehouse_sk#17) - -(17) CometBroadcastExchange -Input [1]: [w_warehouse_sk#17] -Arguments: [w_warehouse_sk#17] - -(18) CometBroadcastHashJoin -Left output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Right output [1]: [w_warehouse_sk#17] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#17], Inner, BuildRight - -(19) CometProject -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16, w_warehouse_sk#17] -Arguments: [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15], [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15] - -(20) CometExpand -Input [5]: [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15] -Arguments: [[inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15, 0], [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, null, 1], [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, null, null, 3], [inv_quantity_on_hand#3, i_product_name#16, null, null, null, 7], [inv_quantity_on_hand#3, null, null, null, null, 15]], [inv_quantity_on_hand#3, i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] - -(21) CometHashAggregate -Input [6]: [inv_quantity_on_hand#3, i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] -Keys [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] -Functions [1]: [partial_avg(inv_quantity_on_hand#3)] - -(22) CometExchange -Input [7]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22, sum#23, count#24] -Arguments: hashpartitioning(i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [7]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22, sum#23, count#24] -Keys [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] -Functions [1]: [avg(inv_quantity_on_hand#3)] - -(24) CometTakeOrderedAndProject -Input [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#25 ASC NULLS FIRST,i_product_name#18 ASC NULLS FIRST,i_brand#19 ASC NULLS FIRST,i_class#20 ASC NULLS FIRST,i_category#21 ASC NULLS FIRST], output=[i_product_name#18,i_brand#19,i_class#20,i_category#21,qoh#25]), [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25], 100, 0, [qoh#25 ASC NULLS FIRST, i_product_name#18 ASC NULLS FIRST, i_brand#19 ASC NULLS FIRST, i_class#20 ASC NULLS FIRST, i_category#21 ASC NULLS FIRST], [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] - -(25) CometColumnarToRow [codegen id : 1] -Input [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6)) - -(28) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(30) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/extended.txt deleted file mode 100644 index 7369619d2f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/extended.txt +++ /dev/null @@ -1,33 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - -Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 9119ee749d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] - CometHashAggregate [spark_grouping_id,sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(inv_quantity_on_hand)] - CometExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - CometHashAggregate [inv_quantity_on_hand] [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] - 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,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,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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange [w_warehouse_sk] #5 - CometFilter [w_warehouse_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/explain.txt deleted file mode 100644 index 029c7fd3df..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/explain.txt +++ /dev/null @@ -1,602 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (69) -+- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometUnion (65) - :- CometProject (47) - : +- CometBroadcastHashJoin (46) - : :- CometProject (41) - : : +- CometSortMergeJoin (40) - : : :- CometSort (25) - : : : +- CometColumnarExchange (24) - : : : +- * Project (23) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : : : :- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : +- BroadcastExchange (21) - : : : +- * Project (20) - : : : +- * Filter (19) - : : : +- * HashAggregate (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometColumnarExchange (16) - : : : +- * HashAggregate (15) - : : : +- * Project (14) - : : : +- * BroadcastHashJoin Inner BuildRight (13) - : : : :- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * Filter (5) - : : : : : +- * ColumnarToRow (4) - : : : : : +- Scan parquet spark_catalog.default.store_sales (3) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (12) - : : : +- * CometColumnarToRow (11) - : : : +- CometFilter (10) - : : : +- CometNativeScan parquet spark_catalog.default.item (9) - : : +- CometSort (39) - : : +- CometProject (38) - : : +- CometFilter (37) - : : +- CometHashAggregate (36) - : : +- CometExchange (35) - : : +- CometHashAggregate (34) - : : +- CometProject (33) - : : +- CometBroadcastHashJoin (32) - : : :- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometNativeScan parquet spark_catalog.default.store_sales (26) - : : +- CometBroadcastExchange (31) - : : +- CometFilter (30) - : : +- CometNativeScan parquet spark_catalog.default.customer (29) - : +- CometBroadcastExchange (45) - : +- CometProject (44) - : +- CometFilter (43) - : +- CometNativeScan parquet spark_catalog.default.date_dim (42) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometSortMergeJoin (60) - : :- CometSort (54) - : : +- CometColumnarExchange (53) - : : +- * Project (52) - : : +- * BroadcastHashJoin LeftSemi BuildRight (51) - : : :- * ColumnarToRow (49) - : : : +- Scan parquet spark_catalog.default.web_sales (48) - : : +- ReusedExchange (50) - : +- CometSort (59) - : +- CometProject (58) - : +- CometFilter (57) - : +- CometHashAggregate (56) - : +- ReusedExchange (55) - +- ReusedExchange (62) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(3) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(4) ColumnarToRow [codegen id : 3] -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] - -(5) Filter [codegen id : 3] -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_item_sk#7) - -(6) ReusedExchange [Reuses operator id: 79] -Output [2]: [d_date_sk#10, d_date#11] - -(7) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(8) Project [codegen id : 3] -Output [2]: [ss_item_sk#7, d_date#11] -Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] - -(9) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#12, i_item_desc#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [i_item_sk#12, i_item_desc#13] -Condition : isnotnull(i_item_sk#12) - -(11) CometColumnarToRow [codegen id : 2] -Input [2]: [i_item_sk#12, i_item_desc#13] - -(12) BroadcastExchange -Input [2]: [i_item_sk#12, i_item_desc#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#7] -Right keys [1]: [i_item_sk#12] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 3] -Output [3]: [d_date#11, i_item_sk#12, substr(i_item_desc#13, 1, 30) AS _groupingexpression#14] -Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#12, i_item_desc#13] - -(15) HashAggregate [codegen id : 3] -Input [3]: [d_date#11, i_item_sk#12, _groupingexpression#14] -Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#15] -Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] - -(16) CometColumnarExchange -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(17) CometColumnarToRow [codegen id : 4] -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] - -(18) HashAggregate [codegen id : 4] -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#17] -Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] - -(19) Filter [codegen id : 4] -Input [2]: [item_sk#18, cnt#19] -Condition : (cnt#19 > 4) - -(20) Project [codegen id : 4] -Output [1]: [item_sk#18] -Input [2]: [item_sk#18, cnt#19] - -(21) BroadcastExchange -Input [1]: [item_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(22) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [item_sk#18] -Join type: LeftSemi -Join condition: None - -(23) Project [codegen id : 5] -Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(24) CometColumnarExchange -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] - -(26) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_customer_sk#20) - -(28) CometProject -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -Arguments: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22], [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] - -(29) CometNativeScan parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(30) CometFilter -Input [1]: [c_customer_sk#24] -Condition : isnotnull(c_customer_sk#24) - -(31) CometBroadcastExchange -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24] - -(32) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] -Right output [1]: [c_customer_sk#24] -Arguments: [ss_customer_sk#20], [c_customer_sk#24], Inner, BuildRight - -(33) CometProject -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -Arguments: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24], [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] - -(34) CometHashAggregate -Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -Keys [1]: [c_customer_sk#24] -Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] - -(35) CometExchange -Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] -Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(36) CometHashAggregate -Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] -Keys [1]: [c_customer_sk#24] -Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] - -(37) CometFilter -Input [2]: [c_customer_sk#24, ssales#27] -Condition : (isnotnull(ssales#27) AND (cast(ssales#27 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#28, [id=#29]))) - -(38) CometProject -Input [2]: [c_customer_sk#24, ssales#27] -Arguments: [c_customer_sk#24], [c_customer_sk#24] - -(39) CometSort -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24], [c_customer_sk#24 ASC NULLS FIRST] - -(40) CometSortMergeJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [c_customer_sk#24] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#24], LeftSemi - -(41) CometProject -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(42) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#30, d_year#31, d_moy#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [3]: [d_date_sk#30, d_year#31, d_moy#32] -Condition : ((((isnotnull(d_year#31) AND isnotnull(d_moy#32)) AND (d_year#31 = 2000)) AND (d_moy#32 = 2)) AND isnotnull(d_date_sk#30)) - -(44) CometProject -Input [3]: [d_date_sk#30, d_year#31, d_moy#32] -Arguments: [d_date_sk#30], [d_date_sk#30] - -(45) CometBroadcastExchange -Input [1]: [d_date_sk#30] -Arguments: [d_date_sk#30] - -(46) CometBroadcastHashJoin -Left output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [d_date_sk#30] -Arguments: [cs_sold_date_sk#5], [d_date_sk#30], Inner, BuildRight - -(47) CometProject -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#30] -Arguments: [sales#33], [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#33] - -(48) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#34, ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#6)] -ReadSchema: struct - -(49) ColumnarToRow [codegen id : 10] -Input [5]: [ws_item_sk#34, ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] - -(50) ReusedExchange [Reuses operator id: 21] -Output [1]: [item_sk#39] - -(51) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [item_sk#39] -Join type: LeftSemi -Join condition: None - -(52) Project [codegen id : 10] -Output [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -Input [5]: [ws_item_sk#34, ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] - -(53) CometColumnarExchange -Input [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -Arguments: hashpartitioning(ws_bill_customer_sk#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(54) CometSort -Input [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -Arguments: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38], [ws_bill_customer_sk#35 ASC NULLS FIRST] - -(55) ReusedExchange [Reuses operator id: 35] -Output [3]: [c_customer_sk#40, sum#41, isEmpty#42] - -(56) CometHashAggregate -Input [3]: [c_customer_sk#40, sum#41, isEmpty#42] -Keys [1]: [c_customer_sk#40] -Functions [1]: [sum((cast(ss_quantity#43 as decimal(10,0)) * ss_sales_price#44))] - -(57) CometFilter -Input [2]: [c_customer_sk#40, ssales#45] -Condition : (isnotnull(ssales#45) AND (cast(ssales#45 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#28, [id=#29]))) - -(58) CometProject -Input [2]: [c_customer_sk#40, ssales#45] -Arguments: [c_customer_sk#40], [c_customer_sk#40] - -(59) CometSort -Input [1]: [c_customer_sk#40] -Arguments: [c_customer_sk#40], [c_customer_sk#40 ASC NULLS FIRST] - -(60) CometSortMergeJoin -Left output [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -Right output [1]: [c_customer_sk#40] -Arguments: [ws_bill_customer_sk#35], [c_customer_sk#40], LeftSemi - -(61) CometProject -Input [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -Arguments: [ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38], [ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] - -(62) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#46] - -(63) CometBroadcastHashJoin -Left output [3]: [ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -Right output [1]: [d_date_sk#46] -Arguments: [ws_sold_date_sk#38], [d_date_sk#46], Inner, BuildRight - -(64) CometProject -Input [4]: [ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38, d_date_sk#46] -Arguments: [sales#47], [(cast(ws_quantity#36 as decimal(10,0)) * ws_list_price#37) AS sales#47] - -(65) CometUnion -Child 0 Input [1]: [sales#33] -Child 1 Input [1]: [sales#47] - -(66) CometHashAggregate -Input [1]: [sales#33] -Keys: [] -Functions [1]: [partial_sum(sales#33)] - -(67) CometExchange -Input [2]: [sum#48, isEmpty#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(68) CometHashAggregate -Input [2]: [sum#48, isEmpty#49] -Keys: [] -Functions [1]: [sum(sales#33)] - -(69) CometColumnarToRow [codegen id : 11] -Input [1]: [sum(sales)#50] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (74) -+- * CometColumnarToRow (73) - +- CometProject (72) - +- CometFilter (71) - +- CometNativeScan parquet spark_catalog.default.date_dim (70) - - -(70) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#30, d_year#31, d_moy#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(71) CometFilter -Input [3]: [d_date_sk#30, d_year#31, d_moy#32] -Condition : ((((isnotnull(d_year#31) AND isnotnull(d_moy#32)) AND (d_year#31 = 2000)) AND (d_moy#32 = 2)) AND isnotnull(d_date_sk#30)) - -(72) CometProject -Input [3]: [d_date_sk#30, d_year#31, d_moy#32] -Arguments: [d_date_sk#30], [d_date_sk#30] - -(73) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#30] - -(74) BroadcastExchange -Input [1]: [d_date_sk#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (79) -+- * CometColumnarToRow (78) - +- CometProject (77) - +- CometFilter (76) - +- CometNativeScan parquet spark_catalog.default.date_dim (75) - - -(75) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(76) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#51] -Condition : (d_year#51 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(77) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#51] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(78) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_date#11] - -(79) BroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:3 Hosting operator id = 37 Hosting Expression = Subquery scalar-subquery#28, [id=#29] -* HashAggregate (99) -+- * CometColumnarToRow (98) - +- CometColumnarExchange (97) - +- * HashAggregate (96) - +- * HashAggregate (95) - +- * CometColumnarToRow (94) - +- CometColumnarExchange (93) - +- * HashAggregate (92) - +- * Project (91) - +- * BroadcastHashJoin Inner BuildRight (90) - :- * Project (88) - : +- * BroadcastHashJoin Inner BuildRight (87) - : :- * Filter (82) - : : +- * ColumnarToRow (81) - : : +- Scan parquet spark_catalog.default.store_sales (80) - : +- BroadcastExchange (86) - : +- * CometColumnarToRow (85) - : +- CometFilter (84) - : +- CometNativeScan parquet spark_catalog.default.customer (83) - +- ReusedExchange (89) - - -(80) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#55), dynamicpruningexpression(ss_sold_date_sk#55 IN dynamicpruning#56)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(81) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55] - -(82) Filter [codegen id : 3] -Input [4]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55] -Condition : isnotnull(ss_customer_sk#52) - -(83) CometNativeScan parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#57] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(84) CometFilter -Input [1]: [c_customer_sk#57] -Condition : isnotnull(c_customer_sk#57) - -(85) CometColumnarToRow [codegen id : 1] -Input [1]: [c_customer_sk#57] - -(86) BroadcastExchange -Input [1]: [c_customer_sk#57] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -(87) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#52] -Right keys [1]: [c_customer_sk#57] -Join type: Inner -Join condition: None - -(88) Project [codegen id : 3] -Output [4]: [ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57] -Input [5]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57] - -(89) ReusedExchange [Reuses operator id: 104] -Output [1]: [d_date_sk#58] - -(90) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#55] -Right keys [1]: [d_date_sk#58] -Join type: Inner -Join condition: None - -(91) Project [codegen id : 3] -Output [3]: [ss_quantity#53, ss_sales_price#54, c_customer_sk#57] -Input [5]: [ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57, d_date_sk#58] - -(92) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#53, ss_sales_price#54, c_customer_sk#57] -Keys [1]: [c_customer_sk#57] -Functions [1]: [partial_sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))] -Aggregate Attributes [2]: [sum#59, isEmpty#60] -Results [3]: [c_customer_sk#57, sum#61, isEmpty#62] - -(93) CometColumnarExchange -Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] -Arguments: hashpartitioning(c_customer_sk#57, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(94) CometColumnarToRow [codegen id : 4] -Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] - -(95) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] -Keys [1]: [c_customer_sk#57] -Functions [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))#63] -Results [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))#63 AS csales#64] - -(96) HashAggregate [codegen id : 4] -Input [1]: [csales#64] -Keys: [] -Functions [1]: [partial_max(csales#64)] -Aggregate Attributes [1]: [max#65] -Results [1]: [max#66] - -(97) CometColumnarExchange -Input [1]: [max#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(98) CometColumnarToRow [codegen id : 5] -Input [1]: [max#66] - -(99) HashAggregate [codegen id : 5] -Input [1]: [max#66] -Keys: [] -Functions [1]: [max(csales#64)] -Aggregate Attributes [1]: [max(csales#64)#67] -Results [1]: [max(csales#64)#67 AS tpcds_cmax#68] - -Subquery:4 Hosting operator id = 80 Hosting Expression = ss_sold_date_sk#55 IN dynamicpruning#56 -BroadcastExchange (104) -+- * CometColumnarToRow (103) - +- CometProject (102) - +- CometFilter (101) - +- CometNativeScan parquet spark_catalog.default.date_dim (100) - - -(100) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#58, d_year#69] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(101) CometFilter -Input [2]: [d_date_sk#58, d_year#69] -Condition : (d_year#69 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#58)) - -(102) CometProject -Input [2]: [d_date_sk#58, d_year#69] -Arguments: [d_date_sk#58], [d_date_sk#58] - -(103) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#58] - -(104) BroadcastExchange -Input [1]: [d_date_sk#58] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:5 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#6 - -Subquery:6 Hosting operator id = 57 Hosting Expression = ReusedSubquery Subquery scalar-subquery#28, [id=#29] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/simplified.txt deleted file mode 100644 index d42a6ba29e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/simplified.txt +++ /dev/null @@ -1,142 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,isEmpty] [sum(sales),sum(sales)] - CometExchange #1 - CometHashAggregate [sales] [sum,isEmpty] - CometUnion [sales] - CometProject [cs_quantity,cs_list_price] [sales] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometColumnarExchange [cs_bill_customer_sk] #2 - WholeStageCodegen (5) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [item_sk] - Filter [cnt] - HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 - WholeStageCodegen (3) - HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (5) - HashAggregate [max] [max(csales),tpcds_cmax,max] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (4) - HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_sk] #11 - WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #12 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] - InputAdapter - ReusedExchange [d_date_sk] #12 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #8 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,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,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #9 - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [ws_quantity,ws_list_price] [sales] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometColumnarExchange [ws_bill_customer_sk] #15 - WholeStageCodegen (10) - Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - BroadcastHashJoin [ws_item_sk,item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #8 - ReusedExchange [d_date_sk] #14 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/explain.txt deleted file mode 100644 index ce21e141e4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,576 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (67) -+- CometHashAggregate (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometUnion (63) - :- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (40) - : : +- CometSortMergeJoin (39) - : : :- CometSort (24) - : : : +- CometExchange (23) - : : : +- CometProject (22) - : : : +- CometBroadcastHashJoin (21) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometBroadcastExchange (20) - : : : +- CometProject (19) - : : : +- CometFilter (18) - : : : +- CometHashAggregate (17) - : : : +- CometExchange (16) - : : : +- CometHashAggregate (15) - : : : +- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometFilter (3) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (2) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometFilter (11) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (10) - : : +- CometSort (38) - : : +- CometProject (37) - : : +- CometFilter (36) - : : +- CometHashAggregate (35) - : : +- CometExchange (34) - : : +- CometHashAggregate (33) - : : +- CometProject (32) - : : +- CometBroadcastHashJoin (31) - : : :- CometProject (27) - : : : +- CometFilter (26) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (25) - : : +- CometBroadcastExchange (30) - : : +- CometFilter (29) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (28) - : +- CometBroadcastExchange (44) - : +- CometProject (43) - : +- CometFilter (42) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - +- CometProject (62) - +- CometBroadcastHashJoin (61) - :- CometProject (59) - : +- CometSortMergeJoin (58) - : :- CometSort (52) - : : +- CometExchange (51) - : : +- CometProject (50) - : : +- CometBroadcastHashJoin (49) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : : +- ReusedExchange (48) - : +- CometSort (57) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometHashAggregate (54) - : +- ReusedExchange (53) - +- ReusedExchange (60) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(2) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(3) CometFilter -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_item_sk#7) - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(6) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(7) CometBroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: [d_date_sk#10, d_date#11] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Right output [2]: [d_date_sk#10, d_date#11] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] -Arguments: [ss_item_sk#7, d_date#11], [ss_item_sk#7, d_date#11] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#13, i_item_desc#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [i_item_sk#13, i_item_desc#14] -Condition : isnotnull(i_item_sk#13) - -(12) CometBroadcastExchange -Input [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [i_item_sk#13, i_item_desc#14] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, d_date#11] -Right output [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [ss_item_sk#7], [i_item_sk#13], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#13, i_item_desc#14] -Arguments: [d_date#11, i_item_sk#13, _groupingexpression#15], [d_date#11, i_item_sk#13, substr(i_item_desc#14, 1, 30) AS _groupingexpression#15] - -(15) CometHashAggregate -Input [3]: [d_date#11, i_item_sk#13, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [partial_count(1)] - -(16) CometExchange -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [count(1)] - -(18) CometFilter -Input [2]: [item_sk#17, cnt#18] -Condition : (cnt#18 > 4) - -(19) CometProject -Input [2]: [item_sk#17, cnt#18] -Arguments: [item_sk#17], [item_sk#17] - -(20) CometBroadcastExchange -Input [1]: [item_sk#17] -Arguments: [item_sk#17] - -(21) CometBroadcastHashJoin -Left output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [item_sk#17] -Arguments: [cs_item_sk#2], [item_sk#17], LeftSemi, BuildRight - -(22) CometProject -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(23) CometExchange -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#19) - -(27) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Arguments: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21], [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(29) CometFilter -Input [1]: [c_customer_sk#23] -Condition : isnotnull(c_customer_sk#23) - -(30) CometBroadcastExchange -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23] - -(31) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] -Right output [1]: [c_customer_sk#23] -Arguments: [ss_customer_sk#19], [c_customer_sk#23], Inner, BuildRight - -(32) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Arguments: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23], [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] - -(33) CometHashAggregate -Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Keys [1]: [c_customer_sk#23] -Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(34) CometExchange -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(35) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(36) CometFilter -Input [2]: [c_customer_sk#23, ssales#26] -Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#27, [id=#28]))) - -(37) CometProject -Input [2]: [c_customer_sk#23, ssales#26] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(38) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(39) CometSortMergeJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [c_customer_sk#23] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#23], LeftSemi - -(40) CometProject -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#29, d_year#30, d_moy#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Condition : ((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2000)) AND (d_moy#31 = 2)) AND isnotnull(d_date_sk#29)) - -(43) CometProject -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Arguments: [d_date_sk#29], [d_date_sk#29] - -(44) CometBroadcastExchange -Input [1]: [d_date_sk#29] -Arguments: [d_date_sk#29] - -(45) CometBroadcastHashJoin -Left output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [d_date_sk#29] -Arguments: [cs_sold_date_sk#5], [d_date_sk#29], Inner, BuildRight - -(46) CometProject -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#29] -Arguments: [sales#32], [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#32] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_sold_date_sk#37 IN dynamicpruning#38)] -ReadSchema: struct - -(48) ReusedExchange [Reuses operator id: 20] -Output [1]: [item_sk#39] - -(49) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Right output [1]: [item_sk#39] -Arguments: [ws_item_sk#33], [item_sk#39], LeftSemi, BuildRight - -(50) CometProject -Input [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] - -(51) CometExchange -Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: hashpartitioning(ws_bill_customer_sk#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(52) CometSort -Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_bill_customer_sk#34 ASC NULLS FIRST] - -(53) ReusedExchange [Reuses operator id: 34] -Output [3]: [c_customer_sk#40, sum#41, isEmpty#42] - -(54) CometHashAggregate -Input [3]: [c_customer_sk#40, sum#41, isEmpty#42] -Keys [1]: [c_customer_sk#40] -Functions [1]: [sum((cast(ss_quantity#43 as decimal(10,0)) * ss_sales_price#44))] - -(55) CometFilter -Input [2]: [c_customer_sk#40, ssales#45] -Condition : (isnotnull(ssales#45) AND (cast(ssales#45 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) - -(56) CometProject -Input [2]: [c_customer_sk#40, ssales#45] -Arguments: [c_customer_sk#40], [c_customer_sk#40] - -(57) CometSort -Input [1]: [c_customer_sk#40] -Arguments: [c_customer_sk#40], [c_customer_sk#40 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Right output [1]: [c_customer_sk#40] -Arguments: [ws_bill_customer_sk#34], [c_customer_sk#40], LeftSemi - -(59) CometProject -Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] - -(60) ReusedExchange [Reuses operator id: 44] -Output [1]: [d_date_sk#46] - -(61) CometBroadcastHashJoin -Left output [3]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Right output [1]: [d_date_sk#46] -Arguments: [ws_sold_date_sk#37], [d_date_sk#46], Inner, BuildRight - -(62) CometProject -Input [4]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37, d_date_sk#46] -Arguments: [sales#47], [(cast(ws_quantity#35 as decimal(10,0)) * ws_list_price#36) AS sales#47] - -(63) CometUnion -Child 0 Input [1]: [sales#32] -Child 1 Input [1]: [sales#47] - -(64) CometHashAggregate -Input [1]: [sales#32] -Keys: [] -Functions [1]: [partial_sum(sales#32)] - -(65) CometExchange -Input [2]: [sum#48, isEmpty#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(66) CometHashAggregate -Input [2]: [sum#48, isEmpty#49] -Keys: [] -Functions [1]: [sum(sales#32)] - -(67) CometColumnarToRow [codegen id : 1] -Input [1]: [sum(sales)#50] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (72) -+- * CometColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) - - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#29, d_year#30, d_moy#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Condition : ((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2000)) AND (d_moy#31 = 2)) AND isnotnull(d_date_sk#29)) - -(70) CometProject -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Arguments: [d_date_sk#29], [d_date_sk#29] - -(71) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#29] - -(72) BroadcastExchange -Input [1]: [d_date_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 2 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (77) -+- * CometColumnarToRow (76) - +- CometProject (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(75) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(76) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_date#11] - -(77) BroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:3 Hosting operator id = 36 Hosting Expression = Subquery scalar-subquery#27, [id=#28] -* CometColumnarToRow (95) -+- CometHashAggregate (94) - +- CometExchange (93) - +- CometHashAggregate (92) - +- CometHashAggregate (91) - +- CometExchange (90) - +- CometHashAggregate (89) - +- CometProject (88) - +- CometBroadcastHashJoin (87) - :- CometProject (82) - : +- CometBroadcastHashJoin (81) - : :- CometFilter (79) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (78) - : +- ReusedExchange (80) - +- CometBroadcastExchange (86) - +- CometProject (85) - +- CometFilter (84) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (83) - - -(78) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#54), dynamicpruningexpression(ss_sold_date_sk#54 IN dynamicpruning#55)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(79) CometFilter -Input [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] -Condition : isnotnull(ss_customer_sk#51) - -(80) ReusedExchange [Reuses operator id: 30] -Output [1]: [c_customer_sk#56] - -(81) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] -Right output [1]: [c_customer_sk#56] -Arguments: [ss_customer_sk#51], [c_customer_sk#56], Inner, BuildRight - -(82) CometProject -Input [5]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56] -Arguments: [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56], [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56] - -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#57, d_year#58] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(84) CometFilter -Input [2]: [d_date_sk#57, d_year#58] -Condition : (d_year#58 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#57)) - -(85) CometProject -Input [2]: [d_date_sk#57, d_year#58] -Arguments: [d_date_sk#57], [d_date_sk#57] - -(86) CometBroadcastExchange -Input [1]: [d_date_sk#57] -Arguments: [d_date_sk#57] - -(87) CometBroadcastHashJoin -Left output [4]: [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56] -Right output [1]: [d_date_sk#57] -Arguments: [ss_sold_date_sk#54], [d_date_sk#57], Inner, BuildRight - -(88) CometProject -Input [5]: [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56, d_date_sk#57] -Arguments: [ss_quantity#52, ss_sales_price#53, c_customer_sk#56], [ss_quantity#52, ss_sales_price#53, c_customer_sk#56] - -(89) CometHashAggregate -Input [3]: [ss_quantity#52, ss_sales_price#53, c_customer_sk#56] -Keys [1]: [c_customer_sk#56] -Functions [1]: [partial_sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))] - -(90) CometExchange -Input [3]: [c_customer_sk#56, sum#59, isEmpty#60] -Arguments: hashpartitioning(c_customer_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(91) CometHashAggregate -Input [3]: [c_customer_sk#56, sum#59, isEmpty#60] -Keys [1]: [c_customer_sk#56] -Functions [1]: [sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))] - -(92) CometHashAggregate -Input [1]: [csales#61] -Keys: [] -Functions [1]: [partial_max(csales#61)] - -(93) CometExchange -Input [1]: [max#62] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(94) CometHashAggregate -Input [1]: [max#62] -Keys: [] -Functions [1]: [max(csales#61)] - -(95) CometColumnarToRow [codegen id : 1] -Input [1]: [tpcds_cmax#63] - -Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#54 IN dynamicpruning#55 -BroadcastExchange (100) -+- * CometColumnarToRow (99) - +- CometProject (98) - +- CometFilter (97) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (96) - - -(96) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#57, d_year#58] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(97) CometFilter -Input [2]: [d_date_sk#57, d_year#58] -Condition : (d_year#58 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#57)) - -(98) CometProject -Input [2]: [d_date_sk#57, d_year#58] -Arguments: [d_date_sk#57], [d_date_sk#57] - -(99) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#57] - -(100) BroadcastExchange -Input [1]: [d_date_sk#57] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] - -Subquery:5 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#6 - -Subquery:6 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 8f1bddf6c0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,116 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,isEmpty] [sum(sales),sum(sales)] - CometExchange #1 - CometHashAggregate [sales] [sum,isEmpty] - CometUnion [sales] - CometProject [cs_quantity,cs_list_price] [sales] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometExchange [cs_bill_customer_sk] #2 - CometProject [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [item_sk] #4 - CometProject [item_sk] - CometFilter [item_sk,cnt] - CometHashAggregate [_groupingexpression,d_date,count] [item_sk,cnt,i_item_sk,count(1)] - CometExchange [_groupingexpression,i_item_sk,d_date] #5 - CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] - CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] - CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [i_item_sk,i_item_desc] #8 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [max] [tpcds_cmax,max(csales)] - CometExchange #11 - CometHashAggregate [csales] [max] - CometHashAggregate [c_customer_sk,sum,isEmpty] [csales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #12 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,c_customer_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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #9 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,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,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #10 - CometFilter [c_customer_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [d_date_sk] #15 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [ws_quantity,ws_list_price] [sales] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometExchange [ws_bill_customer_sk] #16 - CometProject [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,item_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #9 - ReusedExchange [d_date_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/explain.txt deleted file mode 100644 index ce21e141e4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/explain.txt +++ /dev/null @@ -1,576 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (67) -+- CometHashAggregate (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometUnion (63) - :- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (40) - : : +- CometSortMergeJoin (39) - : : :- CometSort (24) - : : : +- CometExchange (23) - : : : +- CometProject (22) - : : : +- CometBroadcastHashJoin (21) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometBroadcastExchange (20) - : : : +- CometProject (19) - : : : +- CometFilter (18) - : : : +- CometHashAggregate (17) - : : : +- CometExchange (16) - : : : +- CometHashAggregate (15) - : : : +- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometFilter (3) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (2) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometFilter (11) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (10) - : : +- CometSort (38) - : : +- CometProject (37) - : : +- CometFilter (36) - : : +- CometHashAggregate (35) - : : +- CometExchange (34) - : : +- CometHashAggregate (33) - : : +- CometProject (32) - : : +- CometBroadcastHashJoin (31) - : : :- CometProject (27) - : : : +- CometFilter (26) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (25) - : : +- CometBroadcastExchange (30) - : : +- CometFilter (29) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (28) - : +- CometBroadcastExchange (44) - : +- CometProject (43) - : +- CometFilter (42) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - +- CometProject (62) - +- CometBroadcastHashJoin (61) - :- CometProject (59) - : +- CometSortMergeJoin (58) - : :- CometSort (52) - : : +- CometExchange (51) - : : +- CometProject (50) - : : +- CometBroadcastHashJoin (49) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : : +- ReusedExchange (48) - : +- CometSort (57) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometHashAggregate (54) - : +- ReusedExchange (53) - +- ReusedExchange (60) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(2) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(3) CometFilter -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_item_sk#7) - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(6) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(7) CometBroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: [d_date_sk#10, d_date#11] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Right output [2]: [d_date_sk#10, d_date#11] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] -Arguments: [ss_item_sk#7, d_date#11], [ss_item_sk#7, d_date#11] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#13, i_item_desc#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [i_item_sk#13, i_item_desc#14] -Condition : isnotnull(i_item_sk#13) - -(12) CometBroadcastExchange -Input [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [i_item_sk#13, i_item_desc#14] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, d_date#11] -Right output [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [ss_item_sk#7], [i_item_sk#13], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#13, i_item_desc#14] -Arguments: [d_date#11, i_item_sk#13, _groupingexpression#15], [d_date#11, i_item_sk#13, substr(i_item_desc#14, 1, 30) AS _groupingexpression#15] - -(15) CometHashAggregate -Input [3]: [d_date#11, i_item_sk#13, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [partial_count(1)] - -(16) CometExchange -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [count(1)] - -(18) CometFilter -Input [2]: [item_sk#17, cnt#18] -Condition : (cnt#18 > 4) - -(19) CometProject -Input [2]: [item_sk#17, cnt#18] -Arguments: [item_sk#17], [item_sk#17] - -(20) CometBroadcastExchange -Input [1]: [item_sk#17] -Arguments: [item_sk#17] - -(21) CometBroadcastHashJoin -Left output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [item_sk#17] -Arguments: [cs_item_sk#2], [item_sk#17], LeftSemi, BuildRight - -(22) CometProject -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(23) CometExchange -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#19) - -(27) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Arguments: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21], [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(29) CometFilter -Input [1]: [c_customer_sk#23] -Condition : isnotnull(c_customer_sk#23) - -(30) CometBroadcastExchange -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23] - -(31) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] -Right output [1]: [c_customer_sk#23] -Arguments: [ss_customer_sk#19], [c_customer_sk#23], Inner, BuildRight - -(32) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Arguments: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23], [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] - -(33) CometHashAggregate -Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Keys [1]: [c_customer_sk#23] -Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(34) CometExchange -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(35) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(36) CometFilter -Input [2]: [c_customer_sk#23, ssales#26] -Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#27, [id=#28]))) - -(37) CometProject -Input [2]: [c_customer_sk#23, ssales#26] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(38) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(39) CometSortMergeJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [c_customer_sk#23] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#23], LeftSemi - -(40) CometProject -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#29, d_year#30, d_moy#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Condition : ((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2000)) AND (d_moy#31 = 2)) AND isnotnull(d_date_sk#29)) - -(43) CometProject -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Arguments: [d_date_sk#29], [d_date_sk#29] - -(44) CometBroadcastExchange -Input [1]: [d_date_sk#29] -Arguments: [d_date_sk#29] - -(45) CometBroadcastHashJoin -Left output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [d_date_sk#29] -Arguments: [cs_sold_date_sk#5], [d_date_sk#29], Inner, BuildRight - -(46) CometProject -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#29] -Arguments: [sales#32], [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#32] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_sold_date_sk#37 IN dynamicpruning#38)] -ReadSchema: struct - -(48) ReusedExchange [Reuses operator id: 20] -Output [1]: [item_sk#39] - -(49) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Right output [1]: [item_sk#39] -Arguments: [ws_item_sk#33], [item_sk#39], LeftSemi, BuildRight - -(50) CometProject -Input [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] - -(51) CometExchange -Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: hashpartitioning(ws_bill_customer_sk#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(52) CometSort -Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_bill_customer_sk#34 ASC NULLS FIRST] - -(53) ReusedExchange [Reuses operator id: 34] -Output [3]: [c_customer_sk#40, sum#41, isEmpty#42] - -(54) CometHashAggregate -Input [3]: [c_customer_sk#40, sum#41, isEmpty#42] -Keys [1]: [c_customer_sk#40] -Functions [1]: [sum((cast(ss_quantity#43 as decimal(10,0)) * ss_sales_price#44))] - -(55) CometFilter -Input [2]: [c_customer_sk#40, ssales#45] -Condition : (isnotnull(ssales#45) AND (cast(ssales#45 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) - -(56) CometProject -Input [2]: [c_customer_sk#40, ssales#45] -Arguments: [c_customer_sk#40], [c_customer_sk#40] - -(57) CometSort -Input [1]: [c_customer_sk#40] -Arguments: [c_customer_sk#40], [c_customer_sk#40 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Right output [1]: [c_customer_sk#40] -Arguments: [ws_bill_customer_sk#34], [c_customer_sk#40], LeftSemi - -(59) CometProject -Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] - -(60) ReusedExchange [Reuses operator id: 44] -Output [1]: [d_date_sk#46] - -(61) CometBroadcastHashJoin -Left output [3]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Right output [1]: [d_date_sk#46] -Arguments: [ws_sold_date_sk#37], [d_date_sk#46], Inner, BuildRight - -(62) CometProject -Input [4]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37, d_date_sk#46] -Arguments: [sales#47], [(cast(ws_quantity#35 as decimal(10,0)) * ws_list_price#36) AS sales#47] - -(63) CometUnion -Child 0 Input [1]: [sales#32] -Child 1 Input [1]: [sales#47] - -(64) CometHashAggregate -Input [1]: [sales#32] -Keys: [] -Functions [1]: [partial_sum(sales#32)] - -(65) CometExchange -Input [2]: [sum#48, isEmpty#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(66) CometHashAggregate -Input [2]: [sum#48, isEmpty#49] -Keys: [] -Functions [1]: [sum(sales#32)] - -(67) CometColumnarToRow [codegen id : 1] -Input [1]: [sum(sales)#50] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (72) -+- * CometColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) - - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#29, d_year#30, d_moy#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Condition : ((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2000)) AND (d_moy#31 = 2)) AND isnotnull(d_date_sk#29)) - -(70) CometProject -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Arguments: [d_date_sk#29], [d_date_sk#29] - -(71) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#29] - -(72) BroadcastExchange -Input [1]: [d_date_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 2 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (77) -+- * CometColumnarToRow (76) - +- CometProject (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(75) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(76) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_date#11] - -(77) BroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:3 Hosting operator id = 36 Hosting Expression = Subquery scalar-subquery#27, [id=#28] -* CometColumnarToRow (95) -+- CometHashAggregate (94) - +- CometExchange (93) - +- CometHashAggregate (92) - +- CometHashAggregate (91) - +- CometExchange (90) - +- CometHashAggregate (89) - +- CometProject (88) - +- CometBroadcastHashJoin (87) - :- CometProject (82) - : +- CometBroadcastHashJoin (81) - : :- CometFilter (79) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (78) - : +- ReusedExchange (80) - +- CometBroadcastExchange (86) - +- CometProject (85) - +- CometFilter (84) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (83) - - -(78) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#54), dynamicpruningexpression(ss_sold_date_sk#54 IN dynamicpruning#55)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(79) CometFilter -Input [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] -Condition : isnotnull(ss_customer_sk#51) - -(80) ReusedExchange [Reuses operator id: 30] -Output [1]: [c_customer_sk#56] - -(81) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] -Right output [1]: [c_customer_sk#56] -Arguments: [ss_customer_sk#51], [c_customer_sk#56], Inner, BuildRight - -(82) CometProject -Input [5]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56] -Arguments: [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56], [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56] - -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#57, d_year#58] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(84) CometFilter -Input [2]: [d_date_sk#57, d_year#58] -Condition : (d_year#58 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#57)) - -(85) CometProject -Input [2]: [d_date_sk#57, d_year#58] -Arguments: [d_date_sk#57], [d_date_sk#57] - -(86) CometBroadcastExchange -Input [1]: [d_date_sk#57] -Arguments: [d_date_sk#57] - -(87) CometBroadcastHashJoin -Left output [4]: [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56] -Right output [1]: [d_date_sk#57] -Arguments: [ss_sold_date_sk#54], [d_date_sk#57], Inner, BuildRight - -(88) CometProject -Input [5]: [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56, d_date_sk#57] -Arguments: [ss_quantity#52, ss_sales_price#53, c_customer_sk#56], [ss_quantity#52, ss_sales_price#53, c_customer_sk#56] - -(89) CometHashAggregate -Input [3]: [ss_quantity#52, ss_sales_price#53, c_customer_sk#56] -Keys [1]: [c_customer_sk#56] -Functions [1]: [partial_sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))] - -(90) CometExchange -Input [3]: [c_customer_sk#56, sum#59, isEmpty#60] -Arguments: hashpartitioning(c_customer_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(91) CometHashAggregate -Input [3]: [c_customer_sk#56, sum#59, isEmpty#60] -Keys [1]: [c_customer_sk#56] -Functions [1]: [sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))] - -(92) CometHashAggregate -Input [1]: [csales#61] -Keys: [] -Functions [1]: [partial_max(csales#61)] - -(93) CometExchange -Input [1]: [max#62] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(94) CometHashAggregate -Input [1]: [max#62] -Keys: [] -Functions [1]: [max(csales#61)] - -(95) CometColumnarToRow [codegen id : 1] -Input [1]: [tpcds_cmax#63] - -Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#54 IN dynamicpruning#55 -BroadcastExchange (100) -+- * CometColumnarToRow (99) - +- CometProject (98) - +- CometFilter (97) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (96) - - -(96) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#57, d_year#58] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(97) CometFilter -Input [2]: [d_date_sk#57, d_year#58] -Condition : (d_year#58 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#57)) - -(98) CometProject -Input [2]: [d_date_sk#57, d_year#58] -Arguments: [d_date_sk#57], [d_date_sk#57] - -(99) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#57] - -(100) BroadcastExchange -Input [1]: [d_date_sk#57] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] - -Subquery:5 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#6 - -Subquery:6 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/extended.txt deleted file mode 100644 index a84226b077..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/extended.txt +++ /dev/null @@ -1,146 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometSort - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometSort - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 127 out of 138 eligible operators (92%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 8f1bddf6c0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a/simplified.txt +++ /dev/null @@ -1,116 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,isEmpty] [sum(sales),sum(sales)] - CometExchange #1 - CometHashAggregate [sales] [sum,isEmpty] - CometUnion [sales] - CometProject [cs_quantity,cs_list_price] [sales] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometExchange [cs_bill_customer_sk] #2 - CometProject [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [item_sk] #4 - CometProject [item_sk] - CometFilter [item_sk,cnt] - CometHashAggregate [_groupingexpression,d_date,count] [item_sk,cnt,i_item_sk,count(1)] - CometExchange [_groupingexpression,i_item_sk,d_date] #5 - CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] - CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] - CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [i_item_sk,i_item_desc] #8 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [max] [tpcds_cmax,max(csales)] - CometExchange #11 - CometHashAggregate [csales] [max] - CometHashAggregate [c_customer_sk,sum,isEmpty] [csales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #12 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,c_customer_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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #9 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,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,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #10 - CometFilter [c_customer_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [d_date_sk] #15 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [ws_quantity,ws_list_price] [sales] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometExchange [ws_bill_customer_sk] #16 - CometProject [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,item_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #9 - ReusedExchange [d_date_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/explain.txt deleted file mode 100644 index 6003575bd0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/explain.txt +++ /dev/null @@ -1,718 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (90) -+- CometTakeOrderedAndProject (89) - +- CometUnion (88) - :- CometHashAggregate (64) - : +- CometExchange (63) - : +- CometHashAggregate (62) - : +- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (55) - : : +- CometBroadcastHashJoin (54) - : : :- CometSortMergeJoin (41) - : : : :- CometSort (26) - : : : : +- CometColumnarExchange (25) - : : : : +- * Project (24) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (22) - : : : : +- * Project (21) - : : : : +- * Filter (20) - : : : : +- * HashAggregate (19) - : : : : +- * CometColumnarToRow (18) - : : : : +- CometColumnarExchange (17) - : : : : +- * HashAggregate (16) - : : : : +- * Project (15) - : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : :- * Project (9) - : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : :- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : : : +- ReusedExchange (7) - : : : : +- BroadcastExchange (13) - : : : : +- * CometColumnarToRow (12) - : : : : +- CometFilter (11) - : : : : +- CometNativeScan parquet spark_catalog.default.item (10) - : : : +- CometSort (40) - : : : +- CometProject (39) - : : : +- CometFilter (38) - : : : +- CometHashAggregate (37) - : : : +- CometExchange (36) - : : : +- CometHashAggregate (35) - : : : +- CometProject (34) - : : : +- CometBroadcastHashJoin (33) - : : : :- CometProject (29) - : : : : +- CometFilter (28) - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (27) - : : : +- CometBroadcastExchange (32) - : : : +- CometFilter (31) - : : : +- CometNativeScan parquet spark_catalog.default.customer (30) - : : +- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometSortMergeJoin (51) - : : :- CometSort (45) - : : : +- CometExchange (44) - : : : +- CometFilter (43) - : : : +- CometNativeScan parquet spark_catalog.default.customer (42) - : : +- CometSort (50) - : : +- CometProject (49) - : : +- CometFilter (48) - : : +- CometHashAggregate (47) - : : +- ReusedExchange (46) - : +- CometBroadcastExchange (59) - : +- CometProject (58) - : +- CometFilter (57) - : +- CometNativeScan parquet spark_catalog.default.date_dim (56) - +- CometHashAggregate (87) - +- CometExchange (86) - +- CometHashAggregate (85) - +- CometProject (84) - +- CometBroadcastHashJoin (83) - :- CometProject (81) - : +- CometBroadcastHashJoin (80) - : :- CometSortMergeJoin (78) - : : :- CometSort (72) - : : : +- CometColumnarExchange (71) - : : : +- * Project (70) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (69) - : : : :- * Filter (67) - : : : : +- * ColumnarToRow (66) - : : : : +- Scan parquet spark_catalog.default.web_sales (65) - : : : +- ReusedExchange (68) - : : +- CometSort (77) - : : +- CometProject (76) - : : +- CometFilter (75) - : : +- CometHashAggregate (74) - : : +- ReusedExchange (73) - : +- ReusedExchange (79) - +- ReusedExchange (82) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(3) Filter [codegen id : 5] -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_bill_customer_sk#1) - -(4) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 3] -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] - -(6) Filter [codegen id : 3] -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_item_sk#7) - -(7) ReusedExchange [Reuses operator id: 100] -Output [2]: [d_date_sk#10, d_date#11] - -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 3] -Output [2]: [ss_item_sk#7, d_date#11] -Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] - -(10) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#12, i_item_desc#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [i_item_sk#12, i_item_desc#13] -Condition : isnotnull(i_item_sk#12) - -(12) CometColumnarToRow [codegen id : 2] -Input [2]: [i_item_sk#12, i_item_desc#13] - -(13) BroadcastExchange -Input [2]: [i_item_sk#12, i_item_desc#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(14) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#7] -Right keys [1]: [i_item_sk#12] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 3] -Output [3]: [d_date#11, i_item_sk#12, substr(i_item_desc#13, 1, 30) AS _groupingexpression#14] -Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#12, i_item_desc#13] - -(16) HashAggregate [codegen id : 3] -Input [3]: [d_date#11, i_item_sk#12, _groupingexpression#14] -Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#15] -Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] - -(17) CometColumnarExchange -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(18) CometColumnarToRow [codegen id : 4] -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] - -(19) HashAggregate [codegen id : 4] -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#17] -Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] - -(20) Filter [codegen id : 4] -Input [2]: [item_sk#18, cnt#19] -Condition : (cnt#19 > 4) - -(21) Project [codegen id : 4] -Output [1]: [item_sk#18] -Input [2]: [item_sk#18, cnt#19] - -(22) BroadcastExchange -Input [1]: [item_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(23) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [item_sk#18] -Join type: LeftSemi -Join condition: None - -(24) Project [codegen id : 5] -Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(25) CometColumnarExchange -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(26) CometSort -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] - -(27) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(28) CometFilter -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_customer_sk#20) - -(29) CometProject -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -Arguments: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22], [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] - -(30) CometNativeScan parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(31) CometFilter -Input [1]: [c_customer_sk#24] -Condition : isnotnull(c_customer_sk#24) - -(32) CometBroadcastExchange -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24] - -(33) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] -Right output [1]: [c_customer_sk#24] -Arguments: [ss_customer_sk#20], [c_customer_sk#24], Inner, BuildRight - -(34) CometProject -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -Arguments: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24], [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] - -(35) CometHashAggregate -Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -Keys [1]: [c_customer_sk#24] -Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] - -(36) CometExchange -Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] -Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(37) CometHashAggregate -Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] -Keys [1]: [c_customer_sk#24] -Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] - -(38) CometFilter -Input [2]: [c_customer_sk#24, ssales#27] -Condition : (isnotnull(ssales#27) AND (cast(ssales#27 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#28, [id=#29]))) - -(39) CometProject -Input [2]: [c_customer_sk#24, ssales#27] -Arguments: [c_customer_sk#24], [c_customer_sk#24] - -(40) CometSort -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24], [c_customer_sk#24 ASC NULLS FIRST] - -(41) CometSortMergeJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [c_customer_sk#24] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#24], LeftSemi - -(42) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(43) CometFilter -Input [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Condition : isnotnull(c_customer_sk#30) - -(44) CometExchange -Input [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(45) CometSort -Input [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Arguments: [c_customer_sk#30, c_first_name#31, c_last_name#32], [c_customer_sk#30 ASC NULLS FIRST] - -(46) ReusedExchange [Reuses operator id: 36] -Output [3]: [c_customer_sk#24, sum#25, isEmpty#26] - -(47) CometHashAggregate -Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] -Keys [1]: [c_customer_sk#24] -Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] - -(48) CometFilter -Input [2]: [c_customer_sk#24, ssales#27] -Condition : (isnotnull(ssales#27) AND (cast(ssales#27 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#28, [id=#29]))) - -(49) CometProject -Input [2]: [c_customer_sk#24, ssales#27] -Arguments: [c_customer_sk#24], [c_customer_sk#24] - -(50) CometSort -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24], [c_customer_sk#24 ASC NULLS FIRST] - -(51) CometSortMergeJoin -Left output [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Right output [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#30], [c_customer_sk#24], LeftSemi - -(52) CometProject -Input [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Arguments: [c_customer_sk#30, c_first_name#33, c_last_name#34], [c_customer_sk#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#31, 20, true, false, true) AS c_first_name#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#32, 30, true, false, true) AS c_last_name#34] - -(53) CometBroadcastExchange -Input [3]: [c_customer_sk#30, c_first_name#33, c_last_name#34] -Arguments: [c_customer_sk#30, c_first_name#33, c_last_name#34] - -(54) CometBroadcastHashJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [3]: [c_customer_sk#30, c_first_name#33, c_last_name#34] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#30], Inner, BuildRight - -(55) CometProject -Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#30, c_first_name#33, c_last_name#34] -Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#33, c_last_name#34], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#33, c_last_name#34] - -(56) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#35, d_year#36, d_moy#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(57) CometFilter -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] -Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2000)) AND (d_moy#37 = 2)) AND isnotnull(d_date_sk#35)) - -(58) CometProject -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] -Arguments: [d_date_sk#35], [d_date_sk#35] - -(59) CometBroadcastExchange -Input [1]: [d_date_sk#35] -Arguments: [d_date_sk#35] - -(60) CometBroadcastHashJoin -Left output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#33, c_last_name#34] -Right output [1]: [d_date_sk#35] -Arguments: [cs_sold_date_sk#5], [d_date_sk#35], Inner, BuildRight - -(61) CometProject -Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#33, c_last_name#34, d_date_sk#35] -Arguments: [cs_quantity#3, cs_list_price#4, c_first_name#33, c_last_name#34], [cs_quantity#3, cs_list_price#4, c_first_name#33, c_last_name#34] - -(62) CometHashAggregate -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#33, c_last_name#34] -Keys [2]: [c_last_name#34, c_first_name#33] -Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] - -(63) CometExchange -Input [4]: [c_last_name#34, c_first_name#33, sum#38, isEmpty#39] -Arguments: hashpartitioning(c_last_name#34, c_first_name#33, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(64) CometHashAggregate -Input [4]: [c_last_name#34, c_first_name#33, sum#38, isEmpty#39] -Keys [2]: [c_last_name#34, c_first_name#33] -Functions [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] - -(65) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#40, ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(66) ColumnarToRow [codegen id : 10] -Input [5]: [ws_item_sk#40, ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] - -(67) Filter [codegen id : 10] -Input [5]: [ws_item_sk#40, ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_bill_customer_sk#41) - -(68) ReusedExchange [Reuses operator id: 22] -Output [1]: [item_sk#45] - -(69) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [item_sk#45] -Join type: LeftSemi -Join condition: None - -(70) Project [codegen id : 10] -Output [4]: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -Input [5]: [ws_item_sk#40, ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] - -(71) CometColumnarExchange -Input [4]: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -Arguments: hashpartitioning(ws_bill_customer_sk#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(72) CometSort -Input [4]: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -Arguments: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44], [ws_bill_customer_sk#41 ASC NULLS FIRST] - -(73) ReusedExchange [Reuses operator id: 36] -Output [3]: [c_customer_sk#46, sum#47, isEmpty#48] - -(74) CometHashAggregate -Input [3]: [c_customer_sk#46, sum#47, isEmpty#48] -Keys [1]: [c_customer_sk#46] -Functions [1]: [sum((cast(ss_quantity#49 as decimal(10,0)) * ss_sales_price#50))] - -(75) CometFilter -Input [2]: [c_customer_sk#46, ssales#51] -Condition : (isnotnull(ssales#51) AND (cast(ssales#51 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#28, [id=#29]))) - -(76) CometProject -Input [2]: [c_customer_sk#46, ssales#51] -Arguments: [c_customer_sk#46], [c_customer_sk#46] - -(77) CometSort -Input [1]: [c_customer_sk#46] -Arguments: [c_customer_sk#46], [c_customer_sk#46 ASC NULLS FIRST] - -(78) CometSortMergeJoin -Left output [4]: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -Right output [1]: [c_customer_sk#46] -Arguments: [ws_bill_customer_sk#41], [c_customer_sk#46], LeftSemi - -(79) ReusedExchange [Reuses operator id: 53] -Output [3]: [c_customer_sk#52, c_first_name#53, c_last_name#54] - -(80) CometBroadcastHashJoin -Left output [4]: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -Right output [3]: [c_customer_sk#52, c_first_name#53, c_last_name#54] -Arguments: [ws_bill_customer_sk#41], [c_customer_sk#52], Inner, BuildRight - -(81) CometProject -Input [7]: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44, c_customer_sk#52, c_first_name#53, c_last_name#54] -Arguments: [ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44, c_first_name#53, c_last_name#54], [ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44, c_first_name#53, c_last_name#54] - -(82) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#55] - -(83) CometBroadcastHashJoin -Left output [5]: [ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44, c_first_name#53, c_last_name#54] -Right output [1]: [d_date_sk#55] -Arguments: [ws_sold_date_sk#44], [d_date_sk#55], Inner, BuildRight - -(84) CometProject -Input [6]: [ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44, c_first_name#53, c_last_name#54, d_date_sk#55] -Arguments: [ws_quantity#42, ws_list_price#43, c_first_name#53, c_last_name#54], [ws_quantity#42, ws_list_price#43, c_first_name#53, c_last_name#54] - -(85) CometHashAggregate -Input [4]: [ws_quantity#42, ws_list_price#43, c_first_name#53, c_last_name#54] -Keys [2]: [c_last_name#54, c_first_name#53] -Functions [1]: [partial_sum((cast(ws_quantity#42 as decimal(10,0)) * ws_list_price#43))] - -(86) CometExchange -Input [4]: [c_last_name#54, c_first_name#53, sum#56, isEmpty#57] -Arguments: hashpartitioning(c_last_name#54, c_first_name#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(87) CometHashAggregate -Input [4]: [c_last_name#54, c_first_name#53, sum#56, isEmpty#57] -Keys [2]: [c_last_name#54, c_first_name#53] -Functions [1]: [sum((cast(ws_quantity#42 as decimal(10,0)) * ws_list_price#43))] - -(88) CometUnion -Child 0 Input [3]: [c_last_name#34, c_first_name#33, sales#58] -Child 1 Input [3]: [c_last_name#54, c_first_name#53, sales#59] - -(89) CometTakeOrderedAndProject -Input [3]: [c_last_name#34, c_first_name#33, sales#58] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#34 ASC NULLS FIRST,c_first_name#33 ASC NULLS FIRST,sales#58 ASC NULLS FIRST], output=[c_last_name#34,c_first_name#33,sales#58]), [c_last_name#34, c_first_name#33, sales#58], 100, 0, [c_last_name#34 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, sales#58 ASC NULLS FIRST], [c_last_name#34, c_first_name#33, sales#58] - -(90) CometColumnarToRow [codegen id : 11] -Input [3]: [c_last_name#34, c_first_name#33, sales#58] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (95) -+- * CometColumnarToRow (94) - +- CometProject (93) - +- CometFilter (92) - +- CometNativeScan parquet spark_catalog.default.date_dim (91) - - -(91) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#35, d_year#36, d_moy#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(92) CometFilter -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] -Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2000)) AND (d_moy#37 = 2)) AND isnotnull(d_date_sk#35)) - -(93) CometProject -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] -Arguments: [d_date_sk#35], [d_date_sk#35] - -(94) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#35] - -(95) BroadcastExchange -Input [1]: [d_date_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (100) -+- * CometColumnarToRow (99) - +- CometProject (98) - +- CometFilter (97) - +- CometNativeScan parquet spark_catalog.default.date_dim (96) - - -(96) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#60] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(97) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#60] -Condition : (d_year#60 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(98) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#60] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(99) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_date#11] - -(100) BroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -Subquery:3 Hosting operator id = 38 Hosting Expression = Subquery scalar-subquery#28, [id=#29] -* HashAggregate (120) -+- * CometColumnarToRow (119) - +- CometColumnarExchange (118) - +- * HashAggregate (117) - +- * HashAggregate (116) - +- * CometColumnarToRow (115) - +- CometColumnarExchange (114) - +- * HashAggregate (113) - +- * Project (112) - +- * BroadcastHashJoin Inner BuildRight (111) - :- * Project (109) - : +- * BroadcastHashJoin Inner BuildRight (108) - : :- * Filter (103) - : : +- * ColumnarToRow (102) - : : +- Scan parquet spark_catalog.default.store_sales (101) - : +- BroadcastExchange (107) - : +- * CometColumnarToRow (106) - : +- CometFilter (105) - : +- CometNativeScan parquet spark_catalog.default.customer (104) - +- ReusedExchange (110) - - -(101) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#64), dynamicpruningexpression(ss_sold_date_sk#64 IN dynamicpruning#65)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(102) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64] - -(103) Filter [codegen id : 3] -Input [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64] -Condition : isnotnull(ss_customer_sk#61) - -(104) CometNativeScan parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(105) CometFilter -Input [1]: [c_customer_sk#66] -Condition : isnotnull(c_customer_sk#66) - -(106) CometColumnarToRow [codegen id : 1] -Input [1]: [c_customer_sk#66] - -(107) BroadcastExchange -Input [1]: [c_customer_sk#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -(108) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#61] -Right keys [1]: [c_customer_sk#66] -Join type: Inner -Join condition: None - -(109) Project [codegen id : 3] -Output [4]: [ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, c_customer_sk#66] -Input [5]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, c_customer_sk#66] - -(110) ReusedExchange [Reuses operator id: 125] -Output [1]: [d_date_sk#67] - -(111) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#64] -Right keys [1]: [d_date_sk#67] -Join type: Inner -Join condition: None - -(112) Project [codegen id : 3] -Output [3]: [ss_quantity#62, ss_sales_price#63, c_customer_sk#66] -Input [5]: [ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, c_customer_sk#66, d_date_sk#67] - -(113) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#62, ss_sales_price#63, c_customer_sk#66] -Keys [1]: [c_customer_sk#66] -Functions [1]: [partial_sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))] -Aggregate Attributes [2]: [sum#68, isEmpty#69] -Results [3]: [c_customer_sk#66, sum#70, isEmpty#71] - -(114) CometColumnarExchange -Input [3]: [c_customer_sk#66, sum#70, isEmpty#71] -Arguments: hashpartitioning(c_customer_sk#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(115) CometColumnarToRow [codegen id : 4] -Input [3]: [c_customer_sk#66, sum#70, isEmpty#71] - -(116) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#66, sum#70, isEmpty#71] -Keys [1]: [c_customer_sk#66] -Functions [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))#72] -Results [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))#72 AS csales#73] - -(117) HashAggregate [codegen id : 4] -Input [1]: [csales#73] -Keys: [] -Functions [1]: [partial_max(csales#73)] -Aggregate Attributes [1]: [max#74] -Results [1]: [max#75] - -(118) CometColumnarExchange -Input [1]: [max#75] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(119) CometColumnarToRow [codegen id : 5] -Input [1]: [max#75] - -(120) HashAggregate [codegen id : 5] -Input [1]: [max#75] -Keys: [] -Functions [1]: [max(csales#73)] -Aggregate Attributes [1]: [max(csales#73)#76] -Results [1]: [max(csales#73)#76 AS tpcds_cmax#77] - -Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#64 IN dynamicpruning#65 -BroadcastExchange (125) -+- * CometColumnarToRow (124) - +- CometProject (123) - +- CometFilter (122) - +- CometNativeScan parquet spark_catalog.default.date_dim (121) - - -(121) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#67, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(122) CometFilter -Input [2]: [d_date_sk#67, d_year#78] -Condition : (d_year#78 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#67)) - -(123) CometProject -Input [2]: [d_date_sk#67, d_year#78] -Arguments: [d_date_sk#67], [d_date_sk#67] - -(124) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#67] - -(125) BroadcastExchange -Input [1]: [d_date_sk#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] - -Subquery:5 Hosting operator id = 48 Hosting Expression = ReusedSubquery Subquery scalar-subquery#28, [id=#29] - -Subquery:6 Hosting operator id = 65 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 - -Subquery:7 Hosting operator id = 75 Hosting Expression = ReusedSubquery Subquery scalar-subquery#28, [id=#29] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/simplified.txt deleted file mode 100644 index 41f01311f8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/simplified.txt +++ /dev/null @@ -1,164 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,sales] - CometUnion [c_last_name,c_first_name,sales] - CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price))] - CometExchange [c_last_name,c_first_name] #1 - CometHashAggregate [cs_quantity,cs_list_price] [c_last_name,c_first_name,sum,isEmpty] - CometProject [cs_quantity,cs_list_price,c_first_name,c_last_name] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometColumnarExchange [cs_bill_customer_sk] #2 - WholeStageCodegen (5) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [item_sk] - Filter [cnt] - HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 - WholeStageCodegen (3) - HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (5) - HashAggregate [max] [max(csales),tpcds_cmax,max] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (4) - HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_sk] #11 - WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #12 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] - InputAdapter - ReusedExchange [d_date_sk] #12 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #8 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,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,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #9 - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #14 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [c_customer_sk,c_first_name,c_last_name,c_customer_sk] - CometSort [c_customer_sk,c_first_name,c_last_name] - CometExchange [c_customer_sk] #15 - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #8 - CometBroadcastExchange [d_date_sk] #16 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price))] - CometExchange [c_last_name,c_first_name] #17 - CometHashAggregate [ws_quantity,ws_list_price] [c_last_name,c_first_name,sum,isEmpty] - CometProject [ws_quantity,ws_list_price,c_first_name,c_last_name] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometColumnarExchange [ws_bill_customer_sk] #18 - WholeStageCodegen (10) - Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - BroadcastHashJoin [ws_item_sk,item_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #8 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #14 - ReusedExchange [d_date_sk] #16 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/explain.txt deleted file mode 100644 index 474a708a44..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,692 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (88) -+- CometTakeOrderedAndProject (87) - +- CometUnion (86) - :- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometSortMergeJoin (40) - : : : :- CometSort (25) - : : : : +- CometExchange (24) - : : : : +- CometProject (23) - : : : : +- CometBroadcastHashJoin (22) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometBroadcastExchange (21) - : : : : +- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometHashAggregate (18) - : : : : +- CometExchange (17) - : : : : +- CometHashAggregate (16) - : : : : +- CometProject (15) - : : : : +- CometBroadcastHashJoin (14) - : : : : :- CometProject (10) - : : : : : +- CometBroadcastHashJoin (9) - : : : : : :- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : : +- CometBroadcastExchange (8) - : : : : : +- CometProject (7) - : : : : : +- CometFilter (6) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (5) - : : : : +- CometBroadcastExchange (13) - : : : : +- CometFilter (12) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : +- CometSort (39) - : : : +- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometHashAggregate (36) - : : : +- CometExchange (35) - : : : +- CometHashAggregate (34) - : : : +- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometProject (28) - : : : : +- CometFilter (27) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (26) - : : : +- CometBroadcastExchange (31) - : : : +- CometFilter (30) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (29) - : : +- CometBroadcastExchange (52) - : : +- CometProject (51) - : : +- CometSortMergeJoin (50) - : : :- CometSort (44) - : : : +- CometExchange (43) - : : : +- CometFilter (42) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (41) - : : +- CometSort (49) - : : +- CometProject (48) - : : +- CometFilter (47) - : : +- CometHashAggregate (46) - : : +- ReusedExchange (45) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometHashAggregate (85) - +- CometExchange (84) - +- CometHashAggregate (83) - +- CometProject (82) - +- CometBroadcastHashJoin (81) - :- CometProject (79) - : +- CometBroadcastHashJoin (78) - : :- CometSortMergeJoin (76) - : : :- CometSort (70) - : : : +- CometExchange (69) - : : : +- CometProject (68) - : : : +- CometBroadcastHashJoin (67) - : : : :- CometFilter (65) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (64) - : : : +- ReusedExchange (66) - : : +- CometSort (75) - : : +- CometProject (74) - : : +- CometFilter (73) - : : +- CometHashAggregate (72) - : : +- ReusedExchange (71) - : +- ReusedExchange (77) - +- ReusedExchange (80) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_bill_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_item_sk#7) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(7) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(8) CometBroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: [d_date_sk#10, d_date#11] - -(9) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Right output [2]: [d_date_sk#10, d_date#11] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(10) CometProject -Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] -Arguments: [ss_item_sk#7, d_date#11], [ss_item_sk#7, d_date#11] - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#13, i_item_desc#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [2]: [i_item_sk#13, i_item_desc#14] -Condition : isnotnull(i_item_sk#13) - -(13) CometBroadcastExchange -Input [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [i_item_sk#13, i_item_desc#14] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, d_date#11] -Right output [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [ss_item_sk#7], [i_item_sk#13], Inner, BuildRight - -(15) CometProject -Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#13, i_item_desc#14] -Arguments: [d_date#11, i_item_sk#13, _groupingexpression#15], [d_date#11, i_item_sk#13, substr(i_item_desc#14, 1, 30) AS _groupingexpression#15] - -(16) CometHashAggregate -Input [3]: [d_date#11, i_item_sk#13, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [partial_count(1)] - -(17) CometExchange -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [count(1)] - -(19) CometFilter -Input [2]: [item_sk#17, cnt#18] -Condition : (cnt#18 > 4) - -(20) CometProject -Input [2]: [item_sk#17, cnt#18] -Arguments: [item_sk#17], [item_sk#17] - -(21) CometBroadcastExchange -Input [1]: [item_sk#17] -Arguments: [item_sk#17] - -(22) CometBroadcastHashJoin -Left output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [item_sk#17] -Arguments: [cs_item_sk#2], [item_sk#17], LeftSemi, BuildRight - -(23) CometProject -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(24) CometExchange -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(25) CometSort -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#19) - -(28) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Arguments: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21], [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(30) CometFilter -Input [1]: [c_customer_sk#23] -Condition : isnotnull(c_customer_sk#23) - -(31) CometBroadcastExchange -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23] - -(32) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] -Right output [1]: [c_customer_sk#23] -Arguments: [ss_customer_sk#19], [c_customer_sk#23], Inner, BuildRight - -(33) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Arguments: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23], [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] - -(34) CometHashAggregate -Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Keys [1]: [c_customer_sk#23] -Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(35) CometExchange -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(36) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(37) CometFilter -Input [2]: [c_customer_sk#23, ssales#26] -Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#27, [id=#28]))) - -(38) CometProject -Input [2]: [c_customer_sk#23, ssales#26] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(39) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(40) CometSortMergeJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [c_customer_sk#23] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#23], LeftSemi - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Condition : isnotnull(c_customer_sk#29) - -(43) CometExchange -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(44) CometSort -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Arguments: [c_customer_sk#29, c_first_name#30, c_last_name#31], [c_customer_sk#29 ASC NULLS FIRST] - -(45) ReusedExchange [Reuses operator id: 35] -Output [3]: [c_customer_sk#23, sum#24, isEmpty#25] - -(46) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(47) CometFilter -Input [2]: [c_customer_sk#23, ssales#26] -Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) - -(48) CometProject -Input [2]: [c_customer_sk#23, ssales#26] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(49) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(50) CometSortMergeJoin -Left output [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Right output [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#29], [c_customer_sk#23], LeftSemi - -(51) CometProject -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Arguments: [c_customer_sk#29, c_first_name#32, c_last_name#33], [c_customer_sk#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#30, 20, true, false, true) AS c_first_name#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#31, 30, true, false, true) AS c_last_name#33] - -(52) CometBroadcastExchange -Input [3]: [c_customer_sk#29, c_first_name#32, c_last_name#33] -Arguments: [c_customer_sk#29, c_first_name#32, c_last_name#33] - -(53) CometBroadcastHashJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [3]: [c_customer_sk#29, c_first_name#32, c_last_name#33] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#29], Inner, BuildRight - -(54) CometProject -Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#29, c_first_name#32, c_last_name#33] -Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#35, d_moy#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Condition : ((((isnotnull(d_year#35) AND isnotnull(d_moy#36)) AND (d_year#35 = 2000)) AND (d_moy#36 = 2)) AND isnotnull(d_date_sk#34)) - -(57) CometProject -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Arguments: [d_date_sk#34], [d_date_sk#34] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#34] -Arguments: [d_date_sk#34] - -(59) CometBroadcastHashJoin -Left output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33] -Right output [1]: [d_date_sk#34] -Arguments: [cs_sold_date_sk#5], [d_date_sk#34], Inner, BuildRight - -(60) CometProject -Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33, d_date_sk#34] -Arguments: [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33], [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33] - -(61) CometHashAggregate -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33] -Keys [2]: [c_last_name#33, c_first_name#32] -Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] - -(62) CometExchange -Input [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] -Arguments: hashpartitioning(c_last_name#33, c_first_name#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(63) CometHashAggregate -Input [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] -Keys [2]: [c_last_name#33, c_first_name#32] -Functions [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#43), dynamicpruningexpression(ws_sold_date_sk#43 IN dynamicpruning#44)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(65) CometFilter -Input [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Condition : isnotnull(ws_bill_customer_sk#40) - -(66) ReusedExchange [Reuses operator id: 21] -Output [1]: [item_sk#45] - -(67) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Right output [1]: [item_sk#45] -Arguments: [ws_item_sk#39], [item_sk#45], LeftSemi, BuildRight - -(68) CometProject -Input [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Arguments: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43], [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] - -(69) CometExchange -Input [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Arguments: hashpartitioning(ws_bill_customer_sk#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(70) CometSort -Input [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Arguments: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43], [ws_bill_customer_sk#40 ASC NULLS FIRST] - -(71) ReusedExchange [Reuses operator id: 35] -Output [3]: [c_customer_sk#46, sum#47, isEmpty#48] - -(72) CometHashAggregate -Input [3]: [c_customer_sk#46, sum#47, isEmpty#48] -Keys [1]: [c_customer_sk#46] -Functions [1]: [sum((cast(ss_quantity#49 as decimal(10,0)) * ss_sales_price#50))] - -(73) CometFilter -Input [2]: [c_customer_sk#46, ssales#51] -Condition : (isnotnull(ssales#51) AND (cast(ssales#51 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) - -(74) CometProject -Input [2]: [c_customer_sk#46, ssales#51] -Arguments: [c_customer_sk#46], [c_customer_sk#46] - -(75) CometSort -Input [1]: [c_customer_sk#46] -Arguments: [c_customer_sk#46], [c_customer_sk#46 ASC NULLS FIRST] - -(76) CometSortMergeJoin -Left output [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Right output [1]: [c_customer_sk#46] -Arguments: [ws_bill_customer_sk#40], [c_customer_sk#46], LeftSemi - -(77) ReusedExchange [Reuses operator id: 52] -Output [3]: [c_customer_sk#52, c_first_name#53, c_last_name#54] - -(78) CometBroadcastHashJoin -Left output [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Right output [3]: [c_customer_sk#52, c_first_name#53, c_last_name#54] -Arguments: [ws_bill_customer_sk#40], [c_customer_sk#52], Inner, BuildRight - -(79) CometProject -Input [7]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_customer_sk#52, c_first_name#53, c_last_name#54] -Arguments: [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#53, c_last_name#54], [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#53, c_last_name#54] - -(80) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#55] - -(81) CometBroadcastHashJoin -Left output [5]: [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#53, c_last_name#54] -Right output [1]: [d_date_sk#55] -Arguments: [ws_sold_date_sk#43], [d_date_sk#55], Inner, BuildRight - -(82) CometProject -Input [6]: [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#53, c_last_name#54, d_date_sk#55] -Arguments: [ws_quantity#41, ws_list_price#42, c_first_name#53, c_last_name#54], [ws_quantity#41, ws_list_price#42, c_first_name#53, c_last_name#54] - -(83) CometHashAggregate -Input [4]: [ws_quantity#41, ws_list_price#42, c_first_name#53, c_last_name#54] -Keys [2]: [c_last_name#54, c_first_name#53] -Functions [1]: [partial_sum((cast(ws_quantity#41 as decimal(10,0)) * ws_list_price#42))] - -(84) CometExchange -Input [4]: [c_last_name#54, c_first_name#53, sum#56, isEmpty#57] -Arguments: hashpartitioning(c_last_name#54, c_first_name#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(85) CometHashAggregate -Input [4]: [c_last_name#54, c_first_name#53, sum#56, isEmpty#57] -Keys [2]: [c_last_name#54, c_first_name#53] -Functions [1]: [sum((cast(ws_quantity#41 as decimal(10,0)) * ws_list_price#42))] - -(86) CometUnion -Child 0 Input [3]: [c_last_name#33, c_first_name#32, sales#58] -Child 1 Input [3]: [c_last_name#54, c_first_name#53, sales#59] - -(87) CometTakeOrderedAndProject -Input [3]: [c_last_name#33, c_first_name#32, sales#58] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#33 ASC NULLS FIRST,c_first_name#32 ASC NULLS FIRST,sales#58 ASC NULLS FIRST], output=[c_last_name#33,c_first_name#32,sales#58]), [c_last_name#33, c_first_name#32, sales#58], 100, 0, [c_last_name#33 ASC NULLS FIRST, c_first_name#32 ASC NULLS FIRST, sales#58 ASC NULLS FIRST], [c_last_name#33, c_first_name#32, sales#58] - -(88) CometColumnarToRow [codegen id : 1] -Input [3]: [c_last_name#33, c_first_name#32, sales#58] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (93) -+- * CometColumnarToRow (92) - +- CometProject (91) - +- CometFilter (90) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (89) - - -(89) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#35, d_moy#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(90) CometFilter -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Condition : ((((isnotnull(d_year#35) AND isnotnull(d_moy#36)) AND (d_year#35 = 2000)) AND (d_moy#36 = 2)) AND isnotnull(d_date_sk#34)) - -(91) CometProject -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Arguments: [d_date_sk#34], [d_date_sk#34] - -(92) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#34] - -(93) BroadcastExchange -Input [1]: [d_date_sk#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (98) -+- * CometColumnarToRow (97) - +- CometProject (96) - +- CometFilter (95) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (94) - - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(95) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(96) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(97) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_date#11] - -(98) BroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:3 Hosting operator id = 37 Hosting Expression = Subquery scalar-subquery#27, [id=#28] -* CometColumnarToRow (116) -+- CometHashAggregate (115) - +- CometExchange (114) - +- CometHashAggregate (113) - +- CometHashAggregate (112) - +- CometExchange (111) - +- CometHashAggregate (110) - +- CometProject (109) - +- CometBroadcastHashJoin (108) - :- CometProject (103) - : +- CometBroadcastHashJoin (102) - : :- CometFilter (100) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (99) - : +- ReusedExchange (101) - +- CometBroadcastExchange (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) - - -(99) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#60, ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#63), dynamicpruningexpression(ss_sold_date_sk#63 IN dynamicpruning#64)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(100) CometFilter -Input [4]: [ss_customer_sk#60, ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63] -Condition : isnotnull(ss_customer_sk#60) - -(101) ReusedExchange [Reuses operator id: 31] -Output [1]: [c_customer_sk#65] - -(102) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#60, ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63] -Right output [1]: [c_customer_sk#65] -Arguments: [ss_customer_sk#60], [c_customer_sk#65], Inner, BuildRight - -(103) CometProject -Input [5]: [ss_customer_sk#60, ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63, c_customer_sk#65] -Arguments: [ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63, c_customer_sk#65], [ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63, c_customer_sk#65] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#66, d_year#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [d_date_sk#66, d_year#67] -Condition : (d_year#67 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#66)) - -(106) CometProject -Input [2]: [d_date_sk#66, d_year#67] -Arguments: [d_date_sk#66], [d_date_sk#66] - -(107) CometBroadcastExchange -Input [1]: [d_date_sk#66] -Arguments: [d_date_sk#66] - -(108) CometBroadcastHashJoin -Left output [4]: [ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63, c_customer_sk#65] -Right output [1]: [d_date_sk#66] -Arguments: [ss_sold_date_sk#63], [d_date_sk#66], Inner, BuildRight - -(109) CometProject -Input [5]: [ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63, c_customer_sk#65, d_date_sk#66] -Arguments: [ss_quantity#61, ss_sales_price#62, c_customer_sk#65], [ss_quantity#61, ss_sales_price#62, c_customer_sk#65] - -(110) CometHashAggregate -Input [3]: [ss_quantity#61, ss_sales_price#62, c_customer_sk#65] -Keys [1]: [c_customer_sk#65] -Functions [1]: [partial_sum((cast(ss_quantity#61 as decimal(10,0)) * ss_sales_price#62))] - -(111) CometExchange -Input [3]: [c_customer_sk#65, sum#68, isEmpty#69] -Arguments: hashpartitioning(c_customer_sk#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(112) CometHashAggregate -Input [3]: [c_customer_sk#65, sum#68, isEmpty#69] -Keys [1]: [c_customer_sk#65] -Functions [1]: [sum((cast(ss_quantity#61 as decimal(10,0)) * ss_sales_price#62))] - -(113) CometHashAggregate -Input [1]: [csales#70] -Keys: [] -Functions [1]: [partial_max(csales#70)] - -(114) CometExchange -Input [1]: [max#71] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(115) CometHashAggregate -Input [1]: [max#71] -Keys: [] -Functions [1]: [max(csales#70)] - -(116) CometColumnarToRow [codegen id : 1] -Input [1]: [tpcds_cmax#72] - -Subquery:4 Hosting operator id = 99 Hosting Expression = ss_sold_date_sk#63 IN dynamicpruning#64 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#66, d_year#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#66, d_year#67] -Condition : (d_year#67 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#66)) - -(119) CometProject -Input [2]: [d_date_sk#66, d_year#67] -Arguments: [d_date_sk#66], [d_date_sk#66] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#66] - -(121) BroadcastExchange -Input [1]: [d_date_sk#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -Subquery:5 Hosting operator id = 47 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] - -Subquery:6 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#43 IN dynamicpruning#6 - -Subquery:7 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/simplified.txt deleted file mode 100644 index 92563114ae..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,138 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,sales] - CometUnion [c_last_name,c_first_name,sales] - CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price))] - CometExchange [c_last_name,c_first_name] #1 - CometHashAggregate [cs_quantity,cs_list_price] [c_last_name,c_first_name,sum,isEmpty] - CometProject [cs_quantity,cs_list_price,c_first_name,c_last_name] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometExchange [cs_bill_customer_sk] #2 - CometProject [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,item_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [item_sk] #4 - CometProject [item_sk] - CometFilter [item_sk,cnt] - CometHashAggregate [_groupingexpression,d_date,count] [item_sk,cnt,i_item_sk,count(1)] - CometExchange [_groupingexpression,i_item_sk,d_date] #5 - CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] - CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] - CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [i_item_sk,i_item_desc] #8 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [max] [tpcds_cmax,max(csales)] - CometExchange #11 - CometHashAggregate [csales] [max] - CometHashAggregate [c_customer_sk,sum,isEmpty] [csales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #12 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,c_customer_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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #9 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,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,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #10 - CometFilter [c_customer_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #15 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [c_customer_sk,c_first_name,c_last_name,c_customer_sk] - CometSort [c_customer_sk,c_first_name,c_last_name] - CometExchange [c_customer_sk] #16 - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #9 - CometBroadcastExchange [d_date_sk] #17 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price))] - CometExchange [c_last_name,c_first_name] #18 - CometHashAggregate [ws_quantity,ws_list_price] [c_last_name,c_first_name,sum,isEmpty] - CometProject [ws_quantity,ws_list_price,c_first_name,c_last_name] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometExchange [ws_bill_customer_sk] #19 - CometProject [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #9 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 - ReusedExchange [d_date_sk] #17 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/explain.txt deleted file mode 100644 index 474a708a44..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/explain.txt +++ /dev/null @@ -1,692 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (88) -+- CometTakeOrderedAndProject (87) - +- CometUnion (86) - :- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometSortMergeJoin (40) - : : : :- CometSort (25) - : : : : +- CometExchange (24) - : : : : +- CometProject (23) - : : : : +- CometBroadcastHashJoin (22) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometBroadcastExchange (21) - : : : : +- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometHashAggregate (18) - : : : : +- CometExchange (17) - : : : : +- CometHashAggregate (16) - : : : : +- CometProject (15) - : : : : +- CometBroadcastHashJoin (14) - : : : : :- CometProject (10) - : : : : : +- CometBroadcastHashJoin (9) - : : : : : :- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : : +- CometBroadcastExchange (8) - : : : : : +- CometProject (7) - : : : : : +- CometFilter (6) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (5) - : : : : +- CometBroadcastExchange (13) - : : : : +- CometFilter (12) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : +- CometSort (39) - : : : +- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometHashAggregate (36) - : : : +- CometExchange (35) - : : : +- CometHashAggregate (34) - : : : +- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometProject (28) - : : : : +- CometFilter (27) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (26) - : : : +- CometBroadcastExchange (31) - : : : +- CometFilter (30) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (29) - : : +- CometBroadcastExchange (52) - : : +- CometProject (51) - : : +- CometSortMergeJoin (50) - : : :- CometSort (44) - : : : +- CometExchange (43) - : : : +- CometFilter (42) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (41) - : : +- CometSort (49) - : : +- CometProject (48) - : : +- CometFilter (47) - : : +- CometHashAggregate (46) - : : +- ReusedExchange (45) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometHashAggregate (85) - +- CometExchange (84) - +- CometHashAggregate (83) - +- CometProject (82) - +- CometBroadcastHashJoin (81) - :- CometProject (79) - : +- CometBroadcastHashJoin (78) - : :- CometSortMergeJoin (76) - : : :- CometSort (70) - : : : +- CometExchange (69) - : : : +- CometProject (68) - : : : +- CometBroadcastHashJoin (67) - : : : :- CometFilter (65) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (64) - : : : +- ReusedExchange (66) - : : +- CometSort (75) - : : +- CometProject (74) - : : +- CometFilter (73) - : : +- CometHashAggregate (72) - : : +- ReusedExchange (71) - : +- ReusedExchange (77) - +- ReusedExchange (80) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_bill_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_item_sk#7) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(7) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(8) CometBroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: [d_date_sk#10, d_date#11] - -(9) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Right output [2]: [d_date_sk#10, d_date#11] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(10) CometProject -Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] -Arguments: [ss_item_sk#7, d_date#11], [ss_item_sk#7, d_date#11] - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#13, i_item_desc#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [2]: [i_item_sk#13, i_item_desc#14] -Condition : isnotnull(i_item_sk#13) - -(13) CometBroadcastExchange -Input [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [i_item_sk#13, i_item_desc#14] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, d_date#11] -Right output [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [ss_item_sk#7], [i_item_sk#13], Inner, BuildRight - -(15) CometProject -Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#13, i_item_desc#14] -Arguments: [d_date#11, i_item_sk#13, _groupingexpression#15], [d_date#11, i_item_sk#13, substr(i_item_desc#14, 1, 30) AS _groupingexpression#15] - -(16) CometHashAggregate -Input [3]: [d_date#11, i_item_sk#13, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [partial_count(1)] - -(17) CometExchange -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [count(1)] - -(19) CometFilter -Input [2]: [item_sk#17, cnt#18] -Condition : (cnt#18 > 4) - -(20) CometProject -Input [2]: [item_sk#17, cnt#18] -Arguments: [item_sk#17], [item_sk#17] - -(21) CometBroadcastExchange -Input [1]: [item_sk#17] -Arguments: [item_sk#17] - -(22) CometBroadcastHashJoin -Left output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [item_sk#17] -Arguments: [cs_item_sk#2], [item_sk#17], LeftSemi, BuildRight - -(23) CometProject -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(24) CometExchange -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(25) CometSort -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#19) - -(28) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Arguments: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21], [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(30) CometFilter -Input [1]: [c_customer_sk#23] -Condition : isnotnull(c_customer_sk#23) - -(31) CometBroadcastExchange -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23] - -(32) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] -Right output [1]: [c_customer_sk#23] -Arguments: [ss_customer_sk#19], [c_customer_sk#23], Inner, BuildRight - -(33) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Arguments: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23], [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] - -(34) CometHashAggregate -Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Keys [1]: [c_customer_sk#23] -Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(35) CometExchange -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(36) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(37) CometFilter -Input [2]: [c_customer_sk#23, ssales#26] -Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#27, [id=#28]))) - -(38) CometProject -Input [2]: [c_customer_sk#23, ssales#26] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(39) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(40) CometSortMergeJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [c_customer_sk#23] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#23], LeftSemi - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Condition : isnotnull(c_customer_sk#29) - -(43) CometExchange -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(44) CometSort -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Arguments: [c_customer_sk#29, c_first_name#30, c_last_name#31], [c_customer_sk#29 ASC NULLS FIRST] - -(45) ReusedExchange [Reuses operator id: 35] -Output [3]: [c_customer_sk#23, sum#24, isEmpty#25] - -(46) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(47) CometFilter -Input [2]: [c_customer_sk#23, ssales#26] -Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) - -(48) CometProject -Input [2]: [c_customer_sk#23, ssales#26] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(49) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(50) CometSortMergeJoin -Left output [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Right output [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#29], [c_customer_sk#23], LeftSemi - -(51) CometProject -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Arguments: [c_customer_sk#29, c_first_name#32, c_last_name#33], [c_customer_sk#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#30, 20, true, false, true) AS c_first_name#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#31, 30, true, false, true) AS c_last_name#33] - -(52) CometBroadcastExchange -Input [3]: [c_customer_sk#29, c_first_name#32, c_last_name#33] -Arguments: [c_customer_sk#29, c_first_name#32, c_last_name#33] - -(53) CometBroadcastHashJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [3]: [c_customer_sk#29, c_first_name#32, c_last_name#33] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#29], Inner, BuildRight - -(54) CometProject -Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#29, c_first_name#32, c_last_name#33] -Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#35, d_moy#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Condition : ((((isnotnull(d_year#35) AND isnotnull(d_moy#36)) AND (d_year#35 = 2000)) AND (d_moy#36 = 2)) AND isnotnull(d_date_sk#34)) - -(57) CometProject -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Arguments: [d_date_sk#34], [d_date_sk#34] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#34] -Arguments: [d_date_sk#34] - -(59) CometBroadcastHashJoin -Left output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33] -Right output [1]: [d_date_sk#34] -Arguments: [cs_sold_date_sk#5], [d_date_sk#34], Inner, BuildRight - -(60) CometProject -Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33, d_date_sk#34] -Arguments: [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33], [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33] - -(61) CometHashAggregate -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33] -Keys [2]: [c_last_name#33, c_first_name#32] -Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] - -(62) CometExchange -Input [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] -Arguments: hashpartitioning(c_last_name#33, c_first_name#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(63) CometHashAggregate -Input [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] -Keys [2]: [c_last_name#33, c_first_name#32] -Functions [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#43), dynamicpruningexpression(ws_sold_date_sk#43 IN dynamicpruning#44)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(65) CometFilter -Input [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Condition : isnotnull(ws_bill_customer_sk#40) - -(66) ReusedExchange [Reuses operator id: 21] -Output [1]: [item_sk#45] - -(67) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Right output [1]: [item_sk#45] -Arguments: [ws_item_sk#39], [item_sk#45], LeftSemi, BuildRight - -(68) CometProject -Input [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Arguments: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43], [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] - -(69) CometExchange -Input [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Arguments: hashpartitioning(ws_bill_customer_sk#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(70) CometSort -Input [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Arguments: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43], [ws_bill_customer_sk#40 ASC NULLS FIRST] - -(71) ReusedExchange [Reuses operator id: 35] -Output [3]: [c_customer_sk#46, sum#47, isEmpty#48] - -(72) CometHashAggregate -Input [3]: [c_customer_sk#46, sum#47, isEmpty#48] -Keys [1]: [c_customer_sk#46] -Functions [1]: [sum((cast(ss_quantity#49 as decimal(10,0)) * ss_sales_price#50))] - -(73) CometFilter -Input [2]: [c_customer_sk#46, ssales#51] -Condition : (isnotnull(ssales#51) AND (cast(ssales#51 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) - -(74) CometProject -Input [2]: [c_customer_sk#46, ssales#51] -Arguments: [c_customer_sk#46], [c_customer_sk#46] - -(75) CometSort -Input [1]: [c_customer_sk#46] -Arguments: [c_customer_sk#46], [c_customer_sk#46 ASC NULLS FIRST] - -(76) CometSortMergeJoin -Left output [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Right output [1]: [c_customer_sk#46] -Arguments: [ws_bill_customer_sk#40], [c_customer_sk#46], LeftSemi - -(77) ReusedExchange [Reuses operator id: 52] -Output [3]: [c_customer_sk#52, c_first_name#53, c_last_name#54] - -(78) CometBroadcastHashJoin -Left output [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Right output [3]: [c_customer_sk#52, c_first_name#53, c_last_name#54] -Arguments: [ws_bill_customer_sk#40], [c_customer_sk#52], Inner, BuildRight - -(79) CometProject -Input [7]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_customer_sk#52, c_first_name#53, c_last_name#54] -Arguments: [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#53, c_last_name#54], [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#53, c_last_name#54] - -(80) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#55] - -(81) CometBroadcastHashJoin -Left output [5]: [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#53, c_last_name#54] -Right output [1]: [d_date_sk#55] -Arguments: [ws_sold_date_sk#43], [d_date_sk#55], Inner, BuildRight - -(82) CometProject -Input [6]: [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#53, c_last_name#54, d_date_sk#55] -Arguments: [ws_quantity#41, ws_list_price#42, c_first_name#53, c_last_name#54], [ws_quantity#41, ws_list_price#42, c_first_name#53, c_last_name#54] - -(83) CometHashAggregate -Input [4]: [ws_quantity#41, ws_list_price#42, c_first_name#53, c_last_name#54] -Keys [2]: [c_last_name#54, c_first_name#53] -Functions [1]: [partial_sum((cast(ws_quantity#41 as decimal(10,0)) * ws_list_price#42))] - -(84) CometExchange -Input [4]: [c_last_name#54, c_first_name#53, sum#56, isEmpty#57] -Arguments: hashpartitioning(c_last_name#54, c_first_name#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(85) CometHashAggregate -Input [4]: [c_last_name#54, c_first_name#53, sum#56, isEmpty#57] -Keys [2]: [c_last_name#54, c_first_name#53] -Functions [1]: [sum((cast(ws_quantity#41 as decimal(10,0)) * ws_list_price#42))] - -(86) CometUnion -Child 0 Input [3]: [c_last_name#33, c_first_name#32, sales#58] -Child 1 Input [3]: [c_last_name#54, c_first_name#53, sales#59] - -(87) CometTakeOrderedAndProject -Input [3]: [c_last_name#33, c_first_name#32, sales#58] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#33 ASC NULLS FIRST,c_first_name#32 ASC NULLS FIRST,sales#58 ASC NULLS FIRST], output=[c_last_name#33,c_first_name#32,sales#58]), [c_last_name#33, c_first_name#32, sales#58], 100, 0, [c_last_name#33 ASC NULLS FIRST, c_first_name#32 ASC NULLS FIRST, sales#58 ASC NULLS FIRST], [c_last_name#33, c_first_name#32, sales#58] - -(88) CometColumnarToRow [codegen id : 1] -Input [3]: [c_last_name#33, c_first_name#32, sales#58] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (93) -+- * CometColumnarToRow (92) - +- CometProject (91) - +- CometFilter (90) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (89) - - -(89) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#35, d_moy#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(90) CometFilter -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Condition : ((((isnotnull(d_year#35) AND isnotnull(d_moy#36)) AND (d_year#35 = 2000)) AND (d_moy#36 = 2)) AND isnotnull(d_date_sk#34)) - -(91) CometProject -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Arguments: [d_date_sk#34], [d_date_sk#34] - -(92) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#34] - -(93) BroadcastExchange -Input [1]: [d_date_sk#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (98) -+- * CometColumnarToRow (97) - +- CometProject (96) - +- CometFilter (95) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (94) - - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(95) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(96) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(97) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_date#11] - -(98) BroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:3 Hosting operator id = 37 Hosting Expression = Subquery scalar-subquery#27, [id=#28] -* CometColumnarToRow (116) -+- CometHashAggregate (115) - +- CometExchange (114) - +- CometHashAggregate (113) - +- CometHashAggregate (112) - +- CometExchange (111) - +- CometHashAggregate (110) - +- CometProject (109) - +- CometBroadcastHashJoin (108) - :- CometProject (103) - : +- CometBroadcastHashJoin (102) - : :- CometFilter (100) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (99) - : +- ReusedExchange (101) - +- CometBroadcastExchange (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) - - -(99) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#60, ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#63), dynamicpruningexpression(ss_sold_date_sk#63 IN dynamicpruning#64)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(100) CometFilter -Input [4]: [ss_customer_sk#60, ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63] -Condition : isnotnull(ss_customer_sk#60) - -(101) ReusedExchange [Reuses operator id: 31] -Output [1]: [c_customer_sk#65] - -(102) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#60, ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63] -Right output [1]: [c_customer_sk#65] -Arguments: [ss_customer_sk#60], [c_customer_sk#65], Inner, BuildRight - -(103) CometProject -Input [5]: [ss_customer_sk#60, ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63, c_customer_sk#65] -Arguments: [ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63, c_customer_sk#65], [ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63, c_customer_sk#65] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#66, d_year#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [d_date_sk#66, d_year#67] -Condition : (d_year#67 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#66)) - -(106) CometProject -Input [2]: [d_date_sk#66, d_year#67] -Arguments: [d_date_sk#66], [d_date_sk#66] - -(107) CometBroadcastExchange -Input [1]: [d_date_sk#66] -Arguments: [d_date_sk#66] - -(108) CometBroadcastHashJoin -Left output [4]: [ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63, c_customer_sk#65] -Right output [1]: [d_date_sk#66] -Arguments: [ss_sold_date_sk#63], [d_date_sk#66], Inner, BuildRight - -(109) CometProject -Input [5]: [ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63, c_customer_sk#65, d_date_sk#66] -Arguments: [ss_quantity#61, ss_sales_price#62, c_customer_sk#65], [ss_quantity#61, ss_sales_price#62, c_customer_sk#65] - -(110) CometHashAggregate -Input [3]: [ss_quantity#61, ss_sales_price#62, c_customer_sk#65] -Keys [1]: [c_customer_sk#65] -Functions [1]: [partial_sum((cast(ss_quantity#61 as decimal(10,0)) * ss_sales_price#62))] - -(111) CometExchange -Input [3]: [c_customer_sk#65, sum#68, isEmpty#69] -Arguments: hashpartitioning(c_customer_sk#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(112) CometHashAggregate -Input [3]: [c_customer_sk#65, sum#68, isEmpty#69] -Keys [1]: [c_customer_sk#65] -Functions [1]: [sum((cast(ss_quantity#61 as decimal(10,0)) * ss_sales_price#62))] - -(113) CometHashAggregate -Input [1]: [csales#70] -Keys: [] -Functions [1]: [partial_max(csales#70)] - -(114) CometExchange -Input [1]: [max#71] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(115) CometHashAggregate -Input [1]: [max#71] -Keys: [] -Functions [1]: [max(csales#70)] - -(116) CometColumnarToRow [codegen id : 1] -Input [1]: [tpcds_cmax#72] - -Subquery:4 Hosting operator id = 99 Hosting Expression = ss_sold_date_sk#63 IN dynamicpruning#64 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#66, d_year#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#66, d_year#67] -Condition : (d_year#67 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#66)) - -(119) CometProject -Input [2]: [d_date_sk#66, d_year#67] -Arguments: [d_date_sk#66], [d_date_sk#66] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#66] - -(121) BroadcastExchange -Input [1]: [d_date_sk#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -Subquery:5 Hosting operator id = 47 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] - -Subquery:6 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#43 IN dynamicpruning#6 - -Subquery:7 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/extended.txt deleted file mode 100644 index cd91f5de49..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/extended.txt +++ /dev/null @@ -1,198 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometSort - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometSort - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometSort - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometSort - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 177 out of 190 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 92563114ae..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b/simplified.txt +++ /dev/null @@ -1,138 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,sales] - CometUnion [c_last_name,c_first_name,sales] - CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price))] - CometExchange [c_last_name,c_first_name] #1 - CometHashAggregate [cs_quantity,cs_list_price] [c_last_name,c_first_name,sum,isEmpty] - CometProject [cs_quantity,cs_list_price,c_first_name,c_last_name] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometExchange [cs_bill_customer_sk] #2 - CometProject [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,item_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [item_sk] #4 - CometProject [item_sk] - CometFilter [item_sk,cnt] - CometHashAggregate [_groupingexpression,d_date,count] [item_sk,cnt,i_item_sk,count(1)] - CometExchange [_groupingexpression,i_item_sk,d_date] #5 - CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] - CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] - CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [i_item_sk,i_item_desc] #8 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [max] [tpcds_cmax,max(csales)] - CometExchange #11 - CometHashAggregate [csales] [max] - CometHashAggregate [c_customer_sk,sum,isEmpty] [csales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #12 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,c_customer_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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #9 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,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,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #10 - CometFilter [c_customer_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #15 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [c_customer_sk,c_first_name,c_last_name,c_customer_sk] - CometSort [c_customer_sk,c_first_name,c_last_name] - CometExchange [c_customer_sk] #16 - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #9 - CometBroadcastExchange [d_date_sk] #17 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price))] - CometExchange [c_last_name,c_first_name] #18 - CometHashAggregate [ws_quantity,ws_list_price] [c_last_name,c_first_name,sum,isEmpty] - CometProject [ws_quantity,ws_list_price,c_first_name,c_last_name] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometExchange [ws_bill_customer_sk] #19 - CometProject [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #9 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 - ReusedExchange [d_date_sk] #17 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_datafusion/explain.txt deleted file mode 100644 index 67a43617c7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_datafusion/explain.txt +++ /dev/null @@ -1,439 +0,0 @@ -== Physical Plan == -* Filter (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometNativeScan parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometNativeScan parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.customer_address (32) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometNativeScan parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) = pale ) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#19, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#21, 10, true, false, true) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Condition : (isnotnull(c_customer_sk#26) AND isnotnull(c_birth_country#29)) - -(27) CometProject -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29], [c_customer_sk#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#31, c_birth_country#29] - -(28) CometBroadcastExchange -Input [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(31) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(32) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_state#32, ca_zip#33, ca_country#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true))) - -(34) CometProject -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] - -(35) CometColumnarToRow [codegen id : 1] -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] - -(36) BroadcastExchange -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#29, s_zip#16] -Right keys [2]: [upper(ca_country#34), ca_zip#36] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#37] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(40) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [partial_sum(netpaid#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(44) CometColumnarExchange -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [sum(netpaid#40)] -Aggregate Attributes [1]: [sum(netpaid#40)#45] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] -Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (77) -+- * CometColumnarToRow (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (56) - : : : +- CometBroadcastHashJoin (55) - : : : :- CometProject (53) - : : : : +- CometSortMergeJoin (52) - : : : : :- CometSort (49) - : : : : : +- ReusedExchange (48) - : : : : +- CometSort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (54) - : : +- CometBroadcastExchange (60) - : : +- CometProject (59) - : : +- CometFilter (58) - : : +- CometNativeScan parquet spark_catalog.default.item (57) - : +- ReusedExchange (63) - +- ReusedExchange (67) - - -(48) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] - -(49) CometSort -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53], [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST] - -(50) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#54, sr_ticket_number#55] - -(51) CometSort -Input [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [sr_item_sk#54, sr_ticket_number#55], [sr_ticket_number#55 ASC NULLS FIRST, sr_item_sk#54 ASC NULLS FIRST] - -(52) CometSortMergeJoin -Left output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Right output [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_ticket_number#52, ss_item_sk#49], [sr_ticket_number#55, sr_item_sk#54], Inner - -(53) CometProject -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53], [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] - -(54) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] - -(55) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Right output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_store_sk#51], [s_store_sk#56], Inner, BuildRight - -(56) CometProject -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16], [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] - -(57) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Condition : isnotnull(i_item_sk#58) - -(59) CometProject -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [i_item_sk#58, i_current_price#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#60, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#61, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#62, 10, true, false, true) AS i_units#25, i_manager_id#63] - -(60) CometBroadcastExchange -Input [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(61) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_item_sk#49], [i_item_sk#58], Inner, BuildRight - -(62) CometProject -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(63) ReusedExchange [Reuses operator id: 28] -Output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] - -(64) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Right output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight - -(65) CometProject -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(66) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(67) ReusedExchange [Reuses operator id: 36] -Output [3]: [ca_state#35, ca_zip#36, ca_country#66] - -(68) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#65, s_zip#16] -Right keys [2]: [upper(ca_country#66), ca_zip#36] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 2] -Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] - -(70) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#67] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(71) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(72) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(73) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#69] - -(74) HashAggregate [codegen id : 3] -Input [1]: [netpaid#69] -Keys: [] -Functions [1]: [partial_avg(netpaid#69)] -Aggregate Attributes [2]: [sum#70, count#71] -Results [2]: [sum#72, count#73] - -(75) CometColumnarExchange -Input [2]: [sum#72, count#73] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#72, count#73] - -(77) HashAggregate [codegen id : 4] -Input [2]: [sum#72, count#73] -Keys: [] -Functions [1]: [avg(netpaid#69)] -Aggregate Attributes [1]: [avg(netpaid#69)#74] -Results [1]: [(0.05 * avg(netpaid#69)#74) AS (0.05 * avg(netpaid))#75] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_datafusion/simplified.txt deleted file mode 100644 index af8d5ee7aa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_datafusion/simplified.txt +++ /dev/null @@ -1,94 +0,0 @@ -WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #6 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometNativeScan 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/q24a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/explain.txt deleted file mode 100644 index c6e3a30a1f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,439 +0,0 @@ -== Physical Plan == -* Filter (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) = pale ) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#19, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#21, 10, true, false, true) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Condition : (isnotnull(c_customer_sk#26) AND isnotnull(c_birth_country#29)) - -(27) CometProject -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29], [c_customer_sk#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#31, c_birth_country#29] - -(28) CometBroadcastExchange -Input [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(31) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_state#32, ca_zip#33, ca_country#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true))) - -(34) CometProject -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] - -(35) CometColumnarToRow [codegen id : 1] -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] - -(36) BroadcastExchange -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#29, s_zip#16] -Right keys [2]: [upper(ca_country#34), ca_zip#36] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#37] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(40) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [partial_sum(netpaid#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(44) CometColumnarExchange -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [sum(netpaid#40)] -Aggregate Attributes [1]: [sum(netpaid#40)#45] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] -Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (77) -+- * CometColumnarToRow (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (56) - : : : +- CometBroadcastHashJoin (55) - : : : :- CometProject (53) - : : : : +- CometSortMergeJoin (52) - : : : : :- CometSort (49) - : : : : : +- ReusedExchange (48) - : : : : +- CometSort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (54) - : : +- CometBroadcastExchange (60) - : : +- CometProject (59) - : : +- CometFilter (58) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (57) - : +- ReusedExchange (63) - +- ReusedExchange (67) - - -(48) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] - -(49) CometSort -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53], [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST] - -(50) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#54, sr_ticket_number#55] - -(51) CometSort -Input [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [sr_item_sk#54, sr_ticket_number#55], [sr_ticket_number#55 ASC NULLS FIRST, sr_item_sk#54 ASC NULLS FIRST] - -(52) CometSortMergeJoin -Left output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Right output [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_ticket_number#52, ss_item_sk#49], [sr_ticket_number#55, sr_item_sk#54], Inner - -(53) CometProject -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53], [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] - -(54) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] - -(55) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Right output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_store_sk#51], [s_store_sk#56], Inner, BuildRight - -(56) CometProject -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16], [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Condition : isnotnull(i_item_sk#58) - -(59) CometProject -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [i_item_sk#58, i_current_price#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#60, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#61, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#62, 10, true, false, true) AS i_units#25, i_manager_id#63] - -(60) CometBroadcastExchange -Input [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(61) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_item_sk#49], [i_item_sk#58], Inner, BuildRight - -(62) CometProject -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(63) ReusedExchange [Reuses operator id: 28] -Output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] - -(64) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Right output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight - -(65) CometProject -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(66) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(67) ReusedExchange [Reuses operator id: 36] -Output [3]: [ca_state#35, ca_zip#36, ca_country#66] - -(68) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#65, s_zip#16] -Right keys [2]: [upper(ca_country#66), ca_zip#36] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 2] -Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] - -(70) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#67] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(71) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(72) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(73) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#69] - -(74) HashAggregate [codegen id : 3] -Input [1]: [netpaid#69] -Keys: [] -Functions [1]: [partial_avg(netpaid#69)] -Aggregate Attributes [2]: [sum#70, count#71] -Results [2]: [sum#72, count#73] - -(75) CometColumnarExchange -Input [2]: [sum#72, count#73] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#72, count#73] - -(77) HashAggregate [codegen id : 4] -Input [2]: [sum#72, count#73] -Keys: [] -Functions [1]: [avg(netpaid#69)] -Aggregate Attributes [1]: [avg(netpaid#69)#74] -Results [1]: [(0.05 * avg(netpaid#69)#74) AS (0.05 * avg(netpaid))#75] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 440f477d0f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,94 +0,0 @@ -WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #6 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] 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/q24a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/explain.txt deleted file mode 100644 index c6e3a30a1f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/explain.txt +++ /dev/null @@ -1,439 +0,0 @@ -== Physical Plan == -* Filter (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) = pale ) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#19, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#21, 10, true, false, true) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Condition : (isnotnull(c_customer_sk#26) AND isnotnull(c_birth_country#29)) - -(27) CometProject -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29], [c_customer_sk#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#31, c_birth_country#29] - -(28) CometBroadcastExchange -Input [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(31) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_state#32, ca_zip#33, ca_country#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true))) - -(34) CometProject -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] - -(35) CometColumnarToRow [codegen id : 1] -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] - -(36) BroadcastExchange -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#29, s_zip#16] -Right keys [2]: [upper(ca_country#34), ca_zip#36] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#37] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(40) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [partial_sum(netpaid#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(44) CometColumnarExchange -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [sum(netpaid#40)] -Aggregate Attributes [1]: [sum(netpaid#40)#45] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] -Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (77) -+- * CometColumnarToRow (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (56) - : : : +- CometBroadcastHashJoin (55) - : : : :- CometProject (53) - : : : : +- CometSortMergeJoin (52) - : : : : :- CometSort (49) - : : : : : +- ReusedExchange (48) - : : : : +- CometSort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (54) - : : +- CometBroadcastExchange (60) - : : +- CometProject (59) - : : +- CometFilter (58) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (57) - : +- ReusedExchange (63) - +- ReusedExchange (67) - - -(48) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] - -(49) CometSort -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53], [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST] - -(50) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#54, sr_ticket_number#55] - -(51) CometSort -Input [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [sr_item_sk#54, sr_ticket_number#55], [sr_ticket_number#55 ASC NULLS FIRST, sr_item_sk#54 ASC NULLS FIRST] - -(52) CometSortMergeJoin -Left output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Right output [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_ticket_number#52, ss_item_sk#49], [sr_ticket_number#55, sr_item_sk#54], Inner - -(53) CometProject -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53], [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] - -(54) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] - -(55) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Right output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_store_sk#51], [s_store_sk#56], Inner, BuildRight - -(56) CometProject -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16], [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Condition : isnotnull(i_item_sk#58) - -(59) CometProject -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [i_item_sk#58, i_current_price#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#60, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#61, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#62, 10, true, false, true) AS i_units#25, i_manager_id#63] - -(60) CometBroadcastExchange -Input [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(61) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_item_sk#49], [i_item_sk#58], Inner, BuildRight - -(62) CometProject -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(63) ReusedExchange [Reuses operator id: 28] -Output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] - -(64) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Right output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight - -(65) CometProject -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(66) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(67) ReusedExchange [Reuses operator id: 36] -Output [3]: [ca_state#35, ca_zip#36, ca_country#66] - -(68) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#65, s_zip#16] -Right keys [2]: [upper(ca_country#66), ca_zip#36] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 2] -Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] - -(70) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#67] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(71) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(72) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(73) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#69] - -(74) HashAggregate [codegen id : 3] -Input [1]: [netpaid#69] -Keys: [] -Functions [1]: [partial_avg(netpaid#69)] -Aggregate Attributes [2]: [sum#70, count#71] -Results [2]: [sum#72, count#73] - -(75) CometColumnarExchange -Input [2]: [sum#72, count#73] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#72, count#73] - -(77) HashAggregate [codegen id : 4] -Input [2]: [sum#72, count#73] -Keys: [] -Functions [1]: [avg(netpaid#69)] -Aggregate Attributes [1]: [avg(netpaid#69)#74] -Results [1]: [(0.05 * avg(netpaid#69)#74) AS (0.05 * avg(netpaid))#75] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/extended.txt deleted file mode 100644 index 8777e81a37..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/extended.txt +++ /dev/null @@ -1,96 +0,0 @@ -Filter -: +- Subquery -: +- HashAggregate -: +- CometColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- HashAggregate -: +- CometColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- Project -: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometColumnarToRow -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometSortMergeJoin -: : : : : :- CometSort -: : : : : : +- CometExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : +- CometSort -: : : : : +- CometExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -: +- BroadcastExchange -: +- CometColumnarToRow -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 70 out of 86 eligible operators (81%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 440f477d0f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a/simplified.txt +++ /dev/null @@ -1,94 +0,0 @@ -WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #6 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_datafusion/explain.txt deleted file mode 100644 index 65c5d9d521..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_datafusion/explain.txt +++ /dev/null @@ -1,439 +0,0 @@ -== Physical Plan == -* Filter (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometNativeScan parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometNativeScan parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.customer_address (32) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometNativeScan parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) = chiffon ) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#19, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#21, 10, true, false, true) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Condition : (isnotnull(c_customer_sk#26) AND isnotnull(c_birth_country#29)) - -(27) CometProject -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29], [c_customer_sk#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#31, c_birth_country#29] - -(28) CometBroadcastExchange -Input [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(31) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(32) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_state#32, ca_zip#33, ca_country#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true))) - -(34) CometProject -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] - -(35) CometColumnarToRow [codegen id : 1] -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] - -(36) BroadcastExchange -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#29, s_zip#16] -Right keys [2]: [upper(ca_country#34), ca_zip#36] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#37] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(40) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [partial_sum(netpaid#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(44) CometColumnarExchange -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [sum(netpaid#40)] -Aggregate Attributes [1]: [sum(netpaid#40)#45] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] -Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (77) -+- * CometColumnarToRow (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (56) - : : : +- CometBroadcastHashJoin (55) - : : : :- CometProject (53) - : : : : +- CometSortMergeJoin (52) - : : : : :- CometSort (49) - : : : : : +- ReusedExchange (48) - : : : : +- CometSort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (54) - : : +- CometBroadcastExchange (60) - : : +- CometProject (59) - : : +- CometFilter (58) - : : +- CometNativeScan parquet spark_catalog.default.item (57) - : +- ReusedExchange (63) - +- ReusedExchange (67) - - -(48) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] - -(49) CometSort -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53], [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST] - -(50) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#54, sr_ticket_number#55] - -(51) CometSort -Input [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [sr_item_sk#54, sr_ticket_number#55], [sr_ticket_number#55 ASC NULLS FIRST, sr_item_sk#54 ASC NULLS FIRST] - -(52) CometSortMergeJoin -Left output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Right output [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_ticket_number#52, ss_item_sk#49], [sr_ticket_number#55, sr_item_sk#54], Inner - -(53) CometProject -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53], [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] - -(54) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] - -(55) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Right output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_store_sk#51], [s_store_sk#56], Inner, BuildRight - -(56) CometProject -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16], [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] - -(57) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Condition : isnotnull(i_item_sk#58) - -(59) CometProject -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [i_item_sk#58, i_current_price#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#60, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#61, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#62, 10, true, false, true) AS i_units#25, i_manager_id#63] - -(60) CometBroadcastExchange -Input [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(61) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_item_sk#49], [i_item_sk#58], Inner, BuildRight - -(62) CometProject -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(63) ReusedExchange [Reuses operator id: 28] -Output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] - -(64) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Right output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight - -(65) CometProject -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(66) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(67) ReusedExchange [Reuses operator id: 36] -Output [3]: [ca_state#35, ca_zip#36, ca_country#66] - -(68) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#65, s_zip#16] -Right keys [2]: [upper(ca_country#66), ca_zip#36] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 2] -Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] - -(70) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#67] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(71) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(72) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(73) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#69] - -(74) HashAggregate [codegen id : 3] -Input [1]: [netpaid#69] -Keys: [] -Functions [1]: [partial_avg(netpaid#69)] -Aggregate Attributes [2]: [sum#70, count#71] -Results [2]: [sum#72, count#73] - -(75) CometColumnarExchange -Input [2]: [sum#72, count#73] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#72, count#73] - -(77) HashAggregate [codegen id : 4] -Input [2]: [sum#72, count#73] -Keys: [] -Functions [1]: [avg(netpaid#69)] -Aggregate Attributes [1]: [avg(netpaid#69)#74] -Results [1]: [(0.05 * avg(netpaid#69)#74) AS (0.05 * avg(netpaid))#75] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_datafusion/simplified.txt deleted file mode 100644 index af8d5ee7aa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_datafusion/simplified.txt +++ /dev/null @@ -1,94 +0,0 @@ -WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #6 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometNativeScan 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.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/explain.txt deleted file mode 100644 index a77a3f88b3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,439 +0,0 @@ -== Physical Plan == -* Filter (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) = chiffon ) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#19, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#21, 10, true, false, true) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Condition : (isnotnull(c_customer_sk#26) AND isnotnull(c_birth_country#29)) - -(27) CometProject -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29], [c_customer_sk#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#31, c_birth_country#29] - -(28) CometBroadcastExchange -Input [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(31) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_state#32, ca_zip#33, ca_country#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true))) - -(34) CometProject -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] - -(35) CometColumnarToRow [codegen id : 1] -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] - -(36) BroadcastExchange -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#29, s_zip#16] -Right keys [2]: [upper(ca_country#34), ca_zip#36] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#37] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(40) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [partial_sum(netpaid#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(44) CometColumnarExchange -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [sum(netpaid#40)] -Aggregate Attributes [1]: [sum(netpaid#40)#45] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] -Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (77) -+- * CometColumnarToRow (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (56) - : : : +- CometBroadcastHashJoin (55) - : : : :- CometProject (53) - : : : : +- CometSortMergeJoin (52) - : : : : :- CometSort (49) - : : : : : +- ReusedExchange (48) - : : : : +- CometSort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (54) - : : +- CometBroadcastExchange (60) - : : +- CometProject (59) - : : +- CometFilter (58) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (57) - : +- ReusedExchange (63) - +- ReusedExchange (67) - - -(48) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] - -(49) CometSort -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53], [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST] - -(50) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#54, sr_ticket_number#55] - -(51) CometSort -Input [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [sr_item_sk#54, sr_ticket_number#55], [sr_ticket_number#55 ASC NULLS FIRST, sr_item_sk#54 ASC NULLS FIRST] - -(52) CometSortMergeJoin -Left output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Right output [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_ticket_number#52, ss_item_sk#49], [sr_ticket_number#55, sr_item_sk#54], Inner - -(53) CometProject -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53], [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] - -(54) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] - -(55) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Right output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_store_sk#51], [s_store_sk#56], Inner, BuildRight - -(56) CometProject -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16], [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Condition : isnotnull(i_item_sk#58) - -(59) CometProject -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [i_item_sk#58, i_current_price#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#60, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#61, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#62, 10, true, false, true) AS i_units#25, i_manager_id#63] - -(60) CometBroadcastExchange -Input [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(61) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_item_sk#49], [i_item_sk#58], Inner, BuildRight - -(62) CometProject -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(63) ReusedExchange [Reuses operator id: 28] -Output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] - -(64) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Right output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight - -(65) CometProject -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(66) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(67) ReusedExchange [Reuses operator id: 36] -Output [3]: [ca_state#35, ca_zip#36, ca_country#66] - -(68) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#65, s_zip#16] -Right keys [2]: [upper(ca_country#66), ca_zip#36] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 2] -Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] - -(70) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#67] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(71) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(72) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(73) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#69] - -(74) HashAggregate [codegen id : 3] -Input [1]: [netpaid#69] -Keys: [] -Functions [1]: [partial_avg(netpaid#69)] -Aggregate Attributes [2]: [sum#70, count#71] -Results [2]: [sum#72, count#73] - -(75) CometColumnarExchange -Input [2]: [sum#72, count#73] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#72, count#73] - -(77) HashAggregate [codegen id : 4] -Input [2]: [sum#72, count#73] -Keys: [] -Functions [1]: [avg(netpaid#69)] -Aggregate Attributes [1]: [avg(netpaid#69)#74] -Results [1]: [(0.05 * avg(netpaid#69)#74) AS (0.05 * avg(netpaid))#75] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/simplified.txt deleted file mode 100644 index 440f477d0f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,94 +0,0 @@ -WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #6 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/explain.txt deleted file mode 100644 index a77a3f88b3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/explain.txt +++ /dev/null @@ -1,439 +0,0 @@ -== Physical Plan == -* Filter (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) = chiffon ) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#19, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#21, 10, true, false, true) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Condition : (isnotnull(c_customer_sk#26) AND isnotnull(c_birth_country#29)) - -(27) CometProject -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29], [c_customer_sk#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#31, c_birth_country#29] - -(28) CometBroadcastExchange -Input [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(31) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_state#32, ca_zip#33, ca_country#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true))) - -(34) CometProject -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] - -(35) CometColumnarToRow [codegen id : 1] -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] - -(36) BroadcastExchange -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#29, s_zip#16] -Right keys [2]: [upper(ca_country#34), ca_zip#36] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#37] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(40) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [partial_sum(netpaid#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(44) CometColumnarExchange -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [sum(netpaid#40)] -Aggregate Attributes [1]: [sum(netpaid#40)#45] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] -Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (77) -+- * CometColumnarToRow (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (56) - : : : +- CometBroadcastHashJoin (55) - : : : :- CometProject (53) - : : : : +- CometSortMergeJoin (52) - : : : : :- CometSort (49) - : : : : : +- ReusedExchange (48) - : : : : +- CometSort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (54) - : : +- CometBroadcastExchange (60) - : : +- CometProject (59) - : : +- CometFilter (58) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (57) - : +- ReusedExchange (63) - +- ReusedExchange (67) - - -(48) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] - -(49) CometSort -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53], [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST] - -(50) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#54, sr_ticket_number#55] - -(51) CometSort -Input [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [sr_item_sk#54, sr_ticket_number#55], [sr_ticket_number#55 ASC NULLS FIRST, sr_item_sk#54 ASC NULLS FIRST] - -(52) CometSortMergeJoin -Left output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Right output [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_ticket_number#52, ss_item_sk#49], [sr_ticket_number#55, sr_item_sk#54], Inner - -(53) CometProject -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53], [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] - -(54) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] - -(55) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Right output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_store_sk#51], [s_store_sk#56], Inner, BuildRight - -(56) CometProject -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16], [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Condition : isnotnull(i_item_sk#58) - -(59) CometProject -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [i_item_sk#58, i_current_price#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#60, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#61, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#62, 10, true, false, true) AS i_units#25, i_manager_id#63] - -(60) CometBroadcastExchange -Input [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(61) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_item_sk#49], [i_item_sk#58], Inner, BuildRight - -(62) CometProject -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(63) ReusedExchange [Reuses operator id: 28] -Output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] - -(64) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Right output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight - -(65) CometProject -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(66) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(67) ReusedExchange [Reuses operator id: 36] -Output [3]: [ca_state#35, ca_zip#36, ca_country#66] - -(68) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#65, s_zip#16] -Right keys [2]: [upper(ca_country#66), ca_zip#36] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 2] -Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] - -(70) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#67] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(71) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(72) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(73) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#69] - -(74) HashAggregate [codegen id : 3] -Input [1]: [netpaid#69] -Keys: [] -Functions [1]: [partial_avg(netpaid#69)] -Aggregate Attributes [2]: [sum#70, count#71] -Results [2]: [sum#72, count#73] - -(75) CometColumnarExchange -Input [2]: [sum#72, count#73] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#72, count#73] - -(77) HashAggregate [codegen id : 4] -Input [2]: [sum#72, count#73] -Keys: [] -Functions [1]: [avg(netpaid#69)] -Aggregate Attributes [1]: [avg(netpaid#69)#74] -Results [1]: [(0.05 * avg(netpaid#69)#74) AS (0.05 * avg(netpaid))#75] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/extended.txt deleted file mode 100644 index 8777e81a37..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/extended.txt +++ /dev/null @@ -1,96 +0,0 @@ -Filter -: +- Subquery -: +- HashAggregate -: +- CometColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- HashAggregate -: +- CometColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- Project -: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometColumnarToRow -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometSortMergeJoin -: : : : : :- CometSort -: : : : : : +- CometExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : +- CometSort -: : : : : +- CometExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -: +- BroadcastExchange -: +- CometColumnarToRow -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 70 out of 86 eligible operators (81%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 440f477d0f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b/simplified.txt +++ /dev/null @@ -1,94 +0,0 @@ -WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #6 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/explain.txt deleted file mode 100644 index 15bdd29f61..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/explain.txt +++ /dev/null @@ -1,312 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (18) - : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet spark_catalog.default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * Filter (12) - : : : : : +- * ColumnarToRow (11) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (10) - : : : : +- ReusedExchange (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.store (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.item (32) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 8] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] - -(3) Filter [codegen id : 8] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(4) Scan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] - -(6) Filter [codegen id : 1] -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(7) BroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 8] -Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] -Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] - -(10) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] - -(12) Filter [codegen id : 2] -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(13) BroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] -Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] - -(16) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#18] - -(17) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#18] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 8] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#18] - -(19) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#19] - -(20) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [sr_returned_date_sk#12] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#19] - -(22) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#20] - -(23) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 8] -Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#20] - -(25) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#21, s_store_id#22, s_store_name#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(26) CometFilter -Input [3]: [s_store_sk#21, s_store_id#22, s_store_name#23] -Condition : isnotnull(s_store_sk#21) - -(27) CometProject -Input [3]: [s_store_sk#21, s_store_id#22, s_store_name#23] -Arguments: [s_store_sk#21, s_store_id#24, s_store_name#23], [s_store_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#22, 16, true, false, true) AS s_store_id#24, s_store_name#23] - -(28) CometColumnarToRow [codegen id : 6] -Input [3]: [s_store_sk#21, s_store_id#24, s_store_name#23] - -(29) BroadcastExchange -Input [3]: [s_store_sk#21, s_store_id#24, s_store_name#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(30) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#21] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_sk#21, s_store_id#24, s_store_name#23] - -(32) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] -Condition : isnotnull(i_item_sk#25) - -(34) CometProject -Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] -Arguments: [i_item_sk#25, i_item_id#28, i_item_desc#27], [i_item_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#26, 16, true, false, true) AS i_item_id#28, i_item_desc#27] - -(35) CometColumnarToRow [codegen id : 7] -Input [3]: [i_item_sk#25, i_item_id#28, i_item_desc#27] - -(36) BroadcastExchange -Input [3]: [i_item_sk#25, i_item_id#28, i_item_desc#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(37) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#25] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 8] -Output [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23, i_item_id#28, i_item_desc#27] -Input [9]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23, i_item_sk#25, i_item_id#28, i_item_desc#27] - -(39) HashAggregate [codegen id : 8] -Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23, i_item_id#28, i_item_desc#27] -Keys [4]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23] -Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#11)), partial_sum(UnscaledValue(cs_net_profit#16))] -Aggregate Attributes [3]: [sum#29, sum#30, sum#31] -Results [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum#32, sum#33, sum#34] - -(40) CometColumnarExchange -Input [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum#32, sum#33, sum#34] -Arguments: hashpartitioning(i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(41) CometColumnarToRow [codegen id : 9] -Input [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum#32, sum#33, sum#34] - -(42) HashAggregate [codegen id : 9] -Input [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum#32, sum#33, sum#34] -Keys [4]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23] -Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#35, sum(UnscaledValue(sr_net_loss#11))#36, sum(UnscaledValue(cs_net_profit#16))#37] -Results [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#35,17,2) AS store_sales_profit#38, MakeDecimal(sum(UnscaledValue(sr_net_loss#11))#36,17,2) AS store_returns_loss#39, MakeDecimal(sum(UnscaledValue(cs_net_profit#16))#37,17,2) AS catalog_sales_profit#40] - -(43) TakeOrderedAndProject -Input [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, store_sales_profit#38, store_returns_loss#39, catalog_sales_profit#40] -Arguments: 100, [i_item_id#28 ASC NULLS FIRST, i_item_desc#27 ASC NULLS FIRST, s_store_id#24 ASC NULLS FIRST, s_store_name#23 ASC NULLS FIRST], [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, store_sales_profit#38, store_returns_loss#39, catalog_sales_profit#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometNativeScan parquet spark_catalog.default.date_dim (44) - - -(44) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#18, d_year#41, d_moy#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [3]: [d_date_sk#18, d_year#41, d_moy#42] -Condition : ((((isnotnull(d_moy#42) AND isnotnull(d_year#41)) AND (d_moy#42 = 4)) AND (d_year#41 = 2001)) AND isnotnull(d_date_sk#18)) - -(46) CometProject -Input [3]: [d_date_sk#18, d_year#41, d_moy#42] -Arguments: [d_date_sk#18], [d_date_sk#18] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#18] - -(48) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometNativeScan parquet spark_catalog.default.date_dim (49) - - -(49) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#43, d_moy#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#19, d_year#43, d_moy#44] -Condition : (((((isnotnull(d_moy#44) AND isnotnull(d_year#43)) AND (d_moy#44 >= 4)) AND (d_moy#44 <= 10)) AND (d_year#43 = 2001)) AND isnotnull(d_date_sk#19)) - -(51) CometProject -Input [3]: [d_date_sk#19, d_year#43, d_moy#44] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(53) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/simplified.txt deleted file mode 100644 index 288561a740..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/simplified.txt +++ /dev/null @@ -1,79 +0,0 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] - WholeStageCodegen (9) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit)),store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - WholeStageCodegen (8) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] [sum,sum,sum,sum,sum,sum] - Project [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [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] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Filter [cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometNativeScan 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/q25.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/explain.txt deleted file mode 100644 index 6a9f9094d3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,319 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometProject (18) - : : : : +- CometBroadcastHashJoin (17) - : : : : :- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (8) - : : : : +- CometBroadcastExchange (16) - : : : : +- CometProject (15) - : : : : +- CometFilter (14) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : : : +- CometBroadcastExchange (22) - : : : +- CometProject (21) - : : : +- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - : : +- ReusedExchange (25) - : +- CometBroadcastExchange (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (28) - +- CometBroadcastExchange (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (34) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(5) CometBroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] - -(6) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(10) CometBroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] - -(11) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] -Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight - -(12) CometProject -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(15) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 4)) AND (d_moy#24 <= 10)) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(21) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#22] -Arguments: [sr_returned_date_sk#12], [d_date_sk#22], Inner, BuildRight - -(24) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#22] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] - -(25) ReusedExchange [Reuses operator id: 22] -Output [1]: [d_date_sk#25] - -(26) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#25] -Arguments: [cs_sold_date_sk#17], [d_date_sk#25], Inner, BuildRight - -(27) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#25] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(29) CometFilter -Input [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] -Condition : isnotnull(s_store_sk#26) - -(30) CometProject -Input [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] -Arguments: [s_store_sk#26, s_store_id#29, s_store_name#28], [s_store_sk#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#27, 16, true, false, true) AS s_store_id#29, s_store_name#28] - -(31) CometBroadcastExchange -Input [3]: [s_store_sk#26, s_store_id#29, s_store_name#28] -Arguments: [s_store_sk#26, s_store_id#29, s_store_name#28] - -(32) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] -Right output [3]: [s_store_sk#26, s_store_id#29, s_store_name#28] -Arguments: [ss_store_sk#3], [s_store_sk#26], Inner, BuildRight - -(33) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_sk#26, s_store_id#29, s_store_name#28] -Arguments: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28], [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28] - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] -Condition : isnotnull(i_item_sk#30) - -(36) CometProject -Input [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] -Arguments: [i_item_sk#30, i_item_id#33, i_item_desc#32], [i_item_sk#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#31, 16, true, false, true) AS i_item_id#33, i_item_desc#32] - -(37) CometBroadcastExchange -Input [3]: [i_item_sk#30, i_item_id#33, i_item_desc#32] -Arguments: [i_item_sk#30, i_item_id#33, i_item_desc#32] - -(38) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28] -Right output [3]: [i_item_sk#30, i_item_id#33, i_item_desc#32] -Arguments: [ss_item_sk#1], [i_item_sk#30], Inner, BuildRight - -(39) CometProject -Input [9]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_sk#30, i_item_id#33, i_item_desc#32] -Arguments: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_id#33, i_item_desc#32], [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_id#33, i_item_desc#32] - -(40) CometHashAggregate -Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_id#33, i_item_desc#32] -Keys [4]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28] -Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#11)), partial_sum(UnscaledValue(cs_net_profit#16))] - -(41) CometExchange -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, sum#34, sum#35, sum#36] -Arguments: hashpartitioning(i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(42) CometHashAggregate -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, sum#34, sum#35, sum#36] -Keys [4]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28] -Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] - -(43) CometTakeOrderedAndProject -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#33 ASC NULLS FIRST,i_item_desc#32 ASC NULLS FIRST,s_store_id#29 ASC NULLS FIRST,s_store_name#28 ASC NULLS FIRST], output=[i_item_id#33,i_item_desc#32,s_store_id#29,s_store_name#28,store_sales_profit#37,store_returns_loss#38,catalog_sales_profit#39]), [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39], 100, 0, [i_item_id#33 ASC NULLS FIRST, i_item_desc#32 ASC NULLS FIRST, s_store_id#29 ASC NULLS FIRST, s_store_name#28 ASC NULLS FIRST], [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] - -(44) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(47) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(49) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) - - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 4)) AND (d_moy#24 <= 10)) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(52) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(53) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(54) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/simplified.txt deleted file mode 100644 index dcad304452..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,63 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] - CometHashAggregate [sum,sum,sum] [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit))] - CometExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - CometHashAggregate [ss_net_profit,sr_net_loss,cs_net_profit] [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] - 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,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_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 [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 [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_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 [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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] 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 [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #8 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan [native_iceberg_compat] 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/q25/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/explain.txt deleted file mode 100644 index 6a9f9094d3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/explain.txt +++ /dev/null @@ -1,319 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometProject (18) - : : : : +- CometBroadcastHashJoin (17) - : : : : :- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (8) - : : : : +- CometBroadcastExchange (16) - : : : : +- CometProject (15) - : : : : +- CometFilter (14) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : : : +- CometBroadcastExchange (22) - : : : +- CometProject (21) - : : : +- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - : : +- ReusedExchange (25) - : +- CometBroadcastExchange (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (28) - +- CometBroadcastExchange (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (34) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(5) CometBroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] - -(6) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(10) CometBroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] - -(11) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] -Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight - -(12) CometProject -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(15) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 4)) AND (d_moy#24 <= 10)) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(21) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#22] -Arguments: [sr_returned_date_sk#12], [d_date_sk#22], Inner, BuildRight - -(24) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#22] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] - -(25) ReusedExchange [Reuses operator id: 22] -Output [1]: [d_date_sk#25] - -(26) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#25] -Arguments: [cs_sold_date_sk#17], [d_date_sk#25], Inner, BuildRight - -(27) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#25] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(29) CometFilter -Input [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] -Condition : isnotnull(s_store_sk#26) - -(30) CometProject -Input [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] -Arguments: [s_store_sk#26, s_store_id#29, s_store_name#28], [s_store_sk#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#27, 16, true, false, true) AS s_store_id#29, s_store_name#28] - -(31) CometBroadcastExchange -Input [3]: [s_store_sk#26, s_store_id#29, s_store_name#28] -Arguments: [s_store_sk#26, s_store_id#29, s_store_name#28] - -(32) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] -Right output [3]: [s_store_sk#26, s_store_id#29, s_store_name#28] -Arguments: [ss_store_sk#3], [s_store_sk#26], Inner, BuildRight - -(33) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_sk#26, s_store_id#29, s_store_name#28] -Arguments: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28], [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28] - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] -Condition : isnotnull(i_item_sk#30) - -(36) CometProject -Input [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] -Arguments: [i_item_sk#30, i_item_id#33, i_item_desc#32], [i_item_sk#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#31, 16, true, false, true) AS i_item_id#33, i_item_desc#32] - -(37) CometBroadcastExchange -Input [3]: [i_item_sk#30, i_item_id#33, i_item_desc#32] -Arguments: [i_item_sk#30, i_item_id#33, i_item_desc#32] - -(38) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28] -Right output [3]: [i_item_sk#30, i_item_id#33, i_item_desc#32] -Arguments: [ss_item_sk#1], [i_item_sk#30], Inner, BuildRight - -(39) CometProject -Input [9]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_sk#30, i_item_id#33, i_item_desc#32] -Arguments: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_id#33, i_item_desc#32], [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_id#33, i_item_desc#32] - -(40) CometHashAggregate -Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_id#33, i_item_desc#32] -Keys [4]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28] -Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#11)), partial_sum(UnscaledValue(cs_net_profit#16))] - -(41) CometExchange -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, sum#34, sum#35, sum#36] -Arguments: hashpartitioning(i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(42) CometHashAggregate -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, sum#34, sum#35, sum#36] -Keys [4]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28] -Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] - -(43) CometTakeOrderedAndProject -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#33 ASC NULLS FIRST,i_item_desc#32 ASC NULLS FIRST,s_store_id#29 ASC NULLS FIRST,s_store_name#28 ASC NULLS FIRST], output=[i_item_id#33,i_item_desc#32,s_store_id#29,s_store_name#28,store_sales_profit#37,store_returns_loss#38,catalog_sales_profit#39]), [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39], 100, 0, [i_item_id#33 ASC NULLS FIRST, i_item_desc#32 ASC NULLS FIRST, s_store_id#29 ASC NULLS FIRST, s_store_name#28 ASC NULLS FIRST], [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] - -(44) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(47) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(49) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) - - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 4)) AND (d_moy#24 <= 10)) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(52) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(53) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(54) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/extended.txt deleted file mode 100644 index 4e1a4f3ec0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/extended.txt +++ /dev/null @@ -1,62 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index dcad304452..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25/simplified.txt +++ /dev/null @@ -1,63 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] - CometHashAggregate [sum,sum,sum] [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit))] - CometExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - CometHashAggregate [ss_net_profit,sr_net_loss,cs_net_profit] [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] - 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,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_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 [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 [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_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 [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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] 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 [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #8 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/explain.txt deleted file mode 100644 index fb4c487ea9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- * CometColumnarToRow (30) - +- CometColumnarExchange (29) - +- * HashAggregate (28) - +- * Project (27) - +- * BroadcastHashJoin Inner BuildRight (26) - :- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.item (14) - +- BroadcastExchange (25) - +- * CometColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometNativeScan parquet spark_catalog.default.promotion (21) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] - -(3) Filter [codegen id : 5] -Input [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Condition : ((isnotnull(cs_bill_cdemo_sk#1) AND isnotnull(cs_item_sk#2)) AND isnotnull(cs_promo_sk#3)) - -(4) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = S)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#10)) - -(6) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [cd_demo_sk#10] - -(8) BroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_bill_cdemo_sk#1] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 5] -Output [7]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Input [9]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, cd_demo_sk#10] - -(11) ReusedExchange [Reuses operator id: 37] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [6]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] -Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, d_date_sk#14] - -(14) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#15, i_item_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [i_item_sk#15, i_item_id#16] -Condition : isnotnull(i_item_sk#15) - -(16) CometProject -Input [2]: [i_item_sk#15, i_item_id#16] -Arguments: [i_item_sk#15, i_item_id#17], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#16, 16, true, false, true) AS i_item_id#17] - -(17) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#15, i_item_id#17] - -(18) BroadcastExchange -Input [2]: [i_item_sk#15, i_item_id#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [6]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] -Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_sk#15, i_item_id#17] - -(21) CometNativeScan parquet spark_catalog.default.promotion -Output [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_email#19, 1, true, false, true) = N) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_event#20, 1, true, false, true) = N)) AND isnotnull(p_promo_sk#18)) - -(23) CometProject -Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] -Arguments: [p_promo_sk#18], [p_promo_sk#18] - -(24) CometColumnarToRow [codegen id : 4] -Input [1]: [p_promo_sk#18] - -(25) BroadcastExchange -Input [1]: [p_promo_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_promo_sk#3] -Right keys [1]: [p_promo_sk#18] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 5] -Output [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] -Input [7]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17, p_promo_sk#18] - -(28) HashAggregate [codegen id : 5] -Input [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] -Keys [1]: [i_item_id#17] -Functions [4]: [partial_avg(cs_quantity#4), partial_avg(UnscaledValue(cs_list_price#5)), partial_avg(UnscaledValue(cs_coupon_amt#7)), partial_avg(UnscaledValue(cs_sales_price#6))] -Aggregate Attributes [8]: [sum#21, count#22, sum#23, count#24, sum#25, count#26, sum#27, count#28] -Results [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] - -(29) CometColumnarExchange -Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] -Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 6] -Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] - -(31) HashAggregate [codegen id : 6] -Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] -Keys [1]: [i_item_id#17] -Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(UnscaledValue(cs_coupon_amt#7)), avg(UnscaledValue(cs_sales_price#6))] -Aggregate Attributes [4]: [avg(cs_quantity#4)#37, avg(UnscaledValue(cs_list_price#5))#38, avg(UnscaledValue(cs_coupon_amt#7))#39, avg(UnscaledValue(cs_sales_price#6))#40] -Results [5]: [i_item_id#17, avg(cs_quantity#4)#37 AS agg1#41, cast((avg(UnscaledValue(cs_list_price#5))#38 / 100.0) as decimal(11,6)) AS agg2#42, cast((avg(UnscaledValue(cs_coupon_amt#7))#39 / 100.0) as decimal(11,6)) AS agg3#43, cast((avg(UnscaledValue(cs_sales_price#6))#40 / 100.0) as decimal(11,6)) AS agg4#44] - -(32) TakeOrderedAndProject -Input [5]: [i_item_id#17, agg1#41, agg2#42, agg3#43, agg4#44] -Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#41, agg2#42, agg3#43, agg4#44] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometNativeScan parquet spark_catalog.default.date_dim (33) - - -(33) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [d_date_sk#14, d_year#45] -Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2000)) AND isnotnull(d_date_sk#14)) - -(35) CometProject -Input [2]: [d_date_sk#14, d_year#45] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(37) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/simplified.txt deleted file mode 100644 index 7eb9c67611..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) - HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometNativeScan 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/q26.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/explain.txt deleted file mode 100644 index 3d70460689..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,212 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (31) -+- CometTakeOrderedAndProject (30) - +- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Condition : ((isnotnull(cs_bill_cdemo_sk#1) AND isnotnull(cs_item_sk#2)) AND isnotnull(cs_promo_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = S)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [cs_bill_cdemo_sk#1], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, cd_demo_sk#10] -Arguments: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8], [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [cs_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, d_date_sk#14] -Arguments: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7], [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#16, i_item_id#17] -Condition : isnotnull(i_item_sk#16) - -(17) CometProject -Input [2]: [i_item_sk#16, i_item_id#17] -Arguments: [i_item_sk#16, i_item_id#18], [i_item_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#17, 16, true, false, true) AS i_item_id#18] - -(18) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_id#18] -Arguments: [i_item_sk#16, i_item_id#18] - -(19) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] -Right output [2]: [i_item_sk#16, i_item_id#18] -Arguments: [cs_item_sk#2], [i_item_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_sk#16, i_item_id#18] -Arguments: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18], [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_email#20, 1, true, false, true) = N) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_event#21, 1, true, false, true) = N)) AND isnotnull(p_promo_sk#19)) - -(23) CometProject -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Arguments: [p_promo_sk#19], [p_promo_sk#19] - -(24) CometBroadcastExchange -Input [1]: [p_promo_sk#19] -Arguments: [p_promo_sk#19] - -(25) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] -Right output [1]: [p_promo_sk#19] -Arguments: [cs_promo_sk#3], [p_promo_sk#19], Inner, BuildRight - -(26) CometProject -Input [7]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18, p_promo_sk#19] -Arguments: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] - -(27) CometHashAggregate -Input [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [4]: [partial_avg(cs_quantity#4), partial_avg(UnscaledValue(cs_list_price#5)), partial_avg(UnscaledValue(cs_coupon_amt#7)), partial_avg(UnscaledValue(cs_sales_price#6))] - -(28) CometExchange -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Keys [1]: [i_item_id#18] -Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(UnscaledValue(cs_coupon_amt#7)), avg(UnscaledValue(cs_sales_price#6))] - -(30) CometTakeOrderedAndProject -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#18 ASC NULLS FIRST], output=[i_item_id#18,agg1#30,agg2#31,agg3#32,agg4#33]), [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33], 100, 0, [i_item_id#18 ASC NULLS FIRST], [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(34) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(36) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/simplified.txt deleted file mode 100644 index 0b56a47547..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,agg1,agg2,agg3,agg4,avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price))] - CometExchange [i_item_id] #1 - CometHashAggregate [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [i_item_id,sum,count,sum,count,sum,count,sum,count] - CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - 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,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_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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [p_promo_sk] #6 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometScan [native_iceberg_compat] 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/q26/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/explain.txt deleted file mode 100644 index 3d70460689..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/explain.txt +++ /dev/null @@ -1,212 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (31) -+- CometTakeOrderedAndProject (30) - +- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Condition : ((isnotnull(cs_bill_cdemo_sk#1) AND isnotnull(cs_item_sk#2)) AND isnotnull(cs_promo_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = S)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [cs_bill_cdemo_sk#1], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, cd_demo_sk#10] -Arguments: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8], [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [cs_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, d_date_sk#14] -Arguments: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7], [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#16, i_item_id#17] -Condition : isnotnull(i_item_sk#16) - -(17) CometProject -Input [2]: [i_item_sk#16, i_item_id#17] -Arguments: [i_item_sk#16, i_item_id#18], [i_item_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#17, 16, true, false, true) AS i_item_id#18] - -(18) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_id#18] -Arguments: [i_item_sk#16, i_item_id#18] - -(19) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] -Right output [2]: [i_item_sk#16, i_item_id#18] -Arguments: [cs_item_sk#2], [i_item_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_sk#16, i_item_id#18] -Arguments: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18], [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_email#20, 1, true, false, true) = N) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_event#21, 1, true, false, true) = N)) AND isnotnull(p_promo_sk#19)) - -(23) CometProject -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Arguments: [p_promo_sk#19], [p_promo_sk#19] - -(24) CometBroadcastExchange -Input [1]: [p_promo_sk#19] -Arguments: [p_promo_sk#19] - -(25) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] -Right output [1]: [p_promo_sk#19] -Arguments: [cs_promo_sk#3], [p_promo_sk#19], Inner, BuildRight - -(26) CometProject -Input [7]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18, p_promo_sk#19] -Arguments: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] - -(27) CometHashAggregate -Input [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [4]: [partial_avg(cs_quantity#4), partial_avg(UnscaledValue(cs_list_price#5)), partial_avg(UnscaledValue(cs_coupon_amt#7)), partial_avg(UnscaledValue(cs_sales_price#6))] - -(28) CometExchange -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Keys [1]: [i_item_id#18] -Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(UnscaledValue(cs_coupon_amt#7)), avg(UnscaledValue(cs_sales_price#6))] - -(30) CometTakeOrderedAndProject -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#18 ASC NULLS FIRST], output=[i_item_id#18,agg1#30,agg2#31,agg3#32,agg4#33]), [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33], 100, 0, [i_item_id#18 ASC NULLS FIRST], [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(34) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(36) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/extended.txt deleted file mode 100644 index dbea7afe34..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/extended.txt +++ /dev/null @@ -1,39 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 0b56a47547..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,agg1,agg2,agg3,agg4,avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price))] - CometExchange [i_item_id] #1 - CometHashAggregate [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [i_item_id,sum,count,sum,count,sum,count,sum,count] - CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - 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,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_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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [p_promo_sk] #6 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/explain.txt deleted file mode 100644 index b9e3e82684..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (33) -+- * HashAggregate (32) - +- * CometColumnarToRow (31) - +- CometColumnarExchange (30) - +- * HashAggregate (29) - +- * Expand (28) - +- * Project (27) - +- * BroadcastHashJoin Inner BuildRight (26) - :- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.store (14) - +- BroadcastExchange (25) - +- * CometColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometNativeScan parquet spark_catalog.default.item (21) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(3) Filter [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) - -(4) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = S)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#10)) - -(6) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [cd_demo_sk#10] - -(8) BroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 5] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] - -(11) ReusedExchange [Reuses operator id: 38] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#15, s_state#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#15, s_state#16] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#16, 2, true, false, true) = TN) AND isnotnull(s_store_sk#15)) - -(16) CometProject -Input [2]: [s_store_sk#15, s_state#16] -Arguments: [s_store_sk#15, s_state#17], [s_store_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#16, 2, true, false, true) AS s_state#17] - -(17) CometColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#15, s_state#17] - -(18) BroadcastExchange -Input [2]: [s_store_sk#15, s_state#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15, s_state#17] - -(21) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#18, i_item_id#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_item_sk#18, i_item_id#19] -Condition : isnotnull(i_item_sk#18) - -(23) CometProject -Input [2]: [i_item_sk#18, i_item_id#19] -Arguments: [i_item_sk#18, i_item_id#20], [i_item_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#19, 16, true, false, true) AS i_item_id#20] - -(24) CometColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#18, i_item_id#20] - -(25) BroadcastExchange -Input [2]: [i_item_sk#18, i_item_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#18] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 5] -Output [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#17] -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17, i_item_sk#18, i_item_id#20] - -(28) Expand [codegen id : 5] -Input [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#17] -Arguments: [[ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#17, 0], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, null, 1], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, null, null, 3]], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#22, spark_grouping_id#23] - -(29) HashAggregate [codegen id : 5] -Input [7]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#22, spark_grouping_id#23] -Keys [3]: [i_item_id#21, s_state#22, spark_grouping_id#23] -Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [8]: [sum#24, count#25, sum#26, count#27, sum#28, count#29, sum#30, count#31] -Results [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] - -(30) CometColumnarExchange -Input [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] -Arguments: hashpartitioning(i_item_id#21, s_state#22, spark_grouping_id#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(31) CometColumnarToRow [codegen id : 6] -Input [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] - -(32) HashAggregate [codegen id : 6] -Input [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] -Keys [3]: [i_item_id#21, s_state#22, spark_grouping_id#23] -Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [4]: [avg(ss_quantity#4)#40, avg(UnscaledValue(ss_list_price#5))#41, avg(UnscaledValue(ss_coupon_amt#7))#42, avg(UnscaledValue(ss_sales_price#6))#43] -Results [7]: [i_item_id#21, s_state#22, cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint) AS g_state#44, avg(ss_quantity#4)#40 AS agg1#45, cast((avg(UnscaledValue(ss_list_price#5))#41 / 100.0) as decimal(11,6)) AS agg2#46, cast((avg(UnscaledValue(ss_coupon_amt#7))#42 / 100.0) as decimal(11,6)) AS agg3#47, cast((avg(UnscaledValue(ss_sales_price#6))#43 / 100.0) as decimal(11,6)) AS agg4#48] - -(33) TakeOrderedAndProject -Input [7]: [i_item_id#21, s_state#22, g_state#44, agg1#45, agg2#46, agg3#47, agg4#48] -Arguments: 100, [i_item_id#21 ASC NULLS FIRST, s_state#22 ASC NULLS FIRST], [i_item_id#21, s_state#22, g_state#44, agg1#45, agg2#46, agg3#47, agg4#48] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometNativeScan parquet spark_catalog.default.date_dim (34) - - -(34) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [d_date_sk#14, d_year#49] -Condition : ((isnotnull(d_year#49) AND (d_year#49 = 2002)) AND isnotnull(d_date_sk#14)) - -(36) CometProject -Input [2]: [d_date_sk#14, d_year#49] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(37) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(38) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/simplified.txt deleted file mode 100644 index dcb4953707..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/simplified.txt +++ /dev/null @@ -1,55 +0,0 @@ -TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) - HashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,s_state,spark_grouping_id] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Expand [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Filter [ss_cdemo_sk,ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan 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/q27.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/explain.txt deleted file mode 100644 index eb158b2889..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (32) -+- CometTakeOrderedAndProject (31) - +- CometHashAggregate (30) - +- CometExchange (29) - +- CometHashAggregate (28) - +- CometExpand (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = S)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#16, s_state#17] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) = TN) AND isnotnull(s_store_sk#16)) - -(17) CometProject -Input [2]: [s_store_sk#16, s_state#17] -Arguments: [s_store_sk#16, s_state#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) AS s_state#18] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_state#18] -Arguments: [s_store_sk#16, s_state#18] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Right output [2]: [s_store_sk#16, s_state#18] -Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16, s_state#18] -Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_item_id#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_item_sk#19, i_item_id#20] -Condition : isnotnull(i_item_sk#19) - -(23) CometProject -Input [2]: [i_item_sk#19, i_item_id#20] -Arguments: [i_item_sk#19, i_item_id#21], [i_item_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#20, 16, true, false, true) AS i_item_id#21] - -(24) CometBroadcastExchange -Input [2]: [i_item_sk#19, i_item_id#21] -Arguments: [i_item_sk#19, i_item_id#21] - -(25) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] -Right output [2]: [i_item_sk#19, i_item_id#21] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(26) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18, i_item_sk#19, i_item_id#21] -Arguments: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18] - -(27) CometExpand -Input [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18] -Arguments: [[ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18, 0], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, null, 1], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, null, null, 3]], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#22, s_state#23, spark_grouping_id#24] - -(28) CometHashAggregate -Input [7]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#22, s_state#23, spark_grouping_id#24] -Keys [3]: [i_item_id#22, s_state#23, spark_grouping_id#24] -Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] - -(29) CometExchange -Input [11]: [i_item_id#22, s_state#23, spark_grouping_id#24, sum#25, count#26, sum#27, count#28, sum#29, count#30, sum#31, count#32] -Arguments: hashpartitioning(i_item_id#22, s_state#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(30) CometHashAggregate -Input [11]: [i_item_id#22, s_state#23, spark_grouping_id#24, sum#25, count#26, sum#27, count#28, sum#29, count#30, sum#31, count#32] -Keys [3]: [i_item_id#22, s_state#23, spark_grouping_id#24] -Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] - -(31) CometTakeOrderedAndProject -Input [7]: [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#22 ASC NULLS FIRST,s_state#23 ASC NULLS FIRST], output=[i_item_id#22,s_state#23,g_state#33,agg1#34,agg2#35,agg3#36,agg4#37]), [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37], 100, 0, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37] - -(32) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (33) - - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_sk#14)) - -(35) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(37) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/simplified.txt deleted file mode 100644 index bdae0cc477..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,42 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,spark_grouping_id,avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price))] - CometExchange [i_item_id,s_state,spark_grouping_id] #1 - CometHashAggregate [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] - 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,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_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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_state] #5 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] 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/q27/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/explain.txt deleted file mode 100644 index eb158b2889..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (32) -+- CometTakeOrderedAndProject (31) - +- CometHashAggregate (30) - +- CometExchange (29) - +- CometHashAggregate (28) - +- CometExpand (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = S)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#16, s_state#17] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) = TN) AND isnotnull(s_store_sk#16)) - -(17) CometProject -Input [2]: [s_store_sk#16, s_state#17] -Arguments: [s_store_sk#16, s_state#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) AS s_state#18] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_state#18] -Arguments: [s_store_sk#16, s_state#18] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Right output [2]: [s_store_sk#16, s_state#18] -Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16, s_state#18] -Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_item_id#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_item_sk#19, i_item_id#20] -Condition : isnotnull(i_item_sk#19) - -(23) CometProject -Input [2]: [i_item_sk#19, i_item_id#20] -Arguments: [i_item_sk#19, i_item_id#21], [i_item_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#20, 16, true, false, true) AS i_item_id#21] - -(24) CometBroadcastExchange -Input [2]: [i_item_sk#19, i_item_id#21] -Arguments: [i_item_sk#19, i_item_id#21] - -(25) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] -Right output [2]: [i_item_sk#19, i_item_id#21] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(26) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18, i_item_sk#19, i_item_id#21] -Arguments: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18] - -(27) CometExpand -Input [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18] -Arguments: [[ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18, 0], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, null, 1], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, null, null, 3]], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#22, s_state#23, spark_grouping_id#24] - -(28) CometHashAggregate -Input [7]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#22, s_state#23, spark_grouping_id#24] -Keys [3]: [i_item_id#22, s_state#23, spark_grouping_id#24] -Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] - -(29) CometExchange -Input [11]: [i_item_id#22, s_state#23, spark_grouping_id#24, sum#25, count#26, sum#27, count#28, sum#29, count#30, sum#31, count#32] -Arguments: hashpartitioning(i_item_id#22, s_state#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(30) CometHashAggregate -Input [11]: [i_item_id#22, s_state#23, spark_grouping_id#24, sum#25, count#26, sum#27, count#28, sum#29, count#30, sum#31, count#32] -Keys [3]: [i_item_id#22, s_state#23, spark_grouping_id#24] -Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] - -(31) CometTakeOrderedAndProject -Input [7]: [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#22 ASC NULLS FIRST,s_state#23 ASC NULLS FIRST], output=[i_item_id#22,s_state#23,g_state#33,agg1#34,agg2#35,agg3#36,agg4#37]), [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37], 100, 0, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37] - -(32) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (33) - - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_sk#14)) - -(35) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(37) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/extended.txt deleted file mode 100644 index f550f3855c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/extended.txt +++ /dev/null @@ -1,40 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index bdae0cc477..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27/simplified.txt +++ /dev/null @@ -1,42 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,spark_grouping_id,avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price))] - CometExchange [i_item_id,s_state,spark_grouping_id] #1 - CometHashAggregate [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] - 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,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_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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_state] #5 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_datafusion/explain.txt deleted file mode 100644 index 985274408b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_datafusion/explain.txt +++ /dev/null @@ -1,437 +0,0 @@ -== Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (76) -:- * BroadcastNestedLoopJoin Inner BuildRight (63) -: :- * BroadcastNestedLoopJoin Inner BuildRight (50) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (37) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (24) -: : : : :- * CometColumnarToRow (11) -: : : : : +- CometHashAggregate (10) -: : : : : +- CometColumnarExchange (9) -: : : : : +- * HashAggregate (8) -: : : : : +- * HashAggregate (7) -: : : : : +- * CometColumnarToRow (6) -: : : : : +- CometExchange (5) -: : : : : +- CometHashAggregate (4) -: : : : : +- CometProject (3) -: : : : : +- CometFilter (2) -: : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) -: : : : +- BroadcastExchange (23) -: : : : +- * CometColumnarToRow (22) -: : : : +- CometHashAggregate (21) -: : : : +- CometColumnarExchange (20) -: : : : +- * HashAggregate (19) -: : : : +- * HashAggregate (18) -: : : : +- * CometColumnarToRow (17) -: : : : +- CometExchange (16) -: : : : +- CometHashAggregate (15) -: : : : +- CometProject (14) -: : : : +- CometFilter (13) -: : : : +- CometNativeScan parquet spark_catalog.default.store_sales (12) -: : : +- BroadcastExchange (36) -: : : +- * CometColumnarToRow (35) -: : : +- CometHashAggregate (34) -: : : +- CometColumnarExchange (33) -: : : +- * HashAggregate (32) -: : : +- * HashAggregate (31) -: : : +- * CometColumnarToRow (30) -: : : +- CometExchange (29) -: : : +- CometHashAggregate (28) -: : : +- CometProject (27) -: : : +- CometFilter (26) -: : : +- CometNativeScan parquet spark_catalog.default.store_sales (25) -: : +- BroadcastExchange (49) -: : +- * CometColumnarToRow (48) -: : +- CometHashAggregate (47) -: : +- CometColumnarExchange (46) -: : +- * HashAggregate (45) -: : +- * HashAggregate (44) -: : +- * CometColumnarToRow (43) -: : +- CometExchange (42) -: : +- CometHashAggregate (41) -: : +- CometProject (40) -: : +- CometFilter (39) -: : +- CometNativeScan parquet spark_catalog.default.store_sales (38) -: +- BroadcastExchange (62) -: +- * CometColumnarToRow (61) -: +- CometHashAggregate (60) -: +- CometColumnarExchange (59) -: +- * HashAggregate (58) -: +- * HashAggregate (57) -: +- * CometColumnarToRow (56) -: +- CometExchange (55) -: +- CometHashAggregate (54) -: +- CometProject (53) -: +- CometFilter (52) -: +- CometNativeScan parquet spark_catalog.default.store_sales (51) -+- BroadcastExchange (75) - +- * CometColumnarToRow (74) - +- CometHashAggregate (73) - +- CometColumnarExchange (72) - +- * HashAggregate (71) - +- * HashAggregate (70) - +- * CometColumnarToRow (69) - +- CometExchange (68) - +- CometHashAggregate (67) - +- CometProject (66) - +- CometFilter (65) - +- CometNativeScan parquet spark_catalog.default.store_sales (64) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,0), LessThanOrEqual(ss_quantity,5), Or(Or(And(GreaterThanOrEqual(ss_list_price,8.00),LessThanOrEqual(ss_list_price,18.00)),And(GreaterThanOrEqual(ss_coupon_amt,459.00),LessThanOrEqual(ss_coupon_amt,1459.00))),And(GreaterThanOrEqual(ss_wholesale_cost,57.00),LessThanOrEqual(ss_wholesale_cost,77.00)))] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 0)) AND (ss_quantity#1 <= 5)) AND ((((ss_list_price#3 >= 8.00) AND (ss_list_price#3 <= 18.00)) OR ((ss_coupon_amt#4 >= 459.00) AND (ss_coupon_amt#4 <= 1459.00))) OR ((ss_wholesale_cost#2 >= 57.00) AND (ss_wholesale_cost#2 <= 77.00)))) - -(3) CometProject -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Arguments: [ss_list_price#3], [ss_list_price#3] - -(4) CometHashAggregate -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] - -(5) CometExchange -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(6) CometColumnarToRow [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] - -(7) HashAggregate [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10] -Results [4]: [ss_list_price#3, sum#6, count#7, count#8] - -(8) HashAggregate [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10, count(ss_list_price#3)#11] -Results [4]: [sum#6, count#7, count#8, count#12] - -(9) CometColumnarExchange -Input [4]: [sum#6, count#7, count#8, count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(10) CometHashAggregate -Input [4]: [sum#6, count#7, count#8, count#12] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] - -(11) CometColumnarToRow [codegen id : 12] -Input [3]: [B1_LP#13, B1_CNT#14, B1_CNTD#15] - -(12) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] -ReadSchema: struct - -(13) CometFilter -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Condition : (((isnotnull(ss_quantity#16) AND (ss_quantity#16 >= 6)) AND (ss_quantity#16 <= 10)) AND ((((ss_list_price#18 >= 90.00) AND (ss_list_price#18 <= 100.00)) OR ((ss_coupon_amt#19 >= 2323.00) AND (ss_coupon_amt#19 <= 3323.00))) OR ((ss_wholesale_cost#17 >= 31.00) AND (ss_wholesale_cost#17 <= 51.00)))) - -(14) CometProject -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Arguments: [ss_list_price#18], [ss_list_price#18] - -(15) CometHashAggregate -Input [1]: [ss_list_price#18] -Keys [1]: [ss_list_price#18] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_list_price#18)] - -(16) CometExchange -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(17) CometColumnarToRow [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] - -(18) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Keys [1]: [ss_list_price#18] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25] -Results [4]: [ss_list_price#18, sum#21, count#22, count#23] - -(19) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18), partial_count(distinct ss_list_price#18)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25, count(ss_list_price#18)#26] -Results [4]: [sum#21, count#22, count#23, count#27] - -(20) CometColumnarExchange -Input [4]: [sum#21, count#22, count#23, count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(21) CometHashAggregate -Input [4]: [sum#21, count#22, count#23, count#27] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#18)), count(ss_list_price#18), count(distinct ss_list_price#18)] - -(22) CometColumnarToRow [codegen id : 3] -Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] - -(23) BroadcastExchange -Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] -Arguments: IdentityBroadcastMode, [plan_id=5] - -(24) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(25) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] -ReadSchema: struct - -(26) CometFilter -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Condition : (((isnotnull(ss_quantity#31) AND (ss_quantity#31 >= 11)) AND (ss_quantity#31 <= 15)) AND ((((ss_list_price#33 >= 142.00) AND (ss_list_price#33 <= 152.00)) OR ((ss_coupon_amt#34 >= 12214.00) AND (ss_coupon_amt#34 <= 13214.00))) OR ((ss_wholesale_cost#32 >= 79.00) AND (ss_wholesale_cost#32 <= 99.00)))) - -(27) CometProject -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Arguments: [ss_list_price#33], [ss_list_price#33] - -(28) CometHashAggregate -Input [1]: [ss_list_price#33] -Keys [1]: [ss_list_price#33] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_list_price#33)] - -(29) CometExchange -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(30) CometColumnarToRow [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] - -(31) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Keys [1]: [ss_list_price#33] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40] -Results [4]: [ss_list_price#33, sum#36, count#37, count#38] - -(32) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33), partial_count(distinct ss_list_price#33)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40, count(ss_list_price#33)#41] -Results [4]: [sum#36, count#37, count#38, count#42] - -(33) CometColumnarExchange -Input [4]: [sum#36, count#37, count#38, count#42] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(34) CometHashAggregate -Input [4]: [sum#36, count#37, count#38, count#42] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#33)), count(ss_list_price#33), count(distinct ss_list_price#33)] - -(35) CometColumnarToRow [codegen id : 5] -Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] - -(36) BroadcastExchange -Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] -Arguments: IdentityBroadcastMode, [plan_id=8] - -(37) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(38) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] -ReadSchema: struct - -(39) CometFilter -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Condition : (((isnotnull(ss_quantity#46) AND (ss_quantity#46 >= 16)) AND (ss_quantity#46 <= 20)) AND ((((ss_list_price#48 >= 135.00) AND (ss_list_price#48 <= 145.00)) OR ((ss_coupon_amt#49 >= 6071.00) AND (ss_coupon_amt#49 <= 7071.00))) OR ((ss_wholesale_cost#47 >= 38.00) AND (ss_wholesale_cost#47 <= 58.00)))) - -(40) CometProject -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Arguments: [ss_list_price#48], [ss_list_price#48] - -(41) CometHashAggregate -Input [1]: [ss_list_price#48] -Keys [1]: [ss_list_price#48] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_list_price#48)] - -(42) CometExchange -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(43) CometColumnarToRow [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] - -(44) HashAggregate [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Keys [1]: [ss_list_price#48] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55] -Results [4]: [ss_list_price#48, sum#51, count#52, count#53] - -(45) HashAggregate [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48), partial_count(distinct ss_list_price#48)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55, count(ss_list_price#48)#56] -Results [4]: [sum#51, count#52, count#53, count#57] - -(46) CometColumnarExchange -Input [4]: [sum#51, count#52, count#53, count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(47) CometHashAggregate -Input [4]: [sum#51, count#52, count#53, count#57] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#48)), count(ss_list_price#48), count(distinct ss_list_price#48)] - -(48) CometColumnarToRow [codegen id : 7] -Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] - -(49) BroadcastExchange -Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] -Arguments: IdentityBroadcastMode, [plan_id=11] - -(50) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(51) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] -ReadSchema: struct - -(52) CometFilter -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Condition : (((isnotnull(ss_quantity#61) AND (ss_quantity#61 >= 21)) AND (ss_quantity#61 <= 25)) AND ((((ss_list_price#63 >= 122.00) AND (ss_list_price#63 <= 132.00)) OR ((ss_coupon_amt#64 >= 836.00) AND (ss_coupon_amt#64 <= 1836.00))) OR ((ss_wholesale_cost#62 >= 17.00) AND (ss_wholesale_cost#62 <= 37.00)))) - -(53) CometProject -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Arguments: [ss_list_price#63], [ss_list_price#63] - -(54) CometHashAggregate -Input [1]: [ss_list_price#63] -Keys [1]: [ss_list_price#63] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_list_price#63)] - -(55) CometExchange -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(56) CometColumnarToRow [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] - -(57) HashAggregate [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Keys [1]: [ss_list_price#63] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70] -Results [4]: [ss_list_price#63, sum#66, count#67, count#68] - -(58) HashAggregate [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63), partial_count(distinct ss_list_price#63)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70, count(ss_list_price#63)#71] -Results [4]: [sum#66, count#67, count#68, count#72] - -(59) CometColumnarExchange -Input [4]: [sum#66, count#67, count#68, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(60) CometHashAggregate -Input [4]: [sum#66, count#67, count#68, count#72] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#63)), count(ss_list_price#63), count(distinct ss_list_price#63)] - -(61) CometColumnarToRow [codegen id : 9] -Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] - -(62) BroadcastExchange -Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] -Arguments: IdentityBroadcastMode, [plan_id=14] - -(63) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(64) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] -ReadSchema: struct - -(65) CometFilter -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Condition : (((isnotnull(ss_quantity#76) AND (ss_quantity#76 >= 26)) AND (ss_quantity#76 <= 30)) AND ((((ss_list_price#78 >= 154.00) AND (ss_list_price#78 <= 164.00)) OR ((ss_coupon_amt#79 >= 7326.00) AND (ss_coupon_amt#79 <= 8326.00))) OR ((ss_wholesale_cost#77 >= 7.00) AND (ss_wholesale_cost#77 <= 27.00)))) - -(66) CometProject -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Arguments: [ss_list_price#78], [ss_list_price#78] - -(67) CometHashAggregate -Input [1]: [ss_list_price#78] -Keys [1]: [ss_list_price#78] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_list_price#78)] - -(68) CometExchange -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(69) CometColumnarToRow [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] - -(70) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Keys [1]: [ss_list_price#78] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85] -Results [4]: [ss_list_price#78, sum#81, count#82, count#83] - -(71) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78), partial_count(distinct ss_list_price#78)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85, count(ss_list_price#78)#86] -Results [4]: [sum#81, count#82, count#83, count#87] - -(72) CometColumnarExchange -Input [4]: [sum#81, count#82, count#83, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(73) CometHashAggregate -Input [4]: [sum#81, count#82, count#83, count#87] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#78)), count(ss_list_price#78), count(distinct ss_list_price#78)] - -(74) CometColumnarToRow [codegen id : 11] -Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] - -(75) BroadcastExchange -Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] -Arguments: IdentityBroadcastMode, [plan_id=17] - -(76) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_datafusion/simplified.txt deleted file mode 100644 index a8540a4abb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_datafusion/simplified.txt +++ /dev/null @@ -1,105 +0,0 @@ -WholeStageCodegen (12) - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B1_LP,B1_CNT,B1_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #2 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B2_LP,B2_CNT,B2_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #4 - WholeStageCodegen (2) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #5 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B3_LP,B3_CNT,B3_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #7 - WholeStageCodegen (4) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #8 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B4_LP,B4_CNT,B4_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #10 - WholeStageCodegen (6) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #11 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B5_LP,B5_CNT,B5_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #13 - WholeStageCodegen (8) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #14 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B6_LP,B6_CNT,B6_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #16 - WholeStageCodegen (10) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #17 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan 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/q28.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_iceberg_compat/explain.txt deleted file mode 100644 index 98ac957fc7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,437 +0,0 @@ -== Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (76) -:- * BroadcastNestedLoopJoin Inner BuildRight (63) -: :- * BroadcastNestedLoopJoin Inner BuildRight (50) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (37) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (24) -: : : : :- * CometColumnarToRow (11) -: : : : : +- CometHashAggregate (10) -: : : : : +- CometColumnarExchange (9) -: : : : : +- * HashAggregate (8) -: : : : : +- * HashAggregate (7) -: : : : : +- * CometColumnarToRow (6) -: : : : : +- CometExchange (5) -: : : : : +- CometHashAggregate (4) -: : : : : +- CometProject (3) -: : : : : +- CometFilter (2) -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) -: : : : +- BroadcastExchange (23) -: : : : +- * CometColumnarToRow (22) -: : : : +- CometHashAggregate (21) -: : : : +- CometColumnarExchange (20) -: : : : +- * HashAggregate (19) -: : : : +- * HashAggregate (18) -: : : : +- * CometColumnarToRow (17) -: : : : +- CometExchange (16) -: : : : +- CometHashAggregate (15) -: : : : +- CometProject (14) -: : : : +- CometFilter (13) -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (12) -: : : +- BroadcastExchange (36) -: : : +- * CometColumnarToRow (35) -: : : +- CometHashAggregate (34) -: : : +- CometColumnarExchange (33) -: : : +- * HashAggregate (32) -: : : +- * HashAggregate (31) -: : : +- * CometColumnarToRow (30) -: : : +- CometExchange (29) -: : : +- CometHashAggregate (28) -: : : +- CometProject (27) -: : : +- CometFilter (26) -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (25) -: : +- BroadcastExchange (49) -: : +- * CometColumnarToRow (48) -: : +- CometHashAggregate (47) -: : +- CometColumnarExchange (46) -: : +- * HashAggregate (45) -: : +- * HashAggregate (44) -: : +- * CometColumnarToRow (43) -: : +- CometExchange (42) -: : +- CometHashAggregate (41) -: : +- CometProject (40) -: : +- CometFilter (39) -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (38) -: +- BroadcastExchange (62) -: +- * CometColumnarToRow (61) -: +- CometHashAggregate (60) -: +- CometColumnarExchange (59) -: +- * HashAggregate (58) -: +- * HashAggregate (57) -: +- * CometColumnarToRow (56) -: +- CometExchange (55) -: +- CometHashAggregate (54) -: +- CometProject (53) -: +- CometFilter (52) -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (51) -+- BroadcastExchange (75) - +- * CometColumnarToRow (74) - +- CometHashAggregate (73) - +- CometColumnarExchange (72) - +- * HashAggregate (71) - +- * HashAggregate (70) - +- * CometColumnarToRow (69) - +- CometExchange (68) - +- CometHashAggregate (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (64) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,0), LessThanOrEqual(ss_quantity,5), Or(Or(And(GreaterThanOrEqual(ss_list_price,8.00),LessThanOrEqual(ss_list_price,18.00)),And(GreaterThanOrEqual(ss_coupon_amt,459.00),LessThanOrEqual(ss_coupon_amt,1459.00))),And(GreaterThanOrEqual(ss_wholesale_cost,57.00),LessThanOrEqual(ss_wholesale_cost,77.00)))] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 0)) AND (ss_quantity#1 <= 5)) AND ((((ss_list_price#3 >= 8.00) AND (ss_list_price#3 <= 18.00)) OR ((ss_coupon_amt#4 >= 459.00) AND (ss_coupon_amt#4 <= 1459.00))) OR ((ss_wholesale_cost#2 >= 57.00) AND (ss_wholesale_cost#2 <= 77.00)))) - -(3) CometProject -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Arguments: [ss_list_price#3], [ss_list_price#3] - -(4) CometHashAggregate -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] - -(5) CometExchange -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(6) CometColumnarToRow [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] - -(7) HashAggregate [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10] -Results [4]: [ss_list_price#3, sum#6, count#7, count#8] - -(8) HashAggregate [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10, count(ss_list_price#3)#11] -Results [4]: [sum#6, count#7, count#8, count#12] - -(9) CometColumnarExchange -Input [4]: [sum#6, count#7, count#8, count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(10) CometHashAggregate -Input [4]: [sum#6, count#7, count#8, count#12] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] - -(11) CometColumnarToRow [codegen id : 12] -Input [3]: [B1_LP#13, B1_CNT#14, B1_CNTD#15] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] -ReadSchema: struct - -(13) CometFilter -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Condition : (((isnotnull(ss_quantity#16) AND (ss_quantity#16 >= 6)) AND (ss_quantity#16 <= 10)) AND ((((ss_list_price#18 >= 90.00) AND (ss_list_price#18 <= 100.00)) OR ((ss_coupon_amt#19 >= 2323.00) AND (ss_coupon_amt#19 <= 3323.00))) OR ((ss_wholesale_cost#17 >= 31.00) AND (ss_wholesale_cost#17 <= 51.00)))) - -(14) CometProject -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Arguments: [ss_list_price#18], [ss_list_price#18] - -(15) CometHashAggregate -Input [1]: [ss_list_price#18] -Keys [1]: [ss_list_price#18] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_list_price#18)] - -(16) CometExchange -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(17) CometColumnarToRow [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] - -(18) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Keys [1]: [ss_list_price#18] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25] -Results [4]: [ss_list_price#18, sum#21, count#22, count#23] - -(19) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18), partial_count(distinct ss_list_price#18)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25, count(ss_list_price#18)#26] -Results [4]: [sum#21, count#22, count#23, count#27] - -(20) CometColumnarExchange -Input [4]: [sum#21, count#22, count#23, count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(21) CometHashAggregate -Input [4]: [sum#21, count#22, count#23, count#27] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#18)), count(ss_list_price#18), count(distinct ss_list_price#18)] - -(22) CometColumnarToRow [codegen id : 3] -Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] - -(23) BroadcastExchange -Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] -Arguments: IdentityBroadcastMode, [plan_id=5] - -(24) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] -ReadSchema: struct - -(26) CometFilter -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Condition : (((isnotnull(ss_quantity#31) AND (ss_quantity#31 >= 11)) AND (ss_quantity#31 <= 15)) AND ((((ss_list_price#33 >= 142.00) AND (ss_list_price#33 <= 152.00)) OR ((ss_coupon_amt#34 >= 12214.00) AND (ss_coupon_amt#34 <= 13214.00))) OR ((ss_wholesale_cost#32 >= 79.00) AND (ss_wholesale_cost#32 <= 99.00)))) - -(27) CometProject -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Arguments: [ss_list_price#33], [ss_list_price#33] - -(28) CometHashAggregate -Input [1]: [ss_list_price#33] -Keys [1]: [ss_list_price#33] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_list_price#33)] - -(29) CometExchange -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(30) CometColumnarToRow [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] - -(31) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Keys [1]: [ss_list_price#33] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40] -Results [4]: [ss_list_price#33, sum#36, count#37, count#38] - -(32) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33), partial_count(distinct ss_list_price#33)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40, count(ss_list_price#33)#41] -Results [4]: [sum#36, count#37, count#38, count#42] - -(33) CometColumnarExchange -Input [4]: [sum#36, count#37, count#38, count#42] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(34) CometHashAggregate -Input [4]: [sum#36, count#37, count#38, count#42] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#33)), count(ss_list_price#33), count(distinct ss_list_price#33)] - -(35) CometColumnarToRow [codegen id : 5] -Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] - -(36) BroadcastExchange -Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] -Arguments: IdentityBroadcastMode, [plan_id=8] - -(37) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] -ReadSchema: struct - -(39) CometFilter -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Condition : (((isnotnull(ss_quantity#46) AND (ss_quantity#46 >= 16)) AND (ss_quantity#46 <= 20)) AND ((((ss_list_price#48 >= 135.00) AND (ss_list_price#48 <= 145.00)) OR ((ss_coupon_amt#49 >= 6071.00) AND (ss_coupon_amt#49 <= 7071.00))) OR ((ss_wholesale_cost#47 >= 38.00) AND (ss_wholesale_cost#47 <= 58.00)))) - -(40) CometProject -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Arguments: [ss_list_price#48], [ss_list_price#48] - -(41) CometHashAggregate -Input [1]: [ss_list_price#48] -Keys [1]: [ss_list_price#48] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_list_price#48)] - -(42) CometExchange -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(43) CometColumnarToRow [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] - -(44) HashAggregate [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Keys [1]: [ss_list_price#48] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55] -Results [4]: [ss_list_price#48, sum#51, count#52, count#53] - -(45) HashAggregate [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48), partial_count(distinct ss_list_price#48)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55, count(ss_list_price#48)#56] -Results [4]: [sum#51, count#52, count#53, count#57] - -(46) CometColumnarExchange -Input [4]: [sum#51, count#52, count#53, count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(47) CometHashAggregate -Input [4]: [sum#51, count#52, count#53, count#57] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#48)), count(ss_list_price#48), count(distinct ss_list_price#48)] - -(48) CometColumnarToRow [codegen id : 7] -Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] - -(49) BroadcastExchange -Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] -Arguments: IdentityBroadcastMode, [plan_id=11] - -(50) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] -ReadSchema: struct - -(52) CometFilter -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Condition : (((isnotnull(ss_quantity#61) AND (ss_quantity#61 >= 21)) AND (ss_quantity#61 <= 25)) AND ((((ss_list_price#63 >= 122.00) AND (ss_list_price#63 <= 132.00)) OR ((ss_coupon_amt#64 >= 836.00) AND (ss_coupon_amt#64 <= 1836.00))) OR ((ss_wholesale_cost#62 >= 17.00) AND (ss_wholesale_cost#62 <= 37.00)))) - -(53) CometProject -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Arguments: [ss_list_price#63], [ss_list_price#63] - -(54) CometHashAggregate -Input [1]: [ss_list_price#63] -Keys [1]: [ss_list_price#63] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_list_price#63)] - -(55) CometExchange -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(56) CometColumnarToRow [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] - -(57) HashAggregate [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Keys [1]: [ss_list_price#63] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70] -Results [4]: [ss_list_price#63, sum#66, count#67, count#68] - -(58) HashAggregate [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63), partial_count(distinct ss_list_price#63)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70, count(ss_list_price#63)#71] -Results [4]: [sum#66, count#67, count#68, count#72] - -(59) CometColumnarExchange -Input [4]: [sum#66, count#67, count#68, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(60) CometHashAggregate -Input [4]: [sum#66, count#67, count#68, count#72] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#63)), count(ss_list_price#63), count(distinct ss_list_price#63)] - -(61) CometColumnarToRow [codegen id : 9] -Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] - -(62) BroadcastExchange -Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] -Arguments: IdentityBroadcastMode, [plan_id=14] - -(63) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] -ReadSchema: struct - -(65) CometFilter -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Condition : (((isnotnull(ss_quantity#76) AND (ss_quantity#76 >= 26)) AND (ss_quantity#76 <= 30)) AND ((((ss_list_price#78 >= 154.00) AND (ss_list_price#78 <= 164.00)) OR ((ss_coupon_amt#79 >= 7326.00) AND (ss_coupon_amt#79 <= 8326.00))) OR ((ss_wholesale_cost#77 >= 7.00) AND (ss_wholesale_cost#77 <= 27.00)))) - -(66) CometProject -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Arguments: [ss_list_price#78], [ss_list_price#78] - -(67) CometHashAggregate -Input [1]: [ss_list_price#78] -Keys [1]: [ss_list_price#78] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_list_price#78)] - -(68) CometExchange -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(69) CometColumnarToRow [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] - -(70) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Keys [1]: [ss_list_price#78] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85] -Results [4]: [ss_list_price#78, sum#81, count#82, count#83] - -(71) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78), partial_count(distinct ss_list_price#78)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85, count(ss_list_price#78)#86] -Results [4]: [sum#81, count#82, count#83, count#87] - -(72) CometColumnarExchange -Input [4]: [sum#81, count#82, count#83, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(73) CometHashAggregate -Input [4]: [sum#81, count#82, count#83, count#87] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#78)), count(ss_list_price#78), count(distinct ss_list_price#78)] - -(74) CometColumnarToRow [codegen id : 11] -Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] - -(75) BroadcastExchange -Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] -Arguments: IdentityBroadcastMode, [plan_id=17] - -(76) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_iceberg_compat/simplified.txt deleted file mode 100644 index ce476affda..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,105 +0,0 @@ -WholeStageCodegen (12) - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B1_LP,B1_CNT,B1_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #2 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B2_LP,B2_CNT,B2_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #4 - WholeStageCodegen (2) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #5 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B3_LP,B3_CNT,B3_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #7 - WholeStageCodegen (4) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #8 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B4_LP,B4_CNT,B4_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #10 - WholeStageCodegen (6) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #11 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B5_LP,B5_CNT,B5_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #13 - WholeStageCodegen (8) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #14 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B6_LP,B6_CNT,B6_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #16 - WholeStageCodegen (10) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #17 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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/q28/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/explain.txt deleted file mode 100644 index 98ac957fc7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/explain.txt +++ /dev/null @@ -1,437 +0,0 @@ -== Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (76) -:- * BroadcastNestedLoopJoin Inner BuildRight (63) -: :- * BroadcastNestedLoopJoin Inner BuildRight (50) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (37) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (24) -: : : : :- * CometColumnarToRow (11) -: : : : : +- CometHashAggregate (10) -: : : : : +- CometColumnarExchange (9) -: : : : : +- * HashAggregate (8) -: : : : : +- * HashAggregate (7) -: : : : : +- * CometColumnarToRow (6) -: : : : : +- CometExchange (5) -: : : : : +- CometHashAggregate (4) -: : : : : +- CometProject (3) -: : : : : +- CometFilter (2) -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) -: : : : +- BroadcastExchange (23) -: : : : +- * CometColumnarToRow (22) -: : : : +- CometHashAggregate (21) -: : : : +- CometColumnarExchange (20) -: : : : +- * HashAggregate (19) -: : : : +- * HashAggregate (18) -: : : : +- * CometColumnarToRow (17) -: : : : +- CometExchange (16) -: : : : +- CometHashAggregate (15) -: : : : +- CometProject (14) -: : : : +- CometFilter (13) -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (12) -: : : +- BroadcastExchange (36) -: : : +- * CometColumnarToRow (35) -: : : +- CometHashAggregate (34) -: : : +- CometColumnarExchange (33) -: : : +- * HashAggregate (32) -: : : +- * HashAggregate (31) -: : : +- * CometColumnarToRow (30) -: : : +- CometExchange (29) -: : : +- CometHashAggregate (28) -: : : +- CometProject (27) -: : : +- CometFilter (26) -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (25) -: : +- BroadcastExchange (49) -: : +- * CometColumnarToRow (48) -: : +- CometHashAggregate (47) -: : +- CometColumnarExchange (46) -: : +- * HashAggregate (45) -: : +- * HashAggregate (44) -: : +- * CometColumnarToRow (43) -: : +- CometExchange (42) -: : +- CometHashAggregate (41) -: : +- CometProject (40) -: : +- CometFilter (39) -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (38) -: +- BroadcastExchange (62) -: +- * CometColumnarToRow (61) -: +- CometHashAggregate (60) -: +- CometColumnarExchange (59) -: +- * HashAggregate (58) -: +- * HashAggregate (57) -: +- * CometColumnarToRow (56) -: +- CometExchange (55) -: +- CometHashAggregate (54) -: +- CometProject (53) -: +- CometFilter (52) -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (51) -+- BroadcastExchange (75) - +- * CometColumnarToRow (74) - +- CometHashAggregate (73) - +- CometColumnarExchange (72) - +- * HashAggregate (71) - +- * HashAggregate (70) - +- * CometColumnarToRow (69) - +- CometExchange (68) - +- CometHashAggregate (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (64) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,0), LessThanOrEqual(ss_quantity,5), Or(Or(And(GreaterThanOrEqual(ss_list_price,8.00),LessThanOrEqual(ss_list_price,18.00)),And(GreaterThanOrEqual(ss_coupon_amt,459.00),LessThanOrEqual(ss_coupon_amt,1459.00))),And(GreaterThanOrEqual(ss_wholesale_cost,57.00),LessThanOrEqual(ss_wholesale_cost,77.00)))] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 0)) AND (ss_quantity#1 <= 5)) AND ((((ss_list_price#3 >= 8.00) AND (ss_list_price#3 <= 18.00)) OR ((ss_coupon_amt#4 >= 459.00) AND (ss_coupon_amt#4 <= 1459.00))) OR ((ss_wholesale_cost#2 >= 57.00) AND (ss_wholesale_cost#2 <= 77.00)))) - -(3) CometProject -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Arguments: [ss_list_price#3], [ss_list_price#3] - -(4) CometHashAggregate -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] - -(5) CometExchange -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(6) CometColumnarToRow [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] - -(7) HashAggregate [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10] -Results [4]: [ss_list_price#3, sum#6, count#7, count#8] - -(8) HashAggregate [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10, count(ss_list_price#3)#11] -Results [4]: [sum#6, count#7, count#8, count#12] - -(9) CometColumnarExchange -Input [4]: [sum#6, count#7, count#8, count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(10) CometHashAggregate -Input [4]: [sum#6, count#7, count#8, count#12] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] - -(11) CometColumnarToRow [codegen id : 12] -Input [3]: [B1_LP#13, B1_CNT#14, B1_CNTD#15] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] -ReadSchema: struct - -(13) CometFilter -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Condition : (((isnotnull(ss_quantity#16) AND (ss_quantity#16 >= 6)) AND (ss_quantity#16 <= 10)) AND ((((ss_list_price#18 >= 90.00) AND (ss_list_price#18 <= 100.00)) OR ((ss_coupon_amt#19 >= 2323.00) AND (ss_coupon_amt#19 <= 3323.00))) OR ((ss_wholesale_cost#17 >= 31.00) AND (ss_wholesale_cost#17 <= 51.00)))) - -(14) CometProject -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Arguments: [ss_list_price#18], [ss_list_price#18] - -(15) CometHashAggregate -Input [1]: [ss_list_price#18] -Keys [1]: [ss_list_price#18] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_list_price#18)] - -(16) CometExchange -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(17) CometColumnarToRow [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] - -(18) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Keys [1]: [ss_list_price#18] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25] -Results [4]: [ss_list_price#18, sum#21, count#22, count#23] - -(19) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18), partial_count(distinct ss_list_price#18)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25, count(ss_list_price#18)#26] -Results [4]: [sum#21, count#22, count#23, count#27] - -(20) CometColumnarExchange -Input [4]: [sum#21, count#22, count#23, count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(21) CometHashAggregate -Input [4]: [sum#21, count#22, count#23, count#27] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#18)), count(ss_list_price#18), count(distinct ss_list_price#18)] - -(22) CometColumnarToRow [codegen id : 3] -Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] - -(23) BroadcastExchange -Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] -Arguments: IdentityBroadcastMode, [plan_id=5] - -(24) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] -ReadSchema: struct - -(26) CometFilter -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Condition : (((isnotnull(ss_quantity#31) AND (ss_quantity#31 >= 11)) AND (ss_quantity#31 <= 15)) AND ((((ss_list_price#33 >= 142.00) AND (ss_list_price#33 <= 152.00)) OR ((ss_coupon_amt#34 >= 12214.00) AND (ss_coupon_amt#34 <= 13214.00))) OR ((ss_wholesale_cost#32 >= 79.00) AND (ss_wholesale_cost#32 <= 99.00)))) - -(27) CometProject -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Arguments: [ss_list_price#33], [ss_list_price#33] - -(28) CometHashAggregate -Input [1]: [ss_list_price#33] -Keys [1]: [ss_list_price#33] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_list_price#33)] - -(29) CometExchange -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(30) CometColumnarToRow [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] - -(31) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Keys [1]: [ss_list_price#33] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40] -Results [4]: [ss_list_price#33, sum#36, count#37, count#38] - -(32) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33), partial_count(distinct ss_list_price#33)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40, count(ss_list_price#33)#41] -Results [4]: [sum#36, count#37, count#38, count#42] - -(33) CometColumnarExchange -Input [4]: [sum#36, count#37, count#38, count#42] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(34) CometHashAggregate -Input [4]: [sum#36, count#37, count#38, count#42] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#33)), count(ss_list_price#33), count(distinct ss_list_price#33)] - -(35) CometColumnarToRow [codegen id : 5] -Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] - -(36) BroadcastExchange -Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] -Arguments: IdentityBroadcastMode, [plan_id=8] - -(37) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] -ReadSchema: struct - -(39) CometFilter -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Condition : (((isnotnull(ss_quantity#46) AND (ss_quantity#46 >= 16)) AND (ss_quantity#46 <= 20)) AND ((((ss_list_price#48 >= 135.00) AND (ss_list_price#48 <= 145.00)) OR ((ss_coupon_amt#49 >= 6071.00) AND (ss_coupon_amt#49 <= 7071.00))) OR ((ss_wholesale_cost#47 >= 38.00) AND (ss_wholesale_cost#47 <= 58.00)))) - -(40) CometProject -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Arguments: [ss_list_price#48], [ss_list_price#48] - -(41) CometHashAggregate -Input [1]: [ss_list_price#48] -Keys [1]: [ss_list_price#48] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_list_price#48)] - -(42) CometExchange -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(43) CometColumnarToRow [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] - -(44) HashAggregate [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Keys [1]: [ss_list_price#48] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55] -Results [4]: [ss_list_price#48, sum#51, count#52, count#53] - -(45) HashAggregate [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48), partial_count(distinct ss_list_price#48)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55, count(ss_list_price#48)#56] -Results [4]: [sum#51, count#52, count#53, count#57] - -(46) CometColumnarExchange -Input [4]: [sum#51, count#52, count#53, count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(47) CometHashAggregate -Input [4]: [sum#51, count#52, count#53, count#57] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#48)), count(ss_list_price#48), count(distinct ss_list_price#48)] - -(48) CometColumnarToRow [codegen id : 7] -Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] - -(49) BroadcastExchange -Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] -Arguments: IdentityBroadcastMode, [plan_id=11] - -(50) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] -ReadSchema: struct - -(52) CometFilter -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Condition : (((isnotnull(ss_quantity#61) AND (ss_quantity#61 >= 21)) AND (ss_quantity#61 <= 25)) AND ((((ss_list_price#63 >= 122.00) AND (ss_list_price#63 <= 132.00)) OR ((ss_coupon_amt#64 >= 836.00) AND (ss_coupon_amt#64 <= 1836.00))) OR ((ss_wholesale_cost#62 >= 17.00) AND (ss_wholesale_cost#62 <= 37.00)))) - -(53) CometProject -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Arguments: [ss_list_price#63], [ss_list_price#63] - -(54) CometHashAggregate -Input [1]: [ss_list_price#63] -Keys [1]: [ss_list_price#63] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_list_price#63)] - -(55) CometExchange -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(56) CometColumnarToRow [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] - -(57) HashAggregate [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Keys [1]: [ss_list_price#63] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70] -Results [4]: [ss_list_price#63, sum#66, count#67, count#68] - -(58) HashAggregate [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63), partial_count(distinct ss_list_price#63)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70, count(ss_list_price#63)#71] -Results [4]: [sum#66, count#67, count#68, count#72] - -(59) CometColumnarExchange -Input [4]: [sum#66, count#67, count#68, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(60) CometHashAggregate -Input [4]: [sum#66, count#67, count#68, count#72] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#63)), count(ss_list_price#63), count(distinct ss_list_price#63)] - -(61) CometColumnarToRow [codegen id : 9] -Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] - -(62) BroadcastExchange -Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] -Arguments: IdentityBroadcastMode, [plan_id=14] - -(63) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] -ReadSchema: struct - -(65) CometFilter -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Condition : (((isnotnull(ss_quantity#76) AND (ss_quantity#76 >= 26)) AND (ss_quantity#76 <= 30)) AND ((((ss_list_price#78 >= 154.00) AND (ss_list_price#78 <= 164.00)) OR ((ss_coupon_amt#79 >= 7326.00) AND (ss_coupon_amt#79 <= 8326.00))) OR ((ss_wholesale_cost#77 >= 7.00) AND (ss_wholesale_cost#77 <= 27.00)))) - -(66) CometProject -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Arguments: [ss_list_price#78], [ss_list_price#78] - -(67) CometHashAggregate -Input [1]: [ss_list_price#78] -Keys [1]: [ss_list_price#78] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_list_price#78)] - -(68) CometExchange -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(69) CometColumnarToRow [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] - -(70) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Keys [1]: [ss_list_price#78] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85] -Results [4]: [ss_list_price#78, sum#81, count#82, count#83] - -(71) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78), partial_count(distinct ss_list_price#78)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85, count(ss_list_price#78)#86] -Results [4]: [sum#81, count#82, count#83, count#87] - -(72) CometColumnarExchange -Input [4]: [sum#81, count#82, count#83, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(73) CometHashAggregate -Input [4]: [sum#81, count#82, count#83, count#87] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#78)), count(ss_list_price#78), count(distinct ss_list_price#78)] - -(74) CometColumnarToRow [codegen id : 11] -Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] - -(75) BroadcastExchange -Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] -Arguments: IdentityBroadcastMode, [plan_id=17] - -(76) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/extended.txt deleted file mode 100644 index 809265b4d5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/extended.txt +++ /dev/null @@ -1,78 +0,0 @@ -BroadcastNestedLoopJoin -:- BroadcastNestedLoopJoin -: :- BroadcastNestedLoopJoin -: : :- BroadcastNestedLoopJoin -: : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] -: : : : :- CometColumnarToRow -: : : : : +- CometHashAggregate -: : : : : +- CometColumnarExchange -: : : : : +- HashAggregate -: : : : : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : : : : +- CometColumnarToRow -: : : : : +- CometExchange -: : : : : +- CometHashAggregate -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : +- BroadcastExchange -: : : : +- CometColumnarToRow -: : : : +- CometHashAggregate -: : : : +- CometColumnarExchange -: : : : +- HashAggregate -: : : : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : : : +- CometColumnarToRow -: : : : +- CometExchange -: : : : +- CometHashAggregate -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : +- BroadcastExchange -: : : +- CometColumnarToRow -: : : +- CometHashAggregate -: : : +- CometColumnarExchange -: : : +- HashAggregate -: : : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : : +- CometColumnarToRow -: : : +- CometExchange -: : : +- CometHashAggregate -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : +- BroadcastExchange -: : +- CometColumnarToRow -: : +- CometHashAggregate -: : +- CometColumnarExchange -: : +- HashAggregate -: : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : +- CometColumnarToRow -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: +- BroadcastExchange -: +- CometColumnarToRow -: +- CometHashAggregate -: +- CometColumnarExchange -: +- HashAggregate -: +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: +- CometColumnarToRow -: +- CometExchange -: +- CometHashAggregate -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -+- BroadcastExchange - +- CometColumnarToRow - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - -Comet accelerated 42 out of 64 eligible operators (65%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index ce476affda..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/simplified.txt +++ /dev/null @@ -1,105 +0,0 @@ -WholeStageCodegen (12) - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B1_LP,B1_CNT,B1_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #2 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B2_LP,B2_CNT,B2_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #4 - WholeStageCodegen (2) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #5 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B3_LP,B3_CNT,B3_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #7 - WholeStageCodegen (4) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #8 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B4_LP,B4_CNT,B4_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #10 - WholeStageCodegen (6) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #11 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B5_LP,B5_CNT,B5_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #13 - WholeStageCodegen (8) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #14 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B6_LP,B6_CNT,B6_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #16 - WholeStageCodegen (10) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #17 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/explain.txt deleted file mode 100644 index 20f1508413..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/explain.txt +++ /dev/null @@ -1,340 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (18) - : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet spark_catalog.default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * Filter (12) - : : : : : +- * ColumnarToRow (11) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (10) - : : : : +- ReusedExchange (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.store (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.item (32) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 8] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] - -(3) Filter [codegen id : 8] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(4) Scan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(6) Filter [codegen id : 1] -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(7) BroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 8] -Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] -Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(10) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(12) Filter [codegen id : 2] -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(13) BroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] -Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(16) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#19] - -(17) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 8] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] - -(19) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#20] - -(20) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [sr_returned_date_sk#12] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#20] - -(22) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#21] - -(23) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#21] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 8] -Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#21] - -(25) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(26) CometFilter -Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] -Condition : isnotnull(s_store_sk#22) - -(27) CometProject -Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] -Arguments: [s_store_sk#22, s_store_id#25, s_store_name#24], [s_store_sk#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#23, 16, true, false, true) AS s_store_id#25, s_store_name#24] - -(28) CometColumnarToRow [codegen id : 6] -Input [3]: [s_store_sk#22, s_store_id#25, s_store_name#24] - -(29) BroadcastExchange -Input [3]: [s_store_sk#22, s_store_id#25, s_store_name#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(30) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#22] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#22, s_store_id#25, s_store_name#24] - -(32) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] -Condition : isnotnull(i_item_sk#26) - -(34) CometProject -Input [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] -Arguments: [i_item_sk#26, i_item_id#29, i_item_desc#28], [i_item_sk#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#27, 16, true, false, true) AS i_item_id#29, i_item_desc#28] - -(35) CometColumnarToRow [codegen id : 7] -Input [3]: [i_item_sk#26, i_item_id#29, i_item_desc#28] - -(36) BroadcastExchange -Input [3]: [i_item_sk#26, i_item_id#29, i_item_desc#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(37) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#26] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 8] -Output [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24, i_item_id#29, i_item_desc#28] -Input [9]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24, i_item_sk#26, i_item_id#29, i_item_desc#28] - -(39) HashAggregate [codegen id : 8] -Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24, i_item_id#29, i_item_desc#28] -Keys [4]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24] -Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#11), partial_sum(cs_quantity#16)] -Aggregate Attributes [3]: [sum#30, sum#31, sum#32] -Results [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum#33, sum#34, sum#35] - -(40) CometColumnarExchange -Input [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum#33, sum#34, sum#35] -Arguments: hashpartitioning(i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(41) CometColumnarToRow [codegen id : 9] -Input [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum#33, sum#34, sum#35] - -(42) HashAggregate [codegen id : 9] -Input [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum#33, sum#34, sum#35] -Keys [4]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24] -Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity#16)] -Aggregate Attributes [3]: [sum(ss_quantity#5)#36, sum(sr_return_quantity#11)#37, sum(cs_quantity#16)#38] -Results [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum(ss_quantity#5)#36 AS store_sales_quantity#39, sum(sr_return_quantity#11)#37 AS store_returns_quantity#40, sum(cs_quantity#16)#38 AS catalog_sales_quantity#41] - -(43) TakeOrderedAndProject -Input [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, store_sales_quantity#39, store_returns_quantity#40, catalog_sales_quantity#41] -Arguments: 100, [i_item_id#29 ASC NULLS FIRST, i_item_desc#28 ASC NULLS FIRST, s_store_id#25 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST], [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, store_sales_quantity#39, store_returns_quantity#40, catalog_sales_quantity#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometNativeScan parquet spark_catalog.default.date_dim (44) - - -(44) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#42, d_moy#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [3]: [d_date_sk#19, d_year#42, d_moy#43] -Condition : ((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 = 9)) AND (d_year#42 = 1999)) AND isnotnull(d_date_sk#19)) - -(46) CometProject -Input [3]: [d_date_sk#19, d_year#42, d_moy#43] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(48) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometNativeScan parquet spark_catalog.default.date_dim (49) - - -(49) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#20, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#20, d_year#44, d_moy#45] -Condition : (((((isnotnull(d_moy#45) AND isnotnull(d_year#44)) AND (d_moy#45 >= 9)) AND (d_moy#45 <= 12)) AND (d_year#44 = 1999)) AND isnotnull(d_date_sk#20)) - -(51) CometProject -Input [3]: [d_date_sk#20, d_year#44, d_moy#45] -Arguments: [d_date_sk#20], [d_date_sk#20] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#20] - -(53) BroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 -BroadcastExchange (58) -+- * CometColumnarToRow (57) - +- CometProject (56) - +- CometFilter (55) - +- CometNativeScan parquet spark_catalog.default.date_dim (54) - - -(54) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#21, d_year#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(55) CometFilter -Input [2]: [d_date_sk#21, d_year#46] -Condition : (d_year#46 IN (1999,2000,2001) AND isnotnull(d_date_sk#21)) - -(56) CometProject -Input [2]: [d_date_sk#21, d_year#46] -Arguments: [d_date_sk#21], [d_date_sk#21] - -(57) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#21] - -(58) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/simplified.txt deleted file mode 100644 index 54bdbeeb5a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/simplified.txt +++ /dev/null @@ -1,86 +0,0 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] - WholeStageCodegen (9) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity),store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - WholeStageCodegen (8) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] [sum,sum,sum,sum,sum,sum] - Project [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [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] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Filter [cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometNativeScan 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/q29.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/explain.txt deleted file mode 100644 index 88c51b72ee..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,366 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometTakeOrderedAndProject (46) - +- CometHashAggregate (45) - +- CometExchange (44) - +- CometHashAggregate (43) - +- CometProject (42) - +- CometBroadcastHashJoin (41) - :- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometProject (18) - : : : : +- CometBroadcastHashJoin (17) - : : : : :- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (8) - : : : : +- CometBroadcastExchange (16) - : : : : +- CometProject (15) - : : : : +- CometFilter (14) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : : : +- CometBroadcastExchange (22) - : : : +- CometProject (21) - : : : +- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - : : +- CometBroadcastExchange (28) - : : +- CometProject (27) - : : +- CometFilter (26) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - : +- CometBroadcastExchange (34) - : +- CometProject (33) - : +- CometFilter (32) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (31) - +- CometBroadcastExchange (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (37) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(5) CometBroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(6) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(10) CometBroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(11) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] -Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight - -(12) CometProject -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9)) AND (d_year#20 = 1999)) AND isnotnull(d_date_sk#19)) - -(15) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 9)) AND (d_moy#24 <= 12)) AND (d_year#23 = 1999)) AND isnotnull(d_date_sk#22)) - -(21) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#22] -Arguments: [sr_returned_date_sk#12], [d_date_sk#22], Inner, BuildRight - -(24) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#22] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) - -(27) CometProject -Input [2]: [d_date_sk#25, d_year#26] -Arguments: [d_date_sk#25], [d_date_sk#25] - -(28) CometBroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: [d_date_sk#25] - -(29) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#25] -Arguments: [cs_sold_date_sk#17], [d_date_sk#25], Inner, BuildRight - -(30) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#25] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] -Condition : isnotnull(s_store_sk#27) - -(33) CometProject -Input [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] -Arguments: [s_store_sk#27, s_store_id#30, s_store_name#29], [s_store_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#28, 16, true, false, true) AS s_store_id#30, s_store_name#29] - -(34) CometBroadcastExchange -Input [3]: [s_store_sk#27, s_store_id#30, s_store_name#29] -Arguments: [s_store_sk#27, s_store_id#30, s_store_name#29] - -(35) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Right output [3]: [s_store_sk#27, s_store_id#30, s_store_name#29] -Arguments: [ss_store_sk#3], [s_store_sk#27], Inner, BuildRight - -(36) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#27, s_store_id#30, s_store_name#29] -Arguments: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29], [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(38) CometFilter -Input [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Condition : isnotnull(i_item_sk#31) - -(39) CometProject -Input [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Arguments: [i_item_sk#31, i_item_id#34, i_item_desc#33], [i_item_sk#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#32, 16, true, false, true) AS i_item_id#34, i_item_desc#33] - -(40) CometBroadcastExchange -Input [3]: [i_item_sk#31, i_item_id#34, i_item_desc#33] -Arguments: [i_item_sk#31, i_item_id#34, i_item_desc#33] - -(41) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29] -Right output [3]: [i_item_sk#31, i_item_id#34, i_item_desc#33] -Arguments: [ss_item_sk#1], [i_item_sk#31], Inner, BuildRight - -(42) CometProject -Input [9]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_sk#31, i_item_id#34, i_item_desc#33] -Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_id#34, i_item_desc#33], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_id#34, i_item_desc#33] - -(43) CometHashAggregate -Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_id#34, i_item_desc#33] -Keys [4]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29] -Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#11), partial_sum(cs_quantity#16)] - -(44) CometExchange -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, sum#35, sum#36, sum#37] -Arguments: hashpartitioning(i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(45) CometHashAggregate -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, sum#35, sum#36, sum#37] -Keys [4]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29] -Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity#16)] - -(46) CometTakeOrderedAndProject -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#34 ASC NULLS FIRST,i_item_desc#33 ASC NULLS FIRST,s_store_id#30 ASC NULLS FIRST,s_store_name#29 ASC NULLS FIRST], output=[i_item_id#34,i_item_desc#33,s_store_id#30,s_store_name#29,store_sales_quantity#38,store_returns_quantity#39,catalog_sales_quantity#40]), [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40], 100, 0, [i_item_id#34 ASC NULLS FIRST, i_item_desc#33 ASC NULLS FIRST, s_store_id#30 ASC NULLS FIRST, s_store_name#29 ASC NULLS FIRST], [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40] - -(47) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9)) AND (d_year#20 = 1999)) AND isnotnull(d_date_sk#19)) - -(50) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(52) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 9)) AND (d_moy#24 <= 12)) AND (d_year#23 = 1999)) AND isnotnull(d_date_sk#22)) - -(55) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(56) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(57) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 -BroadcastExchange (62) -+- * CometColumnarToRow (61) - +- CometProject (60) - +- CometFilter (59) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) - - -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(59) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) - -(60) CometProject -Input [2]: [d_date_sk#25, d_year#26] -Arguments: [d_date_sk#25], [d_date_sk#25] - -(61) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#25] - -(62) BroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/simplified.txt deleted file mode 100644 index 15b992d3a4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] - CometHashAggregate [sum,sum,sum] [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity)] - CometExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - CometHashAggregate [ss_quantity,sr_return_quantity,cs_quantity] [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] - CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - 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_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 [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 [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_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 [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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #8 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #9 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan [native_iceberg_compat] 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/q29/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/explain.txt deleted file mode 100644 index 88c51b72ee..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/explain.txt +++ /dev/null @@ -1,366 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometTakeOrderedAndProject (46) - +- CometHashAggregate (45) - +- CometExchange (44) - +- CometHashAggregate (43) - +- CometProject (42) - +- CometBroadcastHashJoin (41) - :- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometProject (18) - : : : : +- CometBroadcastHashJoin (17) - : : : : :- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (8) - : : : : +- CometBroadcastExchange (16) - : : : : +- CometProject (15) - : : : : +- CometFilter (14) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : : : +- CometBroadcastExchange (22) - : : : +- CometProject (21) - : : : +- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - : : +- CometBroadcastExchange (28) - : : +- CometProject (27) - : : +- CometFilter (26) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - : +- CometBroadcastExchange (34) - : +- CometProject (33) - : +- CometFilter (32) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (31) - +- CometBroadcastExchange (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (37) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(5) CometBroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(6) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(10) CometBroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(11) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] -Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight - -(12) CometProject -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9)) AND (d_year#20 = 1999)) AND isnotnull(d_date_sk#19)) - -(15) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 9)) AND (d_moy#24 <= 12)) AND (d_year#23 = 1999)) AND isnotnull(d_date_sk#22)) - -(21) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#22] -Arguments: [sr_returned_date_sk#12], [d_date_sk#22], Inner, BuildRight - -(24) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#22] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) - -(27) CometProject -Input [2]: [d_date_sk#25, d_year#26] -Arguments: [d_date_sk#25], [d_date_sk#25] - -(28) CometBroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: [d_date_sk#25] - -(29) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#25] -Arguments: [cs_sold_date_sk#17], [d_date_sk#25], Inner, BuildRight - -(30) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#25] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] -Condition : isnotnull(s_store_sk#27) - -(33) CometProject -Input [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] -Arguments: [s_store_sk#27, s_store_id#30, s_store_name#29], [s_store_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#28, 16, true, false, true) AS s_store_id#30, s_store_name#29] - -(34) CometBroadcastExchange -Input [3]: [s_store_sk#27, s_store_id#30, s_store_name#29] -Arguments: [s_store_sk#27, s_store_id#30, s_store_name#29] - -(35) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Right output [3]: [s_store_sk#27, s_store_id#30, s_store_name#29] -Arguments: [ss_store_sk#3], [s_store_sk#27], Inner, BuildRight - -(36) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#27, s_store_id#30, s_store_name#29] -Arguments: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29], [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(38) CometFilter -Input [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Condition : isnotnull(i_item_sk#31) - -(39) CometProject -Input [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Arguments: [i_item_sk#31, i_item_id#34, i_item_desc#33], [i_item_sk#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#32, 16, true, false, true) AS i_item_id#34, i_item_desc#33] - -(40) CometBroadcastExchange -Input [3]: [i_item_sk#31, i_item_id#34, i_item_desc#33] -Arguments: [i_item_sk#31, i_item_id#34, i_item_desc#33] - -(41) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29] -Right output [3]: [i_item_sk#31, i_item_id#34, i_item_desc#33] -Arguments: [ss_item_sk#1], [i_item_sk#31], Inner, BuildRight - -(42) CometProject -Input [9]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_sk#31, i_item_id#34, i_item_desc#33] -Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_id#34, i_item_desc#33], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_id#34, i_item_desc#33] - -(43) CometHashAggregate -Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_id#34, i_item_desc#33] -Keys [4]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29] -Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#11), partial_sum(cs_quantity#16)] - -(44) CometExchange -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, sum#35, sum#36, sum#37] -Arguments: hashpartitioning(i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(45) CometHashAggregate -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, sum#35, sum#36, sum#37] -Keys [4]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29] -Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity#16)] - -(46) CometTakeOrderedAndProject -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#34 ASC NULLS FIRST,i_item_desc#33 ASC NULLS FIRST,s_store_id#30 ASC NULLS FIRST,s_store_name#29 ASC NULLS FIRST], output=[i_item_id#34,i_item_desc#33,s_store_id#30,s_store_name#29,store_sales_quantity#38,store_returns_quantity#39,catalog_sales_quantity#40]), [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40], 100, 0, [i_item_id#34 ASC NULLS FIRST, i_item_desc#33 ASC NULLS FIRST, s_store_id#30 ASC NULLS FIRST, s_store_name#29 ASC NULLS FIRST], [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40] - -(47) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9)) AND (d_year#20 = 1999)) AND isnotnull(d_date_sk#19)) - -(50) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(52) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 9)) AND (d_moy#24 <= 12)) AND (d_year#23 = 1999)) AND isnotnull(d_date_sk#22)) - -(55) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(56) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(57) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 -BroadcastExchange (62) -+- * CometColumnarToRow (61) - +- CometProject (60) - +- CometFilter (59) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) - - -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(59) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) - -(60) CometProject -Input [2]: [d_date_sk#25, d_year#26] -Arguments: [d_date_sk#25], [d_date_sk#25] - -(61) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#25] - -(62) BroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/extended.txt deleted file mode 100644 index fe03fef8a7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/extended.txt +++ /dev/null @@ -1,67 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 15b992d3a4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] - CometHashAggregate [sum,sum,sum] [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity)] - CometExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - CometHashAggregate [ss_quantity,sr_return_quantity,cs_quantity] [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] - CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - 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_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 [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 [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_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 [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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #8 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #9 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_datafusion/explain.txt deleted file mode 100644 index 9a5ab78af8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_datafusion/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometNativeScan parquet spark_catalog.default.item (9) - - -(1) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((isnotnull(d_moy#3) AND (d_moy#3 = 11)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometNativeScan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,128), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Condition : ((isnotnull(i_manufact_id#10) AND (i_manufact_id#10 = 128)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,sum_agg#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,sum_agg#15]), [d_year#2, brand_id#13, brand#14, sum_agg#15], 100, 0, [d_year#2 ASC NULLS FIRST, sum_agg#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, sum_agg#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_datafusion/simplified.txt deleted file mode 100644 index c19e6caf52..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_datafusion/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,sum_agg] - CometHashAggregate [sum] [d_year,brand_id,brand,sum_agg,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] - CometNativeScan 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/q3.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_iceberg_compat/explain.txt deleted file mode 100644 index 6828b7a3aa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((isnotnull(d_moy#3) AND (d_moy#3 = 11)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,128), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Condition : ((isnotnull(i_manufact_id#10) AND (i_manufact_id#10 = 128)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,sum_agg#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,sum_agg#15]), [d_year#2, brand_id#13, brand#14, sum_agg#15], 100, 0, [d_year#2 ASC NULLS FIRST, sum_agg#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, sum_agg#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_iceberg_compat/simplified.txt deleted file mode 100644 index 7bdad4b94f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,sum_agg] - CometHashAggregate [sum] [d_year,brand_id,brand,sum_agg,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] - CometScan [native_iceberg_compat] 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/q3/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/explain.txt deleted file mode 100644 index 6828b7a3aa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((isnotnull(d_moy#3) AND (d_moy#3 = 11)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,128), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Condition : ((isnotnull(i_manufact_id#10) AND (i_manufact_id#10 = 128)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,sum_agg#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,sum_agg#15]), [d_year#2, brand_id#13, brand#14, sum_agg#15], 100, 0, [d_year#2 ASC NULLS FIRST, sum_agg#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, sum_agg#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/extended.txt deleted file mode 100644 index a292badf5d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 7bdad4b94f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q3/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,sum_agg] - CometHashAggregate [sum] [d_year,brand_id,brand,sum_agg,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/explain.txt deleted file mode 100644 index b3dd4f1415..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/explain.txt +++ /dev/null @@ -1,346 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (54) -+- * Project (53) - +- * BroadcastHashJoin Inner BuildRight (52) - :- * Project (46) - : +- * BroadcastHashJoin Inner BuildRight (45) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- * HashAggregate (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.web_returns (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (11) - : : : +- * CometColumnarToRow (10) - : : : +- CometProject (9) - : : : +- CometFilter (8) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) - : : +- BroadcastExchange (37) - : : +- * Filter (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * HashAggregate (31) - : : +- * CometColumnarToRow (30) - : : +- CometColumnarExchange (29) - : : +- * HashAggregate (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (24) - : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : :- * Filter (21) - : : : : +- * ColumnarToRow (20) - : : : : +- Scan parquet spark_catalog.default.web_returns (19) - : : : +- ReusedExchange (22) - : : +- ReusedExchange (25) - : +- BroadcastExchange (44) - : +- * CometColumnarToRow (43) - : +- CometProject (42) - : +- CometFilter (41) - : +- CometNativeScan parquet spark_catalog.default.customer (40) - +- BroadcastExchange (51) - +- * CometColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometNativeScan parquet spark_catalog.default.customer_address (47) - - -(1) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_returning_addr_sk), IsNotNull(wr_returning_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] - -(3) Filter [codegen id : 3] -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Condition : (isnotnull(wr_returning_addr_sk#2) AND isnotnull(wr_returning_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [wr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_state#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_state#8] -Condition : (isnotnull(ca_address_sk#7) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#8, 2, true, false, true))) - -(9) CometProject -Input [2]: [ca_address_sk#7, ca_state#8] -Arguments: [ca_address_sk#7, ca_state#9], [ca_address_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#8, 2, true, false, true) AS ca_state#9] - -(10) CometColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#7, ca_state#9] - -(11) BroadcastExchange -Input [2]: [ca_address_sk#7, ca_state#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [wr_returning_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#7, ca_state#9] - -(14) HashAggregate [codegen id : 3] -Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] -Keys [2]: [wr_returning_customer_sk#1, ca_state#9] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum#10] -Results [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] - -(15) CometColumnarExchange -Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 11] -Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] - -(17) HashAggregate [codegen id : 11] -Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] -Keys [2]: [wr_returning_customer_sk#1, ca_state#9] -Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#12] -Results [3]: [wr_returning_customer_sk#1 AS ctr_customer_sk#13, ca_state#9 AS ctr_state#14, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#12,17,2) AS ctr_total_return#15] - -(18) Filter [codegen id : 11] -Input [3]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15] -Condition : isnotnull(ctr_total_return#15) - -(19) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#19), dynamicpruningexpression(wr_returned_date_sk#19 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_returning_addr_sk)] -ReadSchema: struct - -(20) ColumnarToRow [codegen id : 6] -Input [4]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19] - -(21) Filter [codegen id : 6] -Input [4]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19] -Condition : isnotnull(wr_returning_addr_sk#17) - -(22) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#20] - -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returned_date_sk#19] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18] -Input [5]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19, d_date_sk#20] - -(25) ReusedExchange [Reuses operator id: 11] -Output [2]: [ca_address_sk#21, ca_state#9] - -(26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returning_addr_sk#17] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#9] -Input [5]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, ca_address_sk#21, ca_state#9] - -(28) HashAggregate [codegen id : 6] -Input [3]: [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#9] -Keys [2]: [wr_returning_customer_sk#16, ca_state#9] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#18))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [wr_returning_customer_sk#16, ca_state#9, sum#23] - -(29) CometColumnarExchange -Input [3]: [wr_returning_customer_sk#16, ca_state#9, sum#23] -Arguments: hashpartitioning(wr_returning_customer_sk#16, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(30) CometColumnarToRow [codegen id : 7] -Input [3]: [wr_returning_customer_sk#16, ca_state#9, sum#23] - -(31) HashAggregate [codegen id : 7] -Input [3]: [wr_returning_customer_sk#16, ca_state#9, sum#23] -Keys [2]: [wr_returning_customer_sk#16, ca_state#9] -Functions [1]: [sum(UnscaledValue(wr_return_amt#18))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#18))#12] -Results [2]: [ca_state#9 AS ctr_state#24, MakeDecimal(sum(UnscaledValue(wr_return_amt#18))#12,17,2) AS ctr_total_return#25] - -(32) HashAggregate [codegen id : 7] -Input [2]: [ctr_state#24, ctr_total_return#25] -Keys [1]: [ctr_state#24] -Functions [1]: [partial_avg(ctr_total_return#25)] -Aggregate Attributes [2]: [sum#26, count#27] -Results [3]: [ctr_state#24, sum#28, count#29] - -(33) CometColumnarExchange -Input [3]: [ctr_state#24, sum#28, count#29] -Arguments: hashpartitioning(ctr_state#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 8] -Input [3]: [ctr_state#24, sum#28, count#29] - -(35) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#24, sum#28, count#29] -Keys [1]: [ctr_state#24] -Functions [1]: [avg(ctr_total_return#25)] -Aggregate Attributes [1]: [avg(ctr_total_return#25)#30] -Results [2]: [(avg(ctr_total_return#25)#30 * 1.2) AS (avg(ctr_total_return) * 1.2)#31, ctr_state#24] - -(36) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#24] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) - -(37) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#24] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] - -(38) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_state#14] -Right keys [1]: [ctr_state#24] -Join type: Inner -Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) - -(39) Project [codegen id : 11] -Output [2]: [ctr_customer_sk#13, ctr_total_return#15] -Input [5]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#31, ctr_state#24] - -(40) CometNativeScan parquet spark_catalog.default.customer -Output [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(41) CometFilter -Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#34)) - -(42) CometProject -Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] -Arguments: [c_customer_sk#32, c_customer_id#46, c_current_addr_sk#34, c_salutation#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#51, c_email_address#52, c_last_review_date#45], [c_customer_sk#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true) AS c_customer_id#46, c_current_addr_sk#34, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#35, 10, true, false, true) AS c_salutation#47, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#36, 20, true, false, true) AS c_first_name#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#37, 30, true, false, true) AS c_last_name#49, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#38, 1, true, false, true) AS c_preferred_cust_flag#50, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#43, 13, true, false, true) AS c_login#51, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#44, 50, true, false, true) AS c_email_address#52, c_last_review_date#45] - -(43) CometColumnarToRow [codegen id : 9] -Input [14]: [c_customer_sk#32, c_customer_id#46, c_current_addr_sk#34, c_salutation#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#51, c_email_address#52, c_last_review_date#45] - -(44) BroadcastExchange -Input [14]: [c_customer_sk#32, c_customer_id#46, c_current_addr_sk#34, c_salutation#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#51, c_email_address#52, c_last_review_date#45] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(45) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_customer_sk#13] -Right keys [1]: [c_customer_sk#32] -Join type: Inner -Join condition: None - -(46) Project [codegen id : 11] -Output [14]: [ctr_total_return#15, c_customer_id#46, c_current_addr_sk#34, c_salutation#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#51, c_email_address#52, c_last_review_date#45] -Input [16]: [ctr_customer_sk#13, ctr_total_return#15, c_customer_sk#32, c_customer_id#46, c_current_addr_sk#34, c_salutation#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#51, c_email_address#52, c_last_review_date#45] - -(47) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#53, ca_state#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(48) CometFilter -Input [2]: [ca_address_sk#53, ca_state#54] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#54, 2, true, false, true) = GA) AND isnotnull(ca_address_sk#53)) - -(49) CometProject -Input [2]: [ca_address_sk#53, ca_state#54] -Arguments: [ca_address_sk#53], [ca_address_sk#53] - -(50) CometColumnarToRow [codegen id : 10] -Input [1]: [ca_address_sk#53] - -(51) BroadcastExchange -Input [1]: [ca_address_sk#53] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(52) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#34] -Right keys [1]: [ca_address_sk#53] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 11] -Output [13]: [c_customer_id#46, c_salutation#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#51, c_email_address#52, c_last_review_date#45, ctr_total_return#15] -Input [15]: [ctr_total_return#15, c_customer_id#46, c_current_addr_sk#34, c_salutation#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#51, c_email_address#52, c_last_review_date#45, ca_address_sk#53] - -(54) TakeOrderedAndProject -Input [13]: [c_customer_id#46, c_salutation#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#51, c_email_address#52, c_last_review_date#45, ctr_total_return#15] -Arguments: 100, [c_customer_id#46 ASC NULLS FIRST, c_salutation#47 ASC NULLS FIRST, c_first_name#48 ASC NULLS FIRST, c_last_name#49 ASC NULLS FIRST, c_preferred_cust_flag#50 ASC NULLS FIRST, c_birth_day#39 ASC NULLS FIRST, c_birth_month#40 ASC NULLS FIRST, c_birth_year#41 ASC NULLS FIRST, c_birth_country#42 ASC NULLS FIRST, c_login#51 ASC NULLS FIRST, c_email_address#52 ASC NULLS FIRST, c_last_review_date#45 ASC NULLS FIRST, ctr_total_return#15 ASC NULLS FIRST], [c_customer_id#46, c_salutation#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#51, c_email_address#52, c_last_review_date#45, ctr_total_return#15] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (59) -+- * CometColumnarToRow (58) - +- CometProject (57) - +- CometFilter (56) - +- CometNativeScan parquet spark_catalog.default.date_dim (55) - - -(55) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date_sk#6, d_year#55] -Condition : ((isnotnull(d_year#55) AND (d_year#55 = 2002)) AND isnotnull(d_date_sk#6)) - -(57) CometProject -Input [2]: [d_date_sk#6, d_year#55] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(58) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(59) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 19 Hosting Expression = wr_returned_date_sk#19 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/simplified.txt deleted file mode 100644 index 72b6fbd1ba..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/simplified.txt +++ /dev/null @@ -1,86 +0,0 @@ -TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - WholeStageCodegen (11) - Project [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (3) - HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] - Project [wr_returning_customer_sk,wr_return_amt,ca_state] - BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] - Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Filter [wr_returning_addr_sk,wr_returning_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (8) - Filter [(avg(ctr_total_return) * 1.2)] - HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ctr_state] #5 - WholeStageCodegen (7) - HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] - HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wr_returning_customer_sk,ca_state] #6 - WholeStageCodegen (6) - HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] - Project [wr_returning_customer_sk,wr_return_amt,ca_state] - BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] - Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Filter [wr_returning_addr_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [ca_address_sk,ca_state] #3 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [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] - 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] - CometNativeScan 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 #8 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan 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/q30.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/explain.txt deleted file mode 100644 index 4c7261693d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,318 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (50) -+- CometTakeOrderedAndProject (49) - +- CometProject (48) - +- CometBroadcastHashJoin (47) - :- CometProject (42) - : +- CometBroadcastHashJoin (41) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometFilter (18) - : : : +- CometHashAggregate (17) - : : : +- CometExchange (16) - : : : +- CometHashAggregate (15) - : : : +- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : : +- CometBroadcastExchange (34) - : : +- CometFilter (33) - : : +- CometHashAggregate (32) - : : +- CometExchange (31) - : : +- CometHashAggregate (30) - : : +- CometHashAggregate (29) - : : +- CometExchange (28) - : : +- CometHashAggregate (27) - : : +- CometProject (26) - : : +- CometBroadcastHashJoin (25) - : : :- CometProject (23) - : : : +- CometBroadcastHashJoin (22) - : : : :- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (19) - : : : +- ReusedExchange (21) - : : +- ReusedExchange (24) - : +- CometBroadcastExchange (40) - : +- CometProject (39) - : +- CometFilter (38) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) - +- CometBroadcastExchange (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (43) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_returning_addr_sk), IsNotNull(wr_returning_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Condition : (isnotnull(wr_returning_addr_sk#2) AND isnotnull(wr_returning_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [wr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6] -Arguments: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3], [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#8, ca_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#8, ca_state#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#9, 2, true, false, true))) - -(11) CometProject -Input [2]: [ca_address_sk#8, ca_state#9] -Arguments: [ca_address_sk#8, ca_state#10], [ca_address_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#9, 2, true, false, true) AS ca_state#10] - -(12) CometBroadcastExchange -Input [2]: [ca_address_sk#8, ca_state#10] -Arguments: [ca_address_sk#8, ca_state#10] - -(13) CometBroadcastHashJoin -Left output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] -Right output [2]: [ca_address_sk#8, ca_state#10] -Arguments: [wr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight - -(14) CometProject -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#8, ca_state#10] -Arguments: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10], [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] - -(15) CometHashAggregate -Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] -Keys [2]: [wr_returning_customer_sk#1, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] - -(16) CometExchange -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#11] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#11] -Keys [2]: [wr_returning_customer_sk#1, ca_state#10] -Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] - -(18) CometFilter -Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Condition : isnotnull(ctr_total_return#14) - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#18), dynamicpruningexpression(wr_returned_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(wr_returning_addr_sk)] -ReadSchema: struct - -(20) CometFilter -Input [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] -Condition : isnotnull(wr_returning_addr_sk#16) - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#20] - -(22) CometBroadcastHashJoin -Left output [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] -Right output [1]: [d_date_sk#20] -Arguments: [wr_returned_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(23) CometProject -Input [5]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18, d_date_sk#20] -Arguments: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17], [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17] - -(24) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#21, ca_state#10] - -(25) CometBroadcastHashJoin -Left output [3]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17] -Right output [2]: [ca_address_sk#21, ca_state#10] -Arguments: [wr_returning_addr_sk#16], [ca_address_sk#21], Inner, BuildRight - -(26) CometProject -Input [5]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, ca_address_sk#21, ca_state#10] -Arguments: [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#10], [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#10] - -(27) CometHashAggregate -Input [3]: [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#10] -Keys [2]: [wr_returning_customer_sk#15, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#17))] - -(28) CometExchange -Input [3]: [wr_returning_customer_sk#15, ca_state#10, sum#22] -Arguments: hashpartitioning(wr_returning_customer_sk#15, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [3]: [wr_returning_customer_sk#15, ca_state#10, sum#22] -Keys [2]: [wr_returning_customer_sk#15, ca_state#10] -Functions [1]: [sum(UnscaledValue(wr_return_amt#17))] - -(30) CometHashAggregate -Input [2]: [ctr_state#23, ctr_total_return#24] -Keys [1]: [ctr_state#23] -Functions [1]: [partial_avg(ctr_total_return#24)] - -(31) CometExchange -Input [3]: [ctr_state#23, sum#25, count#26] -Arguments: hashpartitioning(ctr_state#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometHashAggregate -Input [3]: [ctr_state#23, sum#25, count#26] -Keys [1]: [ctr_state#23] -Functions [1]: [avg(ctr_total_return#24)] - -(33) CometFilter -Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#27) - -(34) CometBroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] -Arguments: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] - -(35) CometBroadcastHashJoin -Left output [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Right output [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] -Arguments: [ctr_state#13], [ctr_state#23], Inner, (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#27), BuildRight - -(36) CometProject -Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#27, ctr_state#23] -Arguments: [ctr_customer_sk#12, ctr_total_return#14], [ctr_customer_sk#12, ctr_total_return#14] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [14]: [c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(38) CometFilter -Input [14]: [c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41] -Condition : (isnotnull(c_customer_sk#28) AND isnotnull(c_current_addr_sk#30)) - -(39) CometProject -Input [14]: [c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41] -Arguments: [c_customer_sk#28, c_customer_id#42, c_current_addr_sk#30, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41], [c_customer_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#29, 16, true, false, true) AS c_customer_id#42, c_current_addr_sk#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#31, 10, true, false, true) AS c_salutation#43, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#32, 20, true, false, true) AS c_first_name#44, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#33, 30, true, false, true) AS c_last_name#45, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#34, 1, true, false, true) AS c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#39, 13, true, false, true) AS c_login#47, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#40, 50, true, false, true) AS c_email_address#48, c_last_review_date#41] - -(40) CometBroadcastExchange -Input [14]: [c_customer_sk#28, c_customer_id#42, c_current_addr_sk#30, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41] -Arguments: [c_customer_sk#28, c_customer_id#42, c_current_addr_sk#30, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41] - -(41) CometBroadcastHashJoin -Left output [2]: [ctr_customer_sk#12, ctr_total_return#14] -Right output [14]: [c_customer_sk#28, c_customer_id#42, c_current_addr_sk#30, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41] -Arguments: [ctr_customer_sk#12], [c_customer_sk#28], Inner, BuildRight - -(42) CometProject -Input [16]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#28, c_customer_id#42, c_current_addr_sk#30, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41] -Arguments: [ctr_total_return#14, c_customer_id#42, c_current_addr_sk#30, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41], [ctr_total_return#14, c_customer_id#42, c_current_addr_sk#30, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#49, ca_state#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(44) CometFilter -Input [2]: [ca_address_sk#49, ca_state#50] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#50, 2, true, false, true) = GA) AND isnotnull(ca_address_sk#49)) - -(45) CometProject -Input [2]: [ca_address_sk#49, ca_state#50] -Arguments: [ca_address_sk#49], [ca_address_sk#49] - -(46) CometBroadcastExchange -Input [1]: [ca_address_sk#49] -Arguments: [ca_address_sk#49] - -(47) CometBroadcastHashJoin -Left output [14]: [ctr_total_return#14, c_customer_id#42, c_current_addr_sk#30, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41] -Right output [1]: [ca_address_sk#49] -Arguments: [c_current_addr_sk#30], [ca_address_sk#49], Inner, BuildRight - -(48) CometProject -Input [15]: [ctr_total_return#14, c_customer_id#42, c_current_addr_sk#30, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41, ca_address_sk#49] -Arguments: [c_customer_id#42, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41, ctr_total_return#14], [c_customer_id#42, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41, ctr_total_return#14] - -(49) CometTakeOrderedAndProject -Input [13]: [c_customer_id#42, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41, ctr_total_return#14] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#42 ASC NULLS FIRST,c_salutation#43 ASC NULLS FIRST,c_first_name#44 ASC NULLS FIRST,c_last_name#45 ASC NULLS FIRST,c_preferred_cust_flag#46 ASC NULLS FIRST,c_birth_day#35 ASC NULLS FIRST,c_birth_month#36 ASC NULLS FIRST,c_birth_year#37 ASC NULLS FIRST,c_birth_country#38 ASC NULLS FIRST,c_login#47 ASC NULLS FIRST,c_email_address#48 ASC NULLS FIRST,c_last_review_date#41 ASC NULLS FIRST,ctr_total_return#14 ASC NULLS FIRST], output=[c_customer_id#42,c_salutation#43,c_first_name#44,c_last_name#45,c_preferred_cust_flag#46,c_birth_day#35,c_birth_month#36,c_birth_year#37,c_birth_country#38,c_login#47,c_email_address#48,c_last_review_date#41,ctr_total_return#14]), [c_customer_id#42, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41, ctr_total_return#14], 100, 0, [c_customer_id#42 ASC NULLS FIRST, c_salutation#43 ASC NULLS FIRST, c_first_name#44 ASC NULLS FIRST, c_last_name#45 ASC NULLS FIRST, c_preferred_cust_flag#46 ASC NULLS FIRST, c_birth_day#35 ASC NULLS FIRST, c_birth_month#36 ASC NULLS FIRST, c_birth_year#37 ASC NULLS FIRST, c_birth_country#38 ASC NULLS FIRST, c_login#47 ASC NULLS FIRST, c_email_address#48 ASC NULLS FIRST, c_last_review_date#41 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#42, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41, ctr_total_return#14] - -(50) CometColumnarToRow [codegen id : 1] -Input [13]: [c_customer_id#42, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41, ctr_total_return#14] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) - - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(52) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) - -(53) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(54) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(55) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 19 Hosting Expression = wr_returned_date_sk#18 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/simplified.txt deleted file mode 100644 index 0227534fa5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - CometBroadcastHashJoin [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ca_address_sk] - CometProject [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - CometBroadcastHashJoin [ctr_customer_sk,ctr_total_return,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] - CometProject [ctr_customer_sk,ctr_total_return] - CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] - CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] - CometHashAggregate [sum] [ctr_customer_sk,ctr_state,ctr_total_return,wr_returning_customer_sk,ca_state,sum(UnscaledValue(wr_return_amt))] - CometExchange [wr_returning_customer_sk,ca_state] #1 - CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] - CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 - CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] - CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_state,avg(ctr_total_return)] - CometExchange [ctr_state] #6 - CometHashAggregate [ctr_total_return] [ctr_state,sum,count] - CometHashAggregate [wr_returning_customer_sk,sum] [ctr_state,ctr_total_return,ca_state,sum(UnscaledValue(wr_return_amt))] - CometExchange [wr_returning_customer_sk,ca_state] #7 - CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] - CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 - ReusedExchange [ca_address_sk,ca_state] #4 - CometBroadcastExchange [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] #8 - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [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] - 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 [native_iceberg_compat] 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] - CometBroadcastExchange [ca_address_sk] #9 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] 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/q30/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/explain.txt deleted file mode 100644 index 4c7261693d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/explain.txt +++ /dev/null @@ -1,318 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (50) -+- CometTakeOrderedAndProject (49) - +- CometProject (48) - +- CometBroadcastHashJoin (47) - :- CometProject (42) - : +- CometBroadcastHashJoin (41) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometFilter (18) - : : : +- CometHashAggregate (17) - : : : +- CometExchange (16) - : : : +- CometHashAggregate (15) - : : : +- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : : +- CometBroadcastExchange (34) - : : +- CometFilter (33) - : : +- CometHashAggregate (32) - : : +- CometExchange (31) - : : +- CometHashAggregate (30) - : : +- CometHashAggregate (29) - : : +- CometExchange (28) - : : +- CometHashAggregate (27) - : : +- CometProject (26) - : : +- CometBroadcastHashJoin (25) - : : :- CometProject (23) - : : : +- CometBroadcastHashJoin (22) - : : : :- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (19) - : : : +- ReusedExchange (21) - : : +- ReusedExchange (24) - : +- CometBroadcastExchange (40) - : +- CometProject (39) - : +- CometFilter (38) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) - +- CometBroadcastExchange (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (43) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_returning_addr_sk), IsNotNull(wr_returning_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Condition : (isnotnull(wr_returning_addr_sk#2) AND isnotnull(wr_returning_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [wr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6] -Arguments: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3], [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#8, ca_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#8, ca_state#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#9, 2, true, false, true))) - -(11) CometProject -Input [2]: [ca_address_sk#8, ca_state#9] -Arguments: [ca_address_sk#8, ca_state#10], [ca_address_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#9, 2, true, false, true) AS ca_state#10] - -(12) CometBroadcastExchange -Input [2]: [ca_address_sk#8, ca_state#10] -Arguments: [ca_address_sk#8, ca_state#10] - -(13) CometBroadcastHashJoin -Left output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] -Right output [2]: [ca_address_sk#8, ca_state#10] -Arguments: [wr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight - -(14) CometProject -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#8, ca_state#10] -Arguments: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10], [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] - -(15) CometHashAggregate -Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] -Keys [2]: [wr_returning_customer_sk#1, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] - -(16) CometExchange -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#11] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#11] -Keys [2]: [wr_returning_customer_sk#1, ca_state#10] -Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] - -(18) CometFilter -Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Condition : isnotnull(ctr_total_return#14) - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#18), dynamicpruningexpression(wr_returned_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(wr_returning_addr_sk)] -ReadSchema: struct - -(20) CometFilter -Input [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] -Condition : isnotnull(wr_returning_addr_sk#16) - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#20] - -(22) CometBroadcastHashJoin -Left output [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] -Right output [1]: [d_date_sk#20] -Arguments: [wr_returned_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(23) CometProject -Input [5]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18, d_date_sk#20] -Arguments: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17], [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17] - -(24) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#21, ca_state#10] - -(25) CometBroadcastHashJoin -Left output [3]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17] -Right output [2]: [ca_address_sk#21, ca_state#10] -Arguments: [wr_returning_addr_sk#16], [ca_address_sk#21], Inner, BuildRight - -(26) CometProject -Input [5]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, ca_address_sk#21, ca_state#10] -Arguments: [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#10], [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#10] - -(27) CometHashAggregate -Input [3]: [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#10] -Keys [2]: [wr_returning_customer_sk#15, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#17))] - -(28) CometExchange -Input [3]: [wr_returning_customer_sk#15, ca_state#10, sum#22] -Arguments: hashpartitioning(wr_returning_customer_sk#15, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [3]: [wr_returning_customer_sk#15, ca_state#10, sum#22] -Keys [2]: [wr_returning_customer_sk#15, ca_state#10] -Functions [1]: [sum(UnscaledValue(wr_return_amt#17))] - -(30) CometHashAggregate -Input [2]: [ctr_state#23, ctr_total_return#24] -Keys [1]: [ctr_state#23] -Functions [1]: [partial_avg(ctr_total_return#24)] - -(31) CometExchange -Input [3]: [ctr_state#23, sum#25, count#26] -Arguments: hashpartitioning(ctr_state#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometHashAggregate -Input [3]: [ctr_state#23, sum#25, count#26] -Keys [1]: [ctr_state#23] -Functions [1]: [avg(ctr_total_return#24)] - -(33) CometFilter -Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#27) - -(34) CometBroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] -Arguments: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] - -(35) CometBroadcastHashJoin -Left output [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Right output [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] -Arguments: [ctr_state#13], [ctr_state#23], Inner, (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#27), BuildRight - -(36) CometProject -Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#27, ctr_state#23] -Arguments: [ctr_customer_sk#12, ctr_total_return#14], [ctr_customer_sk#12, ctr_total_return#14] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [14]: [c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(38) CometFilter -Input [14]: [c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41] -Condition : (isnotnull(c_customer_sk#28) AND isnotnull(c_current_addr_sk#30)) - -(39) CometProject -Input [14]: [c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33, c_preferred_cust_flag#34, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#39, c_email_address#40, c_last_review_date#41] -Arguments: [c_customer_sk#28, c_customer_id#42, c_current_addr_sk#30, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41], [c_customer_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#29, 16, true, false, true) AS c_customer_id#42, c_current_addr_sk#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#31, 10, true, false, true) AS c_salutation#43, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#32, 20, true, false, true) AS c_first_name#44, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#33, 30, true, false, true) AS c_last_name#45, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#34, 1, true, false, true) AS c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#39, 13, true, false, true) AS c_login#47, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#40, 50, true, false, true) AS c_email_address#48, c_last_review_date#41] - -(40) CometBroadcastExchange -Input [14]: [c_customer_sk#28, c_customer_id#42, c_current_addr_sk#30, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41] -Arguments: [c_customer_sk#28, c_customer_id#42, c_current_addr_sk#30, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41] - -(41) CometBroadcastHashJoin -Left output [2]: [ctr_customer_sk#12, ctr_total_return#14] -Right output [14]: [c_customer_sk#28, c_customer_id#42, c_current_addr_sk#30, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41] -Arguments: [ctr_customer_sk#12], [c_customer_sk#28], Inner, BuildRight - -(42) CometProject -Input [16]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#28, c_customer_id#42, c_current_addr_sk#30, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41] -Arguments: [ctr_total_return#14, c_customer_id#42, c_current_addr_sk#30, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41], [ctr_total_return#14, c_customer_id#42, c_current_addr_sk#30, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#49, ca_state#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(44) CometFilter -Input [2]: [ca_address_sk#49, ca_state#50] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#50, 2, true, false, true) = GA) AND isnotnull(ca_address_sk#49)) - -(45) CometProject -Input [2]: [ca_address_sk#49, ca_state#50] -Arguments: [ca_address_sk#49], [ca_address_sk#49] - -(46) CometBroadcastExchange -Input [1]: [ca_address_sk#49] -Arguments: [ca_address_sk#49] - -(47) CometBroadcastHashJoin -Left output [14]: [ctr_total_return#14, c_customer_id#42, c_current_addr_sk#30, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41] -Right output [1]: [ca_address_sk#49] -Arguments: [c_current_addr_sk#30], [ca_address_sk#49], Inner, BuildRight - -(48) CometProject -Input [15]: [ctr_total_return#14, c_customer_id#42, c_current_addr_sk#30, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41, ca_address_sk#49] -Arguments: [c_customer_id#42, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41, ctr_total_return#14], [c_customer_id#42, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41, ctr_total_return#14] - -(49) CometTakeOrderedAndProject -Input [13]: [c_customer_id#42, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41, ctr_total_return#14] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#42 ASC NULLS FIRST,c_salutation#43 ASC NULLS FIRST,c_first_name#44 ASC NULLS FIRST,c_last_name#45 ASC NULLS FIRST,c_preferred_cust_flag#46 ASC NULLS FIRST,c_birth_day#35 ASC NULLS FIRST,c_birth_month#36 ASC NULLS FIRST,c_birth_year#37 ASC NULLS FIRST,c_birth_country#38 ASC NULLS FIRST,c_login#47 ASC NULLS FIRST,c_email_address#48 ASC NULLS FIRST,c_last_review_date#41 ASC NULLS FIRST,ctr_total_return#14 ASC NULLS FIRST], output=[c_customer_id#42,c_salutation#43,c_first_name#44,c_last_name#45,c_preferred_cust_flag#46,c_birth_day#35,c_birth_month#36,c_birth_year#37,c_birth_country#38,c_login#47,c_email_address#48,c_last_review_date#41,ctr_total_return#14]), [c_customer_id#42, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41, ctr_total_return#14], 100, 0, [c_customer_id#42 ASC NULLS FIRST, c_salutation#43 ASC NULLS FIRST, c_first_name#44 ASC NULLS FIRST, c_last_name#45 ASC NULLS FIRST, c_preferred_cust_flag#46 ASC NULLS FIRST, c_birth_day#35 ASC NULLS FIRST, c_birth_month#36 ASC NULLS FIRST, c_birth_year#37 ASC NULLS FIRST, c_birth_country#38 ASC NULLS FIRST, c_login#47 ASC NULLS FIRST, c_email_address#48 ASC NULLS FIRST, c_last_review_date#41 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#42, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41, ctr_total_return#14] - -(50) CometColumnarToRow [codegen id : 1] -Input [13]: [c_customer_id#42, c_salutation#43, c_first_name#44, c_last_name#45, c_preferred_cust_flag#46, c_birth_day#35, c_birth_month#36, c_birth_year#37, c_birth_country#38, c_login#47, c_email_address#48, c_last_review_date#41, ctr_total_return#14] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) - - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(52) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) - -(53) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(54) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(55) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 19 Hosting Expression = wr_returned_date_sk#18 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/extended.txt deleted file mode 100644 index 7ebd69d340..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 0227534fa5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - CometBroadcastHashJoin [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ca_address_sk] - CometProject [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - CometBroadcastHashJoin [ctr_customer_sk,ctr_total_return,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] - CometProject [ctr_customer_sk,ctr_total_return] - CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] - CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] - CometHashAggregate [sum] [ctr_customer_sk,ctr_state,ctr_total_return,wr_returning_customer_sk,ca_state,sum(UnscaledValue(wr_return_amt))] - CometExchange [wr_returning_customer_sk,ca_state] #1 - CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] - CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 - CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] - CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_state,avg(ctr_total_return)] - CometExchange [ctr_state] #6 - CometHashAggregate [ctr_total_return] [ctr_state,sum,count] - CometHashAggregate [wr_returning_customer_sk,sum] [ctr_state,ctr_total_return,ca_state,sum(UnscaledValue(wr_return_amt))] - CometExchange [wr_returning_customer_sk,ca_state] #7 - CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] - CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 - ReusedExchange [ca_address_sk,ca_state] #4 - CometBroadcastExchange [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] #8 - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [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] - 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 [native_iceberg_compat] 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] - CometBroadcastExchange [ca_address_sk] #9 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/explain.txt deleted file mode 100644 index 9ce8c738e8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/explain.txt +++ /dev/null @@ -1,644 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (97) -+- CometSort (96) - +- CometColumnarExchange (95) - +- * Project (94) - +- * BroadcastHashJoin Inner BuildRight (93) - :- * Project (78) - : +- * BroadcastHashJoin Inner BuildRight (77) - : :- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (47) - : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : :- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * HashAggregate (16) - : : : : : +- * CometColumnarToRow (15) - : : : : : +- CometColumnarExchange (14) - : : : : : +- * HashAggregate (13) - : : : : : +- * Project (12) - : : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : : :- * Project (6) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : +- ReusedExchange (4) - : : : : : +- BroadcastExchange (10) - : : : : : +- * CometColumnarToRow (9) - : : : : : +- CometFilter (8) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) - : : : : +- BroadcastExchange (30) - : : : : +- * HashAggregate (29) - : : : : +- * CometColumnarToRow (28) - : : : : +- CometColumnarExchange (27) - : : : : +- * HashAggregate (26) - : : : : +- * Project (25) - : : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : : :- * Project (22) - : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : :- * Filter (19) - : : : : : : +- * ColumnarToRow (18) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (17) - : : : : : +- ReusedExchange (20) - : : : : +- ReusedExchange (23) - : : : +- BroadcastExchange (45) - : : : +- * HashAggregate (44) - : : : +- * CometColumnarToRow (43) - : : : +- CometColumnarExchange (42) - : : : +- * HashAggregate (41) - : : : +- * Project (40) - : : : +- * BroadcastHashJoin Inner BuildRight (39) - : : : :- * Project (37) - : : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : : :- * Filter (34) - : : : : : +- * ColumnarToRow (33) - : : : : : +- Scan parquet spark_catalog.default.store_sales (32) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- BroadcastExchange (61) - : : +- * HashAggregate (60) - : : +- * CometColumnarToRow (59) - : : +- CometColumnarExchange (58) - : : +- * HashAggregate (57) - : : +- * Project (56) - : : +- * BroadcastHashJoin Inner BuildRight (55) - : : :- * Project (53) - : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : :- * Filter (50) - : : : : +- * ColumnarToRow (49) - : : : : +- Scan parquet spark_catalog.default.web_sales (48) - : : : +- ReusedExchange (51) - : : +- ReusedExchange (54) - : +- BroadcastExchange (76) - : +- * HashAggregate (75) - : +- * CometColumnarToRow (74) - : +- CometColumnarExchange (73) - : +- * HashAggregate (72) - : +- * Project (71) - : +- * BroadcastHashJoin Inner BuildRight (70) - : :- * Project (68) - : : +- * BroadcastHashJoin Inner BuildRight (67) - : : :- * Filter (65) - : : : +- * ColumnarToRow (64) - : : : +- Scan parquet spark_catalog.default.web_sales (63) - : : +- ReusedExchange (66) - : +- ReusedExchange (69) - +- BroadcastExchange (92) - +- * HashAggregate (91) - +- * CometColumnarToRow (90) - +- CometColumnarExchange (89) - +- * HashAggregate (88) - +- * Project (87) - +- * BroadcastHashJoin Inner BuildRight (86) - :- * Project (84) - : +- * BroadcastHashJoin Inner BuildRight (83) - : :- * Filter (81) - : : +- * ColumnarToRow (80) - : : +- Scan parquet spark_catalog.default.web_sales (79) - : +- ReusedExchange (82) - +- ReusedExchange (85) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_addr_sk#1) - -(4) ReusedExchange [Reuses operator id: 101] -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6, d_qoy#7] - -(7) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#8, ca_county#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [ca_address_sk#8, ca_county#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_county#9)) - -(9) CometColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#8, ca_county#9] - -(10) BroadcastExchange -Input [2]: [ca_address_sk#8, ca_county#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_addr_sk#1] -Right keys [1]: [ca_address_sk#8] -Join type: Inner -Join condition: None - -(12) Project [codegen id : 3] -Output [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_address_sk#8, ca_county#9] - -(13) HashAggregate [codegen id : 3] -Input [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] -Keys [3]: [ca_county#9, d_qoy#7, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#10] -Results [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] - -(14) CometColumnarExchange -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] -Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(15) CometColumnarToRow [codegen id : 24] -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] - -(16) HashAggregate [codegen id : 24] -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] -Keys [3]: [ca_county#9, d_qoy#7, d_year#6] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#12] -Results [3]: [ca_county#9, d_year#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS store_sales#13] - -(17) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#16), dynamicpruningexpression(ss_sold_date_sk#16 IN dynamicpruning#17)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 6] -Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] - -(19) Filter [codegen id : 6] -Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] -Condition : isnotnull(ss_addr_sk#14) - -(20) ReusedExchange [Reuses operator id: 105] -Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#16] -Right keys [1]: [d_date_sk#18] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [4]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20] -Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16, d_date_sk#18, d_year#19, d_qoy#20] - -(23) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#21, ca_county#22] - -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#14] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] -Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_address_sk#21, ca_county#22] - -(26) HashAggregate [codegen id : 6] -Input [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] -Keys [3]: [ca_county#22, d_qoy#20, d_year#19] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#15))] -Aggregate Attributes [1]: [sum#23] -Results [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] - -(27) CometColumnarExchange -Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] -Arguments: hashpartitioning(ca_county#22, d_qoy#20, d_year#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometColumnarToRow [codegen id : 7] -Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] - -(29) HashAggregate [codegen id : 7] -Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] -Keys [3]: [ca_county#22, d_qoy#20, d_year#19] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#15))#12] -Results [2]: [ca_county#22, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#15))#12,17,2) AS store_sales#25] - -(30) BroadcastExchange -Input [2]: [ca_county#22, store_sales#25] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#9] -Right keys [1]: [ca_county#22] -Join type: Inner -Join condition: None - -(32) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#29)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(33) ColumnarToRow [codegen id : 10] -Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] - -(34) Filter [codegen id : 10] -Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -Condition : isnotnull(ss_addr_sk#26) - -(35) ReusedExchange [Reuses operator id: 109] -Output [3]: [d_date_sk#30, d_year#31, d_qoy#32] - -(36) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#28] -Right keys [1]: [d_date_sk#30] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 10] -Output [4]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32] -Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#30, d_year#31, d_qoy#32] - -(38) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#33, ca_county#34] - -(39) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_addr_sk#26] -Right keys [1]: [ca_address_sk#33] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 10] -Output [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] -Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_address_sk#33, ca_county#34] - -(41) HashAggregate [codegen id : 10] -Input [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] -Keys [3]: [ca_county#34, d_qoy#32, d_year#31] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum#35] -Results [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] - -(42) CometColumnarExchange -Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] -Arguments: hashpartitioning(ca_county#34, d_qoy#32, d_year#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(43) CometColumnarToRow [codegen id : 11] -Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] - -(44) HashAggregate [codegen id : 11] -Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] -Keys [3]: [ca_county#34, d_qoy#32, d_year#31] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#12] -Results [2]: [ca_county#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#12,17,2) AS store_sales#37] - -(45) BroadcastExchange -Input [2]: [ca_county#34, store_sales#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] - -(46) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#22] -Right keys [1]: [ca_county#34] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 24] -Output [5]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37] -Input [7]: [ca_county#9, d_year#6, store_sales#13, ca_county#22, store_sales#25, ca_county#34, store_sales#37] - -(48) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(49) ColumnarToRow [codegen id : 14] -Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] - -(50) Filter [codegen id : 14] -Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -Condition : isnotnull(ws_bill_addr_sk#38) - -(51) ReusedExchange [Reuses operator id: 101] -Output [3]: [d_date_sk#41, d_year#42, d_qoy#43] - -(52) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#41] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 14] -Output [4]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#42, d_qoy#43] -Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41, d_year#42, d_qoy#43] - -(54) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#44, ca_county#45] - -(55) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#38] -Right keys [1]: [ca_address_sk#44] -Join type: Inner -Join condition: None - -(56) Project [codegen id : 14] -Output [4]: [ws_ext_sales_price#39, d_year#42, d_qoy#43, ca_county#45] -Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#42, d_qoy#43, ca_address_sk#44, ca_county#45] - -(57) HashAggregate [codegen id : 14] -Input [4]: [ws_ext_sales_price#39, d_year#42, d_qoy#43, ca_county#45] -Keys [3]: [ca_county#45, d_qoy#43, d_year#42] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum#46] -Results [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] - -(58) CometColumnarExchange -Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] -Arguments: hashpartitioning(ca_county#45, d_qoy#43, d_year#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(59) CometColumnarToRow [codegen id : 15] -Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] - -(60) HashAggregate [codegen id : 15] -Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] -Keys [3]: [ca_county#45, d_qoy#43, d_year#42] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] -Results [2]: [ca_county#45, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS web_sales#49] - -(61) BroadcastExchange -Input [2]: [ca_county#45, web_sales#49] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(62) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#9] -Right keys [1]: [ca_county#45] -Join type: Inner -Join condition: None - -(63) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#17)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(64) ColumnarToRow [codegen id : 18] -Input [3]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] - -(65) Filter [codegen id : 18] -Input [3]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] -Condition : isnotnull(ws_bill_addr_sk#50) - -(66) ReusedExchange [Reuses operator id: 105] -Output [3]: [d_date_sk#53, d_year#54, d_qoy#55] - -(67) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#52] -Right keys [1]: [d_date_sk#53] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 18] -Output [4]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, d_year#54, d_qoy#55] -Input [6]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52, d_date_sk#53, d_year#54, d_qoy#55] - -(69) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#56, ca_county#57] - -(70) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_bill_addr_sk#50] -Right keys [1]: [ca_address_sk#56] -Join type: Inner -Join condition: None - -(71) Project [codegen id : 18] -Output [4]: [ws_ext_sales_price#51, d_year#54, d_qoy#55, ca_county#57] -Input [6]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, d_year#54, d_qoy#55, ca_address_sk#56, ca_county#57] - -(72) HashAggregate [codegen id : 18] -Input [4]: [ws_ext_sales_price#51, d_year#54, d_qoy#55, ca_county#57] -Keys [3]: [ca_county#57, d_qoy#55, d_year#54] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#51))] -Aggregate Attributes [1]: [sum#58] -Results [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] - -(73) CometColumnarExchange -Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] -Arguments: hashpartitioning(ca_county#57, d_qoy#55, d_year#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(74) CometColumnarToRow [codegen id : 19] -Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] - -(75) HashAggregate [codegen id : 19] -Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] -Keys [3]: [ca_county#57, d_qoy#55, d_year#54] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#51))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#51))#48] -Results [2]: [ca_county#57, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#51))#48,17,2) AS web_sales#60] - -(76) BroadcastExchange -Input [2]: [ca_county#57, web_sales#60] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=10] - -(77) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#45] -Right keys [1]: [ca_county#57] -Join type: Inner -Join condition: (CASE WHEN (web_sales#49 > 0.00) THEN (web_sales#60 / web_sales#49) END > CASE WHEN (store_sales#13 > 0.00) THEN (store_sales#25 / store_sales#13) END) - -(78) Project [codegen id : 24] -Output [8]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#45, web_sales#49, web_sales#60] -Input [9]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#45, web_sales#49, ca_county#57, web_sales#60] - -(79) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#29)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(80) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] - -(81) Filter [codegen id : 22] -Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_bill_addr_sk#61) - -(82) ReusedExchange [Reuses operator id: 109] -Output [3]: [d_date_sk#64, d_year#65, d_qoy#66] - -(83) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#63] -Right keys [1]: [d_date_sk#64] -Join type: Inner -Join condition: None - -(84) Project [codegen id : 22] -Output [4]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#65, d_qoy#66] -Input [6]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63, d_date_sk#64, d_year#65, d_qoy#66] - -(85) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#67, ca_county#68] - -(86) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_bill_addr_sk#61] -Right keys [1]: [ca_address_sk#67] -Join type: Inner -Join condition: None - -(87) Project [codegen id : 22] -Output [4]: [ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_county#68] -Input [6]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_address_sk#67, ca_county#68] - -(88) HashAggregate [codegen id : 22] -Input [4]: [ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_county#68] -Keys [3]: [ca_county#68, d_qoy#66, d_year#65] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#62))] -Aggregate Attributes [1]: [sum#69] -Results [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] - -(89) CometColumnarExchange -Input [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] -Arguments: hashpartitioning(ca_county#68, d_qoy#66, d_year#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(90) CometColumnarToRow [codegen id : 23] -Input [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] - -(91) HashAggregate [codegen id : 23] -Input [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] -Keys [3]: [ca_county#68, d_qoy#66, d_year#65] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#62))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#62))#48] -Results [2]: [ca_county#68, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#62))#48,17,2) AS web_sales#71] - -(92) BroadcastExchange -Input [2]: [ca_county#68, web_sales#71] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=12] - -(93) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#45] -Right keys [1]: [ca_county#68] -Join type: Inner -Join condition: (CASE WHEN (web_sales#60 > 0.00) THEN (web_sales#71 / web_sales#60) END > CASE WHEN (store_sales#25 > 0.00) THEN (store_sales#37 / store_sales#25) END) - -(94) Project [codegen id : 24] -Output [6]: [ca_county#9, d_year#6, (web_sales#60 / web_sales#49) AS web_q1_q2_increase#72, (store_sales#25 / store_sales#13) AS store_q1_q2_increase#73, (web_sales#71 / web_sales#60) AS web_q2_q3_increase#74, (store_sales#37 / store_sales#25) AS store_q2_q3_increase#75] -Input [10]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#45, web_sales#49, web_sales#60, ca_county#68, web_sales#71] - -(95) CometColumnarExchange -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(96) CometSort -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75] -Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75], [ca_county#9 ASC NULLS FIRST] - -(97) CometColumnarToRow [codegen id : 25] -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (101) -+- * CometColumnarToRow (100) - +- CometFilter (99) - +- CometNativeScan parquet spark_catalog.default.date_dim (98) - - -(98) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(99) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) - -(100) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] - -(101) BroadcastExchange -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=14] - -Subquery:2 Hosting operator id = 17 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#17 -BroadcastExchange (105) -+- * CometColumnarToRow (104) - +- CometFilter (103) - +- CometNativeScan parquet spark_catalog.default.date_dim (102) - - -(102) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(103) CometFilter -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] -Condition : ((((isnotnull(d_qoy#20) AND isnotnull(d_year#19)) AND (d_qoy#20 = 2)) AND (d_year#19 = 2000)) AND isnotnull(d_date_sk#18)) - -(104) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] - -(105) BroadcastExchange -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] - -Subquery:3 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 -BroadcastExchange (109) -+- * CometColumnarToRow (108) - +- CometFilter (107) - +- CometNativeScan parquet spark_catalog.default.date_dim (106) - - -(106) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#30, d_year#31, d_qoy#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(107) CometFilter -Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] -Condition : ((((isnotnull(d_qoy#32) AND isnotnull(d_year#31)) AND (d_qoy#32 = 3)) AND (d_year#31 = 2000)) AND isnotnull(d_date_sk#30)) - -(108) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] - -(109) BroadcastExchange -Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] - -Subquery:4 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#4 - -Subquery:5 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#17 - -Subquery:6 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#29 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/simplified.txt deleted file mode 100644 index f3a0572081..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/simplified.txt +++ /dev/null @@ -1,166 +0,0 @@ -WholeStageCodegen (25) - CometColumnarToRow - InputAdapter - CometSort [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] - CometColumnarExchange [ca_county] #1 - WholeStageCodegen (24) - Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] - Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - Project [ca_county,d_year,store_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - BroadcastHashJoin [ca_county,ca_county] - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #2 - WholeStageCodegen (3) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_county] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #6 - WholeStageCodegen (6) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #7 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #9 - WholeStageCodegen (10) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #10 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #12 - WholeStageCodegen (14) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (19) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #14 - WholeStageCodegen (18) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #7 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (23) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #16 - WholeStageCodegen (22) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/explain.txt deleted file mode 100644 index 83d717e295..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,593 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (90) -+- CometSort (89) - +- CometExchange (88) - +- CometProject (87) - +- CometBroadcastHashJoin (86) - :- CometProject (73) - : +- CometBroadcastHashJoin (72) - : :- CometBroadcastHashJoin (59) - : : :- CometProject (46) - : : : +- CometBroadcastHashJoin (45) - : : : :- CometBroadcastHashJoin (30) - : : : : :- CometHashAggregate (15) - : : : : : +- CometExchange (14) - : : : : : +- CometHashAggregate (13) - : : : : : +- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - : : : : +- CometBroadcastExchange (29) - : : : : +- CometHashAggregate (28) - : : : : +- CometExchange (27) - : : : : +- CometHashAggregate (26) - : : : : +- CometProject (25) - : : : : +- CometBroadcastHashJoin (24) - : : : : :- CometProject (22) - : : : : : +- CometBroadcastHashJoin (21) - : : : : : :- CometFilter (17) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (16) - : : : : : +- CometBroadcastExchange (20) - : : : : : +- CometFilter (19) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (18) - : : : : +- ReusedExchange (23) - : : : +- CometBroadcastExchange (44) - : : : +- CometHashAggregate (43) - : : : +- CometExchange (42) - : : : +- CometHashAggregate (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (32) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (31) - : : : : +- CometBroadcastExchange (35) - : : : : +- CometFilter (34) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (33) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (58) - : : +- CometHashAggregate (57) - : : +- CometExchange (56) - : : +- CometHashAggregate (55) - : : +- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometFilter (48) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : : : +- ReusedExchange (49) - : : +- ReusedExchange (52) - : +- CometBroadcastExchange (71) - : +- CometHashAggregate (70) - : +- CometExchange (69) - : +- CometHashAggregate (68) - : +- CometProject (67) - : +- CometBroadcastHashJoin (66) - : :- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometFilter (61) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (60) - : : +- ReusedExchange (62) - : +- ReusedExchange (65) - +- CometBroadcastExchange (85) - +- CometHashAggregate (84) - +- CometExchange (83) - +- CometHashAggregate (82) - +- CometProject (81) - +- CometBroadcastHashJoin (80) - :- CometProject (78) - : +- CometBroadcastHashJoin (77) - : :- CometFilter (75) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (74) - : +- ReusedExchange (76) - +- ReusedExchange (79) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_addr_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) - -(5) CometBroadcastExchange -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [d_date_sk#5, d_year#6, d_qoy#7] - -(6) CometBroadcastHashJoin -Left output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(7) CometProject -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7], [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#8, ca_county#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [ca_address_sk#8, ca_county#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_county#9)) - -(10) CometBroadcastExchange -Input [2]: [ca_address_sk#8, ca_county#9] -Arguments: [ca_address_sk#8, ca_county#9] - -(11) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] -Right output [2]: [ca_address_sk#8, ca_county#9] -Arguments: [ss_addr_sk#1], [ca_address_sk#8], Inner, BuildRight - -(12) CometProject -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_address_sk#8, ca_county#9] -Arguments: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9], [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] - -(13) CometHashAggregate -Input [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] -Keys [3]: [ca_county#9, d_qoy#7, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(14) CometExchange -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10] -Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(15) CometHashAggregate -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10] -Keys [3]: [ca_county#9, d_qoy#7, d_year#6] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(17) CometFilter -Input [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] -Condition : isnotnull(ss_addr_sk#11) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(19) CometFilter -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) - -(20) CometBroadcastExchange -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Arguments: [d_date_sk#15, d_year#16, d_qoy#17] - -(21) CometBroadcastHashJoin -Left output [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] -Right output [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(22) CometProject -Input [6]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_year#16, d_qoy#17] -Arguments: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17], [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17] - -(23) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#18, ca_county#19] - -(24) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17] -Right output [2]: [ca_address_sk#18, ca_county#19] -Arguments: [ss_addr_sk#11], [ca_address_sk#18], Inner, BuildRight - -(25) CometProject -Input [6]: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_address_sk#18, ca_county#19] -Arguments: [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19], [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19] - -(26) CometHashAggregate -Input [4]: [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19] -Keys [3]: [ca_county#19, d_qoy#17, d_year#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#12))] - -(27) CometExchange -Input [4]: [ca_county#19, d_qoy#17, d_year#16, sum#20] -Arguments: hashpartitioning(ca_county#19, d_qoy#17, d_year#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [4]: [ca_county#19, d_qoy#17, d_year#16, sum#20] -Keys [3]: [ca_county#19, d_qoy#17, d_year#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#12))] - -(29) CometBroadcastExchange -Input [2]: [ca_county#19, store_sales#21] -Arguments: [ca_county#19, store_sales#21] - -(30) CometBroadcastHashJoin -Left output [3]: [ca_county#9, d_year#6, store_sales#22] -Right output [2]: [ca_county#19, store_sales#21] -Arguments: [ca_county#9], [ca_county#19], Inner, BuildRight - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_addr_sk#23) - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) - -(35) CometBroadcastExchange -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Arguments: [d_date_sk#27, d_year#28, d_qoy#29] - -(36) CometBroadcastHashJoin -Left output [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] -Right output [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight - -(37) CometProject -Input [6]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_year#28, d_qoy#29] -Arguments: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29], [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29] - -(38) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#30, ca_county#31] - -(39) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29] -Right output [2]: [ca_address_sk#30, ca_county#31] -Arguments: [ss_addr_sk#23], [ca_address_sk#30], Inner, BuildRight - -(40) CometProject -Input [6]: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_address_sk#30, ca_county#31] -Arguments: [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31], [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31] - -(41) CometHashAggregate -Input [4]: [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31] -Keys [3]: [ca_county#31, d_qoy#29, d_year#28] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#24))] - -(42) CometExchange -Input [4]: [ca_county#31, d_qoy#29, d_year#28, sum#32] -Arguments: hashpartitioning(ca_county#31, d_qoy#29, d_year#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(43) CometHashAggregate -Input [4]: [ca_county#31, d_qoy#29, d_year#28, sum#32] -Keys [3]: [ca_county#31, d_qoy#29, d_year#28] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#24))] - -(44) CometBroadcastExchange -Input [2]: [ca_county#31, store_sales#33] -Arguments: [ca_county#31, store_sales#33] - -(45) CometBroadcastHashJoin -Left output [5]: [ca_county#9, d_year#6, store_sales#22, ca_county#19, store_sales#21] -Right output [2]: [ca_county#31, store_sales#33] -Arguments: [ca_county#19], [ca_county#31], Inner, BuildRight - -(46) CometProject -Input [7]: [ca_county#9, d_year#6, store_sales#22, ca_county#19, store_sales#21, ca_county#31, store_sales#33] -Arguments: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33], [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#37)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(48) CometFilter -Input [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_bill_addr_sk#34) - -(49) ReusedExchange [Reuses operator id: 5] -Output [3]: [d_date_sk#38, d_year#39, d_qoy#40] - -(50) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Right output [3]: [d_date_sk#38, d_year#39, d_qoy#40] -Arguments: [ws_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight - -(51) CometProject -Input [6]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#38, d_year#39, d_qoy#40] -Arguments: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40], [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40] - -(52) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#41, ca_county#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40] -Right output [2]: [ca_address_sk#41, ca_county#42] -Arguments: [ws_bill_addr_sk#34], [ca_address_sk#41], Inner, BuildRight - -(54) CometProject -Input [6]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_address_sk#41, ca_county#42] -Arguments: [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42], [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42] - -(55) CometHashAggregate -Input [4]: [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42] -Keys [3]: [ca_county#42, d_qoy#40, d_year#39] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] - -(56) CometExchange -Input [4]: [ca_county#42, d_qoy#40, d_year#39, sum#43] -Arguments: hashpartitioning(ca_county#42, d_qoy#40, d_year#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(57) CometHashAggregate -Input [4]: [ca_county#42, d_qoy#40, d_year#39, sum#43] -Keys [3]: [ca_county#42, d_qoy#40, d_year#39] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] - -(58) CometBroadcastExchange -Input [2]: [ca_county#42, web_sales#44] -Arguments: [ca_county#42, web_sales#44] - -(59) CometBroadcastHashJoin -Left output [5]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33] -Right output [2]: [ca_county#42, web_sales#44] -Arguments: [ca_county#9], [ca_county#42], Inner, BuildRight - -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#48)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(61) CometFilter -Input [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] -Condition : isnotnull(ws_bill_addr_sk#45) - -(62) ReusedExchange [Reuses operator id: 20] -Output [3]: [d_date_sk#49, d_year#50, d_qoy#51] - -(63) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] -Right output [3]: [d_date_sk#49, d_year#50, d_qoy#51] -Arguments: [ws_sold_date_sk#47], [d_date_sk#49], Inner, BuildRight - -(64) CometProject -Input [6]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47, d_date_sk#49, d_year#50, d_qoy#51] -Arguments: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51], [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51] - -(65) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#52, ca_county#53] - -(66) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51] -Right output [2]: [ca_address_sk#52, ca_county#53] -Arguments: [ws_bill_addr_sk#45], [ca_address_sk#52], Inner, BuildRight - -(67) CometProject -Input [6]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_address_sk#52, ca_county#53] -Arguments: [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53], [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53] - -(68) CometHashAggregate -Input [4]: [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53] -Keys [3]: [ca_county#53, d_qoy#51, d_year#50] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#46))] - -(69) CometExchange -Input [4]: [ca_county#53, d_qoy#51, d_year#50, sum#54] -Arguments: hashpartitioning(ca_county#53, d_qoy#51, d_year#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(70) CometHashAggregate -Input [4]: [ca_county#53, d_qoy#51, d_year#50, sum#54] -Keys [3]: [ca_county#53, d_qoy#51, d_year#50] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#46))] - -(71) CometBroadcastExchange -Input [2]: [ca_county#53, web_sales#55] -Arguments: [ca_county#53, web_sales#55] - -(72) CometBroadcastHashJoin -Left output [7]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44] -Right output [2]: [ca_county#53, web_sales#55] -Arguments: [ca_county#42], [ca_county#53], Inner, (CASE WHEN (web_sales#44 > 0.00) THEN (web_sales#55 / web_sales#44) END > CASE WHEN (store_sales#22 > 0.00) THEN (store_sales#21 / store_sales#22) END), BuildRight - -(73) CometProject -Input [9]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, ca_county#53, web_sales#55] -Arguments: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55], [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55] - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(75) CometFilter -Input [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] -Condition : isnotnull(ws_bill_addr_sk#56) - -(76) ReusedExchange [Reuses operator id: 35] -Output [3]: [d_date_sk#60, d_year#61, d_qoy#62] - -(77) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] -Right output [3]: [d_date_sk#60, d_year#61, d_qoy#62] -Arguments: [ws_sold_date_sk#58], [d_date_sk#60], Inner, BuildRight - -(78) CometProject -Input [6]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58, d_date_sk#60, d_year#61, d_qoy#62] -Arguments: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62], [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62] - -(79) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#63, ca_county#64] - -(80) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62] -Right output [2]: [ca_address_sk#63, ca_county#64] -Arguments: [ws_bill_addr_sk#56], [ca_address_sk#63], Inner, BuildRight - -(81) CometProject -Input [6]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_address_sk#63, ca_county#64] -Arguments: [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64], [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64] - -(82) CometHashAggregate -Input [4]: [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64] -Keys [3]: [ca_county#64, d_qoy#62, d_year#61] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#57))] - -(83) CometExchange -Input [4]: [ca_county#64, d_qoy#62, d_year#61, sum#65] -Arguments: hashpartitioning(ca_county#64, d_qoy#62, d_year#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(84) CometHashAggregate -Input [4]: [ca_county#64, d_qoy#62, d_year#61, sum#65] -Keys [3]: [ca_county#64, d_qoy#62, d_year#61] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#57))] - -(85) CometBroadcastExchange -Input [2]: [ca_county#64, web_sales#66] -Arguments: [ca_county#64, web_sales#66] - -(86) CometBroadcastHashJoin -Left output [8]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55] -Right output [2]: [ca_county#64, web_sales#66] -Arguments: [ca_county#42], [ca_county#64], Inner, (CASE WHEN (web_sales#55 > 0.00) THEN (web_sales#66 / web_sales#55) END > CASE WHEN (store_sales#21 > 0.00) THEN (store_sales#33 / store_sales#21) END), BuildRight - -(87) CometProject -Input [10]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55, ca_county#64, web_sales#66] -Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9, d_year#6, (web_sales#55 / web_sales#44) AS web_q1_q2_increase#67, (store_sales#21 / store_sales#22) AS store_q1_q2_increase#68, (web_sales#66 / web_sales#55) AS web_q2_q3_increase#69, (store_sales#33 / store_sales#21) AS store_q2_q3_increase#70] - -(88) CometExchange -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(89) CometSort -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] -Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9 ASC NULLS FIRST] - -(90) CometColumnarToRow [codegen id : 1] -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (94) -+- * CometColumnarToRow (93) - +- CometFilter (92) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (91) - - -(91) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(92) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) - -(93) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] - -(94) BroadcastExchange -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (98) -+- * CometColumnarToRow (97) - +- CometFilter (96) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (95) - - -(95) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(96) CometFilter -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) - -(97) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] - -(98) BroadcastExchange -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -Subquery:3 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 -BroadcastExchange (102) -+- * CometColumnarToRow (101) - +- CometFilter (100) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (99) - - -(99) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(100) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) - -(101) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] - -(102) BroadcastExchange -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:4 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#4 - -Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#14 - -Subquery:6 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#26 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/simplified.txt deleted file mode 100644 index 2f97384320..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,116 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] - CometExchange [ca_county] #1 - CometProject [web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] - CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales,ca_county,web_sales] - CometProject [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,ca_county,web_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales] - CometProject [ca_county,d_year,store_sales,store_sales,store_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales,ca_county,store_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales] - CometHashAggregate [d_qoy,sum] [ca_county,d_year,store_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #2 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [ca_address_sk,ca_county] #5 - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange [ca_county,store_sales] #6 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,store_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #7 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,store_sales] #10 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,store_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #11 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,web_sales] #14 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #15 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_year,d_qoy] #4 - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,web_sales] #16 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #17 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [d_date_sk,d_year,d_qoy] #9 - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,web_sales] #18 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #19 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [d_date_sk,d_year,d_qoy] #13 - ReusedExchange [ca_address_sk,ca_county] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt deleted file mode 100644 index 83d717e295..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt +++ /dev/null @@ -1,593 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (90) -+- CometSort (89) - +- CometExchange (88) - +- CometProject (87) - +- CometBroadcastHashJoin (86) - :- CometProject (73) - : +- CometBroadcastHashJoin (72) - : :- CometBroadcastHashJoin (59) - : : :- CometProject (46) - : : : +- CometBroadcastHashJoin (45) - : : : :- CometBroadcastHashJoin (30) - : : : : :- CometHashAggregate (15) - : : : : : +- CometExchange (14) - : : : : : +- CometHashAggregate (13) - : : : : : +- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - : : : : +- CometBroadcastExchange (29) - : : : : +- CometHashAggregate (28) - : : : : +- CometExchange (27) - : : : : +- CometHashAggregate (26) - : : : : +- CometProject (25) - : : : : +- CometBroadcastHashJoin (24) - : : : : :- CometProject (22) - : : : : : +- CometBroadcastHashJoin (21) - : : : : : :- CometFilter (17) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (16) - : : : : : +- CometBroadcastExchange (20) - : : : : : +- CometFilter (19) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (18) - : : : : +- ReusedExchange (23) - : : : +- CometBroadcastExchange (44) - : : : +- CometHashAggregate (43) - : : : +- CometExchange (42) - : : : +- CometHashAggregate (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (32) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (31) - : : : : +- CometBroadcastExchange (35) - : : : : +- CometFilter (34) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (33) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (58) - : : +- CometHashAggregate (57) - : : +- CometExchange (56) - : : +- CometHashAggregate (55) - : : +- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometFilter (48) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : : : +- ReusedExchange (49) - : : +- ReusedExchange (52) - : +- CometBroadcastExchange (71) - : +- CometHashAggregate (70) - : +- CometExchange (69) - : +- CometHashAggregate (68) - : +- CometProject (67) - : +- CometBroadcastHashJoin (66) - : :- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometFilter (61) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (60) - : : +- ReusedExchange (62) - : +- ReusedExchange (65) - +- CometBroadcastExchange (85) - +- CometHashAggregate (84) - +- CometExchange (83) - +- CometHashAggregate (82) - +- CometProject (81) - +- CometBroadcastHashJoin (80) - :- CometProject (78) - : +- CometBroadcastHashJoin (77) - : :- CometFilter (75) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (74) - : +- ReusedExchange (76) - +- ReusedExchange (79) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_addr_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) - -(5) CometBroadcastExchange -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [d_date_sk#5, d_year#6, d_qoy#7] - -(6) CometBroadcastHashJoin -Left output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(7) CometProject -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7], [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#8, ca_county#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [ca_address_sk#8, ca_county#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_county#9)) - -(10) CometBroadcastExchange -Input [2]: [ca_address_sk#8, ca_county#9] -Arguments: [ca_address_sk#8, ca_county#9] - -(11) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] -Right output [2]: [ca_address_sk#8, ca_county#9] -Arguments: [ss_addr_sk#1], [ca_address_sk#8], Inner, BuildRight - -(12) CometProject -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_address_sk#8, ca_county#9] -Arguments: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9], [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] - -(13) CometHashAggregate -Input [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] -Keys [3]: [ca_county#9, d_qoy#7, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(14) CometExchange -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10] -Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(15) CometHashAggregate -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10] -Keys [3]: [ca_county#9, d_qoy#7, d_year#6] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(17) CometFilter -Input [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] -Condition : isnotnull(ss_addr_sk#11) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(19) CometFilter -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) - -(20) CometBroadcastExchange -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Arguments: [d_date_sk#15, d_year#16, d_qoy#17] - -(21) CometBroadcastHashJoin -Left output [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] -Right output [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(22) CometProject -Input [6]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_year#16, d_qoy#17] -Arguments: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17], [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17] - -(23) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#18, ca_county#19] - -(24) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17] -Right output [2]: [ca_address_sk#18, ca_county#19] -Arguments: [ss_addr_sk#11], [ca_address_sk#18], Inner, BuildRight - -(25) CometProject -Input [6]: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_address_sk#18, ca_county#19] -Arguments: [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19], [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19] - -(26) CometHashAggregate -Input [4]: [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19] -Keys [3]: [ca_county#19, d_qoy#17, d_year#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#12))] - -(27) CometExchange -Input [4]: [ca_county#19, d_qoy#17, d_year#16, sum#20] -Arguments: hashpartitioning(ca_county#19, d_qoy#17, d_year#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [4]: [ca_county#19, d_qoy#17, d_year#16, sum#20] -Keys [3]: [ca_county#19, d_qoy#17, d_year#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#12))] - -(29) CometBroadcastExchange -Input [2]: [ca_county#19, store_sales#21] -Arguments: [ca_county#19, store_sales#21] - -(30) CometBroadcastHashJoin -Left output [3]: [ca_county#9, d_year#6, store_sales#22] -Right output [2]: [ca_county#19, store_sales#21] -Arguments: [ca_county#9], [ca_county#19], Inner, BuildRight - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_addr_sk#23) - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) - -(35) CometBroadcastExchange -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Arguments: [d_date_sk#27, d_year#28, d_qoy#29] - -(36) CometBroadcastHashJoin -Left output [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] -Right output [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight - -(37) CometProject -Input [6]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_year#28, d_qoy#29] -Arguments: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29], [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29] - -(38) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#30, ca_county#31] - -(39) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29] -Right output [2]: [ca_address_sk#30, ca_county#31] -Arguments: [ss_addr_sk#23], [ca_address_sk#30], Inner, BuildRight - -(40) CometProject -Input [6]: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_address_sk#30, ca_county#31] -Arguments: [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31], [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31] - -(41) CometHashAggregate -Input [4]: [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31] -Keys [3]: [ca_county#31, d_qoy#29, d_year#28] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#24))] - -(42) CometExchange -Input [4]: [ca_county#31, d_qoy#29, d_year#28, sum#32] -Arguments: hashpartitioning(ca_county#31, d_qoy#29, d_year#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(43) CometHashAggregate -Input [4]: [ca_county#31, d_qoy#29, d_year#28, sum#32] -Keys [3]: [ca_county#31, d_qoy#29, d_year#28] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#24))] - -(44) CometBroadcastExchange -Input [2]: [ca_county#31, store_sales#33] -Arguments: [ca_county#31, store_sales#33] - -(45) CometBroadcastHashJoin -Left output [5]: [ca_county#9, d_year#6, store_sales#22, ca_county#19, store_sales#21] -Right output [2]: [ca_county#31, store_sales#33] -Arguments: [ca_county#19], [ca_county#31], Inner, BuildRight - -(46) CometProject -Input [7]: [ca_county#9, d_year#6, store_sales#22, ca_county#19, store_sales#21, ca_county#31, store_sales#33] -Arguments: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33], [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#37)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(48) CometFilter -Input [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_bill_addr_sk#34) - -(49) ReusedExchange [Reuses operator id: 5] -Output [3]: [d_date_sk#38, d_year#39, d_qoy#40] - -(50) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Right output [3]: [d_date_sk#38, d_year#39, d_qoy#40] -Arguments: [ws_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight - -(51) CometProject -Input [6]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#38, d_year#39, d_qoy#40] -Arguments: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40], [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40] - -(52) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#41, ca_county#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40] -Right output [2]: [ca_address_sk#41, ca_county#42] -Arguments: [ws_bill_addr_sk#34], [ca_address_sk#41], Inner, BuildRight - -(54) CometProject -Input [6]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_address_sk#41, ca_county#42] -Arguments: [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42], [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42] - -(55) CometHashAggregate -Input [4]: [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42] -Keys [3]: [ca_county#42, d_qoy#40, d_year#39] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] - -(56) CometExchange -Input [4]: [ca_county#42, d_qoy#40, d_year#39, sum#43] -Arguments: hashpartitioning(ca_county#42, d_qoy#40, d_year#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(57) CometHashAggregate -Input [4]: [ca_county#42, d_qoy#40, d_year#39, sum#43] -Keys [3]: [ca_county#42, d_qoy#40, d_year#39] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] - -(58) CometBroadcastExchange -Input [2]: [ca_county#42, web_sales#44] -Arguments: [ca_county#42, web_sales#44] - -(59) CometBroadcastHashJoin -Left output [5]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33] -Right output [2]: [ca_county#42, web_sales#44] -Arguments: [ca_county#9], [ca_county#42], Inner, BuildRight - -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#48)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(61) CometFilter -Input [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] -Condition : isnotnull(ws_bill_addr_sk#45) - -(62) ReusedExchange [Reuses operator id: 20] -Output [3]: [d_date_sk#49, d_year#50, d_qoy#51] - -(63) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] -Right output [3]: [d_date_sk#49, d_year#50, d_qoy#51] -Arguments: [ws_sold_date_sk#47], [d_date_sk#49], Inner, BuildRight - -(64) CometProject -Input [6]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47, d_date_sk#49, d_year#50, d_qoy#51] -Arguments: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51], [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51] - -(65) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#52, ca_county#53] - -(66) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51] -Right output [2]: [ca_address_sk#52, ca_county#53] -Arguments: [ws_bill_addr_sk#45], [ca_address_sk#52], Inner, BuildRight - -(67) CometProject -Input [6]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_address_sk#52, ca_county#53] -Arguments: [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53], [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53] - -(68) CometHashAggregate -Input [4]: [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53] -Keys [3]: [ca_county#53, d_qoy#51, d_year#50] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#46))] - -(69) CometExchange -Input [4]: [ca_county#53, d_qoy#51, d_year#50, sum#54] -Arguments: hashpartitioning(ca_county#53, d_qoy#51, d_year#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(70) CometHashAggregate -Input [4]: [ca_county#53, d_qoy#51, d_year#50, sum#54] -Keys [3]: [ca_county#53, d_qoy#51, d_year#50] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#46))] - -(71) CometBroadcastExchange -Input [2]: [ca_county#53, web_sales#55] -Arguments: [ca_county#53, web_sales#55] - -(72) CometBroadcastHashJoin -Left output [7]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44] -Right output [2]: [ca_county#53, web_sales#55] -Arguments: [ca_county#42], [ca_county#53], Inner, (CASE WHEN (web_sales#44 > 0.00) THEN (web_sales#55 / web_sales#44) END > CASE WHEN (store_sales#22 > 0.00) THEN (store_sales#21 / store_sales#22) END), BuildRight - -(73) CometProject -Input [9]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, ca_county#53, web_sales#55] -Arguments: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55], [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55] - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(75) CometFilter -Input [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] -Condition : isnotnull(ws_bill_addr_sk#56) - -(76) ReusedExchange [Reuses operator id: 35] -Output [3]: [d_date_sk#60, d_year#61, d_qoy#62] - -(77) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] -Right output [3]: [d_date_sk#60, d_year#61, d_qoy#62] -Arguments: [ws_sold_date_sk#58], [d_date_sk#60], Inner, BuildRight - -(78) CometProject -Input [6]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58, d_date_sk#60, d_year#61, d_qoy#62] -Arguments: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62], [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62] - -(79) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#63, ca_county#64] - -(80) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62] -Right output [2]: [ca_address_sk#63, ca_county#64] -Arguments: [ws_bill_addr_sk#56], [ca_address_sk#63], Inner, BuildRight - -(81) CometProject -Input [6]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_address_sk#63, ca_county#64] -Arguments: [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64], [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64] - -(82) CometHashAggregate -Input [4]: [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64] -Keys [3]: [ca_county#64, d_qoy#62, d_year#61] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#57))] - -(83) CometExchange -Input [4]: [ca_county#64, d_qoy#62, d_year#61, sum#65] -Arguments: hashpartitioning(ca_county#64, d_qoy#62, d_year#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(84) CometHashAggregate -Input [4]: [ca_county#64, d_qoy#62, d_year#61, sum#65] -Keys [3]: [ca_county#64, d_qoy#62, d_year#61] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#57))] - -(85) CometBroadcastExchange -Input [2]: [ca_county#64, web_sales#66] -Arguments: [ca_county#64, web_sales#66] - -(86) CometBroadcastHashJoin -Left output [8]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55] -Right output [2]: [ca_county#64, web_sales#66] -Arguments: [ca_county#42], [ca_county#64], Inner, (CASE WHEN (web_sales#55 > 0.00) THEN (web_sales#66 / web_sales#55) END > CASE WHEN (store_sales#21 > 0.00) THEN (store_sales#33 / store_sales#21) END), BuildRight - -(87) CometProject -Input [10]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55, ca_county#64, web_sales#66] -Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9, d_year#6, (web_sales#55 / web_sales#44) AS web_q1_q2_increase#67, (store_sales#21 / store_sales#22) AS store_q1_q2_increase#68, (web_sales#66 / web_sales#55) AS web_q2_q3_increase#69, (store_sales#33 / store_sales#21) AS store_q2_q3_increase#70] - -(88) CometExchange -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(89) CometSort -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] -Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9 ASC NULLS FIRST] - -(90) CometColumnarToRow [codegen id : 1] -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (94) -+- * CometColumnarToRow (93) - +- CometFilter (92) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (91) - - -(91) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(92) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) - -(93) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] - -(94) BroadcastExchange -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (98) -+- * CometColumnarToRow (97) - +- CometFilter (96) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (95) - - -(95) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(96) CometFilter -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) - -(97) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] - -(98) BroadcastExchange -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -Subquery:3 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 -BroadcastExchange (102) -+- * CometColumnarToRow (101) - +- CometFilter (100) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (99) - - -(99) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(100) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) - -(101) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] - -(102) BroadcastExchange -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:4 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#4 - -Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#14 - -Subquery:6 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#26 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/extended.txt deleted file mode 100644 index b4e7aa4db6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/extended.txt +++ /dev/null @@ -1,126 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : : +- CometBroadcastExchange - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 111 out of 120 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 2f97384320..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt +++ /dev/null @@ -1,116 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] - CometExchange [ca_county] #1 - CometProject [web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] - CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales,ca_county,web_sales] - CometProject [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,ca_county,web_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales] - CometProject [ca_county,d_year,store_sales,store_sales,store_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales,ca_county,store_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales] - CometHashAggregate [d_qoy,sum] [ca_county,d_year,store_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #2 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [ca_address_sk,ca_county] #5 - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange [ca_county,store_sales] #6 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,store_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #7 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,store_sales] #10 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,store_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #11 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,web_sales] #14 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #15 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_year,d_qoy] #4 - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,web_sales] #16 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #17 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [d_date_sk,d_year,d_qoy] #9 - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,web_sales] #18 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #19 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [d_date_sk,d_year,d_qoy] #13 - ReusedExchange [ca_address_sk,ca_county] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/explain.txt deleted file mode 100644 index df3987b0a0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -* HashAggregate (31) -+- * CometColumnarToRow (30) - +- CometColumnarExchange (29) - +- * HashAggregate (28) - +- * Project (27) - +- * BroadcastHashJoin Inner BuildRight (26) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : +- BroadcastExchange (8) - : : +- * CometColumnarToRow (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometNativeScan parquet spark_catalog.default.item (4) - : +- BroadcastExchange (22) - : +- * Filter (21) - : +- * HashAggregate (20) - : +- * CometColumnarToRow (19) - : +- CometColumnarExchange (18) - : +- * HashAggregate (17) - : +- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet spark_catalog.default.catalog_sales (11) - : +- ReusedExchange (14) - +- ReusedExchange (25) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_ext_discount_amt)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] - -(3) Filter [codegen id : 6] -Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Condition : (isnotnull(cs_item_sk#1) AND isnotnull(cs_ext_discount_amt#2)) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_manufact_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,977), IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [i_item_sk#5, i_manufact_id#6] -Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 977)) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [2]: [i_item_sk#5, i_manufact_id#6] -Arguments: [i_item_sk#5], [i_item_sk#5] - -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [i_item_sk#5] - -(8) BroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 6] -Output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] -Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] - -(11) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] - -(13) Filter [codegen id : 3] -Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Condition : isnotnull(cs_item_sk#7) - -(14) ReusedExchange [Reuses operator id: 36] -Output [1]: [d_date_sk#10] - -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(16) Project [codegen id : 3] -Output [2]: [cs_item_sk#7, cs_ext_discount_amt#8] -Input [4]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9, d_date_sk#10] - -(17) HashAggregate [codegen id : 3] -Input [2]: [cs_item_sk#7, cs_ext_discount_amt#8] -Keys [1]: [cs_item_sk#7] -Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#8))] -Aggregate Attributes [2]: [sum#11, count#12] -Results [3]: [cs_item_sk#7, sum#13, count#14] - -(18) CometColumnarExchange -Input [3]: [cs_item_sk#7, sum#13, count#14] -Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(19) CometColumnarToRow [codegen id : 4] -Input [3]: [cs_item_sk#7, sum#13, count#14] - -(20) HashAggregate [codegen id : 4] -Input [3]: [cs_item_sk#7, sum#13, count#14] -Keys [1]: [cs_item_sk#7] -Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))] -Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))#15] -Results [2]: [(1.3 * cast((avg(UnscaledValue(cs_ext_discount_amt#8))#15 / 100.0) as decimal(11,6))) AS (1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] - -(21) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] -Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#16) - -(22) BroadcastExchange -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] - -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [i_item_sk#5] -Right keys [1]: [cs_item_sk#7] -Join type: Inner -Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#16) - -(24) Project [codegen id : 6] -Output [2]: [cs_ext_discount_amt#2, cs_sold_date_sk#3] -Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] - -(25) ReusedExchange [Reuses operator id: 36] -Output [1]: [d_date_sk#17] - -(26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 6] -Output [1]: [cs_ext_discount_amt#2] -Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#17] - -(28) HashAggregate [codegen id : 6] -Input [1]: [cs_ext_discount_amt#2] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#18] -Results [1]: [sum#19] - -(29) CometColumnarExchange -Input [1]: [sum#19] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 7] -Input [1]: [sum#19] - -(31) HashAggregate [codegen id : 7] -Input [1]: [sum#19] -Keys: [] -Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))#20] -Results [1]: [MakeDecimal(sum(UnscaledValue(cs_ext_discount_amt#2))#20,17,2) AS excess discount amount#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.date_dim (32) - - -(32) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#17, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 2000-01-27)) AND (d_date#22 <= 2000-04-26)) AND isnotnull(d_date_sk#17)) - -(34) CometProject -Input [2]: [d_date_sk#17, d_date#22] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] - -(36) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/simplified.txt deleted file mode 100644 index 14c21b32a5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -WholeStageCodegen (7) - HashAggregate [sum] [sum(UnscaledValue(cs_ext_discount_amt)),excess discount amount,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (6) - HashAggregate [cs_ext_discount_amt] [sum,sum] - Project [cs_ext_discount_amt] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_discount_amt,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk,cs_ext_discount_amt,(1.3 * avg(cs_ext_discount_amt))] - Project [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk,cs_ext_discount_amt] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Filter [(1.3 * avg(cs_ext_discount_amt))] - HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_item_sk] #5 - WholeStageCodegen (3) - HashAggregate [cs_item_sk,cs_ext_discount_amt] [sum,count,sum,count] - Project [cs_item_sk,cs_ext_discount_amt] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/explain.txt deleted file mode 100644 index d6c10008ec..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (30) -+- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : +- CometBroadcastExchange (21) - : +- CometFilter (20) - : +- CometHashAggregate (19) - : +- CometExchange (18) - : +- CometHashAggregate (17) - : +- CometProject (16) - : +- CometBroadcastHashJoin (15) - : :- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : +- CometBroadcastExchange (14) - : +- CometProject (13) - : +- CometFilter (12) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (11) - +- ReusedExchange (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_ext_discount_amt)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Condition : (isnotnull(cs_item_sk#1) AND isnotnull(cs_ext_discount_amt#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_manufact_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,977), IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_manufact_id#6] -Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 977)) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [2]: [i_item_sk#5, i_manufact_id#6] -Arguments: [i_item_sk#5], [i_item_sk#5] - -(6) CometBroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: [i_item_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Right output [1]: [i_item_sk#5] -Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] -Arguments: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5], [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Condition : isnotnull(cs_item_sk#7) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) - -(13) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(15) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(16) CometProject -Input [4]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9, d_date_sk#11] -Arguments: [cs_item_sk#7, cs_ext_discount_amt#8], [cs_item_sk#7, cs_ext_discount_amt#8] - -(17) CometHashAggregate -Input [2]: [cs_item_sk#7, cs_ext_discount_amt#8] -Keys [1]: [cs_item_sk#7] -Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#8))] - -(18) CometExchange -Input [3]: [cs_item_sk#7, sum#13, count#14] -Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [3]: [cs_item_sk#7, sum#13, count#14] -Keys [1]: [cs_item_sk#7] -Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))] - -(20) CometFilter -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#15) - -(21) CometBroadcastExchange -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Arguments: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] - -(22) CometBroadcastHashJoin -Left output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] -Right output [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Arguments: [i_item_sk#5], [cs_item_sk#7], Inner, (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#15), BuildRight - -(23) CometProject -Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Arguments: [cs_ext_discount_amt#2, cs_sold_date_sk#3], [cs_ext_discount_amt#2, cs_sold_date_sk#3] - -(24) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#16] - -(25) CometBroadcastHashJoin -Left output [2]: [cs_ext_discount_amt#2, cs_sold_date_sk#3] -Right output [1]: [d_date_sk#16] -Arguments: [cs_sold_date_sk#3], [d_date_sk#16], Inner, BuildRight - -(26) CometProject -Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#16] -Arguments: [cs_ext_discount_amt#2], [cs_ext_discount_amt#2] - -(27) CometHashAggregate -Input [1]: [cs_ext_discount_amt#2] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))] - -(28) CometExchange -Input [1]: [sum#17] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [1]: [sum#17] -Keys: [] -Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [excess discount amount#18] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#16, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#19 <= 2000-04-26)) AND isnotnull(d_date_sk#16)) - -(33) CometProject -Input [2]: [d_date_sk#16, d_date#19] -Arguments: [d_date_sk#16], [d_date_sk#16] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#16] - -(35) BroadcastExchange -Input [1]: [d_date_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/simplified.txt deleted file mode 100644 index 07619fc999..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [excess discount amount,sum(UnscaledValue(cs_ext_discount_amt))] - CometExchange #1 - CometHashAggregate [cs_ext_discount_amt] [sum] - CometProject [cs_ext_discount_amt] - CometBroadcastHashJoin [cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] - CometProject [cs_ext_discount_amt,cs_sold_date_sk] - CometBroadcastHashJoin [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk,(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] - CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk] #3 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] #4 - CometFilter [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] - CometHashAggregate [sum,count] [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,avg(UnscaledValue(cs_ext_discount_amt))] - CometExchange [cs_item_sk] #5 - CometHashAggregate [cs_ext_discount_amt] [cs_item_sk,sum,count] - CometProject [cs_item_sk,cs_ext_discount_amt] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/explain.txt deleted file mode 100644 index d6c10008ec..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (30) -+- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : +- CometBroadcastExchange (21) - : +- CometFilter (20) - : +- CometHashAggregate (19) - : +- CometExchange (18) - : +- CometHashAggregate (17) - : +- CometProject (16) - : +- CometBroadcastHashJoin (15) - : :- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : +- CometBroadcastExchange (14) - : +- CometProject (13) - : +- CometFilter (12) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (11) - +- ReusedExchange (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_ext_discount_amt)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Condition : (isnotnull(cs_item_sk#1) AND isnotnull(cs_ext_discount_amt#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_manufact_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,977), IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_manufact_id#6] -Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 977)) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [2]: [i_item_sk#5, i_manufact_id#6] -Arguments: [i_item_sk#5], [i_item_sk#5] - -(6) CometBroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: [i_item_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Right output [1]: [i_item_sk#5] -Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] -Arguments: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5], [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Condition : isnotnull(cs_item_sk#7) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) - -(13) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(15) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(16) CometProject -Input [4]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9, d_date_sk#11] -Arguments: [cs_item_sk#7, cs_ext_discount_amt#8], [cs_item_sk#7, cs_ext_discount_amt#8] - -(17) CometHashAggregate -Input [2]: [cs_item_sk#7, cs_ext_discount_amt#8] -Keys [1]: [cs_item_sk#7] -Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#8))] - -(18) CometExchange -Input [3]: [cs_item_sk#7, sum#13, count#14] -Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [3]: [cs_item_sk#7, sum#13, count#14] -Keys [1]: [cs_item_sk#7] -Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))] - -(20) CometFilter -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#15) - -(21) CometBroadcastExchange -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Arguments: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] - -(22) CometBroadcastHashJoin -Left output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] -Right output [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Arguments: [i_item_sk#5], [cs_item_sk#7], Inner, (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#15), BuildRight - -(23) CometProject -Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Arguments: [cs_ext_discount_amt#2, cs_sold_date_sk#3], [cs_ext_discount_amt#2, cs_sold_date_sk#3] - -(24) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#16] - -(25) CometBroadcastHashJoin -Left output [2]: [cs_ext_discount_amt#2, cs_sold_date_sk#3] -Right output [1]: [d_date_sk#16] -Arguments: [cs_sold_date_sk#3], [d_date_sk#16], Inner, BuildRight - -(26) CometProject -Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#16] -Arguments: [cs_ext_discount_amt#2], [cs_ext_discount_amt#2] - -(27) CometHashAggregate -Input [1]: [cs_ext_discount_amt#2] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))] - -(28) CometExchange -Input [1]: [sum#17] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [1]: [sum#17] -Keys: [] -Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [excess discount amount#18] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#16, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#19 <= 2000-04-26)) AND isnotnull(d_date_sk#16)) - -(33) CometProject -Input [2]: [d_date_sk#16, d_date#19] -Arguments: [d_date_sk#16], [d_date_sk#16] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#16] - -(35) BroadcastExchange -Input [1]: [d_date_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/extended.txt deleted file mode 100644 index 943bb147ff..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 07619fc999..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [excess discount amount,sum(UnscaledValue(cs_ext_discount_amt))] - CometExchange #1 - CometHashAggregate [cs_ext_discount_amt] [sum] - CometProject [cs_ext_discount_amt] - CometBroadcastHashJoin [cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] - CometProject [cs_ext_discount_amt,cs_sold_date_sk] - CometBroadcastHashJoin [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk,(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] - CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk] #3 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] #4 - CometFilter [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] - CometHashAggregate [sum,count] [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,avg(UnscaledValue(cs_ext_discount_amt))] - CometExchange [cs_item_sk] #5 - CometHashAggregate [cs_ext_discount_amt] [cs_item_sk,sum,count] - CometProject [cs_item_sk,cs_ext_discount_amt] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/explain.txt deleted file mode 100644 index 0cc777296b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/explain.txt +++ /dev/null @@ -1,415 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (66) -+- * HashAggregate (65) - +- * CometColumnarToRow (64) - +- CometColumnarExchange (63) - +- * HashAggregate (62) - +- Union (61) - :- * HashAggregate (28) - : +- * CometColumnarToRow (27) - : +- CometColumnarExchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (22) - : +- * CometColumnarToRow (21) - : +- CometBroadcastHashJoin (20) - : :- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.item (14) - : +- CometBroadcastExchange (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometNativeScan parquet spark_catalog.default.item (16) - :- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- CometColumnarExchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (34) - : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : :- * Filter (31) - : : : : +- * ColumnarToRow (30) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) - +- * HashAggregate (60) - +- * CometColumnarToRow (59) - +- CometColumnarExchange (58) - +- * HashAggregate (57) - +- * Project (56) - +- * BroadcastHashJoin Inner BuildRight (55) - :- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) - : :- * Project (50) - : : +- * BroadcastHashJoin Inner BuildRight (49) - : : :- * Filter (47) - : : : +- * ColumnarToRow (46) - : : : +- Scan parquet spark_catalog.default.web_sales (45) - : : +- ReusedExchange (48) - : +- ReusedExchange (51) - +- ReusedExchange (54) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(4) ReusedExchange [Reuses operator id: 71] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_gmt_offset#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isnotnull(ca_address_sk#7)) - -(9) CometProject -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Arguments: [ca_address_sk#7], [ca_address_sk#7] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [ca_address_sk#7] - -(11) BroadcastExchange -Input [1]: [ca_address_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] - -(14) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#9, i_manufact_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [i_item_sk#9, i_manufact_id#10] -Condition : isnotnull(i_item_sk#9) - -(16) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_category#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(17) CometFilter -Input [2]: [i_category#11, i_manufact_id#12] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) = Electronics ) - -(18) CometProject -Input [2]: [i_category#11, i_manufact_id#12] -Arguments: [i_manufact_id#12], [i_manufact_id#12] - -(19) CometBroadcastExchange -Input [1]: [i_manufact_id#12] -Arguments: [i_manufact_id#12] - -(20) CometBroadcastHashJoin -Left output [2]: [i_item_sk#9, i_manufact_id#10] -Right output [1]: [i_manufact_id#12] -Arguments: [i_manufact_id#10], [i_manufact_id#12], LeftSemi, BuildRight - -(21) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#9, i_manufact_id#10] - -(22) BroadcastExchange -Input [2]: [i_item_sk#9, i_manufact_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#9] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 4] -Output [2]: [ss_ext_sales_price#3, i_manufact_id#10] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_manufact_id#10] - -(25) HashAggregate [codegen id : 4] -Input [2]: [ss_ext_sales_price#3, i_manufact_id#10] -Keys [1]: [i_manufact_id#10] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum#13] -Results [2]: [i_manufact_id#10, sum#14] - -(26) CometColumnarExchange -Input [2]: [i_manufact_id#10, sum#14] -Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(27) CometColumnarToRow [codegen id : 5] -Input [2]: [i_manufact_id#10, sum#14] - -(28) HashAggregate [codegen id : 5] -Input [2]: [i_manufact_id#10, sum#14] -Keys [1]: [i_manufact_id#10] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#15] -Results [2]: [i_manufact_id#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#15,17,2) AS total_sales#16] - -(29) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(30) ColumnarToRow [codegen id : 9] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] - -(31) Filter [codegen id : 9] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) - -(32) ReusedExchange [Reuses operator id: 71] -Output [1]: [d_date_sk#21] - -(33) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#20] -Right keys [1]: [d_date_sk#21] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 9] -Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] -Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#21] - -(35) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#22] - -(36) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_bill_addr_sk#17] -Right keys [1]: [ca_address_sk#22] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 9] -Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#22] - -(38) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#23, i_manufact_id#24] - -(39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#23] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 9] -Output [2]: [cs_ext_sales_price#19, i_manufact_id#24] -Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#23, i_manufact_id#24] - -(41) HashAggregate [codegen id : 9] -Input [2]: [cs_ext_sales_price#19, i_manufact_id#24] -Keys [1]: [i_manufact_id#24] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum#25] -Results [2]: [i_manufact_id#24, sum#26] - -(42) CometColumnarExchange -Input [2]: [i_manufact_id#24, sum#26] -Arguments: hashpartitioning(i_manufact_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(43) CometColumnarToRow [codegen id : 10] -Input [2]: [i_manufact_id#24, sum#26] - -(44) HashAggregate [codegen id : 10] -Input [2]: [i_manufact_id#24, sum#26] -Keys [1]: [i_manufact_id#24] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#19))#27] -Results [2]: [i_manufact_id#24, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#19))#27,17,2) AS total_sales#28] - -(45) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#32), dynamicpruningexpression(ws_sold_date_sk#32 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 14] -Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32] - -(47) Filter [codegen id : 14] -Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32] -Condition : (isnotnull(ws_bill_addr_sk#30) AND isnotnull(ws_item_sk#29)) - -(48) ReusedExchange [Reuses operator id: 71] -Output [1]: [d_date_sk#33] - -(49) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#32] -Right keys [1]: [d_date_sk#33] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 14] -Output [3]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31] -Input [5]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32, d_date_sk#33] - -(51) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#34] - -(52) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#30] -Right keys [1]: [ca_address_sk#34] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 14] -Output [2]: [ws_item_sk#29, ws_ext_sales_price#31] -Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ca_address_sk#34] - -(54) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#35, i_manufact_id#36] - -(55) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_item_sk#29] -Right keys [1]: [i_item_sk#35] -Join type: Inner -Join condition: None - -(56) Project [codegen id : 14] -Output [2]: [ws_ext_sales_price#31, i_manufact_id#36] -Input [4]: [ws_item_sk#29, ws_ext_sales_price#31, i_item_sk#35, i_manufact_id#36] - -(57) HashAggregate [codegen id : 14] -Input [2]: [ws_ext_sales_price#31, i_manufact_id#36] -Keys [1]: [i_manufact_id#36] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#31))] -Aggregate Attributes [1]: [sum#37] -Results [2]: [i_manufact_id#36, sum#38] - -(58) CometColumnarExchange -Input [2]: [i_manufact_id#36, sum#38] -Arguments: hashpartitioning(i_manufact_id#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(59) CometColumnarToRow [codegen id : 15] -Input [2]: [i_manufact_id#36, sum#38] - -(60) HashAggregate [codegen id : 15] -Input [2]: [i_manufact_id#36, sum#38] -Keys [1]: [i_manufact_id#36] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#31))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#31))#39] -Results [2]: [i_manufact_id#36, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#31))#39,17,2) AS total_sales#40] - -(61) Union - -(62) HashAggregate [codegen id : 16] -Input [2]: [i_manufact_id#10, total_sales#16] -Keys [1]: [i_manufact_id#10] -Functions [1]: [partial_sum(total_sales#16)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [3]: [i_manufact_id#10, sum#43, isEmpty#44] - -(63) CometColumnarExchange -Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] -Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(64) CometColumnarToRow [codegen id : 17] -Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] - -(65) HashAggregate [codegen id : 17] -Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] -Keys [1]: [i_manufact_id#10] -Functions [1]: [sum(total_sales#16)] -Aggregate Attributes [1]: [sum(total_sales#16)#45] -Results [2]: [i_manufact_id#10, sum(total_sales#16)#45 AS total_sales#46] - -(66) TakeOrderedAndProject -Input [2]: [i_manufact_id#10, total_sales#46] -Arguments: 100, [total_sales#46 ASC NULLS FIRST], [i_manufact_id#10, total_sales#46] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometProject (69) - +- CometFilter (68) - +- CometNativeScan parquet spark_catalog.default.date_dim (67) - - -(67) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#47, d_moy#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] -ReadSchema: struct - -(68) CometFilter -Input [3]: [d_date_sk#6, d_year#47, d_moy#48] -Condition : ((((isnotnull(d_year#47) AND isnotnull(d_moy#48)) AND (d_year#47 = 1998)) AND (d_moy#48 = 5)) AND isnotnull(d_date_sk#6)) - -(69) CometProject -Input [3]: [d_date_sk#6, d_year#47, d_moy#48] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(70) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(71) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#32 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/simplified.txt deleted file mode 100644 index fd1c1d0ef2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/simplified.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject [total_sales,i_manufact_id] - WholeStageCodegen (17) - HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id] #1 - WholeStageCodegen (16) - HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id] #2 - WholeStageCodegen (4) - HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] - CometFilter [i_item_sk,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [i_manufact_id] #6 - CometProject [i_manufact_id] - CometFilter [i_category,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_category,i_manufact_id] - WholeStageCodegen (10) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id] #7 - WholeStageCodegen (9) - HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_addr_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 - WholeStageCodegen (15) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id] #8 - WholeStageCodegen (14) - HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/explain.txt deleted file mode 100644 index a178860ccc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,380 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (61) -+- CometTakeOrderedAndProject (60) - +- CometHashAggregate (59) - +- CometExchange (58) - +- CometHashAggregate (57) - +- CometUnion (56) - :- CometHashAggregate (27) - : +- CometExchange (26) - : +- CometHashAggregate (25) - : +- CometProject (24) - : +- CometBroadcastHashJoin (23) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : +- CometBroadcastExchange (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometFilter (29) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (28) - : : : +- ReusedExchange (30) - : : +- ReusedExchange (33) - : +- ReusedExchange (36) - +- CometHashAggregate (55) - +- CometExchange (54) - +- CometHashAggregate (53) - +- CometProject (52) - +- CometBroadcastHashJoin (51) - :- CometProject (49) - : +- CometBroadcastHashJoin (48) - : :- CometProject (46) - : : +- CometBroadcastHashJoin (45) - : : :- CometFilter (43) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : : +- ReusedExchange (44) - : +- ReusedExchange (47) - +- ReusedExchange (50) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(11) CometProject -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Arguments: [ca_address_sk#9], [ca_address_sk#9] - -(12) CometBroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: [ca_address_sk#9] - -(13) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Right output [1]: [ca_address_sk#9] -Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] -Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#11, i_manufact_id#12] -Condition : isnotnull(i_item_sk#11) - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_category#13, i_manufact_id#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(18) CometFilter -Input [2]: [i_category#13, i_manufact_id#14] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#13, 50, true, false, true) = Electronics ) - -(19) CometProject -Input [2]: [i_category#13, i_manufact_id#14] -Arguments: [i_manufact_id#14], [i_manufact_id#14] - -(20) CometBroadcastExchange -Input [1]: [i_manufact_id#14] -Arguments: [i_manufact_id#14] - -(21) CometBroadcastHashJoin -Left output [2]: [i_item_sk#11, i_manufact_id#12] -Right output [1]: [i_manufact_id#14] -Arguments: [i_manufact_id#12], [i_manufact_id#14], LeftSemi, BuildRight - -(22) CometBroadcastExchange -Input [2]: [i_item_sk#11, i_manufact_id#12] -Arguments: [i_item_sk#11, i_manufact_id#12] - -(23) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Right output [2]: [i_item_sk#11, i_manufact_id#12] -Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight - -(24) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_manufact_id#12] -Arguments: [ss_ext_sales_price#3, i_manufact_id#12], [ss_ext_sales_price#3, i_manufact_id#12] - -(25) CometHashAggregate -Input [2]: [ss_ext_sales_price#3, i_manufact_id#12] -Keys [1]: [i_manufact_id#12] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(26) CometExchange -Input [2]: [i_manufact_id#12, sum#15] -Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [2]: [i_manufact_id#12, sum#15] -Keys [1]: [i_manufact_id#12] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#20)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(29) CometFilter -Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] -Condition : (isnotnull(cs_bill_addr_sk#16) AND isnotnull(cs_item_sk#17)) - -(30) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#21] - -(31) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] -Right output [1]: [d_date_sk#21] -Arguments: [cs_sold_date_sk#19], [d_date_sk#21], Inner, BuildRight - -(32) CometProject -Input [5]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19, d_date_sk#21] -Arguments: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18], [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] - -(33) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#22] - -(34) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] -Right output [1]: [ca_address_sk#22] -Arguments: [cs_bill_addr_sk#16], [ca_address_sk#22], Inner, BuildRight - -(35) CometProject -Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, ca_address_sk#22] -Arguments: [cs_item_sk#17, cs_ext_sales_price#18], [cs_item_sk#17, cs_ext_sales_price#18] - -(36) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#23, i_manufact_id#24] - -(37) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_ext_sales_price#18] -Right output [2]: [i_item_sk#23, i_manufact_id#24] -Arguments: [cs_item_sk#17], [i_item_sk#23], Inner, BuildRight - -(38) CometProject -Input [4]: [cs_item_sk#17, cs_ext_sales_price#18, i_item_sk#23, i_manufact_id#24] -Arguments: [cs_ext_sales_price#18, i_manufact_id#24], [cs_ext_sales_price#18, i_manufact_id#24] - -(39) CometHashAggregate -Input [2]: [cs_ext_sales_price#18, i_manufact_id#24] -Keys [1]: [i_manufact_id#24] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#18))] - -(40) CometExchange -Input [2]: [i_manufact_id#24, sum#25] -Arguments: hashpartitioning(i_manufact_id#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(41) CometHashAggregate -Input [2]: [i_manufact_id#24, sum#25] -Keys [1]: [i_manufact_id#24] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#18))] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(43) CometFilter -Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Condition : (isnotnull(ws_bill_addr_sk#27) AND isnotnull(ws_item_sk#26)) - -(44) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#31] - -(45) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Right output [1]: [d_date_sk#31] -Arguments: [ws_sold_date_sk#29], [d_date_sk#31], Inner, BuildRight - -(46) CometProject -Input [5]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29, d_date_sk#31] -Arguments: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28], [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] - -(47) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#32] - -(48) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] -Right output [1]: [ca_address_sk#32] -Arguments: [ws_bill_addr_sk#27], [ca_address_sk#32], Inner, BuildRight - -(49) CometProject -Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ca_address_sk#32] -Arguments: [ws_item_sk#26, ws_ext_sales_price#28], [ws_item_sk#26, ws_ext_sales_price#28] - -(50) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#33, i_manufact_id#34] - -(51) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#26, ws_ext_sales_price#28] -Right output [2]: [i_item_sk#33, i_manufact_id#34] -Arguments: [ws_item_sk#26], [i_item_sk#33], Inner, BuildRight - -(52) CometProject -Input [4]: [ws_item_sk#26, ws_ext_sales_price#28, i_item_sk#33, i_manufact_id#34] -Arguments: [ws_ext_sales_price#28, i_manufact_id#34], [ws_ext_sales_price#28, i_manufact_id#34] - -(53) CometHashAggregate -Input [2]: [ws_ext_sales_price#28, i_manufact_id#34] -Keys [1]: [i_manufact_id#34] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#28))] - -(54) CometExchange -Input [2]: [i_manufact_id#34, sum#35] -Arguments: hashpartitioning(i_manufact_id#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(55) CometHashAggregate -Input [2]: [i_manufact_id#34, sum#35] -Keys [1]: [i_manufact_id#34] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#28))] - -(56) CometUnion -Child 0 Input [2]: [i_manufact_id#12, total_sales#36] -Child 1 Input [2]: [i_manufact_id#24, total_sales#37] -Child 2 Input [2]: [i_manufact_id#34, total_sales#38] - -(57) CometHashAggregate -Input [2]: [i_manufact_id#12, total_sales#36] -Keys [1]: [i_manufact_id#12] -Functions [1]: [partial_sum(total_sales#36)] - -(58) CometExchange -Input [3]: [i_manufact_id#12, sum#39, isEmpty#40] -Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(59) CometHashAggregate -Input [3]: [i_manufact_id#12, sum#39, isEmpty#40] -Keys [1]: [i_manufact_id#12] -Functions [1]: [sum(total_sales#36)] - -(60) CometTakeOrderedAndProject -Input [2]: [i_manufact_id#12, total_sales#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#41 ASC NULLS FIRST], output=[i_manufact_id#12,total_sales#41]), [i_manufact_id#12, total_sales#41], 100, 0, [total_sales#41 ASC NULLS FIRST], [i_manufact_id#12, total_sales#41] - -(61) CometColumnarToRow [codegen id : 1] -Input [2]: [i_manufact_id#12, total_sales#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (66) -+- * CometColumnarToRow (65) - +- CometProject (64) - +- CometFilter (63) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (62) - - -(62) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] -ReadSchema: struct - -(63) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6)) - -(64) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(65) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(66) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/simplified.txt deleted file mode 100644 index b4b84c2ac6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_manufact_id,total_sales] - CometHashAggregate [sum,isEmpty] [i_manufact_id,total_sales,sum(total_sales)] - CometExchange [i_manufact_id] #1 - CometHashAggregate [total_sales] [i_manufact_id,sum,isEmpty] - CometUnion [i_manufact_id,total_sales] - CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_manufact_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_manufact_id,sum] - CometProject [ss_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [i_manufact_id] #7 - CometProject [i_manufact_id] - CometFilter [i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact_id] - CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_manufact_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_manufact_id,sum] - CometProject [cs_ext_sales_price,i_manufact_id] - 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,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_manufact_id] #6 - CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_manufact_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_manufact_id,sum] - CometProject [ws_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [ws_item_sk,ws_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/explain.txt deleted file mode 100644 index a178860ccc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/explain.txt +++ /dev/null @@ -1,380 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (61) -+- CometTakeOrderedAndProject (60) - +- CometHashAggregate (59) - +- CometExchange (58) - +- CometHashAggregate (57) - +- CometUnion (56) - :- CometHashAggregate (27) - : +- CometExchange (26) - : +- CometHashAggregate (25) - : +- CometProject (24) - : +- CometBroadcastHashJoin (23) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : +- CometBroadcastExchange (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometFilter (29) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (28) - : : : +- ReusedExchange (30) - : : +- ReusedExchange (33) - : +- ReusedExchange (36) - +- CometHashAggregate (55) - +- CometExchange (54) - +- CometHashAggregate (53) - +- CometProject (52) - +- CometBroadcastHashJoin (51) - :- CometProject (49) - : +- CometBroadcastHashJoin (48) - : :- CometProject (46) - : : +- CometBroadcastHashJoin (45) - : : :- CometFilter (43) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : : +- ReusedExchange (44) - : +- ReusedExchange (47) - +- ReusedExchange (50) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(11) CometProject -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Arguments: [ca_address_sk#9], [ca_address_sk#9] - -(12) CometBroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: [ca_address_sk#9] - -(13) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Right output [1]: [ca_address_sk#9] -Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] -Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#11, i_manufact_id#12] -Condition : isnotnull(i_item_sk#11) - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_category#13, i_manufact_id#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(18) CometFilter -Input [2]: [i_category#13, i_manufact_id#14] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#13, 50, true, false, true) = Electronics ) - -(19) CometProject -Input [2]: [i_category#13, i_manufact_id#14] -Arguments: [i_manufact_id#14], [i_manufact_id#14] - -(20) CometBroadcastExchange -Input [1]: [i_manufact_id#14] -Arguments: [i_manufact_id#14] - -(21) CometBroadcastHashJoin -Left output [2]: [i_item_sk#11, i_manufact_id#12] -Right output [1]: [i_manufact_id#14] -Arguments: [i_manufact_id#12], [i_manufact_id#14], LeftSemi, BuildRight - -(22) CometBroadcastExchange -Input [2]: [i_item_sk#11, i_manufact_id#12] -Arguments: [i_item_sk#11, i_manufact_id#12] - -(23) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Right output [2]: [i_item_sk#11, i_manufact_id#12] -Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight - -(24) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_manufact_id#12] -Arguments: [ss_ext_sales_price#3, i_manufact_id#12], [ss_ext_sales_price#3, i_manufact_id#12] - -(25) CometHashAggregate -Input [2]: [ss_ext_sales_price#3, i_manufact_id#12] -Keys [1]: [i_manufact_id#12] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(26) CometExchange -Input [2]: [i_manufact_id#12, sum#15] -Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [2]: [i_manufact_id#12, sum#15] -Keys [1]: [i_manufact_id#12] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#20)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(29) CometFilter -Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] -Condition : (isnotnull(cs_bill_addr_sk#16) AND isnotnull(cs_item_sk#17)) - -(30) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#21] - -(31) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] -Right output [1]: [d_date_sk#21] -Arguments: [cs_sold_date_sk#19], [d_date_sk#21], Inner, BuildRight - -(32) CometProject -Input [5]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19, d_date_sk#21] -Arguments: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18], [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] - -(33) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#22] - -(34) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] -Right output [1]: [ca_address_sk#22] -Arguments: [cs_bill_addr_sk#16], [ca_address_sk#22], Inner, BuildRight - -(35) CometProject -Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, ca_address_sk#22] -Arguments: [cs_item_sk#17, cs_ext_sales_price#18], [cs_item_sk#17, cs_ext_sales_price#18] - -(36) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#23, i_manufact_id#24] - -(37) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_ext_sales_price#18] -Right output [2]: [i_item_sk#23, i_manufact_id#24] -Arguments: [cs_item_sk#17], [i_item_sk#23], Inner, BuildRight - -(38) CometProject -Input [4]: [cs_item_sk#17, cs_ext_sales_price#18, i_item_sk#23, i_manufact_id#24] -Arguments: [cs_ext_sales_price#18, i_manufact_id#24], [cs_ext_sales_price#18, i_manufact_id#24] - -(39) CometHashAggregate -Input [2]: [cs_ext_sales_price#18, i_manufact_id#24] -Keys [1]: [i_manufact_id#24] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#18))] - -(40) CometExchange -Input [2]: [i_manufact_id#24, sum#25] -Arguments: hashpartitioning(i_manufact_id#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(41) CometHashAggregate -Input [2]: [i_manufact_id#24, sum#25] -Keys [1]: [i_manufact_id#24] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#18))] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(43) CometFilter -Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Condition : (isnotnull(ws_bill_addr_sk#27) AND isnotnull(ws_item_sk#26)) - -(44) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#31] - -(45) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Right output [1]: [d_date_sk#31] -Arguments: [ws_sold_date_sk#29], [d_date_sk#31], Inner, BuildRight - -(46) CometProject -Input [5]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29, d_date_sk#31] -Arguments: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28], [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] - -(47) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#32] - -(48) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] -Right output [1]: [ca_address_sk#32] -Arguments: [ws_bill_addr_sk#27], [ca_address_sk#32], Inner, BuildRight - -(49) CometProject -Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ca_address_sk#32] -Arguments: [ws_item_sk#26, ws_ext_sales_price#28], [ws_item_sk#26, ws_ext_sales_price#28] - -(50) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#33, i_manufact_id#34] - -(51) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#26, ws_ext_sales_price#28] -Right output [2]: [i_item_sk#33, i_manufact_id#34] -Arguments: [ws_item_sk#26], [i_item_sk#33], Inner, BuildRight - -(52) CometProject -Input [4]: [ws_item_sk#26, ws_ext_sales_price#28, i_item_sk#33, i_manufact_id#34] -Arguments: [ws_ext_sales_price#28, i_manufact_id#34], [ws_ext_sales_price#28, i_manufact_id#34] - -(53) CometHashAggregate -Input [2]: [ws_ext_sales_price#28, i_manufact_id#34] -Keys [1]: [i_manufact_id#34] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#28))] - -(54) CometExchange -Input [2]: [i_manufact_id#34, sum#35] -Arguments: hashpartitioning(i_manufact_id#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(55) CometHashAggregate -Input [2]: [i_manufact_id#34, sum#35] -Keys [1]: [i_manufact_id#34] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#28))] - -(56) CometUnion -Child 0 Input [2]: [i_manufact_id#12, total_sales#36] -Child 1 Input [2]: [i_manufact_id#24, total_sales#37] -Child 2 Input [2]: [i_manufact_id#34, total_sales#38] - -(57) CometHashAggregate -Input [2]: [i_manufact_id#12, total_sales#36] -Keys [1]: [i_manufact_id#12] -Functions [1]: [partial_sum(total_sales#36)] - -(58) CometExchange -Input [3]: [i_manufact_id#12, sum#39, isEmpty#40] -Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(59) CometHashAggregate -Input [3]: [i_manufact_id#12, sum#39, isEmpty#40] -Keys [1]: [i_manufact_id#12] -Functions [1]: [sum(total_sales#36)] - -(60) CometTakeOrderedAndProject -Input [2]: [i_manufact_id#12, total_sales#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#41 ASC NULLS FIRST], output=[i_manufact_id#12,total_sales#41]), [i_manufact_id#12, total_sales#41], 100, 0, [total_sales#41 ASC NULLS FIRST], [i_manufact_id#12, total_sales#41] - -(61) CometColumnarToRow [codegen id : 1] -Input [2]: [i_manufact_id#12, total_sales#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (66) -+- * CometColumnarToRow (65) - +- CometProject (64) - +- CometFilter (63) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (62) - - -(62) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] -ReadSchema: struct - -(63) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6)) - -(64) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(65) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(66) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/extended.txt deleted file mode 100644 index 063fdb6faf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/extended.txt +++ /dev/null @@ -1,97 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 89 out of 93 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index b4b84c2ac6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_manufact_id,total_sales] - CometHashAggregate [sum,isEmpty] [i_manufact_id,total_sales,sum(total_sales)] - CometExchange [i_manufact_id] #1 - CometHashAggregate [total_sales] [i_manufact_id,sum,isEmpty] - CometUnion [i_manufact_id,total_sales] - CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_manufact_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_manufact_id,sum] - CometProject [ss_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [i_manufact_id] #7 - CometProject [i_manufact_id] - CometFilter [i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact_id] - CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_manufact_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_manufact_id,sum] - CometProject [cs_ext_sales_price,i_manufact_id] - 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,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_manufact_id] #6 - CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_manufact_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_manufact_id,sum] - CometProject [ws_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [ws_item_sk,ws_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/explain.txt deleted file mode 100644 index 18c904ceed..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/explain.txt +++ /dev/null @@ -1,231 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (35) -+- CometSort (34) - +- CometColumnarExchange (33) - +- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * Filter (25) - : +- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (30) - +- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometNativeScan parquet spark_catalog.default.customer (26) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 40] -Output [1]: [d_date_sk#7] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_county#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [s_store_sk#8, s_county#9] -Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND isnotnull(s_store_sk#8)) - -(9) CometProject -Input [2]: [s_store_sk#8, s_county#9] -Arguments: [s_store_sk#8], [s_store_sk#8] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#8] - -(11) BroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] - -(14) CometNativeScan parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : ((((isnotnull(hd_vehicle_count#13) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#11, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#11, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#12 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#13 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#10)) - -(16) CometProject -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Arguments: [hd_demo_sk#10], [hd_demo_sk#10] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#10] - -(18) BroadcastExchange -Input [1]: [hd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#10] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [2]: [ss_customer_sk#1, ss_ticket_number#4] -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] - -(21) HashAggregate [codegen id : 4] -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#14] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(22) CometColumnarExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(24) HashAggregate [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#16] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] - -(25) Filter [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) - -(26) CometNativeScan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(27) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(28) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] - -(29) CometColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(30) BroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#18] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 6] -Output [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(33) CometColumnarExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(34) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST] - -(35) CometColumnarToRow [codegen id : 7] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (40) -+- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometNativeScan parquet spark_catalog.default.date_dim (36) - - -(36) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#27, d_dom#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [d_date_sk#7, d_year#27, d_dom#28] -Condition : (((((d_dom#28 >= 1) AND (d_dom#28 <= 3)) OR ((d_dom#28 >= 25) AND (d_dom#28 <= 28))) AND d_year#27 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(38) CometProject -Input [3]: [d_date_sk#7, d_year#27, d_dom#28] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(39) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(40) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/simplified.txt deleted file mode 100644 index 10af5ceadc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/simplified.txt +++ /dev/null @@ -1,59 +0,0 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometNativeScan 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/q34.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/explain.txt deleted file mode 100644 index 287e323952..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometSort (32) - +- CometExchange (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (24) - : +- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_county#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#10, s_county#11] -Condition : ((isnotnull(s_county#11) AND (s_county#11 = Williamson County)) AND isnotnull(s_store_sk#10)) - -(11) CometProject -Input [2]: [s_store_sk#10, s_county#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : ((((isnotnull(hd_vehicle_count#15) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#14 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#15 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#12)) - -(17) CometProject -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#12], [hd_demo_sk#12] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#12] -Arguments: [hd_demo_sk#12] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Right output [1]: [hd_demo_sk#12] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight - -(20) CometProject -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] -Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] - -(21) CometHashAggregate -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] - -(22) CometExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] - -(24) CometFilter -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(27) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(29) CometBroadcastHashJoin -Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Right output [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -(31) CometExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(32) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST] - -(33) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(36) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(37) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(38) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/simplified.txt deleted file mode 100644 index 47d0e35dd2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 - CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] - CometExchange [ss_ticket_number,ss_customer_sk] #2 - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - 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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/explain.txt deleted file mode 100644 index 287e323952..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometSort (32) - +- CometExchange (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (24) - : +- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_county#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#10, s_county#11] -Condition : ((isnotnull(s_county#11) AND (s_county#11 = Williamson County)) AND isnotnull(s_store_sk#10)) - -(11) CometProject -Input [2]: [s_store_sk#10, s_county#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : ((((isnotnull(hd_vehicle_count#15) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#14 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#15 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#12)) - -(17) CometProject -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#12], [hd_demo_sk#12] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#12] -Arguments: [hd_demo_sk#12] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Right output [1]: [hd_demo_sk#12] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight - -(20) CometProject -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] -Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] - -(21) CometHashAggregate -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] - -(22) CometExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] - -(24) CometFilter -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(27) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(29) CometBroadcastHashJoin -Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Right output [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -(31) CometExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(32) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST] - -(33) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(36) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(37) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(38) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/extended.txt deleted file mode 100644 index 3d9d23d2cc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/extended.txt +++ /dev/null @@ -1,41 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 47d0e35dd2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 - CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] - CometExchange [ss_ticket_number,ss_customer_sk] #2 - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - 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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/explain.txt deleted file mode 100644 index 35d06d16ae..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/explain.txt +++ /dev/null @@ -1,295 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (45) -+- * HashAggregate (44) - +- * CometColumnarToRow (43) - +- CometColumnarExchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometNativeScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (38) - +- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometNativeScan parquet spark_catalog.default.customer_demographics (34) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(4) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] - -(6) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#9] - -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None - -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#6] -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] - -(9) BroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#6] -Join type: LeftSemi -Join condition: None - -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] - -(13) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#12] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#10] -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] - -(16) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(17) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#10] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] - -(20) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#15] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#13] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] - -(23) BroadcastExchange -Input [1]: [cs_ship_customer_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#13] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(25) Filter [codegen id : 9] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(26) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(27) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : isnotnull(ca_address_sk#16) - -(29) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16, ca_state#18], [ca_address_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#17, 2, true, false, true) AS ca_state#18] - -(30) CometColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#16, ca_state#18] - -(31) BroadcastExchange -Input [2]: [ca_address_sk#16, ca_state#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#16] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#18] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#16, ca_state#18] - -(34) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(35) CometFilter -Input [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Condition : isnotnull(cd_demo_sk#19) - -(36) CometProject -Input [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Arguments: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24], [cd_demo_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#20, 1, true, false, true) AS cd_gender#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#21, 1, true, false, true) AS cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] - -(37) CometColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] - -(38) BroadcastExchange -Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#19] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 9] -Output [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Input [8]: [c_current_cdemo_sk#4, ca_state#18, cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] - -(41) HashAggregate [codegen id : 9] -Input [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#22), partial_max(cd_dep_count#22), partial_avg(cd_dep_count#22), partial_min(cd_dep_employed_count#23), partial_max(cd_dep_employed_count#23), partial_avg(cd_dep_employed_count#23), partial_min(cd_dep_college_count#24), partial_max(cd_dep_college_count#24), partial_avg(cd_dep_college_count#24)] -Aggregate Attributes [13]: [count#27, min#28, max#29, sum#30, count#31, min#32, max#33, sum#34, count#35, min#36, max#37, sum#38, count#39] -Results [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, min#41, max#42, sum#43, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52] - -(42) CometColumnarExchange -Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, min#41, max#42, sum#43, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52] -Arguments: hashpartitioning(ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) CometColumnarToRow [codegen id : 10] -Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, min#41, max#42, sum#43, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52] - -(44) HashAggregate [codegen id : 10] -Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, min#41, max#42, sum#43, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52] -Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Functions [10]: [count(1), min(cd_dep_count#22), max(cd_dep_count#22), avg(cd_dep_count#22), min(cd_dep_employed_count#23), max(cd_dep_employed_count#23), avg(cd_dep_employed_count#23), min(cd_dep_college_count#24), max(cd_dep_college_count#24), avg(cd_dep_college_count#24)] -Aggregate Attributes [10]: [count(1)#53, min(cd_dep_count#22)#54, max(cd_dep_count#22)#55, avg(cd_dep_count#22)#56, min(cd_dep_employed_count#23)#57, max(cd_dep_employed_count#23)#58, avg(cd_dep_employed_count#23)#59, min(cd_dep_college_count#24)#60, max(cd_dep_college_count#24)#61, avg(cd_dep_college_count#24)#62] -Results [18]: [ca_state#18, cd_gender#25, cd_marital_status#26, count(1)#53 AS cnt1#63, min(cd_dep_count#22)#54 AS min(cd_dep_count)#64, max(cd_dep_count#22)#55 AS max(cd_dep_count)#65, avg(cd_dep_count#22)#56 AS avg(cd_dep_count)#66, cd_dep_employed_count#23, count(1)#53 AS cnt2#67, min(cd_dep_employed_count#23)#57 AS min(cd_dep_employed_count)#68, max(cd_dep_employed_count#23)#58 AS max(cd_dep_employed_count)#69, avg(cd_dep_employed_count#23)#59 AS avg(cd_dep_employed_count)#70, cd_dep_college_count#24, count(1)#53 AS cnt3#71, min(cd_dep_college_count#24)#60 AS min(cd_dep_college_count)#72, max(cd_dep_college_count#24)#61 AS max(cd_dep_college_count)#73, avg(cd_dep_college_count#24)#62 AS avg(cd_dep_college_count)#74, cd_dep_count#22] - -(45) TakeOrderedAndProject -Input [18]: [ca_state#18, cd_gender#25, cd_marital_status#26, cnt1#63, min(cd_dep_count)#64, max(cd_dep_count)#65, avg(cd_dep_count)#66, cd_dep_employed_count#23, cnt2#67, min(cd_dep_employed_count)#68, max(cd_dep_employed_count)#69, avg(cd_dep_employed_count)#70, cd_dep_college_count#24, cnt3#71, min(cd_dep_college_count)#72, max(cd_dep_college_count)#73, avg(cd_dep_college_count)#74, cd_dep_count#22] -Arguments: 100, [ca_state#18 ASC NULLS FIRST, cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_dep_count#22 ASC NULLS FIRST, cd_dep_employed_count#23 ASC NULLS FIRST, cd_dep_college_count#24 ASC NULLS FIRST], [ca_state#18, cd_gender#25, cd_marital_status#26, cnt1#63, min(cd_dep_count)#64, max(cd_dep_count)#65, avg(cd_dep_count)#66, cd_dep_employed_count#23, cnt2#67, min(cd_dep_employed_count)#68, max(cd_dep_employed_count)#69, avg(cd_dep_employed_count)#70, cd_dep_college_count#24, cnt3#71, min(cd_dep_college_count)#72, max(cd_dep_college_count)#73, avg(cd_dep_college_count)#74] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (50) -+- * CometColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometNativeScan parquet spark_catalog.default.date_dim (46) - - -(46) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#75, d_qoy#76] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_date_sk#9, d_year#75, d_qoy#76] -Condition : ((((isnotnull(d_year#75) AND isnotnull(d_qoy#76)) AND (d_year#75 = 2002)) AND (d_qoy#76 < 4)) AND isnotnull(d_date_sk#9)) - -(48) CometProject -Input [3]: [d_date_sk#9, d_year#75, d_qoy#76] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(49) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(50) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/simplified.txt deleted file mode 100644 index 48bc5637d0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/simplified.txt +++ /dev/null @@ -1,77 +0,0 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (10) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan 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/q35.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/explain.txt deleted file mode 100644 index fedc50de1e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * Filter (27) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(6) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: [d_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#9] -Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -Arguments: [ss_customer_sk#6], [ss_customer_sk#6] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: [ss_customer_sk#6] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Right output [1]: [ss_customer_sk#6] -Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#15] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Right output [1]: [d_date_sk#15] -Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] -Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#12] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#12] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#19] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#16] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#16] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(27) Filter [codegen id : 5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(28) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : isnotnull(ca_address_sk#20) - -(31) CometProject -Input [2]: [ca_address_sk#20, ca_state#21] -Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#21, 2, true, false, true) AS ca_state#22] - -(32) CometColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#20, ca_state#22] - -(33) BroadcastExchange -Input [2]: [ca_address_sk#20, ca_state#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(34) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, ca_state#22] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#22] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(37) CometFilter -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#23) - -(38) CometProject -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#24, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(39) CometColumnarToRow [codegen id : 4] -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(40) BroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(41) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#23] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 5] -Output [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [8]: [c_current_cdemo_sk#4, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(43) HashAggregate [codegen id : 5] -Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#26), partial_max(cd_dep_count#26), partial_avg(cd_dep_count#26), partial_min(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_avg(cd_dep_employed_count#27), partial_min(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_avg(cd_dep_college_count#28)] -Aggregate Attributes [13]: [count#31, min#32, max#33, sum#34, count#35, min#36, max#37, sum#38, count#39, min#40, max#41, sum#42, count#43] -Results [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] - -(44) CometColumnarExchange -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] -Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 6] -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] - -(46) HashAggregate [codegen id : 6] -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [count(1), min(cd_dep_count#26), max(cd_dep_count#26), avg(cd_dep_count#26), min(cd_dep_employed_count#27), max(cd_dep_employed_count#27), avg(cd_dep_employed_count#27), min(cd_dep_college_count#28), max(cd_dep_college_count#28), avg(cd_dep_college_count#28)] -Aggregate Attributes [10]: [count(1)#57, min(cd_dep_count#26)#58, max(cd_dep_count#26)#59, avg(cd_dep_count#26)#60, min(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, avg(cd_dep_employed_count#27)#63, min(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, avg(cd_dep_college_count#28)#66] -Results [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, count(1)#57 AS cnt1#67, min(cd_dep_count#26)#58 AS min(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, avg(cd_dep_count#26)#60 AS avg(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, min(cd_dep_employed_count#27)#61 AS min(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, avg(cd_dep_employed_count#27)#63 AS avg(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, min(cd_dep_college_count#28)#64 AS min(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, avg(cd_dep_college_count#28)#66 AS avg(cd_dep_college_count)#78, cd_dep_count#26] - -(47) TakeOrderedAndProject -Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78, cd_dep_count#26] -Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#22, cd_gender#29, cd_marital_status#30, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(52) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/simplified.txt deleted file mode 100644 index f1fe09fb46..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (6) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] 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/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/explain.txt deleted file mode 100644 index fedc50de1e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * Filter (27) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(6) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: [d_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#9] -Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -Arguments: [ss_customer_sk#6], [ss_customer_sk#6] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: [ss_customer_sk#6] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Right output [1]: [ss_customer_sk#6] -Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#15] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Right output [1]: [d_date_sk#15] -Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] -Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#12] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#12] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#19] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#16] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#16] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(27) Filter [codegen id : 5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(28) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : isnotnull(ca_address_sk#20) - -(31) CometProject -Input [2]: [ca_address_sk#20, ca_state#21] -Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#21, 2, true, false, true) AS ca_state#22] - -(32) CometColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#20, ca_state#22] - -(33) BroadcastExchange -Input [2]: [ca_address_sk#20, ca_state#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(34) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, ca_state#22] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#22] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(37) CometFilter -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#23) - -(38) CometProject -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#24, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(39) CometColumnarToRow [codegen id : 4] -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(40) BroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(41) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#23] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 5] -Output [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [8]: [c_current_cdemo_sk#4, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(43) HashAggregate [codegen id : 5] -Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#26), partial_max(cd_dep_count#26), partial_avg(cd_dep_count#26), partial_min(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_avg(cd_dep_employed_count#27), partial_min(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_avg(cd_dep_college_count#28)] -Aggregate Attributes [13]: [count#31, min#32, max#33, sum#34, count#35, min#36, max#37, sum#38, count#39, min#40, max#41, sum#42, count#43] -Results [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] - -(44) CometColumnarExchange -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] -Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 6] -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] - -(46) HashAggregate [codegen id : 6] -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [count(1), min(cd_dep_count#26), max(cd_dep_count#26), avg(cd_dep_count#26), min(cd_dep_employed_count#27), max(cd_dep_employed_count#27), avg(cd_dep_employed_count#27), min(cd_dep_college_count#28), max(cd_dep_college_count#28), avg(cd_dep_college_count#28)] -Aggregate Attributes [10]: [count(1)#57, min(cd_dep_count#26)#58, max(cd_dep_count#26)#59, avg(cd_dep_count#26)#60, min(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, avg(cd_dep_employed_count#27)#63, min(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, avg(cd_dep_college_count#28)#66] -Results [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, count(1)#57 AS cnt1#67, min(cd_dep_count#26)#58 AS min(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, avg(cd_dep_count#26)#60 AS avg(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, min(cd_dep_employed_count#27)#61 AS min(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, avg(cd_dep_employed_count#27)#63 AS avg(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, min(cd_dep_college_count#28)#64 AS min(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, avg(cd_dep_college_count#28)#66 AS avg(cd_dep_college_count)#78, cd_dep_count#26] - -(47) TakeOrderedAndProject -Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78, cd_dep_count#26] -Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#22, cd_gender#29, cd_marital_status#30, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(52) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/extended.txt deleted file mode 100644 index a6f33d6f7e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/extended.txt +++ /dev/null @@ -1,63 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index f1fe09fb46..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (6) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/explain.txt deleted file mode 100644 index 58c6701f56..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/explain.txt +++ /dev/null @@ -1,207 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (31) -+- * Project (30) - +- Window (29) - +- * CometColumnarToRow (28) - +- CometSort (27) - +- CometColumnarExchange (26) - +- * HashAggregate (25) - +- * CometColumnarToRow (24) - +- CometColumnarExchange (23) - +- * HashAggregate (22) - +- * Expand (21) - +- * Project (20) - +- * BroadcastHashJoin Inner BuildRight (19) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) - : +- CometProject (9) - : +- CometFilter (8) - : +- CometNativeScan parquet spark_catalog.default.item (7) - +- BroadcastExchange (18) - +- * CometColumnarToRow (17) - +- CometProject (16) - +- CometFilter (15) - +- CometNativeScan parquet spark_catalog.default.store (14) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) - -(4) ReusedExchange [Reuses operator id: 36] -Output [1]: [d_date_sk#7] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] - -(7) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Condition : isnotnull(i_item_sk#8) - -(9) CometProject -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Arguments: [i_item_sk#8, i_class#11, i_category#12], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#12] - -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#8, i_class#11, i_category#12] - -(11) BroadcastExchange -Input [3]: [i_item_sk#8, i_class#11, i_category#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#8] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#8, i_class#11, i_category#12] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#13, s_state#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#13, s_state#14] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#14, 2, true, false, true) = TN) AND isnotnull(s_store_sk#13)) - -(16) CometProject -Input [2]: [s_store_sk#13, s_state#14] -Arguments: [s_store_sk#13], [s_store_sk#13] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#13] - -(18) BroadcastExchange -Input [1]: [s_store_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#13] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#12, i_class#11] -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12, s_store_sk#13] - -(21) Expand [codegen id : 4] -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#12, i_class#11] -Arguments: [[ss_ext_sales_price#3, ss_net_profit#4, i_category#12, i_class#11, 0], [ss_ext_sales_price#3, ss_net_profit#4, i_category#12, null, 1], [ss_ext_sales_price#3, ss_net_profit#4, null, null, 3]], [ss_ext_sales_price#3, ss_net_profit#4, i_category#15, i_class#16, spark_grouping_id#17] - -(22) HashAggregate [codegen id : 4] -Input [5]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#15, i_class#16, spark_grouping_id#17] -Keys [3]: [i_category#15, i_class#16, spark_grouping_id#17] -Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum#18, sum#19] -Results [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] - -(23) CometColumnarExchange -Input [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] -Arguments: hashpartitioning(i_category#15, i_class#16, spark_grouping_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometColumnarToRow [codegen id : 5] -Input [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] - -(25) HashAggregate [codegen id : 5] -Input [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] -Keys [3]: [i_category#15, i_class#16, spark_grouping_id#17] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#22, sum(UnscaledValue(ss_ext_sales_price#3))#23] -Results [7]: [(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#22,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#23,17,2)) AS gross_margin#24, i_category#15, i_class#16, (cast((shiftright(spark_grouping_id#17, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#17, 0) & 1) as tinyint)) AS lochierarchy#25, (MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#22,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#23,17,2)) AS _w0#26, (cast((shiftright(spark_grouping_id#17, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#17, 0) & 1) as tinyint)) AS _w1#27, CASE WHEN (cast((shiftright(spark_grouping_id#17, 0) & 1) as tinyint) = 0) THEN i_category#15 END AS _w2#28] - -(26) CometColumnarExchange -Input [7]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28] -Arguments: hashpartitioning(_w1#27, _w2#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometSort -Input [7]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28] -Arguments: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28], [_w1#27 ASC NULLS FIRST, _w2#28 ASC NULLS FIRST, _w0#26 ASC NULLS FIRST] - -(28) CometColumnarToRow [codegen id : 6] -Input [7]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28] - -(29) Window -Input [7]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28] -Arguments: [rank(_w0#26) windowspecdefinition(_w1#27, _w2#28, _w0#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#29], [_w1#27, _w2#28], [_w0#26 ASC NULLS FIRST] - -(30) Project [codegen id : 7] -Output [5]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, rank_within_parent#29] -Input [8]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28, rank_within_parent#29] - -(31) TakeOrderedAndProject -Input [5]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, rank_within_parent#29] -Arguments: 100, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#29 ASC NULLS FIRST], [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, rank_within_parent#29] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.date_dim (32) - - -(32) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#7, d_year#30] -Condition : ((isnotnull(d_year#30) AND (d_year#30 = 2001)) AND isnotnull(d_date_sk#7)) - -(34) CometProject -Input [2]: [d_date_sk#7, d_year#30] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(36) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/simplified.txt deleted file mode 100644 index d6f8548c89..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (7) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Expand [ss_ext_sales_price,ss_net_profit,i_category,i_class] - Project [ss_ext_sales_price,ss_net_profit,i_category,i_class] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan 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/q36.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/explain.txt deleted file mode 100644 index ae1d4c242b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,203 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- Window (28) - +- * CometColumnarToRow (27) - +- CometSort (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExpand (21) - +- CometProject (20) - +- CometBroadcastHashJoin (19) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - +- CometBroadcastExchange (18) - +- CometProject (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4], [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#9, i_class#10, i_category#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Condition : isnotnull(i_item_sk#9) - -(11) CometProject -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Arguments: [i_item_sk#9, i_class#12, i_category#13], [i_item_sk#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#13] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [i_item_sk#9, i_class#12, i_category#13] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Right output [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#9], Inner, BuildRight - -(14) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) = TN) AND isnotnull(s_store_sk#14)) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14], [s_store_sk#14] - -(18) CometBroadcastExchange -Input [1]: [s_store_sk#14] -Arguments: [s_store_sk#14] - -(19) CometBroadcastHashJoin -Left output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -Right output [1]: [s_store_sk#14] -Arguments: [ss_store_sk#2], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13, s_store_sk#14] -Arguments: [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12], [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12] - -(21) CometExpand -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12] -Arguments: [[ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12, 0], [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, null, 1], [ss_ext_sales_price#3, ss_net_profit#4, null, null, 3]], [ss_ext_sales_price#3, ss_net_profit#4, i_category#16, i_class#17, spark_grouping_id#18] - -(22) CometHashAggregate -Input [5]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#16, i_class#17, spark_grouping_id#18] -Keys [3]: [i_category#16, i_class#17, spark_grouping_id#18] -Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(23) CometExchange -Input [5]: [i_category#16, i_class#17, spark_grouping_id#18, sum#19, sum#20] -Arguments: hashpartitioning(i_category#16, i_class#17, spark_grouping_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [5]: [i_category#16, i_class#17, spark_grouping_id#18, sum#19, sum#20] -Keys [3]: [i_category#16, i_class#17, spark_grouping_id#18] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] - -(25) CometExchange -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -Arguments: hashpartitioning(_w1#24, _w2#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(26) CometSort -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -Arguments: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25], [_w1#24 ASC NULLS FIRST, _w2#25 ASC NULLS FIRST, _w0#23 ASC NULLS FIRST] - -(27) CometColumnarToRow [codegen id : 1] -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] - -(28) Window -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -Arguments: [rank(_w0#23) windowspecdefinition(_w1#24, _w2#25, _w0#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#26], [_w1#24, _w2#25], [_w0#23 ASC NULLS FIRST] - -(29) Project [codegen id : 2] -Output [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] -Input [8]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25, rank_within_parent#26] - -(30) TakeOrderedAndProject -Input [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] -Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#26 ASC NULLS FIRST], [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(33) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(35) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/simplified.txt deleted file mode 100644 index 61782e3011..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,42 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (2) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometExchange [_w1,_w2] #1 - CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),spark_grouping_id] - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] - 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,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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #6 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] 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/q36/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/explain.txt deleted file mode 100644 index ae1d4c242b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/explain.txt +++ /dev/null @@ -1,203 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- Window (28) - +- * CometColumnarToRow (27) - +- CometSort (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExpand (21) - +- CometProject (20) - +- CometBroadcastHashJoin (19) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - +- CometBroadcastExchange (18) - +- CometProject (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4], [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#9, i_class#10, i_category#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Condition : isnotnull(i_item_sk#9) - -(11) CometProject -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Arguments: [i_item_sk#9, i_class#12, i_category#13], [i_item_sk#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#13] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [i_item_sk#9, i_class#12, i_category#13] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Right output [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#9], Inner, BuildRight - -(14) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) = TN) AND isnotnull(s_store_sk#14)) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14], [s_store_sk#14] - -(18) CometBroadcastExchange -Input [1]: [s_store_sk#14] -Arguments: [s_store_sk#14] - -(19) CometBroadcastHashJoin -Left output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -Right output [1]: [s_store_sk#14] -Arguments: [ss_store_sk#2], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13, s_store_sk#14] -Arguments: [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12], [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12] - -(21) CometExpand -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12] -Arguments: [[ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12, 0], [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, null, 1], [ss_ext_sales_price#3, ss_net_profit#4, null, null, 3]], [ss_ext_sales_price#3, ss_net_profit#4, i_category#16, i_class#17, spark_grouping_id#18] - -(22) CometHashAggregate -Input [5]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#16, i_class#17, spark_grouping_id#18] -Keys [3]: [i_category#16, i_class#17, spark_grouping_id#18] -Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(23) CometExchange -Input [5]: [i_category#16, i_class#17, spark_grouping_id#18, sum#19, sum#20] -Arguments: hashpartitioning(i_category#16, i_class#17, spark_grouping_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [5]: [i_category#16, i_class#17, spark_grouping_id#18, sum#19, sum#20] -Keys [3]: [i_category#16, i_class#17, spark_grouping_id#18] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] - -(25) CometExchange -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -Arguments: hashpartitioning(_w1#24, _w2#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(26) CometSort -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -Arguments: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25], [_w1#24 ASC NULLS FIRST, _w2#25 ASC NULLS FIRST, _w0#23 ASC NULLS FIRST] - -(27) CometColumnarToRow [codegen id : 1] -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] - -(28) Window -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -Arguments: [rank(_w0#23) windowspecdefinition(_w1#24, _w2#25, _w0#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#26], [_w1#24, _w2#25], [_w0#23 ASC NULLS FIRST] - -(29) Project [codegen id : 2] -Output [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] -Input [8]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25, rank_within_parent#26] - -(30) TakeOrderedAndProject -Input [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] -Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#26 ASC NULLS FIRST], [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(33) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(35) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/extended.txt deleted file mode 100644 index 89c523a388..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/extended.txt +++ /dev/null @@ -1,38 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 61782e3011..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/simplified.txt +++ /dev/null @@ -1,42 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (2) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometExchange [_w1,_w2] #1 - CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),spark_grouping_id] - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] - 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,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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #6 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/explain.txt deleted file mode 100644 index e36c995169..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/explain.txt +++ /dev/null @@ -1,181 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometTakeOrderedAndProject (25) - +- CometHashAggregate (24) - +- CometColumnarExchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildLeft (20) - :- BroadcastExchange (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * CometColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (9) - : : +- * Project (8) - : : +- * Filter (7) - : : +- * ColumnarToRow (6) - : : +- Scan parquet spark_catalog.default.inventory (5) - : +- ReusedExchange (12) - +- * CometColumnarToRow (19) - +- CometProject (18) - +- CometFilter (17) - +- CometNativeScan parquet spark_catalog.default.catalog_sales (16) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,68.00), LessThanOrEqual(i_current_price,98.00), In(i_manufact_id, [677,694,808,940]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 68.00)) AND (i_current_price#4 <= 98.00)) AND i_manufact_id#5 IN (677,940,694,808)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#2, 16, true, false, true) AS i_item_id#6, i_item_desc#3, i_current_price#4] - -(4) CometColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(5) Scan parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#9), dynamicpruningexpression(inv_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] - -(7) Filter [codegen id : 1] -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Condition : (((isnotnull(inv_quantity_on_hand#8) AND (inv_quantity_on_hand#8 >= 100)) AND (inv_quantity_on_hand#8 <= 500)) AND isnotnull(inv_item_sk#7)) - -(8) Project [codegen id : 1] -Output [2]: [inv_item_sk#7, inv_date_sk#9] -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] - -(9) BroadcastExchange -Input [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [inv_item_sk#7] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 3] -Output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] - -(12) ReusedExchange [Reuses operator id: 31] -Output [1]: [d_date_sk#11] - -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [inv_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 3] -Output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] - -(15) BroadcastExchange -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#12, cs_sold_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [cs_item_sk#12, cs_sold_date_sk#13] -Condition : isnotnull(cs_item_sk#12) - -(18) CometProject -Input [2]: [cs_item_sk#12, cs_sold_date_sk#13] -Arguments: [cs_item_sk#12], [cs_item_sk#12] - -(19) CometColumnarToRow -Input [1]: [cs_item_sk#12] - -(20) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#12] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 4] -Output [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, cs_item_sk#12] - -(22) HashAggregate [codegen id : 4] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -(23) CometColumnarExchange -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(25) CometTakeOrderedAndProject -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(26) CometColumnarToRow [codegen id : 5] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometNativeScan parquet spark_catalog.default.date_dim (27) - - -(27) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#11, d_date#14] -Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 2000-02-01)) AND (d_date#14 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) - -(29) CometProject -Input [2]: [d_date_sk#11, d_date#14] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(31) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/simplified.txt deleted file mode 100644 index 88a3895990..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_current_price] - Project [i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [i_item_sk,cs_item_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (3) - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [inv_item_sk,inv_date_sk] - Filter [inv_quantity_on_hand,inv_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #4 - CometColumnarToRow - InputAdapter - CometProject [cs_item_sk] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometNativeScan 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/q37.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/explain.txt deleted file mode 100644 index e4cfdbf6d1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometTakeOrderedAndProject (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (4) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - +- CometProject (19) - +- CometFilter (18) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (17) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,68.00), LessThanOrEqual(i_current_price,98.00), In(i_manufact_id, [677,694,808,940]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 68.00)) AND (i_current_price#4 <= 98.00)) AND i_manufact_id#5 IN (677,940,694,808)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#2, 16, true, false, true) AS i_item_id#6, i_item_desc#3, i_current_price#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#9), dynamicpruningexpression(inv_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Condition : (((isnotnull(inv_quantity_on_hand#8) AND (inv_quantity_on_hand#8 >= 100)) AND (inv_quantity_on_hand#8 <= 500)) AND isnotnull(inv_item_sk#7)) - -(6) CometProject -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9], [inv_item_sk#7, inv_date_sk#9] - -(7) CometBroadcastExchange -Input [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1], [inv_item_sk#7], Inner, BuildRight - -(9) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#12 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) - -(12) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(14) CometBroadcastHashJoin -Left output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [inv_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(15) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(16) CometBroadcastExchange -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#13, cs_sold_date_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [cs_item_sk#13, cs_sold_date_sk#14] -Condition : isnotnull(cs_item_sk#13) - -(19) CometProject -Input [2]: [cs_item_sk#13, cs_sold_date_sk#14] -Arguments: [cs_item_sk#13], [cs_item_sk#13] - -(20) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [1]: [cs_item_sk#13] -Arguments: [i_item_sk#1], [cs_item_sk#13], Inner, BuildLeft - -(21) CometProject -Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, cs_item_sk#13] -Arguments: [i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(22) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(23) CometExchange -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(25) CometTakeOrderedAndProject -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(26) CometColumnarToRow [codegen id : 1] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#12 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) - -(29) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(31) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/simplified.txt deleted file mode 100644 index 7ab646a628..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometExchange [i_item_id,i_item_desc,i_current_price] #1 - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometProject [i_item_id,i_item_desc,i_current_price] - 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 [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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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/q37/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/explain.txt deleted file mode 100644 index e4cfdbf6d1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometTakeOrderedAndProject (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (4) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - +- CometProject (19) - +- CometFilter (18) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (17) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,68.00), LessThanOrEqual(i_current_price,98.00), In(i_manufact_id, [677,694,808,940]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 68.00)) AND (i_current_price#4 <= 98.00)) AND i_manufact_id#5 IN (677,940,694,808)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#2, 16, true, false, true) AS i_item_id#6, i_item_desc#3, i_current_price#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#9), dynamicpruningexpression(inv_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Condition : (((isnotnull(inv_quantity_on_hand#8) AND (inv_quantity_on_hand#8 >= 100)) AND (inv_quantity_on_hand#8 <= 500)) AND isnotnull(inv_item_sk#7)) - -(6) CometProject -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9], [inv_item_sk#7, inv_date_sk#9] - -(7) CometBroadcastExchange -Input [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1], [inv_item_sk#7], Inner, BuildRight - -(9) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#12 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) - -(12) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(14) CometBroadcastHashJoin -Left output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [inv_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(15) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(16) CometBroadcastExchange -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#13, cs_sold_date_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [cs_item_sk#13, cs_sold_date_sk#14] -Condition : isnotnull(cs_item_sk#13) - -(19) CometProject -Input [2]: [cs_item_sk#13, cs_sold_date_sk#14] -Arguments: [cs_item_sk#13], [cs_item_sk#13] - -(20) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [1]: [cs_item_sk#13] -Arguments: [i_item_sk#1], [cs_item_sk#13], Inner, BuildLeft - -(21) CometProject -Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, cs_item_sk#13] -Arguments: [i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(22) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(23) CometExchange -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(25) CometTakeOrderedAndProject -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(26) CometColumnarToRow [codegen id : 1] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#12 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) - -(29) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(31) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/extended.txt deleted file mode 100644 index abe0937d1d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/extended.txt +++ /dev/null @@ -1,34 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 7ab646a628..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometExchange [i_item_id,i_item_desc,i_current_price] #1 - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometProject [i_item_id,i_item_desc,i_current_price] - 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 [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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/explain.txt deleted file mode 100644 index f1a8620998..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/explain.txt +++ /dev/null @@ -1,317 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (49) -+- CometHashAggregate (48) - +- CometExchange (47) - +- CometHashAggregate (46) - +- CometProject (45) - +- CometBroadcastHashJoin (44) - :- CometBroadcastHashJoin (30) - : :- CometHashAggregate (16) - : : +- CometColumnarExchange (15) - : : +- * HashAggregate (14) - : : +- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.customer (7) - : +- CometBroadcastExchange (29) - : +- CometHashAggregate (28) - : +- CometColumnarExchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet spark_catalog.default.catalog_sales (17) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - +- CometBroadcastExchange (43) - +- CometHashAggregate (42) - +- CometColumnarExchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Filter (33) - : : +- * ColumnarToRow (32) - : : +- Scan parquet spark_catalog.default.web_sales (31) - : +- ReusedExchange (34) - +- ReusedExchange (37) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sold_date_sk#2 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] - -(3) Filter [codegen id : 3] -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Condition : isnotnull(ss_customer_sk#1) - -(4) ReusedExchange [Reuses operator id: 54] -Output [2]: [d_date_sk#4, d_date#5] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#2] -Right keys [1]: [d_date_sk#4] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [2]: [ss_customer_sk#1, d_date#5] -Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] - -(7) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Condition : isnotnull(c_customer_sk#6) - -(9) CometProject -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Arguments: [c_customer_sk#6, c_first_name#9, c_last_name#10], [c_customer_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#7, 20, true, false, true) AS c_first_name#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#8, 30, true, false, true) AS c_last_name#10] - -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] - -(11) BroadcastExchange -Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#6] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [3]: [c_last_name#10, c_first_name#9, d_date#5] -Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#6, c_first_name#9, c_last_name#10] - -(14) HashAggregate [codegen id : 3] -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#5] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#5] - -(15) CometColumnarExchange -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] -Arguments: hashpartitioning(c_last_name#10, c_first_name#9, d_date#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometHashAggregate -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#5] -Functions: [] - -(17) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#12), dynamicpruningexpression(cs_sold_date_sk#12 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 6] -Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] - -(19) Filter [codegen id : 6] -Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] -Condition : isnotnull(cs_bill_customer_sk#11) - -(20) ReusedExchange [Reuses operator id: 54] -Output [2]: [d_date_sk#13, d_date#14] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#12] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [2]: [cs_bill_customer_sk#11, d_date#14] -Input [4]: [cs_bill_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13, d_date#14] - -(23) ReusedExchange [Reuses operator id: 11] -Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] - -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_bill_customer_sk#11] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [3]: [c_last_name#17, c_first_name#16, d_date#14] -Input [5]: [cs_bill_customer_sk#11, d_date#14, c_customer_sk#15, c_first_name#16, c_last_name#17] - -(26) HashAggregate [codegen id : 6] -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#17, c_first_name#16, d_date#14] - -(27) CometColumnarExchange -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, d_date#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometHashAggregate -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] -Functions: [] - -(29) CometBroadcastExchange -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Arguments: [c_last_name#17, c_first_name#16, d_date#14] - -(30) CometBroadcastHashJoin -Left output [3]: [c_last_name#10, c_first_name#9, d_date#5] -Right output [3]: [c_last_name#17, c_first_name#16, d_date#14] -Arguments: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)], [coalesce(c_last_name#17, ), isnull(c_last_name#17), coalesce(c_first_name#16, ), isnull(c_first_name#16), coalesce(d_date#14, 1970-01-01), isnull(d_date#14)], LeftSemi, BuildRight - -(31) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_sold_date_sk#19 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(32) ColumnarToRow [codegen id : 9] -Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] - -(33) Filter [codegen id : 9] -Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] -Condition : isnotnull(ws_bill_customer_sk#18) - -(34) ReusedExchange [Reuses operator id: 54] -Output [2]: [d_date_sk#20, d_date#21] - -(35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#19] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(36) Project [codegen id : 9] -Output [2]: [ws_bill_customer_sk#18, d_date#21] -Input [4]: [ws_bill_customer_sk#18, ws_sold_date_sk#19, d_date_sk#20, d_date#21] - -(37) ReusedExchange [Reuses operator id: 11] -Output [3]: [c_customer_sk#22, c_first_name#23, c_last_name#24] - -(38) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_bill_customer_sk#18] -Right keys [1]: [c_customer_sk#22] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 9] -Output [3]: [c_last_name#24, c_first_name#23, d_date#21] -Input [5]: [ws_bill_customer_sk#18, d_date#21, c_customer_sk#22, c_first_name#23, c_last_name#24] - -(40) HashAggregate [codegen id : 9] -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#24, c_first_name#23, d_date#21] - -(41) CometColumnarExchange -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, d_date#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(42) CometHashAggregate -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] -Functions: [] - -(43) CometBroadcastExchange -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: [c_last_name#24, c_first_name#23, d_date#21] - -(44) CometBroadcastHashJoin -Left output [3]: [c_last_name#10, c_first_name#9, d_date#5] -Right output [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)], [coalesce(c_last_name#24, ), isnull(c_last_name#24), coalesce(c_first_name#23, ), isnull(c_first_name#23), coalesce(d_date#21, 1970-01-01), isnull(d_date#21)], LeftSemi, BuildRight - -(45) CometProject -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] - -(46) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(47) CometExchange -Input [1]: [count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(48) CometHashAggregate -Input [1]: [count#25] -Keys: [] -Functions [1]: [count(1)] - -(49) CometColumnarToRow [codegen id : 10] -Input [1]: [count(1)#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometNativeScan parquet spark_catalog.default.date_dim (50) - - -(50) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#27] -Condition : (((isnotnull(d_month_seq#27) AND (d_month_seq#27 >= 1200)) AND (d_month_seq#27 <= 1211)) AND isnotnull(d_date_sk#4)) - -(52) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#27] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(53) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] - -(54) BroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 17 Hosting Expression = cs_sold_date_sk#12 IN dynamicpruning#3 - -Subquery:3 Hosting operator id = 31 Hosting Expression = ws_sold_date_sk#19 IN dynamicpruning#3 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/simplified.txt deleted file mode 100644 index 547965d09c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/simplified.txt +++ /dev/null @@ -1,75 +0,0 @@ -WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometBroadcastExchange [c_last_name,c_first_name,d_date] #5 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 - CometBroadcastExchange [c_last_name,c_first_name,d_date] #7 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (9) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/explain.txt deleted file mode 100644 index 591fa114fb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,308 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (48) -+- CometHashAggregate (47) - +- CometExchange (46) - +- CometHashAggregate (45) - +- CometProject (44) - +- CometBroadcastHashJoin (43) - :- CometBroadcastHashJoin (30) - : :- CometHashAggregate (17) - : : +- CometExchange (16) - : : +- CometHashAggregate (15) - : : +- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : +- CometBroadcastExchange (29) - : +- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometFilter (19) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - +- CometBroadcastExchange (42) - +- CometHashAggregate (41) - +- CometExchange (40) - +- CometHashAggregate (39) - +- CometProject (38) - +- CometBroadcastHashJoin (37) - :- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (31) - : +- ReusedExchange (33) - +- ReusedExchange (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sold_date_sk#2 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Condition : isnotnull(ss_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(5) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: [d_date_sk#4, d_date#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Right output [2]: [d_date_sk#4, d_date#5] -Arguments: [ss_sold_date_sk#2], [d_date_sk#4], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] -Arguments: [ss_customer_sk#1, d_date#5], [ss_customer_sk#1, d_date#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Condition : isnotnull(c_customer_sk#7) - -(11) CometProject -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11], [c_customer_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#8, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#9, 30, true, false, true) AS c_last_name#11] - -(12) CometBroadcastExchange -Input [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, d_date#5] -Right output [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [ss_customer_sk#1], [c_customer_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_last_name#11, c_first_name#10, d_date#5], [c_last_name#11, c_first_name#10, d_date#5] - -(15) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(16) CometExchange -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(c_last_name#11, c_first_name#10, d_date#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Condition : isnotnull(cs_bill_customer_sk#12) - -(20) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#15, d_date#16] - -(21) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_date#16] -Arguments: [cs_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(22) CometProject -Input [4]: [cs_bill_customer_sk#12, cs_sold_date_sk#13, d_date_sk#15, d_date#16] -Arguments: [cs_bill_customer_sk#12, d_date#16], [cs_bill_customer_sk#12, d_date#16] - -(23) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] - -(24) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, d_date#16] -Right output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [cs_bill_customer_sk#12], [c_customer_sk#17], Inner, BuildRight - -(25) CometProject -Input [5]: [cs_bill_customer_sk#12, d_date#16, c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [c_last_name#19, c_first_name#18, d_date#16], [c_last_name#19, c_first_name#18, d_date#16] - -(26) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(27) CometExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: hashpartitioning(c_last_name#19, c_first_name#18, d_date#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(29) CometBroadcastExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: [c_last_name#19, c_first_name#18, d_date#16] - -(30) CometBroadcastHashJoin -Left output [3]: [c_last_name#11, c_first_name#10, d_date#5] -Right output [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)], [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 1970-01-01), isnull(d_date#16)], LeftSemi, BuildRight - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#21), dynamicpruningexpression(ws_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Condition : isnotnull(ws_bill_customer_sk#20) - -(33) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#23, d_date#24] - -(34) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Right output [2]: [d_date_sk#23, d_date#24] -Arguments: [ws_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(35) CometProject -Input [4]: [ws_bill_customer_sk#20, ws_sold_date_sk#21, d_date_sk#23, d_date#24] -Arguments: [ws_bill_customer_sk#20, d_date#24], [ws_bill_customer_sk#20, d_date#24] - -(36) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] - -(37) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, d_date#24] -Right output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [ws_bill_customer_sk#20], [c_customer_sk#25], Inner, BuildRight - -(38) CometProject -Input [5]: [ws_bill_customer_sk#20, d_date#24, c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_last_name#27, c_first_name#26, d_date#24], [c_last_name#27, c_first_name#26, d_date#24] - -(39) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(40) CometExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: hashpartitioning(c_last_name#27, c_first_name#26, d_date#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(41) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(42) CometBroadcastExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: [c_last_name#27, c_first_name#26, d_date#24] - -(43) CometBroadcastHashJoin -Left output [3]: [c_last_name#11, c_first_name#10, d_date#5] -Right output [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)], [coalesce(c_last_name#27, ), isnull(c_last_name#27), coalesce(c_first_name#26, ), isnull(c_first_name#26), coalesce(d_date#24, 1970-01-01), isnull(d_date#24)], LeftSemi, BuildRight - -(44) CometProject -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] - -(45) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(46) CometExchange -Input [1]: [count#28] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(47) CometHashAggregate -Input [1]: [count#28] -Keys: [] -Functions [1]: [count(1)] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [count(1)#29] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(51) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(52) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] - -(53) BroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#3 - -Subquery:3 Hosting operator id = 31 Hosting Expression = ws_sold_date_sk#21 IN dynamicpruning#3 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/simplified.txt deleted file mode 100644 index 7ceccf5d2c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,60 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #2 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] - CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_customer_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #4 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometBroadcastExchange [c_last_name,c_first_name,d_date] #6 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #7 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] - CometFilter [cs_bill_customer_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometBroadcastExchange [c_last_name,c_first_name,d_date] #8 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #9 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_bill_customer_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/explain.txt deleted file mode 100644 index 591fa114fb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/explain.txt +++ /dev/null @@ -1,308 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (48) -+- CometHashAggregate (47) - +- CometExchange (46) - +- CometHashAggregate (45) - +- CometProject (44) - +- CometBroadcastHashJoin (43) - :- CometBroadcastHashJoin (30) - : :- CometHashAggregate (17) - : : +- CometExchange (16) - : : +- CometHashAggregate (15) - : : +- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : +- CometBroadcastExchange (29) - : +- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometFilter (19) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - +- CometBroadcastExchange (42) - +- CometHashAggregate (41) - +- CometExchange (40) - +- CometHashAggregate (39) - +- CometProject (38) - +- CometBroadcastHashJoin (37) - :- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (31) - : +- ReusedExchange (33) - +- ReusedExchange (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sold_date_sk#2 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Condition : isnotnull(ss_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(5) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: [d_date_sk#4, d_date#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Right output [2]: [d_date_sk#4, d_date#5] -Arguments: [ss_sold_date_sk#2], [d_date_sk#4], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] -Arguments: [ss_customer_sk#1, d_date#5], [ss_customer_sk#1, d_date#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Condition : isnotnull(c_customer_sk#7) - -(11) CometProject -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11], [c_customer_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#8, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#9, 30, true, false, true) AS c_last_name#11] - -(12) CometBroadcastExchange -Input [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, d_date#5] -Right output [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [ss_customer_sk#1], [c_customer_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_last_name#11, c_first_name#10, d_date#5], [c_last_name#11, c_first_name#10, d_date#5] - -(15) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(16) CometExchange -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(c_last_name#11, c_first_name#10, d_date#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Condition : isnotnull(cs_bill_customer_sk#12) - -(20) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#15, d_date#16] - -(21) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_date#16] -Arguments: [cs_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(22) CometProject -Input [4]: [cs_bill_customer_sk#12, cs_sold_date_sk#13, d_date_sk#15, d_date#16] -Arguments: [cs_bill_customer_sk#12, d_date#16], [cs_bill_customer_sk#12, d_date#16] - -(23) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] - -(24) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, d_date#16] -Right output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [cs_bill_customer_sk#12], [c_customer_sk#17], Inner, BuildRight - -(25) CometProject -Input [5]: [cs_bill_customer_sk#12, d_date#16, c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [c_last_name#19, c_first_name#18, d_date#16], [c_last_name#19, c_first_name#18, d_date#16] - -(26) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(27) CometExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: hashpartitioning(c_last_name#19, c_first_name#18, d_date#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(29) CometBroadcastExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: [c_last_name#19, c_first_name#18, d_date#16] - -(30) CometBroadcastHashJoin -Left output [3]: [c_last_name#11, c_first_name#10, d_date#5] -Right output [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)], [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 1970-01-01), isnull(d_date#16)], LeftSemi, BuildRight - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#21), dynamicpruningexpression(ws_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Condition : isnotnull(ws_bill_customer_sk#20) - -(33) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#23, d_date#24] - -(34) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Right output [2]: [d_date_sk#23, d_date#24] -Arguments: [ws_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(35) CometProject -Input [4]: [ws_bill_customer_sk#20, ws_sold_date_sk#21, d_date_sk#23, d_date#24] -Arguments: [ws_bill_customer_sk#20, d_date#24], [ws_bill_customer_sk#20, d_date#24] - -(36) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] - -(37) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, d_date#24] -Right output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [ws_bill_customer_sk#20], [c_customer_sk#25], Inner, BuildRight - -(38) CometProject -Input [5]: [ws_bill_customer_sk#20, d_date#24, c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_last_name#27, c_first_name#26, d_date#24], [c_last_name#27, c_first_name#26, d_date#24] - -(39) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(40) CometExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: hashpartitioning(c_last_name#27, c_first_name#26, d_date#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(41) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(42) CometBroadcastExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: [c_last_name#27, c_first_name#26, d_date#24] - -(43) CometBroadcastHashJoin -Left output [3]: [c_last_name#11, c_first_name#10, d_date#5] -Right output [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)], [coalesce(c_last_name#27, ), isnull(c_last_name#27), coalesce(c_first_name#26, ), isnull(c_first_name#26), coalesce(d_date#24, 1970-01-01), isnull(d_date#24)], LeftSemi, BuildRight - -(44) CometProject -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] - -(45) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(46) CometExchange -Input [1]: [count#28] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(47) CometHashAggregate -Input [1]: [count#28] -Keys: [] -Functions [1]: [count(1)] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [count(1)#29] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(51) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(52) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] - -(53) BroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#3 - -Subquery:3 Hosting operator id = 31 Hosting Expression = ws_sold_date_sk#21 IN dynamicpruning#3 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/extended.txt deleted file mode 100644 index 55af64efc6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/extended.txt +++ /dev/null @@ -1,70 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 62 out of 66 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 7ceccf5d2c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38/simplified.txt +++ /dev/null @@ -1,60 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #2 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] - CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_customer_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #4 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometBroadcastExchange [c_last_name,c_first_name,d_date] #6 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #7 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] - CometFilter [cs_bill_customer_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometBroadcastExchange [c_last_name,c_first_name,d_date] #8 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #9 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_bill_customer_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/explain.txt deleted file mode 100644 index b8ed3a361b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/explain.txt +++ /dev/null @@ -1,330 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometSort (46) - +- CometColumnarExchange (45) - +- * BroadcastHashJoin Inner BuildRight (44) - :- * Project (24) - : +- * Filter (23) - : +- * HashAggregate (22) - : +- * CometColumnarToRow (21) - : +- CometColumnarExchange (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.inventory (1) - : : : +- BroadcastExchange (7) - : : : +- * CometColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : +- BroadcastExchange (13) - : : +- * CometColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) - : +- ReusedExchange (16) - +- BroadcastExchange (43) - +- * Project (42) - +- * Filter (41) - +- * HashAggregate (40) - +- * CometColumnarToRow (39) - +- CometColumnarExchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * Filter (27) - : : : +- * ColumnarToRow (26) - : : : +- Scan parquet spark_catalog.default.inventory (25) - : : +- ReusedExchange (28) - : +- ReusedExchange (31) - +- ReusedExchange (34) - - -(1) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [1]: [i_item_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [1]: [i_item_sk#6] -Condition : isnotnull(i_item_sk#6) - -(6) CometColumnarToRow [codegen id : 1] -Input [1]: [i_item_sk#6] - -(7) BroadcastExchange -Input [1]: [i_item_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 4] -Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] - -(10) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Condition : isnotnull(w_warehouse_sk#7) - -(12) CometColumnarToRow [codegen id : 2] -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] - -(13) BroadcastExchange -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#7] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] - -(16) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#9, d_moy#10] - -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#10] - -(19) HashAggregate [codegen id : 4] -Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [5]: [n#11, avg#12, m2#13, sum#14, count#15] -Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] - -(20) CometColumnarExchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(21) CometColumnarToRow [codegen id : 10] -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] - -(22) HashAggregate [codegen id : 10] -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#21, avg(inv_quantity_on_hand#3)#22] -Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stddev_samp(cast(inv_quantity_on_hand#3 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#3)#22 AS mean#24] - -(23) Filter [codegen id : 10] -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.0) END - -(24) Project [codegen id : 10] -Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN null ELSE (stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))) END AS cov#25] -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] - -(25) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#29), dynamicpruningexpression(inv_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(26) ColumnarToRow [codegen id : 8] -Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] - -(27) Filter [codegen id : 8] -Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -Condition : (isnotnull(inv_item_sk#26) AND isnotnull(inv_warehouse_sk#27)) - -(28) ReusedExchange [Reuses operator id: 7] -Output [1]: [i_item_sk#31] - -(29) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_item_sk#26] -Right keys [1]: [i_item_sk#31] -Join type: Inner -Join condition: None - -(30) Project [codegen id : 8] -Output [4]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] -Input [5]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] - -(31) ReusedExchange [Reuses operator id: 13] -Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] - -(32) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_warehouse_sk#27] -Right keys [1]: [w_warehouse_sk#32] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] -Input [6]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] - -(34) ReusedExchange [Reuses operator id: 57] -Output [2]: [d_date_sk#34, d_moy#35] - -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_date_sk#29] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None - -(36) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] -Input [7]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_date_sk#34, d_moy#35] - -(37) HashAggregate [codegen id : 8] -Input [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] -Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#28 as double)), partial_avg(inv_quantity_on_hand#28)] -Aggregate Attributes [5]: [n#36, avg#37, m2#38, sum#39, count#40] -Results [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] - -(38) CometColumnarExchange -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(39) CometColumnarToRow [codegen id : 9] -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] - -(40) HashAggregate [codegen id : 9] -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] -Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#28)#22 AS mean#24] - -(41) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.0) END - -(42) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#24 AS mean#46, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN null ELSE (stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))) END AS cov#47] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] - -(43) BroadcastExchange -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] - -(44) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] -Right keys [2]: [i_item_sk#31, w_warehouse_sk#32] -Join type: Inner -Join condition: None - -(45) CometColumnarExchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(46) CometSort -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 11] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometNativeScan parquet spark_catalog.default.date_dim (48) - - -(48) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#48, d_moy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#48, d_moy#10] -Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#10)) AND (d_year#48 = 2001)) AND (d_moy#10 = 1)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#48, d_moy#10] -Arguments: [d_date_sk#9, d_moy#10], [d_date_sk#9, d_moy#10] - -(51) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#9, d_moy#10] - -(52) BroadcastExchange -Input [2]: [d_date_sk#9, d_moy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 25 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometNativeScan parquet spark_catalog.default.date_dim (53) - - -(53) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#49, d_moy#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#34, d_year#49, d_moy#35] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#35)) AND (d_year#49 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#34)) - -(55) CometProject -Input [3]: [d_date_sk#34, d_year#49, d_moy#35] -Arguments: [d_date_sk#34, d_moy#35], [d_date_sk#34, d_moy#35] - -(56) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#34, d_moy#35] - -(57) BroadcastExchange -Input [2]: [d_date_sk#34, d_moy#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/simplified.txt deleted file mode 100644 index 11dd59a97f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/simplified.txt +++ /dev/null @@ -1,84 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 - WholeStageCodegen (8) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [i_item_sk] #4 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - InputAdapter - ReusedExchange [d_date_sk,d_moy] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/explain.txt deleted file mode 100644 index ed74780c2c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,329 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometSort (46) - +- CometExchange (45) - +- CometBroadcastHashJoin (44) - :- CometProject (23) - : +- CometFilter (22) - : +- CometHashAggregate (21) - : +- CometExchange (20) - : +- CometHashAggregate (19) - : +- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (12) - : : +- CometBroadcastHashJoin (11) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : +- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - +- CometBroadcastExchange (43) - +- CometProject (42) - +- CometFilter (41) - +- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (28) - : : +- CometBroadcastHashJoin (27) - : : :- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (24) - : : +- ReusedExchange (26) - : +- ReusedExchange (29) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [i_item_sk#6] -Condition : isnotnull(i_item_sk#6) - -(5) CometBroadcastExchange -Input [1]: [i_item_sk#6] -Arguments: [i_item_sk#6] - -(6) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [i_item_sk#6] -Arguments: [inv_item_sk#1], [i_item_sk#6], Inner, BuildRight - -(7) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6], [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Condition : isnotnull(w_warehouse_sk#7) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [w_warehouse_sk#7, w_warehouse_name#8] - -(11) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Right output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#7], Inner, BuildRight - -(12) CometProject -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8], [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(15) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11] - -(17) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Right output [2]: [d_date_sk#9, d_moy#11] -Arguments: [inv_date_sk#4], [d_date_sk#9], Inner, BuildRight - -(18) CometProject -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#11] -Arguments: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11], [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] - -(19) CometHashAggregate -Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] - -(20) CometExchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] - -(22) CometFilter -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.0) END - -(23) CometProject -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19], [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN null ELSE (stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))) END AS cov#19] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#23), dynamicpruningexpression(inv_date_sk#23 IN dynamicpruning#24)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Condition : (isnotnull(inv_item_sk#20) AND isnotnull(inv_warehouse_sk#21)) - -(26) ReusedExchange [Reuses operator id: 5] -Output [1]: [i_item_sk#25] - -(27) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Right output [1]: [i_item_sk#25] -Arguments: [inv_item_sk#20], [i_item_sk#25], Inner, BuildRight - -(28) CometProject -Input [5]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Arguments: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25], [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] - -(29) ReusedExchange [Reuses operator id: 10] -Output [2]: [w_warehouse_sk#26, w_warehouse_name#27] - -(30) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Right output [2]: [w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_warehouse_sk#21], [w_warehouse_sk#26], Inner, BuildRight - -(31) CometProject -Input [6]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27], [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(34) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(35) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30] - -(36) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Right output [2]: [d_date_sk#28, d_moy#30] -Arguments: [inv_date_sk#23], [d_date_sk#28], Inner, BuildRight - -(37) CometProject -Input [7]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_date_sk#28, d_moy#30] -Arguments: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30], [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] - -(38) CometHashAggregate -Input [5]: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#22 as double)), partial_avg(inv_quantity_on_hand#22)] - -(39) CometExchange -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Arguments: hashpartitioning(w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(40) CometHashAggregate -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#22 as double)), avg(inv_quantity_on_hand#22)] - -(41) CometFilter -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.0) END - -(42) CometProject -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#18 AS mean#36, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN null ELSE (stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))) END AS cov#37] - -(43) CometBroadcastExchange -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] - -(44) CometBroadcastHashJoin -Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] -Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight - -(45) CometExchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(46) CometSort -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 1] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(51) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#9, d_moy#11] - -(52) BroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(55) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(56) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#28, d_moy#30] - -(57) BroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 2cd1f70c8a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,65 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [i_item_sk] #4 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [d_date_sk,d_moy] #6 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov] #7 - CometProject [mean,stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_moy] #10 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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/q39a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/explain.txt deleted file mode 100644 index ed74780c2c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/explain.txt +++ /dev/null @@ -1,329 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometSort (46) - +- CometExchange (45) - +- CometBroadcastHashJoin (44) - :- CometProject (23) - : +- CometFilter (22) - : +- CometHashAggregate (21) - : +- CometExchange (20) - : +- CometHashAggregate (19) - : +- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (12) - : : +- CometBroadcastHashJoin (11) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : +- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - +- CometBroadcastExchange (43) - +- CometProject (42) - +- CometFilter (41) - +- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (28) - : : +- CometBroadcastHashJoin (27) - : : :- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (24) - : : +- ReusedExchange (26) - : +- ReusedExchange (29) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [i_item_sk#6] -Condition : isnotnull(i_item_sk#6) - -(5) CometBroadcastExchange -Input [1]: [i_item_sk#6] -Arguments: [i_item_sk#6] - -(6) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [i_item_sk#6] -Arguments: [inv_item_sk#1], [i_item_sk#6], Inner, BuildRight - -(7) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6], [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Condition : isnotnull(w_warehouse_sk#7) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [w_warehouse_sk#7, w_warehouse_name#8] - -(11) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Right output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#7], Inner, BuildRight - -(12) CometProject -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8], [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(15) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11] - -(17) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Right output [2]: [d_date_sk#9, d_moy#11] -Arguments: [inv_date_sk#4], [d_date_sk#9], Inner, BuildRight - -(18) CometProject -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#11] -Arguments: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11], [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] - -(19) CometHashAggregate -Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] - -(20) CometExchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] - -(22) CometFilter -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.0) END - -(23) CometProject -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19], [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN null ELSE (stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))) END AS cov#19] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#23), dynamicpruningexpression(inv_date_sk#23 IN dynamicpruning#24)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Condition : (isnotnull(inv_item_sk#20) AND isnotnull(inv_warehouse_sk#21)) - -(26) ReusedExchange [Reuses operator id: 5] -Output [1]: [i_item_sk#25] - -(27) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Right output [1]: [i_item_sk#25] -Arguments: [inv_item_sk#20], [i_item_sk#25], Inner, BuildRight - -(28) CometProject -Input [5]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Arguments: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25], [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] - -(29) ReusedExchange [Reuses operator id: 10] -Output [2]: [w_warehouse_sk#26, w_warehouse_name#27] - -(30) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Right output [2]: [w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_warehouse_sk#21], [w_warehouse_sk#26], Inner, BuildRight - -(31) CometProject -Input [6]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27], [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(34) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(35) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30] - -(36) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Right output [2]: [d_date_sk#28, d_moy#30] -Arguments: [inv_date_sk#23], [d_date_sk#28], Inner, BuildRight - -(37) CometProject -Input [7]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_date_sk#28, d_moy#30] -Arguments: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30], [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] - -(38) CometHashAggregate -Input [5]: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#22 as double)), partial_avg(inv_quantity_on_hand#22)] - -(39) CometExchange -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Arguments: hashpartitioning(w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(40) CometHashAggregate -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#22 as double)), avg(inv_quantity_on_hand#22)] - -(41) CometFilter -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.0) END - -(42) CometProject -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#18 AS mean#36, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN null ELSE (stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))) END AS cov#37] - -(43) CometBroadcastExchange -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] - -(44) CometBroadcastHashJoin -Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] -Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight - -(45) CometExchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(46) CometSort -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 1] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(51) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#9, d_moy#11] - -(52) BroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(55) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(56) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#28, d_moy#30] - -(57) BroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/extended.txt deleted file mode 100644 index 8ff1442cf8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 2cd1f70c8a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt +++ /dev/null @@ -1,65 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [i_item_sk] #4 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [d_date_sk,d_moy] #6 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov] #7 - CometProject [mean,stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_moy] #10 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/explain.txt deleted file mode 100644 index 69fbdb52f0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/explain.txt +++ /dev/null @@ -1,330 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometSort (46) - +- CometColumnarExchange (45) - +- * BroadcastHashJoin Inner BuildRight (44) - :- * Project (24) - : +- * Filter (23) - : +- * HashAggregate (22) - : +- * CometColumnarToRow (21) - : +- CometColumnarExchange (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.inventory (1) - : : : +- BroadcastExchange (7) - : : : +- * CometColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : +- BroadcastExchange (13) - : : +- * CometColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) - : +- ReusedExchange (16) - +- BroadcastExchange (43) - +- * Project (42) - +- * Filter (41) - +- * HashAggregate (40) - +- * CometColumnarToRow (39) - +- CometColumnarExchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * Filter (27) - : : : +- * ColumnarToRow (26) - : : : +- Scan parquet spark_catalog.default.inventory (25) - : : +- ReusedExchange (28) - : +- ReusedExchange (31) - +- ReusedExchange (34) - - -(1) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [1]: [i_item_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [1]: [i_item_sk#6] -Condition : isnotnull(i_item_sk#6) - -(6) CometColumnarToRow [codegen id : 1] -Input [1]: [i_item_sk#6] - -(7) BroadcastExchange -Input [1]: [i_item_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 4] -Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] - -(10) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Condition : isnotnull(w_warehouse_sk#7) - -(12) CometColumnarToRow [codegen id : 2] -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] - -(13) BroadcastExchange -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#7] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] - -(16) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#9, d_moy#10] - -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#10] - -(19) HashAggregate [codegen id : 4] -Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [5]: [n#11, avg#12, m2#13, sum#14, count#15] -Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] - -(20) CometColumnarExchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(21) CometColumnarToRow [codegen id : 10] -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] - -(22) HashAggregate [codegen id : 10] -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#21, avg(inv_quantity_on_hand#3)#22] -Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stddev_samp(cast(inv_quantity_on_hand#3 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#3)#22 AS mean#24] - -(23) Filter [codegen id : 10] -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] -Condition : (CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.0) END AND CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.5) END) - -(24) Project [codegen id : 10] -Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN null ELSE (stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))) END AS cov#25] -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] - -(25) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#29), dynamicpruningexpression(inv_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(26) ColumnarToRow [codegen id : 8] -Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] - -(27) Filter [codegen id : 8] -Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -Condition : (isnotnull(inv_item_sk#26) AND isnotnull(inv_warehouse_sk#27)) - -(28) ReusedExchange [Reuses operator id: 7] -Output [1]: [i_item_sk#31] - -(29) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_item_sk#26] -Right keys [1]: [i_item_sk#31] -Join type: Inner -Join condition: None - -(30) Project [codegen id : 8] -Output [4]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] -Input [5]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] - -(31) ReusedExchange [Reuses operator id: 13] -Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] - -(32) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_warehouse_sk#27] -Right keys [1]: [w_warehouse_sk#32] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] -Input [6]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] - -(34) ReusedExchange [Reuses operator id: 57] -Output [2]: [d_date_sk#34, d_moy#35] - -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_date_sk#29] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None - -(36) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] -Input [7]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_date_sk#34, d_moy#35] - -(37) HashAggregate [codegen id : 8] -Input [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] -Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#28 as double)), partial_avg(inv_quantity_on_hand#28)] -Aggregate Attributes [5]: [n#36, avg#37, m2#38, sum#39, count#40] -Results [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] - -(38) CometColumnarExchange -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(39) CometColumnarToRow [codegen id : 9] -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] - -(40) HashAggregate [codegen id : 9] -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] -Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#28)#22 AS mean#24] - -(41) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.0) END - -(42) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#24 AS mean#46, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN null ELSE (stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))) END AS cov#47] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] - -(43) BroadcastExchange -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] - -(44) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] -Right keys [2]: [i_item_sk#31, w_warehouse_sk#32] -Join type: Inner -Join condition: None - -(45) CometColumnarExchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(46) CometSort -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 11] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometNativeScan parquet spark_catalog.default.date_dim (48) - - -(48) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#48, d_moy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#48, d_moy#10] -Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#10)) AND (d_year#48 = 2001)) AND (d_moy#10 = 1)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#48, d_moy#10] -Arguments: [d_date_sk#9, d_moy#10], [d_date_sk#9, d_moy#10] - -(51) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#9, d_moy#10] - -(52) BroadcastExchange -Input [2]: [d_date_sk#9, d_moy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 25 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometNativeScan parquet spark_catalog.default.date_dim (53) - - -(53) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#49, d_moy#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#34, d_year#49, d_moy#35] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#35)) AND (d_year#49 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#34)) - -(55) CometProject -Input [3]: [d_date_sk#34, d_year#49, d_moy#35] -Arguments: [d_date_sk#34, d_moy#35], [d_date_sk#34, d_moy#35] - -(56) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#34, d_moy#35] - -(57) BroadcastExchange -Input [2]: [d_date_sk#34, d_moy#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/simplified.txt deleted file mode 100644 index 11dd59a97f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/simplified.txt +++ /dev/null @@ -1,84 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 - WholeStageCodegen (8) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [i_item_sk] #4 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - InputAdapter - ReusedExchange [d_date_sk,d_moy] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/explain.txt deleted file mode 100644 index 2a31b0d8b2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,329 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometSort (46) - +- CometExchange (45) - +- CometBroadcastHashJoin (44) - :- CometProject (23) - : +- CometFilter (22) - : +- CometHashAggregate (21) - : +- CometExchange (20) - : +- CometHashAggregate (19) - : +- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (12) - : : +- CometBroadcastHashJoin (11) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : +- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - +- CometBroadcastExchange (43) - +- CometProject (42) - +- CometFilter (41) - +- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (28) - : : +- CometBroadcastHashJoin (27) - : : :- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (24) - : : +- ReusedExchange (26) - : +- ReusedExchange (29) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [i_item_sk#6] -Condition : isnotnull(i_item_sk#6) - -(5) CometBroadcastExchange -Input [1]: [i_item_sk#6] -Arguments: [i_item_sk#6] - -(6) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [i_item_sk#6] -Arguments: [inv_item_sk#1], [i_item_sk#6], Inner, BuildRight - -(7) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6], [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Condition : isnotnull(w_warehouse_sk#7) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [w_warehouse_sk#7, w_warehouse_name#8] - -(11) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Right output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#7], Inner, BuildRight - -(12) CometProject -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8], [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(15) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11] - -(17) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Right output [2]: [d_date_sk#9, d_moy#11] -Arguments: [inv_date_sk#4], [d_date_sk#9], Inner, BuildRight - -(18) CometProject -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#11] -Arguments: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11], [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] - -(19) CometHashAggregate -Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] - -(20) CometExchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] - -(22) CometFilter -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Condition : (CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.0) END AND CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.5) END) - -(23) CometProject -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19], [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN null ELSE (stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))) END AS cov#19] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#23), dynamicpruningexpression(inv_date_sk#23 IN dynamicpruning#24)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Condition : (isnotnull(inv_item_sk#20) AND isnotnull(inv_warehouse_sk#21)) - -(26) ReusedExchange [Reuses operator id: 5] -Output [1]: [i_item_sk#25] - -(27) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Right output [1]: [i_item_sk#25] -Arguments: [inv_item_sk#20], [i_item_sk#25], Inner, BuildRight - -(28) CometProject -Input [5]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Arguments: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25], [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] - -(29) ReusedExchange [Reuses operator id: 10] -Output [2]: [w_warehouse_sk#26, w_warehouse_name#27] - -(30) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Right output [2]: [w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_warehouse_sk#21], [w_warehouse_sk#26], Inner, BuildRight - -(31) CometProject -Input [6]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27], [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(34) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(35) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30] - -(36) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Right output [2]: [d_date_sk#28, d_moy#30] -Arguments: [inv_date_sk#23], [d_date_sk#28], Inner, BuildRight - -(37) CometProject -Input [7]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_date_sk#28, d_moy#30] -Arguments: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30], [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] - -(38) CometHashAggregate -Input [5]: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#22 as double)), partial_avg(inv_quantity_on_hand#22)] - -(39) CometExchange -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Arguments: hashpartitioning(w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(40) CometHashAggregate -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#22 as double)), avg(inv_quantity_on_hand#22)] - -(41) CometFilter -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.0) END - -(42) CometProject -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#18 AS mean#36, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN null ELSE (stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))) END AS cov#37] - -(43) CometBroadcastExchange -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] - -(44) CometBroadcastHashJoin -Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] -Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight - -(45) CometExchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(46) CometSort -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 1] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(51) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#9, d_moy#11] - -(52) BroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(55) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(56) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#28, d_moy#30] - -(57) BroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/simplified.txt deleted file mode 100644 index 2cd1f70c8a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,65 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [i_item_sk] #4 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [d_date_sk,d_moy] #6 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov] #7 - CometProject [mean,stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_moy] #10 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/explain.txt deleted file mode 100644 index 2a31b0d8b2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/explain.txt +++ /dev/null @@ -1,329 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometSort (46) - +- CometExchange (45) - +- CometBroadcastHashJoin (44) - :- CometProject (23) - : +- CometFilter (22) - : +- CometHashAggregate (21) - : +- CometExchange (20) - : +- CometHashAggregate (19) - : +- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (12) - : : +- CometBroadcastHashJoin (11) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : +- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - +- CometBroadcastExchange (43) - +- CometProject (42) - +- CometFilter (41) - +- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (28) - : : +- CometBroadcastHashJoin (27) - : : :- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (24) - : : +- ReusedExchange (26) - : +- ReusedExchange (29) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [i_item_sk#6] -Condition : isnotnull(i_item_sk#6) - -(5) CometBroadcastExchange -Input [1]: [i_item_sk#6] -Arguments: [i_item_sk#6] - -(6) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [i_item_sk#6] -Arguments: [inv_item_sk#1], [i_item_sk#6], Inner, BuildRight - -(7) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6], [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Condition : isnotnull(w_warehouse_sk#7) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [w_warehouse_sk#7, w_warehouse_name#8] - -(11) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Right output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#7], Inner, BuildRight - -(12) CometProject -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8], [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(15) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11] - -(17) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Right output [2]: [d_date_sk#9, d_moy#11] -Arguments: [inv_date_sk#4], [d_date_sk#9], Inner, BuildRight - -(18) CometProject -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#11] -Arguments: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11], [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] - -(19) CometHashAggregate -Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] - -(20) CometExchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] - -(22) CometFilter -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Condition : (CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.0) END AND CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.5) END) - -(23) CometProject -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19], [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN null ELSE (stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))) END AS cov#19] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#23), dynamicpruningexpression(inv_date_sk#23 IN dynamicpruning#24)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Condition : (isnotnull(inv_item_sk#20) AND isnotnull(inv_warehouse_sk#21)) - -(26) ReusedExchange [Reuses operator id: 5] -Output [1]: [i_item_sk#25] - -(27) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Right output [1]: [i_item_sk#25] -Arguments: [inv_item_sk#20], [i_item_sk#25], Inner, BuildRight - -(28) CometProject -Input [5]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Arguments: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25], [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] - -(29) ReusedExchange [Reuses operator id: 10] -Output [2]: [w_warehouse_sk#26, w_warehouse_name#27] - -(30) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Right output [2]: [w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_warehouse_sk#21], [w_warehouse_sk#26], Inner, BuildRight - -(31) CometProject -Input [6]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27], [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(34) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(35) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30] - -(36) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Right output [2]: [d_date_sk#28, d_moy#30] -Arguments: [inv_date_sk#23], [d_date_sk#28], Inner, BuildRight - -(37) CometProject -Input [7]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_date_sk#28, d_moy#30] -Arguments: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30], [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] - -(38) CometHashAggregate -Input [5]: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#22 as double)), partial_avg(inv_quantity_on_hand#22)] - -(39) CometExchange -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Arguments: hashpartitioning(w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(40) CometHashAggregate -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#22 as double)), avg(inv_quantity_on_hand#22)] - -(41) CometFilter -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.0) END - -(42) CometProject -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#18 AS mean#36, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN null ELSE (stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))) END AS cov#37] - -(43) CometBroadcastExchange -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] - -(44) CometBroadcastHashJoin -Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] -Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight - -(45) CometExchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(46) CometSort -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 1] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(51) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#9, d_moy#11] - -(52) BroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(55) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(56) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#28, d_moy#30] - -(57) BroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/extended.txt deleted file mode 100644 index 8ff1442cf8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 2cd1f70c8a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt +++ /dev/null @@ -1,65 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [i_item_sk] #4 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [d_date_sk,d_moy] #6 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov] #7 - CometProject [mean,stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_moy] #10 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/explain.txt deleted file mode 100644 index 9fffd1dde5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/explain.txt +++ /dev/null @@ -1,752 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (120) -+- * Project (119) - +- * BroadcastHashJoin Inner BuildRight (118) - :- * Project (99) - : +- * BroadcastHashJoin Inner BuildRight (98) - : :- * Project (78) - : : +- * BroadcastHashJoin Inner BuildRight (77) - : : :- * Project (58) - : : : +- * BroadcastHashJoin Inner BuildRight (57) - : : : :- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Filter (18) - : : : : : +- * HashAggregate (17) - : : : : : +- * CometColumnarToRow (16) - : : : : : +- CometColumnarExchange (15) - : : : : : +- * HashAggregate (14) - : : : : : +- * Project (13) - : : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * CometColumnarToRow (4) - : : : : : : : +- CometProject (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * Filter (7) - : : : : : : +- * ColumnarToRow (6) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (5) - : : : : : +- ReusedExchange (11) - : : : : +- BroadcastExchange (36) - : : : : +- * HashAggregate (35) - : : : : +- * CometColumnarToRow (34) - : : : : +- CometColumnarExchange (33) - : : : : +- * HashAggregate (32) - : : : : +- * Project (31) - : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : :- * Project (28) - : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : :- * CometColumnarToRow (22) - : : : : : : +- CometProject (21) - : : : : : : +- CometFilter (20) - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) - : : : : : +- BroadcastExchange (26) - : : : : : +- * Filter (25) - : : : : : +- * ColumnarToRow (24) - : : : : : +- Scan parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (29) - : : : +- BroadcastExchange (56) - : : : +- * Filter (55) - : : : +- * HashAggregate (54) - : : : +- * CometColumnarToRow (53) - : : : +- CometColumnarExchange (52) - : : : +- * HashAggregate (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * Project (47) - : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : :- * CometColumnarToRow (41) - : : : : : +- CometProject (40) - : : : : : +- CometFilter (39) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (38) - : : : : +- BroadcastExchange (45) - : : : : +- * Filter (44) - : : : : +- * ColumnarToRow (43) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (42) - : : : +- ReusedExchange (48) - : : +- BroadcastExchange (76) - : : +- * HashAggregate (75) - : : +- * CometColumnarToRow (74) - : : +- CometColumnarExchange (73) - : : +- * HashAggregate (72) - : : +- * Project (71) - : : +- * BroadcastHashJoin Inner BuildRight (70) - : : :- * Project (68) - : : : +- * BroadcastHashJoin Inner BuildRight (67) - : : : :- * CometColumnarToRow (62) - : : : : +- CometProject (61) - : : : : +- CometFilter (60) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (59) - : : : +- BroadcastExchange (66) - : : : +- * Filter (65) - : : : +- * ColumnarToRow (64) - : : : +- Scan parquet spark_catalog.default.catalog_sales (63) - : : +- ReusedExchange (69) - : +- BroadcastExchange (97) - : +- * Filter (96) - : +- * HashAggregate (95) - : +- * CometColumnarToRow (94) - : +- CometColumnarExchange (93) - : +- * HashAggregate (92) - : +- * Project (91) - : +- * BroadcastHashJoin Inner BuildRight (90) - : :- * Project (88) - : : +- * BroadcastHashJoin Inner BuildRight (87) - : : :- * CometColumnarToRow (82) - : : : +- CometProject (81) - : : : +- CometFilter (80) - : : : +- CometNativeScan parquet spark_catalog.default.customer (79) - : : +- BroadcastExchange (86) - : : +- * Filter (85) - : : +- * ColumnarToRow (84) - : : +- Scan parquet spark_catalog.default.web_sales (83) - : +- ReusedExchange (89) - +- BroadcastExchange (117) - +- * HashAggregate (116) - +- * CometColumnarToRow (115) - +- CometColumnarExchange (114) - +- * HashAggregate (113) - +- * Project (112) - +- * BroadcastHashJoin Inner BuildRight (111) - :- * Project (109) - : +- * BroadcastHashJoin Inner BuildRight (108) - : :- * CometColumnarToRow (103) - : : +- CometProject (102) - : : +- CometFilter (101) - : : +- CometNativeScan parquet spark_catalog.default.customer (100) - : +- BroadcastExchange (107) - : +- * Filter (106) - : +- * ColumnarToRow (105) - : +- Scan parquet spark_catalog.default.web_sales (104) - +- ReusedExchange (110) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] - -(4) CometColumnarToRow [codegen id : 3] -Input [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] - -(5) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#20), dynamicpruningexpression(ss_sold_date_sk#20 IN dynamicpruning#21)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] - -(7) Filter [codegen id : 1] -Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Condition : isnotnull(ss_customer_sk#15) - -(8) BroadcastExchange -Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#15] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Input [14]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] - -(11) ReusedExchange [Reuses operator id: 124] -Output [2]: [d_date_sk#22, d_year#23] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#20] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] -Input [14]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20, d_date_sk#22, d_year#23] - -(14) HashAggregate [codegen id : 3] -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] -Functions [1]: [partial_sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] -Aggregate Attributes [2]: [sum#24, isEmpty#25] -Results [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#26, isEmpty#27] - -(15) CometColumnarExchange -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#26, isEmpty#27] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 24] -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#26, isEmpty#27] - -(17) HashAggregate [codegen id : 24] -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#26, isEmpty#27] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] -Functions [1]: [sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] -Aggregate Attributes [1]: [sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))#28] -Results [2]: [c_customer_id#9 AS customer_id#29, sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))#28 AS year_total#30] - -(18) Filter [codegen id : 24] -Input [2]: [customer_id#29, year_total#30] -Condition : (isnotnull(year_total#30) AND (year_total#30 > 0.000000)) - -(19) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_country#36, c_login#37, c_email_address#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(20) CometFilter -Input [8]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_country#36, c_login#37, c_email_address#38] -Condition : (isnotnull(c_customer_sk#31) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#32, 16, true, false, true))) - -(21) CometProject -Input [8]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_country#36, c_login#37, c_email_address#38] -Arguments: [c_customer_sk#31, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14], [c_customer_sk#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#32, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#33, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#34, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#35, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#37, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#38, 50, true, false, true) AS c_email_address#14] - -(22) CometColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#31, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14] - -(23) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#44), dynamicpruningexpression(ss_sold_date_sk#44 IN dynamicpruning#45)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [6]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44] - -(25) Filter [codegen id : 4] -Input [6]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44] -Condition : isnotnull(ss_customer_sk#39) - -(26) BroadcastExchange -Input [6]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#31] -Right keys [1]: [ss_customer_sk#39] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44] -Input [14]: [c_customer_sk#31, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44] - -(29) ReusedExchange [Reuses operator id: 128] -Output [2]: [d_date_sk#46, d_year#47] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#44] -Right keys [1]: [d_date_sk#46] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, d_year#47] -Input [14]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44, d_date_sk#46, d_year#47] - -(32) HashAggregate [codegen id : 6] -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, d_year#47] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47] -Functions [1]: [partial_sum(((((ss_ext_list_price#43 - ss_ext_wholesale_cost#42) - ss_ext_discount_amt#40) + ss_ext_sales_price#41) / 2))] -Aggregate Attributes [2]: [sum#48, isEmpty#49] -Results [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47, sum#50, isEmpty#51] - -(33) CometColumnarExchange -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47, sum#50, isEmpty#51] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 7] -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47, sum#50, isEmpty#51] - -(35) HashAggregate [codegen id : 7] -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47, sum#50, isEmpty#51] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47] -Functions [1]: [sum(((((ss_ext_list_price#43 - ss_ext_wholesale_cost#42) - ss_ext_discount_amt#40) + ss_ext_sales_price#41) / 2))] -Aggregate Attributes [1]: [sum(((((ss_ext_list_price#43 - ss_ext_wholesale_cost#42) - ss_ext_discount_amt#40) + ss_ext_sales_price#41) / 2))#28] -Results [8]: [c_customer_id#9 AS customer_id#52, c_first_name#10 AS customer_first_name#53, c_last_name#11 AS customer_last_name#54, c_preferred_cust_flag#12 AS customer_preferred_cust_flag#55, c_birth_country#36 AS customer_birth_country#56, c_login#13 AS customer_login#57, c_email_address#14 AS customer_email_address#58, sum(((((ss_ext_list_price#43 - ss_ext_wholesale_cost#42) - ss_ext_discount_amt#40) + ss_ext_sales_price#41) / 2))#28 AS year_total#59] - -(36) BroadcastExchange -Input [8]: [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#59] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(37) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#29] -Right keys [1]: [customer_id#52] -Join type: Inner -Join condition: None - -(38) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(39) CometFilter -Input [8]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67] -Condition : (isnotnull(c_customer_sk#60) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#61, 16, true, false, true))) - -(40) CometProject -Input [8]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67] -Arguments: [c_customer_sk#60, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73], [c_customer_sk#60, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#61, 16, true, false, true) AS c_customer_id#68, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#62, 20, true, false, true) AS c_first_name#69, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#63, 30, true, false, true) AS c_last_name#70, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#64, 1, true, false, true) AS c_preferred_cust_flag#71, c_birth_country#65, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#66, 13, true, false, true) AS c_login#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#67, 50, true, false, true) AS c_email_address#73] - -(41) CometColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#60, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73] - -(42) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#74, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#79), dynamicpruningexpression(cs_sold_date_sk#79 IN dynamicpruning#21)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 8] -Input [6]: [cs_bill_customer_sk#74, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79] - -(44) Filter [codegen id : 8] -Input [6]: [cs_bill_customer_sk#74, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79] -Condition : isnotnull(cs_bill_customer_sk#74) - -(45) BroadcastExchange -Input [6]: [cs_bill_customer_sk#74, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -(46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#60] -Right keys [1]: [cs_bill_customer_sk#74] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 10] -Output [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79] -Input [14]: [c_customer_sk#60, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, cs_bill_customer_sk#74, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79] - -(48) ReusedExchange [Reuses operator id: 124] -Output [2]: [d_date_sk#80, d_year#81] - -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#79] -Right keys [1]: [d_date_sk#80] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 10] -Output [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, d_year#81] -Input [14]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79, d_date_sk#80, d_year#81] - -(51) HashAggregate [codegen id : 10] -Input [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, d_year#81] -Keys [8]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81] -Functions [1]: [partial_sum(((((cs_ext_list_price#78 - cs_ext_wholesale_cost#77) - cs_ext_discount_amt#75) + cs_ext_sales_price#76) / 2))] -Aggregate Attributes [2]: [sum#82, isEmpty#83] -Results [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81, sum#84, isEmpty#85] - -(52) CometColumnarExchange -Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81, sum#84, isEmpty#85] -Arguments: hashpartitioning(c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(53) CometColumnarToRow [codegen id : 11] -Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81, sum#84, isEmpty#85] - -(54) HashAggregate [codegen id : 11] -Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81, sum#84, isEmpty#85] -Keys [8]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81] -Functions [1]: [sum(((((cs_ext_list_price#78 - cs_ext_wholesale_cost#77) - cs_ext_discount_amt#75) + cs_ext_sales_price#76) / 2))] -Aggregate Attributes [1]: [sum(((((cs_ext_list_price#78 - cs_ext_wholesale_cost#77) - cs_ext_discount_amt#75) + cs_ext_sales_price#76) / 2))#86] -Results [2]: [c_customer_id#68 AS customer_id#87, sum(((((cs_ext_list_price#78 - cs_ext_wholesale_cost#77) - cs_ext_discount_amt#75) + cs_ext_sales_price#76) / 2))#86 AS year_total#88] - -(55) Filter [codegen id : 11] -Input [2]: [customer_id#87, year_total#88] -Condition : (isnotnull(year_total#88) AND (year_total#88 > 0.000000)) - -(56) BroadcastExchange -Input [2]: [customer_id#87, year_total#88] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(57) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#29] -Right keys [1]: [customer_id#87] -Join type: Inner -Join condition: None - -(58) Project [codegen id : 24] -Output [11]: [customer_id#29, year_total#30, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#59, year_total#88] -Input [12]: [customer_id#29, year_total#30, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#59, customer_id#87, year_total#88] - -(59) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#89, c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(60) CometFilter -Input [8]: [c_customer_sk#89, c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96] -Condition : (isnotnull(c_customer_sk#89) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#90, 16, true, false, true))) - -(61) CometProject -Input [8]: [c_customer_sk#89, c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96] -Arguments: [c_customer_sk#89, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73], [c_customer_sk#89, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#90, 16, true, false, true) AS c_customer_id#68, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#91, 20, true, false, true) AS c_first_name#69, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#92, 30, true, false, true) AS c_last_name#70, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#93, 1, true, false, true) AS c_preferred_cust_flag#71, c_birth_country#94, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#95, 13, true, false, true) AS c_login#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#96, 50, true, false, true) AS c_email_address#73] - -(62) CometColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#89, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73] - -(63) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#102), dynamicpruningexpression(cs_sold_date_sk#102 IN dynamicpruning#45)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(64) ColumnarToRow [codegen id : 12] -Input [6]: [cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] - -(65) Filter [codegen id : 12] -Input [6]: [cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] -Condition : isnotnull(cs_bill_customer_sk#97) - -(66) BroadcastExchange -Input [6]: [cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#89] -Right keys [1]: [cs_bill_customer_sk#97] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 14] -Output [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] -Input [14]: [c_customer_sk#89, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] - -(69) ReusedExchange [Reuses operator id: 128] -Output [2]: [d_date_sk#103, d_year#104] - -(70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [cs_sold_date_sk#102] -Right keys [1]: [d_date_sk#103] -Join type: Inner -Join condition: None - -(71) Project [codegen id : 14] -Output [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, d_year#104] -Input [14]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102, d_date_sk#103, d_year#104] - -(72) HashAggregate [codegen id : 14] -Input [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, d_year#104] -Keys [8]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104] -Functions [1]: [partial_sum(((((cs_ext_list_price#101 - cs_ext_wholesale_cost#100) - cs_ext_discount_amt#98) + cs_ext_sales_price#99) / 2))] -Aggregate Attributes [2]: [sum#105, isEmpty#106] -Results [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104, sum#107, isEmpty#108] - -(73) CometColumnarExchange -Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104, sum#107, isEmpty#108] -Arguments: hashpartitioning(c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(74) CometColumnarToRow [codegen id : 15] -Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104, sum#107, isEmpty#108] - -(75) HashAggregate [codegen id : 15] -Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104, sum#107, isEmpty#108] -Keys [8]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104] -Functions [1]: [sum(((((cs_ext_list_price#101 - cs_ext_wholesale_cost#100) - cs_ext_discount_amt#98) + cs_ext_sales_price#99) / 2))] -Aggregate Attributes [1]: [sum(((((cs_ext_list_price#101 - cs_ext_wholesale_cost#100) - cs_ext_discount_amt#98) + cs_ext_sales_price#99) / 2))#86] -Results [2]: [c_customer_id#68 AS customer_id#109, sum(((((cs_ext_list_price#101 - cs_ext_wholesale_cost#100) - cs_ext_discount_amt#98) + cs_ext_sales_price#99) / 2))#86 AS year_total#110] - -(76) BroadcastExchange -Input [2]: [customer_id#109, year_total#110] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] - -(77) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#29] -Right keys [1]: [customer_id#109] -Join type: Inner -Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN (year_total#110 / year_total#88) END > CASE WHEN (year_total#30 > 0.000000) THEN (year_total#59 / year_total#30) END) - -(78) Project [codegen id : 24] -Output [10]: [customer_id#29, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#88, year_total#110] -Input [13]: [customer_id#29, year_total#30, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#59, year_total#88, customer_id#109, year_total#110] - -(79) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#111, c_customer_id#112, c_first_name#113, c_last_name#114, c_preferred_cust_flag#115, c_birth_country#116, c_login#117, c_email_address#118] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(80) CometFilter -Input [8]: [c_customer_sk#111, c_customer_id#112, c_first_name#113, c_last_name#114, c_preferred_cust_flag#115, c_birth_country#116, c_login#117, c_email_address#118] -Condition : (isnotnull(c_customer_sk#111) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#112, 16, true, false, true))) - -(81) CometProject -Input [8]: [c_customer_sk#111, c_customer_id#112, c_first_name#113, c_last_name#114, c_preferred_cust_flag#115, c_birth_country#116, c_login#117, c_email_address#118] -Arguments: [c_customer_sk#111, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124], [c_customer_sk#111, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#112, 16, true, false, true) AS c_customer_id#119, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#113, 20, true, false, true) AS c_first_name#120, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#114, 30, true, false, true) AS c_last_name#121, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#115, 1, true, false, true) AS c_preferred_cust_flag#122, c_birth_country#116, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#117, 13, true, false, true) AS c_login#123, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#118, 50, true, false, true) AS c_email_address#124] - -(82) CometColumnarToRow [codegen id : 18] -Input [8]: [c_customer_sk#111, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124] - -(83) Scan parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#125, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#130), dynamicpruningexpression(ws_sold_date_sk#130 IN dynamicpruning#21)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(84) ColumnarToRow [codegen id : 16] -Input [6]: [ws_bill_customer_sk#125, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130] - -(85) Filter [codegen id : 16] -Input [6]: [ws_bill_customer_sk#125, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130] -Condition : isnotnull(ws_bill_customer_sk#125) - -(86) BroadcastExchange -Input [6]: [ws_bill_customer_sk#125, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -(87) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_customer_sk#111] -Right keys [1]: [ws_bill_customer_sk#125] -Join type: Inner -Join condition: None - -(88) Project [codegen id : 18] -Output [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130] -Input [14]: [c_customer_sk#111, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, ws_bill_customer_sk#125, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130] - -(89) ReusedExchange [Reuses operator id: 124] -Output [2]: [d_date_sk#131, d_year#132] - -(90) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#130] -Right keys [1]: [d_date_sk#131] -Join type: Inner -Join condition: None - -(91) Project [codegen id : 18] -Output [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, d_year#132] -Input [14]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130, d_date_sk#131, d_year#132] - -(92) HashAggregate [codegen id : 18] -Input [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, d_year#132] -Keys [8]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132] -Functions [1]: [partial_sum(((((ws_ext_list_price#129 - ws_ext_wholesale_cost#128) - ws_ext_discount_amt#126) + ws_ext_sales_price#127) / 2))] -Aggregate Attributes [2]: [sum#133, isEmpty#134] -Results [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132, sum#135, isEmpty#136] - -(93) CometColumnarExchange -Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132, sum#135, isEmpty#136] -Arguments: hashpartitioning(c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(94) CometColumnarToRow [codegen id : 19] -Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132, sum#135, isEmpty#136] - -(95) HashAggregate [codegen id : 19] -Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132, sum#135, isEmpty#136] -Keys [8]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132] -Functions [1]: [sum(((((ws_ext_list_price#129 - ws_ext_wholesale_cost#128) - ws_ext_discount_amt#126) + ws_ext_sales_price#127) / 2))] -Aggregate Attributes [1]: [sum(((((ws_ext_list_price#129 - ws_ext_wholesale_cost#128) - ws_ext_discount_amt#126) + ws_ext_sales_price#127) / 2))#137] -Results [2]: [c_customer_id#119 AS customer_id#138, sum(((((ws_ext_list_price#129 - ws_ext_wholesale_cost#128) - ws_ext_discount_amt#126) + ws_ext_sales_price#127) / 2))#137 AS year_total#139] - -(96) Filter [codegen id : 19] -Input [2]: [customer_id#138, year_total#139] -Condition : (isnotnull(year_total#139) AND (year_total#139 > 0.000000)) - -(97) BroadcastExchange -Input [2]: [customer_id#138, year_total#139] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=14] - -(98) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#29] -Right keys [1]: [customer_id#138] -Join type: Inner -Join condition: None - -(99) Project [codegen id : 24] -Output [11]: [customer_id#29, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#88, year_total#110, year_total#139] -Input [12]: [customer_id#29, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#88, year_total#110, customer_id#138, year_total#139] - -(100) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#140, c_customer_id#141, c_first_name#142, c_last_name#143, c_preferred_cust_flag#144, c_birth_country#145, c_login#146, c_email_address#147] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(101) CometFilter -Input [8]: [c_customer_sk#140, c_customer_id#141, c_first_name#142, c_last_name#143, c_preferred_cust_flag#144, c_birth_country#145, c_login#146, c_email_address#147] -Condition : (isnotnull(c_customer_sk#140) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#141, 16, true, false, true))) - -(102) CometProject -Input [8]: [c_customer_sk#140, c_customer_id#141, c_first_name#142, c_last_name#143, c_preferred_cust_flag#144, c_birth_country#145, c_login#146, c_email_address#147] -Arguments: [c_customer_sk#140, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124], [c_customer_sk#140, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#141, 16, true, false, true) AS c_customer_id#119, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#142, 20, true, false, true) AS c_first_name#120, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#143, 30, true, false, true) AS c_last_name#121, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#144, 1, true, false, true) AS c_preferred_cust_flag#122, c_birth_country#145, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#146, 13, true, false, true) AS c_login#123, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#147, 50, true, false, true) AS c_email_address#124] - -(103) CometColumnarToRow [codegen id : 22] -Input [8]: [c_customer_sk#140, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124] - -(104) Scan parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#153), dynamicpruningexpression(ws_sold_date_sk#153 IN dynamicpruning#45)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(105) ColumnarToRow [codegen id : 20] -Input [6]: [ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] - -(106) Filter [codegen id : 20] -Input [6]: [ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] -Condition : isnotnull(ws_bill_customer_sk#148) - -(107) BroadcastExchange -Input [6]: [ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] - -(108) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [c_customer_sk#140] -Right keys [1]: [ws_bill_customer_sk#148] -Join type: Inner -Join condition: None - -(109) Project [codegen id : 22] -Output [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] -Input [14]: [c_customer_sk#140, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] - -(110) ReusedExchange [Reuses operator id: 128] -Output [2]: [d_date_sk#154, d_year#155] - -(111) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#153] -Right keys [1]: [d_date_sk#154] -Join type: Inner -Join condition: None - -(112) Project [codegen id : 22] -Output [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, d_year#155] -Input [14]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153, d_date_sk#154, d_year#155] - -(113) HashAggregate [codegen id : 22] -Input [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, d_year#155] -Keys [8]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155] -Functions [1]: [partial_sum(((((ws_ext_list_price#152 - ws_ext_wholesale_cost#151) - ws_ext_discount_amt#149) + ws_ext_sales_price#150) / 2))] -Aggregate Attributes [2]: [sum#156, isEmpty#157] -Results [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155, sum#158, isEmpty#159] - -(114) CometColumnarExchange -Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155, sum#158, isEmpty#159] -Arguments: hashpartitioning(c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(115) CometColumnarToRow [codegen id : 23] -Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155, sum#158, isEmpty#159] - -(116) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155, sum#158, isEmpty#159] -Keys [8]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155] -Functions [1]: [sum(((((ws_ext_list_price#152 - ws_ext_wholesale_cost#151) - ws_ext_discount_amt#149) + ws_ext_sales_price#150) / 2))] -Aggregate Attributes [1]: [sum(((((ws_ext_list_price#152 - ws_ext_wholesale_cost#151) - ws_ext_discount_amt#149) + ws_ext_sales_price#150) / 2))#137] -Results [2]: [c_customer_id#119 AS customer_id#160, sum(((((ws_ext_list_price#152 - ws_ext_wholesale_cost#151) - ws_ext_discount_amt#149) + ws_ext_sales_price#150) / 2))#137 AS year_total#161] - -(117) BroadcastExchange -Input [2]: [customer_id#160, year_total#161] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=17] - -(118) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#29] -Right keys [1]: [customer_id#160] -Join type: Inner -Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN (year_total#110 / year_total#88) END > CASE WHEN (year_total#139 > 0.000000) THEN (year_total#161 / year_total#139) END) - -(119) Project [codegen id : 24] -Output [7]: [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58] -Input [13]: [customer_id#29, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#88, year_total#110, year_total#139, customer_id#160, year_total#161] - -(120) TakeOrderedAndProject -Input [7]: [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58] -Arguments: 100, [customer_id#52 ASC NULLS FIRST, customer_first_name#53 ASC NULLS FIRST, customer_last_name#54 ASC NULLS FIRST, customer_preferred_cust_flag#55 ASC NULLS FIRST, customer_birth_country#56 ASC NULLS FIRST, customer_login#57 ASC NULLS FIRST, customer_email_address#58 ASC NULLS FIRST], [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 -BroadcastExchange (124) -+- * CometColumnarToRow (123) - +- CometFilter (122) - +- CometNativeScan parquet spark_catalog.default.date_dim (121) - - -(121) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_year#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(122) CometFilter -Input [2]: [d_date_sk#22, d_year#23] -Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(123) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#22, d_year#23] - -(124) BroadcastExchange -Input [2]: [d_date_sk#22, d_year#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#44 IN dynamicpruning#45 -BroadcastExchange (128) -+- * CometColumnarToRow (127) - +- CometFilter (126) - +- CometNativeScan parquet spark_catalog.default.date_dim (125) - - -(125) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#46, d_year#47] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#46, d_year#47] -Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2002)) AND isnotnull(d_date_sk#46)) - -(127) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#46, d_year#47] - -(128) BroadcastExchange -Input [2]: [d_date_sk#46, d_year#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] - -Subquery:3 Hosting operator id = 42 Hosting Expression = cs_sold_date_sk#79 IN dynamicpruning#21 - -Subquery:4 Hosting operator id = 63 Hosting Expression = cs_sold_date_sk#102 IN dynamicpruning#45 - -Subquery:5 Hosting operator id = 83 Hosting Expression = ws_sold_date_sk#130 IN dynamicpruning#21 - -Subquery:6 Hosting operator id = 104 Hosting Expression = ws_sold_date_sk#153 IN dynamicpruning#45 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/simplified.txt deleted file mode 100644 index 56bc32e144..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/simplified.txt +++ /dev/null @@ -1,191 +0,0 @@ -TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - WholeStageCodegen (24) - Project [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 - WholeStageCodegen (3) - HashAggregate [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] [sum,isEmpty,sum,isEmpty] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #5 - WholeStageCodegen (6) - HashAggregate [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] [sum,isEmpty,sum,isEmpty] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) - HashAggregate [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] [sum,isEmpty,sum,isEmpty] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) - HashAggregate [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] [sum,isEmpty,sum,isEmpty] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (19) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #15 - WholeStageCodegen (18) - HashAggregate [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] [sum,isEmpty,sum,isEmpty] - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (16) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (23) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #18 - WholeStageCodegen (22) - HashAggregate [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] [sum,isEmpty,sum,isEmpty] - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #19 - WholeStageCodegen (20) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/explain.txt deleted file mode 100644 index cea1c1d430..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,671 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (107) -+- CometTakeOrderedAndProject (106) - +- CometProject (105) - +- CometBroadcastHashJoin (104) - :- CometProject (88) - : +- CometBroadcastHashJoin (87) - : :- CometProject (70) - : : +- CometBroadcastHashJoin (69) - : : :- CometProject (53) - : : : +- CometBroadcastHashJoin (52) - : : : :- CometBroadcastHashJoin (35) - : : : : :- CometFilter (17) - : : : : : +- CometHashAggregate (16) - : : : : : +- CometExchange (15) - : : : : : +- CometHashAggregate (14) - : : : : : +- CometProject (13) - : : : : : +- CometBroadcastHashJoin (12) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometProject (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : : : +- CometBroadcastExchange (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : : : +- CometBroadcastExchange (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : : : +- CometBroadcastExchange (34) - : : : : +- CometHashAggregate (33) - : : : : +- CometExchange (32) - : : : : +- CometHashAggregate (31) - : : : : +- CometProject (30) - : : : : +- CometBroadcastHashJoin (29) - : : : : :- CometProject (25) - : : : : : +- CometBroadcastHashJoin (24) - : : : : : :- CometProject (20) - : : : : : : +- CometFilter (19) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : : : +- CometBroadcastExchange (23) - : : : : : +- CometFilter (22) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : : : +- CometBroadcastExchange (28) - : : : : +- CometFilter (27) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : : : +- CometBroadcastExchange (51) - : : : +- CometFilter (50) - : : : +- CometHashAggregate (49) - : : : +- CometExchange (48) - : : : +- CometHashAggregate (47) - : : : +- CometProject (46) - : : : +- CometBroadcastHashJoin (45) - : : : :- CometProject (43) - : : : : +- CometBroadcastHashJoin (42) - : : : : :- CometProject (38) - : : : : : +- CometFilter (37) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : : : +- CometBroadcastExchange (41) - : : : : +- CometFilter (40) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) - : : : +- ReusedExchange (44) - : : +- CometBroadcastExchange (68) - : : +- CometHashAggregate (67) - : : +- CometExchange (66) - : : +- CometHashAggregate (65) - : : +- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometProject (61) - : : : +- CometBroadcastHashJoin (60) - : : : :- CometProject (56) - : : : : +- CometFilter (55) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : : : +- CometBroadcastExchange (59) - : : : +- CometFilter (58) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (57) - : : +- ReusedExchange (62) - : +- CometBroadcastExchange (86) - : +- CometFilter (85) - : +- CometHashAggregate (84) - : +- CometExchange (83) - : +- CometHashAggregate (82) - : +- CometProject (81) - : +- CometBroadcastHashJoin (80) - : :- CometProject (78) - : : +- CometBroadcastHashJoin (77) - : : :- CometProject (73) - : : : +- CometFilter (72) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (71) - : : +- CometBroadcastExchange (76) - : : +- CometFilter (75) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (74) - : +- ReusedExchange (79) - +- CometBroadcastExchange (103) - +- CometHashAggregate (102) - +- CometExchange (101) - +- CometHashAggregate (100) - +- CometProject (99) - +- CometBroadcastHashJoin (98) - :- CometProject (96) - : +- CometBroadcastHashJoin (95) - : :- CometProject (91) - : : +- CometFilter (90) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (89) - : +- CometBroadcastExchange (94) - : +- CometFilter (93) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (92) - +- ReusedExchange (97) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#20), dynamicpruningexpression(ss_sold_date_sk#20 IN dynamicpruning#21)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Condition : isnotnull(ss_customer_sk#15) - -(6) CometBroadcastExchange -Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Arguments: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] - -(7) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Right output [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Arguments: [c_customer_sk#1], [ss_customer_sk#15], Inner, BuildRight - -(8) CometProject -Input [14]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_year#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#22, d_year#23] -Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#22, d_year#23] -Arguments: [d_date_sk#22, d_year#23] - -(12) CometBroadcastHashJoin -Left output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Right output [2]: [d_date_sk#22, d_year#23] -Arguments: [ss_sold_date_sk#20], [d_date_sk#22], Inner, BuildRight - -(13) CometProject -Input [14]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20, d_date_sk#22, d_year#23] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] - -(14) CometHashAggregate -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] -Functions [1]: [partial_sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] - -(15) CometExchange -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#24, isEmpty#25] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] -Functions [1]: [sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] - -(17) CometFilter -Input [2]: [customer_id#26, year_total#27] -Condition : (isnotnull(year_total#27) AND (year_total#27 > 0.000000)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] -Condition : (isnotnull(c_customer_sk#28) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#29, 16, true, false, true))) - -(20) CometProject -Input [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] -Arguments: [c_customer_sk#28, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14], [c_customer_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#29, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#30, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#31, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#32, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#34, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#35, 50, true, false, true) AS c_email_address#14] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#41), dynamicpruningexpression(ss_sold_date_sk#41 IN dynamicpruning#42)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Condition : isnotnull(ss_customer_sk#36) - -(23) CometBroadcastExchange -Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Arguments: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] - -(24) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#28, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14] -Right output [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Arguments: [c_customer_sk#28], [ss_customer_sk#36], Inner, BuildRight - -(25) CometProject -Input [14]: [c_customer_sk#28, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_year#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#43, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2002)) AND isnotnull(d_date_sk#43)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#43, d_year#44] -Arguments: [d_date_sk#43, d_year#44] - -(29) CometBroadcastHashJoin -Left output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Right output [2]: [d_date_sk#43, d_year#44] -Arguments: [ss_sold_date_sk#41], [d_date_sk#43], Inner, BuildRight - -(30) CometProject -Input [14]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41, d_date_sk#43, d_year#44] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, d_year#44], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, d_year#44] - -(31) CometHashAggregate -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, d_year#44] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, d_year#44] -Functions [1]: [partial_sum(((((ss_ext_list_price#40 - ss_ext_wholesale_cost#39) - ss_ext_discount_amt#37) + ss_ext_sales_price#38) / 2))] - -(32) CometExchange -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, d_year#44, sum#45, isEmpty#46] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, d_year#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, d_year#44, sum#45, isEmpty#46] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, d_year#44] -Functions [1]: [sum(((((ss_ext_list_price#40 - ss_ext_wholesale_cost#39) - ss_ext_discount_amt#37) + ss_ext_sales_price#38) / 2))] - -(34) CometBroadcastExchange -Input [8]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54] -Arguments: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#26, year_total#27] -Right output [8]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54] -Arguments: [customer_id#26], [customer_id#47], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [8]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62] -Condition : (isnotnull(c_customer_sk#55) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#56, 16, true, false, true))) - -(38) CometProject -Input [8]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62] -Arguments: [c_customer_sk#55, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68], [c_customer_sk#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#56, 16, true, false, true) AS c_customer_id#63, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#57, 20, true, false, true) AS c_first_name#64, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#58, 30, true, false, true) AS c_last_name#65, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#59, 1, true, false, true) AS c_preferred_cust_flag#66, c_birth_country#60, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#61, 13, true, false, true) AS c_login#67, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#62, 50, true, false, true) AS c_email_address#68] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_sold_date_sk#74 IN dynamicpruning#75)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [6]: [cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Condition : isnotnull(cs_bill_customer_sk#69) - -(41) CometBroadcastExchange -Input [6]: [cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Arguments: [cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] - -(42) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#55, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68] -Right output [6]: [cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Arguments: [c_customer_sk#55], [cs_bill_customer_sk#69], Inner, BuildRight - -(43) CometProject -Input [14]: [c_customer_sk#55, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Arguments: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74], [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#76, d_year#77] - -(45) CometBroadcastHashJoin -Left output [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Right output [2]: [d_date_sk#76, d_year#77] -Arguments: [cs_sold_date_sk#74], [d_date_sk#76], Inner, BuildRight - -(46) CometProject -Input [14]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74, d_date_sk#76, d_year#77] -Arguments: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, d_year#77], [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, d_year#77] - -(47) CometHashAggregate -Input [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, d_year#77] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#77] -Functions [1]: [partial_sum(((((cs_ext_list_price#73 - cs_ext_wholesale_cost#72) - cs_ext_discount_amt#70) + cs_ext_sales_price#71) / 2))] - -(48) CometExchange -Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#77, sum#78, isEmpty#79] -Arguments: hashpartitioning(c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#77, sum#78, isEmpty#79] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#77] -Functions [1]: [sum(((((cs_ext_list_price#73 - cs_ext_wholesale_cost#72) - cs_ext_discount_amt#70) + cs_ext_sales_price#71) / 2))] - -(50) CometFilter -Input [2]: [customer_id#80, year_total#81] -Condition : (isnotnull(year_total#81) AND (year_total#81 > 0.000000)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#80, year_total#81] -Arguments: [customer_id#80, year_total#81] - -(52) CometBroadcastHashJoin -Left output [10]: [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54] -Right output [2]: [customer_id#80, year_total#81] -Arguments: [customer_id#26], [customer_id#80], Inner, BuildRight - -(53) CometProject -Input [12]: [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54, customer_id#80, year_total#81] -Arguments: [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54, year_total#81], [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54, year_total#81] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] -Condition : (isnotnull(c_customer_sk#82) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#83, 16, true, false, true))) - -(56) CometProject -Input [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] -Arguments: [c_customer_sk#82, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68], [c_customer_sk#82, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#83, 16, true, false, true) AS c_customer_id#63, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#84, 20, true, false, true) AS c_first_name#64, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#85, 30, true, false, true) AS c_last_name#65, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#86, 1, true, false, true) AS c_preferred_cust_flag#66, c_birth_country#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#88, 13, true, false, true) AS c_login#67, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#89, 50, true, false, true) AS c_email_address#68] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#95), dynamicpruningexpression(cs_sold_date_sk#95 IN dynamicpruning#96)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Condition : isnotnull(cs_bill_customer_sk#90) - -(59) CometBroadcastExchange -Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Arguments: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] - -(60) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#82, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68] -Right output [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Arguments: [c_customer_sk#82], [cs_bill_customer_sk#90], Inner, BuildRight - -(61) CometProject -Input [14]: [c_customer_sk#82, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Arguments: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95], [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#97, d_year#98] - -(63) CometBroadcastHashJoin -Left output [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Right output [2]: [d_date_sk#97, d_year#98] -Arguments: [cs_sold_date_sk#95], [d_date_sk#97], Inner, BuildRight - -(64) CometProject -Input [14]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95, d_date_sk#97, d_year#98] -Arguments: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#98], [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#98] - -(65) CometHashAggregate -Input [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#98] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, d_year#98] -Functions [1]: [partial_sum(((((cs_ext_list_price#94 - cs_ext_wholesale_cost#93) - cs_ext_discount_amt#91) + cs_ext_sales_price#92) / 2))] - -(66) CometExchange -Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, d_year#98, sum#99, isEmpty#100] -Arguments: hashpartitioning(c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, d_year#98, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, d_year#98, sum#99, isEmpty#100] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, d_year#98] -Functions [1]: [sum(((((cs_ext_list_price#94 - cs_ext_wholesale_cost#93) - cs_ext_discount_amt#91) + cs_ext_sales_price#92) / 2))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#101, year_total#102] -Arguments: [customer_id#101, year_total#102] - -(69) CometBroadcastHashJoin -Left output [11]: [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54, year_total#81] -Right output [2]: [customer_id#101, year_total#102] -Arguments: [customer_id#26], [customer_id#101], Inner, (CASE WHEN (year_total#81 > 0.000000) THEN (year_total#102 / year_total#81) END > CASE WHEN (year_total#27 > 0.000000) THEN (year_total#54 / year_total#27) END), BuildRight - -(70) CometProject -Input [13]: [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54, year_total#81, customer_id#101, year_total#102] -Arguments: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102], [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102] - -(71) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#103, c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(72) CometFilter -Input [8]: [c_customer_sk#103, c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110] -Condition : (isnotnull(c_customer_sk#103) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#104, 16, true, false, true))) - -(73) CometProject -Input [8]: [c_customer_sk#103, c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110] -Arguments: [c_customer_sk#103, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116], [c_customer_sk#103, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#104, 16, true, false, true) AS c_customer_id#111, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#105, 20, true, false, true) AS c_first_name#112, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#106, 30, true, false, true) AS c_last_name#113, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#107, 1, true, false, true) AS c_preferred_cust_flag#114, c_birth_country#108, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#109, 13, true, false, true) AS c_login#115, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#110, 50, true, false, true) AS c_email_address#116] - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#122), dynamicpruningexpression(ws_sold_date_sk#122 IN dynamicpruning#123)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(75) CometFilter -Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Condition : isnotnull(ws_bill_customer_sk#117) - -(76) CometBroadcastExchange -Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Arguments: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] - -(77) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#103, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116] -Right output [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Arguments: [c_customer_sk#103], [ws_bill_customer_sk#117], Inner, BuildRight - -(78) CometProject -Input [14]: [c_customer_sk#103, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Arguments: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122], [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] - -(79) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#124, d_year#125] - -(80) CometBroadcastHashJoin -Left output [12]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Right output [2]: [d_date_sk#124, d_year#125] -Arguments: [ws_sold_date_sk#122], [d_date_sk#124], Inner, BuildRight - -(81) CometProject -Input [14]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122, d_date_sk#124, d_year#125] -Arguments: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#125], [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#125] - -(82) CometHashAggregate -Input [12]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#125] -Keys [8]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, d_year#125] -Functions [1]: [partial_sum(((((ws_ext_list_price#121 - ws_ext_wholesale_cost#120) - ws_ext_discount_amt#118) + ws_ext_sales_price#119) / 2))] - -(83) CometExchange -Input [10]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, d_year#125, sum#126, isEmpty#127] -Arguments: hashpartitioning(c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, d_year#125, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(84) CometHashAggregate -Input [10]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, d_year#125, sum#126, isEmpty#127] -Keys [8]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, d_year#125] -Functions [1]: [sum(((((ws_ext_list_price#121 - ws_ext_wholesale_cost#120) - ws_ext_discount_amt#118) + ws_ext_sales_price#119) / 2))] - -(85) CometFilter -Input [2]: [customer_id#128, year_total#129] -Condition : (isnotnull(year_total#129) AND (year_total#129 > 0.000000)) - -(86) CometBroadcastExchange -Input [2]: [customer_id#128, year_total#129] -Arguments: [customer_id#128, year_total#129] - -(87) CometBroadcastHashJoin -Left output [10]: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102] -Right output [2]: [customer_id#128, year_total#129] -Arguments: [customer_id#26], [customer_id#128], Inner, BuildRight - -(88) CometProject -Input [12]: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102, customer_id#128, year_total#129] -Arguments: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102, year_total#129], [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102, year_total#129] - -(89) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#130, c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(90) CometFilter -Input [8]: [c_customer_sk#130, c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137] -Condition : (isnotnull(c_customer_sk#130) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#131, 16, true, false, true))) - -(91) CometProject -Input [8]: [c_customer_sk#130, c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137] -Arguments: [c_customer_sk#130, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116], [c_customer_sk#130, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#131, 16, true, false, true) AS c_customer_id#111, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#132, 20, true, false, true) AS c_first_name#112, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#133, 30, true, false, true) AS c_last_name#113, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#134, 1, true, false, true) AS c_preferred_cust_flag#114, c_birth_country#135, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#136, 13, true, false, true) AS c_login#115, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#137, 50, true, false, true) AS c_email_address#116] - -(92) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#143), dynamicpruningexpression(ws_sold_date_sk#143 IN dynamicpruning#144)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(93) CometFilter -Input [6]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Condition : isnotnull(ws_bill_customer_sk#138) - -(94) CometBroadcastExchange -Input [6]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Arguments: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] - -(95) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#130, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116] -Right output [6]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Arguments: [c_customer_sk#130], [ws_bill_customer_sk#138], Inner, BuildRight - -(96) CometProject -Input [14]: [c_customer_sk#130, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Arguments: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143], [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] - -(97) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#145, d_year#146] - -(98) CometBroadcastHashJoin -Left output [12]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Right output [2]: [d_date_sk#145, d_year#146] -Arguments: [ws_sold_date_sk#143], [d_date_sk#145], Inner, BuildRight - -(99) CometProject -Input [14]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143, d_date_sk#145, d_year#146] -Arguments: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, d_year#146], [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, d_year#146] - -(100) CometHashAggregate -Input [12]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, d_year#146] -Keys [8]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, d_year#146] -Functions [1]: [partial_sum(((((ws_ext_list_price#142 - ws_ext_wholesale_cost#141) - ws_ext_discount_amt#139) + ws_ext_sales_price#140) / 2))] - -(101) CometExchange -Input [10]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, d_year#146, sum#147, isEmpty#148] -Arguments: hashpartitioning(c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, d_year#146, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(102) CometHashAggregate -Input [10]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, d_year#146, sum#147, isEmpty#148] -Keys [8]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, d_year#146] -Functions [1]: [sum(((((ws_ext_list_price#142 - ws_ext_wholesale_cost#141) - ws_ext_discount_amt#139) + ws_ext_sales_price#140) / 2))] - -(103) CometBroadcastExchange -Input [2]: [customer_id#149, year_total#150] -Arguments: [customer_id#149, year_total#150] - -(104) CometBroadcastHashJoin -Left output [11]: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102, year_total#129] -Right output [2]: [customer_id#149, year_total#150] -Arguments: [customer_id#26], [customer_id#149], Inner, (CASE WHEN (year_total#81 > 0.000000) THEN (year_total#102 / year_total#81) END > CASE WHEN (year_total#129 > 0.000000) THEN (year_total#150 / year_total#129) END), BuildRight - -(105) CometProject -Input [13]: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102, year_total#129, customer_id#149, year_total#150] -Arguments: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53], [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53] - -(106) CometTakeOrderedAndProject -Input [7]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#47 ASC NULLS FIRST,customer_first_name#48 ASC NULLS FIRST,customer_last_name#49 ASC NULLS FIRST,customer_preferred_cust_flag#50 ASC NULLS FIRST,customer_birth_country#51 ASC NULLS FIRST,customer_login#52 ASC NULLS FIRST,customer_email_address#53 ASC NULLS FIRST], output=[customer_id#47,customer_first_name#48,customer_last_name#49,customer_preferred_cust_flag#50,customer_birth_country#51,customer_login#52,customer_email_address#53]), [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53], 100, 0, [customer_id#47 ASC NULLS FIRST, customer_first_name#48 ASC NULLS FIRST, customer_last_name#49 ASC NULLS FIRST, customer_preferred_cust_flag#50 ASC NULLS FIRST, customer_birth_country#51 ASC NULLS FIRST, customer_login#52 ASC NULLS FIRST, customer_email_address#53 ASC NULLS FIRST], [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53] - -(107) CometColumnarToRow [codegen id : 1] -Input [7]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 -BroadcastExchange (111) -+- * CometColumnarToRow (110) - +- CometFilter (109) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) - - -(108) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_year#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(109) CometFilter -Input [2]: [d_date_sk#22, d_year#23] -Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(110) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#22, d_year#23] - -(111) BroadcastExchange -Input [2]: [d_date_sk#22, d_year#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#41 IN dynamicpruning#42 -BroadcastExchange (115) -+- * CometColumnarToRow (114) - +- CometFilter (113) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) - - -(112) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_year#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(113) CometFilter -Input [2]: [d_date_sk#43, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2002)) AND isnotnull(d_date_sk#43)) - -(114) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#43, d_year#44] - -(115) BroadcastExchange -Input [2]: [d_date_sk#43, d_year#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -Subquery:3 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#21 - -Subquery:4 Hosting operator id = 57 Hosting Expression = cs_sold_date_sk#95 IN dynamicpruning#42 - -Subquery:5 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#122 IN dynamicpruning#21 - -Subquery:6 Hosting operator id = 92 Hosting Expression = ws_sold_date_sk#143 IN dynamicpruning#42 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/simplified.txt deleted file mode 100644 index 36be965b48..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,127 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - CometProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - CometBroadcastHashJoin [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total,customer_id,year_total] - CometProject [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] - CometBroadcastHashJoin [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,customer_id,year_total] - CometProject [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 - CometHashAggregate [ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total] #5 - CometHashAggregate [d_year,sum,isEmpty] [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #6 - CometHashAggregate [ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #16 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #17 - CometHashAggregate [ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #19 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #20 - CometHashAggregate [ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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/q4/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/explain.txt deleted file mode 100644 index cea1c1d430..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/explain.txt +++ /dev/null @@ -1,671 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (107) -+- CometTakeOrderedAndProject (106) - +- CometProject (105) - +- CometBroadcastHashJoin (104) - :- CometProject (88) - : +- CometBroadcastHashJoin (87) - : :- CometProject (70) - : : +- CometBroadcastHashJoin (69) - : : :- CometProject (53) - : : : +- CometBroadcastHashJoin (52) - : : : :- CometBroadcastHashJoin (35) - : : : : :- CometFilter (17) - : : : : : +- CometHashAggregate (16) - : : : : : +- CometExchange (15) - : : : : : +- CometHashAggregate (14) - : : : : : +- CometProject (13) - : : : : : +- CometBroadcastHashJoin (12) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometProject (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : : : +- CometBroadcastExchange (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : : : +- CometBroadcastExchange (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : : : +- CometBroadcastExchange (34) - : : : : +- CometHashAggregate (33) - : : : : +- CometExchange (32) - : : : : +- CometHashAggregate (31) - : : : : +- CometProject (30) - : : : : +- CometBroadcastHashJoin (29) - : : : : :- CometProject (25) - : : : : : +- CometBroadcastHashJoin (24) - : : : : : :- CometProject (20) - : : : : : : +- CometFilter (19) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : : : +- CometBroadcastExchange (23) - : : : : : +- CometFilter (22) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : : : +- CometBroadcastExchange (28) - : : : : +- CometFilter (27) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : : : +- CometBroadcastExchange (51) - : : : +- CometFilter (50) - : : : +- CometHashAggregate (49) - : : : +- CometExchange (48) - : : : +- CometHashAggregate (47) - : : : +- CometProject (46) - : : : +- CometBroadcastHashJoin (45) - : : : :- CometProject (43) - : : : : +- CometBroadcastHashJoin (42) - : : : : :- CometProject (38) - : : : : : +- CometFilter (37) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : : : +- CometBroadcastExchange (41) - : : : : +- CometFilter (40) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) - : : : +- ReusedExchange (44) - : : +- CometBroadcastExchange (68) - : : +- CometHashAggregate (67) - : : +- CometExchange (66) - : : +- CometHashAggregate (65) - : : +- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometProject (61) - : : : +- CometBroadcastHashJoin (60) - : : : :- CometProject (56) - : : : : +- CometFilter (55) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : : : +- CometBroadcastExchange (59) - : : : +- CometFilter (58) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (57) - : : +- ReusedExchange (62) - : +- CometBroadcastExchange (86) - : +- CometFilter (85) - : +- CometHashAggregate (84) - : +- CometExchange (83) - : +- CometHashAggregate (82) - : +- CometProject (81) - : +- CometBroadcastHashJoin (80) - : :- CometProject (78) - : : +- CometBroadcastHashJoin (77) - : : :- CometProject (73) - : : : +- CometFilter (72) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (71) - : : +- CometBroadcastExchange (76) - : : +- CometFilter (75) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (74) - : +- ReusedExchange (79) - +- CometBroadcastExchange (103) - +- CometHashAggregate (102) - +- CometExchange (101) - +- CometHashAggregate (100) - +- CometProject (99) - +- CometBroadcastHashJoin (98) - :- CometProject (96) - : +- CometBroadcastHashJoin (95) - : :- CometProject (91) - : : +- CometFilter (90) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (89) - : +- CometBroadcastExchange (94) - : +- CometFilter (93) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (92) - +- ReusedExchange (97) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#20), dynamicpruningexpression(ss_sold_date_sk#20 IN dynamicpruning#21)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Condition : isnotnull(ss_customer_sk#15) - -(6) CometBroadcastExchange -Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Arguments: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] - -(7) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Right output [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Arguments: [c_customer_sk#1], [ss_customer_sk#15], Inner, BuildRight - -(8) CometProject -Input [14]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_year#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#22, d_year#23] -Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#22, d_year#23] -Arguments: [d_date_sk#22, d_year#23] - -(12) CometBroadcastHashJoin -Left output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Right output [2]: [d_date_sk#22, d_year#23] -Arguments: [ss_sold_date_sk#20], [d_date_sk#22], Inner, BuildRight - -(13) CometProject -Input [14]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20, d_date_sk#22, d_year#23] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] - -(14) CometHashAggregate -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] -Functions [1]: [partial_sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] - -(15) CometExchange -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#24, isEmpty#25] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] -Functions [1]: [sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] - -(17) CometFilter -Input [2]: [customer_id#26, year_total#27] -Condition : (isnotnull(year_total#27) AND (year_total#27 > 0.000000)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] -Condition : (isnotnull(c_customer_sk#28) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#29, 16, true, false, true))) - -(20) CometProject -Input [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] -Arguments: [c_customer_sk#28, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14], [c_customer_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#29, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#30, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#31, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#32, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#34, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#35, 50, true, false, true) AS c_email_address#14] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#41), dynamicpruningexpression(ss_sold_date_sk#41 IN dynamicpruning#42)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Condition : isnotnull(ss_customer_sk#36) - -(23) CometBroadcastExchange -Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Arguments: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] - -(24) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#28, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14] -Right output [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Arguments: [c_customer_sk#28], [ss_customer_sk#36], Inner, BuildRight - -(25) CometProject -Input [14]: [c_customer_sk#28, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_year#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#43, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2002)) AND isnotnull(d_date_sk#43)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#43, d_year#44] -Arguments: [d_date_sk#43, d_year#44] - -(29) CometBroadcastHashJoin -Left output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Right output [2]: [d_date_sk#43, d_year#44] -Arguments: [ss_sold_date_sk#41], [d_date_sk#43], Inner, BuildRight - -(30) CometProject -Input [14]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41, d_date_sk#43, d_year#44] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, d_year#44], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, d_year#44] - -(31) CometHashAggregate -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, d_year#44] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, d_year#44] -Functions [1]: [partial_sum(((((ss_ext_list_price#40 - ss_ext_wholesale_cost#39) - ss_ext_discount_amt#37) + ss_ext_sales_price#38) / 2))] - -(32) CometExchange -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, d_year#44, sum#45, isEmpty#46] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, d_year#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, d_year#44, sum#45, isEmpty#46] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, d_year#44] -Functions [1]: [sum(((((ss_ext_list_price#40 - ss_ext_wholesale_cost#39) - ss_ext_discount_amt#37) + ss_ext_sales_price#38) / 2))] - -(34) CometBroadcastExchange -Input [8]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54] -Arguments: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#26, year_total#27] -Right output [8]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54] -Arguments: [customer_id#26], [customer_id#47], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [8]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62] -Condition : (isnotnull(c_customer_sk#55) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#56, 16, true, false, true))) - -(38) CometProject -Input [8]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62] -Arguments: [c_customer_sk#55, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68], [c_customer_sk#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#56, 16, true, false, true) AS c_customer_id#63, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#57, 20, true, false, true) AS c_first_name#64, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#58, 30, true, false, true) AS c_last_name#65, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#59, 1, true, false, true) AS c_preferred_cust_flag#66, c_birth_country#60, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#61, 13, true, false, true) AS c_login#67, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#62, 50, true, false, true) AS c_email_address#68] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_sold_date_sk#74 IN dynamicpruning#75)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [6]: [cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Condition : isnotnull(cs_bill_customer_sk#69) - -(41) CometBroadcastExchange -Input [6]: [cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Arguments: [cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] - -(42) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#55, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68] -Right output [6]: [cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Arguments: [c_customer_sk#55], [cs_bill_customer_sk#69], Inner, BuildRight - -(43) CometProject -Input [14]: [c_customer_sk#55, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Arguments: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74], [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#76, d_year#77] - -(45) CometBroadcastHashJoin -Left output [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Right output [2]: [d_date_sk#76, d_year#77] -Arguments: [cs_sold_date_sk#74], [d_date_sk#76], Inner, BuildRight - -(46) CometProject -Input [14]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74, d_date_sk#76, d_year#77] -Arguments: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, d_year#77], [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, d_year#77] - -(47) CometHashAggregate -Input [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, d_year#77] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#77] -Functions [1]: [partial_sum(((((cs_ext_list_price#73 - cs_ext_wholesale_cost#72) - cs_ext_discount_amt#70) + cs_ext_sales_price#71) / 2))] - -(48) CometExchange -Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#77, sum#78, isEmpty#79] -Arguments: hashpartitioning(c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#77, sum#78, isEmpty#79] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#77] -Functions [1]: [sum(((((cs_ext_list_price#73 - cs_ext_wholesale_cost#72) - cs_ext_discount_amt#70) + cs_ext_sales_price#71) / 2))] - -(50) CometFilter -Input [2]: [customer_id#80, year_total#81] -Condition : (isnotnull(year_total#81) AND (year_total#81 > 0.000000)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#80, year_total#81] -Arguments: [customer_id#80, year_total#81] - -(52) CometBroadcastHashJoin -Left output [10]: [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54] -Right output [2]: [customer_id#80, year_total#81] -Arguments: [customer_id#26], [customer_id#80], Inner, BuildRight - -(53) CometProject -Input [12]: [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54, customer_id#80, year_total#81] -Arguments: [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54, year_total#81], [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54, year_total#81] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] -Condition : (isnotnull(c_customer_sk#82) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#83, 16, true, false, true))) - -(56) CometProject -Input [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] -Arguments: [c_customer_sk#82, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68], [c_customer_sk#82, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#83, 16, true, false, true) AS c_customer_id#63, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#84, 20, true, false, true) AS c_first_name#64, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#85, 30, true, false, true) AS c_last_name#65, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#86, 1, true, false, true) AS c_preferred_cust_flag#66, c_birth_country#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#88, 13, true, false, true) AS c_login#67, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#89, 50, true, false, true) AS c_email_address#68] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#95), dynamicpruningexpression(cs_sold_date_sk#95 IN dynamicpruning#96)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Condition : isnotnull(cs_bill_customer_sk#90) - -(59) CometBroadcastExchange -Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Arguments: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] - -(60) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#82, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68] -Right output [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Arguments: [c_customer_sk#82], [cs_bill_customer_sk#90], Inner, BuildRight - -(61) CometProject -Input [14]: [c_customer_sk#82, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Arguments: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95], [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#97, d_year#98] - -(63) CometBroadcastHashJoin -Left output [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Right output [2]: [d_date_sk#97, d_year#98] -Arguments: [cs_sold_date_sk#95], [d_date_sk#97], Inner, BuildRight - -(64) CometProject -Input [14]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95, d_date_sk#97, d_year#98] -Arguments: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#98], [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#98] - -(65) CometHashAggregate -Input [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#98] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, d_year#98] -Functions [1]: [partial_sum(((((cs_ext_list_price#94 - cs_ext_wholesale_cost#93) - cs_ext_discount_amt#91) + cs_ext_sales_price#92) / 2))] - -(66) CometExchange -Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, d_year#98, sum#99, isEmpty#100] -Arguments: hashpartitioning(c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, d_year#98, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, d_year#98, sum#99, isEmpty#100] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, d_year#98] -Functions [1]: [sum(((((cs_ext_list_price#94 - cs_ext_wholesale_cost#93) - cs_ext_discount_amt#91) + cs_ext_sales_price#92) / 2))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#101, year_total#102] -Arguments: [customer_id#101, year_total#102] - -(69) CometBroadcastHashJoin -Left output [11]: [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54, year_total#81] -Right output [2]: [customer_id#101, year_total#102] -Arguments: [customer_id#26], [customer_id#101], Inner, (CASE WHEN (year_total#81 > 0.000000) THEN (year_total#102 / year_total#81) END > CASE WHEN (year_total#27 > 0.000000) THEN (year_total#54 / year_total#27) END), BuildRight - -(70) CometProject -Input [13]: [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54, year_total#81, customer_id#101, year_total#102] -Arguments: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102], [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102] - -(71) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#103, c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(72) CometFilter -Input [8]: [c_customer_sk#103, c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110] -Condition : (isnotnull(c_customer_sk#103) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#104, 16, true, false, true))) - -(73) CometProject -Input [8]: [c_customer_sk#103, c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110] -Arguments: [c_customer_sk#103, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116], [c_customer_sk#103, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#104, 16, true, false, true) AS c_customer_id#111, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#105, 20, true, false, true) AS c_first_name#112, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#106, 30, true, false, true) AS c_last_name#113, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#107, 1, true, false, true) AS c_preferred_cust_flag#114, c_birth_country#108, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#109, 13, true, false, true) AS c_login#115, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#110, 50, true, false, true) AS c_email_address#116] - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#122), dynamicpruningexpression(ws_sold_date_sk#122 IN dynamicpruning#123)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(75) CometFilter -Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Condition : isnotnull(ws_bill_customer_sk#117) - -(76) CometBroadcastExchange -Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Arguments: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] - -(77) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#103, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116] -Right output [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Arguments: [c_customer_sk#103], [ws_bill_customer_sk#117], Inner, BuildRight - -(78) CometProject -Input [14]: [c_customer_sk#103, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Arguments: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122], [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] - -(79) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#124, d_year#125] - -(80) CometBroadcastHashJoin -Left output [12]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Right output [2]: [d_date_sk#124, d_year#125] -Arguments: [ws_sold_date_sk#122], [d_date_sk#124], Inner, BuildRight - -(81) CometProject -Input [14]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122, d_date_sk#124, d_year#125] -Arguments: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#125], [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#125] - -(82) CometHashAggregate -Input [12]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#125] -Keys [8]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, d_year#125] -Functions [1]: [partial_sum(((((ws_ext_list_price#121 - ws_ext_wholesale_cost#120) - ws_ext_discount_amt#118) + ws_ext_sales_price#119) / 2))] - -(83) CometExchange -Input [10]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, d_year#125, sum#126, isEmpty#127] -Arguments: hashpartitioning(c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, d_year#125, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(84) CometHashAggregate -Input [10]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, d_year#125, sum#126, isEmpty#127] -Keys [8]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, d_year#125] -Functions [1]: [sum(((((ws_ext_list_price#121 - ws_ext_wholesale_cost#120) - ws_ext_discount_amt#118) + ws_ext_sales_price#119) / 2))] - -(85) CometFilter -Input [2]: [customer_id#128, year_total#129] -Condition : (isnotnull(year_total#129) AND (year_total#129 > 0.000000)) - -(86) CometBroadcastExchange -Input [2]: [customer_id#128, year_total#129] -Arguments: [customer_id#128, year_total#129] - -(87) CometBroadcastHashJoin -Left output [10]: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102] -Right output [2]: [customer_id#128, year_total#129] -Arguments: [customer_id#26], [customer_id#128], Inner, BuildRight - -(88) CometProject -Input [12]: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102, customer_id#128, year_total#129] -Arguments: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102, year_total#129], [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102, year_total#129] - -(89) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#130, c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(90) CometFilter -Input [8]: [c_customer_sk#130, c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137] -Condition : (isnotnull(c_customer_sk#130) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#131, 16, true, false, true))) - -(91) CometProject -Input [8]: [c_customer_sk#130, c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137] -Arguments: [c_customer_sk#130, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116], [c_customer_sk#130, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#131, 16, true, false, true) AS c_customer_id#111, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#132, 20, true, false, true) AS c_first_name#112, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#133, 30, true, false, true) AS c_last_name#113, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#134, 1, true, false, true) AS c_preferred_cust_flag#114, c_birth_country#135, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#136, 13, true, false, true) AS c_login#115, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#137, 50, true, false, true) AS c_email_address#116] - -(92) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#143), dynamicpruningexpression(ws_sold_date_sk#143 IN dynamicpruning#144)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(93) CometFilter -Input [6]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Condition : isnotnull(ws_bill_customer_sk#138) - -(94) CometBroadcastExchange -Input [6]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Arguments: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] - -(95) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#130, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116] -Right output [6]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Arguments: [c_customer_sk#130], [ws_bill_customer_sk#138], Inner, BuildRight - -(96) CometProject -Input [14]: [c_customer_sk#130, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Arguments: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143], [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] - -(97) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#145, d_year#146] - -(98) CometBroadcastHashJoin -Left output [12]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Right output [2]: [d_date_sk#145, d_year#146] -Arguments: [ws_sold_date_sk#143], [d_date_sk#145], Inner, BuildRight - -(99) CometProject -Input [14]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143, d_date_sk#145, d_year#146] -Arguments: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, d_year#146], [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, d_year#146] - -(100) CometHashAggregate -Input [12]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, d_year#146] -Keys [8]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, d_year#146] -Functions [1]: [partial_sum(((((ws_ext_list_price#142 - ws_ext_wholesale_cost#141) - ws_ext_discount_amt#139) + ws_ext_sales_price#140) / 2))] - -(101) CometExchange -Input [10]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, d_year#146, sum#147, isEmpty#148] -Arguments: hashpartitioning(c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, d_year#146, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(102) CometHashAggregate -Input [10]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, d_year#146, sum#147, isEmpty#148] -Keys [8]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, d_year#146] -Functions [1]: [sum(((((ws_ext_list_price#142 - ws_ext_wholesale_cost#141) - ws_ext_discount_amt#139) + ws_ext_sales_price#140) / 2))] - -(103) CometBroadcastExchange -Input [2]: [customer_id#149, year_total#150] -Arguments: [customer_id#149, year_total#150] - -(104) CometBroadcastHashJoin -Left output [11]: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102, year_total#129] -Right output [2]: [customer_id#149, year_total#150] -Arguments: [customer_id#26], [customer_id#149], Inner, (CASE WHEN (year_total#81 > 0.000000) THEN (year_total#102 / year_total#81) END > CASE WHEN (year_total#129 > 0.000000) THEN (year_total#150 / year_total#129) END), BuildRight - -(105) CometProject -Input [13]: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102, year_total#129, customer_id#149, year_total#150] -Arguments: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53], [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53] - -(106) CometTakeOrderedAndProject -Input [7]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#47 ASC NULLS FIRST,customer_first_name#48 ASC NULLS FIRST,customer_last_name#49 ASC NULLS FIRST,customer_preferred_cust_flag#50 ASC NULLS FIRST,customer_birth_country#51 ASC NULLS FIRST,customer_login#52 ASC NULLS FIRST,customer_email_address#53 ASC NULLS FIRST], output=[customer_id#47,customer_first_name#48,customer_last_name#49,customer_preferred_cust_flag#50,customer_birth_country#51,customer_login#52,customer_email_address#53]), [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53], 100, 0, [customer_id#47 ASC NULLS FIRST, customer_first_name#48 ASC NULLS FIRST, customer_last_name#49 ASC NULLS FIRST, customer_preferred_cust_flag#50 ASC NULLS FIRST, customer_birth_country#51 ASC NULLS FIRST, customer_login#52 ASC NULLS FIRST, customer_email_address#53 ASC NULLS FIRST], [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53] - -(107) CometColumnarToRow [codegen id : 1] -Input [7]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 -BroadcastExchange (111) -+- * CometColumnarToRow (110) - +- CometFilter (109) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) - - -(108) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_year#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(109) CometFilter -Input [2]: [d_date_sk#22, d_year#23] -Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(110) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#22, d_year#23] - -(111) BroadcastExchange -Input [2]: [d_date_sk#22, d_year#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#41 IN dynamicpruning#42 -BroadcastExchange (115) -+- * CometColumnarToRow (114) - +- CometFilter (113) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) - - -(112) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_year#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(113) CometFilter -Input [2]: [d_date_sk#43, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2002)) AND isnotnull(d_date_sk#43)) - -(114) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#43, d_year#44] - -(115) BroadcastExchange -Input [2]: [d_date_sk#43, d_year#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -Subquery:3 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#21 - -Subquery:4 Hosting operator id = 57 Hosting Expression = cs_sold_date_sk#95 IN dynamicpruning#42 - -Subquery:5 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#122 IN dynamicpruning#21 - -Subquery:6 Hosting operator id = 92 Hosting Expression = ws_sold_date_sk#143 IN dynamicpruning#42 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/extended.txt deleted file mode 100644 index 1be0816ee1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/extended.txt +++ /dev/null @@ -1,131 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 118 out of 126 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 36be965b48..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4/simplified.txt +++ /dev/null @@ -1,127 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - CometProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - CometBroadcastHashJoin [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total,customer_id,year_total] - CometProject [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] - CometBroadcastHashJoin [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,customer_id,year_total] - CometProject [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 - CometHashAggregate [ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total] #5 - CometHashAggregate [d_year,sum,isEmpty] [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #6 - CometHashAggregate [ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #16 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #17 - CometHashAggregate [ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #19 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #20 - CometHashAggregate [ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/explain.txt deleted file mode 100644 index fe074b9df2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/explain.txt +++ /dev/null @@ -1,221 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (34) -+- CometTakeOrderedAndProject (33) - +- CometHashAggregate (32) - +- CometExchange (31) - +- CometHashAggregate (30) - +- CometProject (29) - +- CometBroadcastHashJoin (28) - :- CometProject (24) - : +- CometBroadcastHashJoin (23) - : :- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (12) - : : : +- CometSortMergeJoin (11) - : : : :- CometSort (5) - : : : : +- CometColumnarExchange (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : +- CometSort (10) - : : : +- CometExchange (9) - : : : +- CometProject (8) - : : : +- CometFilter (7) - : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (6) - : : +- CometBroadcastExchange (16) - : : +- CometProject (15) - : : +- CometFilter (14) - : : +- CometNativeScan parquet spark_catalog.default.warehouse (13) - : +- CometBroadcastExchange (22) - : +- CometProject (21) - : +- CometFilter (20) - : +- CometNativeScan parquet spark_catalog.default.item (19) - +- CometBroadcastExchange (27) - +- CometFilter (26) - +- CometNativeScan parquet spark_catalog.default.date_dim (25) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] - -(3) Filter [codegen id : 1] -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Condition : (isnotnull(cs_warehouse_sk#1) AND isnotnull(cs_item_sk#2)) - -(4) CometColumnarExchange -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_order_number#3, cs_item_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5], [cs_order_number#3 ASC NULLS FIRST, cs_item_sk#2 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Condition : (isnotnull(cr_order_number#8) AND isnotnull(cr_item_sk#7)) - -(8) CometProject -Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] - -(9) CometExchange -Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: hashpartitioning(cr_order_number#8, cr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_order_number#8 ASC NULLS FIRST, cr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Right output [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cs_order_number#3, cs_item_sk#2], [cr_order_number#8, cr_item_sk#7], LeftOuter - -(12) CometProject -Input [8]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5, cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9], [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] - -(13) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#11, w_state#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [w_warehouse_sk#11, w_state#12] -Condition : isnotnull(w_warehouse_sk#11) - -(15) CometProject -Input [2]: [w_warehouse_sk#11, w_state#12] -Arguments: [w_warehouse_sk#11, w_state#13], [w_warehouse_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, w_state#12, 2, true, false, true) AS w_state#13] - -(16) CometBroadcastExchange -Input [2]: [w_warehouse_sk#11, w_state#13] -Arguments: [w_warehouse_sk#11, w_state#13] - -(17) CometBroadcastHashJoin -Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] -Right output [2]: [w_warehouse_sk#11, w_state#13] -Arguments: [cs_warehouse_sk#1], [w_warehouse_sk#11], Inner, BuildRight - -(18) CometProject -Input [7]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_warehouse_sk#11, w_state#13] -Arguments: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13], [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13] - -(19) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Condition : (((isnotnull(i_current_price#16) AND (i_current_price#16 >= 0.99)) AND (i_current_price#16 <= 1.49)) AND isnotnull(i_item_sk#14)) - -(21) CometProject -Input [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Arguments: [i_item_sk#14, i_item_id#17], [i_item_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#15, 16, true, false, true) AS i_item_id#17] - -(22) CometBroadcastExchange -Input [2]: [i_item_sk#14, i_item_id#17] -Arguments: [i_item_sk#14, i_item_id#17] - -(23) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13] -Right output [2]: [i_item_sk#14, i_item_id#17] -Arguments: [cs_item_sk#2], [i_item_sk#14], Inner, BuildRight - -(24) CometProject -Input [7]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_sk#14, i_item_id#17] -Arguments: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17], [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17] - -(25) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#18, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) - -(27) CometBroadcastExchange -Input [2]: [d_date_sk#18, d_date#19] -Arguments: [d_date_sk#18, d_date#19] - -(28) CometBroadcastHashJoin -Left output [5]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17] -Right output [2]: [d_date_sk#18, d_date#19] -Arguments: [cs_sold_date_sk#5], [d_date_sk#18], Inner, BuildRight - -(29) CometProject -Input [7]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date_sk#18, d_date#19] -Arguments: [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19], [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19] - -(30) CometHashAggregate -Input [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19] -Keys [2]: [w_state#13, i_item_id#17] -Functions [2]: [partial_sum(CASE WHEN (d_date#19 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), partial_sum(CASE WHEN (d_date#19 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] - -(31) CometExchange -Input [6]: [w_state#13, i_item_id#17, sum#20, isEmpty#21, sum#22, isEmpty#23] -Arguments: hashpartitioning(w_state#13, i_item_id#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometHashAggregate -Input [6]: [w_state#13, i_item_id#17, sum#20, isEmpty#21, sum#22, isEmpty#23] -Keys [2]: [w_state#13, i_item_id#17] -Functions [2]: [sum(CASE WHEN (d_date#19 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), sum(CASE WHEN (d_date#19 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] - -(33) CometTakeOrderedAndProject -Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_state#13 ASC NULLS FIRST,i_item_id#17 ASC NULLS FIRST], output=[w_state#13,i_item_id#17,sales_before#24,sales_after#25]), [w_state#13, i_item_id#17, sales_before#24, sales_after#25], 100, 0, [w_state#13 ASC NULLS FIRST, i_item_id#17 ASC NULLS FIRST], [w_state#13, i_item_id#17, sales_before#24, sales_after#25] - -(34) CometColumnarToRow [codegen id : 2] -Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometFilter (36) - +- CometNativeScan parquet spark_catalog.default.date_dim (35) - - -(35) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(36) CometFilter -Input [2]: [d_date_sk#18, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) - -(37) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#18, d_date#19] - -(38) BroadcastExchange -Input [2]: [d_date_sk#18, d_date#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/simplified.txt deleted file mode 100644 index 13760fcb0b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] - CometHashAggregate [sum,isEmpty,sum,isEmpty] [w_state,i_item_id,sales_before,sales_after,sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END)] - CometExchange [w_state,i_item_id] #1 - CometHashAggregate [d_date,cs_sales_price,cr_refunded_cash] [w_state,i_item_id,sum,isEmpty,sum,isEmpty] - CometProject [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id,d_date_sk,d_date] - CometProject [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] - CometBroadcastHashJoin [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_warehouse_sk,w_state] - CometProject [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] - CometSortMergeJoin [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_refunded_cash] - CometSort [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - CometColumnarExchange [cs_order_number,cs_item_sk] #2 - WholeStageCodegen (1) - Filter [cs_warehouse_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] - CometExchange [cr_order_number,cr_item_sk] #4 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_state] #5 - CometProject [w_state] [w_warehouse_sk,w_state] - CometFilter [w_warehouse_sk,w_state] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometFilter [d_date_sk,d_date] - CometNativeScan 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/q40.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/explain.txt deleted file mode 100644 index 00e2b8fbc1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometTakeOrderedAndProject (32) - +- CometHashAggregate (31) - +- CometExchange (30) - +- CometHashAggregate (29) - +- CometProject (28) - +- CometBroadcastHashJoin (27) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometProject (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (5) - : : +- CometBroadcastExchange (15) - : : +- CometProject (14) - : : +- CometFilter (13) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (12) - : +- CometBroadcastExchange (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (18) - +- CometBroadcastExchange (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Condition : (isnotnull(cs_warehouse_sk#1) AND isnotnull(cs_item_sk#2)) - -(3) CometExchange -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_order_number#3, cs_item_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5], [cs_order_number#3 ASC NULLS FIRST, cs_item_sk#2 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Condition : (isnotnull(cr_order_number#8) AND isnotnull(cr_item_sk#7)) - -(7) CometProject -Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] - -(8) CometExchange -Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: hashpartitioning(cr_order_number#8, cr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_order_number#8 ASC NULLS FIRST, cr_item_sk#7 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Right output [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cs_order_number#3, cs_item_sk#2], [cr_order_number#8, cr_item_sk#7], LeftOuter - -(11) CometProject -Input [8]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5, cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9], [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#11, w_state#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [w_warehouse_sk#11, w_state#12] -Condition : isnotnull(w_warehouse_sk#11) - -(14) CometProject -Input [2]: [w_warehouse_sk#11, w_state#12] -Arguments: [w_warehouse_sk#11, w_state#13], [w_warehouse_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, w_state#12, 2, true, false, true) AS w_state#13] - -(15) CometBroadcastExchange -Input [2]: [w_warehouse_sk#11, w_state#13] -Arguments: [w_warehouse_sk#11, w_state#13] - -(16) CometBroadcastHashJoin -Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] -Right output [2]: [w_warehouse_sk#11, w_state#13] -Arguments: [cs_warehouse_sk#1], [w_warehouse_sk#11], Inner, BuildRight - -(17) CometProject -Input [7]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_warehouse_sk#11, w_state#13] -Arguments: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13], [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] -ReadSchema: struct - -(19) CometFilter -Input [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Condition : (((isnotnull(i_current_price#16) AND (i_current_price#16 >= 0.99)) AND (i_current_price#16 <= 1.49)) AND isnotnull(i_item_sk#14)) - -(20) CometProject -Input [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Arguments: [i_item_sk#14, i_item_id#17], [i_item_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#15, 16, true, false, true) AS i_item_id#17] - -(21) CometBroadcastExchange -Input [2]: [i_item_sk#14, i_item_id#17] -Arguments: [i_item_sk#14, i_item_id#17] - -(22) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13] -Right output [2]: [i_item_sk#14, i_item_id#17] -Arguments: [cs_item_sk#2], [i_item_sk#14], Inner, BuildRight - -(23) CometProject -Input [7]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_sk#14, i_item_id#17] -Arguments: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17], [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#18, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) - -(26) CometBroadcastExchange -Input [2]: [d_date_sk#18, d_date#19] -Arguments: [d_date_sk#18, d_date#19] - -(27) CometBroadcastHashJoin -Left output [5]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17] -Right output [2]: [d_date_sk#18, d_date#19] -Arguments: [cs_sold_date_sk#5], [d_date_sk#18], Inner, BuildRight - -(28) CometProject -Input [7]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date_sk#18, d_date#19] -Arguments: [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19], [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19] - -(29) CometHashAggregate -Input [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19] -Keys [2]: [w_state#13, i_item_id#17] -Functions [2]: [partial_sum(CASE WHEN (d_date#19 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), partial_sum(CASE WHEN (d_date#19 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] - -(30) CometExchange -Input [6]: [w_state#13, i_item_id#17, sum#20, isEmpty#21, sum#22, isEmpty#23] -Arguments: hashpartitioning(w_state#13, i_item_id#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(31) CometHashAggregate -Input [6]: [w_state#13, i_item_id#17, sum#20, isEmpty#21, sum#22, isEmpty#23] -Keys [2]: [w_state#13, i_item_id#17] -Functions [2]: [sum(CASE WHEN (d_date#19 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), sum(CASE WHEN (d_date#19 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] - -(32) CometTakeOrderedAndProject -Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_state#13 ASC NULLS FIRST,i_item_id#17 ASC NULLS FIRST], output=[w_state#13,i_item_id#17,sales_before#24,sales_after#25]), [w_state#13, i_item_id#17, sales_before#24, sales_after#25], 100, 0, [w_state#13 ASC NULLS FIRST, i_item_id#17 ASC NULLS FIRST], [w_state#13, i_item_id#17, sales_before#24, sales_after#25] - -(33) CometColumnarToRow [codegen id : 1] -Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [d_date_sk#18, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) - -(36) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#18, d_date#19] - -(37) BroadcastExchange -Input [2]: [d_date_sk#18, d_date#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/simplified.txt deleted file mode 100644 index 79e3556750..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,42 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] - CometHashAggregate [sum,isEmpty,sum,isEmpty] [w_state,i_item_id,sales_before,sales_after,sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END)] - CometExchange [w_state,i_item_id] #1 - CometHashAggregate [d_date,cs_sales_price,cr_refunded_cash] [w_state,i_item_id,sum,isEmpty,sum,isEmpty] - CometProject [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id,d_date_sk,d_date] - CometProject [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] - CometBroadcastHashJoin [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_warehouse_sk,w_state] - CometProject [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] - CometSortMergeJoin [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_refunded_cash] - CometSort [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - CometExchange [cs_order_number,cs_item_sk] #2 - CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] - CometExchange [cr_order_number,cr_item_sk] #4 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_state] #5 - CometProject [w_state] [w_warehouse_sk,w_state] - CometFilter [w_warehouse_sk,w_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q40/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/explain.txt deleted file mode 100644 index 00e2b8fbc1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometTakeOrderedAndProject (32) - +- CometHashAggregate (31) - +- CometExchange (30) - +- CometHashAggregate (29) - +- CometProject (28) - +- CometBroadcastHashJoin (27) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometProject (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (5) - : : +- CometBroadcastExchange (15) - : : +- CometProject (14) - : : +- CometFilter (13) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (12) - : +- CometBroadcastExchange (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (18) - +- CometBroadcastExchange (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Condition : (isnotnull(cs_warehouse_sk#1) AND isnotnull(cs_item_sk#2)) - -(3) CometExchange -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_order_number#3, cs_item_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5], [cs_order_number#3 ASC NULLS FIRST, cs_item_sk#2 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Condition : (isnotnull(cr_order_number#8) AND isnotnull(cr_item_sk#7)) - -(7) CometProject -Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] - -(8) CometExchange -Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: hashpartitioning(cr_order_number#8, cr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_order_number#8 ASC NULLS FIRST, cr_item_sk#7 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Right output [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cs_order_number#3, cs_item_sk#2], [cr_order_number#8, cr_item_sk#7], LeftOuter - -(11) CometProject -Input [8]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5, cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9], [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#11, w_state#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [w_warehouse_sk#11, w_state#12] -Condition : isnotnull(w_warehouse_sk#11) - -(14) CometProject -Input [2]: [w_warehouse_sk#11, w_state#12] -Arguments: [w_warehouse_sk#11, w_state#13], [w_warehouse_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, w_state#12, 2, true, false, true) AS w_state#13] - -(15) CometBroadcastExchange -Input [2]: [w_warehouse_sk#11, w_state#13] -Arguments: [w_warehouse_sk#11, w_state#13] - -(16) CometBroadcastHashJoin -Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] -Right output [2]: [w_warehouse_sk#11, w_state#13] -Arguments: [cs_warehouse_sk#1], [w_warehouse_sk#11], Inner, BuildRight - -(17) CometProject -Input [7]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_warehouse_sk#11, w_state#13] -Arguments: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13], [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] -ReadSchema: struct - -(19) CometFilter -Input [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Condition : (((isnotnull(i_current_price#16) AND (i_current_price#16 >= 0.99)) AND (i_current_price#16 <= 1.49)) AND isnotnull(i_item_sk#14)) - -(20) CometProject -Input [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Arguments: [i_item_sk#14, i_item_id#17], [i_item_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#15, 16, true, false, true) AS i_item_id#17] - -(21) CometBroadcastExchange -Input [2]: [i_item_sk#14, i_item_id#17] -Arguments: [i_item_sk#14, i_item_id#17] - -(22) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13] -Right output [2]: [i_item_sk#14, i_item_id#17] -Arguments: [cs_item_sk#2], [i_item_sk#14], Inner, BuildRight - -(23) CometProject -Input [7]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_sk#14, i_item_id#17] -Arguments: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17], [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#18, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) - -(26) CometBroadcastExchange -Input [2]: [d_date_sk#18, d_date#19] -Arguments: [d_date_sk#18, d_date#19] - -(27) CometBroadcastHashJoin -Left output [5]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17] -Right output [2]: [d_date_sk#18, d_date#19] -Arguments: [cs_sold_date_sk#5], [d_date_sk#18], Inner, BuildRight - -(28) CometProject -Input [7]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date_sk#18, d_date#19] -Arguments: [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19], [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19] - -(29) CometHashAggregate -Input [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19] -Keys [2]: [w_state#13, i_item_id#17] -Functions [2]: [partial_sum(CASE WHEN (d_date#19 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), partial_sum(CASE WHEN (d_date#19 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] - -(30) CometExchange -Input [6]: [w_state#13, i_item_id#17, sum#20, isEmpty#21, sum#22, isEmpty#23] -Arguments: hashpartitioning(w_state#13, i_item_id#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(31) CometHashAggregate -Input [6]: [w_state#13, i_item_id#17, sum#20, isEmpty#21, sum#22, isEmpty#23] -Keys [2]: [w_state#13, i_item_id#17] -Functions [2]: [sum(CASE WHEN (d_date#19 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), sum(CASE WHEN (d_date#19 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] - -(32) CometTakeOrderedAndProject -Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_state#13 ASC NULLS FIRST,i_item_id#17 ASC NULLS FIRST], output=[w_state#13,i_item_id#17,sales_before#24,sales_after#25]), [w_state#13, i_item_id#17, sales_before#24, sales_after#25], 100, 0, [w_state#13 ASC NULLS FIRST, i_item_id#17 ASC NULLS FIRST], [w_state#13, i_item_id#17, sales_before#24, sales_after#25] - -(33) CometColumnarToRow [codegen id : 1] -Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [d_date_sk#18, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) - -(36) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#18, d_date#19] - -(37) BroadcastExchange -Input [2]: [d_date_sk#18, d_date#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/extended.txt deleted file mode 100644 index 54a7b21486..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/extended.txt +++ /dev/null @@ -1,40 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 79e3556750..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40/simplified.txt +++ /dev/null @@ -1,42 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] - CometHashAggregate [sum,isEmpty,sum,isEmpty] [w_state,i_item_id,sales_before,sales_after,sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END)] - CometExchange [w_state,i_item_id] #1 - CometHashAggregate [d_date,cs_sales_price,cr_refunded_cash] [w_state,i_item_id,sum,isEmpty,sum,isEmpty] - CometProject [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id,d_date_sk,d_date] - CometProject [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] - CometBroadcastHashJoin [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_warehouse_sk,w_state] - CometProject [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] - CometSortMergeJoin [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_refunded_cash] - CometSort [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - CometExchange [cs_order_number,cs_item_sk] #2 - CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] - CometExchange [cr_order_number,cr_item_sk] #4 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_state] #5 - CometProject [w_state] [w_warehouse_sk,w_state] - CometFilter [w_warehouse_sk,w_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q41.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_datafusion/explain.txt deleted file mode 100644 index 625df9cd57..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_datafusion/explain.txt +++ /dev/null @@ -1,107 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (3) - : +- CometFilter (2) - : +- CometNativeScan parquet spark_catalog.default.item (1) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometHashAggregate (9) - +- CometExchange (8) - +- CometHashAggregate (7) - +- CometProject (6) - +- CometFilter (5) - +- CometNativeScan parquet spark_catalog.default.item (4) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), GreaterThanOrEqual(i_manufact_id,738), LessThanOrEqual(i_manufact_id,778)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Condition : ((isnotnull(i_manufact_id#1) AND (i_manufact_id#1 >= 738)) AND (i_manufact_id#1 <= 778)) - -(3) CometProject -Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Arguments: [i_manufact#2, i_product_name#3], [i_manufact#2, i_product_name#3] - -(4) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(5) CometFilter -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] -Condition : (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Women ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = powder ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = khaki )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Ounce ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Oz ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = extra large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = brown ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = honeydew )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Bunch ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Ton ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = N/A ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = small ))))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Men ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = floral ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = deep )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = N/A ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Dozen ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = petite ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = light ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = cornflower )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Box ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Pound ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = extra large )))))) OR (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Women ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = midnight ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = snow )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Pallet ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Gross ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = extra large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = cyan ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = papaya )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Cup ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Dram ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = N/A ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = small ))))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Men ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = orange ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = frosted )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Each ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Tbl ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = petite ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = forest ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = ghost )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Lb ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Bundle ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = extra large ))))))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#5, 50, true, false, true))) - -(6) CometProject -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] -Arguments: [i_manufact#9], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#5, 50, true, false, true) AS i_manufact#9] - -(7) CometHashAggregate -Input [1]: [i_manufact#9] -Keys [1]: [i_manufact#9] -Functions [1]: [partial_count(1)] - -(8) CometExchange -Input [2]: [i_manufact#9, count#10] -Arguments: hashpartitioning(i_manufact#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(9) CometHashAggregate -Input [2]: [i_manufact#9, count#10] -Keys [1]: [i_manufact#9] -Functions [1]: [count(1)] - -(10) CometFilter -Input [2]: [item_cnt#11, i_manufact#9] -Condition : (item_cnt#11 > 0) - -(11) CometProject -Input [2]: [item_cnt#11, i_manufact#9] -Arguments: [i_manufact#9], [i_manufact#9] - -(12) CometBroadcastExchange -Input [1]: [i_manufact#9] -Arguments: [i_manufact#9] - -(13) CometBroadcastHashJoin -Left output [2]: [i_manufact#2, i_product_name#3] -Right output [1]: [i_manufact#9] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#2, 50, true, false, true)], [i_manufact#9], Inner, BuildRight - -(14) CometProject -Input [3]: [i_manufact#2, i_product_name#3, i_manufact#9] -Arguments: [i_product_name#12], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#3, 50, true, false, true) AS i_product_name#12] - -(15) CometHashAggregate -Input [1]: [i_product_name#12] -Keys [1]: [i_product_name#12] -Functions: [] - -(16) CometExchange -Input [1]: [i_product_name#12] -Arguments: hashpartitioning(i_product_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(17) CometHashAggregate -Input [1]: [i_product_name#12] -Keys [1]: [i_product_name#12] -Functions: [] - -(18) CometTakeOrderedAndProject -Input [1]: [i_product_name#12] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#12 ASC NULLS FIRST], output=[i_product_name#12]), [i_product_name#12], 100, 0, [i_product_name#12 ASC NULLS FIRST], [i_product_name#12] - -(19) CometColumnarToRow [codegen id : 1] -Input [1]: [i_product_name#12] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_datafusion/simplified.txt deleted file mode 100644 index 2557f51684..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_datafusion/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name] - CometHashAggregate [i_product_name] - CometExchange [i_product_name] #1 - CometHashAggregate [i_product_name] - CometProject [i_product_name] [i_product_name] - CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - CometBroadcastExchange [i_manufact] #2 - CometProject [i_manufact] - CometFilter [item_cnt,i_manufact] - CometHashAggregate [count] [item_cnt,i_manufact,count(1)] - CometExchange [i_manufact] #3 - CometHashAggregate [i_manufact,count] - CometProject [i_manufact] [i_manufact] - CometFilter [i_category,i_manufact,i_size,i_color,i_units] - CometNativeScan 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/q41.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_iceberg_compat/explain.txt deleted file mode 100644 index 568da58fa0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,107 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (3) - : +- CometFilter (2) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometHashAggregate (9) - +- CometExchange (8) - +- CometHashAggregate (7) - +- CometProject (6) - +- CometFilter (5) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), GreaterThanOrEqual(i_manufact_id,738), LessThanOrEqual(i_manufact_id,778)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Condition : ((isnotnull(i_manufact_id#1) AND (i_manufact_id#1 >= 738)) AND (i_manufact_id#1 <= 778)) - -(3) CometProject -Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Arguments: [i_manufact#2, i_product_name#3], [i_manufact#2, i_product_name#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(5) CometFilter -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] -Condition : (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Women ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = powder ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = khaki )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Ounce ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Oz ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = extra large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = brown ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = honeydew )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Bunch ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Ton ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = N/A ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = small ))))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Men ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = floral ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = deep )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = N/A ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Dozen ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = petite ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = light ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = cornflower )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Box ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Pound ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = extra large )))))) OR (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Women ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = midnight ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = snow )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Pallet ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Gross ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = extra large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = cyan ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = papaya )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Cup ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Dram ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = N/A ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = small ))))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Men ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = orange ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = frosted )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Each ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Tbl ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = petite ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = forest ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = ghost )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Lb ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Bundle ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = extra large ))))))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#5, 50, true, false, true))) - -(6) CometProject -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] -Arguments: [i_manufact#9], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#5, 50, true, false, true) AS i_manufact#9] - -(7) CometHashAggregate -Input [1]: [i_manufact#9] -Keys [1]: [i_manufact#9] -Functions [1]: [partial_count(1)] - -(8) CometExchange -Input [2]: [i_manufact#9, count#10] -Arguments: hashpartitioning(i_manufact#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(9) CometHashAggregate -Input [2]: [i_manufact#9, count#10] -Keys [1]: [i_manufact#9] -Functions [1]: [count(1)] - -(10) CometFilter -Input [2]: [item_cnt#11, i_manufact#9] -Condition : (item_cnt#11 > 0) - -(11) CometProject -Input [2]: [item_cnt#11, i_manufact#9] -Arguments: [i_manufact#9], [i_manufact#9] - -(12) CometBroadcastExchange -Input [1]: [i_manufact#9] -Arguments: [i_manufact#9] - -(13) CometBroadcastHashJoin -Left output [2]: [i_manufact#2, i_product_name#3] -Right output [1]: [i_manufact#9] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#2, 50, true, false, true)], [i_manufact#9], Inner, BuildRight - -(14) CometProject -Input [3]: [i_manufact#2, i_product_name#3, i_manufact#9] -Arguments: [i_product_name#12], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#3, 50, true, false, true) AS i_product_name#12] - -(15) CometHashAggregate -Input [1]: [i_product_name#12] -Keys [1]: [i_product_name#12] -Functions: [] - -(16) CometExchange -Input [1]: [i_product_name#12] -Arguments: hashpartitioning(i_product_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(17) CometHashAggregate -Input [1]: [i_product_name#12] -Keys [1]: [i_product_name#12] -Functions: [] - -(18) CometTakeOrderedAndProject -Input [1]: [i_product_name#12] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#12 ASC NULLS FIRST], output=[i_product_name#12]), [i_product_name#12], 100, 0, [i_product_name#12 ASC NULLS FIRST], [i_product_name#12] - -(19) CometColumnarToRow [codegen id : 1] -Input [1]: [i_product_name#12] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_iceberg_compat/simplified.txt deleted file mode 100644 index 99c935aaa4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name] - CometHashAggregate [i_product_name] - CometExchange [i_product_name] #1 - CometHashAggregate [i_product_name] - CometProject [i_product_name] [i_product_name] - CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - CometBroadcastExchange [i_manufact] #2 - CometProject [i_manufact] - CometFilter [item_cnt,i_manufact] - CometHashAggregate [count] [item_cnt,i_manufact,count(1)] - CometExchange [i_manufact] #3 - CometHashAggregate [i_manufact,count] - CometProject [i_manufact] [i_manufact] - CometFilter [i_category,i_manufact,i_size,i_color,i_units] - CometScan [native_iceberg_compat] 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/q41/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/explain.txt deleted file mode 100644 index 568da58fa0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/explain.txt +++ /dev/null @@ -1,107 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (3) - : +- CometFilter (2) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometHashAggregate (9) - +- CometExchange (8) - +- CometHashAggregate (7) - +- CometProject (6) - +- CometFilter (5) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), GreaterThanOrEqual(i_manufact_id,738), LessThanOrEqual(i_manufact_id,778)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Condition : ((isnotnull(i_manufact_id#1) AND (i_manufact_id#1 >= 738)) AND (i_manufact_id#1 <= 778)) - -(3) CometProject -Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Arguments: [i_manufact#2, i_product_name#3], [i_manufact#2, i_product_name#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(5) CometFilter -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] -Condition : (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Women ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = powder ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = khaki )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Ounce ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Oz ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = extra large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = brown ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = honeydew )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Bunch ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Ton ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = N/A ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = small ))))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Men ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = floral ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = deep )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = N/A ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Dozen ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = petite ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = light ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = cornflower )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Box ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Pound ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = extra large )))))) OR (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Women ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = midnight ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = snow )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Pallet ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Gross ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = extra large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = cyan ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = papaya )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Cup ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Dram ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = N/A ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = small ))))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Men ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = orange ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = frosted )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Each ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Tbl ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = petite ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = forest ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#7, 20, true, false, true) = ghost )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Lb ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#8, 10, true, false, true) = Bundle ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#6, 20, true, false, true) = extra large ))))))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#5, 50, true, false, true))) - -(6) CometProject -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] -Arguments: [i_manufact#9], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#5, 50, true, false, true) AS i_manufact#9] - -(7) CometHashAggregate -Input [1]: [i_manufact#9] -Keys [1]: [i_manufact#9] -Functions [1]: [partial_count(1)] - -(8) CometExchange -Input [2]: [i_manufact#9, count#10] -Arguments: hashpartitioning(i_manufact#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(9) CometHashAggregate -Input [2]: [i_manufact#9, count#10] -Keys [1]: [i_manufact#9] -Functions [1]: [count(1)] - -(10) CometFilter -Input [2]: [item_cnt#11, i_manufact#9] -Condition : (item_cnt#11 > 0) - -(11) CometProject -Input [2]: [item_cnt#11, i_manufact#9] -Arguments: [i_manufact#9], [i_manufact#9] - -(12) CometBroadcastExchange -Input [1]: [i_manufact#9] -Arguments: [i_manufact#9] - -(13) CometBroadcastHashJoin -Left output [2]: [i_manufact#2, i_product_name#3] -Right output [1]: [i_manufact#9] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#2, 50, true, false, true)], [i_manufact#9], Inner, BuildRight - -(14) CometProject -Input [3]: [i_manufact#2, i_product_name#3, i_manufact#9] -Arguments: [i_product_name#12], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#3, 50, true, false, true) AS i_product_name#12] - -(15) CometHashAggregate -Input [1]: [i_product_name#12] -Keys [1]: [i_product_name#12] -Functions: [] - -(16) CometExchange -Input [1]: [i_product_name#12] -Arguments: hashpartitioning(i_product_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(17) CometHashAggregate -Input [1]: [i_product_name#12] -Keys [1]: [i_product_name#12] -Functions: [] - -(18) CometTakeOrderedAndProject -Input [1]: [i_product_name#12] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#12 ASC NULLS FIRST], output=[i_product_name#12]), [i_product_name#12], 100, 0, [i_product_name#12 ASC NULLS FIRST], [i_product_name#12] - -(19) CometColumnarToRow [codegen id : 1] -Input [1]: [i_product_name#12] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/extended.txt deleted file mode 100644 index 9d802b5033..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 99c935aaa4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q41/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name] - CometHashAggregate [i_product_name] - CometExchange [i_product_name] #1 - CometHashAggregate [i_product_name] - CometProject [i_product_name] [i_product_name] - CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - CometBroadcastExchange [i_manufact] #2 - CometProject [i_manufact] - CometFilter [item_cnt,i_manufact] - CometHashAggregate [count] [item_cnt,i_manufact,count(1)] - CometExchange [i_manufact] #3 - CometHashAggregate [i_manufact,count] - CometProject [i_manufact] [i_manufact] - CometFilter [i_category,i_manufact,i_size,i_color,i_units] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_datafusion/explain.txt deleted file mode 100644 index af01446009..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_datafusion/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometNativeScan parquet spark_catalog.default.item (9) - - -(1) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometNativeScan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_category_id#8, i_category#11], [i_item_sk#7, i_category_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [i_item_sk#7, i_category_id#8, i_category#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11], [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11] -Keys [3]: [d_year#2, i_category_id#8, i_category#11] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] -Arguments: hashpartitioning(d_year#2, i_category_id#8, i_category#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] -Keys [3]: [d_year#2, i_category_id#8, i_category#11] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sum(ss_ext_sales_price)#13 DESC NULLS LAST,d_year#2 ASC NULLS FIRST,i_category_id#8 ASC NULLS FIRST,i_category#11 ASC NULLS FIRST], output=[d_year#2,i_category_id#8,i_category#11,sum(ss_ext_sales_price)#13]), [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13], 100, 0, [sum(ss_ext_sales_price)#13 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_datafusion/simplified.txt deleted file mode 100644 index 96ab34454d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_datafusion/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,i_category_id,i_category,sum(ss_ext_sales_price)] - CometHashAggregate [sum] [d_year,i_category_id,i_category,sum(ss_ext_sales_price),sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_category_id,i_category] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_category_id,i_category,sum] - CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 - CometProject [i_category] [i_item_sk,i_category_id,i_category] - CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] - CometNativeScan 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/q42.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_iceberg_compat/explain.txt deleted file mode 100644 index 183fffdf00..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_category_id#8, i_category#11], [i_item_sk#7, i_category_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [i_item_sk#7, i_category_id#8, i_category#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11], [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11] -Keys [3]: [d_year#2, i_category_id#8, i_category#11] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] -Arguments: hashpartitioning(d_year#2, i_category_id#8, i_category#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] -Keys [3]: [d_year#2, i_category_id#8, i_category#11] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sum(ss_ext_sales_price)#13 DESC NULLS LAST,d_year#2 ASC NULLS FIRST,i_category_id#8 ASC NULLS FIRST,i_category#11 ASC NULLS FIRST], output=[d_year#2,i_category_id#8,i_category#11,sum(ss_ext_sales_price)#13]), [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13], 100, 0, [sum(ss_ext_sales_price)#13 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_iceberg_compat/simplified.txt deleted file mode 100644 index 839937d40c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,i_category_id,i_category,sum(ss_ext_sales_price)] - CometHashAggregate [sum] [d_year,i_category_id,i_category,sum(ss_ext_sales_price),sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_category_id,i_category] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_category_id,i_category,sum] - CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 - CometProject [i_category] [i_item_sk,i_category_id,i_category] - CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] - CometScan [native_iceberg_compat] 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/q42/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/explain.txt deleted file mode 100644 index 183fffdf00..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_category_id#8, i_category#11], [i_item_sk#7, i_category_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [i_item_sk#7, i_category_id#8, i_category#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11], [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11] -Keys [3]: [d_year#2, i_category_id#8, i_category#11] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] -Arguments: hashpartitioning(d_year#2, i_category_id#8, i_category#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] -Keys [3]: [d_year#2, i_category_id#8, i_category#11] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sum(ss_ext_sales_price)#13 DESC NULLS LAST,d_year#2 ASC NULLS FIRST,i_category_id#8 ASC NULLS FIRST,i_category#11 ASC NULLS FIRST], output=[d_year#2,i_category_id#8,i_category#11,sum(ss_ext_sales_price)#13]), [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13], 100, 0, [sum(ss_ext_sales_price)#13 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/extended.txt deleted file mode 100644 index a292badf5d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 839937d40c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q42/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,i_category_id,i_category,sum(ss_ext_sales_price)] - CometHashAggregate [sum] [d_year,i_category_id,i_category,sum(ss_ext_sales_price),sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_category_id,i_category] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_category_id,i_category,sum] - CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 - CometProject [i_category] [i_item_sk,i_category_id,i_category] - CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_datafusion/explain.txt deleted file mode 100644 index 261e6ddbaa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_datafusion/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometNativeScan parquet spark_catalog.default.store (9) - - -(1) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Condition : ((isnotnull(d_year#2) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Arguments: [d_date_sk#1, d_day_name#4], [d_date_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_day_name#3, 9, true, false, true) AS d_day_name#4] - -(4) CometNativeScan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_store_sk#5) - -(6) CometBroadcastExchange -Input [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_day_name#4] -Right output [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [d_date_sk#1], [ss_sold_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_day_name#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [d_day_name#4, ss_store_sk#5, ss_sales_price#6], [d_day_name#4, ss_store_sk#5, ss_sales_price#6] - -(9) CometNativeScan parquet spark_catalog.default.store -Output [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Condition : ((isnotnull(s_gmt_offset#11) AND (s_gmt_offset#11 = -5.00)) AND isnotnull(s_store_sk#8)) - -(11) CometProject -Input [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Arguments: [s_store_sk#8, s_store_id#12, s_store_name#10], [s_store_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#9, 16, true, false, true) AS s_store_id#12, s_store_name#10] - -(12) CometBroadcastExchange -Input [3]: [s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [s_store_sk#8, s_store_id#12, s_store_name#10] - -(13) CometBroadcastHashJoin -Left output [3]: [d_day_name#4, ss_store_sk#5, ss_sales_price#6] -Right output [3]: [s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [ss_store_sk#5], [s_store_sk#8], Inner, BuildRight - -(14) CometProject -Input [6]: [d_day_name#4, ss_store_sk#5, ss_sales_price#6, s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10], [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10] - -(15) CometHashAggregate -Input [4]: [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10] -Keys [2]: [s_store_name#10, s_store_id#12] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))] - -(16) CometExchange -Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_name#10, s_store_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Keys [2]: [s_store_name#10, s_store_id#12] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))] - -(18) CometTakeOrderedAndProject -Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#10 ASC NULLS FIRST,s_store_id#12 ASC NULLS FIRST,sun_sales#20 ASC NULLS FIRST,mon_sales#21 ASC NULLS FIRST,tue_sales#22 ASC NULLS FIRST,wed_sales#23 ASC NULLS FIRST,thu_sales#24 ASC NULLS FIRST,fri_sales#25 ASC NULLS FIRST,sat_sales#26 ASC NULLS FIRST], output=[s_store_name#10,s_store_id#12,sun_sales#20,mon_sales#21,tue_sales#22,wed_sales#23,thu_sales#24,fri_sales#25,sat_sales#26]), [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26], 100, 0, [s_store_name#10 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sun_sales#20 ASC NULLS FIRST, mon_sales#21 ASC NULLS FIRST, tue_sales#22 ASC NULLS FIRST, wed_sales#23 ASC NULLS FIRST, thu_sales#24 ASC NULLS FIRST, fri_sales#25 ASC NULLS FIRST, sat_sales#26 ASC NULLS FIRST], [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] - -(19) CometColumnarToRow [codegen id : 1] -Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_datafusion/simplified.txt deleted file mode 100644 index c7064b3ad8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_datafusion/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - CometExchange [s_store_name,s_store_id] #1 - CometHashAggregate [d_day_name,ss_sales_price] [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] - CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - 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,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [d_day_name] [d_date_sk,d_day_name] - CometFilter [d_date_sk,d_year,d_day_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] - CometNativeScan 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/q43.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_iceberg_compat/explain.txt deleted file mode 100644 index 62ddd93781..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Condition : ((isnotnull(d_year#2) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Arguments: [d_date_sk#1, d_day_name#4], [d_date_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_day_name#3, 9, true, false, true) AS d_day_name#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_store_sk#5) - -(6) CometBroadcastExchange -Input [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_day_name#4] -Right output [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [d_date_sk#1], [ss_sold_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_day_name#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [d_day_name#4, ss_store_sk#5, ss_sales_price#6], [d_day_name#4, ss_store_sk#5, ss_sales_price#6] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Condition : ((isnotnull(s_gmt_offset#11) AND (s_gmt_offset#11 = -5.00)) AND isnotnull(s_store_sk#8)) - -(11) CometProject -Input [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Arguments: [s_store_sk#8, s_store_id#12, s_store_name#10], [s_store_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#9, 16, true, false, true) AS s_store_id#12, s_store_name#10] - -(12) CometBroadcastExchange -Input [3]: [s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [s_store_sk#8, s_store_id#12, s_store_name#10] - -(13) CometBroadcastHashJoin -Left output [3]: [d_day_name#4, ss_store_sk#5, ss_sales_price#6] -Right output [3]: [s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [ss_store_sk#5], [s_store_sk#8], Inner, BuildRight - -(14) CometProject -Input [6]: [d_day_name#4, ss_store_sk#5, ss_sales_price#6, s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10], [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10] - -(15) CometHashAggregate -Input [4]: [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10] -Keys [2]: [s_store_name#10, s_store_id#12] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))] - -(16) CometExchange -Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_name#10, s_store_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Keys [2]: [s_store_name#10, s_store_id#12] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))] - -(18) CometTakeOrderedAndProject -Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#10 ASC NULLS FIRST,s_store_id#12 ASC NULLS FIRST,sun_sales#20 ASC NULLS FIRST,mon_sales#21 ASC NULLS FIRST,tue_sales#22 ASC NULLS FIRST,wed_sales#23 ASC NULLS FIRST,thu_sales#24 ASC NULLS FIRST,fri_sales#25 ASC NULLS FIRST,sat_sales#26 ASC NULLS FIRST], output=[s_store_name#10,s_store_id#12,sun_sales#20,mon_sales#21,tue_sales#22,wed_sales#23,thu_sales#24,fri_sales#25,sat_sales#26]), [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26], 100, 0, [s_store_name#10 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sun_sales#20 ASC NULLS FIRST, mon_sales#21 ASC NULLS FIRST, tue_sales#22 ASC NULLS FIRST, wed_sales#23 ASC NULLS FIRST, thu_sales#24 ASC NULLS FIRST, fri_sales#25 ASC NULLS FIRST, sat_sales#26 ASC NULLS FIRST], [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] - -(19) CometColumnarToRow [codegen id : 1] -Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_iceberg_compat/simplified.txt deleted file mode 100644 index 59ad7611a5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - CometExchange [s_store_name,s_store_id] #1 - CometHashAggregate [d_day_name,ss_sales_price] [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] - CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - 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,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [d_day_name] [d_date_sk,d_day_name] - CometFilter [d_date_sk,d_year,d_day_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] - CometScan [native_iceberg_compat] 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/q43/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/explain.txt deleted file mode 100644 index 62ddd93781..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Condition : ((isnotnull(d_year#2) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Arguments: [d_date_sk#1, d_day_name#4], [d_date_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_day_name#3, 9, true, false, true) AS d_day_name#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_store_sk#5) - -(6) CometBroadcastExchange -Input [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_day_name#4] -Right output [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [d_date_sk#1], [ss_sold_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_day_name#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [d_day_name#4, ss_store_sk#5, ss_sales_price#6], [d_day_name#4, ss_store_sk#5, ss_sales_price#6] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Condition : ((isnotnull(s_gmt_offset#11) AND (s_gmt_offset#11 = -5.00)) AND isnotnull(s_store_sk#8)) - -(11) CometProject -Input [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Arguments: [s_store_sk#8, s_store_id#12, s_store_name#10], [s_store_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#9, 16, true, false, true) AS s_store_id#12, s_store_name#10] - -(12) CometBroadcastExchange -Input [3]: [s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [s_store_sk#8, s_store_id#12, s_store_name#10] - -(13) CometBroadcastHashJoin -Left output [3]: [d_day_name#4, ss_store_sk#5, ss_sales_price#6] -Right output [3]: [s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [ss_store_sk#5], [s_store_sk#8], Inner, BuildRight - -(14) CometProject -Input [6]: [d_day_name#4, ss_store_sk#5, ss_sales_price#6, s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10], [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10] - -(15) CometHashAggregate -Input [4]: [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10] -Keys [2]: [s_store_name#10, s_store_id#12] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))] - -(16) CometExchange -Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_name#10, s_store_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Keys [2]: [s_store_name#10, s_store_id#12] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))] - -(18) CometTakeOrderedAndProject -Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#10 ASC NULLS FIRST,s_store_id#12 ASC NULLS FIRST,sun_sales#20 ASC NULLS FIRST,mon_sales#21 ASC NULLS FIRST,tue_sales#22 ASC NULLS FIRST,wed_sales#23 ASC NULLS FIRST,thu_sales#24 ASC NULLS FIRST,fri_sales#25 ASC NULLS FIRST,sat_sales#26 ASC NULLS FIRST], output=[s_store_name#10,s_store_id#12,sun_sales#20,mon_sales#21,tue_sales#22,wed_sales#23,thu_sales#24,fri_sales#25,sat_sales#26]), [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26], 100, 0, [s_store_name#10 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sun_sales#20 ASC NULLS FIRST, mon_sales#21 ASC NULLS FIRST, tue_sales#22 ASC NULLS FIRST, wed_sales#23 ASC NULLS FIRST, thu_sales#24 ASC NULLS FIRST, fri_sales#25 ASC NULLS FIRST, sat_sales#26 ASC NULLS FIRST], [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] - -(19) CometColumnarToRow [codegen id : 1] -Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/extended.txt deleted file mode 100644 index cb125e2456..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 59ad7611a5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q43/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - CometExchange [s_store_name,s_store_id] #1 - CometHashAggregate [d_day_name,ss_sales_price] [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] - CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - 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,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [d_day_name] [d_date_sk,d_day_name] - CometFilter [d_date_sk,d_year,d_day_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_datafusion/explain.txt deleted file mode 100644 index b2a660d9d6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_datafusion/explain.txt +++ /dev/null @@ -1,283 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (45) -+- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (34) - : : +- * SortMergeJoin Inner (33) - : : :- * Sort (18) - : : : +- * Project (17) - : : : +- * Filter (16) - : : : +- Window (15) - : : : +- WindowGroupLimit (14) - : : : +- * CometColumnarToRow (13) - : : : +- CometSort (12) - : : : +- CometColumnarExchange (11) - : : : +- WindowGroupLimit (10) - : : : +- * CometColumnarToRow (9) - : : : +- CometSort (8) - : : : +- CometFilter (7) - : : : +- CometHashAggregate (6) - : : : +- CometExchange (5) - : : : +- CometHashAggregate (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : +- * Sort (32) - : : +- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- WindowGroupLimit (28) - : : +- * CometColumnarToRow (27) - : : +- CometSort (26) - : : +- CometColumnarExchange (25) - : : +- WindowGroupLimit (24) - : : +- * CometColumnarToRow (23) - : : +- CometSort (22) - : : +- CometFilter (21) - : : +- CometHashAggregate (20) - : : +- ReusedExchange (19) - : +- BroadcastExchange (39) - : +- * CometColumnarToRow (38) - : +- CometProject (37) - : +- CometFilter (36) - : +- CometNativeScan parquet spark_catalog.default.item (35) - +- ReusedExchange (42) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4)) - -(3) CometProject -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Arguments: [ss_item_sk#1, ss_net_profit#3], [ss_item_sk#1, ss_net_profit#3] - -(4) CometHashAggregate -Input [2]: [ss_item_sk#1, ss_net_profit#3] -Keys [1]: [ss_item_sk#1] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] - -(5) CometExchange -Input [3]: [ss_item_sk#1, sum#5, count#6] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(6) CometHashAggregate -Input [3]: [ss_item_sk#1, sum#5, count#6] -Keys [1]: [ss_item_sk#1] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] - -(7) CometFilter -Input [2]: [item_sk#7, rank_col#8] -Condition : (isnotnull(rank_col#8) AND (cast(rank_col#8 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#9, [id=#10]))) - -(8) CometSort -Input [2]: [item_sk#7, rank_col#8] -Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] - -(9) CometColumnarToRow [codegen id : 1] -Input [2]: [item_sk#7, rank_col#8] - -(10) WindowGroupLimit -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank_col#8 ASC NULLS FIRST], rank(rank_col#8), 10, Partial - -(11) CometColumnarExchange -Input [2]: [item_sk#7, rank_col#8] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(12) CometSort -Input [2]: [item_sk#7, rank_col#8] -Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] - -(13) CometColumnarToRow [codegen id : 2] -Input [2]: [item_sk#7, rank_col#8] - -(14) WindowGroupLimit -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank_col#8 ASC NULLS FIRST], rank(rank_col#8), 10, Final - -(15) Window -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] - -(16) Filter [codegen id : 3] -Input [3]: [item_sk#7, rank_col#8, rnk#11] -Condition : ((rnk#11 < 11) AND isnotnull(item_sk#7)) - -(17) Project [codegen id : 3] -Output [2]: [item_sk#7, rnk#11] -Input [3]: [item_sk#7, rank_col#8, rnk#11] - -(18) Sort [codegen id : 3] -Input [2]: [item_sk#7, rnk#11] -Arguments: [rnk#11 ASC NULLS FIRST], false, 0 - -(19) ReusedExchange [Reuses operator id: 5] -Output [3]: [ss_item_sk#12, sum#13, count#14] - -(20) CometHashAggregate -Input [3]: [ss_item_sk#12, sum#13, count#14] -Keys [1]: [ss_item_sk#12] -Functions [1]: [avg(UnscaledValue(ss_net_profit#15))] - -(21) CometFilter -Input [2]: [item_sk#16, rank_col#17] -Condition : (isnotnull(rank_col#17) AND (cast(rank_col#17 as decimal(13,7)) > (0.9 * ReusedSubquery Subquery scalar-subquery#9, [id=#10]))) - -(22) CometSort -Input [2]: [item_sk#16, rank_col#17] -Arguments: [item_sk#16, rank_col#17], [rank_col#17 DESC NULLS LAST] - -(23) CometColumnarToRow [codegen id : 4] -Input [2]: [item_sk#16, rank_col#17] - -(24) WindowGroupLimit -Input [2]: [item_sk#16, rank_col#17] -Arguments: [rank_col#17 DESC NULLS LAST], rank(rank_col#17), 10, Partial - -(25) CometColumnarExchange -Input [2]: [item_sk#16, rank_col#17] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(26) CometSort -Input [2]: [item_sk#16, rank_col#17] -Arguments: [item_sk#16, rank_col#17], [rank_col#17 DESC NULLS LAST] - -(27) CometColumnarToRow [codegen id : 5] -Input [2]: [item_sk#16, rank_col#17] - -(28) WindowGroupLimit -Input [2]: [item_sk#16, rank_col#17] -Arguments: [rank_col#17 DESC NULLS LAST], rank(rank_col#17), 10, Final - -(29) Window -Input [2]: [item_sk#16, rank_col#17] -Arguments: [rank(rank_col#17) windowspecdefinition(rank_col#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#18], [rank_col#17 DESC NULLS LAST] - -(30) Filter [codegen id : 6] -Input [3]: [item_sk#16, rank_col#17, rnk#18] -Condition : ((rnk#18 < 11) AND isnotnull(item_sk#16)) - -(31) Project [codegen id : 6] -Output [2]: [item_sk#16, rnk#18] -Input [3]: [item_sk#16, rank_col#17, rnk#18] - -(32) Sort [codegen id : 6] -Input [2]: [item_sk#16, rnk#18] -Arguments: [rnk#18 ASC NULLS FIRST], false, 0 - -(33) SortMergeJoin [codegen id : 9] -Left keys [1]: [rnk#11] -Right keys [1]: [rnk#18] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 9] -Output [3]: [item_sk#7, rnk#11, item_sk#16] -Input [4]: [item_sk#7, rnk#11, item_sk#16, rnk#18] - -(35) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_product_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(36) CometFilter -Input [2]: [i_item_sk#19, i_product_name#20] -Condition : isnotnull(i_item_sk#19) - -(37) CometProject -Input [2]: [i_item_sk#19, i_product_name#20] -Arguments: [i_item_sk#19, i_product_name#21], [i_item_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#20, 50, true, false, true) AS i_product_name#21] - -(38) CometColumnarToRow [codegen id : 7] -Input [2]: [i_item_sk#19, i_product_name#21] - -(39) BroadcastExchange -Input [2]: [i_item_sk#19, i_product_name#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(40) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [item_sk#7] -Right keys [1]: [i_item_sk#19] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 9] -Output [3]: [rnk#11, item_sk#16, i_product_name#21] -Input [5]: [item_sk#7, rnk#11, item_sk#16, i_item_sk#19, i_product_name#21] - -(42) ReusedExchange [Reuses operator id: 39] -Output [2]: [i_item_sk#22, i_product_name#23] - -(43) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [item_sk#16] -Right keys [1]: [i_item_sk#22] -Join type: Inner -Join condition: None - -(44) Project [codegen id : 9] -Output [3]: [rnk#11, i_product_name#21 AS best_performing#24, i_product_name#23 AS worst_performing#25] -Input [5]: [rnk#11, item_sk#16, i_product_name#21, i_item_sk#22, i_product_name#23] - -(45) TakeOrderedAndProject -Input [3]: [rnk#11, best_performing#24, worst_performing#25] -Arguments: 100, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#24, worst_performing#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometNativeScan parquet spark_catalog.default.store_sales (46) - - -(46) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_addr_sk#26, ss_store_sk#27, ss_net_profit#28, ss_sold_date_sk#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] -ReadSchema: struct - -(47) CometFilter -Input [4]: [ss_addr_sk#26, ss_store_sk#27, ss_net_profit#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_store_sk#27) AND (ss_store_sk#27 = 4)) AND isnull(ss_addr_sk#26)) - -(48) CometProject -Input [4]: [ss_addr_sk#26, ss_store_sk#27, ss_net_profit#28, ss_sold_date_sk#29] -Arguments: [ss_store_sk#27, ss_net_profit#28], [ss_store_sk#27, ss_net_profit#28] - -(49) CometHashAggregate -Input [2]: [ss_store_sk#27, ss_net_profit#28] -Keys [1]: [ss_store_sk#27] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#28))] - -(50) CometExchange -Input [3]: [ss_store_sk#27, sum#30, count#31] -Arguments: hashpartitioning(ss_store_sk#27, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(51) CometHashAggregate -Input [3]: [ss_store_sk#27, sum#30, count#31] -Keys [1]: [ss_store_sk#27] -Functions [1]: [avg(UnscaledValue(ss_net_profit#28))] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [rank_col#32] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ReusedSubquery Subquery scalar-subquery#9, [id=#10] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_datafusion/simplified.txt deleted file mode 100644 index a6db5c3397..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_datafusion/simplified.txt +++ /dev/null @@ -1,75 +0,0 @@ -TakeOrderedAndProject [rnk,best_performing,worst_performing] - WholeStageCodegen (9) - Project [rnk,i_product_name,i_product_name] - BroadcastHashJoin [item_sk,i_item_sk] - Project [rnk,item_sk,i_product_name] - BroadcastHashJoin [item_sk,i_item_sk] - Project [item_sk,rnk,item_sk] - SortMergeJoin [rnk,rnk] - InputAdapter - WholeStageCodegen (3) - Sort [rnk] - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WindowGroupLimit [rank_col] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometColumnarExchange #1 - WindowGroupLimit [rank_col] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometFilter [item_sk,rank_col] - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_store_sk] #3 - CometHashAggregate [ss_net_profit] [ss_store_sk,sum,count] - CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_item_sk] #2 - CometHashAggregate [ss_net_profit] [ss_item_sk,sum,count] - CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - InputAdapter - WholeStageCodegen (6) - Sort [rnk] - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WindowGroupLimit [rank_col] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometColumnarExchange #4 - WindowGroupLimit [rank_col] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometFilter [item_sk,rank_col] - ReusedSubquery [rank_col] #1 - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - ReusedExchange [ss_item_sk,sum,count] #2 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_product_name] - CometNativeScan 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/q44.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/explain.txt deleted file mode 100644 index 4b30c7d51f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,283 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (45) -+- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (34) - : : +- * SortMergeJoin Inner (33) - : : :- * Sort (18) - : : : +- * Project (17) - : : : +- * Filter (16) - : : : +- Window (15) - : : : +- WindowGroupLimit (14) - : : : +- * CometColumnarToRow (13) - : : : +- CometSort (12) - : : : +- CometColumnarExchange (11) - : : : +- WindowGroupLimit (10) - : : : +- * CometColumnarToRow (9) - : : : +- CometSort (8) - : : : +- CometFilter (7) - : : : +- CometHashAggregate (6) - : : : +- CometExchange (5) - : : : +- CometHashAggregate (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- * Sort (32) - : : +- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- WindowGroupLimit (28) - : : +- * CometColumnarToRow (27) - : : +- CometSort (26) - : : +- CometColumnarExchange (25) - : : +- WindowGroupLimit (24) - : : +- * CometColumnarToRow (23) - : : +- CometSort (22) - : : +- CometFilter (21) - : : +- CometHashAggregate (20) - : : +- ReusedExchange (19) - : +- BroadcastExchange (39) - : +- * CometColumnarToRow (38) - : +- CometProject (37) - : +- CometFilter (36) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (35) - +- ReusedExchange (42) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4)) - -(3) CometProject -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Arguments: [ss_item_sk#1, ss_net_profit#3], [ss_item_sk#1, ss_net_profit#3] - -(4) CometHashAggregate -Input [2]: [ss_item_sk#1, ss_net_profit#3] -Keys [1]: [ss_item_sk#1] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] - -(5) CometExchange -Input [3]: [ss_item_sk#1, sum#5, count#6] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(6) CometHashAggregate -Input [3]: [ss_item_sk#1, sum#5, count#6] -Keys [1]: [ss_item_sk#1] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] - -(7) CometFilter -Input [2]: [item_sk#7, rank_col#8] -Condition : (isnotnull(rank_col#8) AND (cast(rank_col#8 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#9, [id=#10]))) - -(8) CometSort -Input [2]: [item_sk#7, rank_col#8] -Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] - -(9) CometColumnarToRow [codegen id : 1] -Input [2]: [item_sk#7, rank_col#8] - -(10) WindowGroupLimit -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank_col#8 ASC NULLS FIRST], rank(rank_col#8), 10, Partial - -(11) CometColumnarExchange -Input [2]: [item_sk#7, rank_col#8] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(12) CometSort -Input [2]: [item_sk#7, rank_col#8] -Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] - -(13) CometColumnarToRow [codegen id : 2] -Input [2]: [item_sk#7, rank_col#8] - -(14) WindowGroupLimit -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank_col#8 ASC NULLS FIRST], rank(rank_col#8), 10, Final - -(15) Window -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] - -(16) Filter [codegen id : 3] -Input [3]: [item_sk#7, rank_col#8, rnk#11] -Condition : ((rnk#11 < 11) AND isnotnull(item_sk#7)) - -(17) Project [codegen id : 3] -Output [2]: [item_sk#7, rnk#11] -Input [3]: [item_sk#7, rank_col#8, rnk#11] - -(18) Sort [codegen id : 3] -Input [2]: [item_sk#7, rnk#11] -Arguments: [rnk#11 ASC NULLS FIRST], false, 0 - -(19) ReusedExchange [Reuses operator id: 5] -Output [3]: [ss_item_sk#12, sum#13, count#14] - -(20) CometHashAggregate -Input [3]: [ss_item_sk#12, sum#13, count#14] -Keys [1]: [ss_item_sk#12] -Functions [1]: [avg(UnscaledValue(ss_net_profit#15))] - -(21) CometFilter -Input [2]: [item_sk#16, rank_col#17] -Condition : (isnotnull(rank_col#17) AND (cast(rank_col#17 as decimal(13,7)) > (0.9 * ReusedSubquery Subquery scalar-subquery#9, [id=#10]))) - -(22) CometSort -Input [2]: [item_sk#16, rank_col#17] -Arguments: [item_sk#16, rank_col#17], [rank_col#17 DESC NULLS LAST] - -(23) CometColumnarToRow [codegen id : 4] -Input [2]: [item_sk#16, rank_col#17] - -(24) WindowGroupLimit -Input [2]: [item_sk#16, rank_col#17] -Arguments: [rank_col#17 DESC NULLS LAST], rank(rank_col#17), 10, Partial - -(25) CometColumnarExchange -Input [2]: [item_sk#16, rank_col#17] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(26) CometSort -Input [2]: [item_sk#16, rank_col#17] -Arguments: [item_sk#16, rank_col#17], [rank_col#17 DESC NULLS LAST] - -(27) CometColumnarToRow [codegen id : 5] -Input [2]: [item_sk#16, rank_col#17] - -(28) WindowGroupLimit -Input [2]: [item_sk#16, rank_col#17] -Arguments: [rank_col#17 DESC NULLS LAST], rank(rank_col#17), 10, Final - -(29) Window -Input [2]: [item_sk#16, rank_col#17] -Arguments: [rank(rank_col#17) windowspecdefinition(rank_col#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#18], [rank_col#17 DESC NULLS LAST] - -(30) Filter [codegen id : 6] -Input [3]: [item_sk#16, rank_col#17, rnk#18] -Condition : ((rnk#18 < 11) AND isnotnull(item_sk#16)) - -(31) Project [codegen id : 6] -Output [2]: [item_sk#16, rnk#18] -Input [3]: [item_sk#16, rank_col#17, rnk#18] - -(32) Sort [codegen id : 6] -Input [2]: [item_sk#16, rnk#18] -Arguments: [rnk#18 ASC NULLS FIRST], false, 0 - -(33) SortMergeJoin [codegen id : 9] -Left keys [1]: [rnk#11] -Right keys [1]: [rnk#18] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 9] -Output [3]: [item_sk#7, rnk#11, item_sk#16] -Input [4]: [item_sk#7, rnk#11, item_sk#16, rnk#18] - -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_product_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(36) CometFilter -Input [2]: [i_item_sk#19, i_product_name#20] -Condition : isnotnull(i_item_sk#19) - -(37) CometProject -Input [2]: [i_item_sk#19, i_product_name#20] -Arguments: [i_item_sk#19, i_product_name#21], [i_item_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#20, 50, true, false, true) AS i_product_name#21] - -(38) CometColumnarToRow [codegen id : 7] -Input [2]: [i_item_sk#19, i_product_name#21] - -(39) BroadcastExchange -Input [2]: [i_item_sk#19, i_product_name#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(40) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [item_sk#7] -Right keys [1]: [i_item_sk#19] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 9] -Output [3]: [rnk#11, item_sk#16, i_product_name#21] -Input [5]: [item_sk#7, rnk#11, item_sk#16, i_item_sk#19, i_product_name#21] - -(42) ReusedExchange [Reuses operator id: 39] -Output [2]: [i_item_sk#22, i_product_name#23] - -(43) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [item_sk#16] -Right keys [1]: [i_item_sk#22] -Join type: Inner -Join condition: None - -(44) Project [codegen id : 9] -Output [3]: [rnk#11, i_product_name#21 AS best_performing#24, i_product_name#23 AS worst_performing#25] -Input [5]: [rnk#11, item_sk#16, i_product_name#21, i_item_sk#22, i_product_name#23] - -(45) TakeOrderedAndProject -Input [3]: [rnk#11, best_performing#24, worst_performing#25] -Arguments: 100, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#24, worst_performing#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (46) - - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_addr_sk#26, ss_store_sk#27, ss_net_profit#28, ss_sold_date_sk#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] -ReadSchema: struct - -(47) CometFilter -Input [4]: [ss_addr_sk#26, ss_store_sk#27, ss_net_profit#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_store_sk#27) AND (ss_store_sk#27 = 4)) AND isnull(ss_addr_sk#26)) - -(48) CometProject -Input [4]: [ss_addr_sk#26, ss_store_sk#27, ss_net_profit#28, ss_sold_date_sk#29] -Arguments: [ss_store_sk#27, ss_net_profit#28], [ss_store_sk#27, ss_net_profit#28] - -(49) CometHashAggregate -Input [2]: [ss_store_sk#27, ss_net_profit#28] -Keys [1]: [ss_store_sk#27] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#28))] - -(50) CometExchange -Input [3]: [ss_store_sk#27, sum#30, count#31] -Arguments: hashpartitioning(ss_store_sk#27, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(51) CometHashAggregate -Input [3]: [ss_store_sk#27, sum#30, count#31] -Keys [1]: [ss_store_sk#27] -Functions [1]: [avg(UnscaledValue(ss_net_profit#28))] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [rank_col#32] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ReusedSubquery Subquery scalar-subquery#9, [id=#10] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/simplified.txt deleted file mode 100644 index d0c3ad435b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,75 +0,0 @@ -TakeOrderedAndProject [rnk,best_performing,worst_performing] - WholeStageCodegen (9) - Project [rnk,i_product_name,i_product_name] - BroadcastHashJoin [item_sk,i_item_sk] - Project [rnk,item_sk,i_product_name] - BroadcastHashJoin [item_sk,i_item_sk] - Project [item_sk,rnk,item_sk] - SortMergeJoin [rnk,rnk] - InputAdapter - WholeStageCodegen (3) - Sort [rnk] - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WindowGroupLimit [rank_col] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometColumnarExchange #1 - WindowGroupLimit [rank_col] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometFilter [item_sk,rank_col] - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_store_sk] #3 - CometHashAggregate [ss_net_profit] [ss_store_sk,sum,count] - CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_item_sk] #2 - CometHashAggregate [ss_net_profit] [ss_item_sk,sum,count] - CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - InputAdapter - WholeStageCodegen (6) - Sort [rnk] - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WindowGroupLimit [rank_col] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometColumnarExchange #4 - WindowGroupLimit [rank_col] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometFilter [item_sk,rank_col] - ReusedSubquery [rank_col] #1 - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - ReusedExchange [ss_item_sk,sum,count] #2 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_product_name] - CometScan [native_iceberg_compat] 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/q44/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/explain.txt deleted file mode 100644 index 4b30c7d51f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/explain.txt +++ /dev/null @@ -1,283 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (45) -+- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (34) - : : +- * SortMergeJoin Inner (33) - : : :- * Sort (18) - : : : +- * Project (17) - : : : +- * Filter (16) - : : : +- Window (15) - : : : +- WindowGroupLimit (14) - : : : +- * CometColumnarToRow (13) - : : : +- CometSort (12) - : : : +- CometColumnarExchange (11) - : : : +- WindowGroupLimit (10) - : : : +- * CometColumnarToRow (9) - : : : +- CometSort (8) - : : : +- CometFilter (7) - : : : +- CometHashAggregate (6) - : : : +- CometExchange (5) - : : : +- CometHashAggregate (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- * Sort (32) - : : +- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- WindowGroupLimit (28) - : : +- * CometColumnarToRow (27) - : : +- CometSort (26) - : : +- CometColumnarExchange (25) - : : +- WindowGroupLimit (24) - : : +- * CometColumnarToRow (23) - : : +- CometSort (22) - : : +- CometFilter (21) - : : +- CometHashAggregate (20) - : : +- ReusedExchange (19) - : +- BroadcastExchange (39) - : +- * CometColumnarToRow (38) - : +- CometProject (37) - : +- CometFilter (36) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (35) - +- ReusedExchange (42) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4)) - -(3) CometProject -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Arguments: [ss_item_sk#1, ss_net_profit#3], [ss_item_sk#1, ss_net_profit#3] - -(4) CometHashAggregate -Input [2]: [ss_item_sk#1, ss_net_profit#3] -Keys [1]: [ss_item_sk#1] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] - -(5) CometExchange -Input [3]: [ss_item_sk#1, sum#5, count#6] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(6) CometHashAggregate -Input [3]: [ss_item_sk#1, sum#5, count#6] -Keys [1]: [ss_item_sk#1] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] - -(7) CometFilter -Input [2]: [item_sk#7, rank_col#8] -Condition : (isnotnull(rank_col#8) AND (cast(rank_col#8 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#9, [id=#10]))) - -(8) CometSort -Input [2]: [item_sk#7, rank_col#8] -Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] - -(9) CometColumnarToRow [codegen id : 1] -Input [2]: [item_sk#7, rank_col#8] - -(10) WindowGroupLimit -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank_col#8 ASC NULLS FIRST], rank(rank_col#8), 10, Partial - -(11) CometColumnarExchange -Input [2]: [item_sk#7, rank_col#8] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(12) CometSort -Input [2]: [item_sk#7, rank_col#8] -Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] - -(13) CometColumnarToRow [codegen id : 2] -Input [2]: [item_sk#7, rank_col#8] - -(14) WindowGroupLimit -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank_col#8 ASC NULLS FIRST], rank(rank_col#8), 10, Final - -(15) Window -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] - -(16) Filter [codegen id : 3] -Input [3]: [item_sk#7, rank_col#8, rnk#11] -Condition : ((rnk#11 < 11) AND isnotnull(item_sk#7)) - -(17) Project [codegen id : 3] -Output [2]: [item_sk#7, rnk#11] -Input [3]: [item_sk#7, rank_col#8, rnk#11] - -(18) Sort [codegen id : 3] -Input [2]: [item_sk#7, rnk#11] -Arguments: [rnk#11 ASC NULLS FIRST], false, 0 - -(19) ReusedExchange [Reuses operator id: 5] -Output [3]: [ss_item_sk#12, sum#13, count#14] - -(20) CometHashAggregate -Input [3]: [ss_item_sk#12, sum#13, count#14] -Keys [1]: [ss_item_sk#12] -Functions [1]: [avg(UnscaledValue(ss_net_profit#15))] - -(21) CometFilter -Input [2]: [item_sk#16, rank_col#17] -Condition : (isnotnull(rank_col#17) AND (cast(rank_col#17 as decimal(13,7)) > (0.9 * ReusedSubquery Subquery scalar-subquery#9, [id=#10]))) - -(22) CometSort -Input [2]: [item_sk#16, rank_col#17] -Arguments: [item_sk#16, rank_col#17], [rank_col#17 DESC NULLS LAST] - -(23) CometColumnarToRow [codegen id : 4] -Input [2]: [item_sk#16, rank_col#17] - -(24) WindowGroupLimit -Input [2]: [item_sk#16, rank_col#17] -Arguments: [rank_col#17 DESC NULLS LAST], rank(rank_col#17), 10, Partial - -(25) CometColumnarExchange -Input [2]: [item_sk#16, rank_col#17] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(26) CometSort -Input [2]: [item_sk#16, rank_col#17] -Arguments: [item_sk#16, rank_col#17], [rank_col#17 DESC NULLS LAST] - -(27) CometColumnarToRow [codegen id : 5] -Input [2]: [item_sk#16, rank_col#17] - -(28) WindowGroupLimit -Input [2]: [item_sk#16, rank_col#17] -Arguments: [rank_col#17 DESC NULLS LAST], rank(rank_col#17), 10, Final - -(29) Window -Input [2]: [item_sk#16, rank_col#17] -Arguments: [rank(rank_col#17) windowspecdefinition(rank_col#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#18], [rank_col#17 DESC NULLS LAST] - -(30) Filter [codegen id : 6] -Input [3]: [item_sk#16, rank_col#17, rnk#18] -Condition : ((rnk#18 < 11) AND isnotnull(item_sk#16)) - -(31) Project [codegen id : 6] -Output [2]: [item_sk#16, rnk#18] -Input [3]: [item_sk#16, rank_col#17, rnk#18] - -(32) Sort [codegen id : 6] -Input [2]: [item_sk#16, rnk#18] -Arguments: [rnk#18 ASC NULLS FIRST], false, 0 - -(33) SortMergeJoin [codegen id : 9] -Left keys [1]: [rnk#11] -Right keys [1]: [rnk#18] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 9] -Output [3]: [item_sk#7, rnk#11, item_sk#16] -Input [4]: [item_sk#7, rnk#11, item_sk#16, rnk#18] - -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_product_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(36) CometFilter -Input [2]: [i_item_sk#19, i_product_name#20] -Condition : isnotnull(i_item_sk#19) - -(37) CometProject -Input [2]: [i_item_sk#19, i_product_name#20] -Arguments: [i_item_sk#19, i_product_name#21], [i_item_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#20, 50, true, false, true) AS i_product_name#21] - -(38) CometColumnarToRow [codegen id : 7] -Input [2]: [i_item_sk#19, i_product_name#21] - -(39) BroadcastExchange -Input [2]: [i_item_sk#19, i_product_name#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(40) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [item_sk#7] -Right keys [1]: [i_item_sk#19] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 9] -Output [3]: [rnk#11, item_sk#16, i_product_name#21] -Input [5]: [item_sk#7, rnk#11, item_sk#16, i_item_sk#19, i_product_name#21] - -(42) ReusedExchange [Reuses operator id: 39] -Output [2]: [i_item_sk#22, i_product_name#23] - -(43) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [item_sk#16] -Right keys [1]: [i_item_sk#22] -Join type: Inner -Join condition: None - -(44) Project [codegen id : 9] -Output [3]: [rnk#11, i_product_name#21 AS best_performing#24, i_product_name#23 AS worst_performing#25] -Input [5]: [rnk#11, item_sk#16, i_product_name#21, i_item_sk#22, i_product_name#23] - -(45) TakeOrderedAndProject -Input [3]: [rnk#11, best_performing#24, worst_performing#25] -Arguments: 100, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#24, worst_performing#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (46) - - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_addr_sk#26, ss_store_sk#27, ss_net_profit#28, ss_sold_date_sk#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] -ReadSchema: struct - -(47) CometFilter -Input [4]: [ss_addr_sk#26, ss_store_sk#27, ss_net_profit#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_store_sk#27) AND (ss_store_sk#27 = 4)) AND isnull(ss_addr_sk#26)) - -(48) CometProject -Input [4]: [ss_addr_sk#26, ss_store_sk#27, ss_net_profit#28, ss_sold_date_sk#29] -Arguments: [ss_store_sk#27, ss_net_profit#28], [ss_store_sk#27, ss_net_profit#28] - -(49) CometHashAggregate -Input [2]: [ss_store_sk#27, ss_net_profit#28] -Keys [1]: [ss_store_sk#27] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#28))] - -(50) CometExchange -Input [3]: [ss_store_sk#27, sum#30, count#31] -Arguments: hashpartitioning(ss_store_sk#27, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(51) CometHashAggregate -Input [3]: [ss_store_sk#27, sum#30, count#31] -Keys [1]: [ss_store_sk#27] -Functions [1]: [avg(UnscaledValue(ss_net_profit#28))] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [rank_col#32] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ReusedSubquery Subquery scalar-subquery#9, [id=#10] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/extended.txt deleted file mode 100644 index 6b58597252..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/extended.txt +++ /dev/null @@ -1,64 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- SortMergeJoin - : : :- Sort - : : : +- Project - : : : +- Filter - : : : +- Window - : : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : : +- CometColumnarToRow - : : : +- CometSort - : : : +- CometColumnarExchange - : : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : : +- CometColumnarToRow - : : : +- CometSort - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- Sort - : : +- Project - : : +- Filter - : : +- Window - : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 32 out of 55 eligible operators (58%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d0c3ad435b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/simplified.txt +++ /dev/null @@ -1,75 +0,0 @@ -TakeOrderedAndProject [rnk,best_performing,worst_performing] - WholeStageCodegen (9) - Project [rnk,i_product_name,i_product_name] - BroadcastHashJoin [item_sk,i_item_sk] - Project [rnk,item_sk,i_product_name] - BroadcastHashJoin [item_sk,i_item_sk] - Project [item_sk,rnk,item_sk] - SortMergeJoin [rnk,rnk] - InputAdapter - WholeStageCodegen (3) - Sort [rnk] - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WindowGroupLimit [rank_col] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometColumnarExchange #1 - WindowGroupLimit [rank_col] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometFilter [item_sk,rank_col] - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_store_sk] #3 - CometHashAggregate [ss_net_profit] [ss_store_sk,sum,count] - CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_item_sk] #2 - CometHashAggregate [ss_net_profit] [ss_item_sk,sum,count] - CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - InputAdapter - WholeStageCodegen (6) - Sort [rnk] - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WindowGroupLimit [rank_col] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometColumnarExchange #4 - WindowGroupLimit [rank_col] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometFilter [item_sk,rank_col] - ReusedSubquery [rank_col] #1 - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - ReusedExchange [ss_item_sk,sum,count] #2 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_product_name] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/explain.txt deleted file mode 100644 index 2114d6e892..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/explain.txt +++ /dev/null @@ -1,256 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (39) -+- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * Filter (33) - +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (32) - :- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * CometColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (4) - : : : +- BroadcastExchange (14) - : : : +- * CometColumnarToRow (13) - : : : +- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (10) - : : +- ReusedExchange (17) - : +- BroadcastExchange (24) - : +- * CometColumnarToRow (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometNativeScan parquet spark_catalog.default.item (20) - +- BroadcastExchange (31) - +- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometNativeScan parquet spark_catalog.default.item (27) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#5), dynamicpruningexpression(ws_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] - -(3) Filter [codegen id : 6] -Input [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Condition : (isnotnull(ws_bill_customer_sk#3) AND isnotnull(ws_item_sk#2)) - -(4) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#7, c_current_addr_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Condition : (isnotnull(c_customer_sk#7) AND isnotnull(c_current_addr_sk#8)) - -(6) CometColumnarToRow [codegen id : 1] -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] - -(7) BroadcastExchange -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_bill_customer_sk#3] -Right keys [1]: [c_customer_sk#7] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 6] -Output [4]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] -Input [6]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5, c_customer_sk#7, c_current_addr_sk#8] - -(10) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Condition : isnotnull(ca_address_sk#9) - -(12) CometProject -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Arguments: [ca_address_sk#9, ca_city#10, ca_zip#12], [ca_address_sk#9, ca_city#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#11, 10, true, false, true) AS ca_zip#12] - -(13) CometColumnarToRow [codegen id : 2] -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] - -(14) BroadcastExchange -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_current_addr_sk#8] -Right keys [1]: [ca_address_sk#9] -Join type: Inner -Join condition: None - -(16) Project [codegen id : 6] -Output [5]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12] -Input [7]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8, ca_address_sk#9, ca_city#10, ca_zip#12] - -(17) ReusedExchange [Reuses operator id: 44] -Output [1]: [d_date_sk#13] - -(18) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#5] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 6] -Output [4]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12] -Input [6]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12, d_date_sk#13] - -(20) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#14, i_item_id#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [i_item_sk#14, i_item_id#15] -Condition : isnotnull(i_item_sk#14) - -(22) CometProject -Input [2]: [i_item_sk#14, i_item_id#15] -Arguments: [i_item_sk#14, i_item_id#16], [i_item_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#15, 16, true, false, true) AS i_item_id#16] - -(23) CometColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#14, i_item_id#16] - -(24) BroadcastExchange -Input [2]: [i_item_sk#14, i_item_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#2] -Right keys [1]: [i_item_sk#14] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 6] -Output [4]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#16] -Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12, i_item_sk#14, i_item_id#16] - -(27) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#17, i_item_id#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_item_sk, [11,13,17,19,2,23,29,3,5,7])] -ReadSchema: struct - -(28) CometFilter -Input [2]: [i_item_sk#17, i_item_id#18] -Condition : i_item_sk#17 IN (2,3,5,7,11,13,17,19,23,29) - -(29) CometProject -Input [2]: [i_item_sk#17, i_item_id#18] -Arguments: [i_item_id#19], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#18, 16, true, false, true) AS i_item_id#19] - -(30) CometColumnarToRow [codegen id : 5] -Input [1]: [i_item_id#19] - -(31) BroadcastExchange -Input [1]: [i_item_id#19] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [i_item_id#16] -Right keys [1]: [i_item_id#19] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(33) Filter [codegen id : 6] -Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#16, exists#1] -Condition : (substr(ca_zip#12, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) - -(34) Project [codegen id : 6] -Output [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] -Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#16, exists#1] - -(35) HashAggregate [codegen id : 6] -Input [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] -Keys [2]: [ca_zip#12, ca_city#10] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum#20] -Results [3]: [ca_zip#12, ca_city#10, sum#21] - -(36) CometColumnarExchange -Input [3]: [ca_zip#12, ca_city#10, sum#21] -Arguments: hashpartitioning(ca_zip#12, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(37) CometColumnarToRow [codegen id : 7] -Input [3]: [ca_zip#12, ca_city#10, sum#21] - -(38) HashAggregate [codegen id : 7] -Input [3]: [ca_zip#12, ca_city#10, sum#21] -Keys [2]: [ca_zip#12, ca_city#10] -Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#22] -Results [3]: [ca_zip#12, ca_city#10, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#22,17,2) AS sum(ws_sales_price)#23] - -(39) TakeOrderedAndProject -Input [3]: [ca_zip#12, ca_city#10, sum(ws_sales_price)#23] -Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip#12, ca_city#10, sum(ws_sales_price)#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometNativeScan parquet spark_catalog.default.date_dim (40) - - -(40) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#24, d_qoy#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(41) CometFilter -Input [3]: [d_date_sk#13, d_year#24, d_qoy#25] -Condition : ((((isnotnull(d_qoy#25) AND isnotnull(d_year#24)) AND (d_qoy#25 = 2)) AND (d_year#24 = 2001)) AND isnotnull(d_date_sk#13)) - -(42) CometProject -Input [3]: [d_date_sk#13, d_year#24, d_qoy#25] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(43) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(44) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/simplified.txt deleted file mode 100644 index 81ecde84b2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/simplified.txt +++ /dev/null @@ -1,64 +0,0 @@ -TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - WholeStageCodegen (7) - HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_zip,ca_city] #1 - WholeStageCodegen (6) - HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] - Project [ws_sales_price,ca_city,ca_zip] - Filter [ca_zip,exists] - BroadcastHashJoin [i_item_id,i_item_id] - Project [ws_sales_price,ca_city,ca_zip,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_sales_price,ca_city,ca_zip] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Filter [ws_bill_customer_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_zip] [ca_address_sk,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_city,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan 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/q45.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/explain.txt deleted file mode 100644 index f6ab9a8d53..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,259 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (39) -+- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * Filter (33) - +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (32) - :- * CometColumnarToRow (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (7) - : : : : +- CometBroadcastHashJoin (6) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometBroadcastExchange (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- BroadcastExchange (31) - +- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (27) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#5), dynamicpruningexpression(ws_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Condition : (isnotnull(ws_bill_customer_sk#3) AND isnotnull(ws_item_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#7, c_current_addr_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Condition : (isnotnull(c_customer_sk#7) AND isnotnull(c_current_addr_sk#8)) - -(5) CometBroadcastExchange -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Arguments: [c_customer_sk#7, c_current_addr_sk#8] - -(6) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Right output [2]: [c_customer_sk#7, c_current_addr_sk#8] -Arguments: [ws_bill_customer_sk#3], [c_customer_sk#7], Inner, BuildRight - -(7) CometProject -Input [6]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5, c_customer_sk#7, c_current_addr_sk#8] -Arguments: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8], [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Condition : isnotnull(ca_address_sk#9) - -(10) CometProject -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Arguments: [ca_address_sk#9, ca_city#10, ca_zip#12], [ca_address_sk#9, ca_city#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#11, 10, true, false, true) AS ca_zip#12] - -(11) CometBroadcastExchange -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] -Arguments: [ca_address_sk#9, ca_city#10, ca_zip#12] - -(12) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] -Right output [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] -Arguments: [c_current_addr_sk#8], [ca_address_sk#9], Inner, BuildRight - -(13) CometProject -Input [7]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8, ca_address_sk#9, ca_city#10, ca_zip#12] -Arguments: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12], [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Condition : ((((isnotnull(d_qoy#15) AND isnotnull(d_year#14)) AND (d_qoy#15 = 2)) AND (d_year#14 = 2001)) AND isnotnull(d_date_sk#13)) - -(16) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(18) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight - -(19) CometProject -Input [6]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12, d_date_sk#13] -Arguments: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12], [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [i_item_sk#16, i_item_id#17] -Condition : isnotnull(i_item_sk#16) - -(22) CometProject -Input [2]: [i_item_sk#16, i_item_id#17] -Arguments: [i_item_sk#16, i_item_id#18], [i_item_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#17, 16, true, false, true) AS i_item_id#18] - -(23) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_id#18] -Arguments: [i_item_sk#16, i_item_id#18] - -(24) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12] -Right output [2]: [i_item_sk#16, i_item_id#18] -Arguments: [ws_item_sk#2], [i_item_sk#16], Inner, BuildRight - -(25) CometProject -Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12, i_item_sk#16, i_item_id#18] -Arguments: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18], [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] - -(26) CometColumnarToRow [codegen id : 2] -Input [4]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_item_id#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_item_sk, [11,13,17,19,2,23,29,3,5,7])] -ReadSchema: struct - -(28) CometFilter -Input [2]: [i_item_sk#19, i_item_id#20] -Condition : i_item_sk#19 IN (2,3,5,7,11,13,17,19,23,29) - -(29) CometProject -Input [2]: [i_item_sk#19, i_item_id#20] -Arguments: [i_item_id#21], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#20, 16, true, false, true) AS i_item_id#21] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [i_item_id#21] - -(31) BroadcastExchange -Input [1]: [i_item_id#21] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=1] - -(32) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [i_item_id#18] -Right keys [1]: [i_item_id#21] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(33) Filter [codegen id : 2] -Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] -Condition : (substr(ca_zip#12, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) - -(34) Project [codegen id : 2] -Output [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] -Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] - -(35) HashAggregate [codegen id : 2] -Input [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] -Keys [2]: [ca_zip#12, ca_city#10] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [ca_zip#12, ca_city#10, sum#23] - -(36) CometColumnarExchange -Input [3]: [ca_zip#12, ca_city#10, sum#23] -Arguments: hashpartitioning(ca_zip#12, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(37) CometColumnarToRow [codegen id : 3] -Input [3]: [ca_zip#12, ca_city#10, sum#23] - -(38) HashAggregate [codegen id : 3] -Input [3]: [ca_zip#12, ca_city#10, sum#23] -Keys [2]: [ca_zip#12, ca_city#10] -Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#24] -Results [3]: [ca_zip#12, ca_city#10, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#24,17,2) AS sum(ws_sales_price)#25] - -(39) TakeOrderedAndProject -Input [3]: [ca_zip#12, ca_city#10, sum(ws_sales_price)#25] -Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip#12, ca_city#10, sum(ws_sales_price)#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) - - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(41) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Condition : ((((isnotnull(d_qoy#15) AND isnotnull(d_year#14)) AND (d_qoy#15 = 2)) AND (d_year#14 = 2001)) AND isnotnull(d_date_sk#13)) - -(42) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(43) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(44) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/simplified.txt deleted file mode 100644 index d78fc716ab..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - WholeStageCodegen (3) - HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_zip,ca_city] #1 - WholeStageCodegen (2) - HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] - Project [ws_sales_price,ca_city,ca_zip] - Filter [ca_zip,exists] - BroadcastHashJoin [i_item_id,i_item_id] - CometColumnarToRow - InputAdapter - CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] - 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_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 [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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 - CometProject [ca_zip] [ca_address_sk,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_city,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] 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/q45/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/explain.txt deleted file mode 100644 index f6ab9a8d53..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/explain.txt +++ /dev/null @@ -1,259 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (39) -+- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * Filter (33) - +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (32) - :- * CometColumnarToRow (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (7) - : : : : +- CometBroadcastHashJoin (6) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometBroadcastExchange (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- BroadcastExchange (31) - +- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (27) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#5), dynamicpruningexpression(ws_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Condition : (isnotnull(ws_bill_customer_sk#3) AND isnotnull(ws_item_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#7, c_current_addr_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Condition : (isnotnull(c_customer_sk#7) AND isnotnull(c_current_addr_sk#8)) - -(5) CometBroadcastExchange -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Arguments: [c_customer_sk#7, c_current_addr_sk#8] - -(6) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Right output [2]: [c_customer_sk#7, c_current_addr_sk#8] -Arguments: [ws_bill_customer_sk#3], [c_customer_sk#7], Inner, BuildRight - -(7) CometProject -Input [6]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5, c_customer_sk#7, c_current_addr_sk#8] -Arguments: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8], [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Condition : isnotnull(ca_address_sk#9) - -(10) CometProject -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Arguments: [ca_address_sk#9, ca_city#10, ca_zip#12], [ca_address_sk#9, ca_city#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#11, 10, true, false, true) AS ca_zip#12] - -(11) CometBroadcastExchange -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] -Arguments: [ca_address_sk#9, ca_city#10, ca_zip#12] - -(12) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] -Right output [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] -Arguments: [c_current_addr_sk#8], [ca_address_sk#9], Inner, BuildRight - -(13) CometProject -Input [7]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8, ca_address_sk#9, ca_city#10, ca_zip#12] -Arguments: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12], [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Condition : ((((isnotnull(d_qoy#15) AND isnotnull(d_year#14)) AND (d_qoy#15 = 2)) AND (d_year#14 = 2001)) AND isnotnull(d_date_sk#13)) - -(16) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(18) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight - -(19) CometProject -Input [6]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12, d_date_sk#13] -Arguments: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12], [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [i_item_sk#16, i_item_id#17] -Condition : isnotnull(i_item_sk#16) - -(22) CometProject -Input [2]: [i_item_sk#16, i_item_id#17] -Arguments: [i_item_sk#16, i_item_id#18], [i_item_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#17, 16, true, false, true) AS i_item_id#18] - -(23) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_id#18] -Arguments: [i_item_sk#16, i_item_id#18] - -(24) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12] -Right output [2]: [i_item_sk#16, i_item_id#18] -Arguments: [ws_item_sk#2], [i_item_sk#16], Inner, BuildRight - -(25) CometProject -Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12, i_item_sk#16, i_item_id#18] -Arguments: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18], [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] - -(26) CometColumnarToRow [codegen id : 2] -Input [4]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_item_id#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_item_sk, [11,13,17,19,2,23,29,3,5,7])] -ReadSchema: struct - -(28) CometFilter -Input [2]: [i_item_sk#19, i_item_id#20] -Condition : i_item_sk#19 IN (2,3,5,7,11,13,17,19,23,29) - -(29) CometProject -Input [2]: [i_item_sk#19, i_item_id#20] -Arguments: [i_item_id#21], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#20, 16, true, false, true) AS i_item_id#21] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [i_item_id#21] - -(31) BroadcastExchange -Input [1]: [i_item_id#21] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=1] - -(32) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [i_item_id#18] -Right keys [1]: [i_item_id#21] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(33) Filter [codegen id : 2] -Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] -Condition : (substr(ca_zip#12, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) - -(34) Project [codegen id : 2] -Output [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] -Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] - -(35) HashAggregate [codegen id : 2] -Input [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] -Keys [2]: [ca_zip#12, ca_city#10] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [ca_zip#12, ca_city#10, sum#23] - -(36) CometColumnarExchange -Input [3]: [ca_zip#12, ca_city#10, sum#23] -Arguments: hashpartitioning(ca_zip#12, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(37) CometColumnarToRow [codegen id : 3] -Input [3]: [ca_zip#12, ca_city#10, sum#23] - -(38) HashAggregate [codegen id : 3] -Input [3]: [ca_zip#12, ca_city#10, sum#23] -Keys [2]: [ca_zip#12, ca_city#10] -Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#24] -Results [3]: [ca_zip#12, ca_city#10, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#24,17,2) AS sum(ws_sales_price)#25] - -(39) TakeOrderedAndProject -Input [3]: [ca_zip#12, ca_city#10, sum(ws_sales_price)#25] -Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip#12, ca_city#10, sum(ws_sales_price)#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) - - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(41) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Condition : ((((isnotnull(d_qoy#15) AND isnotnull(d_year#14)) AND (d_qoy#15 = 2)) AND (d_year#14 = 2001)) AND isnotnull(d_date_sk#13)) - -(42) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(43) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(44) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/extended.txt deleted file mode 100644 index 4a57091056..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/extended.txt +++ /dev/null @@ -1,47 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- Filter - +- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 32 out of 41 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d78fc716ab..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - WholeStageCodegen (3) - HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_zip,ca_city] #1 - WholeStageCodegen (2) - HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] - Project [ws_sales_price,ca_city,ca_zip] - Filter [ca_zip,exists] - BroadcastHashJoin [i_item_id,i_item_id] - CometColumnarToRow - InputAdapter - CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] - 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_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 [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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 - CometProject [ca_zip] [ca_address_sk,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_city,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/explain.txt deleted file mode 100644 index 994cd12469..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/explain.txt +++ /dev/null @@ -1,267 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (41) -+- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * HashAggregate (30) - : : +- * CometColumnarToRow (29) - : : +- CometColumnarExchange (28) - : : +- * HashAggregate (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Project (6) - : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : :- * Filter (3) - : : : : : : +- * ColumnarToRow (2) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : +- ReusedExchange (4) - : : : : +- BroadcastExchange (11) - : : : : +- * CometColumnarToRow (10) - : : : : +- CometProject (9) - : : : : +- CometFilter (8) - : : : : +- CometNativeScan parquet spark_catalog.default.store (7) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) - : : +- BroadcastExchange (24) - : : +- * CometColumnarToRow (23) - : : +- CometFilter (22) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (21) - : +- BroadcastExchange (35) - : +- * CometColumnarToRow (34) - : +- CometProject (33) - : +- CometFilter (32) - : +- CometNativeScan parquet spark_catalog.default.customer (31) - +- ReusedExchange (38) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] - -(3) Filter [codegen id : 5] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 46] -Output [1]: [d_date_sk#10] - -(5) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 5] -Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_city#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [s_store_sk#11, s_city#12] -Condition : (s_city#12 IN (Fairview,Midway) AND isnotnull(s_store_sk#11)) - -(9) CometProject -Input [2]: [s_store_sk#11, s_city#12] -Arguments: [s_store_sk#11], [s_store_sk#11] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#11] - -(11) BroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#11] - -(14) CometNativeScan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#13, hd_dep_count#14, hd_vehicle_count#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [hd_demo_sk#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : (((hd_dep_count#14 = 4) OR (hd_vehicle_count#15 = 3)) AND isnotnull(hd_demo_sk#13)) - -(16) CometProject -Input [3]: [hd_demo_sk#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#13], [hd_demo_sk#13] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#13] - -(18) BroadcastExchange -Input [1]: [hd_demo_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#13] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [5]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Input [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, hd_demo_sk#13] - -(21) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_city#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [ca_address_sk#16, ca_city#17] -Condition : (isnotnull(ca_address_sk#16) AND isnotnull(ca_city#17)) - -(23) CometColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#16, ca_city#17] - -(24) BroadcastExchange -Input [2]: [ca_address_sk#16, ca_city#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#16] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 5] -Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#17] -Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_address_sk#16, ca_city#17] - -(27) HashAggregate [codegen id : 5] -Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#17] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17] -Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum#18, sum#19] -Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] - -(28) CometColumnarExchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(29) CometColumnarToRow [codegen id : 8] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] - -(30) HashAggregate [codegen id : 8] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17] -Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#6))#22, sum(UnscaledValue(ss_net_profit#7))#23] -Results [5]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#17 AS bought_city#24, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#22,17,2) AS amt#25, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#23,17,2) AS profit#26] - -(31) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(32) CometFilter -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] -Condition : (isnotnull(c_customer_sk#27) AND isnotnull(c_current_addr_sk#28)) - -(33) CometProject -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] -Arguments: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32], [c_customer_sk#27, c_current_addr_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#29, 20, true, false, true) AS c_first_name#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#30, 30, true, false, true) AS c_last_name#32] - -(34) CometColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] - -(35) BroadcastExchange -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(36) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#27] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 8] -Output [7]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#31, c_last_name#32] -Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#24, amt#25, profit#26, c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] - -(38) ReusedExchange [Reuses operator id: 24] -Output [2]: [ca_address_sk#33, ca_city#34] - -(39) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [c_current_addr_sk#28] -Right keys [1]: [ca_address_sk#33] -Join type: Inner -Join condition: NOT (ca_city#34 = bought_city#24) - -(40) Project [codegen id : 8] -Output [7]: [c_last_name#32, c_first_name#31, ca_city#34, bought_city#24, ss_ticket_number#5, amt#25, profit#26] -Input [9]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#31, c_last_name#32, ca_address_sk#33, ca_city#34] - -(41) TakeOrderedAndProject -Input [7]: [c_last_name#32, c_first_name#31, ca_city#34, bought_city#24, ss_ticket_number#5, amt#25, profit#26] -Arguments: 100, [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, ca_city#34 ASC NULLS FIRST, bought_city#24 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#32, c_first_name#31, ca_city#34, bought_city#24, ss_ticket_number#5, amt#25, profit#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometNativeScan parquet spark_catalog.default.date_dim (42) - - -(42) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#35, d_dow#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [3]: [d_date_sk#10, d_year#35, d_dow#36] -Condition : ((d_dow#36 IN (6,0) AND d_year#35 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(44) CometProject -Input [3]: [d_date_sk#10, d_year#35, d_dow#36] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(45) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(46) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/simplified.txt deleted file mode 100644 index b4b78689b1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/simplified.txt +++ /dev/null @@ -1,67 +0,0 @@ -TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - WholeStageCodegen (8) - Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] - Project [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - WholeStageCodegen (5) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_city] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - InputAdapter - ReusedExchange [ca_address_sk,ca_city] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/explain.txt deleted file mode 100644 index ca7ae9df0c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,256 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (39) -+- CometTakeOrderedAndProject (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (34) - : +- CometBroadcastHashJoin (33) - : :- CometHashAggregate (28) - : : +- CometExchange (27) - : : +- CometHashAggregate (26) - : : +- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (20) - : : : +- CometBroadcastHashJoin (19) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometProject (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : : : +- CometBroadcastExchange (18) - : : : +- CometProject (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - : : +- CometBroadcastExchange (23) - : : +- CometFilter (22) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (21) - : +- CometBroadcastExchange (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (29) - +- ReusedExchange (35) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(5) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#13, s_city#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#13, s_city#14] -Condition : (s_city#14 IN (Fairview,Midway) AND isnotnull(s_store_sk#13)) - -(11) CometProject -Input [2]: [s_store_sk#13, s_city#14] -Arguments: [s_store_sk#13], [s_store_sk#13] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#13] -Arguments: [s_store_sk#13] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [1]: [s_store_sk#13] -Arguments: [ss_store_sk#4], [s_store_sk#13], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#13] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((hd_dep_count#16 = 4) OR (hd_vehicle_count#17 = 3)) AND isnotnull(hd_demo_sk#15)) - -(17) CometProject -Input [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] -Arguments: [hd_demo_sk#15], [hd_demo_sk#15] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#15] -Arguments: [hd_demo_sk#15] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [1]: [hd_demo_sk#15] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#15], Inner, BuildRight - -(20) CometProject -Input [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, hd_demo_sk#15] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_city#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [ca_address_sk#18, ca_city#19] -Condition : (isnotnull(ca_address_sk#18) AND isnotnull(ca_city#19)) - -(23) CometBroadcastExchange -Input [2]: [ca_address_sk#18, ca_city#19] -Arguments: [ca_address_sk#18, ca_city#19] - -(24) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [2]: [ca_address_sk#18, ca_city#19] -Arguments: [ss_addr_sk#3], [ca_address_sk#18], Inner, BuildRight - -(25) CometProject -Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_address_sk#18, ca_city#19] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19] - -(26) CometHashAggregate -Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19] -Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] - -(27) CometExchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#20, sum#21] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#20, sum#21] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19] -Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] -Condition : (isnotnull(c_customer_sk#22) AND isnotnull(c_current_addr_sk#23)) - -(31) CometProject -Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] -Arguments: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27], [c_customer_sk#22, c_current_addr_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#24, 20, true, false, true) AS c_first_name#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#25, 30, true, false, true) AS c_last_name#27] - -(32) CometBroadcastExchange -Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Arguments: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] - -(33) CometBroadcastHashJoin -Left output [5]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, amt#29, profit#30] -Right output [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Arguments: [ss_customer_sk#1], [c_customer_sk#22], Inner, BuildRight - -(34) CometProject -Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, amt#29, profit#30, c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Arguments: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27], [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27] - -(35) ReusedExchange [Reuses operator id: 23] -Output [2]: [ca_address_sk#31, ca_city#32] - -(36) CometBroadcastHashJoin -Left output [7]: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Right output [2]: [ca_address_sk#31, ca_city#32] -Arguments: [c_current_addr_sk#23], [ca_address_sk#31], Inner, NOT (ca_city#32 = bought_city#28), BuildRight - -(37) CometProject -Input [9]: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27, ca_address_sk#31, ca_city#32] -Arguments: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30], [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] - -(38) CometTakeOrderedAndProject -Input [7]: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#27 ASC NULLS FIRST,c_first_name#26 ASC NULLS FIRST,ca_city#32 ASC NULLS FIRST,bought_city#28 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#27,c_first_name#26,ca_city#32,bought_city#28,ss_ticket_number#5,amt#29,profit#30]), [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30], 100, 0, [c_last_name#27 ASC NULLS FIRST, c_first_name#26 ASC NULLS FIRST, ca_city#32 ASC NULLS FIRST, bought_city#28 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] - -(39) CometColumnarToRow [codegen id : 1] -Input [7]: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) - - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(41) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(42) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(43) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(44) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/simplified.txt deleted file mode 100644 index e646cb620a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - CometBroadcastHashJoin [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] - CometProject [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometHashAggregate [ss_addr_sk,sum,sum] [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,ca_city,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] - CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - CometHashAggregate [ss_coupon_amt,ss_net_profit] [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] - CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - 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_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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [ca_address_sk,ca_city] #6 - CometFilter [ca_address_sk,ca_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - ReusedExchange [ca_address_sk,ca_city] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/explain.txt deleted file mode 100644 index ca7ae9df0c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/explain.txt +++ /dev/null @@ -1,256 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (39) -+- CometTakeOrderedAndProject (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (34) - : +- CometBroadcastHashJoin (33) - : :- CometHashAggregate (28) - : : +- CometExchange (27) - : : +- CometHashAggregate (26) - : : +- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (20) - : : : +- CometBroadcastHashJoin (19) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometProject (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : : : +- CometBroadcastExchange (18) - : : : +- CometProject (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - : : +- CometBroadcastExchange (23) - : : +- CometFilter (22) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (21) - : +- CometBroadcastExchange (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (29) - +- ReusedExchange (35) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(5) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#13, s_city#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#13, s_city#14] -Condition : (s_city#14 IN (Fairview,Midway) AND isnotnull(s_store_sk#13)) - -(11) CometProject -Input [2]: [s_store_sk#13, s_city#14] -Arguments: [s_store_sk#13], [s_store_sk#13] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#13] -Arguments: [s_store_sk#13] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [1]: [s_store_sk#13] -Arguments: [ss_store_sk#4], [s_store_sk#13], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#13] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((hd_dep_count#16 = 4) OR (hd_vehicle_count#17 = 3)) AND isnotnull(hd_demo_sk#15)) - -(17) CometProject -Input [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] -Arguments: [hd_demo_sk#15], [hd_demo_sk#15] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#15] -Arguments: [hd_demo_sk#15] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [1]: [hd_demo_sk#15] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#15], Inner, BuildRight - -(20) CometProject -Input [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, hd_demo_sk#15] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_city#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [ca_address_sk#18, ca_city#19] -Condition : (isnotnull(ca_address_sk#18) AND isnotnull(ca_city#19)) - -(23) CometBroadcastExchange -Input [2]: [ca_address_sk#18, ca_city#19] -Arguments: [ca_address_sk#18, ca_city#19] - -(24) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [2]: [ca_address_sk#18, ca_city#19] -Arguments: [ss_addr_sk#3], [ca_address_sk#18], Inner, BuildRight - -(25) CometProject -Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_address_sk#18, ca_city#19] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19] - -(26) CometHashAggregate -Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19] -Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] - -(27) CometExchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#20, sum#21] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#20, sum#21] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19] -Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] -Condition : (isnotnull(c_customer_sk#22) AND isnotnull(c_current_addr_sk#23)) - -(31) CometProject -Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] -Arguments: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27], [c_customer_sk#22, c_current_addr_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#24, 20, true, false, true) AS c_first_name#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#25, 30, true, false, true) AS c_last_name#27] - -(32) CometBroadcastExchange -Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Arguments: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] - -(33) CometBroadcastHashJoin -Left output [5]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, amt#29, profit#30] -Right output [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Arguments: [ss_customer_sk#1], [c_customer_sk#22], Inner, BuildRight - -(34) CometProject -Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, amt#29, profit#30, c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Arguments: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27], [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27] - -(35) ReusedExchange [Reuses operator id: 23] -Output [2]: [ca_address_sk#31, ca_city#32] - -(36) CometBroadcastHashJoin -Left output [7]: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Right output [2]: [ca_address_sk#31, ca_city#32] -Arguments: [c_current_addr_sk#23], [ca_address_sk#31], Inner, NOT (ca_city#32 = bought_city#28), BuildRight - -(37) CometProject -Input [9]: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27, ca_address_sk#31, ca_city#32] -Arguments: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30], [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] - -(38) CometTakeOrderedAndProject -Input [7]: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#27 ASC NULLS FIRST,c_first_name#26 ASC NULLS FIRST,ca_city#32 ASC NULLS FIRST,bought_city#28 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#27,c_first_name#26,ca_city#32,bought_city#28,ss_ticket_number#5,amt#29,profit#30]), [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30], 100, 0, [c_last_name#27 ASC NULLS FIRST, c_first_name#26 ASC NULLS FIRST, ca_city#32 ASC NULLS FIRST, bought_city#28 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] - -(39) CometColumnarToRow [codegen id : 1] -Input [7]: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) - - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(41) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(42) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(43) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(44) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/extended.txt deleted file mode 100644 index c842000e11..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/extended.txt +++ /dev/null @@ -1,49 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index e646cb620a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - CometBroadcastHashJoin [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] - CometProject [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometHashAggregate [ss_addr_sk,sum,sum] [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,ca_city,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] - CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - CometHashAggregate [ss_coupon_amt,ss_net_profit] [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] - CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - 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_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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [ca_address_sk,ca_city] #6 - CometFilter [ca_address_sk,ca_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - ReusedExchange [ca_address_sk,ca_city] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/explain.txt deleted file mode 100644 index 3f98e1bd21..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/explain.txt +++ /dev/null @@ -1,304 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (51) -+- * Project (50) - +- * BroadcastHashJoin Inner BuildRight (49) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * CometColumnarToRow (26) - : : +- CometSort (25) - : : +- CometColumnarExchange (24) - : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- * HashAggregate (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.store (14) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- Window (38) - : +- * CometColumnarToRow (37) - : +- CometSort (36) - : +- CometColumnarExchange (35) - : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) - : +- ReusedExchange (32) - +- BroadcastExchange (48) - +- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- ReusedExchange (43) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#4, i_category#5] - -(5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(7) Filter [codegen id : 1] -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) - -(8) BroadcastExchange -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#6] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(11) ReusedExchange [Reuses operator id: 55] -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] -Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) - -(16) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] - -(17) BroadcastExchange -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#7] -Right keys [1]: [s_store_sk#14] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] - -(20) HashAggregate [codegen id : 4] -Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] -Aggregate Attributes [1]: [sum#17] -Results [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] - -(21) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] - -(23) HashAggregate [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#8))#19] -Results [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#19,17,2) AS sum_sales#20, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#19,17,2) AS _w0#21] - -(24) CometColumnarExchange -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) CometColumnarToRow [codegen id : 6] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] - -(27) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(28) Filter [codegen id : 7] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(29) Window -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] -Arguments: [avg(_w0#21) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(30) Filter [codegen id : 22] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] -Condition : ((isnotnull(avg_monthly_sales#23) AND (avg_monthly_sales#23 > 0.000000)) AND CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END) - -(31) Project [codegen id : 22] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] - -(32) ReusedExchange [Reuses operator id: 21] -Output [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] - -(33) CometColumnarToRow [codegen id : 12] -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] - -(34) HashAggregate [codegen id : 12] -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -Keys [6]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] -Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#19] -Results [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#19,17,2) AS sum_sales#20] - -(35) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(36) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] -Arguments: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] - -(37) CometColumnarToRow [codegen id : 13] -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] - -(38) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] -Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#5, i_brand#4, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] - -(39) Project [codegen id : 14] -Output [6]: [i_category#5 AS i_category#31, i_brand#4 AS i_brand#32, s_store_name#24, s_company_name#25, sum_sales#20 AS sum_sales#33, rn#30] -Input [8]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20, rn#30] - -(40) BroadcastExchange -Input [6]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] - -(41) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] -Right keys [5]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, (rn#30 + 1)] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 22] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33] -Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] - -(43) ReusedExchange [Reuses operator id: 35] -Output [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] - -(44) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] -Arguments: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] - -(45) CometColumnarToRow [codegen id : 20] -Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] - -(46) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] -Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#5, i_brand#4, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] - -(47) Project [codegen id : 21] -Output [6]: [i_category#5 AS i_category#39, i_brand#4 AS i_brand#40, s_store_name#34, s_company_name#35, sum_sales#20 AS sum_sales#41, rn#38] -Input [8]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20, rn#38] - -(48) BroadcastExchange -Input [6]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] - -(49) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] -Right keys [5]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, (rn#38 - 1)] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 22] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, sum_sales#33 AS psum#42, sum_sales#41 AS nsum#43] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33, i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] - -(51) TakeOrderedAndProject -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] -Arguments: 100, [(sum_sales#20 - avg_monthly_sales#23) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometFilter (53) - +- CometNativeScan parquet spark_catalog.default.date_dim (52) - - -(52) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(54) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(55) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/simplified.txt deleted file mode 100644 index 017805e5b2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/simplified.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (22) - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (7) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/explain.txt deleted file mode 100644 index 1c039203ce..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_item_sk#1], [ss_item_sk#6], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) - -(16) CometBroadcastExchange -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [s_store_sk#14, s_store_name#15, s_company_name#16] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] -Right output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [ss_store_sk#7], [s_store_sk#14], Inner, BuildRight - -(18) CometProject -Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16], [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] - -(19) CometHashAggregate -Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] - -(20) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] - -(22) CometExchange -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] - -(25) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(28) Filter [codegen id : 7] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] - -(30) ReusedExchange [Reuses operator id: 20] -Output [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] - -(31) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] -Keys [6]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(ss_sales_price#27))] - -(32) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] - -(35) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [6]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] -Input [8]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28] - -(37) BroadcastExchange -Input [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] - -(40) ReusedExchange [Reuses operator id: 32] -Output [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(41) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(43) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#5, i_brand#4, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [6]: [i_category#5 AS i_category#37, i_brand#4 AS i_brand#38, s_store_name#32, s_company_name#33, sum_sales#18 AS sum_sales#39, rn#36] -Input [8]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18, rn#36] - -(45) BroadcastExchange -Input [6]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, (rn#36 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] - -(48) TakeOrderedAndProject -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/simplified.txt deleted file mode 100644 index 19124a9e20..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - 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 [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,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/explain.txt deleted file mode 100644 index 1c039203ce..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_item_sk#1], [ss_item_sk#6], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) - -(16) CometBroadcastExchange -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [s_store_sk#14, s_store_name#15, s_company_name#16] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] -Right output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [ss_store_sk#7], [s_store_sk#14], Inner, BuildRight - -(18) CometProject -Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16], [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] - -(19) CometHashAggregate -Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] - -(20) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] - -(22) CometExchange -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] - -(25) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(28) Filter [codegen id : 7] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] - -(30) ReusedExchange [Reuses operator id: 20] -Output [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] - -(31) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] -Keys [6]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(ss_sales_price#27))] - -(32) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] - -(35) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [6]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] -Input [8]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28] - -(37) BroadcastExchange -Input [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] - -(40) ReusedExchange [Reuses operator id: 32] -Output [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(41) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(43) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#5, i_brand#4, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [6]: [i_category#5 AS i_category#37, i_brand#4 AS i_brand#38, s_store_name#32, s_company_name#33, sum_sales#18 AS sum_sales#39, rn#36] -Input [8]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18, rn#36] - -(45) BroadcastExchange -Input [6]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, (rn#36 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] - -(48) TakeOrderedAndProject -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/extended.txt deleted file mode 100644 index b50b570b4b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 19124a9e20..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - 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 [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,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/explain.txt deleted file mode 100644 index 494c1d4338..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/explain.txt +++ /dev/null @@ -1,207 +0,0 @@ -== Physical Plan == -* HashAggregate (30) -+- * CometColumnarToRow (29) - +- CometColumnarExchange (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * CometColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.store (4) - : : +- BroadcastExchange (14) - : : +- * CometColumnarToRow (13) - : : +- CometProject (12) - : : +- CometFilter (11) - : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (10) - : +- BroadcastExchange (21) - : +- * CometColumnarToRow (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometNativeScan parquet spark_catalog.default.customer_address (17) - +- ReusedExchange (24) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_addr_sk), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ss_net_profit,0.00),LessThanOrEqual(ss_net_profit,2000.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,3000.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,25000.00)))] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] - -(3) Filter [codegen id : 5] -Input [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((((isnotnull(ss_store_sk#3) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_addr_sk#2)) AND ((((ss_sales_price#5 >= 100.00) AND (ss_sales_price#5 <= 150.00)) OR ((ss_sales_price#5 >= 50.00) AND (ss_sales_price#5 <= 100.00))) OR ((ss_sales_price#5 >= 150.00) AND (ss_sales_price#5 <= 200.00)))) AND ((((ss_net_profit#6 >= 0.00) AND (ss_net_profit#6 <= 2000.00)) OR ((ss_net_profit#6 >= 150.00) AND (ss_net_profit#6 <= 3000.00))) OR ((ss_net_profit#6 >= 50.00) AND (ss_net_profit#6 <= 25000.00)))) - -(4) CometNativeScan parquet spark_catalog.default.store -Output [1]: [s_store_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [1]: [s_store_sk#9] -Condition : isnotnull(s_store_sk#9) - -(6) CometColumnarToRow [codegen id : 1] -Input [1]: [s_store_sk#9] - -(7) BroadcastExchange -Input [1]: [s_store_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 5] -Output [6]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Input [8]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, s_store_sk#9] - -(10) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Condition : (isnotnull(cd_demo_sk#10) AND ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) = 4 yr Degree )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) = D) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) = 2 yr Degree ))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) = S) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) = College )))) - -(12) CometProject -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Arguments: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14], [cd_demo_sk#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) AS cd_marital_status#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) AS cd_education_status#14] - -(13) CometColumnarToRow [codegen id : 2] -Input [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] - -(14) BroadcastExchange -Input [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#1] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: ((((((cd_marital_status#13 = M) AND (cd_education_status#14 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#13 = D) AND (cd_education_status#14 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#13 = S) AND (cd_education_status#14 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) - -(16) Project [codegen id : 5] -Output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] -Input [9]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] - -(17) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(18) CometFilter -Input [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Condition : (((isnotnull(ca_country#17) AND (ca_country#17 = United States)) AND isnotnull(ca_address_sk#15)) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) IN (CO,OH,TX) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) IN (OR,MN,KY)) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) IN (VA,CA,MS))) - -(19) CometProject -Input [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Arguments: [ca_address_sk#15, ca_state#18], [ca_address_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) AS ca_state#18] - -(20) CometColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#15, ca_state#18] - -(21) BroadcastExchange -Input [2]: [ca_address_sk#15, ca_state#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(22) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#15] -Join type: Inner -Join condition: ((((ca_state#18 IN (CO,OH,TX) AND (ss_net_profit#6 >= 0.00)) AND (ss_net_profit#6 <= 2000.00)) OR ((ca_state#18 IN (OR,MN,KY) AND (ss_net_profit#6 >= 150.00)) AND (ss_net_profit#6 <= 3000.00))) OR ((ca_state#18 IN (VA,CA,MS) AND (ss_net_profit#6 >= 50.00)) AND (ss_net_profit#6 <= 25000.00))) - -(23) Project [codegen id : 5] -Output [2]: [ss_quantity#4, ss_sold_date_sk#7] -Input [6]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7, ca_address_sk#15, ca_state#18] - -(24) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#19] - -(25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 5] -Output [1]: [ss_quantity#4] -Input [3]: [ss_quantity#4, ss_sold_date_sk#7, d_date_sk#19] - -(27) HashAggregate [codegen id : 5] -Input [1]: [ss_quantity#4] -Keys: [] -Functions [1]: [partial_sum(ss_quantity#4)] -Aggregate Attributes [1]: [sum#20] -Results [1]: [sum#21] - -(28) CometColumnarExchange -Input [1]: [sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(29) CometColumnarToRow [codegen id : 6] -Input [1]: [sum#21] - -(30) HashAggregate [codegen id : 6] -Input [1]: [sum#21] -Keys: [] -Functions [1]: [sum(ss_quantity#4)] -Aggregate Attributes [1]: [sum(ss_quantity#4)#22] -Results [1]: [sum(ss_quantity#4)#22 AS sum(ss_quantity)#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometNativeScan parquet spark_catalog.default.date_dim (31) - - -(31) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_year#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#19, d_year#24] -Condition : ((isnotnull(d_year#24) AND (d_year#24 = 2001)) AND isnotnull(d_date_sk#19)) - -(33) CometProject -Input [2]: [d_date_sk#19, d_year#24] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(35) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/simplified.txt deleted file mode 100644 index 0cd7e900b0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (6) - HashAggregate [sum] [sum(ss_quantity),sum(ss_quantity),sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (5) - HashAggregate [ss_quantity] [sum,sum] - Project [ss_quantity] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sold_date_sk] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] - Project [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] - Project [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/explain.txt deleted file mode 100644 index 71924d1e6d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,202 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometHashAggregate (28) - +- CometExchange (27) - +- CometHashAggregate (26) - +- CometProject (25) - +- CometBroadcastHashJoin (24) - :- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (8) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (14) - +- CometBroadcastExchange (23) - +- CometProject (22) - +- CometFilter (21) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (20) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_addr_sk), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ss_net_profit,0.00),LessThanOrEqual(ss_net_profit,2000.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,3000.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,25000.00)))] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((((isnotnull(ss_store_sk#3) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_addr_sk#2)) AND ((((ss_sales_price#5 >= 100.00) AND (ss_sales_price#5 <= 150.00)) OR ((ss_sales_price#5 >= 50.00) AND (ss_sales_price#5 <= 100.00))) OR ((ss_sales_price#5 >= 150.00) AND (ss_sales_price#5 <= 200.00)))) AND ((((ss_net_profit#6 >= 0.00) AND (ss_net_profit#6 <= 2000.00)) OR ((ss_net_profit#6 >= 150.00) AND (ss_net_profit#6 <= 3000.00))) OR ((ss_net_profit#6 >= 50.00) AND (ss_net_profit#6 <= 25000.00)))) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [s_store_sk#9] -Condition : isnotnull(s_store_sk#9) - -(5) CometBroadcastExchange -Input [1]: [s_store_sk#9] -Arguments: [s_store_sk#9] - -(6) CometBroadcastHashJoin -Left output [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [1]: [s_store_sk#9] -Arguments: [ss_store_sk#3], [s_store_sk#9], Inner, BuildRight - -(7) CometProject -Input [8]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, s_store_sk#9] -Arguments: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Condition : (isnotnull(cd_demo_sk#10) AND ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) = 4 yr Degree )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) = D) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) = 2 yr Degree ))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) = S) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) = College )))) - -(10) CometProject -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Arguments: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14], [cd_demo_sk#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) AS cd_marital_status#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) AS cd_education_status#14] - -(11) CometBroadcastExchange -Input [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] -Arguments: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] - -(12) CometBroadcastHashJoin -Left output [6]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] -Arguments: [ss_cdemo_sk#1], [cd_demo_sk#10], Inner, ((((((cd_marital_status#13 = M) AND (cd_education_status#14 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#13 = D) AND (cd_education_status#14 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#13 = S) AND (cd_education_status#14 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))), BuildRight - -(13) CometProject -Input [9]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] -Arguments: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7], [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Condition : (((isnotnull(ca_country#17) AND (ca_country#17 = United States)) AND isnotnull(ca_address_sk#15)) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) IN (CO,OH,TX) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) IN (OR,MN,KY)) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) IN (VA,CA,MS))) - -(16) CometProject -Input [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Arguments: [ca_address_sk#15, ca_state#18], [ca_address_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) AS ca_state#18] - -(17) CometBroadcastExchange -Input [2]: [ca_address_sk#15, ca_state#18] -Arguments: [ca_address_sk#15, ca_state#18] - -(18) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] -Right output [2]: [ca_address_sk#15, ca_state#18] -Arguments: [ss_addr_sk#2], [ca_address_sk#15], Inner, ((((ca_state#18 IN (CO,OH,TX) AND (ss_net_profit#6 >= 0.00)) AND (ss_net_profit#6 <= 2000.00)) OR ((ca_state#18 IN (OR,MN,KY) AND (ss_net_profit#6 >= 150.00)) AND (ss_net_profit#6 <= 3000.00))) OR ((ca_state#18 IN (VA,CA,MS) AND (ss_net_profit#6 >= 50.00)) AND (ss_net_profit#6 <= 25000.00))), BuildRight - -(19) CometProject -Input [6]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7, ca_address_sk#15, ca_state#18] -Arguments: [ss_quantity#4, ss_sold_date_sk#7], [ss_quantity#4, ss_sold_date_sk#7] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_year#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [d_date_sk#19, d_year#20] -Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(22) CometProject -Input [2]: [d_date_sk#19, d_year#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(23) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(24) CometBroadcastHashJoin -Left output [2]: [ss_quantity#4, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#7], [d_date_sk#19], Inner, BuildRight - -(25) CometProject -Input [3]: [ss_quantity#4, ss_sold_date_sk#7, d_date_sk#19] -Arguments: [ss_quantity#4], [ss_quantity#4] - -(26) CometHashAggregate -Input [1]: [ss_quantity#4] -Keys: [] -Functions [1]: [partial_sum(ss_quantity#4)] - -(27) CometExchange -Input [1]: [sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [1]: [sum#21] -Keys: [] -Functions [1]: [sum(ss_quantity#4)] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [sum(ss_quantity)#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_year#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [d_date_sk#19, d_year#20] -Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(32) CometProject -Input [2]: [d_date_sk#19, d_year#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(33) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(34) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/simplified.txt deleted file mode 100644 index e29e101c7e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,39 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [sum(ss_quantity),sum(ss_quantity)] - CometExchange #1 - CometHashAggregate [ss_quantity] [sum] - CometProject [ss_quantity] - CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] - CometProject [ss_quantity,ss_sold_date_sk] - 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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk] #3 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #4 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [ca_address_sk,ca_state] #5 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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/q48/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/explain.txt deleted file mode 100644 index 71924d1e6d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/explain.txt +++ /dev/null @@ -1,202 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometHashAggregate (28) - +- CometExchange (27) - +- CometHashAggregate (26) - +- CometProject (25) - +- CometBroadcastHashJoin (24) - :- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (8) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (14) - +- CometBroadcastExchange (23) - +- CometProject (22) - +- CometFilter (21) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (20) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_addr_sk), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ss_net_profit,0.00),LessThanOrEqual(ss_net_profit,2000.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,3000.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,25000.00)))] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((((isnotnull(ss_store_sk#3) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_addr_sk#2)) AND ((((ss_sales_price#5 >= 100.00) AND (ss_sales_price#5 <= 150.00)) OR ((ss_sales_price#5 >= 50.00) AND (ss_sales_price#5 <= 100.00))) OR ((ss_sales_price#5 >= 150.00) AND (ss_sales_price#5 <= 200.00)))) AND ((((ss_net_profit#6 >= 0.00) AND (ss_net_profit#6 <= 2000.00)) OR ((ss_net_profit#6 >= 150.00) AND (ss_net_profit#6 <= 3000.00))) OR ((ss_net_profit#6 >= 50.00) AND (ss_net_profit#6 <= 25000.00)))) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [s_store_sk#9] -Condition : isnotnull(s_store_sk#9) - -(5) CometBroadcastExchange -Input [1]: [s_store_sk#9] -Arguments: [s_store_sk#9] - -(6) CometBroadcastHashJoin -Left output [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [1]: [s_store_sk#9] -Arguments: [ss_store_sk#3], [s_store_sk#9], Inner, BuildRight - -(7) CometProject -Input [8]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, s_store_sk#9] -Arguments: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Condition : (isnotnull(cd_demo_sk#10) AND ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) = 4 yr Degree )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) = D) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) = 2 yr Degree ))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) = S) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) = College )))) - -(10) CometProject -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Arguments: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14], [cd_demo_sk#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) AS cd_marital_status#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) AS cd_education_status#14] - -(11) CometBroadcastExchange -Input [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] -Arguments: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] - -(12) CometBroadcastHashJoin -Left output [6]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] -Arguments: [ss_cdemo_sk#1], [cd_demo_sk#10], Inner, ((((((cd_marital_status#13 = M) AND (cd_education_status#14 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#13 = D) AND (cd_education_status#14 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#13 = S) AND (cd_education_status#14 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))), BuildRight - -(13) CometProject -Input [9]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] -Arguments: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7], [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Condition : (((isnotnull(ca_country#17) AND (ca_country#17 = United States)) AND isnotnull(ca_address_sk#15)) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) IN (CO,OH,TX) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) IN (OR,MN,KY)) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) IN (VA,CA,MS))) - -(16) CometProject -Input [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Arguments: [ca_address_sk#15, ca_state#18], [ca_address_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) AS ca_state#18] - -(17) CometBroadcastExchange -Input [2]: [ca_address_sk#15, ca_state#18] -Arguments: [ca_address_sk#15, ca_state#18] - -(18) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] -Right output [2]: [ca_address_sk#15, ca_state#18] -Arguments: [ss_addr_sk#2], [ca_address_sk#15], Inner, ((((ca_state#18 IN (CO,OH,TX) AND (ss_net_profit#6 >= 0.00)) AND (ss_net_profit#6 <= 2000.00)) OR ((ca_state#18 IN (OR,MN,KY) AND (ss_net_profit#6 >= 150.00)) AND (ss_net_profit#6 <= 3000.00))) OR ((ca_state#18 IN (VA,CA,MS) AND (ss_net_profit#6 >= 50.00)) AND (ss_net_profit#6 <= 25000.00))), BuildRight - -(19) CometProject -Input [6]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7, ca_address_sk#15, ca_state#18] -Arguments: [ss_quantity#4, ss_sold_date_sk#7], [ss_quantity#4, ss_sold_date_sk#7] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_year#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [d_date_sk#19, d_year#20] -Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(22) CometProject -Input [2]: [d_date_sk#19, d_year#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(23) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(24) CometBroadcastHashJoin -Left output [2]: [ss_quantity#4, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#7], [d_date_sk#19], Inner, BuildRight - -(25) CometProject -Input [3]: [ss_quantity#4, ss_sold_date_sk#7, d_date_sk#19] -Arguments: [ss_quantity#4], [ss_quantity#4] - -(26) CometHashAggregate -Input [1]: [ss_quantity#4] -Keys: [] -Functions [1]: [partial_sum(ss_quantity#4)] - -(27) CometExchange -Input [1]: [sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [1]: [sum#21] -Keys: [] -Functions [1]: [sum(ss_quantity#4)] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [sum(ss_quantity)#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_year#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [d_date_sk#19, d_year#20] -Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(32) CometProject -Input [2]: [d_date_sk#19, d_year#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(33) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(34) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/extended.txt deleted file mode 100644 index 57fbc2d035..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index e29e101c7e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48/simplified.txt +++ /dev/null @@ -1,39 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [sum(ss_quantity),sum(ss_quantity)] - CometExchange #1 - CometHashAggregate [ss_quantity] [sum] - CometProject [ss_quantity] - CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] - CometProject [ss_quantity,ss_sold_date_sk] - 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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk] #3 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #4 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [ca_address_sk,ca_state] #5 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/explain.txt deleted file mode 100644 index 0a43ebc390..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/explain.txt +++ /dev/null @@ -1,497 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (84) -+- CometTakeOrderedAndProject (83) - +- CometHashAggregate (82) - +- CometColumnarExchange (81) - +- * HashAggregate (80) - +- Union (79) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometColumnarExchange (19) - : +- * HashAggregate (18) - : +- * CometColumnarToRow (17) - : +- CometColumnarExchange (16) - : +- * HashAggregate (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildLeft (10) - : : :- BroadcastExchange (5) - : : : +- * Project (4) - : : : +- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : +- * CometColumnarToRow (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometNativeScan parquet spark_catalog.default.web_returns (6) - : +- ReusedExchange (12) - :- * Project (52) - : +- * Filter (51) - : +- Window (50) - : +- * Sort (49) - : +- Window (48) - : +- * CometColumnarToRow (47) - : +- CometSort (46) - : +- CometColumnarExchange (45) - : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- CometColumnarExchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildLeft (36) - : : :- BroadcastExchange (31) - : : : +- * Project (30) - : : : +- * Filter (29) - : : : +- * ColumnarToRow (28) - : : : +- Scan parquet spark_catalog.default.catalog_sales (27) - : : +- * CometColumnarToRow (35) - : : +- CometProject (34) - : : +- CometFilter (33) - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (32) - : +- ReusedExchange (38) - +- * Project (78) - +- * Filter (77) - +- Window (76) - +- * Sort (75) - +- Window (74) - +- * CometColumnarToRow (73) - +- CometSort (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * CometColumnarToRow (69) - +- CometColumnarExchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * BroadcastHashJoin Inner BuildRight (65) - :- * Project (63) - : +- * BroadcastHashJoin Inner BuildLeft (62) - : :- BroadcastExchange (57) - : : +- * Project (56) - : : +- * Filter (55) - : : +- * ColumnarToRow (54) - : : +- Scan parquet spark_catalog.default.store_sales (53) - : +- * CometColumnarToRow (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometNativeScan parquet spark_catalog.default.store_returns (58) - +- ReusedExchange (64) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] - -(3) Filter [codegen id : 1] -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND isnotnull(ws_quantity#3)) AND (ws_net_profit#5 > 1.00)) AND (ws_net_paid#4 > 0.00)) AND (ws_quantity#3 > 0)) AND isnotnull(ws_order_number#2)) AND isnotnull(ws_item_sk#1)) - -(4) Project [codegen id : 1] -Output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] - -(5) BroadcastExchange -Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=1] - -(6) CometNativeScan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) - -(8) CometProject -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(9) CometColumnarToRow -Input [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(10) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [ws_order_number#2, ws_item_sk#1] -Right keys [2]: [wr_order_number#9, wr_item_sk#8] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 3] -Output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(12) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#13] - -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#6] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 3] -Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] - -(15) HashAggregate [codegen id : 3] -Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Keys [1]: [ws_item_sk#1] -Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#14, sum#15, sum#16, isEmpty#17, sum#18, isEmpty#19] -Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] - -(16) CometColumnarExchange -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(17) CometColumnarToRow [codegen id : 4] -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] - -(18) HashAggregate [codegen id : 4] -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Keys [1]: [ws_item_sk#1] -Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] -Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] - -(19) CometColumnarExchange -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(20) CometSort -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [item#30, return_ratio#31, currency_ratio#32], [return_ratio#31 ASC NULLS FIRST] - -(21) CometColumnarToRow [codegen id : 5] -Input [3]: [item#30, return_ratio#31, currency_ratio#32] - -(22) Window -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] - -(23) Sort [codegen id : 6] -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 - -(24) Window -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] - -(25) Filter [codegen id : 7] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] -Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) - -(26) Project [codegen id : 7] -Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] - -(27) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(28) ColumnarToRow [codegen id : 8] -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] - -(29) Filter [codegen id : 8] -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) - -(30) Project [codegen id : 8] -Output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] - -(31) BroadcastExchange -Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=4] - -(32) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] -Condition : (((isnotnull(cr_return_amount#45) AND (cr_return_amount#45 > 10000.00)) AND isnotnull(cr_order_number#43)) AND isnotnull(cr_item_sk#42)) - -(34) CometProject -Input [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] -Arguments: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45], [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] - -(35) CometColumnarToRow -Input [4]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] - -(36) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [cs_order_number#37, cs_item_sk#36] -Right keys [2]: [cr_order_number#43, cr_item_sk#42] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 10] -Output [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45] -Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] - -(38) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#47] - -(39) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#41] -Right keys [1]: [d_date_sk#47] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 10] -Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] -Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45, d_date_sk#47] - -(41) HashAggregate [codegen id : 10] -Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] -Keys [1]: [cs_item_sk#36] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#44, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#48, sum#49, sum#50, isEmpty#51, sum#52, isEmpty#53] -Results [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] - -(42) CometColumnarExchange -Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(43) CometColumnarToRow [codegen id : 11] -Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] - -(44) HashAggregate [codegen id : 11] -Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Keys [1]: [cs_item_sk#36] -Functions [4]: [sum(coalesce(cr_return_quantity#44, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#44, 0))#60, sum(coalesce(cs_quantity#38, 0))#61, sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#62, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63] -Results [3]: [cs_item_sk#36 AS item#64, (cast(sum(coalesce(cr_return_quantity#44, 0))#60 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#61 as decimal(15,4))) AS return_ratio#65, (cast(sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#62 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63 as decimal(15,4))) AS currency_ratio#66] - -(45) CometColumnarExchange -Input [3]: [item#64, return_ratio#65, currency_ratio#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(46) CometSort -Input [3]: [item#64, return_ratio#65, currency_ratio#66] -Arguments: [item#64, return_ratio#65, currency_ratio#66], [return_ratio#65 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 12] -Input [3]: [item#64, return_ratio#65, currency_ratio#66] - -(48) Window -Input [3]: [item#64, return_ratio#65, currency_ratio#66] -Arguments: [rank(return_ratio#65) windowspecdefinition(return_ratio#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#67], [return_ratio#65 ASC NULLS FIRST] - -(49) Sort [codegen id : 13] -Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] -Arguments: [currency_ratio#66 ASC NULLS FIRST], false, 0 - -(50) Window -Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] -Arguments: [rank(currency_ratio#66) windowspecdefinition(currency_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#68], [currency_ratio#66 ASC NULLS FIRST] - -(51) Filter [codegen id : 14] -Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] -Condition : ((return_rank#67 <= 10) OR (currency_rank#68 <= 10)) - -(52) Project [codegen id : 14] -Output [5]: [catalog AS channel#69, item#64, return_ratio#65, return_rank#67, currency_rank#68] -Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] - -(53) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#75), dynamicpruningexpression(ss_sold_date_sk#75 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(54) ColumnarToRow [codegen id : 15] -Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] - -(55) Filter [codegen id : 15] -Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] -Condition : (((((((isnotnull(ss_net_profit#74) AND isnotnull(ss_net_paid#73)) AND isnotnull(ss_quantity#72)) AND (ss_net_profit#74 > 1.00)) AND (ss_net_paid#73 > 0.00)) AND (ss_quantity#72 > 0)) AND isnotnull(ss_ticket_number#71)) AND isnotnull(ss_item_sk#70)) - -(56) Project [codegen id : 15] -Output [5]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75] -Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] - -(57) BroadcastExchange -Input [5]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=7] - -(58) CometNativeScan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(59) CometFilter -Input [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] -Condition : (((isnotnull(sr_return_amt#79) AND (sr_return_amt#79 > 10000.00)) AND isnotnull(sr_ticket_number#77)) AND isnotnull(sr_item_sk#76)) - -(60) CometProject -Input [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] -Arguments: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79], [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] - -(61) CometColumnarToRow -Input [4]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] - -(62) BroadcastHashJoin [codegen id : 17] -Left keys [2]: [ss_ticket_number#71, ss_item_sk#70] -Right keys [2]: [sr_ticket_number#77, sr_item_sk#76] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 17] -Output [6]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79] -Input [9]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] - -(64) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#81] - -(65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#75] -Right keys [1]: [d_date_sk#81] -Join type: Inner -Join condition: None - -(66) Project [codegen id : 17] -Output [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] -Input [7]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79, d_date_sk#81] - -(67) HashAggregate [codegen id : 17] -Input [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] -Keys [1]: [ss_item_sk#70] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#78, 0)), partial_sum(coalesce(ss_quantity#72, 0)), partial_sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#82, sum#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Results [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] - -(68) CometColumnarExchange -Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] -Arguments: hashpartitioning(ss_item_sk#70, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(69) CometColumnarToRow [codegen id : 18] -Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] - -(70) HashAggregate [codegen id : 18] -Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] -Keys [1]: [ss_item_sk#70] -Functions [4]: [sum(coalesce(sr_return_quantity#78, 0)), sum(coalesce(ss_quantity#72, 0)), sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#78, 0))#94, sum(coalesce(ss_quantity#72, 0))#95, sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00))#96, sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97] -Results [3]: [ss_item_sk#70 AS item#98, (cast(sum(coalesce(sr_return_quantity#78, 0))#94 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#72, 0))#95 as decimal(15,4))) AS return_ratio#99, (cast(sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00))#96 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97 as decimal(15,4))) AS currency_ratio#100] - -(71) CometColumnarExchange -Input [3]: [item#98, return_ratio#99, currency_ratio#100] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(72) CometSort -Input [3]: [item#98, return_ratio#99, currency_ratio#100] -Arguments: [item#98, return_ratio#99, currency_ratio#100], [return_ratio#99 ASC NULLS FIRST] - -(73) CometColumnarToRow [codegen id : 19] -Input [3]: [item#98, return_ratio#99, currency_ratio#100] - -(74) Window -Input [3]: [item#98, return_ratio#99, currency_ratio#100] -Arguments: [rank(return_ratio#99) windowspecdefinition(return_ratio#99 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#101], [return_ratio#99 ASC NULLS FIRST] - -(75) Sort [codegen id : 20] -Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] -Arguments: [currency_ratio#100 ASC NULLS FIRST], false, 0 - -(76) Window -Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] -Arguments: [rank(currency_ratio#100) windowspecdefinition(currency_ratio#100 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#102], [currency_ratio#100 ASC NULLS FIRST] - -(77) Filter [codegen id : 21] -Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] -Condition : ((return_rank#101 <= 10) OR (currency_rank#102 <= 10)) - -(78) Project [codegen id : 21] -Output [5]: [store AS channel#103, item#98, return_ratio#99, return_rank#101, currency_rank#102] -Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] - -(79) Union - -(80) HashAggregate [codegen id : 22] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] - -(81) CometColumnarExchange -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(82) CometHashAggregate -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Functions: [] - -(83) CometTakeOrderedAndProject -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,return_rank#33 ASC NULLS FIRST,currency_rank#34 ASC NULLS FIRST], output=[channel#35,item#30,return_ratio#31,return_rank#33,currency_rank#34]), [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34], 100, 0, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] - -(84) CometColumnarToRow [codegen id : 23] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (89) -+- * CometColumnarToRow (88) - +- CometProject (87) - +- CometFilter (86) - +- CometNativeScan parquet spark_catalog.default.date_dim (85) - - -(85) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#104, d_moy#105] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(86) CometFilter -Input [3]: [d_date_sk#13, d_year#104, d_moy#105] -Condition : ((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND (d_year#104 = 2001)) AND (d_moy#105 = 12)) AND isnotnull(d_date_sk#13)) - -(87) CometProject -Input [3]: [d_date_sk#13, d_year#104, d_moy#105] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(88) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(89) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 - -Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#75 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/simplified.txt deleted file mode 100644 index 047269434d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/simplified.txt +++ /dev/null @@ -1,140 +0,0 @@ -WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (22) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (7) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (6) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #2 - WholeStageCodegen (4) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (14) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (13) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #6 - WholeStageCodegen (11) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_item_sk] #7 - WholeStageCodegen (10) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (21) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (20) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #9 - WholeStageCodegen (18) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk] #10 - WholeStageCodegen (17) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/explain.txt deleted file mode 100644 index 338f5f4201..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,462 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- * Project (49) - : +- * Filter (48) - : +- Window (47) - : +- * Sort (46) - : +- Window (45) - : +- * CometColumnarToRow (44) - : +- CometSort (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometBroadcastExchange (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometBroadcastHashJoin (57) - : :- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (59) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND isnotnull(ws_quantity#3)) AND (ws_net_profit#5 > 1.00)) AND (ws_net_paid#4 > 0.00)) AND (ws_quantity#3 > 0)) AND isnotnull(ws_order_number#2)) AND isnotnull(ws_item_sk#1)) - -(3) CometProject -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6], [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(4) CometBroadcastExchange -Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) - -(7) CometProject -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(8) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner, BuildLeft - -(9) CometProject -Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(12) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] - -(16) CometHashAggregate -Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Keys [1]: [ws_item_sk#1] -Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(17) CometExchange -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Keys [1]: [ws_item_sk#1] -Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(19) CometExchange -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] - -(21) CometColumnarToRow [codegen id : 1] -Input [3]: [item#22, return_ratio#23, currency_ratio#24] - -(22) Window -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] - -(23) Sort [codegen id : 2] -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 - -(24) Window -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] - -(25) Filter [codegen id : 3] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] -Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) - -(26) Project [codegen id : 3] -Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_sold_date_sk#33 IN dynamicpruning#34)] -PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(28) CometFilter -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) - -(29) CometProject -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(30) CometBroadcastExchange -Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(32) CometFilter -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) - -(33) CometProject -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] - -(34) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft - -(35) CometProject -Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] - -(36) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#40] - -(37) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -Right output [1]: [d_date_sk#40] -Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight - -(38) CometProject -Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] - -(39) CometHashAggregate -Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -Keys [1]: [cs_item_sk#28] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(40) CometExchange -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(41) CometHashAggregate -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Keys [1]: [cs_item_sk#28] -Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(42) CometExchange -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(43) CometSort -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] - -(44) CometColumnarToRow [codegen id : 4] -Input [3]: [item#47, return_ratio#48, currency_ratio#49] - -(45) Window -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] - -(46) Sort [codegen id : 5] -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 - -(47) Window -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] - -(48) Filter [codegen id : 6] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] -Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) - -(49) Project [codegen id : 6] -Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) - -(52) CometProject -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(53) CometBroadcastExchange -Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(55) CometFilter -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) - -(56) CometProject -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] - -(57) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft - -(58) CometProject -Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] - -(59) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#65] - -(60) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(61) CometProject -Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] - -(62) CometHashAggregate -Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -Keys [1]: [ss_item_sk#53] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(63) CometExchange -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(64) CometHashAggregate -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Keys [1]: [ss_item_sk#53] -Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(65) CometExchange -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(66) CometSort -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] - -(67) CometColumnarToRow [codegen id : 7] -Input [3]: [item#72, return_ratio#73, currency_ratio#74] - -(68) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] - -(69) Sort [codegen id : 8] -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 - -(70) Window -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] - -(71) Filter [codegen id : 9] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] -Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) - -(72) Project [codegen id : 9] -Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] - -(73) Union - -(74) HashAggregate [codegen id : 10] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(75) CometColumnarExchange -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometHashAggregate -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] - -(77) CometTakeOrderedAndProject -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(78) CometColumnarToRow [codegen id : 11] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) - - -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(80) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(81) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(82) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(83) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 - -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/simplified.txt deleted file mode 100644 index ca80833ee5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,110 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (10) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (3) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (2) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - 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_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_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - 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_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_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 (9) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (8) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - 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_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_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/explain.txt deleted file mode 100644 index 338f5f4201..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/explain.txt +++ /dev/null @@ -1,462 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- * Project (49) - : +- * Filter (48) - : +- Window (47) - : +- * Sort (46) - : +- Window (45) - : +- * CometColumnarToRow (44) - : +- CometSort (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometBroadcastExchange (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometBroadcastHashJoin (57) - : :- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (59) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND isnotnull(ws_quantity#3)) AND (ws_net_profit#5 > 1.00)) AND (ws_net_paid#4 > 0.00)) AND (ws_quantity#3 > 0)) AND isnotnull(ws_order_number#2)) AND isnotnull(ws_item_sk#1)) - -(3) CometProject -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6], [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(4) CometBroadcastExchange -Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) - -(7) CometProject -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(8) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner, BuildLeft - -(9) CometProject -Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(12) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] - -(16) CometHashAggregate -Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Keys [1]: [ws_item_sk#1] -Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(17) CometExchange -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Keys [1]: [ws_item_sk#1] -Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(19) CometExchange -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] - -(21) CometColumnarToRow [codegen id : 1] -Input [3]: [item#22, return_ratio#23, currency_ratio#24] - -(22) Window -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] - -(23) Sort [codegen id : 2] -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 - -(24) Window -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] - -(25) Filter [codegen id : 3] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] -Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) - -(26) Project [codegen id : 3] -Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_sold_date_sk#33 IN dynamicpruning#34)] -PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(28) CometFilter -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) - -(29) CometProject -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(30) CometBroadcastExchange -Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(32) CometFilter -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) - -(33) CometProject -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] - -(34) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft - -(35) CometProject -Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] - -(36) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#40] - -(37) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -Right output [1]: [d_date_sk#40] -Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight - -(38) CometProject -Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] - -(39) CometHashAggregate -Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -Keys [1]: [cs_item_sk#28] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(40) CometExchange -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(41) CometHashAggregate -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Keys [1]: [cs_item_sk#28] -Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(42) CometExchange -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(43) CometSort -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] - -(44) CometColumnarToRow [codegen id : 4] -Input [3]: [item#47, return_ratio#48, currency_ratio#49] - -(45) Window -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] - -(46) Sort [codegen id : 5] -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 - -(47) Window -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] - -(48) Filter [codegen id : 6] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] -Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) - -(49) Project [codegen id : 6] -Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) - -(52) CometProject -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(53) CometBroadcastExchange -Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(55) CometFilter -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) - -(56) CometProject -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] - -(57) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft - -(58) CometProject -Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] - -(59) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#65] - -(60) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(61) CometProject -Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] - -(62) CometHashAggregate -Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -Keys [1]: [ss_item_sk#53] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(63) CometExchange -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(64) CometHashAggregate -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Keys [1]: [ss_item_sk#53] -Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(65) CometExchange -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(66) CometSort -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] - -(67) CometColumnarToRow [codegen id : 7] -Input [3]: [item#72, return_ratio#73, currency_ratio#74] - -(68) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] - -(69) Sort [codegen id : 8] -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 - -(70) Window -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] - -(71) Filter [codegen id : 9] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] -Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) - -(72) Project [codegen id : 9] -Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] - -(73) Union - -(74) HashAggregate [codegen id : 10] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(75) CometColumnarExchange -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometHashAggregate -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] - -(77) CometTakeOrderedAndProject -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(78) CometColumnarToRow [codegen id : 11] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) - - -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(80) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(81) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(82) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(83) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 - -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/extended.txt deleted file mode 100644 index 75684a9669..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/extended.txt +++ /dev/null @@ -1,94 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index ca80833ee5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/simplified.txt +++ /dev/null @@ -1,110 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (10) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (3) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (2) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - 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_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_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - 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_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_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 (9) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (8) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - 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_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_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/explain.txt deleted file mode 100644 index a0a95bd28b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/explain.txt +++ /dev/null @@ -1,499 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (81) -+- * HashAggregate (80) - +- * CometColumnarToRow (79) - +- CometColumnarExchange (78) - +- * HashAggregate (77) - +- * Expand (76) - +- Union (75) - :- * HashAggregate (23) - : +- * CometColumnarToRow (22) - : +- CometColumnarExchange (21) - : +- * HashAggregate (20) - : +- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- Union (9) - : : : :- * Project (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- * Project (8) - : : : +- * Filter (7) - : : : +- * ColumnarToRow (6) - : : : +- Scan parquet spark_catalog.default.store_returns (5) - : : +- ReusedExchange (10) - : +- BroadcastExchange (17) - : +- * CometColumnarToRow (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometNativeScan parquet spark_catalog.default.store (13) - :- * HashAggregate (46) - : +- * CometColumnarToRow (45) - : +- CometColumnarExchange (44) - : +- * HashAggregate (43) - : +- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- Union (32) - : : : :- * Project (27) - : : : : +- * Filter (26) - : : : : +- * ColumnarToRow (25) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (24) - : : : +- * Project (31) - : : : +- * Filter (30) - : : : +- * ColumnarToRow (29) - : : : +- Scan parquet spark_catalog.default.catalog_returns (28) - : : +- ReusedExchange (33) - : +- BroadcastExchange (40) - : +- * CometColumnarToRow (39) - : +- CometProject (38) - : +- CometFilter (37) - : +- CometNativeScan parquet spark_catalog.default.catalog_page (36) - +- * HashAggregate (74) - +- * CometColumnarToRow (73) - +- CometColumnarExchange (72) - +- * HashAggregate (71) - +- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * Project (63) - : +- * BroadcastHashJoin Inner BuildRight (62) - : :- Union (60) - : : :- * Project (50) - : : : +- * Filter (49) - : : : +- * ColumnarToRow (48) - : : : +- Scan parquet spark_catalog.default.web_sales (47) - : : +- * Project (59) - : : +- * BroadcastHashJoin Inner BuildLeft (58) - : : :- BroadcastExchange (53) - : : : +- * ColumnarToRow (52) - : : : +- Scan parquet spark_catalog.default.web_returns (51) - : : +- * CometColumnarToRow (57) - : : +- CometProject (56) - : : +- CometFilter (55) - : : +- CometNativeScan parquet spark_catalog.default.web_sales (54) - : +- ReusedExchange (61) - +- BroadcastExchange (68) - +- * CometColumnarToRow (67) - +- CometProject (66) - +- CometFilter (65) - +- CometNativeScan parquet spark_catalog.default.web_site (64) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 1] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(4) Project [codegen id : 1] -Output [6]: [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] - -(5) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 2] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] - -(7) Filter [codegen id : 2] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#12) - -(8) Project [codegen id : 2] -Output [6]: [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] - -(9) Union - -(10) ReusedExchange [Reuses operator id: 86] -Output [1]: [d_date_sk#22] - -(11) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [date_sk#7] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None - -(12) Project [codegen id : 5] -Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] - -(13) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#23, s_store_id#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [s_store_sk#23, s_store_id#24] -Condition : isnotnull(s_store_sk#23) - -(15) CometProject -Input [2]: [s_store_sk#23, s_store_id#24] -Arguments: [s_store_sk#23, s_store_id#25], [s_store_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#24, 16, true, false, true) AS s_store_id#25] - -(16) CometColumnarToRow [codegen id : 4] -Input [2]: [s_store_sk#23, s_store_id#25] - -(17) BroadcastExchange -Input [2]: [s_store_sk#23, s_store_id#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(18) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [store_sk#6] -Right keys [1]: [s_store_sk#23] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 5] -Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#25] - -(20) HashAggregate [codegen id : 5] -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -Keys [1]: [s_store_id#25] -Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum#26, sum#27, sum#28, sum#29] -Results [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] - -(21) CometColumnarExchange -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(22) CometColumnarToRow [codegen id : 6] -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] - -(23) HashAggregate [codegen id : 6] -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -Keys [1]: [s_store_id#25] -Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#34, sum(UnscaledValue(return_amt#10))#35, sum(UnscaledValue(profit#9))#36, sum(UnscaledValue(net_loss#11))#37] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#8))#34,17,2) AS sales#38, MakeDecimal(sum(UnscaledValue(return_amt#10))#35,17,2) AS returns#39, (MakeDecimal(sum(UnscaledValue(profit#9))#36,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#37,17,2)) AS profit#40, store channel AS channel#41, concat(store, s_store_id#25) AS id#42] - -(24) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_sold_date_sk#46 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cs_catalog_page_sk)] -ReadSchema: struct - -(25) ColumnarToRow [codegen id : 7] -Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] - -(26) Filter [codegen id : 7] -Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Condition : isnotnull(cs_catalog_page_sk#43) - -(27) Project [codegen id : 7] -Output [6]: [cs_catalog_page_sk#43 AS page_sk#47, cs_sold_date_sk#46 AS date_sk#48, cs_ext_sales_price#44 AS sales_price#49, cs_net_profit#45 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] -Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] - -(28) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cr_catalog_page_sk)] -ReadSchema: struct - -(29) ColumnarToRow [codegen id : 8] -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] - -(30) Filter [codegen id : 8] -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -Condition : isnotnull(cr_catalog_page_sk#53) - -(31) Project [codegen id : 8] -Output [6]: [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] - -(32) Union - -(33) ReusedExchange [Reuses operator id: 86] -Output [1]: [d_date_sk#63] - -(34) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [date_sk#48] -Right keys [1]: [d_date_sk#63] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 11] -Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] -Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] - -(36) CometNativeScan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Condition : isnotnull(cp_catalog_page_sk#64) - -(38) CometProject -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Arguments: [cp_catalog_page_sk#64, cp_catalog_page_id#66], [cp_catalog_page_sk#64, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#65, 16, true, false, true) AS cp_catalog_page_id#66] - -(39) CometColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] - -(40) BroadcastExchange -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(41) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [page_sk#47] -Right keys [1]: [cp_catalog_page_sk#64] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 11] -Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] -Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#66] - -(43) HashAggregate [codegen id : 11] -Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] -Keys [1]: [cp_catalog_page_id#66] -Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] -Aggregate Attributes [4]: [sum#67, sum#68, sum#69, sum#70] -Results [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] - -(44) CometColumnarExchange -Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] -Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(45) CometColumnarToRow [codegen id : 12] -Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] - -(46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] -Keys [1]: [cp_catalog_page_id#66] -Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#75, sum(UnscaledValue(return_amt#51))#76, sum(UnscaledValue(profit#50))#77, sum(UnscaledValue(net_loss#52))#78] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#49))#75,17,2) AS sales#79, MakeDecimal(sum(UnscaledValue(return_amt#51))#76,17,2) AS returns#80, (MakeDecimal(sum(UnscaledValue(profit#50))#77,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#52))#78,17,2)) AS profit#81, catalog channel AS channel#82, concat(catalog_page, cp_catalog_page_id#66) AS id#83] - -(47) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_sold_date_sk#87 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] - -(49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] -Condition : isnotnull(ws_web_site_sk#84) - -(50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#84 AS wsr_web_site_sk#88, ws_sold_date_sk#87 AS date_sk#89, ws_ext_sales_price#85 AS sales_price#90, ws_net_profit#86 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] -Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] - -(51) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#5)] -ReadSchema: struct - -(52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] - -(53) BroadcastExchange -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [plan_id=5] - -(54) CometNativeScan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(55) CometFilter -Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) - -(56) CometProject -Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] - -(57) CometColumnarToRow -Input [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] - -(58) BroadcastHashJoin [codegen id : 15] -Left keys [2]: [wr_item_sk#94, wr_order_number#95] -Right keys [2]: [ws_item_sk#99, ws_order_number#101] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 15] -Output [6]: [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] -Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] - -(60) Union - -(61) ReusedExchange [Reuses operator id: 86] -Output [1]: [d_date_sk#109] - -(62) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [date_sk#89] -Right keys [1]: [d_date_sk#109] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 18] -Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] -Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] - -(64) CometNativeScan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#110, web_site_id#111] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(65) CometFilter -Input [2]: [web_site_sk#110, web_site_id#111] -Condition : isnotnull(web_site_sk#110) - -(66) CometProject -Input [2]: [web_site_sk#110, web_site_id#111] -Arguments: [web_site_sk#110, web_site_id#112], [web_site_sk#110, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#111, 16, true, false, true) AS web_site_id#112] - -(67) CometColumnarToRow [codegen id : 17] -Input [2]: [web_site_sk#110, web_site_id#112] - -(68) BroadcastExchange -Input [2]: [web_site_sk#110, web_site_id#112] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(69) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [wsr_web_site_sk#88] -Right keys [1]: [web_site_sk#110] -Join type: Inner -Join condition: None - -(70) Project [codegen id : 18] -Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] -Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#112] - -(71) HashAggregate [codegen id : 18] -Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] -Keys [1]: [web_site_id#112] -Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum#113, sum#114, sum#115, sum#116] -Results [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] - -(72) CometColumnarExchange -Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] -Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(73) CometColumnarToRow [codegen id : 19] -Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] - -(74) HashAggregate [codegen id : 19] -Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] -Keys [1]: [web_site_id#112] -Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#121, sum(UnscaledValue(return_amt#92))#122, sum(UnscaledValue(profit#91))#123, sum(UnscaledValue(net_loss#93))#124] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#90))#121,17,2) AS sales#125, MakeDecimal(sum(UnscaledValue(return_amt#92))#122,17,2) AS returns#126, (MakeDecimal(sum(UnscaledValue(profit#91))#123,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#124,17,2)) AS profit#127, web channel AS channel#128, concat(web_site, web_site_id#112) AS id#129] - -(75) Union - -(76) Expand [codegen id : 20] -Input [5]: [sales#38, returns#39, profit#40, channel#41, id#42] -Arguments: [[sales#38, returns#39, profit#40, channel#41, id#42, 0], [sales#38, returns#39, profit#40, channel#41, null, 1], [sales#38, returns#39, profit#40, null, null, 3]], [sales#38, returns#39, profit#40, channel#130, id#131, spark_grouping_id#132] - -(77) HashAggregate [codegen id : 20] -Input [6]: [sales#38, returns#39, profit#40, channel#130, id#131, spark_grouping_id#132] -Keys [3]: [channel#130, id#131, spark_grouping_id#132] -Functions [3]: [partial_sum(sales#38), partial_sum(returns#39), partial_sum(profit#40)] -Aggregate Attributes [6]: [sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -Results [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] - -(78) CometColumnarExchange -Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] -Arguments: hashpartitioning(channel#130, id#131, spark_grouping_id#132, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(79) CometColumnarToRow [codegen id : 21] -Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] - -(80) HashAggregate [codegen id : 21] -Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] -Keys [3]: [channel#130, id#131, spark_grouping_id#132] -Functions [3]: [sum(sales#38), sum(returns#39), sum(profit#40)] -Aggregate Attributes [3]: [sum(sales#38)#145, sum(returns#39)#146, sum(profit#40)#147] -Results [5]: [channel#130, id#131, sum(sales#38)#145 AS sales#148, sum(returns#39)#146 AS returns#149, sum(profit#40)#147 AS profit#150] - -(81) TakeOrderedAndProject -Input [5]: [channel#130, id#131, sales#148, returns#149, profit#150] -Arguments: 100, [channel#130 ASC NULLS FIRST, id#131 ASC NULLS FIRST], [channel#130, id#131, sales#148, returns#149, profit#150] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (86) -+- * CometColumnarToRow (85) - +- CometProject (84) - +- CometFilter (83) - +- CometNativeScan parquet spark_catalog.default.date_dim (82) - - -(82) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#151] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] -ReadSchema: struct - -(83) CometFilter -Input [2]: [d_date_sk#22, d_date#151] -Condition : (((isnotnull(d_date#151) AND (d_date#151 >= 2000-08-23)) AND (d_date#151 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) - -(84) CometProject -Input [2]: [d_date_sk#22, d_date#151] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(85) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(86) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 28 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#87 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/simplified.txt deleted file mode 100644 index 9dc4fc19c7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/simplified.txt +++ /dev/null @@ -1,137 +0,0 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (21) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (20) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [s_store_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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_id] #2 - WholeStageCodegen (5) - HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,s_store_id] - BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (1) - Project [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - WholeStageCodegen (2) - Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] - Filter [sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (12) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cp_catalog_page_id] #5 - WholeStageCodegen (11) - HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (7) - Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] - Filter [cs_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (8) - Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] - Filter [cr_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (19) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [web_site_id] #7 - WholeStageCodegen (18) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (13) - Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] - Filter [ws_web_site_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (15) - Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (14) - ColumnarToRow - InputAdapter - Scan 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 - CometColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (17) - CometColumnarToRow - InputAdapter - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan 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/q5.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/explain.txt deleted file mode 100644 index b4a1dedd61..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,452 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (71) -+- CometTakeOrderedAndProject (70) - +- CometHashAggregate (69) - +- CometExchange (68) - +- CometHashAggregate (67) - +- CometExpand (66) - +- CometUnion (65) - :- CometHashAggregate (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometUnion (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - :- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (32) - : : +- CometBroadcastHashJoin (31) - : : :- CometUnion (29) - : : : :- CometProject (25) - : : : : +- CometFilter (24) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) - : : : +- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) - : : +- ReusedExchange (30) - : +- CometBroadcastExchange (36) - : +- CometProject (35) - : +- CometFilter (34) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (55) - : +- CometBroadcastHashJoin (54) - : :- CometUnion (52) - : : :- CometProject (44) - : : : +- CometFilter (43) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : : +- CometProject (51) - : : +- CometBroadcastHashJoin (50) - : : :- CometBroadcastExchange (46) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) - : : +- CometProject (49) - : : +- CometFilter (48) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : +- ReusedExchange (53) - +- CometBroadcastExchange (59) - +- CometProject (58) - +- CometFilter (57) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Arguments: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11], [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#12) - -(6) CometProject -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Arguments: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21], [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] - -(7) CometUnion -Child 0 Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) - -(10) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(11) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(12) CometBroadcastHashJoin -Left output [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [1]: [d_date_sk#22] -Arguments: [date_sk#7], [d_date_sk#22], Inner, BuildRight - -(13) CometProject -Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] -Arguments: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11], [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_store_id#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#24, s_store_id#25] -Condition : isnotnull(s_store_sk#24) - -(16) CometProject -Input [2]: [s_store_sk#24, s_store_id#25] -Arguments: [s_store_sk#24, s_store_id#26], [s_store_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#25, 16, true, false, true) AS s_store_id#26] - -(17) CometBroadcastExchange -Input [2]: [s_store_sk#24, s_store_id#26] -Arguments: [s_store_sk#24, s_store_id#26] - -(18) CometBroadcastHashJoin -Left output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [2]: [s_store_sk#24, s_store_id#26] -Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight - -(19) CometProject -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#26] -Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] - -(20) CometHashAggregate -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] -Keys [1]: [s_store_id#26] -Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] - -(21) CometExchange -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Keys [1]: [s_store_id#26] -Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk)] -ReadSchema: struct - -(24) CometFilter -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : isnotnull(cs_catalog_page_sk#31) - -(25) CometProject -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cr_catalog_page_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Condition : isnotnull(cr_catalog_page_sk#42) - -(28) CometProject -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] - -(29) CometUnion -Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] - -(30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#52] - -(31) CometBroadcastHashJoin -Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [1]: [d_date_sk#52] -Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight - -(32) CometProject -Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] -Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Condition : isnotnull(cp_catalog_page_sk#53) - -(35) CometProject -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#54, 16, true, false, true) AS cp_catalog_page_id#55] - -(36) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] - -(37) CometBroadcastHashJoin -Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight - -(38) CometProject -Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] - -(39) CometHashAggregate -Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] - -(40) CometExchange -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(41) CometHashAggregate -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] -PushedFilters: [IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(43) CometFilter -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_web_site_sk#60) - -(44) CometProject -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] -ReadSchema: struct - -(46) CometBroadcastExchange -Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(48) CometFilter -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) - -(49) CometProject -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] - -(50) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft - -(51) CometProject -Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] - -(52) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] - -(53) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#86] - -(54) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [1]: [d_date_sk#86] -Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight - -(55) CometProject -Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] -Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#87, web_site_id#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(57) CometFilter -Input [2]: [web_site_sk#87, web_site_id#88] -Condition : isnotnull(web_site_sk#87) - -(58) CometProject -Input [2]: [web_site_sk#87, web_site_id#88] -Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#88, 16, true, false, true) AS web_site_id#89] - -(59) CometBroadcastExchange -Input [2]: [web_site_sk#87, web_site_id#89] -Arguments: [web_site_sk#87, web_site_id#89] - -(60) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [2]: [web_site_sk#87, web_site_id#89] -Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight - -(61) CometProject -Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] -Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] - -(62) CometHashAggregate -Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] -Keys [1]: [web_site_id#89] -Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] - -(63) CometExchange -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(64) CometHashAggregate -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Keys [1]: [web_site_id#89] -Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] - -(65) CometUnion -Child 0 Input [5]: [sales#94, returns#95, profit#96, channel#97, id#98] -Child 1 Input [5]: [sales#99, returns#100, profit#101, channel#102, id#103] -Child 2 Input [5]: [sales#104, returns#105, profit#106, channel#107, id#108] - -(66) CometExpand -Input [5]: [sales#94, returns#95, profit#96, channel#97, id#98] -Arguments: [[sales#94, returns#95, profit#96, channel#97, id#98, 0], [sales#94, returns#95, profit#96, channel#97, null, 1], [sales#94, returns#95, profit#96, null, null, 3]], [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] - -(67) CometHashAggregate -Input [6]: [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] -Keys [3]: [channel#109, id#110, spark_grouping_id#111] -Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] - -(68) CometExchange -Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Arguments: hashpartitioning(channel#109, id#110, spark_grouping_id#111, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(69) CometHashAggregate -Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Keys [3]: [channel#109, id#110, spark_grouping_id#111] -Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] - -(70) CometTakeOrderedAndProject -Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#109 ASC NULLS FIRST,id#110 ASC NULLS FIRST], output=[channel#109,id#110,sales#118,returns#119,profit#120]), [channel#109, id#110, sales#118, returns#119, profit#120], 100, 0, [channel#109 ASC NULLS FIRST, id#110 ASC NULLS FIRST], [channel#109, id#110, sales#118, returns#119, profit#120] - -(71) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) - -(74) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(75) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(76) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/simplified.txt deleted file mode 100644 index 6fff89a25c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,86 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id,spark_grouping_id] #1 - CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] - CometUnion [sales,returns,profit,channel,id] - CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),s_store_id] - CometExchange [s_store_id] #2 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - 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 [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,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),cp_catalog_page_id] - CometExchange [cp_catalog_page_id] #6 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - 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 [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,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [cp_catalog_page_sk,cp_catalog_page_id] #7 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),web_site_id] - CometExchange [web_site_id] #8 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - 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 [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,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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,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 [native_iceberg_compat] 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_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 [web_site_sk,web_site_id] #10 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] 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/q5/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/explain.txt deleted file mode 100644 index b4a1dedd61..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/explain.txt +++ /dev/null @@ -1,452 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (71) -+- CometTakeOrderedAndProject (70) - +- CometHashAggregate (69) - +- CometExchange (68) - +- CometHashAggregate (67) - +- CometExpand (66) - +- CometUnion (65) - :- CometHashAggregate (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometUnion (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - :- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (32) - : : +- CometBroadcastHashJoin (31) - : : :- CometUnion (29) - : : : :- CometProject (25) - : : : : +- CometFilter (24) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) - : : : +- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) - : : +- ReusedExchange (30) - : +- CometBroadcastExchange (36) - : +- CometProject (35) - : +- CometFilter (34) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (55) - : +- CometBroadcastHashJoin (54) - : :- CometUnion (52) - : : :- CometProject (44) - : : : +- CometFilter (43) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : : +- CometProject (51) - : : +- CometBroadcastHashJoin (50) - : : :- CometBroadcastExchange (46) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) - : : +- CometProject (49) - : : +- CometFilter (48) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : +- ReusedExchange (53) - +- CometBroadcastExchange (59) - +- CometProject (58) - +- CometFilter (57) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Arguments: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11], [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#12) - -(6) CometProject -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Arguments: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21], [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] - -(7) CometUnion -Child 0 Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) - -(10) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(11) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(12) CometBroadcastHashJoin -Left output [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [1]: [d_date_sk#22] -Arguments: [date_sk#7], [d_date_sk#22], Inner, BuildRight - -(13) CometProject -Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] -Arguments: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11], [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_store_id#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#24, s_store_id#25] -Condition : isnotnull(s_store_sk#24) - -(16) CometProject -Input [2]: [s_store_sk#24, s_store_id#25] -Arguments: [s_store_sk#24, s_store_id#26], [s_store_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#25, 16, true, false, true) AS s_store_id#26] - -(17) CometBroadcastExchange -Input [2]: [s_store_sk#24, s_store_id#26] -Arguments: [s_store_sk#24, s_store_id#26] - -(18) CometBroadcastHashJoin -Left output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [2]: [s_store_sk#24, s_store_id#26] -Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight - -(19) CometProject -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#26] -Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] - -(20) CometHashAggregate -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] -Keys [1]: [s_store_id#26] -Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] - -(21) CometExchange -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Keys [1]: [s_store_id#26] -Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk)] -ReadSchema: struct - -(24) CometFilter -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : isnotnull(cs_catalog_page_sk#31) - -(25) CometProject -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cr_catalog_page_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Condition : isnotnull(cr_catalog_page_sk#42) - -(28) CometProject -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] - -(29) CometUnion -Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] - -(30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#52] - -(31) CometBroadcastHashJoin -Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [1]: [d_date_sk#52] -Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight - -(32) CometProject -Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] -Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Condition : isnotnull(cp_catalog_page_sk#53) - -(35) CometProject -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#54, 16, true, false, true) AS cp_catalog_page_id#55] - -(36) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] - -(37) CometBroadcastHashJoin -Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight - -(38) CometProject -Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] - -(39) CometHashAggregate -Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] - -(40) CometExchange -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(41) CometHashAggregate -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] -PushedFilters: [IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(43) CometFilter -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_web_site_sk#60) - -(44) CometProject -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] -ReadSchema: struct - -(46) CometBroadcastExchange -Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(48) CometFilter -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) - -(49) CometProject -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] - -(50) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft - -(51) CometProject -Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] - -(52) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] - -(53) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#86] - -(54) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [1]: [d_date_sk#86] -Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight - -(55) CometProject -Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] -Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#87, web_site_id#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(57) CometFilter -Input [2]: [web_site_sk#87, web_site_id#88] -Condition : isnotnull(web_site_sk#87) - -(58) CometProject -Input [2]: [web_site_sk#87, web_site_id#88] -Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#88, 16, true, false, true) AS web_site_id#89] - -(59) CometBroadcastExchange -Input [2]: [web_site_sk#87, web_site_id#89] -Arguments: [web_site_sk#87, web_site_id#89] - -(60) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [2]: [web_site_sk#87, web_site_id#89] -Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight - -(61) CometProject -Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] -Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] - -(62) CometHashAggregate -Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] -Keys [1]: [web_site_id#89] -Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] - -(63) CometExchange -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(64) CometHashAggregate -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Keys [1]: [web_site_id#89] -Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] - -(65) CometUnion -Child 0 Input [5]: [sales#94, returns#95, profit#96, channel#97, id#98] -Child 1 Input [5]: [sales#99, returns#100, profit#101, channel#102, id#103] -Child 2 Input [5]: [sales#104, returns#105, profit#106, channel#107, id#108] - -(66) CometExpand -Input [5]: [sales#94, returns#95, profit#96, channel#97, id#98] -Arguments: [[sales#94, returns#95, profit#96, channel#97, id#98, 0], [sales#94, returns#95, profit#96, channel#97, null, 1], [sales#94, returns#95, profit#96, null, null, 3]], [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] - -(67) CometHashAggregate -Input [6]: [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] -Keys [3]: [channel#109, id#110, spark_grouping_id#111] -Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] - -(68) CometExchange -Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Arguments: hashpartitioning(channel#109, id#110, spark_grouping_id#111, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(69) CometHashAggregate -Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Keys [3]: [channel#109, id#110, spark_grouping_id#111] -Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] - -(70) CometTakeOrderedAndProject -Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#109 ASC NULLS FIRST,id#110 ASC NULLS FIRST], output=[channel#109,id#110,sales#118,returns#119,profit#120]), [channel#109, id#110, sales#118, returns#119, profit#120], 100, 0, [channel#109 ASC NULLS FIRST, id#110 ASC NULLS FIRST], [channel#109, id#110, sales#118, returns#119, profit#120] - -(71) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) - -(74) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(75) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(76) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/extended.txt deleted file mode 100644 index d128acaf06..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/extended.txt +++ /dev/null @@ -1,90 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - -Comet accelerated 79 out of 86 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 6fff89a25c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt +++ /dev/null @@ -1,86 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id,spark_grouping_id] #1 - CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] - CometUnion [sales,returns,profit,channel,id] - CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),s_store_id] - CometExchange [s_store_id] #2 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - 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 [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,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),cp_catalog_page_id] - CometExchange [cp_catalog_page_id] #6 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - 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 [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,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [cp_catalog_page_sk,cp_catalog_page_id] #7 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),web_site_id] - CometExchange [web_site_id] #8 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - 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 [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,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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,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 [native_iceberg_compat] 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_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 [web_site_sk,web_site_id] #10 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/explain.txt deleted file mode 100644 index de9a1cd3ec..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (30) -+- * HashAggregate (29) - +- * CometColumnarToRow (28) - +- CometColumnarExchange (27) - +- * HashAggregate (26) - +- * Project (25) - +- * BroadcastHashJoin Inner BuildRight (24) - :- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet spark_catalog.default.store_returns (4) - : : +- BroadcastExchange (14) - : : +- * CometColumnarToRow (13) - : : +- CometProject (12) - : : +- CometFilter (11) - : : +- CometNativeScan parquet spark_catalog.default.store (10) - : +- BroadcastExchange (20) - : +- * CometColumnarToRow (19) - : +- CometFilter (18) - : +- CometNativeScan parquet spark_catalog.default.date_dim (17) - +- ReusedExchange (23) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#5)] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_store_sk#3)) - -(3) CometColumnarToRow [codegen id : 5] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] - -(4) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#9), dynamicpruningexpression(sr_returned_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk), IsNotNull(sr_customer_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] - -(6) Filter [codegen id : 1] -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : ((isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#6)) AND isnotnull(sr_customer_sk#7)) - -(7) BroadcastExchange -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(input[2, int, false], input[0, int, false], input[1, int, false]),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 5] -Left keys [3]: [ss_ticket_number#4, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [sr_ticket_number#8, sr_item_sk#6, sr_customer_sk#7] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 5] -Output [3]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9] -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] - -(10) CometNativeScan parquet spark_catalog.default.store -Output [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(11) CometFilter -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Condition : isnotnull(s_store_sk#11) - -(12) CometProject -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Arguments: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_street_type#16, 15, true, false, true) AS s_street_type#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_suite_number#17, 10, true, false, true) AS s_suite_number#23, s_city#18, s_county#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#20, 2, true, false, true) AS s_state#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#21, 10, true, false, true) AS s_zip#25] - -(13) CometColumnarToRow [codegen id : 2] -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(14) BroadcastExchange -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None - -(16) Project [codegen id : 5] -Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Input [14]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(17) CometNativeScan parquet spark_catalog.default.date_dim -Output [1]: [d_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) CometFilter -Input [1]: [d_date_sk#26] -Condition : isnotnull(d_date_sk#26) - -(19) CometColumnarToRow [codegen id : 3] -Input [1]: [d_date_sk#26] - -(20) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#26] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 5] -Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#26] - -(23) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#27] - -(24) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [sr_returned_date_sk#9] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 5] -Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#27] - -(26) HashAggregate [codegen id : 5] -Input [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Functions [5]: [partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] -Aggregate Attributes [5]: [sum#28, sum#29, sum#30, sum#31, sum#32] -Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#33, sum#34, sum#35, sum#36, sum#37] - -(27) CometColumnarExchange -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#33, sum#34, sum#35, sum#36, sum#37] -Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(28) CometColumnarToRow [codegen id : 6] -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#33, sum#34, sum#35, sum#36, sum#37] - -(29) HashAggregate [codegen id : 6] -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#33, sum#34, sum#35, sum#36, sum#37] -Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Functions [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] -Aggregate Attributes [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#38, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#39, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#40, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#41, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#42] -Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#38 AS 30 days #43, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#39 AS 31 - 60 days #44, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#40 AS 61 - 90 days #45, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#41 AS 91 - 120 days #46, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#42 AS >120 days #47] - -(30) TakeOrderedAndProject -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #43, 31 - 60 days #44, 61 - 90 days #45, 91 - 120 days #46, >120 days #47] -Arguments: 100, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRST, s_street_number#14 ASC NULLS FIRST, s_street_name#15 ASC NULLS FIRST, s_street_type#22 ASC NULLS FIRST, s_suite_number#23 ASC NULLS FIRST, s_city#18 ASC NULLS FIRST, s_county#19 ASC NULLS FIRST, s_state#24 ASC NULLS FIRST, s_zip#25 ASC NULLS FIRST], [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #43, 31 - 60 days #44, 61 - 90 days #45, 91 - 120 days #46, >120 days #47] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometNativeScan parquet spark_catalog.default.date_dim (31) - - -(31) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#48, d_moy#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [d_date_sk#27, d_year#48, d_moy#49] -Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#49)) AND (d_year#48 = 2001)) AND (d_moy#49 = 8)) AND isnotnull(d_date_sk#27)) - -(33) CometProject -Input [3]: [d_date_sk#27, d_year#48, d_moy#49] -Arguments: [d_date_sk#27], [d_date_sk#27] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#27] - -(35) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/simplified.txt deleted file mode 100644 index 186c041b09..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -TakeOrderedAndProject [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,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - WholeStageCodegen (6) - HashAggregate [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] [sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 30) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 60) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 90) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) > 120) THEN 1 ELSE 0 END),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [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] #1 - WholeStageCodegen (5) - HashAggregate [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] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [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] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [sr_ticket_number,sr_item_sk,sr_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_street_type,s_suite_number,s_state,s_zip] [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] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/explain.txt deleted file mode 100644 index ff37c8f5fb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,203 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5)] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#9), dynamicpruningexpression(sr_returned_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk), IsNotNull(sr_customer_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : ((isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#6)) AND isnotnull(sr_customer_sk#7)) - -(5) CometBroadcastExchange -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] - -(6) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [ss_ticket_number#4, ss_item_sk#1, ss_customer_sk#2], [sr_ticket_number#8, sr_item_sk#6, sr_customer_sk#7], Inner, BuildRight - -(7) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9], [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(9) CometFilter -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Condition : isnotnull(s_store_sk#11) - -(10) CometProject -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Arguments: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_street_type#16, 15, true, false, true) AS s_street_type#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_suite_number#17, 10, true, false, true) AS s_suite_number#23, s_city#18, s_county#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#20, 2, true, false, true) AS s_state#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#21, 10, true, false, true) AS s_zip#25] - -(11) CometBroadcastExchange -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Arguments: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(12) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9] -Right output [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Arguments: [ss_store_sk#3], [s_store_sk#11], Inner, BuildRight - -(13) CometProject -Input [14]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [1]: [d_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [1]: [d_date_sk#26] -Condition : isnotnull(d_date_sk#26) - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] - -(17) CometBroadcastHashJoin -Left output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#5], [d_date_sk#26], Inner, BuildRight - -(18) CometProject -Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#26] -Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_moy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Condition : ((((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2001)) AND (d_moy#29 = 8)) AND isnotnull(d_date_sk#27)) - -(21) CometProject -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Arguments: [d_date_sk#27], [d_date_sk#27] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: [d_date_sk#27] - -(23) CometBroadcastHashJoin -Left output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Right output [1]: [d_date_sk#27] -Arguments: [sr_returned_date_sk#9], [d_date_sk#27], Inner, BuildRight - -(24) CometProject -Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#27] -Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(25) CometHashAggregate -Input [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Functions [5]: [partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#30, sum#31, sum#32, sum#33, sum#34] -Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#30, sum#31, sum#32, sum#33, sum#34] -Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Functions [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#12 ASC NULLS FIRST,s_company_id#13 ASC NULLS FIRST,s_street_number#14 ASC NULLS FIRST,s_street_name#15 ASC NULLS FIRST,s_street_type#22 ASC NULLS FIRST,s_suite_number#23 ASC NULLS FIRST,s_city#18 ASC NULLS FIRST,s_county#19 ASC NULLS FIRST,s_state#24 ASC NULLS FIRST,s_zip#25 ASC NULLS FIRST], output=[s_store_name#12,s_company_id#13,s_street_number#14,s_street_name#15,s_street_type#22,s_suite_number#23,s_city#18,s_county#19,s_state#24,s_zip#25,30 days #35,31 - 60 days #36,61 - 90 days #37,91 - 120 days #38,>120 days #39]), [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39], 100, 0, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRST, s_street_number#14 ASC NULLS FIRST, s_street_name#15 ASC NULLS FIRST, s_street_type#22 ASC NULLS FIRST, s_suite_number#23 ASC NULLS FIRST, s_city#18 ASC NULLS FIRST, s_county#19 ASC NULLS FIRST, s_state#24 ASC NULLS FIRST, s_zip#25 ASC NULLS FIRST], [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39] - -(29) CometColumnarToRow [codegen id : 1] -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_moy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Condition : ((((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2001)) AND (d_moy#29 = 8)) AND isnotnull(d_date_sk#27)) - -(32) CometProject -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Arguments: [d_date_sk#27], [d_date_sk#27] - -(33) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#27] - -(34) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/simplified.txt deleted file mode 100644 index 98a44eaf40..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,39 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [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,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [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,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 30) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 60) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 90) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [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] #1 - CometHashAggregate [sr_returned_date_sk,ss_sold_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,sum,sum,sum,sum,sum] - 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,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,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,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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 - CometProject [s_street_type,s_suite_number,s_state,s_zip] [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] - 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 [native_iceberg_compat] 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 [d_date_sk] #5 - CometFilter [d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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/q50/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/explain.txt deleted file mode 100644 index ff37c8f5fb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/explain.txt +++ /dev/null @@ -1,203 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5)] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#9), dynamicpruningexpression(sr_returned_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk), IsNotNull(sr_customer_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : ((isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#6)) AND isnotnull(sr_customer_sk#7)) - -(5) CometBroadcastExchange -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] - -(6) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [ss_ticket_number#4, ss_item_sk#1, ss_customer_sk#2], [sr_ticket_number#8, sr_item_sk#6, sr_customer_sk#7], Inner, BuildRight - -(7) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9], [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(9) CometFilter -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Condition : isnotnull(s_store_sk#11) - -(10) CometProject -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Arguments: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_street_type#16, 15, true, false, true) AS s_street_type#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_suite_number#17, 10, true, false, true) AS s_suite_number#23, s_city#18, s_county#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#20, 2, true, false, true) AS s_state#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#21, 10, true, false, true) AS s_zip#25] - -(11) CometBroadcastExchange -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Arguments: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(12) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9] -Right output [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Arguments: [ss_store_sk#3], [s_store_sk#11], Inner, BuildRight - -(13) CometProject -Input [14]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [1]: [d_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [1]: [d_date_sk#26] -Condition : isnotnull(d_date_sk#26) - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] - -(17) CometBroadcastHashJoin -Left output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#5], [d_date_sk#26], Inner, BuildRight - -(18) CometProject -Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#26] -Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_moy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Condition : ((((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2001)) AND (d_moy#29 = 8)) AND isnotnull(d_date_sk#27)) - -(21) CometProject -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Arguments: [d_date_sk#27], [d_date_sk#27] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: [d_date_sk#27] - -(23) CometBroadcastHashJoin -Left output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Right output [1]: [d_date_sk#27] -Arguments: [sr_returned_date_sk#9], [d_date_sk#27], Inner, BuildRight - -(24) CometProject -Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#27] -Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(25) CometHashAggregate -Input [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Functions [5]: [partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#30, sum#31, sum#32, sum#33, sum#34] -Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#30, sum#31, sum#32, sum#33, sum#34] -Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Functions [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#12 ASC NULLS FIRST,s_company_id#13 ASC NULLS FIRST,s_street_number#14 ASC NULLS FIRST,s_street_name#15 ASC NULLS FIRST,s_street_type#22 ASC NULLS FIRST,s_suite_number#23 ASC NULLS FIRST,s_city#18 ASC NULLS FIRST,s_county#19 ASC NULLS FIRST,s_state#24 ASC NULLS FIRST,s_zip#25 ASC NULLS FIRST], output=[s_store_name#12,s_company_id#13,s_street_number#14,s_street_name#15,s_street_type#22,s_suite_number#23,s_city#18,s_county#19,s_state#24,s_zip#25,30 days #35,31 - 60 days #36,61 - 90 days #37,91 - 120 days #38,>120 days #39]), [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39], 100, 0, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRST, s_street_number#14 ASC NULLS FIRST, s_street_name#15 ASC NULLS FIRST, s_street_type#22 ASC NULLS FIRST, s_suite_number#23 ASC NULLS FIRST, s_city#18 ASC NULLS FIRST, s_county#19 ASC NULLS FIRST, s_state#24 ASC NULLS FIRST, s_zip#25 ASC NULLS FIRST], [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39] - -(29) CometColumnarToRow [codegen id : 1] -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_moy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Condition : ((((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2001)) AND (d_moy#29 = 8)) AND isnotnull(d_date_sk#27)) - -(32) CometProject -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Arguments: [d_date_sk#27], [d_date_sk#27] - -(33) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#27] - -(34) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/extended.txt deleted file mode 100644 index 0fbba7d1e8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 98a44eaf40..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50/simplified.txt +++ /dev/null @@ -1,39 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [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,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [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,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 30) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 60) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 90) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [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] #1 - CometHashAggregate [sr_returned_date_sk,ss_sold_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,sum,sum,sum,sum,sum] - 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,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,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,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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 - CometProject [s_street_type,s_suite_number,s_state,s_zip] [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] - 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 [native_iceberg_compat] 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 [d_date_sk] #5 - CometFilter [d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/explain.txt deleted file mode 100644 index 9af40029c8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/explain.txt +++ /dev/null @@ -1,264 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (42) -+- * Filter (41) - +- Window (40) - +- * CometColumnarToRow (39) - +- CometSort (38) - +- CometExchange (37) - +- CometProject (36) - +- CometSortMergeJoin (35) - :- CometSort (17) - : +- CometColumnarExchange (16) - : +- * Project (15) - : +- Window (14) - : +- * CometColumnarToRow (13) - : +- CometSort (12) - : +- CometColumnarExchange (11) - : +- * HashAggregate (10) - : +- * CometColumnarToRow (9) - : +- CometColumnarExchange (8) - : +- * HashAggregate (7) - : +- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.web_sales (1) - : +- ReusedExchange (4) - +- CometSort (34) - +- CometColumnarExchange (33) - +- * Project (32) - +- Window (31) - +- * CometColumnarToRow (30) - +- CometSort (29) - +- CometColumnarExchange (28) - +- * HashAggregate (27) - +- * CometColumnarToRow (26) - +- CometColumnarExchange (25) - +- * HashAggregate (24) - +- * Project (23) - +- * BroadcastHashJoin Inner BuildRight (22) - :- * Filter (20) - : +- * ColumnarToRow (19) - : +- Scan parquet spark_catalog.default.store_sales (18) - +- ReusedExchange (21) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] - -(3) Filter [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(4) ReusedExchange [Reuses operator id: 47] -Output [2]: [d_date_sk#5, d_date#6] - -(5) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 2] -Output [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] - -(7) HashAggregate [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum#7] -Results [3]: [ws_item_sk#1, d_date#6, sum#8] - -(8) CometColumnarExchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(9) CometColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, d_date#6, sum#8] - -(10) HashAggregate [codegen id : 3] -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#9] -Results [4]: [ws_item_sk#1 AS item_sk#10, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#9,17,2) AS _w0#11, ws_item_sk#1] - -(11) CometColumnarExchange -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(12) CometSort -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(13) CometColumnarToRow [codegen id : 4] -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] - -(14) Window -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: [sum(_w0#11) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(15) Project [codegen id : 5] -Output [3]: [item_sk#10, d_date#6, cume_sales#12] -Input [5]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1, cume_sales#12] - -(16) CometColumnarExchange -Input [3]: [item_sk#10, d_date#6, cume_sales#12] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(17) CometSort -Input [3]: [item_sk#10, d_date#6, cume_sales#12] -Arguments: [item_sk#10, d_date#6, cume_sales#12], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(18) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_sold_date_sk#15 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 7] -Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] - -(20) Filter [codegen id : 7] -Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -Condition : isnotnull(ss_item_sk#13) - -(21) ReusedExchange [Reuses operator id: 47] -Output [2]: [d_date_sk#16, d_date#17] - -(22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#15] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None - -(23) Project [codegen id : 7] -Output [3]: [ss_item_sk#13, ss_sales_price#14, d_date#17] -Input [5]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#16, d_date#17] - -(24) HashAggregate [codegen id : 7] -Input [3]: [ss_item_sk#13, ss_sales_price#14, d_date#17] -Keys [2]: [ss_item_sk#13, d_date#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [ss_item_sk#13, d_date#17, sum#19] - -(25) CometColumnarExchange -Input [3]: [ss_item_sk#13, d_date#17, sum#19] -Arguments: hashpartitioning(ss_item_sk#13, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(26) CometColumnarToRow [codegen id : 8] -Input [3]: [ss_item_sk#13, d_date#17, sum#19] - -(27) HashAggregate [codegen id : 8] -Input [3]: [ss_item_sk#13, d_date#17, sum#19] -Keys [2]: [ss_item_sk#13, d_date#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#14))#20] -Results [4]: [ss_item_sk#13 AS item_sk#21, d_date#17, MakeDecimal(sum(UnscaledValue(ss_sales_price#14))#20,17,2) AS _w0#22, ss_item_sk#13] - -(28) CometColumnarExchange -Input [4]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13] -Arguments: hashpartitioning(ss_item_sk#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(29) CometSort -Input [4]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13] -Arguments: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13], [ss_item_sk#13 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] - -(30) CometColumnarToRow [codegen id : 9] -Input [4]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13] - -(31) Window -Input [4]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13] -Arguments: [sum(_w0#22) windowspecdefinition(ss_item_sk#13, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#23], [ss_item_sk#13], [d_date#17 ASC NULLS FIRST] - -(32) Project [codegen id : 10] -Output [3]: [item_sk#21, d_date#17, cume_sales#23] -Input [5]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13, cume_sales#23] - -(33) CometColumnarExchange -Input [3]: [item_sk#21, d_date#17, cume_sales#23] -Arguments: hashpartitioning(item_sk#21, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(34) CometSort -Input [3]: [item_sk#21, d_date#17, cume_sales#23] -Arguments: [item_sk#21, d_date#17, cume_sales#23], [item_sk#21 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] - -(35) CometSortMergeJoin -Left output [3]: [item_sk#10, d_date#6, cume_sales#12] -Right output [3]: [item_sk#21, d_date#17, cume_sales#23] -Arguments: [item_sk#10, d_date#6], [item_sk#21, d_date#17], FullOuter - -(36) CometProject -Input [6]: [item_sk#10, d_date#6, cume_sales#12, item_sk#21, d_date#17, cume_sales#23] -Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#21 END AS item_sk#24, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#17 END AS d_date#25, cume_sales#12 AS web_sales#26, cume_sales#23 AS store_sales#27] - -(37) CometExchange -Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] -Arguments: hashpartitioning(item_sk#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(38) CometSort -Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] -Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27], [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST] - -(39) CometColumnarToRow [codegen id : 11] -Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] - -(40) Window -Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] -Arguments: [max(web_sales#26) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#28, max(store_sales#27) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#29], [item_sk#24], [d_date#25 ASC NULLS FIRST] - -(41) Filter [codegen id : 12] -Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] -Condition : ((isnotnull(web_cumulative#28) AND isnotnull(store_cumulative#29)) AND (web_cumulative#28 > store_cumulative#29)) - -(42) TakeOrderedAndProject -Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] -Arguments: 100, [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST], [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (47) -+- * CometColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometNativeScan parquet spark_catalog.default.date_dim (43) - - -(43) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(44) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#30] -Condition : (((isnotnull(d_month_seq#30) AND (d_month_seq#30 >= 1200)) AND (d_month_seq#30 <= 1211)) AND isnotnull(d_date_sk#5)) - -(45) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#30] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(46) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] - -(47) BroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/simplified.txt deleted file mode 100644 index 51d3f11ae6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (12) - Filter [web_cumulative,store_cumulative] - InputAdapter - Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (5) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ws_item_sk] - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk,d_date] #4 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #6 - WholeStageCodegen (10) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ss_item_sk] - CometColumnarExchange [ss_item_sk] #7 - WholeStageCodegen (8) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk,d_date] #8 - WholeStageCodegen (7) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/explain.txt deleted file mode 100644 index 683cc25d24..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,257 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (41) -+- * Filter (40) - +- Window (39) - +- * CometColumnarToRow (38) - +- CometSort (37) - +- CometExchange (36) - +- CometProject (35) - +- CometSortMergeJoin (34) - :- CometSort (18) - : +- CometColumnarExchange (17) - : +- * Project (16) - : +- Window (15) - : +- * CometColumnarToRow (14) - : +- CometSort (13) - : +- CometExchange (12) - : +- CometHashAggregate (11) - : +- CometExchange (10) - : +- CometHashAggregate (9) - : +- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- CometSort (33) - +- CometColumnarExchange (32) - +- * Project (31) - +- Window (30) - +- * CometColumnarToRow (29) - +- CometSort (28) - +- CometExchange (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometBroadcastHashJoin (22) - :- CometFilter (20) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (19) - +- ReusedExchange (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: [d_date_sk#5, d_date#6] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Right output [2]: [d_date_sk#5, d_date#6] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] -Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] - -(9) CometHashAggregate -Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] - -(10) CometExchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] - -(12) CometExchange -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(13) CometSort -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(14) CometColumnarToRow [codegen id : 1] -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] - -(15) Window -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: [sum(_w0#10) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(16) Project [codegen id : 2] -Output [3]: [item_sk#9, d_date#6, cume_sales#11] -Input [5]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1, cume_sales#11] - -(17) CometColumnarExchange -Input [3]: [item_sk#9, d_date#6, cume_sales#11] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(18) CometSort -Input [3]: [item_sk#9, d_date#6, cume_sales#11] -Arguments: [item_sk#9, d_date#6, cume_sales#11], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#14), dynamicpruningexpression(ss_sold_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] -Condition : isnotnull(ss_item_sk#12) - -(21) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#16, d_date#17] - -(22) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] -Right output [2]: [d_date_sk#16, d_date#17] -Arguments: [ss_sold_date_sk#14], [d_date_sk#16], Inner, BuildRight - -(23) CometProject -Input [5]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14, d_date_sk#16, d_date#17] -Arguments: [ss_item_sk#12, ss_sales_price#13, d_date#17], [ss_item_sk#12, ss_sales_price#13, d_date#17] - -(24) CometHashAggregate -Input [3]: [ss_item_sk#12, ss_sales_price#13, d_date#17] -Keys [2]: [ss_item_sk#12, d_date#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#13))] - -(25) CometExchange -Input [3]: [ss_item_sk#12, d_date#17, sum#18] -Arguments: hashpartitioning(ss_item_sk#12, d_date#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(26) CometHashAggregate -Input [3]: [ss_item_sk#12, d_date#17, sum#18] -Keys [2]: [ss_item_sk#12, d_date#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#13))] - -(27) CometExchange -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -Arguments: hashpartitioning(ss_item_sk#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(28) CometSort -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -Arguments: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12], [ss_item_sk#12 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] - -(29) CometColumnarToRow [codegen id : 3] -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] - -(30) Window -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -Arguments: [sum(_w0#20) windowspecdefinition(ss_item_sk#12, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#21], [ss_item_sk#12], [d_date#17 ASC NULLS FIRST] - -(31) Project [codegen id : 4] -Output [3]: [item_sk#19, d_date#17, cume_sales#21] -Input [5]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12, cume_sales#21] - -(32) CometColumnarExchange -Input [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: hashpartitioning(item_sk#19, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(33) CometSort -Input [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: [item_sk#19, d_date#17, cume_sales#21], [item_sk#19 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] - -(34) CometSortMergeJoin -Left output [3]: [item_sk#9, d_date#6, cume_sales#11] -Right output [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: [item_sk#9, d_date#6], [item_sk#19, d_date#17], FullOuter - -(35) CometProject -Input [6]: [item_sk#9, d_date#6, cume_sales#11, item_sk#19, d_date#17, cume_sales#21] -Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#19 END AS item_sk#22, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#17 END AS d_date#23, cume_sales#11 AS web_sales#24, cume_sales#21 AS store_sales#25] - -(36) CometExchange -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: hashpartitioning(item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(37) CometSort -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST] - -(38) CometColumnarToRow [codegen id : 5] -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] - -(39) Window -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: [max(web_sales#24) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#26, max(store_sales#25) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#27], [item_sk#22], [d_date#23 ASC NULLS FIRST] - -(40) Filter [codegen id : 6] -Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] -Condition : ((isnotnull(web_cumulative#26) AND isnotnull(store_cumulative#27)) AND (web_cumulative#26 > store_cumulative#27)) - -(41) TakeOrderedAndProject -Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] -Arguments: 100, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) - - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) - -(44) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(45) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] - -(46) BroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/simplified.txt deleted file mode 100644 index a5af2e114b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,62 +0,0 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (6) - Filter [web_cumulative,store_cumulative] - InputAdapter - Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (2) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ws_item_sk] - CometExchange [ws_item_sk] #3 - CometHashAggregate [sum] [item_sk,d_date,_w0,ws_item_sk,sum(UnscaledValue(ws_sales_price))] - CometExchange [ws_item_sk,d_date] #4 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #6 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #7 - WholeStageCodegen (4) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ss_item_sk] - CometExchange [ss_item_sk] #8 - CometHashAggregate [sum] [item_sk,d_date,_w0,ss_item_sk,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_item_sk,d_date] #9 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - 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 [native_iceberg_compat] 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/q51/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/explain.txt deleted file mode 100644 index 683cc25d24..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/explain.txt +++ /dev/null @@ -1,257 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (41) -+- * Filter (40) - +- Window (39) - +- * CometColumnarToRow (38) - +- CometSort (37) - +- CometExchange (36) - +- CometProject (35) - +- CometSortMergeJoin (34) - :- CometSort (18) - : +- CometColumnarExchange (17) - : +- * Project (16) - : +- Window (15) - : +- * CometColumnarToRow (14) - : +- CometSort (13) - : +- CometExchange (12) - : +- CometHashAggregate (11) - : +- CometExchange (10) - : +- CometHashAggregate (9) - : +- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- CometSort (33) - +- CometColumnarExchange (32) - +- * Project (31) - +- Window (30) - +- * CometColumnarToRow (29) - +- CometSort (28) - +- CometExchange (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometBroadcastHashJoin (22) - :- CometFilter (20) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (19) - +- ReusedExchange (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: [d_date_sk#5, d_date#6] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Right output [2]: [d_date_sk#5, d_date#6] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] -Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] - -(9) CometHashAggregate -Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] - -(10) CometExchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] - -(12) CometExchange -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(13) CometSort -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(14) CometColumnarToRow [codegen id : 1] -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] - -(15) Window -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: [sum(_w0#10) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(16) Project [codegen id : 2] -Output [3]: [item_sk#9, d_date#6, cume_sales#11] -Input [5]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1, cume_sales#11] - -(17) CometColumnarExchange -Input [3]: [item_sk#9, d_date#6, cume_sales#11] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(18) CometSort -Input [3]: [item_sk#9, d_date#6, cume_sales#11] -Arguments: [item_sk#9, d_date#6, cume_sales#11], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#14), dynamicpruningexpression(ss_sold_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] -Condition : isnotnull(ss_item_sk#12) - -(21) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#16, d_date#17] - -(22) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] -Right output [2]: [d_date_sk#16, d_date#17] -Arguments: [ss_sold_date_sk#14], [d_date_sk#16], Inner, BuildRight - -(23) CometProject -Input [5]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14, d_date_sk#16, d_date#17] -Arguments: [ss_item_sk#12, ss_sales_price#13, d_date#17], [ss_item_sk#12, ss_sales_price#13, d_date#17] - -(24) CometHashAggregate -Input [3]: [ss_item_sk#12, ss_sales_price#13, d_date#17] -Keys [2]: [ss_item_sk#12, d_date#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#13))] - -(25) CometExchange -Input [3]: [ss_item_sk#12, d_date#17, sum#18] -Arguments: hashpartitioning(ss_item_sk#12, d_date#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(26) CometHashAggregate -Input [3]: [ss_item_sk#12, d_date#17, sum#18] -Keys [2]: [ss_item_sk#12, d_date#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#13))] - -(27) CometExchange -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -Arguments: hashpartitioning(ss_item_sk#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(28) CometSort -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -Arguments: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12], [ss_item_sk#12 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] - -(29) CometColumnarToRow [codegen id : 3] -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] - -(30) Window -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -Arguments: [sum(_w0#20) windowspecdefinition(ss_item_sk#12, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#21], [ss_item_sk#12], [d_date#17 ASC NULLS FIRST] - -(31) Project [codegen id : 4] -Output [3]: [item_sk#19, d_date#17, cume_sales#21] -Input [5]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12, cume_sales#21] - -(32) CometColumnarExchange -Input [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: hashpartitioning(item_sk#19, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(33) CometSort -Input [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: [item_sk#19, d_date#17, cume_sales#21], [item_sk#19 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] - -(34) CometSortMergeJoin -Left output [3]: [item_sk#9, d_date#6, cume_sales#11] -Right output [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: [item_sk#9, d_date#6], [item_sk#19, d_date#17], FullOuter - -(35) CometProject -Input [6]: [item_sk#9, d_date#6, cume_sales#11, item_sk#19, d_date#17, cume_sales#21] -Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#19 END AS item_sk#22, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#17 END AS d_date#23, cume_sales#11 AS web_sales#24, cume_sales#21 AS store_sales#25] - -(36) CometExchange -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: hashpartitioning(item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(37) CometSort -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST] - -(38) CometColumnarToRow [codegen id : 5] -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] - -(39) Window -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: [max(web_sales#24) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#26, max(store_sales#25) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#27], [item_sk#22], [d_date#23 ASC NULLS FIRST] - -(40) Filter [codegen id : 6] -Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] -Condition : ((isnotnull(web_cumulative#26) AND isnotnull(store_cumulative#27)) AND (web_cumulative#26 > store_cumulative#27)) - -(41) TakeOrderedAndProject -Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] -Arguments: 100, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) - - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) - -(44) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(45) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] - -(46) BroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/extended.txt deleted file mode 100644 index 66c5717cc7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/extended.txt +++ /dev/null @@ -1,53 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometColumnarExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 37 out of 47 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index a5af2e114b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/simplified.txt +++ /dev/null @@ -1,62 +0,0 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (6) - Filter [web_cumulative,store_cumulative] - InputAdapter - Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (2) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ws_item_sk] - CometExchange [ws_item_sk] #3 - CometHashAggregate [sum] [item_sk,d_date,_w0,ws_item_sk,sum(UnscaledValue(ws_sales_price))] - CometExchange [ws_item_sk,d_date] #4 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #6 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #7 - WholeStageCodegen (4) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ss_item_sk] - CometExchange [ss_item_sk] #8 - CometHashAggregate [sum] [item_sk,d_date,_w0,ss_item_sk,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_item_sk,d_date] #9 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - 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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_datafusion/explain.txt deleted file mode 100644 index 4af1b4f562..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_datafusion/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometNativeScan parquet spark_catalog.default.item (9) - - -(1) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometNativeScan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,ext_price#15]), [d_year#2, brand_id#13, brand#14, ext_price#15], 100, 0, [d_year#2 ASC NULLS FIRST, ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, ext_price#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_datafusion/simplified.txt deleted file mode 100644 index cee223d027..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_datafusion/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,ext_price] - CometHashAggregate [sum] [d_year,brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometNativeScan 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/q52.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_iceberg_compat/explain.txt deleted file mode 100644 index a05651a422..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,ext_price#15]), [d_year#2, brand_id#13, brand#14, ext_price#15], 100, 0, [d_year#2 ASC NULLS FIRST, ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, ext_price#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_iceberg_compat/simplified.txt deleted file mode 100644 index 218062c3ef..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,ext_price] - CometHashAggregate [sum] [d_year,brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan [native_iceberg_compat] 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/q52/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/explain.txt deleted file mode 100644 index a05651a422..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,ext_price#15]), [d_year#2, brand_id#13, brand#14, ext_price#15], 100, 0, [d_year#2 ASC NULLS FIRST, ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, ext_price#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/extended.txt deleted file mode 100644 index a292badf5d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 218062c3ef..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q52/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,ext_price] - CometHashAggregate [sum] [d_year,brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/explain.txt deleted file mode 100644 index f8c8dd3bf2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/explain.txt +++ /dev/null @@ -1,202 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * CometColumnarToRow (26) - +- CometSort (25) - +- CometColumnarExchange (24) - +- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometColumnarExchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * CometColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (8) - : : +- * Filter (7) - : : +- * ColumnarToRow (6) - : : +- Scan parquet spark_catalog.default.store_sales (5) - : +- ReusedExchange (11) - +- BroadcastExchange (17) - +- * CometColumnarToRow (16) - +- CometFilter (15) - +- CometNativeScan parquet spark_catalog.default.store (14) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Books ,Children ,Electronics ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (personal ,portable ,reference ,self-help )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Women ,Music ,Men ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (accessories ,classical ,fragrances ,pants )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_manufact_id#5], [i_item_sk#1, i_manufact_id#5] - -(4) CometColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#1, i_manufact_id#5] - -(5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(7) Filter [codegen id : 1] -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) - -(8) BroadcastExchange -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#10] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Input [6]: [i_item_sk#1, i_manufact_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(11) ReusedExchange [Reuses operator id: 35] -Output [2]: [d_date_sk#15, d_qoy#16] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#13] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#16] -Input [6]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_qoy#16] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [1]: [s_store_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [1]: [s_store_sk#17] -Condition : isnotnull(s_store_sk#17) - -(16) CometColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#17] - -(17) BroadcastExchange -Input [1]: [s_store_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#11] -Right keys [1]: [s_store_sk#17] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#16] -Input [5]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#16, s_store_sk#17] - -(20) HashAggregate [codegen id : 4] -Input [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#16] -Keys [2]: [i_manufact_id#5, d_qoy#16] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [i_manufact_id#5, d_qoy#16, sum#19] - -(21) CometColumnarExchange -Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] -Arguments: hashpartitioning(i_manufact_id#5, d_qoy#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] - -(23) HashAggregate [codegen id : 5] -Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] -Keys [2]: [i_manufact_id#5, d_qoy#16] -Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#20] -Results [3]: [i_manufact_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS _w0#22] - -(24) CometColumnarExchange -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: [i_manufact_id#5, sum_sales#21, _w0#22], [i_manufact_id#5 ASC NULLS FIRST] - -(26) CometColumnarToRow [codegen id : 6] -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] - -(27) Window -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#23], [i_manufact_id#5] - -(28) Filter [codegen id : 7] -Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] -Condition : CASE WHEN (avg_quarterly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_quarterly_sales#23)) / avg_quarterly_sales#23) > 0.1000000000000000) ELSE false END - -(29) Project [codegen id : 7] -Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] -Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] - -(30) TakeOrderedAndProject -Input [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] -Arguments: 100, [avg_quarterly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometNativeScan parquet spark_catalog.default.date_dim (31) - - -(31) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] -Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(33) CometProject -Input [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] -Arguments: [d_date_sk#15, d_qoy#16], [d_date_sk#15, d_qoy#16] - -(34) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_qoy#16] - -(35) BroadcastExchange -Input [2]: [d_date_sk#15, d_qoy#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/simplified.txt deleted file mode 100644 index 63c3e1a17a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/simplified.txt +++ /dev/null @@ -1,53 +0,0 @@ -TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (7) - Project [i_manufact_id,sum_sales,avg_quarterly_sales] - Filter [avg_quarterly_sales,sum_sales] - InputAdapter - Window [_w0,i_manufact_id] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_manufact_id,sum_sales,_w0] - CometColumnarExchange [i_manufact_id] #1 - WholeStageCodegen (5) - HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id,d_qoy] #2 - WholeStageCodegen (4) - HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] - Project [i_manufact_id,ss_sales_price,d_qoy] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_sk,i_manufact_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_qoy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/explain.txt deleted file mode 100644 index 2fc38bf838..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,198 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) - +- CometSort (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Books ,Children ,Electronics ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (personal ,portable ,reference ,self-help )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Women ,Music ,Men ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (accessories ,classical ,fragrances ,pants )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_manufact_id#5], [i_item_sk#1, i_manufact_id#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(7) CometBroadcastHashJoin -Left output [2]: [i_item_sk#1, i_manufact_id#5] -Right output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_item_sk#1], [ss_item_sk#10], Inner, BuildRight - -(8) CometProject -Input [6]: [i_item_sk#1, i_manufact_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13], [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(11) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] - -(12) CometBroadcastExchange -Input [2]: [d_date_sk#15, d_qoy#17] -Arguments: [d_date_sk#15, d_qoy#17] - -(13) CometBroadcastHashJoin -Left output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_qoy#17] -Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(14) CometProject -Input [6]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_qoy#17] -Arguments: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17], [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [s_store_sk#18] -Condition : isnotnull(s_store_sk#18) - -(17) CometBroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: [s_store_sk#18] - -(18) CometBroadcastHashJoin -Left output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17] -Right output [1]: [s_store_sk#18] -Arguments: [ss_store_sk#11], [s_store_sk#18], Inner, BuildRight - -(19) CometProject -Input [5]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17, s_store_sk#18] -Arguments: [i_manufact_id#5, ss_sales_price#12, d_qoy#17], [i_manufact_id#5, ss_sales_price#12, d_qoy#17] - -(20) CometHashAggregate -Input [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#17] -Keys [2]: [i_manufact_id#5, d_qoy#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] - -(21) CometExchange -Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] -Arguments: hashpartitioning(i_manufact_id#5, d_qoy#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] -Keys [2]: [i_manufact_id#5, d_qoy#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] - -(23) CometExchange -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: [i_manufact_id#5, sum_sales#20, _w0#21], [i_manufact_id#5 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 1] -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] - -(26) Window -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#22], [i_manufact_id#5] - -(27) Filter [codegen id : 2] -Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] -Condition : CASE WHEN (avg_quarterly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_quarterly_sales#22)) / avg_quarterly_sales#22) > 0.1000000000000000) ELSE false END - -(28) Project [codegen id : 2] -Output [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] -Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] - -(29) TakeOrderedAndProject -Input [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] -Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(32) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_qoy#17] - -(34) BroadcastExchange -Input [2]: [d_date_sk#15, d_qoy#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/simplified.txt deleted file mode 100644 index 0c123eb728..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (2) - Project [i_manufact_id,sum_sales,avg_quarterly_sales] - Filter [avg_quarterly_sales,sum_sales] - InputAdapter - Window [_w0,i_manufact_id] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_manufact_id,sum_sales,_w0] - CometExchange [i_manufact_id] #1 - CometHashAggregate [d_qoy,sum] [i_manufact_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_manufact_id,d_qoy] #2 - CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] - CometProject [i_manufact_id,ss_sales_price,d_qoy] - 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 [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,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_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [d_date_sk,d_qoy] #5 - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/explain.txt deleted file mode 100644 index 2fc38bf838..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/explain.txt +++ /dev/null @@ -1,198 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) - +- CometSort (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Books ,Children ,Electronics ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (personal ,portable ,reference ,self-help )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Women ,Music ,Men ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (accessories ,classical ,fragrances ,pants )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_manufact_id#5], [i_item_sk#1, i_manufact_id#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(7) CometBroadcastHashJoin -Left output [2]: [i_item_sk#1, i_manufact_id#5] -Right output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_item_sk#1], [ss_item_sk#10], Inner, BuildRight - -(8) CometProject -Input [6]: [i_item_sk#1, i_manufact_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13], [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(11) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] - -(12) CometBroadcastExchange -Input [2]: [d_date_sk#15, d_qoy#17] -Arguments: [d_date_sk#15, d_qoy#17] - -(13) CometBroadcastHashJoin -Left output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_qoy#17] -Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(14) CometProject -Input [6]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_qoy#17] -Arguments: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17], [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [s_store_sk#18] -Condition : isnotnull(s_store_sk#18) - -(17) CometBroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: [s_store_sk#18] - -(18) CometBroadcastHashJoin -Left output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17] -Right output [1]: [s_store_sk#18] -Arguments: [ss_store_sk#11], [s_store_sk#18], Inner, BuildRight - -(19) CometProject -Input [5]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17, s_store_sk#18] -Arguments: [i_manufact_id#5, ss_sales_price#12, d_qoy#17], [i_manufact_id#5, ss_sales_price#12, d_qoy#17] - -(20) CometHashAggregate -Input [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#17] -Keys [2]: [i_manufact_id#5, d_qoy#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] - -(21) CometExchange -Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] -Arguments: hashpartitioning(i_manufact_id#5, d_qoy#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] -Keys [2]: [i_manufact_id#5, d_qoy#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] - -(23) CometExchange -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: [i_manufact_id#5, sum_sales#20, _w0#21], [i_manufact_id#5 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 1] -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] - -(26) Window -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#22], [i_manufact_id#5] - -(27) Filter [codegen id : 2] -Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] -Condition : CASE WHEN (avg_quarterly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_quarterly_sales#22)) / avg_quarterly_sales#22) > 0.1000000000000000) ELSE false END - -(28) Project [codegen id : 2] -Output [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] -Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] - -(29) TakeOrderedAndProject -Input [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] -Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(32) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_qoy#17] - -(34) BroadcastExchange -Input [2]: [d_date_sk#15, d_qoy#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/extended.txt deleted file mode 100644 index 825b1ed81c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 0c123eb728..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (2) - Project [i_manufact_id,sum_sales,avg_quarterly_sales] - Filter [avg_quarterly_sales,sum_sales] - InputAdapter - Window [_w0,i_manufact_id] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_manufact_id,sum_sales,_w0] - CometExchange [i_manufact_id] #1 - CometHashAggregate [d_qoy,sum] [i_manufact_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_manufact_id,d_qoy] #2 - CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] - CometProject [i_manufact_id,ss_sales_price,d_qoy] - 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 [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,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_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [d_date_sk,d_qoy] #5 - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/explain.txt deleted file mode 100644 index bf3e53cb21..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/explain.txt +++ /dev/null @@ -1,497 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (61) -+- * HashAggregate (60) - +- * CometColumnarToRow (59) - +- CometColumnarExchange (58) - +- * HashAggregate (57) - +- * HashAggregate (56) - +- * CometColumnarToRow (55) - +- CometColumnarExchange (54) - +- * HashAggregate (53) - +- * Project (52) - +- * BroadcastHashJoin Inner BuildRight (51) - :- * Project (49) - : +- * BroadcastHashJoin Inner BuildRight (48) - : :- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * CometColumnarToRow (29) - : : : : +- CometHashAggregate (28) - : : : : +- CometColumnarExchange (27) - : : : : +- * HashAggregate (26) - : : : : +- * Project (25) - : : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : : :- * Project (19) - : : : : : +- * BroadcastHashJoin Inner BuildRight (18) - : : : : : :- * Project (16) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : : : : :- Union (9) - : : : : : : : :- * Project (4) - : : : : : : : : +- * Filter (3) - : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : +- * Project (8) - : : : : : : : +- * Filter (7) - : : : : : : : +- * ColumnarToRow (6) - : : : : : : : +- Scan parquet spark_catalog.default.web_sales (5) - : : : : : : +- BroadcastExchange (14) - : : : : : : +- * CometColumnarToRow (13) - : : : : : : +- CometProject (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) - : : : : : +- ReusedExchange (17) - : : : : +- BroadcastExchange (23) - : : : : +- * CometColumnarToRow (22) - : : : : +- CometFilter (21) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (20) - : : : +- BroadcastExchange (33) - : : : +- * Filter (32) - : : : +- * ColumnarToRow (31) - : : : +- Scan parquet spark_catalog.default.store_sales (30) - : : +- BroadcastExchange (40) - : : +- * CometColumnarToRow (39) - : : +- CometProject (38) - : : +- CometFilter (37) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (36) - : +- BroadcastExchange (47) - : +- * CometColumnarToRow (46) - : +- CometProject (45) - : +- CometFilter (44) - : +- CometNativeScan parquet spark_catalog.default.store (43) - +- ReusedExchange (50) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] - -(3) Filter [codegen id : 1] -Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_bill_customer_sk#1)) - -(4) Project [codegen id : 1] -Output [3]: [cs_sold_date_sk#3 AS sold_date_sk#5, cs_bill_customer_sk#1 AS customer_sk#6, cs_item_sk#2 AS item_sk#7] -Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] - -(5) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#10), dynamicpruningexpression(ws_sold_date_sk#10 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 2] -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] - -(7) Filter [codegen id : 2] -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Condition : (isnotnull(ws_item_sk#8) AND isnotnull(ws_bill_customer_sk#9)) - -(8) Project [codegen id : 2] -Output [3]: [ws_sold_date_sk#10 AS sold_date_sk#11, ws_bill_customer_sk#9 AS customer_sk#12, ws_item_sk#8 AS item_sk#13] -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] - -(9) Union - -(10) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_class#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true) = Women ) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#15, 50, true, false, true) = maternity )) AND isnotnull(i_item_sk#14)) - -(12) CometProject -Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Arguments: [i_item_sk#14], [i_item_sk#14] - -(13) CometColumnarToRow [codegen id : 3] -Input [1]: [i_item_sk#14] - -(14) BroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(15) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [item_sk#7] -Right keys [1]: [i_item_sk#14] -Join type: Inner -Join condition: None - -(16) Project [codegen id : 6] -Output [2]: [sold_date_sk#5, customer_sk#6] -Input [4]: [sold_date_sk#5, customer_sk#6, item_sk#7, i_item_sk#14] - -(17) ReusedExchange [Reuses operator id: 66] -Output [1]: [d_date_sk#17] - -(18) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sold_date_sk#5] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 6] -Output [1]: [customer_sk#6] -Input [3]: [sold_date_sk#5, customer_sk#6, d_date_sk#17] - -(20) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#18, c_current_addr_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Condition : (isnotnull(c_customer_sk#18) AND isnotnull(c_current_addr_sk#19)) - -(22) CometColumnarToRow [codegen id : 5] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] - -(23) BroadcastExchange -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [customer_sk#6] -Right keys [1]: [c_customer_sk#18] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [2]: [c_customer_sk#18, c_current_addr_sk#19] -Input [3]: [customer_sk#6, c_customer_sk#18, c_current_addr_sk#19] - -(26) HashAggregate [codegen id : 6] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Keys [2]: [c_customer_sk#18, c_current_addr_sk#19] -Functions: [] -Aggregate Attributes: [] -Results [2]: [c_customer_sk#18, c_current_addr_sk#19] - -(27) CometColumnarExchange -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Arguments: hashpartitioning(c_customer_sk#18, c_current_addr_sk#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometHashAggregate -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Keys [2]: [c_customer_sk#18, c_current_addr_sk#19] -Functions: [] - -(29) CometColumnarToRow [codegen id : 11] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] - -(30) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 7] -Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] - -(32) Filter [codegen id : 7] -Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#20) - -(33) BroadcastExchange -Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - -(34) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_customer_sk#18] -Right keys [1]: [ss_customer_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 11] -Output [4]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22] -Input [5]: [c_customer_sk#18, c_current_addr_sk#19, ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] - -(36) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#24, ca_county#25, ca_state#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] -Condition : ((isnotnull(ca_address_sk#24) AND isnotnull(ca_county#25)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#26, 2, true, false, true))) - -(38) CometProject -Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] -Arguments: [ca_address_sk#24, ca_county#25, ca_state#27], [ca_address_sk#24, ca_county#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#26, 2, true, false, true) AS ca_state#27] - -(39) CometColumnarToRow [codegen id : 8] -Input [3]: [ca_address_sk#24, ca_county#25, ca_state#27] - -(40) BroadcastExchange -Input [3]: [ca_address_sk#24, ca_county#25, ca_state#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(41) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#19] -Right keys [1]: [ca_address_sk#24] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 11] -Output [5]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#27] -Input [7]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_address_sk#24, ca_county#25, ca_state#27] - -(43) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_county#28, s_state#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county)] -ReadSchema: struct - -(44) CometFilter -Input [2]: [s_county#28, s_state#29] -Condition : (isnotnull(s_county#28) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#29, 2, true, false, true))) - -(45) CometProject -Input [2]: [s_county#28, s_state#29] -Arguments: [s_county#28, s_state#30], [s_county#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#29, 2, true, false, true) AS s_state#30] - -(46) CometColumnarToRow [codegen id : 9] -Input [2]: [s_county#28, s_state#30] - -(47) BroadcastExchange -Input [2]: [s_county#28, s_state#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true]),false), [plan_id=6] - -(48) BroadcastHashJoin [codegen id : 11] -Left keys [2]: [ca_county#25, ca_state#27] -Right keys [2]: [s_county#28, s_state#30] -Join type: Inner -Join condition: None - -(49) Project [codegen id : 11] -Output [3]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22] -Input [7]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#27, s_county#28, s_state#30] - -(50) ReusedExchange [Reuses operator id: 71] -Output [1]: [d_date_sk#31] - -(51) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#22] -Right keys [1]: [d_date_sk#31] -Join type: Inner -Join condition: None - -(52) Project [codegen id : 11] -Output [2]: [c_customer_sk#18, ss_ext_sales_price#21] -Input [4]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, d_date_sk#31] - -(53) HashAggregate [codegen id : 11] -Input [2]: [c_customer_sk#18, ss_ext_sales_price#21] -Keys [1]: [c_customer_sk#18] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#21))] -Aggregate Attributes [1]: [sum#32] -Results [2]: [c_customer_sk#18, sum#33] - -(54) CometColumnarExchange -Input [2]: [c_customer_sk#18, sum#33] -Arguments: hashpartitioning(c_customer_sk#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(55) CometColumnarToRow [codegen id : 12] -Input [2]: [c_customer_sk#18, sum#33] - -(56) HashAggregate [codegen id : 12] -Input [2]: [c_customer_sk#18, sum#33] -Keys [1]: [c_customer_sk#18] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#21))#34] -Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#21))#34,17,2) / 50) as int) AS segment#35] - -(57) HashAggregate [codegen id : 12] -Input [1]: [segment#35] -Keys [1]: [segment#35] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#36] -Results [2]: [segment#35, count#37] - -(58) CometColumnarExchange -Input [2]: [segment#35, count#37] -Arguments: hashpartitioning(segment#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(59) CometColumnarToRow [codegen id : 13] -Input [2]: [segment#35, count#37] - -(60) HashAggregate [codegen id : 13] -Input [2]: [segment#35, count#37] -Keys [1]: [segment#35] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#38] -Results [3]: [segment#35, count(1)#38 AS num_customers#39, (segment#35 * 50) AS segment_base#40] - -(61) TakeOrderedAndProject -Input [3]: [segment#35, num_customers#39, segment_base#40] -Arguments: 100, [segment#35 ASC NULLS FIRST, num_customers#39 ASC NULLS FIRST], [segment#35, num_customers#39, segment_base#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (66) -+- * CometColumnarToRow (65) - +- CometProject (64) - +- CometFilter (63) - +- CometNativeScan parquet spark_catalog.default.date_dim (62) - - -(62) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#41, d_moy#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(63) CometFilter -Input [3]: [d_date_sk#17, d_year#41, d_moy#42] -Condition : ((((isnotnull(d_moy#42) AND isnotnull(d_year#41)) AND (d_moy#42 = 12)) AND (d_year#41 = 1998)) AND isnotnull(d_date_sk#17)) - -(64) CometProject -Input [3]: [d_date_sk#17, d_year#41, d_moy#42] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(65) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] - -(66) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 - -Subquery:3 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometProject (69) - +- CometFilter (68) - +- CometNativeScan parquet spark_catalog.default.date_dim (67) - - -(67) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#31, d_month_seq#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(68) CometFilter -Input [2]: [d_date_sk#31, d_month_seq#43] -Condition : (((isnotnull(d_month_seq#43) AND (d_month_seq#43 >= Subquery scalar-subquery#44, [id=#45])) AND (d_month_seq#43 <= Subquery scalar-subquery#46, [id=#47])) AND isnotnull(d_date_sk#31)) - -(69) CometProject -Input [2]: [d_date_sk#31, d_month_seq#43] -Arguments: [d_date_sk#31], [d_date_sk#31] - -(70) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#31] - -(71) BroadcastExchange -Input [1]: [d_date_sk#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] - -Subquery:4 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#44, [id=#45] -* CometColumnarToRow (78) -+- CometHashAggregate (77) - +- CometExchange (76) - +- CometHashAggregate (75) - +- CometProject (74) - +- CometFilter (73) - +- CometNativeScan parquet spark_catalog.default.date_dim (72) - - -(72) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#48, d_year#49, d_moy#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] -ReadSchema: struct - -(73) CometFilter -Input [3]: [d_month_seq#48, d_year#49, d_moy#50] -Condition : (((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 1998)) AND (d_moy#50 = 12)) - -(74) CometProject -Input [3]: [d_month_seq#48, d_year#49, d_moy#50] -Arguments: [(d_month_seq + 1)#51], [(d_month_seq#48 + 1) AS (d_month_seq + 1)#51] - -(75) CometHashAggregate -Input [1]: [(d_month_seq + 1)#51] -Keys [1]: [(d_month_seq + 1)#51] -Functions: [] - -(76) CometExchange -Input [1]: [(d_month_seq + 1)#51] -Arguments: hashpartitioning((d_month_seq + 1)#51, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(77) CometHashAggregate -Input [1]: [(d_month_seq + 1)#51] -Keys [1]: [(d_month_seq + 1)#51] -Functions: [] - -(78) CometColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 1)#51] - -Subquery:5 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#46, [id=#47] -* CometColumnarToRow (85) -+- CometHashAggregate (84) - +- CometExchange (83) - +- CometHashAggregate (82) - +- CometProject (81) - +- CometFilter (80) - +- CometNativeScan parquet spark_catalog.default.date_dim (79) - - -(79) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#52, d_year#53, d_moy#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] -ReadSchema: struct - -(80) CometFilter -Input [3]: [d_month_seq#52, d_year#53, d_moy#54] -Condition : (((isnotnull(d_year#53) AND isnotnull(d_moy#54)) AND (d_year#53 = 1998)) AND (d_moy#54 = 12)) - -(81) CometProject -Input [3]: [d_month_seq#52, d_year#53, d_moy#54] -Arguments: [(d_month_seq + 3)#55], [(d_month_seq#52 + 3) AS (d_month_seq + 3)#55] - -(82) CometHashAggregate -Input [1]: [(d_month_seq + 3)#55] -Keys [1]: [(d_month_seq + 3)#55] -Functions: [] - -(83) CometExchange -Input [1]: [(d_month_seq + 3)#55] -Arguments: hashpartitioning((d_month_seq + 3)#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(84) CometHashAggregate -Input [1]: [(d_month_seq + 3)#55] -Keys [1]: [(d_month_seq + 3)#55] -Functions: [] - -(85) CometColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 3)#55] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/simplified.txt deleted file mode 100644 index ba04de874d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/simplified.txt +++ /dev/null @@ -1,127 +0,0 @@ -TakeOrderedAndProject [segment,num_customers,segment_base] - WholeStageCodegen (13) - HashAggregate [segment,count] [count(1),num_customers,segment_base,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [segment] #1 - WholeStageCodegen (12) - HashAggregate [segment] [count,count] - HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_sk] #2 - WholeStageCodegen (11) - HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] - Project [c_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ca_county,ca_state,s_county,s_state] - Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometHashAggregate [c_customer_sk,c_current_addr_sk] - CometColumnarExchange [c_customer_sk,c_current_addr_sk] #3 - WholeStageCodegen (6) - HashAggregate [c_customer_sk,c_current_addr_sk] - Project [c_customer_sk,c_current_addr_sk] - BroadcastHashJoin [customer_sk,c_customer_sk] - Project [customer_sk] - BroadcastHashJoin [sold_date_sk,d_date_sk] - Project [sold_date_sk,customer_sk] - BroadcastHashJoin [item_sk,i_item_sk] - InputAdapter - Union - WholeStageCodegen (1) - Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] - Filter [cs_item_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (2) - Project [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] - Filter [ws_item_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometExchange [(d_month_seq + 1)] #9 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #10 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_county,ca_state] - CometFilter [ca_address_sk,ca_county,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_county,s_state] - CometFilter [s_county,s_state] - CometNativeScan parquet spark_catalog.default.store [s_county,s_state] - InputAdapter - ReusedExchange [d_date_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/explain.txt deleted file mode 100644 index 26ce2a5731..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,488 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (58) -+- CometTakeOrderedAndProject (57) - +- CometHashAggregate (56) - +- CometExchange (55) - +- CometHashAggregate (54) - +- CometHashAggregate (53) - +- CometExchange (52) - +- CometHashAggregate (51) - +- CometProject (50) - +- CometBroadcastHashJoin (49) - :- CometProject (44) - : +- CometBroadcastHashJoin (43) - : :- CometProject (38) - : : +- CometBroadcastHashJoin (37) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometHashAggregate (27) - : : : : +- CometExchange (26) - : : : : +- CometHashAggregate (25) - : : : : +- CometProject (24) - : : : : +- CometBroadcastHashJoin (23) - : : : : :- CometProject (19) - : : : : : +- CometBroadcastHashJoin (18) - : : : : : :- CometProject (13) - : : : : : : +- CometBroadcastHashJoin (12) - : : : : : : :- CometUnion (7) - : : : : : : : :- CometProject (3) - : : : : : : : : +- CometFilter (2) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : +- CometProject (6) - : : : : : : : +- CometFilter (5) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (4) - : : : : : : +- CometBroadcastExchange (11) - : : : : : : +- CometProject (10) - : : : : : : +- CometFilter (9) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (8) - : : : : : +- CometBroadcastExchange (17) - : : : : : +- CometProject (16) - : : : : : +- CometFilter (15) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - : : : : +- CometBroadcastExchange (22) - : : : : +- CometFilter (21) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (20) - : : : +- CometBroadcastExchange (30) - : : : +- CometFilter (29) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (28) - : : +- CometBroadcastExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (33) - : +- CometBroadcastExchange (42) - : +- CometProject (41) - : +- CometFilter (40) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (39) - +- CometBroadcastExchange (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_bill_customer_sk#1)) - -(3) CometProject -Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Arguments: [sold_date_sk#5, customer_sk#6, item_sk#7], [cs_sold_date_sk#3 AS sold_date_sk#5, cs_bill_customer_sk#1 AS customer_sk#6, cs_item_sk#2 AS item_sk#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#10), dynamicpruningexpression(ws_sold_date_sk#10 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Condition : (isnotnull(ws_item_sk#8) AND isnotnull(ws_bill_customer_sk#9)) - -(6) CometProject -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Arguments: [sold_date_sk#11, customer_sk#12, item_sk#13], [ws_sold_date_sk#10 AS sold_date_sk#11, ws_bill_customer_sk#9 AS customer_sk#12, ws_item_sk#8 AS item_sk#13] - -(7) CometUnion -Child 0 Input [3]: [sold_date_sk#5, customer_sk#6, item_sk#7] -Child 1 Input [3]: [sold_date_sk#11, customer_sk#12, item_sk#13] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_class#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true) = Women ) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#15, 50, true, false, true) = maternity )) AND isnotnull(i_item_sk#14)) - -(10) CometProject -Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Arguments: [i_item_sk#14], [i_item_sk#14] - -(11) CometBroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: [i_item_sk#14] - -(12) CometBroadcastHashJoin -Left output [3]: [sold_date_sk#5, customer_sk#6, item_sk#7] -Right output [1]: [i_item_sk#14] -Arguments: [item_sk#7], [i_item_sk#14], Inner, BuildRight - -(13) CometProject -Input [4]: [sold_date_sk#5, customer_sk#6, item_sk#7, i_item_sk#14] -Arguments: [sold_date_sk#5, customer_sk#6], [sold_date_sk#5, customer_sk#6] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12)) AND (d_year#18 = 1998)) AND isnotnull(d_date_sk#17)) - -(16) CometProject -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: [d_date_sk#17] - -(18) CometBroadcastHashJoin -Left output [2]: [sold_date_sk#5, customer_sk#6] -Right output [1]: [d_date_sk#17] -Arguments: [sold_date_sk#5], [d_date_sk#17], Inner, BuildRight - -(19) CometProject -Input [3]: [sold_date_sk#5, customer_sk#6, d_date_sk#17] -Arguments: [customer_sk#6], [customer_sk#6] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#20, c_current_addr_sk#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Condition : (isnotnull(c_customer_sk#20) AND isnotnull(c_current_addr_sk#21)) - -(22) CometBroadcastExchange -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Arguments: [c_customer_sk#20, c_current_addr_sk#21] - -(23) CometBroadcastHashJoin -Left output [1]: [customer_sk#6] -Right output [2]: [c_customer_sk#20, c_current_addr_sk#21] -Arguments: [customer_sk#6], [c_customer_sk#20], Inner, BuildRight - -(24) CometProject -Input [3]: [customer_sk#6, c_customer_sk#20, c_current_addr_sk#21] -Arguments: [c_customer_sk#20, c_current_addr_sk#21], [c_customer_sk#20, c_current_addr_sk#21] - -(25) CometHashAggregate -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Keys [2]: [c_customer_sk#20, c_current_addr_sk#21] -Functions: [] - -(26) CometExchange -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Arguments: hashpartitioning(c_customer_sk#20, c_current_addr_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Keys [2]: [c_customer_sk#20, c_current_addr_sk#21] -Functions: [] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#24), dynamicpruningexpression(ss_sold_date_sk#24 IN dynamicpruning#25)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(29) CometFilter -Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Condition : isnotnull(ss_customer_sk#22) - -(30) CometBroadcastExchange -Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Arguments: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] - -(31) CometBroadcastHashJoin -Left output [2]: [c_customer_sk#20, c_current_addr_sk#21] -Right output [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Arguments: [c_customer_sk#20], [ss_customer_sk#22], Inner, BuildRight - -(32) CometProject -Input [5]: [c_customer_sk#20, c_current_addr_sk#21, ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Arguments: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24], [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#26, ca_county#27, ca_state#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct - -(34) CometFilter -Input [3]: [ca_address_sk#26, ca_county#27, ca_state#28] -Condition : ((isnotnull(ca_address_sk#26) AND isnotnull(ca_county#27)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#28, 2, true, false, true))) - -(35) CometProject -Input [3]: [ca_address_sk#26, ca_county#27, ca_state#28] -Arguments: [ca_address_sk#26, ca_county#27, ca_state#29], [ca_address_sk#26, ca_county#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#28, 2, true, false, true) AS ca_state#29] - -(36) CometBroadcastExchange -Input [3]: [ca_address_sk#26, ca_county#27, ca_state#29] -Arguments: [ca_address_sk#26, ca_county#27, ca_state#29] - -(37) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24] -Right output [3]: [ca_address_sk#26, ca_county#27, ca_state#29] -Arguments: [c_current_addr_sk#21], [ca_address_sk#26], Inner, BuildRight - -(38) CometProject -Input [7]: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_address_sk#26, ca_county#27, ca_state#29] -Arguments: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29], [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_county#30, s_state#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county)] -ReadSchema: struct - -(40) CometFilter -Input [2]: [s_county#30, s_state#31] -Condition : (isnotnull(s_county#30) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#31, 2, true, false, true))) - -(41) CometProject -Input [2]: [s_county#30, s_state#31] -Arguments: [s_county#30, s_state#32], [s_county#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#31, 2, true, false, true) AS s_state#32] - -(42) CometBroadcastExchange -Input [2]: [s_county#30, s_state#32] -Arguments: [s_county#30, s_state#32] - -(43) CometBroadcastHashJoin -Left output [5]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29] -Right output [2]: [s_county#30, s_state#32] -Arguments: [ca_county#27, ca_state#29], [s_county#30, s_state#32], Inner, BuildRight - -(44) CometProject -Input [7]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29, s_county#30, s_state#32] -Arguments: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24], [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_month_seq#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [d_date_sk#33, d_month_seq#34] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= ReusedSubquery Subquery scalar-subquery#35, [id=#36])) AND (d_month_seq#34 <= ReusedSubquery Subquery scalar-subquery#37, [id=#38])) AND isnotnull(d_date_sk#33)) - -(47) CometProject -Input [2]: [d_date_sk#33, d_month_seq#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(48) CometBroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: [d_date_sk#33] - -(49) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] -Right output [1]: [d_date_sk#33] -Arguments: [ss_sold_date_sk#24], [d_date_sk#33], Inner, BuildRight - -(50) CometProject -Input [4]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, d_date_sk#33] -Arguments: [c_customer_sk#20, ss_ext_sales_price#23], [c_customer_sk#20, ss_ext_sales_price#23] - -(51) CometHashAggregate -Input [2]: [c_customer_sk#20, ss_ext_sales_price#23] -Keys [1]: [c_customer_sk#20] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#23))] - -(52) CometExchange -Input [2]: [c_customer_sk#20, sum#39] -Arguments: hashpartitioning(c_customer_sk#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(53) CometHashAggregate -Input [2]: [c_customer_sk#20, sum#39] -Keys [1]: [c_customer_sk#20] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#23))] - -(54) CometHashAggregate -Input [1]: [segment#40] -Keys [1]: [segment#40] -Functions [1]: [partial_count(1)] - -(55) CometExchange -Input [2]: [segment#40, count#41] -Arguments: hashpartitioning(segment#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(56) CometHashAggregate -Input [2]: [segment#40, count#41] -Keys [1]: [segment#40] -Functions [1]: [count(1)] - -(57) CometTakeOrderedAndProject -Input [3]: [segment#40, num_customers#42, segment_base#43] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[segment#40 ASC NULLS FIRST,num_customers#42 ASC NULLS FIRST], output=[segment#40,num_customers#42,segment_base#43]), [segment#40, num_customers#42, segment_base#43], 100, 0, [segment#40 ASC NULLS FIRST, num_customers#42 ASC NULLS FIRST], [segment#40, num_customers#42, segment_base#43] - -(58) CometColumnarToRow [codegen id : 1] -Input [3]: [segment#40, num_customers#42, segment_base#43] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (63) -+- * CometColumnarToRow (62) - +- CometProject (61) - +- CometFilter (60) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (59) - - -(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(60) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12)) AND (d_year#18 = 1998)) AND isnotnull(d_date_sk#17)) - -(61) CometProject -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(62) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] - -(63) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 - -Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 -BroadcastExchange (68) -+- * CometColumnarToRow (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (64) - - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_month_seq#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(65) CometFilter -Input [2]: [d_date_sk#33, d_month_seq#34] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= Subquery scalar-subquery#35, [id=#36])) AND (d_month_seq#34 <= Subquery scalar-subquery#37, [id=#38])) AND isnotnull(d_date_sk#33)) - -(66) CometProject -Input [2]: [d_date_sk#33, d_month_seq#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(67) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#33] - -(68) BroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:4 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#35, [id=#36] -* CometColumnarToRow (75) -+- CometHashAggregate (74) - +- CometExchange (73) - +- CometHashAggregate (72) - +- CometProject (71) - +- CometFilter (70) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (69) - - -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#44, d_year#45, d_moy#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] -ReadSchema: struct - -(70) CometFilter -Input [3]: [d_month_seq#44, d_year#45, d_moy#46] -Condition : (((isnotnull(d_year#45) AND isnotnull(d_moy#46)) AND (d_year#45 = 1998)) AND (d_moy#46 = 12)) - -(71) CometProject -Input [3]: [d_month_seq#44, d_year#45, d_moy#46] -Arguments: [(d_month_seq + 1)#47], [(d_month_seq#44 + 1) AS (d_month_seq + 1)#47] - -(72) CometHashAggregate -Input [1]: [(d_month_seq + 1)#47] -Keys [1]: [(d_month_seq + 1)#47] -Functions: [] - -(73) CometExchange -Input [1]: [(d_month_seq + 1)#47] -Arguments: hashpartitioning((d_month_seq + 1)#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(74) CometHashAggregate -Input [1]: [(d_month_seq + 1)#47] -Keys [1]: [(d_month_seq + 1)#47] -Functions: [] - -(75) CometColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 1)#47] - -Subquery:5 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#37, [id=#38] -* CometColumnarToRow (82) -+- CometHashAggregate (81) - +- CometExchange (80) - +- CometHashAggregate (79) - +- CometProject (78) - +- CometFilter (77) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) - - -(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#48, d_year#49, d_moy#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] -ReadSchema: struct - -(77) CometFilter -Input [3]: [d_month_seq#48, d_year#49, d_moy#50] -Condition : (((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 1998)) AND (d_moy#50 = 12)) - -(78) CometProject -Input [3]: [d_month_seq#48, d_year#49, d_moy#50] -Arguments: [(d_month_seq + 3)#51], [(d_month_seq#48 + 3) AS (d_month_seq + 3)#51] - -(79) CometHashAggregate -Input [1]: [(d_month_seq + 3)#51] -Keys [1]: [(d_month_seq + 3)#51] -Functions: [] - -(80) CometExchange -Input [1]: [(d_month_seq + 3)#51] -Arguments: hashpartitioning((d_month_seq + 3)#51, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(81) CometHashAggregate -Input [1]: [(d_month_seq + 3)#51] -Keys [1]: [(d_month_seq + 3)#51] -Functions: [] - -(82) CometColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 3)#51] - -Subquery:6 Hosting operator id = 46 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#36] - -Subquery:7 Hosting operator id = 46 Hosting Expression = ReusedSubquery Subquery scalar-subquery#37, [id=#38] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/simplified.txt deleted file mode 100644 index ae0cecb1ba..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,99 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [segment,num_customers,segment_base] - CometHashAggregate [count] [segment,num_customers,segment_base,count(1)] - CometExchange [segment] #1 - CometHashAggregate [segment,count] - CometHashAggregate [c_customer_sk,sum] [segment,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [c_customer_sk] #2 - CometHashAggregate [ss_ext_sales_price] [c_customer_sk,sum] - CometProject [c_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state,s_county,s_state] - CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] - CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk,ca_address_sk,ca_county,ca_state] - CometProject [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometHashAggregate [c_customer_sk,c_current_addr_sk] - CometExchange [c_customer_sk,c_current_addr_sk] #3 - CometHashAggregate [c_customer_sk,c_current_addr_sk] - CometProject [c_customer_sk,c_current_addr_sk] - CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] - CometProject [customer_sk] - CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] - CometProject [sold_date_sk,customer_sk] - 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_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [i_item_sk] #5 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] #8 - CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometExchange [(d_month_seq + 1)] #10 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #11 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state] #12 - CometProject [ca_state] [ca_address_sk,ca_county,ca_state] - CometFilter [ca_address_sk,ca_county,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] - CometBroadcastExchange [s_county,s_state] #13 - CometProject [s_state] [s_county,s_state] - CometFilter [s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_county,s_state] - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [(d_month_seq + 1)] #3 - ReusedSubquery [(d_month_seq + 3)] #4 - CometScan [native_iceberg_compat] 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/q54/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/explain.txt deleted file mode 100644 index 26ce2a5731..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/explain.txt +++ /dev/null @@ -1,488 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (58) -+- CometTakeOrderedAndProject (57) - +- CometHashAggregate (56) - +- CometExchange (55) - +- CometHashAggregate (54) - +- CometHashAggregate (53) - +- CometExchange (52) - +- CometHashAggregate (51) - +- CometProject (50) - +- CometBroadcastHashJoin (49) - :- CometProject (44) - : +- CometBroadcastHashJoin (43) - : :- CometProject (38) - : : +- CometBroadcastHashJoin (37) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometHashAggregate (27) - : : : : +- CometExchange (26) - : : : : +- CometHashAggregate (25) - : : : : +- CometProject (24) - : : : : +- CometBroadcastHashJoin (23) - : : : : :- CometProject (19) - : : : : : +- CometBroadcastHashJoin (18) - : : : : : :- CometProject (13) - : : : : : : +- CometBroadcastHashJoin (12) - : : : : : : :- CometUnion (7) - : : : : : : : :- CometProject (3) - : : : : : : : : +- CometFilter (2) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : +- CometProject (6) - : : : : : : : +- CometFilter (5) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (4) - : : : : : : +- CometBroadcastExchange (11) - : : : : : : +- CometProject (10) - : : : : : : +- CometFilter (9) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (8) - : : : : : +- CometBroadcastExchange (17) - : : : : : +- CometProject (16) - : : : : : +- CometFilter (15) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - : : : : +- CometBroadcastExchange (22) - : : : : +- CometFilter (21) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (20) - : : : +- CometBroadcastExchange (30) - : : : +- CometFilter (29) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (28) - : : +- CometBroadcastExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (33) - : +- CometBroadcastExchange (42) - : +- CometProject (41) - : +- CometFilter (40) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (39) - +- CometBroadcastExchange (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_bill_customer_sk#1)) - -(3) CometProject -Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Arguments: [sold_date_sk#5, customer_sk#6, item_sk#7], [cs_sold_date_sk#3 AS sold_date_sk#5, cs_bill_customer_sk#1 AS customer_sk#6, cs_item_sk#2 AS item_sk#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#10), dynamicpruningexpression(ws_sold_date_sk#10 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Condition : (isnotnull(ws_item_sk#8) AND isnotnull(ws_bill_customer_sk#9)) - -(6) CometProject -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Arguments: [sold_date_sk#11, customer_sk#12, item_sk#13], [ws_sold_date_sk#10 AS sold_date_sk#11, ws_bill_customer_sk#9 AS customer_sk#12, ws_item_sk#8 AS item_sk#13] - -(7) CometUnion -Child 0 Input [3]: [sold_date_sk#5, customer_sk#6, item_sk#7] -Child 1 Input [3]: [sold_date_sk#11, customer_sk#12, item_sk#13] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_class#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true) = Women ) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#15, 50, true, false, true) = maternity )) AND isnotnull(i_item_sk#14)) - -(10) CometProject -Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Arguments: [i_item_sk#14], [i_item_sk#14] - -(11) CometBroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: [i_item_sk#14] - -(12) CometBroadcastHashJoin -Left output [3]: [sold_date_sk#5, customer_sk#6, item_sk#7] -Right output [1]: [i_item_sk#14] -Arguments: [item_sk#7], [i_item_sk#14], Inner, BuildRight - -(13) CometProject -Input [4]: [sold_date_sk#5, customer_sk#6, item_sk#7, i_item_sk#14] -Arguments: [sold_date_sk#5, customer_sk#6], [sold_date_sk#5, customer_sk#6] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12)) AND (d_year#18 = 1998)) AND isnotnull(d_date_sk#17)) - -(16) CometProject -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: [d_date_sk#17] - -(18) CometBroadcastHashJoin -Left output [2]: [sold_date_sk#5, customer_sk#6] -Right output [1]: [d_date_sk#17] -Arguments: [sold_date_sk#5], [d_date_sk#17], Inner, BuildRight - -(19) CometProject -Input [3]: [sold_date_sk#5, customer_sk#6, d_date_sk#17] -Arguments: [customer_sk#6], [customer_sk#6] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#20, c_current_addr_sk#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Condition : (isnotnull(c_customer_sk#20) AND isnotnull(c_current_addr_sk#21)) - -(22) CometBroadcastExchange -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Arguments: [c_customer_sk#20, c_current_addr_sk#21] - -(23) CometBroadcastHashJoin -Left output [1]: [customer_sk#6] -Right output [2]: [c_customer_sk#20, c_current_addr_sk#21] -Arguments: [customer_sk#6], [c_customer_sk#20], Inner, BuildRight - -(24) CometProject -Input [3]: [customer_sk#6, c_customer_sk#20, c_current_addr_sk#21] -Arguments: [c_customer_sk#20, c_current_addr_sk#21], [c_customer_sk#20, c_current_addr_sk#21] - -(25) CometHashAggregate -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Keys [2]: [c_customer_sk#20, c_current_addr_sk#21] -Functions: [] - -(26) CometExchange -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Arguments: hashpartitioning(c_customer_sk#20, c_current_addr_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Keys [2]: [c_customer_sk#20, c_current_addr_sk#21] -Functions: [] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#24), dynamicpruningexpression(ss_sold_date_sk#24 IN dynamicpruning#25)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(29) CometFilter -Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Condition : isnotnull(ss_customer_sk#22) - -(30) CometBroadcastExchange -Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Arguments: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] - -(31) CometBroadcastHashJoin -Left output [2]: [c_customer_sk#20, c_current_addr_sk#21] -Right output [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Arguments: [c_customer_sk#20], [ss_customer_sk#22], Inner, BuildRight - -(32) CometProject -Input [5]: [c_customer_sk#20, c_current_addr_sk#21, ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Arguments: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24], [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#26, ca_county#27, ca_state#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct - -(34) CometFilter -Input [3]: [ca_address_sk#26, ca_county#27, ca_state#28] -Condition : ((isnotnull(ca_address_sk#26) AND isnotnull(ca_county#27)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#28, 2, true, false, true))) - -(35) CometProject -Input [3]: [ca_address_sk#26, ca_county#27, ca_state#28] -Arguments: [ca_address_sk#26, ca_county#27, ca_state#29], [ca_address_sk#26, ca_county#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#28, 2, true, false, true) AS ca_state#29] - -(36) CometBroadcastExchange -Input [3]: [ca_address_sk#26, ca_county#27, ca_state#29] -Arguments: [ca_address_sk#26, ca_county#27, ca_state#29] - -(37) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24] -Right output [3]: [ca_address_sk#26, ca_county#27, ca_state#29] -Arguments: [c_current_addr_sk#21], [ca_address_sk#26], Inner, BuildRight - -(38) CometProject -Input [7]: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_address_sk#26, ca_county#27, ca_state#29] -Arguments: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29], [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_county#30, s_state#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county)] -ReadSchema: struct - -(40) CometFilter -Input [2]: [s_county#30, s_state#31] -Condition : (isnotnull(s_county#30) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#31, 2, true, false, true))) - -(41) CometProject -Input [2]: [s_county#30, s_state#31] -Arguments: [s_county#30, s_state#32], [s_county#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#31, 2, true, false, true) AS s_state#32] - -(42) CometBroadcastExchange -Input [2]: [s_county#30, s_state#32] -Arguments: [s_county#30, s_state#32] - -(43) CometBroadcastHashJoin -Left output [5]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29] -Right output [2]: [s_county#30, s_state#32] -Arguments: [ca_county#27, ca_state#29], [s_county#30, s_state#32], Inner, BuildRight - -(44) CometProject -Input [7]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29, s_county#30, s_state#32] -Arguments: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24], [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_month_seq#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [d_date_sk#33, d_month_seq#34] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= ReusedSubquery Subquery scalar-subquery#35, [id=#36])) AND (d_month_seq#34 <= ReusedSubquery Subquery scalar-subquery#37, [id=#38])) AND isnotnull(d_date_sk#33)) - -(47) CometProject -Input [2]: [d_date_sk#33, d_month_seq#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(48) CometBroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: [d_date_sk#33] - -(49) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] -Right output [1]: [d_date_sk#33] -Arguments: [ss_sold_date_sk#24], [d_date_sk#33], Inner, BuildRight - -(50) CometProject -Input [4]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, d_date_sk#33] -Arguments: [c_customer_sk#20, ss_ext_sales_price#23], [c_customer_sk#20, ss_ext_sales_price#23] - -(51) CometHashAggregate -Input [2]: [c_customer_sk#20, ss_ext_sales_price#23] -Keys [1]: [c_customer_sk#20] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#23))] - -(52) CometExchange -Input [2]: [c_customer_sk#20, sum#39] -Arguments: hashpartitioning(c_customer_sk#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(53) CometHashAggregate -Input [2]: [c_customer_sk#20, sum#39] -Keys [1]: [c_customer_sk#20] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#23))] - -(54) CometHashAggregate -Input [1]: [segment#40] -Keys [1]: [segment#40] -Functions [1]: [partial_count(1)] - -(55) CometExchange -Input [2]: [segment#40, count#41] -Arguments: hashpartitioning(segment#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(56) CometHashAggregate -Input [2]: [segment#40, count#41] -Keys [1]: [segment#40] -Functions [1]: [count(1)] - -(57) CometTakeOrderedAndProject -Input [3]: [segment#40, num_customers#42, segment_base#43] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[segment#40 ASC NULLS FIRST,num_customers#42 ASC NULLS FIRST], output=[segment#40,num_customers#42,segment_base#43]), [segment#40, num_customers#42, segment_base#43], 100, 0, [segment#40 ASC NULLS FIRST, num_customers#42 ASC NULLS FIRST], [segment#40, num_customers#42, segment_base#43] - -(58) CometColumnarToRow [codegen id : 1] -Input [3]: [segment#40, num_customers#42, segment_base#43] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (63) -+- * CometColumnarToRow (62) - +- CometProject (61) - +- CometFilter (60) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (59) - - -(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(60) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12)) AND (d_year#18 = 1998)) AND isnotnull(d_date_sk#17)) - -(61) CometProject -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(62) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] - -(63) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 - -Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 -BroadcastExchange (68) -+- * CometColumnarToRow (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (64) - - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_month_seq#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(65) CometFilter -Input [2]: [d_date_sk#33, d_month_seq#34] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= Subquery scalar-subquery#35, [id=#36])) AND (d_month_seq#34 <= Subquery scalar-subquery#37, [id=#38])) AND isnotnull(d_date_sk#33)) - -(66) CometProject -Input [2]: [d_date_sk#33, d_month_seq#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(67) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#33] - -(68) BroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:4 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#35, [id=#36] -* CometColumnarToRow (75) -+- CometHashAggregate (74) - +- CometExchange (73) - +- CometHashAggregate (72) - +- CometProject (71) - +- CometFilter (70) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (69) - - -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#44, d_year#45, d_moy#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] -ReadSchema: struct - -(70) CometFilter -Input [3]: [d_month_seq#44, d_year#45, d_moy#46] -Condition : (((isnotnull(d_year#45) AND isnotnull(d_moy#46)) AND (d_year#45 = 1998)) AND (d_moy#46 = 12)) - -(71) CometProject -Input [3]: [d_month_seq#44, d_year#45, d_moy#46] -Arguments: [(d_month_seq + 1)#47], [(d_month_seq#44 + 1) AS (d_month_seq + 1)#47] - -(72) CometHashAggregate -Input [1]: [(d_month_seq + 1)#47] -Keys [1]: [(d_month_seq + 1)#47] -Functions: [] - -(73) CometExchange -Input [1]: [(d_month_seq + 1)#47] -Arguments: hashpartitioning((d_month_seq + 1)#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(74) CometHashAggregate -Input [1]: [(d_month_seq + 1)#47] -Keys [1]: [(d_month_seq + 1)#47] -Functions: [] - -(75) CometColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 1)#47] - -Subquery:5 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#37, [id=#38] -* CometColumnarToRow (82) -+- CometHashAggregate (81) - +- CometExchange (80) - +- CometHashAggregate (79) - +- CometProject (78) - +- CometFilter (77) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) - - -(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#48, d_year#49, d_moy#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] -ReadSchema: struct - -(77) CometFilter -Input [3]: [d_month_seq#48, d_year#49, d_moy#50] -Condition : (((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 1998)) AND (d_moy#50 = 12)) - -(78) CometProject -Input [3]: [d_month_seq#48, d_year#49, d_moy#50] -Arguments: [(d_month_seq + 3)#51], [(d_month_seq#48 + 3) AS (d_month_seq + 3)#51] - -(79) CometHashAggregate -Input [1]: [(d_month_seq + 3)#51] -Keys [1]: [(d_month_seq + 3)#51] -Functions: [] - -(80) CometExchange -Input [1]: [(d_month_seq + 3)#51] -Arguments: hashpartitioning((d_month_seq + 3)#51, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(81) CometHashAggregate -Input [1]: [(d_month_seq + 3)#51] -Keys [1]: [(d_month_seq + 3)#51] -Functions: [] - -(82) CometColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 3)#51] - -Subquery:6 Hosting operator id = 46 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#36] - -Subquery:7 Hosting operator id = 46 Hosting Expression = ReusedSubquery Subquery scalar-subquery#37, [id=#38] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/extended.txt deleted file mode 100644 index 98fce92fb1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/extended.txt +++ /dev/null @@ -1,91 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometUnion - : : : : : : : :- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : :- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : :- ReusedSubquery - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 75 out of 84 eligible operators (89%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index ae0cecb1ba..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54/simplified.txt +++ /dev/null @@ -1,99 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [segment,num_customers,segment_base] - CometHashAggregate [count] [segment,num_customers,segment_base,count(1)] - CometExchange [segment] #1 - CometHashAggregate [segment,count] - CometHashAggregate [c_customer_sk,sum] [segment,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [c_customer_sk] #2 - CometHashAggregate [ss_ext_sales_price] [c_customer_sk,sum] - CometProject [c_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state,s_county,s_state] - CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] - CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk,ca_address_sk,ca_county,ca_state] - CometProject [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometHashAggregate [c_customer_sk,c_current_addr_sk] - CometExchange [c_customer_sk,c_current_addr_sk] #3 - CometHashAggregate [c_customer_sk,c_current_addr_sk] - CometProject [c_customer_sk,c_current_addr_sk] - CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] - CometProject [customer_sk] - CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] - CometProject [sold_date_sk,customer_sk] - 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_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [i_item_sk] #5 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] #8 - CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometExchange [(d_month_seq + 1)] #10 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #11 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state] #12 - CometProject [ca_state] [ca_address_sk,ca_county,ca_state] - CometFilter [ca_address_sk,ca_county,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] - CometBroadcastExchange [s_county,s_state] #13 - CometProject [s_state] [s_county,s_state] - CometFilter [s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_county,s_state] - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [(d_month_seq + 1)] #3 - ReusedSubquery [(d_month_seq + 3)] #4 - CometScan [native_iceberg_compat] 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/q55.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_datafusion/explain.txt deleted file mode 100644 index 358eb94c69..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_datafusion/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometNativeScan parquet spark_catalog.default.item (9) - - -(1) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1999)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1], [d_date_sk#1] - -(4) CometNativeScan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [1]: [d_date_sk#1] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [4]: [d_date_sk#1, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5], [ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,28), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 28)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [3]: [ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [2]: [i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [3]: [i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [i_brand#11, i_brand_id#8, sum#12] -Keys [2]: [i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [3]: [brand_id#13, brand#14, ext_price#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[brand_id#13,brand#14,ext_price#15]), [brand_id#13, brand#14, ext_price#15], 100, 0, [ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [brand_id#13, brand#14, ext_price#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [3]: [brand_id#13, brand#14, ext_price#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_datafusion/simplified.txt deleted file mode 100644 index a2adacc531..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_datafusion/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,ext_price] - CometHashAggregate [sum] [brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,sum] - CometProject [ss_ext_sales_price,i_brand_id,i_brand] - 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_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometNativeScan 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/q55.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_iceberg_compat/explain.txt deleted file mode 100644 index 47e0117481..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1999)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1], [d_date_sk#1] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [1]: [d_date_sk#1] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [4]: [d_date_sk#1, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5], [ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,28), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 28)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [3]: [ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [2]: [i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [3]: [i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [i_brand#11, i_brand_id#8, sum#12] -Keys [2]: [i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [3]: [brand_id#13, brand#14, ext_price#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[brand_id#13,brand#14,ext_price#15]), [brand_id#13, brand#14, ext_price#15], 100, 0, [ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [brand_id#13, brand#14, ext_price#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [3]: [brand_id#13, brand#14, ext_price#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_iceberg_compat/simplified.txt deleted file mode 100644 index dd7658cbac..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,ext_price] - CometHashAggregate [sum] [brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,sum] - CometProject [ss_ext_sales_price,i_brand_id,i_brand] - 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_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan [native_iceberg_compat] 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/q55/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/explain.txt deleted file mode 100644 index 47e0117481..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1999)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1], [d_date_sk#1] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [1]: [d_date_sk#1] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [4]: [d_date_sk#1, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5], [ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,28), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 28)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [3]: [ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [2]: [i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [3]: [i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [i_brand#11, i_brand_id#8, sum#12] -Keys [2]: [i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [3]: [brand_id#13, brand#14, ext_price#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[brand_id#13,brand#14,ext_price#15]), [brand_id#13, brand#14, ext_price#15], 100, 0, [ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [brand_id#13, brand#14, ext_price#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [3]: [brand_id#13, brand#14, ext_price#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/extended.txt deleted file mode 100644 index a292badf5d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index dd7658cbac..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q55/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,ext_price] - CometHashAggregate [sum] [brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,sum] - CometProject [ss_ext_sales_price,i_brand_id,i_brand] - 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_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/explain.txt deleted file mode 100644 index 62396854a7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/explain.txt +++ /dev/null @@ -1,420 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (67) -+- * HashAggregate (66) - +- * CometColumnarToRow (65) - +- CometColumnarExchange (64) - +- * HashAggregate (63) - +- Union (62) - :- * HashAggregate (29) - : +- * CometColumnarToRow (28) - : +- CometColumnarExchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * CometColumnarToRow (22) - : +- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.item (14) - : +- CometBroadcastExchange (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometNativeScan parquet spark_catalog.default.item (16) - :- * HashAggregate (45) - : +- * CometColumnarToRow (44) - : +- CometColumnarExchange (43) - : +- * HashAggregate (42) - : +- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (38) - : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * Filter (32) - : : : : +- * ColumnarToRow (31) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (30) - : : : +- ReusedExchange (33) - : : +- ReusedExchange (36) - : +- ReusedExchange (39) - +- * HashAggregate (61) - +- * CometColumnarToRow (60) - +- CometColumnarExchange (59) - +- * HashAggregate (58) - +- * Project (57) - +- * BroadcastHashJoin Inner BuildRight (56) - :- * Project (54) - : +- * BroadcastHashJoin Inner BuildRight (53) - : :- * Project (51) - : : +- * BroadcastHashJoin Inner BuildRight (50) - : : :- * Filter (48) - : : : +- * ColumnarToRow (47) - : : : +- Scan parquet spark_catalog.default.web_sales (46) - : : +- ReusedExchange (49) - : +- ReusedExchange (52) - +- ReusedExchange (55) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(4) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_gmt_offset#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isnotnull(ca_address_sk#7)) - -(9) CometProject -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Arguments: [ca_address_sk#7], [ca_address_sk#7] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [ca_address_sk#7] - -(11) BroadcastExchange -Input [1]: [ca_address_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] - -(14) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#9, i_item_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [i_item_sk#9, i_item_id#10] -Condition : isnotnull(i_item_sk#9) - -(16) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_id#11, i_color#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(17) CometFilter -Input [2]: [i_item_id#11, i_color#12] -Condition : staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#12, 20, true, false, true) IN (slate ,blanched ,burnished ) - -(18) CometProject -Input [2]: [i_item_id#11, i_color#12] -Arguments: [i_item_id#13], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#11, 16, true, false, true) AS i_item_id#13] - -(19) CometBroadcastExchange -Input [1]: [i_item_id#13] -Arguments: [i_item_id#13] - -(20) CometBroadcastHashJoin -Left output [2]: [i_item_sk#9, i_item_id#10] -Right output [1]: [i_item_id#13] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#10, 16, true, false, true)], [i_item_id#13], LeftSemi, BuildRight - -(21) CometProject -Input [2]: [i_item_sk#9, i_item_id#10] -Arguments: [i_item_sk#9, i_item_id#14], [i_item_sk#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#10, 16, true, false, true) AS i_item_id#14] - -(22) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#9, i_item_id#14] - -(23) BroadcastExchange -Input [2]: [i_item_sk#9, i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#9] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 4] -Output [2]: [ss_ext_sales_price#3, i_item_id#14] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_item_id#14] - -(26) HashAggregate [codegen id : 4] -Input [2]: [ss_ext_sales_price#3, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum#15] -Results [2]: [i_item_id#14, sum#16] - -(27) CometColumnarExchange -Input [2]: [i_item_id#14, sum#16] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometColumnarToRow [codegen id : 5] -Input [2]: [i_item_id#14, sum#16] - -(29) HashAggregate [codegen id : 5] -Input [2]: [i_item_id#14, sum#16] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#17] -Results [2]: [i_item_id#14 AS i_item_id#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#17,17,2) AS total_sales#19] - -(30) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 9] -Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] - -(32) Filter [codegen id : 9] -Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -Condition : (isnotnull(cs_bill_addr_sk#20) AND isnotnull(cs_item_sk#21)) - -(33) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#24] - -(34) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 9] -Output [3]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22] -Input [5]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#24] - -(36) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#25] - -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_bill_addr_sk#20] -Right keys [1]: [ca_address_sk#25] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 9] -Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] -Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, ca_address_sk#25] - -(39) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#26, i_item_id#27] - -(40) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#21] -Right keys [1]: [i_item_sk#26] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 9] -Output [2]: [cs_ext_sales_price#22, i_item_id#27] -Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#26, i_item_id#27] - -(42) HashAggregate [codegen id : 9] -Input [2]: [cs_ext_sales_price#22, i_item_id#27] -Keys [1]: [i_item_id#27] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#27, sum#29] - -(43) CometColumnarExchange -Input [2]: [i_item_id#27, sum#29] -Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(44) CometColumnarToRow [codegen id : 10] -Input [2]: [i_item_id#27, sum#29] - -(45) HashAggregate [codegen id : 10] -Input [2]: [i_item_id#27, sum#29] -Keys [1]: [i_item_id#27] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] -Results [2]: [i_item_id#27, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS total_sales#31] - -(46) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(47) ColumnarToRow [codegen id : 14] -Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] - -(48) Filter [codegen id : 14] -Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] -Condition : (isnotnull(ws_bill_addr_sk#33) AND isnotnull(ws_item_sk#32)) - -(49) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#36] - -(50) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#36] -Join type: Inner -Join condition: None - -(51) Project [codegen id : 14] -Output [3]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34] -Input [5]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35, d_date_sk#36] - -(52) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#37] - -(53) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#33] -Right keys [1]: [ca_address_sk#37] -Join type: Inner -Join condition: None - -(54) Project [codegen id : 14] -Output [2]: [ws_item_sk#32, ws_ext_sales_price#34] -Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ca_address_sk#37] - -(55) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#38, i_item_id#39] - -(56) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_item_sk#32] -Right keys [1]: [i_item_sk#38] -Join type: Inner -Join condition: None - -(57) Project [codegen id : 14] -Output [2]: [ws_ext_sales_price#34, i_item_id#39] -Input [4]: [ws_item_sk#32, ws_ext_sales_price#34, i_item_sk#38, i_item_id#39] - -(58) HashAggregate [codegen id : 14] -Input [2]: [ws_ext_sales_price#34, i_item_id#39] -Keys [1]: [i_item_id#39] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#34))] -Aggregate Attributes [1]: [sum#40] -Results [2]: [i_item_id#39, sum#41] - -(59) CometColumnarExchange -Input [2]: [i_item_id#39, sum#41] -Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(60) CometColumnarToRow [codegen id : 15] -Input [2]: [i_item_id#39, sum#41] - -(61) HashAggregate [codegen id : 15] -Input [2]: [i_item_id#39, sum#41] -Keys [1]: [i_item_id#39] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#34))#42] -Results [2]: [i_item_id#39, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#34))#42,17,2) AS total_sales#43] - -(62) Union - -(63) HashAggregate [codegen id : 16] -Input [2]: [i_item_id#18, total_sales#19] -Keys [1]: [i_item_id#18] -Functions [1]: [partial_sum(total_sales#19)] -Aggregate Attributes [2]: [sum#44, isEmpty#45] -Results [3]: [i_item_id#18, sum#46, isEmpty#47] - -(64) CometColumnarExchange -Input [3]: [i_item_id#18, sum#46, isEmpty#47] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(65) CometColumnarToRow [codegen id : 17] -Input [3]: [i_item_id#18, sum#46, isEmpty#47] - -(66) HashAggregate [codegen id : 17] -Input [3]: [i_item_id#18, sum#46, isEmpty#47] -Keys [1]: [i_item_id#18] -Functions [1]: [sum(total_sales#19)] -Aggregate Attributes [1]: [sum(total_sales#19)#48] -Results [2]: [i_item_id#18, sum(total_sales#19)#48 AS total_sales#49] - -(67) TakeOrderedAndProject -Input [2]: [i_item_id#18, total_sales#49] -Arguments: 100, [total_sales#49 ASC NULLS FIRST], [i_item_id#18, total_sales#49] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (72) -+- * CometColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometNativeScan parquet spark_catalog.default.date_dim (68) - - -(68) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#50, d_moy#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [3]: [d_date_sk#6, d_year#50, d_moy#51] -Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 2001)) AND (d_moy#51 = 2)) AND isnotnull(d_date_sk#6)) - -(70) CometProject -Input [3]: [d_date_sk#6, d_year#50, d_moy#51] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(71) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(72) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/simplified.txt deleted file mode 100644 index 81d90bdfb2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/simplified.txt +++ /dev/null @@ -1,106 +0,0 @@ -TakeOrderedAndProject [total_sales,i_item_id] - WholeStageCodegen (17) - HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (16) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),i_item_id,total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #6 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_color] - CometNativeScan parquet spark_catalog.default.item [i_item_id,i_color] - WholeStageCodegen (10) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #7 - WholeStageCodegen (9) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_addr_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (15) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #8 - WholeStageCodegen (14) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/explain.txt deleted file mode 100644 index 5562e86b91..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,385 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (62) -+- CometTakeOrderedAndProject (61) - +- CometHashAggregate (60) - +- CometExchange (59) - +- CometHashAggregate (58) - +- CometUnion (57) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- CometHashAggregate (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometFilter (30) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- CometHashAggregate (56) - +- CometExchange (55) - +- CometHashAggregate (54) - +- CometProject (53) - +- CometBroadcastHashJoin (52) - :- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometFilter (44) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (43) - : : +- ReusedExchange (45) - : +- ReusedExchange (48) - +- ReusedExchange (51) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001)) AND (d_moy#8 = 2)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(11) CometProject -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Arguments: [ca_address_sk#9], [ca_address_sk#9] - -(12) CometBroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: [ca_address_sk#9] - -(13) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Right output [1]: [ca_address_sk#9] -Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] -Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#11, i_item_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#11, i_item_id#12] -Condition : isnotnull(i_item_sk#11) - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_id#13, i_color#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(18) CometFilter -Input [2]: [i_item_id#13, i_color#14] -Condition : staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#14, 20, true, false, true) IN (slate ,blanched ,burnished ) - -(19) CometProject -Input [2]: [i_item_id#13, i_color#14] -Arguments: [i_item_id#15], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#13, 16, true, false, true) AS i_item_id#15] - -(20) CometBroadcastExchange -Input [1]: [i_item_id#15] -Arguments: [i_item_id#15] - -(21) CometBroadcastHashJoin -Left output [2]: [i_item_sk#11, i_item_id#12] -Right output [1]: [i_item_id#15] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#12, 16, true, false, true)], [i_item_id#15], LeftSemi, BuildRight - -(22) CometProject -Input [2]: [i_item_sk#11, i_item_id#12] -Arguments: [i_item_sk#11, i_item_id#16], [i_item_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#12, 16, true, false, true) AS i_item_id#16] - -(23) CometBroadcastExchange -Input [2]: [i_item_sk#11, i_item_id#16] -Arguments: [i_item_sk#11, i_item_id#16] - -(24) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Right output [2]: [i_item_sk#11, i_item_id#16] -Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight - -(25) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_item_id#16] -Arguments: [ss_ext_sales_price#3, i_item_id#16], [ss_ext_sales_price#3, i_item_id#16] - -(26) CometHashAggregate -Input [2]: [ss_ext_sales_price#3, i_item_id#16] -Keys [1]: [i_item_id#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(27) CometExchange -Input [2]: [i_item_id#16, sum#17] -Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [2]: [i_item_id#16, sum#17] -Keys [1]: [i_item_id#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Condition : (isnotnull(cs_bill_addr_sk#18) AND isnotnull(cs_item_sk#19)) - -(31) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#23] - -(32) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Right output [1]: [d_date_sk#23] -Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(33) CometProject -Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] -Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] - -(34) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#24] - -(35) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -Right output [1]: [ca_address_sk#24] -Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner, BuildRight - -(36) CometProject -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] -Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] - -(37) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#25, i_item_id#26] - -(38) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] -Right output [2]: [i_item_sk#25, i_item_id#26] -Arguments: [cs_item_sk#19], [i_item_sk#25], Inner, BuildRight - -(39) CometProject -Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_item_id#26] -Arguments: [cs_ext_sales_price#20, i_item_id#26], [cs_ext_sales_price#20, i_item_id#26] - -(40) CometHashAggregate -Input [2]: [cs_ext_sales_price#20, i_item_id#26] -Keys [1]: [i_item_id#26] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] - -(41) CometExchange -Input [2]: [i_item_id#26, sum#27] -Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(42) CometHashAggregate -Input [2]: [i_item_id#26, sum#27] -Keys [1]: [i_item_id#26] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(44) CometFilter -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Condition : (isnotnull(ws_bill_addr_sk#29) AND isnotnull(ws_item_sk#28)) - -(45) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(46) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [ws_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(47) CometProject -Input [5]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31, d_date_sk#33] -Arguments: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30], [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] - -(48) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#34] - -(49) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] -Right output [1]: [ca_address_sk#34] -Arguments: [ws_bill_addr_sk#29], [ca_address_sk#34], Inner, BuildRight - -(50) CometProject -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ca_address_sk#34] -Arguments: [ws_item_sk#28, ws_ext_sales_price#30], [ws_item_sk#28, ws_ext_sales_price#30] - -(51) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#35, i_item_id#36] - -(52) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#28, ws_ext_sales_price#30] -Right output [2]: [i_item_sk#35, i_item_id#36] -Arguments: [ws_item_sk#28], [i_item_sk#35], Inner, BuildRight - -(53) CometProject -Input [4]: [ws_item_sk#28, ws_ext_sales_price#30, i_item_sk#35, i_item_id#36] -Arguments: [ws_ext_sales_price#30, i_item_id#36], [ws_ext_sales_price#30, i_item_id#36] - -(54) CometHashAggregate -Input [2]: [ws_ext_sales_price#30, i_item_id#36] -Keys [1]: [i_item_id#36] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#30))] - -(55) CometExchange -Input [2]: [i_item_id#36, sum#37] -Arguments: hashpartitioning(i_item_id#36, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(56) CometHashAggregate -Input [2]: [i_item_id#36, sum#37] -Keys [1]: [i_item_id#36] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#30))] - -(57) CometUnion -Child 0 Input [2]: [i_item_id#38, total_sales#39] -Child 1 Input [2]: [i_item_id#26, total_sales#40] -Child 2 Input [2]: [i_item_id#36, total_sales#41] - -(58) CometHashAggregate -Input [2]: [i_item_id#38, total_sales#39] -Keys [1]: [i_item_id#38] -Functions [1]: [partial_sum(total_sales#39)] - -(59) CometExchange -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(60) CometHashAggregate -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Keys [1]: [i_item_id#38] -Functions [1]: [sum(total_sales#39)] - -(61) CometTakeOrderedAndProject -Input [2]: [i_item_id#38, total_sales#44] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#44 ASC NULLS FIRST], output=[i_item_id#38,total_sales#44]), [i_item_id#38, total_sales#44], 100, 0, [total_sales#44 ASC NULLS FIRST], [i_item_id#38, total_sales#44] - -(62) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_id#38, total_sales#44] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (67) -+- * CometColumnarToRow (66) - +- CometProject (65) - +- CometFilter (64) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) - - -(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(64) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001)) AND (d_moy#8 = 2)) AND isnotnull(d_date_sk#6)) - -(65) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(66) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(67) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/simplified.txt deleted file mode 100644 index 19eca782cd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,total_sales] - CometHashAggregate [sum,isEmpty] [i_item_id,total_sales,sum(total_sales)] - CometExchange [i_item_id] #1 - CometHashAggregate [total_sales] [i_item_id,sum,isEmpty] - CometUnion [i_item_id,total_sales] - CometHashAggregate [sum] [i_item_id,total_sales,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #7 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_color] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_color] - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - 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,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/explain.txt deleted file mode 100644 index 5562e86b91..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/explain.txt +++ /dev/null @@ -1,385 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (62) -+- CometTakeOrderedAndProject (61) - +- CometHashAggregate (60) - +- CometExchange (59) - +- CometHashAggregate (58) - +- CometUnion (57) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- CometHashAggregate (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometFilter (30) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- CometHashAggregate (56) - +- CometExchange (55) - +- CometHashAggregate (54) - +- CometProject (53) - +- CometBroadcastHashJoin (52) - :- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometFilter (44) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (43) - : : +- ReusedExchange (45) - : +- ReusedExchange (48) - +- ReusedExchange (51) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001)) AND (d_moy#8 = 2)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(11) CometProject -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Arguments: [ca_address_sk#9], [ca_address_sk#9] - -(12) CometBroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: [ca_address_sk#9] - -(13) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Right output [1]: [ca_address_sk#9] -Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] -Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#11, i_item_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#11, i_item_id#12] -Condition : isnotnull(i_item_sk#11) - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_id#13, i_color#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(18) CometFilter -Input [2]: [i_item_id#13, i_color#14] -Condition : staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#14, 20, true, false, true) IN (slate ,blanched ,burnished ) - -(19) CometProject -Input [2]: [i_item_id#13, i_color#14] -Arguments: [i_item_id#15], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#13, 16, true, false, true) AS i_item_id#15] - -(20) CometBroadcastExchange -Input [1]: [i_item_id#15] -Arguments: [i_item_id#15] - -(21) CometBroadcastHashJoin -Left output [2]: [i_item_sk#11, i_item_id#12] -Right output [1]: [i_item_id#15] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#12, 16, true, false, true)], [i_item_id#15], LeftSemi, BuildRight - -(22) CometProject -Input [2]: [i_item_sk#11, i_item_id#12] -Arguments: [i_item_sk#11, i_item_id#16], [i_item_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#12, 16, true, false, true) AS i_item_id#16] - -(23) CometBroadcastExchange -Input [2]: [i_item_sk#11, i_item_id#16] -Arguments: [i_item_sk#11, i_item_id#16] - -(24) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Right output [2]: [i_item_sk#11, i_item_id#16] -Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight - -(25) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_item_id#16] -Arguments: [ss_ext_sales_price#3, i_item_id#16], [ss_ext_sales_price#3, i_item_id#16] - -(26) CometHashAggregate -Input [2]: [ss_ext_sales_price#3, i_item_id#16] -Keys [1]: [i_item_id#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(27) CometExchange -Input [2]: [i_item_id#16, sum#17] -Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [2]: [i_item_id#16, sum#17] -Keys [1]: [i_item_id#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Condition : (isnotnull(cs_bill_addr_sk#18) AND isnotnull(cs_item_sk#19)) - -(31) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#23] - -(32) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Right output [1]: [d_date_sk#23] -Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(33) CometProject -Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] -Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] - -(34) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#24] - -(35) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -Right output [1]: [ca_address_sk#24] -Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner, BuildRight - -(36) CometProject -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] -Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] - -(37) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#25, i_item_id#26] - -(38) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] -Right output [2]: [i_item_sk#25, i_item_id#26] -Arguments: [cs_item_sk#19], [i_item_sk#25], Inner, BuildRight - -(39) CometProject -Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_item_id#26] -Arguments: [cs_ext_sales_price#20, i_item_id#26], [cs_ext_sales_price#20, i_item_id#26] - -(40) CometHashAggregate -Input [2]: [cs_ext_sales_price#20, i_item_id#26] -Keys [1]: [i_item_id#26] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] - -(41) CometExchange -Input [2]: [i_item_id#26, sum#27] -Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(42) CometHashAggregate -Input [2]: [i_item_id#26, sum#27] -Keys [1]: [i_item_id#26] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(44) CometFilter -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Condition : (isnotnull(ws_bill_addr_sk#29) AND isnotnull(ws_item_sk#28)) - -(45) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(46) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [ws_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(47) CometProject -Input [5]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31, d_date_sk#33] -Arguments: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30], [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] - -(48) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#34] - -(49) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] -Right output [1]: [ca_address_sk#34] -Arguments: [ws_bill_addr_sk#29], [ca_address_sk#34], Inner, BuildRight - -(50) CometProject -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ca_address_sk#34] -Arguments: [ws_item_sk#28, ws_ext_sales_price#30], [ws_item_sk#28, ws_ext_sales_price#30] - -(51) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#35, i_item_id#36] - -(52) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#28, ws_ext_sales_price#30] -Right output [2]: [i_item_sk#35, i_item_id#36] -Arguments: [ws_item_sk#28], [i_item_sk#35], Inner, BuildRight - -(53) CometProject -Input [4]: [ws_item_sk#28, ws_ext_sales_price#30, i_item_sk#35, i_item_id#36] -Arguments: [ws_ext_sales_price#30, i_item_id#36], [ws_ext_sales_price#30, i_item_id#36] - -(54) CometHashAggregate -Input [2]: [ws_ext_sales_price#30, i_item_id#36] -Keys [1]: [i_item_id#36] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#30))] - -(55) CometExchange -Input [2]: [i_item_id#36, sum#37] -Arguments: hashpartitioning(i_item_id#36, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(56) CometHashAggregate -Input [2]: [i_item_id#36, sum#37] -Keys [1]: [i_item_id#36] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#30))] - -(57) CometUnion -Child 0 Input [2]: [i_item_id#38, total_sales#39] -Child 1 Input [2]: [i_item_id#26, total_sales#40] -Child 2 Input [2]: [i_item_id#36, total_sales#41] - -(58) CometHashAggregate -Input [2]: [i_item_id#38, total_sales#39] -Keys [1]: [i_item_id#38] -Functions [1]: [partial_sum(total_sales#39)] - -(59) CometExchange -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(60) CometHashAggregate -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Keys [1]: [i_item_id#38] -Functions [1]: [sum(total_sales#39)] - -(61) CometTakeOrderedAndProject -Input [2]: [i_item_id#38, total_sales#44] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#44 ASC NULLS FIRST], output=[i_item_id#38,total_sales#44]), [i_item_id#38, total_sales#44], 100, 0, [total_sales#44 ASC NULLS FIRST], [i_item_id#38, total_sales#44] - -(62) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_id#38, total_sales#44] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (67) -+- * CometColumnarToRow (66) - +- CometProject (65) - +- CometFilter (64) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) - - -(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(64) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001)) AND (d_moy#8 = 2)) AND isnotnull(d_date_sk#6)) - -(65) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(66) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(67) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/extended.txt deleted file mode 100644 index 36b9a8306d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/extended.txt +++ /dev/null @@ -1,100 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 19eca782cd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,total_sales] - CometHashAggregate [sum,isEmpty] [i_item_id,total_sales,sum(total_sales)] - CometExchange [i_item_id] #1 - CometHashAggregate [total_sales] [i_item_id,sum,isEmpty] - CometUnion [i_item_id,total_sales] - CometHashAggregate [sum] [i_item_id,total_sales,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #7 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_color] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_color] - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - 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,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/explain.txt deleted file mode 100644 index c3ba92766c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/explain.txt +++ /dev/null @@ -1,304 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (51) -+- * Project (50) - +- * BroadcastHashJoin Inner BuildRight (49) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * CometColumnarToRow (26) - : : +- CometSort (25) - : : +- CometColumnarExchange (24) - : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- * HashAggregate (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- Window (38) - : +- * CometColumnarToRow (37) - : +- CometSort (36) - : +- CometColumnarExchange (35) - : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) - : +- ReusedExchange (32) - +- BroadcastExchange (48) - +- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- ReusedExchange (43) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#4, i_category#5] - -(5) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(7) Filter [codegen id : 1] -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) - -(8) BroadcastExchange -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#7] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(11) ReusedExchange [Reuses operator id: 55] -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] - -(14) CometNativeScan parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#14, cc_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#14, cc_name#15] -Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) - -(16) CometColumnarToRow [codegen id : 3] -Input [2]: [cc_call_center_sk#14, cc_name#15] - -(17) BroadcastExchange -Input [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_call_center_sk#6] -Right keys [1]: [cc_call_center_sk#14] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] - -(20) HashAggregate [codegen id : 4] -Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] -Aggregate Attributes [1]: [sum#16] -Results [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] - -(21) CometColumnarExchange -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] - -(23) HashAggregate [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#8))#18] -Results [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#18,17,2) AS sum_sales#19, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#18,17,2) AS _w0#20] - -(24) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) CometColumnarToRow [codegen id : 6] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] - -(27) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(28) Filter [codegen id : 7] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(29) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] -Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(30) Filter [codegen id : 22] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] -Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) - -(31) Project [codegen id : 22] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] - -(32) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] - -(33) CometColumnarToRow [codegen id : 12] -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] - -(34) HashAggregate [codegen id : 12] -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] -Keys [5]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#27))#18] -Results [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#18,17,2) AS sum_sales#19] - -(35) CometColumnarExchange -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(36) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] -Arguments: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(37) CometColumnarToRow [codegen id : 13] -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] - -(38) Window -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(39) Project [codegen id : 14] -Output [5]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, cc_name#23, sum_sales#19 AS sum_sales#31, rn#28] -Input [7]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19, rn#28] - -(40) BroadcastExchange -Input [5]: [i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] - -(41) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] -Right keys [4]: [i_category#29, i_brand#30, cc_name#23, (rn#28 + 1)] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 22] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] - -(43) ReusedExchange [Reuses operator id: 35] -Output [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] - -(44) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] -Arguments: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] - -(45) CometColumnarToRow [codegen id : 20] -Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] - -(46) Window -Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] -Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#5, i_brand#4, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#5, i_brand#4, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] - -(47) Project [codegen id : 21] -Output [5]: [i_category#5 AS i_category#36, i_brand#4 AS i_brand#37, cc_name#32, sum_sales#19 AS sum_sales#38, rn#35] -Input [7]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19, rn#35] - -(48) BroadcastExchange -Input [5]: [i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] - -(49) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] -Right keys [4]: [i_category#36, i_brand#37, cc_name#32, (rn#35 - 1)] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 22] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#39, sum_sales#38 AS nsum#40] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] - -(51) TakeOrderedAndProject -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] -Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometFilter (53) - +- CometNativeScan parquet spark_catalog.default.date_dim (52) - - -(52) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(54) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(55) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/simplified.txt deleted file mode 100644 index 4f9ac35f4d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/simplified.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (22) - Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (7) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [cs_item_sk,cs_call_center_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk,cc_name] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/explain.txt deleted file mode 100644 index eb21a3abd7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) - -(6) CometBroadcastExchange -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_item_sk#1], [cs_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#14, cc_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#14, cc_name#15] -Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) - -(16) CometBroadcastExchange -Input [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cc_call_center_sk#14, cc_name#15] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] -Right output [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cs_call_center_sk#6], [cc_call_center_sk#14], Inner, BuildRight - -(18) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] -Arguments: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15], [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] - -(19) CometHashAggregate -Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] - -(20) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] - -(22) CometExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] - -(25) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(28) Filter [codegen id : 7] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] - -(30) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] - -(31) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] -Keys [5]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23] -Functions [1]: [sum(UnscaledValue(cs_sales_price#25))] - -(32) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] - -(35) Window -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [5]: [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] -Input [7]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26] - -(37) BroadcastExchange -Input [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] - -(40) ReusedExchange [Reuses operator id: 32] -Output [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(41) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(43) Window -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#5, i_brand#4, cc_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#5, i_brand#4, cc_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [5]: [i_category#5 AS i_category#34, i_brand#4 AS i_brand#35, cc_name#30, sum_sales#17 AS sum_sales#36, rn#33] -Input [7]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17, rn#33] - -(45) BroadcastExchange -Input [5]: [i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#34, i_brand#35, cc_name#30, (rn#33 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] - -(48) TakeOrderedAndProject -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/simplified.txt deleted file mode 100644 index b854e818db..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - 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 [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,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/explain.txt deleted file mode 100644 index eb21a3abd7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) - -(6) CometBroadcastExchange -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_item_sk#1], [cs_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#14, cc_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#14, cc_name#15] -Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) - -(16) CometBroadcastExchange -Input [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cc_call_center_sk#14, cc_name#15] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] -Right output [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cs_call_center_sk#6], [cc_call_center_sk#14], Inner, BuildRight - -(18) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] -Arguments: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15], [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] - -(19) CometHashAggregate -Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] - -(20) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] - -(22) CometExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] - -(25) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(28) Filter [codegen id : 7] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] - -(30) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] - -(31) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] -Keys [5]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23] -Functions [1]: [sum(UnscaledValue(cs_sales_price#25))] - -(32) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] - -(35) Window -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [5]: [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] -Input [7]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26] - -(37) BroadcastExchange -Input [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] - -(40) ReusedExchange [Reuses operator id: 32] -Output [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(41) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(43) Window -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#5, i_brand#4, cc_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#5, i_brand#4, cc_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [5]: [i_category#5 AS i_category#34, i_brand#4 AS i_brand#35, cc_name#30, sum_sales#17 AS sum_sales#36, rn#33] -Input [7]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17, rn#33] - -(45) BroadcastExchange -Input [5]: [i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#34, i_brand#35, cc_name#30, (rn#33 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] - -(48) TakeOrderedAndProject -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/extended.txt deleted file mode 100644 index de7d26cd06..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index b854e818db..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - 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 [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,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/explain.txt deleted file mode 100644 index 34406110bc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/explain.txt +++ /dev/null @@ -1,396 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (53) -+- * Project (52) - +- * BroadcastHashJoin Inner BuildRight (51) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Filter (18) - : : +- * HashAggregate (17) - : : +- * CometColumnarToRow (16) - : : +- CometColumnarExchange (15) - : : +- * HashAggregate (14) - : : +- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (33) - : +- * Filter (32) - : +- * HashAggregate (31) - : +- * CometColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- * HashAggregate (28) - : +- * Project (27) - : +- * BroadcastHashJoin Inner BuildRight (26) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Filter (21) - : : : +- * ColumnarToRow (20) - : : : +- Scan parquet spark_catalog.default.catalog_sales (19) - : : +- ReusedExchange (22) - : +- ReusedExchange (25) - +- BroadcastExchange (50) - +- * Filter (49) - +- * HashAggregate (48) - +- * CometColumnarToRow (47) - +- CometColumnarExchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Filter (38) - : : +- * ColumnarToRow (37) - : : +- Scan parquet spark_catalog.default.web_sales (36) - : +- ReusedExchange (39) - +- ReusedExchange (42) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_item_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [i_item_sk#5, i_item_id#6] -Condition : (isnotnull(i_item_sk#5) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true))) - -(6) CometProject -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#7] - -(7) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_sk#5, i_item_id#7] - -(8) BroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7] -Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#7] - -(11) ReusedExchange [Reuses operator id: 63] -Output [1]: [d_date_sk#8] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#8] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [2]: [ss_ext_sales_price#2, i_item_id#7] -Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7, d_date_sk#8] - -(14) HashAggregate [codegen id : 3] -Input [2]: [ss_ext_sales_price#2, i_item_id#7] -Keys [1]: [i_item_id#7] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#9] -Results [2]: [i_item_id#7, sum#10] - -(15) CometColumnarExchange -Input [2]: [i_item_id#7, sum#10] -Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 12] -Input [2]: [i_item_id#7, sum#10] - -(17) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#7, sum#10] -Keys [1]: [i_item_id#7] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#11] -Results [2]: [i_item_id#7 AS item_id#12, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#11,17,2) AS ss_item_rev#13] - -(18) Filter [codegen id : 12] -Input [2]: [item_id#12, ss_item_rev#13] -Condition : isnotnull(ss_item_rev#13) - -(19) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(20) ColumnarToRow [codegen id : 6] -Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] - -(21) Filter [codegen id : 6] -Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#14) - -(22) ReusedExchange [Reuses operator id: 8] -Output [2]: [i_item_sk#17, i_item_id#18] - -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_item_sk#14] -Right keys [1]: [i_item_sk#17] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 6] -Output [3]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#18] -Input [5]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_sk#17, i_item_id#18] - -(25) ReusedExchange [Reuses operator id: 63] -Output [1]: [d_date_sk#19] - -(26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 6] -Output [2]: [cs_ext_sales_price#15, i_item_id#18] -Input [4]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#18, d_date_sk#19] - -(28) HashAggregate [codegen id : 6] -Input [2]: [cs_ext_sales_price#15, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#15))] -Aggregate Attributes [1]: [sum#20] -Results [2]: [i_item_id#18, sum#21] - -(29) CometColumnarExchange -Input [2]: [i_item_id#18, sum#21] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(30) CometColumnarToRow [codegen id : 7] -Input [2]: [i_item_id#18, sum#21] - -(31) HashAggregate [codegen id : 7] -Input [2]: [i_item_id#18, sum#21] -Keys [1]: [i_item_id#18] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#15))#22] -Results [2]: [i_item_id#18 AS item_id#23, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#15))#22,17,2) AS cs_item_rev#24] - -(32) Filter [codegen id : 7] -Input [2]: [item_id#23, cs_item_rev#24] -Condition : isnotnull(cs_item_rev#24) - -(33) BroadcastExchange -Input [2]: [item_id#23, cs_item_rev#24] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] - -(34) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [item_id#12] -Right keys [1]: [item_id#23] -Join type: Inner -Join condition: ((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * cs_item_rev#24)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) - -(35) Project [codegen id : 12] -Output [3]: [item_id#12, ss_item_rev#13, cs_item_rev#24] -Input [4]: [item_id#12, ss_item_rev#13, item_id#23, cs_item_rev#24] - -(36) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(37) ColumnarToRow [codegen id : 10] -Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] - -(38) Filter [codegen id : 10] -Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#25) - -(39) ReusedExchange [Reuses operator id: 8] -Output [2]: [i_item_sk#28, i_item_id#29] - -(40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_item_sk#25] -Right keys [1]: [i_item_sk#28] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 10] -Output [3]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#29] -Input [5]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_sk#28, i_item_id#29] - -(42) ReusedExchange [Reuses operator id: 63] -Output [1]: [d_date_sk#30] - -(43) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#30] -Join type: Inner -Join condition: None - -(44) Project [codegen id : 10] -Output [2]: [ws_ext_sales_price#26, i_item_id#29] -Input [4]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#29, d_date_sk#30] - -(45) HashAggregate [codegen id : 10] -Input [2]: [ws_ext_sales_price#26, i_item_id#29] -Keys [1]: [i_item_id#29] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#31] -Results [2]: [i_item_id#29, sum#32] - -(46) CometColumnarExchange -Input [2]: [i_item_id#29, sum#32] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(47) CometColumnarToRow [codegen id : 11] -Input [2]: [i_item_id#29, sum#32] - -(48) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#29, sum#32] -Keys [1]: [i_item_id#29] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#26))#33] -Results [2]: [i_item_id#29 AS item_id#34, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#26))#33,17,2) AS ws_item_rev#35] - -(49) Filter [codegen id : 11] -Input [2]: [item_id#34, ws_item_rev#35] -Condition : isnotnull(ws_item_rev#35) - -(50) BroadcastExchange -Input [2]: [item_id#34, ws_item_rev#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] - -(51) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [item_id#12] -Right keys [1]: [item_id#34] -Join type: Inner -Join condition: ((((((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * ws_item_rev#35)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * ws_item_rev#35))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ws_item_rev#35))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ws_item_rev#35))) AND (cast(ws_item_rev#35 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(ws_item_rev#35 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) AND (cast(ws_item_rev#35 as decimal(19,3)) >= (0.9 * cs_item_rev#24))) AND (cast(ws_item_rev#35 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) - -(52) Project [codegen id : 12] -Output [8]: [item_id#12, ss_item_rev#13, (((ss_item_rev#13 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35)) / 3) * 100) AS ss_dev#36, cs_item_rev#24, (((cs_item_rev#24 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35)) / 3) * 100) AS cs_dev#37, ws_item_rev#35, (((ws_item_rev#35 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35)) / 3) * 100) AS ws_dev#38, (((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35) / 3) AS average#39] -Input [5]: [item_id#12, ss_item_rev#13, cs_item_rev#24, item_id#34, ws_item_rev#35] - -(53) TakeOrderedAndProject -Input [8]: [item_id#12, ss_item_rev#13, ss_dev#36, cs_item_rev#24, cs_dev#37, ws_item_rev#35, ws_dev#38, average#39] -Arguments: 100, [item_id#12 ASC NULLS FIRST, ss_item_rev#13 ASC NULLS FIRST], [item_id#12, ss_item_rev#13, ss_dev#36, cs_item_rev#24, cs_dev#37, ws_item_rev#35, ws_dev#38, average#39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (63) -+- * CometColumnarToRow (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometFilter (55) - : +- CometNativeScan parquet spark_catalog.default.date_dim (54) - +- CometBroadcastExchange (59) - +- CometProject (58) - +- CometFilter (57) - +- CometNativeScan parquet spark_catalog.default.date_dim (56) - - -(54) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(55) CometFilter -Input [2]: [d_date_sk#8, d_date#40] -Condition : isnotnull(d_date_sk#8) - -(56) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date#41, d_week_seq#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] -ReadSchema: struct - -(57) CometFilter -Input [2]: [d_date#41, d_week_seq#42] -Condition : (isnotnull(d_week_seq#42) AND (d_week_seq#42 = Subquery scalar-subquery#43, [id=#44])) - -(58) CometProject -Input [2]: [d_date#41, d_week_seq#42] -Arguments: [d_date#41], [d_date#41] - -(59) CometBroadcastExchange -Input [1]: [d_date#41] -Arguments: [d_date#41] - -(60) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#40] -Right output [1]: [d_date#41] -Arguments: [d_date#40], [d_date#41], LeftSemi, BuildRight - -(61) CometProject -Input [2]: [d_date_sk#8, d_date#40] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(62) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(63) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 57 Hosting Expression = Subquery scalar-subquery#43, [id=#44] -* CometColumnarToRow (67) -+- CometProject (66) - +- CometFilter (65) - +- CometNativeScan parquet spark_catalog.default.date_dim (64) - - -(64) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date#45, d_week_seq#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] -ReadSchema: struct - -(65) CometFilter -Input [2]: [d_date#45, d_week_seq#46] -Condition : (isnotnull(d_date#45) AND (d_date#45 = 2000-01-03)) - -(66) CometProject -Input [2]: [d_date#45, d_week_seq#46] -Arguments: [d_week_seq#46], [d_week_seq#46] - -(67) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#46] - -Subquery:3 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#4 - -Subquery:4 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/simplified.txt deleted file mode 100644 index 94adc941c8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/simplified.txt +++ /dev/null @@ -1,97 +0,0 @@ -TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - WholeStageCodegen (12) - Project [item_id,ss_item_rev,cs_item_rev,ws_item_rev] - BroadcastHashJoin [item_id,item_id,ss_item_rev,ws_item_rev,cs_item_rev] - Project [item_id,ss_item_rev,cs_item_rev] - BroadcastHashJoin [item_id,item_id,ss_item_rev,cs_item_rev] - Filter [ss_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (3) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometFilter [d_date,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - Filter [cs_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #6 - WholeStageCodegen (6) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #4 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) - Filter [ws_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #8 - WholeStageCodegen (10) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #4 - InputAdapter - ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/explain.txt deleted file mode 100644 index 0307d0df0e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,381 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (52) -+- CometTakeOrderedAndProject (51) - +- CometProject (50) - +- CometBroadcastHashJoin (49) - :- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometFilter (20) - : : +- CometHashAggregate (19) - : : +- CometExchange (18) - : : +- CometHashAggregate (17) - : : +- CometProject (16) - : : +- CometBroadcastHashJoin (15) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (14) - : : +- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- ReusedExchange (11) - : +- CometBroadcastExchange (33) - : +- CometFilter (32) - : +- CometHashAggregate (31) - : +- CometExchange (30) - : +- CometHashAggregate (29) - : +- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (21) - : : +- ReusedExchange (23) - : +- ReusedExchange (26) - +- CometBroadcastExchange (48) - +- CometFilter (47) - +- CometHashAggregate (46) - +- CometExchange (45) - +- CometHashAggregate (44) - +- CometProject (43) - +- CometBroadcastHashJoin (42) - :- CometProject (40) - : +- CometBroadcastHashJoin (39) - : :- CometFilter (37) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (36) - : +- ReusedExchange (38) - +- ReusedExchange (41) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_item_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_item_id#6] -Condition : (isnotnull(i_item_sk#5) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true))) - -(5) CometProject -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#7] - -(6) CometBroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#7] -Arguments: [i_item_sk#5, i_item_id#7] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [2]: [i_item_sk#5, i_item_id#7] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#7] -Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(11) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date#10] - -(12) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#10] -Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight - -(13) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: [d_date_sk#8] - -(15) CometBroadcastHashJoin -Left output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7] -Right output [1]: [d_date_sk#8] -Arguments: [ss_sold_date_sk#3], [d_date_sk#8], Inner, BuildRight - -(16) CometProject -Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7, d_date_sk#8] -Arguments: [ss_ext_sales_price#2, i_item_id#7], [ss_ext_sales_price#2, i_item_id#7] - -(17) CometHashAggregate -Input [2]: [ss_ext_sales_price#2, i_item_id#7] -Keys [1]: [i_item_id#7] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(18) CometExchange -Input [2]: [i_item_id#7, sum#11] -Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [2]: [i_item_id#7, sum#11] -Keys [1]: [i_item_id#7] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(20) CometFilter -Input [2]: [item_id#12, ss_item_rev#13] -Condition : isnotnull(ss_item_rev#13) - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#14) - -(23) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#18, i_item_id#19] - -(24) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Right output [2]: [i_item_sk#18, i_item_id#19] -Arguments: [cs_item_sk#14], [i_item_sk#18], Inner, BuildRight - -(25) CometProject -Input [5]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_sk#18, i_item_id#19] -Arguments: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19], [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19] - -(26) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#20] - -(27) CometBroadcastHashJoin -Left output [3]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19] -Right output [1]: [d_date_sk#20] -Arguments: [cs_sold_date_sk#16], [d_date_sk#20], Inner, BuildRight - -(28) CometProject -Input [4]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19, d_date_sk#20] -Arguments: [cs_ext_sales_price#15, i_item_id#19], [cs_ext_sales_price#15, i_item_id#19] - -(29) CometHashAggregate -Input [2]: [cs_ext_sales_price#15, i_item_id#19] -Keys [1]: [i_item_id#19] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#15))] - -(30) CometExchange -Input [2]: [i_item_id#19, sum#21] -Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(31) CometHashAggregate -Input [2]: [i_item_id#19, sum#21] -Keys [1]: [i_item_id#19] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#15))] - -(32) CometFilter -Input [2]: [item_id#22, cs_item_rev#23] -Condition : isnotnull(cs_item_rev#23) - -(33) CometBroadcastExchange -Input [2]: [item_id#22, cs_item_rev#23] -Arguments: [item_id#22, cs_item_rev#23] - -(34) CometBroadcastHashJoin -Left output [2]: [item_id#12, ss_item_rev#13] -Right output [2]: [item_id#22, cs_item_rev#23] -Arguments: [item_id#12], [item_id#22], Inner, ((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * cs_item_rev#23)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * cs_item_rev#23))) AND (cast(cs_item_rev#23 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(cs_item_rev#23 as decimal(20,3)) <= (1.1 * ss_item_rev#13))), BuildRight - -(35) CometProject -Input [4]: [item_id#12, ss_item_rev#13, item_id#22, cs_item_rev#23] -Arguments: [item_id#12, ss_item_rev#13, cs_item_rev#23], [item_id#12, ss_item_rev#13, cs_item_rev#23] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#26), dynamicpruningexpression(ws_sold_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26] -Condition : isnotnull(ws_item_sk#24) - -(38) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#28, i_item_id#29] - -(39) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26] -Right output [2]: [i_item_sk#28, i_item_id#29] -Arguments: [ws_item_sk#24], [i_item_sk#28], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_sk#28, i_item_id#29] -Arguments: [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29], [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29] - -(41) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#30] - -(42) CometBroadcastHashJoin -Left output [3]: [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29] -Right output [1]: [d_date_sk#30] -Arguments: [ws_sold_date_sk#26], [d_date_sk#30], Inner, BuildRight - -(43) CometProject -Input [4]: [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29, d_date_sk#30] -Arguments: [ws_ext_sales_price#25, i_item_id#29], [ws_ext_sales_price#25, i_item_id#29] - -(44) CometHashAggregate -Input [2]: [ws_ext_sales_price#25, i_item_id#29] -Keys [1]: [i_item_id#29] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#25))] - -(45) CometExchange -Input [2]: [i_item_id#29, sum#31] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(46) CometHashAggregate -Input [2]: [i_item_id#29, sum#31] -Keys [1]: [i_item_id#29] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#25))] - -(47) CometFilter -Input [2]: [item_id#32, ws_item_rev#33] -Condition : isnotnull(ws_item_rev#33) - -(48) CometBroadcastExchange -Input [2]: [item_id#32, ws_item_rev#33] -Arguments: [item_id#32, ws_item_rev#33] - -(49) CometBroadcastHashJoin -Left output [3]: [item_id#12, ss_item_rev#13, cs_item_rev#23] -Right output [2]: [item_id#32, ws_item_rev#33] -Arguments: [item_id#12], [item_id#32], Inner, ((((((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * ws_item_rev#33)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * ws_item_rev#33))) AND (cast(cs_item_rev#23 as decimal(19,3)) >= (0.9 * ws_item_rev#33))) AND (cast(cs_item_rev#23 as decimal(20,3)) <= (1.1 * ws_item_rev#33))) AND (cast(ws_item_rev#33 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(ws_item_rev#33 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) AND (cast(ws_item_rev#33 as decimal(19,3)) >= (0.9 * cs_item_rev#23))) AND (cast(ws_item_rev#33 as decimal(20,3)) <= (1.1 * cs_item_rev#23))), BuildRight - -(50) CometProject -Input [5]: [item_id#12, ss_item_rev#13, cs_item_rev#23, item_id#32, ws_item_rev#33] -Arguments: [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37], [item_id#12, ss_item_rev#13, (((ss_item_rev#13 / ((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33)) / 3) * 100) AS ss_dev#34, cs_item_rev#23, (((cs_item_rev#23 / ((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33)) / 3) * 100) AS cs_dev#35, ws_item_rev#33, (((ws_item_rev#33 / ((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33)) / 3) * 100) AS ws_dev#36, (((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33) / 3) AS average#37] - -(51) CometTakeOrderedAndProject -Input [8]: [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_id#12 ASC NULLS FIRST,ss_item_rev#13 ASC NULLS FIRST], output=[item_id#12,ss_item_rev#13,ss_dev#34,cs_item_rev#23,cs_dev#35,ws_item_rev#33,ws_dev#36,average#37]), [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37], 100, 0, [item_id#12 ASC NULLS FIRST, ss_item_rev#13 ASC NULLS FIRST], [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37] - -(52) CometColumnarToRow [codegen id : 1] -Input [8]: [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (62) -+- * CometColumnarToRow (61) - +- CometProject (60) - +- CometBroadcastHashJoin (59) - :- CometFilter (54) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - +- CometBroadcastExchange (58) - +- CometProject (57) - +- CometFilter (56) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#10, d_week_seq#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date#10, d_week_seq#38] -Condition : (isnotnull(d_week_seq#38) AND (d_week_seq#38 = Subquery scalar-subquery#39, [id=#40])) - -(57) CometProject -Input [2]: [d_date#10, d_week_seq#38] -Arguments: [d_date#10], [d_date#10] - -(58) CometBroadcastExchange -Input [1]: [d_date#10] -Arguments: [d_date#10] - -(59) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#10] -Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight - -(60) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(61) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(62) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 56 Hosting Expression = Subquery scalar-subquery#39, [id=#40] -* CometColumnarToRow (66) -+- CometProject (65) - +- CometFilter (64) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) - - -(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#41, d_week_seq#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] -ReadSchema: struct - -(64) CometFilter -Input [2]: [d_date#41, d_week_seq#42] -Condition : (isnotnull(d_date#41) AND (d_date#41 = 2000-01-03)) - -(65) CometProject -Input [2]: [d_date#41, d_week_seq#42] -Arguments: [d_week_seq#42], [d_week_seq#42] - -(66) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#42] - -Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#4 - -Subquery:4 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#26 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/simplified.txt deleted file mode 100644 index 96e11bc0c7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,76 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - CometProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - CometBroadcastHashJoin [item_id,ss_item_rev,cs_item_rev,item_id,ws_item_rev] - CometProject [item_id,ss_item_rev,cs_item_rev] - CometBroadcastHashJoin [item_id,ss_item_rev,item_id,cs_item_rev] - CometFilter [item_id,ss_item_rev] - CometHashAggregate [sum] [item_id,ss_item_rev,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id] #1 - CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] - CometProject [ss_ext_sales_price,i_item_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometFilter [d_date,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [i_item_sk,i_item_id] #4 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #3 - CometBroadcastExchange [item_id,cs_item_rev] #6 - CometFilter [item_id,cs_item_rev] - CometHashAggregate [sum] [item_id,cs_item_rev,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id] #7 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - 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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [item_id,ws_item_rev] #8 - CometFilter [item_id,ws_item_rev] - CometHashAggregate [sum] [item_id,ws_item_rev,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - 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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/explain.txt deleted file mode 100644 index 0307d0df0e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/explain.txt +++ /dev/null @@ -1,381 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (52) -+- CometTakeOrderedAndProject (51) - +- CometProject (50) - +- CometBroadcastHashJoin (49) - :- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometFilter (20) - : : +- CometHashAggregate (19) - : : +- CometExchange (18) - : : +- CometHashAggregate (17) - : : +- CometProject (16) - : : +- CometBroadcastHashJoin (15) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (14) - : : +- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- ReusedExchange (11) - : +- CometBroadcastExchange (33) - : +- CometFilter (32) - : +- CometHashAggregate (31) - : +- CometExchange (30) - : +- CometHashAggregate (29) - : +- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (21) - : : +- ReusedExchange (23) - : +- ReusedExchange (26) - +- CometBroadcastExchange (48) - +- CometFilter (47) - +- CometHashAggregate (46) - +- CometExchange (45) - +- CometHashAggregate (44) - +- CometProject (43) - +- CometBroadcastHashJoin (42) - :- CometProject (40) - : +- CometBroadcastHashJoin (39) - : :- CometFilter (37) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (36) - : +- ReusedExchange (38) - +- ReusedExchange (41) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_item_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_item_id#6] -Condition : (isnotnull(i_item_sk#5) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true))) - -(5) CometProject -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#7] - -(6) CometBroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#7] -Arguments: [i_item_sk#5, i_item_id#7] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [2]: [i_item_sk#5, i_item_id#7] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#7] -Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(11) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date#10] - -(12) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#10] -Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight - -(13) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: [d_date_sk#8] - -(15) CometBroadcastHashJoin -Left output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7] -Right output [1]: [d_date_sk#8] -Arguments: [ss_sold_date_sk#3], [d_date_sk#8], Inner, BuildRight - -(16) CometProject -Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7, d_date_sk#8] -Arguments: [ss_ext_sales_price#2, i_item_id#7], [ss_ext_sales_price#2, i_item_id#7] - -(17) CometHashAggregate -Input [2]: [ss_ext_sales_price#2, i_item_id#7] -Keys [1]: [i_item_id#7] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(18) CometExchange -Input [2]: [i_item_id#7, sum#11] -Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [2]: [i_item_id#7, sum#11] -Keys [1]: [i_item_id#7] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(20) CometFilter -Input [2]: [item_id#12, ss_item_rev#13] -Condition : isnotnull(ss_item_rev#13) - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#14) - -(23) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#18, i_item_id#19] - -(24) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Right output [2]: [i_item_sk#18, i_item_id#19] -Arguments: [cs_item_sk#14], [i_item_sk#18], Inner, BuildRight - -(25) CometProject -Input [5]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_sk#18, i_item_id#19] -Arguments: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19], [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19] - -(26) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#20] - -(27) CometBroadcastHashJoin -Left output [3]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19] -Right output [1]: [d_date_sk#20] -Arguments: [cs_sold_date_sk#16], [d_date_sk#20], Inner, BuildRight - -(28) CometProject -Input [4]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19, d_date_sk#20] -Arguments: [cs_ext_sales_price#15, i_item_id#19], [cs_ext_sales_price#15, i_item_id#19] - -(29) CometHashAggregate -Input [2]: [cs_ext_sales_price#15, i_item_id#19] -Keys [1]: [i_item_id#19] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#15))] - -(30) CometExchange -Input [2]: [i_item_id#19, sum#21] -Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(31) CometHashAggregate -Input [2]: [i_item_id#19, sum#21] -Keys [1]: [i_item_id#19] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#15))] - -(32) CometFilter -Input [2]: [item_id#22, cs_item_rev#23] -Condition : isnotnull(cs_item_rev#23) - -(33) CometBroadcastExchange -Input [2]: [item_id#22, cs_item_rev#23] -Arguments: [item_id#22, cs_item_rev#23] - -(34) CometBroadcastHashJoin -Left output [2]: [item_id#12, ss_item_rev#13] -Right output [2]: [item_id#22, cs_item_rev#23] -Arguments: [item_id#12], [item_id#22], Inner, ((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * cs_item_rev#23)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * cs_item_rev#23))) AND (cast(cs_item_rev#23 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(cs_item_rev#23 as decimal(20,3)) <= (1.1 * ss_item_rev#13))), BuildRight - -(35) CometProject -Input [4]: [item_id#12, ss_item_rev#13, item_id#22, cs_item_rev#23] -Arguments: [item_id#12, ss_item_rev#13, cs_item_rev#23], [item_id#12, ss_item_rev#13, cs_item_rev#23] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#26), dynamicpruningexpression(ws_sold_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26] -Condition : isnotnull(ws_item_sk#24) - -(38) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#28, i_item_id#29] - -(39) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26] -Right output [2]: [i_item_sk#28, i_item_id#29] -Arguments: [ws_item_sk#24], [i_item_sk#28], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_sk#28, i_item_id#29] -Arguments: [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29], [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29] - -(41) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#30] - -(42) CometBroadcastHashJoin -Left output [3]: [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29] -Right output [1]: [d_date_sk#30] -Arguments: [ws_sold_date_sk#26], [d_date_sk#30], Inner, BuildRight - -(43) CometProject -Input [4]: [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29, d_date_sk#30] -Arguments: [ws_ext_sales_price#25, i_item_id#29], [ws_ext_sales_price#25, i_item_id#29] - -(44) CometHashAggregate -Input [2]: [ws_ext_sales_price#25, i_item_id#29] -Keys [1]: [i_item_id#29] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#25))] - -(45) CometExchange -Input [2]: [i_item_id#29, sum#31] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(46) CometHashAggregate -Input [2]: [i_item_id#29, sum#31] -Keys [1]: [i_item_id#29] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#25))] - -(47) CometFilter -Input [2]: [item_id#32, ws_item_rev#33] -Condition : isnotnull(ws_item_rev#33) - -(48) CometBroadcastExchange -Input [2]: [item_id#32, ws_item_rev#33] -Arguments: [item_id#32, ws_item_rev#33] - -(49) CometBroadcastHashJoin -Left output [3]: [item_id#12, ss_item_rev#13, cs_item_rev#23] -Right output [2]: [item_id#32, ws_item_rev#33] -Arguments: [item_id#12], [item_id#32], Inner, ((((((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * ws_item_rev#33)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * ws_item_rev#33))) AND (cast(cs_item_rev#23 as decimal(19,3)) >= (0.9 * ws_item_rev#33))) AND (cast(cs_item_rev#23 as decimal(20,3)) <= (1.1 * ws_item_rev#33))) AND (cast(ws_item_rev#33 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(ws_item_rev#33 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) AND (cast(ws_item_rev#33 as decimal(19,3)) >= (0.9 * cs_item_rev#23))) AND (cast(ws_item_rev#33 as decimal(20,3)) <= (1.1 * cs_item_rev#23))), BuildRight - -(50) CometProject -Input [5]: [item_id#12, ss_item_rev#13, cs_item_rev#23, item_id#32, ws_item_rev#33] -Arguments: [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37], [item_id#12, ss_item_rev#13, (((ss_item_rev#13 / ((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33)) / 3) * 100) AS ss_dev#34, cs_item_rev#23, (((cs_item_rev#23 / ((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33)) / 3) * 100) AS cs_dev#35, ws_item_rev#33, (((ws_item_rev#33 / ((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33)) / 3) * 100) AS ws_dev#36, (((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33) / 3) AS average#37] - -(51) CometTakeOrderedAndProject -Input [8]: [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_id#12 ASC NULLS FIRST,ss_item_rev#13 ASC NULLS FIRST], output=[item_id#12,ss_item_rev#13,ss_dev#34,cs_item_rev#23,cs_dev#35,ws_item_rev#33,ws_dev#36,average#37]), [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37], 100, 0, [item_id#12 ASC NULLS FIRST, ss_item_rev#13 ASC NULLS FIRST], [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37] - -(52) CometColumnarToRow [codegen id : 1] -Input [8]: [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (62) -+- * CometColumnarToRow (61) - +- CometProject (60) - +- CometBroadcastHashJoin (59) - :- CometFilter (54) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - +- CometBroadcastExchange (58) - +- CometProject (57) - +- CometFilter (56) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#10, d_week_seq#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date#10, d_week_seq#38] -Condition : (isnotnull(d_week_seq#38) AND (d_week_seq#38 = Subquery scalar-subquery#39, [id=#40])) - -(57) CometProject -Input [2]: [d_date#10, d_week_seq#38] -Arguments: [d_date#10], [d_date#10] - -(58) CometBroadcastExchange -Input [1]: [d_date#10] -Arguments: [d_date#10] - -(59) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#10] -Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight - -(60) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(61) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(62) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 56 Hosting Expression = Subquery scalar-subquery#39, [id=#40] -* CometColumnarToRow (66) -+- CometProject (65) - +- CometFilter (64) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) - - -(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#41, d_week_seq#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] -ReadSchema: struct - -(64) CometFilter -Input [2]: [d_date#41, d_week_seq#42] -Condition : (isnotnull(d_date#41) AND (d_date#41 = 2000-01-03)) - -(65) CometProject -Input [2]: [d_date#41, d_week_seq#42] -Arguments: [d_week_seq#42], [d_week_seq#42] - -(66) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#42] - -Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#4 - -Subquery:4 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#26 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/extended.txt deleted file mode 100644 index 3d3dd68990..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/extended.txt +++ /dev/null @@ -1,112 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- Subquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 96 out of 104 eligible operators (92%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 96e11bc0c7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/simplified.txt +++ /dev/null @@ -1,76 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - CometProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - CometBroadcastHashJoin [item_id,ss_item_rev,cs_item_rev,item_id,ws_item_rev] - CometProject [item_id,ss_item_rev,cs_item_rev] - CometBroadcastHashJoin [item_id,ss_item_rev,item_id,cs_item_rev] - CometFilter [item_id,ss_item_rev] - CometHashAggregate [sum] [item_id,ss_item_rev,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id] #1 - CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] - CometProject [ss_ext_sales_price,i_item_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometFilter [d_date,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [i_item_sk,i_item_id] #4 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #3 - CometBroadcastExchange [item_id,cs_item_rev] #6 - CometFilter [item_id,cs_item_rev] - CometHashAggregate [sum] [item_id,cs_item_rev,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id] #7 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - 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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [item_id,ws_item_rev] #8 - CometFilter [item_id,ws_item_rev] - CometHashAggregate [sum] [item_id,ws_item_rev,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - 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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_datafusion/explain.txt deleted file mode 100644 index 1bb4b7dc8f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_datafusion/explain.txt +++ /dev/null @@ -1,239 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (42) -+- CometTakeOrderedAndProject (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometNativeScan parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (15) - : : +- CometProject (14) - : : +- CometFilter (13) - : : +- CometNativeScan parquet spark_catalog.default.store (12) - : +- CometBroadcastExchange (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometNativeScan parquet spark_catalog.default.date_dim (18) - +- CometBroadcastExchange (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometHashAggregate (25) - : : +- ReusedExchange (24) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometNativeScan parquet spark_catalog.default.store (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.date_dim (32) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#3)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Condition : (isnotnull(d_date_sk#4) AND isnotnull(d_week_seq#5)) - -(5) CometProject -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Arguments: [d_date_sk#4, d_week_seq#5, d_day_name#7], [d_date_sk#4, d_week_seq#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_day_name#6, 9, true, false, true) AS d_day_name#7] - -(6) CometBroadcastExchange -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [d_date_sk#4, d_week_seq#5, d_day_name#7] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Right output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [ss_sold_date_sk#3], [d_date_sk#4], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3, d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7], [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7] - -(9) CometHashAggregate -Input [4]: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7] -Keys [2]: [d_week_seq#5, ss_store_sk#1] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] - -(10) CometExchange -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] -Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] -Keys [2]: [d_week_seq#5, ss_store_sk#1] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] - -(12) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(13) CometFilter -Input [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Condition : (isnotnull(s_store_sk#15) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#16, 16, true, false, true))) - -(14) CometProject -Input [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Arguments: [s_store_sk#15, s_store_id#18, s_store_name#17], [s_store_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#16, 16, true, false, true) AS s_store_id#18, s_store_name#17] - -(15) CometBroadcastExchange -Input [3]: [s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [s_store_sk#15, s_store_id#18, s_store_name#17] - -(16) CometBroadcastHashJoin -Left output [9]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25] -Right output [3]: [s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [ss_store_sk#1], [s_store_sk#15], Inner, BuildRight - -(17) CometProject -Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17], [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17] - -(18) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#26, d_week_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_week_seq)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [d_month_seq#26, d_week_seq#27] -Condition : (((isnotnull(d_month_seq#26) AND (d_month_seq#26 >= 1212)) AND (d_month_seq#26 <= 1223)) AND isnotnull(d_week_seq#27)) - -(20) CometProject -Input [2]: [d_month_seq#26, d_week_seq#27] -Arguments: [d_week_seq#27], [d_week_seq#27] - -(21) CometBroadcastExchange -Input [1]: [d_week_seq#27] -Arguments: [d_week_seq#27] - -(22) CometBroadcastHashJoin -Left output [10]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17] -Right output [1]: [d_week_seq#27] -Arguments: [d_week_seq#5], [d_week_seq#27], Inner, BuildRight - -(23) CometProject -Input [11]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17, d_week_seq#27] -Arguments: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37], [s_store_name#17 AS s_store_name1#28, d_week_seq#5 AS d_week_seq1#29, s_store_id#18 AS s_store_id1#30, sun_sales#19 AS sun_sales1#31, mon_sales#20 AS mon_sales1#32, tue_sales#21 AS tue_sales1#33, wed_sales#22 AS wed_sales1#34, thu_sales#23 AS thu_sales1#35, fri_sales#24 AS fri_sales1#36, sat_sales#25 AS sat_sales1#37] - -(24) ReusedExchange [Reuses operator id: 10] -Output [9]: [d_week_seq#38, ss_store_sk#39, sum#40, sum#41, sum#42, sum#43, sum#44, sum#45, sum#46] - -(25) CometHashAggregate -Input [9]: [d_week_seq#38, ss_store_sk#39, sum#40, sum#41, sum#42, sum#43, sum#44, sum#45, sum#46] -Keys [2]: [d_week_seq#38, ss_store_sk#39] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#47 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#47 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#47 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#47 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#47 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#47 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#47 END))] - -(26) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#48, s_store_id#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [s_store_sk#48, s_store_id#49] -Condition : (isnotnull(s_store_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#49, 16, true, false, true))) - -(28) CometProject -Input [2]: [s_store_sk#48, s_store_id#49] -Arguments: [s_store_sk#48, s_store_id#50], [s_store_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#49, 16, true, false, true) AS s_store_id#50] - -(29) CometBroadcastExchange -Input [2]: [s_store_sk#48, s_store_id#50] -Arguments: [s_store_sk#48, s_store_id#50] - -(30) CometBroadcastHashJoin -Left output [9]: [d_week_seq#38, ss_store_sk#39, sun_sales#51, mon_sales#52, tue_sales#53, wed_sales#54, thu_sales#55, fri_sales#56, sat_sales#57] -Right output [2]: [s_store_sk#48, s_store_id#50] -Arguments: [ss_store_sk#39], [s_store_sk#48], Inner, BuildRight - -(31) CometProject -Input [11]: [d_week_seq#38, ss_store_sk#39, sun_sales#51, mon_sales#52, tue_sales#53, wed_sales#54, thu_sales#55, fri_sales#56, sat_sales#57, s_store_sk#48, s_store_id#50] -Arguments: [d_week_seq#38, sun_sales#51, mon_sales#52, tue_sales#53, wed_sales#54, thu_sales#55, fri_sales#56, sat_sales#57, s_store_id#50], [d_week_seq#38, sun_sales#51, mon_sales#52, tue_sales#53, wed_sales#54, thu_sales#55, fri_sales#56, sat_sales#57, s_store_id#50] - -(32) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#58, d_week_seq#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1224), LessThanOrEqual(d_month_seq,1235), IsNotNull(d_week_seq)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_month_seq#58, d_week_seq#59] -Condition : (((isnotnull(d_month_seq#58) AND (d_month_seq#58 >= 1224)) AND (d_month_seq#58 <= 1235)) AND isnotnull(d_week_seq#59)) - -(34) CometProject -Input [2]: [d_month_seq#58, d_week_seq#59] -Arguments: [d_week_seq#59], [d_week_seq#59] - -(35) CometBroadcastExchange -Input [1]: [d_week_seq#59] -Arguments: [d_week_seq#59] - -(36) CometBroadcastHashJoin -Left output [9]: [d_week_seq#38, sun_sales#51, mon_sales#52, tue_sales#53, wed_sales#54, thu_sales#55, fri_sales#56, sat_sales#57, s_store_id#50] -Right output [1]: [d_week_seq#59] -Arguments: [d_week_seq#38], [d_week_seq#59], Inner, BuildRight - -(37) CometProject -Input [10]: [d_week_seq#38, sun_sales#51, mon_sales#52, tue_sales#53, wed_sales#54, thu_sales#55, fri_sales#56, sat_sales#57, s_store_id#50, d_week_seq#59] -Arguments: [d_week_seq2#60, s_store_id2#61, sun_sales2#62, mon_sales2#63, tue_sales2#64, wed_sales2#65, thu_sales2#66, fri_sales2#67, sat_sales2#68], [d_week_seq#38 AS d_week_seq2#60, s_store_id#50 AS s_store_id2#61, sun_sales#51 AS sun_sales2#62, mon_sales#52 AS mon_sales2#63, tue_sales#53 AS tue_sales2#64, wed_sales#54 AS wed_sales2#65, thu_sales#55 AS thu_sales2#66, fri_sales#56 AS fri_sales2#67, sat_sales#57 AS sat_sales2#68] - -(38) CometBroadcastExchange -Input [9]: [d_week_seq2#60, s_store_id2#61, sun_sales2#62, mon_sales2#63, tue_sales2#64, wed_sales2#65, thu_sales2#66, fri_sales2#67, sat_sales2#68] -Arguments: [d_week_seq2#60, s_store_id2#61, sun_sales2#62, mon_sales2#63, tue_sales2#64, wed_sales2#65, thu_sales2#66, fri_sales2#67, sat_sales2#68] - -(39) CometBroadcastHashJoin -Left output [10]: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37] -Right output [9]: [d_week_seq2#60, s_store_id2#61, sun_sales2#62, mon_sales2#63, tue_sales2#64, wed_sales2#65, thu_sales2#66, fri_sales2#67, sat_sales2#68] -Arguments: [s_store_id1#30, d_week_seq1#29], [s_store_id2#61, (d_week_seq2#60 - 52)], Inner, BuildRight - -(40) CometProject -Input [19]: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37, d_week_seq2#60, s_store_id2#61, sun_sales2#62, mon_sales2#63, tue_sales2#64, wed_sales2#65, thu_sales2#66, fri_sales2#67, sat_sales2#68] -Arguments: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#69, (mon_sales1 / mon_sales2)#70, (tue_sales1 / tue_sales2)#71, (wed_sales1 / wed_sales2)#72, (thu_sales1 / thu_sales2)#73, (fri_sales1 / fri_sales2)#74, (sat_sales1 / sat_sales2)#75], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1#31 / sun_sales2#62) AS (sun_sales1 / sun_sales2)#69, (mon_sales1#32 / mon_sales2#63) AS (mon_sales1 / mon_sales2)#70, (tue_sales1#33 / tue_sales2#64) AS (tue_sales1 / tue_sales2)#71, (wed_sales1#34 / wed_sales2#65) AS (wed_sales1 / wed_sales2)#72, (thu_sales1#35 / thu_sales2#66) AS (thu_sales1 / thu_sales2)#73, (fri_sales1#36 / fri_sales2#67) AS (fri_sales1 / fri_sales2)#74, (sat_sales1#37 / sat_sales2#68) AS (sat_sales1 / sat_sales2)#75] - -(41) CometTakeOrderedAndProject -Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#69, (mon_sales1 / mon_sales2)#70, (tue_sales1 / tue_sales2)#71, (wed_sales1 / wed_sales2)#72, (thu_sales1 / thu_sales2)#73, (fri_sales1 / fri_sales2)#74, (sat_sales1 / sat_sales2)#75] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name1#28 ASC NULLS FIRST,s_store_id1#30 ASC NULLS FIRST,d_week_seq1#29 ASC NULLS FIRST], output=[s_store_name1#28,s_store_id1#30,d_week_seq1#29,(sun_sales1 / sun_sales2)#69,(mon_sales1 / mon_sales2)#70,(tue_sales1 / tue_sales2)#71,(wed_sales1 / wed_sales2)#72,(thu_sales1 / thu_sales2)#73,(fri_sales1 / fri_sales2)#74,(sat_sales1 / sat_sales2)#75]), [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#69, (mon_sales1 / mon_sales2)#70, (tue_sales1 / tue_sales2)#71, (wed_sales1 / wed_sales2)#72, (thu_sales1 / thu_sales2)#73, (fri_sales1 / fri_sales2)#74, (sat_sales1 / sat_sales2)#75], 100, 0, [s_store_name1#28 ASC NULLS FIRST, s_store_id1#30 ASC NULLS FIRST, d_week_seq1#29 ASC NULLS FIRST], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#69, (mon_sales1 / mon_sales2)#70, (tue_sales1 / tue_sales2)#71, (wed_sales1 / wed_sales2)#72, (thu_sales1 / thu_sales2)#73, (fri_sales1 / fri_sales2)#74, (sat_sales1 / sat_sales2)#75] - -(42) CometColumnarToRow [codegen id : 1] -Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#69, (mon_sales1 / mon_sales2)#70, (tue_sales1 / tue_sales2)#71, (wed_sales1 / wed_sales2)#72, (thu_sales1 / thu_sales2)#73, (fri_sales1 / fri_sales2)#74, (sat_sales1 / sat_sales2)#75] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_datafusion/simplified.txt deleted file mode 100644 index 72823b14c1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_datafusion/simplified.txt +++ /dev/null @@ -1,44 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometBroadcastHashJoin [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometProject [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id,s_store_name] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - CometExchange [d_week_seq,ss_store_sk] #1 - CometHashAggregate [d_day_name,ss_sales_price] [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] - CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] - CometBroadcastExchange [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 - CometBroadcastExchange [s_store_sk,s_store_id] #6 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [d_week_seq] #7 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_iceberg_compat/explain.txt deleted file mode 100644 index 4366c901e1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,239 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (42) -+- CometTakeOrderedAndProject (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (15) - : : +- CometProject (14) - : : +- CometFilter (13) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (12) - : +- CometBroadcastExchange (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (18) - +- CometBroadcastExchange (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometHashAggregate (25) - : : +- ReusedExchange (24) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Condition : (isnotnull(d_date_sk#4) AND isnotnull(d_week_seq#5)) - -(5) CometProject -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Arguments: [d_date_sk#4, d_week_seq#5, d_day_name#7], [d_date_sk#4, d_week_seq#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_day_name#6, 9, true, false, true) AS d_day_name#7] - -(6) CometBroadcastExchange -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [d_date_sk#4, d_week_seq#5, d_day_name#7] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Right output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [ss_sold_date_sk#3], [d_date_sk#4], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3, d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7], [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7] - -(9) CometHashAggregate -Input [4]: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7] -Keys [2]: [d_week_seq#5, ss_store_sk#1] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] - -(10) CometExchange -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] -Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] -Keys [2]: [d_week_seq#5, ss_store_sk#1] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(13) CometFilter -Input [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Condition : (isnotnull(s_store_sk#15) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#16, 16, true, false, true))) - -(14) CometProject -Input [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Arguments: [s_store_sk#15, s_store_id#18, s_store_name#17], [s_store_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#16, 16, true, false, true) AS s_store_id#18, s_store_name#17] - -(15) CometBroadcastExchange -Input [3]: [s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [s_store_sk#15, s_store_id#18, s_store_name#17] - -(16) CometBroadcastHashJoin -Left output [9]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25] -Right output [3]: [s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [ss_store_sk#1], [s_store_sk#15], Inner, BuildRight - -(17) CometProject -Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17], [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#26, d_week_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_week_seq)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [d_month_seq#26, d_week_seq#27] -Condition : (((isnotnull(d_month_seq#26) AND (d_month_seq#26 >= 1212)) AND (d_month_seq#26 <= 1223)) AND isnotnull(d_week_seq#27)) - -(20) CometProject -Input [2]: [d_month_seq#26, d_week_seq#27] -Arguments: [d_week_seq#27], [d_week_seq#27] - -(21) CometBroadcastExchange -Input [1]: [d_week_seq#27] -Arguments: [d_week_seq#27] - -(22) CometBroadcastHashJoin -Left output [10]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17] -Right output [1]: [d_week_seq#27] -Arguments: [d_week_seq#5], [d_week_seq#27], Inner, BuildRight - -(23) CometProject -Input [11]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17, d_week_seq#27] -Arguments: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37], [s_store_name#17 AS s_store_name1#28, d_week_seq#5 AS d_week_seq1#29, s_store_id#18 AS s_store_id1#30, sun_sales#19 AS sun_sales1#31, mon_sales#20 AS mon_sales1#32, tue_sales#21 AS tue_sales1#33, wed_sales#22 AS wed_sales1#34, thu_sales#23 AS thu_sales1#35, fri_sales#24 AS fri_sales1#36, sat_sales#25 AS sat_sales1#37] - -(24) ReusedExchange [Reuses operator id: 10] -Output [9]: [d_week_seq#38, ss_store_sk#39, sum#40, sum#41, sum#42, sum#43, sum#44, sum#45, sum#46] - -(25) CometHashAggregate -Input [9]: [d_week_seq#38, ss_store_sk#39, sum#40, sum#41, sum#42, sum#43, sum#44, sum#45, sum#46] -Keys [2]: [d_week_seq#38, ss_store_sk#39] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#47 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#47 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#47 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#47 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#47 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#47 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#47 END))] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#48, s_store_id#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [s_store_sk#48, s_store_id#49] -Condition : (isnotnull(s_store_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#49, 16, true, false, true))) - -(28) CometProject -Input [2]: [s_store_sk#48, s_store_id#49] -Arguments: [s_store_sk#48, s_store_id#50], [s_store_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#49, 16, true, false, true) AS s_store_id#50] - -(29) CometBroadcastExchange -Input [2]: [s_store_sk#48, s_store_id#50] -Arguments: [s_store_sk#48, s_store_id#50] - -(30) CometBroadcastHashJoin -Left output [9]: [d_week_seq#38, ss_store_sk#39, sun_sales#51, mon_sales#52, tue_sales#53, wed_sales#54, thu_sales#55, fri_sales#56, sat_sales#57] -Right output [2]: [s_store_sk#48, s_store_id#50] -Arguments: [ss_store_sk#39], [s_store_sk#48], Inner, BuildRight - -(31) CometProject -Input [11]: [d_week_seq#38, ss_store_sk#39, sun_sales#51, mon_sales#52, tue_sales#53, wed_sales#54, thu_sales#55, fri_sales#56, sat_sales#57, s_store_sk#48, s_store_id#50] -Arguments: [d_week_seq#38, sun_sales#51, mon_sales#52, tue_sales#53, wed_sales#54, thu_sales#55, fri_sales#56, sat_sales#57, s_store_id#50], [d_week_seq#38, sun_sales#51, mon_sales#52, tue_sales#53, wed_sales#54, thu_sales#55, fri_sales#56, sat_sales#57, s_store_id#50] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#58, d_week_seq#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1224), LessThanOrEqual(d_month_seq,1235), IsNotNull(d_week_seq)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_month_seq#58, d_week_seq#59] -Condition : (((isnotnull(d_month_seq#58) AND (d_month_seq#58 >= 1224)) AND (d_month_seq#58 <= 1235)) AND isnotnull(d_week_seq#59)) - -(34) CometProject -Input [2]: [d_month_seq#58, d_week_seq#59] -Arguments: [d_week_seq#59], [d_week_seq#59] - -(35) CometBroadcastExchange -Input [1]: [d_week_seq#59] -Arguments: [d_week_seq#59] - -(36) CometBroadcastHashJoin -Left output [9]: [d_week_seq#38, sun_sales#51, mon_sales#52, tue_sales#53, wed_sales#54, thu_sales#55, fri_sales#56, sat_sales#57, s_store_id#50] -Right output [1]: [d_week_seq#59] -Arguments: [d_week_seq#38], [d_week_seq#59], Inner, BuildRight - -(37) CometProject -Input [10]: [d_week_seq#38, sun_sales#51, mon_sales#52, tue_sales#53, wed_sales#54, thu_sales#55, fri_sales#56, sat_sales#57, s_store_id#50, d_week_seq#59] -Arguments: [d_week_seq2#60, s_store_id2#61, sun_sales2#62, mon_sales2#63, tue_sales2#64, wed_sales2#65, thu_sales2#66, fri_sales2#67, sat_sales2#68], [d_week_seq#38 AS d_week_seq2#60, s_store_id#50 AS s_store_id2#61, sun_sales#51 AS sun_sales2#62, mon_sales#52 AS mon_sales2#63, tue_sales#53 AS tue_sales2#64, wed_sales#54 AS wed_sales2#65, thu_sales#55 AS thu_sales2#66, fri_sales#56 AS fri_sales2#67, sat_sales#57 AS sat_sales2#68] - -(38) CometBroadcastExchange -Input [9]: [d_week_seq2#60, s_store_id2#61, sun_sales2#62, mon_sales2#63, tue_sales2#64, wed_sales2#65, thu_sales2#66, fri_sales2#67, sat_sales2#68] -Arguments: [d_week_seq2#60, s_store_id2#61, sun_sales2#62, mon_sales2#63, tue_sales2#64, wed_sales2#65, thu_sales2#66, fri_sales2#67, sat_sales2#68] - -(39) CometBroadcastHashJoin -Left output [10]: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37] -Right output [9]: [d_week_seq2#60, s_store_id2#61, sun_sales2#62, mon_sales2#63, tue_sales2#64, wed_sales2#65, thu_sales2#66, fri_sales2#67, sat_sales2#68] -Arguments: [s_store_id1#30, d_week_seq1#29], [s_store_id2#61, (d_week_seq2#60 - 52)], Inner, BuildRight - -(40) CometProject -Input [19]: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37, d_week_seq2#60, s_store_id2#61, sun_sales2#62, mon_sales2#63, tue_sales2#64, wed_sales2#65, thu_sales2#66, fri_sales2#67, sat_sales2#68] -Arguments: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#69, (mon_sales1 / mon_sales2)#70, (tue_sales1 / tue_sales2)#71, (wed_sales1 / wed_sales2)#72, (thu_sales1 / thu_sales2)#73, (fri_sales1 / fri_sales2)#74, (sat_sales1 / sat_sales2)#75], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1#31 / sun_sales2#62) AS (sun_sales1 / sun_sales2)#69, (mon_sales1#32 / mon_sales2#63) AS (mon_sales1 / mon_sales2)#70, (tue_sales1#33 / tue_sales2#64) AS (tue_sales1 / tue_sales2)#71, (wed_sales1#34 / wed_sales2#65) AS (wed_sales1 / wed_sales2)#72, (thu_sales1#35 / thu_sales2#66) AS (thu_sales1 / thu_sales2)#73, (fri_sales1#36 / fri_sales2#67) AS (fri_sales1 / fri_sales2)#74, (sat_sales1#37 / sat_sales2#68) AS (sat_sales1 / sat_sales2)#75] - -(41) CometTakeOrderedAndProject -Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#69, (mon_sales1 / mon_sales2)#70, (tue_sales1 / tue_sales2)#71, (wed_sales1 / wed_sales2)#72, (thu_sales1 / thu_sales2)#73, (fri_sales1 / fri_sales2)#74, (sat_sales1 / sat_sales2)#75] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name1#28 ASC NULLS FIRST,s_store_id1#30 ASC NULLS FIRST,d_week_seq1#29 ASC NULLS FIRST], output=[s_store_name1#28,s_store_id1#30,d_week_seq1#29,(sun_sales1 / sun_sales2)#69,(mon_sales1 / mon_sales2)#70,(tue_sales1 / tue_sales2)#71,(wed_sales1 / wed_sales2)#72,(thu_sales1 / thu_sales2)#73,(fri_sales1 / fri_sales2)#74,(sat_sales1 / sat_sales2)#75]), [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#69, (mon_sales1 / mon_sales2)#70, (tue_sales1 / tue_sales2)#71, (wed_sales1 / wed_sales2)#72, (thu_sales1 / thu_sales2)#73, (fri_sales1 / fri_sales2)#74, (sat_sales1 / sat_sales2)#75], 100, 0, [s_store_name1#28 ASC NULLS FIRST, s_store_id1#30 ASC NULLS FIRST, d_week_seq1#29 ASC NULLS FIRST], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#69, (mon_sales1 / mon_sales2)#70, (tue_sales1 / tue_sales2)#71, (wed_sales1 / wed_sales2)#72, (thu_sales1 / thu_sales2)#73, (fri_sales1 / fri_sales2)#74, (sat_sales1 / sat_sales2)#75] - -(42) CometColumnarToRow [codegen id : 1] -Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#69, (mon_sales1 / mon_sales2)#70, (tue_sales1 / tue_sales2)#71, (wed_sales1 / wed_sales2)#72, (thu_sales1 / thu_sales2)#73, (fri_sales1 / fri_sales2)#74, (sat_sales1 / sat_sales2)#75] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_iceberg_compat/simplified.txt deleted file mode 100644 index bca9c6463d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,44 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometBroadcastHashJoin [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometProject [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id,s_store_name] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - CometExchange [d_week_seq,ss_store_sk] #1 - CometHashAggregate [d_day_name,ss_sales_price] [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] - CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] - CometBroadcastExchange [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 - CometBroadcastExchange [s_store_sk,s_store_id] #6 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [d_week_seq] #7 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/explain.txt deleted file mode 100644 index 4366c901e1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/explain.txt +++ /dev/null @@ -1,239 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (42) -+- CometTakeOrderedAndProject (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (15) - : : +- CometProject (14) - : : +- CometFilter (13) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (12) - : +- CometBroadcastExchange (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (18) - +- CometBroadcastExchange (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometHashAggregate (25) - : : +- ReusedExchange (24) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Condition : (isnotnull(d_date_sk#4) AND isnotnull(d_week_seq#5)) - -(5) CometProject -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Arguments: [d_date_sk#4, d_week_seq#5, d_day_name#7], [d_date_sk#4, d_week_seq#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_day_name#6, 9, true, false, true) AS d_day_name#7] - -(6) CometBroadcastExchange -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [d_date_sk#4, d_week_seq#5, d_day_name#7] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Right output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [ss_sold_date_sk#3], [d_date_sk#4], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3, d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7], [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7] - -(9) CometHashAggregate -Input [4]: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7] -Keys [2]: [d_week_seq#5, ss_store_sk#1] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] - -(10) CometExchange -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] -Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] -Keys [2]: [d_week_seq#5, ss_store_sk#1] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(13) CometFilter -Input [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Condition : (isnotnull(s_store_sk#15) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#16, 16, true, false, true))) - -(14) CometProject -Input [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Arguments: [s_store_sk#15, s_store_id#18, s_store_name#17], [s_store_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#16, 16, true, false, true) AS s_store_id#18, s_store_name#17] - -(15) CometBroadcastExchange -Input [3]: [s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [s_store_sk#15, s_store_id#18, s_store_name#17] - -(16) CometBroadcastHashJoin -Left output [9]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25] -Right output [3]: [s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [ss_store_sk#1], [s_store_sk#15], Inner, BuildRight - -(17) CometProject -Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17], [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#26, d_week_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_week_seq)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [d_month_seq#26, d_week_seq#27] -Condition : (((isnotnull(d_month_seq#26) AND (d_month_seq#26 >= 1212)) AND (d_month_seq#26 <= 1223)) AND isnotnull(d_week_seq#27)) - -(20) CometProject -Input [2]: [d_month_seq#26, d_week_seq#27] -Arguments: [d_week_seq#27], [d_week_seq#27] - -(21) CometBroadcastExchange -Input [1]: [d_week_seq#27] -Arguments: [d_week_seq#27] - -(22) CometBroadcastHashJoin -Left output [10]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17] -Right output [1]: [d_week_seq#27] -Arguments: [d_week_seq#5], [d_week_seq#27], Inner, BuildRight - -(23) CometProject -Input [11]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17, d_week_seq#27] -Arguments: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37], [s_store_name#17 AS s_store_name1#28, d_week_seq#5 AS d_week_seq1#29, s_store_id#18 AS s_store_id1#30, sun_sales#19 AS sun_sales1#31, mon_sales#20 AS mon_sales1#32, tue_sales#21 AS tue_sales1#33, wed_sales#22 AS wed_sales1#34, thu_sales#23 AS thu_sales1#35, fri_sales#24 AS fri_sales1#36, sat_sales#25 AS sat_sales1#37] - -(24) ReusedExchange [Reuses operator id: 10] -Output [9]: [d_week_seq#38, ss_store_sk#39, sum#40, sum#41, sum#42, sum#43, sum#44, sum#45, sum#46] - -(25) CometHashAggregate -Input [9]: [d_week_seq#38, ss_store_sk#39, sum#40, sum#41, sum#42, sum#43, sum#44, sum#45, sum#46] -Keys [2]: [d_week_seq#38, ss_store_sk#39] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#47 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#47 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#47 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#47 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#47 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#47 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#47 END))] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#48, s_store_id#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [s_store_sk#48, s_store_id#49] -Condition : (isnotnull(s_store_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#49, 16, true, false, true))) - -(28) CometProject -Input [2]: [s_store_sk#48, s_store_id#49] -Arguments: [s_store_sk#48, s_store_id#50], [s_store_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#49, 16, true, false, true) AS s_store_id#50] - -(29) CometBroadcastExchange -Input [2]: [s_store_sk#48, s_store_id#50] -Arguments: [s_store_sk#48, s_store_id#50] - -(30) CometBroadcastHashJoin -Left output [9]: [d_week_seq#38, ss_store_sk#39, sun_sales#51, mon_sales#52, tue_sales#53, wed_sales#54, thu_sales#55, fri_sales#56, sat_sales#57] -Right output [2]: [s_store_sk#48, s_store_id#50] -Arguments: [ss_store_sk#39], [s_store_sk#48], Inner, BuildRight - -(31) CometProject -Input [11]: [d_week_seq#38, ss_store_sk#39, sun_sales#51, mon_sales#52, tue_sales#53, wed_sales#54, thu_sales#55, fri_sales#56, sat_sales#57, s_store_sk#48, s_store_id#50] -Arguments: [d_week_seq#38, sun_sales#51, mon_sales#52, tue_sales#53, wed_sales#54, thu_sales#55, fri_sales#56, sat_sales#57, s_store_id#50], [d_week_seq#38, sun_sales#51, mon_sales#52, tue_sales#53, wed_sales#54, thu_sales#55, fri_sales#56, sat_sales#57, s_store_id#50] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#58, d_week_seq#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1224), LessThanOrEqual(d_month_seq,1235), IsNotNull(d_week_seq)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_month_seq#58, d_week_seq#59] -Condition : (((isnotnull(d_month_seq#58) AND (d_month_seq#58 >= 1224)) AND (d_month_seq#58 <= 1235)) AND isnotnull(d_week_seq#59)) - -(34) CometProject -Input [2]: [d_month_seq#58, d_week_seq#59] -Arguments: [d_week_seq#59], [d_week_seq#59] - -(35) CometBroadcastExchange -Input [1]: [d_week_seq#59] -Arguments: [d_week_seq#59] - -(36) CometBroadcastHashJoin -Left output [9]: [d_week_seq#38, sun_sales#51, mon_sales#52, tue_sales#53, wed_sales#54, thu_sales#55, fri_sales#56, sat_sales#57, s_store_id#50] -Right output [1]: [d_week_seq#59] -Arguments: [d_week_seq#38], [d_week_seq#59], Inner, BuildRight - -(37) CometProject -Input [10]: [d_week_seq#38, sun_sales#51, mon_sales#52, tue_sales#53, wed_sales#54, thu_sales#55, fri_sales#56, sat_sales#57, s_store_id#50, d_week_seq#59] -Arguments: [d_week_seq2#60, s_store_id2#61, sun_sales2#62, mon_sales2#63, tue_sales2#64, wed_sales2#65, thu_sales2#66, fri_sales2#67, sat_sales2#68], [d_week_seq#38 AS d_week_seq2#60, s_store_id#50 AS s_store_id2#61, sun_sales#51 AS sun_sales2#62, mon_sales#52 AS mon_sales2#63, tue_sales#53 AS tue_sales2#64, wed_sales#54 AS wed_sales2#65, thu_sales#55 AS thu_sales2#66, fri_sales#56 AS fri_sales2#67, sat_sales#57 AS sat_sales2#68] - -(38) CometBroadcastExchange -Input [9]: [d_week_seq2#60, s_store_id2#61, sun_sales2#62, mon_sales2#63, tue_sales2#64, wed_sales2#65, thu_sales2#66, fri_sales2#67, sat_sales2#68] -Arguments: [d_week_seq2#60, s_store_id2#61, sun_sales2#62, mon_sales2#63, tue_sales2#64, wed_sales2#65, thu_sales2#66, fri_sales2#67, sat_sales2#68] - -(39) CometBroadcastHashJoin -Left output [10]: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37] -Right output [9]: [d_week_seq2#60, s_store_id2#61, sun_sales2#62, mon_sales2#63, tue_sales2#64, wed_sales2#65, thu_sales2#66, fri_sales2#67, sat_sales2#68] -Arguments: [s_store_id1#30, d_week_seq1#29], [s_store_id2#61, (d_week_seq2#60 - 52)], Inner, BuildRight - -(40) CometProject -Input [19]: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37, d_week_seq2#60, s_store_id2#61, sun_sales2#62, mon_sales2#63, tue_sales2#64, wed_sales2#65, thu_sales2#66, fri_sales2#67, sat_sales2#68] -Arguments: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#69, (mon_sales1 / mon_sales2)#70, (tue_sales1 / tue_sales2)#71, (wed_sales1 / wed_sales2)#72, (thu_sales1 / thu_sales2)#73, (fri_sales1 / fri_sales2)#74, (sat_sales1 / sat_sales2)#75], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1#31 / sun_sales2#62) AS (sun_sales1 / sun_sales2)#69, (mon_sales1#32 / mon_sales2#63) AS (mon_sales1 / mon_sales2)#70, (tue_sales1#33 / tue_sales2#64) AS (tue_sales1 / tue_sales2)#71, (wed_sales1#34 / wed_sales2#65) AS (wed_sales1 / wed_sales2)#72, (thu_sales1#35 / thu_sales2#66) AS (thu_sales1 / thu_sales2)#73, (fri_sales1#36 / fri_sales2#67) AS (fri_sales1 / fri_sales2)#74, (sat_sales1#37 / sat_sales2#68) AS (sat_sales1 / sat_sales2)#75] - -(41) CometTakeOrderedAndProject -Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#69, (mon_sales1 / mon_sales2)#70, (tue_sales1 / tue_sales2)#71, (wed_sales1 / wed_sales2)#72, (thu_sales1 / thu_sales2)#73, (fri_sales1 / fri_sales2)#74, (sat_sales1 / sat_sales2)#75] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name1#28 ASC NULLS FIRST,s_store_id1#30 ASC NULLS FIRST,d_week_seq1#29 ASC NULLS FIRST], output=[s_store_name1#28,s_store_id1#30,d_week_seq1#29,(sun_sales1 / sun_sales2)#69,(mon_sales1 / mon_sales2)#70,(tue_sales1 / tue_sales2)#71,(wed_sales1 / wed_sales2)#72,(thu_sales1 / thu_sales2)#73,(fri_sales1 / fri_sales2)#74,(sat_sales1 / sat_sales2)#75]), [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#69, (mon_sales1 / mon_sales2)#70, (tue_sales1 / tue_sales2)#71, (wed_sales1 / wed_sales2)#72, (thu_sales1 / thu_sales2)#73, (fri_sales1 / fri_sales2)#74, (sat_sales1 / sat_sales2)#75], 100, 0, [s_store_name1#28 ASC NULLS FIRST, s_store_id1#30 ASC NULLS FIRST, d_week_seq1#29 ASC NULLS FIRST], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#69, (mon_sales1 / mon_sales2)#70, (tue_sales1 / tue_sales2)#71, (wed_sales1 / wed_sales2)#72, (thu_sales1 / thu_sales2)#73, (fri_sales1 / fri_sales2)#74, (sat_sales1 / sat_sales2)#75] - -(42) CometColumnarToRow [codegen id : 1] -Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#69, (mon_sales1 / mon_sales2)#70, (tue_sales1 / tue_sales2)#71, (wed_sales1 / wed_sales2)#72, (thu_sales1 / thu_sales2)#73, (fri_sales1 / fri_sales2)#74, (sat_sales1 / sat_sales2)#75] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/extended.txt deleted file mode 100644 index 3d50a2131e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/extended.txt +++ /dev/null @@ -1,53 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 50 out of 50 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index bca9c6463d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q59/simplified.txt +++ /dev/null @@ -1,44 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometBroadcastHashJoin [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometProject [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id,s_store_name] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - CometExchange [d_week_seq,ss_store_sk] #1 - CometHashAggregate [d_day_name,ss_sales_price] [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] - CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] - CometBroadcastExchange [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 - CometBroadcastExchange [s_store_sk,s_store_id] #6 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [d_week_seq] #7 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/explain.txt deleted file mode 100644 index 93d8a46c2d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/explain.txt +++ /dev/null @@ -1,304 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (40) -+- * Filter (39) - +- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * CometColumnarToRow (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.customer (4) - : : +- BroadcastExchange (13) - : : +- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet spark_catalog.default.store_sales (10) - : +- ReusedExchange (16) - +- BroadcastExchange (32) - +- * CometColumnarToRow (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (20) - : +- CometNativeScan parquet spark_catalog.default.item (19) - +- CometBroadcastExchange (28) - +- CometFilter (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometFilter (22) - +- CometNativeScan parquet spark_catalog.default.item (21) - - -(1) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#1, ca_state#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ca_address_sk#1, ca_state#2] -Condition : isnotnull(ca_address_sk#1) - -(3) CometProject -Input [2]: [ca_address_sk#1, ca_state#2] -Arguments: [ca_address_sk#1, ca_state#3], [ca_address_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#2, 2, true, false, true) AS ca_state#3] - -(4) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_customer_sk#4)) - -(6) CometBroadcastExchange -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [c_customer_sk#4, c_current_addr_sk#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#1, ca_state#3] -Right output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] - -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ca_state#3, c_customer_sk#4] - -(10) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 1] -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(12) Filter [codegen id : 1] -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) - -(13) BroadcastExchange -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [c_customer_sk#4] -Right keys [1]: [ss_customer_sk#7] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(16) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#10] - -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [2]: [ca_state#3, ss_item_sk#6] -Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] - -(19) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#11, i_current_price#12, i_category#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [i_item_sk#11, i_current_price#12, i_category#13] -Condition : (isnotnull(i_current_price#12) AND isnotnull(i_item_sk#11)) - -(21) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_current_price#14, i_category#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_current_price#14, i_category#15] -Condition : isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#15, 50, true, false, true)) - -(23) CometProject -Input [2]: [i_current_price#14, i_category#15] -Arguments: [i_current_price#14, i_category#16], [i_current_price#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#15, 50, true, false, true) AS i_category#16] - -(24) CometHashAggregate -Input [2]: [i_current_price#14, i_category#16] -Keys [1]: [i_category#16] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#14))] - -(25) CometExchange -Input [3]: [i_category#16, sum#17, count#18] -Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(26) CometHashAggregate -Input [3]: [i_category#16, sum#17, count#18] -Keys [1]: [i_category#16] -Functions [1]: [avg(UnscaledValue(i_current_price#14))] - -(27) CometFilter -Input [2]: [avg(i_current_price)#19, i_category#16] -Condition : isnotnull(avg(i_current_price)#19) - -(28) CometBroadcastExchange -Input [2]: [avg(i_current_price)#19, i_category#16] -Arguments: [avg(i_current_price)#19, i_category#16] - -(29) CometBroadcastHashJoin -Left output [3]: [i_item_sk#11, i_current_price#12, i_category#13] -Right output [2]: [avg(i_current_price)#19, i_category#16] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#13, 50, true, false, true)], [i_category#16], Inner, (cast(i_current_price#12 as decimal(14,7)) > (1.2 * avg(i_current_price)#19)), BuildRight - -(30) CometProject -Input [5]: [i_item_sk#11, i_current_price#12, i_category#13, avg(i_current_price)#19, i_category#16] -Arguments: [i_item_sk#11], [i_item_sk#11] - -(31) CometColumnarToRow [codegen id : 3] -Input [1]: [i_item_sk#11] - -(32) BroadcastExchange -Input [1]: [i_item_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(33) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#6] -Right keys [1]: [i_item_sk#11] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 4] -Output [1]: [ca_state#3] -Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#11] - -(35) HashAggregate [codegen id : 4] -Input [1]: [ca_state#3] -Keys [1]: [ca_state#3] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#20] -Results [2]: [ca_state#3, count#21] - -(36) CometColumnarExchange -Input [2]: [ca_state#3, count#21] -Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(37) CometColumnarToRow [codegen id : 5] -Input [2]: [ca_state#3, count#21] - -(38) HashAggregate [codegen id : 5] -Input [2]: [ca_state#3, count#21] -Keys [1]: [ca_state#3] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#22] -Results [2]: [ca_state#3 AS state#23, count(1)#22 AS cnt#24] - -(39) Filter [codegen id : 5] -Input [2]: [state#23, cnt#24] -Condition : (cnt#24 >= 10) - -(40) TakeOrderedAndProject -Input [2]: [state#23, cnt#24] -Arguments: 100, [cnt#24 ASC NULLS FIRST], [state#23, cnt#24] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometNativeScan parquet spark_catalog.default.date_dim (41) - - -(41) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#25] -Condition : ((isnotnull(d_month_seq#25) AND (d_month_seq#25 = Subquery scalar-subquery#26, [id=#27])) AND isnotnull(d_date_sk#10)) - -(43) CometProject -Input [2]: [d_date_sk#10, d_month_seq#25] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(45) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#26, [id=#27] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometNativeScan parquet spark_catalog.default.date_dim (46) - - -(46) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_month_seq#28, d_year#29, d_moy#30] -Condition : (((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2000)) AND (d_moy#30 = 1)) - -(48) CometProject -Input [3]: [d_month_seq#28, d_year#29, d_moy#30] -Arguments: [d_month_seq#28], [d_month_seq#28] - -(49) CometHashAggregate -Input [1]: [d_month_seq#28] -Keys [1]: [d_month_seq#28] -Functions: [] - -(50) CometExchange -Input [1]: [d_month_seq#28] -Arguments: hashpartitioning(d_month_seq#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(51) CometHashAggregate -Input [1]: [d_month_seq#28] -Keys [1]: [d_month_seq#28] -Functions: [] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#28] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/simplified.txt deleted file mode 100644 index cec4eecbc6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/simplified.txt +++ /dev/null @@ -1,69 +0,0 @@ -TakeOrderedAndProject [cnt,state] - WholeStageCodegen (5) - Filter [cnt] - HashAggregate [ca_state,count] [count(1),state,cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state] #1 - WholeStageCodegen (4) - HashAggregate [ca_state] [count,count] - Project [ca_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ca_state,ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ca_state,ss_item_sk,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_customer_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #7 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #8 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_current_price,i_category] - CometFilter [i_current_price,i_category] - CometNativeScan 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/q6.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/explain.txt deleted file mode 100644 index da1b420416..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometFilter (38) - +- CometHashAggregate (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (4) - : : +- CometBroadcastExchange (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (9) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- CometBroadcastExchange (29) - +- CometFilter (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (22) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#1, ca_state#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ca_address_sk#1, ca_state#2] -Condition : isnotnull(ca_address_sk#1) - -(3) CometProject -Input [2]: [ca_address_sk#1, ca_state#2] -Arguments: [ca_address_sk#1, ca_state#3], [ca_address_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#2, 2, true, false, true) AS ca_state#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_customer_sk#4)) - -(6) CometBroadcastExchange -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [c_customer_sk#4, c_current_addr_sk#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#1, ca_state#3] -Right output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) - -(11) CometBroadcastExchange -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(12) CometBroadcastHashJoin -Left output [2]: [ca_state#3, c_customer_sk#4] -Right output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [c_customer_sk#4], [ss_customer_sk#7], Inner, BuildRight - -(13) CometProject -Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8], [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(16) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(18) CometBroadcastHashJoin -Left output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(19) CometProject -Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ca_state#3, ss_item_sk#6], [ca_state#3, ss_item_sk#6] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Condition : (isnotnull(i_current_price#15) AND isnotnull(i_item_sk#14)) - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_current_price#17, i_category#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(23) CometFilter -Input [2]: [i_current_price#17, i_category#18] -Condition : isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true)) - -(24) CometProject -Input [2]: [i_current_price#17, i_category#18] -Arguments: [i_current_price#17, i_category#19], [i_current_price#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true) AS i_category#19] - -(25) CometHashAggregate -Input [2]: [i_current_price#17, i_category#19] -Keys [1]: [i_category#19] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#17))] - -(26) CometExchange -Input [3]: [i_category#19, sum#20, count#21] -Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [3]: [i_category#19, sum#20, count#21] -Keys [1]: [i_category#19] -Functions [1]: [avg(UnscaledValue(i_current_price#17))] - -(28) CometFilter -Input [2]: [avg(i_current_price)#22, i_category#19] -Condition : isnotnull(avg(i_current_price)#22) - -(29) CometBroadcastExchange -Input [2]: [avg(i_current_price)#22, i_category#19] -Arguments: [avg(i_current_price)#22, i_category#19] - -(30) CometBroadcastHashJoin -Left output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Right output [2]: [avg(i_current_price)#22, i_category#19] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true)], [i_category#19], Inner, (cast(i_current_price#15 as decimal(14,7)) > (1.2 * avg(i_current_price)#22)), BuildRight - -(31) CometProject -Input [5]: [i_item_sk#14, i_current_price#15, i_category#16, avg(i_current_price)#22, i_category#19] -Arguments: [i_item_sk#14], [i_item_sk#14] - -(32) CometBroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: [i_item_sk#14] - -(33) CometBroadcastHashJoin -Left output [2]: [ca_state#3, ss_item_sk#6] -Right output [1]: [i_item_sk#14] -Arguments: [ss_item_sk#6], [i_item_sk#14], Inner, BuildRight - -(34) CometProject -Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#14] -Arguments: [ca_state#3], [ca_state#3] - -(35) CometHashAggregate -Input [1]: [ca_state#3] -Keys [1]: [ca_state#3] -Functions [1]: [partial_count(1)] - -(36) CometExchange -Input [2]: [ca_state#3, count#23] -Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(37) CometHashAggregate -Input [2]: [ca_state#3, count#23] -Keys [1]: [ca_state#3] -Functions [1]: [count(1)] - -(38) CometFilter -Input [2]: [state#24, cnt#25] -Condition : (cnt#25 >= 10) - -(39) CometTakeOrderedAndProject -Input [2]: [state#24, cnt#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#25 ASC NULLS FIRST], output=[state#24,cnt#25]), [state#24, cnt#25], 100, 0, [cnt#25 ASC NULLS FIRST], [state#24, cnt#25] - -(40) CometColumnarToRow [codegen id : 1] -Input [2]: [state#24, cnt#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(43) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(45) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) - - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#26, d_year#27, d_moy#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Condition : (((isnotnull(d_year#27) AND isnotnull(d_moy#28)) AND (d_year#27 = 2000)) AND (d_moy#28 = 1)) - -(48) CometProject -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Arguments: [d_month_seq#26], [d_month_seq#26] - -(49) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] -Functions: [] - -(50) CometExchange -Input [1]: [d_month_seq#26] -Arguments: hashpartitioning(d_month_seq#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(51) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] -Functions: [] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#26] - -Subquery:3 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/simplified.txt deleted file mode 100644 index 2c772b5998..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [state,cnt] - CometFilter [state,cnt] - CometHashAggregate [count] [state,cnt,ca_state,count(1)] - CometExchange [ca_state] #1 - CometHashAggregate [ca_state,count] - CometProject [ca_state] - CometBroadcastHashJoin [ca_state,ss_item_sk,i_item_sk] - CometProject [ca_state,ss_item_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 [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,ca_state,c_customer_sk,c_current_addr_sk] - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk] #7 - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #8 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #9 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_current_price,i_category] - CometFilter [i_current_price,i_category] - CometScan [native_iceberg_compat] 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/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/explain.txt deleted file mode 100644 index da1b420416..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometFilter (38) - +- CometHashAggregate (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (4) - : : +- CometBroadcastExchange (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (9) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- CometBroadcastExchange (29) - +- CometFilter (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (22) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#1, ca_state#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ca_address_sk#1, ca_state#2] -Condition : isnotnull(ca_address_sk#1) - -(3) CometProject -Input [2]: [ca_address_sk#1, ca_state#2] -Arguments: [ca_address_sk#1, ca_state#3], [ca_address_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#2, 2, true, false, true) AS ca_state#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_customer_sk#4)) - -(6) CometBroadcastExchange -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [c_customer_sk#4, c_current_addr_sk#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#1, ca_state#3] -Right output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) - -(11) CometBroadcastExchange -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(12) CometBroadcastHashJoin -Left output [2]: [ca_state#3, c_customer_sk#4] -Right output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [c_customer_sk#4], [ss_customer_sk#7], Inner, BuildRight - -(13) CometProject -Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8], [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(16) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(18) CometBroadcastHashJoin -Left output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(19) CometProject -Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ca_state#3, ss_item_sk#6], [ca_state#3, ss_item_sk#6] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Condition : (isnotnull(i_current_price#15) AND isnotnull(i_item_sk#14)) - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_current_price#17, i_category#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(23) CometFilter -Input [2]: [i_current_price#17, i_category#18] -Condition : isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true)) - -(24) CometProject -Input [2]: [i_current_price#17, i_category#18] -Arguments: [i_current_price#17, i_category#19], [i_current_price#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true) AS i_category#19] - -(25) CometHashAggregate -Input [2]: [i_current_price#17, i_category#19] -Keys [1]: [i_category#19] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#17))] - -(26) CometExchange -Input [3]: [i_category#19, sum#20, count#21] -Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [3]: [i_category#19, sum#20, count#21] -Keys [1]: [i_category#19] -Functions [1]: [avg(UnscaledValue(i_current_price#17))] - -(28) CometFilter -Input [2]: [avg(i_current_price)#22, i_category#19] -Condition : isnotnull(avg(i_current_price)#22) - -(29) CometBroadcastExchange -Input [2]: [avg(i_current_price)#22, i_category#19] -Arguments: [avg(i_current_price)#22, i_category#19] - -(30) CometBroadcastHashJoin -Left output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Right output [2]: [avg(i_current_price)#22, i_category#19] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true)], [i_category#19], Inner, (cast(i_current_price#15 as decimal(14,7)) > (1.2 * avg(i_current_price)#22)), BuildRight - -(31) CometProject -Input [5]: [i_item_sk#14, i_current_price#15, i_category#16, avg(i_current_price)#22, i_category#19] -Arguments: [i_item_sk#14], [i_item_sk#14] - -(32) CometBroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: [i_item_sk#14] - -(33) CometBroadcastHashJoin -Left output [2]: [ca_state#3, ss_item_sk#6] -Right output [1]: [i_item_sk#14] -Arguments: [ss_item_sk#6], [i_item_sk#14], Inner, BuildRight - -(34) CometProject -Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#14] -Arguments: [ca_state#3], [ca_state#3] - -(35) CometHashAggregate -Input [1]: [ca_state#3] -Keys [1]: [ca_state#3] -Functions [1]: [partial_count(1)] - -(36) CometExchange -Input [2]: [ca_state#3, count#23] -Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(37) CometHashAggregate -Input [2]: [ca_state#3, count#23] -Keys [1]: [ca_state#3] -Functions [1]: [count(1)] - -(38) CometFilter -Input [2]: [state#24, cnt#25] -Condition : (cnt#25 >= 10) - -(39) CometTakeOrderedAndProject -Input [2]: [state#24, cnt#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#25 ASC NULLS FIRST], output=[state#24,cnt#25]), [state#24, cnt#25], 100, 0, [cnt#25 ASC NULLS FIRST], [state#24, cnt#25] - -(40) CometColumnarToRow [codegen id : 1] -Input [2]: [state#24, cnt#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(43) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(45) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) - - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#26, d_year#27, d_moy#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Condition : (((isnotnull(d_year#27) AND isnotnull(d_moy#28)) AND (d_year#27 = 2000)) AND (d_moy#28 = 1)) - -(48) CometProject -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Arguments: [d_month_seq#26], [d_month_seq#26] - -(49) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] -Functions: [] - -(50) CometExchange -Input [1]: [d_month_seq#26] -Arguments: hashpartitioning(d_month_seq#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(51) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] -Functions: [] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#26] - -Subquery:3 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/extended.txt deleted file mode 100644 index f5b69fc6cf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/extended.txt +++ /dev/null @@ -1,57 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 2c772b5998..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [state,cnt] - CometFilter [state,cnt] - CometHashAggregate [count] [state,cnt,ca_state,count(1)] - CometExchange [ca_state] #1 - CometHashAggregate [ca_state,count] - CometProject [ca_state] - CometBroadcastHashJoin [ca_state,ss_item_sk,i_item_sk] - CometProject [ca_state,ss_item_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 [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,ca_state,c_customer_sk,c_current_addr_sk] - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk] #7 - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #8 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #9 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_current_price,i_category] - CometFilter [i_current_price,i_category] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/explain.txt deleted file mode 100644 index 5584119109..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/explain.txt +++ /dev/null @@ -1,420 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (67) -+- * HashAggregate (66) - +- * CometColumnarToRow (65) - +- CometColumnarExchange (64) - +- * HashAggregate (63) - +- Union (62) - :- * HashAggregate (29) - : +- * CometColumnarToRow (28) - : +- CometColumnarExchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * CometColumnarToRow (22) - : +- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.item (14) - : +- CometBroadcastExchange (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometNativeScan parquet spark_catalog.default.item (16) - :- * HashAggregate (45) - : +- * CometColumnarToRow (44) - : +- CometColumnarExchange (43) - : +- * HashAggregate (42) - : +- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (38) - : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * Filter (32) - : : : : +- * ColumnarToRow (31) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (30) - : : : +- ReusedExchange (33) - : : +- ReusedExchange (36) - : +- ReusedExchange (39) - +- * HashAggregate (61) - +- * CometColumnarToRow (60) - +- CometColumnarExchange (59) - +- * HashAggregate (58) - +- * Project (57) - +- * BroadcastHashJoin Inner BuildRight (56) - :- * Project (54) - : +- * BroadcastHashJoin Inner BuildRight (53) - : :- * Project (51) - : : +- * BroadcastHashJoin Inner BuildRight (50) - : : :- * Filter (48) - : : : +- * ColumnarToRow (47) - : : : +- Scan parquet spark_catalog.default.web_sales (46) - : : +- ReusedExchange (49) - : +- ReusedExchange (52) - +- ReusedExchange (55) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(4) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_gmt_offset#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isnotnull(ca_address_sk#7)) - -(9) CometProject -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Arguments: [ca_address_sk#7], [ca_address_sk#7] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [ca_address_sk#7] - -(11) BroadcastExchange -Input [1]: [ca_address_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] - -(14) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#9, i_item_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [i_item_sk#9, i_item_id#10] -Condition : isnotnull(i_item_sk#9) - -(16) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_id#11, i_category#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(17) CometFilter -Input [2]: [i_item_id#11, i_category#12] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#12, 50, true, false, true) = Music ) - -(18) CometProject -Input [2]: [i_item_id#11, i_category#12] -Arguments: [i_item_id#13], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#11, 16, true, false, true) AS i_item_id#13] - -(19) CometBroadcastExchange -Input [1]: [i_item_id#13] -Arguments: [i_item_id#13] - -(20) CometBroadcastHashJoin -Left output [2]: [i_item_sk#9, i_item_id#10] -Right output [1]: [i_item_id#13] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#10, 16, true, false, true)], [i_item_id#13], LeftSemi, BuildRight - -(21) CometProject -Input [2]: [i_item_sk#9, i_item_id#10] -Arguments: [i_item_sk#9, i_item_id#14], [i_item_sk#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#10, 16, true, false, true) AS i_item_id#14] - -(22) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#9, i_item_id#14] - -(23) BroadcastExchange -Input [2]: [i_item_sk#9, i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#9] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 4] -Output [2]: [ss_ext_sales_price#3, i_item_id#14] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_item_id#14] - -(26) HashAggregate [codegen id : 4] -Input [2]: [ss_ext_sales_price#3, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum#15] -Results [2]: [i_item_id#14, sum#16] - -(27) CometColumnarExchange -Input [2]: [i_item_id#14, sum#16] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometColumnarToRow [codegen id : 5] -Input [2]: [i_item_id#14, sum#16] - -(29) HashAggregate [codegen id : 5] -Input [2]: [i_item_id#14, sum#16] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#17] -Results [2]: [i_item_id#14 AS i_item_id#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#17,17,2) AS total_sales#19] - -(30) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 9] -Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] - -(32) Filter [codegen id : 9] -Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -Condition : (isnotnull(cs_bill_addr_sk#20) AND isnotnull(cs_item_sk#21)) - -(33) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#24] - -(34) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 9] -Output [3]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22] -Input [5]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#24] - -(36) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#25] - -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_bill_addr_sk#20] -Right keys [1]: [ca_address_sk#25] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 9] -Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] -Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, ca_address_sk#25] - -(39) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#26, i_item_id#27] - -(40) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#21] -Right keys [1]: [i_item_sk#26] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 9] -Output [2]: [cs_ext_sales_price#22, i_item_id#27] -Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#26, i_item_id#27] - -(42) HashAggregate [codegen id : 9] -Input [2]: [cs_ext_sales_price#22, i_item_id#27] -Keys [1]: [i_item_id#27] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#27, sum#29] - -(43) CometColumnarExchange -Input [2]: [i_item_id#27, sum#29] -Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(44) CometColumnarToRow [codegen id : 10] -Input [2]: [i_item_id#27, sum#29] - -(45) HashAggregate [codegen id : 10] -Input [2]: [i_item_id#27, sum#29] -Keys [1]: [i_item_id#27] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] -Results [2]: [i_item_id#27, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS total_sales#31] - -(46) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(47) ColumnarToRow [codegen id : 14] -Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] - -(48) Filter [codegen id : 14] -Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] -Condition : (isnotnull(ws_bill_addr_sk#33) AND isnotnull(ws_item_sk#32)) - -(49) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#36] - -(50) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#36] -Join type: Inner -Join condition: None - -(51) Project [codegen id : 14] -Output [3]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34] -Input [5]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35, d_date_sk#36] - -(52) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#37] - -(53) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#33] -Right keys [1]: [ca_address_sk#37] -Join type: Inner -Join condition: None - -(54) Project [codegen id : 14] -Output [2]: [ws_item_sk#32, ws_ext_sales_price#34] -Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ca_address_sk#37] - -(55) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#38, i_item_id#39] - -(56) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_item_sk#32] -Right keys [1]: [i_item_sk#38] -Join type: Inner -Join condition: None - -(57) Project [codegen id : 14] -Output [2]: [ws_ext_sales_price#34, i_item_id#39] -Input [4]: [ws_item_sk#32, ws_ext_sales_price#34, i_item_sk#38, i_item_id#39] - -(58) HashAggregate [codegen id : 14] -Input [2]: [ws_ext_sales_price#34, i_item_id#39] -Keys [1]: [i_item_id#39] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#34))] -Aggregate Attributes [1]: [sum#40] -Results [2]: [i_item_id#39, sum#41] - -(59) CometColumnarExchange -Input [2]: [i_item_id#39, sum#41] -Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(60) CometColumnarToRow [codegen id : 15] -Input [2]: [i_item_id#39, sum#41] - -(61) HashAggregate [codegen id : 15] -Input [2]: [i_item_id#39, sum#41] -Keys [1]: [i_item_id#39] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#34))#42] -Results [2]: [i_item_id#39, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#34))#42,17,2) AS total_sales#43] - -(62) Union - -(63) HashAggregate [codegen id : 16] -Input [2]: [i_item_id#18, total_sales#19] -Keys [1]: [i_item_id#18] -Functions [1]: [partial_sum(total_sales#19)] -Aggregate Attributes [2]: [sum#44, isEmpty#45] -Results [3]: [i_item_id#18, sum#46, isEmpty#47] - -(64) CometColumnarExchange -Input [3]: [i_item_id#18, sum#46, isEmpty#47] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(65) CometColumnarToRow [codegen id : 17] -Input [3]: [i_item_id#18, sum#46, isEmpty#47] - -(66) HashAggregate [codegen id : 17] -Input [3]: [i_item_id#18, sum#46, isEmpty#47] -Keys [1]: [i_item_id#18] -Functions [1]: [sum(total_sales#19)] -Aggregate Attributes [1]: [sum(total_sales#19)#48] -Results [2]: [i_item_id#18, sum(total_sales#19)#48 AS total_sales#49] - -(67) TakeOrderedAndProject -Input [2]: [i_item_id#18, total_sales#49] -Arguments: 100, [i_item_id#18 ASC NULLS FIRST, total_sales#49 ASC NULLS FIRST], [i_item_id#18, total_sales#49] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (72) -+- * CometColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometNativeScan parquet spark_catalog.default.date_dim (68) - - -(68) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#50, d_moy#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [3]: [d_date_sk#6, d_year#50, d_moy#51] -Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 1998)) AND (d_moy#51 = 9)) AND isnotnull(d_date_sk#6)) - -(70) CometProject -Input [3]: [d_date_sk#6, d_year#50, d_moy#51] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(71) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(72) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/simplified.txt deleted file mode 100644 index 4c05038b34..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/simplified.txt +++ /dev/null @@ -1,106 +0,0 @@ -TakeOrderedAndProject [i_item_id,total_sales] - WholeStageCodegen (17) - HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (16) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),i_item_id,total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #6 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_id,i_category] - WholeStageCodegen (10) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #7 - WholeStageCodegen (9) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_addr_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (15) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #8 - WholeStageCodegen (14) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/explain.txt deleted file mode 100644 index e5e165b5ec..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,385 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (62) -+- CometTakeOrderedAndProject (61) - +- CometHashAggregate (60) - +- CometExchange (59) - +- CometHashAggregate (58) - +- CometUnion (57) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- CometHashAggregate (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometFilter (30) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- CometHashAggregate (56) - +- CometExchange (55) - +- CometHashAggregate (54) - +- CometProject (53) - +- CometBroadcastHashJoin (52) - :- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometFilter (44) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (43) - : : +- ReusedExchange (45) - : +- ReusedExchange (48) - +- ReusedExchange (51) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 9)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(11) CometProject -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Arguments: [ca_address_sk#9], [ca_address_sk#9] - -(12) CometBroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: [ca_address_sk#9] - -(13) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Right output [1]: [ca_address_sk#9] -Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] -Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#11, i_item_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#11, i_item_id#12] -Condition : isnotnull(i_item_sk#11) - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_id#13, i_category#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(18) CometFilter -Input [2]: [i_item_id#13, i_category#14] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#14, 50, true, false, true) = Music ) - -(19) CometProject -Input [2]: [i_item_id#13, i_category#14] -Arguments: [i_item_id#15], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#13, 16, true, false, true) AS i_item_id#15] - -(20) CometBroadcastExchange -Input [1]: [i_item_id#15] -Arguments: [i_item_id#15] - -(21) CometBroadcastHashJoin -Left output [2]: [i_item_sk#11, i_item_id#12] -Right output [1]: [i_item_id#15] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#12, 16, true, false, true)], [i_item_id#15], LeftSemi, BuildRight - -(22) CometProject -Input [2]: [i_item_sk#11, i_item_id#12] -Arguments: [i_item_sk#11, i_item_id#16], [i_item_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#12, 16, true, false, true) AS i_item_id#16] - -(23) CometBroadcastExchange -Input [2]: [i_item_sk#11, i_item_id#16] -Arguments: [i_item_sk#11, i_item_id#16] - -(24) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Right output [2]: [i_item_sk#11, i_item_id#16] -Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight - -(25) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_item_id#16] -Arguments: [ss_ext_sales_price#3, i_item_id#16], [ss_ext_sales_price#3, i_item_id#16] - -(26) CometHashAggregate -Input [2]: [ss_ext_sales_price#3, i_item_id#16] -Keys [1]: [i_item_id#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(27) CometExchange -Input [2]: [i_item_id#16, sum#17] -Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [2]: [i_item_id#16, sum#17] -Keys [1]: [i_item_id#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Condition : (isnotnull(cs_bill_addr_sk#18) AND isnotnull(cs_item_sk#19)) - -(31) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#23] - -(32) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Right output [1]: [d_date_sk#23] -Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(33) CometProject -Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] -Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] - -(34) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#24] - -(35) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -Right output [1]: [ca_address_sk#24] -Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner, BuildRight - -(36) CometProject -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] -Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] - -(37) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#25, i_item_id#26] - -(38) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] -Right output [2]: [i_item_sk#25, i_item_id#26] -Arguments: [cs_item_sk#19], [i_item_sk#25], Inner, BuildRight - -(39) CometProject -Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_item_id#26] -Arguments: [cs_ext_sales_price#20, i_item_id#26], [cs_ext_sales_price#20, i_item_id#26] - -(40) CometHashAggregate -Input [2]: [cs_ext_sales_price#20, i_item_id#26] -Keys [1]: [i_item_id#26] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] - -(41) CometExchange -Input [2]: [i_item_id#26, sum#27] -Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(42) CometHashAggregate -Input [2]: [i_item_id#26, sum#27] -Keys [1]: [i_item_id#26] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(44) CometFilter -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Condition : (isnotnull(ws_bill_addr_sk#29) AND isnotnull(ws_item_sk#28)) - -(45) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(46) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [ws_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(47) CometProject -Input [5]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31, d_date_sk#33] -Arguments: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30], [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] - -(48) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#34] - -(49) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] -Right output [1]: [ca_address_sk#34] -Arguments: [ws_bill_addr_sk#29], [ca_address_sk#34], Inner, BuildRight - -(50) CometProject -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ca_address_sk#34] -Arguments: [ws_item_sk#28, ws_ext_sales_price#30], [ws_item_sk#28, ws_ext_sales_price#30] - -(51) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#35, i_item_id#36] - -(52) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#28, ws_ext_sales_price#30] -Right output [2]: [i_item_sk#35, i_item_id#36] -Arguments: [ws_item_sk#28], [i_item_sk#35], Inner, BuildRight - -(53) CometProject -Input [4]: [ws_item_sk#28, ws_ext_sales_price#30, i_item_sk#35, i_item_id#36] -Arguments: [ws_ext_sales_price#30, i_item_id#36], [ws_ext_sales_price#30, i_item_id#36] - -(54) CometHashAggregate -Input [2]: [ws_ext_sales_price#30, i_item_id#36] -Keys [1]: [i_item_id#36] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#30))] - -(55) CometExchange -Input [2]: [i_item_id#36, sum#37] -Arguments: hashpartitioning(i_item_id#36, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(56) CometHashAggregate -Input [2]: [i_item_id#36, sum#37] -Keys [1]: [i_item_id#36] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#30))] - -(57) CometUnion -Child 0 Input [2]: [i_item_id#38, total_sales#39] -Child 1 Input [2]: [i_item_id#26, total_sales#40] -Child 2 Input [2]: [i_item_id#36, total_sales#41] - -(58) CometHashAggregate -Input [2]: [i_item_id#38, total_sales#39] -Keys [1]: [i_item_id#38] -Functions [1]: [partial_sum(total_sales#39)] - -(59) CometExchange -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(60) CometHashAggregate -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Keys [1]: [i_item_id#38] -Functions [1]: [sum(total_sales#39)] - -(61) CometTakeOrderedAndProject -Input [2]: [i_item_id#38, total_sales#44] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#38 ASC NULLS FIRST,total_sales#44 ASC NULLS FIRST], output=[i_item_id#38,total_sales#44]), [i_item_id#38, total_sales#44], 100, 0, [i_item_id#38 ASC NULLS FIRST, total_sales#44 ASC NULLS FIRST], [i_item_id#38, total_sales#44] - -(62) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_id#38, total_sales#44] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (67) -+- * CometColumnarToRow (66) - +- CometProject (65) - +- CometFilter (64) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) - - -(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] -ReadSchema: struct - -(64) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 9)) AND isnotnull(d_date_sk#6)) - -(65) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(66) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(67) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/simplified.txt deleted file mode 100644 index 086da3b776..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,total_sales] - CometHashAggregate [sum,isEmpty] [i_item_id,total_sales,sum(total_sales)] - CometExchange [i_item_id] #1 - CometHashAggregate [total_sales] [i_item_id,sum,isEmpty] - CometUnion [i_item_id,total_sales] - CometHashAggregate [sum] [i_item_id,total_sales,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #7 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_category] - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - 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,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/explain.txt deleted file mode 100644 index e5e165b5ec..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/explain.txt +++ /dev/null @@ -1,385 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (62) -+- CometTakeOrderedAndProject (61) - +- CometHashAggregate (60) - +- CometExchange (59) - +- CometHashAggregate (58) - +- CometUnion (57) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- CometHashAggregate (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometFilter (30) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- CometHashAggregate (56) - +- CometExchange (55) - +- CometHashAggregate (54) - +- CometProject (53) - +- CometBroadcastHashJoin (52) - :- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometFilter (44) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (43) - : : +- ReusedExchange (45) - : +- ReusedExchange (48) - +- ReusedExchange (51) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 9)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(11) CometProject -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Arguments: [ca_address_sk#9], [ca_address_sk#9] - -(12) CometBroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: [ca_address_sk#9] - -(13) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Right output [1]: [ca_address_sk#9] -Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] -Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#11, i_item_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#11, i_item_id#12] -Condition : isnotnull(i_item_sk#11) - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_id#13, i_category#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(18) CometFilter -Input [2]: [i_item_id#13, i_category#14] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#14, 50, true, false, true) = Music ) - -(19) CometProject -Input [2]: [i_item_id#13, i_category#14] -Arguments: [i_item_id#15], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#13, 16, true, false, true) AS i_item_id#15] - -(20) CometBroadcastExchange -Input [1]: [i_item_id#15] -Arguments: [i_item_id#15] - -(21) CometBroadcastHashJoin -Left output [2]: [i_item_sk#11, i_item_id#12] -Right output [1]: [i_item_id#15] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#12, 16, true, false, true)], [i_item_id#15], LeftSemi, BuildRight - -(22) CometProject -Input [2]: [i_item_sk#11, i_item_id#12] -Arguments: [i_item_sk#11, i_item_id#16], [i_item_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#12, 16, true, false, true) AS i_item_id#16] - -(23) CometBroadcastExchange -Input [2]: [i_item_sk#11, i_item_id#16] -Arguments: [i_item_sk#11, i_item_id#16] - -(24) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Right output [2]: [i_item_sk#11, i_item_id#16] -Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight - -(25) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_item_id#16] -Arguments: [ss_ext_sales_price#3, i_item_id#16], [ss_ext_sales_price#3, i_item_id#16] - -(26) CometHashAggregate -Input [2]: [ss_ext_sales_price#3, i_item_id#16] -Keys [1]: [i_item_id#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(27) CometExchange -Input [2]: [i_item_id#16, sum#17] -Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [2]: [i_item_id#16, sum#17] -Keys [1]: [i_item_id#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Condition : (isnotnull(cs_bill_addr_sk#18) AND isnotnull(cs_item_sk#19)) - -(31) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#23] - -(32) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Right output [1]: [d_date_sk#23] -Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(33) CometProject -Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] -Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] - -(34) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#24] - -(35) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -Right output [1]: [ca_address_sk#24] -Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner, BuildRight - -(36) CometProject -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] -Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] - -(37) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#25, i_item_id#26] - -(38) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] -Right output [2]: [i_item_sk#25, i_item_id#26] -Arguments: [cs_item_sk#19], [i_item_sk#25], Inner, BuildRight - -(39) CometProject -Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_item_id#26] -Arguments: [cs_ext_sales_price#20, i_item_id#26], [cs_ext_sales_price#20, i_item_id#26] - -(40) CometHashAggregate -Input [2]: [cs_ext_sales_price#20, i_item_id#26] -Keys [1]: [i_item_id#26] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] - -(41) CometExchange -Input [2]: [i_item_id#26, sum#27] -Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(42) CometHashAggregate -Input [2]: [i_item_id#26, sum#27] -Keys [1]: [i_item_id#26] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(44) CometFilter -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Condition : (isnotnull(ws_bill_addr_sk#29) AND isnotnull(ws_item_sk#28)) - -(45) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(46) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [ws_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(47) CometProject -Input [5]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31, d_date_sk#33] -Arguments: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30], [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] - -(48) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#34] - -(49) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] -Right output [1]: [ca_address_sk#34] -Arguments: [ws_bill_addr_sk#29], [ca_address_sk#34], Inner, BuildRight - -(50) CometProject -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ca_address_sk#34] -Arguments: [ws_item_sk#28, ws_ext_sales_price#30], [ws_item_sk#28, ws_ext_sales_price#30] - -(51) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#35, i_item_id#36] - -(52) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#28, ws_ext_sales_price#30] -Right output [2]: [i_item_sk#35, i_item_id#36] -Arguments: [ws_item_sk#28], [i_item_sk#35], Inner, BuildRight - -(53) CometProject -Input [4]: [ws_item_sk#28, ws_ext_sales_price#30, i_item_sk#35, i_item_id#36] -Arguments: [ws_ext_sales_price#30, i_item_id#36], [ws_ext_sales_price#30, i_item_id#36] - -(54) CometHashAggregate -Input [2]: [ws_ext_sales_price#30, i_item_id#36] -Keys [1]: [i_item_id#36] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#30))] - -(55) CometExchange -Input [2]: [i_item_id#36, sum#37] -Arguments: hashpartitioning(i_item_id#36, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(56) CometHashAggregate -Input [2]: [i_item_id#36, sum#37] -Keys [1]: [i_item_id#36] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#30))] - -(57) CometUnion -Child 0 Input [2]: [i_item_id#38, total_sales#39] -Child 1 Input [2]: [i_item_id#26, total_sales#40] -Child 2 Input [2]: [i_item_id#36, total_sales#41] - -(58) CometHashAggregate -Input [2]: [i_item_id#38, total_sales#39] -Keys [1]: [i_item_id#38] -Functions [1]: [partial_sum(total_sales#39)] - -(59) CometExchange -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(60) CometHashAggregate -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Keys [1]: [i_item_id#38] -Functions [1]: [sum(total_sales#39)] - -(61) CometTakeOrderedAndProject -Input [2]: [i_item_id#38, total_sales#44] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#38 ASC NULLS FIRST,total_sales#44 ASC NULLS FIRST], output=[i_item_id#38,total_sales#44]), [i_item_id#38, total_sales#44], 100, 0, [i_item_id#38 ASC NULLS FIRST, total_sales#44 ASC NULLS FIRST], [i_item_id#38, total_sales#44] - -(62) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_id#38, total_sales#44] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (67) -+- * CometColumnarToRow (66) - +- CometProject (65) - +- CometFilter (64) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) - - -(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] -ReadSchema: struct - -(64) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 9)) AND isnotnull(d_date_sk#6)) - -(65) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(66) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(67) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/extended.txt deleted file mode 100644 index 36b9a8306d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/extended.txt +++ /dev/null @@ -1,100 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 086da3b776..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,total_sales] - CometHashAggregate [sum,isEmpty] [i_item_id,total_sales,sum(total_sales)] - CometExchange [i_item_id] #1 - CometHashAggregate [total_sales] [i_item_id,sum,isEmpty] - CometUnion [i_item_id,total_sales] - CometHashAggregate [sum] [i_item_id,total_sales,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #7 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_category] - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - 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,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/explain.txt deleted file mode 100644 index e50623a19c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/explain.txt +++ /dev/null @@ -1,425 +0,0 @@ -== Physical Plan == -* Project (69) -+- * BroadcastNestedLoopJoin Inner BuildRight (68) - :- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- CometColumnarExchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (33) - : : +- * BroadcastHashJoin Inner BuildRight (32) - : : :- * Project (26) - : : : +- * BroadcastHashJoin Inner BuildRight (25) - : : : :- * Project (20) - : : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : : :- * Project (17) - : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * CometColumnarToRow (7) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometNativeScan parquet spark_catalog.default.store (4) - : : : : : +- BroadcastExchange (15) - : : : : : +- * CometColumnarToRow (14) - : : : : : +- CometProject (13) - : : : : : +- CometFilter (12) - : : : : : +- CometNativeScan parquet spark_catalog.default.promotion (11) - : : : : +- ReusedExchange (18) - : : : +- BroadcastExchange (24) - : : : +- * CometColumnarToRow (23) - : : : +- CometFilter (22) - : : : +- CometNativeScan parquet spark_catalog.default.customer (21) - : : +- BroadcastExchange (31) - : : +- * CometColumnarToRow (30) - : : +- CometProject (29) - : : +- CometFilter (28) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (27) - : +- BroadcastExchange (38) - : +- * CometColumnarToRow (37) - : +- CometProject (36) - : +- CometFilter (35) - : +- CometNativeScan parquet spark_catalog.default.item (34) - +- BroadcastExchange (67) - +- * HashAggregate (66) - +- * CometColumnarToRow (65) - +- CometColumnarExchange (64) - +- * HashAggregate (63) - +- * Project (62) - +- * BroadcastHashJoin Inner BuildRight (61) - :- * Project (59) - : +- * BroadcastHashJoin Inner BuildRight (58) - : :- * Project (56) - : : +- * BroadcastHashJoin Inner BuildRight (55) - : : :- * Project (53) - : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : :- * Project (50) - : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : :- * Filter (47) - : : : : : +- * ColumnarToRow (46) - : : : : : +- Scan parquet spark_catalog.default.store_sales (45) - : : : : +- ReusedExchange (48) - : : : +- ReusedExchange (51) - : : +- ReusedExchange (54) - : +- ReusedExchange (57) - +- ReusedExchange (60) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 7] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(3) Filter [codegen id : 7] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_store_sk#3) AND isnotnull(ss_promo_sk#4)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1)) - -(4) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_gmt_offset#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [s_store_sk#8, s_gmt_offset#9] -Condition : ((isnotnull(s_gmt_offset#9) AND (s_gmt_offset#9 = -5.00)) AND isnotnull(s_store_sk#8)) - -(6) CometProject -Input [2]: [s_store_sk#8, s_gmt_offset#9] -Arguments: [s_store_sk#8], [s_store_sk#8] - -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [s_store_sk#8] - -(8) BroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 7] -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, s_store_sk#8] - -(11) CometNativeScan parquet spark_catalog.default.promotion -Output [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_dmail#11, 1, true, false, true) = Y) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_email#12, 1, true, false, true) = Y)) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#13, 1, true, false, true) = Y)) AND isnotnull(p_promo_sk#10)) - -(13) CometProject -Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Arguments: [p_promo_sk#10], [p_promo_sk#10] - -(14) CometColumnarToRow [codegen id : 2] -Input [1]: [p_promo_sk#10] - -(15) BroadcastExchange -Input [1]: [p_promo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_promo_sk#4] -Right keys [1]: [p_promo_sk#10] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 7] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, p_promo_sk#10] - -(18) ReusedExchange [Reuses operator id: 74] -Output [1]: [d_date_sk#14] - -(19) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 7] -Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#14] - -(21) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#15, c_current_addr_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [c_customer_sk#15, c_current_addr_sk#16] -Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_current_addr_sk#16)) - -(23) CometColumnarToRow [codegen id : 4] -Input [2]: [c_customer_sk#15, c_current_addr_sk#16] - -(24) BroadcastExchange -Input [2]: [c_customer_sk#15, c_current_addr_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(25) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 7] -Output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#16] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#15, c_current_addr_sk#16] - -(27) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#17, ca_gmt_offset#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [ca_address_sk#17, ca_gmt_offset#18] -Condition : ((isnotnull(ca_gmt_offset#18) AND (ca_gmt_offset#18 = -5.00)) AND isnotnull(ca_address_sk#17)) - -(29) CometProject -Input [2]: [ca_address_sk#17, ca_gmt_offset#18] -Arguments: [ca_address_sk#17], [ca_address_sk#17] - -(30) CometColumnarToRow [codegen id : 5] -Input [1]: [ca_address_sk#17] - -(31) BroadcastExchange -Input [1]: [ca_address_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#16] -Right keys [1]: [ca_address_sk#17] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 7] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#5] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#16, ca_address_sk#17] - -(34) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_category#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [i_item_sk#19, i_category#20] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#20, 50, true, false, true) = Jewelry ) AND isnotnull(i_item_sk#19)) - -(36) CometProject -Input [2]: [i_item_sk#19, i_category#20] -Arguments: [i_item_sk#19], [i_item_sk#19] - -(37) CometColumnarToRow [codegen id : 6] -Input [1]: [i_item_sk#19] - -(38) BroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#19] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 7] -Output [1]: [ss_ext_sales_price#5] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#19] - -(41) HashAggregate [codegen id : 7] -Input [1]: [ss_ext_sales_price#5] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum#21] -Results [1]: [sum#22] - -(42) CometColumnarExchange -Input [1]: [sum#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) CometColumnarToRow [codegen id : 15] -Input [1]: [sum#22] - -(44) HashAggregate [codegen id : 15] -Input [1]: [sum#22] -Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#23] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#23,17,2) AS promotions#24] - -(45) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 13] -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] - -(47) Filter [codegen id : 13] -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_store_sk#27) AND isnotnull(ss_customer_sk#26)) AND isnotnull(ss_item_sk#25)) - -(48) ReusedExchange [Reuses operator id: 8] -Output [1]: [s_store_sk#30] - -(49) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#27] -Right keys [1]: [s_store_sk#30] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 13] -Output [4]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] -Input [6]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, s_store_sk#30] - -(51) ReusedExchange [Reuses operator id: 74] -Output [1]: [d_date_sk#31] - -(52) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_date_sk#29] -Right keys [1]: [d_date_sk#31] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 13] -Output [3]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#31] - -(54) ReusedExchange [Reuses operator id: 24] -Output [2]: [c_customer_sk#32, c_current_addr_sk#33] - -(55) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#32] -Join type: Inner -Join condition: None - -(56) Project [codegen id : 13] -Output [3]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#33] -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, c_customer_sk#32, c_current_addr_sk#33] - -(57) ReusedExchange [Reuses operator id: 31] -Output [1]: [ca_address_sk#34] - -(58) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [c_current_addr_sk#33] -Right keys [1]: [ca_address_sk#34] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 13] -Output [2]: [ss_item_sk#25, ss_ext_sales_price#28] -Input [4]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#33, ca_address_sk#34] - -(60) ReusedExchange [Reuses operator id: 38] -Output [1]: [i_item_sk#35] - -(61) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_item_sk#25] -Right keys [1]: [i_item_sk#35] -Join type: Inner -Join condition: None - -(62) Project [codegen id : 13] -Output [1]: [ss_ext_sales_price#28] -Input [3]: [ss_item_sk#25, ss_ext_sales_price#28, i_item_sk#35] - -(63) HashAggregate [codegen id : 13] -Input [1]: [ss_ext_sales_price#28] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#28))] -Aggregate Attributes [1]: [sum#36] -Results [1]: [sum#37] - -(64) CometColumnarExchange -Input [1]: [sum#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(65) CometColumnarToRow [codegen id : 14] -Input [1]: [sum#37] - -(66) HashAggregate [codegen id : 14] -Input [1]: [sum#37] -Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#28))#38] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#28))#38,17,2) AS total#39] - -(67) BroadcastExchange -Input [1]: [total#39] -Arguments: IdentityBroadcastMode, [plan_id=8] - -(68) BroadcastNestedLoopJoin [codegen id : 15] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 15] -Output [3]: [promotions#24, total#39, ((cast(promotions#24 as decimal(15,4)) / cast(total#39 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] -Input [2]: [promotions#24, total#39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (74) -+- * CometColumnarToRow (73) - +- CometProject (72) - +- CometFilter (71) - +- CometNativeScan parquet spark_catalog.default.date_dim (70) - - -(70) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#14, d_year#41, d_moy#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(71) CometFilter -Input [3]: [d_date_sk#14, d_year#41, d_moy#42] -Condition : ((((isnotnull(d_year#41) AND isnotnull(d_moy#42)) AND (d_year#41 = 1998)) AND (d_moy#42 = 11)) AND isnotnull(d_date_sk#14)) - -(72) CometProject -Input [3]: [d_date_sk#14, d_year#41, d_moy#42] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(73) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(74) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 45 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/simplified.txt deleted file mode 100644 index f5bacac9c6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/simplified.txt +++ /dev/null @@ -1,108 +0,0 @@ -WholeStageCodegen (15) - Project [promotions,total] - BroadcastNestedLoopJoin - HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (7) - HashAggregate [ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_gmt_offset] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (14) - HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (13) - HashAggregate [ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk,ss_customer_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [s_store_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [c_customer_sk,c_current_addr_sk] #5 - InputAdapter - ReusedExchange [ca_address_sk] #6 - InputAdapter - ReusedExchange [i_item_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/explain.txt deleted file mode 100644 index 4f090653c8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,397 +0,0 @@ -== Physical Plan == -* Project (65) -+- * BroadcastNestedLoopJoin Inner BuildRight (64) - :- * CometColumnarToRow (41) - : +- CometHashAggregate (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometBroadcastHashJoin (19) - : : : : :- CometProject (14) - : : : : : +- CometBroadcastHashJoin (13) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (6) - : : : : : : +- CometProject (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (3) - : : : : : +- CometBroadcastExchange (12) - : : : : : +- CometProject (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (9) - : : : : +- CometBroadcastExchange (18) - : : : : +- CometProject (17) - : : : : +- CometFilter (16) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (15) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (21) - : : +- CometBroadcastExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (26) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) - +- BroadcastExchange (63) - +- * CometColumnarToRow (62) - +- CometHashAggregate (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometProject (55) - : +- CometBroadcastHashJoin (54) - : :- CometProject (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometProject (49) - : : : +- CometBroadcastHashJoin (48) - : : : :- CometProject (46) - : : : : +- CometBroadcastHashJoin (45) - : : : : :- CometFilter (43) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (42) - : : : : +- ReusedExchange (44) - : : : +- ReusedExchange (47) - : : +- ReusedExchange (50) - : +- ReusedExchange (53) - +- ReusedExchange (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_store_sk#3) AND isnotnull(ss_promo_sk#4)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_gmt_offset#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [s_store_sk#8, s_gmt_offset#9] -Condition : ((isnotnull(s_gmt_offset#9) AND (s_gmt_offset#9 = -5.00)) AND isnotnull(s_store_sk#8)) - -(5) CometProject -Input [2]: [s_store_sk#8, s_gmt_offset#9] -Arguments: [s_store_sk#8], [s_store_sk#8] - -(6) CometBroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: [s_store_sk#8] - -(7) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [s_store_sk#8] -Arguments: [ss_store_sk#3], [s_store_sk#8], Inner, BuildRight - -(8) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, s_store_sk#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6], [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_dmail#11, 1, true, false, true) = Y) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_email#12, 1, true, false, true) = Y)) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#13, 1, true, false, true) = Y)) AND isnotnull(p_promo_sk#10)) - -(11) CometProject -Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Arguments: [p_promo_sk#10], [p_promo_sk#10] - -(12) CometBroadcastExchange -Input [1]: [p_promo_sk#10] -Arguments: [p_promo_sk#10] - -(13) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [p_promo_sk#10] -Arguments: [ss_promo_sk#4], [p_promo_sk#10], Inner, BuildRight - -(14) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, p_promo_sk#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6], [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#14, d_year#15, d_moy#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1998)) AND (d_moy#16 = 11)) AND isnotnull(d_date_sk#14)) - -(17) CometProject -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(18) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(19) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#6], [d_date_sk#14], Inner, BuildRight - -(20) CometProject -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5], [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(c_current_addr_sk#18)) - -(23) CometBroadcastExchange -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [c_customer_sk#17, c_current_addr_sk#18] - -(24) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] -Right output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_customer_sk#2], [c_customer_sk#17], Inner, BuildRight - -(25) CometProject -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18], [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_gmt_offset#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [ca_address_sk#19, ca_gmt_offset#20] -Condition : ((isnotnull(ca_gmt_offset#20) AND (ca_gmt_offset#20 = -5.00)) AND isnotnull(ca_address_sk#19)) - -(28) CometProject -Input [2]: [ca_address_sk#19, ca_gmt_offset#20] -Arguments: [ca_address_sk#19], [ca_address_sk#19] - -(29) CometBroadcastExchange -Input [1]: [ca_address_sk#19] -Arguments: [ca_address_sk#19] - -(30) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18] -Right output [1]: [ca_address_sk#19] -Arguments: [c_current_addr_sk#18], [ca_address_sk#19], Inner, BuildRight - -(31) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18, ca_address_sk#19] -Arguments: [ss_item_sk#1, ss_ext_sales_price#5], [ss_item_sk#1, ss_ext_sales_price#5] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#21, i_category#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [i_item_sk#21, i_category#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#22, 50, true, false, true) = Jewelry ) AND isnotnull(i_item_sk#21)) - -(34) CometProject -Input [2]: [i_item_sk#21, i_category#22] -Arguments: [i_item_sk#21], [i_item_sk#21] - -(35) CometBroadcastExchange -Input [1]: [i_item_sk#21] -Arguments: [i_item_sk#21] - -(36) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#5] -Right output [1]: [i_item_sk#21] -Arguments: [ss_item_sk#1], [i_item_sk#21], Inner, BuildRight - -(37) CometProject -Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#21] -Arguments: [ss_ext_sales_price#5], [ss_ext_sales_price#5] - -(38) CometHashAggregate -Input [1]: [ss_ext_sales_price#5] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(39) CometExchange -Input [1]: [sum#23] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(40) CometHashAggregate -Input [1]: [sum#23] -Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(41) CometColumnarToRow [codegen id : 2] -Input [1]: [promotions#24] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(43) CometFilter -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_store_sk#27) AND isnotnull(ss_customer_sk#26)) AND isnotnull(ss_item_sk#25)) - -(44) ReusedExchange [Reuses operator id: 6] -Output [1]: [s_store_sk#31] - -(45) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Right output [1]: [s_store_sk#31] -Arguments: [ss_store_sk#27], [s_store_sk#31], Inner, BuildRight - -(46) CometProject -Input [6]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, s_store_sk#31] -Arguments: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29], [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] - -(47) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#32] - -(48) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] -Right output [1]: [d_date_sk#32] -Arguments: [ss_sold_date_sk#29], [d_date_sk#32], Inner, BuildRight - -(49) CometProject -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#32] -Arguments: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28], [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] - -(50) ReusedExchange [Reuses operator id: 23] -Output [2]: [c_customer_sk#33, c_current_addr_sk#34] - -(51) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] -Right output [2]: [c_customer_sk#33, c_current_addr_sk#34] -Arguments: [ss_customer_sk#26], [c_customer_sk#33], Inner, BuildRight - -(52) CometProject -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, c_customer_sk#33, c_current_addr_sk#34] -Arguments: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34], [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] - -(53) ReusedExchange [Reuses operator id: 29] -Output [1]: [ca_address_sk#35] - -(54) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] -Right output [1]: [ca_address_sk#35] -Arguments: [c_current_addr_sk#34], [ca_address_sk#35], Inner, BuildRight - -(55) CometProject -Input [4]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34, ca_address_sk#35] -Arguments: [ss_item_sk#25, ss_ext_sales_price#28], [ss_item_sk#25, ss_ext_sales_price#28] - -(56) ReusedExchange [Reuses operator id: 35] -Output [1]: [i_item_sk#36] - -(57) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#25, ss_ext_sales_price#28] -Right output [1]: [i_item_sk#36] -Arguments: [ss_item_sk#25], [i_item_sk#36], Inner, BuildRight - -(58) CometProject -Input [3]: [ss_item_sk#25, ss_ext_sales_price#28, i_item_sk#36] -Arguments: [ss_ext_sales_price#28], [ss_ext_sales_price#28] - -(59) CometHashAggregate -Input [1]: [ss_ext_sales_price#28] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#28))] - -(60) CometExchange -Input [1]: [sum#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(61) CometHashAggregate -Input [1]: [sum#37] -Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] - -(62) CometColumnarToRow [codegen id : 1] -Input [1]: [total#38] - -(63) BroadcastExchange -Input [1]: [total#38] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(64) BroadcastNestedLoopJoin [codegen id : 2] -Join type: Inner -Join condition: None - -(65) Project [codegen id : 2] -Output [3]: [promotions#24, total#38, ((cast(promotions#24 as decimal(15,4)) / cast(total#38 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#39] -Input [2]: [promotions#24, total#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (70) -+- * CometColumnarToRow (69) - +- CometProject (68) - +- CometFilter (67) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) - - -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#14, d_year#15, d_moy#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(67) CometFilter -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1998)) AND (d_moy#16 = 11)) AND isnotnull(d_date_sk#14)) - -(68) CometProject -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(69) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(70) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 42 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/simplified.txt deleted file mode 100644 index 6ebe25b753..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,79 +0,0 @@ -WholeStageCodegen (2) - Project [promotions,total] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [promotions,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange #1 - CometHashAggregate [ss_ext_sales_price] [sum] - CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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_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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk] #3 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - CometBroadcastExchange [p_promo_sk] #4 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk] #7 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_category] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [total,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange #10 - CometHashAggregate [ss_ext_sales_price] [sum] - CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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_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_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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #5 - ReusedExchange [c_customer_sk,c_current_addr_sk] #6 - ReusedExchange [ca_address_sk] #7 - ReusedExchange [i_item_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/explain.txt deleted file mode 100644 index 4f090653c8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/explain.txt +++ /dev/null @@ -1,397 +0,0 @@ -== Physical Plan == -* Project (65) -+- * BroadcastNestedLoopJoin Inner BuildRight (64) - :- * CometColumnarToRow (41) - : +- CometHashAggregate (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometBroadcastHashJoin (19) - : : : : :- CometProject (14) - : : : : : +- CometBroadcastHashJoin (13) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (6) - : : : : : : +- CometProject (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (3) - : : : : : +- CometBroadcastExchange (12) - : : : : : +- CometProject (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (9) - : : : : +- CometBroadcastExchange (18) - : : : : +- CometProject (17) - : : : : +- CometFilter (16) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (15) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (21) - : : +- CometBroadcastExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (26) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) - +- BroadcastExchange (63) - +- * CometColumnarToRow (62) - +- CometHashAggregate (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometProject (55) - : +- CometBroadcastHashJoin (54) - : :- CometProject (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometProject (49) - : : : +- CometBroadcastHashJoin (48) - : : : :- CometProject (46) - : : : : +- CometBroadcastHashJoin (45) - : : : : :- CometFilter (43) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (42) - : : : : +- ReusedExchange (44) - : : : +- ReusedExchange (47) - : : +- ReusedExchange (50) - : +- ReusedExchange (53) - +- ReusedExchange (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_store_sk#3) AND isnotnull(ss_promo_sk#4)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_gmt_offset#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [s_store_sk#8, s_gmt_offset#9] -Condition : ((isnotnull(s_gmt_offset#9) AND (s_gmt_offset#9 = -5.00)) AND isnotnull(s_store_sk#8)) - -(5) CometProject -Input [2]: [s_store_sk#8, s_gmt_offset#9] -Arguments: [s_store_sk#8], [s_store_sk#8] - -(6) CometBroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: [s_store_sk#8] - -(7) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [s_store_sk#8] -Arguments: [ss_store_sk#3], [s_store_sk#8], Inner, BuildRight - -(8) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, s_store_sk#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6], [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_dmail#11, 1, true, false, true) = Y) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_email#12, 1, true, false, true) = Y)) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#13, 1, true, false, true) = Y)) AND isnotnull(p_promo_sk#10)) - -(11) CometProject -Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Arguments: [p_promo_sk#10], [p_promo_sk#10] - -(12) CometBroadcastExchange -Input [1]: [p_promo_sk#10] -Arguments: [p_promo_sk#10] - -(13) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [p_promo_sk#10] -Arguments: [ss_promo_sk#4], [p_promo_sk#10], Inner, BuildRight - -(14) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, p_promo_sk#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6], [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#14, d_year#15, d_moy#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1998)) AND (d_moy#16 = 11)) AND isnotnull(d_date_sk#14)) - -(17) CometProject -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(18) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(19) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#6], [d_date_sk#14], Inner, BuildRight - -(20) CometProject -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5], [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(c_current_addr_sk#18)) - -(23) CometBroadcastExchange -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [c_customer_sk#17, c_current_addr_sk#18] - -(24) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] -Right output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_customer_sk#2], [c_customer_sk#17], Inner, BuildRight - -(25) CometProject -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18], [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_gmt_offset#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [ca_address_sk#19, ca_gmt_offset#20] -Condition : ((isnotnull(ca_gmt_offset#20) AND (ca_gmt_offset#20 = -5.00)) AND isnotnull(ca_address_sk#19)) - -(28) CometProject -Input [2]: [ca_address_sk#19, ca_gmt_offset#20] -Arguments: [ca_address_sk#19], [ca_address_sk#19] - -(29) CometBroadcastExchange -Input [1]: [ca_address_sk#19] -Arguments: [ca_address_sk#19] - -(30) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18] -Right output [1]: [ca_address_sk#19] -Arguments: [c_current_addr_sk#18], [ca_address_sk#19], Inner, BuildRight - -(31) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18, ca_address_sk#19] -Arguments: [ss_item_sk#1, ss_ext_sales_price#5], [ss_item_sk#1, ss_ext_sales_price#5] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#21, i_category#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [i_item_sk#21, i_category#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#22, 50, true, false, true) = Jewelry ) AND isnotnull(i_item_sk#21)) - -(34) CometProject -Input [2]: [i_item_sk#21, i_category#22] -Arguments: [i_item_sk#21], [i_item_sk#21] - -(35) CometBroadcastExchange -Input [1]: [i_item_sk#21] -Arguments: [i_item_sk#21] - -(36) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#5] -Right output [1]: [i_item_sk#21] -Arguments: [ss_item_sk#1], [i_item_sk#21], Inner, BuildRight - -(37) CometProject -Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#21] -Arguments: [ss_ext_sales_price#5], [ss_ext_sales_price#5] - -(38) CometHashAggregate -Input [1]: [ss_ext_sales_price#5] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(39) CometExchange -Input [1]: [sum#23] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(40) CometHashAggregate -Input [1]: [sum#23] -Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(41) CometColumnarToRow [codegen id : 2] -Input [1]: [promotions#24] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(43) CometFilter -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_store_sk#27) AND isnotnull(ss_customer_sk#26)) AND isnotnull(ss_item_sk#25)) - -(44) ReusedExchange [Reuses operator id: 6] -Output [1]: [s_store_sk#31] - -(45) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Right output [1]: [s_store_sk#31] -Arguments: [ss_store_sk#27], [s_store_sk#31], Inner, BuildRight - -(46) CometProject -Input [6]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, s_store_sk#31] -Arguments: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29], [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] - -(47) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#32] - -(48) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] -Right output [1]: [d_date_sk#32] -Arguments: [ss_sold_date_sk#29], [d_date_sk#32], Inner, BuildRight - -(49) CometProject -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#32] -Arguments: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28], [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] - -(50) ReusedExchange [Reuses operator id: 23] -Output [2]: [c_customer_sk#33, c_current_addr_sk#34] - -(51) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] -Right output [2]: [c_customer_sk#33, c_current_addr_sk#34] -Arguments: [ss_customer_sk#26], [c_customer_sk#33], Inner, BuildRight - -(52) CometProject -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, c_customer_sk#33, c_current_addr_sk#34] -Arguments: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34], [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] - -(53) ReusedExchange [Reuses operator id: 29] -Output [1]: [ca_address_sk#35] - -(54) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] -Right output [1]: [ca_address_sk#35] -Arguments: [c_current_addr_sk#34], [ca_address_sk#35], Inner, BuildRight - -(55) CometProject -Input [4]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34, ca_address_sk#35] -Arguments: [ss_item_sk#25, ss_ext_sales_price#28], [ss_item_sk#25, ss_ext_sales_price#28] - -(56) ReusedExchange [Reuses operator id: 35] -Output [1]: [i_item_sk#36] - -(57) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#25, ss_ext_sales_price#28] -Right output [1]: [i_item_sk#36] -Arguments: [ss_item_sk#25], [i_item_sk#36], Inner, BuildRight - -(58) CometProject -Input [3]: [ss_item_sk#25, ss_ext_sales_price#28, i_item_sk#36] -Arguments: [ss_ext_sales_price#28], [ss_ext_sales_price#28] - -(59) CometHashAggregate -Input [1]: [ss_ext_sales_price#28] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#28))] - -(60) CometExchange -Input [1]: [sum#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(61) CometHashAggregate -Input [1]: [sum#37] -Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] - -(62) CometColumnarToRow [codegen id : 1] -Input [1]: [total#38] - -(63) BroadcastExchange -Input [1]: [total#38] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(64) BroadcastNestedLoopJoin [codegen id : 2] -Join type: Inner -Join condition: None - -(65) Project [codegen id : 2] -Output [3]: [promotions#24, total#38, ((cast(promotions#24 as decimal(15,4)) / cast(total#38 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#39] -Input [2]: [promotions#24, total#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (70) -+- * CometColumnarToRow (69) - +- CometProject (68) - +- CometFilter (67) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) - - -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#14, d_year#15, d_moy#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(67) CometFilter -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1998)) AND (d_moy#16 = 11)) AND isnotnull(d_date_sk#14)) - -(68) CometProject -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(69) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(70) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 42 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/extended.txt deleted file mode 100644 index 7bae7d5724..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/extended.txt +++ /dev/null @@ -1,88 +0,0 @@ -Project -+- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 77 out of 83 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 6ebe25b753..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/simplified.txt +++ /dev/null @@ -1,79 +0,0 @@ -WholeStageCodegen (2) - Project [promotions,total] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [promotions,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange #1 - CometHashAggregate [ss_ext_sales_price] [sum] - CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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_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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk] #3 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - CometBroadcastExchange [p_promo_sk] #4 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk] #7 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_category] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [total,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange #10 - CometHashAggregate [ss_ext_sales_price] [sum] - CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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_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_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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #5 - ReusedExchange [c_customer_sk,c_current_addr_sk] #6 - ReusedExchange [ca_address_sk] #7 - ReusedExchange [i_item_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/explain.txt deleted file mode 100644 index 1d9748055e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometNativeScan parquet spark_catalog.default.warehouse (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometNativeScan parquet spark_catalog.default.ship_mode (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.web_site (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometNativeScan parquet spark_catalog.default.date_dim (19) - - -(1) CometNativeScan parquet spark_catalog.default.web_sales -Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_ship_mode_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Condition : (((isnotnull(ws_warehouse_sk#4) AND isnotnull(ws_ship_mode_sk#3)) AND isnotnull(ws_web_site_sk#2)) AND isnotnull(ws_ship_date_sk#1)) - -(3) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [ws_warehouse_sk#4], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7], [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] - -(8) CometNativeScan parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#8, sm_type#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Condition : isnotnull(sm_ship_mode_sk#8) - -(10) CometProject -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Arguments: [sm_ship_mode_sk#8, sm_type#10], [sm_ship_mode_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, sm_type#9, 30, true, false, true) AS sm_type#10] - -(11) CometBroadcastExchange -Input [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [sm_ship_mode_sk#8, sm_type#10] - -(12) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] -Right output [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [ws_ship_mode_sk#3], [sm_ship_mode_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#10] -Arguments: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10], [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10] - -(14) CometNativeScan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#11, web_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [web_site_sk#11, web_name#12] -Condition : isnotnull(web_site_sk#11) - -(16) CometBroadcastExchange -Input [2]: [web_site_sk#11, web_name#12] -Arguments: [web_site_sk#11, web_name#12] - -(17) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10] -Right output [2]: [web_site_sk#11, web_name#12] -Arguments: [ws_web_site_sk#2], [web_site_sk#11], Inner, BuildRight - -(18) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_site_sk#11, web_name#12] -Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12], [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12] - -(19) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_month_seq#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#13, d_month_seq#14] -Condition : (((isnotnull(d_month_seq#14) AND (d_month_seq#14 >= 1200)) AND (d_month_seq#14 <= 1211)) AND isnotnull(d_date_sk#13)) - -(21) CometProject -Input [2]: [d_date_sk#13, d_month_seq#14] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(23) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12] -Right output [1]: [d_date_sk#13] -Arguments: [ws_ship_date_sk#1], [d_date_sk#13], Inner, BuildRight - -(24) CometProject -Input [6]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12, d_date_sk#13] -Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, _groupingexpression#15], [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#15] - -(25) CometHashAggregate -Input [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, sm_type#10, web_name#12] -Functions [5]: [partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [8]: [_groupingexpression#15, sm_type#10, web_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Arguments: hashpartitioning(_groupingexpression#15, sm_type#10, web_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [8]: [_groupingexpression#15, sm_type#10, web_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Keys [3]: [_groupingexpression#15, sm_type#10, web_name#12] -Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,web_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,web_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, web_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - -(29) CometColumnarToRow [codegen id : 1] -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/simplified.txt deleted file mode 100644 index de80c17349..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_datafusion/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 30) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 60) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 90) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [_groupingexpression,sm_type,web_name] #1 - CometHashAggregate [ws_ship_date_sk,ws_sold_date_sk] [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum] - CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] - 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_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_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_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] - CometNativeScan 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 [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [web_site_sk,web_name] #4 - CometFilter [web_site_sk,web_name] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan 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/q62.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_iceberg_compat/explain.txt deleted file mode 100644 index a006a92dd0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_ship_mode_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Condition : (((isnotnull(ws_warehouse_sk#4) AND isnotnull(ws_ship_mode_sk#3)) AND isnotnull(ws_web_site_sk#2)) AND isnotnull(ws_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [ws_warehouse_sk#4], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7], [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#8, sm_type#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Condition : isnotnull(sm_ship_mode_sk#8) - -(10) CometProject -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Arguments: [sm_ship_mode_sk#8, sm_type#10], [sm_ship_mode_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, sm_type#9, 30, true, false, true) AS sm_type#10] - -(11) CometBroadcastExchange -Input [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [sm_ship_mode_sk#8, sm_type#10] - -(12) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] -Right output [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [ws_ship_mode_sk#3], [sm_ship_mode_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#10] -Arguments: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10], [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#11, web_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [web_site_sk#11, web_name#12] -Condition : isnotnull(web_site_sk#11) - -(16) CometBroadcastExchange -Input [2]: [web_site_sk#11, web_name#12] -Arguments: [web_site_sk#11, web_name#12] - -(17) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10] -Right output [2]: [web_site_sk#11, web_name#12] -Arguments: [ws_web_site_sk#2], [web_site_sk#11], Inner, BuildRight - -(18) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_site_sk#11, web_name#12] -Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12], [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_month_seq#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#13, d_month_seq#14] -Condition : (((isnotnull(d_month_seq#14) AND (d_month_seq#14 >= 1200)) AND (d_month_seq#14 <= 1211)) AND isnotnull(d_date_sk#13)) - -(21) CometProject -Input [2]: [d_date_sk#13, d_month_seq#14] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(23) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12] -Right output [1]: [d_date_sk#13] -Arguments: [ws_ship_date_sk#1], [d_date_sk#13], Inner, BuildRight - -(24) CometProject -Input [6]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12, d_date_sk#13] -Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, _groupingexpression#15], [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#15] - -(25) CometHashAggregate -Input [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, sm_type#10, web_name#12] -Functions [5]: [partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [8]: [_groupingexpression#15, sm_type#10, web_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Arguments: hashpartitioning(_groupingexpression#15, sm_type#10, web_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [8]: [_groupingexpression#15, sm_type#10, web_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Keys [3]: [_groupingexpression#15, sm_type#10, web_name#12] -Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,web_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,web_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, web_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - -(29) CometColumnarToRow [codegen id : 1] -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_iceberg_compat/simplified.txt deleted file mode 100644 index dee2e434c7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 30) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 60) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 90) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [_groupingexpression,sm_type,web_name] #1 - CometHashAggregate [ws_ship_date_sk,ws_sold_date_sk] [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum] - CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] - 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_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_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_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 [native_iceberg_compat] 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 [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [web_site_sk,web_name] #4 - CometFilter [web_site_sk,web_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] 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/q62/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/explain.txt deleted file mode 100644 index a006a92dd0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_ship_mode_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Condition : (((isnotnull(ws_warehouse_sk#4) AND isnotnull(ws_ship_mode_sk#3)) AND isnotnull(ws_web_site_sk#2)) AND isnotnull(ws_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [ws_warehouse_sk#4], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7], [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#8, sm_type#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Condition : isnotnull(sm_ship_mode_sk#8) - -(10) CometProject -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Arguments: [sm_ship_mode_sk#8, sm_type#10], [sm_ship_mode_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, sm_type#9, 30, true, false, true) AS sm_type#10] - -(11) CometBroadcastExchange -Input [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [sm_ship_mode_sk#8, sm_type#10] - -(12) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] -Right output [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [ws_ship_mode_sk#3], [sm_ship_mode_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#10] -Arguments: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10], [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#11, web_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [web_site_sk#11, web_name#12] -Condition : isnotnull(web_site_sk#11) - -(16) CometBroadcastExchange -Input [2]: [web_site_sk#11, web_name#12] -Arguments: [web_site_sk#11, web_name#12] - -(17) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10] -Right output [2]: [web_site_sk#11, web_name#12] -Arguments: [ws_web_site_sk#2], [web_site_sk#11], Inner, BuildRight - -(18) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_site_sk#11, web_name#12] -Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12], [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_month_seq#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#13, d_month_seq#14] -Condition : (((isnotnull(d_month_seq#14) AND (d_month_seq#14 >= 1200)) AND (d_month_seq#14 <= 1211)) AND isnotnull(d_date_sk#13)) - -(21) CometProject -Input [2]: [d_date_sk#13, d_month_seq#14] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(23) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12] -Right output [1]: [d_date_sk#13] -Arguments: [ws_ship_date_sk#1], [d_date_sk#13], Inner, BuildRight - -(24) CometProject -Input [6]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12, d_date_sk#13] -Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, _groupingexpression#15], [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#15] - -(25) CometHashAggregate -Input [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, sm_type#10, web_name#12] -Functions [5]: [partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [8]: [_groupingexpression#15, sm_type#10, web_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Arguments: hashpartitioning(_groupingexpression#15, sm_type#10, web_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [8]: [_groupingexpression#15, sm_type#10, web_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Keys [3]: [_groupingexpression#15, sm_type#10, web_name#12] -Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,web_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,web_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, web_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - -(29) CometColumnarToRow [codegen id : 1] -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/extended.txt deleted file mode 100644 index a5fd7359a3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 28 out of 28 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index dee2e434c7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q62/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 30) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 60) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 90) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [_groupingexpression,sm_type,web_name] #1 - CometHashAggregate [ws_ship_date_sk,ws_sold_date_sk] [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum] - CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] - 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_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_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_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 [native_iceberg_compat] 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 [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [web_site_sk,web_name] #4 - CometFilter [web_site_sk,web_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/explain.txt deleted file mode 100644 index 32f70c3d33..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/explain.txt +++ /dev/null @@ -1,202 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * CometColumnarToRow (26) - +- CometSort (25) - +- CometColumnarExchange (24) - +- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometColumnarExchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * CometColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (8) - : : +- * Filter (7) - : : +- * ColumnarToRow (6) - : : +- Scan parquet spark_catalog.default.store_sales (5) - : +- ReusedExchange (11) - +- BroadcastExchange (17) - +- * CometColumnarToRow (16) - +- CometFilter (15) - +- CometNativeScan parquet spark_catalog.default.store (14) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Books ,Children ,Electronics ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (personal ,portable ,refernece ,self-help )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Women ,Music ,Men ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (accessories ,classical ,fragrances ,pants )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Arguments: [i_item_sk#1, i_manager_id#5], [i_item_sk#1, i_manager_id#5] - -(4) CometColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#1, i_manager_id#5] - -(5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(7) Filter [codegen id : 1] -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) - -(8) BroadcastExchange -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#10] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Input [6]: [i_item_sk#1, i_manager_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(11) ReusedExchange [Reuses operator id: 35] -Output [2]: [d_date_sk#15, d_moy#16] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#13] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#16] -Input [6]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_moy#16] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [1]: [s_store_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [1]: [s_store_sk#17] -Condition : isnotnull(s_store_sk#17) - -(16) CometColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#17] - -(17) BroadcastExchange -Input [1]: [s_store_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#11] -Right keys [1]: [s_store_sk#17] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [3]: [i_manager_id#5, ss_sales_price#12, d_moy#16] -Input [5]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#16, s_store_sk#17] - -(20) HashAggregate [codegen id : 4] -Input [3]: [i_manager_id#5, ss_sales_price#12, d_moy#16] -Keys [2]: [i_manager_id#5, d_moy#16] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [i_manager_id#5, d_moy#16, sum#19] - -(21) CometColumnarExchange -Input [3]: [i_manager_id#5, d_moy#16, sum#19] -Arguments: hashpartitioning(i_manager_id#5, d_moy#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [3]: [i_manager_id#5, d_moy#16, sum#19] - -(23) HashAggregate [codegen id : 5] -Input [3]: [i_manager_id#5, d_moy#16, sum#19] -Keys [2]: [i_manager_id#5, d_moy#16] -Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#20] -Results [3]: [i_manager_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS _w0#22] - -(24) CometColumnarExchange -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: [i_manager_id#5, sum_sales#21, _w0#22], [i_manager_id#5 ASC NULLS FIRST] - -(26) CometColumnarToRow [codegen id : 6] -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] - -(27) Window -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_manager_id#5] - -(28) Filter [codegen id : 7] -Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] -Condition : CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) ELSE false END - -(29) Project [codegen id : 7] -Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] -Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] - -(30) TakeOrderedAndProject -Input [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] -Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometNativeScan parquet spark_catalog.default.date_dim (31) - - -(31) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] -Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(33) CometProject -Input [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] -Arguments: [d_date_sk#15, d_moy#16], [d_date_sk#15, d_moy#16] - -(34) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_moy#16] - -(35) BroadcastExchange -Input [2]: [d_date_sk#15, d_moy#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/simplified.txt deleted file mode 100644 index 2933149226..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/simplified.txt +++ /dev/null @@ -1,53 +0,0 @@ -TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (7) - Project [i_manager_id,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_manager_id] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_manager_id,sum_sales,_w0] - CometColumnarExchange [i_manager_id] #1 - WholeStageCodegen (5) - HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manager_id,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] - Project [i_manager_id,ss_sales_price,d_moy] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_sk,i_manager_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/explain.txt deleted file mode 100644 index e2a0cceb1a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,198 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) - +- CometSort (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Books ,Children ,Electronics ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (personal ,portable ,refernece ,self-help )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Women ,Music ,Men ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (accessories ,classical ,fragrances ,pants )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Arguments: [i_item_sk#1, i_manager_id#5], [i_item_sk#1, i_manager_id#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(7) CometBroadcastHashJoin -Left output [2]: [i_item_sk#1, i_manager_id#5] -Right output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_item_sk#1], [ss_item_sk#10], Inner, BuildRight - -(8) CometProject -Input [6]: [i_item_sk#1, i_manager_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13], [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(11) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] - -(12) CometBroadcastExchange -Input [2]: [d_date_sk#15, d_moy#17] -Arguments: [d_date_sk#15, d_moy#17] - -(13) CometBroadcastHashJoin -Left output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_moy#17] -Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(14) CometProject -Input [6]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_moy#17] -Arguments: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17], [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [s_store_sk#18] -Condition : isnotnull(s_store_sk#18) - -(17) CometBroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: [s_store_sk#18] - -(18) CometBroadcastHashJoin -Left output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17] -Right output [1]: [s_store_sk#18] -Arguments: [ss_store_sk#11], [s_store_sk#18], Inner, BuildRight - -(19) CometProject -Input [5]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17, s_store_sk#18] -Arguments: [i_manager_id#5, ss_sales_price#12, d_moy#17], [i_manager_id#5, ss_sales_price#12, d_moy#17] - -(20) CometHashAggregate -Input [3]: [i_manager_id#5, ss_sales_price#12, d_moy#17] -Keys [2]: [i_manager_id#5, d_moy#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] - -(21) CometExchange -Input [3]: [i_manager_id#5, d_moy#17, sum#19] -Arguments: hashpartitioning(i_manager_id#5, d_moy#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [3]: [i_manager_id#5, d_moy#17, sum#19] -Keys [2]: [i_manager_id#5, d_moy#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] - -(23) CometExchange -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: [i_manager_id#5, sum_sales#20, _w0#21], [i_manager_id#5 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 1] -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] - -(26) Window -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_manager_id#5] - -(27) Filter [codegen id : 2] -Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] -Condition : CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) ELSE false END - -(28) Project [codegen id : 2] -Output [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] -Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] - -(29) TakeOrderedAndProject -Input [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] -Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(32) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_moy#17] - -(34) BroadcastExchange -Input [2]: [d_date_sk#15, d_moy#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/simplified.txt deleted file mode 100644 index 02166879a9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (2) - Project [i_manager_id,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_manager_id] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_manager_id,sum_sales,_w0] - CometExchange [i_manager_id] #1 - CometHashAggregate [d_moy,sum] [i_manager_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_manager_id,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] - CometProject [i_manager_id,ss_sales_price,d_moy] - 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 [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,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_item_sk,i_brand,i_class,i_category,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/explain.txt deleted file mode 100644 index e2a0cceb1a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/explain.txt +++ /dev/null @@ -1,198 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) - +- CometSort (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Books ,Children ,Electronics ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (personal ,portable ,refernece ,self-help )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Women ,Music ,Men ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (accessories ,classical ,fragrances ,pants )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Arguments: [i_item_sk#1, i_manager_id#5], [i_item_sk#1, i_manager_id#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(7) CometBroadcastHashJoin -Left output [2]: [i_item_sk#1, i_manager_id#5] -Right output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_item_sk#1], [ss_item_sk#10], Inner, BuildRight - -(8) CometProject -Input [6]: [i_item_sk#1, i_manager_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13], [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(11) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] - -(12) CometBroadcastExchange -Input [2]: [d_date_sk#15, d_moy#17] -Arguments: [d_date_sk#15, d_moy#17] - -(13) CometBroadcastHashJoin -Left output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_moy#17] -Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(14) CometProject -Input [6]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_moy#17] -Arguments: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17], [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [s_store_sk#18] -Condition : isnotnull(s_store_sk#18) - -(17) CometBroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: [s_store_sk#18] - -(18) CometBroadcastHashJoin -Left output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17] -Right output [1]: [s_store_sk#18] -Arguments: [ss_store_sk#11], [s_store_sk#18], Inner, BuildRight - -(19) CometProject -Input [5]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17, s_store_sk#18] -Arguments: [i_manager_id#5, ss_sales_price#12, d_moy#17], [i_manager_id#5, ss_sales_price#12, d_moy#17] - -(20) CometHashAggregate -Input [3]: [i_manager_id#5, ss_sales_price#12, d_moy#17] -Keys [2]: [i_manager_id#5, d_moy#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] - -(21) CometExchange -Input [3]: [i_manager_id#5, d_moy#17, sum#19] -Arguments: hashpartitioning(i_manager_id#5, d_moy#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [3]: [i_manager_id#5, d_moy#17, sum#19] -Keys [2]: [i_manager_id#5, d_moy#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] - -(23) CometExchange -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: [i_manager_id#5, sum_sales#20, _w0#21], [i_manager_id#5 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 1] -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] - -(26) Window -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_manager_id#5] - -(27) Filter [codegen id : 2] -Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] -Condition : CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) ELSE false END - -(28) Project [codegen id : 2] -Output [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] -Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] - -(29) TakeOrderedAndProject -Input [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] -Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(32) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_moy#17] - -(34) BroadcastExchange -Input [2]: [d_date_sk#15, d_moy#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/extended.txt deleted file mode 100644 index 825b1ed81c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 02166879a9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (2) - Project [i_manager_id,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_manager_id] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_manager_id,sum_sales,_w0] - CometExchange [i_manager_id] #1 - CometHashAggregate [d_moy,sum] [i_manager_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_manager_id,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] - CometProject [i_manager_id,ss_sales_price,d_moy] - 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 [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,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_item_sk,i_brand,i_class,i_category,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/explain.txt deleted file mode 100644 index c021738ca6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/explain.txt +++ /dev/null @@ -1,1029 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (180) -+- CometSort (179) - +- CometExchange (178) - +- CometProject (177) - +- CometSortMergeJoin (176) - :- CometSort (105) - : +- CometExchange (104) - : +- CometHashAggregate (103) - : +- CometHashAggregate (102) - : +- CometProject (101) - : +- CometBroadcastHashJoin (100) - : :- CometProject (95) - : : +- CometBroadcastHashJoin (94) - : : :- CometProject (92) - : : : +- CometBroadcastHashJoin (91) - : : : :- CometProject (87) - : : : : +- CometBroadcastHashJoin (86) - : : : : :- CometProject (84) - : : : : : +- CometBroadcastHashJoin (83) - : : : : : :- CometProject (78) - : : : : : : +- CometBroadcastHashJoin (77) - : : : : : : :- CometProject (75) - : : : : : : : +- CometBroadcastHashJoin (74) - : : : : : : : :- CometProject (70) - : : : : : : : : +- CometBroadcastHashJoin (69) - : : : : : : : : :- CometProject (65) - : : : : : : : : : +- CometBroadcastHashJoin (64) - : : : : : : : : : :- CometProject (62) - : : : : : : : : : : +- CometBroadcastHashJoin (61) - : : : : : : : : : : :- CometProject (56) - : : : : : : : : : : : +- CometBroadcastHashJoin (55) - : : : : : : : : : : : :- CometProject (53) - : : : : : : : : : : : : +- CometBroadcastHashJoin (52) - : : : : : : : : : : : : :- CometProject (48) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (47) - : : : : : : : : : : : : : :- CometProject (43) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (42) - : : : : : : : : : : : : : : :- CometProject (37) - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (36) - : : : : : : : : : : : : : : : :- CometProject (32) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (31) - : : : : : : : : : : : : : : : : :- CometSort (12) - : : : : : : : : : : : : : : : : : +- CometColumnarExchange (11) - : : : : : : : : : : : : : : : : : +- * Project (10) - : : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : : : : : : : : : : : : : : :- BroadcastExchange (4) - : : : : : : : : : : : : : : : : : : +- * Filter (3) - : : : : : : : : : : : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (8) - : : : : : : : : : : : : : : : : : +- CometProject (7) - : : : : : : : : : : : : : : : : : +- CometFilter (6) - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (5) - : : : : : : : : : : : : : : : : +- CometSort (30) - : : : : : : : : : : : : : : : : +- CometProject (29) - : : : : : : : : : : : : : : : : +- CometFilter (28) - : : : : : : : : : : : : : : : : +- CometHashAggregate (27) - : : : : : : : : : : : : : : : : +- CometExchange (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometProject (24) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (23) - : : : : : : : : : : : : : : : : :- CometSort (17) - : : : : : : : : : : : : : : : : : +- CometExchange (16) - : : : : : : : : : : : : : : : : : +- CometProject (15) - : : : : : : : : : : : : : : : : : +- CometFilter (14) - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : : : : : : : : : : : +- CometSort (22) - : : : : : : : : : : : : : : : : +- CometExchange (21) - : : : : : : : : : : : : : : : : +- CometProject (20) - : : : : : : : : : : : : : : : : +- CometFilter (19) - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (18) - : : : : : : : : : : : : : : : +- CometBroadcastExchange (35) - : : : : : : : : : : : : : : : +- CometFilter (34) - : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (33) - : : : : : : : : : : : : : : +- CometBroadcastExchange (41) - : : : : : : : : : : : : : : +- CometProject (40) - : : : : : : : : : : : : : : +- CometFilter (39) - : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store (38) - : : : : : : : : : : : : : +- CometBroadcastExchange (46) - : : : : : : : : : : : : : +- CometFilter (45) - : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer (44) - : : : : : : : : : : : : +- CometBroadcastExchange (51) - : : : : : : : : : : : : +- CometFilter (50) - : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (49) - : : : : : : : : : : : +- ReusedExchange (54) - : : : : : : : : : : +- CometBroadcastExchange (60) - : : : : : : : : : : +- CometProject (59) - : : : : : : : : : : +- CometFilter (58) - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (57) - : : : : : : : : : +- ReusedExchange (63) - : : : : : : : : +- CometBroadcastExchange (68) - : : : : : : : : +- CometFilter (67) - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.promotion (66) - : : : : : : : +- CometBroadcastExchange (73) - : : : : : : : +- CometFilter (72) - : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (71) - : : : : : : +- ReusedExchange (76) - : : : : : +- CometBroadcastExchange (82) - : : : : : +- CometProject (81) - : : : : : +- CometFilter (80) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address (79) - : : : : +- ReusedExchange (85) - : : : +- CometBroadcastExchange (90) - : : : +- CometFilter (89) - : : : +- CometNativeScan parquet spark_catalog.default.income_band (88) - : : +- ReusedExchange (93) - : +- CometBroadcastExchange (99) - : +- CometProject (98) - : +- CometFilter (97) - : +- CometNativeScan parquet spark_catalog.default.item (96) - +- CometSort (175) - +- CometExchange (174) - +- CometHashAggregate (173) - +- CometHashAggregate (172) - +- CometProject (171) - +- CometBroadcastHashJoin (170) - :- CometProject (168) - : +- CometBroadcastHashJoin (167) - : :- CometProject (165) - : : +- CometBroadcastHashJoin (164) - : : :- CometProject (162) - : : : +- CometBroadcastHashJoin (161) - : : : :- CometProject (159) - : : : : +- CometBroadcastHashJoin (158) - : : : : :- CometProject (156) - : : : : : +- CometBroadcastHashJoin (155) - : : : : : :- CometProject (153) - : : : : : : +- CometBroadcastHashJoin (152) - : : : : : : :- CometProject (150) - : : : : : : : +- CometBroadcastHashJoin (149) - : : : : : : : :- CometProject (147) - : : : : : : : : +- CometBroadcastHashJoin (146) - : : : : : : : : :- CometProject (144) - : : : : : : : : : +- CometBroadcastHashJoin (143) - : : : : : : : : : :- CometProject (141) - : : : : : : : : : : +- CometBroadcastHashJoin (140) - : : : : : : : : : : :- CometProject (138) - : : : : : : : : : : : +- CometBroadcastHashJoin (137) - : : : : : : : : : : : :- CometProject (135) - : : : : : : : : : : : : +- CometBroadcastHashJoin (134) - : : : : : : : : : : : : :- CometProject (132) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (131) - : : : : : : : : : : : : : :- CometProject (129) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (128) - : : : : : : : : : : : : : : :- CometProject (124) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (123) - : : : : : : : : : : : : : : : :- CometSort (117) - : : : : : : : : : : : : : : : : +- CometColumnarExchange (116) - : : : : : : : : : : : : : : : : +- * Project (115) - : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (114) - : : : : : : : : : : : : : : : : :- BroadcastExchange (109) - : : : : : : : : : : : : : : : : : +- * Filter (108) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (107) - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (106) - : : : : : : : : : : : : : : : : +- * CometColumnarToRow (113) - : : : : : : : : : : : : : : : : +- CometProject (112) - : : : : : : : : : : : : : : : : +- CometFilter (111) - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (110) - : : : : : : : : : : : : : : : +- CometSort (122) - : : : : : : : : : : : : : : : +- CometProject (121) - : : : : : : : : : : : : : : : +- CometFilter (120) - : : : : : : : : : : : : : : : +- CometHashAggregate (119) - : : : : : : : : : : : : : : : +- ReusedExchange (118) - : : : : : : : : : : : : : : +- CometBroadcastExchange (127) - : : : : : : : : : : : : : : +- CometFilter (126) - : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (125) - : : : : : : : : : : : : : +- ReusedExchange (130) - : : : : : : : : : : : : +- ReusedExchange (133) - : : : : : : : : : : : +- ReusedExchange (136) - : : : : : : : : : : +- ReusedExchange (139) - : : : : : : : : : +- ReusedExchange (142) - : : : : : : : : +- ReusedExchange (145) - : : : : : : : +- ReusedExchange (148) - : : : : : : +- ReusedExchange (151) - : : : : : +- ReusedExchange (154) - : : : : +- ReusedExchange (157) - : : : +- ReusedExchange (160) - : : +- ReusedExchange (163) - : +- ReusedExchange (166) - +- ReusedExchange (169) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(3) Filter [codegen id : 1] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) - -(4) BroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=1] - -(5) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) - -(7) CometProject -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] - -(8) CometColumnarToRow -Input [2]: [sr_item_sk#14, sr_ticket_number#15] - -(9) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [ss_item_sk#1, ss_ticket_number#8] -Right keys [2]: [sr_item_sk#14, sr_ticket_number#15] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 2] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] - -(11) CometColumnarExchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(12) CometSort -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] - -(13) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) - -(15) CometProject -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] - -(16) CometExchange -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(17) CometSort -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] - -(18) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(19) CometFilter -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) - -(20) CometProject -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(21) CometExchange -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(22) CometSort -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] - -(23) CometSortMergeJoin -Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner - -(24) CometProject -Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(25) CometHashAggregate -Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Keys [1]: [cs_item_sk#17] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(26) CometExchange -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(27) CometHashAggregate -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Keys [1]: [cs_item_sk#17] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(28) CometFilter -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(29) CometProject -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Arguments: [cs_item_sk#17], [cs_item_sk#17] - -(30) CometSort -Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] - -(31) CometSortMergeJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [1]: [cs_item_sk#17] -Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner - -(32) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(33) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(35) CometBroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: [d_date_sk#32, d_year#33] - -(36) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight - -(37) CometProject -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] - -(38) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name)] -ReadSchema: struct - -(39) CometFilter -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true))) - -(40) CometProject -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37], [s_store_sk#34, s_store_name#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true) AS s_zip#37] - -(41) CometBroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37] - -(42) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight - -(43) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] - -(44) CometNativeScan parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(45) CometFilter -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Condition : (((((isnotnull(c_customer_sk#38) AND isnotnull(c_first_sales_date_sk#43)) AND isnotnull(c_first_shipto_date_sk#42)) AND isnotnull(c_current_cdemo_sk#39)) AND isnotnull(c_current_hdemo_sk#40)) AND isnotnull(c_current_addr_sk#41)) - -(46) CometBroadcastExchange -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(47) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -Right output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_customer_sk#2], [c_customer_sk#38], Inner, BuildRight - -(48) CometProject -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(49) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : isnotnull(d_date_sk#44) - -(51) CometBroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] - -(52) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [c_first_sales_date_sk#43], [d_date_sk#44], Inner, BuildRight - -(53) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43, d_date_sk#44, d_year#45] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] - -(54) ReusedExchange [Reuses operator id: 51] -Output [2]: [d_date_sk#46, d_year#47] - -(55) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [c_first_shipto_date_sk#42], [d_date_sk#46], Inner, BuildRight - -(56) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45, d_date_sk#46, d_year#47] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(57) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#48, cd_marital_status#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(58) CometFilter -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true))) - -(59) CometProject -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Arguments: [cd_demo_sk#48, cd_marital_status#50], [cd_demo_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true) AS cd_marital_status#50] - -(60) CometBroadcastExchange -Input [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [cd_demo_sk#48, cd_marital_status#50] - -(61) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_cdemo_sk#3], [cd_demo_sk#48], Inner, BuildRight - -(62) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] - -(63) ReusedExchange [Reuses operator id: 60] -Output [2]: [cd_demo_sk#51, cd_marital_status#52] - -(64) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -Right output [2]: [cd_demo_sk#51, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#39], [cd_demo_sk#51], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(65) CometProject -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50, cd_demo_sk#51, cd_marital_status#52] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(66) CometNativeScan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(67) CometFilter -Input [1]: [p_promo_sk#53] -Condition : isnotnull(p_promo_sk#53) - -(68) CometBroadcastExchange -Input [1]: [p_promo_sk#53] -Arguments: [p_promo_sk#53] - -(69) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [1]: [p_promo_sk#53] -Arguments: [ss_promo_sk#7], [p_promo_sk#53], Inner, BuildRight - -(70) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, p_promo_sk#53] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(71) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(72) CometFilter -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Condition : (isnotnull(hd_demo_sk#54) AND isnotnull(hd_income_band_sk#55)) - -(73) CometBroadcastExchange -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [hd_demo_sk#54, hd_income_band_sk#55] - -(74) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_hdemo_sk#4], [hd_demo_sk#54], Inner, BuildRight - -(75) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] - -(76) ReusedExchange [Reuses operator id: 73] -Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] - -(77) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -Right output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [c_current_hdemo_sk#40], [hd_demo_sk#56], Inner, BuildRight - -(78) CometProject -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] - -(79) CometNativeScan parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(80) CometFilter -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Condition : isnotnull(ca_address_sk#58) - -(81) CometProject -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ca_address_sk#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#59, 10, true, false, true) AS ca_street_number#63, ca_street_name#60, ca_city#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#62, 10, true, false, true) AS ca_zip#64] - -(82) CometBroadcastExchange -Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(83) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -Right output [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_addr_sk#5], [ca_address_sk#58], Inner, BuildRight - -(84) CometProject -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(85) ReusedExchange [Reuses operator id: 82] -Output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(86) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Right output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [c_current_addr_sk#41], [ca_address_sk#65], Inner, BuildRight - -(87) CometProject -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(88) CometNativeScan parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(89) CometFilter -Input [1]: [ib_income_band_sk#70] -Condition : isnotnull(ib_income_band_sk#70) - -(90) CometBroadcastExchange -Input [1]: [ib_income_band_sk#70] -Arguments: [ib_income_band_sk#70] - -(91) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#70] -Arguments: [hd_income_band_sk#55], [ib_income_band_sk#70], Inner, BuildRight - -(92) CometProject -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#70] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(93) ReusedExchange [Reuses operator id: 90] -Output [1]: [ib_income_band_sk#71] - -(94) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#71] -Arguments: [hd_income_band_sk#57], [ib_income_band_sk#71], Inner, BuildRight - -(95) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#71] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(96) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(97) CometFilter -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Condition : ((((((isnotnull(i_current_price#73) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#74, 20, true, false, true) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#73 >= 64.00)) AND (i_current_price#73 <= 74.00)) AND (i_current_price#73 >= 65.00)) AND (i_current_price#73 <= 79.00)) AND isnotnull(i_item_sk#72)) - -(98) CometProject -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Arguments: [i_item_sk#72, i_product_name#76], [i_item_sk#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#75, 50, true, false, true) AS i_product_name#76] - -(99) CometBroadcastExchange -Input [2]: [i_item_sk#72, i_product_name#76] -Arguments: [i_item_sk#72, i_product_name#76] - -(100) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [2]: [i_item_sk#72, i_product_name#76] -Arguments: [ss_item_sk#1], [i_item_sk#72], Inner, BuildRight - -(101) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] - -(102) CometHashAggregate -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] - -(103) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#77, sum#78, sum#79, sum#80] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] - -(104) CometExchange -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: hashpartitioning(item_sk#82, store_name#83, store_zip#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(105) CometSort -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97], [item_sk#82 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, store_zip#84 ASC NULLS FIRST] - -(106) Scan parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#109), dynamicpruningexpression(ss_sold_date_sk#109 IN dynamicpruning#110)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(107) ColumnarToRow [codegen id : 3] -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(108) Filter [codegen id : 3] -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Condition : (((((((isnotnull(ss_item_sk#98) AND isnotnull(ss_ticket_number#105)) AND isnotnull(ss_store_sk#103)) AND isnotnull(ss_customer_sk#99)) AND isnotnull(ss_cdemo_sk#100)) AND isnotnull(ss_promo_sk#104)) AND isnotnull(ss_hdemo_sk#101)) AND isnotnull(ss_addr_sk#102)) - -(109) BroadcastExchange -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=7] - -(110) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(111) CometFilter -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) - -(112) CometProject -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] - -(113) CometColumnarToRow -Input [2]: [sr_item_sk#111, sr_ticket_number#112] - -(114) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [ss_item_sk#98, ss_ticket_number#105] -Right keys [2]: [sr_item_sk#111, sr_ticket_number#112] -Join type: Inner -Join condition: None - -(115) Project [codegen id : 4] -Output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, sr_item_sk#111, sr_ticket_number#112] - -(116) CometColumnarExchange -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: hashpartitioning(ss_item_sk#98, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(117) CometSort -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98 ASC NULLS FIRST] - -(118) ReusedExchange [Reuses operator id: 26] -Output [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] - -(119) CometHashAggregate -Input [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] -Keys [1]: [cs_item_sk#114] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#118)), sum(((cr_refunded_cash#119 + cr_reversed_charge#120) + cr_store_credit#121))] - -(120) CometFilter -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(121) CometProject -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Arguments: [cs_item_sk#114], [cs_item_sk#114] - -(122) CometSort -Input [1]: [cs_item_sk#114] -Arguments: [cs_item_sk#114], [cs_item_sk#114 ASC NULLS FIRST] - -(123) CometSortMergeJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [1]: [cs_item_sk#114] -Arguments: [ss_item_sk#98], [cs_item_sk#114], Inner - -(124) CometProject -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, cs_item_sk#114] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(125) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(127) CometBroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: [d_date_sk#122, d_year#123] - -(128) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [d_date_sk#122, d_year#123] -Arguments: [ss_sold_date_sk#109], [d_date_sk#122], Inner, BuildRight - -(129) CometProject -Input [13]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, d_date_sk#122, d_year#123] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] - -(130) ReusedExchange [Reuses operator id: 41] -Output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] - -(131) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] -Right output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_store_sk#103], [s_store_sk#124], Inner, BuildRight - -(132) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] - -(133) ReusedExchange [Reuses operator id: 46] -Output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(134) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] -Right output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_customer_sk#99], [c_customer_sk#126], Inner, BuildRight - -(135) CometProject -Input [18]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(136) ReusedExchange [Reuses operator id: 51] -Output [2]: [d_date_sk#132, d_year#133] - -(137) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Right output [2]: [d_date_sk#132, d_year#133] -Arguments: [c_first_sales_date_sk#131], [d_date_sk#132], Inner, BuildRight - -(138) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131, d_date_sk#132, d_year#133] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] - -(139) ReusedExchange [Reuses operator id: 51] -Output [2]: [d_date_sk#134, d_year#135] - -(140) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] -Right output [2]: [d_date_sk#134, d_year#135] -Arguments: [c_first_shipto_date_sk#130], [d_date_sk#134], Inner, BuildRight - -(141) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133, d_date_sk#134, d_year#135] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(142) ReusedExchange [Reuses operator id: 60] -Output [2]: [cd_demo_sk#136, cd_marital_status#50] - -(143) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_cdemo_sk#100], [cd_demo_sk#136], Inner, BuildRight - -(144) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] - -(145) ReusedExchange [Reuses operator id: 60] -Output [2]: [cd_demo_sk#137, cd_marital_status#52] - -(146) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] -Right output [2]: [cd_demo_sk#137, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#127], [cd_demo_sk#137], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(147) CometProject -Input [18]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50, cd_demo_sk#137, cd_marital_status#52] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(148) ReusedExchange [Reuses operator id: 68] -Output [1]: [p_promo_sk#138] - -(149) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [1]: [p_promo_sk#138] -Arguments: [ss_promo_sk#104], [p_promo_sk#138], Inner, BuildRight - -(150) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, p_promo_sk#138] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(151) ReusedExchange [Reuses operator id: 73] -Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] - -(152) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_hdemo_sk#101], [hd_demo_sk#139], Inner, BuildRight - -(153) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] - -(154) ReusedExchange [Reuses operator id: 73] -Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] - -(155) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] -Right output [2]: [hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [c_current_hdemo_sk#128], [hd_demo_sk#141], Inner, BuildRight - -(156) CometProject -Input [15]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] - -(157) ReusedExchange [Reuses operator id: 82] -Output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(158) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] -Right output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_addr_sk#102], [ca_address_sk#143], Inner, BuildRight - -(159) CometProject -Input [18]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(160) ReusedExchange [Reuses operator id: 82] -Output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(161) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Right output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [c_current_addr_sk#129], [ca_address_sk#146], Inner, BuildRight - -(162) CometProject -Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(163) ReusedExchange [Reuses operator id: 90] -Output [1]: [ib_income_band_sk#149] - -(164) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#149] -Arguments: [hd_income_band_sk#140], [ib_income_band_sk#149], Inner, BuildRight - -(165) CometProject -Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#149] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(166) ReusedExchange [Reuses operator id: 90] -Output [1]: [ib_income_band_sk#150] - -(167) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#150] -Arguments: [hd_income_band_sk#142], [ib_income_band_sk#150], Inner, BuildRight - -(168) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#150] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(169) ReusedExchange [Reuses operator id: 99] -Output [2]: [i_item_sk#151, i_product_name#76] - -(170) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [2]: [i_item_sk#151, i_product_name#76] -Arguments: [ss_item_sk#98], [i_item_sk#151], Inner, BuildRight - -(171) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] - -(172) CometHashAggregate -Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#106)), partial_sum(UnscaledValue(ss_list_price#107)), partial_sum(UnscaledValue(ss_coupon_amt#108))] - -(173) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135, count#77, sum#152, sum#153, sum#154] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#106)), sum(UnscaledValue(ss_list_price#107)), sum(UnscaledValue(ss_coupon_amt#108))] - -(174) CometExchange -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: hashpartitioning(item_sk#155, store_name#156, store_zip#157, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(175) CometSort -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162], [item_sk#155 ASC NULLS FIRST, store_name#156 ASC NULLS FIRST, store_zip#157 ASC NULLS FIRST] - -(176) CometSortMergeJoin -Left output [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Right output [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#156, store_zip#157], Inner, (cnt#159 <= cnt#94) - -(177) CometProject -Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -(178) CometExchange -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(179) CometSort -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST] - -(180) CometColumnarToRow [codegen id : 5] -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (184) -+- * CometColumnarToRow (183) - +- CometFilter (182) - +- CometNativeScan parquet spark_catalog.default.date_dim (181) - - -(181) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(182) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(183) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#33] - -(184) BroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] - -Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 -BroadcastExchange (188) -+- * CometColumnarToRow (187) - +- CometFilter (186) - +- CometNativeScan parquet spark_catalog.default.date_dim (185) - - -(185) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(186) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(187) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#122, d_year#123] - -(188) BroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/simplified.txt deleted file mode 100644 index f954616bb5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/simplified.txt +++ /dev/null @@ -1,206 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometExchange [product_name,store_name,cnt] #1 - CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometColumnarExchange [ss_item_sk] #3 - WholeStageCodegen (2) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometNativeScan 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] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [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] #11 - 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] - CometNativeScan 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] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk] - ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometColumnarExchange [ss_item_sk] #20 - WholeStageCodegen (4) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #21 - WholeStageCodegen (3) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet 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 #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - CometBroadcastExchange [d_date_sk,d_year] #23 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/explain.txt deleted file mode 100644 index cbe790cd00..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,1011 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (176) -+- CometSort (175) - +- CometExchange (174) - +- CometProject (173) - +- CometSortMergeJoin (172) - :- CometSort (103) - : +- CometExchange (102) - : +- CometHashAggregate (101) - : +- CometHashAggregate (100) - : +- CometProject (99) - : +- CometBroadcastHashJoin (98) - : :- CometProject (93) - : : +- CometBroadcastHashJoin (92) - : : :- CometProject (90) - : : : +- CometBroadcastHashJoin (89) - : : : :- CometProject (85) - : : : : +- CometBroadcastHashJoin (84) - : : : : :- CometProject (82) - : : : : : +- CometBroadcastHashJoin (81) - : : : : : :- CometProject (76) - : : : : : : +- CometBroadcastHashJoin (75) - : : : : : : :- CometProject (73) - : : : : : : : +- CometBroadcastHashJoin (72) - : : : : : : : :- CometProject (68) - : : : : : : : : +- CometBroadcastHashJoin (67) - : : : : : : : : :- CometProject (63) - : : : : : : : : : +- CometBroadcastHashJoin (62) - : : : : : : : : : :- CometProject (60) - : : : : : : : : : : +- CometBroadcastHashJoin (59) - : : : : : : : : : : :- CometProject (54) - : : : : : : : : : : : +- CometBroadcastHashJoin (53) - : : : : : : : : : : : :- CometProject (51) - : : : : : : : : : : : : +- CometBroadcastHashJoin (50) - : : : : : : : : : : : : :- CometProject (46) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (45) - : : : : : : : : : : : : : :- CometProject (41) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (40) - : : : : : : : : : : : : : : :- CometProject (35) - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (34) - : : : : : : : : : : : : : : : :- CometProject (30) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) - : : : : : : : : : : : : : : : : :- CometSort (10) - : : : : : : : : : : : : : : : : : +- CometExchange (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- CometSort (28) - : : : : : : : : : : : : : : : : +- CometProject (27) - : : : : : : : : : : : : : : : : +- CometFilter (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometExchange (24) - : : : : : : : : : : : : : : : : +- CometHashAggregate (23) - : : : : : : : : : : : : : : : : +- CometProject (22) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) - : : : : : : : : : : : : : : : : :- CometSort (15) - : : : : : : : : : : : : : : : : : +- CometExchange (14) - : : : : : : : : : : : : : : : : : +- CometProject (13) - : : : : : : : : : : : : : : : : : +- CometFilter (12) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (11) - : : : : : : : : : : : : : : : : +- CometSort (20) - : : : : : : : : : : : : : : : : +- CometExchange (19) - : : : : : : : : : : : : : : : : +- CometProject (18) - : : : : : : : : : : : : : : : : +- CometFilter (17) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : : : : : : : : : : : : : : : +- CometBroadcastExchange (33) - : : : : : : : : : : : : : : : +- CometFilter (32) - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - : : : : : : : : : : : : : : +- CometBroadcastExchange (39) - : : : : : : : : : : : : : : +- CometProject (38) - : : : : : : : : : : : : : : +- CometFilter (37) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (36) - : : : : : : : : : : : : : +- CometBroadcastExchange (44) - : : : : : : : : : : : : : +- CometFilter (43) - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (42) - : : : : : : : : : : : : +- CometBroadcastExchange (49) - : : : : : : : : : : : : +- CometFilter (48) - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) - : : : : : : : : : : : +- ReusedExchange (52) - : : : : : : : : : : +- CometBroadcastExchange (58) - : : : : : : : : : : +- CometProject (57) - : : : : : : : : : : +- CometFilter (56) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (55) - : : : : : : : : : +- ReusedExchange (61) - : : : : : : : : +- CometBroadcastExchange (66) - : : : : : : : : +- CometFilter (65) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (64) - : : : : : : : +- CometBroadcastExchange (71) - : : : : : : : +- CometFilter (70) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (69) - : : : : : : +- ReusedExchange (74) - : : : : : +- CometBroadcastExchange (80) - : : : : : +- CometProject (79) - : : : : : +- CometFilter (78) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (77) - : : : : +- ReusedExchange (83) - : : : +- CometBroadcastExchange (88) - : : : +- CometFilter (87) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (86) - : : +- ReusedExchange (91) - : +- CometBroadcastExchange (97) - : +- CometProject (96) - : +- CometFilter (95) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (94) - +- CometSort (171) - +- CometExchange (170) - +- CometHashAggregate (169) - +- CometHashAggregate (168) - +- CometProject (167) - +- CometBroadcastHashJoin (166) - :- CometProject (164) - : +- CometBroadcastHashJoin (163) - : :- CometProject (161) - : : +- CometBroadcastHashJoin (160) - : : :- CometProject (158) - : : : +- CometBroadcastHashJoin (157) - : : : :- CometProject (155) - : : : : +- CometBroadcastHashJoin (154) - : : : : :- CometProject (152) - : : : : : +- CometBroadcastHashJoin (151) - : : : : : :- CometProject (149) - : : : : : : +- CometBroadcastHashJoin (148) - : : : : : : :- CometProject (146) - : : : : : : : +- CometBroadcastHashJoin (145) - : : : : : : : :- CometProject (143) - : : : : : : : : +- CometBroadcastHashJoin (142) - : : : : : : : : :- CometProject (140) - : : : : : : : : : +- CometBroadcastHashJoin (139) - : : : : : : : : : :- CometProject (137) - : : : : : : : : : : +- CometBroadcastHashJoin (136) - : : : : : : : : : : :- CometProject (134) - : : : : : : : : : : : +- CometBroadcastHashJoin (133) - : : : : : : : : : : : :- CometProject (131) - : : : : : : : : : : : : +- CometBroadcastHashJoin (130) - : : : : : : : : : : : : :- CometProject (128) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (127) - : : : : : : : : : : : : : :- CometProject (125) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (124) - : : : : : : : : : : : : : : :- CometProject (120) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (119) - : : : : : : : : : : : : : : : :- CometSort (113) - : : : : : : : : : : : : : : : : +- CometExchange (112) - : : : : : : : : : : : : : : : : +- CometProject (111) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (110) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (106) - : : : : : : : : : : : : : : : : : +- CometFilter (105) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (104) - : : : : : : : : : : : : : : : : +- CometProject (109) - : : : : : : : : : : : : : : : : +- CometFilter (108) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (107) - : : : : : : : : : : : : : : : +- CometSort (118) - : : : : : : : : : : : : : : : +- CometProject (117) - : : : : : : : : : : : : : : : +- CometFilter (116) - : : : : : : : : : : : : : : : +- CometHashAggregate (115) - : : : : : : : : : : : : : : : +- ReusedExchange (114) - : : : : : : : : : : : : : : +- CometBroadcastExchange (123) - : : : : : : : : : : : : : : +- CometFilter (122) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (121) - : : : : : : : : : : : : : +- ReusedExchange (126) - : : : : : : : : : : : : +- ReusedExchange (129) - : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : +- ReusedExchange (141) - : : : : : : : +- ReusedExchange (144) - : : : : : : +- ReusedExchange (147) - : : : : : +- ReusedExchange (150) - : : : : +- ReusedExchange (153) - : : : +- ReusedExchange (156) - : : +- ReusedExchange (159) - : +- ReusedExchange (162) - +- ReusedExchange (165) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) CometFilter -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) - -(3) CometBroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) - -(6) CometProject -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] - -(7) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner, BuildLeft - -(8) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(9) CometExchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometSort -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) - -(13) CometProject -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] - -(14) CometExchange -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(15) CometSort -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(17) CometFilter -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) - -(18) CometProject -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(19) CometExchange -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner - -(22) CometProject -Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(23) CometHashAggregate -Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Keys [1]: [cs_item_sk#17] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(24) CometExchange -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometHashAggregate -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Keys [1]: [cs_item_sk#17] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(26) CometFilter -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(27) CometProject -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Arguments: [cs_item_sk#17], [cs_item_sk#17] - -(28) CometSort -Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] - -(29) CometSortMergeJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [1]: [cs_item_sk#17] -Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner - -(30) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(33) CometBroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: [d_date_sk#32, d_year#33] - -(34) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight - -(35) CometProject -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true))) - -(38) CometProject -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37], [s_store_sk#34, s_store_name#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true) AS s_zip#37] - -(39) CometBroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37] - -(40) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight - -(41) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(43) CometFilter -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Condition : (((((isnotnull(c_customer_sk#38) AND isnotnull(c_first_sales_date_sk#43)) AND isnotnull(c_first_shipto_date_sk#42)) AND isnotnull(c_current_cdemo_sk#39)) AND isnotnull(c_current_hdemo_sk#40)) AND isnotnull(c_current_addr_sk#41)) - -(44) CometBroadcastExchange -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(45) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -Right output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_customer_sk#2], [c_customer_sk#38], Inner, BuildRight - -(46) CometProject -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(48) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : isnotnull(d_date_sk#44) - -(49) CometBroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] - -(50) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [c_first_sales_date_sk#43], [d_date_sk#44], Inner, BuildRight - -(51) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43, d_date_sk#44, d_year#45] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] - -(52) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#46, d_year#47] - -(53) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [c_first_shipto_date_sk#42], [d_date_sk#46], Inner, BuildRight - -(54) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45, d_date_sk#46, d_year#47] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#48, cd_marital_status#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true))) - -(57) CometProject -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Arguments: [cd_demo_sk#48, cd_marital_status#50], [cd_demo_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true) AS cd_marital_status#50] - -(58) CometBroadcastExchange -Input [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [cd_demo_sk#48, cd_marital_status#50] - -(59) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_cdemo_sk#3], [cd_demo_sk#48], Inner, BuildRight - -(60) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] - -(61) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#51, cd_marital_status#52] - -(62) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -Right output [2]: [cd_demo_sk#51, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#39], [cd_demo_sk#51], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(63) CometProject -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50, cd_demo_sk#51, cd_marital_status#52] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(65) CometFilter -Input [1]: [p_promo_sk#53] -Condition : isnotnull(p_promo_sk#53) - -(66) CometBroadcastExchange -Input [1]: [p_promo_sk#53] -Arguments: [p_promo_sk#53] - -(67) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [1]: [p_promo_sk#53] -Arguments: [ss_promo_sk#7], [p_promo_sk#53], Inner, BuildRight - -(68) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, p_promo_sk#53] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(70) CometFilter -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Condition : (isnotnull(hd_demo_sk#54) AND isnotnull(hd_income_band_sk#55)) - -(71) CometBroadcastExchange -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [hd_demo_sk#54, hd_income_band_sk#55] - -(72) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_hdemo_sk#4], [hd_demo_sk#54], Inner, BuildRight - -(73) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] - -(74) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] - -(75) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -Right output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [c_current_hdemo_sk#40], [hd_demo_sk#56], Inner, BuildRight - -(76) CometProject -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(78) CometFilter -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Condition : isnotnull(ca_address_sk#58) - -(79) CometProject -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ca_address_sk#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#59, 10, true, false, true) AS ca_street_number#63, ca_street_name#60, ca_city#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#62, 10, true, false, true) AS ca_zip#64] - -(80) CometBroadcastExchange -Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(81) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -Right output [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_addr_sk#5], [ca_address_sk#58], Inner, BuildRight - -(82) CometProject -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(83) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(84) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Right output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [c_current_addr_sk#41], [ca_address_sk#65], Inner, BuildRight - -(85) CometProject -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(87) CometFilter -Input [1]: [ib_income_band_sk#70] -Condition : isnotnull(ib_income_band_sk#70) - -(88) CometBroadcastExchange -Input [1]: [ib_income_band_sk#70] -Arguments: [ib_income_band_sk#70] - -(89) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#70] -Arguments: [hd_income_band_sk#55], [ib_income_band_sk#70], Inner, BuildRight - -(90) CometProject -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#70] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(91) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#71] - -(92) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#71] -Arguments: [hd_income_band_sk#57], [ib_income_band_sk#71], Inner, BuildRight - -(93) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#71] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(95) CometFilter -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Condition : ((((((isnotnull(i_current_price#73) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#74, 20, true, false, true) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#73 >= 64.00)) AND (i_current_price#73 <= 74.00)) AND (i_current_price#73 >= 65.00)) AND (i_current_price#73 <= 79.00)) AND isnotnull(i_item_sk#72)) - -(96) CometProject -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Arguments: [i_item_sk#72, i_product_name#76], [i_item_sk#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#75, 50, true, false, true) AS i_product_name#76] - -(97) CometBroadcastExchange -Input [2]: [i_item_sk#72, i_product_name#76] -Arguments: [i_item_sk#72, i_product_name#76] - -(98) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [2]: [i_item_sk#72, i_product_name#76] -Arguments: [ss_item_sk#1], [i_item_sk#72], Inner, BuildRight - -(99) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] - -(100) CometHashAggregate -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] - -(101) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#77, sum#78, sum#79, sum#80] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] - -(102) CometExchange -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: hashpartitioning(item_sk#82, store_name#83, store_zip#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(103) CometSort -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97], [item_sk#82 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, store_zip#84 ASC NULLS FIRST] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#109), dynamicpruningexpression(ss_sold_date_sk#109 IN dynamicpruning#110)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(105) CometFilter -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Condition : (((((((isnotnull(ss_item_sk#98) AND isnotnull(ss_ticket_number#105)) AND isnotnull(ss_store_sk#103)) AND isnotnull(ss_customer_sk#99)) AND isnotnull(ss_cdemo_sk#100)) AND isnotnull(ss_promo_sk#104)) AND isnotnull(ss_hdemo_sk#101)) AND isnotnull(ss_addr_sk#102)) - -(106) CometBroadcastExchange -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(108) CometFilter -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) - -(109) CometProject -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] - -(110) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_ticket_number#105], [sr_item_sk#111, sr_ticket_number#112], Inner, BuildLeft - -(111) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(112) CometExchange -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: hashpartitioning(ss_item_sk#98, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(113) CometSort -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98 ASC NULLS FIRST] - -(114) ReusedExchange [Reuses operator id: 24] -Output [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] - -(115) CometHashAggregate -Input [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] -Keys [1]: [cs_item_sk#114] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#118)), sum(((cr_refunded_cash#119 + cr_reversed_charge#120) + cr_store_credit#121))] - -(116) CometFilter -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(117) CometProject -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Arguments: [cs_item_sk#114], [cs_item_sk#114] - -(118) CometSort -Input [1]: [cs_item_sk#114] -Arguments: [cs_item_sk#114], [cs_item_sk#114 ASC NULLS FIRST] - -(119) CometSortMergeJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [1]: [cs_item_sk#114] -Arguments: [ss_item_sk#98], [cs_item_sk#114], Inner - -(120) CometProject -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, cs_item_sk#114] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(121) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(122) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(123) CometBroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: [d_date_sk#122, d_year#123] - -(124) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [d_date_sk#122, d_year#123] -Arguments: [ss_sold_date_sk#109], [d_date_sk#122], Inner, BuildRight - -(125) CometProject -Input [13]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, d_date_sk#122, d_year#123] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] - -(126) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] - -(127) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] -Right output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_store_sk#103], [s_store_sk#124], Inner, BuildRight - -(128) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] - -(129) ReusedExchange [Reuses operator id: 44] -Output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(130) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] -Right output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_customer_sk#99], [c_customer_sk#126], Inner, BuildRight - -(131) CometProject -Input [18]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(132) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#132, d_year#133] - -(133) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Right output [2]: [d_date_sk#132, d_year#133] -Arguments: [c_first_sales_date_sk#131], [d_date_sk#132], Inner, BuildRight - -(134) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131, d_date_sk#132, d_year#133] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] - -(135) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#134, d_year#135] - -(136) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] -Right output [2]: [d_date_sk#134, d_year#135] -Arguments: [c_first_shipto_date_sk#130], [d_date_sk#134], Inner, BuildRight - -(137) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133, d_date_sk#134, d_year#135] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(138) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#136, cd_marital_status#50] - -(139) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_cdemo_sk#100], [cd_demo_sk#136], Inner, BuildRight - -(140) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] - -(141) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#137, cd_marital_status#52] - -(142) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] -Right output [2]: [cd_demo_sk#137, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#127], [cd_demo_sk#137], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(143) CometProject -Input [18]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50, cd_demo_sk#137, cd_marital_status#52] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(144) ReusedExchange [Reuses operator id: 66] -Output [1]: [p_promo_sk#138] - -(145) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [1]: [p_promo_sk#138] -Arguments: [ss_promo_sk#104], [p_promo_sk#138], Inner, BuildRight - -(146) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, p_promo_sk#138] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(147) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] - -(148) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_hdemo_sk#101], [hd_demo_sk#139], Inner, BuildRight - -(149) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] - -(150) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] - -(151) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] -Right output [2]: [hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [c_current_hdemo_sk#128], [hd_demo_sk#141], Inner, BuildRight - -(152) CometProject -Input [15]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] - -(153) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(154) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] -Right output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_addr_sk#102], [ca_address_sk#143], Inner, BuildRight - -(155) CometProject -Input [18]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(156) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(157) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Right output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [c_current_addr_sk#129], [ca_address_sk#146], Inner, BuildRight - -(158) CometProject -Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(159) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#149] - -(160) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#149] -Arguments: [hd_income_band_sk#140], [ib_income_band_sk#149], Inner, BuildRight - -(161) CometProject -Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#149] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(162) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#150] - -(163) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#150] -Arguments: [hd_income_band_sk#142], [ib_income_band_sk#150], Inner, BuildRight - -(164) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#150] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(165) ReusedExchange [Reuses operator id: 97] -Output [2]: [i_item_sk#151, i_product_name#76] - -(166) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [2]: [i_item_sk#151, i_product_name#76] -Arguments: [ss_item_sk#98], [i_item_sk#151], Inner, BuildRight - -(167) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] - -(168) CometHashAggregate -Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#106)), partial_sum(UnscaledValue(ss_list_price#107)), partial_sum(UnscaledValue(ss_coupon_amt#108))] - -(169) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135, count#77, sum#152, sum#153, sum#154] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#106)), sum(UnscaledValue(ss_list_price#107)), sum(UnscaledValue(ss_coupon_amt#108))] - -(170) CometExchange -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: hashpartitioning(item_sk#155, store_name#156, store_zip#157, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(171) CometSort -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162], [item_sk#155 ASC NULLS FIRST, store_name#156 ASC NULLS FIRST, store_zip#157 ASC NULLS FIRST] - -(172) CometSortMergeJoin -Left output [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Right output [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#156, store_zip#157], Inner, (cnt#159 <= cnt#94) - -(173) CometProject -Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -(174) CometExchange -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(175) CometSort -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST] - -(176) CometColumnarToRow [codegen id : 1] -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (180) -+- * CometColumnarToRow (179) - +- CometFilter (178) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) - - -(177) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(178) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(179) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#33] - -(180) BroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 -BroadcastExchange (184) -+- * CometColumnarToRow (183) - +- CometFilter (182) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) - - -(181) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(182) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(183) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#122, d_year#123] - -(184) BroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/simplified.txt deleted file mode 100644 index 43527978d8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,192 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometExchange [product_name,store_name,cnt] #1 - CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #3 - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [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] #11 - 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 [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk] - ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #20 - 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_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] #21 - 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 [native_iceberg_compat] 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 #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - CometBroadcastExchange [d_date_sk,d_year] #23 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/explain.txt deleted file mode 100644 index cbe790cd00..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/explain.txt +++ /dev/null @@ -1,1011 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (176) -+- CometSort (175) - +- CometExchange (174) - +- CometProject (173) - +- CometSortMergeJoin (172) - :- CometSort (103) - : +- CometExchange (102) - : +- CometHashAggregate (101) - : +- CometHashAggregate (100) - : +- CometProject (99) - : +- CometBroadcastHashJoin (98) - : :- CometProject (93) - : : +- CometBroadcastHashJoin (92) - : : :- CometProject (90) - : : : +- CometBroadcastHashJoin (89) - : : : :- CometProject (85) - : : : : +- CometBroadcastHashJoin (84) - : : : : :- CometProject (82) - : : : : : +- CometBroadcastHashJoin (81) - : : : : : :- CometProject (76) - : : : : : : +- CometBroadcastHashJoin (75) - : : : : : : :- CometProject (73) - : : : : : : : +- CometBroadcastHashJoin (72) - : : : : : : : :- CometProject (68) - : : : : : : : : +- CometBroadcastHashJoin (67) - : : : : : : : : :- CometProject (63) - : : : : : : : : : +- CometBroadcastHashJoin (62) - : : : : : : : : : :- CometProject (60) - : : : : : : : : : : +- CometBroadcastHashJoin (59) - : : : : : : : : : : :- CometProject (54) - : : : : : : : : : : : +- CometBroadcastHashJoin (53) - : : : : : : : : : : : :- CometProject (51) - : : : : : : : : : : : : +- CometBroadcastHashJoin (50) - : : : : : : : : : : : : :- CometProject (46) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (45) - : : : : : : : : : : : : : :- CometProject (41) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (40) - : : : : : : : : : : : : : : :- CometProject (35) - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (34) - : : : : : : : : : : : : : : : :- CometProject (30) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) - : : : : : : : : : : : : : : : : :- CometSort (10) - : : : : : : : : : : : : : : : : : +- CometExchange (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- CometSort (28) - : : : : : : : : : : : : : : : : +- CometProject (27) - : : : : : : : : : : : : : : : : +- CometFilter (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometExchange (24) - : : : : : : : : : : : : : : : : +- CometHashAggregate (23) - : : : : : : : : : : : : : : : : +- CometProject (22) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) - : : : : : : : : : : : : : : : : :- CometSort (15) - : : : : : : : : : : : : : : : : : +- CometExchange (14) - : : : : : : : : : : : : : : : : : +- CometProject (13) - : : : : : : : : : : : : : : : : : +- CometFilter (12) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (11) - : : : : : : : : : : : : : : : : +- CometSort (20) - : : : : : : : : : : : : : : : : +- CometExchange (19) - : : : : : : : : : : : : : : : : +- CometProject (18) - : : : : : : : : : : : : : : : : +- CometFilter (17) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : : : : : : : : : : : : : : : +- CometBroadcastExchange (33) - : : : : : : : : : : : : : : : +- CometFilter (32) - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - : : : : : : : : : : : : : : +- CometBroadcastExchange (39) - : : : : : : : : : : : : : : +- CometProject (38) - : : : : : : : : : : : : : : +- CometFilter (37) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (36) - : : : : : : : : : : : : : +- CometBroadcastExchange (44) - : : : : : : : : : : : : : +- CometFilter (43) - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (42) - : : : : : : : : : : : : +- CometBroadcastExchange (49) - : : : : : : : : : : : : +- CometFilter (48) - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) - : : : : : : : : : : : +- ReusedExchange (52) - : : : : : : : : : : +- CometBroadcastExchange (58) - : : : : : : : : : : +- CometProject (57) - : : : : : : : : : : +- CometFilter (56) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (55) - : : : : : : : : : +- ReusedExchange (61) - : : : : : : : : +- CometBroadcastExchange (66) - : : : : : : : : +- CometFilter (65) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (64) - : : : : : : : +- CometBroadcastExchange (71) - : : : : : : : +- CometFilter (70) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (69) - : : : : : : +- ReusedExchange (74) - : : : : : +- CometBroadcastExchange (80) - : : : : : +- CometProject (79) - : : : : : +- CometFilter (78) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (77) - : : : : +- ReusedExchange (83) - : : : +- CometBroadcastExchange (88) - : : : +- CometFilter (87) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (86) - : : +- ReusedExchange (91) - : +- CometBroadcastExchange (97) - : +- CometProject (96) - : +- CometFilter (95) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (94) - +- CometSort (171) - +- CometExchange (170) - +- CometHashAggregate (169) - +- CometHashAggregate (168) - +- CometProject (167) - +- CometBroadcastHashJoin (166) - :- CometProject (164) - : +- CometBroadcastHashJoin (163) - : :- CometProject (161) - : : +- CometBroadcastHashJoin (160) - : : :- CometProject (158) - : : : +- CometBroadcastHashJoin (157) - : : : :- CometProject (155) - : : : : +- CometBroadcastHashJoin (154) - : : : : :- CometProject (152) - : : : : : +- CometBroadcastHashJoin (151) - : : : : : :- CometProject (149) - : : : : : : +- CometBroadcastHashJoin (148) - : : : : : : :- CometProject (146) - : : : : : : : +- CometBroadcastHashJoin (145) - : : : : : : : :- CometProject (143) - : : : : : : : : +- CometBroadcastHashJoin (142) - : : : : : : : : :- CometProject (140) - : : : : : : : : : +- CometBroadcastHashJoin (139) - : : : : : : : : : :- CometProject (137) - : : : : : : : : : : +- CometBroadcastHashJoin (136) - : : : : : : : : : : :- CometProject (134) - : : : : : : : : : : : +- CometBroadcastHashJoin (133) - : : : : : : : : : : : :- CometProject (131) - : : : : : : : : : : : : +- CometBroadcastHashJoin (130) - : : : : : : : : : : : : :- CometProject (128) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (127) - : : : : : : : : : : : : : :- CometProject (125) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (124) - : : : : : : : : : : : : : : :- CometProject (120) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (119) - : : : : : : : : : : : : : : : :- CometSort (113) - : : : : : : : : : : : : : : : : +- CometExchange (112) - : : : : : : : : : : : : : : : : +- CometProject (111) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (110) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (106) - : : : : : : : : : : : : : : : : : +- CometFilter (105) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (104) - : : : : : : : : : : : : : : : : +- CometProject (109) - : : : : : : : : : : : : : : : : +- CometFilter (108) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (107) - : : : : : : : : : : : : : : : +- CometSort (118) - : : : : : : : : : : : : : : : +- CometProject (117) - : : : : : : : : : : : : : : : +- CometFilter (116) - : : : : : : : : : : : : : : : +- CometHashAggregate (115) - : : : : : : : : : : : : : : : +- ReusedExchange (114) - : : : : : : : : : : : : : : +- CometBroadcastExchange (123) - : : : : : : : : : : : : : : +- CometFilter (122) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (121) - : : : : : : : : : : : : : +- ReusedExchange (126) - : : : : : : : : : : : : +- ReusedExchange (129) - : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : +- ReusedExchange (141) - : : : : : : : +- ReusedExchange (144) - : : : : : : +- ReusedExchange (147) - : : : : : +- ReusedExchange (150) - : : : : +- ReusedExchange (153) - : : : +- ReusedExchange (156) - : : +- ReusedExchange (159) - : +- ReusedExchange (162) - +- ReusedExchange (165) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) CometFilter -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) - -(3) CometBroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) - -(6) CometProject -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] - -(7) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner, BuildLeft - -(8) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(9) CometExchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometSort -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) - -(13) CometProject -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] - -(14) CometExchange -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(15) CometSort -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(17) CometFilter -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) - -(18) CometProject -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(19) CometExchange -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner - -(22) CometProject -Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(23) CometHashAggregate -Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Keys [1]: [cs_item_sk#17] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(24) CometExchange -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometHashAggregate -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Keys [1]: [cs_item_sk#17] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(26) CometFilter -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(27) CometProject -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Arguments: [cs_item_sk#17], [cs_item_sk#17] - -(28) CometSort -Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] - -(29) CometSortMergeJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [1]: [cs_item_sk#17] -Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner - -(30) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(33) CometBroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: [d_date_sk#32, d_year#33] - -(34) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight - -(35) CometProject -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true))) - -(38) CometProject -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37], [s_store_sk#34, s_store_name#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true) AS s_zip#37] - -(39) CometBroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37] - -(40) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight - -(41) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(43) CometFilter -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Condition : (((((isnotnull(c_customer_sk#38) AND isnotnull(c_first_sales_date_sk#43)) AND isnotnull(c_first_shipto_date_sk#42)) AND isnotnull(c_current_cdemo_sk#39)) AND isnotnull(c_current_hdemo_sk#40)) AND isnotnull(c_current_addr_sk#41)) - -(44) CometBroadcastExchange -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(45) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -Right output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_customer_sk#2], [c_customer_sk#38], Inner, BuildRight - -(46) CometProject -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(48) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : isnotnull(d_date_sk#44) - -(49) CometBroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] - -(50) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [c_first_sales_date_sk#43], [d_date_sk#44], Inner, BuildRight - -(51) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43, d_date_sk#44, d_year#45] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] - -(52) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#46, d_year#47] - -(53) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [c_first_shipto_date_sk#42], [d_date_sk#46], Inner, BuildRight - -(54) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45, d_date_sk#46, d_year#47] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#48, cd_marital_status#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true))) - -(57) CometProject -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Arguments: [cd_demo_sk#48, cd_marital_status#50], [cd_demo_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true) AS cd_marital_status#50] - -(58) CometBroadcastExchange -Input [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [cd_demo_sk#48, cd_marital_status#50] - -(59) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_cdemo_sk#3], [cd_demo_sk#48], Inner, BuildRight - -(60) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] - -(61) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#51, cd_marital_status#52] - -(62) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -Right output [2]: [cd_demo_sk#51, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#39], [cd_demo_sk#51], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(63) CometProject -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50, cd_demo_sk#51, cd_marital_status#52] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(65) CometFilter -Input [1]: [p_promo_sk#53] -Condition : isnotnull(p_promo_sk#53) - -(66) CometBroadcastExchange -Input [1]: [p_promo_sk#53] -Arguments: [p_promo_sk#53] - -(67) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [1]: [p_promo_sk#53] -Arguments: [ss_promo_sk#7], [p_promo_sk#53], Inner, BuildRight - -(68) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, p_promo_sk#53] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(70) CometFilter -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Condition : (isnotnull(hd_demo_sk#54) AND isnotnull(hd_income_band_sk#55)) - -(71) CometBroadcastExchange -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [hd_demo_sk#54, hd_income_band_sk#55] - -(72) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_hdemo_sk#4], [hd_demo_sk#54], Inner, BuildRight - -(73) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] - -(74) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] - -(75) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -Right output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [c_current_hdemo_sk#40], [hd_demo_sk#56], Inner, BuildRight - -(76) CometProject -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(78) CometFilter -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Condition : isnotnull(ca_address_sk#58) - -(79) CometProject -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ca_address_sk#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#59, 10, true, false, true) AS ca_street_number#63, ca_street_name#60, ca_city#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#62, 10, true, false, true) AS ca_zip#64] - -(80) CometBroadcastExchange -Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(81) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -Right output [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_addr_sk#5], [ca_address_sk#58], Inner, BuildRight - -(82) CometProject -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(83) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(84) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Right output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [c_current_addr_sk#41], [ca_address_sk#65], Inner, BuildRight - -(85) CometProject -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(87) CometFilter -Input [1]: [ib_income_band_sk#70] -Condition : isnotnull(ib_income_band_sk#70) - -(88) CometBroadcastExchange -Input [1]: [ib_income_band_sk#70] -Arguments: [ib_income_band_sk#70] - -(89) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#70] -Arguments: [hd_income_band_sk#55], [ib_income_band_sk#70], Inner, BuildRight - -(90) CometProject -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#70] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(91) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#71] - -(92) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#71] -Arguments: [hd_income_band_sk#57], [ib_income_band_sk#71], Inner, BuildRight - -(93) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#71] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(95) CometFilter -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Condition : ((((((isnotnull(i_current_price#73) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#74, 20, true, false, true) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#73 >= 64.00)) AND (i_current_price#73 <= 74.00)) AND (i_current_price#73 >= 65.00)) AND (i_current_price#73 <= 79.00)) AND isnotnull(i_item_sk#72)) - -(96) CometProject -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Arguments: [i_item_sk#72, i_product_name#76], [i_item_sk#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#75, 50, true, false, true) AS i_product_name#76] - -(97) CometBroadcastExchange -Input [2]: [i_item_sk#72, i_product_name#76] -Arguments: [i_item_sk#72, i_product_name#76] - -(98) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [2]: [i_item_sk#72, i_product_name#76] -Arguments: [ss_item_sk#1], [i_item_sk#72], Inner, BuildRight - -(99) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] - -(100) CometHashAggregate -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] - -(101) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#77, sum#78, sum#79, sum#80] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] - -(102) CometExchange -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: hashpartitioning(item_sk#82, store_name#83, store_zip#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(103) CometSort -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97], [item_sk#82 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, store_zip#84 ASC NULLS FIRST] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#109), dynamicpruningexpression(ss_sold_date_sk#109 IN dynamicpruning#110)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(105) CometFilter -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Condition : (((((((isnotnull(ss_item_sk#98) AND isnotnull(ss_ticket_number#105)) AND isnotnull(ss_store_sk#103)) AND isnotnull(ss_customer_sk#99)) AND isnotnull(ss_cdemo_sk#100)) AND isnotnull(ss_promo_sk#104)) AND isnotnull(ss_hdemo_sk#101)) AND isnotnull(ss_addr_sk#102)) - -(106) CometBroadcastExchange -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(108) CometFilter -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) - -(109) CometProject -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] - -(110) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_ticket_number#105], [sr_item_sk#111, sr_ticket_number#112], Inner, BuildLeft - -(111) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(112) CometExchange -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: hashpartitioning(ss_item_sk#98, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(113) CometSort -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98 ASC NULLS FIRST] - -(114) ReusedExchange [Reuses operator id: 24] -Output [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] - -(115) CometHashAggregate -Input [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] -Keys [1]: [cs_item_sk#114] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#118)), sum(((cr_refunded_cash#119 + cr_reversed_charge#120) + cr_store_credit#121))] - -(116) CometFilter -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(117) CometProject -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Arguments: [cs_item_sk#114], [cs_item_sk#114] - -(118) CometSort -Input [1]: [cs_item_sk#114] -Arguments: [cs_item_sk#114], [cs_item_sk#114 ASC NULLS FIRST] - -(119) CometSortMergeJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [1]: [cs_item_sk#114] -Arguments: [ss_item_sk#98], [cs_item_sk#114], Inner - -(120) CometProject -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, cs_item_sk#114] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(121) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(122) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(123) CometBroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: [d_date_sk#122, d_year#123] - -(124) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [d_date_sk#122, d_year#123] -Arguments: [ss_sold_date_sk#109], [d_date_sk#122], Inner, BuildRight - -(125) CometProject -Input [13]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, d_date_sk#122, d_year#123] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] - -(126) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] - -(127) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] -Right output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_store_sk#103], [s_store_sk#124], Inner, BuildRight - -(128) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] - -(129) ReusedExchange [Reuses operator id: 44] -Output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(130) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] -Right output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_customer_sk#99], [c_customer_sk#126], Inner, BuildRight - -(131) CometProject -Input [18]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(132) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#132, d_year#133] - -(133) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Right output [2]: [d_date_sk#132, d_year#133] -Arguments: [c_first_sales_date_sk#131], [d_date_sk#132], Inner, BuildRight - -(134) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131, d_date_sk#132, d_year#133] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] - -(135) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#134, d_year#135] - -(136) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] -Right output [2]: [d_date_sk#134, d_year#135] -Arguments: [c_first_shipto_date_sk#130], [d_date_sk#134], Inner, BuildRight - -(137) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133, d_date_sk#134, d_year#135] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(138) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#136, cd_marital_status#50] - -(139) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_cdemo_sk#100], [cd_demo_sk#136], Inner, BuildRight - -(140) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] - -(141) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#137, cd_marital_status#52] - -(142) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] -Right output [2]: [cd_demo_sk#137, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#127], [cd_demo_sk#137], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(143) CometProject -Input [18]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50, cd_demo_sk#137, cd_marital_status#52] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(144) ReusedExchange [Reuses operator id: 66] -Output [1]: [p_promo_sk#138] - -(145) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [1]: [p_promo_sk#138] -Arguments: [ss_promo_sk#104], [p_promo_sk#138], Inner, BuildRight - -(146) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, p_promo_sk#138] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(147) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] - -(148) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_hdemo_sk#101], [hd_demo_sk#139], Inner, BuildRight - -(149) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] - -(150) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] - -(151) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] -Right output [2]: [hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [c_current_hdemo_sk#128], [hd_demo_sk#141], Inner, BuildRight - -(152) CometProject -Input [15]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] - -(153) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(154) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] -Right output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_addr_sk#102], [ca_address_sk#143], Inner, BuildRight - -(155) CometProject -Input [18]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(156) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(157) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Right output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [c_current_addr_sk#129], [ca_address_sk#146], Inner, BuildRight - -(158) CometProject -Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(159) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#149] - -(160) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#149] -Arguments: [hd_income_band_sk#140], [ib_income_band_sk#149], Inner, BuildRight - -(161) CometProject -Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#149] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(162) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#150] - -(163) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#150] -Arguments: [hd_income_band_sk#142], [ib_income_band_sk#150], Inner, BuildRight - -(164) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#150] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(165) ReusedExchange [Reuses operator id: 97] -Output [2]: [i_item_sk#151, i_product_name#76] - -(166) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [2]: [i_item_sk#151, i_product_name#76] -Arguments: [ss_item_sk#98], [i_item_sk#151], Inner, BuildRight - -(167) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] - -(168) CometHashAggregate -Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#106)), partial_sum(UnscaledValue(ss_list_price#107)), partial_sum(UnscaledValue(ss_coupon_amt#108))] - -(169) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135, count#77, sum#152, sum#153, sum#154] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#106)), sum(UnscaledValue(ss_list_price#107)), sum(UnscaledValue(ss_coupon_amt#108))] - -(170) CometExchange -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: hashpartitioning(item_sk#155, store_name#156, store_zip#157, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(171) CometSort -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162], [item_sk#155 ASC NULLS FIRST, store_name#156 ASC NULLS FIRST, store_zip#157 ASC NULLS FIRST] - -(172) CometSortMergeJoin -Left output [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Right output [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#156, store_zip#157], Inner, (cnt#159 <= cnt#94) - -(173) CometProject -Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -(174) CometExchange -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(175) CometSort -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST] - -(176) CometColumnarToRow [codegen id : 1] -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (180) -+- * CometColumnarToRow (179) - +- CometFilter (178) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) - - -(177) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(178) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(179) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#33] - -(180) BroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 -BroadcastExchange (184) -+- * CometColumnarToRow (183) - +- CometFilter (182) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) - - -(181) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(182) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(183) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#122, d_year#123] - -(184) BroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/extended.txt deleted file mode 100644 index 059acab385..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/extended.txt +++ /dev/null @@ -1,247 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 43527978d8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt +++ /dev/null @@ -1,192 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometExchange [product_name,store_name,cnt] #1 - CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #3 - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [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] #11 - 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 [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk] - ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #20 - 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_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] #21 - 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 [native_iceberg_compat] 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 #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - CometBroadcastExchange [d_date_sk,d_year] #23 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/explain.txt deleted file mode 100644 index 7cbfb795e8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/explain.txt +++ /dev/null @@ -1,286 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * CometColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.store (1) - : : +- BroadcastExchange (15) - : : +- * Filter (14) - : : +- * HashAggregate (13) - : : +- * CometColumnarToRow (12) - : : +- CometColumnarExchange (11) - : : +- * HashAggregate (10) - : : +- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : +- ReusedExchange (7) - : +- BroadcastExchange (22) - : +- * CometColumnarToRow (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometNativeScan parquet spark_catalog.default.item (18) - +- BroadcastExchange (40) - +- * Filter (39) - +- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * HashAggregate (34) - +- * CometColumnarToRow (33) - +- CometColumnarExchange (32) - +- * HashAggregate (31) - +- * Project (30) - +- * BroadcastHashJoin Inner BuildRight (29) - :- * Filter (27) - : +- * ColumnarToRow (26) - : +- Scan parquet spark_catalog.default.store_sales (25) - +- ReusedExchange (28) - - -(1) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#1, s_store_name#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [s_store_sk#1, s_store_name#2] -Condition : isnotnull(s_store_sk#1) - -(3) CometColumnarToRow [codegen id : 9] -Input [2]: [s_store_sk#1, s_store_name#2] - -(4) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] - -(6) Filter [codegen id : 2] -Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3)) - -(7) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#8] - -(8) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 2] -Output [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] -Input [5]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] - -(10) HashAggregate [codegen id : 2] -Input [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] -Keys [2]: [ss_store_sk#4, ss_item_sk#3] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#5))] -Aggregate Attributes [1]: [sum#9] -Results [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] - -(11) CometColumnarExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(12) CometColumnarToRow [codegen id : 3] -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] - -(13) HashAggregate [codegen id : 3] -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -Keys [2]: [ss_store_sk#4, ss_item_sk#3] -Functions [1]: [sum(UnscaledValue(ss_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#5))#11] -Results [3]: [ss_store_sk#4, ss_item_sk#3, MakeDecimal(sum(UnscaledValue(ss_sales_price#5))#11,17,2) AS revenue#12] - -(14) Filter [codegen id : 3] -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] -Condition : isnotnull(revenue#12) - -(15) BroadcastExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [s_store_sk#1] -Right keys [1]: [ss_store_sk#4] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 9] -Output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] -Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] - -(18) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(19) CometFilter -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Condition : isnotnull(i_item_sk#13) - -(20) CometProject -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Arguments: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18], [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#17, 50, true, false, true) AS i_brand#18] - -(21) CometColumnarToRow [codegen id : 4] -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] - -(22) BroadcastExchange -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(23) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#3] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 9] -Output [7]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] -Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12, i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] - -(25) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(26) ColumnarToRow [codegen id : 6] -Input [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] - -(27) Filter [codegen id : 6] -Input [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_store_sk#20) - -(28) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#23] - -(29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#22] -Right keys [1]: [d_date_sk#23] -Join type: Inner -Join condition: None - -(30) Project [codegen id : 6] -Output [3]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21] -Input [5]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22, d_date_sk#23] - -(31) HashAggregate [codegen id : 6] -Input [3]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21] -Keys [2]: [ss_store_sk#20, ss_item_sk#19] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#21))] -Aggregate Attributes [1]: [sum#24] -Results [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] - -(32) CometColumnarExchange -Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] -Arguments: hashpartitioning(ss_store_sk#20, ss_item_sk#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(33) CometColumnarToRow [codegen id : 7] -Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] - -(34) HashAggregate [codegen id : 7] -Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] -Keys [2]: [ss_store_sk#20, ss_item_sk#19] -Functions [1]: [sum(UnscaledValue(ss_sales_price#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#21))#26] -Results [2]: [ss_store_sk#20, MakeDecimal(sum(UnscaledValue(ss_sales_price#21))#26,17,2) AS revenue#27] - -(35) HashAggregate [codegen id : 7] -Input [2]: [ss_store_sk#20, revenue#27] -Keys [1]: [ss_store_sk#20] -Functions [1]: [partial_avg(revenue#27)] -Aggregate Attributes [2]: [sum#28, count#29] -Results [3]: [ss_store_sk#20, sum#30, count#31] - -(36) CometColumnarExchange -Input [3]: [ss_store_sk#20, sum#30, count#31] -Arguments: hashpartitioning(ss_store_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(37) CometColumnarToRow [codegen id : 8] -Input [3]: [ss_store_sk#20, sum#30, count#31] - -(38) HashAggregate [codegen id : 8] -Input [3]: [ss_store_sk#20, sum#30, count#31] -Keys [1]: [ss_store_sk#20] -Functions [1]: [avg(revenue#27)] -Aggregate Attributes [1]: [avg(revenue#27)#32] -Results [2]: [ss_store_sk#20, avg(revenue#27)#32 AS ave#33] - -(39) Filter [codegen id : 8] -Input [2]: [ss_store_sk#20, ave#33] -Condition : isnotnull(ave#33) - -(40) BroadcastExchange -Input [2]: [ss_store_sk#20, ave#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [ss_store_sk#20] -Join type: Inner -Join condition: (cast(revenue#12 as decimal(23,7)) <= (0.1 * ave#33)) - -(42) Project [codegen id : 9] -Output [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#18] -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18, ss_store_sk#20, ave#33] - -(43) TakeOrderedAndProject -Input [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#18] -Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#14 ASC NULLS FIRST], [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#18] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometNativeScan parquet spark_catalog.default.date_dim (44) - - -(44) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [2]: [d_date_sk#8, d_month_seq#34] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1176)) AND (d_month_seq#34 <= 1187)) AND isnotnull(d_date_sk#8)) - -(46) CometProject -Input [2]: [d_date_sk#8, d_month_seq#34] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(48) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/simplified.txt deleted file mode 100644 index 2695c9fb89..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - WholeStageCodegen (9) - Project [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] - Project [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [s_store_name,ss_store_sk,ss_item_sk,revenue] - BroadcastHashJoin [s_store_sk,ss_store_sk] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name] - InputAdapter - BroadcastExchange #1 - WholeStageCodegen (3) - Filter [revenue] - HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_store_sk,ss_item_sk] #2 - WholeStageCodegen (2) - HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_brand] [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) - Filter [ave] - HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_store_sk] #6 - WholeStageCodegen (7) - HashAggregate [ss_store_sk,revenue] [sum,count,sum,count] - HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_store_sk,ss_item_sk] #7 - WholeStageCodegen (6) - HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/explain.txt deleted file mode 100644 index 8b38832289..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,264 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometProject (38) - +- CometBroadcastHashJoin (37) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (1) - : : +- CometBroadcastExchange (15) - : : +- CometFilter (14) - : : +- CometHashAggregate (13) - : : +- CometExchange (12) - : : +- CometHashAggregate (11) - : : +- CometProject (10) - : : +- CometBroadcastHashJoin (9) - : : :- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : +- CometBroadcastExchange (8) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (5) - : +- CometBroadcastExchange (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (18) - +- CometBroadcastExchange (36) - +- CometFilter (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometHashAggregate (31) - +- CometExchange (30) - +- CometHashAggregate (29) - +- CometProject (28) - +- CometBroadcastHashJoin (27) - :- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (24) - +- ReusedExchange (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#1, s_store_name#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [s_store_sk#1, s_store_name#2] -Condition : isnotnull(s_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [d_date_sk#8, d_month_seq#9] -Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_month_seq#9 <= 1187)) AND isnotnull(d_date_sk#8)) - -(7) CometProject -Input [2]: [d_date_sk#8, d_month_seq#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(8) CometBroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: [d_date_sk#8] - -(9) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [d_date_sk#8] -Arguments: [ss_sold_date_sk#6], [d_date_sk#8], Inner, BuildRight - -(10) CometProject -Input [5]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] -Arguments: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5], [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] - -(11) CometHashAggregate -Input [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] -Keys [2]: [ss_store_sk#4, ss_item_sk#3] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#5))] - -(12) CometExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(13) CometHashAggregate -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -Keys [2]: [ss_store_sk#4, ss_item_sk#3] -Functions [1]: [sum(UnscaledValue(ss_sales_price#5))] - -(14) CometFilter -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] -Condition : isnotnull(revenue#11) - -(15) CometBroadcastExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] -Arguments: [ss_store_sk#4, ss_item_sk#3, revenue#11] - -(16) CometBroadcastHashJoin -Left output [2]: [s_store_sk#1, s_store_name#2] -Right output [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] -Arguments: [s_store_sk#1], [ss_store_sk#4], Inner, BuildRight - -(17) CometProject -Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] -Arguments: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11], [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(19) CometFilter -Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -Condition : isnotnull(i_item_sk#12) - -(20) CometProject -Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -Arguments: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17], [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#16, 50, true, false, true) AS i_brand#17] - -(21) CometBroadcastExchange -Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(22) CometBroadcastHashJoin -Left output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] -Right output [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: [ss_item_sk#3], [i_item_sk#12], Inner, BuildRight - -(23) CometProject -Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11, i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17], [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#21), dynamicpruningexpression(ss_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] -Condition : isnotnull(ss_store_sk#19) - -(26) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#23] - -(27) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] -Right output [1]: [d_date_sk#23] -Arguments: [ss_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(28) CometProject -Input [5]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21, d_date_sk#23] -Arguments: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20], [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] - -(29) CometHashAggregate -Input [3]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] -Keys [2]: [ss_store_sk#19, ss_item_sk#18] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#20))] - -(30) CometExchange -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#24] -Arguments: hashpartitioning(ss_store_sk#19, ss_item_sk#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(31) CometHashAggregate -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#24] -Keys [2]: [ss_store_sk#19, ss_item_sk#18] -Functions [1]: [sum(UnscaledValue(ss_sales_price#20))] - -(32) CometHashAggregate -Input [2]: [ss_store_sk#19, revenue#25] -Keys [1]: [ss_store_sk#19] -Functions [1]: [partial_avg(revenue#25)] - -(33) CometExchange -Input [3]: [ss_store_sk#19, sum#26, count#27] -Arguments: hashpartitioning(ss_store_sk#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(34) CometHashAggregate -Input [3]: [ss_store_sk#19, sum#26, count#27] -Keys [1]: [ss_store_sk#19] -Functions [1]: [avg(revenue#25)] - -(35) CometFilter -Input [2]: [ss_store_sk#19, ave#28] -Condition : isnotnull(ave#28) - -(36) CometBroadcastExchange -Input [2]: [ss_store_sk#19, ave#28] -Arguments: [ss_store_sk#19, ave#28] - -(37) CometBroadcastHashJoin -Left output [7]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Right output [2]: [ss_store_sk#19, ave#28] -Arguments: [ss_store_sk#4], [ss_store_sk#19], Inner, (cast(revenue#11 as decimal(23,7)) <= (0.1 * ave#28)), BuildRight - -(38) CometProject -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17, ss_store_sk#19, ave#28] -Arguments: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(39) CometTakeOrderedAndProject -Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#2 ASC NULLS FIRST,i_item_desc#13 ASC NULLS FIRST], output=[s_store_name#2,i_item_desc#13,revenue#11,i_current_price#14,i_wholesale_cost#15,i_brand#17]), [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17], 100, 0, [s_store_name#2 ASC NULLS FIRST, i_item_desc#13 ASC NULLS FIRST], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(40) CometColumnarToRow [codegen id : 1] -Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#8, d_month_seq#9] -Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_month_seq#9 <= 1187)) AND isnotnull(d_date_sk#8)) - -(43) CometProject -Input [2]: [d_date_sk#8, d_month_seq#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(45) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/simplified.txt deleted file mode 100644 index cd1c84b6f2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,51 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - CometProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - CometBroadcastHashJoin [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand,ss_store_sk,ave] - CometProject [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometBroadcastHashJoin [s_store_name,ss_store_sk,ss_item_sk,revenue,i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometProject [s_store_name,ss_store_sk,ss_item_sk,revenue] - CometBroadcastHashJoin [s_store_sk,s_store_name,ss_store_sk,ss_item_sk,revenue] - CometFilter [s_store_sk,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name] - CometBroadcastExchange [ss_store_sk,ss_item_sk,revenue] #1 - CometFilter [ss_store_sk,ss_item_sk,revenue] - CometHashAggregate [sum] [ss_store_sk,ss_item_sk,revenue,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_store_sk,ss_item_sk] #2 - CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] - CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] #5 - CometProject [i_brand] [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometBroadcastExchange [ss_store_sk,ave] #6 - CometFilter [ss_store_sk,ave] - CometHashAggregate [sum,count] [ss_store_sk,ave,avg(revenue)] - CometExchange [ss_store_sk] #7 - CometHashAggregate [revenue] [ss_store_sk,sum,count] - CometHashAggregate [ss_item_sk,sum] [ss_store_sk,revenue,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_store_sk,ss_item_sk] #8 - CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] - CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - 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 [native_iceberg_compat] 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/q65/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/explain.txt deleted file mode 100644 index 8b38832289..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/explain.txt +++ /dev/null @@ -1,264 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometProject (38) - +- CometBroadcastHashJoin (37) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (1) - : : +- CometBroadcastExchange (15) - : : +- CometFilter (14) - : : +- CometHashAggregate (13) - : : +- CometExchange (12) - : : +- CometHashAggregate (11) - : : +- CometProject (10) - : : +- CometBroadcastHashJoin (9) - : : :- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : +- CometBroadcastExchange (8) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (5) - : +- CometBroadcastExchange (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (18) - +- CometBroadcastExchange (36) - +- CometFilter (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometHashAggregate (31) - +- CometExchange (30) - +- CometHashAggregate (29) - +- CometProject (28) - +- CometBroadcastHashJoin (27) - :- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (24) - +- ReusedExchange (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#1, s_store_name#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [s_store_sk#1, s_store_name#2] -Condition : isnotnull(s_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [d_date_sk#8, d_month_seq#9] -Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_month_seq#9 <= 1187)) AND isnotnull(d_date_sk#8)) - -(7) CometProject -Input [2]: [d_date_sk#8, d_month_seq#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(8) CometBroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: [d_date_sk#8] - -(9) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [d_date_sk#8] -Arguments: [ss_sold_date_sk#6], [d_date_sk#8], Inner, BuildRight - -(10) CometProject -Input [5]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] -Arguments: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5], [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] - -(11) CometHashAggregate -Input [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] -Keys [2]: [ss_store_sk#4, ss_item_sk#3] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#5))] - -(12) CometExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(13) CometHashAggregate -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -Keys [2]: [ss_store_sk#4, ss_item_sk#3] -Functions [1]: [sum(UnscaledValue(ss_sales_price#5))] - -(14) CometFilter -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] -Condition : isnotnull(revenue#11) - -(15) CometBroadcastExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] -Arguments: [ss_store_sk#4, ss_item_sk#3, revenue#11] - -(16) CometBroadcastHashJoin -Left output [2]: [s_store_sk#1, s_store_name#2] -Right output [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] -Arguments: [s_store_sk#1], [ss_store_sk#4], Inner, BuildRight - -(17) CometProject -Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] -Arguments: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11], [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(19) CometFilter -Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -Condition : isnotnull(i_item_sk#12) - -(20) CometProject -Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -Arguments: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17], [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#16, 50, true, false, true) AS i_brand#17] - -(21) CometBroadcastExchange -Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(22) CometBroadcastHashJoin -Left output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] -Right output [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: [ss_item_sk#3], [i_item_sk#12], Inner, BuildRight - -(23) CometProject -Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11, i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17], [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#21), dynamicpruningexpression(ss_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] -Condition : isnotnull(ss_store_sk#19) - -(26) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#23] - -(27) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] -Right output [1]: [d_date_sk#23] -Arguments: [ss_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(28) CometProject -Input [5]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21, d_date_sk#23] -Arguments: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20], [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] - -(29) CometHashAggregate -Input [3]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] -Keys [2]: [ss_store_sk#19, ss_item_sk#18] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#20))] - -(30) CometExchange -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#24] -Arguments: hashpartitioning(ss_store_sk#19, ss_item_sk#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(31) CometHashAggregate -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#24] -Keys [2]: [ss_store_sk#19, ss_item_sk#18] -Functions [1]: [sum(UnscaledValue(ss_sales_price#20))] - -(32) CometHashAggregate -Input [2]: [ss_store_sk#19, revenue#25] -Keys [1]: [ss_store_sk#19] -Functions [1]: [partial_avg(revenue#25)] - -(33) CometExchange -Input [3]: [ss_store_sk#19, sum#26, count#27] -Arguments: hashpartitioning(ss_store_sk#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(34) CometHashAggregate -Input [3]: [ss_store_sk#19, sum#26, count#27] -Keys [1]: [ss_store_sk#19] -Functions [1]: [avg(revenue#25)] - -(35) CometFilter -Input [2]: [ss_store_sk#19, ave#28] -Condition : isnotnull(ave#28) - -(36) CometBroadcastExchange -Input [2]: [ss_store_sk#19, ave#28] -Arguments: [ss_store_sk#19, ave#28] - -(37) CometBroadcastHashJoin -Left output [7]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Right output [2]: [ss_store_sk#19, ave#28] -Arguments: [ss_store_sk#4], [ss_store_sk#19], Inner, (cast(revenue#11 as decimal(23,7)) <= (0.1 * ave#28)), BuildRight - -(38) CometProject -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17, ss_store_sk#19, ave#28] -Arguments: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(39) CometTakeOrderedAndProject -Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#2 ASC NULLS FIRST,i_item_desc#13 ASC NULLS FIRST], output=[s_store_name#2,i_item_desc#13,revenue#11,i_current_price#14,i_wholesale_cost#15,i_brand#17]), [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17], 100, 0, [s_store_name#2 ASC NULLS FIRST, i_item_desc#13 ASC NULLS FIRST], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(40) CometColumnarToRow [codegen id : 1] -Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#8, d_month_seq#9] -Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_month_seq#9 <= 1187)) AND isnotnull(d_date_sk#8)) - -(43) CometProject -Input [2]: [d_date_sk#8, d_month_seq#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(45) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/extended.txt deleted file mode 100644 index 1fc09bd2e3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/extended.txt +++ /dev/null @@ -1,52 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 45 out of 48 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index cd1c84b6f2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65/simplified.txt +++ /dev/null @@ -1,51 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - CometProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - CometBroadcastHashJoin [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand,ss_store_sk,ave] - CometProject [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometBroadcastHashJoin [s_store_name,ss_store_sk,ss_item_sk,revenue,i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometProject [s_store_name,ss_store_sk,ss_item_sk,revenue] - CometBroadcastHashJoin [s_store_sk,s_store_name,ss_store_sk,ss_item_sk,revenue] - CometFilter [s_store_sk,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name] - CometBroadcastExchange [ss_store_sk,ss_item_sk,revenue] #1 - CometFilter [ss_store_sk,ss_item_sk,revenue] - CometHashAggregate [sum] [ss_store_sk,ss_item_sk,revenue,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_store_sk,ss_item_sk] #2 - CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] - CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] #5 - CometProject [i_brand] [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometBroadcastExchange [ss_store_sk,ave] #6 - CometFilter [ss_store_sk,ave] - CometHashAggregate [sum,count] [ss_store_sk,ave,avg(revenue)] - CometExchange [ss_store_sk] #7 - CometHashAggregate [revenue] [ss_store_sk,sum,count] - CometHashAggregate [ss_item_sk,sum] [ss_store_sk,revenue,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_store_sk,ss_item_sk] #8 - CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] - CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - 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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/explain.txt deleted file mode 100644 index 56c5025634..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/explain.txt +++ /dev/null @@ -1,349 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (56) -+- * HashAggregate (55) - +- * CometColumnarToRow (54) - +- CometColumnarExchange (53) - +- * HashAggregate (52) - +- Union (51) - :- * HashAggregate (31) - : +- * CometColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- * HashAggregate (28) - : +- * Project (27) - : +- * BroadcastHashJoin Inner BuildRight (26) - : :- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.warehouse (4) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (18) - : : +- * CometColumnarToRow (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.time_dim (14) - : +- BroadcastExchange (25) - : +- * CometColumnarToRow (24) - : +- CometProject (23) - : +- CometFilter (22) - : +- CometNativeScan parquet spark_catalog.default.ship_mode (21) - +- * HashAggregate (50) - +- * CometColumnarToRow (49) - +- CometColumnarExchange (48) - +- * HashAggregate (47) - +- * Project (46) - +- * BroadcastHashJoin Inner BuildRight (45) - :- * Project (43) - : +- * BroadcastHashJoin Inner BuildRight (42) - : :- * Project (40) - : : +- * BroadcastHashJoin Inner BuildRight (39) - : : :- * Project (37) - : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : :- * Filter (34) - : : : : +- * ColumnarToRow (33) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (32) - : : : +- ReusedExchange (35) - : : +- ReusedExchange (38) - : +- ReusedExchange (41) - +- ReusedExchange (44) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_ship_mode_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] - -(3) Filter [codegen id : 5] -Input [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Condition : ((isnotnull(ws_warehouse_sk#3) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_ship_mode_sk#2)) - -(4) CometNativeScan parquet spark_catalog.default.warehouse -Output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(5) CometFilter -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Condition : isnotnull(w_warehouse_sk#9) - -(6) CometProject -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Arguments: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15], [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, w_state#14, 2, true, false, true) AS w_state#16, w_country#15] - -(7) CometColumnarToRow [codegen id : 1] -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] - -(8) BroadcastExchange -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_warehouse_sk#3] -Right keys [1]: [w_warehouse_sk#9] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 5] -Output [12]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] - -(11) ReusedExchange [Reuses operator id: 60] -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_sold_date_sk#7] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [13]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Input [15]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_date_sk#17, d_year#18, d_moy#19] - -(14) CometNativeScan parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#20, t_time#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_time), GreaterThanOrEqual(t_time,30838), LessThanOrEqual(t_time,59638), IsNotNull(t_time_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [t_time_sk#20, t_time#21] -Condition : (((isnotnull(t_time#21) AND (t_time#21 >= 30838)) AND (t_time#21 <= 59638)) AND isnotnull(t_time_sk#20)) - -(16) CometProject -Input [2]: [t_time_sk#20, t_time#21] -Arguments: [t_time_sk#20], [t_time_sk#20] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [t_time_sk#20] - -(18) BroadcastExchange -Input [1]: [t_time_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_sold_time_sk#1] -Right keys [1]: [t_time_sk#20] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [12]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, t_time_sk#20] - -(21) CometNativeScan parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, sm_carrier#23, 20, true, false, true) IN (DHL ,BARIAN ) AND isnotnull(sm_ship_mode_sk#22)) - -(23) CometProject -Input [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Arguments: [sm_ship_mode_sk#22], [sm_ship_mode_sk#22] - -(24) CometColumnarToRow [codegen id : 4] -Input [1]: [sm_ship_mode_sk#22] - -(25) BroadcastExchange -Input [1]: [sm_ship_mode_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_ship_mode_sk#2] -Right keys [1]: [sm_ship_mode_sk#22] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 5] -Output [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Input [13]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, sm_ship_mode_sk#22] - -(28) HashAggregate [codegen id : 5] -Input [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] -Functions [24]: [partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] -Aggregate Attributes [48]: [sum#24, isEmpty#25, sum#26, isEmpty#27, sum#28, isEmpty#29, sum#30, isEmpty#31, sum#32, isEmpty#33, sum#34, isEmpty#35, sum#36, isEmpty#37, sum#38, isEmpty#39, sum#40, isEmpty#41, sum#42, isEmpty#43, sum#44, isEmpty#45, sum#46, isEmpty#47, sum#48, isEmpty#49, sum#50, isEmpty#51, sum#52, isEmpty#53, sum#54, isEmpty#55, sum#56, isEmpty#57, sum#58, isEmpty#59, sum#60, isEmpty#61, sum#62, isEmpty#63, sum#64, isEmpty#65, sum#66, isEmpty#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Results [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#72, isEmpty#73, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] - -(29) CometColumnarExchange -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#72, isEmpty#73, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 6] -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#72, isEmpty#73, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] - -(31) HashAggregate [codegen id : 6] -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#72, isEmpty#73, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] -Functions [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#120, sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#121, sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#122, sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#123, sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#124, sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#125, sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#126, sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#127, sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#128, sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#129, sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#130, sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#131, sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#132, sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#133, sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#134, sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#135, sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#136, sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#137, sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#138, sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#139, sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#140, sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#141, sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#142, sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#143] -Results [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, DHL,BARIAN AS ship_carriers#144, d_year#18 AS year#145, sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#120 AS jan_sales#146, sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#121 AS feb_sales#147, sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#122 AS mar_sales#148, sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#123 AS apr_sales#149, sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#124 AS may_sales#150, sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#125 AS jun_sales#151, sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#126 AS jul_sales#152, sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#127 AS aug_sales#153, sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#128 AS sep_sales#154, sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#129 AS oct_sales#155, sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#130 AS nov_sales#156, sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#131 AS dec_sales#157, sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#132 AS jan_net#158, sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#133 AS feb_net#159, sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#134 AS mar_net#160, sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#135 AS apr_net#161, sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#136 AS may_net#162, sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#137 AS jun_net#163, sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#138 AS jul_net#164, sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#139 AS aug_net#165, sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#140 AS sep_net#166, sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#141 AS oct_net#167, sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#142 AS nov_net#168, sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#143 AS dec_net#169] - -(32) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_warehouse_sk#172, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#176), dynamicpruningexpression(cs_sold_date_sk#176 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] -ReadSchema: struct - -(33) ColumnarToRow [codegen id : 11] -Input [7]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_warehouse_sk#172, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176] - -(34) Filter [codegen id : 11] -Input [7]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_warehouse_sk#172, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176] -Condition : ((isnotnull(cs_warehouse_sk#172) AND isnotnull(cs_sold_time_sk#170)) AND isnotnull(cs_ship_mode_sk#171)) - -(35) ReusedExchange [Reuses operator id: 8] -Output [7]: [w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] - -(36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_warehouse_sk#172] -Right keys [1]: [w_warehouse_sk#177] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 11] -Output [12]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] -Input [14]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_warehouse_sk#172, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176, w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] - -(38) ReusedExchange [Reuses operator id: 60] -Output [3]: [d_date_sk#184, d_year#185, d_moy#186] - -(39) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#176] -Right keys [1]: [d_date_sk#184] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 11] -Output [13]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -Input [15]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_date_sk#184, d_year#185, d_moy#186] - -(41) ReusedExchange [Reuses operator id: 18] -Output [1]: [t_time_sk#187] - -(42) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_time_sk#170] -Right keys [1]: [t_time_sk#187] -Join type: Inner -Join condition: None - -(43) Project [codegen id : 11] -Output [12]: [cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -Input [14]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, t_time_sk#187] - -(44) ReusedExchange [Reuses operator id: 25] -Output [1]: [sm_ship_mode_sk#188] - -(45) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_ship_mode_sk#171] -Right keys [1]: [sm_ship_mode_sk#188] -Join type: Inner -Join condition: None - -(46) Project [codegen id : 11] -Output [11]: [cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -Input [13]: [cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, sm_ship_mode_sk#188] - -(47) HashAggregate [codegen id : 11] -Input [11]: [cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] -Functions [24]: [partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)] -Aggregate Attributes [48]: [sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236] -Results [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] - -(48) CometColumnarExchange -Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] -Arguments: hashpartitioning(w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(49) CometColumnarToRow [codegen id : 12] -Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] - -(50) HashAggregate [codegen id : 12] -Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] -Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] -Functions [24]: [sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#286, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#287, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#288, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#289, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#301, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#302, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#303, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#304, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#305, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#306, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#307, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#308] -Results [32]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, DHL,BARIAN AS ship_carriers#309, d_year#185 AS year#310, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#285 AS jan_sales#311, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#286 AS feb_sales#312, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#287 AS mar_sales#313, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#288 AS apr_sales#314, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#289 AS may_sales#315, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#290 AS jun_sales#316, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#291 AS jul_sales#317, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#292 AS aug_sales#318, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#293 AS sep_sales#319, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#294 AS oct_sales#320, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#295 AS nov_sales#321, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#296 AS dec_sales#322, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#297 AS jan_net#323, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#298 AS feb_net#324, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#299 AS mar_net#325, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#300 AS apr_net#326, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#301 AS may_net#327, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#302 AS jun_net#328, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#303 AS jul_net#329, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#304 AS aug_net#330, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#305 AS sep_net#331, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#306 AS oct_net#332, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#307 AS nov_net#333, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#308 AS dec_net#334] - -(51) Union - -(52) HashAggregate [codegen id : 13] -Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, jan_sales#146, feb_sales#147, mar_sales#148, apr_sales#149, may_sales#150, jun_sales#151, jul_sales#152, aug_sales#153, sep_sales#154, oct_sales#155, nov_sales#156, dec_sales#157, jan_net#158, feb_net#159, mar_net#160, apr_net#161, may_net#162, jun_net#163, jul_net#164, aug_net#165, sep_net#166, oct_net#167, nov_net#168, dec_net#169] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145] -Functions [36]: [partial_sum(jan_sales#146), partial_sum(feb_sales#147), partial_sum(mar_sales#148), partial_sum(apr_sales#149), partial_sum(may_sales#150), partial_sum(jun_sales#151), partial_sum(jul_sales#152), partial_sum(aug_sales#153), partial_sum(sep_sales#154), partial_sum(oct_sales#155), partial_sum(nov_sales#156), partial_sum(dec_sales#157), partial_sum((jan_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#157 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#158), partial_sum(feb_net#159), partial_sum(mar_net#160), partial_sum(apr_net#161), partial_sum(may_net#162), partial_sum(jun_net#163), partial_sum(jul_net#164), partial_sum(aug_net#165), partial_sum(sep_net#166), partial_sum(oct_net#167), partial_sum(nov_net#168), partial_sum(dec_net#169)] -Aggregate Attributes [72]: [sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358, sum#359, isEmpty#360, sum#361, isEmpty#362, sum#363, isEmpty#364, sum#365, isEmpty#366, sum#367, isEmpty#368, sum#369, isEmpty#370, sum#371, isEmpty#372, sum#373, isEmpty#374, sum#375, isEmpty#376, sum#377, isEmpty#378, sum#379, isEmpty#380, sum#381, isEmpty#382, sum#383, isEmpty#384, sum#385, isEmpty#386, sum#387, isEmpty#388, sum#389, isEmpty#390, sum#391, isEmpty#392, sum#393, isEmpty#394, sum#395, isEmpty#396, sum#397, isEmpty#398, sum#399, isEmpty#400, sum#401, isEmpty#402, sum#403, isEmpty#404, sum#405, isEmpty#406] -Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] - -(53) CometColumnarExchange -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(54) CometColumnarToRow [codegen id : 14] -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] - -(55) HashAggregate [codegen id : 14] -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145] -Functions [36]: [sum(jan_sales#146), sum(feb_sales#147), sum(mar_sales#148), sum(apr_sales#149), sum(may_sales#150), sum(jun_sales#151), sum(jul_sales#152), sum(aug_sales#153), sum(sep_sales#154), sum(oct_sales#155), sum(nov_sales#156), sum(dec_sales#157), sum((jan_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#157 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#158), sum(feb_net#159), sum(mar_net#160), sum(apr_net#161), sum(may_net#162), sum(jun_net#163), sum(jul_net#164), sum(aug_net#165), sum(sep_net#166), sum(oct_net#167), sum(nov_net#168), sum(dec_net#169)] -Aggregate Attributes [36]: [sum(jan_sales#146)#479, sum(feb_sales#147)#480, sum(mar_sales#148)#481, sum(apr_sales#149)#482, sum(may_sales#150)#483, sum(jun_sales#151)#484, sum(jul_sales#152)#485, sum(aug_sales#153)#486, sum(sep_sales#154)#487, sum(oct_sales#155)#488, sum(nov_sales#156)#489, sum(dec_sales#157)#490, sum((jan_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#491, sum((feb_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#492, sum((mar_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#493, sum((apr_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#494, sum((may_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#495, sum((jun_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#496, sum((jul_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#497, sum((aug_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#498, sum((sep_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#499, sum((oct_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#500, sum((nov_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#501, sum((dec_sales#157 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#502, sum(jan_net#158)#503, sum(feb_net#159)#504, sum(mar_net#160)#505, sum(apr_net#161)#506, sum(may_net#162)#507, sum(jun_net#163)#508, sum(jul_net#164)#509, sum(aug_net#165)#510, sum(sep_net#166)#511, sum(oct_net#167)#512, sum(nov_net#168)#513, sum(dec_net#169)#514] -Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum(jan_sales#146)#479 AS jan_sales#515, sum(feb_sales#147)#480 AS feb_sales#516, sum(mar_sales#148)#481 AS mar_sales#517, sum(apr_sales#149)#482 AS apr_sales#518, sum(may_sales#150)#483 AS may_sales#519, sum(jun_sales#151)#484 AS jun_sales#520, sum(jul_sales#152)#485 AS jul_sales#521, sum(aug_sales#153)#486 AS aug_sales#522, sum(sep_sales#154)#487 AS sep_sales#523, sum(oct_sales#155)#488 AS oct_sales#524, sum(nov_sales#156)#489 AS nov_sales#525, sum(dec_sales#157)#490 AS dec_sales#526, sum((jan_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#491 AS jan_sales_per_sq_foot#527, sum((feb_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#492 AS feb_sales_per_sq_foot#528, sum((mar_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#493 AS mar_sales_per_sq_foot#529, sum((apr_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#494 AS apr_sales_per_sq_foot#530, sum((may_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#495 AS may_sales_per_sq_foot#531, sum((jun_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#496 AS jun_sales_per_sq_foot#532, sum((jul_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#497 AS jul_sales_per_sq_foot#533, sum((aug_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#498 AS aug_sales_per_sq_foot#534, sum((sep_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#499 AS sep_sales_per_sq_foot#535, sum((oct_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#500 AS oct_sales_per_sq_foot#536, sum((nov_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#501 AS nov_sales_per_sq_foot#537, sum((dec_sales#157 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#502 AS dec_sales_per_sq_foot#538, sum(jan_net#158)#503 AS jan_net#539, sum(feb_net#159)#504 AS feb_net#540, sum(mar_net#160)#505 AS mar_net#541, sum(apr_net#161)#506 AS apr_net#542, sum(may_net#162)#507 AS may_net#543, sum(jun_net#163)#508 AS jun_net#544, sum(jul_net#164)#509 AS jul_net#545, sum(aug_net#165)#510 AS aug_net#546, sum(sep_net#166)#511 AS sep_net#547, sum(oct_net#167)#512 AS oct_net#548, sum(nov_net#168)#513 AS nov_net#549, sum(dec_net#169)#514 AS dec_net#550] - -(56) TakeOrderedAndProject -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, jan_sales#515, feb_sales#516, mar_sales#517, apr_sales#518, may_sales#519, jun_sales#520, jul_sales#521, aug_sales#522, sep_sales#523, oct_sales#524, nov_sales#525, dec_sales#526, jan_sales_per_sq_foot#527, feb_sales_per_sq_foot#528, mar_sales_per_sq_foot#529, apr_sales_per_sq_foot#530, may_sales_per_sq_foot#531, jun_sales_per_sq_foot#532, jul_sales_per_sq_foot#533, aug_sales_per_sq_foot#534, sep_sales_per_sq_foot#535, oct_sales_per_sq_foot#536, nov_sales_per_sq_foot#537, dec_sales_per_sq_foot#538, jan_net#539, feb_net#540, mar_net#541, apr_net#542, may_net#543, jun_net#544, jul_net#545, aug_net#546, sep_net#547, oct_net#548, nov_net#549, dec_net#550] -Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, jan_sales#515, feb_sales#516, mar_sales#517, apr_sales#518, may_sales#519, jun_sales#520, jul_sales#521, aug_sales#522, sep_sales#523, oct_sales#524, nov_sales#525, dec_sales#526, jan_sales_per_sq_foot#527, feb_sales_per_sq_foot#528, mar_sales_per_sq_foot#529, apr_sales_per_sq_foot#530, may_sales_per_sq_foot#531, jun_sales_per_sq_foot#532, jul_sales_per_sq_foot#533, aug_sales_per_sq_foot#534, sep_sales_per_sq_foot#535, oct_sales_per_sq_foot#536, nov_sales_per_sq_foot#537, dec_sales_per_sq_foot#538, jan_net#539, feb_net#540, mar_net#541, apr_net#542, may_net#543, jun_net#544, jul_net#545, aug_net#546, sep_net#547, oct_net#548, nov_net#549, dec_net#550] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (60) -+- * CometColumnarToRow (59) - +- CometFilter (58) - +- CometNativeScan parquet spark_catalog.default.date_dim (57) - - -(57) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(59) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] - -(60) BroadcastExchange -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#176 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/simplified.txt deleted file mode 100644 index 3a1f053d60..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - WholeStageCodegen (14) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,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,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum((jan_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((feb_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((mar_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((apr_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((may_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jun_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jul_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((aug_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((sep_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((oct_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((nov_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((dec_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net),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_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,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,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - WholeStageCodegen (13) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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,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,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] - InputAdapter - Union - WholeStageCodegen (6) - 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 (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - WholeStageCodegen (5) - HashAggregate [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] [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,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] - Project [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] - BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] - Project [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] - BroadcastHashJoin [ws_sold_time_sk,t_time_sk] - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - Filter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [w_state] [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_time_sk,t_time] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [sm_ship_mode_sk] - CometFilter [sm_ship_mode_sk,sm_carrier] - CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] - WholeStageCodegen (12) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #7 - WholeStageCodegen (11) - HashAggregate [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] [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,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] - Project [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] - BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] - Project [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] - BroadcastHashJoin [cs_sold_time_sk,t_time_sk] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - Filter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #3 - InputAdapter - ReusedExchange [t_time_sk] #5 - InputAdapter - ReusedExchange [sm_ship_mode_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/explain.txt deleted file mode 100644 index 7599e9ecd3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,317 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (51) -+- CometTakeOrderedAndProject (50) - +- CometHashAggregate (49) - +- CometExchange (48) - +- CometHashAggregate (47) - +- CometUnion (46) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (14) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode (20) - +- CometHashAggregate (45) - +- CometExchange (44) - +- CometHashAggregate (43) - +- CometProject (42) - +- CometBroadcastHashJoin (41) - :- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometFilter (30) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_ship_mode_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Condition : ((isnotnull(ws_warehouse_sk#3) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_ship_mode_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Condition : isnotnull(w_warehouse_sk#9) - -(5) CometProject -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Arguments: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15], [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, w_state#14, 2, true, false, true) AS w_state#16, w_country#15] - -(6) CometBroadcastExchange -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Arguments: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] - -(7) CometBroadcastHashJoin -Left output [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Right output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Arguments: [ws_warehouse_sk#3], [w_warehouse_sk#9], Inner, BuildRight - -(8) CometProject -Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Arguments: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15], [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [d_date_sk#17, d_year#18, d_moy#19] - -(12) CometBroadcastHashJoin -Left output [12]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Right output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [ws_sold_date_sk#7], [d_date_sk#17], Inner, BuildRight - -(13) CometProject -Input [15]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_date_sk#17, d_year#18, d_moy#19] -Arguments: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19], [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#20, t_time#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_time), GreaterThanOrEqual(t_time,30838), LessThanOrEqual(t_time,59638), IsNotNull(t_time_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [t_time_sk#20, t_time#21] -Condition : (((isnotnull(t_time#21) AND (t_time#21 >= 30838)) AND (t_time#21 <= 59638)) AND isnotnull(t_time_sk#20)) - -(16) CometProject -Input [2]: [t_time_sk#20, t_time#21] -Arguments: [t_time_sk#20], [t_time_sk#20] - -(17) CometBroadcastExchange -Input [1]: [t_time_sk#20] -Arguments: [t_time_sk#20] - -(18) CometBroadcastHashJoin -Left output [13]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Right output [1]: [t_time_sk#20] -Arguments: [ws_sold_time_sk#1], [t_time_sk#20], Inner, BuildRight - -(19) CometProject -Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, t_time_sk#20] -Arguments: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19], [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, sm_carrier#23, 20, true, false, true) IN (DHL ,BARIAN ) AND isnotnull(sm_ship_mode_sk#22)) - -(22) CometProject -Input [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Arguments: [sm_ship_mode_sk#22], [sm_ship_mode_sk#22] - -(23) CometBroadcastExchange -Input [1]: [sm_ship_mode_sk#22] -Arguments: [sm_ship_mode_sk#22] - -(24) CometBroadcastHashJoin -Left output [12]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Right output [1]: [sm_ship_mode_sk#22] -Arguments: [ws_ship_mode_sk#2], [sm_ship_mode_sk#22], Inner, BuildRight - -(25) CometProject -Input [13]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, sm_ship_mode_sk#22] -Arguments: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19], [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] - -(26) CometHashAggregate -Input [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] -Functions [24]: [partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] - -(27) CometExchange -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#24, isEmpty#25, sum#26, isEmpty#27, sum#28, isEmpty#29, sum#30, isEmpty#31, sum#32, isEmpty#33, sum#34, isEmpty#35, sum#36, isEmpty#37, sum#38, isEmpty#39, sum#40, isEmpty#41, sum#42, isEmpty#43, sum#44, isEmpty#45, sum#46, isEmpty#47, sum#48, isEmpty#49, sum#50, isEmpty#51, sum#52, isEmpty#53, sum#54, isEmpty#55, sum#56, isEmpty#57, sum#58, isEmpty#59, sum#60, isEmpty#61, sum#62, isEmpty#63, sum#64, isEmpty#65, sum#66, isEmpty#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#24, isEmpty#25, sum#26, isEmpty#27, sum#28, isEmpty#29, sum#30, isEmpty#31, sum#32, isEmpty#33, sum#34, isEmpty#35, sum#36, isEmpty#37, sum#38, isEmpty#39, sum#40, isEmpty#41, sum#42, isEmpty#43, sum#44, isEmpty#45, sum#46, isEmpty#47, sum#48, isEmpty#49, sum#50, isEmpty#51, sum#52, isEmpty#53, sum#54, isEmpty#55, sum#56, isEmpty#57, sum#58, isEmpty#59, sum#60, isEmpty#61, sum#62, isEmpty#63, sum#64, isEmpty#65, sum#66, isEmpty#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] -Functions [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#78), dynamicpruningexpression(cs_sold_date_sk#78 IN dynamicpruning#79)] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] -ReadSchema: struct - -(30) CometFilter -Input [7]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78] -Condition : ((isnotnull(cs_warehouse_sk#74) AND isnotnull(cs_sold_time_sk#72)) AND isnotnull(cs_ship_mode_sk#73)) - -(31) ReusedExchange [Reuses operator id: 6] -Output [7]: [w_warehouse_sk#80, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] - -(32) CometBroadcastHashJoin -Left output [7]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78] -Right output [7]: [w_warehouse_sk#80, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] -Arguments: [cs_warehouse_sk#74], [w_warehouse_sk#80], Inner, BuildRight - -(33) CometProject -Input [14]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_sk#80, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] -Arguments: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86], [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] - -(34) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#87, d_year#88, d_moy#89] - -(35) CometBroadcastHashJoin -Left output [12]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] -Right output [3]: [d_date_sk#87, d_year#88, d_moy#89] -Arguments: [cs_sold_date_sk#78], [d_date_sk#87], Inner, BuildRight - -(36) CometProject -Input [15]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_date_sk#87, d_year#88, d_moy#89] -Arguments: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89], [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] - -(37) ReusedExchange [Reuses operator id: 17] -Output [1]: [t_time_sk#90] - -(38) CometBroadcastHashJoin -Left output [13]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] -Right output [1]: [t_time_sk#90] -Arguments: [cs_sold_time_sk#72], [t_time_sk#90], Inner, BuildRight - -(39) CometProject -Input [14]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89, t_time_sk#90] -Arguments: [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89], [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] - -(40) ReusedExchange [Reuses operator id: 23] -Output [1]: [sm_ship_mode_sk#91] - -(41) CometBroadcastHashJoin -Left output [12]: [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] -Right output [1]: [sm_ship_mode_sk#91] -Arguments: [cs_ship_mode_sk#73], [sm_ship_mode_sk#91], Inner, BuildRight - -(42) CometProject -Input [13]: [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89, sm_ship_mode_sk#91] -Arguments: [cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89], [cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] - -(43) CometHashAggregate -Input [11]: [cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] -Keys [7]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88] -Functions [24]: [partial_sum(CASE WHEN (d_moy#89 = 1) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 2) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 3) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 4) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 5) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 6) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 7) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 8) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 9) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 10) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 11) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 12) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 1) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 2) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 3) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 4) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 5) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 6) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 7) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 8) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 9) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 10) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 11) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 12) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END)] - -(44) CometExchange -Input [55]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Arguments: hashpartitioning(w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(45) CometHashAggregate -Input [55]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Keys [7]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88] -Functions [24]: [sum(CASE WHEN (d_moy#89 = 1) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 2) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 3) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 4) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 5) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 6) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 7) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 8) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 9) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 10) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 11) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 12) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 1) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 2) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 3) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 4) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 5) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 6) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 7) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 8) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 9) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 10) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 11) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 12) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END)] - -(46) CometUnion -Child 0 Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#142, feb_sales#143, mar_sales#144, apr_sales#145, may_sales#146, jun_sales#147, jul_sales#148, aug_sales#149, sep_sales#150, oct_sales#151, nov_sales#152, dec_sales#153, jan_net#154, feb_net#155, mar_net#156, apr_net#157, may_net#158, jun_net#159, jul_net#160, aug_net#161, sep_net#162, oct_net#163, nov_net#164, dec_net#165] -Child 1 Input [32]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, ship_carriers#166, year#167, jan_sales#168, feb_sales#169, mar_sales#170, apr_sales#171, may_sales#172, jun_sales#173, jul_sales#174, aug_sales#175, sep_sales#176, oct_sales#177, nov_sales#178, dec_sales#179, jan_net#180, feb_net#181, mar_net#182, apr_net#183, may_net#184, jun_net#185, jul_net#186, aug_net#187, sep_net#188, oct_net#189, nov_net#190, dec_net#191] - -(47) CometHashAggregate -Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#142, feb_sales#143, mar_sales#144, apr_sales#145, may_sales#146, jun_sales#147, jul_sales#148, aug_sales#149, sep_sales#150, oct_sales#151, nov_sales#152, dec_sales#153, jan_net#154, feb_net#155, mar_net#156, apr_net#157, may_net#158, jun_net#159, jul_net#160, aug_net#161, sep_net#162, oct_net#163, nov_net#164, dec_net#165] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141] -Functions [36]: [partial_sum(jan_sales#142), partial_sum(feb_sales#143), partial_sum(mar_sales#144), partial_sum(apr_sales#145), partial_sum(may_sales#146), partial_sum(jun_sales#147), partial_sum(jul_sales#148), partial_sum(aug_sales#149), partial_sum(sep_sales#150), partial_sum(oct_sales#151), partial_sum(nov_sales#152), partial_sum(dec_sales#153), partial_sum((jan_sales#142 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#143 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#144 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#154), partial_sum(feb_net#155), partial_sum(mar_net#156), partial_sum(apr_net#157), partial_sum(may_net#158), partial_sum(jun_net#159), partial_sum(jul_net#160), partial_sum(aug_net#161), partial_sum(sep_net#162), partial_sum(oct_net#163), partial_sum(nov_net#164), partial_sum(dec_net#165)] - -(48) CometExchange -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211, sum#212, isEmpty#213, sum#214, isEmpty#215, sum#216, isEmpty#217, sum#218, isEmpty#219, sum#220, isEmpty#221, sum#222, isEmpty#223, sum#224, isEmpty#225, sum#226, isEmpty#227, sum#228, isEmpty#229, sum#230, isEmpty#231, sum#232, isEmpty#233, sum#234, isEmpty#235, sum#236, isEmpty#237, sum#238, isEmpty#239, sum#240, isEmpty#241, sum#242, isEmpty#243, sum#244, isEmpty#245, sum#246, isEmpty#247, sum#248, isEmpty#249, sum#250, isEmpty#251, sum#252, isEmpty#253, sum#254, isEmpty#255, sum#256, isEmpty#257, sum#258, isEmpty#259, sum#260, isEmpty#261, sum#262, isEmpty#263] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211, sum#212, isEmpty#213, sum#214, isEmpty#215, sum#216, isEmpty#217, sum#218, isEmpty#219, sum#220, isEmpty#221, sum#222, isEmpty#223, sum#224, isEmpty#225, sum#226, isEmpty#227, sum#228, isEmpty#229, sum#230, isEmpty#231, sum#232, isEmpty#233, sum#234, isEmpty#235, sum#236, isEmpty#237, sum#238, isEmpty#239, sum#240, isEmpty#241, sum#242, isEmpty#243, sum#244, isEmpty#245, sum#246, isEmpty#247, sum#248, isEmpty#249, sum#250, isEmpty#251, sum#252, isEmpty#253, sum#254, isEmpty#255, sum#256, isEmpty#257, sum#258, isEmpty#259, sum#260, isEmpty#261, sum#262, isEmpty#263] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141] -Functions [36]: [sum(jan_sales#142), sum(feb_sales#143), sum(mar_sales#144), sum(apr_sales#145), sum(may_sales#146), sum(jun_sales#147), sum(jul_sales#148), sum(aug_sales#149), sum(sep_sales#150), sum(oct_sales#151), sum(nov_sales#152), sum(dec_sales#153), sum((jan_sales#142 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#143 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#144 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#154), sum(feb_net#155), sum(mar_net#156), sum(apr_net#157), sum(may_net#158), sum(jun_net#159), sum(jul_net#160), sum(aug_net#161), sum(sep_net#162), sum(oct_net#163), sum(nov_net#164), sum(dec_net#165)] - -(50) CometTakeOrderedAndProject -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#10 ASC NULLS FIRST], output=[w_warehouse_name#10,w_warehouse_sq_ft#11,w_city#12,w_county#13,w_state#16,w_country#15,ship_carriers#140,year#141,jan_sales#264,feb_sales#265,mar_sales#266,apr_sales#267,may_sales#268,jun_sales#269,jul_sales#270,aug_sales#271,sep_sales#272,oct_sales#273,nov_sales#274,dec_sales#275,jan_sales_per_sq_foot#276,feb_sales_per_sq_foot#277,mar_sales_per_sq_foot#278,apr_sales_per_sq_foot#279,may_sales_per_sq_foot#280,jun_sales_per_sq_foot#281,jul_sales_per_sq_foot#282,aug_sales_per_sq_foot#283,sep_sales_per_sq_foot#284,oct_sales_per_sq_foot#285,nov_sales_per_sq_foot#286,dec_sales_per_sq_foot#287,jan_net#288,feb_net#289,mar_net#290,apr_net#291,may_net#292,jun_net#293,jul_net#294,aug_net#295,sep_net#296,oct_net#297,nov_net#298,dec_net#299]), [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299], 100, 0, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299] - -(51) CometColumnarToRow [codegen id : 1] -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometFilter (53) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) - - -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(54) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] - -(55) BroadcastExchange -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#78 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/simplified.txt deleted file mode 100644 index bc6bde24b2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - CometHashAggregate [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,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,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(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum((jan_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((feb_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((mar_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((apr_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((may_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jun_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jul_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((aug_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((sep_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((oct_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((nov_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((dec_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net)] - CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - CometHashAggregate [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] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,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,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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] - CometHashAggregate [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] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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,d_year,sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END)] - CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - CometHashAggregate [d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [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] - 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,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,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_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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 - CometProject [w_state] [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [t_time_sk] #6 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_time] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_time] - CometBroadcastExchange [sm_ship_mode_sk] #7 - CometProject [sm_ship_mode_sk] - CometFilter [sm_ship_mode_sk,sm_carrier] - CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] - CometHashAggregate [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] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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,d_year,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)] - CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #8 - CometHashAggregate [d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [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] - 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,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,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_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_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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year,d_moy] #5 - ReusedExchange [t_time_sk] #6 - ReusedExchange [sm_ship_mode_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/explain.txt deleted file mode 100644 index 7599e9ecd3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/explain.txt +++ /dev/null @@ -1,317 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (51) -+- CometTakeOrderedAndProject (50) - +- CometHashAggregate (49) - +- CometExchange (48) - +- CometHashAggregate (47) - +- CometUnion (46) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (14) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode (20) - +- CometHashAggregate (45) - +- CometExchange (44) - +- CometHashAggregate (43) - +- CometProject (42) - +- CometBroadcastHashJoin (41) - :- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometFilter (30) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_ship_mode_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Condition : ((isnotnull(ws_warehouse_sk#3) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_ship_mode_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Condition : isnotnull(w_warehouse_sk#9) - -(5) CometProject -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Arguments: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15], [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, w_state#14, 2, true, false, true) AS w_state#16, w_country#15] - -(6) CometBroadcastExchange -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Arguments: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] - -(7) CometBroadcastHashJoin -Left output [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Right output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Arguments: [ws_warehouse_sk#3], [w_warehouse_sk#9], Inner, BuildRight - -(8) CometProject -Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Arguments: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15], [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [d_date_sk#17, d_year#18, d_moy#19] - -(12) CometBroadcastHashJoin -Left output [12]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Right output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [ws_sold_date_sk#7], [d_date_sk#17], Inner, BuildRight - -(13) CometProject -Input [15]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_date_sk#17, d_year#18, d_moy#19] -Arguments: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19], [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#20, t_time#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_time), GreaterThanOrEqual(t_time,30838), LessThanOrEqual(t_time,59638), IsNotNull(t_time_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [t_time_sk#20, t_time#21] -Condition : (((isnotnull(t_time#21) AND (t_time#21 >= 30838)) AND (t_time#21 <= 59638)) AND isnotnull(t_time_sk#20)) - -(16) CometProject -Input [2]: [t_time_sk#20, t_time#21] -Arguments: [t_time_sk#20], [t_time_sk#20] - -(17) CometBroadcastExchange -Input [1]: [t_time_sk#20] -Arguments: [t_time_sk#20] - -(18) CometBroadcastHashJoin -Left output [13]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Right output [1]: [t_time_sk#20] -Arguments: [ws_sold_time_sk#1], [t_time_sk#20], Inner, BuildRight - -(19) CometProject -Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, t_time_sk#20] -Arguments: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19], [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, sm_carrier#23, 20, true, false, true) IN (DHL ,BARIAN ) AND isnotnull(sm_ship_mode_sk#22)) - -(22) CometProject -Input [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Arguments: [sm_ship_mode_sk#22], [sm_ship_mode_sk#22] - -(23) CometBroadcastExchange -Input [1]: [sm_ship_mode_sk#22] -Arguments: [sm_ship_mode_sk#22] - -(24) CometBroadcastHashJoin -Left output [12]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Right output [1]: [sm_ship_mode_sk#22] -Arguments: [ws_ship_mode_sk#2], [sm_ship_mode_sk#22], Inner, BuildRight - -(25) CometProject -Input [13]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, sm_ship_mode_sk#22] -Arguments: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19], [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] - -(26) CometHashAggregate -Input [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] -Functions [24]: [partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] - -(27) CometExchange -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#24, isEmpty#25, sum#26, isEmpty#27, sum#28, isEmpty#29, sum#30, isEmpty#31, sum#32, isEmpty#33, sum#34, isEmpty#35, sum#36, isEmpty#37, sum#38, isEmpty#39, sum#40, isEmpty#41, sum#42, isEmpty#43, sum#44, isEmpty#45, sum#46, isEmpty#47, sum#48, isEmpty#49, sum#50, isEmpty#51, sum#52, isEmpty#53, sum#54, isEmpty#55, sum#56, isEmpty#57, sum#58, isEmpty#59, sum#60, isEmpty#61, sum#62, isEmpty#63, sum#64, isEmpty#65, sum#66, isEmpty#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#24, isEmpty#25, sum#26, isEmpty#27, sum#28, isEmpty#29, sum#30, isEmpty#31, sum#32, isEmpty#33, sum#34, isEmpty#35, sum#36, isEmpty#37, sum#38, isEmpty#39, sum#40, isEmpty#41, sum#42, isEmpty#43, sum#44, isEmpty#45, sum#46, isEmpty#47, sum#48, isEmpty#49, sum#50, isEmpty#51, sum#52, isEmpty#53, sum#54, isEmpty#55, sum#56, isEmpty#57, sum#58, isEmpty#59, sum#60, isEmpty#61, sum#62, isEmpty#63, sum#64, isEmpty#65, sum#66, isEmpty#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] -Functions [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#78), dynamicpruningexpression(cs_sold_date_sk#78 IN dynamicpruning#79)] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] -ReadSchema: struct - -(30) CometFilter -Input [7]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78] -Condition : ((isnotnull(cs_warehouse_sk#74) AND isnotnull(cs_sold_time_sk#72)) AND isnotnull(cs_ship_mode_sk#73)) - -(31) ReusedExchange [Reuses operator id: 6] -Output [7]: [w_warehouse_sk#80, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] - -(32) CometBroadcastHashJoin -Left output [7]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78] -Right output [7]: [w_warehouse_sk#80, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] -Arguments: [cs_warehouse_sk#74], [w_warehouse_sk#80], Inner, BuildRight - -(33) CometProject -Input [14]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_sk#80, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] -Arguments: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86], [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] - -(34) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#87, d_year#88, d_moy#89] - -(35) CometBroadcastHashJoin -Left output [12]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] -Right output [3]: [d_date_sk#87, d_year#88, d_moy#89] -Arguments: [cs_sold_date_sk#78], [d_date_sk#87], Inner, BuildRight - -(36) CometProject -Input [15]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_date_sk#87, d_year#88, d_moy#89] -Arguments: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89], [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] - -(37) ReusedExchange [Reuses operator id: 17] -Output [1]: [t_time_sk#90] - -(38) CometBroadcastHashJoin -Left output [13]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] -Right output [1]: [t_time_sk#90] -Arguments: [cs_sold_time_sk#72], [t_time_sk#90], Inner, BuildRight - -(39) CometProject -Input [14]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89, t_time_sk#90] -Arguments: [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89], [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] - -(40) ReusedExchange [Reuses operator id: 23] -Output [1]: [sm_ship_mode_sk#91] - -(41) CometBroadcastHashJoin -Left output [12]: [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] -Right output [1]: [sm_ship_mode_sk#91] -Arguments: [cs_ship_mode_sk#73], [sm_ship_mode_sk#91], Inner, BuildRight - -(42) CometProject -Input [13]: [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89, sm_ship_mode_sk#91] -Arguments: [cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89], [cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] - -(43) CometHashAggregate -Input [11]: [cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] -Keys [7]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88] -Functions [24]: [partial_sum(CASE WHEN (d_moy#89 = 1) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 2) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 3) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 4) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 5) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 6) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 7) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 8) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 9) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 10) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 11) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 12) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 1) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 2) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 3) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 4) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 5) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 6) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 7) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 8) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 9) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 10) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 11) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 12) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END)] - -(44) CometExchange -Input [55]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Arguments: hashpartitioning(w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(45) CometHashAggregate -Input [55]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Keys [7]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88] -Functions [24]: [sum(CASE WHEN (d_moy#89 = 1) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 2) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 3) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 4) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 5) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 6) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 7) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 8) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 9) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 10) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 11) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 12) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 1) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 2) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 3) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 4) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 5) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 6) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 7) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 8) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 9) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 10) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 11) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 12) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END)] - -(46) CometUnion -Child 0 Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#142, feb_sales#143, mar_sales#144, apr_sales#145, may_sales#146, jun_sales#147, jul_sales#148, aug_sales#149, sep_sales#150, oct_sales#151, nov_sales#152, dec_sales#153, jan_net#154, feb_net#155, mar_net#156, apr_net#157, may_net#158, jun_net#159, jul_net#160, aug_net#161, sep_net#162, oct_net#163, nov_net#164, dec_net#165] -Child 1 Input [32]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, ship_carriers#166, year#167, jan_sales#168, feb_sales#169, mar_sales#170, apr_sales#171, may_sales#172, jun_sales#173, jul_sales#174, aug_sales#175, sep_sales#176, oct_sales#177, nov_sales#178, dec_sales#179, jan_net#180, feb_net#181, mar_net#182, apr_net#183, may_net#184, jun_net#185, jul_net#186, aug_net#187, sep_net#188, oct_net#189, nov_net#190, dec_net#191] - -(47) CometHashAggregate -Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#142, feb_sales#143, mar_sales#144, apr_sales#145, may_sales#146, jun_sales#147, jul_sales#148, aug_sales#149, sep_sales#150, oct_sales#151, nov_sales#152, dec_sales#153, jan_net#154, feb_net#155, mar_net#156, apr_net#157, may_net#158, jun_net#159, jul_net#160, aug_net#161, sep_net#162, oct_net#163, nov_net#164, dec_net#165] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141] -Functions [36]: [partial_sum(jan_sales#142), partial_sum(feb_sales#143), partial_sum(mar_sales#144), partial_sum(apr_sales#145), partial_sum(may_sales#146), partial_sum(jun_sales#147), partial_sum(jul_sales#148), partial_sum(aug_sales#149), partial_sum(sep_sales#150), partial_sum(oct_sales#151), partial_sum(nov_sales#152), partial_sum(dec_sales#153), partial_sum((jan_sales#142 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#143 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#144 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#154), partial_sum(feb_net#155), partial_sum(mar_net#156), partial_sum(apr_net#157), partial_sum(may_net#158), partial_sum(jun_net#159), partial_sum(jul_net#160), partial_sum(aug_net#161), partial_sum(sep_net#162), partial_sum(oct_net#163), partial_sum(nov_net#164), partial_sum(dec_net#165)] - -(48) CometExchange -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211, sum#212, isEmpty#213, sum#214, isEmpty#215, sum#216, isEmpty#217, sum#218, isEmpty#219, sum#220, isEmpty#221, sum#222, isEmpty#223, sum#224, isEmpty#225, sum#226, isEmpty#227, sum#228, isEmpty#229, sum#230, isEmpty#231, sum#232, isEmpty#233, sum#234, isEmpty#235, sum#236, isEmpty#237, sum#238, isEmpty#239, sum#240, isEmpty#241, sum#242, isEmpty#243, sum#244, isEmpty#245, sum#246, isEmpty#247, sum#248, isEmpty#249, sum#250, isEmpty#251, sum#252, isEmpty#253, sum#254, isEmpty#255, sum#256, isEmpty#257, sum#258, isEmpty#259, sum#260, isEmpty#261, sum#262, isEmpty#263] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211, sum#212, isEmpty#213, sum#214, isEmpty#215, sum#216, isEmpty#217, sum#218, isEmpty#219, sum#220, isEmpty#221, sum#222, isEmpty#223, sum#224, isEmpty#225, sum#226, isEmpty#227, sum#228, isEmpty#229, sum#230, isEmpty#231, sum#232, isEmpty#233, sum#234, isEmpty#235, sum#236, isEmpty#237, sum#238, isEmpty#239, sum#240, isEmpty#241, sum#242, isEmpty#243, sum#244, isEmpty#245, sum#246, isEmpty#247, sum#248, isEmpty#249, sum#250, isEmpty#251, sum#252, isEmpty#253, sum#254, isEmpty#255, sum#256, isEmpty#257, sum#258, isEmpty#259, sum#260, isEmpty#261, sum#262, isEmpty#263] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141] -Functions [36]: [sum(jan_sales#142), sum(feb_sales#143), sum(mar_sales#144), sum(apr_sales#145), sum(may_sales#146), sum(jun_sales#147), sum(jul_sales#148), sum(aug_sales#149), sum(sep_sales#150), sum(oct_sales#151), sum(nov_sales#152), sum(dec_sales#153), sum((jan_sales#142 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#143 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#144 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#154), sum(feb_net#155), sum(mar_net#156), sum(apr_net#157), sum(may_net#158), sum(jun_net#159), sum(jul_net#160), sum(aug_net#161), sum(sep_net#162), sum(oct_net#163), sum(nov_net#164), sum(dec_net#165)] - -(50) CometTakeOrderedAndProject -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#10 ASC NULLS FIRST], output=[w_warehouse_name#10,w_warehouse_sq_ft#11,w_city#12,w_county#13,w_state#16,w_country#15,ship_carriers#140,year#141,jan_sales#264,feb_sales#265,mar_sales#266,apr_sales#267,may_sales#268,jun_sales#269,jul_sales#270,aug_sales#271,sep_sales#272,oct_sales#273,nov_sales#274,dec_sales#275,jan_sales_per_sq_foot#276,feb_sales_per_sq_foot#277,mar_sales_per_sq_foot#278,apr_sales_per_sq_foot#279,may_sales_per_sq_foot#280,jun_sales_per_sq_foot#281,jul_sales_per_sq_foot#282,aug_sales_per_sq_foot#283,sep_sales_per_sq_foot#284,oct_sales_per_sq_foot#285,nov_sales_per_sq_foot#286,dec_sales_per_sq_foot#287,jan_net#288,feb_net#289,mar_net#290,apr_net#291,may_net#292,jun_net#293,jul_net#294,aug_net#295,sep_net#296,oct_net#297,nov_net#298,dec_net#299]), [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299], 100, 0, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299] - -(51) CometColumnarToRow [codegen id : 1] -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometFilter (53) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) - - -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(54) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] - -(55) BroadcastExchange -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#78 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/extended.txt deleted file mode 100644 index ca39e001b0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/extended.txt +++ /dev/null @@ -1,70 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - -Comet accelerated 63 out of 66 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index bc6bde24b2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - CometHashAggregate [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,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,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(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum((jan_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((feb_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((mar_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((apr_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((may_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jun_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jul_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((aug_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((sep_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((oct_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((nov_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((dec_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net)] - CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - CometHashAggregate [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] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,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,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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] - CometHashAggregate [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] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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,d_year,sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END)] - CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - CometHashAggregate [d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [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] - 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,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,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_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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 - CometProject [w_state] [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [t_time_sk] #6 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_time] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_time] - CometBroadcastExchange [sm_ship_mode_sk] #7 - CometProject [sm_ship_mode_sk] - CometFilter [sm_ship_mode_sk,sm_carrier] - CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] - CometHashAggregate [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] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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,d_year,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)] - CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #8 - CometHashAggregate [d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [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] - 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,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,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_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_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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year,d_moy] #5 - ReusedExchange [t_time_sk] #6 - ReusedExchange [sm_ship_mode_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/explain.txt deleted file mode 100644 index 547ca84576..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (34) -+- * Filter (33) - +- Window (32) - +- WindowGroupLimit (31) - +- * CometColumnarToRow (30) - +- CometSort (29) - +- CometColumnarExchange (28) - +- WindowGroupLimit (27) - +- * Sort (26) - +- * HashAggregate (25) - +- * CometColumnarToRow (24) - +- CometColumnarExchange (23) - +- * HashAggregate (22) - +- * Expand (21) - +- * Project (20) - +- * BroadcastHashJoin Inner BuildRight (19) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) - : +- CometProject (9) - : +- CometFilter (8) - : +- CometNativeScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (18) - +- * CometColumnarToRow (17) - +- CometProject (16) - +- CometFilter (15) - +- CometNativeScan parquet spark_catalog.default.item (14) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(4) ReusedExchange [Reuses operator id: 39] -Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_store_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [s_store_sk#11, s_store_id#12] -Condition : isnotnull(s_store_sk#11) - -(9) CometProject -Input [2]: [s_store_sk#11, s_store_id#12] -Arguments: [s_store_sk#11, s_store_id#13], [s_store_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#12, 16, true, false, true) AS s_store_id#13] - -(10) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#11, s_store_id#13] - -(11) BroadcastExchange -Input [2]: [s_store_sk#11, s_store_id#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#13] - -(14) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -Condition : isnotnull(i_item_sk#14) - -(16) CometProject -Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -Arguments: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22], [i_item_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#15, 50, true, false, true) AS i_brand#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#16, 50, true, false, true) AS i_class#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#17, 50, true, false, true) AS i_category#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#18, 50, true, false, true) AS i_product_name#22] - -(17) CometColumnarToRow [codegen id : 3] -Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] - -(18) BroadcastExchange -Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#14] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [10]: [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] - -(21) Expand [codegen id : 4] -Input [10]: [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] -Arguments: [[ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, 0], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, null, 1], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, null, null, 3], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, null, null, null, 7], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, null, null, null, null, 15], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, null, null, null, null, null, 31], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, null, null, null, null, null, null, 63], [ss_quantity#3, ss_sales_price#4, i_category#21, null, null, null, null, null, null, null, 127], [ss_quantity#3, ss_sales_price#4, null, null, null, null, null, null, null, null, 255]], [ss_quantity#3, ss_sales_price#4, i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] - -(22) HashAggregate [codegen id : 4] -Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] -Keys [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] -Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [2]: [sum#32, isEmpty#33] -Results [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, sum#34, isEmpty#35] - -(23) CometColumnarExchange -Input [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, sum#34, isEmpty#35] -Arguments: hashpartitioning(i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometColumnarToRow [codegen id : 5] -Input [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, sum#34, isEmpty#35] - -(25) HashAggregate [codegen id : 5] -Input [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, sum#34, isEmpty#35] -Keys [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] -Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#36] -Results [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#36 AS sumsales#37] - -(26) Sort [codegen id : 5] -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] -Arguments: [i_category#23 ASC NULLS FIRST, sumsales#37 DESC NULLS LAST], false, 0 - -(27) WindowGroupLimit -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] -Arguments: [i_category#23], [sumsales#37 DESC NULLS LAST], rank(sumsales#37), 100, Partial - -(28) CometColumnarExchange -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] -Arguments: hashpartitioning(i_category#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(29) CometSort -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] -Arguments: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37], [i_category#23 ASC NULLS FIRST, sumsales#37 DESC NULLS LAST] - -(30) CometColumnarToRow [codegen id : 6] -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] - -(31) WindowGroupLimit -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] -Arguments: [i_category#23], [sumsales#37 DESC NULLS LAST], rank(sumsales#37), 100, Final - -(32) Window -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] -Arguments: [rank(sumsales#37) windowspecdefinition(i_category#23, sumsales#37 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#38], [i_category#23], [sumsales#37 DESC NULLS LAST] - -(33) Filter [codegen id : 7] -Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37, rk#38] -Condition : (rk#38 <= 100) - -(34) TakeOrderedAndProject -Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37, rk#38] -Arguments: 100, [i_category#23 ASC NULLS FIRST, i_class#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_product_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_qoy#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, s_store_id#30 ASC NULLS FIRST, sumsales#37 ASC NULLS FIRST, rk#38 ASC NULLS FIRST], [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37, rk#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (39) -+- * CometColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometNativeScan parquet spark_catalog.default.date_dim (35) - - -(35) CometNativeScan parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#39, d_year#8, d_moy#9, d_qoy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(36) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#39, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#39) AND (d_month_seq#39 >= 1200)) AND (d_month_seq#39 <= 1211)) AND isnotnull(d_date_sk#7)) - -(37) CometProject -Input [5]: [d_date_sk#7, d_month_seq#39, d_year#8, d_moy#9, d_qoy#10] -Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(38) CometColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(39) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/simplified.txt deleted file mode 100644 index 7c5b24a903..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/simplified.txt +++ /dev/null @@ -1,57 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (7) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (5) - Sort [i_category,sumsales] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [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] [sum,isEmpty,sum,isEmpty] - Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometNativeScan 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/q67.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/explain.txt deleted file mode 100644 index 3e9bbb06ae..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (34) -+- * Filter (33) - +- Window (32) - +- WindowGroupLimit (31) - +- * CometColumnarToRow (30) - +- CometSort (29) - +- CometColumnarExchange (28) - +- WindowGroupLimit (27) - +- * CometColumnarToRow (26) - +- CometSort (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExpand (21) - +- CometProject (20) - +- CometBroadcastHashJoin (19) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - +- CometBroadcastExchange (18) - +- CometProject (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_month_seq#8 <= 1211)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(6) CometBroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Right output [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11], [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_store_id#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#12, s_store_id#13] -Condition : isnotnull(s_store_sk#12) - -(11) CometProject -Input [2]: [s_store_sk#12, s_store_id#13] -Arguments: [s_store_sk#12, s_store_id#14], [s_store_sk#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#13, 16, true, false, true) AS s_store_id#14] - -(12) CometBroadcastExchange -Input [2]: [s_store_sk#12, s_store_id#14] -Arguments: [s_store_sk#12, s_store_id#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] -Right output [2]: [s_store_sk#12, s_store_id#14] -Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_sk#12, s_store_id#14] -Arguments: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14], [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Condition : isnotnull(i_item_sk#15) - -(17) CometProject -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#16, 50, true, false, true) AS i_brand#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#17, 50, true, false, true) AS i_class#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true) AS i_category#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#19, 50, true, false, true) AS i_product_name#23] - -(18) CometBroadcastExchange -Input [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] -Right output [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight - -(20) CometProject -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] - -(21) CometExpand -Input [10]: [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -Arguments: [[ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 0], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, null, 1], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, null, null, 3], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, null, null, null, 7], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, null, null, null, null, 15], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, null, null, null, null, null, 31], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, null, null, null, null, null, null, 63], [ss_quantity#3, ss_sales_price#4, i_category#22, null, null, null, null, null, null, null, 127], [ss_quantity#3, ss_sales_price#4, null, null, null, null, null, null, null, null, 255]], [ss_quantity#3, ss_sales_price#4, i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] - -(22) CometHashAggregate -Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] -Keys [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] -Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(23) CometExchange -Input [11]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32, sum#33, isEmpty#34] -Arguments: hashpartitioning(i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [11]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32, sum#33, isEmpty#34] -Keys [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] -Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(25) CometSort -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35], [i_category#24 ASC NULLS FIRST, sumsales#35 DESC NULLS LAST] - -(26) CometColumnarToRow [codegen id : 1] -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] - -(27) WindowGroupLimit -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [i_category#24], [sumsales#35 DESC NULLS LAST], rank(sumsales#35), 100, Partial - -(28) CometColumnarExchange -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: hashpartitioning(i_category#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(29) CometSort -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35], [i_category#24 ASC NULLS FIRST, sumsales#35 DESC NULLS LAST] - -(30) CometColumnarToRow [codegen id : 2] -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] - -(31) WindowGroupLimit -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [i_category#24], [sumsales#35 DESC NULLS LAST], rank(sumsales#35), 100, Final - -(32) Window -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [rank(sumsales#35) windowspecdefinition(i_category#24, sumsales#35 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#36], [i_category#24], [sumsales#35 DESC NULLS LAST] - -(33) Filter [codegen id : 3] -Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] -Condition : (rk#36 <= 100) - -(34) TakeOrderedAndProject -Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] -Arguments: 100, [i_category#24 ASC NULLS FIRST, i_class#25 ASC NULLS FIRST, i_brand#26 ASC NULLS FIRST, i_product_name#27 ASC NULLS FIRST, d_year#28 ASC NULLS FIRST, d_qoy#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, s_store_id#31 ASC NULLS FIRST, sumsales#35 ASC NULLS FIRST, rk#36 ASC NULLS FIRST], [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (39) -+- * CometColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (35) - - -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(36) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_month_seq#8 <= 1211)) AND isnotnull(d_date_sk#7)) - -(37) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(38) CometColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(39) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/simplified.txt deleted file mode 100644 index 9069117a5b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,48 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (3) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometHashAggregate [spark_grouping_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 - CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] - 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,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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] 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/q67/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/explain.txt deleted file mode 100644 index 3e9bbb06ae..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (34) -+- * Filter (33) - +- Window (32) - +- WindowGroupLimit (31) - +- * CometColumnarToRow (30) - +- CometSort (29) - +- CometColumnarExchange (28) - +- WindowGroupLimit (27) - +- * CometColumnarToRow (26) - +- CometSort (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExpand (21) - +- CometProject (20) - +- CometBroadcastHashJoin (19) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - +- CometBroadcastExchange (18) - +- CometProject (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_month_seq#8 <= 1211)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(6) CometBroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Right output [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11], [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_store_id#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#12, s_store_id#13] -Condition : isnotnull(s_store_sk#12) - -(11) CometProject -Input [2]: [s_store_sk#12, s_store_id#13] -Arguments: [s_store_sk#12, s_store_id#14], [s_store_sk#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#13, 16, true, false, true) AS s_store_id#14] - -(12) CometBroadcastExchange -Input [2]: [s_store_sk#12, s_store_id#14] -Arguments: [s_store_sk#12, s_store_id#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] -Right output [2]: [s_store_sk#12, s_store_id#14] -Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_sk#12, s_store_id#14] -Arguments: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14], [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Condition : isnotnull(i_item_sk#15) - -(17) CometProject -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#16, 50, true, false, true) AS i_brand#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#17, 50, true, false, true) AS i_class#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true) AS i_category#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#19, 50, true, false, true) AS i_product_name#23] - -(18) CometBroadcastExchange -Input [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] -Right output [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight - -(20) CometProject -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] - -(21) CometExpand -Input [10]: [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -Arguments: [[ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 0], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, null, 1], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, null, null, 3], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, null, null, null, 7], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, null, null, null, null, 15], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, null, null, null, null, null, 31], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, null, null, null, null, null, null, 63], [ss_quantity#3, ss_sales_price#4, i_category#22, null, null, null, null, null, null, null, 127], [ss_quantity#3, ss_sales_price#4, null, null, null, null, null, null, null, null, 255]], [ss_quantity#3, ss_sales_price#4, i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] - -(22) CometHashAggregate -Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] -Keys [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] -Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(23) CometExchange -Input [11]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32, sum#33, isEmpty#34] -Arguments: hashpartitioning(i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [11]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32, sum#33, isEmpty#34] -Keys [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] -Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(25) CometSort -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35], [i_category#24 ASC NULLS FIRST, sumsales#35 DESC NULLS LAST] - -(26) CometColumnarToRow [codegen id : 1] -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] - -(27) WindowGroupLimit -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [i_category#24], [sumsales#35 DESC NULLS LAST], rank(sumsales#35), 100, Partial - -(28) CometColumnarExchange -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: hashpartitioning(i_category#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(29) CometSort -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35], [i_category#24 ASC NULLS FIRST, sumsales#35 DESC NULLS LAST] - -(30) CometColumnarToRow [codegen id : 2] -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] - -(31) WindowGroupLimit -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [i_category#24], [sumsales#35 DESC NULLS LAST], rank(sumsales#35), 100, Final - -(32) Window -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [rank(sumsales#35) windowspecdefinition(i_category#24, sumsales#35 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#36], [i_category#24], [sumsales#35 DESC NULLS LAST] - -(33) Filter [codegen id : 3] -Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] -Condition : (rk#36 <= 100) - -(34) TakeOrderedAndProject -Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] -Arguments: 100, [i_category#24 ASC NULLS FIRST, i_class#25 ASC NULLS FIRST, i_brand#26 ASC NULLS FIRST, i_product_name#27 ASC NULLS FIRST, d_year#28 ASC NULLS FIRST, d_qoy#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, s_store_id#31 ASC NULLS FIRST, sumsales#35 ASC NULLS FIRST, rk#36 ASC NULLS FIRST], [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (39) -+- * CometColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (35) - - -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(36) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_month_seq#8 <= 1211)) AND isnotnull(d_date_sk#7)) - -(37) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(38) CometColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(39) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/extended.txt deleted file mode 100644 index 7504243c7a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 30 out of 37 eligible operators (81%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 9069117a5b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/simplified.txt +++ /dev/null @@ -1,48 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (3) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometHashAggregate [spark_grouping_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 - CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] - 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,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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/explain.txt deleted file mode 100644 index 7da946625e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/explain.txt +++ /dev/null @@ -1,267 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (41) -+- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * HashAggregate (30) - : : +- * CometColumnarToRow (29) - : : +- CometColumnarExchange (28) - : : +- * HashAggregate (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Project (6) - : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : :- * Filter (3) - : : : : : : +- * ColumnarToRow (2) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : +- ReusedExchange (4) - : : : : +- BroadcastExchange (11) - : : : : +- * CometColumnarToRow (10) - : : : : +- CometProject (9) - : : : : +- CometFilter (8) - : : : : +- CometNativeScan parquet spark_catalog.default.store (7) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) - : : +- BroadcastExchange (24) - : : +- * CometColumnarToRow (23) - : : +- CometFilter (22) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (21) - : +- BroadcastExchange (35) - : +- * CometColumnarToRow (34) - : +- CometProject (33) - : +- CometFilter (32) - : +- CometNativeScan parquet spark_catalog.default.customer (31) - +- ReusedExchange (38) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] - -(3) Filter [codegen id : 5] -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 46] -Output [1]: [d_date_sk#11] - -(5) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 5] -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Input [10]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9, d_date_sk#11] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_city#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [s_store_sk#12, s_city#13] -Condition : (s_city#13 IN (Midway,Fairview) AND isnotnull(s_store_sk#12)) - -(9) CometProject -Input [2]: [s_store_sk#12, s_city#13] -Arguments: [s_store_sk#12], [s_store_sk#12] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#12] - -(11) BroadcastExchange -Input [1]: [s_store_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#12] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, s_store_sk#12] - -(14) CometNativeScan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Condition : (((hd_dep_count#15 = 4) OR (hd_vehicle_count#16 = 3)) AND isnotnull(hd_demo_sk#14)) - -(16) CometProject -Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Arguments: [hd_demo_sk#14], [hd_demo_sk#14] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#14] - -(18) BroadcastExchange -Input [1]: [hd_demo_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#14] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, hd_demo_sk#14] - -(21) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#17, ca_city#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [ca_address_sk#17, ca_city#18] -Condition : (isnotnull(ca_address_sk#17) AND isnotnull(ca_city#18)) - -(23) CometColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#17, ca_city#18] - -(24) BroadcastExchange -Input [2]: [ca_address_sk#17, ca_city#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#17] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 5] -Output [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#18] -Input [8]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_address_sk#17, ca_city#18] - -(27) HashAggregate [codegen id : 5] -Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#18] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#6)), partial_sum(UnscaledValue(ss_ext_list_price#7)), partial_sum(UnscaledValue(ss_ext_tax#8))] -Aggregate Attributes [3]: [sum#19, sum#20, sum#21] -Results [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] - -(28) CometColumnarExchange -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(29) CometColumnarToRow [codegen id : 8] -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] - -(30) HashAggregate [codegen id : 8] -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#6)), sum(UnscaledValue(ss_ext_list_price#7)), sum(UnscaledValue(ss_ext_tax#8))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#6))#25, sum(UnscaledValue(ss_ext_list_price#7))#26, sum(UnscaledValue(ss_ext_tax#8))#27] -Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#18 AS bought_city#28, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#25,17,2) AS extended_price#29, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#7))#26,17,2) AS list_price#30, MakeDecimal(sum(UnscaledValue(ss_ext_tax#8))#27,17,2) AS extended_tax#31] - -(31) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(32) CometFilter -Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#33)) - -(33) CometProject -Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] -Arguments: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37], [c_customer_sk#32, c_current_addr_sk#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#34, 20, true, false, true) AS c_first_name#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#35, 30, true, false, true) AS c_last_name#37] - -(34) CometColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37] - -(35) BroadcastExchange -Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(36) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#32] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 8] -Output [8]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#36, c_last_name#37] -Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37] - -(38) ReusedExchange [Reuses operator id: 24] -Output [2]: [ca_address_sk#38, ca_city#39] - -(39) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [c_current_addr_sk#33] -Right keys [1]: [ca_address_sk#38] -Join type: Inner -Join condition: NOT (ca_city#39 = bought_city#28) - -(40) Project [codegen id : 8] -Output [8]: [c_last_name#37, c_first_name#36, ca_city#39, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] -Input [10]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#36, c_last_name#37, ca_address_sk#38, ca_city#39] - -(41) TakeOrderedAndProject -Input [8]: [c_last_name#37, c_first_name#36, ca_city#39, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] -Arguments: 100, [c_last_name#37 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#39, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometNativeScan parquet spark_catalog.default.date_dim (42) - - -(42) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#40, d_dom#41] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [3]: [d_date_sk#11, d_year#40, d_dom#41] -Condition : ((((isnotnull(d_dom#41) AND (d_dom#41 >= 1)) AND (d_dom#41 <= 2)) AND d_year#40 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) - -(44) CometProject -Input [3]: [d_date_sk#11, d_year#40, d_dom#41] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(45) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(46) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/simplified.txt deleted file mode 100644 index 05b883b1e6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/simplified.txt +++ /dev/null @@ -1,67 +0,0 @@ -TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_city,extended_price,extended_tax,list_price] - WholeStageCodegen (8) - Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] - Project [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax)),bought_city,extended_price,list_price,extended_tax,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - WholeStageCodegen (5) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [sum,sum,sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_city] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - InputAdapter - ReusedExchange [ca_address_sk,ca_city] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/explain.txt deleted file mode 100644 index bc17101a84..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,256 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (39) -+- CometTakeOrderedAndProject (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (34) - : +- CometBroadcastHashJoin (33) - : :- CometHashAggregate (28) - : : +- CometExchange (27) - : : +- CometHashAggregate (26) - : : +- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (20) - : : : +- CometBroadcastHashJoin (19) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometProject (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : : : +- CometBroadcastExchange (18) - : : : +- CometProject (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - : : +- CometBroadcastExchange (23) - : : +- CometFilter (22) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (21) - : +- CometBroadcastExchange (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (29) - +- ReusedExchange (35) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_dom#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AND d_year#12 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) - -(5) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(7) CometBroadcastHashJoin -Left output [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(8) CometProject -Input [10]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9, d_date_sk#11] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_city#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#14, s_city#15] -Condition : (s_city#15 IN (Midway,Fairview) AND isnotnull(s_store_sk#14)) - -(11) CometProject -Input [2]: [s_store_sk#14, s_city#15] -Arguments: [s_store_sk#14], [s_store_sk#14] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#14] -Arguments: [s_store_sk#14] - -(13) CometBroadcastHashJoin -Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Right output [1]: [s_store_sk#14] -Arguments: [ss_store_sk#4], [s_store_sk#14], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, s_store_sk#14] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Condition : (((hd_dep_count#17 = 4) OR (hd_vehicle_count#18 = 3)) AND isnotnull(hd_demo_sk#16)) - -(17) CometProject -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Arguments: [hd_demo_sk#16], [hd_demo_sk#16] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#16] -Arguments: [hd_demo_sk#16] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Right output [1]: [hd_demo_sk#16] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, hd_demo_sk#16] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_city#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [ca_address_sk#19, ca_city#20] -Condition : (isnotnull(ca_address_sk#19) AND isnotnull(ca_city#20)) - -(23) CometBroadcastExchange -Input [2]: [ca_address_sk#19, ca_city#20] -Arguments: [ca_address_sk#19, ca_city#20] - -(24) CometBroadcastHashJoin -Left output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Right output [2]: [ca_address_sk#19, ca_city#20] -Arguments: [ss_addr_sk#3], [ca_address_sk#19], Inner, BuildRight - -(25) CometProject -Input [8]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_address_sk#19, ca_city#20] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20] - -(26) CometHashAggregate -Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#6)), partial_sum(UnscaledValue(ss_ext_list_price#7)), partial_sum(UnscaledValue(ss_ext_tax#8))] - -(27) CometExchange -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#21, sum#22, sum#23] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#21, sum#22, sum#23] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#6)), sum(UnscaledValue(ss_ext_list_price#7)), sum(UnscaledValue(ss_ext_tax#8))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] -Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_current_addr_sk#25)) - -(31) CometProject -Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29], [c_customer_sk#24, c_current_addr_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#26, 20, true, false, true) AS c_first_name#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#27, 30, true, false, true) AS c_last_name#29] - -(32) CometBroadcastExchange -Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Arguments: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] - -(33) CometBroadcastHashJoin -Left output [6]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#30, extended_price#31, list_price#32, extended_tax#33] -Right output [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Arguments: [ss_customer_sk#1], [c_customer_sk#24], Inner, BuildRight - -(34) CometProject -Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Arguments: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29], [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29] - -(35) ReusedExchange [Reuses operator id: 23] -Output [2]: [ca_address_sk#34, ca_city#35] - -(36) CometBroadcastHashJoin -Left output [8]: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Right output [2]: [ca_address_sk#34, ca_city#35] -Arguments: [c_current_addr_sk#25], [ca_address_sk#34], Inner, NOT (ca_city#35 = bought_city#30), BuildRight - -(37) CometProject -Input [10]: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29, ca_address_sk#34, ca_city#35] -Arguments: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32], [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] - -(38) CometTakeOrderedAndProject -Input [8]: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#29 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#29,c_first_name#28,ca_city#35,bought_city#30,ss_ticket_number#5,extended_price#31,extended_tax#33,list_price#32]), [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32], 100, 0, [c_last_name#29 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] - -(39) CometColumnarToRow [codegen id : 1] -Input [8]: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) - - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_dom#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(41) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AND d_year#12 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) - -(42) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(43) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(44) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/simplified.txt deleted file mode 100644 index fdddd82de0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] - CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] - CometBroadcastHashJoin [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] - CometProject [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometHashAggregate [ss_addr_sk,sum,sum,sum] [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,ca_city,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax))] - CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - CometHashAggregate [ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] - 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_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_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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [ca_address_sk,ca_city] #6 - CometFilter [ca_address_sk,ca_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - ReusedExchange [ca_address_sk,ca_city] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/explain.txt deleted file mode 100644 index bc17101a84..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/explain.txt +++ /dev/null @@ -1,256 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (39) -+- CometTakeOrderedAndProject (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (34) - : +- CometBroadcastHashJoin (33) - : :- CometHashAggregate (28) - : : +- CometExchange (27) - : : +- CometHashAggregate (26) - : : +- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (20) - : : : +- CometBroadcastHashJoin (19) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometProject (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : : : +- CometBroadcastExchange (18) - : : : +- CometProject (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - : : +- CometBroadcastExchange (23) - : : +- CometFilter (22) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (21) - : +- CometBroadcastExchange (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (29) - +- ReusedExchange (35) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_dom#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AND d_year#12 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) - -(5) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(7) CometBroadcastHashJoin -Left output [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(8) CometProject -Input [10]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9, d_date_sk#11] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_city#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#14, s_city#15] -Condition : (s_city#15 IN (Midway,Fairview) AND isnotnull(s_store_sk#14)) - -(11) CometProject -Input [2]: [s_store_sk#14, s_city#15] -Arguments: [s_store_sk#14], [s_store_sk#14] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#14] -Arguments: [s_store_sk#14] - -(13) CometBroadcastHashJoin -Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Right output [1]: [s_store_sk#14] -Arguments: [ss_store_sk#4], [s_store_sk#14], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, s_store_sk#14] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Condition : (((hd_dep_count#17 = 4) OR (hd_vehicle_count#18 = 3)) AND isnotnull(hd_demo_sk#16)) - -(17) CometProject -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Arguments: [hd_demo_sk#16], [hd_demo_sk#16] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#16] -Arguments: [hd_demo_sk#16] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Right output [1]: [hd_demo_sk#16] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, hd_demo_sk#16] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_city#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [ca_address_sk#19, ca_city#20] -Condition : (isnotnull(ca_address_sk#19) AND isnotnull(ca_city#20)) - -(23) CometBroadcastExchange -Input [2]: [ca_address_sk#19, ca_city#20] -Arguments: [ca_address_sk#19, ca_city#20] - -(24) CometBroadcastHashJoin -Left output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Right output [2]: [ca_address_sk#19, ca_city#20] -Arguments: [ss_addr_sk#3], [ca_address_sk#19], Inner, BuildRight - -(25) CometProject -Input [8]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_address_sk#19, ca_city#20] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20] - -(26) CometHashAggregate -Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#6)), partial_sum(UnscaledValue(ss_ext_list_price#7)), partial_sum(UnscaledValue(ss_ext_tax#8))] - -(27) CometExchange -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#21, sum#22, sum#23] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#21, sum#22, sum#23] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#6)), sum(UnscaledValue(ss_ext_list_price#7)), sum(UnscaledValue(ss_ext_tax#8))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] -Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_current_addr_sk#25)) - -(31) CometProject -Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29], [c_customer_sk#24, c_current_addr_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#26, 20, true, false, true) AS c_first_name#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#27, 30, true, false, true) AS c_last_name#29] - -(32) CometBroadcastExchange -Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Arguments: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] - -(33) CometBroadcastHashJoin -Left output [6]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#30, extended_price#31, list_price#32, extended_tax#33] -Right output [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Arguments: [ss_customer_sk#1], [c_customer_sk#24], Inner, BuildRight - -(34) CometProject -Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Arguments: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29], [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29] - -(35) ReusedExchange [Reuses operator id: 23] -Output [2]: [ca_address_sk#34, ca_city#35] - -(36) CometBroadcastHashJoin -Left output [8]: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Right output [2]: [ca_address_sk#34, ca_city#35] -Arguments: [c_current_addr_sk#25], [ca_address_sk#34], Inner, NOT (ca_city#35 = bought_city#30), BuildRight - -(37) CometProject -Input [10]: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29, ca_address_sk#34, ca_city#35] -Arguments: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32], [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] - -(38) CometTakeOrderedAndProject -Input [8]: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#29 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#29,c_first_name#28,ca_city#35,bought_city#30,ss_ticket_number#5,extended_price#31,extended_tax#33,list_price#32]), [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32], 100, 0, [c_last_name#29 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] - -(39) CometColumnarToRow [codegen id : 1] -Input [8]: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) - - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_dom#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(41) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AND d_year#12 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) - -(42) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(43) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(44) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/extended.txt deleted file mode 100644 index c842000e11..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/extended.txt +++ /dev/null @@ -1,49 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index fdddd82de0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] - CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] - CometBroadcastHashJoin [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] - CometProject [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometHashAggregate [ss_addr_sk,sum,sum,sum] [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,ca_city,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax))] - CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - CometHashAggregate [ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] - 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_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_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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [ca_address_sk,ca_city] #6 - CometFilter [ca_address_sk,ca_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - ReusedExchange [ca_address_sk,ca_city] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/explain.txt deleted file mode 100644 index 556dd015a5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/explain.txt +++ /dev/null @@ -1,290 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (44) -+- * HashAggregate (43) - +- * CometColumnarToRow (42) - +- CometColumnarExchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (25) - : : +- * BroadcastHashJoin LeftAnti BuildRight (24) - : : :- * BroadcastHashJoin LeftAnti BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (30) - : +- * CometColumnarToRow (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometNativeScan parquet spark_catalog.default.customer_address (26) - +- BroadcastExchange (37) - +- * CometColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometNativeScan parquet spark_catalog.default.customer_demographics (33) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(4) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] - -(6) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#7] - -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#4] -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] - -(9) BroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#4] -Join type: LeftSemi -Join condition: None - -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#6)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] - -(13) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#10] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#8] -Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#10] - -(16) BroadcastExchange -Input [1]: [ws_bill_customer_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(17) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#8] -Join type: LeftAnti -Join condition: None - -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#11, cs_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#12), dynamicpruningexpression(cs_sold_date_sk#12 IN dynamicpruning#6)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#11, cs_sold_date_sk#12] - -(20) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#13] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#12] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#11] -Input [3]: [cs_ship_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13] - -(23) BroadcastExchange -Input [1]: [cs_ship_customer_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_ship_customer_sk#11] -Join type: LeftAnti -Join condition: None - -(25) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(26) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#14, ca_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [ca_address_sk#14, ca_state#15] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#15, 2, true, false, true) IN (KY,GA,NM) AND isnotnull(ca_address_sk#14)) - -(28) CometProject -Input [2]: [ca_address_sk#14, ca_state#15] -Arguments: [ca_address_sk#14], [ca_address_sk#14] - -(29) CometColumnarToRow [codegen id : 7] -Input [1]: [ca_address_sk#14] - -(30) BroadcastExchange -Input [1]: [ca_address_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#14] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 9] -Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#14] - -(33) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#16, cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(34) CometFilter -Input [6]: [cd_demo_sk#16, cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21] -Condition : isnotnull(cd_demo_sk#16) - -(35) CometProject -Input [6]: [cd_demo_sk#16, cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21] -Arguments: [cd_demo_sk#16, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25], [cd_demo_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#17, 1, true, false, true) AS cd_gender#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#18, 1, true, false, true) AS cd_marital_status#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#19, 20, true, false, true) AS cd_education_status#24, cd_purchase_estimate#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#21, 10, true, false, true) AS cd_credit_rating#25] - -(36) CometColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#16, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] - -(37) BroadcastExchange -Input [6]: [cd_demo_sk#16, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(38) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#16] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 9] -Output [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#16, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] - -(40) HashAggregate [codegen id : 9] -Input [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] -Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#26] -Results [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25, count#27] - -(41) CometColumnarExchange -Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25, count#27] -Arguments: hashpartitioning(cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(42) CometColumnarToRow [codegen id : 10] -Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25, count#27] - -(43) HashAggregate [codegen id : 10] -Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25, count#27] -Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#28] -Results [8]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, count(1)#28 AS cnt1#29, cd_purchase_estimate#20, count(1)#28 AS cnt2#30, cd_credit_rating#25, count(1)#28 AS cnt3#31] - -(44) TakeOrderedAndProject -Input [8]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#29, cd_purchase_estimate#20, cnt2#30, cd_credit_rating#25, cnt3#31] -Arguments: 100, [cd_gender#22 ASC NULLS FIRST, cd_marital_status#23 ASC NULLS FIRST, cd_education_status#24 ASC NULLS FIRST, cd_purchase_estimate#20 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST], [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#29, cd_purchase_estimate#20, cnt2#30, cd_credit_rating#25, cnt3#31] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometNativeScan parquet spark_catalog.default.date_dim (45) - - -(45) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#32, d_moy#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [3]: [d_date_sk#7, d_year#32, d_moy#33] -Condition : (((((isnotnull(d_year#32) AND isnotnull(d_moy#33)) AND (d_year#32 = 2001)) AND (d_moy#33 >= 4)) AND (d_moy#33 <= 6)) AND isnotnull(d_date_sk#7)) - -(47) CometProject -Input [3]: [d_date_sk#7, d_year#32, d_moy#33] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(49) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#12 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/simplified.txt deleted file mode 100644 index f8868e38be..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/simplified.txt +++ /dev/null @@ -1,76 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (10) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (9) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometNativeScan 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/q69.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/explain.txt deleted file mode 100644 index af81415f33..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,301 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (46) -+- * HashAggregate (45) - +- * CometColumnarToRow (44) - +- CometColumnarExchange (43) - +- * HashAggregate (42) - +- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (34) - : +- * BroadcastHashJoin Inner BuildRight (33) - : :- * Project (27) - : : +- * BroadcastHashJoin LeftAnti BuildRight (26) - : : :- * BroadcastHashJoin LeftAnti BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (32) - : +- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (28) - +- BroadcastExchange (39) - +- * CometColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (35) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2001)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 6)) AND isnotnull(d_date_sk#7)) - -(6) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#4], [ss_customer_sk#4] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: [ss_customer_sk#4] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ss_customer_sk#4] -Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#13] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] -Arguments: [ws_bill_customer_sk#10], [ws_bill_customer_sk#10] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#10] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#10] -Join type: LeftAnti -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#17] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -Right output [1]: [d_date_sk#17] -Arguments: [cs_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] -Arguments: [cs_ship_customer_sk#14], [cs_ship_customer_sk#14] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#14] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_ship_customer_sk#14] -Join type: LeftAnti -Join condition: None - -(27) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_state#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(29) CometFilter -Input [2]: [ca_address_sk#18, ca_state#19] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#19, 2, true, false, true) IN (KY,GA,NM) AND isnotnull(ca_address_sk#18)) - -(30) CometProject -Input [2]: [ca_address_sk#18, ca_state#19] -Arguments: [ca_address_sk#18], [ca_address_sk#18] - -(31) CometColumnarToRow [codegen id : 3] -Input [1]: [ca_address_sk#18] - -(32) BroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(33) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#18] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 5] -Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] - -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(36) CometFilter -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -Condition : isnotnull(cd_demo_sk#20) - -(37) CometProject -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -Arguments: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29], [cd_demo_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#21, 1, true, false, true) AS cd_gender#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#22, 1, true, false, true) AS cd_marital_status#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#23, 20, true, false, true) AS cd_education_status#28, cd_purchase_estimate#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#25, 10, true, false, true) AS cd_credit_rating#29] - -(38) CometColumnarToRow [codegen id : 4] -Input [6]: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] - -(39) BroadcastExchange -Input [6]: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(40) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 5] -Output [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] - -(42) HashAggregate [codegen id : 5] -Input [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#30] -Results [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] - -(43) CometColumnarExchange -Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] -Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(44) CometColumnarToRow [codegen id : 6] -Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] - -(45) HashAggregate [codegen id : 6] -Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] -Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#32] -Results [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#32 AS cnt1#33, cd_purchase_estimate#24, count(1)#32 AS cnt2#34, cd_credit_rating#29, count(1)#32 AS cnt3#35] - -(46) TakeOrderedAndProject -Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#33, cd_purchase_estimate#24, cnt2#34, cd_credit_rating#29, cnt3#35] -Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#33, cd_purchase_estimate#24, cnt2#34, cd_credit_rating#29, cnt3#35] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (51) -+- * CometColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) - - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] -ReadSchema: struct - -(48) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2001)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 6)) AND isnotnull(d_date_sk#7)) - -(49) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(50) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(51) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/simplified.txt deleted file mode 100644 index 68bf32c40b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,72 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (6) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (5) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometScan [native_iceberg_compat] 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/q69/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/explain.txt deleted file mode 100644 index af81415f33..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/explain.txt +++ /dev/null @@ -1,301 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (46) -+- * HashAggregate (45) - +- * CometColumnarToRow (44) - +- CometColumnarExchange (43) - +- * HashAggregate (42) - +- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (34) - : +- * BroadcastHashJoin Inner BuildRight (33) - : :- * Project (27) - : : +- * BroadcastHashJoin LeftAnti BuildRight (26) - : : :- * BroadcastHashJoin LeftAnti BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (32) - : +- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (28) - +- BroadcastExchange (39) - +- * CometColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (35) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2001)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 6)) AND isnotnull(d_date_sk#7)) - -(6) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#4], [ss_customer_sk#4] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: [ss_customer_sk#4] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ss_customer_sk#4] -Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#13] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] -Arguments: [ws_bill_customer_sk#10], [ws_bill_customer_sk#10] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#10] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#10] -Join type: LeftAnti -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#17] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -Right output [1]: [d_date_sk#17] -Arguments: [cs_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] -Arguments: [cs_ship_customer_sk#14], [cs_ship_customer_sk#14] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#14] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_ship_customer_sk#14] -Join type: LeftAnti -Join condition: None - -(27) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_state#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(29) CometFilter -Input [2]: [ca_address_sk#18, ca_state#19] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#19, 2, true, false, true) IN (KY,GA,NM) AND isnotnull(ca_address_sk#18)) - -(30) CometProject -Input [2]: [ca_address_sk#18, ca_state#19] -Arguments: [ca_address_sk#18], [ca_address_sk#18] - -(31) CometColumnarToRow [codegen id : 3] -Input [1]: [ca_address_sk#18] - -(32) BroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(33) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#18] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 5] -Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] - -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(36) CometFilter -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -Condition : isnotnull(cd_demo_sk#20) - -(37) CometProject -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -Arguments: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29], [cd_demo_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#21, 1, true, false, true) AS cd_gender#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#22, 1, true, false, true) AS cd_marital_status#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#23, 20, true, false, true) AS cd_education_status#28, cd_purchase_estimate#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#25, 10, true, false, true) AS cd_credit_rating#29] - -(38) CometColumnarToRow [codegen id : 4] -Input [6]: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] - -(39) BroadcastExchange -Input [6]: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(40) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 5] -Output [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] - -(42) HashAggregate [codegen id : 5] -Input [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#30] -Results [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] - -(43) CometColumnarExchange -Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] -Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(44) CometColumnarToRow [codegen id : 6] -Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] - -(45) HashAggregate [codegen id : 6] -Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] -Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#32] -Results [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#32 AS cnt1#33, cd_purchase_estimate#24, count(1)#32 AS cnt2#34, cd_credit_rating#29, count(1)#32 AS cnt3#35] - -(46) TakeOrderedAndProject -Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#33, cd_purchase_estimate#24, cnt2#34, cd_credit_rating#29, cnt3#35] -Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#33, cd_purchase_estimate#24, cnt2#34, cd_credit_rating#29, cnt3#35] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (51) -+- * CometColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) - - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] -ReadSchema: struct - -(48) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2001)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 6)) AND isnotnull(d_date_sk#7)) - -(49) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(50) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(51) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/extended.txt deleted file mode 100644 index b4fbb455ef..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/extended.txt +++ /dev/null @@ -1,62 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 68bf32c40b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69/simplified.txt +++ /dev/null @@ -1,72 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (6) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (5) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/explain.txt deleted file mode 100644 index 1a5a7efc91..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- * CometColumnarToRow (30) - +- CometColumnarExchange (29) - +- * HashAggregate (28) - +- * Project (27) - +- * BroadcastHashJoin Inner BuildRight (26) - :- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.item (14) - +- BroadcastExchange (25) - +- * CometColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometNativeScan parquet spark_catalog.default.promotion (21) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(3) Filter [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(4) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = S)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#10)) - -(6) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [cd_demo_sk#10] - -(8) BroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 5] -Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] - -(11) ReusedExchange [Reuses operator id: 37] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] - -(14) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#15, i_item_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [i_item_sk#15, i_item_id#16] -Condition : isnotnull(i_item_sk#15) - -(16) CometProject -Input [2]: [i_item_sk#15, i_item_id#16] -Arguments: [i_item_sk#15, i_item_id#17], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#16, 16, true, false, true) AS i_item_id#17] - -(17) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#15, i_item_id#17] - -(18) BroadcastExchange -Input [2]: [i_item_sk#15, i_item_id#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [6]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#15, i_item_id#17] - -(21) CometNativeScan parquet spark_catalog.default.promotion -Output [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_email#19, 1, true, false, true) = N) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_event#20, 1, true, false, true) = N)) AND isnotnull(p_promo_sk#18)) - -(23) CometProject -Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] -Arguments: [p_promo_sk#18], [p_promo_sk#18] - -(24) CometColumnarToRow [codegen id : 4] -Input [1]: [p_promo_sk#18] - -(25) BroadcastExchange -Input [1]: [p_promo_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_promo_sk#3] -Right keys [1]: [p_promo_sk#18] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 5] -Output [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] -Input [7]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17, p_promo_sk#18] - -(28) HashAggregate [codegen id : 5] -Input [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] -Keys [1]: [i_item_id#17] -Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [8]: [sum#21, count#22, sum#23, count#24, sum#25, count#26, sum#27, count#28] -Results [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] - -(29) CometColumnarExchange -Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] -Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 6] -Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] - -(31) HashAggregate [codegen id : 6] -Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] -Keys [1]: [i_item_id#17] -Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [4]: [avg(ss_quantity#4)#37, avg(UnscaledValue(ss_list_price#5))#38, avg(UnscaledValue(ss_coupon_amt#7))#39, avg(UnscaledValue(ss_sales_price#6))#40] -Results [5]: [i_item_id#17, avg(ss_quantity#4)#37 AS agg1#41, cast((avg(UnscaledValue(ss_list_price#5))#38 / 100.0) as decimal(11,6)) AS agg2#42, cast((avg(UnscaledValue(ss_coupon_amt#7))#39 / 100.0) as decimal(11,6)) AS agg3#43, cast((avg(UnscaledValue(ss_sales_price#6))#40 / 100.0) as decimal(11,6)) AS agg4#44] - -(32) TakeOrderedAndProject -Input [5]: [i_item_id#17, agg1#41, agg2#42, agg3#43, agg4#44] -Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#41, agg2#42, agg3#43, agg4#44] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometNativeScan parquet spark_catalog.default.date_dim (33) - - -(33) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [d_date_sk#14, d_year#45] -Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2000)) AND isnotnull(d_date_sk#14)) - -(35) CometProject -Input [2]: [d_date_sk#14, d_year#45] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(37) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/simplified.txt deleted file mode 100644 index bb670b4a73..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) - HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Filter [ss_cdemo_sk,ss_item_sk,ss_promo_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometNativeScan 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/q7.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/explain.txt deleted file mode 100644 index 0e32283618..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,212 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (31) -+- CometTakeOrderedAndProject (30) - +- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = S)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#16, i_item_id#17] -Condition : isnotnull(i_item_sk#16) - -(17) CometProject -Input [2]: [i_item_sk#16, i_item_id#17] -Arguments: [i_item_sk#16, i_item_id#18], [i_item_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#17, 16, true, false, true) AS i_item_id#18] - -(18) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_id#18] -Arguments: [i_item_sk#16, i_item_id#18] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Right output [2]: [i_item_sk#16, i_item_id#18] -Arguments: [ss_item_sk#1], [i_item_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#16, i_item_id#18] -Arguments: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18], [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_email#20, 1, true, false, true) = N) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_event#21, 1, true, false, true) = N)) AND isnotnull(p_promo_sk#19)) - -(23) CometProject -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Arguments: [p_promo_sk#19], [p_promo_sk#19] - -(24) CometBroadcastExchange -Input [1]: [p_promo_sk#19] -Arguments: [p_promo_sk#19] - -(25) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] -Right output [1]: [p_promo_sk#19] -Arguments: [ss_promo_sk#3], [p_promo_sk#19], Inner, BuildRight - -(26) CometProject -Input [7]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18, p_promo_sk#19] -Arguments: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] - -(27) CometHashAggregate -Input [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] - -(28) CometExchange -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Keys [1]: [i_item_id#18] -Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] - -(30) CometTakeOrderedAndProject -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#18 ASC NULLS FIRST], output=[i_item_id#18,agg1#30,agg2#31,agg3#32,agg4#33]), [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33], 100, 0, [i_item_id#18 ASC NULLS FIRST], [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(34) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(36) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/simplified.txt deleted file mode 100644 index a15bcd5c0d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,agg1,agg2,agg3,agg4,avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price))] - CometExchange [i_item_id] #1 - CometHashAggregate [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,sum,count,sum,count,sum,count,sum,count] - CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - 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,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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [p_promo_sk] #6 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometScan [native_iceberg_compat] 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/q7/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/explain.txt deleted file mode 100644 index 0e32283618..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/explain.txt +++ /dev/null @@ -1,212 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (31) -+- CometTakeOrderedAndProject (30) - +- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = S)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#16, i_item_id#17] -Condition : isnotnull(i_item_sk#16) - -(17) CometProject -Input [2]: [i_item_sk#16, i_item_id#17] -Arguments: [i_item_sk#16, i_item_id#18], [i_item_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#17, 16, true, false, true) AS i_item_id#18] - -(18) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_id#18] -Arguments: [i_item_sk#16, i_item_id#18] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Right output [2]: [i_item_sk#16, i_item_id#18] -Arguments: [ss_item_sk#1], [i_item_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#16, i_item_id#18] -Arguments: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18], [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_email#20, 1, true, false, true) = N) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_event#21, 1, true, false, true) = N)) AND isnotnull(p_promo_sk#19)) - -(23) CometProject -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Arguments: [p_promo_sk#19], [p_promo_sk#19] - -(24) CometBroadcastExchange -Input [1]: [p_promo_sk#19] -Arguments: [p_promo_sk#19] - -(25) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] -Right output [1]: [p_promo_sk#19] -Arguments: [ss_promo_sk#3], [p_promo_sk#19], Inner, BuildRight - -(26) CometProject -Input [7]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18, p_promo_sk#19] -Arguments: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] - -(27) CometHashAggregate -Input [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] - -(28) CometExchange -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Keys [1]: [i_item_id#18] -Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] - -(30) CometTakeOrderedAndProject -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#18 ASC NULLS FIRST], output=[i_item_id#18,agg1#30,agg2#31,agg3#32,agg4#33]), [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33], 100, 0, [i_item_id#18 ASC NULLS FIRST], [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(34) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(36) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/extended.txt deleted file mode 100644 index 655f651a41..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/extended.txt +++ /dev/null @@ -1,39 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index a15bcd5c0d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,agg1,agg2,agg3,agg4,avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price))] - CometExchange [i_item_id] #1 - CometHashAggregate [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,sum,count,sum,count,sum,count,sum,count] - CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - 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,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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [p_promo_sk] #6 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/explain.txt deleted file mode 100644 index c1925c206e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/explain.txt +++ /dev/null @@ -1,305 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- CometColumnarExchange (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Expand (38) - +- * Project (37) - +- * BroadcastHashJoin Inner BuildRight (36) - :- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.store_sales (1) - : +- ReusedExchange (4) - +- BroadcastExchange (35) - +- * Project (34) - +- * BroadcastHashJoin LeftSemi BuildRight (33) - :- * CometColumnarToRow (9) - : +- CometFilter (8) - : +- CometNativeScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (32) - +- * Project (31) - +- * Filter (30) - +- Window (29) - +- WindowGroupLimit (28) - +- * Sort (27) - +- * HashAggregate (26) - +- * CometColumnarToRow (25) - +- CometColumnarExchange (24) - +- * HashAggregate (23) - +- * Project (22) - +- * BroadcastHashJoin Inner BuildRight (21) - :- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet spark_catalog.default.store_sales (10) - : +- BroadcastExchange (17) - : +- * CometColumnarToRow (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometNativeScan parquet spark_catalog.default.store (13) - +- ReusedExchange (20) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 8] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] - -(3) Filter [codegen id : 8] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(4) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#5] - -(5) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 8] -Output [2]: [ss_store_sk#1, ss_net_profit#2] -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#6, s_county#7, s_state#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Condition : isnotnull(s_store_sk#6) - -(9) CometColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#6, s_county#7, s_state#8] - -(10) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] - -(12) Filter [codegen id : 4] -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_store_sk#9) - -(13) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_state#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [s_store_sk#12, s_state#13] -Condition : isnotnull(s_store_sk#12) - -(15) CometProject -Input [2]: [s_store_sk#12, s_state#13] -Arguments: [s_store_sk#12, s_state#14], [s_store_sk#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#14] - -(16) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#12, s_state#14] - -(17) BroadcastExchange -Input [2]: [s_store_sk#12, s_state#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#9] -Right keys [1]: [s_store_sk#12] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14] -Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#12, s_state#14] - -(20) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#15] - -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 4] -Output [2]: [ss_net_profit#10, s_state#14] -Input [4]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14, d_date_sk#15] - -(23) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#10, s_state#14] -Keys [1]: [s_state#14] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum#16] -Results [2]: [s_state#14, sum#17] - -(24) CometColumnarExchange -Input [2]: [s_state#14, sum#17] -Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(25) CometColumnarToRow [codegen id : 5] -Input [2]: [s_state#14, sum#17] - -(26) HashAggregate [codegen id : 5] -Input [2]: [s_state#14, sum#17] -Keys [1]: [s_state#14] -Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#18] -Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#18,17,2) AS _w0#19, s_state#14] - -(27) Sort [codegen id : 5] -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 - -(28) WindowGroupLimit -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [s_state#14], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final - -(29) Window -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#14, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#14], [_w0#19 DESC NULLS LAST] - -(30) Filter [codegen id : 6] -Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] -Condition : (ranking#20 <= 5) - -(31) Project [codegen id : 6] -Output [1]: [s_state#14] -Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] - -(32) BroadcastExchange -Input [1]: [s_state#14] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(33) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true)] -Right keys [1]: [s_state#14] -Join type: LeftSemi -Join condition: None - -(34) Project [codegen id : 7] -Output [3]: [s_store_sk#6, s_county#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true) AS s_state#21] -Input [3]: [s_store_sk#6, s_county#7, s_state#8] - -(35) BroadcastExchange -Input [3]: [s_store_sk#6, s_county#7, s_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(36) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#6] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 8] -Output [3]: [ss_net_profit#2, s_state#21, s_county#7] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#21] - -(38) Expand [codegen id : 8] -Input [3]: [ss_net_profit#2, s_state#21, s_county#7] -Arguments: [[ss_net_profit#2, s_state#21, s_county#7, 0], [ss_net_profit#2, s_state#21, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] - -(39) HashAggregate [codegen id : 8] -Input [4]: [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#25] -Results [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] - -(40) CometColumnarExchange -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Arguments: hashpartitioning(s_state#22, s_county#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(41) CometColumnarToRow [codegen id : 9] -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] - -(42) HashAggregate [codegen id : 9] -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#27] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS total_sum#28, s_state#22, s_county#23, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS lochierarchy#29, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS _w0#30, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS _w1#31, CASE WHEN (cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint) = 0) THEN s_state#22 END AS _w2#32] - -(43) CometColumnarExchange -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] -Arguments: hashpartitioning(_w1#31, _w2#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(44) CometSort -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] -Arguments: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32], [_w1#31 ASC NULLS FIRST, _w2#32 ASC NULLS FIRST, _w0#30 DESC NULLS LAST] - -(45) CometColumnarToRow [codegen id : 10] -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] - -(46) Window -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] -Arguments: [rank(_w0#30) windowspecdefinition(_w1#31, _w2#32, _w0#30 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#33], [_w1#31, _w2#32], [_w0#30 DESC NULLS LAST] - -(47) Project [codegen id : 11] -Output [5]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] -Input [8]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32, rank_within_parent#33] - -(48) TakeOrderedAndProject -Input [5]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] -Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0) THEN s_state#22 END ASC NULLS FIRST, rank_within_parent#33 ASC NULLS FIRST], [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometNativeScan parquet spark_catalog.default.date_dim (49) - - -(49) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#34] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1200)) AND (d_month_seq#34 <= 1211)) AND isnotnull(d_date_sk#5)) - -(51) CometProject -Input [2]: [d_date_sk#5, d_month_seq#34] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(53) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/simplified.txt deleted file mode 100644 index f61238590a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/simplified.txt +++ /dev/null @@ -1,80 +0,0 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (11) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (9) - HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (8) - HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [ss_net_profit,s_state,s_county] - Project [ss_net_profit,s_state,s_county] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (5) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #6 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/explain.txt deleted file mode 100644 index 95dc8874aa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,309 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (49) -+- * Project (48) - +- Window (47) - +- * CometColumnarToRow (46) - +- CometSort (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * CometColumnarToRow (42) - +- CometColumnarExchange (41) - +- * HashAggregate (40) - +- * Expand (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (9) - : +- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- BroadcastExchange (36) - +- * Project (35) - +- * BroadcastHashJoin LeftSemi BuildRight (34) - :- * CometColumnarToRow (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) - +- BroadcastExchange (33) - +- * Project (32) - +- * Filter (31) - +- Window (30) - +- WindowGroupLimit (29) - +- * CometColumnarToRow (28) - +- CometSort (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometBroadcastHashJoin (22) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - +- ReusedExchange (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] - -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ss_store_sk#1, ss_net_profit#2] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#7, s_county#8, s_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Condition : isnotnull(s_store_sk#7) - -(12) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_store_sk#10) - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : isnotnull(s_store_sk#14) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14, s_state#16], [s_store_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) AS s_state#16] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#14, s_state#16] -Arguments: [s_store_sk#14, s_state#16] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Right output [2]: [s_store_sk#14, s_state#16] -Arguments: [ss_store_sk#10], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#16] -Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16], [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#17] - -(22) CometBroadcastHashJoin -Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] -Right output [1]: [d_date_sk#17] -Arguments: [ss_sold_date_sk#12], [d_date_sk#17], Inner, BuildRight - -(23) CometProject -Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16, d_date_sk#17] -Arguments: [ss_net_profit#11, s_state#16], [ss_net_profit#11, s_state#16] - -(24) CometHashAggregate -Input [2]: [ss_net_profit#11, s_state#16] -Keys [1]: [s_state#16] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] - -(25) CometExchange -Input [2]: [s_state#16, sum#18] -Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(26) CometHashAggregate -Input [2]: [s_state#16, sum#18] -Keys [1]: [s_state#16] -Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] - -(27) CometSort -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16, _w0#19, s_state#16], [s_state#16 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] - -(28) CometColumnarToRow [codegen id : 1] -Input [3]: [s_state#16, _w0#19, s_state#16] - -(29) WindowGroupLimit -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final - -(30) Window -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#16, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#16], [_w0#19 DESC NULLS LAST] - -(31) Filter [codegen id : 2] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] -Condition : (ranking#20 <= 5) - -(32) Project [codegen id : 2] -Output [1]: [s_state#16] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] - -(33) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] - -(34) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)] -Right keys [1]: [s_state#16] -Join type: LeftSemi -Join condition: None - -(35) Project [codegen id : 3] -Output [3]: [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#21] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(36) BroadcastExchange -Input [3]: [s_store_sk#7, s_county#8, s_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_state#21, s_county#8] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#21] - -(39) Expand [codegen id : 4] -Input [3]: [ss_net_profit#2, s_state#21, s_county#8] -Arguments: [[ss_net_profit#2, s_state#21, s_county#8, 0], [ss_net_profit#2, s_state#21, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] - -(40) HashAggregate [codegen id : 4] -Input [4]: [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#25] -Results [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] - -(41) CometColumnarExchange -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Arguments: hashpartitioning(s_state#22, s_county#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(42) CometColumnarToRow [codegen id : 5] -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] - -(43) HashAggregate [codegen id : 5] -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#27] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS total_sum#28, s_state#22, s_county#23, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS lochierarchy#29, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS _w0#30, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS _w1#31, CASE WHEN (cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint) = 0) THEN s_state#22 END AS _w2#32] - -(44) CometColumnarExchange -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] -Arguments: hashpartitioning(_w1#31, _w2#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometSort -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] -Arguments: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32], [_w1#31 ASC NULLS FIRST, _w2#32 ASC NULLS FIRST, _w0#30 DESC NULLS LAST] - -(46) CometColumnarToRow [codegen id : 6] -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] - -(47) Window -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] -Arguments: [rank(_w0#30) windowspecdefinition(_w1#31, _w2#32, _w0#30 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#33], [_w1#31, _w2#32], [_w0#30 DESC NULLS LAST] - -(48) Project [codegen id : 7] -Output [5]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] -Input [8]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32, rank_within_parent#33] - -(49) TakeOrderedAndProject -Input [5]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] -Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0) THEN s_state#22 END ASC NULLS FIRST, rank_within_parent#33 ASC NULLS FIRST], [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) - - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(52) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(53) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(54) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/simplified.txt deleted file mode 100644 index 80f0cfc8c2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (7) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (5) - HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [ss_net_profit,s_state,s_county] - Project [ss_net_profit,s_state,s_county] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [s_state,_w0] - CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] - CometExchange [s_state] #7 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - 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,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] 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/q70/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/explain.txt deleted file mode 100644 index 95dc8874aa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/explain.txt +++ /dev/null @@ -1,309 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (49) -+- * Project (48) - +- Window (47) - +- * CometColumnarToRow (46) - +- CometSort (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * CometColumnarToRow (42) - +- CometColumnarExchange (41) - +- * HashAggregate (40) - +- * Expand (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (9) - : +- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- BroadcastExchange (36) - +- * Project (35) - +- * BroadcastHashJoin LeftSemi BuildRight (34) - :- * CometColumnarToRow (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) - +- BroadcastExchange (33) - +- * Project (32) - +- * Filter (31) - +- Window (30) - +- WindowGroupLimit (29) - +- * CometColumnarToRow (28) - +- CometSort (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometBroadcastHashJoin (22) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - +- ReusedExchange (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] - -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ss_store_sk#1, ss_net_profit#2] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#7, s_county#8, s_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Condition : isnotnull(s_store_sk#7) - -(12) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_store_sk#10) - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : isnotnull(s_store_sk#14) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14, s_state#16], [s_store_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) AS s_state#16] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#14, s_state#16] -Arguments: [s_store_sk#14, s_state#16] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Right output [2]: [s_store_sk#14, s_state#16] -Arguments: [ss_store_sk#10], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#16] -Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16], [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#17] - -(22) CometBroadcastHashJoin -Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] -Right output [1]: [d_date_sk#17] -Arguments: [ss_sold_date_sk#12], [d_date_sk#17], Inner, BuildRight - -(23) CometProject -Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16, d_date_sk#17] -Arguments: [ss_net_profit#11, s_state#16], [ss_net_profit#11, s_state#16] - -(24) CometHashAggregate -Input [2]: [ss_net_profit#11, s_state#16] -Keys [1]: [s_state#16] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] - -(25) CometExchange -Input [2]: [s_state#16, sum#18] -Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(26) CometHashAggregate -Input [2]: [s_state#16, sum#18] -Keys [1]: [s_state#16] -Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] - -(27) CometSort -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16, _w0#19, s_state#16], [s_state#16 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] - -(28) CometColumnarToRow [codegen id : 1] -Input [3]: [s_state#16, _w0#19, s_state#16] - -(29) WindowGroupLimit -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final - -(30) Window -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#16, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#16], [_w0#19 DESC NULLS LAST] - -(31) Filter [codegen id : 2] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] -Condition : (ranking#20 <= 5) - -(32) Project [codegen id : 2] -Output [1]: [s_state#16] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] - -(33) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] - -(34) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)] -Right keys [1]: [s_state#16] -Join type: LeftSemi -Join condition: None - -(35) Project [codegen id : 3] -Output [3]: [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#21] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(36) BroadcastExchange -Input [3]: [s_store_sk#7, s_county#8, s_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_state#21, s_county#8] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#21] - -(39) Expand [codegen id : 4] -Input [3]: [ss_net_profit#2, s_state#21, s_county#8] -Arguments: [[ss_net_profit#2, s_state#21, s_county#8, 0], [ss_net_profit#2, s_state#21, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] - -(40) HashAggregate [codegen id : 4] -Input [4]: [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#25] -Results [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] - -(41) CometColumnarExchange -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Arguments: hashpartitioning(s_state#22, s_county#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(42) CometColumnarToRow [codegen id : 5] -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] - -(43) HashAggregate [codegen id : 5] -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#27] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS total_sum#28, s_state#22, s_county#23, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS lochierarchy#29, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS _w0#30, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS _w1#31, CASE WHEN (cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint) = 0) THEN s_state#22 END AS _w2#32] - -(44) CometColumnarExchange -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] -Arguments: hashpartitioning(_w1#31, _w2#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometSort -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] -Arguments: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32], [_w1#31 ASC NULLS FIRST, _w2#32 ASC NULLS FIRST, _w0#30 DESC NULLS LAST] - -(46) CometColumnarToRow [codegen id : 6] -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] - -(47) Window -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] -Arguments: [rank(_w0#30) windowspecdefinition(_w1#31, _w2#32, _w0#30 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#33], [_w1#31, _w2#32], [_w0#30 DESC NULLS LAST] - -(48) Project [codegen id : 7] -Output [5]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] -Input [8]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32, rank_within_parent#33] - -(49) TakeOrderedAndProject -Input [5]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] -Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0) THEN s_state#22 END ASC NULLS FIRST, rank_within_parent#33 ASC NULLS FIRST], [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) - - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(52) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(53) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(54) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/extended.txt deleted file mode 100644 index 45a2c7a669..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/extended.txt +++ /dev/null @@ -1,61 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 34 out of 53 eligible operators (64%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 80f0cfc8c2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (7) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (5) - HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [ss_net_profit,s_state,s_county] - Project [ss_net_profit,s_state,s_county] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [s_state,_w0] - CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] - CometExchange [s_state] #7 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - 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,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/explain.txt deleted file mode 100644 index 7c4f959123..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/explain.txt +++ /dev/null @@ -1,262 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometSort (39) - +- CometColumnarExchange (38) - +- * HashAggregate (37) - +- * CometColumnarToRow (36) - +- CometColumnarExchange (35) - +- * HashAggregate (34) - +- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * Project (26) - : +- * BroadcastHashJoin Inner BuildLeft (25) - : :- BroadcastExchange (5) - : : +- * CometColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometNativeScan parquet spark_catalog.default.item (1) - : +- Union (24) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * Filter (8) - : : : +- * ColumnarToRow (7) - : : : +- Scan parquet spark_catalog.default.web_sales (6) - : : +- ReusedExchange (9) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Filter (14) - : : : +- * ColumnarToRow (13) - : : : +- Scan parquet spark_catalog.default.catalog_sales (12) - : : +- ReusedExchange (15) - : +- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * Filter (20) - : : +- * ColumnarToRow (19) - : : +- Scan parquet spark_catalog.default.store_sales (18) - : +- ReusedExchange (21) - +- BroadcastExchange (31) - +- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometNativeScan parquet spark_catalog.default.time_dim (27) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Condition : ((isnotnull(i_manager_id#4) AND (i_manager_id#4 = 1)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Arguments: [i_item_sk#1, i_brand_id#2, i_brand#5], [i_item_sk#1, i_brand_id#2, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#3, 50, true, false, true) AS i_brand#5] - -(4) CometColumnarToRow [codegen id : 1] -Input [3]: [i_item_sk#1, i_brand_id#2, i_brand#5] - -(5) BroadcastExchange -Input [3]: [i_item_sk#1, i_brand_id#2, i_brand#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(6) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_time_sk)] -ReadSchema: struct - -(7) ColumnarToRow [codegen id : 3] -Input [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] - -(8) Filter [codegen id : 3] -Input [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Condition : (isnotnull(ws_item_sk#7) AND isnotnull(ws_sold_time_sk#6)) - -(9) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#11] - -(10) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 3] -Output [3]: [ws_ext_sales_price#8 AS ext_price#12, ws_item_sk#7 AS sold_item_sk#13, ws_sold_time_sk#6 AS time_sk#14] -Input [5]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9, d_date_sk#11] - -(12) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_sold_time_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_time_sk)] -ReadSchema: struct - -(13) ColumnarToRow [codegen id : 5] -Input [4]: [cs_sold_time_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] - -(14) Filter [codegen id : 5] -Input [4]: [cs_sold_time_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] -Condition : (isnotnull(cs_item_sk#16) AND isnotnull(cs_sold_time_sk#15)) - -(15) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#19] - -(16) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 5] -Output [3]: [cs_ext_sales_price#17 AS ext_price#20, cs_item_sk#16 AS sold_item_sk#21, cs_sold_time_sk#15 AS time_sk#22] -Input [5]: [cs_sold_time_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18, d_date_sk#19] - -(18) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 7] -Input [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] - -(20) Filter [codegen id : 7] -Input [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : (isnotnull(ss_item_sk#24) AND isnotnull(ss_sold_time_sk#23)) - -(21) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#27] - -(22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: None - -(23) Project [codegen id : 7] -Output [3]: [ss_ext_sales_price#25 AS ext_price#28, ss_item_sk#24 AS sold_item_sk#29, ss_sold_time_sk#23 AS time_sk#30] -Input [5]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26, d_date_sk#27] - -(24) Union - -(25) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [sold_item_sk#13] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 9] -Output [4]: [i_brand_id#2, i_brand#5, ext_price#12, time_sk#14] -Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#5, ext_price#12, sold_item_sk#13, time_sk#14] - -(27) CometNativeScan parquet spark_catalog.default.time_dim -Output [4]: [t_time_sk#31, t_hour#32, t_minute#33, t_meal_time#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_time_sk)] -ReadSchema: struct - -(28) CometFilter -Input [4]: [t_time_sk#31, t_hour#32, t_minute#33, t_meal_time#34] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, t_meal_time#34, 20, true, false, true) = breakfast ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, t_meal_time#34, 20, true, false, true) = dinner )) AND isnotnull(t_time_sk#31)) - -(29) CometProject -Input [4]: [t_time_sk#31, t_hour#32, t_minute#33, t_meal_time#34] -Arguments: [t_time_sk#31, t_hour#32, t_minute#33], [t_time_sk#31, t_hour#32, t_minute#33] - -(30) CometColumnarToRow [codegen id : 8] -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] - -(31) BroadcastExchange -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [time_sk#14] -Right keys [1]: [t_time_sk#31] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 9] -Output [5]: [i_brand_id#2, i_brand#5, ext_price#12, t_hour#32, t_minute#33] -Input [7]: [i_brand_id#2, i_brand#5, ext_price#12, time_sk#14, t_time_sk#31, t_hour#32, t_minute#33] - -(34) HashAggregate [codegen id : 9] -Input [5]: [i_brand_id#2, i_brand#5, ext_price#12, t_hour#32, t_minute#33] -Keys [4]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33] -Functions [1]: [partial_sum(UnscaledValue(ext_price#12))] -Aggregate Attributes [1]: [sum#35] -Results [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] - -(35) CometColumnarExchange -Input [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] -Arguments: hashpartitioning(i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(36) CometColumnarToRow [codegen id : 10] -Input [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] - -(37) HashAggregate [codegen id : 10] -Input [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] -Keys [4]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33] -Functions [1]: [sum(UnscaledValue(ext_price#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#12))#37] -Results [5]: [i_brand_id#2 AS brand_id#38, i_brand#5 AS brand#39, t_hour#32, t_minute#33, MakeDecimal(sum(UnscaledValue(ext_price#12))#37,17,2) AS ext_price#40] - -(38) CometColumnarExchange -Input [5]: [brand_id#38, brand#39, t_hour#32, t_minute#33, ext_price#40] -Arguments: rangepartitioning(ext_price#40 DESC NULLS LAST, brand_id#38 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(39) CometSort -Input [5]: [brand_id#38, brand#39, t_hour#32, t_minute#33, ext_price#40] -Arguments: [brand_id#38, brand#39, t_hour#32, t_minute#33, ext_price#40], [ext_price#40 DESC NULLS LAST, brand_id#38 ASC NULLS FIRST] - -(40) CometColumnarToRow [codegen id : 11] -Input [5]: [brand_id#38, brand#39, t_hour#32, t_minute#33, ext_price#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 6 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometNativeScan parquet spark_catalog.default.date_dim (41) - - -(41) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#41, d_moy#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#11, d_year#41, d_moy#42] -Condition : ((((isnotnull(d_moy#42) AND isnotnull(d_year#41)) AND (d_moy#42 = 11)) AND (d_year#41 = 1999)) AND isnotnull(d_date_sk#11)) - -(43) CometProject -Input [3]: [d_date_sk#11, d_year#41, d_moy#42] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(45) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 12 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#10 - -Subquery:3 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#10 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/simplified.txt deleted file mode 100644 index 838a3e6604..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometColumnarExchange [ext_price,brand_id] #1 - WholeStageCodegen (10) - HashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] [sum(UnscaledValue(ext_price)),brand_id,brand,ext_price,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand,i_brand_id,t_hour,t_minute] #2 - WholeStageCodegen (9) - HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] - Project [i_brand_id,i_brand,ext_price,t_hour,t_minute] - BroadcastHashJoin [time_sk,t_time_sk] - Project [i_brand_id,i_brand,ext_price,time_sk] - BroadcastHashJoin [i_item_sk,sold_item_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - InputAdapter - Union - WholeStageCodegen (3) - Project [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk,ws_sold_time_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (5) - Project [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_time_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (7) - Project [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_sold_time_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] - CometNativeScan 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/q71.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/explain.txt deleted file mode 100644 index bb217d6417..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,251 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (37) -+- CometSort (36) - +- CometExchange (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometBroadcastExchange (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : +- CometUnion (23) - : :- CometProject (12) - : : +- CometBroadcastHashJoin (11) - : : :- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (5) - : : +- CometBroadcastExchange (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (7) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (13) - : : +- ReusedExchange (15) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (19) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (18) - : +- ReusedExchange (20) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Condition : ((isnotnull(i_manager_id#4) AND (i_manager_id#4 = 1)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Arguments: [i_item_sk#1, i_brand_id#2, i_brand#5], [i_item_sk#1, i_brand_id#2, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#3, 50, true, false, true) AS i_brand#5] - -(4) CometBroadcastExchange -Input [3]: [i_item_sk#1, i_brand_id#2, i_brand#5] -Arguments: [i_item_sk#1, i_brand_id#2, i_brand#5] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_time_sk)] -ReadSchema: struct - -(6) CometFilter -Input [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Condition : (isnotnull(ws_item_sk#7) AND isnotnull(ws_sold_time_sk#6)) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_moy#13) AND isnotnull(d_year#12)) AND (d_moy#13 = 11)) AND (d_year#12 = 1999)) AND isnotnull(d_date_sk#11)) - -(9) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(10) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(11) CometBroadcastHashJoin -Left output [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [ws_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(12) CometProject -Input [5]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9, d_date_sk#11] -Arguments: [ext_price#14, sold_item_sk#15, time_sk#16], [ws_ext_sales_price#8 AS ext_price#14, ws_item_sk#7 AS sold_item_sk#15, ws_sold_time_sk#6 AS time_sk#16] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#21)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_time_sk)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#18) AND isnotnull(cs_sold_time_sk#17)) - -(15) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#22] - -(16) CometBroadcastHashJoin -Left output [4]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Right output [1]: [d_date_sk#22] -Arguments: [cs_sold_date_sk#20], [d_date_sk#22], Inner, BuildRight - -(17) CometProject -Input [5]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] -Arguments: [ext_price#23, sold_item_sk#24, time_sk#25], [cs_ext_sales_price#19 AS ext_price#23, cs_item_sk#18 AS sold_item_sk#24, cs_sold_time_sk#17 AS time_sk#25] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] -ReadSchema: struct - -(19) CometFilter -Input [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Condition : (isnotnull(ss_item_sk#27) AND isnotnull(ss_sold_time_sk#26)) - -(20) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#31] - -(21) CometBroadcastHashJoin -Left output [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Right output [1]: [d_date_sk#31] -Arguments: [ss_sold_date_sk#29], [d_date_sk#31], Inner, BuildRight - -(22) CometProject -Input [5]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#31] -Arguments: [ext_price#32, sold_item_sk#33, time_sk#34], [ss_ext_sales_price#28 AS ext_price#32, ss_item_sk#27 AS sold_item_sk#33, ss_sold_time_sk#26 AS time_sk#34] - -(23) CometUnion -Child 0 Input [3]: [ext_price#14, sold_item_sk#15, time_sk#16] -Child 1 Input [3]: [ext_price#23, sold_item_sk#24, time_sk#25] -Child 2 Input [3]: [ext_price#32, sold_item_sk#33, time_sk#34] - -(24) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand_id#2, i_brand#5] -Right output [3]: [ext_price#14, sold_item_sk#15, time_sk#16] -Arguments: [i_item_sk#1], [sold_item_sk#15], Inner, BuildLeft - -(25) CometProject -Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#5, ext_price#14, sold_item_sk#15, time_sk#16] -Arguments: [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16], [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [4]: [t_time_sk#35, t_hour#36, t_minute#37, t_meal_time#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_time_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [t_time_sk#35, t_hour#36, t_minute#37, t_meal_time#38] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, t_meal_time#38, 20, true, false, true) = breakfast ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, t_meal_time#38, 20, true, false, true) = dinner )) AND isnotnull(t_time_sk#35)) - -(28) CometProject -Input [4]: [t_time_sk#35, t_hour#36, t_minute#37, t_meal_time#38] -Arguments: [t_time_sk#35, t_hour#36, t_minute#37], [t_time_sk#35, t_hour#36, t_minute#37] - -(29) CometBroadcastExchange -Input [3]: [t_time_sk#35, t_hour#36, t_minute#37] -Arguments: [t_time_sk#35, t_hour#36, t_minute#37] - -(30) CometBroadcastHashJoin -Left output [4]: [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16] -Right output [3]: [t_time_sk#35, t_hour#36, t_minute#37] -Arguments: [time_sk#16], [t_time_sk#35], Inner, BuildRight - -(31) CometProject -Input [7]: [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16, t_time_sk#35, t_hour#36, t_minute#37] -Arguments: [i_brand_id#2, i_brand#5, ext_price#14, t_hour#36, t_minute#37], [i_brand_id#2, i_brand#5, ext_price#14, t_hour#36, t_minute#37] - -(32) CometHashAggregate -Input [5]: [i_brand_id#2, i_brand#5, ext_price#14, t_hour#36, t_minute#37] -Keys [4]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37] -Functions [1]: [partial_sum(UnscaledValue(ext_price#14))] - -(33) CometExchange -Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] -Arguments: hashpartitioning(i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(34) CometHashAggregate -Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] -Keys [4]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37] -Functions [1]: [sum(UnscaledValue(ext_price#14))] - -(35) CometExchange -Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] -Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(36) CometSort -Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] -Arguments: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42], [ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST] - -(37) CometColumnarToRow [codegen id : 1] -Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (42) -+- * CometColumnarToRow (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) - - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(39) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_moy#13) AND isnotnull(d_year#12)) AND (d_moy#13 = 11)) AND (d_year#12 = 1999)) AND isnotnull(d_date_sk#11)) - -(40) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(41) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(42) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#10 - -Subquery:3 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#10 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/simplified.txt deleted file mode 100644 index 63e7c353ac..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometExchange [ext_price,brand_id] #1 - CometHashAggregate [sum] [brand_id,brand,t_hour,t_minute,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ext_price))] - CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 - CometHashAggregate [ext_price] [i_brand,i_brand_id,t_hour,t_minute,sum] - CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] - 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,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - 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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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_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 [native_iceberg_compat] 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_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 [native_iceberg_compat] 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 [t_time_sk,t_hour,t_minute] #6 - CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] - CometScan [native_iceberg_compat] 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/q71/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/explain.txt deleted file mode 100644 index bb217d6417..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/explain.txt +++ /dev/null @@ -1,251 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (37) -+- CometSort (36) - +- CometExchange (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometBroadcastExchange (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : +- CometUnion (23) - : :- CometProject (12) - : : +- CometBroadcastHashJoin (11) - : : :- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (5) - : : +- CometBroadcastExchange (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (7) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (13) - : : +- ReusedExchange (15) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (19) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (18) - : +- ReusedExchange (20) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Condition : ((isnotnull(i_manager_id#4) AND (i_manager_id#4 = 1)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Arguments: [i_item_sk#1, i_brand_id#2, i_brand#5], [i_item_sk#1, i_brand_id#2, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#3, 50, true, false, true) AS i_brand#5] - -(4) CometBroadcastExchange -Input [3]: [i_item_sk#1, i_brand_id#2, i_brand#5] -Arguments: [i_item_sk#1, i_brand_id#2, i_brand#5] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_time_sk)] -ReadSchema: struct - -(6) CometFilter -Input [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Condition : (isnotnull(ws_item_sk#7) AND isnotnull(ws_sold_time_sk#6)) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_moy#13) AND isnotnull(d_year#12)) AND (d_moy#13 = 11)) AND (d_year#12 = 1999)) AND isnotnull(d_date_sk#11)) - -(9) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(10) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(11) CometBroadcastHashJoin -Left output [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [ws_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(12) CometProject -Input [5]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9, d_date_sk#11] -Arguments: [ext_price#14, sold_item_sk#15, time_sk#16], [ws_ext_sales_price#8 AS ext_price#14, ws_item_sk#7 AS sold_item_sk#15, ws_sold_time_sk#6 AS time_sk#16] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#21)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_time_sk)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#18) AND isnotnull(cs_sold_time_sk#17)) - -(15) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#22] - -(16) CometBroadcastHashJoin -Left output [4]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Right output [1]: [d_date_sk#22] -Arguments: [cs_sold_date_sk#20], [d_date_sk#22], Inner, BuildRight - -(17) CometProject -Input [5]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] -Arguments: [ext_price#23, sold_item_sk#24, time_sk#25], [cs_ext_sales_price#19 AS ext_price#23, cs_item_sk#18 AS sold_item_sk#24, cs_sold_time_sk#17 AS time_sk#25] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] -ReadSchema: struct - -(19) CometFilter -Input [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Condition : (isnotnull(ss_item_sk#27) AND isnotnull(ss_sold_time_sk#26)) - -(20) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#31] - -(21) CometBroadcastHashJoin -Left output [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Right output [1]: [d_date_sk#31] -Arguments: [ss_sold_date_sk#29], [d_date_sk#31], Inner, BuildRight - -(22) CometProject -Input [5]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#31] -Arguments: [ext_price#32, sold_item_sk#33, time_sk#34], [ss_ext_sales_price#28 AS ext_price#32, ss_item_sk#27 AS sold_item_sk#33, ss_sold_time_sk#26 AS time_sk#34] - -(23) CometUnion -Child 0 Input [3]: [ext_price#14, sold_item_sk#15, time_sk#16] -Child 1 Input [3]: [ext_price#23, sold_item_sk#24, time_sk#25] -Child 2 Input [3]: [ext_price#32, sold_item_sk#33, time_sk#34] - -(24) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand_id#2, i_brand#5] -Right output [3]: [ext_price#14, sold_item_sk#15, time_sk#16] -Arguments: [i_item_sk#1], [sold_item_sk#15], Inner, BuildLeft - -(25) CometProject -Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#5, ext_price#14, sold_item_sk#15, time_sk#16] -Arguments: [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16], [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [4]: [t_time_sk#35, t_hour#36, t_minute#37, t_meal_time#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_time_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [t_time_sk#35, t_hour#36, t_minute#37, t_meal_time#38] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, t_meal_time#38, 20, true, false, true) = breakfast ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, t_meal_time#38, 20, true, false, true) = dinner )) AND isnotnull(t_time_sk#35)) - -(28) CometProject -Input [4]: [t_time_sk#35, t_hour#36, t_minute#37, t_meal_time#38] -Arguments: [t_time_sk#35, t_hour#36, t_minute#37], [t_time_sk#35, t_hour#36, t_minute#37] - -(29) CometBroadcastExchange -Input [3]: [t_time_sk#35, t_hour#36, t_minute#37] -Arguments: [t_time_sk#35, t_hour#36, t_minute#37] - -(30) CometBroadcastHashJoin -Left output [4]: [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16] -Right output [3]: [t_time_sk#35, t_hour#36, t_minute#37] -Arguments: [time_sk#16], [t_time_sk#35], Inner, BuildRight - -(31) CometProject -Input [7]: [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16, t_time_sk#35, t_hour#36, t_minute#37] -Arguments: [i_brand_id#2, i_brand#5, ext_price#14, t_hour#36, t_minute#37], [i_brand_id#2, i_brand#5, ext_price#14, t_hour#36, t_minute#37] - -(32) CometHashAggregate -Input [5]: [i_brand_id#2, i_brand#5, ext_price#14, t_hour#36, t_minute#37] -Keys [4]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37] -Functions [1]: [partial_sum(UnscaledValue(ext_price#14))] - -(33) CometExchange -Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] -Arguments: hashpartitioning(i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(34) CometHashAggregate -Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] -Keys [4]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37] -Functions [1]: [sum(UnscaledValue(ext_price#14))] - -(35) CometExchange -Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] -Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(36) CometSort -Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] -Arguments: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42], [ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST] - -(37) CometColumnarToRow [codegen id : 1] -Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (42) -+- * CometColumnarToRow (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) - - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(39) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_moy#13) AND isnotnull(d_year#12)) AND (d_moy#13 = 11)) AND (d_year#12 = 1999)) AND isnotnull(d_date_sk#11)) - -(40) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(41) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(42) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#10 - -Subquery:3 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#10 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/extended.txt deleted file mode 100644 index 25ea15c536..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/extended.txt +++ /dev/null @@ -1,53 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometUnion - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - -Comet accelerated 45 out of 49 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 63e7c353ac..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometExchange [ext_price,brand_id] #1 - CometHashAggregate [sum] [brand_id,brand,t_hour,t_minute,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ext_price))] - CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 - CometHashAggregate [ext_price] [i_brand,i_brand_id,t_hour,t_minute,sum] - CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] - 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,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - 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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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_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 [native_iceberg_compat] 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_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 [native_iceberg_compat] 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 [t_time_sk,t_hour,t_minute] #6 - CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/explain.txt deleted file mode 100644 index ed3a4a7aea..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/explain.txt +++ /dev/null @@ -1,428 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (70) -+- CometTakeOrderedAndProject (69) - +- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometProject (65) - +- CometSortMergeJoin (64) - :- CometSort (58) - : +- CometColumnarExchange (57) - : +- * Project (56) - : +- * BroadcastHashJoin LeftOuter BuildRight (55) - : :- * Project (50) - : : +- * BroadcastHashJoin Inner BuildRight (49) - : : :- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (28) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Project (15) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : : :- * Project (9) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : : : : :- * Filter (3) - : : : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * CometColumnarToRow (6) - : : : : : : : : : +- CometFilter (5) - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory (4) - : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * CometColumnarToRow (12) - : : : : : : : : +- CometFilter (11) - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * CometColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) - : : : : : : +- BroadcastExchange (26) - : : : : : : +- * CometColumnarToRow (25) - : : : : : : +- CometProject (24) - : : : : : : +- CometFilter (23) - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (22) - : : : : : +- BroadcastExchange (33) - : : : : : +- * CometColumnarToRow (32) - : : : : : +- CometProject (31) - : : : : : +- CometFilter (30) - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (29) - : : : : +- ReusedExchange (36) - : : : +- BroadcastExchange (42) - : : : +- * CometColumnarToRow (41) - : : : +- CometFilter (40) - : : : +- CometNativeScan parquet spark_catalog.default.date_dim (39) - : : +- BroadcastExchange (48) - : : +- * CometColumnarToRow (47) - : : +- CometFilter (46) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (45) - : +- BroadcastExchange (54) - : +- * CometColumnarToRow (53) - : +- CometFilter (52) - : +- CometNativeScan parquet spark_catalog.default.promotion (51) - +- CometSort (63) - +- CometExchange (62) - +- CometProject (61) - +- CometFilter (60) - +- CometNativeScan parquet spark_catalog.default.catalog_returns (59) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 10] -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] - -(3) Filter [codegen id : 10] -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) - -(4) CometNativeScan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(inv_date_sk#13)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) - -(6) CometColumnarToRow [codegen id : 1] -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(7) BroadcastExchange -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [inv_item_sk#10] -Join type: Inner -Join condition: (inv_quantity_on_hand#12 < cs_quantity#7) - -(9) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(10) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Condition : isnotnull(w_warehouse_sk#14) - -(12) CometColumnarToRow [codegen id : 2] -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] - -(13) BroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_warehouse_sk#11] -Right keys [1]: [w_warehouse_sk#14] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] - -(16) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_desc#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [i_item_sk#16, i_item_desc#17] -Condition : isnotnull(i_item_sk#16) - -(18) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#16, i_item_desc#17] - -(19) BroadcastExchange -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(20) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#16] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 10] -Output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] - -(22) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#18, cd_marital_status#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(23) CometFilter -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = D) AND isnotnull(cd_demo_sk#18)) - -(24) CometProject -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Arguments: [cd_demo_sk#18], [cd_demo_sk#18] - -(25) CometColumnarToRow [codegen id : 4] -Input [1]: [cd_demo_sk#18] - -(26) BroadcastExchange -Input [1]: [cd_demo_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(27) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] - -(29) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_buy_potential#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#21, 15, true, false, true) = >10000 ) AND isnotnull(hd_demo_sk#20)) - -(31) CometProject -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Arguments: [hd_demo_sk#20], [hd_demo_sk#20] - -(32) CometColumnarToRow [codegen id : 5] -Input [1]: [hd_demo_sk#20] - -(33) BroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] - -(36) ReusedExchange [Reuses operator id: 75] -Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#8] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] - -(39) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_week_seq#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(40) CometFilter -Input [2]: [d_date_sk#25, d_week_seq#26] -Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) - -(41) CometColumnarToRow [codegen id : 7] -Input [2]: [d_date_sk#25, d_week_seq#26] - -(42) BroadcastExchange -Input [2]: [d_date_sk#25, d_week_seq#26] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] - -(43) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [d_week_seq#24, inv_date_sk#13] -Right keys [2]: [d_week_seq#26, d_date_sk#25] -Join type: Inner -Join condition: None - -(44) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#25, d_week_seq#26] - -(45) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#27, d_date#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [d_date_sk#27, d_date#28] -Condition : (isnotnull(d_date#28) AND isnotnull(d_date_sk#27)) - -(47) CometColumnarToRow [codegen id : 8] -Input [2]: [d_date_sk#27, d_date#28] - -(48) BroadcastExchange -Input [2]: [d_date_sk#27, d_date#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: (d_date#28 > date_add(d_date#23, 5)) - -(50) Project [codegen id : 10] -Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#27, d_date#28] - -(51) CometNativeScan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(52) CometFilter -Input [1]: [p_promo_sk#29] -Condition : isnotnull(p_promo_sk#29) - -(53) CometColumnarToRow [codegen id : 9] -Input [1]: [p_promo_sk#29] - -(54) BroadcastExchange -Input [1]: [p_promo_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(55) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_promo_sk#5] -Right keys [1]: [p_promo_sk#29] -Join type: LeftOuter -Join condition: None - -(56) Project [codegen id : 10] -Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] - -(57) CometColumnarExchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(58) CometSort -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] - -(59) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(60) CometFilter -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Condition : (isnotnull(cr_item_sk#30) AND isnotnull(cr_order_number#31)) - -(61) CometProject -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30, cr_order_number#31] - -(62) CometExchange -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(63) CometSort -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST] - -(64) CometSortMergeJoin -Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#30, cr_order_number#31], LeftOuter - -(65) CometProject -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#30, cr_order_number#31] -Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(66) CometHashAggregate -Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [partial_count(1)] - -(67) CometExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(68) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [count(1)] - -(69) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#36 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#34,promo#35,total_cnt#36]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36], 100, 0, [total_cnt#36 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] - -(70) CometColumnarToRow [codegen id : 11] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (75) -+- * CometColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometNativeScan parquet spark_catalog.default.date_dim (71) - - -(71) CometNativeScan parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(72) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] -Condition : ((((isnotnull(d_year#37) AND (d_year#37 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(73) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(74) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(75) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/simplified.txt deleted file mode 100644 index 025217a607..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/simplified.txt +++ /dev/null @@ -1,107 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] - CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] - CometProject [w_warehouse_name,i_item_desc,d_week_seq] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometColumnarExchange [cs_item_sk,cs_order_number] #2 - WholeStageCodegen (10) - Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [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] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometNativeScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometNativeScan 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/q72.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/explain.txt deleted file mode 100644 index bf5bcb475b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,402 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (64) -+- CometTakeOrderedAndProject (63) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometSortMergeJoin (58) - :- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (45) - : : +- CometBroadcastHashJoin (44) - : : :- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (35) - : : : : +- CometBroadcastHashJoin (34) - : : : : :- CometProject (29) - : : : : : +- CometBroadcastHashJoin (28) - : : : : : :- CometProject (23) - : : : : : : +- CometBroadcastHashJoin (22) - : : : : : : :- CometProject (17) - : : : : : : : +- CometBroadcastHashJoin (16) - : : : : : : : :- CometProject (12) - : : : : : : : : +- CometBroadcastHashJoin (11) - : : : : : : : : :- CometProject (7) - : : : : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : : : : :- CometFilter (2) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- CometBroadcastExchange (5) - : : : : : : : : : +- CometFilter (4) - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (3) - : : : : : : : : +- CometBroadcastExchange (10) - : : : : : : : : +- CometFilter (9) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : : : : : : : +- CometBroadcastExchange (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (13) - : : : : : : +- CometBroadcastExchange (21) - : : : : : : +- CometProject (20) - : : : : : : +- CometFilter (19) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (18) - : : : : : +- CometBroadcastExchange (27) - : : : : : +- CometProject (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (24) - : : : : +- CometBroadcastExchange (33) - : : : : +- CometProject (32) - : : : : +- CometFilter (31) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - : : : +- CometBroadcastExchange (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (36) - : : +- CometBroadcastExchange (43) - : : +- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - : +- CometBroadcastExchange (48) - : +- CometFilter (47) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (46) - +- CometSort (57) - +- CometExchange (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (53) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#13)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) - -(5) CometBroadcastExchange -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(6) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Right output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_item_sk#4], [inv_item_sk#10], Inner, (inv_quantity_on_hand#12 < cs_quantity#7), BuildRight - -(7) CometProject -Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Condition : isnotnull(w_warehouse_sk#14) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [w_warehouse_sk#14, w_warehouse_name#15] - -(11) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -Right output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [inv_warehouse_sk#11], [w_warehouse_sk#14], Inner, BuildRight - -(12) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_desc#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [i_item_sk#16, i_item_desc#17] -Condition : isnotnull(i_item_sk#16) - -(15) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [i_item_sk#16, i_item_desc#17] - -(16) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -Right output [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [cs_item_sk#4], [i_item_sk#16], Inner, BuildRight - -(17) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#18, cd_marital_status#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = D) AND isnotnull(cd_demo_sk#18)) - -(20) CometProject -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Arguments: [cd_demo_sk#18], [cd_demo_sk#18] - -(21) CometBroadcastExchange -Input [1]: [cd_demo_sk#18] -Arguments: [cd_demo_sk#18] - -(22) CometBroadcastHashJoin -Left output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [cd_demo_sk#18] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#18], Inner, BuildRight - -(23) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] -Arguments: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_buy_potential#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#21, 15, true, false, true) = >10000 ) AND isnotnull(hd_demo_sk#20)) - -(26) CometProject -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Arguments: [hd_demo_sk#20], [hd_demo_sk#20] - -(27) CometBroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: [hd_demo_sk#20] - -(28) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [hd_demo_sk#20] -Arguments: [cs_bill_hdemo_sk#3], [hd_demo_sk#20], Inner, BuildRight - -(29) CometProject -Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(31) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(32) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(33) CometBroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24] - -(34) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_sold_date_sk#8], [d_date_sk#22], Inner, BuildRight - -(35) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_week_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [d_date_sk#26, d_week_seq#27] -Condition : (isnotnull(d_week_seq#27) AND isnotnull(d_date_sk#26)) - -(38) CometBroadcastExchange -Input [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_date_sk#26, d_week_seq#27] - -(39) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_week_seq#24, inv_date_sk#13], [d_week_seq#27, d_date_sk#26], Inner, BuildRight - -(40) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#26, d_week_seq#27] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_date#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#28, d_date#29] -Condition : (isnotnull(d_date#29) AND isnotnull(d_date_sk#28)) - -(43) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_date#29] -Arguments: [d_date_sk#28, d_date#29] - -(44) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#28, d_date#29] -Arguments: [cs_ship_date_sk#1], [d_date_sk#28], Inner, (d_date#29 > date_add(d_date#23, 5)), BuildRight - -(45) CometProject -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#28, d_date#29] -Arguments: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(47) CometFilter -Input [1]: [p_promo_sk#30] -Condition : isnotnull(p_promo_sk#30) - -(48) CometBroadcastExchange -Input [1]: [p_promo_sk#30] -Arguments: [p_promo_sk#30] - -(49) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [1]: [p_promo_sk#30] -Arguments: [cs_promo_sk#5], [p_promo_sk#30], LeftOuter, BuildRight - -(50) CometProject -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#30] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(51) CometExchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(52) CometSort -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Condition : (isnotnull(cr_item_sk#31) AND isnotnull(cr_order_number#32)) - -(55) CometProject -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31, cr_order_number#32] - -(56) CometExchange -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(57) CometSort -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#31, cr_order_number#32], LeftOuter - -(59) CometProject -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#31, cr_order_number#32] -Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(60) CometHashAggregate -Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [partial_count(1)] - -(61) CometExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(62) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [count(1)] - -(63) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, 0, [total_cnt#37 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -(64) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (69) -+- * CometColumnarToRow (68) - +- CometProject (67) - +- CometFilter (66) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) - - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(67) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(68) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(69) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/simplified.txt deleted file mode 100644 index 678f8c67df..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] - CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] - CometProject [w_warehouse_name,i_item_desc,d_week_seq] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometExchange [cs_item_sk,cs_order_number] #2 - CometProject [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,p_promo_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_date] - 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 [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_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_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_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_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 [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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_desc] #6 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange [cd_demo_sk] #7 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange [hd_demo_sk] #8 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [d_date_sk,d_week_seq] #10 - CometFilter [d_date_sk,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [d_date_sk,d_date] #11 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [p_promo_sk] #12 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/explain.txt deleted file mode 100644 index bf5bcb475b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/explain.txt +++ /dev/null @@ -1,402 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (64) -+- CometTakeOrderedAndProject (63) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometSortMergeJoin (58) - :- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (45) - : : +- CometBroadcastHashJoin (44) - : : :- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (35) - : : : : +- CometBroadcastHashJoin (34) - : : : : :- CometProject (29) - : : : : : +- CometBroadcastHashJoin (28) - : : : : : :- CometProject (23) - : : : : : : +- CometBroadcastHashJoin (22) - : : : : : : :- CometProject (17) - : : : : : : : +- CometBroadcastHashJoin (16) - : : : : : : : :- CometProject (12) - : : : : : : : : +- CometBroadcastHashJoin (11) - : : : : : : : : :- CometProject (7) - : : : : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : : : : :- CometFilter (2) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- CometBroadcastExchange (5) - : : : : : : : : : +- CometFilter (4) - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (3) - : : : : : : : : +- CometBroadcastExchange (10) - : : : : : : : : +- CometFilter (9) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : : : : : : : +- CometBroadcastExchange (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (13) - : : : : : : +- CometBroadcastExchange (21) - : : : : : : +- CometProject (20) - : : : : : : +- CometFilter (19) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (18) - : : : : : +- CometBroadcastExchange (27) - : : : : : +- CometProject (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (24) - : : : : +- CometBroadcastExchange (33) - : : : : +- CometProject (32) - : : : : +- CometFilter (31) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - : : : +- CometBroadcastExchange (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (36) - : : +- CometBroadcastExchange (43) - : : +- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - : +- CometBroadcastExchange (48) - : +- CometFilter (47) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (46) - +- CometSort (57) - +- CometExchange (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (53) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#13)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) - -(5) CometBroadcastExchange -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(6) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Right output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_item_sk#4], [inv_item_sk#10], Inner, (inv_quantity_on_hand#12 < cs_quantity#7), BuildRight - -(7) CometProject -Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Condition : isnotnull(w_warehouse_sk#14) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [w_warehouse_sk#14, w_warehouse_name#15] - -(11) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -Right output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [inv_warehouse_sk#11], [w_warehouse_sk#14], Inner, BuildRight - -(12) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_desc#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [i_item_sk#16, i_item_desc#17] -Condition : isnotnull(i_item_sk#16) - -(15) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [i_item_sk#16, i_item_desc#17] - -(16) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -Right output [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [cs_item_sk#4], [i_item_sk#16], Inner, BuildRight - -(17) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#18, cd_marital_status#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = D) AND isnotnull(cd_demo_sk#18)) - -(20) CometProject -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Arguments: [cd_demo_sk#18], [cd_demo_sk#18] - -(21) CometBroadcastExchange -Input [1]: [cd_demo_sk#18] -Arguments: [cd_demo_sk#18] - -(22) CometBroadcastHashJoin -Left output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [cd_demo_sk#18] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#18], Inner, BuildRight - -(23) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] -Arguments: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_buy_potential#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#21, 15, true, false, true) = >10000 ) AND isnotnull(hd_demo_sk#20)) - -(26) CometProject -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Arguments: [hd_demo_sk#20], [hd_demo_sk#20] - -(27) CometBroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: [hd_demo_sk#20] - -(28) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [hd_demo_sk#20] -Arguments: [cs_bill_hdemo_sk#3], [hd_demo_sk#20], Inner, BuildRight - -(29) CometProject -Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(31) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(32) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(33) CometBroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24] - -(34) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_sold_date_sk#8], [d_date_sk#22], Inner, BuildRight - -(35) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_week_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [d_date_sk#26, d_week_seq#27] -Condition : (isnotnull(d_week_seq#27) AND isnotnull(d_date_sk#26)) - -(38) CometBroadcastExchange -Input [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_date_sk#26, d_week_seq#27] - -(39) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_week_seq#24, inv_date_sk#13], [d_week_seq#27, d_date_sk#26], Inner, BuildRight - -(40) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#26, d_week_seq#27] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_date#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#28, d_date#29] -Condition : (isnotnull(d_date#29) AND isnotnull(d_date_sk#28)) - -(43) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_date#29] -Arguments: [d_date_sk#28, d_date#29] - -(44) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#28, d_date#29] -Arguments: [cs_ship_date_sk#1], [d_date_sk#28], Inner, (d_date#29 > date_add(d_date#23, 5)), BuildRight - -(45) CometProject -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#28, d_date#29] -Arguments: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(47) CometFilter -Input [1]: [p_promo_sk#30] -Condition : isnotnull(p_promo_sk#30) - -(48) CometBroadcastExchange -Input [1]: [p_promo_sk#30] -Arguments: [p_promo_sk#30] - -(49) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [1]: [p_promo_sk#30] -Arguments: [cs_promo_sk#5], [p_promo_sk#30], LeftOuter, BuildRight - -(50) CometProject -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#30] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(51) CometExchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(52) CometSort -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Condition : (isnotnull(cr_item_sk#31) AND isnotnull(cr_order_number#32)) - -(55) CometProject -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31, cr_order_number#32] - -(56) CometExchange -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(57) CometSort -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#31, cr_order_number#32], LeftOuter - -(59) CometProject -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#31, cr_order_number#32] -Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(60) CometHashAggregate -Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [partial_count(1)] - -(61) CometExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(62) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [count(1)] - -(63) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, 0, [total_cnt#37 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -(64) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (69) -+- * CometColumnarToRow (68) - +- CometProject (67) - +- CometFilter (66) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) - - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(67) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(68) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(69) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/extended.txt deleted file mode 100644 index 9000d27d1b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/extended.txt +++ /dev/null @@ -1,72 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 678f8c67df..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] - CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] - CometProject [w_warehouse_name,i_item_desc,d_week_seq] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometExchange [cs_item_sk,cs_order_number] #2 - CometProject [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,p_promo_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_date] - 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 [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_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_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_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_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 [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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_desc] #6 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange [cd_demo_sk] #7 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange [hd_demo_sk] #8 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [d_date_sk,d_week_seq] #10 - CometFilter [d_date_sk,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [d_date_sk,d_date] #11 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [p_promo_sk] #12 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/explain.txt deleted file mode 100644 index 7fe1aa758c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/explain.txt +++ /dev/null @@ -1,231 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (35) -+- CometSort (34) - +- CometColumnarExchange (33) - +- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * Filter (25) - : +- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (30) - +- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometNativeScan parquet spark_catalog.default.customer (26) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 40] -Output [1]: [d_date_sk#7] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_county#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_county, [Bronx County,Franklin Parish,Orange County,Williamson County]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [s_store_sk#8, s_county#9] -Condition : (s_county#9 IN (Williamson County,Franklin Parish,Bronx County,Orange County) AND isnotnull(s_store_sk#8)) - -(9) CometProject -Input [2]: [s_store_sk#8, s_county#9] -Arguments: [s_store_sk#8], [s_store_sk#8] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#8] - -(11) BroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] - -(14) CometNativeScan parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : ((((isnotnull(hd_vehicle_count#13) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#11, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#11, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#12 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#13 as double)))))) > 1.0) END) AND isnotnull(hd_demo_sk#10)) - -(16) CometProject -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Arguments: [hd_demo_sk#10], [hd_demo_sk#10] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#10] - -(18) BroadcastExchange -Input [1]: [hd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#10] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [2]: [ss_customer_sk#1, ss_ticket_number#4] -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] - -(21) HashAggregate [codegen id : 4] -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#14] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(22) CometColumnarExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(24) HashAggregate [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#16] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] - -(25) Filter [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 1) AND (cnt#17 <= 5)) - -(26) CometNativeScan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(27) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(28) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] - -(29) CometColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(30) BroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#18] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 6] -Output [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(33) CometColumnarExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(34) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [cnt#17 DESC NULLS LAST] - -(35) CometColumnarToRow [codegen id : 7] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (40) -+- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometNativeScan parquet spark_catalog.default.date_dim (36) - - -(36) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#27, d_dom#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [d_date_sk#7, d_year#27, d_dom#28] -Condition : ((((isnotnull(d_dom#28) AND (d_dom#28 >= 1)) AND (d_dom#28 <= 2)) AND d_year#27 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(38) CometProject -Input [3]: [d_date_sk#7, d_year#27, d_dom#28] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(39) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(40) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/simplified.txt deleted file mode 100644 index 4d2a6a74ab..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/simplified.txt +++ /dev/null @@ -1,59 +0,0 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [cnt] #1 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometNativeScan 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/q73.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/explain.txt deleted file mode 100644 index 346e110371..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometSort (32) - +- CometExchange (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (24) - : +- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_county#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_county, [Bronx County,Franklin Parish,Orange County,Williamson County]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#10, s_county#11] -Condition : (s_county#11 IN (Williamson County,Franklin Parish,Bronx County,Orange County) AND isnotnull(s_store_sk#10)) - -(11) CometProject -Input [2]: [s_store_sk#10, s_county#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : ((((isnotnull(hd_vehicle_count#15) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#14 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#15 as double)))))) > 1.0) END) AND isnotnull(hd_demo_sk#12)) - -(17) CometProject -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#12], [hd_demo_sk#12] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#12] -Arguments: [hd_demo_sk#12] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Right output [1]: [hd_demo_sk#12] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight - -(20) CometProject -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] -Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] - -(21) CometHashAggregate -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] - -(22) CometExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] - -(24) CometFilter -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 1) AND (cnt#17 <= 5)) - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(27) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(29) CometBroadcastHashJoin -Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Right output [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -(31) CometExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(32) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [cnt#17 DESC NULLS LAST] - -(33) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(36) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(37) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(38) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/simplified.txt deleted file mode 100644 index 227dd00f2b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometExchange [cnt] #1 - CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] - CometExchange [ss_ticket_number,ss_customer_sk] #2 - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - 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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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/q73/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/explain.txt deleted file mode 100644 index 346e110371..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometSort (32) - +- CometExchange (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (24) - : +- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_county#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_county, [Bronx County,Franklin Parish,Orange County,Williamson County]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#10, s_county#11] -Condition : (s_county#11 IN (Williamson County,Franklin Parish,Bronx County,Orange County) AND isnotnull(s_store_sk#10)) - -(11) CometProject -Input [2]: [s_store_sk#10, s_county#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : ((((isnotnull(hd_vehicle_count#15) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#14 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#15 as double)))))) > 1.0) END) AND isnotnull(hd_demo_sk#12)) - -(17) CometProject -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#12], [hd_demo_sk#12] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#12] -Arguments: [hd_demo_sk#12] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Right output [1]: [hd_demo_sk#12] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight - -(20) CometProject -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] -Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] - -(21) CometHashAggregate -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] - -(22) CometExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] - -(24) CometFilter -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 1) AND (cnt#17 <= 5)) - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(27) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(29) CometBroadcastHashJoin -Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Right output [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -(31) CometExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(32) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [cnt#17 DESC NULLS LAST] - -(33) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(36) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(37) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(38) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/extended.txt deleted file mode 100644 index 3d9d23d2cc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/extended.txt +++ /dev/null @@ -1,41 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 227dd00f2b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometExchange [cnt] #1 - CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] - CometExchange [ss_ticket_number,ss_customer_sk] #2 - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - 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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/explain.txt deleted file mode 100644 index c019668fc8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/explain.txt +++ /dev/null @@ -1,513 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (79) -+- * Project (78) - +- * BroadcastHashJoin Inner BuildRight (77) - :- * Project (58) - : +- * BroadcastHashJoin Inner BuildRight (57) - : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- * HashAggregate (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * Project (28) - : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (26) - : : : +- * Filter (25) - : : : +- * ColumnarToRow (24) - : : : +- Scan parquet spark_catalog.default.store_sales (23) - : : +- ReusedExchange (29) - : +- BroadcastExchange (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- * CometColumnarToRow (53) - : +- CometColumnarExchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (41) - : : : +- CometProject (40) - : : : +- CometFilter (39) - : : : +- CometNativeScan parquet spark_catalog.default.customer (38) - : : +- BroadcastExchange (45) - : : +- * Filter (44) - : : +- * ColumnarToRow (43) - : : +- Scan parquet spark_catalog.default.web_sales (42) - : +- ReusedExchange (48) - +- BroadcastExchange (76) - +- * HashAggregate (75) - +- * CometColumnarToRow (74) - +- CometColumnarExchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * CometColumnarToRow (62) - : : +- CometProject (61) - : : +- CometFilter (60) - : : +- CometNativeScan parquet spark_catalog.default.customer (59) - : +- BroadcastExchange (66) - : +- * Filter (65) - : +- * ColumnarToRow (64) - : +- Scan parquet spark_catalog.default.web_sales (63) - +- ReusedExchange (69) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#7] - -(4) CometColumnarToRow [codegen id : 3] -Input [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] - -(5) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(7) Filter [codegen id : 1] -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Condition : isnotnull(ss_customer_sk#8) - -(8) BroadcastExchange -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#8] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(11) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#12, d_year#13] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#10] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] - -(14) HashAggregate [codegen id : 3] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] -Aggregate Attributes [1]: [sum#14] -Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] - -(15) CometColumnarExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 16] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] - -(17) HashAggregate [codegen id : 16] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#9))#16] -Results [2]: [c_customer_id#5 AS customer_id#17, MakeDecimal(sum(UnscaledValue(ss_net_paid#9))#16,17,2) AS year_total#18] - -(18) Filter [codegen id : 16] -Input [2]: [customer_id#17, year_total#18] -Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) - -(19) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(20) CometFilter -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Condition : (isnotnull(c_customer_sk#19) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#20, 16, true, false, true))) - -(21) CometProject -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Arguments: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#20, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#21, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#22, 30, true, false, true) AS c_last_name#7] - -(22) CometColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7] - -(23) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] - -(25) Filter [codegen id : 4] -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_customer_sk#23) - -(26) BroadcastExchange -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#19] -Right keys [1]: [ss_customer_sk#23] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25] -Input [7]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] - -(29) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#27, d_year#28] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#25] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25, d_date_sk#27, d_year#28] - -(32) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#24))] -Aggregate Attributes [1]: [sum#29] -Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] - -(33) CometColumnarExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 7] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] - -(35) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] -Functions [1]: [sum(UnscaledValue(ss_net_paid#24))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#24))#16] -Results [4]: [c_customer_id#5 AS customer_id#31, c_first_name#6 AS customer_first_name#32, c_last_name#7 AS customer_last_name#33, MakeDecimal(sum(UnscaledValue(ss_net_paid#24))#16,17,2) AS year_total#34] - -(36) BroadcastExchange -Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(37) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#31] -Join type: Inner -Join condition: None - -(38) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(39) CometFilter -Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Condition : (isnotnull(c_customer_sk#35) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#36, 16, true, false, true))) - -(40) CometProject -Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Arguments: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#36, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#37, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#38, 30, true, false, true) AS c_last_name#41] - -(41) CometColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41] - -(42) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] - -(44) Filter [codegen id : 8] -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_bill_customer_sk#42) - -(45) BroadcastExchange -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -(46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#35] -Right keys [1]: [ws_bill_customer_sk#42] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 10] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] -Input [7]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] - -(48) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#45, d_year#46] - -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#45] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 10] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#46] -Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#45, d_year#46] - -(51) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#46] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] -Aggregate Attributes [1]: [sum#47] -Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] - -(52) CometColumnarExchange -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] -Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(53) CometColumnarToRow [codegen id : 11] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] - -(54) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46] -Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#49] -Results [2]: [c_customer_id#39 AS customer_id#50, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#49,17,2) AS year_total#51] - -(55) Filter [codegen id : 11] -Input [2]: [customer_id#50, year_total#51] -Condition : (isnotnull(year_total#51) AND (year_total#51 > 0.00)) - -(56) BroadcastExchange -Input [2]: [customer_id#50, year_total#51] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(57) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#50] -Join type: Inner -Join condition: None - -(58) Project [codegen id : 16] -Output [7]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#51] -Input [8]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, customer_id#50, year_total#51] - -(59) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(60) CometFilter -Input [4]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55] -Condition : (isnotnull(c_customer_sk#52) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#53, 16, true, false, true))) - -(61) CometProject -Input [4]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55] -Arguments: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#52, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#53, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#54, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#55, 30, true, false, true) AS c_last_name#41] - -(62) CometColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41] - -(63) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#26)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(64) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] - -(65) Filter [codegen id : 12] -Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] -Condition : isnotnull(ws_bill_customer_sk#56) - -(66) BroadcastExchange -Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#52] -Right keys [1]: [ws_bill_customer_sk#56] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 14] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, ws_sold_date_sk#58] -Input [7]: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] - -(69) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#59, d_year#60] - -(70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#58] -Right keys [1]: [d_date_sk#59] -Join type: Inner -Join condition: None - -(71) Project [codegen id : 14] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, d_year#60] -Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, ws_sold_date_sk#58, d_date_sk#59, d_year#60] - -(72) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, d_year#60] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#57))] -Aggregate Attributes [1]: [sum#61] -Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] - -(73) CometColumnarExchange -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] -Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(74) CometColumnarToRow [codegen id : 15] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] - -(75) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60] -Functions [1]: [sum(UnscaledValue(ws_net_paid#57))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#57))#49] -Results [2]: [c_customer_id#39 AS customer_id#63, MakeDecimal(sum(UnscaledValue(ws_net_paid#57))#49,17,2) AS year_total#64] - -(76) BroadcastExchange -Input [2]: [customer_id#63, year_total#64] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] - -(77) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#63] -Join type: Inner -Join condition: (CASE WHEN (year_total#51 > 0.00) THEN (year_total#64 / year_total#51) END > CASE WHEN (year_total#18 > 0.00) THEN (year_total#34 / year_total#18) END) - -(78) Project [codegen id : 16] -Output [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] -Input [9]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#51, customer_id#63, year_total#64] - -(79) TakeOrderedAndProject -Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] -Arguments: 100, [customer_id#31 ASC NULLS FIRST, customer_id#31 ASC NULLS FIRST, customer_id#31 ASC NULLS FIRST], [customer_id#31, customer_first_name#32, customer_last_name#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometFilter (81) - +- CometNativeScan parquet spark_catalog.default.date_dim (80) - - -(80) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(81) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(82) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(83) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 -BroadcastExchange (87) -+- * CometColumnarToRow (86) - +- CometFilter (85) - +- CometNativeScan parquet spark_catalog.default.date_dim (84) - - -(84) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#27, d_year#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(85) CometFilter -Input [2]: [d_date_sk#27, d_year#28] -Condition : (((isnotnull(d_year#28) AND (d_year#28 = 2002)) AND d_year#28 IN (2001,2002)) AND isnotnull(d_date_sk#27)) - -(86) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#28] - -(87) BroadcastExchange -Input [2]: [d_date_sk#27, d_year#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] - -Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#11 - -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#26 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/simplified.txt deleted file mode 100644 index f488e18c7c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/simplified.txt +++ /dev/null @@ -1,130 +0,0 @@ -TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - WholeStageCodegen (16) - Project [customer_id,customer_first_name,customer_last_name] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/explain.txt deleted file mode 100644 index 649b5dc4d5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,470 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometProject (53) - : +- CometBroadcastHashJoin (52) - : :- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (51) - : +- CometFilter (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : +- CometBroadcastExchange (41) - : : +- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) - : +- ReusedExchange (44) - +- CometBroadcastExchange (68) - +- CometHashAggregate (67) - +- CometExchange (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometFilter (55) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (62) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Condition : isnotnull(ss_customer_sk#8) - -(6) CometBroadcastExchange -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_sk#1], [ss_customer_sk#8], Inner, BuildRight - -(8) CometProject -Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(12) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#10], [d_date_sk#12], Inner, BuildRight - -(13) CometProject -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] - -(14) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] - -(15) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] - -(17) CometFilter -Input [2]: [customer_id#15, year_total#16] -Condition : (isnotnull(year_total#16) AND (year_total#16 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true))) - -(20) CometProject -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Arguments: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#19, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#20, 30, true, false, true) AS c_last_name#7] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_customer_sk#21) - -(23) CometBroadcastExchange -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] - -(24) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [c_customer_sk#17], [ss_customer_sk#21], Inner, BuildRight - -(25) CometProject -Input [7]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: [d_date_sk#25, d_year#26] - -(29) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] -Right output [2]: [d_date_sk#25, d_year#26] -Arguments: [ss_sold_date_sk#23], [d_date_sk#25], Inner, BuildRight - -(30) CometProject -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23, d_date_sk#25, d_year#26] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] - -(31) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] - -(32) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] -Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] - -(34) CometBroadcastExchange -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#15, year_total#16] -Right output [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#15], [customer_id#28], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true))) - -(38) CometProject -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Arguments: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#34, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#35, 30, true, false, true) AS c_last_name#38] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(ws_sold_date_sk#41 IN dynamicpruning#42)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Condition : isnotnull(ws_bill_customer_sk#39) - -(41) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] - -(42) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38] -Right output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [c_customer_sk#32], [ws_bill_customer_sk#39], Inner, BuildRight - -(43) CometProject -Input [7]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#43, d_year#44] - -(45) CometBroadcastHashJoin -Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] -Right output [2]: [d_date_sk#43, d_year#44] -Arguments: [ws_sold_date_sk#41], [d_date_sk#43], Inner, BuildRight - -(46) CometProject -Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41, d_date_sk#43, d_year#44] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] - -(47) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#40))] - -(48) CometExchange -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] -Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] - -(50) CometFilter -Input [2]: [customer_id#46, year_total#47] -Condition : (isnotnull(year_total#47) AND (year_total#47 > 0.00)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#46, year_total#47] -Arguments: [customer_id#46, year_total#47] - -(52) CometBroadcastHashJoin -Left output [6]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Right output [2]: [customer_id#46, year_total#47] -Arguments: [customer_id#15], [customer_id#46], Inner, BuildRight - -(53) CometProject -Input [8]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#46, year_total#47] -Arguments: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47], [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Condition : (isnotnull(c_customer_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true))) - -(56) CometProject -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Arguments: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#50, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#51, 30, true, false, true) AS c_last_name#38] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#55)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Condition : isnotnull(ws_bill_customer_sk#52) - -(59) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] - -(60) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38] -Right output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [c_customer_sk#48], [ws_bill_customer_sk#52], Inner, BuildRight - -(61) CometProject -Input [7]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#56, d_year#57] - -(63) CometBroadcastHashJoin -Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] -Right output [2]: [d_date_sk#56, d_year#57] -Arguments: [ws_sold_date_sk#54], [d_date_sk#56], Inner, BuildRight - -(64) CometProject -Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54, d_date_sk#56, d_year#57] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] - -(65) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#53))] - -(66) CometExchange -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] -Functions [1]: [sum(UnscaledValue(ws_net_paid#53))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#59, year_total#60] -Arguments: [customer_id#59, year_total#60] - -(69) CometBroadcastHashJoin -Left output [7]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] -Right output [2]: [customer_id#59, year_total#60] -Arguments: [customer_id#15], [customer_id#59], Inner, (CASE WHEN (year_total#47 > 0.00) THEN (year_total#60 / year_total#47) END > CASE WHEN (year_total#16 > 0.00) THEN (year_total#31 / year_total#16) END), BuildRight - -(70) CometProject -Input [9]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47, customer_id#59, year_total#60] -Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30], [customer_id#28, customer_first_name#29, customer_last_name#30] - -(71) CometTakeOrderedAndProject -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#28 ASC NULLS FIRST,customer_id#28 ASC NULLS FIRST,customer_id#28 ASC NULLS FIRST], output=[customer_id#28,customer_first_name#29,customer_last_name#30]), [customer_id#28, customer_first_name#29, customer_last_name#30], 100, 0, [customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] - -(72) CometColumnarToRow [codegen id : 1] -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(75) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(76) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) - -(79) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#25, d_year#26] - -(80) BroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#41 IN dynamicpruning#11 - -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#24 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/simplified.txt deleted file mode 100644 index 72e39422e0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - CometProject [customer_id,customer_first_name,customer_last_name] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 - CometHashAggregate [d_year,sum] [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/explain.txt deleted file mode 100644 index 649b5dc4d5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/explain.txt +++ /dev/null @@ -1,470 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometProject (53) - : +- CometBroadcastHashJoin (52) - : :- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (51) - : +- CometFilter (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : +- CometBroadcastExchange (41) - : : +- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) - : +- ReusedExchange (44) - +- CometBroadcastExchange (68) - +- CometHashAggregate (67) - +- CometExchange (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometFilter (55) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (62) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Condition : isnotnull(ss_customer_sk#8) - -(6) CometBroadcastExchange -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_sk#1], [ss_customer_sk#8], Inner, BuildRight - -(8) CometProject -Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(12) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#10], [d_date_sk#12], Inner, BuildRight - -(13) CometProject -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] - -(14) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] - -(15) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] - -(17) CometFilter -Input [2]: [customer_id#15, year_total#16] -Condition : (isnotnull(year_total#16) AND (year_total#16 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true))) - -(20) CometProject -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Arguments: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#19, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#20, 30, true, false, true) AS c_last_name#7] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_customer_sk#21) - -(23) CometBroadcastExchange -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] - -(24) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [c_customer_sk#17], [ss_customer_sk#21], Inner, BuildRight - -(25) CometProject -Input [7]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: [d_date_sk#25, d_year#26] - -(29) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] -Right output [2]: [d_date_sk#25, d_year#26] -Arguments: [ss_sold_date_sk#23], [d_date_sk#25], Inner, BuildRight - -(30) CometProject -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23, d_date_sk#25, d_year#26] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] - -(31) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] - -(32) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] -Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] - -(34) CometBroadcastExchange -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#15, year_total#16] -Right output [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#15], [customer_id#28], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true))) - -(38) CometProject -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Arguments: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#34, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#35, 30, true, false, true) AS c_last_name#38] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(ws_sold_date_sk#41 IN dynamicpruning#42)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Condition : isnotnull(ws_bill_customer_sk#39) - -(41) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] - -(42) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38] -Right output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [c_customer_sk#32], [ws_bill_customer_sk#39], Inner, BuildRight - -(43) CometProject -Input [7]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#43, d_year#44] - -(45) CometBroadcastHashJoin -Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] -Right output [2]: [d_date_sk#43, d_year#44] -Arguments: [ws_sold_date_sk#41], [d_date_sk#43], Inner, BuildRight - -(46) CometProject -Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41, d_date_sk#43, d_year#44] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] - -(47) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#40))] - -(48) CometExchange -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] -Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] - -(50) CometFilter -Input [2]: [customer_id#46, year_total#47] -Condition : (isnotnull(year_total#47) AND (year_total#47 > 0.00)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#46, year_total#47] -Arguments: [customer_id#46, year_total#47] - -(52) CometBroadcastHashJoin -Left output [6]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Right output [2]: [customer_id#46, year_total#47] -Arguments: [customer_id#15], [customer_id#46], Inner, BuildRight - -(53) CometProject -Input [8]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#46, year_total#47] -Arguments: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47], [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Condition : (isnotnull(c_customer_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true))) - -(56) CometProject -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Arguments: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#50, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#51, 30, true, false, true) AS c_last_name#38] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#55)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Condition : isnotnull(ws_bill_customer_sk#52) - -(59) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] - -(60) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38] -Right output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [c_customer_sk#48], [ws_bill_customer_sk#52], Inner, BuildRight - -(61) CometProject -Input [7]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#56, d_year#57] - -(63) CometBroadcastHashJoin -Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] -Right output [2]: [d_date_sk#56, d_year#57] -Arguments: [ws_sold_date_sk#54], [d_date_sk#56], Inner, BuildRight - -(64) CometProject -Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54, d_date_sk#56, d_year#57] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] - -(65) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#53))] - -(66) CometExchange -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] -Functions [1]: [sum(UnscaledValue(ws_net_paid#53))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#59, year_total#60] -Arguments: [customer_id#59, year_total#60] - -(69) CometBroadcastHashJoin -Left output [7]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] -Right output [2]: [customer_id#59, year_total#60] -Arguments: [customer_id#15], [customer_id#59], Inner, (CASE WHEN (year_total#47 > 0.00) THEN (year_total#60 / year_total#47) END > CASE WHEN (year_total#16 > 0.00) THEN (year_total#31 / year_total#16) END), BuildRight - -(70) CometProject -Input [9]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47, customer_id#59, year_total#60] -Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30], [customer_id#28, customer_first_name#29, customer_last_name#30] - -(71) CometTakeOrderedAndProject -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#28 ASC NULLS FIRST,customer_id#28 ASC NULLS FIRST,customer_id#28 ASC NULLS FIRST], output=[customer_id#28,customer_first_name#29,customer_last_name#30]), [customer_id#28, customer_first_name#29, customer_last_name#30], 100, 0, [customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] - -(72) CometColumnarToRow [codegen id : 1] -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(75) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(76) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) - -(79) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#25, d_year#26] - -(80) BroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#41 IN dynamicpruning#11 - -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#24 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/extended.txt deleted file mode 100644 index 476c7be954..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/extended.txt +++ /dev/null @@ -1,90 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 72e39422e0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - CometProject [customer_id,customer_first_name,customer_last_name] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 - CometHashAggregate [d_year,sum] [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/explain.txt deleted file mode 100644 index abecd7ac24..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/explain.txt +++ /dev/null @@ -1,754 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (127) -+- CometTakeOrderedAndProject (126) - +- CometProject (125) - +- CometSortMergeJoin (124) - :- CometSort (68) - : +- CometExchange (67) - : +- CometFilter (66) - : +- CometHashAggregate (65) - : +- CometExchange (64) - : +- CometHashAggregate (63) - : +- CometHashAggregate (62) - : +- CometExchange (61) - : +- CometHashAggregate (60) - : +- CometUnion (59) - : :- CometProject (22) - : : +- CometSortMergeJoin (21) - : : :- CometSort (15) - : : : +- CometColumnarExchange (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : : +- ReusedExchange (11) - : : +- CometSort (20) - : : +- CometExchange (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (40) - : : +- CometSortMergeJoin (39) - : : :- CometSort (33) - : : : +- CometColumnarExchange (32) - : : : +- * Project (31) - : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : :- * Project (28) - : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : :- * Filter (25) - : : : : : +- * ColumnarToRow (24) - : : : : : +- Scan parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (26) - : : : +- ReusedExchange (29) - : : +- CometSort (38) - : : +- CometExchange (37) - : : +- CometProject (36) - : : +- CometFilter (35) - : : +- CometNativeScan parquet spark_catalog.default.store_returns (34) - : +- CometProject (58) - : +- CometSortMergeJoin (57) - : :- CometSort (51) - : : +- CometColumnarExchange (50) - : : +- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * Project (46) - : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : :- * Filter (43) - : : : : +- * ColumnarToRow (42) - : : : : +- Scan parquet spark_catalog.default.web_sales (41) - : : : +- ReusedExchange (44) - : : +- ReusedExchange (47) - : +- CometSort (56) - : +- CometExchange (55) - : +- CometProject (54) - : +- CometFilter (53) - : +- CometNativeScan parquet spark_catalog.default.web_returns (52) - +- CometSort (123) - +- CometExchange (122) - +- CometFilter (121) - +- CometHashAggregate (120) - +- CometExchange (119) - +- CometHashAggregate (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometUnion (114) - :- CometProject (83) - : +- CometSortMergeJoin (82) - : :- CometSort (79) - : : +- CometColumnarExchange (78) - : : +- * Project (77) - : : +- * BroadcastHashJoin Inner BuildRight (76) - : : :- * Project (74) - : : : +- * BroadcastHashJoin Inner BuildRight (73) - : : : :- * Filter (71) - : : : : +- * ColumnarToRow (70) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (69) - : : : +- ReusedExchange (72) - : : +- ReusedExchange (75) - : +- CometSort (81) - : +- ReusedExchange (80) - :- CometProject (98) - : +- CometSortMergeJoin (97) - : :- CometSort (94) - : : +- CometColumnarExchange (93) - : : +- * Project (92) - : : +- * BroadcastHashJoin Inner BuildRight (91) - : : :- * Project (89) - : : : +- * BroadcastHashJoin Inner BuildRight (88) - : : : :- * Filter (86) - : : : : +- * ColumnarToRow (85) - : : : : +- Scan parquet spark_catalog.default.store_sales (84) - : : : +- ReusedExchange (87) - : : +- ReusedExchange (90) - : +- CometSort (96) - : +- ReusedExchange (95) - +- CometProject (113) - +- CometSortMergeJoin (112) - :- CometSort (109) - : +- CometColumnarExchange (108) - : +- * Project (107) - : +- * BroadcastHashJoin Inner BuildRight (106) - : :- * Project (104) - : : +- * BroadcastHashJoin Inner BuildRight (103) - : : :- * Filter (101) - : : : +- * ColumnarToRow (100) - : : : +- Scan parquet spark_catalog.default.web_sales (99) - : : +- ReusedExchange (102) - : +- ReusedExchange (105) - +- CometSort (111) - +- ReusedExchange (110) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] - -(3) Filter [codegen id : 3] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) = Books ) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) - -(6) CometProject -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(7) CometColumnarToRow [codegen id : 1] -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(8) BroadcastExchange -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(11) ReusedExchange [Reuses operator id: 131] -Output [2]: [d_date_sk#13, d_year#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] - -(14) CometColumnarExchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(15) CometSort -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] - -(16) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) - -(18) CometProject -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(19) CometExchange -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter - -(22) CometProject -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21], [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] - -(23) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 6] -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] - -(25) Filter [codegen id : 6] -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#22) - -(26) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#22] -Right keys [1]: [i_item_sk#27] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] -Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] - -(29) ReusedExchange [Reuses operator id: 131] -Output [2]: [d_date_sk#32, d_year#33] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#32] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_date_sk#32, d_year#33] - -(32) CometColumnarExchange -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(33) CometSort -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33], [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] - -(34) CometNativeScan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37, sr_returned_date_sk#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [5]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37, sr_returned_date_sk#38] -Condition : (isnotnull(sr_ticket_number#35) AND isnotnull(sr_item_sk#34)) - -(36) CometProject -Input [5]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37, sr_returned_date_sk#38] -Arguments: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37], [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] - -(37) CometExchange -Input [4]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: hashpartitioning(sr_ticket_number#35, sr_item_sk#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(38) CometSort -Input [4]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37], [sr_ticket_number#35 ASC NULLS FIRST, sr_item_sk#34 ASC NULLS FIRST] - -(39) CometSortMergeJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Right output [4]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#35, sr_item_sk#34], LeftOuter - -(40) CometProject -Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33, sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: [d_year#33, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, sales_cnt#39, sales_amt#40], [d_year#33, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, (ss_quantity#24 - coalesce(sr_return_quantity#36, 0)) AS sales_cnt#39, (ss_ext_sales_price#25 - coalesce(sr_return_amt#37, 0.00)) AS sales_amt#40] - -(41) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(42) ColumnarToRow [codegen id : 9] -Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] - -(43) Filter [codegen id : 9] -Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -Condition : isnotnull(ws_item_sk#41) - -(44) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] - -(45) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#41] -Right keys [1]: [i_item_sk#46] -Join type: Inner -Join condition: None - -(46) Project [codegen id : 9] -Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] -Input [10]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] - -(47) ReusedExchange [Reuses operator id: 131] -Output [2]: [d_date_sk#51, d_year#52] - -(48) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#45] -Right keys [1]: [d_date_sk#51] -Join type: Inner -Join condition: None - -(49) Project [codegen id : 9] -Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Input [11]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_date_sk#51, d_year#52] - -(50) CometColumnarExchange -Input [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Arguments: hashpartitioning(ws_order_number#42, ws_item_sk#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(51) CometSort -Input [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Arguments: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52], [ws_order_number#42 ASC NULLS FIRST, ws_item_sk#41 ASC NULLS FIRST] - -(52) CometNativeScan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56, wr_returned_date_sk#57] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(53) CometFilter -Input [5]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56, wr_returned_date_sk#57] -Condition : (isnotnull(wr_order_number#54) AND isnotnull(wr_item_sk#53)) - -(54) CometProject -Input [5]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56, wr_returned_date_sk#57] -Arguments: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56], [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] - -(55) CometExchange -Input [4]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: hashpartitioning(wr_order_number#54, wr_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(56) CometSort -Input [4]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56], [wr_order_number#54 ASC NULLS FIRST, wr_item_sk#53 ASC NULLS FIRST] - -(57) CometSortMergeJoin -Left output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Right output [4]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: [ws_order_number#42, ws_item_sk#41], [wr_order_number#54, wr_item_sk#53], LeftOuter - -(58) CometProject -Input [13]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52, wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: [d_year#52, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, sales_cnt#58, sales_amt#59], [d_year#52, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, (ws_quantity#43 - coalesce(wr_return_quantity#55, 0)) AS sales_cnt#58, (ws_ext_sales_price#44 - coalesce(wr_return_amt#56, 0.00)) AS sales_amt#59] - -(59) CometUnion -Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#33, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, sales_cnt#39, sales_amt#40] -Child 2 Input [7]: [d_year#52, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, sales_cnt#58, sales_amt#59] - -(60) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(61) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(62) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(63) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(64) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#60, sum#61] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(65) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#60, sum#61] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(66) CometFilter -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Condition : isnotnull(sales_cnt#62) - -(67) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(68) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] - -(69) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#68), dynamicpruningexpression(cs_sold_date_sk#68 IN dynamicpruning#69)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(70) ColumnarToRow [codegen id : 12] -Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] - -(71) Filter [codegen id : 12] -Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] -Condition : isnotnull(cs_item_sk#64) - -(72) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] - -(73) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cs_item_sk#64] -Right keys [1]: [i_item_sk#70] -Join type: Inner -Join condition: None - -(74) Project [codegen id : 12] -Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -Input [10]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] - -(75) ReusedExchange [Reuses operator id: 135] -Output [2]: [d_date_sk#75, d_year#76] - -(76) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cs_sold_date_sk#68] -Right keys [1]: [d_date_sk#75] -Join type: Inner -Join condition: None - -(77) Project [codegen id : 12] -Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Input [11]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_date_sk#75, d_year#76] - -(78) CometColumnarExchange -Input [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Arguments: hashpartitioning(cs_order_number#65, cs_item_sk#64, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(79) CometSort -Input [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Arguments: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76], [cs_order_number#65 ASC NULLS FIRST, cs_item_sk#64 ASC NULLS FIRST] - -(80) ReusedExchange [Reuses operator id: 19] -Output [4]: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] - -(81) CometSort -Input [4]: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] -Arguments: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80], [cr_order_number#78 ASC NULLS FIRST, cr_item_sk#77 ASC NULLS FIRST] - -(82) CometSortMergeJoin -Left output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Right output [4]: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] -Arguments: [cs_order_number#65, cs_item_sk#64], [cr_order_number#78, cr_item_sk#77], LeftOuter - -(83) CometProject -Input [13]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76, cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] -Arguments: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21], [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, (cs_quantity#66 - coalesce(cr_return_quantity#79, 0)) AS sales_cnt#20, (cs_ext_sales_price#67 - coalesce(cr_return_amount#80, 0.00)) AS sales_amt#21] - -(84) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#85), dynamicpruningexpression(ss_sold_date_sk#85 IN dynamicpruning#69)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(85) ColumnarToRow [codegen id : 15] -Input [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85] - -(86) Filter [codegen id : 15] -Input [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85] -Condition : isnotnull(ss_item_sk#81) - -(87) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] - -(88) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ss_item_sk#81] -Right keys [1]: [i_item_sk#86] -Join type: Inner -Join condition: None - -(89) Project [codegen id : 15] -Output [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] -Input [10]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] - -(90) ReusedExchange [Reuses operator id: 135] -Output [2]: [d_date_sk#91, d_year#92] - -(91) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ss_sold_date_sk#85] -Right keys [1]: [d_date_sk#91] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 15] -Output [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92] -Input [11]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_date_sk#91, d_year#92] - -(93) CometColumnarExchange -Input [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92] -Arguments: hashpartitioning(ss_ticket_number#82, ss_item_sk#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(94) CometSort -Input [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92] -Arguments: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92], [ss_ticket_number#82 ASC NULLS FIRST, ss_item_sk#81 ASC NULLS FIRST] - -(95) ReusedExchange [Reuses operator id: 37] -Output [4]: [sr_item_sk#93, sr_ticket_number#94, sr_return_quantity#95, sr_return_amt#96] - -(96) CometSort -Input [4]: [sr_item_sk#93, sr_ticket_number#94, sr_return_quantity#95, sr_return_amt#96] -Arguments: [sr_item_sk#93, sr_ticket_number#94, sr_return_quantity#95, sr_return_amt#96], [sr_ticket_number#94 ASC NULLS FIRST, sr_item_sk#93 ASC NULLS FIRST] - -(97) CometSortMergeJoin -Left output [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92] -Right output [4]: [sr_item_sk#93, sr_ticket_number#94, sr_return_quantity#95, sr_return_amt#96] -Arguments: [ss_ticket_number#82, ss_item_sk#81], [sr_ticket_number#94, sr_item_sk#93], LeftOuter - -(98) CometProject -Input [13]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92, sr_item_sk#93, sr_ticket_number#94, sr_return_quantity#95, sr_return_amt#96] -Arguments: [d_year#92, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, sales_cnt#39, sales_amt#40], [d_year#92, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, (ss_quantity#83 - coalesce(sr_return_quantity#95, 0)) AS sales_cnt#39, (ss_ext_sales_price#84 - coalesce(sr_return_amt#96, 0.00)) AS sales_amt#40] - -(99) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#101), dynamicpruningexpression(ws_sold_date_sk#101 IN dynamicpruning#69)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(100) ColumnarToRow [codegen id : 18] -Input [5]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101] - -(101) Filter [codegen id : 18] -Input [5]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101] -Condition : isnotnull(ws_item_sk#97) - -(102) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#102, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] - -(103) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_item_sk#97] -Right keys [1]: [i_item_sk#102] -Join type: Inner -Join condition: None - -(104) Project [codegen id : 18] -Output [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] -Input [10]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_item_sk#102, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] - -(105) ReusedExchange [Reuses operator id: 135] -Output [2]: [d_date_sk#107, d_year#108] - -(106) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#101] -Right keys [1]: [d_date_sk#107] -Join type: Inner -Join condition: None - -(107) Project [codegen id : 18] -Output [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108] -Input [11]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_date_sk#107, d_year#108] - -(108) CometColumnarExchange -Input [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108] -Arguments: hashpartitioning(ws_order_number#98, ws_item_sk#97, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(109) CometSort -Input [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108] -Arguments: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108], [ws_order_number#98 ASC NULLS FIRST, ws_item_sk#97 ASC NULLS FIRST] - -(110) ReusedExchange [Reuses operator id: 55] -Output [4]: [wr_item_sk#109, wr_order_number#110, wr_return_quantity#111, wr_return_amt#112] - -(111) CometSort -Input [4]: [wr_item_sk#109, wr_order_number#110, wr_return_quantity#111, wr_return_amt#112] -Arguments: [wr_item_sk#109, wr_order_number#110, wr_return_quantity#111, wr_return_amt#112], [wr_order_number#110 ASC NULLS FIRST, wr_item_sk#109 ASC NULLS FIRST] - -(112) CometSortMergeJoin -Left output [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108] -Right output [4]: [wr_item_sk#109, wr_order_number#110, wr_return_quantity#111, wr_return_amt#112] -Arguments: [ws_order_number#98, ws_item_sk#97], [wr_order_number#110, wr_item_sk#109], LeftOuter - -(113) CometProject -Input [13]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108, wr_item_sk#109, wr_order_number#110, wr_return_quantity#111, wr_return_amt#112] -Arguments: [d_year#108, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, sales_cnt#58, sales_amt#59], [d_year#108, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, (ws_quantity#99 - coalesce(wr_return_quantity#111, 0)) AS sales_cnt#58, (ws_ext_sales_price#100 - coalesce(wr_return_amt#112, 0.00)) AS sales_amt#59] - -(114) CometUnion -Child 0 Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#92, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, sales_cnt#39, sales_amt#40] -Child 2 Input [7]: [d_year#108, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, sales_cnt#58, sales_amt#59] - -(115) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Functions: [] - -(116) CometExchange -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(117) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Functions: [] - -(118) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(119) CometExchange -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sum#60, sum#113] -Arguments: hashpartitioning(d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(120) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sum#60, sum#113] -Keys [5]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(121) CometFilter -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115] -Condition : isnotnull(sales_cnt#114) - -(122) CometExchange -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115] -Arguments: hashpartitioning(i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=16] - -(123) CometSort -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115] -Arguments: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115], [i_brand_id#71 ASC NULLS FIRST, i_class_id#72 ASC NULLS FIRST, i_category_id#73 ASC NULLS FIRST, i_manufact_id#74 ASC NULLS FIRST] - -(124) CometSortMergeJoin -Left output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Right output [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115] -Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74], Inner, ((cast(sales_cnt#62 as decimal(17,2)) / cast(sales_cnt#114 as decimal(17,2))) < 0.90000000000000000000) - -(125) CometProject -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115] -Arguments: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121], [d_year#76 AS prev_year#116, d_year#14 AS year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#114 AS prev_yr_cnt#118, sales_cnt#62 AS curr_yr_cnt#119, (sales_cnt#62 - sales_cnt#114) AS sales_cnt_diff#120, (sales_amt#63 - sales_amt#115) AS sales_amt_diff#121] - -(126) CometTakeOrderedAndProject -Input [10]: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#120 ASC NULLS FIRST], output=[prev_year#116,year#117,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#118,curr_yr_cnt#119,sales_cnt_diff#120,sales_amt_diff#121]), [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121], 100, 0, [sales_cnt_diff#120 ASC NULLS FIRST], [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] - -(127) CometColumnarToRow [codegen id : 19] -Input [10]: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (131) -+- * CometColumnarToRow (130) - +- CometFilter (129) - +- CometNativeScan parquet spark_catalog.default.date_dim (128) - - -(128) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(129) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(130) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_year#14] - -(131) BroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#6 - -Subquery:4 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#68 IN dynamicpruning#69 -BroadcastExchange (135) -+- * CometColumnarToRow (134) - +- CometFilter (133) - +- CometNativeScan parquet spark_catalog.default.date_dim (132) - - -(132) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#75, d_year#76] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(133) CometFilter -Input [2]: [d_date_sk#75, d_year#76] -Condition : ((isnotnull(d_year#76) AND (d_year#76 = 2001)) AND isnotnull(d_date_sk#75)) - -(134) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#75, d_year#76] - -(135) BroadcastExchange -Input [2]: [d_date_sk#75, d_year#76] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] - -Subquery:5 Hosting operator id = 84 Hosting Expression = ss_sold_date_sk#85 IN dynamicpruning#69 - -Subquery:6 Hosting operator id = 99 Hosting Expression = ws_sold_date_sk#101 IN dynamicpruning#69 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/simplified.txt deleted file mode 100644 index aaf98f4b09..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/simplified.txt +++ /dev/null @@ -1,173 +0,0 @@ -WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometColumnarExchange [cs_order_number,cs_item_sk] #4 - WholeStageCodegen (3) - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [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] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #7 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 - WholeStageCodegen (6) - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #9 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometColumnarExchange [ws_order_number,ws_item_sk] #10 - WholeStageCodegen (9) - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #11 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometColumnarExchange [cs_order_number,cs_item_sk] #15 - WholeStageCodegen (12) - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 #16 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 - WholeStageCodegen (15) - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometColumnarExchange [ws_order_number,ws_item_sk] #18 - WholeStageCodegen (18) - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/explain.txt deleted file mode 100644 index 7afb1e3be6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,742 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (124) -+- CometTakeOrderedAndProject (123) - +- CometProject (122) - +- CometSortMergeJoin (121) - :- CometSort (66) - : +- CometExchange (65) - : +- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometHashAggregate (60) - : +- CometExchange (59) - : +- CometHashAggregate (58) - : +- CometUnion (57) - : :- CometProject (22) - : : +- CometSortMergeJoin (21) - : : :- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometSort (20) - : : +- CometExchange (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (39) - : : +- CometSortMergeJoin (38) - : : :- CometSort (32) - : : : +- CometExchange (31) - : : : +- CometProject (30) - : : : +- CometBroadcastHashJoin (29) - : : : :- CometProject (27) - : : : : +- CometBroadcastHashJoin (26) - : : : : :- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (25) - : : : +- ReusedExchange (28) - : : +- CometSort (37) - : : +- CometExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (33) - : +- CometProject (56) - : +- CometSortMergeJoin (55) - : :- CometSort (49) - : : +- CometExchange (48) - : : +- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometProject (44) - : : : +- CometBroadcastHashJoin (43) - : : : :- CometFilter (41) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) - : : : +- ReusedExchange (42) - : : +- ReusedExchange (45) - : +- CometSort (54) - : +- CometExchange (53) - : +- CometProject (52) - : +- CometFilter (51) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (50) - +- CometSort (120) - +- CometExchange (119) - +- CometFilter (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometHashAggregate (114) - +- CometExchange (113) - +- CometHashAggregate (112) - +- CometUnion (111) - :- CometProject (82) - : +- CometSortMergeJoin (81) - : :- CometSort (78) - : : +- CometExchange (77) - : : +- CometProject (76) - : : +- CometBroadcastHashJoin (75) - : : :- CometProject (71) - : : : +- CometBroadcastHashJoin (70) - : : : :- CometFilter (68) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) - : : : +- ReusedExchange (69) - : : +- CometBroadcastExchange (74) - : : +- CometFilter (73) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - : +- CometSort (80) - : +- ReusedExchange (79) - :- CometProject (96) - : +- CometSortMergeJoin (95) - : :- CometSort (92) - : : +- CometExchange (91) - : : +- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (87) - : : : +- CometBroadcastHashJoin (86) - : : : :- CometFilter (84) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (83) - : : : +- ReusedExchange (85) - : : +- ReusedExchange (88) - : +- CometSort (94) - : +- ReusedExchange (93) - +- CometProject (110) - +- CometSortMergeJoin (109) - :- CometSort (106) - : +- CometExchange (105) - : +- CometProject (104) - : +- CometBroadcastHashJoin (103) - : :- CometProject (101) - : : +- CometBroadcastHashJoin (100) - : : :- CometFilter (98) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (97) - : : +- ReusedExchange (99) - : +- ReusedExchange (102) - +- CometSort (108) - +- ReusedExchange (107) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) = Books ) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) - -(5) CometProject -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(6) CometBroadcastExchange -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(7) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Right output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: [d_date_sk#13, d_year#14] - -(12) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Right output [2]: [d_date_sk#13, d_year#14] -Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight - -(13) CometProject -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] - -(14) CometExchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(15) CometSort -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) - -(18) CometProject -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(19) CometExchange -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter - -(22) CometProject -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21], [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(24) CometFilter -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#22) - -(25) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(26) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Right output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22], [i_item_sk#28], Inner, BuildRight - -(27) CometProject -Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(28) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#33, d_year#34] - -(29) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Right output [2]: [d_date_sk#33, d_year#34] -Arguments: [ss_sold_date_sk#26], [d_date_sk#33], Inner, BuildRight - -(30) CometProject -Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_date_sk#33, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] - -(31) CometExchange -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometSort -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) - -(35) CometProject -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] - -(36) CometExchange -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(37) CometSort -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] - -(38) CometSortMergeJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter - -(39) CometProject -Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#47)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(41) CometFilter -Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Condition : isnotnull(ws_item_sk#42) - -(42) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(43) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Right output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42], [i_item_sk#48], Inner, BuildRight - -(44) CometProject -Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(45) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#53, d_year#54] - -(46) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [ws_sold_date_sk#46], [d_date_sk#53], Inner, BuildRight - -(47) CometProject -Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] - -(48) CometExchange -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(49) CometSort -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) - -(52) CometProject -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] - -(53) CometExchange -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(54) CometSort -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] - -(55) CometSortMergeJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter - -(56) CometProject -Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] - -(57) CometUnion -Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] - -(58) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(59) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(60) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(61) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(62) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(63) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(64) CometFilter -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Condition : isnotnull(sales_cnt#64) - -(65) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(66) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(68) CometFilter -Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Condition : isnotnull(cs_item_sk#66) - -(69) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(70) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Right output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66], [i_item_sk#72], Inner, BuildRight - -(71) CometProject -Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(74) CometBroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: [d_date_sk#77, d_year#78] - -(75) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Right output [2]: [d_date_sk#77, d_year#78] -Arguments: [cs_sold_date_sk#70], [d_date_sk#77], Inner, BuildRight - -(76) CometProject -Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] - -(77) CometExchange -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(78) CometSort -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] - -(79) ReusedExchange [Reuses operator id: 19] -Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] - -(80) CometSort -Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] - -(81) CometSortMergeJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter - -(82) CometProject -Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#20, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#21] - -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(84) CometFilter -Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Condition : isnotnull(ss_item_sk#83) - -(85) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] - -(86) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Right output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Arguments: [ss_item_sk#83], [i_item_sk#89], Inner, BuildRight - -(87) CometProject -Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] - -(88) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#94, d_year#95] - -(89) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Right output [2]: [d_date_sk#94, d_year#95] -Arguments: [ss_sold_date_sk#87], [d_date_sk#94], Inner, BuildRight - -(90) CometProject -Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_date_sk#94, d_year#95] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] - -(91) CometExchange -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(92) CometSort -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_ticket_number#84 ASC NULLS FIRST, ss_item_sk#83 ASC NULLS FIRST] - -(93) ReusedExchange [Reuses operator id: 36] -Output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] - -(94) CometSort -Input [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99], [sr_ticket_number#97 ASC NULLS FIRST, sr_item_sk#96 ASC NULLS FIRST] - -(95) CometSortMergeJoin -Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Right output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [ss_ticket_number#84, ss_item_sk#83], [sr_ticket_number#97, sr_item_sk#96], LeftOuter - -(96) CometProject -Input [13]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95, sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41], [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, (ss_quantity#85 - coalesce(sr_return_quantity#98, 0)) AS sales_cnt#40, (ss_ext_sales_price#86 - coalesce(sr_return_amt#99, 0.00)) AS sales_amt#41] - -(97) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_sold_date_sk#104 IN dynamicpruning#105)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(98) CometFilter -Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Condition : isnotnull(ws_item_sk#100) - -(99) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] - -(100) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Right output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Arguments: [ws_item_sk#100], [i_item_sk#106], Inner, BuildRight - -(101) CometProject -Input [10]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] - -(102) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#111, d_year#112] - -(103) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Right output [2]: [d_date_sk#111, d_year#112] -Arguments: [ws_sold_date_sk#104], [d_date_sk#111], Inner, BuildRight - -(104) CometProject -Input [11]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] - -(105) CometExchange -Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(106) CometSort -Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_order_number#101 ASC NULLS FIRST, ws_item_sk#100 ASC NULLS FIRST] - -(107) ReusedExchange [Reuses operator id: 53] -Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] - -(108) CometSort -Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116], [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST] - -(109) CometSortMergeJoin -Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Right output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [ws_order_number#101, ws_item_sk#100], [wr_order_number#114, wr_item_sk#113], LeftOuter - -(110) CometProject -Input [13]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61], [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#102 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#60, (ws_ext_sales_price#103 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#61] - -(111) CometUnion -Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61] - -(112) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Functions: [] - -(113) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] - -(114) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Functions: [] - -(115) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(116) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(117) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(118) CometFilter -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Condition : isnotnull(sales_cnt#118) - -(119) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(120) CometSort -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] - -(121) CometSortMergeJoin -Left output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Right output [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], Inner, ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#118 as decimal(17,2))) < 0.90000000000000000000) - -(122) CometProject -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], [d_year#78 AS prev_year#120, d_year#14 AS year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#118 AS prev_yr_cnt#122, sales_cnt#64 AS curr_yr_cnt#123, (sales_cnt#64 - sales_cnt#118) AS sales_cnt_diff#124, (sales_amt#65 - sales_amt#119) AS sales_amt_diff#125] - -(123) CometTakeOrderedAndProject -Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#124 ASC NULLS FIRST], output=[prev_year#120,year#121,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#122,curr_yr_cnt#123,sales_cnt_diff#124,sales_amt_diff#125]), [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], 100, 0, [sales_cnt_diff#124 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] - -(124) CometColumnarToRow [codegen id : 1] -Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (128) -+- * CometColumnarToRow (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) - - -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(127) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_year#14] - -(128) BroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 - -Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 -BroadcastExchange (132) -+- * CometColumnarToRow (131) - +- CometFilter (130) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) - - -(129) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(130) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(131) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#77, d_year#78] - -(132) BroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] - -Subquery:5 Hosting operator id = 83 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 - -Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/simplified.txt deleted file mode 100644 index d0520c8b63..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,144 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #4 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange [d_date_sk,d_year] #7 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #8 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #9 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #10 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #11 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #12 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #16 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_year] #18 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #19 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #20 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/explain.txt deleted file mode 100644 index 7afb1e3be6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/explain.txt +++ /dev/null @@ -1,742 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (124) -+- CometTakeOrderedAndProject (123) - +- CometProject (122) - +- CometSortMergeJoin (121) - :- CometSort (66) - : +- CometExchange (65) - : +- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometHashAggregate (60) - : +- CometExchange (59) - : +- CometHashAggregate (58) - : +- CometUnion (57) - : :- CometProject (22) - : : +- CometSortMergeJoin (21) - : : :- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometSort (20) - : : +- CometExchange (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (39) - : : +- CometSortMergeJoin (38) - : : :- CometSort (32) - : : : +- CometExchange (31) - : : : +- CometProject (30) - : : : +- CometBroadcastHashJoin (29) - : : : :- CometProject (27) - : : : : +- CometBroadcastHashJoin (26) - : : : : :- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (25) - : : : +- ReusedExchange (28) - : : +- CometSort (37) - : : +- CometExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (33) - : +- CometProject (56) - : +- CometSortMergeJoin (55) - : :- CometSort (49) - : : +- CometExchange (48) - : : +- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometProject (44) - : : : +- CometBroadcastHashJoin (43) - : : : :- CometFilter (41) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) - : : : +- ReusedExchange (42) - : : +- ReusedExchange (45) - : +- CometSort (54) - : +- CometExchange (53) - : +- CometProject (52) - : +- CometFilter (51) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (50) - +- CometSort (120) - +- CometExchange (119) - +- CometFilter (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometHashAggregate (114) - +- CometExchange (113) - +- CometHashAggregate (112) - +- CometUnion (111) - :- CometProject (82) - : +- CometSortMergeJoin (81) - : :- CometSort (78) - : : +- CometExchange (77) - : : +- CometProject (76) - : : +- CometBroadcastHashJoin (75) - : : :- CometProject (71) - : : : +- CometBroadcastHashJoin (70) - : : : :- CometFilter (68) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) - : : : +- ReusedExchange (69) - : : +- CometBroadcastExchange (74) - : : +- CometFilter (73) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - : +- CometSort (80) - : +- ReusedExchange (79) - :- CometProject (96) - : +- CometSortMergeJoin (95) - : :- CometSort (92) - : : +- CometExchange (91) - : : +- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (87) - : : : +- CometBroadcastHashJoin (86) - : : : :- CometFilter (84) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (83) - : : : +- ReusedExchange (85) - : : +- ReusedExchange (88) - : +- CometSort (94) - : +- ReusedExchange (93) - +- CometProject (110) - +- CometSortMergeJoin (109) - :- CometSort (106) - : +- CometExchange (105) - : +- CometProject (104) - : +- CometBroadcastHashJoin (103) - : :- CometProject (101) - : : +- CometBroadcastHashJoin (100) - : : :- CometFilter (98) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (97) - : : +- ReusedExchange (99) - : +- ReusedExchange (102) - +- CometSort (108) - +- ReusedExchange (107) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) = Books ) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) - -(5) CometProject -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(6) CometBroadcastExchange -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(7) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Right output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: [d_date_sk#13, d_year#14] - -(12) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Right output [2]: [d_date_sk#13, d_year#14] -Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight - -(13) CometProject -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] - -(14) CometExchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(15) CometSort -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) - -(18) CometProject -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(19) CometExchange -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter - -(22) CometProject -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21], [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(24) CometFilter -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#22) - -(25) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(26) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Right output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22], [i_item_sk#28], Inner, BuildRight - -(27) CometProject -Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(28) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#33, d_year#34] - -(29) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Right output [2]: [d_date_sk#33, d_year#34] -Arguments: [ss_sold_date_sk#26], [d_date_sk#33], Inner, BuildRight - -(30) CometProject -Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_date_sk#33, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] - -(31) CometExchange -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometSort -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) - -(35) CometProject -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] - -(36) CometExchange -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(37) CometSort -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] - -(38) CometSortMergeJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter - -(39) CometProject -Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#47)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(41) CometFilter -Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Condition : isnotnull(ws_item_sk#42) - -(42) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(43) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Right output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42], [i_item_sk#48], Inner, BuildRight - -(44) CometProject -Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(45) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#53, d_year#54] - -(46) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [ws_sold_date_sk#46], [d_date_sk#53], Inner, BuildRight - -(47) CometProject -Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] - -(48) CometExchange -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(49) CometSort -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) - -(52) CometProject -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] - -(53) CometExchange -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(54) CometSort -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] - -(55) CometSortMergeJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter - -(56) CometProject -Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] - -(57) CometUnion -Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] - -(58) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(59) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(60) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(61) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(62) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(63) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(64) CometFilter -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Condition : isnotnull(sales_cnt#64) - -(65) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(66) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(68) CometFilter -Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Condition : isnotnull(cs_item_sk#66) - -(69) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(70) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Right output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66], [i_item_sk#72], Inner, BuildRight - -(71) CometProject -Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(74) CometBroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: [d_date_sk#77, d_year#78] - -(75) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Right output [2]: [d_date_sk#77, d_year#78] -Arguments: [cs_sold_date_sk#70], [d_date_sk#77], Inner, BuildRight - -(76) CometProject -Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] - -(77) CometExchange -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(78) CometSort -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] - -(79) ReusedExchange [Reuses operator id: 19] -Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] - -(80) CometSort -Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] - -(81) CometSortMergeJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter - -(82) CometProject -Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#20, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#21] - -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(84) CometFilter -Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Condition : isnotnull(ss_item_sk#83) - -(85) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] - -(86) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Right output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Arguments: [ss_item_sk#83], [i_item_sk#89], Inner, BuildRight - -(87) CometProject -Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] - -(88) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#94, d_year#95] - -(89) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Right output [2]: [d_date_sk#94, d_year#95] -Arguments: [ss_sold_date_sk#87], [d_date_sk#94], Inner, BuildRight - -(90) CometProject -Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_date_sk#94, d_year#95] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] - -(91) CometExchange -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(92) CometSort -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_ticket_number#84 ASC NULLS FIRST, ss_item_sk#83 ASC NULLS FIRST] - -(93) ReusedExchange [Reuses operator id: 36] -Output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] - -(94) CometSort -Input [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99], [sr_ticket_number#97 ASC NULLS FIRST, sr_item_sk#96 ASC NULLS FIRST] - -(95) CometSortMergeJoin -Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Right output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [ss_ticket_number#84, ss_item_sk#83], [sr_ticket_number#97, sr_item_sk#96], LeftOuter - -(96) CometProject -Input [13]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95, sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41], [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, (ss_quantity#85 - coalesce(sr_return_quantity#98, 0)) AS sales_cnt#40, (ss_ext_sales_price#86 - coalesce(sr_return_amt#99, 0.00)) AS sales_amt#41] - -(97) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_sold_date_sk#104 IN dynamicpruning#105)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(98) CometFilter -Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Condition : isnotnull(ws_item_sk#100) - -(99) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] - -(100) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Right output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Arguments: [ws_item_sk#100], [i_item_sk#106], Inner, BuildRight - -(101) CometProject -Input [10]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] - -(102) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#111, d_year#112] - -(103) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Right output [2]: [d_date_sk#111, d_year#112] -Arguments: [ws_sold_date_sk#104], [d_date_sk#111], Inner, BuildRight - -(104) CometProject -Input [11]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] - -(105) CometExchange -Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(106) CometSort -Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_order_number#101 ASC NULLS FIRST, ws_item_sk#100 ASC NULLS FIRST] - -(107) ReusedExchange [Reuses operator id: 53] -Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] - -(108) CometSort -Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116], [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST] - -(109) CometSortMergeJoin -Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Right output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [ws_order_number#101, ws_item_sk#100], [wr_order_number#114, wr_item_sk#113], LeftOuter - -(110) CometProject -Input [13]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61], [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#102 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#60, (ws_ext_sales_price#103 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#61] - -(111) CometUnion -Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61] - -(112) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Functions: [] - -(113) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] - -(114) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Functions: [] - -(115) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(116) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(117) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(118) CometFilter -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Condition : isnotnull(sales_cnt#118) - -(119) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(120) CometSort -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] - -(121) CometSortMergeJoin -Left output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Right output [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], Inner, ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#118 as decimal(17,2))) < 0.90000000000000000000) - -(122) CometProject -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], [d_year#78 AS prev_year#120, d_year#14 AS year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#118 AS prev_yr_cnt#122, sales_cnt#64 AS curr_yr_cnt#123, (sales_cnt#64 - sales_cnt#118) AS sales_cnt_diff#124, (sales_amt#65 - sales_amt#119) AS sales_amt_diff#125] - -(123) CometTakeOrderedAndProject -Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#124 ASC NULLS FIRST], output=[prev_year#120,year#121,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#122,curr_yr_cnt#123,sales_cnt_diff#124,sales_amt_diff#125]), [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], 100, 0, [sales_cnt_diff#124 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] - -(124) CometColumnarToRow [codegen id : 1] -Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (128) -+- * CometColumnarToRow (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) - - -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(127) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_year#14] - -(128) BroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 - -Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 -BroadcastExchange (132) -+- * CometColumnarToRow (131) - +- CometFilter (130) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) - - -(129) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(130) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(131) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#77, d_year#78] - -(132) BroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] - -Subquery:5 Hosting operator id = 83 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 - -Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/extended.txt deleted file mode 100644 index 22ccc17641..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/extended.txt +++ /dev/null @@ -1,172 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : +- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - +- CometSort - +- CometExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - -Comet accelerated 159 out of 167 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d0520c8b63..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75/simplified.txt +++ /dev/null @@ -1,144 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #4 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange [d_date_sk,d_year] #7 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #8 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #9 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #10 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #11 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #12 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #16 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_year] #18 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #19 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #20 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_datafusion/explain.txt deleted file mode 100644 index f04f5a8a5a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_datafusion/explain.txt +++ /dev/null @@ -1,200 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (35) -+- CometTakeOrderedAndProject (34) - +- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometUnion (30) - :- CometProject (13) - : +- CometBroadcastHashJoin (12) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometNativeScan parquet spark_catalog.default.item (3) - : +- CometBroadcastExchange (11) - : +- CometFilter (10) - : +- CometNativeScan parquet spark_catalog.default.date_dim (9) - :- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometFilter (15) - : : : +- CometNativeScan parquet spark_catalog.default.web_sales (14) - : : +- ReusedExchange (16) - : +- ReusedExchange (19) - +- CometProject (29) - +- CometBroadcastHashJoin (28) - :- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometFilter (23) - : : +- CometNativeScan parquet spark_catalog.default.catalog_sales (22) - : +- ReusedExchange (24) - +- ReusedExchange (27) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#4)] -PushedFilters: [IsNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_category#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_category#6] -Condition : isnotnull(i_item_sk#5) - -(5) CometProject -Input [2]: [i_item_sk#5, i_category#6] -Arguments: [i_item_sk#5, i_category#7], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#6, 50, true, false, true) AS i_category#7] - -(6) CometBroadcastExchange -Input [2]: [i_item_sk#5, i_category#7] -Arguments: [i_item_sk#5, i_category#7] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [2]: [i_item_sk#5, i_category#7] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_item_sk#5, i_category#7] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7], [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7] - -(9) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Condition : isnotnull(d_date_sk#8) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [d_date_sk#8, d_year#9, d_qoy#10] - -(12) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7] -Right output [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [ss_sold_date_sk#4], [d_date_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7, d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13], [store AS channel#11, ss_store_sk#2 AS col_name#12, d_year#9, d_qoy#10, i_category#7, ss_ext_sales_price#3 AS ext_sales_price#13] - -(14) CometNativeScan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ws_sold_date_sk#17)] -PushedFilters: [IsNull(ws_ship_customer_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Condition : (isnull(ws_ship_customer_sk#15) AND isnotnull(ws_item_sk#14)) - -(16) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#18, i_category#19] - -(17) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Right output [2]: [i_item_sk#18, i_category#19] -Arguments: [ws_item_sk#14], [i_item_sk#18], Inner, BuildRight - -(18) CometProject -Input [6]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_item_sk#18, i_category#19] -Arguments: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19], [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19] - -(19) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#20, d_year#21, d_qoy#22] - -(20) CometBroadcastHashJoin -Left output [4]: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19] -Right output [3]: [d_date_sk#20, d_year#21, d_qoy#22] -Arguments: [ws_sold_date_sk#17], [d_date_sk#20], Inner, BuildRight - -(21) CometProject -Input [7]: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19, d_date_sk#20, d_year#21, d_qoy#22] -Arguments: [channel#23, col_name#24, d_year#21, d_qoy#22, i_category#19, ext_sales_price#25], [web AS channel#23, ws_ship_customer_sk#15 AS col_name#24, d_year#21, d_qoy#22, i_category#19, ws_ext_sales_price#16 AS ext_sales_price#25] - -(22) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(cs_sold_date_sk#29)] -PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(23) CometFilter -Input [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Condition : (isnull(cs_ship_addr_sk#26) AND isnotnull(cs_item_sk#27)) - -(24) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#30, i_category#31] - -(25) CometBroadcastHashJoin -Left output [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Right output [2]: [i_item_sk#30, i_category#31] -Arguments: [cs_item_sk#27], [i_item_sk#30], Inner, BuildRight - -(26) CometProject -Input [6]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29, i_item_sk#30, i_category#31] -Arguments: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31], [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31] - -(27) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#32, d_year#33, d_qoy#34] - -(28) CometBroadcastHashJoin -Left output [4]: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31] -Right output [3]: [d_date_sk#32, d_year#33, d_qoy#34] -Arguments: [cs_sold_date_sk#29], [d_date_sk#32], Inner, BuildRight - -(29) CometProject -Input [7]: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31, d_date_sk#32, d_year#33, d_qoy#34] -Arguments: [channel#35, col_name#36, d_year#33, d_qoy#34, i_category#31, ext_sales_price#37], [catalog AS channel#35, cs_ship_addr_sk#26 AS col_name#36, d_year#33, d_qoy#34, i_category#31, cs_ext_sales_price#28 AS ext_sales_price#37] - -(30) CometUnion -Child 0 Input [6]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13] -Child 1 Input [6]: [channel#23, col_name#24, d_year#21, d_qoy#22, i_category#19, ext_sales_price#25] -Child 2 Input [6]: [channel#35, col_name#36, d_year#33, d_qoy#34, i_category#31, ext_sales_price#37] - -(31) CometHashAggregate -Input [6]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13] -Keys [5]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7] -Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#13))] - -(32) CometExchange -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] -Arguments: hashpartitioning(channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(33) CometHashAggregate -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] -Keys [5]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7] -Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#13))] - -(34) CometTakeOrderedAndProject -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#11 ASC NULLS FIRST,col_name#12 ASC NULLS FIRST,d_year#9 ASC NULLS FIRST,d_qoy#10 ASC NULLS FIRST,i_category#7 ASC NULLS FIRST], output=[channel#11,col_name#12,d_year#9,d_qoy#10,i_category#7,sales_cnt#40,sales_amt#41]), [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41], 100, 0, [channel#11 ASC NULLS FIRST, col_name#12 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#7 ASC NULLS FIRST], [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] - -(35) CometColumnarToRow [codegen id : 1] -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_datafusion/simplified.txt deleted file mode 100644 index abb482c3d5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_datafusion/simplified.txt +++ /dev/null @@ -1,37 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] - CometHashAggregate [count,sum] [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt,count(1),sum(UnscaledValue(ext_sales_price))] - CometExchange [channel,col_name,d_year,d_qoy,i_category] #1 - CometHashAggregate [ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,count,sum] - 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_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,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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category] #2 - CometProject [i_category] [i_item_sk,i_category] - CometFilter [i_item_sk,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan 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_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,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] - CometNativeScan 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_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_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] - CometNativeScan 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/q76.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_iceberg_compat/explain.txt deleted file mode 100644 index d862a6cb29..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,200 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (35) -+- CometTakeOrderedAndProject (34) - +- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometUnion (30) - :- CometProject (13) - : +- CometBroadcastHashJoin (12) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : +- CometBroadcastExchange (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - :- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometFilter (15) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (14) - : : +- ReusedExchange (16) - : +- ReusedExchange (19) - +- CometProject (29) - +- CometBroadcastHashJoin (28) - :- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometFilter (23) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (22) - : +- ReusedExchange (24) - +- ReusedExchange (27) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4)] -PushedFilters: [IsNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_category#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_category#6] -Condition : isnotnull(i_item_sk#5) - -(5) CometProject -Input [2]: [i_item_sk#5, i_category#6] -Arguments: [i_item_sk#5, i_category#7], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#6, 50, true, false, true) AS i_category#7] - -(6) CometBroadcastExchange -Input [2]: [i_item_sk#5, i_category#7] -Arguments: [i_item_sk#5, i_category#7] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [2]: [i_item_sk#5, i_category#7] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_item_sk#5, i_category#7] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7], [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Condition : isnotnull(d_date_sk#8) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [d_date_sk#8, d_year#9, d_qoy#10] - -(12) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7] -Right output [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [ss_sold_date_sk#4], [d_date_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7, d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13], [store AS channel#11, ss_store_sk#2 AS col_name#12, d_year#9, d_qoy#10, i_category#7, ss_ext_sales_price#3 AS ext_sales_price#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#17)] -PushedFilters: [IsNull(ws_ship_customer_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Condition : (isnull(ws_ship_customer_sk#15) AND isnotnull(ws_item_sk#14)) - -(16) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#18, i_category#19] - -(17) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Right output [2]: [i_item_sk#18, i_category#19] -Arguments: [ws_item_sk#14], [i_item_sk#18], Inner, BuildRight - -(18) CometProject -Input [6]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_item_sk#18, i_category#19] -Arguments: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19], [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19] - -(19) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#20, d_year#21, d_qoy#22] - -(20) CometBroadcastHashJoin -Left output [4]: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19] -Right output [3]: [d_date_sk#20, d_year#21, d_qoy#22] -Arguments: [ws_sold_date_sk#17], [d_date_sk#20], Inner, BuildRight - -(21) CometProject -Input [7]: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19, d_date_sk#20, d_year#21, d_qoy#22] -Arguments: [channel#23, col_name#24, d_year#21, d_qoy#22, i_category#19, ext_sales_price#25], [web AS channel#23, ws_ship_customer_sk#15 AS col_name#24, d_year#21, d_qoy#22, i_category#19, ws_ext_sales_price#16 AS ext_sales_price#25] - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#29)] -PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(23) CometFilter -Input [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Condition : (isnull(cs_ship_addr_sk#26) AND isnotnull(cs_item_sk#27)) - -(24) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#30, i_category#31] - -(25) CometBroadcastHashJoin -Left output [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Right output [2]: [i_item_sk#30, i_category#31] -Arguments: [cs_item_sk#27], [i_item_sk#30], Inner, BuildRight - -(26) CometProject -Input [6]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29, i_item_sk#30, i_category#31] -Arguments: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31], [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31] - -(27) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#32, d_year#33, d_qoy#34] - -(28) CometBroadcastHashJoin -Left output [4]: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31] -Right output [3]: [d_date_sk#32, d_year#33, d_qoy#34] -Arguments: [cs_sold_date_sk#29], [d_date_sk#32], Inner, BuildRight - -(29) CometProject -Input [7]: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31, d_date_sk#32, d_year#33, d_qoy#34] -Arguments: [channel#35, col_name#36, d_year#33, d_qoy#34, i_category#31, ext_sales_price#37], [catalog AS channel#35, cs_ship_addr_sk#26 AS col_name#36, d_year#33, d_qoy#34, i_category#31, cs_ext_sales_price#28 AS ext_sales_price#37] - -(30) CometUnion -Child 0 Input [6]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13] -Child 1 Input [6]: [channel#23, col_name#24, d_year#21, d_qoy#22, i_category#19, ext_sales_price#25] -Child 2 Input [6]: [channel#35, col_name#36, d_year#33, d_qoy#34, i_category#31, ext_sales_price#37] - -(31) CometHashAggregate -Input [6]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13] -Keys [5]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7] -Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#13))] - -(32) CometExchange -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] -Arguments: hashpartitioning(channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(33) CometHashAggregate -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] -Keys [5]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7] -Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#13))] - -(34) CometTakeOrderedAndProject -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#11 ASC NULLS FIRST,col_name#12 ASC NULLS FIRST,d_year#9 ASC NULLS FIRST,d_qoy#10 ASC NULLS FIRST,i_category#7 ASC NULLS FIRST], output=[channel#11,col_name#12,d_year#9,d_qoy#10,i_category#7,sales_cnt#40,sales_amt#41]), [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41], 100, 0, [channel#11 ASC NULLS FIRST, col_name#12 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#7 ASC NULLS FIRST], [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] - -(35) CometColumnarToRow [codegen id : 1] -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_iceberg_compat/simplified.txt deleted file mode 100644 index b5449831cb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,37 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] - CometHashAggregate [count,sum] [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt,count(1),sum(UnscaledValue(ext_sales_price))] - CometExchange [channel,col_name,d_year,d_qoy,i_category] #1 - CometHashAggregate [ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,count,sum] - 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_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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category] #2 - CometProject [i_category] [i_item_sk,i_category] - CometFilter [i_item_sk,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] 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_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,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 [native_iceberg_compat] 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_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_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 [native_iceberg_compat] 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/q76/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/explain.txt deleted file mode 100644 index d862a6cb29..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/explain.txt +++ /dev/null @@ -1,200 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (35) -+- CometTakeOrderedAndProject (34) - +- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometUnion (30) - :- CometProject (13) - : +- CometBroadcastHashJoin (12) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : +- CometBroadcastExchange (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - :- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometFilter (15) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (14) - : : +- ReusedExchange (16) - : +- ReusedExchange (19) - +- CometProject (29) - +- CometBroadcastHashJoin (28) - :- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometFilter (23) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (22) - : +- ReusedExchange (24) - +- ReusedExchange (27) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4)] -PushedFilters: [IsNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_category#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_category#6] -Condition : isnotnull(i_item_sk#5) - -(5) CometProject -Input [2]: [i_item_sk#5, i_category#6] -Arguments: [i_item_sk#5, i_category#7], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#6, 50, true, false, true) AS i_category#7] - -(6) CometBroadcastExchange -Input [2]: [i_item_sk#5, i_category#7] -Arguments: [i_item_sk#5, i_category#7] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [2]: [i_item_sk#5, i_category#7] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_item_sk#5, i_category#7] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7], [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Condition : isnotnull(d_date_sk#8) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [d_date_sk#8, d_year#9, d_qoy#10] - -(12) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7] -Right output [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [ss_sold_date_sk#4], [d_date_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7, d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13], [store AS channel#11, ss_store_sk#2 AS col_name#12, d_year#9, d_qoy#10, i_category#7, ss_ext_sales_price#3 AS ext_sales_price#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#17)] -PushedFilters: [IsNull(ws_ship_customer_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Condition : (isnull(ws_ship_customer_sk#15) AND isnotnull(ws_item_sk#14)) - -(16) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#18, i_category#19] - -(17) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Right output [2]: [i_item_sk#18, i_category#19] -Arguments: [ws_item_sk#14], [i_item_sk#18], Inner, BuildRight - -(18) CometProject -Input [6]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_item_sk#18, i_category#19] -Arguments: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19], [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19] - -(19) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#20, d_year#21, d_qoy#22] - -(20) CometBroadcastHashJoin -Left output [4]: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19] -Right output [3]: [d_date_sk#20, d_year#21, d_qoy#22] -Arguments: [ws_sold_date_sk#17], [d_date_sk#20], Inner, BuildRight - -(21) CometProject -Input [7]: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19, d_date_sk#20, d_year#21, d_qoy#22] -Arguments: [channel#23, col_name#24, d_year#21, d_qoy#22, i_category#19, ext_sales_price#25], [web AS channel#23, ws_ship_customer_sk#15 AS col_name#24, d_year#21, d_qoy#22, i_category#19, ws_ext_sales_price#16 AS ext_sales_price#25] - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#29)] -PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(23) CometFilter -Input [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Condition : (isnull(cs_ship_addr_sk#26) AND isnotnull(cs_item_sk#27)) - -(24) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#30, i_category#31] - -(25) CometBroadcastHashJoin -Left output [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Right output [2]: [i_item_sk#30, i_category#31] -Arguments: [cs_item_sk#27], [i_item_sk#30], Inner, BuildRight - -(26) CometProject -Input [6]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29, i_item_sk#30, i_category#31] -Arguments: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31], [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31] - -(27) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#32, d_year#33, d_qoy#34] - -(28) CometBroadcastHashJoin -Left output [4]: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31] -Right output [3]: [d_date_sk#32, d_year#33, d_qoy#34] -Arguments: [cs_sold_date_sk#29], [d_date_sk#32], Inner, BuildRight - -(29) CometProject -Input [7]: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31, d_date_sk#32, d_year#33, d_qoy#34] -Arguments: [channel#35, col_name#36, d_year#33, d_qoy#34, i_category#31, ext_sales_price#37], [catalog AS channel#35, cs_ship_addr_sk#26 AS col_name#36, d_year#33, d_qoy#34, i_category#31, cs_ext_sales_price#28 AS ext_sales_price#37] - -(30) CometUnion -Child 0 Input [6]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13] -Child 1 Input [6]: [channel#23, col_name#24, d_year#21, d_qoy#22, i_category#19, ext_sales_price#25] -Child 2 Input [6]: [channel#35, col_name#36, d_year#33, d_qoy#34, i_category#31, ext_sales_price#37] - -(31) CometHashAggregate -Input [6]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13] -Keys [5]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7] -Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#13))] - -(32) CometExchange -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] -Arguments: hashpartitioning(channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(33) CometHashAggregate -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] -Keys [5]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7] -Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#13))] - -(34) CometTakeOrderedAndProject -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#11 ASC NULLS FIRST,col_name#12 ASC NULLS FIRST,d_year#9 ASC NULLS FIRST,d_qoy#10 ASC NULLS FIRST,i_category#7 ASC NULLS FIRST], output=[channel#11,col_name#12,d_year#9,d_qoy#10,i_category#7,sales_cnt#40,sales_amt#41]), [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41], 100, 0, [channel#11 ASC NULLS FIRST, col_name#12 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#7 ASC NULLS FIRST], [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] - -(35) CometColumnarToRow [codegen id : 1] -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/extended.txt deleted file mode 100644 index b56a6590db..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/extended.txt +++ /dev/null @@ -1,47 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 44 out of 44 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index b5449831cb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q76/simplified.txt +++ /dev/null @@ -1,37 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] - CometHashAggregate [count,sum] [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt,count(1),sum(UnscaledValue(ext_sales_price))] - CometExchange [channel,col_name,d_year,d_qoy,i_category] #1 - CometHashAggregate [ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,count,sum] - 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_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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category] #2 - CometProject [i_category] [i_item_sk,i_category] - CometFilter [i_item_sk,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] 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_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,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 [native_iceberg_compat] 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_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_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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/explain.txt deleted file mode 100644 index ae530b4900..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/explain.txt +++ /dev/null @@ -1,575 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (92) -+- * HashAggregate (91) - +- * CometColumnarToRow (90) - +- CometColumnarExchange (89) - +- * HashAggregate (88) - +- * Expand (87) - +- Union (86) - :- * Project (32) - : +- * BroadcastHashJoin LeftOuter BuildRight (31) - : :- * HashAggregate (16) - : : +- * CometColumnarToRow (15) - : : +- CometColumnarExchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * CometColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (30) - : +- * HashAggregate (29) - : +- * CometColumnarToRow (28) - : +- CometColumnarExchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet spark_catalog.default.store_returns (17) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - :- * Project (53) - : +- * BroadcastNestedLoopJoin Inner BuildLeft (52) - : :- BroadcastExchange (42) - : : +- * HashAggregate (41) - : : +- * CometColumnarToRow (40) - : : +- CometColumnarExchange (39) - : : +- * HashAggregate (38) - : : +- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * ColumnarToRow (34) - : : : +- Scan parquet spark_catalog.default.catalog_sales (33) - : : +- ReusedExchange (35) - : +- * HashAggregate (51) - : +- * CometColumnarToRow (50) - : +- CometColumnarExchange (49) - : +- * HashAggregate (48) - : +- * Project (47) - : +- * BroadcastHashJoin Inner BuildRight (46) - : :- * ColumnarToRow (44) - : : +- Scan parquet spark_catalog.default.catalog_returns (43) - : +- ReusedExchange (45) - +- * Project (85) - +- * BroadcastHashJoin LeftOuter BuildRight (84) - :- * HashAggregate (69) - : +- * CometColumnarToRow (68) - : +- CometColumnarExchange (67) - : +- * HashAggregate (66) - : +- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * Filter (56) - : : : +- * ColumnarToRow (55) - : : : +- Scan parquet spark_catalog.default.web_sales (54) - : : +- ReusedExchange (57) - : +- BroadcastExchange (63) - : +- * CometColumnarToRow (62) - : +- CometFilter (61) - : +- CometNativeScan parquet spark_catalog.default.web_page (60) - +- BroadcastExchange (83) - +- * HashAggregate (82) - +- * CometColumnarToRow (81) - +- CometColumnarExchange (80) - +- * HashAggregate (79) - +- * Project (78) - +- * BroadcastHashJoin Inner BuildRight (77) - :- * Project (75) - : +- * BroadcastHashJoin Inner BuildRight (74) - : :- * Filter (72) - : : +- * ColumnarToRow (71) - : : +- Scan parquet spark_catalog.default.web_returns (70) - : +- ReusedExchange (73) - +- ReusedExchange (76) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 3] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(4) ReusedExchange [Reuses operator id: 97] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [1]: [s_store_sk#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [1]: [s_store_sk#7] -Condition : isnotnull(s_store_sk#7) - -(9) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#7] - -(10) BroadcastExchange -Input [1]: [s_store_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None - -(12) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] - -(13) HashAggregate [codegen id : 3] -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Keys [1]: [s_store_sk#7] -Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum#8, sum#9] -Results [3]: [s_store_sk#7, sum#10, sum#11] - -(14) CometColumnarExchange -Input [3]: [s_store_sk#7, sum#10, sum#11] -Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(15) CometColumnarToRow [codegen id : 8] -Input [3]: [s_store_sk#7, sum#10, sum#11] - -(16) HashAggregate [codegen id : 8] -Input [3]: [s_store_sk#7, sum#10, sum#11] -Keys [1]: [s_store_sk#7] -Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#12, sum(UnscaledValue(ss_net_profit#3))#13] -Results [3]: [s_store_sk#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS sales#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#13,17,2) AS profit#15] - -(17) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 6] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] - -(19) Filter [codegen id : 6] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_store_sk#16) - -(20) ReusedExchange [Reuses operator id: 97] -Output [1]: [d_date_sk#20] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_returned_date_sk#19] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] -Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#20] - -(23) ReusedExchange [Reuses operator id: 10] -Output [1]: [s_store_sk#21] - -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_store_sk#16] -Right keys [1]: [s_store_sk#21] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#21] - -(26) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] -Keys [1]: [s_store_sk#21] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum#22, sum#23] -Results [3]: [s_store_sk#21, sum#24, sum#25] - -(27) CometColumnarExchange -Input [3]: [s_store_sk#21, sum#24, sum#25] -Arguments: hashpartitioning(s_store_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#21, sum#24, sum#25] - -(29) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#21, sum#24, sum#25] -Keys [1]: [s_store_sk#21] -Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#17))#26, sum(UnscaledValue(sr_net_loss#18))#27] -Results [3]: [s_store_sk#21, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#26,17,2) AS returns#28, MakeDecimal(sum(UnscaledValue(sr_net_loss#18))#27,17,2) AS profit_loss#29] - -(30) BroadcastExchange -Input [3]: [s_store_sk#21, returns#28, profit_loss#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [s_store_sk#7] -Right keys [1]: [s_store_sk#21] -Join type: LeftOuter -Join condition: None - -(32) Project [codegen id : 8] -Output [5]: [sales#14, coalesce(returns#28, 0.00) AS returns#30, (profit#15 - coalesce(profit_loss#29, 0.00)) AS profit#31, store channel AS channel#32, s_store_sk#7 AS id#33] -Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#21, returns#28, profit_loss#29] - -(33) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#5)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37] - -(35) ReusedExchange [Reuses operator id: 97] -Output [1]: [d_date_sk#38] - -(36) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#37] -Right keys [1]: [d_date_sk#38] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] -Input [5]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37, d_date_sk#38] - -(38) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] -Keys [1]: [cs_call_center_sk#34] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#35)), partial_sum(UnscaledValue(cs_net_profit#36))] -Aggregate Attributes [2]: [sum#39, sum#40] -Results [3]: [cs_call_center_sk#34, sum#41, sum#42] - -(39) CometColumnarExchange -Input [3]: [cs_call_center_sk#34, sum#41, sum#42] -Arguments: hashpartitioning(cs_call_center_sk#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(40) CometColumnarToRow [codegen id : 11] -Input [3]: [cs_call_center_sk#34, sum#41, sum#42] - -(41) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#34, sum#41, sum#42] -Keys [1]: [cs_call_center_sk#34] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#35)), sum(UnscaledValue(cs_net_profit#36))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#35))#43, sum(UnscaledValue(cs_net_profit#36))#44] -Results [3]: [cs_call_center_sk#34, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#35))#43,17,2) AS sales#45, MakeDecimal(sum(UnscaledValue(cs_net_profit#36))#44,17,2) AS profit#46] - -(42) BroadcastExchange -Input [3]: [cs_call_center_sk#34, sales#45, profit#46] -Arguments: IdentityBroadcastMode, [plan_id=6] - -(43) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#49), dynamicpruningexpression(cr_returned_date_sk#49 IN dynamicpruning#5)] -ReadSchema: struct - -(44) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49] - -(45) ReusedExchange [Reuses operator id: 97] -Output [1]: [d_date_sk#50] - -(46) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#49] -Right keys [1]: [d_date_sk#50] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 13] -Output [2]: [cr_return_amount#47, cr_net_loss#48] -Input [4]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49, d_date_sk#50] - -(48) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#47, cr_net_loss#48] -Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#47)), partial_sum(UnscaledValue(cr_net_loss#48))] -Aggregate Attributes [2]: [sum#51, sum#52] -Results [2]: [sum#53, sum#54] - -(49) CometColumnarExchange -Input [2]: [sum#53, sum#54] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(50) CometColumnarToRow -Input [2]: [sum#53, sum#54] - -(51) HashAggregate -Input [2]: [sum#53, sum#54] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#47)), sum(UnscaledValue(cr_net_loss#48))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#47))#55, sum(UnscaledValue(cr_net_loss#48))#56] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#47))#55,17,2) AS returns#57, MakeDecimal(sum(UnscaledValue(cr_net_loss#48))#56,17,2) AS profit_loss#58] - -(52) BroadcastNestedLoopJoin [codegen id : 14] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 14] -Output [5]: [sales#45, returns#57, (profit#46 - profit_loss#58) AS profit#59, catalog channel AS channel#60, cs_call_center_sk#34 AS id#61] -Input [5]: [cs_call_center_sk#34, sales#45, profit#46, returns#57, profit_loss#58] - -(54) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] - -(56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] -Condition : isnotnull(ws_web_page_sk#62) - -(57) ReusedExchange [Reuses operator id: 97] -Output [1]: [d_date_sk#66] - -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#65] -Right keys [1]: [d_date_sk#66] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64] -Input [5]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65, d_date_sk#66] - -(60) CometNativeScan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(61) CometFilter -Input [1]: [wp_web_page_sk#67] -Condition : isnotnull(wp_web_page_sk#67) - -(62) CometColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#67] - -(63) BroadcastExchange -Input [1]: [wp_web_page_sk#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#62] -Right keys [1]: [wp_web_page_sk#67] -Join type: Inner -Join condition: None - -(65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] -Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] - -(66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] -Keys [1]: [wp_web_page_sk#67] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#63)), partial_sum(UnscaledValue(ws_net_profit#64))] -Aggregate Attributes [2]: [sum#68, sum#69] -Results [3]: [wp_web_page_sk#67, sum#70, sum#71] - -(67) CometColumnarExchange -Input [3]: [wp_web_page_sk#67, sum#70, sum#71] -Arguments: hashpartitioning(wp_web_page_sk#67, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(68) CometColumnarToRow [codegen id : 22] -Input [3]: [wp_web_page_sk#67, sum#70, sum#71] - -(69) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#67, sum#70, sum#71] -Keys [1]: [wp_web_page_sk#67] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#63)), sum(UnscaledValue(ws_net_profit#64))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#63))#72, sum(UnscaledValue(ws_net_profit#64))#73] -Results [3]: [wp_web_page_sk#67, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#63))#72,17,2) AS sales#74, MakeDecimal(sum(UnscaledValue(ws_net_profit#64))#73,17,2) AS profit#75] - -(70) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#79), dynamicpruningexpression(wr_returned_date_sk#79 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_web_page_sk)] -ReadSchema: struct - -(71) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] - -(72) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] -Condition : isnotnull(wr_web_page_sk#76) - -(73) ReusedExchange [Reuses operator id: 97] -Output [1]: [d_date_sk#80] - -(74) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#79] -Right keys [1]: [d_date_sk#80] -Join type: Inner -Join condition: None - -(75) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78] -Input [5]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79, d_date_sk#80] - -(76) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#81] - -(77) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#76] -Right keys [1]: [wp_web_page_sk#81] -Join type: Inner -Join condition: None - -(78) Project [codegen id : 20] -Output [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] -Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] - -(79) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] -Keys [1]: [wp_web_page_sk#81] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#77)), partial_sum(UnscaledValue(wr_net_loss#78))] -Aggregate Attributes [2]: [sum#82, sum#83] -Results [3]: [wp_web_page_sk#81, sum#84, sum#85] - -(80) CometColumnarExchange -Input [3]: [wp_web_page_sk#81, sum#84, sum#85] -Arguments: hashpartitioning(wp_web_page_sk#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(81) CometColumnarToRow [codegen id : 21] -Input [3]: [wp_web_page_sk#81, sum#84, sum#85] - -(82) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#81, sum#84, sum#85] -Keys [1]: [wp_web_page_sk#81] -Functions [2]: [sum(UnscaledValue(wr_return_amt#77)), sum(UnscaledValue(wr_net_loss#78))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#77))#86, sum(UnscaledValue(wr_net_loss#78))#87] -Results [3]: [wp_web_page_sk#81, MakeDecimal(sum(UnscaledValue(wr_return_amt#77))#86,17,2) AS returns#88, MakeDecimal(sum(UnscaledValue(wr_net_loss#78))#87,17,2) AS profit_loss#89] - -(83) BroadcastExchange -Input [3]: [wp_web_page_sk#81, returns#88, profit_loss#89] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -(84) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#67] -Right keys [1]: [wp_web_page_sk#81] -Join type: LeftOuter -Join condition: None - -(85) Project [codegen id : 22] -Output [5]: [sales#74, coalesce(returns#88, 0.00) AS returns#90, (profit#75 - coalesce(profit_loss#89, 0.00)) AS profit#91, web channel AS channel#92, wp_web_page_sk#67 AS id#93] -Input [6]: [wp_web_page_sk#67, sales#74, profit#75, wp_web_page_sk#81, returns#88, profit_loss#89] - -(86) Union - -(87) Expand [codegen id : 23] -Input [5]: [sales#14, returns#30, profit#31, channel#32, id#33] -Arguments: [[sales#14, returns#30, profit#31, channel#32, id#33, 0], [sales#14, returns#30, profit#31, channel#32, null, 1], [sales#14, returns#30, profit#31, null, null, 3]], [sales#14, returns#30, profit#31, channel#94, id#95, spark_grouping_id#96] - -(88) HashAggregate [codegen id : 23] -Input [6]: [sales#14, returns#30, profit#31, channel#94, id#95, spark_grouping_id#96] -Keys [3]: [channel#94, id#95, spark_grouping_id#96] -Functions [3]: [partial_sum(sales#14), partial_sum(returns#30), partial_sum(profit#31)] -Aggregate Attributes [6]: [sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Results [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] - -(89) CometColumnarExchange -Input [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Arguments: hashpartitioning(channel#94, id#95, spark_grouping_id#96, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(90) CometColumnarToRow [codegen id : 24] -Input [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] - -(91) HashAggregate [codegen id : 24] -Input [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Keys [3]: [channel#94, id#95, spark_grouping_id#96] -Functions [3]: [sum(sales#14), sum(returns#30), sum(profit#31)] -Aggregate Attributes [3]: [sum(sales#14)#109, sum(returns#30)#110, sum(profit#31)#111] -Results [5]: [channel#94, id#95, sum(sales#14)#109 AS sales#112, sum(returns#30)#110 AS returns#113, sum(profit#31)#111 AS profit#114] - -(92) TakeOrderedAndProject -Input [5]: [channel#94, id#95, sales#112, returns#113, profit#114] -Arguments: 100, [channel#94 ASC NULLS FIRST, id#95 ASC NULLS FIRST], [channel#94, id#95, sales#112, returns#113, profit#114] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (97) -+- * CometColumnarToRow (96) - +- CometProject (95) - +- CometFilter (94) - +- CometNativeScan parquet spark_catalog.default.date_dim (93) - - -(93) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#115] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(94) CometFilter -Input [2]: [d_date_sk#6, d_date#115] -Condition : (((isnotnull(d_date#115) AND (d_date#115 >= 2000-08-03)) AND (d_date#115 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) - -(95) CometProject -Input [2]: [d_date_sk#6, d_date#115] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(96) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(97) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#49 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 70 Hosting Expression = wr_returned_date_sk#79 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/simplified.txt deleted file mode 100644 index 4256e90759..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/simplified.txt +++ /dev/null @@ -1,150 +0,0 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (24) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (23) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (8) - Project [sales,returns,profit,profit_loss,s_store_sk] - BroadcastHashJoin [s_store_sk,s_store_sk] - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #2 - WholeStageCodegen (3) - HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,s_store_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #6 - WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - Project [sr_return_amt,sr_net_loss,s_store_sk] - BroadcastHashJoin [sr_store_sk,s_store_sk] - Project [sr_store_sk,sr_return_amt,sr_net_loss] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [s_store_sk] #4 - WholeStageCodegen (14) - Project [sales,returns,profit,profit_loss,cs_call_center_sk] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) - HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_call_center_sk] #8 - WholeStageCodegen (10) - HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (13) - HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - Project [cr_return_amount,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (22) - Project [sales,returns,profit,profit_loss,wp_web_page_sk] - BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wp_web_page_sk] #10 - WholeStageCodegen (17) - HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_web_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (21) - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wp_web_page_sk] #13 - WholeStageCodegen (20) - HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - Project [wr_return_amt,wr_net_loss,wp_web_page_sk] - BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] - Project [wr_web_page_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Filter [wr_web_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [wp_web_page_sk] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/explain.txt deleted file mode 100644 index bb7ed0a50c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,518 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (85) -+- * HashAggregate (84) - +- * CometColumnarToRow (83) - +- CometColumnarExchange (82) - +- * HashAggregate (81) - +- * Expand (80) - +- Union (79) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometHashAggregate (16) - : : +- CometExchange (15) - : : +- CometHashAggregate (14) - : : +- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (28) - : +- CometHashAggregate (27) - : +- CometExchange (26) - : +- CometHashAggregate (25) - : +- CometProject (24) - : +- CometBroadcastHashJoin (23) - : :- CometProject (21) - : : +- CometBroadcastHashJoin (20) - : : :- CometFilter (18) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (17) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - :- * Project (50) - : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) - : :- BroadcastExchange (40) - : : +- * CometColumnarToRow (39) - : : +- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (32) - : : +- ReusedExchange (33) - : +- * CometColumnarToRow (48) - : +- CometHashAggregate (47) - : +- CometExchange (46) - : +- CometHashAggregate (45) - : +- CometProject (44) - : +- CometBroadcastHashJoin (43) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (41) - : +- ReusedExchange (42) - +- * CometColumnarToRow (78) - +- CometProject (77) - +- CometBroadcastHashJoin (76) - :- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (55) - : : +- CometBroadcastHashJoin (54) - : : :- CometFilter (52) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (51) - : : +- ReusedExchange (53) - : +- CometBroadcastExchange (58) - : +- CometFilter (57) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (56) - +- CometBroadcastExchange (75) - +- CometHashAggregate (74) - +- CometExchange (73) - +- CometHashAggregate (72) - +- CometProject (71) - +- CometBroadcastHashJoin (70) - :- CometProject (68) - : +- CometBroadcastHashJoin (67) - : :- CometFilter (65) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (64) - : +- ReusedExchange (66) - +- ReusedExchange (69) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3], [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [1]: [s_store_sk#8] -Condition : isnotnull(s_store_sk#8) - -(11) CometBroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: [s_store_sk#8] - -(12) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -Right output [1]: [s_store_sk#8] -Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight - -(13) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] - -(14) CometHashAggregate -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Keys [1]: [s_store_sk#8] -Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] - -(15) CometExchange -Input [3]: [s_store_sk#8, sum#9, sum#10] -Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [3]: [s_store_sk#8, sum#9, sum#10] -Keys [1]: [s_store_sk#8] -Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(18) CometFilter -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Condition : isnotnull(sr_store_sk#11) - -(19) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#16] - -(20) CometBroadcastHashJoin -Left output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Right output [1]: [d_date_sk#16] -Arguments: [sr_returned_date_sk#14], [d_date_sk#16], Inner, BuildRight - -(21) CometProject -Input [5]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14, d_date_sk#16] -Arguments: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13], [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] - -(22) ReusedExchange [Reuses operator id: 11] -Output [1]: [s_store_sk#17] - -(23) CometBroadcastHashJoin -Left output [3]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] -Right output [1]: [s_store_sk#17] -Arguments: [sr_store_sk#11], [s_store_sk#17], Inner, BuildRight - -(24) CometProject -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Arguments: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17], [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] - -(25) CometHashAggregate -Input [3]: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Keys [1]: [s_store_sk#17] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#12)), partial_sum(UnscaledValue(sr_net_loss#13))] - -(26) CometExchange -Input [3]: [s_store_sk#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(27) CometHashAggregate -Input [3]: [s_store_sk#17, sum#18, sum#19] -Keys [1]: [s_store_sk#17] -Functions [2]: [sum(UnscaledValue(sr_return_amt#12)), sum(UnscaledValue(sr_net_loss#13))] - -(28) CometBroadcastExchange -Input [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#17, returns#20, profit_loss#21] - -(29) CometBroadcastHashJoin -Left output [3]: [s_store_sk#8, sales#22, profit#23] -Right output [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight - -(30) CometProject -Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] -Arguments: [sales#22, returns#24, profit#25, channel#26, id#27], [sales#22, coalesce(returns#20, 0.00) AS returns#24, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#25, store channel AS channel#26, s_store_sk#8 AS id#27] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [sales#22, returns#24, profit#25, channel#26, id#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] -ReadSchema: struct - -(33) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(34) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [cs_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(35) CometProject -Input [5]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31, d_date_sk#33] -Arguments: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30], [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] - -(36) CometHashAggregate -Input [3]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#29)), partial_sum(UnscaledValue(cs_net_profit#30))] - -(37) CometExchange -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Arguments: hashpartitioning(cs_call_center_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] - -(39) CometColumnarToRow [codegen id : 2] -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] - -(40) BroadcastExchange -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] -Arguments: IdentityBroadcastMode, [plan_id=4] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] -ReadSchema: struct - -(42) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#42] - -(43) CometBroadcastHashJoin -Left output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Right output [1]: [d_date_sk#42] -Arguments: [cr_returned_date_sk#40], [d_date_sk#42], Inner, BuildRight - -(44) CometProject -Input [4]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40, d_date_sk#42] -Arguments: [cr_return_amount#38, cr_net_loss#39], [cr_return_amount#38, cr_net_loss#39] - -(45) CometHashAggregate -Input [2]: [cr_return_amount#38, cr_net_loss#39] -Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] - -(46) CometExchange -Input [2]: [sum#43, sum#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometHashAggregate -Input [2]: [sum#43, sum#44] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] - -(48) CometColumnarToRow -Input [2]: [returns#45, profit_loss#46] - -(49) BroadcastNestedLoopJoin [codegen id : 3] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 3] -Output [5]: [sales#36, returns#45, (profit#37 - profit_loss#46) AS profit#47, catalog channel AS channel#48, cs_call_center_sk#28 AS id#49] -Input [5]: [cs_call_center_sk#28, sales#36, profit#37, returns#45, profit_loss#46] - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] -PushedFilters: [IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(52) CometFilter -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Condition : isnotnull(ws_web_page_sk#50) - -(53) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#55] - -(54) CometBroadcastHashJoin -Left output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Right output [1]: [d_date_sk#55] -Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight - -(55) CometProject -Input [5]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53, d_date_sk#55] -Arguments: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52], [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(57) CometFilter -Input [1]: [wp_web_page_sk#56] -Condition : isnotnull(wp_web_page_sk#56) - -(58) CometBroadcastExchange -Input [1]: [wp_web_page_sk#56] -Arguments: [wp_web_page_sk#56] - -(59) CometBroadcastHashJoin -Left output [3]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] -Right output [1]: [wp_web_page_sk#56] -Arguments: [ws_web_page_sk#50], [wp_web_page_sk#56], Inner, BuildRight - -(60) CometProject -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Arguments: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56], [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] - -(61) CometHashAggregate -Input [3]: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#51)), partial_sum(UnscaledValue(ws_net_profit#52))] - -(62) CometExchange -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Arguments: hashpartitioning(wp_web_page_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(63) CometHashAggregate -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#51)), sum(UnscaledValue(ws_net_profit#52))] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] -PushedFilters: [IsNotNull(wr_web_page_sk)] -ReadSchema: struct - -(65) CometFilter -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Condition : isnotnull(wr_web_page_sk#59) - -(66) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#64] - -(67) CometBroadcastHashJoin -Left output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Right output [1]: [d_date_sk#64] -Arguments: [wr_returned_date_sk#62], [d_date_sk#64], Inner, BuildRight - -(68) CometProject -Input [5]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62, d_date_sk#64] -Arguments: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61], [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] - -(69) ReusedExchange [Reuses operator id: 58] -Output [1]: [wp_web_page_sk#65] - -(70) CometBroadcastHashJoin -Left output [3]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] -Right output [1]: [wp_web_page_sk#65] -Arguments: [wr_web_page_sk#59], [wp_web_page_sk#65], Inner, BuildRight - -(71) CometProject -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Arguments: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65], [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] - -(72) CometHashAggregate -Input [3]: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#60)), partial_sum(UnscaledValue(wr_net_loss#61))] - -(73) CometExchange -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Arguments: hashpartitioning(wp_web_page_sk#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(74) CometHashAggregate -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [sum(UnscaledValue(wr_return_amt#60)), sum(UnscaledValue(wr_net_loss#61))] - -(75) CometBroadcastExchange -Input [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#65, returns#68, profit_loss#69] - -(76) CometBroadcastHashJoin -Left output [3]: [wp_web_page_sk#56, sales#70, profit#71] -Right output [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight - -(77) CometProject -Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [sales#70, returns#72, profit#73, channel#74, id#75], [sales#70, coalesce(returns#68, 0.00) AS returns#72, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#73, web channel AS channel#74, wp_web_page_sk#56 AS id#75] - -(78) CometColumnarToRow [codegen id : 4] -Input [5]: [sales#70, returns#72, profit#73, channel#74, id#75] - -(79) Union - -(80) Expand [codegen id : 5] -Input [5]: [sales#22, returns#24, profit#25, channel#26, id#27] -Arguments: [[sales#22, returns#24, profit#25, channel#26, id#27, 0], [sales#22, returns#24, profit#25, channel#26, null, 1], [sales#22, returns#24, profit#25, null, null, 3]], [sales#22, returns#24, profit#25, channel#76, id#77, spark_grouping_id#78] - -(81) HashAggregate [codegen id : 5] -Input [6]: [sales#22, returns#24, profit#25, channel#76, id#77, spark_grouping_id#78] -Keys [3]: [channel#76, id#77, spark_grouping_id#78] -Functions [3]: [partial_sum(sales#22), partial_sum(returns#24), partial_sum(profit#25)] -Aggregate Attributes [6]: [sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84] -Results [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] - -(82) CometColumnarExchange -Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] -Arguments: hashpartitioning(channel#76, id#77, spark_grouping_id#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(83) CometColumnarToRow [codegen id : 6] -Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] - -(84) HashAggregate [codegen id : 6] -Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] -Keys [3]: [channel#76, id#77, spark_grouping_id#78] -Functions [3]: [sum(sales#22), sum(returns#24), sum(profit#25)] -Aggregate Attributes [3]: [sum(sales#22)#91, sum(returns#24)#92, sum(profit#25)#93] -Results [5]: [channel#76, id#77, sum(sales#22)#91 AS sales#94, sum(returns#24)#92 AS returns#95, sum(profit#25)#93 AS profit#96] - -(85) TakeOrderedAndProject -Input [5]: [channel#76, id#77, sales#94, returns#95, profit#96] -Arguments: 100, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#94, returns#95, profit#96] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (90) -+- * CometColumnarToRow (89) - +- CometProject (88) - +- CometFilter (87) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (86) - - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(87) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) - -(88) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(89) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(90) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#14 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 64 Hosting Expression = wr_returned_date_sk#62 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/simplified.txt deleted file mode 100644 index a1243769e5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,111 +0,0 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (6) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (5) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [returns,profit,profit_loss,s_store_sk] [sales,returns,profit,channel,id] - CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [s_store_sk,sales,profit,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] - CometExchange [s_store_sk] #2 - CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk] #5 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [s_store_sk,returns,profit_loss] #6 - CometHashAggregate [sum,sum] [s_store_sk,returns,profit_loss,sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss))] - CometExchange [s_store_sk] #7 - CometHashAggregate [sr_return_amt,sr_net_loss] [s_store_sk,sum,sum] - CometProject [sr_return_amt,sr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [s_store_sk] #5 - WholeStageCodegen (3) - Project [sales,returns,profit,profit_loss,cs_call_center_sk] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [cs_call_center_sk,sales,profit,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] - CometExchange [cs_call_center_sk] #9 - CometHashAggregate [cs_ext_sales_price,cs_net_profit] [cs_call_center_sk,sum,sum] - CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [returns,profit_loss,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] - CometExchange #10 - CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] - CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [returns,profit,profit_loss,wp_web_page_sk] [sales,returns,profit,channel,id] - CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [wp_web_page_sk,sales,profit,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] - CometExchange [wp_web_page_sk] #11 - CometHashAggregate [ws_ext_sales_price,ws_net_profit] [wp_web_page_sk,sum,sum] - CometProject [ws_ext_sales_price,ws_net_profit,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_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 [native_iceberg_compat] 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 [wp_web_page_sk] #12 - CometFilter [wp_web_page_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [wp_web_page_sk,returns,profit_loss] #13 - CometHashAggregate [sum,sum] [wp_web_page_sk,returns,profit_loss,sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss))] - CometExchange [wp_web_page_sk] #14 - CometHashAggregate [wr_return_amt,wr_net_loss] [wp_web_page_sk,sum,sum] - CometProject [wr_return_amt,wr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [wp_web_page_sk] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/explain.txt deleted file mode 100644 index bb7ed0a50c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/explain.txt +++ /dev/null @@ -1,518 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (85) -+- * HashAggregate (84) - +- * CometColumnarToRow (83) - +- CometColumnarExchange (82) - +- * HashAggregate (81) - +- * Expand (80) - +- Union (79) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometHashAggregate (16) - : : +- CometExchange (15) - : : +- CometHashAggregate (14) - : : +- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (28) - : +- CometHashAggregate (27) - : +- CometExchange (26) - : +- CometHashAggregate (25) - : +- CometProject (24) - : +- CometBroadcastHashJoin (23) - : :- CometProject (21) - : : +- CometBroadcastHashJoin (20) - : : :- CometFilter (18) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (17) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - :- * Project (50) - : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) - : :- BroadcastExchange (40) - : : +- * CometColumnarToRow (39) - : : +- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (32) - : : +- ReusedExchange (33) - : +- * CometColumnarToRow (48) - : +- CometHashAggregate (47) - : +- CometExchange (46) - : +- CometHashAggregate (45) - : +- CometProject (44) - : +- CometBroadcastHashJoin (43) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (41) - : +- ReusedExchange (42) - +- * CometColumnarToRow (78) - +- CometProject (77) - +- CometBroadcastHashJoin (76) - :- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (55) - : : +- CometBroadcastHashJoin (54) - : : :- CometFilter (52) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (51) - : : +- ReusedExchange (53) - : +- CometBroadcastExchange (58) - : +- CometFilter (57) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (56) - +- CometBroadcastExchange (75) - +- CometHashAggregate (74) - +- CometExchange (73) - +- CometHashAggregate (72) - +- CometProject (71) - +- CometBroadcastHashJoin (70) - :- CometProject (68) - : +- CometBroadcastHashJoin (67) - : :- CometFilter (65) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (64) - : +- ReusedExchange (66) - +- ReusedExchange (69) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3], [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [1]: [s_store_sk#8] -Condition : isnotnull(s_store_sk#8) - -(11) CometBroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: [s_store_sk#8] - -(12) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -Right output [1]: [s_store_sk#8] -Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight - -(13) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] - -(14) CometHashAggregate -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Keys [1]: [s_store_sk#8] -Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] - -(15) CometExchange -Input [3]: [s_store_sk#8, sum#9, sum#10] -Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [3]: [s_store_sk#8, sum#9, sum#10] -Keys [1]: [s_store_sk#8] -Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(18) CometFilter -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Condition : isnotnull(sr_store_sk#11) - -(19) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#16] - -(20) CometBroadcastHashJoin -Left output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Right output [1]: [d_date_sk#16] -Arguments: [sr_returned_date_sk#14], [d_date_sk#16], Inner, BuildRight - -(21) CometProject -Input [5]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14, d_date_sk#16] -Arguments: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13], [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] - -(22) ReusedExchange [Reuses operator id: 11] -Output [1]: [s_store_sk#17] - -(23) CometBroadcastHashJoin -Left output [3]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] -Right output [1]: [s_store_sk#17] -Arguments: [sr_store_sk#11], [s_store_sk#17], Inner, BuildRight - -(24) CometProject -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Arguments: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17], [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] - -(25) CometHashAggregate -Input [3]: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Keys [1]: [s_store_sk#17] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#12)), partial_sum(UnscaledValue(sr_net_loss#13))] - -(26) CometExchange -Input [3]: [s_store_sk#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(27) CometHashAggregate -Input [3]: [s_store_sk#17, sum#18, sum#19] -Keys [1]: [s_store_sk#17] -Functions [2]: [sum(UnscaledValue(sr_return_amt#12)), sum(UnscaledValue(sr_net_loss#13))] - -(28) CometBroadcastExchange -Input [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#17, returns#20, profit_loss#21] - -(29) CometBroadcastHashJoin -Left output [3]: [s_store_sk#8, sales#22, profit#23] -Right output [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight - -(30) CometProject -Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] -Arguments: [sales#22, returns#24, profit#25, channel#26, id#27], [sales#22, coalesce(returns#20, 0.00) AS returns#24, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#25, store channel AS channel#26, s_store_sk#8 AS id#27] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [sales#22, returns#24, profit#25, channel#26, id#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] -ReadSchema: struct - -(33) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(34) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [cs_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(35) CometProject -Input [5]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31, d_date_sk#33] -Arguments: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30], [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] - -(36) CometHashAggregate -Input [3]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#29)), partial_sum(UnscaledValue(cs_net_profit#30))] - -(37) CometExchange -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Arguments: hashpartitioning(cs_call_center_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] - -(39) CometColumnarToRow [codegen id : 2] -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] - -(40) BroadcastExchange -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] -Arguments: IdentityBroadcastMode, [plan_id=4] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] -ReadSchema: struct - -(42) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#42] - -(43) CometBroadcastHashJoin -Left output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Right output [1]: [d_date_sk#42] -Arguments: [cr_returned_date_sk#40], [d_date_sk#42], Inner, BuildRight - -(44) CometProject -Input [4]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40, d_date_sk#42] -Arguments: [cr_return_amount#38, cr_net_loss#39], [cr_return_amount#38, cr_net_loss#39] - -(45) CometHashAggregate -Input [2]: [cr_return_amount#38, cr_net_loss#39] -Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] - -(46) CometExchange -Input [2]: [sum#43, sum#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometHashAggregate -Input [2]: [sum#43, sum#44] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] - -(48) CometColumnarToRow -Input [2]: [returns#45, profit_loss#46] - -(49) BroadcastNestedLoopJoin [codegen id : 3] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 3] -Output [5]: [sales#36, returns#45, (profit#37 - profit_loss#46) AS profit#47, catalog channel AS channel#48, cs_call_center_sk#28 AS id#49] -Input [5]: [cs_call_center_sk#28, sales#36, profit#37, returns#45, profit_loss#46] - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] -PushedFilters: [IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(52) CometFilter -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Condition : isnotnull(ws_web_page_sk#50) - -(53) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#55] - -(54) CometBroadcastHashJoin -Left output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Right output [1]: [d_date_sk#55] -Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight - -(55) CometProject -Input [5]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53, d_date_sk#55] -Arguments: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52], [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(57) CometFilter -Input [1]: [wp_web_page_sk#56] -Condition : isnotnull(wp_web_page_sk#56) - -(58) CometBroadcastExchange -Input [1]: [wp_web_page_sk#56] -Arguments: [wp_web_page_sk#56] - -(59) CometBroadcastHashJoin -Left output [3]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] -Right output [1]: [wp_web_page_sk#56] -Arguments: [ws_web_page_sk#50], [wp_web_page_sk#56], Inner, BuildRight - -(60) CometProject -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Arguments: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56], [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] - -(61) CometHashAggregate -Input [3]: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#51)), partial_sum(UnscaledValue(ws_net_profit#52))] - -(62) CometExchange -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Arguments: hashpartitioning(wp_web_page_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(63) CometHashAggregate -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#51)), sum(UnscaledValue(ws_net_profit#52))] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] -PushedFilters: [IsNotNull(wr_web_page_sk)] -ReadSchema: struct - -(65) CometFilter -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Condition : isnotnull(wr_web_page_sk#59) - -(66) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#64] - -(67) CometBroadcastHashJoin -Left output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Right output [1]: [d_date_sk#64] -Arguments: [wr_returned_date_sk#62], [d_date_sk#64], Inner, BuildRight - -(68) CometProject -Input [5]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62, d_date_sk#64] -Arguments: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61], [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] - -(69) ReusedExchange [Reuses operator id: 58] -Output [1]: [wp_web_page_sk#65] - -(70) CometBroadcastHashJoin -Left output [3]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] -Right output [1]: [wp_web_page_sk#65] -Arguments: [wr_web_page_sk#59], [wp_web_page_sk#65], Inner, BuildRight - -(71) CometProject -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Arguments: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65], [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] - -(72) CometHashAggregate -Input [3]: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#60)), partial_sum(UnscaledValue(wr_net_loss#61))] - -(73) CometExchange -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Arguments: hashpartitioning(wp_web_page_sk#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(74) CometHashAggregate -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [sum(UnscaledValue(wr_return_amt#60)), sum(UnscaledValue(wr_net_loss#61))] - -(75) CometBroadcastExchange -Input [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#65, returns#68, profit_loss#69] - -(76) CometBroadcastHashJoin -Left output [3]: [wp_web_page_sk#56, sales#70, profit#71] -Right output [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight - -(77) CometProject -Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [sales#70, returns#72, profit#73, channel#74, id#75], [sales#70, coalesce(returns#68, 0.00) AS returns#72, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#73, web channel AS channel#74, wp_web_page_sk#56 AS id#75] - -(78) CometColumnarToRow [codegen id : 4] -Input [5]: [sales#70, returns#72, profit#73, channel#74, id#75] - -(79) Union - -(80) Expand [codegen id : 5] -Input [5]: [sales#22, returns#24, profit#25, channel#26, id#27] -Arguments: [[sales#22, returns#24, profit#25, channel#26, id#27, 0], [sales#22, returns#24, profit#25, channel#26, null, 1], [sales#22, returns#24, profit#25, null, null, 3]], [sales#22, returns#24, profit#25, channel#76, id#77, spark_grouping_id#78] - -(81) HashAggregate [codegen id : 5] -Input [6]: [sales#22, returns#24, profit#25, channel#76, id#77, spark_grouping_id#78] -Keys [3]: [channel#76, id#77, spark_grouping_id#78] -Functions [3]: [partial_sum(sales#22), partial_sum(returns#24), partial_sum(profit#25)] -Aggregate Attributes [6]: [sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84] -Results [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] - -(82) CometColumnarExchange -Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] -Arguments: hashpartitioning(channel#76, id#77, spark_grouping_id#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(83) CometColumnarToRow [codegen id : 6] -Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] - -(84) HashAggregate [codegen id : 6] -Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] -Keys [3]: [channel#76, id#77, spark_grouping_id#78] -Functions [3]: [sum(sales#22), sum(returns#24), sum(profit#25)] -Aggregate Attributes [3]: [sum(sales#22)#91, sum(returns#24)#92, sum(profit#25)#93] -Results [5]: [channel#76, id#77, sum(sales#22)#91 AS sales#94, sum(returns#24)#92 AS returns#95, sum(profit#25)#93 AS profit#96] - -(85) TakeOrderedAndProject -Input [5]: [channel#76, id#77, sales#94, returns#95, profit#96] -Arguments: 100, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#94, returns#95, profit#96] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (90) -+- * CometColumnarToRow (89) - +- CometProject (88) - +- CometFilter (87) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (86) - - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(87) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) - -(88) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(89) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(90) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#14 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 64 Hosting Expression = wr_returned_date_sk#62 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/extended.txt deleted file mode 100644 index ed8a9e38ca..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/extended.txt +++ /dev/null @@ -1,117 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : :- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - -Comet accelerated 94 out of 109 eligible operators (86%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index a1243769e5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/simplified.txt +++ /dev/null @@ -1,111 +0,0 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (6) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (5) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [returns,profit,profit_loss,s_store_sk] [sales,returns,profit,channel,id] - CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [s_store_sk,sales,profit,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] - CometExchange [s_store_sk] #2 - CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk] #5 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [s_store_sk,returns,profit_loss] #6 - CometHashAggregate [sum,sum] [s_store_sk,returns,profit_loss,sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss))] - CometExchange [s_store_sk] #7 - CometHashAggregate [sr_return_amt,sr_net_loss] [s_store_sk,sum,sum] - CometProject [sr_return_amt,sr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [s_store_sk] #5 - WholeStageCodegen (3) - Project [sales,returns,profit,profit_loss,cs_call_center_sk] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [cs_call_center_sk,sales,profit,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] - CometExchange [cs_call_center_sk] #9 - CometHashAggregate [cs_ext_sales_price,cs_net_profit] [cs_call_center_sk,sum,sum] - CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [returns,profit_loss,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] - CometExchange #10 - CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] - CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [returns,profit,profit_loss,wp_web_page_sk] [sales,returns,profit,channel,id] - CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [wp_web_page_sk,sales,profit,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] - CometExchange [wp_web_page_sk] #11 - CometHashAggregate [ws_ext_sales_price,ws_net_profit] [wp_web_page_sk,sum,sum] - CometProject [ws_ext_sales_price,ws_net_profit,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_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 [native_iceberg_compat] 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 [wp_web_page_sk] #12 - CometFilter [wp_web_page_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [wp_web_page_sk,returns,profit_loss] #13 - CometHashAggregate [sum,sum] [wp_web_page_sk,returns,profit_loss,sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss))] - CometExchange [wp_web_page_sk] #14 - CometHashAggregate [wr_return_amt,wr_net_loss] [wp_web_page_sk,sum,sum] - CometProject [wr_return_amt,wr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [wp_web_page_sk] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/explain.txt deleted file mode 100644 index 4dc7ac380a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/explain.txt +++ /dev/null @@ -1,417 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (70) -+- * Project (69) - +- * CometColumnarToRow (68) - +- CometSortMergeJoin (67) - :- CometProject (45) - : +- CometSortMergeJoin (44) - : :- CometSort (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometFilter (12) - : : : +- CometSortMergeJoin (11) - : : : :- CometSort (5) - : : : : +- CometColumnarExchange (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (10) - : : : +- CometExchange (9) - : : : +- CometProject (8) - : : : +- CometFilter (7) - : : : +- CometNativeScan parquet spark_catalog.default.store_returns (6) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (14) - : +- CometSort (43) - : +- CometFilter (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometFilter (34) - : : +- CometSortMergeJoin (33) - : : :- CometSort (27) - : : : +- CometColumnarExchange (26) - : : : +- * Filter (25) - : : : +- * ColumnarToRow (24) - : : : +- Scan parquet spark_catalog.default.web_sales (23) - : : +- CometSort (32) - : : +- CometExchange (31) - : : +- CometProject (30) - : : +- CometFilter (29) - : : +- CometNativeScan parquet spark_catalog.default.web_returns (28) - : +- ReusedExchange (36) - +- CometSort (66) - +- CometFilter (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometFilter (57) - : +- CometSortMergeJoin (56) - : :- CometSort (50) - : : +- CometColumnarExchange (49) - : : +- * Filter (48) - : : +- * ColumnarToRow (47) - : : +- Scan parquet spark_catalog.default.catalog_sales (46) - : +- CometSort (55) - : +- CometExchange (54) - : +- CometProject (53) - : +- CometFilter (52) - : +- CometNativeScan parquet spark_catalog.default.catalog_returns (51) - +- ReusedExchange (59) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(3) Filter [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_customer_sk#2)) - -(4) CometColumnarExchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_ticket_number#3, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_ticket_number#3 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) - -(8) CometProject -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] - -(9) CometExchange -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter - -(12) CometFilter -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Condition : isnull(sr_ticket_number#10) - -(13) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(14) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(17) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] - -(19) CometHashAggregate -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] - -(20) CometExchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(21) CometHashAggregate -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] - -(22) CometSort -Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] - -(23) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 2] -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(25) Filter [codegen id : 2] -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) - -(26) CometColumnarExchange -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometSort -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] - -(28) CometNativeScan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#28, wr_order_number#29, wr_returned_date_sk#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(29) CometFilter -Input [3]: [wr_item_sk#28, wr_order_number#29, wr_returned_date_sk#30] -Condition : (isnotnull(wr_order_number#29) AND isnotnull(wr_item_sk#28)) - -(30) CometProject -Input [3]: [wr_item_sk#28, wr_order_number#29, wr_returned_date_sk#30] -Arguments: [wr_item_sk#28, wr_order_number#29], [wr_item_sk#28, wr_order_number#29] - -(31) CometExchange -Input [2]: [wr_item_sk#28, wr_order_number#29] -Arguments: hashpartitioning(wr_order_number#29, wr_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(32) CometSort -Input [2]: [wr_item_sk#28, wr_order_number#29] -Arguments: [wr_item_sk#28, wr_order_number#29], [wr_order_number#29 ASC NULLS FIRST, wr_item_sk#28 ASC NULLS FIRST] - -(33) CometSortMergeJoin -Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [wr_item_sk#28, wr_order_number#29] -Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#29, wr_item_sk#28], LeftOuter - -(34) CometFilter -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#28, wr_order_number#29] -Condition : isnull(wr_order_number#29) - -(35) CometProject -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#28, wr_order_number#29] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(36) ReusedExchange [Reuses operator id: 16] -Output [2]: [d_date_sk#31, d_year#32] - -(37) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [d_date_sk#31, d_year#32] -Arguments: [ws_sold_date_sk#27], [d_date_sk#31], Inner, BuildRight - -(38) CometProject -Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#31, d_year#32] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#32], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#32] - -(39) CometHashAggregate -Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#32] -Keys [3]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] - -(40) CometExchange -Input [6]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22, sum#33, sum#34, sum#35] -Arguments: hashpartitioning(d_year#32, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(41) CometHashAggregate -Input [6]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22, sum#33, sum#34, sum#35] -Keys [3]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] - -(42) CometFilter -Input [6]: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Condition : (coalesce(ws_qty#38, 0) > 0) - -(43) CometSort -Input [6]: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Arguments: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40], [ws_sold_year#36 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#37 ASC NULLS FIRST] - -(44) CometSortMergeJoin -Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Right output [6]: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37], Inner - -(45) CometProject -Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40] - -(46) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#47), dynamicpruningexpression(cs_sold_date_sk#47 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(47) ColumnarToRow [codegen id : 3] -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] - -(48) Filter [codegen id : 3] -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Condition : (isnotnull(cs_item_sk#42) AND isnotnull(cs_bill_customer_sk#41)) - -(49) CometColumnarExchange -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Arguments: hashpartitioning(cs_order_number#43, cs_item_sk#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(50) CometSort -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Arguments: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47], [cs_order_number#43 ASC NULLS FIRST, cs_item_sk#42 ASC NULLS FIRST] - -(51) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#48, cr_order_number#49, cr_returned_date_sk#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(52) CometFilter -Input [3]: [cr_item_sk#48, cr_order_number#49, cr_returned_date_sk#50] -Condition : (isnotnull(cr_order_number#49) AND isnotnull(cr_item_sk#48)) - -(53) CometProject -Input [3]: [cr_item_sk#48, cr_order_number#49, cr_returned_date_sk#50] -Arguments: [cr_item_sk#48, cr_order_number#49], [cr_item_sk#48, cr_order_number#49] - -(54) CometExchange -Input [2]: [cr_item_sk#48, cr_order_number#49] -Arguments: hashpartitioning(cr_order_number#49, cr_item_sk#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(55) CometSort -Input [2]: [cr_item_sk#48, cr_order_number#49] -Arguments: [cr_item_sk#48, cr_order_number#49], [cr_order_number#49 ASC NULLS FIRST, cr_item_sk#48 ASC NULLS FIRST] - -(56) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Right output [2]: [cr_item_sk#48, cr_order_number#49] -Arguments: [cs_order_number#43, cs_item_sk#42], [cr_order_number#49, cr_item_sk#48], LeftOuter - -(57) CometFilter -Input [9]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47, cr_item_sk#48, cr_order_number#49] -Condition : isnull(cr_order_number#49) - -(58) CometProject -Input [9]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47, cr_item_sk#48, cr_order_number#49] -Arguments: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47], [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] - -(59) ReusedExchange [Reuses operator id: 16] -Output [2]: [d_date_sk#51, d_year#52] - -(60) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Right output [2]: [d_date_sk#51, d_year#52] -Arguments: [cs_sold_date_sk#47], [d_date_sk#51], Inner, BuildRight - -(61) CometProject -Input [8]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47, d_date_sk#51, d_year#52] -Arguments: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, d_year#52], [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, d_year#52] - -(62) CometHashAggregate -Input [6]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, d_year#52] -Keys [3]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41] -Functions [3]: [partial_sum(cs_quantity#44), partial_sum(UnscaledValue(cs_wholesale_cost#45)), partial_sum(UnscaledValue(cs_sales_price#46))] - -(63) CometExchange -Input [6]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41, sum#53, sum#54, sum#55] -Arguments: hashpartitioning(d_year#52, cs_item_sk#42, cs_bill_customer_sk#41, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(64) CometHashAggregate -Input [6]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41, sum#53, sum#54, sum#55] -Keys [3]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41] -Functions [3]: [sum(cs_quantity#44), sum(UnscaledValue(cs_wholesale_cost#45)), sum(UnscaledValue(cs_sales_price#46))] - -(65) CometFilter -Input [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] -Condition : (coalesce(cs_qty#58, 0) > 0) - -(66) CometSort -Input [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] -Arguments: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60], [cs_sold_year#56 ASC NULLS FIRST, cs_item_sk#42 ASC NULLS FIRST, cs_customer_sk#57 ASC NULLS FIRST] - -(67) CometSortMergeJoin -Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40] -Right output [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57], Inner - -(68) CometColumnarToRow [codegen id : 4] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40, cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] - -(69) Project [codegen id : 4] -Output [12]: [round((cast(ss_qty#18 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#38 + cs_qty#58), 1) as double)))), 2) AS ratio#61, ss_qty#18 AS store_qty#62, ss_wc#19 AS store_wholesale_cost#63, ss_sp#20 AS store_sales_price#64, (coalesce(ws_qty#38, 0) + coalesce(cs_qty#58, 0)) AS other_chan_qty#65, (coalesce(ws_wc#39, 0.00) + coalesce(cs_wc#59, 0.00)) AS other_chan_wholesale_cost#66, (coalesce(ws_sp#40, 0.00) + coalesce(cs_sp#60, 0.00)) AS other_chan_sales_price#67, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, cs_qty#58] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40, cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] - -(70) TakeOrderedAndProject -Input [12]: [ratio#61, store_qty#62, store_wholesale_cost#63, store_sales_price#64, other_chan_qty#65, other_chan_wholesale_cost#66, other_chan_sales_price#67, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, cs_qty#58] -Arguments: 100, [ratio#61 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#65 ASC NULLS FIRST, other_chan_wholesale_cost#66 ASC NULLS FIRST, other_chan_sales_price#67 ASC NULLS FIRST, round((cast(ss_qty#18 as double) / cast(coalesce((ws_qty#38 + cs_qty#58), 1) as double)), 2) ASC NULLS FIRST], [ratio#61, store_qty#62, store_wholesale_cost#63, store_sales_price#64, other_chan_qty#65, other_chan_wholesale_cost#66, other_chan_sales_price#67] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (74) -+- * CometColumnarToRow (73) - +- CometFilter (72) - +- CometNativeScan parquet spark_catalog.default.date_dim (71) - - -(71) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(72) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(73) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(74) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#47 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/simplified.txt deleted file mode 100644 index ebd0840eec..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/simplified.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (4) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - CometColumnarToRow - InputAdapter - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] - CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #2 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #5 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] - CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometColumnarExchange [ws_order_number,ws_item_sk] #7 - WholeStageCodegen (2) - Filter [ws_item_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_order_number,wr_item_sk] #8 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] - CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometColumnarExchange [cs_order_number,cs_item_sk] #10 - WholeStageCodegen (3) - Filter [cs_item_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_order_number,cr_item_sk] #11 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/explain.txt deleted file mode 100644 index 3d3ea5ab86..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,405 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (67) -+- * Project (66) - +- * CometColumnarToRow (65) - +- CometSortMergeJoin (64) - :- CometProject (43) - : +- CometSortMergeJoin (42) - : :- CometSort (21) - : : +- CometHashAggregate (20) - : : +- CometExchange (19) - : : +- CometHashAggregate (18) - : : +- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : +- CometBroadcastExchange (15) - : : +- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : +- CometSort (41) - : +- CometFilter (40) - : +- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (33) - : : +- CometFilter (32) - : : +- CometSortMergeJoin (31) - : : :- CometSort (25) - : : : +- CometExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (22) - : : +- CometSort (30) - : : +- CometExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (26) - : +- ReusedExchange (34) - +- CometSort (63) - +- CometFilter (62) - +- CometHashAggregate (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometProject (55) - : +- CometFilter (54) - : +- CometSortMergeJoin (53) - : :- CometSort (47) - : : +- CometExchange (46) - : : +- CometFilter (45) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (44) - : +- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometFilter (49) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (48) - +- ReusedExchange (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_customer_sk#2)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_ticket_number#3, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_ticket_number#3 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) - -(7) CometProject -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] - -(8) CometExchange -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter - -(11) CometFilter -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Condition : isnull(sr_ticket_number#10) - -(12) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(15) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(16) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight - -(17) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] - -(18) CometHashAggregate -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] - -(19) CometExchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometHashAggregate -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] - -(21) CometSort -Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(23) CometFilter -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) - -(24) CometExchange -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometSort -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Condition : (isnotnull(wr_order_number#30) AND isnotnull(wr_item_sk#29)) - -(28) CometProject -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_item_sk#29, wr_order_number#30] - -(29) CometExchange -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: hashpartitioning(wr_order_number#30, wr_item_sk#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(30) CometSort -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_order_number#30 ASC NULLS FIRST, wr_item_sk#29 ASC NULLS FIRST] - -(31) CometSortMergeJoin -Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#30, wr_item_sk#29], LeftOuter - -(32) CometFilter -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Condition : isnull(wr_order_number#30) - -(33) CometProject -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(34) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#32, d_year#33] - -(35) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ws_sold_date_sk#27], [d_date_sk#32], Inner, BuildRight - -(36) CometProject -Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#32, d_year#33] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] - -(37) CometHashAggregate -Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] - -(38) CometExchange -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Arguments: hashpartitioning(d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(39) CometHashAggregate -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] - -(40) CometFilter -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Condition : (coalesce(ws_qty#39, 0) > 0) - -(41) CometSort -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41], [ws_sold_year#37 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#38 ASC NULLS FIRST] - -(42) CometSortMergeJoin -Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Right output [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38], Inner - -(43) CometProject -Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#48), dynamicpruningexpression(cs_sold_date_sk#48 IN dynamicpruning#49)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(45) CometFilter -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Condition : (isnotnull(cs_item_sk#43) AND isnotnull(cs_bill_customer_sk#42)) - -(46) CometExchange -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(47) CometSort -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST] - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Condition : (isnotnull(cr_order_number#51) AND isnotnull(cr_item_sk#50)) - -(50) CometProject -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_item_sk#50, cr_order_number#51] - -(51) CometExchange -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(52) CometSort -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST] - -(53) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cs_order_number#44, cs_item_sk#43], [cr_order_number#51, cr_item_sk#50], LeftOuter - -(54) CometFilter -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Condition : isnull(cr_order_number#51) - -(55) CometProject -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] - -(56) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#53, d_year#54] - -(57) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [cs_sold_date_sk#48], [d_date_sk#53], Inner, BuildRight - -(58) CometProject -Input [8]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, d_date_sk#53, d_year#54] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] - -(59) CometHashAggregate -Input [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [partial_sum(cs_quantity#45), partial_sum(UnscaledValue(cs_wholesale_cost#46)), partial_sum(UnscaledValue(cs_sales_price#47))] - -(60) CometExchange -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Arguments: hashpartitioning(d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(61) CometHashAggregate -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [sum(cs_quantity#45), sum(UnscaledValue(cs_wholesale_cost#46)), sum(UnscaledValue(cs_sales_price#47))] - -(62) CometFilter -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Condition : (coalesce(cs_qty#60, 0) > 0) - -(63) CometSort -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62], [cs_sold_year#58 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST, cs_customer_sk#59 ASC NULLS FIRST] - -(64) CometSortMergeJoin -Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] -Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner - -(65) CometColumnarToRow [codegen id : 1] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(66) Project [codegen id : 1] -Output [12]: [round((cast(ss_qty#18 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)))), 2) AS ratio#63, ss_qty#18 AS store_qty#64, ss_wc#19 AS store_wholesale_cost#65, ss_sp#20 AS store_sales_price#66, (coalesce(ws_qty#39, 0) + coalesce(cs_qty#60, 0)) AS other_chan_qty#67, (coalesce(ws_wc#40, 0.00) + coalesce(cs_wc#61, 0.00)) AS other_chan_wholesale_cost#68, (coalesce(ws_sp#41, 0.00) + coalesce(cs_sp#62, 0.00)) AS other_chan_sales_price#69, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, cs_qty#60] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(67) TakeOrderedAndProject -Input [12]: [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, cs_qty#60] -Arguments: 100, [ratio#63 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#67 ASC NULLS FIRST, other_chan_wholesale_cost#68 ASC NULLS FIRST, other_chan_sales_price#69 ASC NULLS FIRST, round((cast(ss_qty#18 as double) / cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)), 2) ASC NULLS FIRST], [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) - - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(70) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(71) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/simplified.txt deleted file mode 100644 index 77c05217f4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,78 +0,0 @@ -TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (1) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - CometColumnarToRow - InputAdapter - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] - CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #5 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] - CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometExchange [ws_order_number,ws_item_sk] #7 - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_order_number,wr_item_sk] #8 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] - CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometExchange [cs_order_number,cs_item_sk] #10 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_order_number,cr_item_sk] #11 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/explain.txt deleted file mode 100644 index 3d3ea5ab86..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/explain.txt +++ /dev/null @@ -1,405 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (67) -+- * Project (66) - +- * CometColumnarToRow (65) - +- CometSortMergeJoin (64) - :- CometProject (43) - : +- CometSortMergeJoin (42) - : :- CometSort (21) - : : +- CometHashAggregate (20) - : : +- CometExchange (19) - : : +- CometHashAggregate (18) - : : +- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : +- CometBroadcastExchange (15) - : : +- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : +- CometSort (41) - : +- CometFilter (40) - : +- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (33) - : : +- CometFilter (32) - : : +- CometSortMergeJoin (31) - : : :- CometSort (25) - : : : +- CometExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (22) - : : +- CometSort (30) - : : +- CometExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (26) - : +- ReusedExchange (34) - +- CometSort (63) - +- CometFilter (62) - +- CometHashAggregate (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometProject (55) - : +- CometFilter (54) - : +- CometSortMergeJoin (53) - : :- CometSort (47) - : : +- CometExchange (46) - : : +- CometFilter (45) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (44) - : +- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometFilter (49) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (48) - +- ReusedExchange (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_customer_sk#2)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_ticket_number#3, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_ticket_number#3 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) - -(7) CometProject -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] - -(8) CometExchange -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter - -(11) CometFilter -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Condition : isnull(sr_ticket_number#10) - -(12) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(15) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(16) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight - -(17) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] - -(18) CometHashAggregate -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] - -(19) CometExchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometHashAggregate -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] - -(21) CometSort -Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(23) CometFilter -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) - -(24) CometExchange -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometSort -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Condition : (isnotnull(wr_order_number#30) AND isnotnull(wr_item_sk#29)) - -(28) CometProject -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_item_sk#29, wr_order_number#30] - -(29) CometExchange -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: hashpartitioning(wr_order_number#30, wr_item_sk#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(30) CometSort -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_order_number#30 ASC NULLS FIRST, wr_item_sk#29 ASC NULLS FIRST] - -(31) CometSortMergeJoin -Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#30, wr_item_sk#29], LeftOuter - -(32) CometFilter -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Condition : isnull(wr_order_number#30) - -(33) CometProject -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(34) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#32, d_year#33] - -(35) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ws_sold_date_sk#27], [d_date_sk#32], Inner, BuildRight - -(36) CometProject -Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#32, d_year#33] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] - -(37) CometHashAggregate -Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] - -(38) CometExchange -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Arguments: hashpartitioning(d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(39) CometHashAggregate -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] - -(40) CometFilter -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Condition : (coalesce(ws_qty#39, 0) > 0) - -(41) CometSort -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41], [ws_sold_year#37 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#38 ASC NULLS FIRST] - -(42) CometSortMergeJoin -Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Right output [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38], Inner - -(43) CometProject -Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#48), dynamicpruningexpression(cs_sold_date_sk#48 IN dynamicpruning#49)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(45) CometFilter -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Condition : (isnotnull(cs_item_sk#43) AND isnotnull(cs_bill_customer_sk#42)) - -(46) CometExchange -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(47) CometSort -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST] - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Condition : (isnotnull(cr_order_number#51) AND isnotnull(cr_item_sk#50)) - -(50) CometProject -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_item_sk#50, cr_order_number#51] - -(51) CometExchange -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(52) CometSort -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST] - -(53) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cs_order_number#44, cs_item_sk#43], [cr_order_number#51, cr_item_sk#50], LeftOuter - -(54) CometFilter -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Condition : isnull(cr_order_number#51) - -(55) CometProject -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] - -(56) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#53, d_year#54] - -(57) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [cs_sold_date_sk#48], [d_date_sk#53], Inner, BuildRight - -(58) CometProject -Input [8]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, d_date_sk#53, d_year#54] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] - -(59) CometHashAggregate -Input [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [partial_sum(cs_quantity#45), partial_sum(UnscaledValue(cs_wholesale_cost#46)), partial_sum(UnscaledValue(cs_sales_price#47))] - -(60) CometExchange -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Arguments: hashpartitioning(d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(61) CometHashAggregate -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [sum(cs_quantity#45), sum(UnscaledValue(cs_wholesale_cost#46)), sum(UnscaledValue(cs_sales_price#47))] - -(62) CometFilter -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Condition : (coalesce(cs_qty#60, 0) > 0) - -(63) CometSort -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62], [cs_sold_year#58 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST, cs_customer_sk#59 ASC NULLS FIRST] - -(64) CometSortMergeJoin -Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] -Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner - -(65) CometColumnarToRow [codegen id : 1] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(66) Project [codegen id : 1] -Output [12]: [round((cast(ss_qty#18 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)))), 2) AS ratio#63, ss_qty#18 AS store_qty#64, ss_wc#19 AS store_wholesale_cost#65, ss_sp#20 AS store_sales_price#66, (coalesce(ws_qty#39, 0) + coalesce(cs_qty#60, 0)) AS other_chan_qty#67, (coalesce(ws_wc#40, 0.00) + coalesce(cs_wc#61, 0.00)) AS other_chan_wholesale_cost#68, (coalesce(ws_sp#41, 0.00) + coalesce(cs_sp#62, 0.00)) AS other_chan_sales_price#69, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, cs_qty#60] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(67) TakeOrderedAndProject -Input [12]: [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, cs_qty#60] -Arguments: 100, [ratio#63 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#67 ASC NULLS FIRST, other_chan_wholesale_cost#68 ASC NULLS FIRST, other_chan_sales_price#69 ASC NULLS FIRST, round((cast(ss_qty#18 as double) / cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)), 2) ASC NULLS FIRST], [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) - - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(70) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(71) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/extended.txt deleted file mode 100644 index 9d5ba53be1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/extended.txt +++ /dev/null @@ -1,80 +0,0 @@ -TakeOrderedAndProject -+- Project [COMET: Comet does not support Spark's BigDecimal rounding] - +- CometColumnarToRow - +- CometSortMergeJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 77c05217f4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78/simplified.txt +++ /dev/null @@ -1,78 +0,0 @@ -TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (1) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - CometColumnarToRow - InputAdapter - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] - CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #5 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] - CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometExchange [ws_order_number,ws_item_sk] #7 - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_order_number,wr_item_sk] #8 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] - CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometExchange [cs_order_number,cs_item_sk] #10 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_order_number,cr_item_sk] #11 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/explain.txt deleted file mode 100644 index 78ed64abe7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (32) -+- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (29) - +- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometNativeScan parquet spark_catalog.default.customer (25) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] - -(3) Filter [codegen id : 4] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 37] -Output [1]: [d_date_sk#10] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#11, s_number_employees#12, s_city#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_number_employees), GreaterThanOrEqual(s_number_employees,200), LessThanOrEqual(s_number_employees,295), IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [s_store_sk#11, s_number_employees#12, s_city#13] -Condition : (((isnotnull(s_number_employees#12) AND (s_number_employees#12 >= 200)) AND (s_number_employees#12 <= 295)) AND isnotnull(s_store_sk#11)) - -(9) CometProject -Input [3]: [s_store_sk#11, s_number_employees#12, s_city#13] -Arguments: [s_store_sk#11, s_city#13], [s_store_sk#11, s_city#13] - -(10) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#11, s_city#13] - -(11) BroadcastExchange -Input [2]: [s_store_sk#11, s_city#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#11, s_city#13] - -(14) CometNativeScan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,6),GreaterThan(hd_vehicle_count,2)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Condition : (((hd_dep_count#15 = 6) OR (hd_vehicle_count#16 > 2)) AND isnotnull(hd_demo_sk#14)) - -(16) CometProject -Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Arguments: [hd_demo_sk#14], [hd_demo_sk#14] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#14] - -(18) BroadcastExchange -Input [1]: [hd_demo_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#14] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13, hd_demo_sk#14] - -(21) HashAggregate [codegen id : 4] -Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13] -Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum#17, sum#18] -Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] - -(22) CometColumnarExchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 6] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] - -(24) HashAggregate [codegen id : 6] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13] -Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#6))#21, sum(UnscaledValue(ss_net_profit#7))#22] -Results [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#21,17,2) AS amt#23, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#22,17,2) AS profit#24] - -(25) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Condition : isnotnull(c_customer_sk#25) - -(27) CometProject -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_customer_sk#25, c_first_name#28, c_last_name#29], [c_customer_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#26, 20, true, false, true) AS c_first_name#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#27, 30, true, false, true) AS c_last_name#29] - -(28) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#25, c_first_name#28, c_last_name#29] - -(29) BroadcastExchange -Input [3]: [c_customer_sk#25, c_first_name#28, c_last_name#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#25] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [7]: [c_last_name#29, c_first_name#28, substr(s_city#13, 1, 30) AS substr(s_city, 1, 30)#30, ss_ticket_number#5, amt#23, profit#24, s_city#13] -Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, amt#23, profit#24, c_customer_sk#25, c_first_name#28, c_last_name#29] - -(32) TakeOrderedAndProject -Input [7]: [c_last_name#29, c_first_name#28, substr(s_city, 1, 30)#30, ss_ticket_number#5, amt#23, profit#24, s_city#13] -Arguments: 100, [c_last_name#29 ASC NULLS FIRST, c_first_name#28 ASC NULLS FIRST, substr(s_city#13, 1, 30) ASC NULLS FIRST, profit#24 ASC NULLS FIRST], [c_last_name#29, c_first_name#28, substr(s_city, 1, 30)#30, ss_ticket_number#5, amt#23, profit#24] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometNativeScan parquet spark_catalog.default.date_dim (33) - - -(33) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#31, d_dow#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [3]: [d_date_sk#10, d_year#31, d_dow#32] -Condition : (((isnotnull(d_dow#32) AND (d_dow#32 = 1)) AND d_year#31 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(35) CometProject -Input [3]: [d_date_sk#10, d_year#31, d_dow#32] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(37) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/simplified.txt deleted file mode 100644 index 1ee7a286a4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, 30),ss_ticket_number,amt] - WholeStageCodegen (6) - Project [c_last_name,c_first_name,s_city,ss_ticket_number,amt,profit] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),amt,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk,s_city] - CometFilter [s_store_sk,s_number_employees,s_city] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan 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/q79.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/explain.txt deleted file mode 100644 index 2d2c6dc938..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,212 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (31) -+- CometTakeOrderedAndProject (30) - +- CometProject (29) - +- CometBroadcastHashJoin (28) - :- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(5) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#13, s_number_employees#14, s_city#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_number_employees), GreaterThanOrEqual(s_number_employees,200), LessThanOrEqual(s_number_employees,295), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [s_store_sk#13, s_number_employees#14, s_city#15] -Condition : (((isnotnull(s_number_employees#14) AND (s_number_employees#14 >= 200)) AND (s_number_employees#14 <= 295)) AND isnotnull(s_store_sk#13)) - -(11) CometProject -Input [3]: [s_store_sk#13, s_number_employees#14, s_city#15] -Arguments: [s_store_sk#13, s_city#15], [s_store_sk#13, s_city#15] - -(12) CometBroadcastExchange -Input [2]: [s_store_sk#13, s_city#15] -Arguments: [s_store_sk#13, s_city#15] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [2]: [s_store_sk#13, s_city#15] -Arguments: [ss_store_sk#4], [s_store_sk#13], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#13, s_city#15] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,6),GreaterThan(hd_vehicle_count,2)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Condition : (((hd_dep_count#17 = 6) OR (hd_vehicle_count#18 > 2)) AND isnotnull(hd_demo_sk#16)) - -(17) CometProject -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Arguments: [hd_demo_sk#16], [hd_demo_sk#16] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#16] -Arguments: [hd_demo_sk#16] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] -Right output [1]: [hd_demo_sk#16] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15, hd_demo_sk#16] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] - -(21) CometHashAggregate -Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15] -Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] - -(22) CometExchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#19, sum#20] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#19, sum#20] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15] -Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(25) CometFilter -Input [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] -Condition : isnotnull(c_customer_sk#21) - -(26) CometProject -Input [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] -Arguments: [c_customer_sk#21, c_first_name#24, c_last_name#25], [c_customer_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#22, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#23, 30, true, false, true) AS c_last_name#25] - -(27) CometBroadcastExchange -Input [3]: [c_customer_sk#21, c_first_name#24, c_last_name#25] -Arguments: [c_customer_sk#21, c_first_name#24, c_last_name#25] - -(28) CometBroadcastHashJoin -Left output [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#26, profit#27] -Right output [3]: [c_customer_sk#21, c_first_name#24, c_last_name#25] -Arguments: [ss_customer_sk#1], [c_customer_sk#21], Inner, BuildRight - -(29) CometProject -Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#26, profit#27, c_customer_sk#21, c_first_name#24, c_last_name#25] -Arguments: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15], [c_last_name#25, c_first_name#24, substr(s_city#15, 1, 30) AS substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15] - -(30) CometTakeOrderedAndProject -Input [7]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#25 ASC NULLS FIRST,c_first_name#24 ASC NULLS FIRST,substr(s_city#15, 1, 30) ASC NULLS FIRST,profit#27 ASC NULLS FIRST], output=[c_last_name#25,c_first_name#24,substr(s_city, 1, 30)#28,ss_ticket_number#5,amt#26,profit#27]), [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27], 100, 0, [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, substr(s_city#15, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27] - -(31) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(34) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(36) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/simplified.txt deleted file mode 100644 index 5b252a906b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_city] [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit] - CometProject [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit,s_city] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,s_city,amt,profit,c_customer_sk,c_first_name,c_last_name] - CometHashAggregate [ss_addr_sk,sum,sum] [ss_ticket_number,ss_customer_sk,s_city,amt,profit,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] - CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 - CometHashAggregate [ss_coupon_amt,ss_net_profit] [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] - CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - 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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [s_store_sk,s_city] #4 - CometProject [s_store_sk,s_city] - CometFilter [s_store_sk,s_number_employees,s_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #6 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] 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/q79/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/explain.txt deleted file mode 100644 index 2d2c6dc938..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/explain.txt +++ /dev/null @@ -1,212 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (31) -+- CometTakeOrderedAndProject (30) - +- CometProject (29) - +- CometBroadcastHashJoin (28) - :- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(5) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#13, s_number_employees#14, s_city#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_number_employees), GreaterThanOrEqual(s_number_employees,200), LessThanOrEqual(s_number_employees,295), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [s_store_sk#13, s_number_employees#14, s_city#15] -Condition : (((isnotnull(s_number_employees#14) AND (s_number_employees#14 >= 200)) AND (s_number_employees#14 <= 295)) AND isnotnull(s_store_sk#13)) - -(11) CometProject -Input [3]: [s_store_sk#13, s_number_employees#14, s_city#15] -Arguments: [s_store_sk#13, s_city#15], [s_store_sk#13, s_city#15] - -(12) CometBroadcastExchange -Input [2]: [s_store_sk#13, s_city#15] -Arguments: [s_store_sk#13, s_city#15] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [2]: [s_store_sk#13, s_city#15] -Arguments: [ss_store_sk#4], [s_store_sk#13], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#13, s_city#15] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,6),GreaterThan(hd_vehicle_count,2)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Condition : (((hd_dep_count#17 = 6) OR (hd_vehicle_count#18 > 2)) AND isnotnull(hd_demo_sk#16)) - -(17) CometProject -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Arguments: [hd_demo_sk#16], [hd_demo_sk#16] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#16] -Arguments: [hd_demo_sk#16] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] -Right output [1]: [hd_demo_sk#16] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15, hd_demo_sk#16] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] - -(21) CometHashAggregate -Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15] -Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] - -(22) CometExchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#19, sum#20] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#19, sum#20] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15] -Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(25) CometFilter -Input [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] -Condition : isnotnull(c_customer_sk#21) - -(26) CometProject -Input [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] -Arguments: [c_customer_sk#21, c_first_name#24, c_last_name#25], [c_customer_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#22, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#23, 30, true, false, true) AS c_last_name#25] - -(27) CometBroadcastExchange -Input [3]: [c_customer_sk#21, c_first_name#24, c_last_name#25] -Arguments: [c_customer_sk#21, c_first_name#24, c_last_name#25] - -(28) CometBroadcastHashJoin -Left output [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#26, profit#27] -Right output [3]: [c_customer_sk#21, c_first_name#24, c_last_name#25] -Arguments: [ss_customer_sk#1], [c_customer_sk#21], Inner, BuildRight - -(29) CometProject -Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#26, profit#27, c_customer_sk#21, c_first_name#24, c_last_name#25] -Arguments: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15], [c_last_name#25, c_first_name#24, substr(s_city#15, 1, 30) AS substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15] - -(30) CometTakeOrderedAndProject -Input [7]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#25 ASC NULLS FIRST,c_first_name#24 ASC NULLS FIRST,substr(s_city#15, 1, 30) ASC NULLS FIRST,profit#27 ASC NULLS FIRST], output=[c_last_name#25,c_first_name#24,substr(s_city, 1, 30)#28,ss_ticket_number#5,amt#26,profit#27]), [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27], 100, 0, [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, substr(s_city#15, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27] - -(31) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(34) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(36) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/extended.txt deleted file mode 100644 index 012403275a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/extended.txt +++ /dev/null @@ -1,39 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 5b252a906b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_city] [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit] - CometProject [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit,s_city] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,s_city,amt,profit,c_customer_sk,c_first_name,c_last_name] - CometHashAggregate [ss_addr_sk,sum,sum] [ss_ticket_number,ss_customer_sk,s_city,amt,profit,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] - CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 - CometHashAggregate [ss_coupon_amt,ss_net_profit] [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] - CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - 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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [s_store_sk,s_city] #4 - CometProject [s_store_sk,s_city] - CometFilter [s_store_sk,s_number_employees,s_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #6 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/explain.txt deleted file mode 100644 index 754a8871d1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/explain.txt +++ /dev/null @@ -1,284 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (44) -+- * HashAggregate (43) - +- * CometColumnarToRow (42) - +- CometColumnarExchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) - : +- CometProject (9) - : +- CometFilter (8) - : +- CometNativeScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (37) - +- * CometColumnarToRow (36) - +- CometHashAggregate (35) - +- CometExchange (34) - +- CometHashAggregate (33) - +- CometBroadcastHashJoin (32) - :- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.customer_address (14) - +- CometBroadcastExchange (31) - +- CometProject (30) - +- CometFilter (29) - +- CometHashAggregate (28) - +- CometExchange (27) - +- CometHashAggregate (26) - +- CometProject (25) - +- CometBroadcastHashJoin (24) - :- CometProject (19) - : +- CometFilter (18) - : +- CometNativeScan parquet spark_catalog.default.customer_address (17) - +- CometBroadcastExchange (23) - +- CometProject (22) - +- CometFilter (21) - +- CometNativeScan parquet spark_catalog.default.customer (20) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] - -(3) Filter [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(4) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#5] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [2]: [ss_store_sk#1, ss_net_profit#2] -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#6, s_store_name#7, s_zip#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [s_store_sk#6, s_store_name#7, s_zip#8] -Condition : (isnotnull(s_store_sk#6) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#8, 10, true, false, true))) - -(9) CometProject -Input [3]: [s_store_sk#6, s_store_name#7, s_zip#8] -Arguments: [s_store_sk#6, s_store_name#7, s_zip#9], [s_store_sk#6, s_store_name#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#8, 10, true, false, true) AS s_zip#9] - -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [s_store_sk#6, s_store_name#7, s_zip#9] - -(11) BroadcastExchange -Input [3]: [s_store_sk#6, s_store_name#7, s_zip#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#6] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_store_name#7, s_zip#9] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_store_name#7, s_zip#9] - -(14) CometNativeScan parquet spark_catalog.default.customer_address -Output [1]: [ca_zip#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -ReadSchema: struct - -(15) CometFilter -Input [1]: [ca_zip#10] -Condition : (substr(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#10, 10, true, false, true), 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#10, 10, true, false, true), 1, 5))) - -(16) CometProject -Input [1]: [ca_zip#10] -Arguments: [ca_zip#11], [substr(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#10, 10, true, false, true), 1, 5) AS ca_zip#11] - -(17) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#12, ca_zip#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [ca_address_sk#12, ca_zip#13] -Condition : isnotnull(ca_address_sk#12) - -(19) CometProject -Input [2]: [ca_address_sk#12, ca_zip#13] -Arguments: [ca_address_sk#12, ca_zip#14], [ca_address_sk#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#13, 10, true, false, true) AS ca_zip#14] - -(20) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#16, 1, true, false, true) = Y) AND isnotnull(c_current_addr_sk#15)) - -(22) CometProject -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] -Arguments: [c_current_addr_sk#15], [c_current_addr_sk#15] - -(23) CometBroadcastExchange -Input [1]: [c_current_addr_sk#15] -Arguments: [c_current_addr_sk#15] - -(24) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#12, ca_zip#14] -Right output [1]: [c_current_addr_sk#15] -Arguments: [ca_address_sk#12], [c_current_addr_sk#15], Inner, BuildRight - -(25) CometProject -Input [3]: [ca_address_sk#12, ca_zip#14, c_current_addr_sk#15] -Arguments: [ca_zip#14], [ca_zip#14] - -(26) CometHashAggregate -Input [1]: [ca_zip#14] -Keys [1]: [ca_zip#14] -Functions [1]: [partial_count(1)] - -(27) CometExchange -Input [2]: [ca_zip#14, count#17] -Arguments: hashpartitioning(ca_zip#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [2]: [ca_zip#14, count#17] -Keys [1]: [ca_zip#14] -Functions [1]: [count(1)] - -(29) CometFilter -Input [2]: [ca_zip#18, cnt#19] -Condition : (cnt#19 > 10) - -(30) CometProject -Input [2]: [ca_zip#18, cnt#19] -Arguments: [ca_zip#18], [ca_zip#18] - -(31) CometBroadcastExchange -Input [1]: [ca_zip#18] -Arguments: [ca_zip#18] - -(32) CometBroadcastHashJoin -Left output [1]: [ca_zip#11] -Right output [1]: [ca_zip#18] -Arguments: [coalesce(ca_zip#11, ), isnull(ca_zip#11)], [coalesce(ca_zip#18, ), isnull(ca_zip#18)], LeftSemi, BuildRight - -(33) CometHashAggregate -Input [1]: [ca_zip#11] -Keys [1]: [ca_zip#11] -Functions: [] - -(34) CometExchange -Input [1]: [ca_zip#11] -Arguments: hashpartitioning(ca_zip#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(35) CometHashAggregate -Input [1]: [ca_zip#11] -Keys [1]: [ca_zip#11] -Functions: [] - -(36) CometColumnarToRow [codegen id : 3] -Input [1]: [ca_zip#11] - -(37) BroadcastExchange -Input [1]: [ca_zip#11] -Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [substr(s_zip#9, 1, 2)] -Right keys [1]: [substr(ca_zip#11, 1, 2)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 4] -Output [2]: [ss_net_profit#2, s_store_name#7] -Input [4]: [ss_net_profit#2, s_store_name#7, s_zip#9, ca_zip#11] - -(40) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#2, s_store_name#7] -Keys [1]: [s_store_name#7] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#20] -Results [2]: [s_store_name#7, sum#21] - -(41) CometColumnarExchange -Input [2]: [s_store_name#7, sum#21] -Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(42) CometColumnarToRow [codegen id : 5] -Input [2]: [s_store_name#7, sum#21] - -(43) HashAggregate [codegen id : 5] -Input [2]: [s_store_name#7, sum#21] -Keys [1]: [s_store_name#7] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22] -Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS sum(ss_net_profit)#23] - -(44) TakeOrderedAndProject -Input [2]: [s_store_name#7, sum(ss_net_profit)#23] -Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometNativeScan parquet spark_catalog.default.date_dim (45) - - -(45) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#24, d_qoy#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [3]: [d_date_sk#5, d_year#24, d_qoy#25] -Condition : ((((isnotnull(d_qoy#25) AND isnotnull(d_year#24)) AND (d_qoy#25 = 2)) AND (d_year#24 = 1998)) AND isnotnull(d_date_sk#5)) - -(47) CometProject -Input [3]: [d_date_sk#5, d_year#24, d_qoy#25] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(49) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/simplified.txt deleted file mode 100644 index 4df6d8f659..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/simplified.txt +++ /dev/null @@ -1,63 +0,0 @@ -TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - WholeStageCodegen (5) - HashAggregate [s_store_name,sum] [sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit),sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_name] #1 - WholeStageCodegen (4) - HashAggregate [s_store_name,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_store_name] - BroadcastHashJoin [s_zip,ca_zip] - Project [ss_net_profit,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [ca_zip] - CometExchange [ca_zip] #5 - CometHashAggregate [ca_zip] - CometBroadcastHashJoin [ca_zip,ca_zip] - CometProject [ca_zip] [ca_zip] - CometFilter [ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_zip] - CometBroadcastExchange [ca_zip] #6 - CometProject [ca_zip] - CometFilter [ca_zip,cnt] - CometHashAggregate [count] [ca_zip,cnt,ca_zip,count(1)] - CometExchange [ca_zip] #7 - CometHashAggregate [ca_zip,count] - CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [c_current_addr_sk] #8 - CometProject [c_current_addr_sk] - CometFilter [c_current_addr_sk,c_preferred_cust_flag] - CometNativeScan 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/q8.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/explain.txt deleted file mode 100644 index 075f17ea3d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,284 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - +- CometBroadcastExchange (37) - +- CometHashAggregate (36) - +- CometExchange (35) - +- CometHashAggregate (34) - +- CometBroadcastHashJoin (33) - :- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (15) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (18) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) AND (d_year#6 = 1998)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#8, s_store_name#9, s_zip#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [s_store_sk#8, s_store_name#9, s_zip#10] -Condition : (isnotnull(s_store_sk#8) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#10, 10, true, false, true))) - -(11) CometProject -Input [3]: [s_store_sk#8, s_store_name#9, s_zip#10] -Arguments: [s_store_sk#8, s_store_name#9, s_zip#11], [s_store_sk#8, s_store_name#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#10, 10, true, false, true) AS s_zip#11] - -(12) CometBroadcastExchange -Input [3]: [s_store_sk#8, s_store_name#9, s_zip#11] -Arguments: [s_store_sk#8, s_store_name#9, s_zip#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_store_sk#1, ss_net_profit#2] -Right output [3]: [s_store_sk#8, s_store_name#9, s_zip#11] -Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_store_name#9, s_zip#11] -Arguments: [ss_net_profit#2, s_store_name#9, s_zip#11], [ss_net_profit#2, s_store_name#9, s_zip#11] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [1]: [ca_zip#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -ReadSchema: struct - -(16) CometFilter -Input [1]: [ca_zip#12] -Condition : (substr(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#12, 10, true, false, true), 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#12, 10, true, false, true), 1, 5))) - -(17) CometProject -Input [1]: [ca_zip#12] -Arguments: [ca_zip#13], [substr(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#12, 10, true, false, true), 1, 5) AS ca_zip#13] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#14, ca_zip#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [ca_address_sk#14, ca_zip#15] -Condition : isnotnull(ca_address_sk#14) - -(20) CometProject -Input [2]: [ca_address_sk#14, ca_zip#15] -Arguments: [ca_address_sk#14, ca_zip#16], [ca_address_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#15, 10, true, false, true) AS ca_zip#16] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#18, 1, true, false, true) = Y) AND isnotnull(c_current_addr_sk#17)) - -(23) CometProject -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -Arguments: [c_current_addr_sk#17], [c_current_addr_sk#17] - -(24) CometBroadcastExchange -Input [1]: [c_current_addr_sk#17] -Arguments: [c_current_addr_sk#17] - -(25) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#14, ca_zip#16] -Right output [1]: [c_current_addr_sk#17] -Arguments: [ca_address_sk#14], [c_current_addr_sk#17], Inner, BuildRight - -(26) CometProject -Input [3]: [ca_address_sk#14, ca_zip#16, c_current_addr_sk#17] -Arguments: [ca_zip#16], [ca_zip#16] - -(27) CometHashAggregate -Input [1]: [ca_zip#16] -Keys [1]: [ca_zip#16] -Functions [1]: [partial_count(1)] - -(28) CometExchange -Input [2]: [ca_zip#16, count#19] -Arguments: hashpartitioning(ca_zip#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [2]: [ca_zip#16, count#19] -Keys [1]: [ca_zip#16] -Functions [1]: [count(1)] - -(30) CometFilter -Input [2]: [ca_zip#20, cnt#21] -Condition : (cnt#21 > 10) - -(31) CometProject -Input [2]: [ca_zip#20, cnt#21] -Arguments: [ca_zip#20], [ca_zip#20] - -(32) CometBroadcastExchange -Input [1]: [ca_zip#20] -Arguments: [ca_zip#20] - -(33) CometBroadcastHashJoin -Left output [1]: [ca_zip#13] -Right output [1]: [ca_zip#20] -Arguments: [coalesce(ca_zip#13, ), isnull(ca_zip#13)], [coalesce(ca_zip#20, ), isnull(ca_zip#20)], LeftSemi, BuildRight - -(34) CometHashAggregate -Input [1]: [ca_zip#13] -Keys [1]: [ca_zip#13] -Functions: [] - -(35) CometExchange -Input [1]: [ca_zip#13] -Arguments: hashpartitioning(ca_zip#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(36) CometHashAggregate -Input [1]: [ca_zip#13] -Keys [1]: [ca_zip#13] -Functions: [] - -(37) CometBroadcastExchange -Input [1]: [ca_zip#13] -Arguments: [ca_zip#13] - -(38) CometBroadcastHashJoin -Left output [3]: [ss_net_profit#2, s_store_name#9, s_zip#11] -Right output [1]: [ca_zip#13] -Arguments: [substr(s_zip#11, 1, 2)], [substr(ca_zip#13, 1, 2)], Inner, BuildRight - -(39) CometProject -Input [4]: [ss_net_profit#2, s_store_name#9, s_zip#11, ca_zip#13] -Arguments: [ss_net_profit#2, s_store_name#9], [ss_net_profit#2, s_store_name#9] - -(40) CometHashAggregate -Input [2]: [ss_net_profit#2, s_store_name#9] -Keys [1]: [s_store_name#9] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] - -(41) CometExchange -Input [2]: [s_store_name#9, sum#22] -Arguments: hashpartitioning(s_store_name#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(42) CometHashAggregate -Input [2]: [s_store_name#9, sum#22] -Keys [1]: [s_store_name#9] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] - -(43) CometTakeOrderedAndProject -Input [2]: [s_store_name#9, sum(ss_net_profit)#23] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#9 ASC NULLS FIRST], output=[s_store_name#9,sum(ss_net_profit)#23]), [s_store_name#9, sum(ss_net_profit)#23], 100, 0, [s_store_name#9 ASC NULLS FIRST], [s_store_name#9, sum(ss_net_profit)#23] - -(44) CometColumnarToRow [codegen id : 1] -Input [2]: [s_store_name#9, sum(ss_net_profit)#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) AND (d_year#6 = 1998)) AND isnotnull(d_date_sk#5)) - -(47) CometProject -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(49) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/simplified.txt deleted file mode 100644 index d1542ab607..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - CometHashAggregate [sum] [s_store_name,sum(ss_net_profit),sum(UnscaledValue(ss_net_profit))] - CometExchange [s_store_name] #1 - CometHashAggregate [ss_net_profit] [s_store_name,sum] - CometProject [ss_net_profit,s_store_name] - CometBroadcastHashJoin [ss_net_profit,s_store_name,s_zip,ca_zip] - CometProject [ss_net_profit,s_store_name,s_zip] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] - CometProject [ss_store_sk,ss_net_profit] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [ca_zip] #5 - CometHashAggregate [ca_zip] - CometExchange [ca_zip] #6 - CometHashAggregate [ca_zip] - CometBroadcastHashJoin [ca_zip,ca_zip] - CometProject [ca_zip] [ca_zip] - CometFilter [ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_zip] - CometBroadcastExchange [ca_zip] #7 - CometProject [ca_zip] - CometFilter [ca_zip,cnt] - CometHashAggregate [count] [ca_zip,cnt,ca_zip,count(1)] - CometExchange [ca_zip] #8 - CometHashAggregate [ca_zip,count] - CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [c_current_addr_sk] #9 - CometProject [c_current_addr_sk] - CometFilter [c_current_addr_sk,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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/q8/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/explain.txt deleted file mode 100644 index 075f17ea3d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/explain.txt +++ /dev/null @@ -1,284 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - +- CometBroadcastExchange (37) - +- CometHashAggregate (36) - +- CometExchange (35) - +- CometHashAggregate (34) - +- CometBroadcastHashJoin (33) - :- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (15) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (18) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) AND (d_year#6 = 1998)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#8, s_store_name#9, s_zip#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [s_store_sk#8, s_store_name#9, s_zip#10] -Condition : (isnotnull(s_store_sk#8) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#10, 10, true, false, true))) - -(11) CometProject -Input [3]: [s_store_sk#8, s_store_name#9, s_zip#10] -Arguments: [s_store_sk#8, s_store_name#9, s_zip#11], [s_store_sk#8, s_store_name#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#10, 10, true, false, true) AS s_zip#11] - -(12) CometBroadcastExchange -Input [3]: [s_store_sk#8, s_store_name#9, s_zip#11] -Arguments: [s_store_sk#8, s_store_name#9, s_zip#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_store_sk#1, ss_net_profit#2] -Right output [3]: [s_store_sk#8, s_store_name#9, s_zip#11] -Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_store_name#9, s_zip#11] -Arguments: [ss_net_profit#2, s_store_name#9, s_zip#11], [ss_net_profit#2, s_store_name#9, s_zip#11] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [1]: [ca_zip#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -ReadSchema: struct - -(16) CometFilter -Input [1]: [ca_zip#12] -Condition : (substr(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#12, 10, true, false, true), 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#12, 10, true, false, true), 1, 5))) - -(17) CometProject -Input [1]: [ca_zip#12] -Arguments: [ca_zip#13], [substr(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#12, 10, true, false, true), 1, 5) AS ca_zip#13] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#14, ca_zip#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [ca_address_sk#14, ca_zip#15] -Condition : isnotnull(ca_address_sk#14) - -(20) CometProject -Input [2]: [ca_address_sk#14, ca_zip#15] -Arguments: [ca_address_sk#14, ca_zip#16], [ca_address_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#15, 10, true, false, true) AS ca_zip#16] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#18, 1, true, false, true) = Y) AND isnotnull(c_current_addr_sk#17)) - -(23) CometProject -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -Arguments: [c_current_addr_sk#17], [c_current_addr_sk#17] - -(24) CometBroadcastExchange -Input [1]: [c_current_addr_sk#17] -Arguments: [c_current_addr_sk#17] - -(25) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#14, ca_zip#16] -Right output [1]: [c_current_addr_sk#17] -Arguments: [ca_address_sk#14], [c_current_addr_sk#17], Inner, BuildRight - -(26) CometProject -Input [3]: [ca_address_sk#14, ca_zip#16, c_current_addr_sk#17] -Arguments: [ca_zip#16], [ca_zip#16] - -(27) CometHashAggregate -Input [1]: [ca_zip#16] -Keys [1]: [ca_zip#16] -Functions [1]: [partial_count(1)] - -(28) CometExchange -Input [2]: [ca_zip#16, count#19] -Arguments: hashpartitioning(ca_zip#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [2]: [ca_zip#16, count#19] -Keys [1]: [ca_zip#16] -Functions [1]: [count(1)] - -(30) CometFilter -Input [2]: [ca_zip#20, cnt#21] -Condition : (cnt#21 > 10) - -(31) CometProject -Input [2]: [ca_zip#20, cnt#21] -Arguments: [ca_zip#20], [ca_zip#20] - -(32) CometBroadcastExchange -Input [1]: [ca_zip#20] -Arguments: [ca_zip#20] - -(33) CometBroadcastHashJoin -Left output [1]: [ca_zip#13] -Right output [1]: [ca_zip#20] -Arguments: [coalesce(ca_zip#13, ), isnull(ca_zip#13)], [coalesce(ca_zip#20, ), isnull(ca_zip#20)], LeftSemi, BuildRight - -(34) CometHashAggregate -Input [1]: [ca_zip#13] -Keys [1]: [ca_zip#13] -Functions: [] - -(35) CometExchange -Input [1]: [ca_zip#13] -Arguments: hashpartitioning(ca_zip#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(36) CometHashAggregate -Input [1]: [ca_zip#13] -Keys [1]: [ca_zip#13] -Functions: [] - -(37) CometBroadcastExchange -Input [1]: [ca_zip#13] -Arguments: [ca_zip#13] - -(38) CometBroadcastHashJoin -Left output [3]: [ss_net_profit#2, s_store_name#9, s_zip#11] -Right output [1]: [ca_zip#13] -Arguments: [substr(s_zip#11, 1, 2)], [substr(ca_zip#13, 1, 2)], Inner, BuildRight - -(39) CometProject -Input [4]: [ss_net_profit#2, s_store_name#9, s_zip#11, ca_zip#13] -Arguments: [ss_net_profit#2, s_store_name#9], [ss_net_profit#2, s_store_name#9] - -(40) CometHashAggregate -Input [2]: [ss_net_profit#2, s_store_name#9] -Keys [1]: [s_store_name#9] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] - -(41) CometExchange -Input [2]: [s_store_name#9, sum#22] -Arguments: hashpartitioning(s_store_name#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(42) CometHashAggregate -Input [2]: [s_store_name#9, sum#22] -Keys [1]: [s_store_name#9] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] - -(43) CometTakeOrderedAndProject -Input [2]: [s_store_name#9, sum(ss_net_profit)#23] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#9 ASC NULLS FIRST], output=[s_store_name#9,sum(ss_net_profit)#23]), [s_store_name#9, sum(ss_net_profit)#23], 100, 0, [s_store_name#9 ASC NULLS FIRST], [s_store_name#9, sum(ss_net_profit)#23] - -(44) CometColumnarToRow [codegen id : 1] -Input [2]: [s_store_name#9, sum(ss_net_profit)#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) AND (d_year#6 = 1998)) AND isnotnull(d_date_sk#5)) - -(47) CometProject -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(49) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/extended.txt deleted file mode 100644 index 8e04bbeba7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/extended.txt +++ /dev/null @@ -1,52 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d1542ab607..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - CometHashAggregate [sum] [s_store_name,sum(ss_net_profit),sum(UnscaledValue(ss_net_profit))] - CometExchange [s_store_name] #1 - CometHashAggregate [ss_net_profit] [s_store_name,sum] - CometProject [ss_net_profit,s_store_name] - CometBroadcastHashJoin [ss_net_profit,s_store_name,s_zip,ca_zip] - CometProject [ss_net_profit,s_store_name,s_zip] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] - CometProject [ss_store_sk,ss_net_profit] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [ca_zip] #5 - CometHashAggregate [ca_zip] - CometExchange [ca_zip] #6 - CometHashAggregate [ca_zip] - CometBroadcastHashJoin [ca_zip,ca_zip] - CometProject [ca_zip] [ca_zip] - CometFilter [ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_zip] - CometBroadcastExchange [ca_zip] #7 - CometProject [ca_zip] - CometFilter [ca_zip,cnt] - CometHashAggregate [count] [ca_zip,cnt,ca_zip,count(1)] - CometExchange [ca_zip] #8 - CometHashAggregate [ca_zip,count] - CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [c_current_addr_sk] #9 - CometProject [c_current_addr_sk] - CometFilter [c_current_addr_sk,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/explain.txt deleted file mode 100644 index f1e1a2ace1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/explain.txt +++ /dev/null @@ -1,623 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (106) -+- CometTakeOrderedAndProject (105) - +- CometHashAggregate (104) - +- CometExchange (103) - +- CometHashAggregate (102) - +- CometExpand (101) - +- CometUnion (100) - :- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometProject (18) - : : : : +- CometBroadcastHashJoin (17) - : : : : :- CometProject (12) - : : : : : +- CometSortMergeJoin (11) - : : : : : :- CometSort (5) - : : : : : : +- CometColumnarExchange (4) - : : : : : : +- * Filter (3) - : : : : : : +- * ColumnarToRow (2) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : +- CometSort (10) - : : : : : +- CometExchange (9) - : : : : : +- CometProject (8) - : : : : : +- CometFilter (7) - : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (6) - : : : : +- CometBroadcastExchange (16) - : : : : +- CometProject (15) - : : : : +- CometFilter (14) - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (13) - : : : +- CometBroadcastExchange (22) - : : : +- CometProject (21) - : : : +- CometFilter (20) - : : : +- CometNativeScan parquet spark_catalog.default.store (19) - : : +- CometBroadcastExchange (28) - : : +- CometProject (27) - : : +- CometFilter (26) - : : +- CometNativeScan parquet spark_catalog.default.item (25) - : +- CometBroadcastExchange (34) - : +- CometProject (33) - : +- CometFilter (32) - : +- CometNativeScan parquet spark_catalog.default.promotion (31) - :- CometHashAggregate (69) - : +- CometExchange (68) - : +- CometHashAggregate (67) - : +- CometProject (66) - : +- CometBroadcastHashJoin (65) - : :- CometProject (63) - : : +- CometBroadcastHashJoin (62) - : : :- CometProject (60) - : : : +- CometBroadcastHashJoin (59) - : : : :- CometProject (54) - : : : : +- CometBroadcastHashJoin (53) - : : : : :- CometProject (51) - : : : : : +- CometSortMergeJoin (50) - : : : : : :- CometSort (44) - : : : : : : +- CometColumnarExchange (43) - : : : : : : +- * Filter (42) - : : : : : : +- * ColumnarToRow (41) - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (40) - : : : : : +- CometSort (49) - : : : : : +- CometExchange (48) - : : : : : +- CometProject (47) - : : : : : +- CometFilter (46) - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (45) - : : : : +- ReusedExchange (52) - : : : +- CometBroadcastExchange (58) - : : : +- CometProject (57) - : : : +- CometFilter (56) - : : : +- CometNativeScan parquet spark_catalog.default.catalog_page (55) - : : +- ReusedExchange (61) - : +- ReusedExchange (64) - +- CometHashAggregate (99) - +- CometExchange (98) - +- CometHashAggregate (97) - +- CometProject (96) - +- CometBroadcastHashJoin (95) - :- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (84) - : : : +- CometBroadcastHashJoin (83) - : : : :- CometProject (81) - : : : : +- CometSortMergeJoin (80) - : : : : :- CometSort (74) - : : : : : +- CometColumnarExchange (73) - : : : : : +- * Filter (72) - : : : : : +- * ColumnarToRow (71) - : : : : : +- Scan parquet spark_catalog.default.web_sales (70) - : : : : +- CometSort (79) - : : : : +- CometExchange (78) - : : : : +- CometProject (77) - : : : : +- CometFilter (76) - : : : : +- CometNativeScan parquet spark_catalog.default.web_returns (75) - : : : +- ReusedExchange (82) - : : +- CometBroadcastExchange (88) - : : +- CometProject (87) - : : +- CometFilter (86) - : : +- CometNativeScan parquet spark_catalog.default.web_site (85) - : +- ReusedExchange (91) - +- ReusedExchange (94) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] - -(3) Filter [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(4) CometColumnarExchange -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(7) CometFilter -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) - -(8) CometProject -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(9) CometExchange -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter - -(12) CometProject -Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] - -(13) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) - -(15) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] - -(19) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_store_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [s_store_sk#16, s_store_id#17] -Condition : isnotnull(s_store_sk#16) - -(21) CometProject -Input [2]: [s_store_sk#16, s_store_id#17] -Arguments: [s_store_sk#16, s_store_id#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#17, 16, true, false, true) AS s_store_id#18] - -(22) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_store_id#18] -Arguments: [s_store_sk#16, s_store_id#18] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -Right output [2]: [s_store_sk#16, s_store_id#18] -Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight - -(24) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#18] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(25) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_current_price#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [i_item_sk#19, i_current_price#20] -Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) - -(27) CometProject -Input [2]: [i_item_sk#19, i_current_price#20] -Arguments: [i_item_sk#19], [i_item_sk#19] - -(28) CometBroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: [i_item_sk#19] - -(29) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [i_item_sk#19] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, i_item_sk#19] -Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(31) CometNativeScan parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#21, p_channel_tv#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#22, 1, true, false, true) = N) AND isnotnull(p_promo_sk#21)) - -(33) CometProject -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Arguments: [p_promo_sk#21], [p_promo_sk#21] - -(34) CometBroadcastExchange -Input [1]: [p_promo_sk#21] -Arguments: [p_promo_sk#21] - -(35) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [p_promo_sk#21] -Arguments: [ss_promo_sk#3], [p_promo_sk#21], Inner, BuildRight - -(36) CometProject -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, p_promo_sk#21] -Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(37) CometHashAggregate -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Keys [1]: [s_store_id#18] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(38) CometExchange -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(39) CometHashAggregate -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [s_store_id#18] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(40) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(41) ColumnarToRow [codegen id : 2] -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] - -(42) Filter [codegen id : 2] -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) - -(43) CometColumnarExchange -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(44) CometSort -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] - -(45) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38, cr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(46) CometFilter -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38, cr_returned_date_sk#39] -Condition : (isnotnull(cr_item_sk#35) AND isnotnull(cr_order_number#36)) - -(47) CometProject -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38, cr_returned_date_sk#39] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38], [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] - -(48) CometExchange -Input [4]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: hashpartitioning(cr_item_sk#35, cr_order_number#36, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(49) CometSort -Input [4]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38], [cr_item_sk#35 ASC NULLS FIRST, cr_order_number#36 ASC NULLS FIRST] - -(50) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#35, cr_order_number#36], LeftOuter - -(51) CometProject -Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38] - -(52) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#40] - -(53) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38] -Right output [1]: [d_date_sk#40] -Arguments: [cs_sold_date_sk#34], [d_date_sk#40], Inner, BuildRight - -(54) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38, d_date_sk#40] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38] - -(55) CometNativeScan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#42] -Condition : isnotnull(cp_catalog_page_sk#41) - -(57) CometProject -Input [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#42] -Arguments: [cp_catalog_page_sk#41, cp_catalog_page_id#43], [cp_catalog_page_sk#41, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#42, 16, true, false, true) AS cp_catalog_page_id#43] - -(58) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#43] -Arguments: [cp_catalog_page_sk#41, cp_catalog_page_id#43] - -(59) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38] -Right output [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#43] -Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#41], Inner, BuildRight - -(60) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_sk#41, cp_catalog_page_id#43] -Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] - -(61) ReusedExchange [Reuses operator id: 28] -Output [1]: [i_item_sk#44] - -(62) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] -Right output [1]: [i_item_sk#44] -Arguments: [cs_item_sk#29], [i_item_sk#44], Inner, BuildRight - -(63) CometProject -Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43, i_item_sk#44] -Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] - -(64) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#45] - -(65) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] -Right output [1]: [p_promo_sk#45] -Arguments: [cs_promo_sk#30], [p_promo_sk#45], Inner, BuildRight - -(66) CometProject -Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43, p_promo_sk#45] -Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] - -(67) CometHashAggregate -Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] -Keys [1]: [cp_catalog_page_id#43] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#37 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#38 as decimal(12,2)), 0.00)))] - -(68) CometExchange -Input [6]: [cp_catalog_page_id#43, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] -Arguments: hashpartitioning(cp_catalog_page_id#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(69) CometHashAggregate -Input [6]: [cp_catalog_page_id#43, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] -Keys [1]: [cp_catalog_page_id#43] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#37 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#38 as decimal(12,2)), 0.00)))] - -(70) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#57), dynamicpruningexpression(ws_sold_date_sk#57 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] -ReadSchema: struct - -(71) ColumnarToRow [codegen id : 3] -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] - -(72) Filter [codegen id : 3] -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Condition : ((isnotnull(ws_web_site_sk#52) AND isnotnull(ws_item_sk#51)) AND isnotnull(ws_promo_sk#53)) - -(73) CometColumnarExchange -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Arguments: hashpartitioning(ws_item_sk#51, ws_order_number#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(74) CometSort -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Arguments: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57], [ws_item_sk#51 ASC NULLS FIRST, ws_order_number#54 ASC NULLS FIRST] - -(75) CometNativeScan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] -ReadSchema: struct - -(76) CometFilter -Input [5]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Condition : (isnotnull(wr_item_sk#58) AND isnotnull(wr_order_number#59)) - -(77) CometProject -Input [5]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Arguments: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61], [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] - -(78) CometExchange -Input [4]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: hashpartitioning(wr_item_sk#58, wr_order_number#59, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(79) CometSort -Input [4]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61], [wr_item_sk#58 ASC NULLS FIRST, wr_order_number#59 ASC NULLS FIRST] - -(80) CometSortMergeJoin -Left output [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Right output [4]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: [ws_item_sk#51, ws_order_number#54], [wr_item_sk#58, wr_order_number#59], LeftOuter - -(81) CometProject -Input [11]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61], [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61] - -(82) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#63] - -(83) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61] -Right output [1]: [d_date_sk#63] -Arguments: [ws_sold_date_sk#57], [d_date_sk#63], Inner, BuildRight - -(84) CometProject -Input [9]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61, d_date_sk#63] -Arguments: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61], [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61] - -(85) CometNativeScan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#64, web_site_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(86) CometFilter -Input [2]: [web_site_sk#64, web_site_id#65] -Condition : isnotnull(web_site_sk#64) - -(87) CometProject -Input [2]: [web_site_sk#64, web_site_id#65] -Arguments: [web_site_sk#64, web_site_id#66], [web_site_sk#64, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#65, 16, true, false, true) AS web_site_id#66] - -(88) CometBroadcastExchange -Input [2]: [web_site_sk#64, web_site_id#66] -Arguments: [web_site_sk#64, web_site_id#66] - -(89) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61] -Right output [2]: [web_site_sk#64, web_site_id#66] -Arguments: [ws_web_site_sk#52], [web_site_sk#64], Inner, BuildRight - -(90) CometProject -Input [9]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_sk#64, web_site_id#66] -Arguments: [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66], [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] - -(91) ReusedExchange [Reuses operator id: 28] -Output [1]: [i_item_sk#67] - -(92) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] -Right output [1]: [i_item_sk#67] -Arguments: [ws_item_sk#51], [i_item_sk#67], Inner, BuildRight - -(93) CometProject -Input [8]: [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66, i_item_sk#67] -Arguments: [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66], [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] - -(94) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#68] - -(95) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] -Right output [1]: [p_promo_sk#68] -Arguments: [ws_promo_sk#53], [p_promo_sk#68], Inner, BuildRight - -(96) CometProject -Input [7]: [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66, p_promo_sk#68] -Arguments: [ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66], [ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] - -(97) CometHashAggregate -Input [5]: [ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] -Keys [1]: [web_site_id#66] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#55)), partial_sum(coalesce(cast(wr_return_amt#60 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#56 - coalesce(cast(wr_net_loss#61 as decimal(12,2)), 0.00)))] - -(98) CometExchange -Input [6]: [web_site_id#66, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Arguments: hashpartitioning(web_site_id#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(99) CometHashAggregate -Input [6]: [web_site_id#66, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Keys [1]: [web_site_id#66] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#55)), sum(coalesce(cast(wr_return_amt#60 as decimal(12,2)), 0.00)), sum((ws_net_profit#56 - coalesce(cast(wr_net_loss#61 as decimal(12,2)), 0.00)))] - -(100) CometUnion -Child 0 Input [5]: [sales#74, returns#75, profit#76, channel#77, id#78] -Child 1 Input [5]: [sales#79, returns#80, profit#81, channel#82, id#83] -Child 2 Input [5]: [sales#84, returns#85, profit#86, channel#87, id#88] - -(101) CometExpand -Input [5]: [sales#74, returns#75, profit#76, channel#77, id#78] -Arguments: [[sales#74, returns#75, profit#76, channel#77, id#78, 0], [sales#74, returns#75, profit#76, channel#77, null, 1], [sales#74, returns#75, profit#76, null, null, 3]], [sales#74, returns#75, profit#76, channel#89, id#90, spark_grouping_id#91] - -(102) CometHashAggregate -Input [6]: [sales#74, returns#75, profit#76, channel#89, id#90, spark_grouping_id#91] -Keys [3]: [channel#89, id#90, spark_grouping_id#91] -Functions [3]: [partial_sum(sales#74), partial_sum(returns#75), partial_sum(profit#76)] - -(103) CometExchange -Input [9]: [channel#89, id#90, spark_grouping_id#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97] -Arguments: hashpartitioning(channel#89, id#90, spark_grouping_id#91, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(104) CometHashAggregate -Input [9]: [channel#89, id#90, spark_grouping_id#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97] -Keys [3]: [channel#89, id#90, spark_grouping_id#91] -Functions [3]: [sum(sales#74), sum(returns#75), sum(profit#76)] - -(105) CometTakeOrderedAndProject -Input [5]: [channel#89, id#90, sales#98, returns#99, profit#100] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#89 ASC NULLS FIRST,id#90 ASC NULLS FIRST], output=[channel#89,id#90,sales#98,returns#99,profit#100]), [channel#89, id#90, sales#98, returns#99, profit#100], 100, 0, [channel#89 ASC NULLS FIRST, id#90 ASC NULLS FIRST], [channel#89, id#90, sales#98, returns#99, profit#100] - -(106) CometColumnarToRow [codegen id : 4] -Input [5]: [channel#89, id#90, sales#98, returns#99, profit#100] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (111) -+- * CometColumnarToRow (110) - +- CometProject (109) - +- CometFilter (108) - +- CometNativeScan parquet spark_catalog.default.date_dim (107) - - -(107) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] -ReadSchema: struct - -(108) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) - -(109) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(110) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(111) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#57 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/simplified.txt deleted file mode 100644 index 09e9224cef..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/simplified.txt +++ /dev/null @@ -1,124 +0,0 @@ -WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id,spark_grouping_id] #1 - CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] - CometUnion [sales,returns,profit,channel,id] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),s_store_id] - CometExchange [s_store_id] #2 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk,ss_ticket_number] #3 - WholeStageCodegen (1) - Filter [ss_store_sk,ss_item_sk,ss_promo_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #5 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #7 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #9 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,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))),cp_catalog_page_id] - CometExchange [cp_catalog_page_id] #10 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometColumnarExchange [cs_item_sk,cs_order_number] #11 - WholeStageCodegen (2) - Filter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),web_site_id] - CometExchange [web_site_id] #14 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometColumnarExchange [ws_item_sk,ws_order_number] #15 - WholeStageCodegen (3) - Filter [ws_web_site_sk,ws_item_sk,ws_promo_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #16 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [web_site_sk,web_site_id] #17 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/explain.txt deleted file mode 100644 index f0a163bf69..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,611 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (103) -+- CometTakeOrderedAndProject (102) - +- CometHashAggregate (101) - +- CometExchange (100) - +- CometHashAggregate (99) - +- CometExpand (98) - +- CometUnion (97) - :- CometHashAggregate (38) - : +- CometExchange (37) - : +- CometHashAggregate (36) - : +- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometProject (29) - : : +- CometBroadcastHashJoin (28) - : : :- CometProject (23) - : : : +- CometBroadcastHashJoin (22) - : : : :- CometProject (17) - : : : : +- CometBroadcastHashJoin (16) - : : : : :- CometProject (11) - : : : : : +- CometSortMergeJoin (10) - : : : : : :- CometSort (4) - : : : : : : +- CometExchange (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : +- CometSort (9) - : : : : : +- CometExchange (8) - : : : : : +- CometProject (7) - : : : : : +- CometFilter (6) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : +- CometBroadcastExchange (15) - : : : : +- CometProject (14) - : : : : +- CometFilter (13) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) - : : : +- CometBroadcastExchange (21) - : : : +- CometProject (20) - : : : +- CometFilter (19) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) - : : +- CometBroadcastExchange (27) - : : +- CometProject (26) - : : +- CometFilter (25) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - : +- CometBroadcastExchange (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - :- CometHashAggregate (67) - : +- CometExchange (66) - : +- CometHashAggregate (65) - : +- CometProject (64) - : +- CometBroadcastHashJoin (63) - : :- CometProject (61) - : : +- CometBroadcastHashJoin (60) - : : :- CometProject (58) - : : : +- CometBroadcastHashJoin (57) - : : : :- CometProject (52) - : : : : +- CometBroadcastHashJoin (51) - : : : : :- CometProject (49) - : : : : : +- CometSortMergeJoin (48) - : : : : : :- CometSort (42) - : : : : : : +- CometExchange (41) - : : : : : : +- CometFilter (40) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) - : : : : : +- CometSort (47) - : : : : : +- CometExchange (46) - : : : : : +- CometProject (45) - : : : : : +- CometFilter (44) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) - : : : : +- ReusedExchange (50) - : : : +- CometBroadcastExchange (56) - : : : +- CometProject (55) - : : : +- CometFilter (54) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) - : : +- ReusedExchange (59) - : +- ReusedExchange (62) - +- CometHashAggregate (96) - +- CometExchange (95) - +- CometHashAggregate (94) - +- CometProject (93) - +- CometBroadcastHashJoin (92) - :- CometProject (90) - : +- CometBroadcastHashJoin (89) - : :- CometProject (87) - : : +- CometBroadcastHashJoin (86) - : : :- CometProject (81) - : : : +- CometBroadcastHashJoin (80) - : : : :- CometProject (78) - : : : : +- CometSortMergeJoin (77) - : : : : :- CometSort (71) - : : : : : +- CometExchange (70) - : : : : : +- CometFilter (69) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) - : : : : +- CometSort (76) - : : : : +- CometExchange (75) - : : : : +- CometProject (74) - : : : : +- CometFilter (73) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) - : : : +- ReusedExchange (79) - : : +- CometBroadcastExchange (85) - : : +- CometProject (84) - : : +- CometFilter (83) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) - : +- ReusedExchange (88) - +- ReusedExchange (91) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) - -(7) CometProject -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(8) CometExchange -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter - -(11) CometProject -Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) - -(14) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(15) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(16) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight - -(17) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_store_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [s_store_sk#16, s_store_id#17] -Condition : isnotnull(s_store_sk#16) - -(20) CometProject -Input [2]: [s_store_sk#16, s_store_id#17] -Arguments: [s_store_sk#16, s_store_id#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#17, 16, true, false, true) AS s_store_id#18] - -(21) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_store_id#18] -Arguments: [s_store_sk#16, s_store_id#18] - -(22) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -Right output [2]: [s_store_sk#16, s_store_id#18] -Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight - -(23) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#18] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_current_price#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [i_item_sk#19, i_current_price#20] -Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) - -(26) CometProject -Input [2]: [i_item_sk#19, i_current_price#20] -Arguments: [i_item_sk#19], [i_item_sk#19] - -(27) CometBroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: [i_item_sk#19] - -(28) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [i_item_sk#19] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(29) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, i_item_sk#19] -Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#21, p_channel_tv#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#22, 1, true, false, true) = N) AND isnotnull(p_promo_sk#21)) - -(32) CometProject -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Arguments: [p_promo_sk#21], [p_promo_sk#21] - -(33) CometBroadcastExchange -Input [1]: [p_promo_sk#21] -Arguments: [p_promo_sk#21] - -(34) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [p_promo_sk#21] -Arguments: [ss_promo_sk#3], [p_promo_sk#21], Inner, BuildRight - -(35) CometProject -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, p_promo_sk#21] -Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(36) CometHashAggregate -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Keys [1]: [s_store_id#18] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(37) CometExchange -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [s_store_id#18] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(40) CometFilter -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) - -(41) CometExchange -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(42) CometSort -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(44) CometFilter -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) - -(45) CometProject -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] - -(46) CometExchange -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometSort -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] - -(48) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter - -(49) CometProject -Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] - -(50) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#41] - -(51) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -Right output [1]: [d_date_sk#41] -Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight - -(52) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(54) CometFilter -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Condition : isnotnull(cp_catalog_page_sk#42) - -(55) CometProject -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#43, 16, true, false, true) AS cp_catalog_page_id#44] - -(56) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] - -(57) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight - -(58) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(59) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#45] - -(60) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [i_item_sk#45] -Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight - -(61) CometProject -Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] -Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(62) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#46] - -(63) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [p_promo_sk#46] -Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight - -(64) CometProject -Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] -Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(65) CometHashAggregate -Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(66) CometExchange -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(67) CometHashAggregate -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] -ReadSchema: struct - -(69) CometFilter -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) - -(70) CometExchange -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(71) CometSort -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] -ReadSchema: struct - -(73) CometFilter -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) - -(74) CometProject -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] - -(75) CometExchange -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(76) CometSort -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] - -(77) CometSortMergeJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter - -(78) CometProject -Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] - -(79) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#65] - -(80) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -Right output [1]: [d_date_sk#65] -Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(81) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] - -(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#66, web_site_id#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(83) CometFilter -Input [2]: [web_site_sk#66, web_site_id#67] -Condition : isnotnull(web_site_sk#66) - -(84) CometProject -Input [2]: [web_site_sk#66, web_site_id#67] -Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#67, 16, true, false, true) AS web_site_id#68] - -(85) CometBroadcastExchange -Input [2]: [web_site_sk#66, web_site_id#68] -Arguments: [web_site_sk#66, web_site_id#68] - -(86) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -Right output [2]: [web_site_sk#66, web_site_id#68] -Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight - -(87) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] -Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(88) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#69] - -(89) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [i_item_sk#69] -Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight - -(90) CometProject -Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] -Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(91) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#70] - -(92) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [p_promo_sk#70] -Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight - -(93) CometProject -Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] -Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(94) CometHashAggregate -Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Keys [1]: [web_site_id#68] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(95) CometExchange -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(96) CometHashAggregate -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Keys [1]: [web_site_id#68] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(97) CometUnion -Child 0 Input [5]: [sales#76, returns#77, profit#78, channel#79, id#80] -Child 1 Input [5]: [sales#81, returns#82, profit#83, channel#84, id#85] -Child 2 Input [5]: [sales#86, returns#87, profit#88, channel#89, id#90] - -(98) CometExpand -Input [5]: [sales#76, returns#77, profit#78, channel#79, id#80] -Arguments: [[sales#76, returns#77, profit#78, channel#79, id#80, 0], [sales#76, returns#77, profit#78, channel#79, null, 1], [sales#76, returns#77, profit#78, null, null, 3]], [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] - -(99) CometHashAggregate -Input [6]: [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] -Keys [3]: [channel#91, id#92, spark_grouping_id#93] -Functions [3]: [partial_sum(sales#76), partial_sum(returns#77), partial_sum(profit#78)] - -(100) CometExchange -Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] -Arguments: hashpartitioning(channel#91, id#92, spark_grouping_id#93, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(101) CometHashAggregate -Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] -Keys [3]: [channel#91, id#92, spark_grouping_id#93] -Functions [3]: [sum(sales#76), sum(returns#77), sum(profit#78)] - -(102) CometTakeOrderedAndProject -Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#91 ASC NULLS FIRST,id#92 ASC NULLS FIRST], output=[channel#91,id#92,sales#100,returns#101,profit#102]), [channel#91, id#92, sales#100, returns#101, profit#102], 100, 0, [channel#91 ASC NULLS FIRST, id#92 ASC NULLS FIRST], [channel#91, id#92, sales#100, returns#101, profit#102] - -(103) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (108) -+- * CometColumnarToRow (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) - - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) - -(106) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(107) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(108) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/simplified.txt deleted file mode 100644 index 198646e0fd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,115 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id,spark_grouping_id] #1 - CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] - CometUnion [sales,returns,profit,channel,id] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),s_store_id] - CometExchange [s_store_id] #2 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #3 - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #5 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #7 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #9 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,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))),cp_catalog_page_id] - CometExchange [cp_catalog_page_id] #10 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #11 - 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 [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),web_site_id] - CometExchange [web_site_id] #14 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #15 - 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 [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #16 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [web_site_sk,web_site_id] #17 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/explain.txt deleted file mode 100644 index f0a163bf69..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/explain.txt +++ /dev/null @@ -1,611 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (103) -+- CometTakeOrderedAndProject (102) - +- CometHashAggregate (101) - +- CometExchange (100) - +- CometHashAggregate (99) - +- CometExpand (98) - +- CometUnion (97) - :- CometHashAggregate (38) - : +- CometExchange (37) - : +- CometHashAggregate (36) - : +- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometProject (29) - : : +- CometBroadcastHashJoin (28) - : : :- CometProject (23) - : : : +- CometBroadcastHashJoin (22) - : : : :- CometProject (17) - : : : : +- CometBroadcastHashJoin (16) - : : : : :- CometProject (11) - : : : : : +- CometSortMergeJoin (10) - : : : : : :- CometSort (4) - : : : : : : +- CometExchange (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : +- CometSort (9) - : : : : : +- CometExchange (8) - : : : : : +- CometProject (7) - : : : : : +- CometFilter (6) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : +- CometBroadcastExchange (15) - : : : : +- CometProject (14) - : : : : +- CometFilter (13) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) - : : : +- CometBroadcastExchange (21) - : : : +- CometProject (20) - : : : +- CometFilter (19) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) - : : +- CometBroadcastExchange (27) - : : +- CometProject (26) - : : +- CometFilter (25) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - : +- CometBroadcastExchange (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - :- CometHashAggregate (67) - : +- CometExchange (66) - : +- CometHashAggregate (65) - : +- CometProject (64) - : +- CometBroadcastHashJoin (63) - : :- CometProject (61) - : : +- CometBroadcastHashJoin (60) - : : :- CometProject (58) - : : : +- CometBroadcastHashJoin (57) - : : : :- CometProject (52) - : : : : +- CometBroadcastHashJoin (51) - : : : : :- CometProject (49) - : : : : : +- CometSortMergeJoin (48) - : : : : : :- CometSort (42) - : : : : : : +- CometExchange (41) - : : : : : : +- CometFilter (40) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) - : : : : : +- CometSort (47) - : : : : : +- CometExchange (46) - : : : : : +- CometProject (45) - : : : : : +- CometFilter (44) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) - : : : : +- ReusedExchange (50) - : : : +- CometBroadcastExchange (56) - : : : +- CometProject (55) - : : : +- CometFilter (54) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) - : : +- ReusedExchange (59) - : +- ReusedExchange (62) - +- CometHashAggregate (96) - +- CometExchange (95) - +- CometHashAggregate (94) - +- CometProject (93) - +- CometBroadcastHashJoin (92) - :- CometProject (90) - : +- CometBroadcastHashJoin (89) - : :- CometProject (87) - : : +- CometBroadcastHashJoin (86) - : : :- CometProject (81) - : : : +- CometBroadcastHashJoin (80) - : : : :- CometProject (78) - : : : : +- CometSortMergeJoin (77) - : : : : :- CometSort (71) - : : : : : +- CometExchange (70) - : : : : : +- CometFilter (69) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) - : : : : +- CometSort (76) - : : : : +- CometExchange (75) - : : : : +- CometProject (74) - : : : : +- CometFilter (73) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) - : : : +- ReusedExchange (79) - : : +- CometBroadcastExchange (85) - : : +- CometProject (84) - : : +- CometFilter (83) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) - : +- ReusedExchange (88) - +- ReusedExchange (91) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) - -(7) CometProject -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(8) CometExchange -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter - -(11) CometProject -Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) - -(14) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(15) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(16) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight - -(17) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_store_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [s_store_sk#16, s_store_id#17] -Condition : isnotnull(s_store_sk#16) - -(20) CometProject -Input [2]: [s_store_sk#16, s_store_id#17] -Arguments: [s_store_sk#16, s_store_id#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#17, 16, true, false, true) AS s_store_id#18] - -(21) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_store_id#18] -Arguments: [s_store_sk#16, s_store_id#18] - -(22) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -Right output [2]: [s_store_sk#16, s_store_id#18] -Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight - -(23) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#18] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_current_price#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [i_item_sk#19, i_current_price#20] -Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) - -(26) CometProject -Input [2]: [i_item_sk#19, i_current_price#20] -Arguments: [i_item_sk#19], [i_item_sk#19] - -(27) CometBroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: [i_item_sk#19] - -(28) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [i_item_sk#19] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(29) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, i_item_sk#19] -Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#21, p_channel_tv#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#22, 1, true, false, true) = N) AND isnotnull(p_promo_sk#21)) - -(32) CometProject -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Arguments: [p_promo_sk#21], [p_promo_sk#21] - -(33) CometBroadcastExchange -Input [1]: [p_promo_sk#21] -Arguments: [p_promo_sk#21] - -(34) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [p_promo_sk#21] -Arguments: [ss_promo_sk#3], [p_promo_sk#21], Inner, BuildRight - -(35) CometProject -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, p_promo_sk#21] -Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(36) CometHashAggregate -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Keys [1]: [s_store_id#18] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(37) CometExchange -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [s_store_id#18] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(40) CometFilter -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) - -(41) CometExchange -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(42) CometSort -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(44) CometFilter -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) - -(45) CometProject -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] - -(46) CometExchange -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometSort -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] - -(48) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter - -(49) CometProject -Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] - -(50) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#41] - -(51) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -Right output [1]: [d_date_sk#41] -Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight - -(52) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(54) CometFilter -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Condition : isnotnull(cp_catalog_page_sk#42) - -(55) CometProject -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#43, 16, true, false, true) AS cp_catalog_page_id#44] - -(56) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] - -(57) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight - -(58) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(59) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#45] - -(60) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [i_item_sk#45] -Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight - -(61) CometProject -Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] -Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(62) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#46] - -(63) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [p_promo_sk#46] -Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight - -(64) CometProject -Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] -Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(65) CometHashAggregate -Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(66) CometExchange -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(67) CometHashAggregate -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] -ReadSchema: struct - -(69) CometFilter -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) - -(70) CometExchange -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(71) CometSort -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] -ReadSchema: struct - -(73) CometFilter -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) - -(74) CometProject -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] - -(75) CometExchange -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(76) CometSort -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] - -(77) CometSortMergeJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter - -(78) CometProject -Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] - -(79) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#65] - -(80) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -Right output [1]: [d_date_sk#65] -Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(81) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] - -(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#66, web_site_id#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(83) CometFilter -Input [2]: [web_site_sk#66, web_site_id#67] -Condition : isnotnull(web_site_sk#66) - -(84) CometProject -Input [2]: [web_site_sk#66, web_site_id#67] -Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#67, 16, true, false, true) AS web_site_id#68] - -(85) CometBroadcastExchange -Input [2]: [web_site_sk#66, web_site_id#68] -Arguments: [web_site_sk#66, web_site_id#68] - -(86) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -Right output [2]: [web_site_sk#66, web_site_id#68] -Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight - -(87) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] -Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(88) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#69] - -(89) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [i_item_sk#69] -Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight - -(90) CometProject -Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] -Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(91) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#70] - -(92) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [p_promo_sk#70] -Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight - -(93) CometProject -Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] -Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(94) CometHashAggregate -Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Keys [1]: [web_site_id#68] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(95) CometExchange -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(96) CometHashAggregate -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Keys [1]: [web_site_id#68] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(97) CometUnion -Child 0 Input [5]: [sales#76, returns#77, profit#78, channel#79, id#80] -Child 1 Input [5]: [sales#81, returns#82, profit#83, channel#84, id#85] -Child 2 Input [5]: [sales#86, returns#87, profit#88, channel#89, id#90] - -(98) CometExpand -Input [5]: [sales#76, returns#77, profit#78, channel#79, id#80] -Arguments: [[sales#76, returns#77, profit#78, channel#79, id#80, 0], [sales#76, returns#77, profit#78, channel#79, null, 1], [sales#76, returns#77, profit#78, null, null, 3]], [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] - -(99) CometHashAggregate -Input [6]: [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] -Keys [3]: [channel#91, id#92, spark_grouping_id#93] -Functions [3]: [partial_sum(sales#76), partial_sum(returns#77), partial_sum(profit#78)] - -(100) CometExchange -Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] -Arguments: hashpartitioning(channel#91, id#92, spark_grouping_id#93, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(101) CometHashAggregate -Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] -Keys [3]: [channel#91, id#92, spark_grouping_id#93] -Functions [3]: [sum(sales#76), sum(returns#77), sum(profit#78)] - -(102) CometTakeOrderedAndProject -Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#91 ASC NULLS FIRST,id#92 ASC NULLS FIRST], output=[channel#91,id#92,sales#100,returns#101,profit#102]), [channel#91, id#92, sales#100, returns#101, profit#102], 100, 0, [channel#91 ASC NULLS FIRST, id#92 ASC NULLS FIRST], [channel#91, id#92, sales#100, returns#101, profit#102] - -(103) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (108) -+- * CometColumnarToRow (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) - - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) - -(106) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(107) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(108) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/extended.txt deleted file mode 100644 index ae6fc19b60..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/extended.txt +++ /dev/null @@ -1,131 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - -Comet accelerated 123 out of 127 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 198646e0fd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt +++ /dev/null @@ -1,115 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id,spark_grouping_id] #1 - CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] - CometUnion [sales,returns,profit,channel,id] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),s_store_id] - CometExchange [s_store_id] #2 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #3 - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #5 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #7 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #9 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,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))),cp_catalog_page_id] - CometExchange [cp_catalog_page_id] #10 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #11 - 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 [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),web_site_id] - CometExchange [web_site_id] #14 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #15 - 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 [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #16 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [web_site_sk,web_site_id] #17 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/explain.txt deleted file mode 100644 index 098cf9fe54..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/explain.txt +++ /dev/null @@ -1,346 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (54) -+- * Project (53) - +- * BroadcastHashJoin Inner BuildRight (52) - :- * Project (46) - : +- * BroadcastHashJoin Inner BuildRight (45) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- * HashAggregate (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.catalog_returns (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (11) - : : : +- * CometColumnarToRow (10) - : : : +- CometProject (9) - : : : +- CometFilter (8) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) - : : +- BroadcastExchange (37) - : : +- * Filter (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * HashAggregate (31) - : : +- * CometColumnarToRow (30) - : : +- CometColumnarExchange (29) - : : +- * HashAggregate (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (24) - : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : :- * Filter (21) - : : : : +- * ColumnarToRow (20) - : : : : +- Scan parquet spark_catalog.default.catalog_returns (19) - : : : +- ReusedExchange (22) - : : +- ReusedExchange (25) - : +- BroadcastExchange (44) - : +- * CometColumnarToRow (43) - : +- CometProject (42) - : +- CometFilter (41) - : +- CometNativeScan parquet spark_catalog.default.customer (40) - +- BroadcastExchange (51) - +- * CometColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometNativeScan parquet spark_catalog.default.customer_address (47) - - -(1) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] - -(3) Filter [codegen id : 3] -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Condition : (isnotnull(cr_returning_addr_sk#2) AND isnotnull(cr_returning_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_state#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_state#8] -Condition : (isnotnull(ca_address_sk#7) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#8, 2, true, false, true))) - -(9) CometProject -Input [2]: [ca_address_sk#7, ca_state#8] -Arguments: [ca_address_sk#7, ca_state#9], [ca_address_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#8, 2, true, false, true) AS ca_state#9] - -(10) CometColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#7, ca_state#9] - -(11) BroadcastExchange -Input [2]: [ca_address_sk#7, ca_state#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cr_returning_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#7, ca_state#9] - -(14) HashAggregate [codegen id : 3] -Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] -Keys [2]: [cr_returning_customer_sk#1, ca_state#9] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum#10] -Results [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] - -(15) CometColumnarExchange -Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 11] -Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] - -(17) HashAggregate [codegen id : 11] -Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] -Keys [2]: [cr_returning_customer_sk#1, ca_state#9] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#12] -Results [3]: [cr_returning_customer_sk#1 AS ctr_customer_sk#13, ca_state#9 AS ctr_state#14, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#12,17,2) AS ctr_total_return#15] - -(18) Filter [codegen id : 11] -Input [3]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15] -Condition : isnotnull(ctr_total_return#15) - -(19) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#19), dynamicpruningexpression(cr_returned_date_sk#19 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cr_returning_addr_sk)] -ReadSchema: struct - -(20) ColumnarToRow [codegen id : 6] -Input [4]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19] - -(21) Filter [codegen id : 6] -Input [4]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19] -Condition : isnotnull(cr_returning_addr_sk#17) - -(22) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#20] - -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returned_date_sk#19] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18] -Input [5]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19, d_date_sk#20] - -(25) ReusedExchange [Reuses operator id: 11] -Output [2]: [ca_address_sk#21, ca_state#9] - -(26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returning_addr_sk#17] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#16, cr_return_amt_inc_tax#18, ca_state#9] -Input [5]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, ca_address_sk#21, ca_state#9] - -(28) HashAggregate [codegen id : 6] -Input [3]: [cr_returning_customer_sk#16, cr_return_amt_inc_tax#18, ca_state#9] -Keys [2]: [cr_returning_customer_sk#16, ca_state#9] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#18))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [cr_returning_customer_sk#16, ca_state#9, sum#23] - -(29) CometColumnarExchange -Input [3]: [cr_returning_customer_sk#16, ca_state#9, sum#23] -Arguments: hashpartitioning(cr_returning_customer_sk#16, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(30) CometColumnarToRow [codegen id : 7] -Input [3]: [cr_returning_customer_sk#16, ca_state#9, sum#23] - -(31) HashAggregate [codegen id : 7] -Input [3]: [cr_returning_customer_sk#16, ca_state#9, sum#23] -Keys [2]: [cr_returning_customer_sk#16, ca_state#9] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#18))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#18))#12] -Results [2]: [ca_state#9 AS ctr_state#24, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#18))#12,17,2) AS ctr_total_return#25] - -(32) HashAggregate [codegen id : 7] -Input [2]: [ctr_state#24, ctr_total_return#25] -Keys [1]: [ctr_state#24] -Functions [1]: [partial_avg(ctr_total_return#25)] -Aggregate Attributes [2]: [sum#26, count#27] -Results [3]: [ctr_state#24, sum#28, count#29] - -(33) CometColumnarExchange -Input [3]: [ctr_state#24, sum#28, count#29] -Arguments: hashpartitioning(ctr_state#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 8] -Input [3]: [ctr_state#24, sum#28, count#29] - -(35) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#24, sum#28, count#29] -Keys [1]: [ctr_state#24] -Functions [1]: [avg(ctr_total_return#25)] -Aggregate Attributes [1]: [avg(ctr_total_return#25)#30] -Results [2]: [(avg(ctr_total_return#25)#30 * 1.2) AS (avg(ctr_total_return) * 1.2)#31, ctr_state#24] - -(36) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#24] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) - -(37) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#24] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] - -(38) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_state#14] -Right keys [1]: [ctr_state#24] -Join type: Inner -Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) - -(39) Project [codegen id : 11] -Output [2]: [ctr_customer_sk#13, ctr_total_return#15] -Input [5]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#31, ctr_state#24] - -(40) CometNativeScan parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(41) CometFilter -Input [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#34)) - -(42) CometProject -Input [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] -Arguments: [c_customer_sk#32, c_customer_id#38, c_current_addr_sk#34, c_salutation#39, c_first_name#40, c_last_name#41], [c_customer_sk#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true) AS c_customer_id#38, c_current_addr_sk#34, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#35, 10, true, false, true) AS c_salutation#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#36, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#37, 30, true, false, true) AS c_last_name#41] - -(43) CometColumnarToRow [codegen id : 9] -Input [6]: [c_customer_sk#32, c_customer_id#38, c_current_addr_sk#34, c_salutation#39, c_first_name#40, c_last_name#41] - -(44) BroadcastExchange -Input [6]: [c_customer_sk#32, c_customer_id#38, c_current_addr_sk#34, c_salutation#39, c_first_name#40, c_last_name#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(45) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_customer_sk#13] -Right keys [1]: [c_customer_sk#32] -Join type: Inner -Join condition: None - -(46) Project [codegen id : 11] -Output [6]: [ctr_total_return#15, c_customer_id#38, c_current_addr_sk#34, c_salutation#39, c_first_name#40, c_last_name#41] -Input [8]: [ctr_customer_sk#13, ctr_total_return#15, c_customer_sk#32, c_customer_id#38, c_current_addr_sk#34, c_salutation#39, c_first_name#40, c_last_name#41] - -(47) CometNativeScan parquet spark_catalog.default.customer_address -Output [12]: [ca_address_sk#42, ca_street_number#43, ca_street_name#44, ca_street_type#45, ca_suite_number#46, ca_city#47, ca_county#48, ca_state#49, ca_zip#50, ca_country#51, ca_gmt_offset#52, ca_location_type#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(48) CometFilter -Input [12]: [ca_address_sk#42, ca_street_number#43, ca_street_name#44, ca_street_type#45, ca_suite_number#46, ca_city#47, ca_county#48, ca_state#49, ca_zip#50, ca_country#51, ca_gmt_offset#52, ca_location_type#53] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#49, 2, true, false, true) = GA) AND isnotnull(ca_address_sk#42)) - -(49) CometProject -Input [12]: [ca_address_sk#42, ca_street_number#43, ca_street_name#44, ca_street_type#45, ca_suite_number#46, ca_city#47, ca_county#48, ca_state#49, ca_zip#50, ca_country#51, ca_gmt_offset#52, ca_location_type#53] -Arguments: [ca_address_sk#42, ca_street_number#54, ca_street_name#44, ca_street_type#55, ca_suite_number#56, ca_city#47, ca_county#48, ca_state#57, ca_zip#58, ca_country#51, ca_gmt_offset#52, ca_location_type#59], [ca_address_sk#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#43, 10, true, false, true) AS ca_street_number#54, ca_street_name#44, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_type#45, 15, true, false, true) AS ca_street_type#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_suite_number#46, 10, true, false, true) AS ca_suite_number#56, ca_city#47, ca_county#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#49, 2, true, false, true) AS ca_state#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#50, 10, true, false, true) AS ca_zip#58, ca_country#51, ca_gmt_offset#52, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_location_type#53, 20, true, false, true) AS ca_location_type#59] - -(50) CometColumnarToRow [codegen id : 10] -Input [12]: [ca_address_sk#42, ca_street_number#54, ca_street_name#44, ca_street_type#55, ca_suite_number#56, ca_city#47, ca_county#48, ca_state#57, ca_zip#58, ca_country#51, ca_gmt_offset#52, ca_location_type#59] - -(51) BroadcastExchange -Input [12]: [ca_address_sk#42, ca_street_number#54, ca_street_name#44, ca_street_type#55, ca_suite_number#56, ca_city#47, ca_county#48, ca_state#57, ca_zip#58, ca_country#51, ca_gmt_offset#52, ca_location_type#59] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(52) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#34] -Right keys [1]: [ca_address_sk#42] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 11] -Output [16]: [c_customer_id#38, c_salutation#39, c_first_name#40, c_last_name#41, ca_street_number#54, ca_street_name#44, ca_street_type#55, ca_suite_number#56, ca_city#47, ca_county#48, ca_state#57, ca_zip#58, ca_country#51, ca_gmt_offset#52, ca_location_type#59, ctr_total_return#15] -Input [18]: [ctr_total_return#15, c_customer_id#38, c_current_addr_sk#34, c_salutation#39, c_first_name#40, c_last_name#41, ca_address_sk#42, ca_street_number#54, ca_street_name#44, ca_street_type#55, ca_suite_number#56, ca_city#47, ca_county#48, ca_state#57, ca_zip#58, ca_country#51, ca_gmt_offset#52, ca_location_type#59] - -(54) TakeOrderedAndProject -Input [16]: [c_customer_id#38, c_salutation#39, c_first_name#40, c_last_name#41, ca_street_number#54, ca_street_name#44, ca_street_type#55, ca_suite_number#56, ca_city#47, ca_county#48, ca_state#57, ca_zip#58, ca_country#51, ca_gmt_offset#52, ca_location_type#59, ctr_total_return#15] -Arguments: 100, [c_customer_id#38 ASC NULLS FIRST, c_salutation#39 ASC NULLS FIRST, c_first_name#40 ASC NULLS FIRST, c_last_name#41 ASC NULLS FIRST, ca_street_number#54 ASC NULLS FIRST, ca_street_name#44 ASC NULLS FIRST, ca_street_type#55 ASC NULLS FIRST, ca_suite_number#56 ASC NULLS FIRST, ca_city#47 ASC NULLS FIRST, ca_county#48 ASC NULLS FIRST, ca_state#57 ASC NULLS FIRST, ca_zip#58 ASC NULLS FIRST, ca_country#51 ASC NULLS FIRST, ca_gmt_offset#52 ASC NULLS FIRST, ca_location_type#59 ASC NULLS FIRST, ctr_total_return#15 ASC NULLS FIRST], [c_customer_id#38, c_salutation#39, c_first_name#40, c_last_name#41, ca_street_number#54, ca_street_name#44, ca_street_type#55, ca_suite_number#56, ca_city#47, ca_county#48, ca_state#57, ca_zip#58, ca_country#51, ca_gmt_offset#52, ca_location_type#59, ctr_total_return#15] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (59) -+- * CometColumnarToRow (58) - +- CometProject (57) - +- CometFilter (56) - +- CometNativeScan parquet spark_catalog.default.date_dim (55) - - -(55) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#60] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date_sk#6, d_year#60] -Condition : ((isnotnull(d_year#60) AND (d_year#60 = 2000)) AND isnotnull(d_date_sk#6)) - -(57) CometProject -Input [2]: [d_date_sk#6, d_year#60] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(58) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(59) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 19 Hosting Expression = cr_returned_date_sk#19 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/simplified.txt deleted file mode 100644 index da396ce8a9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/simplified.txt +++ /dev/null @@ -1,86 +0,0 @@ -TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_total_return] - WholeStageCodegen (11) - Project [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_total_return] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (3) - HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] - Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Filter [cr_returning_addr_sk,cr_returning_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (8) - Filter [(avg(ctr_total_return) * 1.2)] - HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ctr_state] #5 - WholeStageCodegen (7) - HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cr_returning_customer_sk,ca_state] #6 - WholeStageCodegen (6) - HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] - Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Filter [cr_returning_addr_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [ca_address_sk,ca_state] #3 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometNativeScan 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 #8 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [ca_street_number,ca_street_type,ca_suite_number,ca_state,ca_zip,ca_location_type] [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] - 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] - CometNativeScan 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/q81.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/explain.txt deleted file mode 100644 index 35974f7924..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,318 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (50) -+- CometTakeOrderedAndProject (49) - +- CometProject (48) - +- CometBroadcastHashJoin (47) - :- CometProject (42) - : +- CometBroadcastHashJoin (41) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometFilter (18) - : : : +- CometHashAggregate (17) - : : : +- CometExchange (16) - : : : +- CometHashAggregate (15) - : : : +- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : : +- CometBroadcastExchange (34) - : : +- CometFilter (33) - : : +- CometHashAggregate (32) - : : +- CometExchange (31) - : : +- CometHashAggregate (30) - : : +- CometHashAggregate (29) - : : +- CometExchange (28) - : : +- CometHashAggregate (27) - : : +- CometProject (26) - : : +- CometBroadcastHashJoin (25) - : : :- CometProject (23) - : : : +- CometBroadcastHashJoin (22) - : : : :- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (19) - : : : +- ReusedExchange (21) - : : +- ReusedExchange (24) - : +- CometBroadcastExchange (40) - : +- CometProject (39) - : +- CometFilter (38) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) - +- CometBroadcastExchange (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (43) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Condition : (isnotnull(cr_returning_addr_sk#2) AND isnotnull(cr_returning_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [cr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] -Arguments: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3], [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#8, ca_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#8, ca_state#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#9, 2, true, false, true))) - -(11) CometProject -Input [2]: [ca_address_sk#8, ca_state#9] -Arguments: [ca_address_sk#8, ca_state#10], [ca_address_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#9, 2, true, false, true) AS ca_state#10] - -(12) CometBroadcastExchange -Input [2]: [ca_address_sk#8, ca_state#10] -Arguments: [ca_address_sk#8, ca_state#10] - -(13) CometBroadcastHashJoin -Left output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -Right output [2]: [ca_address_sk#8, ca_state#10] -Arguments: [cr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight - -(14) CometProject -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#8, ca_state#10] -Arguments: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10], [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] - -(15) CometHashAggregate -Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] -Keys [2]: [cr_returning_customer_sk#1, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] - -(16) CometExchange -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#11] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#11] -Keys [2]: [cr_returning_customer_sk#1, ca_state#10] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] - -(18) CometFilter -Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Condition : isnotnull(ctr_total_return#14) - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#18), dynamicpruningexpression(cr_returned_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cr_returning_addr_sk)] -ReadSchema: struct - -(20) CometFilter -Input [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] -Condition : isnotnull(cr_returning_addr_sk#16) - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#20] - -(22) CometBroadcastHashJoin -Left output [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] -Right output [1]: [d_date_sk#20] -Arguments: [cr_returned_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(23) CometProject -Input [5]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18, d_date_sk#20] -Arguments: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17], [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17] - -(24) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#21, ca_state#10] - -(25) CometBroadcastHashJoin -Left output [3]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17] -Right output [2]: [ca_address_sk#21, ca_state#10] -Arguments: [cr_returning_addr_sk#16], [ca_address_sk#21], Inner, BuildRight - -(26) CometProject -Input [5]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, ca_address_sk#21, ca_state#10] -Arguments: [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#10], [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#10] - -(27) CometHashAggregate -Input [3]: [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#10] -Keys [2]: [cr_returning_customer_sk#15, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#17))] - -(28) CometExchange -Input [3]: [cr_returning_customer_sk#15, ca_state#10, sum#22] -Arguments: hashpartitioning(cr_returning_customer_sk#15, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [3]: [cr_returning_customer_sk#15, ca_state#10, sum#22] -Keys [2]: [cr_returning_customer_sk#15, ca_state#10] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#17))] - -(30) CometHashAggregate -Input [2]: [ctr_state#23, ctr_total_return#24] -Keys [1]: [ctr_state#23] -Functions [1]: [partial_avg(ctr_total_return#24)] - -(31) CometExchange -Input [3]: [ctr_state#23, sum#25, count#26] -Arguments: hashpartitioning(ctr_state#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometHashAggregate -Input [3]: [ctr_state#23, sum#25, count#26] -Keys [1]: [ctr_state#23] -Functions [1]: [avg(ctr_total_return#24)] - -(33) CometFilter -Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#27) - -(34) CometBroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] -Arguments: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] - -(35) CometBroadcastHashJoin -Left output [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Right output [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] -Arguments: [ctr_state#13], [ctr_state#23], Inner, (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#27), BuildRight - -(36) CometProject -Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#27, ctr_state#23] -Arguments: [ctr_customer_sk#12, ctr_total_return#14], [ctr_customer_sk#12, ctr_total_return#14] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(38) CometFilter -Input [6]: [c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33] -Condition : (isnotnull(c_customer_sk#28) AND isnotnull(c_current_addr_sk#30)) - -(39) CometProject -Input [6]: [c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33] -Arguments: [c_customer_sk#28, c_customer_id#34, c_current_addr_sk#30, c_salutation#35, c_first_name#36, c_last_name#37], [c_customer_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#29, 16, true, false, true) AS c_customer_id#34, c_current_addr_sk#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#31, 10, true, false, true) AS c_salutation#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#32, 20, true, false, true) AS c_first_name#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#33, 30, true, false, true) AS c_last_name#37] - -(40) CometBroadcastExchange -Input [6]: [c_customer_sk#28, c_customer_id#34, c_current_addr_sk#30, c_salutation#35, c_first_name#36, c_last_name#37] -Arguments: [c_customer_sk#28, c_customer_id#34, c_current_addr_sk#30, c_salutation#35, c_first_name#36, c_last_name#37] - -(41) CometBroadcastHashJoin -Left output [2]: [ctr_customer_sk#12, ctr_total_return#14] -Right output [6]: [c_customer_sk#28, c_customer_id#34, c_current_addr_sk#30, c_salutation#35, c_first_name#36, c_last_name#37] -Arguments: [ctr_customer_sk#12], [c_customer_sk#28], Inner, BuildRight - -(42) CometProject -Input [8]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#28, c_customer_id#34, c_current_addr_sk#30, c_salutation#35, c_first_name#36, c_last_name#37] -Arguments: [ctr_total_return#14, c_customer_id#34, c_current_addr_sk#30, c_salutation#35, c_first_name#36, c_last_name#37], [ctr_total_return#14, c_customer_id#34, c_current_addr_sk#30, c_salutation#35, c_first_name#36, c_last_name#37] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [12]: [ca_address_sk#38, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#45, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(44) CometFilter -Input [12]: [ca_address_sk#38, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#45, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#45, 2, true, false, true) = GA) AND isnotnull(ca_address_sk#38)) - -(45) CometProject -Input [12]: [ca_address_sk#38, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#45, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -Arguments: [ca_address_sk#38, ca_street_number#50, ca_street_name#40, ca_street_type#51, ca_suite_number#52, ca_city#43, ca_county#44, ca_state#53, ca_zip#54, ca_country#47, ca_gmt_offset#48, ca_location_type#55], [ca_address_sk#38, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#39, 10, true, false, true) AS ca_street_number#50, ca_street_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_type#41, 15, true, false, true) AS ca_street_type#51, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_suite_number#42, 10, true, false, true) AS ca_suite_number#52, ca_city#43, ca_county#44, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#45, 2, true, false, true) AS ca_state#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#46, 10, true, false, true) AS ca_zip#54, ca_country#47, ca_gmt_offset#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_location_type#49, 20, true, false, true) AS ca_location_type#55] - -(46) CometBroadcastExchange -Input [12]: [ca_address_sk#38, ca_street_number#50, ca_street_name#40, ca_street_type#51, ca_suite_number#52, ca_city#43, ca_county#44, ca_state#53, ca_zip#54, ca_country#47, ca_gmt_offset#48, ca_location_type#55] -Arguments: [ca_address_sk#38, ca_street_number#50, ca_street_name#40, ca_street_type#51, ca_suite_number#52, ca_city#43, ca_county#44, ca_state#53, ca_zip#54, ca_country#47, ca_gmt_offset#48, ca_location_type#55] - -(47) CometBroadcastHashJoin -Left output [6]: [ctr_total_return#14, c_customer_id#34, c_current_addr_sk#30, c_salutation#35, c_first_name#36, c_last_name#37] -Right output [12]: [ca_address_sk#38, ca_street_number#50, ca_street_name#40, ca_street_type#51, ca_suite_number#52, ca_city#43, ca_county#44, ca_state#53, ca_zip#54, ca_country#47, ca_gmt_offset#48, ca_location_type#55] -Arguments: [c_current_addr_sk#30], [ca_address_sk#38], Inner, BuildRight - -(48) CometProject -Input [18]: [ctr_total_return#14, c_customer_id#34, c_current_addr_sk#30, c_salutation#35, c_first_name#36, c_last_name#37, ca_address_sk#38, ca_street_number#50, ca_street_name#40, ca_street_type#51, ca_suite_number#52, ca_city#43, ca_county#44, ca_state#53, ca_zip#54, ca_country#47, ca_gmt_offset#48, ca_location_type#55] -Arguments: [c_customer_id#34, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#50, ca_street_name#40, ca_street_type#51, ca_suite_number#52, ca_city#43, ca_county#44, ca_state#53, ca_zip#54, ca_country#47, ca_gmt_offset#48, ca_location_type#55, ctr_total_return#14], [c_customer_id#34, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#50, ca_street_name#40, ca_street_type#51, ca_suite_number#52, ca_city#43, ca_county#44, ca_state#53, ca_zip#54, ca_country#47, ca_gmt_offset#48, ca_location_type#55, ctr_total_return#14] - -(49) CometTakeOrderedAndProject -Input [16]: [c_customer_id#34, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#50, ca_street_name#40, ca_street_type#51, ca_suite_number#52, ca_city#43, ca_county#44, ca_state#53, ca_zip#54, ca_country#47, ca_gmt_offset#48, ca_location_type#55, ctr_total_return#14] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#34 ASC NULLS FIRST,c_salutation#35 ASC NULLS FIRST,c_first_name#36 ASC NULLS FIRST,c_last_name#37 ASC NULLS FIRST,ca_street_number#50 ASC NULLS FIRST,ca_street_name#40 ASC NULLS FIRST,ca_street_type#51 ASC NULLS FIRST,ca_suite_number#52 ASC NULLS FIRST,ca_city#43 ASC NULLS FIRST,ca_county#44 ASC NULLS FIRST,ca_state#53 ASC NULLS FIRST,ca_zip#54 ASC NULLS FIRST,ca_country#47 ASC NULLS FIRST,ca_gmt_offset#48 ASC NULLS FIRST,ca_location_type#55 ASC NULLS FIRST,ctr_total_return#14 ASC NULLS FIRST], output=[c_customer_id#34,c_salutation#35,c_first_name#36,c_last_name#37,ca_street_number#50,ca_street_name#40,ca_street_type#51,ca_suite_number#52,ca_city#43,ca_county#44,ca_state#53,ca_zip#54,ca_country#47,ca_gmt_offset#48,ca_location_type#55,ctr_total_return#14]), [c_customer_id#34, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#50, ca_street_name#40, ca_street_type#51, ca_suite_number#52, ca_city#43, ca_county#44, ca_state#53, ca_zip#54, ca_country#47, ca_gmt_offset#48, ca_location_type#55, ctr_total_return#14], 100, 0, [c_customer_id#34 ASC NULLS FIRST, c_salutation#35 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, c_last_name#37 ASC NULLS FIRST, ca_street_number#50 ASC NULLS FIRST, ca_street_name#40 ASC NULLS FIRST, ca_street_type#51 ASC NULLS FIRST, ca_suite_number#52 ASC NULLS FIRST, ca_city#43 ASC NULLS FIRST, ca_county#44 ASC NULLS FIRST, ca_state#53 ASC NULLS FIRST, ca_zip#54 ASC NULLS FIRST, ca_country#47 ASC NULLS FIRST, ca_gmt_offset#48 ASC NULLS FIRST, ca_location_type#55 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#34, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#50, ca_street_name#40, ca_street_type#51, ca_suite_number#52, ca_city#43, ca_county#44, ca_state#53, ca_zip#54, ca_country#47, ca_gmt_offset#48, ca_location_type#55, ctr_total_return#14] - -(50) CometColumnarToRow [codegen id : 1] -Input [16]: [c_customer_id#34, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#50, ca_street_name#40, ca_street_type#51, ca_suite_number#52, ca_city#43, ca_county#44, ca_state#53, ca_zip#54, ca_country#47, ca_gmt_offset#48, ca_location_type#55, ctr_total_return#14] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) - - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(52) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(53) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(54) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(55) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 19 Hosting Expression = cr_returned_date_sk#18 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/simplified.txt deleted file mode 100644 index 1091e272cc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_total_return] - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_total_return] - CometBroadcastHashJoin [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,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] - CometProject [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometBroadcastHashJoin [ctr_customer_sk,ctr_total_return,c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometProject [ctr_customer_sk,ctr_total_return] - CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] - CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] - CometHashAggregate [sum] [ctr_customer_sk,ctr_state,ctr_total_return,cr_returning_customer_sk,ca_state,sum(UnscaledValue(cr_return_amt_inc_tax))] - CometExchange [cr_returning_customer_sk,ca_state] #1 - CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] - CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - 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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 - CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] - CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_state,avg(ctr_total_return)] - CometExchange [ctr_state] #6 - CometHashAggregate [ctr_total_return] [ctr_state,sum,count] - CometHashAggregate [cr_returning_customer_sk,sum] [ctr_state,ctr_total_return,ca_state,sum(UnscaledValue(cr_return_amt_inc_tax))] - CometExchange [cr_returning_customer_sk,ca_state] #7 - CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] - CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - 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_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 [native_iceberg_compat] 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 - ReusedExchange [ca_address_sk,ca_state] #4 - CometBroadcastExchange [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] #8 - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometBroadcastExchange [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] #9 - CometProject [ca_street_number,ca_street_type,ca_suite_number,ca_state,ca_zip,ca_location_type] [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] - 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 [native_iceberg_compat] 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/q81/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/explain.txt deleted file mode 100644 index 35974f7924..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/explain.txt +++ /dev/null @@ -1,318 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (50) -+- CometTakeOrderedAndProject (49) - +- CometProject (48) - +- CometBroadcastHashJoin (47) - :- CometProject (42) - : +- CometBroadcastHashJoin (41) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometFilter (18) - : : : +- CometHashAggregate (17) - : : : +- CometExchange (16) - : : : +- CometHashAggregate (15) - : : : +- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : : +- CometBroadcastExchange (34) - : : +- CometFilter (33) - : : +- CometHashAggregate (32) - : : +- CometExchange (31) - : : +- CometHashAggregate (30) - : : +- CometHashAggregate (29) - : : +- CometExchange (28) - : : +- CometHashAggregate (27) - : : +- CometProject (26) - : : +- CometBroadcastHashJoin (25) - : : :- CometProject (23) - : : : +- CometBroadcastHashJoin (22) - : : : :- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (19) - : : : +- ReusedExchange (21) - : : +- ReusedExchange (24) - : +- CometBroadcastExchange (40) - : +- CometProject (39) - : +- CometFilter (38) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) - +- CometBroadcastExchange (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (43) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Condition : (isnotnull(cr_returning_addr_sk#2) AND isnotnull(cr_returning_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [cr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] -Arguments: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3], [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#8, ca_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#8, ca_state#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#9, 2, true, false, true))) - -(11) CometProject -Input [2]: [ca_address_sk#8, ca_state#9] -Arguments: [ca_address_sk#8, ca_state#10], [ca_address_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#9, 2, true, false, true) AS ca_state#10] - -(12) CometBroadcastExchange -Input [2]: [ca_address_sk#8, ca_state#10] -Arguments: [ca_address_sk#8, ca_state#10] - -(13) CometBroadcastHashJoin -Left output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -Right output [2]: [ca_address_sk#8, ca_state#10] -Arguments: [cr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight - -(14) CometProject -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#8, ca_state#10] -Arguments: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10], [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] - -(15) CometHashAggregate -Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] -Keys [2]: [cr_returning_customer_sk#1, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] - -(16) CometExchange -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#11] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#11] -Keys [2]: [cr_returning_customer_sk#1, ca_state#10] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] - -(18) CometFilter -Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Condition : isnotnull(ctr_total_return#14) - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#18), dynamicpruningexpression(cr_returned_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cr_returning_addr_sk)] -ReadSchema: struct - -(20) CometFilter -Input [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] -Condition : isnotnull(cr_returning_addr_sk#16) - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#20] - -(22) CometBroadcastHashJoin -Left output [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] -Right output [1]: [d_date_sk#20] -Arguments: [cr_returned_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(23) CometProject -Input [5]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18, d_date_sk#20] -Arguments: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17], [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17] - -(24) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#21, ca_state#10] - -(25) CometBroadcastHashJoin -Left output [3]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17] -Right output [2]: [ca_address_sk#21, ca_state#10] -Arguments: [cr_returning_addr_sk#16], [ca_address_sk#21], Inner, BuildRight - -(26) CometProject -Input [5]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, ca_address_sk#21, ca_state#10] -Arguments: [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#10], [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#10] - -(27) CometHashAggregate -Input [3]: [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#10] -Keys [2]: [cr_returning_customer_sk#15, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#17))] - -(28) CometExchange -Input [3]: [cr_returning_customer_sk#15, ca_state#10, sum#22] -Arguments: hashpartitioning(cr_returning_customer_sk#15, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [3]: [cr_returning_customer_sk#15, ca_state#10, sum#22] -Keys [2]: [cr_returning_customer_sk#15, ca_state#10] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#17))] - -(30) CometHashAggregate -Input [2]: [ctr_state#23, ctr_total_return#24] -Keys [1]: [ctr_state#23] -Functions [1]: [partial_avg(ctr_total_return#24)] - -(31) CometExchange -Input [3]: [ctr_state#23, sum#25, count#26] -Arguments: hashpartitioning(ctr_state#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometHashAggregate -Input [3]: [ctr_state#23, sum#25, count#26] -Keys [1]: [ctr_state#23] -Functions [1]: [avg(ctr_total_return#24)] - -(33) CometFilter -Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#27) - -(34) CometBroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] -Arguments: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] - -(35) CometBroadcastHashJoin -Left output [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Right output [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_state#23] -Arguments: [ctr_state#13], [ctr_state#23], Inner, (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#27), BuildRight - -(36) CometProject -Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#27, ctr_state#23] -Arguments: [ctr_customer_sk#12, ctr_total_return#14], [ctr_customer_sk#12, ctr_total_return#14] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(38) CometFilter -Input [6]: [c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33] -Condition : (isnotnull(c_customer_sk#28) AND isnotnull(c_current_addr_sk#30)) - -(39) CometProject -Input [6]: [c_customer_sk#28, c_customer_id#29, c_current_addr_sk#30, c_salutation#31, c_first_name#32, c_last_name#33] -Arguments: [c_customer_sk#28, c_customer_id#34, c_current_addr_sk#30, c_salutation#35, c_first_name#36, c_last_name#37], [c_customer_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#29, 16, true, false, true) AS c_customer_id#34, c_current_addr_sk#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#31, 10, true, false, true) AS c_salutation#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#32, 20, true, false, true) AS c_first_name#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#33, 30, true, false, true) AS c_last_name#37] - -(40) CometBroadcastExchange -Input [6]: [c_customer_sk#28, c_customer_id#34, c_current_addr_sk#30, c_salutation#35, c_first_name#36, c_last_name#37] -Arguments: [c_customer_sk#28, c_customer_id#34, c_current_addr_sk#30, c_salutation#35, c_first_name#36, c_last_name#37] - -(41) CometBroadcastHashJoin -Left output [2]: [ctr_customer_sk#12, ctr_total_return#14] -Right output [6]: [c_customer_sk#28, c_customer_id#34, c_current_addr_sk#30, c_salutation#35, c_first_name#36, c_last_name#37] -Arguments: [ctr_customer_sk#12], [c_customer_sk#28], Inner, BuildRight - -(42) CometProject -Input [8]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#28, c_customer_id#34, c_current_addr_sk#30, c_salutation#35, c_first_name#36, c_last_name#37] -Arguments: [ctr_total_return#14, c_customer_id#34, c_current_addr_sk#30, c_salutation#35, c_first_name#36, c_last_name#37], [ctr_total_return#14, c_customer_id#34, c_current_addr_sk#30, c_salutation#35, c_first_name#36, c_last_name#37] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [12]: [ca_address_sk#38, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#45, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(44) CometFilter -Input [12]: [ca_address_sk#38, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#45, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#45, 2, true, false, true) = GA) AND isnotnull(ca_address_sk#38)) - -(45) CometProject -Input [12]: [ca_address_sk#38, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#45, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -Arguments: [ca_address_sk#38, ca_street_number#50, ca_street_name#40, ca_street_type#51, ca_suite_number#52, ca_city#43, ca_county#44, ca_state#53, ca_zip#54, ca_country#47, ca_gmt_offset#48, ca_location_type#55], [ca_address_sk#38, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#39, 10, true, false, true) AS ca_street_number#50, ca_street_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_type#41, 15, true, false, true) AS ca_street_type#51, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_suite_number#42, 10, true, false, true) AS ca_suite_number#52, ca_city#43, ca_county#44, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#45, 2, true, false, true) AS ca_state#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#46, 10, true, false, true) AS ca_zip#54, ca_country#47, ca_gmt_offset#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_location_type#49, 20, true, false, true) AS ca_location_type#55] - -(46) CometBroadcastExchange -Input [12]: [ca_address_sk#38, ca_street_number#50, ca_street_name#40, ca_street_type#51, ca_suite_number#52, ca_city#43, ca_county#44, ca_state#53, ca_zip#54, ca_country#47, ca_gmt_offset#48, ca_location_type#55] -Arguments: [ca_address_sk#38, ca_street_number#50, ca_street_name#40, ca_street_type#51, ca_suite_number#52, ca_city#43, ca_county#44, ca_state#53, ca_zip#54, ca_country#47, ca_gmt_offset#48, ca_location_type#55] - -(47) CometBroadcastHashJoin -Left output [6]: [ctr_total_return#14, c_customer_id#34, c_current_addr_sk#30, c_salutation#35, c_first_name#36, c_last_name#37] -Right output [12]: [ca_address_sk#38, ca_street_number#50, ca_street_name#40, ca_street_type#51, ca_suite_number#52, ca_city#43, ca_county#44, ca_state#53, ca_zip#54, ca_country#47, ca_gmt_offset#48, ca_location_type#55] -Arguments: [c_current_addr_sk#30], [ca_address_sk#38], Inner, BuildRight - -(48) CometProject -Input [18]: [ctr_total_return#14, c_customer_id#34, c_current_addr_sk#30, c_salutation#35, c_first_name#36, c_last_name#37, ca_address_sk#38, ca_street_number#50, ca_street_name#40, ca_street_type#51, ca_suite_number#52, ca_city#43, ca_county#44, ca_state#53, ca_zip#54, ca_country#47, ca_gmt_offset#48, ca_location_type#55] -Arguments: [c_customer_id#34, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#50, ca_street_name#40, ca_street_type#51, ca_suite_number#52, ca_city#43, ca_county#44, ca_state#53, ca_zip#54, ca_country#47, ca_gmt_offset#48, ca_location_type#55, ctr_total_return#14], [c_customer_id#34, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#50, ca_street_name#40, ca_street_type#51, ca_suite_number#52, ca_city#43, ca_county#44, ca_state#53, ca_zip#54, ca_country#47, ca_gmt_offset#48, ca_location_type#55, ctr_total_return#14] - -(49) CometTakeOrderedAndProject -Input [16]: [c_customer_id#34, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#50, ca_street_name#40, ca_street_type#51, ca_suite_number#52, ca_city#43, ca_county#44, ca_state#53, ca_zip#54, ca_country#47, ca_gmt_offset#48, ca_location_type#55, ctr_total_return#14] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#34 ASC NULLS FIRST,c_salutation#35 ASC NULLS FIRST,c_first_name#36 ASC NULLS FIRST,c_last_name#37 ASC NULLS FIRST,ca_street_number#50 ASC NULLS FIRST,ca_street_name#40 ASC NULLS FIRST,ca_street_type#51 ASC NULLS FIRST,ca_suite_number#52 ASC NULLS FIRST,ca_city#43 ASC NULLS FIRST,ca_county#44 ASC NULLS FIRST,ca_state#53 ASC NULLS FIRST,ca_zip#54 ASC NULLS FIRST,ca_country#47 ASC NULLS FIRST,ca_gmt_offset#48 ASC NULLS FIRST,ca_location_type#55 ASC NULLS FIRST,ctr_total_return#14 ASC NULLS FIRST], output=[c_customer_id#34,c_salutation#35,c_first_name#36,c_last_name#37,ca_street_number#50,ca_street_name#40,ca_street_type#51,ca_suite_number#52,ca_city#43,ca_county#44,ca_state#53,ca_zip#54,ca_country#47,ca_gmt_offset#48,ca_location_type#55,ctr_total_return#14]), [c_customer_id#34, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#50, ca_street_name#40, ca_street_type#51, ca_suite_number#52, ca_city#43, ca_county#44, ca_state#53, ca_zip#54, ca_country#47, ca_gmt_offset#48, ca_location_type#55, ctr_total_return#14], 100, 0, [c_customer_id#34 ASC NULLS FIRST, c_salutation#35 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, c_last_name#37 ASC NULLS FIRST, ca_street_number#50 ASC NULLS FIRST, ca_street_name#40 ASC NULLS FIRST, ca_street_type#51 ASC NULLS FIRST, ca_suite_number#52 ASC NULLS FIRST, ca_city#43 ASC NULLS FIRST, ca_county#44 ASC NULLS FIRST, ca_state#53 ASC NULLS FIRST, ca_zip#54 ASC NULLS FIRST, ca_country#47 ASC NULLS FIRST, ca_gmt_offset#48 ASC NULLS FIRST, ca_location_type#55 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#34, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#50, ca_street_name#40, ca_street_type#51, ca_suite_number#52, ca_city#43, ca_county#44, ca_state#53, ca_zip#54, ca_country#47, ca_gmt_offset#48, ca_location_type#55, ctr_total_return#14] - -(50) CometColumnarToRow [codegen id : 1] -Input [16]: [c_customer_id#34, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#50, ca_street_name#40, ca_street_type#51, ca_suite_number#52, ca_city#43, ca_county#44, ca_state#53, ca_zip#54, ca_country#47, ca_gmt_offset#48, ca_location_type#55, ctr_total_return#14] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) - - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(52) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(53) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(54) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(55) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 19 Hosting Expression = cr_returned_date_sk#18 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/extended.txt deleted file mode 100644 index 9708802e79..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 1091e272cc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_total_return] - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_total_return] - CometBroadcastHashJoin [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,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] - CometProject [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometBroadcastHashJoin [ctr_customer_sk,ctr_total_return,c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometProject [ctr_customer_sk,ctr_total_return] - CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] - CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] - CometHashAggregate [sum] [ctr_customer_sk,ctr_state,ctr_total_return,cr_returning_customer_sk,ca_state,sum(UnscaledValue(cr_return_amt_inc_tax))] - CometExchange [cr_returning_customer_sk,ca_state] #1 - CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] - CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - 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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 - CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] - CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_state,avg(ctr_total_return)] - CometExchange [ctr_state] #6 - CometHashAggregate [ctr_total_return] [ctr_state,sum,count] - CometHashAggregate [cr_returning_customer_sk,sum] [ctr_state,ctr_total_return,ca_state,sum(UnscaledValue(cr_return_amt_inc_tax))] - CometExchange [cr_returning_customer_sk,ca_state] #7 - CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] - CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - 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_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 [native_iceberg_compat] 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 - ReusedExchange [ca_address_sk,ca_state] #4 - CometBroadcastExchange [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] #8 - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometBroadcastExchange [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] #9 - CometProject [ca_street_number,ca_street_type,ca_suite_number,ca_state,ca_zip,ca_location_type] [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] - 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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/explain.txt deleted file mode 100644 index 93da88fcc1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/explain.txt +++ /dev/null @@ -1,181 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometTakeOrderedAndProject (25) - +- CometHashAggregate (24) - +- CometColumnarExchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildLeft (20) - :- BroadcastExchange (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * CometColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (9) - : : +- * Project (8) - : : +- * Filter (7) - : : +- * ColumnarToRow (6) - : : +- Scan parquet spark_catalog.default.inventory (5) - : +- ReusedExchange (12) - +- * CometColumnarToRow (19) - +- CometProject (18) - +- CometFilter (17) - +- CometNativeScan parquet spark_catalog.default.store_sales (16) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,62.00), LessThanOrEqual(i_current_price,92.00), In(i_manufact_id, [129,270,423,821]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 62.00)) AND (i_current_price#4 <= 92.00)) AND i_manufact_id#5 IN (129,270,821,423)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#2, 16, true, false, true) AS i_item_id#6, i_item_desc#3, i_current_price#4] - -(4) CometColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(5) Scan parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#9), dynamicpruningexpression(inv_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] - -(7) Filter [codegen id : 1] -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Condition : (((isnotnull(inv_quantity_on_hand#8) AND (inv_quantity_on_hand#8 >= 100)) AND (inv_quantity_on_hand#8 <= 500)) AND isnotnull(inv_item_sk#7)) - -(8) Project [codegen id : 1] -Output [2]: [inv_item_sk#7, inv_date_sk#9] -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] - -(9) BroadcastExchange -Input [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [inv_item_sk#7] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 3] -Output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] - -(12) ReusedExchange [Reuses operator id: 31] -Output [1]: [d_date_sk#11] - -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [inv_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 3] -Output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] - -(15) BroadcastExchange -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) CometNativeScan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#12, ss_sold_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [ss_item_sk#12, ss_sold_date_sk#13] -Condition : isnotnull(ss_item_sk#12) - -(18) CometProject -Input [2]: [ss_item_sk#12, ss_sold_date_sk#13] -Arguments: [ss_item_sk#12], [ss_item_sk#12] - -(19) CometColumnarToRow -Input [1]: [ss_item_sk#12] - -(20) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#12] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 4] -Output [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, ss_item_sk#12] - -(22) HashAggregate [codegen id : 4] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -(23) CometColumnarExchange -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(25) CometTakeOrderedAndProject -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(26) CometColumnarToRow [codegen id : 5] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometNativeScan parquet spark_catalog.default.date_dim (27) - - -(27) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#11, d_date#14] -Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 2000-05-25)) AND (d_date#14 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) - -(29) CometProject -Input [2]: [d_date_sk#11, d_date#14] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(31) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/simplified.txt deleted file mode 100644 index b60bf92c5b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_current_price] - Project [i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (3) - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [inv_item_sk,inv_date_sk] - Filter [inv_quantity_on_hand,inv_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #4 - CometColumnarToRow - InputAdapter - CometProject [ss_item_sk] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometNativeScan 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/q82.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/explain.txt deleted file mode 100644 index 698608e16d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometTakeOrderedAndProject (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (4) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - +- CometProject (19) - +- CometFilter (18) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (17) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,62.00), LessThanOrEqual(i_current_price,92.00), In(i_manufact_id, [129,270,423,821]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 62.00)) AND (i_current_price#4 <= 92.00)) AND i_manufact_id#5 IN (129,270,821,423)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#2, 16, true, false, true) AS i_item_id#6, i_item_desc#3, i_current_price#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#9), dynamicpruningexpression(inv_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Condition : (((isnotnull(inv_quantity_on_hand#8) AND (inv_quantity_on_hand#8 >= 100)) AND (inv_quantity_on_hand#8 <= 500)) AND isnotnull(inv_item_sk#7)) - -(6) CometProject -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9], [inv_item_sk#7, inv_date_sk#9] - -(7) CometBroadcastExchange -Input [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1], [inv_item_sk#7], Inner, BuildRight - -(9) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#12 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) - -(12) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(14) CometBroadcastHashJoin -Left output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [inv_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(15) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(16) CometBroadcastExchange -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#13, ss_sold_date_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [ss_item_sk#13, ss_sold_date_sk#14] -Condition : isnotnull(ss_item_sk#13) - -(19) CometProject -Input [2]: [ss_item_sk#13, ss_sold_date_sk#14] -Arguments: [ss_item_sk#13], [ss_item_sk#13] - -(20) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [1]: [ss_item_sk#13] -Arguments: [i_item_sk#1], [ss_item_sk#13], Inner, BuildLeft - -(21) CometProject -Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, ss_item_sk#13] -Arguments: [i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(22) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(23) CometExchange -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(25) CometTakeOrderedAndProject -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(26) CometColumnarToRow [codegen id : 1] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#12 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) - -(29) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(31) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/simplified.txt deleted file mode 100644 index 28d160a4a8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometExchange [i_item_id,i_item_desc,i_current_price] #1 - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometProject [i_item_id,i_item_desc,i_current_price] - 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 [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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [ss_item_sk] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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/q82/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/explain.txt deleted file mode 100644 index 698608e16d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometTakeOrderedAndProject (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (4) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - +- CometProject (19) - +- CometFilter (18) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (17) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,62.00), LessThanOrEqual(i_current_price,92.00), In(i_manufact_id, [129,270,423,821]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 62.00)) AND (i_current_price#4 <= 92.00)) AND i_manufact_id#5 IN (129,270,821,423)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#2, 16, true, false, true) AS i_item_id#6, i_item_desc#3, i_current_price#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#9), dynamicpruningexpression(inv_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Condition : (((isnotnull(inv_quantity_on_hand#8) AND (inv_quantity_on_hand#8 >= 100)) AND (inv_quantity_on_hand#8 <= 500)) AND isnotnull(inv_item_sk#7)) - -(6) CometProject -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9], [inv_item_sk#7, inv_date_sk#9] - -(7) CometBroadcastExchange -Input [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1], [inv_item_sk#7], Inner, BuildRight - -(9) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#12 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) - -(12) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(14) CometBroadcastHashJoin -Left output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [inv_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(15) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(16) CometBroadcastExchange -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#13, ss_sold_date_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [ss_item_sk#13, ss_sold_date_sk#14] -Condition : isnotnull(ss_item_sk#13) - -(19) CometProject -Input [2]: [ss_item_sk#13, ss_sold_date_sk#14] -Arguments: [ss_item_sk#13], [ss_item_sk#13] - -(20) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [1]: [ss_item_sk#13] -Arguments: [i_item_sk#1], [ss_item_sk#13], Inner, BuildLeft - -(21) CometProject -Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, ss_item_sk#13] -Arguments: [i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(22) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(23) CometExchange -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(25) CometTakeOrderedAndProject -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(26) CometColumnarToRow [codegen id : 1] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#12 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) - -(29) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(31) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/extended.txt deleted file mode 100644 index 9fbb7893b8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/extended.txt +++ /dev/null @@ -1,34 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 28d160a4a8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometExchange [i_item_id,i_item_desc,i_current_price] #1 - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometProject [i_item_id,i_item_desc,i_current_price] - 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 [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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [ss_item_sk] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/explain.txt deleted file mode 100644 index cae143b42b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/explain.txt +++ /dev/null @@ -1,378 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (50) -+- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * HashAggregate (17) - : : +- * CometColumnarToRow (16) - : : +- CometColumnarExchange (15) - : : +- * HashAggregate (14) - : : +- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_returns (1) - : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (31) - : +- * HashAggregate (30) - : +- * CometColumnarToRow (29) - : +- CometColumnarExchange (28) - : +- * HashAggregate (27) - : +- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * Filter (20) - : : : +- * ColumnarToRow (19) - : : : +- Scan parquet spark_catalog.default.catalog_returns (18) - : : +- ReusedExchange (21) - : +- ReusedExchange (24) - +- BroadcastExchange (47) - +- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Filter (36) - : : +- * ColumnarToRow (35) - : : +- Scan parquet spark_catalog.default.web_returns (34) - : +- ReusedExchange (37) - +- ReusedExchange (40) - - -(1) Scan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(sr_returned_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(sr_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Condition : isnotnull(sr_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_item_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [i_item_sk#5, i_item_id#6] -Condition : (isnotnull(i_item_sk#5) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true))) - -(6) CometProject -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#7] - -(7) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_sk#5, i_item_id#7] - -(8) BroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [sr_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7] -Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#5, i_item_id#7] - -(11) ReusedExchange [Reuses operator id: 64] -Output [1]: [d_date_sk#8] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [sr_returned_date_sk#3] -Right keys [1]: [d_date_sk#8] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [2]: [sr_return_quantity#2, i_item_id#7] -Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7, d_date_sk#8] - -(14) HashAggregate [codegen id : 3] -Input [2]: [sr_return_quantity#2, i_item_id#7] -Keys [1]: [i_item_id#7] -Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#9] -Results [2]: [i_item_id#7, sum#10] - -(15) CometColumnarExchange -Input [2]: [i_item_id#7, sum#10] -Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 12] -Input [2]: [i_item_id#7, sum#10] - -(17) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#7, sum#10] -Keys [1]: [i_item_id#7] -Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#11] -Results [2]: [i_item_id#7 AS item_id#12, sum(sr_return_quantity#2)#11 AS sr_item_qty#13] - -(18) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#16), dynamicpruningexpression(cr_returned_date_sk#16 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cr_item_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 6] -Input [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] - -(20) Filter [codegen id : 6] -Input [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] -Condition : isnotnull(cr_item_sk#14) - -(21) ReusedExchange [Reuses operator id: 8] -Output [2]: [i_item_sk#17, i_item_id#18] - -(22) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_item_sk#14] -Right keys [1]: [i_item_sk#17] -Join type: Inner -Join condition: None - -(23) Project [codegen id : 6] -Output [3]: [cr_return_quantity#15, cr_returned_date_sk#16, i_item_id#18] -Input [5]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16, i_item_sk#17, i_item_id#18] - -(24) ReusedExchange [Reuses operator id: 64] -Output [1]: [d_date_sk#19] - -(25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returned_date_sk#16] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 6] -Output [2]: [cr_return_quantity#15, i_item_id#18] -Input [4]: [cr_return_quantity#15, cr_returned_date_sk#16, i_item_id#18, d_date_sk#19] - -(27) HashAggregate [codegen id : 6] -Input [2]: [cr_return_quantity#15, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [1]: [partial_sum(cr_return_quantity#15)] -Aggregate Attributes [1]: [sum#20] -Results [2]: [i_item_id#18, sum#21] - -(28) CometColumnarExchange -Input [2]: [i_item_id#18, sum#21] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(29) CometColumnarToRow [codegen id : 7] -Input [2]: [i_item_id#18, sum#21] - -(30) HashAggregate [codegen id : 7] -Input [2]: [i_item_id#18, sum#21] -Keys [1]: [i_item_id#18] -Functions [1]: [sum(cr_return_quantity#15)] -Aggregate Attributes [1]: [sum(cr_return_quantity#15)#22] -Results [2]: [i_item_id#18 AS item_id#23, sum(cr_return_quantity#15)#22 AS cr_item_qty#24] - -(31) BroadcastExchange -Input [2]: [item_id#23, cr_item_qty#24] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [item_id#12] -Right keys [1]: [item_id#23] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 12] -Output [3]: [item_id#12, sr_item_qty#13, cr_item_qty#24] -Input [4]: [item_id#12, sr_item_qty#13, item_id#23, cr_item_qty#24] - -(34) Scan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#27), dynamicpruningexpression(wr_returned_date_sk#27 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(wr_item_sk)] -ReadSchema: struct - -(35) ColumnarToRow [codegen id : 10] -Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] - -(36) Filter [codegen id : 10] -Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] -Condition : isnotnull(wr_item_sk#25) - -(37) ReusedExchange [Reuses operator id: 8] -Output [2]: [i_item_sk#28, i_item_id#29] - -(38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [wr_item_sk#25] -Right keys [1]: [i_item_sk#28] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 10] -Output [3]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#29] -Input [5]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27, i_item_sk#28, i_item_id#29] - -(40) ReusedExchange [Reuses operator id: 64] -Output [1]: [d_date_sk#30] - -(41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [wr_returned_date_sk#27] -Right keys [1]: [d_date_sk#30] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 10] -Output [2]: [wr_return_quantity#26, i_item_id#29] -Input [4]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#29, d_date_sk#30] - -(43) HashAggregate [codegen id : 10] -Input [2]: [wr_return_quantity#26, i_item_id#29] -Keys [1]: [i_item_id#29] -Functions [1]: [partial_sum(wr_return_quantity#26)] -Aggregate Attributes [1]: [sum#31] -Results [2]: [i_item_id#29, sum#32] - -(44) CometColumnarExchange -Input [2]: [i_item_id#29, sum#32] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 11] -Input [2]: [i_item_id#29, sum#32] - -(46) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#29, sum#32] -Keys [1]: [i_item_id#29] -Functions [1]: [sum(wr_return_quantity#26)] -Aggregate Attributes [1]: [sum(wr_return_quantity#26)#33] -Results [2]: [i_item_id#29 AS item_id#34, sum(wr_return_quantity#26)#33 AS wr_item_qty#35] - -(47) BroadcastExchange -Input [2]: [item_id#34, wr_item_qty#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] - -(48) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [item_id#12] -Right keys [1]: [item_id#34] -Join type: Inner -Join condition: None - -(49) Project [codegen id : 12] -Output [8]: [item_id#12, sr_item_qty#13, (((cast(sr_item_qty#13 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as double)))) / 3.0) * 100.0) AS sr_dev#36, cr_item_qty#24, (((cast(cr_item_qty#24 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as double)))) / 3.0) * 100.0) AS cr_dev#37, wr_item_qty#35, (((cast(wr_item_qty#35 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as double)))) / 3.0) * 100.0) AS wr_dev#38, (cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as decimal(20,0)) / 3.0) AS average#39] -Input [5]: [item_id#12, sr_item_qty#13, cr_item_qty#24, item_id#34, wr_item_qty#35] - -(50) TakeOrderedAndProject -Input [8]: [item_id#12, sr_item_qty#13, sr_dev#36, cr_item_qty#24, cr_dev#37, wr_item_qty#35, wr_dev#38, average#39] -Arguments: 100, [item_id#12 ASC NULLS FIRST, sr_item_qty#13 ASC NULLS FIRST], [item_id#12, sr_item_qty#13, sr_dev#36, cr_item_qty#24, cr_dev#37, wr_item_qty#35, wr_dev#38, average#39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (64) -+- * CometColumnarToRow (63) - +- CometProject (62) - +- CometBroadcastHashJoin (61) - :- CometFilter (52) - : +- CometNativeScan parquet spark_catalog.default.date_dim (51) - +- CometBroadcastExchange (60) - +- CometProject (59) - +- CometBroadcastHashJoin (58) - :- CometNativeScan parquet spark_catalog.default.date_dim (53) - +- CometBroadcastExchange (57) - +- CometProject (56) - +- CometFilter (55) - +- CometNativeScan parquet spark_catalog.default.date_dim (54) - - -(51) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(52) CometFilter -Input [2]: [d_date_sk#8, d_date#40] -Condition : isnotnull(d_date_sk#8) - -(53) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date#41, d_week_seq#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(54) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(55) CometFilter -Input [2]: [d_date#43, d_week_seq#44] -Condition : cast(d_date#43 as string) IN (2000-06-30,2000-09-27,2000-11-17) - -(56) CometProject -Input [2]: [d_date#43, d_week_seq#44] -Arguments: [d_week_seq#44], [d_week_seq#44] - -(57) CometBroadcastExchange -Input [1]: [d_week_seq#44] -Arguments: [d_week_seq#44] - -(58) CometBroadcastHashJoin -Left output [2]: [d_date#41, d_week_seq#42] -Right output [1]: [d_week_seq#44] -Arguments: [d_week_seq#42], [d_week_seq#44], LeftSemi, BuildRight - -(59) CometProject -Input [2]: [d_date#41, d_week_seq#42] -Arguments: [d_date#41], [d_date#41] - -(60) CometBroadcastExchange -Input [1]: [d_date#41] -Arguments: [d_date#41] - -(61) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#40] -Right output [1]: [d_date#41] -Arguments: [d_date#40], [d_date#41], LeftSemi, BuildRight - -(62) CometProject -Input [2]: [d_date_sk#8, d_date#40] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(63) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(64) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 18 Hosting Expression = cr_returned_date_sk#16 IN dynamicpruning#4 - -Subquery:3 Hosting operator id = 34 Hosting Expression = wr_returned_date_sk#27 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/simplified.txt deleted file mode 100644 index 0673c590f5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/simplified.txt +++ /dev/null @@ -1,91 +0,0 @@ -TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - WholeStageCodegen (12) - Project [item_id,sr_item_qty,cr_item_qty,wr_item_qty] - BroadcastHashJoin [item_id,item_id] - Project [item_id,sr_item_qty,cr_item_qty] - BroadcastHashJoin [item_id,item_id] - HashAggregate [i_item_id,sum] [sum(sr_return_quantity),item_id,sr_item_qty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (3) - HashAggregate [i_item_id,sr_return_quantity] [sum,sum] - Project [sr_return_quantity,i_item_id] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [sr_return_quantity,sr_returned_date_sk,i_item_id] - BroadcastHashJoin [sr_item_sk,i_item_sk] - Filter [sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - HashAggregate [i_item_id,sum] [sum(cr_return_quantity),item_id,cr_item_qty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #7 - WholeStageCodegen (6) - HashAggregate [i_item_id,cr_return_quantity] [sum,sum] - Project [cr_return_quantity,i_item_id] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Project [cr_return_quantity,cr_returned_date_sk,i_item_id] - BroadcastHashJoin [cr_item_sk,i_item_sk] - Filter [cr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #9 - WholeStageCodegen (10) - HashAggregate [i_item_id,wr_return_quantity] [sum,sum] - Project [wr_return_quantity,i_item_id] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Project [wr_return_quantity,wr_returned_date_sk,i_item_id] - BroadcastHashJoin [wr_item_sk,i_item_sk] - Filter [wr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - InputAdapter - ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/explain.txt deleted file mode 100644 index 515e2540f7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,363 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (49) -+- CometTakeOrderedAndProject (48) - +- CometProject (47) - +- CometBroadcastHashJoin (46) - :- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometHashAggregate (19) - : : +- CometExchange (18) - : : +- CometHashAggregate (17) - : : +- CometProject (16) - : : +- CometBroadcastHashJoin (15) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (14) - : : +- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- ReusedExchange (11) - : +- CometBroadcastExchange (31) - : +- CometHashAggregate (30) - : +- CometExchange (29) - : +- CometHashAggregate (28) - : +- CometProject (27) - : +- CometBroadcastHashJoin (26) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometFilter (21) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (20) - : : +- ReusedExchange (22) - : +- ReusedExchange (25) - +- CometBroadcastExchange (45) - +- CometHashAggregate (44) - +- CometExchange (43) - +- CometHashAggregate (42) - +- CometProject (41) - +- CometBroadcastHashJoin (40) - :- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometFilter (35) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (34) - : +- ReusedExchange (36) - +- ReusedExchange (39) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(sr_returned_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(sr_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Condition : isnotnull(sr_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_item_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_item_id#6] -Condition : (isnotnull(i_item_sk#5) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true))) - -(5) CometProject -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#7] - -(6) CometBroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#7] -Arguments: [i_item_sk#5, i_item_id#7] - -(7) CometBroadcastHashJoin -Left output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Right output [2]: [i_item_sk#5, i_item_id#7] -Arguments: [sr_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#5, i_item_id#7] -Arguments: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7], [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(11) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date#10] - -(12) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#10] -Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight - -(13) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: [d_date_sk#8] - -(15) CometBroadcastHashJoin -Left output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7] -Right output [1]: [d_date_sk#8] -Arguments: [sr_returned_date_sk#3], [d_date_sk#8], Inner, BuildRight - -(16) CometProject -Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7, d_date_sk#8] -Arguments: [sr_return_quantity#2, i_item_id#7], [sr_return_quantity#2, i_item_id#7] - -(17) CometHashAggregate -Input [2]: [sr_return_quantity#2, i_item_id#7] -Keys [1]: [i_item_id#7] -Functions [1]: [partial_sum(sr_return_quantity#2)] - -(18) CometExchange -Input [2]: [i_item_id#7, sum#11] -Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [2]: [i_item_id#7, sum#11] -Keys [1]: [i_item_id#7] -Functions [1]: [sum(sr_return_quantity#2)] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#14), dynamicpruningexpression(cr_returned_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(cr_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] -Condition : isnotnull(cr_item_sk#12) - -(22) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#16, i_item_id#17] - -(23) CometBroadcastHashJoin -Left output [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] -Right output [2]: [i_item_sk#16, i_item_id#17] -Arguments: [cr_item_sk#12], [i_item_sk#16], Inner, BuildRight - -(24) CometProject -Input [5]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14, i_item_sk#16, i_item_id#17] -Arguments: [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17], [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17] - -(25) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#18] - -(26) CometBroadcastHashJoin -Left output [3]: [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17] -Right output [1]: [d_date_sk#18] -Arguments: [cr_returned_date_sk#14], [d_date_sk#18], Inner, BuildRight - -(27) CometProject -Input [4]: [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17, d_date_sk#18] -Arguments: [cr_return_quantity#13, i_item_id#17], [cr_return_quantity#13, i_item_id#17] - -(28) CometHashAggregate -Input [2]: [cr_return_quantity#13, i_item_id#17] -Keys [1]: [i_item_id#17] -Functions [1]: [partial_sum(cr_return_quantity#13)] - -(29) CometExchange -Input [2]: [i_item_id#17, sum#19] -Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(30) CometHashAggregate -Input [2]: [i_item_id#17, sum#19] -Keys [1]: [i_item_id#17] -Functions [1]: [sum(cr_return_quantity#13)] - -(31) CometBroadcastExchange -Input [2]: [item_id#20, cr_item_qty#21] -Arguments: [item_id#20, cr_item_qty#21] - -(32) CometBroadcastHashJoin -Left output [2]: [item_id#22, sr_item_qty#23] -Right output [2]: [item_id#20, cr_item_qty#21] -Arguments: [item_id#22], [item_id#20], Inner, BuildRight - -(33) CometProject -Input [4]: [item_id#22, sr_item_qty#23, item_id#20, cr_item_qty#21] -Arguments: [item_id#22, sr_item_qty#23, cr_item_qty#21], [item_id#22, sr_item_qty#23, cr_item_qty#21] - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#24, wr_return_quantity#25, wr_returned_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#26), dynamicpruningexpression(wr_returned_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(wr_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [wr_item_sk#24, wr_return_quantity#25, wr_returned_date_sk#26] -Condition : isnotnull(wr_item_sk#24) - -(36) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#28, i_item_id#29] - -(37) CometBroadcastHashJoin -Left output [3]: [wr_item_sk#24, wr_return_quantity#25, wr_returned_date_sk#26] -Right output [2]: [i_item_sk#28, i_item_id#29] -Arguments: [wr_item_sk#24], [i_item_sk#28], Inner, BuildRight - -(38) CometProject -Input [5]: [wr_item_sk#24, wr_return_quantity#25, wr_returned_date_sk#26, i_item_sk#28, i_item_id#29] -Arguments: [wr_return_quantity#25, wr_returned_date_sk#26, i_item_id#29], [wr_return_quantity#25, wr_returned_date_sk#26, i_item_id#29] - -(39) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#30] - -(40) CometBroadcastHashJoin -Left output [3]: [wr_return_quantity#25, wr_returned_date_sk#26, i_item_id#29] -Right output [1]: [d_date_sk#30] -Arguments: [wr_returned_date_sk#26], [d_date_sk#30], Inner, BuildRight - -(41) CometProject -Input [4]: [wr_return_quantity#25, wr_returned_date_sk#26, i_item_id#29, d_date_sk#30] -Arguments: [wr_return_quantity#25, i_item_id#29], [wr_return_quantity#25, i_item_id#29] - -(42) CometHashAggregate -Input [2]: [wr_return_quantity#25, i_item_id#29] -Keys [1]: [i_item_id#29] -Functions [1]: [partial_sum(wr_return_quantity#25)] - -(43) CometExchange -Input [2]: [i_item_id#29, sum#31] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(44) CometHashAggregate -Input [2]: [i_item_id#29, sum#31] -Keys [1]: [i_item_id#29] -Functions [1]: [sum(wr_return_quantity#25)] - -(45) CometBroadcastExchange -Input [2]: [item_id#32, wr_item_qty#33] -Arguments: [item_id#32, wr_item_qty#33] - -(46) CometBroadcastHashJoin -Left output [3]: [item_id#22, sr_item_qty#23, cr_item_qty#21] -Right output [2]: [item_id#32, wr_item_qty#33] -Arguments: [item_id#22], [item_id#32], Inner, BuildRight - -(47) CometProject -Input [5]: [item_id#22, sr_item_qty#23, cr_item_qty#21, item_id#32, wr_item_qty#33] -Arguments: [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37], [item_id#22, sr_item_qty#23, (((cast(sr_item_qty#23 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#23 + cr_item_qty#21) + wr_item_qty#33) as double)))) / 3.0) * 100.0) AS sr_dev#34, cr_item_qty#21, (((cast(cr_item_qty#21 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#23 + cr_item_qty#21) + wr_item_qty#33) as double)))) / 3.0) * 100.0) AS cr_dev#35, wr_item_qty#33, (((cast(wr_item_qty#33 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#23 + cr_item_qty#21) + wr_item_qty#33) as double)))) / 3.0) * 100.0) AS wr_dev#36, (cast(((sr_item_qty#23 + cr_item_qty#21) + wr_item_qty#33) as decimal(20,0)) / 3.0) AS average#37] - -(48) CometTakeOrderedAndProject -Input [8]: [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_id#22 ASC NULLS FIRST,sr_item_qty#23 ASC NULLS FIRST], output=[item_id#22,sr_item_qty#23,sr_dev#34,cr_item_qty#21,cr_dev#35,wr_item_qty#33,wr_dev#36,average#37]), [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37], 100, 0, [item_id#22 ASC NULLS FIRST, sr_item_qty#23 ASC NULLS FIRST], [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37] - -(49) CometColumnarToRow [codegen id : 1] -Input [8]: [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (63) -+- * CometColumnarToRow (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometFilter (51) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) - +- CometBroadcastExchange (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) - +- CometBroadcastExchange (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#10, d_week_seq#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#39, d_week_seq#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(54) CometFilter -Input [2]: [d_date#39, d_week_seq#40] -Condition : cast(d_date#39 as string) IN (2000-06-30,2000-09-27,2000-11-17) - -(55) CometProject -Input [2]: [d_date#39, d_week_seq#40] -Arguments: [d_week_seq#40], [d_week_seq#40] - -(56) CometBroadcastExchange -Input [1]: [d_week_seq#40] -Arguments: [d_week_seq#40] - -(57) CometBroadcastHashJoin -Left output [2]: [d_date#10, d_week_seq#38] -Right output [1]: [d_week_seq#40] -Arguments: [d_week_seq#38], [d_week_seq#40], LeftSemi, BuildRight - -(58) CometProject -Input [2]: [d_date#10, d_week_seq#38] -Arguments: [d_date#10], [d_date#10] - -(59) CometBroadcastExchange -Input [1]: [d_date#10] -Arguments: [d_date#10] - -(60) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#10] -Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight - -(61) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(62) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(63) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 20 Hosting Expression = cr_returned_date_sk#14 IN dynamicpruning#4 - -Subquery:3 Hosting operator id = 34 Hosting Expression = wr_returned_date_sk#26 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/simplified.txt deleted file mode 100644 index d016a5ee2d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,70 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - CometProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - CometBroadcastHashJoin [item_id,sr_item_qty,cr_item_qty,item_id,wr_item_qty] - CometProject [item_id,sr_item_qty,cr_item_qty] - CometBroadcastHashJoin [item_id,sr_item_qty,item_id,cr_item_qty] - CometHashAggregate [sum] [item_id,sr_item_qty,i_item_id,sum(sr_return_quantity)] - CometExchange [i_item_id] #1 - CometHashAggregate [sr_return_quantity] [i_item_id,sum] - CometProject [sr_return_quantity,i_item_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #3 - CometBroadcastExchange [item_id,cr_item_qty] #7 - CometHashAggregate [sum] [item_id,cr_item_qty,i_item_id,sum(cr_return_quantity)] - CometExchange [i_item_id] #8 - CometHashAggregate [cr_return_quantity] [i_item_id,sum] - CometProject [cr_return_quantity,i_item_id] - 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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [item_id,wr_item_qty] #9 - CometHashAggregate [sum] [item_id,wr_item_qty,i_item_id,sum(wr_return_quantity)] - CometExchange [i_item_id] #10 - CometHashAggregate [wr_return_quantity] [i_item_id,sum] - CometProject [wr_return_quantity,i_item_id] - 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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/explain.txt deleted file mode 100644 index 515e2540f7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/explain.txt +++ /dev/null @@ -1,363 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (49) -+- CometTakeOrderedAndProject (48) - +- CometProject (47) - +- CometBroadcastHashJoin (46) - :- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometHashAggregate (19) - : : +- CometExchange (18) - : : +- CometHashAggregate (17) - : : +- CometProject (16) - : : +- CometBroadcastHashJoin (15) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (14) - : : +- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- ReusedExchange (11) - : +- CometBroadcastExchange (31) - : +- CometHashAggregate (30) - : +- CometExchange (29) - : +- CometHashAggregate (28) - : +- CometProject (27) - : +- CometBroadcastHashJoin (26) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometFilter (21) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (20) - : : +- ReusedExchange (22) - : +- ReusedExchange (25) - +- CometBroadcastExchange (45) - +- CometHashAggregate (44) - +- CometExchange (43) - +- CometHashAggregate (42) - +- CometProject (41) - +- CometBroadcastHashJoin (40) - :- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometFilter (35) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (34) - : +- ReusedExchange (36) - +- ReusedExchange (39) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(sr_returned_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(sr_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Condition : isnotnull(sr_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_item_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_item_id#6] -Condition : (isnotnull(i_item_sk#5) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true))) - -(5) CometProject -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#7] - -(6) CometBroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#7] -Arguments: [i_item_sk#5, i_item_id#7] - -(7) CometBroadcastHashJoin -Left output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Right output [2]: [i_item_sk#5, i_item_id#7] -Arguments: [sr_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#5, i_item_id#7] -Arguments: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7], [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(11) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date#10] - -(12) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#10] -Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight - -(13) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: [d_date_sk#8] - -(15) CometBroadcastHashJoin -Left output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7] -Right output [1]: [d_date_sk#8] -Arguments: [sr_returned_date_sk#3], [d_date_sk#8], Inner, BuildRight - -(16) CometProject -Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7, d_date_sk#8] -Arguments: [sr_return_quantity#2, i_item_id#7], [sr_return_quantity#2, i_item_id#7] - -(17) CometHashAggregate -Input [2]: [sr_return_quantity#2, i_item_id#7] -Keys [1]: [i_item_id#7] -Functions [1]: [partial_sum(sr_return_quantity#2)] - -(18) CometExchange -Input [2]: [i_item_id#7, sum#11] -Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [2]: [i_item_id#7, sum#11] -Keys [1]: [i_item_id#7] -Functions [1]: [sum(sr_return_quantity#2)] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#14), dynamicpruningexpression(cr_returned_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(cr_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] -Condition : isnotnull(cr_item_sk#12) - -(22) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#16, i_item_id#17] - -(23) CometBroadcastHashJoin -Left output [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] -Right output [2]: [i_item_sk#16, i_item_id#17] -Arguments: [cr_item_sk#12], [i_item_sk#16], Inner, BuildRight - -(24) CometProject -Input [5]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14, i_item_sk#16, i_item_id#17] -Arguments: [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17], [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17] - -(25) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#18] - -(26) CometBroadcastHashJoin -Left output [3]: [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17] -Right output [1]: [d_date_sk#18] -Arguments: [cr_returned_date_sk#14], [d_date_sk#18], Inner, BuildRight - -(27) CometProject -Input [4]: [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17, d_date_sk#18] -Arguments: [cr_return_quantity#13, i_item_id#17], [cr_return_quantity#13, i_item_id#17] - -(28) CometHashAggregate -Input [2]: [cr_return_quantity#13, i_item_id#17] -Keys [1]: [i_item_id#17] -Functions [1]: [partial_sum(cr_return_quantity#13)] - -(29) CometExchange -Input [2]: [i_item_id#17, sum#19] -Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(30) CometHashAggregate -Input [2]: [i_item_id#17, sum#19] -Keys [1]: [i_item_id#17] -Functions [1]: [sum(cr_return_quantity#13)] - -(31) CometBroadcastExchange -Input [2]: [item_id#20, cr_item_qty#21] -Arguments: [item_id#20, cr_item_qty#21] - -(32) CometBroadcastHashJoin -Left output [2]: [item_id#22, sr_item_qty#23] -Right output [2]: [item_id#20, cr_item_qty#21] -Arguments: [item_id#22], [item_id#20], Inner, BuildRight - -(33) CometProject -Input [4]: [item_id#22, sr_item_qty#23, item_id#20, cr_item_qty#21] -Arguments: [item_id#22, sr_item_qty#23, cr_item_qty#21], [item_id#22, sr_item_qty#23, cr_item_qty#21] - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#24, wr_return_quantity#25, wr_returned_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#26), dynamicpruningexpression(wr_returned_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(wr_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [wr_item_sk#24, wr_return_quantity#25, wr_returned_date_sk#26] -Condition : isnotnull(wr_item_sk#24) - -(36) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#28, i_item_id#29] - -(37) CometBroadcastHashJoin -Left output [3]: [wr_item_sk#24, wr_return_quantity#25, wr_returned_date_sk#26] -Right output [2]: [i_item_sk#28, i_item_id#29] -Arguments: [wr_item_sk#24], [i_item_sk#28], Inner, BuildRight - -(38) CometProject -Input [5]: [wr_item_sk#24, wr_return_quantity#25, wr_returned_date_sk#26, i_item_sk#28, i_item_id#29] -Arguments: [wr_return_quantity#25, wr_returned_date_sk#26, i_item_id#29], [wr_return_quantity#25, wr_returned_date_sk#26, i_item_id#29] - -(39) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#30] - -(40) CometBroadcastHashJoin -Left output [3]: [wr_return_quantity#25, wr_returned_date_sk#26, i_item_id#29] -Right output [1]: [d_date_sk#30] -Arguments: [wr_returned_date_sk#26], [d_date_sk#30], Inner, BuildRight - -(41) CometProject -Input [4]: [wr_return_quantity#25, wr_returned_date_sk#26, i_item_id#29, d_date_sk#30] -Arguments: [wr_return_quantity#25, i_item_id#29], [wr_return_quantity#25, i_item_id#29] - -(42) CometHashAggregate -Input [2]: [wr_return_quantity#25, i_item_id#29] -Keys [1]: [i_item_id#29] -Functions [1]: [partial_sum(wr_return_quantity#25)] - -(43) CometExchange -Input [2]: [i_item_id#29, sum#31] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(44) CometHashAggregate -Input [2]: [i_item_id#29, sum#31] -Keys [1]: [i_item_id#29] -Functions [1]: [sum(wr_return_quantity#25)] - -(45) CometBroadcastExchange -Input [2]: [item_id#32, wr_item_qty#33] -Arguments: [item_id#32, wr_item_qty#33] - -(46) CometBroadcastHashJoin -Left output [3]: [item_id#22, sr_item_qty#23, cr_item_qty#21] -Right output [2]: [item_id#32, wr_item_qty#33] -Arguments: [item_id#22], [item_id#32], Inner, BuildRight - -(47) CometProject -Input [5]: [item_id#22, sr_item_qty#23, cr_item_qty#21, item_id#32, wr_item_qty#33] -Arguments: [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37], [item_id#22, sr_item_qty#23, (((cast(sr_item_qty#23 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#23 + cr_item_qty#21) + wr_item_qty#33) as double)))) / 3.0) * 100.0) AS sr_dev#34, cr_item_qty#21, (((cast(cr_item_qty#21 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#23 + cr_item_qty#21) + wr_item_qty#33) as double)))) / 3.0) * 100.0) AS cr_dev#35, wr_item_qty#33, (((cast(wr_item_qty#33 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#23 + cr_item_qty#21) + wr_item_qty#33) as double)))) / 3.0) * 100.0) AS wr_dev#36, (cast(((sr_item_qty#23 + cr_item_qty#21) + wr_item_qty#33) as decimal(20,0)) / 3.0) AS average#37] - -(48) CometTakeOrderedAndProject -Input [8]: [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_id#22 ASC NULLS FIRST,sr_item_qty#23 ASC NULLS FIRST], output=[item_id#22,sr_item_qty#23,sr_dev#34,cr_item_qty#21,cr_dev#35,wr_item_qty#33,wr_dev#36,average#37]), [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37], 100, 0, [item_id#22 ASC NULLS FIRST, sr_item_qty#23 ASC NULLS FIRST], [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37] - -(49) CometColumnarToRow [codegen id : 1] -Input [8]: [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (63) -+- * CometColumnarToRow (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometFilter (51) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) - +- CometBroadcastExchange (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) - +- CometBroadcastExchange (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#10, d_week_seq#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#39, d_week_seq#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(54) CometFilter -Input [2]: [d_date#39, d_week_seq#40] -Condition : cast(d_date#39 as string) IN (2000-06-30,2000-09-27,2000-11-17) - -(55) CometProject -Input [2]: [d_date#39, d_week_seq#40] -Arguments: [d_week_seq#40], [d_week_seq#40] - -(56) CometBroadcastExchange -Input [1]: [d_week_seq#40] -Arguments: [d_week_seq#40] - -(57) CometBroadcastHashJoin -Left output [2]: [d_date#10, d_week_seq#38] -Right output [1]: [d_week_seq#40] -Arguments: [d_week_seq#38], [d_week_seq#40], LeftSemi, BuildRight - -(58) CometProject -Input [2]: [d_date#10, d_week_seq#38] -Arguments: [d_date#10], [d_date#10] - -(59) CometBroadcastExchange -Input [1]: [d_date#10] -Arguments: [d_date#10] - -(60) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#10] -Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight - -(61) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(62) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(63) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 20 Hosting Expression = cr_returned_date_sk#14 IN dynamicpruning#4 - -Subquery:3 Hosting operator id = 34 Hosting Expression = wr_returned_date_sk#26 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/extended.txt deleted file mode 100644 index 924b12cf01..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 97 out of 101 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d016a5ee2d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/simplified.txt +++ /dev/null @@ -1,70 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - CometProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - CometBroadcastHashJoin [item_id,sr_item_qty,cr_item_qty,item_id,wr_item_qty] - CometProject [item_id,sr_item_qty,cr_item_qty] - CometBroadcastHashJoin [item_id,sr_item_qty,item_id,cr_item_qty] - CometHashAggregate [sum] [item_id,sr_item_qty,i_item_id,sum(sr_return_quantity)] - CometExchange [i_item_id] #1 - CometHashAggregate [sr_return_quantity] [i_item_id,sum] - CometProject [sr_return_quantity,i_item_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #3 - CometBroadcastExchange [item_id,cr_item_qty] #7 - CometHashAggregate [sum] [item_id,cr_item_qty,i_item_id,sum(cr_return_quantity)] - CometExchange [i_item_id] #8 - CometHashAggregate [cr_return_quantity] [i_item_id,sum] - CometProject [cr_return_quantity,i_item_id] - 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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [item_id,wr_item_qty] #9 - CometHashAggregate [sum] [item_id,wr_item_qty,i_item_id,sum(wr_return_quantity)] - CometExchange [i_item_id] #10 - CometHashAggregate [wr_return_quantity] [i_item_id,sum] - CometProject [wr_return_quantity,i_item_id] - 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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_datafusion/explain.txt deleted file mode 100644 index 63140cfc2d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_datafusion/explain.txt +++ /dev/null @@ -1,190 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometTakeOrderedAndProject (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometFilter (11) - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (10) - : : +- CometBroadcastExchange (17) - : : +- CometFilter (16) - : : +- CometNativeScan parquet spark_catalog.default.household_demographics (15) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometNativeScan parquet spark_catalog.default.income_band (20) - +- CometProject (29) - +- CometFilter (28) - +- CometNativeScan parquet spark_catalog.default.store_returns (27) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Condition : ((isnotnull(c_current_addr_sk#4) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_current_hdemo_sk#3)) - -(3) CometProject -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Arguments: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#1, 16, true, false, true) AS c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#5, 20, true, false, true) AS c_first_name#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#6, 30, true, false, true) AS c_last_name#9] - -(4) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#10, ca_city#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_city), EqualTo(ca_city,Edgewood), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [ca_address_sk#10, ca_city#11] -Condition : ((isnotnull(ca_city#11) AND (ca_city#11 = Edgewood)) AND isnotnull(ca_address_sk#10)) - -(6) CometProject -Input [2]: [ca_address_sk#10, ca_city#11] -Arguments: [ca_address_sk#10], [ca_address_sk#10] - -(7) CometBroadcastExchange -Input [1]: [ca_address_sk#10] -Arguments: [ca_address_sk#10] - -(8) CometBroadcastHashJoin -Left output [6]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9] -Right output [1]: [ca_address_sk#10] -Arguments: [c_current_addr_sk#4], [ca_address_sk#10], Inner, BuildRight - -(9) CometProject -Input [7]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9, ca_address_sk#10] -Arguments: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9], [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9] - -(10) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(11) CometFilter -Input [1]: [cd_demo_sk#12] -Condition : isnotnull(cd_demo_sk#12) - -(12) CometBroadcastExchange -Input [1]: [cd_demo_sk#12] -Arguments: [cd_demo_sk#12] - -(13) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9] -Right output [1]: [cd_demo_sk#12] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#12], Inner, BuildRight - -(14) CometProject -Input [6]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Arguments: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12], [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(15) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Condition : (isnotnull(hd_demo_sk#13) AND isnotnull(hd_income_band_sk#14)) - -(17) CometBroadcastExchange -Input [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [hd_demo_sk#13, hd_income_band_sk#14] - -(18) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Right output [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [c_current_hdemo_sk#3], [hd_demo_sk#13], Inner, BuildRight - -(19) CometProject -Input [7]: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14], [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14] - -(20) CometNativeScan parquet spark_catalog.default.income_band -Output [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_lower_bound), IsNotNull(ib_upper_bound), GreaterThanOrEqual(ib_lower_bound,38128), LessThanOrEqual(ib_upper_bound,88128), IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Condition : ((((isnotnull(ib_lower_bound#16) AND isnotnull(ib_upper_bound#17)) AND (ib_lower_bound#16 >= 38128)) AND (ib_upper_bound#17 <= 88128)) AND isnotnull(ib_income_band_sk#15)) - -(22) CometProject -Input [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Arguments: [ib_income_band_sk#15], [ib_income_band_sk#15] - -(23) CometBroadcastExchange -Input [1]: [ib_income_band_sk#15] -Arguments: [ib_income_band_sk#15] - -(24) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14] -Right output [1]: [ib_income_band_sk#15] -Arguments: [hd_income_band_sk#14], [ib_income_band_sk#15], Inner, BuildRight - -(25) CometProject -Input [6]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14, ib_income_band_sk#15] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12], [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(26) CometBroadcastExchange -Input [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(27) CometNativeScan parquet spark_catalog.default.store_returns -Output [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_cdemo_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_cdemo_sk#18) - -(29) CometProject -Input [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Arguments: [sr_cdemo_sk#18], [sr_cdemo_sk#18] - -(30) CometBroadcastHashJoin -Left output [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Right output [1]: [sr_cdemo_sk#18] -Arguments: [cd_demo_sk#12], [sr_cdemo_sk#18], Inner, BuildLeft - -(31) CometProject -Input [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, sr_cdemo_sk#18] -Arguments: [customer_id#20, customername#21, c_customer_id#7], [c_customer_id#7 AS customer_id#20, concat(c_last_name#9, , , c_first_name#8) AS customername#21, c_customer_id#7] - -(32) CometTakeOrderedAndProject -Input [3]: [customer_id#20, customername#21, c_customer_id#7] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#7 ASC NULLS FIRST], output=[customer_id#20,customername#21]), [customer_id#20, customername#21], 100, 0, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [customer_id#20, customername#21] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_datafusion/simplified.txt deleted file mode 100644 index 9df82b0e98..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_datafusion/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id] [customer_id,customername] - CometProject [c_last_name,c_first_name] [customer_id,customername,c_customer_id] - 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 [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_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_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_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometNativeScan 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 [ca_address_sk] #2 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_city] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [cd_demo_sk] #3 - CometFilter [cd_demo_sk] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange [ib_income_band_sk] #5 - CometProject [ib_income_band_sk] - CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk,sr_returned_date_sk] - CometNativeScan 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/q84.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_iceberg_compat/explain.txt deleted file mode 100644 index 5bafdc5586..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,190 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometTakeOrderedAndProject (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometFilter (11) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (10) - : : +- CometBroadcastExchange (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (20) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (27) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Condition : ((isnotnull(c_current_addr_sk#4) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_current_hdemo_sk#3)) - -(3) CometProject -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Arguments: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#1, 16, true, false, true) AS c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#5, 20, true, false, true) AS c_first_name#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#6, 30, true, false, true) AS c_last_name#9] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#10, ca_city#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_city), EqualTo(ca_city,Edgewood), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [ca_address_sk#10, ca_city#11] -Condition : ((isnotnull(ca_city#11) AND (ca_city#11 = Edgewood)) AND isnotnull(ca_address_sk#10)) - -(6) CometProject -Input [2]: [ca_address_sk#10, ca_city#11] -Arguments: [ca_address_sk#10], [ca_address_sk#10] - -(7) CometBroadcastExchange -Input [1]: [ca_address_sk#10] -Arguments: [ca_address_sk#10] - -(8) CometBroadcastHashJoin -Left output [6]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9] -Right output [1]: [ca_address_sk#10] -Arguments: [c_current_addr_sk#4], [ca_address_sk#10], Inner, BuildRight - -(9) CometProject -Input [7]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9, ca_address_sk#10] -Arguments: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9], [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(11) CometFilter -Input [1]: [cd_demo_sk#12] -Condition : isnotnull(cd_demo_sk#12) - -(12) CometBroadcastExchange -Input [1]: [cd_demo_sk#12] -Arguments: [cd_demo_sk#12] - -(13) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9] -Right output [1]: [cd_demo_sk#12] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#12], Inner, BuildRight - -(14) CometProject -Input [6]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Arguments: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12], [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Condition : (isnotnull(hd_demo_sk#13) AND isnotnull(hd_income_band_sk#14)) - -(17) CometBroadcastExchange -Input [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [hd_demo_sk#13, hd_income_band_sk#14] - -(18) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Right output [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [c_current_hdemo_sk#3], [hd_demo_sk#13], Inner, BuildRight - -(19) CometProject -Input [7]: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14], [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band -Output [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_lower_bound), IsNotNull(ib_upper_bound), GreaterThanOrEqual(ib_lower_bound,38128), LessThanOrEqual(ib_upper_bound,88128), IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Condition : ((((isnotnull(ib_lower_bound#16) AND isnotnull(ib_upper_bound#17)) AND (ib_lower_bound#16 >= 38128)) AND (ib_upper_bound#17 <= 88128)) AND isnotnull(ib_income_band_sk#15)) - -(22) CometProject -Input [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Arguments: [ib_income_band_sk#15], [ib_income_band_sk#15] - -(23) CometBroadcastExchange -Input [1]: [ib_income_band_sk#15] -Arguments: [ib_income_band_sk#15] - -(24) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14] -Right output [1]: [ib_income_band_sk#15] -Arguments: [hd_income_band_sk#14], [ib_income_band_sk#15], Inner, BuildRight - -(25) CometProject -Input [6]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14, ib_income_band_sk#15] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12], [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(26) CometBroadcastExchange -Input [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_cdemo_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_cdemo_sk#18) - -(29) CometProject -Input [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Arguments: [sr_cdemo_sk#18], [sr_cdemo_sk#18] - -(30) CometBroadcastHashJoin -Left output [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Right output [1]: [sr_cdemo_sk#18] -Arguments: [cd_demo_sk#12], [sr_cdemo_sk#18], Inner, BuildLeft - -(31) CometProject -Input [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, sr_cdemo_sk#18] -Arguments: [customer_id#20, customername#21, c_customer_id#7], [c_customer_id#7 AS customer_id#20, concat(c_last_name#9, , , c_first_name#8) AS customername#21, c_customer_id#7] - -(32) CometTakeOrderedAndProject -Input [3]: [customer_id#20, customername#21, c_customer_id#7] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#7 ASC NULLS FIRST], output=[customer_id#20,customername#21]), [customer_id#20, customername#21], 100, 0, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [customer_id#20, customername#21] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_iceberg_compat/simplified.txt deleted file mode 100644 index e43557c27d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id] [customer_id,customername] - CometProject [c_last_name,c_first_name] [customer_id,customername,c_customer_id] - 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 [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_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_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_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] 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 [ca_address_sk] #2 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [cd_demo_sk] #3 - CometFilter [cd_demo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange [ib_income_band_sk] #5 - CometProject [ib_income_band_sk] - CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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/q84/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/explain.txt deleted file mode 100644 index 5bafdc5586..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/explain.txt +++ /dev/null @@ -1,190 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometTakeOrderedAndProject (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometFilter (11) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (10) - : : +- CometBroadcastExchange (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (20) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (27) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Condition : ((isnotnull(c_current_addr_sk#4) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_current_hdemo_sk#3)) - -(3) CometProject -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Arguments: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#1, 16, true, false, true) AS c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#5, 20, true, false, true) AS c_first_name#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#6, 30, true, false, true) AS c_last_name#9] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#10, ca_city#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_city), EqualTo(ca_city,Edgewood), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [ca_address_sk#10, ca_city#11] -Condition : ((isnotnull(ca_city#11) AND (ca_city#11 = Edgewood)) AND isnotnull(ca_address_sk#10)) - -(6) CometProject -Input [2]: [ca_address_sk#10, ca_city#11] -Arguments: [ca_address_sk#10], [ca_address_sk#10] - -(7) CometBroadcastExchange -Input [1]: [ca_address_sk#10] -Arguments: [ca_address_sk#10] - -(8) CometBroadcastHashJoin -Left output [6]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9] -Right output [1]: [ca_address_sk#10] -Arguments: [c_current_addr_sk#4], [ca_address_sk#10], Inner, BuildRight - -(9) CometProject -Input [7]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9, ca_address_sk#10] -Arguments: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9], [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(11) CometFilter -Input [1]: [cd_demo_sk#12] -Condition : isnotnull(cd_demo_sk#12) - -(12) CometBroadcastExchange -Input [1]: [cd_demo_sk#12] -Arguments: [cd_demo_sk#12] - -(13) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9] -Right output [1]: [cd_demo_sk#12] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#12], Inner, BuildRight - -(14) CometProject -Input [6]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Arguments: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12], [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Condition : (isnotnull(hd_demo_sk#13) AND isnotnull(hd_income_band_sk#14)) - -(17) CometBroadcastExchange -Input [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [hd_demo_sk#13, hd_income_band_sk#14] - -(18) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Right output [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [c_current_hdemo_sk#3], [hd_demo_sk#13], Inner, BuildRight - -(19) CometProject -Input [7]: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14], [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band -Output [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_lower_bound), IsNotNull(ib_upper_bound), GreaterThanOrEqual(ib_lower_bound,38128), LessThanOrEqual(ib_upper_bound,88128), IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Condition : ((((isnotnull(ib_lower_bound#16) AND isnotnull(ib_upper_bound#17)) AND (ib_lower_bound#16 >= 38128)) AND (ib_upper_bound#17 <= 88128)) AND isnotnull(ib_income_band_sk#15)) - -(22) CometProject -Input [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Arguments: [ib_income_band_sk#15], [ib_income_band_sk#15] - -(23) CometBroadcastExchange -Input [1]: [ib_income_band_sk#15] -Arguments: [ib_income_band_sk#15] - -(24) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14] -Right output [1]: [ib_income_band_sk#15] -Arguments: [hd_income_band_sk#14], [ib_income_band_sk#15], Inner, BuildRight - -(25) CometProject -Input [6]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14, ib_income_band_sk#15] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12], [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(26) CometBroadcastExchange -Input [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_cdemo_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_cdemo_sk#18) - -(29) CometProject -Input [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Arguments: [sr_cdemo_sk#18], [sr_cdemo_sk#18] - -(30) CometBroadcastHashJoin -Left output [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Right output [1]: [sr_cdemo_sk#18] -Arguments: [cd_demo_sk#12], [sr_cdemo_sk#18], Inner, BuildLeft - -(31) CometProject -Input [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, sr_cdemo_sk#18] -Arguments: [customer_id#20, customername#21, c_customer_id#7], [c_customer_id#7 AS customer_id#20, concat(c_last_name#9, , , c_first_name#8) AS customername#21, c_customer_id#7] - -(32) CometTakeOrderedAndProject -Input [3]: [customer_id#20, customername#21, c_customer_id#7] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#7 ASC NULLS FIRST], output=[customer_id#20,customername#21]), [customer_id#20, customername#21], 100, 0, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [customer_id#20, customername#21] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/extended.txt deleted file mode 100644 index 5fce933878..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/extended.txt +++ /dev/null @@ -1,35 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - -Comet accelerated 32 out of 32 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index e43557c27d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id] [customer_id,customername] - CometProject [c_last_name,c_first_name] [customer_id,customername,c_customer_id] - 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 [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_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_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_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] 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 [ca_address_sk] #2 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [cd_demo_sk] #3 - CometFilter [cd_demo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange [ib_income_band_sk] #5 - CometProject [ib_income_band_sk] - CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/explain.txt deleted file mode 100644 index 204cd4ea7a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/explain.txt +++ /dev/null @@ -1,329 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (52) -+- * HashAggregate (51) - +- * CometColumnarToRow (50) - +- CometColumnarExchange (49) - +- * HashAggregate (48) - +- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (30) - : : : +- * BroadcastHashJoin Inner BuildRight (29) - : : : :- * Project (23) - : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : :- * Project (16) - : : : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : : : :- BroadcastExchange (4) - : : : : : : : +- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : : : : +- * CometColumnarToRow (8) - : : : : : : +- CometProject (7) - : : : : : : +- CometFilter (6) - : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns (5) - : : : : : +- BroadcastExchange (14) - : : : : : +- * CometColumnarToRow (13) - : : : : : +- CometFilter (12) - : : : : : +- CometNativeScan parquet spark_catalog.default.web_page (11) - : : : : +- BroadcastExchange (21) - : : : : +- * CometColumnarToRow (20) - : : : : +- CometProject (19) - : : : : +- CometFilter (18) - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (17) - : : : +- BroadcastExchange (28) - : : : +- * CometColumnarToRow (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (24) - : : +- BroadcastExchange (35) - : : +- * CometColumnarToRow (34) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (31) - : +- ReusedExchange (38) - +- BroadcastExchange (45) - +- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometNativeScan parquet spark_catalog.default.reason (41) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_page_sk), Or(Or(And(GreaterThanOrEqual(ws_sales_price,100.00),LessThanOrEqual(ws_sales_price,150.00)),And(GreaterThanOrEqual(ws_sales_price,50.00),LessThanOrEqual(ws_sales_price,100.00))),And(GreaterThanOrEqual(ws_sales_price,150.00),LessThanOrEqual(ws_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ws_net_profit,100.00),LessThanOrEqual(ws_net_profit,200.00)),And(GreaterThanOrEqual(ws_net_profit,150.00),LessThanOrEqual(ws_net_profit,300.00))),And(GreaterThanOrEqual(ws_net_profit,50.00),LessThanOrEqual(ws_net_profit,250.00)))] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] - -(3) Filter [codegen id : 1] -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((((isnotnull(ws_item_sk#1) AND isnotnull(ws_order_number#3)) AND isnotnull(ws_web_page_sk#2)) AND ((((ws_sales_price#5 >= 100.00) AND (ws_sales_price#5 <= 150.00)) OR ((ws_sales_price#5 >= 50.00) AND (ws_sales_price#5 <= 100.00))) OR ((ws_sales_price#5 >= 150.00) AND (ws_sales_price#5 <= 200.00)))) AND ((((ws_net_profit#6 >= 100.00) AND (ws_net_profit#6 <= 200.00)) OR ((ws_net_profit#6 >= 150.00) AND (ws_net_profit#6 <= 300.00))) OR ((ws_net_profit#6 >= 50.00) AND (ws_net_profit#6 <= 250.00)))) - -(4) BroadcastExchange -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[2, int, false] as bigint) & 4294967295))),false), [plan_id=1] - -(5) CometNativeScan parquet spark_catalog.default.web_returns -Output [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number), IsNotNull(wr_refunded_cdemo_sk), IsNotNull(wr_returning_cdemo_sk), IsNotNull(wr_refunded_addr_sk), IsNotNull(wr_reason_sk)] -ReadSchema: struct - -(6) CometFilter -Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Condition : (((((isnotnull(wr_item_sk#9) AND isnotnull(wr_order_number#14)) AND isnotnull(wr_refunded_cdemo_sk#10)) AND isnotnull(wr_returning_cdemo_sk#12)) AND isnotnull(wr_refunded_addr_sk#11)) AND isnotnull(wr_reason_sk#13)) - -(7) CometProject -Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Arguments: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16], [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] - -(8) CometColumnarToRow -Input [8]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] - -(9) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [ws_item_sk#1, ws_order_number#3] -Right keys [2]: [wr_item_sk#9, wr_order_number#14] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 8] -Output [11]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Input [15]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] - -(11) CometNativeScan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(12) CometFilter -Input [1]: [wp_web_page_sk#18] -Condition : isnotnull(wp_web_page_sk#18) - -(13) CometColumnarToRow [codegen id : 2] -Input [1]: [wp_web_page_sk#18] - -(14) BroadcastExchange -Input [1]: [wp_web_page_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_web_page_sk#2] -Right keys [1]: [wp_web_page_sk#18] -Join type: Inner -Join condition: None - -(16) Project [codegen id : 8] -Output [10]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Input [12]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, wp_web_page_sk#18] - -(17) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(18) CometFilter -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Condition : (((isnotnull(cd_demo_sk#19) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true))) AND ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = Advanced Degree )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = S) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = College ))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = W) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = 2 yr Degree )))) - -(19) CometProject -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) AS cd_marital_status#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) AS cd_education_status#23] - -(20) CometColumnarToRow [codegen id : 3] -Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] - -(21) BroadcastExchange -Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(22) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [wr_refunded_cdemo_sk#10] -Right keys [1]: [cd_demo_sk#19] -Join type: Inner -Join condition: ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) - -(23) Project [codegen id : 8] -Output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23] -Input [13]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] - -(24) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(25) CometFilter -Input [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Condition : ((isnotnull(cd_demo_sk#24) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#26, 20, true, false, true))) - -(26) CometProject -Input [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Arguments: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28], [cd_demo_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#26, 20, true, false, true) AS cd_education_status#28] - -(27) CometColumnarToRow [codegen id : 4] -Input [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] - -(28) BroadcastExchange -Input [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, string, true], input[2, string, true]),false), [plan_id=4] - -(29) BroadcastHashJoin [codegen id : 8] -Left keys [3]: [wr_returning_cdemo_sk#12, cd_marital_status#22, cd_education_status#23] -Right keys [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Join type: Inner -Join condition: None - -(30) Project [codegen id : 8] -Output [7]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Input [13]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23, cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] - -(31) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Condition : (((isnotnull(ca_country#31) AND (ca_country#31 = United States)) AND isnotnull(ca_address_sk#29)) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) IN (IN,OH,NJ) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) IN (WI,CT,KY)) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) IN (LA,IA,AR))) - -(33) CometProject -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Arguments: [ca_address_sk#29, ca_state#32], [ca_address_sk#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) AS ca_state#32] - -(34) CometColumnarToRow [codegen id : 5] -Input [2]: [ca_address_sk#29, ca_state#32] - -(35) BroadcastExchange -Input [2]: [ca_address_sk#29, ca_state#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(36) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [wr_refunded_addr_sk#11] -Right keys [1]: [ca_address_sk#29] -Join type: Inner -Join condition: ((((ca_state#32 IN (IN,OH,NJ) AND (ws_net_profit#6 >= 100.00)) AND (ws_net_profit#6 <= 200.00)) OR ((ca_state#32 IN (WI,CT,KY) AND (ws_net_profit#6 >= 150.00)) AND (ws_net_profit#6 <= 300.00))) OR ((ca_state#32 IN (LA,IA,AR) AND (ws_net_profit#6 >= 50.00)) AND (ws_net_profit#6 <= 250.00))) - -(37) Project [codegen id : 8] -Output [5]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Input [9]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, ca_address_sk#29, ca_state#32] - -(38) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#33] - -(39) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_date_sk#7] -Right keys [1]: [d_date_sk#33] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 8] -Output [4]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Input [6]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, d_date_sk#33] - -(41) CometNativeScan parquet spark_catalog.default.reason -Output [2]: [r_reason_sk#34, r_reason_desc#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [r_reason_sk#34, r_reason_desc#35] -Condition : isnotnull(r_reason_sk#34) - -(43) CometProject -Input [2]: [r_reason_sk#34, r_reason_desc#35] -Arguments: [r_reason_sk#34, r_reason_desc#36], [r_reason_sk#34, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, r_reason_desc#35, 100, true, false, true) AS r_reason_desc#36] - -(44) CometColumnarToRow [codegen id : 7] -Input [2]: [r_reason_sk#34, r_reason_desc#36] - -(45) BroadcastExchange -Input [2]: [r_reason_sk#34, r_reason_desc#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(46) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [wr_reason_sk#13] -Right keys [1]: [r_reason_sk#34] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 8] -Output [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#36] -Input [6]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, r_reason_sk#34, r_reason_desc#36] - -(48) HashAggregate [codegen id : 8] -Input [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#36] -Keys [1]: [r_reason_desc#36] -Functions [3]: [partial_avg(ws_quantity#4), partial_avg(UnscaledValue(wr_refunded_cash#16)), partial_avg(UnscaledValue(wr_fee#15))] -Aggregate Attributes [6]: [sum#37, count#38, sum#39, count#40, sum#41, count#42] -Results [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, count#48] - -(49) CometColumnarExchange -Input [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, count#48] -Arguments: hashpartitioning(r_reason_desc#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(50) CometColumnarToRow [codegen id : 9] -Input [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, count#48] - -(51) HashAggregate [codegen id : 9] -Input [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, count#48] -Keys [1]: [r_reason_desc#36] -Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg(UnscaledValue(wr_fee#15))] -Aggregate Attributes [3]: [avg(ws_quantity#4)#49, avg(UnscaledValue(wr_refunded_cash#16))#50, avg(UnscaledValue(wr_fee#15))#51] -Results [4]: [substr(r_reason_desc#36, 1, 20) AS substr(r_reason_desc, 1, 20)#52, avg(ws_quantity#4)#49 AS avg(ws_quantity)#53, cast((avg(UnscaledValue(wr_refunded_cash#16))#50 / 100.0) as decimal(11,6)) AS avg(wr_refunded_cash)#54, cast((avg(UnscaledValue(wr_fee#15))#51 / 100.0) as decimal(11,6)) AS avg(wr_fee)#55] - -(52) TakeOrderedAndProject -Input [4]: [substr(r_reason_desc, 1, 20)#52, avg(ws_quantity)#53, avg(wr_refunded_cash)#54, avg(wr_fee)#55] -Arguments: 100, [substr(r_reason_desc, 1, 20)#52 ASC NULLS FIRST, avg(ws_quantity)#53 ASC NULLS FIRST, avg(wr_refunded_cash)#54 ASC NULLS FIRST, avg(wr_fee)#55 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#52, avg(ws_quantity)#53, avg(wr_refunded_cash)#54, avg(wr_fee)#55] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometNativeScan parquet spark_catalog.default.date_dim (53) - - -(53) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [2]: [d_date_sk#33, d_year#56] -Condition : ((isnotnull(d_year#56) AND (d_year#56 = 2000)) AND isnotnull(d_date_sk#33)) - -(55) CometProject -Input [2]: [d_date_sk#33, d_year#56] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(56) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#33] - -(57) BroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/simplified.txt deleted file mode 100644 index 68d46e608d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/simplified.txt +++ /dev/null @@ -1,83 +0,0 @@ -TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] - WholeStageCodegen (9) - HashAggregate [r_reason_desc,sum,count,sum,count,sum,count] [avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee)),substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [r_reason_desc] #1 - WholeStageCodegen (8) - HashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - BroadcastHashJoin [wr_reason_sk,r_reason_sk] - Project [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] - Project [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] - Project [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] - BroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] - Project [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] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [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] - BroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - 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_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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [r_reason_desc] [r_reason_sk,r_reason_desc] - CometFilter [r_reason_sk,r_reason_desc] - CometNativeScan 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/q85.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/explain.txt deleted file mode 100644 index c263f03d76..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,309 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (48) -+- CometTakeOrderedAndProject (47) - +- CometHashAggregate (46) - +- CometExchange (45) - +- CometHashAggregate (44) - +- CometProject (43) - +- CometBroadcastHashJoin (42) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (19) - : : : : +- CometBroadcastHashJoin (18) - : : : : :- CometProject (13) - : : : : : +- CometBroadcastHashJoin (12) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometBroadcastExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (4) - : : : : : +- CometBroadcastExchange (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (9) - : : : : +- CometBroadcastExchange (17) - : : : : +- CometProject (16) - : : : : +- CometFilter (15) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (14) - : : : +- CometBroadcastExchange (23) - : : : +- CometProject (22) - : : : +- CometFilter (21) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (20) - : : +- CometBroadcastExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (26) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - +- CometBroadcastExchange (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (38) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_page_sk), Or(Or(And(GreaterThanOrEqual(ws_sales_price,100.00),LessThanOrEqual(ws_sales_price,150.00)),And(GreaterThanOrEqual(ws_sales_price,50.00),LessThanOrEqual(ws_sales_price,100.00))),And(GreaterThanOrEqual(ws_sales_price,150.00),LessThanOrEqual(ws_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ws_net_profit,100.00),LessThanOrEqual(ws_net_profit,200.00)),And(GreaterThanOrEqual(ws_net_profit,150.00),LessThanOrEqual(ws_net_profit,300.00))),And(GreaterThanOrEqual(ws_net_profit,50.00),LessThanOrEqual(ws_net_profit,250.00)))] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((((isnotnull(ws_item_sk#1) AND isnotnull(ws_order_number#3)) AND isnotnull(ws_web_page_sk#2)) AND ((((ws_sales_price#5 >= 100.00) AND (ws_sales_price#5 <= 150.00)) OR ((ws_sales_price#5 >= 50.00) AND (ws_sales_price#5 <= 100.00))) OR ((ws_sales_price#5 >= 150.00) AND (ws_sales_price#5 <= 200.00)))) AND ((((ws_net_profit#6 >= 100.00) AND (ws_net_profit#6 <= 200.00)) OR ((ws_net_profit#6 >= 150.00) AND (ws_net_profit#6 <= 300.00))) OR ((ws_net_profit#6 >= 50.00) AND (ws_net_profit#6 <= 250.00)))) - -(3) CometBroadcastExchange -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Arguments: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number), IsNotNull(wr_refunded_cdemo_sk), IsNotNull(wr_returning_cdemo_sk), IsNotNull(wr_refunded_addr_sk), IsNotNull(wr_reason_sk)] -ReadSchema: struct - -(5) CometFilter -Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Condition : (((((isnotnull(wr_item_sk#9) AND isnotnull(wr_order_number#14)) AND isnotnull(wr_refunded_cdemo_sk#10)) AND isnotnull(wr_returning_cdemo_sk#12)) AND isnotnull(wr_refunded_addr_sk#11)) AND isnotnull(wr_reason_sk#13)) - -(6) CometProject -Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Arguments: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16], [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] - -(7) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Right output [8]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] -Arguments: [ws_item_sk#1, ws_order_number#3], [wr_item_sk#9, wr_order_number#14], Inner, BuildLeft - -(8) CometProject -Input [15]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] -Arguments: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(10) CometFilter -Input [1]: [wp_web_page_sk#18] -Condition : isnotnull(wp_web_page_sk#18) - -(11) CometBroadcastExchange -Input [1]: [wp_web_page_sk#18] -Arguments: [wp_web_page_sk#18] - -(12) CometBroadcastHashJoin -Left output [11]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [1]: [wp_web_page_sk#18] -Arguments: [ws_web_page_sk#2], [wp_web_page_sk#18], Inner, BuildRight - -(13) CometProject -Input [12]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, wp_web_page_sk#18] -Arguments: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Condition : (((isnotnull(cd_demo_sk#19) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true))) AND ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = Advanced Degree )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = S) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = College ))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = W) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = 2 yr Degree )))) - -(16) CometProject -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) AS cd_marital_status#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) AS cd_education_status#23] - -(17) CometBroadcastExchange -Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] - -(18) CometBroadcastHashJoin -Left output [10]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [wr_refunded_cdemo_sk#10], [cd_demo_sk#19], Inner, ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))), BuildRight - -(19) CometProject -Input [13]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23], [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Condition : ((isnotnull(cd_demo_sk#24) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#26, 20, true, false, true))) - -(22) CometProject -Input [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Arguments: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28], [cd_demo_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#26, 20, true, false, true) AS cd_education_status#28] - -(23) CometBroadcastExchange -Input [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Arguments: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] - -(24) CometBroadcastHashJoin -Left output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23] -Right output [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Arguments: [wr_returning_cdemo_sk#12, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28], Inner, BuildRight - -(25) CometProject -Input [13]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23, cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Arguments: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Condition : (((isnotnull(ca_country#31) AND (ca_country#31 = United States)) AND isnotnull(ca_address_sk#29)) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) IN (IN,OH,NJ) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) IN (WI,CT,KY)) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) IN (LA,IA,AR))) - -(28) CometProject -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Arguments: [ca_address_sk#29, ca_state#32], [ca_address_sk#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) AS ca_state#32] - -(29) CometBroadcastExchange -Input [2]: [ca_address_sk#29, ca_state#32] -Arguments: [ca_address_sk#29, ca_state#32] - -(30) CometBroadcastHashJoin -Left output [7]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [2]: [ca_address_sk#29, ca_state#32] -Arguments: [wr_refunded_addr_sk#11], [ca_address_sk#29], Inner, ((((ca_state#32 IN (IN,OH,NJ) AND (ws_net_profit#6 >= 100.00)) AND (ws_net_profit#6 <= 200.00)) OR ((ca_state#32 IN (WI,CT,KY) AND (ws_net_profit#6 >= 150.00)) AND (ws_net_profit#6 <= 300.00))) OR ((ca_state#32 IN (LA,IA,AR) AND (ws_net_profit#6 >= 50.00)) AND (ws_net_profit#6 <= 250.00))), BuildRight - -(31) CometProject -Input [9]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, ca_address_sk#29, ca_state#32] -Arguments: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#33, d_year#34] -Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2000)) AND isnotnull(d_date_sk#33)) - -(34) CometProject -Input [2]: [d_date_sk#33, d_year#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(35) CometBroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: [d_date_sk#33] - -(36) CometBroadcastHashJoin -Left output [5]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [1]: [d_date_sk#33] -Arguments: [ws_sold_date_sk#7], [d_date_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, d_date_sk#33] -Arguments: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Output [2]: [r_reason_sk#35, r_reason_desc#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk)] -ReadSchema: struct - -(39) CometFilter -Input [2]: [r_reason_sk#35, r_reason_desc#36] -Condition : isnotnull(r_reason_sk#35) - -(40) CometProject -Input [2]: [r_reason_sk#35, r_reason_desc#36] -Arguments: [r_reason_sk#35, r_reason_desc#37], [r_reason_sk#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, r_reason_desc#36, 100, true, false, true) AS r_reason_desc#37] - -(41) CometBroadcastExchange -Input [2]: [r_reason_sk#35, r_reason_desc#37] -Arguments: [r_reason_sk#35, r_reason_desc#37] - -(42) CometBroadcastHashJoin -Left output [4]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [2]: [r_reason_sk#35, r_reason_desc#37] -Arguments: [wr_reason_sk#13], [r_reason_sk#35], Inner, BuildRight - -(43) CometProject -Input [6]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, r_reason_sk#35, r_reason_desc#37] -Arguments: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#37], [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#37] - -(44) CometHashAggregate -Input [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#37] -Keys [1]: [r_reason_desc#37] -Functions [3]: [partial_avg(ws_quantity#4), partial_avg(UnscaledValue(wr_refunded_cash#16)), partial_avg(UnscaledValue(wr_fee#15))] - -(45) CometExchange -Input [7]: [r_reason_desc#37, sum#38, count#39, sum#40, count#41, sum#42, count#43] -Arguments: hashpartitioning(r_reason_desc#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(46) CometHashAggregate -Input [7]: [r_reason_desc#37, sum#38, count#39, sum#40, count#41, sum#42, count#43] -Keys [1]: [r_reason_desc#37] -Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg(UnscaledValue(wr_fee#15))] - -(47) CometTakeOrderedAndProject -Input [4]: [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(r_reason_desc, 1, 20)#44 ASC NULLS FIRST,avg(ws_quantity)#45 ASC NULLS FIRST,avg(wr_refunded_cash)#46 ASC NULLS FIRST,avg(wr_fee)#47 ASC NULLS FIRST], output=[substr(r_reason_desc, 1, 20)#44,avg(ws_quantity)#45,avg(wr_refunded_cash)#46,avg(wr_fee)#47]), [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47], 100, 0, [substr(r_reason_desc, 1, 20)#44 ASC NULLS FIRST, avg(ws_quantity)#45 ASC NULLS FIRST, avg(wr_refunded_cash)#46 ASC NULLS FIRST, avg(wr_fee)#47 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47] - -(48) CometColumnarToRow [codegen id : 1] -Input [4]: [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [2]: [d_date_sk#33, d_year#34] -Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2000)) AND isnotnull(d_date_sk#33)) - -(51) CometProject -Input [2]: [d_date_sk#33, d_year#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#33] - -(53) BroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/simplified.txt deleted file mode 100644 index dfcca32f03..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,58 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] - CometHashAggregate [sum,count,sum,count,sum,count] [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),r_reason_desc,avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee))] - CometExchange [r_reason_desc] #1 - CometHashAggregate [ws_quantity,wr_refunded_cash,wr_fee] [r_reason_desc,sum,count,sum,count,sum,count] - CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - 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_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 [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 [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 [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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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_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 [native_iceberg_compat] 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 [wp_web_page_sk] #4 - CometFilter [wp_web_page_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [ca_address_sk,ca_state] #7 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #8 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 - CometProject [r_reason_desc] [r_reason_sk,r_reason_desc] - CometFilter [r_reason_sk,r_reason_desc] - CometScan [native_iceberg_compat] 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/q85/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/explain.txt deleted file mode 100644 index c263f03d76..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/explain.txt +++ /dev/null @@ -1,309 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (48) -+- CometTakeOrderedAndProject (47) - +- CometHashAggregate (46) - +- CometExchange (45) - +- CometHashAggregate (44) - +- CometProject (43) - +- CometBroadcastHashJoin (42) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (19) - : : : : +- CometBroadcastHashJoin (18) - : : : : :- CometProject (13) - : : : : : +- CometBroadcastHashJoin (12) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometBroadcastExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (4) - : : : : : +- CometBroadcastExchange (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (9) - : : : : +- CometBroadcastExchange (17) - : : : : +- CometProject (16) - : : : : +- CometFilter (15) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (14) - : : : +- CometBroadcastExchange (23) - : : : +- CometProject (22) - : : : +- CometFilter (21) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (20) - : : +- CometBroadcastExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (26) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - +- CometBroadcastExchange (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (38) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_page_sk), Or(Or(And(GreaterThanOrEqual(ws_sales_price,100.00),LessThanOrEqual(ws_sales_price,150.00)),And(GreaterThanOrEqual(ws_sales_price,50.00),LessThanOrEqual(ws_sales_price,100.00))),And(GreaterThanOrEqual(ws_sales_price,150.00),LessThanOrEqual(ws_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ws_net_profit,100.00),LessThanOrEqual(ws_net_profit,200.00)),And(GreaterThanOrEqual(ws_net_profit,150.00),LessThanOrEqual(ws_net_profit,300.00))),And(GreaterThanOrEqual(ws_net_profit,50.00),LessThanOrEqual(ws_net_profit,250.00)))] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((((isnotnull(ws_item_sk#1) AND isnotnull(ws_order_number#3)) AND isnotnull(ws_web_page_sk#2)) AND ((((ws_sales_price#5 >= 100.00) AND (ws_sales_price#5 <= 150.00)) OR ((ws_sales_price#5 >= 50.00) AND (ws_sales_price#5 <= 100.00))) OR ((ws_sales_price#5 >= 150.00) AND (ws_sales_price#5 <= 200.00)))) AND ((((ws_net_profit#6 >= 100.00) AND (ws_net_profit#6 <= 200.00)) OR ((ws_net_profit#6 >= 150.00) AND (ws_net_profit#6 <= 300.00))) OR ((ws_net_profit#6 >= 50.00) AND (ws_net_profit#6 <= 250.00)))) - -(3) CometBroadcastExchange -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Arguments: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number), IsNotNull(wr_refunded_cdemo_sk), IsNotNull(wr_returning_cdemo_sk), IsNotNull(wr_refunded_addr_sk), IsNotNull(wr_reason_sk)] -ReadSchema: struct - -(5) CometFilter -Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Condition : (((((isnotnull(wr_item_sk#9) AND isnotnull(wr_order_number#14)) AND isnotnull(wr_refunded_cdemo_sk#10)) AND isnotnull(wr_returning_cdemo_sk#12)) AND isnotnull(wr_refunded_addr_sk#11)) AND isnotnull(wr_reason_sk#13)) - -(6) CometProject -Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Arguments: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16], [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] - -(7) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Right output [8]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] -Arguments: [ws_item_sk#1, ws_order_number#3], [wr_item_sk#9, wr_order_number#14], Inner, BuildLeft - -(8) CometProject -Input [15]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] -Arguments: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(10) CometFilter -Input [1]: [wp_web_page_sk#18] -Condition : isnotnull(wp_web_page_sk#18) - -(11) CometBroadcastExchange -Input [1]: [wp_web_page_sk#18] -Arguments: [wp_web_page_sk#18] - -(12) CometBroadcastHashJoin -Left output [11]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [1]: [wp_web_page_sk#18] -Arguments: [ws_web_page_sk#2], [wp_web_page_sk#18], Inner, BuildRight - -(13) CometProject -Input [12]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, wp_web_page_sk#18] -Arguments: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Condition : (((isnotnull(cd_demo_sk#19) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true))) AND ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = Advanced Degree )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = S) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = College ))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = W) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = 2 yr Degree )))) - -(16) CometProject -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) AS cd_marital_status#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) AS cd_education_status#23] - -(17) CometBroadcastExchange -Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] - -(18) CometBroadcastHashJoin -Left output [10]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [wr_refunded_cdemo_sk#10], [cd_demo_sk#19], Inner, ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))), BuildRight - -(19) CometProject -Input [13]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23], [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Condition : ((isnotnull(cd_demo_sk#24) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#26, 20, true, false, true))) - -(22) CometProject -Input [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Arguments: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28], [cd_demo_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#26, 20, true, false, true) AS cd_education_status#28] - -(23) CometBroadcastExchange -Input [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Arguments: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] - -(24) CometBroadcastHashJoin -Left output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23] -Right output [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Arguments: [wr_returning_cdemo_sk#12, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28], Inner, BuildRight - -(25) CometProject -Input [13]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23, cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Arguments: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Condition : (((isnotnull(ca_country#31) AND (ca_country#31 = United States)) AND isnotnull(ca_address_sk#29)) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) IN (IN,OH,NJ) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) IN (WI,CT,KY)) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) IN (LA,IA,AR))) - -(28) CometProject -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Arguments: [ca_address_sk#29, ca_state#32], [ca_address_sk#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) AS ca_state#32] - -(29) CometBroadcastExchange -Input [2]: [ca_address_sk#29, ca_state#32] -Arguments: [ca_address_sk#29, ca_state#32] - -(30) CometBroadcastHashJoin -Left output [7]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [2]: [ca_address_sk#29, ca_state#32] -Arguments: [wr_refunded_addr_sk#11], [ca_address_sk#29], Inner, ((((ca_state#32 IN (IN,OH,NJ) AND (ws_net_profit#6 >= 100.00)) AND (ws_net_profit#6 <= 200.00)) OR ((ca_state#32 IN (WI,CT,KY) AND (ws_net_profit#6 >= 150.00)) AND (ws_net_profit#6 <= 300.00))) OR ((ca_state#32 IN (LA,IA,AR) AND (ws_net_profit#6 >= 50.00)) AND (ws_net_profit#6 <= 250.00))), BuildRight - -(31) CometProject -Input [9]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, ca_address_sk#29, ca_state#32] -Arguments: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#33, d_year#34] -Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2000)) AND isnotnull(d_date_sk#33)) - -(34) CometProject -Input [2]: [d_date_sk#33, d_year#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(35) CometBroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: [d_date_sk#33] - -(36) CometBroadcastHashJoin -Left output [5]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [1]: [d_date_sk#33] -Arguments: [ws_sold_date_sk#7], [d_date_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, d_date_sk#33] -Arguments: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Output [2]: [r_reason_sk#35, r_reason_desc#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk)] -ReadSchema: struct - -(39) CometFilter -Input [2]: [r_reason_sk#35, r_reason_desc#36] -Condition : isnotnull(r_reason_sk#35) - -(40) CometProject -Input [2]: [r_reason_sk#35, r_reason_desc#36] -Arguments: [r_reason_sk#35, r_reason_desc#37], [r_reason_sk#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, r_reason_desc#36, 100, true, false, true) AS r_reason_desc#37] - -(41) CometBroadcastExchange -Input [2]: [r_reason_sk#35, r_reason_desc#37] -Arguments: [r_reason_sk#35, r_reason_desc#37] - -(42) CometBroadcastHashJoin -Left output [4]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [2]: [r_reason_sk#35, r_reason_desc#37] -Arguments: [wr_reason_sk#13], [r_reason_sk#35], Inner, BuildRight - -(43) CometProject -Input [6]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, r_reason_sk#35, r_reason_desc#37] -Arguments: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#37], [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#37] - -(44) CometHashAggregate -Input [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#37] -Keys [1]: [r_reason_desc#37] -Functions [3]: [partial_avg(ws_quantity#4), partial_avg(UnscaledValue(wr_refunded_cash#16)), partial_avg(UnscaledValue(wr_fee#15))] - -(45) CometExchange -Input [7]: [r_reason_desc#37, sum#38, count#39, sum#40, count#41, sum#42, count#43] -Arguments: hashpartitioning(r_reason_desc#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(46) CometHashAggregate -Input [7]: [r_reason_desc#37, sum#38, count#39, sum#40, count#41, sum#42, count#43] -Keys [1]: [r_reason_desc#37] -Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg(UnscaledValue(wr_fee#15))] - -(47) CometTakeOrderedAndProject -Input [4]: [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(r_reason_desc, 1, 20)#44 ASC NULLS FIRST,avg(ws_quantity)#45 ASC NULLS FIRST,avg(wr_refunded_cash)#46 ASC NULLS FIRST,avg(wr_fee)#47 ASC NULLS FIRST], output=[substr(r_reason_desc, 1, 20)#44,avg(ws_quantity)#45,avg(wr_refunded_cash)#46,avg(wr_fee)#47]), [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47], 100, 0, [substr(r_reason_desc, 1, 20)#44 ASC NULLS FIRST, avg(ws_quantity)#45 ASC NULLS FIRST, avg(wr_refunded_cash)#46 ASC NULLS FIRST, avg(wr_fee)#47 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47] - -(48) CometColumnarToRow [codegen id : 1] -Input [4]: [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [2]: [d_date_sk#33, d_year#34] -Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2000)) AND isnotnull(d_date_sk#33)) - -(51) CometProject -Input [2]: [d_date_sk#33, d_year#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#33] - -(53) BroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/extended.txt deleted file mode 100644 index e2383394d1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/extended.txt +++ /dev/null @@ -1,56 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason - -Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index dfcca32f03..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85/simplified.txt +++ /dev/null @@ -1,58 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] - CometHashAggregate [sum,count,sum,count,sum,count] [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),r_reason_desc,avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee))] - CometExchange [r_reason_desc] #1 - CometHashAggregate [ws_quantity,wr_refunded_cash,wr_fee] [r_reason_desc,sum,count,sum,count,sum,count] - CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - 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_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 [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 [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 [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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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_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 [native_iceberg_compat] 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 [wp_web_page_sk] #4 - CometFilter [wp_web_page_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [ca_address_sk,ca_state] #7 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #8 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 - CometProject [r_reason_desc] [r_reason_sk,r_reason_desc] - CometFilter [r_reason_sk,r_reason_desc] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/explain.txt deleted file mode 100644 index 3f745e99c5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (17) - +- CometColumnarExchange (16) - +- * HashAggregate (15) - +- * Expand (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.web_sales (1) - : +- ReusedExchange (4) - +- BroadcastExchange (11) - +- * CometColumnarToRow (10) - +- CometProject (9) - +- CometFilter (8) - +- CometNativeScan parquet spark_catalog.default.item (7) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(4) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#5] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [2]: [ws_item_sk#1, ws_net_paid#2] -Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] - -(7) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#6, i_class#7, i_category#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Condition : isnotnull(i_item_sk#6) - -(9) CometProject -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Arguments: [i_item_sk#6, i_class#9, i_category#10], [i_item_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#7, 50, true, false, true) AS i_class#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#8, 50, true, false, true) AS i_category#10] - -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#6, i_class#9, i_category#10] - -(11) BroadcastExchange -Input [3]: [i_item_sk#6, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [3]: [ws_net_paid#2, i_category#10, i_class#9] -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#6, i_class#9, i_category#10] - -(14) Expand [codegen id : 3] -Input [3]: [ws_net_paid#2, i_category#10, i_class#9] -Arguments: [[ws_net_paid#2, i_category#10, i_class#9, 0], [ws_net_paid#2, i_category#10, null, 1], [ws_net_paid#2, null, null, 3]], [ws_net_paid#2, i_category#11, i_class#12, spark_grouping_id#13] - -(15) HashAggregate [codegen id : 3] -Input [4]: [ws_net_paid#2, i_category#11, i_class#12, spark_grouping_id#13] -Keys [3]: [i_category#11, i_class#12, spark_grouping_id#13] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum#14] -Results [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] - -(16) CometColumnarExchange -Input [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] -Arguments: hashpartitioning(i_category#11, i_class#12, spark_grouping_id#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(17) CometColumnarToRow [codegen id : 4] -Input [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] - -(18) HashAggregate [codegen id : 4] -Input [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] -Keys [3]: [i_category#11, i_class#12, spark_grouping_id#13] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#16] -Results [7]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#16,17,2) AS total_sum#17, i_category#11, i_class#12, (cast((shiftright(spark_grouping_id#13, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#13, 0) & 1) as tinyint)) AS lochierarchy#18, MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#16,17,2) AS _w0#19, (cast((shiftright(spark_grouping_id#13, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#13, 0) & 1) as tinyint)) AS _w1#20, CASE WHEN (cast((shiftright(spark_grouping_id#13, 0) & 1) as tinyint) = 0) THEN i_category#11 END AS _w2#21] - -(19) CometColumnarExchange -Input [7]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21] -Arguments: hashpartitioning(_w1#20, _w2#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(20) CometSort -Input [7]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21] -Arguments: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21], [_w1#20 ASC NULLS FIRST, _w2#21 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] - -(21) CometColumnarToRow [codegen id : 5] -Input [7]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21] - -(22) Window -Input [7]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21] -Arguments: [rank(_w0#19) windowspecdefinition(_w1#20, _w2#21, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#22], [_w1#20, _w2#21], [_w0#19 DESC NULLS LAST] - -(23) Project [codegen id : 6] -Output [5]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, rank_within_parent#22] -Input [8]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21, rank_within_parent#22] - -(24) TakeOrderedAndProject -Input [5]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, rank_within_parent#22] -Arguments: 100, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0) THEN i_category#11 END ASC NULLS FIRST, rank_within_parent#22 ASC NULLS FIRST], [total_sum#17, i_category#11, i_class#12, lochierarchy#18, rank_within_parent#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometNativeScan parquet spark_catalog.default.date_dim (25) - - -(25) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#23] -Condition : (((isnotnull(d_month_seq#23) AND (d_month_seq#23 >= 1200)) AND (d_month_seq#23 <= 1211)) AND isnotnull(d_date_sk#5)) - -(27) CometProject -Input [2]: [d_date_sk#5, d_month_seq#23] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(29) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/simplified.txt deleted file mode 100644 index 0b24fe234a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/simplified.txt +++ /dev/null @@ -1,44 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (6) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 - WholeStageCodegen (3) - HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] - Expand [ws_net_paid,i_category,i_class] - Project [ws_net_paid,i_category,i_class] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan 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/q86.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/explain.txt deleted file mode 100644 index 1af5e2612c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,169 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometExchange (19) - +- CometHashAggregate (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometExpand (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] -Arguments: [ws_item_sk#1, ws_net_paid#2], [ws_item_sk#1, ws_net_paid#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#7, i_class#8, i_category#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Condition : isnotnull(i_item_sk#7) - -(11) CometProject -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Arguments: [i_item_sk#7, i_class#10, i_category#11], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#8, 50, true, false, true) AS i_class#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [i_item_sk#7, i_class#10, i_category#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#1, ws_net_paid#2] -Right output [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_net_paid#2, i_category#11, i_class#10], [ws_net_paid#2, i_category#11, i_class#10] - -(15) CometExpand -Input [3]: [ws_net_paid#2, i_category#11, i_class#10] -Arguments: [[ws_net_paid#2, i_category#11, i_class#10, 0], [ws_net_paid#2, i_category#11, null, 1], [ws_net_paid#2, null, null, 3]], [ws_net_paid#2, i_category#12, i_class#13, spark_grouping_id#14] - -(16) CometHashAggregate -Input [4]: [ws_net_paid#2, i_category#12, i_class#13, spark_grouping_id#14] -Keys [3]: [i_category#12, i_class#13, spark_grouping_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] - -(17) CometExchange -Input [4]: [i_category#12, i_class#13, spark_grouping_id#14, sum#15] -Arguments: hashpartitioning(i_category#12, i_class#13, spark_grouping_id#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [4]: [i_category#12, i_class#13, spark_grouping_id#14, sum#15] -Keys [3]: [i_category#12, i_class#13, spark_grouping_id#14] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] - -(19) CometExchange -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -Arguments: hashpartitioning(_w1#19, _w2#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -Arguments: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20], [_w1#19 ASC NULLS FIRST, _w2#20 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] - -(21) CometColumnarToRow [codegen id : 1] -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] - -(22) Window -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -Arguments: [rank(_w0#18) windowspecdefinition(_w1#19, _w2#20, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#21], [_w1#19, _w2#20], [_w0#18 DESC NULLS LAST] - -(23) Project [codegen id : 2] -Output [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] -Input [8]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20, rank_within_parent#21] - -(24) TakeOrderedAndProject -Input [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] -Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#21 ASC NULLS FIRST], [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(27) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(29) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/simplified.txt deleted file mode 100644 index 29e2d72920..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (2) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometExchange [_w1,_w2] #1 - CometHashAggregate [sum] [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ws_net_paid)),spark_grouping_id] - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] - 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,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] 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/q86/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/explain.txt deleted file mode 100644 index 1af5e2612c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/explain.txt +++ /dev/null @@ -1,169 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometExchange (19) - +- CometHashAggregate (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometExpand (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] -Arguments: [ws_item_sk#1, ws_net_paid#2], [ws_item_sk#1, ws_net_paid#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#7, i_class#8, i_category#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Condition : isnotnull(i_item_sk#7) - -(11) CometProject -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Arguments: [i_item_sk#7, i_class#10, i_category#11], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#8, 50, true, false, true) AS i_class#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [i_item_sk#7, i_class#10, i_category#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#1, ws_net_paid#2] -Right output [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_net_paid#2, i_category#11, i_class#10], [ws_net_paid#2, i_category#11, i_class#10] - -(15) CometExpand -Input [3]: [ws_net_paid#2, i_category#11, i_class#10] -Arguments: [[ws_net_paid#2, i_category#11, i_class#10, 0], [ws_net_paid#2, i_category#11, null, 1], [ws_net_paid#2, null, null, 3]], [ws_net_paid#2, i_category#12, i_class#13, spark_grouping_id#14] - -(16) CometHashAggregate -Input [4]: [ws_net_paid#2, i_category#12, i_class#13, spark_grouping_id#14] -Keys [3]: [i_category#12, i_class#13, spark_grouping_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] - -(17) CometExchange -Input [4]: [i_category#12, i_class#13, spark_grouping_id#14, sum#15] -Arguments: hashpartitioning(i_category#12, i_class#13, spark_grouping_id#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [4]: [i_category#12, i_class#13, spark_grouping_id#14, sum#15] -Keys [3]: [i_category#12, i_class#13, spark_grouping_id#14] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] - -(19) CometExchange -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -Arguments: hashpartitioning(_w1#19, _w2#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -Arguments: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20], [_w1#19 ASC NULLS FIRST, _w2#20 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] - -(21) CometColumnarToRow [codegen id : 1] -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] - -(22) Window -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -Arguments: [rank(_w0#18) windowspecdefinition(_w1#19, _w2#20, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#21], [_w1#19, _w2#20], [_w0#18 DESC NULLS LAST] - -(23) Project [codegen id : 2] -Output [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] -Input [8]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20, rank_within_parent#21] - -(24) TakeOrderedAndProject -Input [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] -Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#21 ASC NULLS FIRST], [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(27) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(29) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/extended.txt deleted file mode 100644 index dfc810b108..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/extended.txt +++ /dev/null @@ -1,32 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 29e2d72920..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (2) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometExchange [_w1,_w2] #1 - CometHashAggregate [sum] [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ws_net_paid)),spark_grouping_id] - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] - 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,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/explain.txt deleted file mode 100644 index b3086b072a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/explain.txt +++ /dev/null @@ -1,336 +0,0 @@ -== Physical Plan == -* HashAggregate (52) -+- * CometColumnarToRow (51) - +- CometColumnarExchange (50) - +- * HashAggregate (49) - +- * Project (48) - +- * BroadcastHashJoin LeftAnti BuildRight (47) - :- * BroadcastHashJoin LeftAnti BuildRight (32) - : :- * CometColumnarToRow (17) - : : +- CometHashAggregate (16) - : : +- CometColumnarExchange (15) - : : +- * HashAggregate (14) - : : +- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.customer (7) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometHashAggregate (29) - : +- CometColumnarExchange (28) - : +- * HashAggregate (27) - : +- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * Filter (20) - : : : +- * ColumnarToRow (19) - : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (21) - : +- ReusedExchange (24) - +- BroadcastExchange (46) - +- * CometColumnarToRow (45) - +- CometHashAggregate (44) - +- CometColumnarExchange (43) - +- * HashAggregate (42) - +- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Filter (35) - : : +- * ColumnarToRow (34) - : : +- Scan parquet spark_catalog.default.web_sales (33) - : +- ReusedExchange (36) - +- ReusedExchange (39) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sold_date_sk#2 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] - -(3) Filter [codegen id : 3] -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Condition : isnotnull(ss_customer_sk#1) - -(4) ReusedExchange [Reuses operator id: 57] -Output [2]: [d_date_sk#4, d_date#5] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#2] -Right keys [1]: [d_date_sk#4] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [2]: [ss_customer_sk#1, d_date#5] -Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] - -(7) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Condition : isnotnull(c_customer_sk#6) - -(9) CometProject -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Arguments: [c_customer_sk#6, c_first_name#9, c_last_name#10], [c_customer_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#7, 20, true, false, true) AS c_first_name#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#8, 30, true, false, true) AS c_last_name#10] - -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] - -(11) BroadcastExchange -Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#6] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [3]: [c_last_name#10, c_first_name#9, d_date#5] -Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#6, c_first_name#9, c_last_name#10] - -(14) HashAggregate [codegen id : 3] -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#5] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#5] - -(15) CometColumnarExchange -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] -Arguments: hashpartitioning(c_last_name#10, c_first_name#9, d_date#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometHashAggregate -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#5] -Functions: [] - -(17) CometColumnarToRow [codegen id : 12] -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] - -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#12), dynamicpruningexpression(cs_sold_date_sk#12 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] - -(20) Filter [codegen id : 6] -Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] -Condition : isnotnull(cs_bill_customer_sk#11) - -(21) ReusedExchange [Reuses operator id: 57] -Output [2]: [d_date_sk#13, d_date#14] - -(22) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#12] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(23) Project [codegen id : 6] -Output [2]: [cs_bill_customer_sk#11, d_date#14] -Input [4]: [cs_bill_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13, d_date#14] - -(24) ReusedExchange [Reuses operator id: 11] -Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] - -(25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_bill_customer_sk#11] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 6] -Output [3]: [c_last_name#17, c_first_name#16, d_date#14] -Input [5]: [cs_bill_customer_sk#11, d_date#14, c_customer_sk#15, c_first_name#16, c_last_name#17] - -(27) HashAggregate [codegen id : 6] -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#17, c_first_name#16, d_date#14] - -(28) CometColumnarExchange -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, d_date#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(29) CometHashAggregate -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] -Functions: [] - -(30) CometColumnarToRow [codegen id : 7] -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] - -(31) BroadcastExchange -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#17, ), isnull(c_last_name#17), coalesce(c_first_name#16, ), isnull(c_first_name#16), coalesce(d_date#14, 1970-01-01), isnull(d_date#14)] -Join type: LeftAnti -Join condition: None - -(33) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_sold_date_sk#19 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 10] -Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] - -(35) Filter [codegen id : 10] -Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] -Condition : isnotnull(ws_bill_customer_sk#18) - -(36) ReusedExchange [Reuses operator id: 57] -Output [2]: [d_date_sk#20, d_date#21] - -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#19] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 10] -Output [2]: [ws_bill_customer_sk#18, d_date#21] -Input [4]: [ws_bill_customer_sk#18, ws_sold_date_sk#19, d_date_sk#20, d_date#21] - -(39) ReusedExchange [Reuses operator id: 11] -Output [3]: [c_customer_sk#22, c_first_name#23, c_last_name#24] - -(40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_bill_customer_sk#18] -Right keys [1]: [c_customer_sk#22] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 10] -Output [3]: [c_last_name#24, c_first_name#23, d_date#21] -Input [5]: [ws_bill_customer_sk#18, d_date#21, c_customer_sk#22, c_first_name#23, c_last_name#24] - -(42) HashAggregate [codegen id : 10] -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#24, c_first_name#23, d_date#21] - -(43) CometColumnarExchange -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, d_date#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(44) CometHashAggregate -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] -Functions: [] - -(45) CometColumnarToRow [codegen id : 11] -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] - -(46) BroadcastExchange -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=6] - -(47) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#24, ), isnull(c_last_name#24), coalesce(c_first_name#23, ), isnull(c_first_name#23), coalesce(d_date#21, 1970-01-01), isnull(d_date#21)] -Join type: LeftAnti -Join condition: None - -(48) Project [codegen id : 12] -Output: [] -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] - -(49) HashAggregate [codegen id : 12] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#25] -Results [1]: [count#26] - -(50) CometColumnarExchange -Input [1]: [count#26] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(51) CometColumnarToRow [codegen id : 13] -Input [1]: [count#26] - -(52) HashAggregate [codegen id : 13] -Input [1]: [count#26] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#27] -Results [1]: [count(1)#27 AS count(1)#28] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometNativeScan parquet spark_catalog.default.date_dim (53) - - -(53) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] -Condition : (((isnotnull(d_month_seq#29) AND (d_month_seq#29 >= 1200)) AND (d_month_seq#29 <= 1211)) AND isnotnull(d_date_sk#4)) - -(55) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(56) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] - -(57) BroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#12 IN dynamicpruning#3 - -Subquery:3 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#19 IN dynamicpruning#3 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/simplified.txt deleted file mode 100644 index afc302e779..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/simplified.txt +++ /dev/null @@ -1,86 +0,0 @@ -WholeStageCodegen (13) - HashAggregate [count] [count(1),count(1),count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (12) - HashAggregate [count,count] - Project - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/explain.txt deleted file mode 100644 index fc6b12b834..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,327 +0,0 @@ -== Physical Plan == -* HashAggregate (51) -+- * CometColumnarToRow (50) - +- CometColumnarExchange (49) - +- * HashAggregate (48) - +- * Project (47) - +- * BroadcastHashJoin LeftAnti BuildRight (46) - :- * BroadcastHashJoin LeftAnti BuildRight (32) - : :- * CometColumnarToRow (18) - : : +- CometHashAggregate (17) - : : +- CometExchange (16) - : : +- CometHashAggregate (15) - : : +- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometHashAggregate (29) - : +- CometExchange (28) - : +- CometHashAggregate (27) - : +- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (19) - : : +- ReusedExchange (21) - : +- ReusedExchange (24) - +- BroadcastExchange (45) - +- * CometColumnarToRow (44) - +- CometHashAggregate (43) - +- CometExchange (42) - +- CometHashAggregate (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : +- ReusedExchange (35) - +- ReusedExchange (38) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sold_date_sk#2 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Condition : isnotnull(ss_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(5) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: [d_date_sk#4, d_date#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Right output [2]: [d_date_sk#4, d_date#5] -Arguments: [ss_sold_date_sk#2], [d_date_sk#4], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] -Arguments: [ss_customer_sk#1, d_date#5], [ss_customer_sk#1, d_date#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Condition : isnotnull(c_customer_sk#7) - -(11) CometProject -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11], [c_customer_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#8, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#9, 30, true, false, true) AS c_last_name#11] - -(12) CometBroadcastExchange -Input [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, d_date#5] -Right output [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [ss_customer_sk#1], [c_customer_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_last_name#11, c_first_name#10, d_date#5], [c_last_name#11, c_first_name#10, d_date#5] - -(15) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(16) CometExchange -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(c_last_name#11, c_first_name#10, d_date#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(18) CometColumnarToRow [codegen id : 3] -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Condition : isnotnull(cs_bill_customer_sk#12) - -(21) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#15, d_date#16] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_date#16] -Arguments: [cs_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(23) CometProject -Input [4]: [cs_bill_customer_sk#12, cs_sold_date_sk#13, d_date_sk#15, d_date#16] -Arguments: [cs_bill_customer_sk#12, d_date#16], [cs_bill_customer_sk#12, d_date#16] - -(24) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] - -(25) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, d_date#16] -Right output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [cs_bill_customer_sk#12], [c_customer_sk#17], Inner, BuildRight - -(26) CometProject -Input [5]: [cs_bill_customer_sk#12, d_date#16, c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [c_last_name#19, c_first_name#18, d_date#16], [c_last_name#19, c_first_name#18, d_date#16] - -(27) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(28) CometExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: hashpartitioning(c_last_name#19, c_first_name#18, d_date#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(30) CometColumnarToRow [codegen id : 1] -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] - -(31) BroadcastExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=3] - -(32) BroadcastHashJoin [codegen id : 3] -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 1970-01-01), isnull(d_date#16)] -Join type: LeftAnti -Join condition: None - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#21), dynamicpruningexpression(ws_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Condition : isnotnull(ws_bill_customer_sk#20) - -(35) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#23, d_date#24] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Right output [2]: [d_date_sk#23, d_date#24] -Arguments: [ws_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(37) CometProject -Input [4]: [ws_bill_customer_sk#20, ws_sold_date_sk#21, d_date_sk#23, d_date#24] -Arguments: [ws_bill_customer_sk#20, d_date#24], [ws_bill_customer_sk#20, d_date#24] - -(38) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] - -(39) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, d_date#24] -Right output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [ws_bill_customer_sk#20], [c_customer_sk#25], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_bill_customer_sk#20, d_date#24, c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_last_name#27, c_first_name#26, d_date#24], [c_last_name#27, c_first_name#26, d_date#24] - -(41) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(42) CometExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: hashpartitioning(c_last_name#27, c_first_name#26, d_date#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(43) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(44) CometColumnarToRow [codegen id : 2] -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] - -(45) BroadcastExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 3] -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#27, ), isnull(c_last_name#27), coalesce(c_first_name#26, ), isnull(c_first_name#26), coalesce(d_date#24, 1970-01-01), isnull(d_date#24)] -Join type: LeftAnti -Join condition: None - -(47) Project [codegen id : 3] -Output: [] -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] - -(48) HashAggregate [codegen id : 3] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#28] -Results [1]: [count#29] - -(49) CometColumnarExchange -Input [1]: [count#29] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(50) CometColumnarToRow [codegen id : 4] -Input [1]: [count#29] - -(51) HashAggregate [codegen id : 4] -Input [1]: [count#29] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#30] -Results [1]: [count(1)#30 AS count(1)#31] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (56) -+- * CometColumnarToRow (55) - +- CometProject (54) - +- CometFilter (53) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) - - -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(54) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(55) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] - -(56) BroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#3 - -Subquery:3 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#21 IN dynamicpruning#3 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/simplified.txt deleted file mode 100644 index f687139735..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -WholeStageCodegen (4) - HashAggregate [count] [count(1),count(1),count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (3) - HashAggregate [count,count] - Project - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #2 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] - CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_customer_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #4 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #7 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] - CometFilter [cs_bill_customer_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #9 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_bill_customer_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/explain.txt deleted file mode 100644 index fc6b12b834..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/explain.txt +++ /dev/null @@ -1,327 +0,0 @@ -== Physical Plan == -* HashAggregate (51) -+- * CometColumnarToRow (50) - +- CometColumnarExchange (49) - +- * HashAggregate (48) - +- * Project (47) - +- * BroadcastHashJoin LeftAnti BuildRight (46) - :- * BroadcastHashJoin LeftAnti BuildRight (32) - : :- * CometColumnarToRow (18) - : : +- CometHashAggregate (17) - : : +- CometExchange (16) - : : +- CometHashAggregate (15) - : : +- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometHashAggregate (29) - : +- CometExchange (28) - : +- CometHashAggregate (27) - : +- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (19) - : : +- ReusedExchange (21) - : +- ReusedExchange (24) - +- BroadcastExchange (45) - +- * CometColumnarToRow (44) - +- CometHashAggregate (43) - +- CometExchange (42) - +- CometHashAggregate (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : +- ReusedExchange (35) - +- ReusedExchange (38) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sold_date_sk#2 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Condition : isnotnull(ss_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(5) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: [d_date_sk#4, d_date#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Right output [2]: [d_date_sk#4, d_date#5] -Arguments: [ss_sold_date_sk#2], [d_date_sk#4], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] -Arguments: [ss_customer_sk#1, d_date#5], [ss_customer_sk#1, d_date#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Condition : isnotnull(c_customer_sk#7) - -(11) CometProject -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11], [c_customer_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#8, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#9, 30, true, false, true) AS c_last_name#11] - -(12) CometBroadcastExchange -Input [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, d_date#5] -Right output [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [ss_customer_sk#1], [c_customer_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_last_name#11, c_first_name#10, d_date#5], [c_last_name#11, c_first_name#10, d_date#5] - -(15) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(16) CometExchange -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(c_last_name#11, c_first_name#10, d_date#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(18) CometColumnarToRow [codegen id : 3] -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Condition : isnotnull(cs_bill_customer_sk#12) - -(21) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#15, d_date#16] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_date#16] -Arguments: [cs_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(23) CometProject -Input [4]: [cs_bill_customer_sk#12, cs_sold_date_sk#13, d_date_sk#15, d_date#16] -Arguments: [cs_bill_customer_sk#12, d_date#16], [cs_bill_customer_sk#12, d_date#16] - -(24) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] - -(25) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, d_date#16] -Right output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [cs_bill_customer_sk#12], [c_customer_sk#17], Inner, BuildRight - -(26) CometProject -Input [5]: [cs_bill_customer_sk#12, d_date#16, c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [c_last_name#19, c_first_name#18, d_date#16], [c_last_name#19, c_first_name#18, d_date#16] - -(27) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(28) CometExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: hashpartitioning(c_last_name#19, c_first_name#18, d_date#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(30) CometColumnarToRow [codegen id : 1] -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] - -(31) BroadcastExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=3] - -(32) BroadcastHashJoin [codegen id : 3] -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 1970-01-01), isnull(d_date#16)] -Join type: LeftAnti -Join condition: None - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#21), dynamicpruningexpression(ws_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Condition : isnotnull(ws_bill_customer_sk#20) - -(35) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#23, d_date#24] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Right output [2]: [d_date_sk#23, d_date#24] -Arguments: [ws_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(37) CometProject -Input [4]: [ws_bill_customer_sk#20, ws_sold_date_sk#21, d_date_sk#23, d_date#24] -Arguments: [ws_bill_customer_sk#20, d_date#24], [ws_bill_customer_sk#20, d_date#24] - -(38) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] - -(39) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, d_date#24] -Right output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [ws_bill_customer_sk#20], [c_customer_sk#25], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_bill_customer_sk#20, d_date#24, c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_last_name#27, c_first_name#26, d_date#24], [c_last_name#27, c_first_name#26, d_date#24] - -(41) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(42) CometExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: hashpartitioning(c_last_name#27, c_first_name#26, d_date#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(43) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(44) CometColumnarToRow [codegen id : 2] -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] - -(45) BroadcastExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 3] -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#27, ), isnull(c_last_name#27), coalesce(c_first_name#26, ), isnull(c_first_name#26), coalesce(d_date#24, 1970-01-01), isnull(d_date#24)] -Join type: LeftAnti -Join condition: None - -(47) Project [codegen id : 3] -Output: [] -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] - -(48) HashAggregate [codegen id : 3] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#28] -Results [1]: [count#29] - -(49) CometColumnarExchange -Input [1]: [count#29] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(50) CometColumnarToRow [codegen id : 4] -Input [1]: [count#29] - -(51) HashAggregate [codegen id : 4] -Input [1]: [count#29] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#30] -Results [1]: [count(1)#30 AS count(1)#31] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (56) -+- * CometColumnarToRow (55) - +- CometProject (54) - +- CometFilter (53) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) - - -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(54) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(55) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] - -(56) BroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#3 - -Subquery:3 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#21 IN dynamicpruning#3 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/extended.txt deleted file mode 100644 index ddee139acf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/extended.txt +++ /dev/null @@ -1,73 +0,0 @@ -HashAggregate -+- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] - : :- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 55 out of 66 eligible operators (83%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index f687139735..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -WholeStageCodegen (4) - HashAggregate [count] [count(1),count(1),count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (3) - HashAggregate [count,count] - Project - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #2 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] - CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_customer_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #4 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #7 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] - CometFilter [cs_bill_customer_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #9 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_bill_customer_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_datafusion/explain.txt deleted file mode 100644 index 25b4c305f2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_datafusion/explain.txt +++ /dev/null @@ -1,927 +0,0 @@ -== Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (172) -:- * BroadcastNestedLoopJoin Inner BuildRight (151) -: :- * BroadcastNestedLoopJoin Inner BuildRight (130) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (109) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) -: : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) -: : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : : : : : :- * CometColumnarToRow (25) -: : : : : : : +- CometHashAggregate (24) -: : : : : : : +- CometExchange (23) -: : : : : : : +- CometHashAggregate (22) -: : : : : : : +- CometProject (21) -: : : : : : : +- CometBroadcastHashJoin (20) -: : : : : : : :- CometProject (15) -: : : : : : : : +- CometBroadcastHashJoin (14) -: : : : : : : : :- CometProject (9) -: : : : : : : : : +- CometBroadcastHashJoin (8) -: : : : : : : : : :- CometProject (3) -: : : : : : : : : : +- CometFilter (2) -: : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) -: : : : : : : : : +- CometBroadcastExchange (7) -: : : : : : : : : +- CometProject (6) -: : : : : : : : : +- CometFilter (5) -: : : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (4) -: : : : : : : : +- CometBroadcastExchange (13) -: : : : : : : : +- CometProject (12) -: : : : : : : : +- CometFilter (11) -: : : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (10) -: : : : : : : +- CometBroadcastExchange (19) -: : : : : : : +- CometProject (18) -: : : : : : : +- CometFilter (17) -: : : : : : : +- CometNativeScan parquet spark_catalog.default.store (16) -: : : : : : +- BroadcastExchange (45) -: : : : : : +- * CometColumnarToRow (44) -: : : : : : +- CometHashAggregate (43) -: : : : : : +- CometExchange (42) -: : : : : : +- CometHashAggregate (41) -: : : : : : +- CometProject (40) -: : : : : : +- CometBroadcastHashJoin (39) -: : : : : : :- CometProject (37) -: : : : : : : +- CometBroadcastHashJoin (36) -: : : : : : : :- CometProject (31) -: : : : : : : : +- CometBroadcastHashJoin (30) -: : : : : : : : :- CometProject (28) -: : : : : : : : : +- CometFilter (27) -: : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (26) -: : : : : : : : +- ReusedExchange (29) -: : : : : : : +- CometBroadcastExchange (35) -: : : : : : : +- CometProject (34) -: : : : : : : +- CometFilter (33) -: : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (32) -: : : : : : +- ReusedExchange (38) -: : : : : +- BroadcastExchange (66) -: : : : : +- * CometColumnarToRow (65) -: : : : : +- CometHashAggregate (64) -: : : : : +- CometExchange (63) -: : : : : +- CometHashAggregate (62) -: : : : : +- CometProject (61) -: : : : : +- CometBroadcastHashJoin (60) -: : : : : :- CometProject (58) -: : : : : : +- CometBroadcastHashJoin (57) -: : : : : : :- CometProject (52) -: : : : : : : +- CometBroadcastHashJoin (51) -: : : : : : : :- CometProject (49) -: : : : : : : : +- CometFilter (48) -: : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (47) -: : : : : : : +- ReusedExchange (50) -: : : : : : +- CometBroadcastExchange (56) -: : : : : : +- CometProject (55) -: : : : : : +- CometFilter (54) -: : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (53) -: : : : : +- ReusedExchange (59) -: : : : +- BroadcastExchange (87) -: : : : +- * CometColumnarToRow (86) -: : : : +- CometHashAggregate (85) -: : : : +- CometExchange (84) -: : : : +- CometHashAggregate (83) -: : : : +- CometProject (82) -: : : : +- CometBroadcastHashJoin (81) -: : : : :- CometProject (79) -: : : : : +- CometBroadcastHashJoin (78) -: : : : : :- CometProject (73) -: : : : : : +- CometBroadcastHashJoin (72) -: : : : : : :- CometProject (70) -: : : : : : : +- CometFilter (69) -: : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (68) -: : : : : : +- ReusedExchange (71) -: : : : : +- CometBroadcastExchange (77) -: : : : : +- CometProject (76) -: : : : : +- CometFilter (75) -: : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (74) -: : : : +- ReusedExchange (80) -: : : +- BroadcastExchange (108) -: : : +- * CometColumnarToRow (107) -: : : +- CometHashAggregate (106) -: : : +- CometExchange (105) -: : : +- CometHashAggregate (104) -: : : +- CometProject (103) -: : : +- CometBroadcastHashJoin (102) -: : : :- CometProject (100) -: : : : +- CometBroadcastHashJoin (99) -: : : : :- CometProject (94) -: : : : : +- CometBroadcastHashJoin (93) -: : : : : :- CometProject (91) -: : : : : : +- CometFilter (90) -: : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (89) -: : : : : +- ReusedExchange (92) -: : : : +- CometBroadcastExchange (98) -: : : : +- CometProject (97) -: : : : +- CometFilter (96) -: : : : +- CometNativeScan parquet spark_catalog.default.time_dim (95) -: : : +- ReusedExchange (101) -: : +- BroadcastExchange (129) -: : +- * CometColumnarToRow (128) -: : +- CometHashAggregate (127) -: : +- CometExchange (126) -: : +- CometHashAggregate (125) -: : +- CometProject (124) -: : +- CometBroadcastHashJoin (123) -: : :- CometProject (121) -: : : +- CometBroadcastHashJoin (120) -: : : :- CometProject (115) -: : : : +- CometBroadcastHashJoin (114) -: : : : :- CometProject (112) -: : : : : +- CometFilter (111) -: : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (110) -: : : : +- ReusedExchange (113) -: : : +- CometBroadcastExchange (119) -: : : +- CometProject (118) -: : : +- CometFilter (117) -: : : +- CometNativeScan parquet spark_catalog.default.time_dim (116) -: : +- ReusedExchange (122) -: +- BroadcastExchange (150) -: +- * CometColumnarToRow (149) -: +- CometHashAggregate (148) -: +- CometExchange (147) -: +- CometHashAggregate (146) -: +- CometProject (145) -: +- CometBroadcastHashJoin (144) -: :- CometProject (142) -: : +- CometBroadcastHashJoin (141) -: : :- CometProject (136) -: : : +- CometBroadcastHashJoin (135) -: : : :- CometProject (133) -: : : : +- CometFilter (132) -: : : : +- CometNativeScan parquet spark_catalog.default.store_sales (131) -: : : +- ReusedExchange (134) -: : +- CometBroadcastExchange (140) -: : +- CometProject (139) -: : +- CometFilter (138) -: : +- CometNativeScan parquet spark_catalog.default.time_dim (137) -: +- ReusedExchange (143) -+- BroadcastExchange (171) - +- * CometColumnarToRow (170) - +- CometHashAggregate (169) - +- CometExchange (168) - +- CometHashAggregate (167) - +- CometProject (166) - +- CometBroadcastHashJoin (165) - :- CometProject (163) - : +- CometBroadcastHashJoin (162) - : :- CometProject (157) - : : +- CometBroadcastHashJoin (156) - : : :- CometProject (154) - : : : +- CometFilter (153) - : : : +- CometNativeScan parquet spark_catalog.default.store_sales (152) - : : +- ReusedExchange (155) - : +- CometBroadcastExchange (161) - : +- CometProject (160) - : +- CometFilter (159) - : +- CometNativeScan parquet spark_catalog.default.time_dim (158) - +- ReusedExchange (164) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(3) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(4) CometNativeScan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(Or(And(EqualTo(hd_dep_count,4),LessThanOrEqual(hd_vehicle_count,6)),And(EqualTo(hd_dep_count,2),LessThanOrEqual(hd_vehicle_count,4))),And(EqualTo(hd_dep_count,0),LessThanOrEqual(hd_vehicle_count,2))), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Condition : (((((hd_dep_count#6 = 4) AND (hd_vehicle_count#7 <= 6)) OR ((hd_dep_count#6 = 2) AND (hd_vehicle_count#7 <= 4))) OR ((hd_dep_count#6 = 0) AND (hd_vehicle_count#7 <= 2))) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] -Arguments: [ss_sold_time_sk#1, ss_store_sk#3], [ss_sold_time_sk#1, ss_store_sk#3] - -(10) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,8), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Condition : ((((isnotnull(t_hour#9) AND isnotnull(t_minute#10)) AND (t_hour#9 = 8)) AND (t_minute#10 >= 30)) AND isnotnull(t_time_sk#8)) - -(12) CometProject -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Arguments: [t_time_sk#8], [t_time_sk#8] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#8] -Arguments: [t_time_sk#8] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Right output [1]: [t_time_sk#8] -Arguments: [ss_sold_time_sk#1], [t_time_sk#8], Inner, BuildRight - -(15) CometProject -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#8] -Arguments: [ss_store_sk#3], [ss_store_sk#3] - -(16) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_store_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [s_store_sk#11, s_store_name#12] -Condition : ((isnotnull(s_store_name#12) AND (s_store_name#12 = ese)) AND isnotnull(s_store_sk#11)) - -(18) CometProject -Input [2]: [s_store_sk#11, s_store_name#12] -Arguments: [s_store_sk#11], [s_store_sk#11] - -(19) CometBroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: [s_store_sk#11] - -(20) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#3] -Right output [1]: [s_store_sk#11] -Arguments: [ss_store_sk#3], [s_store_sk#11], Inner, BuildRight - -(21) CometProject -Input [2]: [ss_store_sk#3, s_store_sk#11] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#13] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#13] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 8] -Input [1]: [h8_30_to_9#14] - -(26) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Condition : ((isnotnull(ss_hdemo_sk#16) AND isnotnull(ss_sold_time_sk#15)) AND isnotnull(ss_store_sk#17)) - -(28) CometProject -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Arguments: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17], [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] - -(29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#19] - -(30) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] -Right output [1]: [hd_demo_sk#19] -Arguments: [ss_hdemo_sk#16], [hd_demo_sk#19], Inner, BuildRight - -(31) CometProject -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, hd_demo_sk#19] -Arguments: [ss_sold_time_sk#15, ss_store_sk#17], [ss_sold_time_sk#15, ss_store_sk#17] - -(32) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Condition : ((((isnotnull(t_hour#21) AND isnotnull(t_minute#22)) AND (t_hour#21 = 9)) AND (t_minute#22 < 30)) AND isnotnull(t_time_sk#20)) - -(34) CometProject -Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Arguments: [t_time_sk#20], [t_time_sk#20] - -(35) CometBroadcastExchange -Input [1]: [t_time_sk#20] -Arguments: [t_time_sk#20] - -(36) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#15, ss_store_sk#17] -Right output [1]: [t_time_sk#20] -Arguments: [ss_sold_time_sk#15], [t_time_sk#20], Inner, BuildRight - -(37) CometProject -Input [3]: [ss_sold_time_sk#15, ss_store_sk#17, t_time_sk#20] -Arguments: [ss_store_sk#17], [ss_store_sk#17] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#23] - -(39) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#17] -Right output [1]: [s_store_sk#23] -Arguments: [ss_store_sk#17], [s_store_sk#23], Inner, BuildRight - -(40) CometProject -Input [2]: [ss_store_sk#17, s_store_sk#23] - -(41) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(42) CometExchange -Input [1]: [count#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(43) CometHashAggregate -Input [1]: [count#24] -Keys: [] -Functions [1]: [count(1)] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [h9_to_9_30#25] - -(45) BroadcastExchange -Input [1]: [h9_to_9_30#25] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(46) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(47) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(48) CometFilter -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_hdemo_sk#27) AND isnotnull(ss_sold_time_sk#26)) AND isnotnull(ss_store_sk#28)) - -(49) CometProject -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Arguments: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28], [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] - -(50) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#30] - -(51) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] -Right output [1]: [hd_demo_sk#30] -Arguments: [ss_hdemo_sk#27], [hd_demo_sk#30], Inner, BuildRight - -(52) CometProject -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, hd_demo_sk#30] -Arguments: [ss_sold_time_sk#26, ss_store_sk#28], [ss_sold_time_sk#26, ss_store_sk#28] - -(53) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Condition : ((((isnotnull(t_hour#32) AND isnotnull(t_minute#33)) AND (t_hour#32 = 9)) AND (t_minute#33 >= 30)) AND isnotnull(t_time_sk#31)) - -(55) CometProject -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Arguments: [t_time_sk#31], [t_time_sk#31] - -(56) CometBroadcastExchange -Input [1]: [t_time_sk#31] -Arguments: [t_time_sk#31] - -(57) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#26, ss_store_sk#28] -Right output [1]: [t_time_sk#31] -Arguments: [ss_sold_time_sk#26], [t_time_sk#31], Inner, BuildRight - -(58) CometProject -Input [3]: [ss_sold_time_sk#26, ss_store_sk#28, t_time_sk#31] -Arguments: [ss_store_sk#28], [ss_store_sk#28] - -(59) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#34] - -(60) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#28] -Right output [1]: [s_store_sk#34] -Arguments: [ss_store_sk#28], [s_store_sk#34], Inner, BuildRight - -(61) CometProject -Input [2]: [ss_store_sk#28, s_store_sk#34] - -(62) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(63) CometExchange -Input [1]: [count#35] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(64) CometHashAggregate -Input [1]: [count#35] -Keys: [] -Functions [1]: [count(1)] - -(65) CometColumnarToRow [codegen id : 2] -Input [1]: [h9_30_to_10#36] - -(66) BroadcastExchange -Input [1]: [h9_30_to_10#36] -Arguments: IdentityBroadcastMode, [plan_id=5] - -(67) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(68) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(69) CometFilter -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Condition : ((isnotnull(ss_hdemo_sk#38) AND isnotnull(ss_sold_time_sk#37)) AND isnotnull(ss_store_sk#39)) - -(70) CometProject -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Arguments: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39], [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] - -(71) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#41] - -(72) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] -Right output [1]: [hd_demo_sk#41] -Arguments: [ss_hdemo_sk#38], [hd_demo_sk#41], Inner, BuildRight - -(73) CometProject -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, hd_demo_sk#41] -Arguments: [ss_sold_time_sk#37, ss_store_sk#39], [ss_sold_time_sk#37, ss_store_sk#39] - -(74) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(75) CometFilter -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Condition : ((((isnotnull(t_hour#43) AND isnotnull(t_minute#44)) AND (t_hour#43 = 10)) AND (t_minute#44 < 30)) AND isnotnull(t_time_sk#42)) - -(76) CometProject -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Arguments: [t_time_sk#42], [t_time_sk#42] - -(77) CometBroadcastExchange -Input [1]: [t_time_sk#42] -Arguments: [t_time_sk#42] - -(78) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#37, ss_store_sk#39] -Right output [1]: [t_time_sk#42] -Arguments: [ss_sold_time_sk#37], [t_time_sk#42], Inner, BuildRight - -(79) CometProject -Input [3]: [ss_sold_time_sk#37, ss_store_sk#39, t_time_sk#42] -Arguments: [ss_store_sk#39], [ss_store_sk#39] - -(80) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#45] - -(81) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#39] -Right output [1]: [s_store_sk#45] -Arguments: [ss_store_sk#39], [s_store_sk#45], Inner, BuildRight - -(82) CometProject -Input [2]: [ss_store_sk#39, s_store_sk#45] - -(83) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(84) CometExchange -Input [1]: [count#46] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(85) CometHashAggregate -Input [1]: [count#46] -Keys: [] -Functions [1]: [count(1)] - -(86) CometColumnarToRow [codegen id : 3] -Input [1]: [h10_to_10_30#47] - -(87) BroadcastExchange -Input [1]: [h10_to_10_30#47] -Arguments: IdentityBroadcastMode, [plan_id=7] - -(88) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(89) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(90) CometFilter -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Condition : ((isnotnull(ss_hdemo_sk#49) AND isnotnull(ss_sold_time_sk#48)) AND isnotnull(ss_store_sk#50)) - -(91) CometProject -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Arguments: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50], [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] - -(92) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#52] - -(93) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] -Right output [1]: [hd_demo_sk#52] -Arguments: [ss_hdemo_sk#49], [hd_demo_sk#52], Inner, BuildRight - -(94) CometProject -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, hd_demo_sk#52] -Arguments: [ss_sold_time_sk#48, ss_store_sk#50], [ss_sold_time_sk#48, ss_store_sk#50] - -(95) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(96) CometFilter -Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Condition : ((((isnotnull(t_hour#54) AND isnotnull(t_minute#55)) AND (t_hour#54 = 10)) AND (t_minute#55 >= 30)) AND isnotnull(t_time_sk#53)) - -(97) CometProject -Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Arguments: [t_time_sk#53], [t_time_sk#53] - -(98) CometBroadcastExchange -Input [1]: [t_time_sk#53] -Arguments: [t_time_sk#53] - -(99) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#48, ss_store_sk#50] -Right output [1]: [t_time_sk#53] -Arguments: [ss_sold_time_sk#48], [t_time_sk#53], Inner, BuildRight - -(100) CometProject -Input [3]: [ss_sold_time_sk#48, ss_store_sk#50, t_time_sk#53] -Arguments: [ss_store_sk#50], [ss_store_sk#50] - -(101) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#56] - -(102) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#50] -Right output [1]: [s_store_sk#56] -Arguments: [ss_store_sk#50], [s_store_sk#56], Inner, BuildRight - -(103) CometProject -Input [2]: [ss_store_sk#50, s_store_sk#56] - -(104) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(105) CometExchange -Input [1]: [count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(106) CometHashAggregate -Input [1]: [count#57] -Keys: [] -Functions [1]: [count(1)] - -(107) CometColumnarToRow [codegen id : 4] -Input [1]: [h10_30_to_11#58] - -(108) BroadcastExchange -Input [1]: [h10_30_to_11#58] -Arguments: IdentityBroadcastMode, [plan_id=9] - -(109) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(110) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(111) CometFilter -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Condition : ((isnotnull(ss_hdemo_sk#60) AND isnotnull(ss_sold_time_sk#59)) AND isnotnull(ss_store_sk#61)) - -(112) CometProject -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Arguments: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61], [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] - -(113) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#63] - -(114) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] -Right output [1]: [hd_demo_sk#63] -Arguments: [ss_hdemo_sk#60], [hd_demo_sk#63], Inner, BuildRight - -(115) CometProject -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, hd_demo_sk#63] -Arguments: [ss_sold_time_sk#59, ss_store_sk#61], [ss_sold_time_sk#59, ss_store_sk#61] - -(116) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(117) CometFilter -Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Condition : ((((isnotnull(t_hour#65) AND isnotnull(t_minute#66)) AND (t_hour#65 = 11)) AND (t_minute#66 < 30)) AND isnotnull(t_time_sk#64)) - -(118) CometProject -Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Arguments: [t_time_sk#64], [t_time_sk#64] - -(119) CometBroadcastExchange -Input [1]: [t_time_sk#64] -Arguments: [t_time_sk#64] - -(120) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#59, ss_store_sk#61] -Right output [1]: [t_time_sk#64] -Arguments: [ss_sold_time_sk#59], [t_time_sk#64], Inner, BuildRight - -(121) CometProject -Input [3]: [ss_sold_time_sk#59, ss_store_sk#61, t_time_sk#64] -Arguments: [ss_store_sk#61], [ss_store_sk#61] - -(122) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#67] - -(123) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#61] -Right output [1]: [s_store_sk#67] -Arguments: [ss_store_sk#61], [s_store_sk#67], Inner, BuildRight - -(124) CometProject -Input [2]: [ss_store_sk#61, s_store_sk#67] - -(125) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(126) CometExchange -Input [1]: [count#68] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(127) CometHashAggregate -Input [1]: [count#68] -Keys: [] -Functions [1]: [count(1)] - -(128) CometColumnarToRow [codegen id : 5] -Input [1]: [h11_to_11_30#69] - -(129) BroadcastExchange -Input [1]: [h11_to_11_30#69] -Arguments: IdentityBroadcastMode, [plan_id=11] - -(130) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(131) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(132) CometFilter -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Condition : ((isnotnull(ss_hdemo_sk#71) AND isnotnull(ss_sold_time_sk#70)) AND isnotnull(ss_store_sk#72)) - -(133) CometProject -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Arguments: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72], [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] - -(134) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#74] - -(135) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] -Right output [1]: [hd_demo_sk#74] -Arguments: [ss_hdemo_sk#71], [hd_demo_sk#74], Inner, BuildRight - -(136) CometProject -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, hd_demo_sk#74] -Arguments: [ss_sold_time_sk#70, ss_store_sk#72], [ss_sold_time_sk#70, ss_store_sk#72] - -(137) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(138) CometFilter -Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Condition : ((((isnotnull(t_hour#76) AND isnotnull(t_minute#77)) AND (t_hour#76 = 11)) AND (t_minute#77 >= 30)) AND isnotnull(t_time_sk#75)) - -(139) CometProject -Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Arguments: [t_time_sk#75], [t_time_sk#75] - -(140) CometBroadcastExchange -Input [1]: [t_time_sk#75] -Arguments: [t_time_sk#75] - -(141) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#70, ss_store_sk#72] -Right output [1]: [t_time_sk#75] -Arguments: [ss_sold_time_sk#70], [t_time_sk#75], Inner, BuildRight - -(142) CometProject -Input [3]: [ss_sold_time_sk#70, ss_store_sk#72, t_time_sk#75] -Arguments: [ss_store_sk#72], [ss_store_sk#72] - -(143) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#78] - -(144) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#72] -Right output [1]: [s_store_sk#78] -Arguments: [ss_store_sk#72], [s_store_sk#78], Inner, BuildRight - -(145) CometProject -Input [2]: [ss_store_sk#72, s_store_sk#78] - -(146) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(147) CometExchange -Input [1]: [count#79] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(148) CometHashAggregate -Input [1]: [count#79] -Keys: [] -Functions [1]: [count(1)] - -(149) CometColumnarToRow [codegen id : 6] -Input [1]: [h11_30_to_12#80] - -(150) BroadcastExchange -Input [1]: [h11_30_to_12#80] -Arguments: IdentityBroadcastMode, [plan_id=13] - -(151) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(152) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(153) CometFilter -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Condition : ((isnotnull(ss_hdemo_sk#82) AND isnotnull(ss_sold_time_sk#81)) AND isnotnull(ss_store_sk#83)) - -(154) CometProject -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Arguments: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83], [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] - -(155) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#85] - -(156) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] -Right output [1]: [hd_demo_sk#85] -Arguments: [ss_hdemo_sk#82], [hd_demo_sk#85], Inner, BuildRight - -(157) CometProject -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, hd_demo_sk#85] -Arguments: [ss_sold_time_sk#81, ss_store_sk#83], [ss_sold_time_sk#81, ss_store_sk#83] - -(158) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(159) CometFilter -Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Condition : ((((isnotnull(t_hour#87) AND isnotnull(t_minute#88)) AND (t_hour#87 = 12)) AND (t_minute#88 < 30)) AND isnotnull(t_time_sk#86)) - -(160) CometProject -Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Arguments: [t_time_sk#86], [t_time_sk#86] - -(161) CometBroadcastExchange -Input [1]: [t_time_sk#86] -Arguments: [t_time_sk#86] - -(162) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#81, ss_store_sk#83] -Right output [1]: [t_time_sk#86] -Arguments: [ss_sold_time_sk#81], [t_time_sk#86], Inner, BuildRight - -(163) CometProject -Input [3]: [ss_sold_time_sk#81, ss_store_sk#83, t_time_sk#86] -Arguments: [ss_store_sk#83], [ss_store_sk#83] - -(164) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#89] - -(165) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#83] -Right output [1]: [s_store_sk#89] -Arguments: [ss_store_sk#83], [s_store_sk#89], Inner, BuildRight - -(166) CometProject -Input [2]: [ss_store_sk#83, s_store_sk#89] - -(167) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(168) CometExchange -Input [1]: [count#90] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(169) CometHashAggregate -Input [1]: [count#90] -Keys: [] -Functions [1]: [count(1)] - -(170) CometColumnarToRow [codegen id : 7] -Input [1]: [h12_to_12_30#91] - -(171) BroadcastExchange -Input [1]: [h12_to_12_30#91] -Arguments: IdentityBroadcastMode, [plan_id=15] - -(172) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_datafusion/simplified.txt deleted file mode 100644 index cef8e0e760..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_datafusion/simplified.txt +++ /dev/null @@ -1,195 +0,0 @@ -WholeStageCodegen (8) - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h8_30_to_9,count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_to_9_30,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan 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 [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_30_to_10,count(1)] - CometExchange #9 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan 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 [t_time_sk] #10 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_to_10_30,count(1)] - CometExchange #12 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan 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 [t_time_sk] #13 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_30_to_11,count(1)] - CometExchange #15 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan 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 [t_time_sk] #16 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_to_11_30,count(1)] - CometExchange #18 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan 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 [t_time_sk] #19 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_30_to_12,count(1)] - CometExchange #21 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan 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 [t_time_sk] #22 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #23 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h12_to_12_30,count(1)] - CometExchange #24 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan 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 [t_time_sk] #25 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan 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/q88.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_iceberg_compat/explain.txt deleted file mode 100644 index 2e49d20af4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,927 +0,0 @@ -== Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (172) -:- * BroadcastNestedLoopJoin Inner BuildRight (151) -: :- * BroadcastNestedLoopJoin Inner BuildRight (130) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (109) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) -: : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) -: : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : : : : : :- * CometColumnarToRow (25) -: : : : : : : +- CometHashAggregate (24) -: : : : : : : +- CometExchange (23) -: : : : : : : +- CometHashAggregate (22) -: : : : : : : +- CometProject (21) -: : : : : : : +- CometBroadcastHashJoin (20) -: : : : : : : :- CometProject (15) -: : : : : : : : +- CometBroadcastHashJoin (14) -: : : : : : : : :- CometProject (9) -: : : : : : : : : +- CometBroadcastHashJoin (8) -: : : : : : : : : :- CometProject (3) -: : : : : : : : : : +- CometFilter (2) -: : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) -: : : : : : : : : +- CometBroadcastExchange (7) -: : : : : : : : : +- CometProject (6) -: : : : : : : : : +- CometFilter (5) -: : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (4) -: : : : : : : : +- CometBroadcastExchange (13) -: : : : : : : : +- CometProject (12) -: : : : : : : : +- CometFilter (11) -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (10) -: : : : : : : +- CometBroadcastExchange (19) -: : : : : : : +- CometProject (18) -: : : : : : : +- CometFilter (17) -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (16) -: : : : : : +- BroadcastExchange (45) -: : : : : : +- * CometColumnarToRow (44) -: : : : : : +- CometHashAggregate (43) -: : : : : : +- CometExchange (42) -: : : : : : +- CometHashAggregate (41) -: : : : : : +- CometProject (40) -: : : : : : +- CometBroadcastHashJoin (39) -: : : : : : :- CometProject (37) -: : : : : : : +- CometBroadcastHashJoin (36) -: : : : : : : :- CometProject (31) -: : : : : : : : +- CometBroadcastHashJoin (30) -: : : : : : : : :- CometProject (28) -: : : : : : : : : +- CometFilter (27) -: : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (26) -: : : : : : : : +- ReusedExchange (29) -: : : : : : : +- CometBroadcastExchange (35) -: : : : : : : +- CometProject (34) -: : : : : : : +- CometFilter (33) -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (32) -: : : : : : +- ReusedExchange (38) -: : : : : +- BroadcastExchange (66) -: : : : : +- * CometColumnarToRow (65) -: : : : : +- CometHashAggregate (64) -: : : : : +- CometExchange (63) -: : : : : +- CometHashAggregate (62) -: : : : : +- CometProject (61) -: : : : : +- CometBroadcastHashJoin (60) -: : : : : :- CometProject (58) -: : : : : : +- CometBroadcastHashJoin (57) -: : : : : : :- CometProject (52) -: : : : : : : +- CometBroadcastHashJoin (51) -: : : : : : : :- CometProject (49) -: : : : : : : : +- CometFilter (48) -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (47) -: : : : : : : +- ReusedExchange (50) -: : : : : : +- CometBroadcastExchange (56) -: : : : : : +- CometProject (55) -: : : : : : +- CometFilter (54) -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (53) -: : : : : +- ReusedExchange (59) -: : : : +- BroadcastExchange (87) -: : : : +- * CometColumnarToRow (86) -: : : : +- CometHashAggregate (85) -: : : : +- CometExchange (84) -: : : : +- CometHashAggregate (83) -: : : : +- CometProject (82) -: : : : +- CometBroadcastHashJoin (81) -: : : : :- CometProject (79) -: : : : : +- CometBroadcastHashJoin (78) -: : : : : :- CometProject (73) -: : : : : : +- CometBroadcastHashJoin (72) -: : : : : : :- CometProject (70) -: : : : : : : +- CometFilter (69) -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (68) -: : : : : : +- ReusedExchange (71) -: : : : : +- CometBroadcastExchange (77) -: : : : : +- CometProject (76) -: : : : : +- CometFilter (75) -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (74) -: : : : +- ReusedExchange (80) -: : : +- BroadcastExchange (108) -: : : +- * CometColumnarToRow (107) -: : : +- CometHashAggregate (106) -: : : +- CometExchange (105) -: : : +- CometHashAggregate (104) -: : : +- CometProject (103) -: : : +- CometBroadcastHashJoin (102) -: : : :- CometProject (100) -: : : : +- CometBroadcastHashJoin (99) -: : : : :- CometProject (94) -: : : : : +- CometBroadcastHashJoin (93) -: : : : : :- CometProject (91) -: : : : : : +- CometFilter (90) -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (89) -: : : : : +- ReusedExchange (92) -: : : : +- CometBroadcastExchange (98) -: : : : +- CometProject (97) -: : : : +- CometFilter (96) -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (95) -: : : +- ReusedExchange (101) -: : +- BroadcastExchange (129) -: : +- * CometColumnarToRow (128) -: : +- CometHashAggregate (127) -: : +- CometExchange (126) -: : +- CometHashAggregate (125) -: : +- CometProject (124) -: : +- CometBroadcastHashJoin (123) -: : :- CometProject (121) -: : : +- CometBroadcastHashJoin (120) -: : : :- CometProject (115) -: : : : +- CometBroadcastHashJoin (114) -: : : : :- CometProject (112) -: : : : : +- CometFilter (111) -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (110) -: : : : +- ReusedExchange (113) -: : : +- CometBroadcastExchange (119) -: : : +- CometProject (118) -: : : +- CometFilter (117) -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (116) -: : +- ReusedExchange (122) -: +- BroadcastExchange (150) -: +- * CometColumnarToRow (149) -: +- CometHashAggregate (148) -: +- CometExchange (147) -: +- CometHashAggregate (146) -: +- CometProject (145) -: +- CometBroadcastHashJoin (144) -: :- CometProject (142) -: : +- CometBroadcastHashJoin (141) -: : :- CometProject (136) -: : : +- CometBroadcastHashJoin (135) -: : : :- CometProject (133) -: : : : +- CometFilter (132) -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (131) -: : : +- ReusedExchange (134) -: : +- CometBroadcastExchange (140) -: : +- CometProject (139) -: : +- CometFilter (138) -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (137) -: +- ReusedExchange (143) -+- BroadcastExchange (171) - +- * CometColumnarToRow (170) - +- CometHashAggregate (169) - +- CometExchange (168) - +- CometHashAggregate (167) - +- CometProject (166) - +- CometBroadcastHashJoin (165) - :- CometProject (163) - : +- CometBroadcastHashJoin (162) - : :- CometProject (157) - : : +- CometBroadcastHashJoin (156) - : : :- CometProject (154) - : : : +- CometFilter (153) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (152) - : : +- ReusedExchange (155) - : +- CometBroadcastExchange (161) - : +- CometProject (160) - : +- CometFilter (159) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (158) - +- ReusedExchange (164) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(3) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(Or(And(EqualTo(hd_dep_count,4),LessThanOrEqual(hd_vehicle_count,6)),And(EqualTo(hd_dep_count,2),LessThanOrEqual(hd_vehicle_count,4))),And(EqualTo(hd_dep_count,0),LessThanOrEqual(hd_vehicle_count,2))), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Condition : (((((hd_dep_count#6 = 4) AND (hd_vehicle_count#7 <= 6)) OR ((hd_dep_count#6 = 2) AND (hd_vehicle_count#7 <= 4))) OR ((hd_dep_count#6 = 0) AND (hd_vehicle_count#7 <= 2))) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] -Arguments: [ss_sold_time_sk#1, ss_store_sk#3], [ss_sold_time_sk#1, ss_store_sk#3] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,8), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Condition : ((((isnotnull(t_hour#9) AND isnotnull(t_minute#10)) AND (t_hour#9 = 8)) AND (t_minute#10 >= 30)) AND isnotnull(t_time_sk#8)) - -(12) CometProject -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Arguments: [t_time_sk#8], [t_time_sk#8] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#8] -Arguments: [t_time_sk#8] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Right output [1]: [t_time_sk#8] -Arguments: [ss_sold_time_sk#1], [t_time_sk#8], Inner, BuildRight - -(15) CometProject -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#8] -Arguments: [ss_store_sk#3], [ss_store_sk#3] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_store_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [s_store_sk#11, s_store_name#12] -Condition : ((isnotnull(s_store_name#12) AND (s_store_name#12 = ese)) AND isnotnull(s_store_sk#11)) - -(18) CometProject -Input [2]: [s_store_sk#11, s_store_name#12] -Arguments: [s_store_sk#11], [s_store_sk#11] - -(19) CometBroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: [s_store_sk#11] - -(20) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#3] -Right output [1]: [s_store_sk#11] -Arguments: [ss_store_sk#3], [s_store_sk#11], Inner, BuildRight - -(21) CometProject -Input [2]: [ss_store_sk#3, s_store_sk#11] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#13] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#13] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 8] -Input [1]: [h8_30_to_9#14] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Condition : ((isnotnull(ss_hdemo_sk#16) AND isnotnull(ss_sold_time_sk#15)) AND isnotnull(ss_store_sk#17)) - -(28) CometProject -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Arguments: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17], [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] - -(29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#19] - -(30) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] -Right output [1]: [hd_demo_sk#19] -Arguments: [ss_hdemo_sk#16], [hd_demo_sk#19], Inner, BuildRight - -(31) CometProject -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, hd_demo_sk#19] -Arguments: [ss_sold_time_sk#15, ss_store_sk#17], [ss_sold_time_sk#15, ss_store_sk#17] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Condition : ((((isnotnull(t_hour#21) AND isnotnull(t_minute#22)) AND (t_hour#21 = 9)) AND (t_minute#22 < 30)) AND isnotnull(t_time_sk#20)) - -(34) CometProject -Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Arguments: [t_time_sk#20], [t_time_sk#20] - -(35) CometBroadcastExchange -Input [1]: [t_time_sk#20] -Arguments: [t_time_sk#20] - -(36) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#15, ss_store_sk#17] -Right output [1]: [t_time_sk#20] -Arguments: [ss_sold_time_sk#15], [t_time_sk#20], Inner, BuildRight - -(37) CometProject -Input [3]: [ss_sold_time_sk#15, ss_store_sk#17, t_time_sk#20] -Arguments: [ss_store_sk#17], [ss_store_sk#17] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#23] - -(39) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#17] -Right output [1]: [s_store_sk#23] -Arguments: [ss_store_sk#17], [s_store_sk#23], Inner, BuildRight - -(40) CometProject -Input [2]: [ss_store_sk#17, s_store_sk#23] - -(41) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(42) CometExchange -Input [1]: [count#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(43) CometHashAggregate -Input [1]: [count#24] -Keys: [] -Functions [1]: [count(1)] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [h9_to_9_30#25] - -(45) BroadcastExchange -Input [1]: [h9_to_9_30#25] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(46) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(48) CometFilter -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_hdemo_sk#27) AND isnotnull(ss_sold_time_sk#26)) AND isnotnull(ss_store_sk#28)) - -(49) CometProject -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Arguments: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28], [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] - -(50) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#30] - -(51) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] -Right output [1]: [hd_demo_sk#30] -Arguments: [ss_hdemo_sk#27], [hd_demo_sk#30], Inner, BuildRight - -(52) CometProject -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, hd_demo_sk#30] -Arguments: [ss_sold_time_sk#26, ss_store_sk#28], [ss_sold_time_sk#26, ss_store_sk#28] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Condition : ((((isnotnull(t_hour#32) AND isnotnull(t_minute#33)) AND (t_hour#32 = 9)) AND (t_minute#33 >= 30)) AND isnotnull(t_time_sk#31)) - -(55) CometProject -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Arguments: [t_time_sk#31], [t_time_sk#31] - -(56) CometBroadcastExchange -Input [1]: [t_time_sk#31] -Arguments: [t_time_sk#31] - -(57) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#26, ss_store_sk#28] -Right output [1]: [t_time_sk#31] -Arguments: [ss_sold_time_sk#26], [t_time_sk#31], Inner, BuildRight - -(58) CometProject -Input [3]: [ss_sold_time_sk#26, ss_store_sk#28, t_time_sk#31] -Arguments: [ss_store_sk#28], [ss_store_sk#28] - -(59) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#34] - -(60) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#28] -Right output [1]: [s_store_sk#34] -Arguments: [ss_store_sk#28], [s_store_sk#34], Inner, BuildRight - -(61) CometProject -Input [2]: [ss_store_sk#28, s_store_sk#34] - -(62) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(63) CometExchange -Input [1]: [count#35] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(64) CometHashAggregate -Input [1]: [count#35] -Keys: [] -Functions [1]: [count(1)] - -(65) CometColumnarToRow [codegen id : 2] -Input [1]: [h9_30_to_10#36] - -(66) BroadcastExchange -Input [1]: [h9_30_to_10#36] -Arguments: IdentityBroadcastMode, [plan_id=5] - -(67) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(69) CometFilter -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Condition : ((isnotnull(ss_hdemo_sk#38) AND isnotnull(ss_sold_time_sk#37)) AND isnotnull(ss_store_sk#39)) - -(70) CometProject -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Arguments: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39], [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] - -(71) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#41] - -(72) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] -Right output [1]: [hd_demo_sk#41] -Arguments: [ss_hdemo_sk#38], [hd_demo_sk#41], Inner, BuildRight - -(73) CometProject -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, hd_demo_sk#41] -Arguments: [ss_sold_time_sk#37, ss_store_sk#39], [ss_sold_time_sk#37, ss_store_sk#39] - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(75) CometFilter -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Condition : ((((isnotnull(t_hour#43) AND isnotnull(t_minute#44)) AND (t_hour#43 = 10)) AND (t_minute#44 < 30)) AND isnotnull(t_time_sk#42)) - -(76) CometProject -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Arguments: [t_time_sk#42], [t_time_sk#42] - -(77) CometBroadcastExchange -Input [1]: [t_time_sk#42] -Arguments: [t_time_sk#42] - -(78) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#37, ss_store_sk#39] -Right output [1]: [t_time_sk#42] -Arguments: [ss_sold_time_sk#37], [t_time_sk#42], Inner, BuildRight - -(79) CometProject -Input [3]: [ss_sold_time_sk#37, ss_store_sk#39, t_time_sk#42] -Arguments: [ss_store_sk#39], [ss_store_sk#39] - -(80) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#45] - -(81) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#39] -Right output [1]: [s_store_sk#45] -Arguments: [ss_store_sk#39], [s_store_sk#45], Inner, BuildRight - -(82) CometProject -Input [2]: [ss_store_sk#39, s_store_sk#45] - -(83) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(84) CometExchange -Input [1]: [count#46] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(85) CometHashAggregate -Input [1]: [count#46] -Keys: [] -Functions [1]: [count(1)] - -(86) CometColumnarToRow [codegen id : 3] -Input [1]: [h10_to_10_30#47] - -(87) BroadcastExchange -Input [1]: [h10_to_10_30#47] -Arguments: IdentityBroadcastMode, [plan_id=7] - -(88) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(89) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(90) CometFilter -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Condition : ((isnotnull(ss_hdemo_sk#49) AND isnotnull(ss_sold_time_sk#48)) AND isnotnull(ss_store_sk#50)) - -(91) CometProject -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Arguments: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50], [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] - -(92) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#52] - -(93) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] -Right output [1]: [hd_demo_sk#52] -Arguments: [ss_hdemo_sk#49], [hd_demo_sk#52], Inner, BuildRight - -(94) CometProject -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, hd_demo_sk#52] -Arguments: [ss_sold_time_sk#48, ss_store_sk#50], [ss_sold_time_sk#48, ss_store_sk#50] - -(95) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(96) CometFilter -Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Condition : ((((isnotnull(t_hour#54) AND isnotnull(t_minute#55)) AND (t_hour#54 = 10)) AND (t_minute#55 >= 30)) AND isnotnull(t_time_sk#53)) - -(97) CometProject -Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Arguments: [t_time_sk#53], [t_time_sk#53] - -(98) CometBroadcastExchange -Input [1]: [t_time_sk#53] -Arguments: [t_time_sk#53] - -(99) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#48, ss_store_sk#50] -Right output [1]: [t_time_sk#53] -Arguments: [ss_sold_time_sk#48], [t_time_sk#53], Inner, BuildRight - -(100) CometProject -Input [3]: [ss_sold_time_sk#48, ss_store_sk#50, t_time_sk#53] -Arguments: [ss_store_sk#50], [ss_store_sk#50] - -(101) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#56] - -(102) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#50] -Right output [1]: [s_store_sk#56] -Arguments: [ss_store_sk#50], [s_store_sk#56], Inner, BuildRight - -(103) CometProject -Input [2]: [ss_store_sk#50, s_store_sk#56] - -(104) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(105) CometExchange -Input [1]: [count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(106) CometHashAggregate -Input [1]: [count#57] -Keys: [] -Functions [1]: [count(1)] - -(107) CometColumnarToRow [codegen id : 4] -Input [1]: [h10_30_to_11#58] - -(108) BroadcastExchange -Input [1]: [h10_30_to_11#58] -Arguments: IdentityBroadcastMode, [plan_id=9] - -(109) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(110) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(111) CometFilter -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Condition : ((isnotnull(ss_hdemo_sk#60) AND isnotnull(ss_sold_time_sk#59)) AND isnotnull(ss_store_sk#61)) - -(112) CometProject -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Arguments: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61], [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] - -(113) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#63] - -(114) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] -Right output [1]: [hd_demo_sk#63] -Arguments: [ss_hdemo_sk#60], [hd_demo_sk#63], Inner, BuildRight - -(115) CometProject -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, hd_demo_sk#63] -Arguments: [ss_sold_time_sk#59, ss_store_sk#61], [ss_sold_time_sk#59, ss_store_sk#61] - -(116) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(117) CometFilter -Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Condition : ((((isnotnull(t_hour#65) AND isnotnull(t_minute#66)) AND (t_hour#65 = 11)) AND (t_minute#66 < 30)) AND isnotnull(t_time_sk#64)) - -(118) CometProject -Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Arguments: [t_time_sk#64], [t_time_sk#64] - -(119) CometBroadcastExchange -Input [1]: [t_time_sk#64] -Arguments: [t_time_sk#64] - -(120) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#59, ss_store_sk#61] -Right output [1]: [t_time_sk#64] -Arguments: [ss_sold_time_sk#59], [t_time_sk#64], Inner, BuildRight - -(121) CometProject -Input [3]: [ss_sold_time_sk#59, ss_store_sk#61, t_time_sk#64] -Arguments: [ss_store_sk#61], [ss_store_sk#61] - -(122) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#67] - -(123) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#61] -Right output [1]: [s_store_sk#67] -Arguments: [ss_store_sk#61], [s_store_sk#67], Inner, BuildRight - -(124) CometProject -Input [2]: [ss_store_sk#61, s_store_sk#67] - -(125) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(126) CometExchange -Input [1]: [count#68] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(127) CometHashAggregate -Input [1]: [count#68] -Keys: [] -Functions [1]: [count(1)] - -(128) CometColumnarToRow [codegen id : 5] -Input [1]: [h11_to_11_30#69] - -(129) BroadcastExchange -Input [1]: [h11_to_11_30#69] -Arguments: IdentityBroadcastMode, [plan_id=11] - -(130) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(131) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(132) CometFilter -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Condition : ((isnotnull(ss_hdemo_sk#71) AND isnotnull(ss_sold_time_sk#70)) AND isnotnull(ss_store_sk#72)) - -(133) CometProject -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Arguments: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72], [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] - -(134) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#74] - -(135) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] -Right output [1]: [hd_demo_sk#74] -Arguments: [ss_hdemo_sk#71], [hd_demo_sk#74], Inner, BuildRight - -(136) CometProject -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, hd_demo_sk#74] -Arguments: [ss_sold_time_sk#70, ss_store_sk#72], [ss_sold_time_sk#70, ss_store_sk#72] - -(137) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(138) CometFilter -Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Condition : ((((isnotnull(t_hour#76) AND isnotnull(t_minute#77)) AND (t_hour#76 = 11)) AND (t_minute#77 >= 30)) AND isnotnull(t_time_sk#75)) - -(139) CometProject -Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Arguments: [t_time_sk#75], [t_time_sk#75] - -(140) CometBroadcastExchange -Input [1]: [t_time_sk#75] -Arguments: [t_time_sk#75] - -(141) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#70, ss_store_sk#72] -Right output [1]: [t_time_sk#75] -Arguments: [ss_sold_time_sk#70], [t_time_sk#75], Inner, BuildRight - -(142) CometProject -Input [3]: [ss_sold_time_sk#70, ss_store_sk#72, t_time_sk#75] -Arguments: [ss_store_sk#72], [ss_store_sk#72] - -(143) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#78] - -(144) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#72] -Right output [1]: [s_store_sk#78] -Arguments: [ss_store_sk#72], [s_store_sk#78], Inner, BuildRight - -(145) CometProject -Input [2]: [ss_store_sk#72, s_store_sk#78] - -(146) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(147) CometExchange -Input [1]: [count#79] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(148) CometHashAggregate -Input [1]: [count#79] -Keys: [] -Functions [1]: [count(1)] - -(149) CometColumnarToRow [codegen id : 6] -Input [1]: [h11_30_to_12#80] - -(150) BroadcastExchange -Input [1]: [h11_30_to_12#80] -Arguments: IdentityBroadcastMode, [plan_id=13] - -(151) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(152) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(153) CometFilter -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Condition : ((isnotnull(ss_hdemo_sk#82) AND isnotnull(ss_sold_time_sk#81)) AND isnotnull(ss_store_sk#83)) - -(154) CometProject -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Arguments: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83], [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] - -(155) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#85] - -(156) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] -Right output [1]: [hd_demo_sk#85] -Arguments: [ss_hdemo_sk#82], [hd_demo_sk#85], Inner, BuildRight - -(157) CometProject -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, hd_demo_sk#85] -Arguments: [ss_sold_time_sk#81, ss_store_sk#83], [ss_sold_time_sk#81, ss_store_sk#83] - -(158) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(159) CometFilter -Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Condition : ((((isnotnull(t_hour#87) AND isnotnull(t_minute#88)) AND (t_hour#87 = 12)) AND (t_minute#88 < 30)) AND isnotnull(t_time_sk#86)) - -(160) CometProject -Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Arguments: [t_time_sk#86], [t_time_sk#86] - -(161) CometBroadcastExchange -Input [1]: [t_time_sk#86] -Arguments: [t_time_sk#86] - -(162) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#81, ss_store_sk#83] -Right output [1]: [t_time_sk#86] -Arguments: [ss_sold_time_sk#81], [t_time_sk#86], Inner, BuildRight - -(163) CometProject -Input [3]: [ss_sold_time_sk#81, ss_store_sk#83, t_time_sk#86] -Arguments: [ss_store_sk#83], [ss_store_sk#83] - -(164) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#89] - -(165) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#83] -Right output [1]: [s_store_sk#89] -Arguments: [ss_store_sk#83], [s_store_sk#89], Inner, BuildRight - -(166) CometProject -Input [2]: [ss_store_sk#83, s_store_sk#89] - -(167) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(168) CometExchange -Input [1]: [count#90] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(169) CometHashAggregate -Input [1]: [count#90] -Keys: [] -Functions [1]: [count(1)] - -(170) CometColumnarToRow [codegen id : 7] -Input [1]: [h12_to_12_30#91] - -(171) BroadcastExchange -Input [1]: [h12_to_12_30#91] -Arguments: IdentityBroadcastMode, [plan_id=15] - -(172) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_iceberg_compat/simplified.txt deleted file mode 100644 index e3923fb3ea..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,195 +0,0 @@ -WholeStageCodegen (8) - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h8_30_to_9,count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_to_9_30,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_30_to_10,count(1)] - CometExchange #9 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #10 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_to_10_30,count(1)] - CometExchange #12 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #13 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_30_to_11,count(1)] - CometExchange #15 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #16 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_to_11_30,count(1)] - CometExchange #18 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #19 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_30_to_12,count(1)] - CometExchange #21 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #22 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #23 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h12_to_12_30,count(1)] - CometExchange #24 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #25 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] 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/q88/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/explain.txt deleted file mode 100644 index 2e49d20af4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/explain.txt +++ /dev/null @@ -1,927 +0,0 @@ -== Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (172) -:- * BroadcastNestedLoopJoin Inner BuildRight (151) -: :- * BroadcastNestedLoopJoin Inner BuildRight (130) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (109) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) -: : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) -: : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : : : : : :- * CometColumnarToRow (25) -: : : : : : : +- CometHashAggregate (24) -: : : : : : : +- CometExchange (23) -: : : : : : : +- CometHashAggregate (22) -: : : : : : : +- CometProject (21) -: : : : : : : +- CometBroadcastHashJoin (20) -: : : : : : : :- CometProject (15) -: : : : : : : : +- CometBroadcastHashJoin (14) -: : : : : : : : :- CometProject (9) -: : : : : : : : : +- CometBroadcastHashJoin (8) -: : : : : : : : : :- CometProject (3) -: : : : : : : : : : +- CometFilter (2) -: : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) -: : : : : : : : : +- CometBroadcastExchange (7) -: : : : : : : : : +- CometProject (6) -: : : : : : : : : +- CometFilter (5) -: : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (4) -: : : : : : : : +- CometBroadcastExchange (13) -: : : : : : : : +- CometProject (12) -: : : : : : : : +- CometFilter (11) -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (10) -: : : : : : : +- CometBroadcastExchange (19) -: : : : : : : +- CometProject (18) -: : : : : : : +- CometFilter (17) -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (16) -: : : : : : +- BroadcastExchange (45) -: : : : : : +- * CometColumnarToRow (44) -: : : : : : +- CometHashAggregate (43) -: : : : : : +- CometExchange (42) -: : : : : : +- CometHashAggregate (41) -: : : : : : +- CometProject (40) -: : : : : : +- CometBroadcastHashJoin (39) -: : : : : : :- CometProject (37) -: : : : : : : +- CometBroadcastHashJoin (36) -: : : : : : : :- CometProject (31) -: : : : : : : : +- CometBroadcastHashJoin (30) -: : : : : : : : :- CometProject (28) -: : : : : : : : : +- CometFilter (27) -: : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (26) -: : : : : : : : +- ReusedExchange (29) -: : : : : : : +- CometBroadcastExchange (35) -: : : : : : : +- CometProject (34) -: : : : : : : +- CometFilter (33) -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (32) -: : : : : : +- ReusedExchange (38) -: : : : : +- BroadcastExchange (66) -: : : : : +- * CometColumnarToRow (65) -: : : : : +- CometHashAggregate (64) -: : : : : +- CometExchange (63) -: : : : : +- CometHashAggregate (62) -: : : : : +- CometProject (61) -: : : : : +- CometBroadcastHashJoin (60) -: : : : : :- CometProject (58) -: : : : : : +- CometBroadcastHashJoin (57) -: : : : : : :- CometProject (52) -: : : : : : : +- CometBroadcastHashJoin (51) -: : : : : : : :- CometProject (49) -: : : : : : : : +- CometFilter (48) -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (47) -: : : : : : : +- ReusedExchange (50) -: : : : : : +- CometBroadcastExchange (56) -: : : : : : +- CometProject (55) -: : : : : : +- CometFilter (54) -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (53) -: : : : : +- ReusedExchange (59) -: : : : +- BroadcastExchange (87) -: : : : +- * CometColumnarToRow (86) -: : : : +- CometHashAggregate (85) -: : : : +- CometExchange (84) -: : : : +- CometHashAggregate (83) -: : : : +- CometProject (82) -: : : : +- CometBroadcastHashJoin (81) -: : : : :- CometProject (79) -: : : : : +- CometBroadcastHashJoin (78) -: : : : : :- CometProject (73) -: : : : : : +- CometBroadcastHashJoin (72) -: : : : : : :- CometProject (70) -: : : : : : : +- CometFilter (69) -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (68) -: : : : : : +- ReusedExchange (71) -: : : : : +- CometBroadcastExchange (77) -: : : : : +- CometProject (76) -: : : : : +- CometFilter (75) -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (74) -: : : : +- ReusedExchange (80) -: : : +- BroadcastExchange (108) -: : : +- * CometColumnarToRow (107) -: : : +- CometHashAggregate (106) -: : : +- CometExchange (105) -: : : +- CometHashAggregate (104) -: : : +- CometProject (103) -: : : +- CometBroadcastHashJoin (102) -: : : :- CometProject (100) -: : : : +- CometBroadcastHashJoin (99) -: : : : :- CometProject (94) -: : : : : +- CometBroadcastHashJoin (93) -: : : : : :- CometProject (91) -: : : : : : +- CometFilter (90) -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (89) -: : : : : +- ReusedExchange (92) -: : : : +- CometBroadcastExchange (98) -: : : : +- CometProject (97) -: : : : +- CometFilter (96) -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (95) -: : : +- ReusedExchange (101) -: : +- BroadcastExchange (129) -: : +- * CometColumnarToRow (128) -: : +- CometHashAggregate (127) -: : +- CometExchange (126) -: : +- CometHashAggregate (125) -: : +- CometProject (124) -: : +- CometBroadcastHashJoin (123) -: : :- CometProject (121) -: : : +- CometBroadcastHashJoin (120) -: : : :- CometProject (115) -: : : : +- CometBroadcastHashJoin (114) -: : : : :- CometProject (112) -: : : : : +- CometFilter (111) -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (110) -: : : : +- ReusedExchange (113) -: : : +- CometBroadcastExchange (119) -: : : +- CometProject (118) -: : : +- CometFilter (117) -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (116) -: : +- ReusedExchange (122) -: +- BroadcastExchange (150) -: +- * CometColumnarToRow (149) -: +- CometHashAggregate (148) -: +- CometExchange (147) -: +- CometHashAggregate (146) -: +- CometProject (145) -: +- CometBroadcastHashJoin (144) -: :- CometProject (142) -: : +- CometBroadcastHashJoin (141) -: : :- CometProject (136) -: : : +- CometBroadcastHashJoin (135) -: : : :- CometProject (133) -: : : : +- CometFilter (132) -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (131) -: : : +- ReusedExchange (134) -: : +- CometBroadcastExchange (140) -: : +- CometProject (139) -: : +- CometFilter (138) -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (137) -: +- ReusedExchange (143) -+- BroadcastExchange (171) - +- * CometColumnarToRow (170) - +- CometHashAggregate (169) - +- CometExchange (168) - +- CometHashAggregate (167) - +- CometProject (166) - +- CometBroadcastHashJoin (165) - :- CometProject (163) - : +- CometBroadcastHashJoin (162) - : :- CometProject (157) - : : +- CometBroadcastHashJoin (156) - : : :- CometProject (154) - : : : +- CometFilter (153) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (152) - : : +- ReusedExchange (155) - : +- CometBroadcastExchange (161) - : +- CometProject (160) - : +- CometFilter (159) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (158) - +- ReusedExchange (164) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(3) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(Or(And(EqualTo(hd_dep_count,4),LessThanOrEqual(hd_vehicle_count,6)),And(EqualTo(hd_dep_count,2),LessThanOrEqual(hd_vehicle_count,4))),And(EqualTo(hd_dep_count,0),LessThanOrEqual(hd_vehicle_count,2))), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Condition : (((((hd_dep_count#6 = 4) AND (hd_vehicle_count#7 <= 6)) OR ((hd_dep_count#6 = 2) AND (hd_vehicle_count#7 <= 4))) OR ((hd_dep_count#6 = 0) AND (hd_vehicle_count#7 <= 2))) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] -Arguments: [ss_sold_time_sk#1, ss_store_sk#3], [ss_sold_time_sk#1, ss_store_sk#3] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,8), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Condition : ((((isnotnull(t_hour#9) AND isnotnull(t_minute#10)) AND (t_hour#9 = 8)) AND (t_minute#10 >= 30)) AND isnotnull(t_time_sk#8)) - -(12) CometProject -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Arguments: [t_time_sk#8], [t_time_sk#8] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#8] -Arguments: [t_time_sk#8] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Right output [1]: [t_time_sk#8] -Arguments: [ss_sold_time_sk#1], [t_time_sk#8], Inner, BuildRight - -(15) CometProject -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#8] -Arguments: [ss_store_sk#3], [ss_store_sk#3] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_store_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [s_store_sk#11, s_store_name#12] -Condition : ((isnotnull(s_store_name#12) AND (s_store_name#12 = ese)) AND isnotnull(s_store_sk#11)) - -(18) CometProject -Input [2]: [s_store_sk#11, s_store_name#12] -Arguments: [s_store_sk#11], [s_store_sk#11] - -(19) CometBroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: [s_store_sk#11] - -(20) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#3] -Right output [1]: [s_store_sk#11] -Arguments: [ss_store_sk#3], [s_store_sk#11], Inner, BuildRight - -(21) CometProject -Input [2]: [ss_store_sk#3, s_store_sk#11] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#13] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#13] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 8] -Input [1]: [h8_30_to_9#14] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Condition : ((isnotnull(ss_hdemo_sk#16) AND isnotnull(ss_sold_time_sk#15)) AND isnotnull(ss_store_sk#17)) - -(28) CometProject -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Arguments: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17], [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] - -(29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#19] - -(30) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] -Right output [1]: [hd_demo_sk#19] -Arguments: [ss_hdemo_sk#16], [hd_demo_sk#19], Inner, BuildRight - -(31) CometProject -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, hd_demo_sk#19] -Arguments: [ss_sold_time_sk#15, ss_store_sk#17], [ss_sold_time_sk#15, ss_store_sk#17] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Condition : ((((isnotnull(t_hour#21) AND isnotnull(t_minute#22)) AND (t_hour#21 = 9)) AND (t_minute#22 < 30)) AND isnotnull(t_time_sk#20)) - -(34) CometProject -Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Arguments: [t_time_sk#20], [t_time_sk#20] - -(35) CometBroadcastExchange -Input [1]: [t_time_sk#20] -Arguments: [t_time_sk#20] - -(36) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#15, ss_store_sk#17] -Right output [1]: [t_time_sk#20] -Arguments: [ss_sold_time_sk#15], [t_time_sk#20], Inner, BuildRight - -(37) CometProject -Input [3]: [ss_sold_time_sk#15, ss_store_sk#17, t_time_sk#20] -Arguments: [ss_store_sk#17], [ss_store_sk#17] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#23] - -(39) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#17] -Right output [1]: [s_store_sk#23] -Arguments: [ss_store_sk#17], [s_store_sk#23], Inner, BuildRight - -(40) CometProject -Input [2]: [ss_store_sk#17, s_store_sk#23] - -(41) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(42) CometExchange -Input [1]: [count#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(43) CometHashAggregate -Input [1]: [count#24] -Keys: [] -Functions [1]: [count(1)] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [h9_to_9_30#25] - -(45) BroadcastExchange -Input [1]: [h9_to_9_30#25] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(46) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(48) CometFilter -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_hdemo_sk#27) AND isnotnull(ss_sold_time_sk#26)) AND isnotnull(ss_store_sk#28)) - -(49) CometProject -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Arguments: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28], [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] - -(50) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#30] - -(51) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] -Right output [1]: [hd_demo_sk#30] -Arguments: [ss_hdemo_sk#27], [hd_demo_sk#30], Inner, BuildRight - -(52) CometProject -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, hd_demo_sk#30] -Arguments: [ss_sold_time_sk#26, ss_store_sk#28], [ss_sold_time_sk#26, ss_store_sk#28] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Condition : ((((isnotnull(t_hour#32) AND isnotnull(t_minute#33)) AND (t_hour#32 = 9)) AND (t_minute#33 >= 30)) AND isnotnull(t_time_sk#31)) - -(55) CometProject -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Arguments: [t_time_sk#31], [t_time_sk#31] - -(56) CometBroadcastExchange -Input [1]: [t_time_sk#31] -Arguments: [t_time_sk#31] - -(57) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#26, ss_store_sk#28] -Right output [1]: [t_time_sk#31] -Arguments: [ss_sold_time_sk#26], [t_time_sk#31], Inner, BuildRight - -(58) CometProject -Input [3]: [ss_sold_time_sk#26, ss_store_sk#28, t_time_sk#31] -Arguments: [ss_store_sk#28], [ss_store_sk#28] - -(59) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#34] - -(60) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#28] -Right output [1]: [s_store_sk#34] -Arguments: [ss_store_sk#28], [s_store_sk#34], Inner, BuildRight - -(61) CometProject -Input [2]: [ss_store_sk#28, s_store_sk#34] - -(62) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(63) CometExchange -Input [1]: [count#35] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(64) CometHashAggregate -Input [1]: [count#35] -Keys: [] -Functions [1]: [count(1)] - -(65) CometColumnarToRow [codegen id : 2] -Input [1]: [h9_30_to_10#36] - -(66) BroadcastExchange -Input [1]: [h9_30_to_10#36] -Arguments: IdentityBroadcastMode, [plan_id=5] - -(67) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(69) CometFilter -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Condition : ((isnotnull(ss_hdemo_sk#38) AND isnotnull(ss_sold_time_sk#37)) AND isnotnull(ss_store_sk#39)) - -(70) CometProject -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Arguments: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39], [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] - -(71) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#41] - -(72) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] -Right output [1]: [hd_demo_sk#41] -Arguments: [ss_hdemo_sk#38], [hd_demo_sk#41], Inner, BuildRight - -(73) CometProject -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, hd_demo_sk#41] -Arguments: [ss_sold_time_sk#37, ss_store_sk#39], [ss_sold_time_sk#37, ss_store_sk#39] - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(75) CometFilter -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Condition : ((((isnotnull(t_hour#43) AND isnotnull(t_minute#44)) AND (t_hour#43 = 10)) AND (t_minute#44 < 30)) AND isnotnull(t_time_sk#42)) - -(76) CometProject -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Arguments: [t_time_sk#42], [t_time_sk#42] - -(77) CometBroadcastExchange -Input [1]: [t_time_sk#42] -Arguments: [t_time_sk#42] - -(78) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#37, ss_store_sk#39] -Right output [1]: [t_time_sk#42] -Arguments: [ss_sold_time_sk#37], [t_time_sk#42], Inner, BuildRight - -(79) CometProject -Input [3]: [ss_sold_time_sk#37, ss_store_sk#39, t_time_sk#42] -Arguments: [ss_store_sk#39], [ss_store_sk#39] - -(80) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#45] - -(81) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#39] -Right output [1]: [s_store_sk#45] -Arguments: [ss_store_sk#39], [s_store_sk#45], Inner, BuildRight - -(82) CometProject -Input [2]: [ss_store_sk#39, s_store_sk#45] - -(83) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(84) CometExchange -Input [1]: [count#46] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(85) CometHashAggregate -Input [1]: [count#46] -Keys: [] -Functions [1]: [count(1)] - -(86) CometColumnarToRow [codegen id : 3] -Input [1]: [h10_to_10_30#47] - -(87) BroadcastExchange -Input [1]: [h10_to_10_30#47] -Arguments: IdentityBroadcastMode, [plan_id=7] - -(88) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(89) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(90) CometFilter -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Condition : ((isnotnull(ss_hdemo_sk#49) AND isnotnull(ss_sold_time_sk#48)) AND isnotnull(ss_store_sk#50)) - -(91) CometProject -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Arguments: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50], [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] - -(92) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#52] - -(93) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] -Right output [1]: [hd_demo_sk#52] -Arguments: [ss_hdemo_sk#49], [hd_demo_sk#52], Inner, BuildRight - -(94) CometProject -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, hd_demo_sk#52] -Arguments: [ss_sold_time_sk#48, ss_store_sk#50], [ss_sold_time_sk#48, ss_store_sk#50] - -(95) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(96) CometFilter -Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Condition : ((((isnotnull(t_hour#54) AND isnotnull(t_minute#55)) AND (t_hour#54 = 10)) AND (t_minute#55 >= 30)) AND isnotnull(t_time_sk#53)) - -(97) CometProject -Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Arguments: [t_time_sk#53], [t_time_sk#53] - -(98) CometBroadcastExchange -Input [1]: [t_time_sk#53] -Arguments: [t_time_sk#53] - -(99) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#48, ss_store_sk#50] -Right output [1]: [t_time_sk#53] -Arguments: [ss_sold_time_sk#48], [t_time_sk#53], Inner, BuildRight - -(100) CometProject -Input [3]: [ss_sold_time_sk#48, ss_store_sk#50, t_time_sk#53] -Arguments: [ss_store_sk#50], [ss_store_sk#50] - -(101) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#56] - -(102) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#50] -Right output [1]: [s_store_sk#56] -Arguments: [ss_store_sk#50], [s_store_sk#56], Inner, BuildRight - -(103) CometProject -Input [2]: [ss_store_sk#50, s_store_sk#56] - -(104) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(105) CometExchange -Input [1]: [count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(106) CometHashAggregate -Input [1]: [count#57] -Keys: [] -Functions [1]: [count(1)] - -(107) CometColumnarToRow [codegen id : 4] -Input [1]: [h10_30_to_11#58] - -(108) BroadcastExchange -Input [1]: [h10_30_to_11#58] -Arguments: IdentityBroadcastMode, [plan_id=9] - -(109) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(110) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(111) CometFilter -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Condition : ((isnotnull(ss_hdemo_sk#60) AND isnotnull(ss_sold_time_sk#59)) AND isnotnull(ss_store_sk#61)) - -(112) CometProject -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Arguments: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61], [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] - -(113) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#63] - -(114) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] -Right output [1]: [hd_demo_sk#63] -Arguments: [ss_hdemo_sk#60], [hd_demo_sk#63], Inner, BuildRight - -(115) CometProject -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, hd_demo_sk#63] -Arguments: [ss_sold_time_sk#59, ss_store_sk#61], [ss_sold_time_sk#59, ss_store_sk#61] - -(116) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(117) CometFilter -Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Condition : ((((isnotnull(t_hour#65) AND isnotnull(t_minute#66)) AND (t_hour#65 = 11)) AND (t_minute#66 < 30)) AND isnotnull(t_time_sk#64)) - -(118) CometProject -Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Arguments: [t_time_sk#64], [t_time_sk#64] - -(119) CometBroadcastExchange -Input [1]: [t_time_sk#64] -Arguments: [t_time_sk#64] - -(120) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#59, ss_store_sk#61] -Right output [1]: [t_time_sk#64] -Arguments: [ss_sold_time_sk#59], [t_time_sk#64], Inner, BuildRight - -(121) CometProject -Input [3]: [ss_sold_time_sk#59, ss_store_sk#61, t_time_sk#64] -Arguments: [ss_store_sk#61], [ss_store_sk#61] - -(122) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#67] - -(123) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#61] -Right output [1]: [s_store_sk#67] -Arguments: [ss_store_sk#61], [s_store_sk#67], Inner, BuildRight - -(124) CometProject -Input [2]: [ss_store_sk#61, s_store_sk#67] - -(125) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(126) CometExchange -Input [1]: [count#68] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(127) CometHashAggregate -Input [1]: [count#68] -Keys: [] -Functions [1]: [count(1)] - -(128) CometColumnarToRow [codegen id : 5] -Input [1]: [h11_to_11_30#69] - -(129) BroadcastExchange -Input [1]: [h11_to_11_30#69] -Arguments: IdentityBroadcastMode, [plan_id=11] - -(130) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(131) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(132) CometFilter -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Condition : ((isnotnull(ss_hdemo_sk#71) AND isnotnull(ss_sold_time_sk#70)) AND isnotnull(ss_store_sk#72)) - -(133) CometProject -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Arguments: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72], [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] - -(134) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#74] - -(135) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] -Right output [1]: [hd_demo_sk#74] -Arguments: [ss_hdemo_sk#71], [hd_demo_sk#74], Inner, BuildRight - -(136) CometProject -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, hd_demo_sk#74] -Arguments: [ss_sold_time_sk#70, ss_store_sk#72], [ss_sold_time_sk#70, ss_store_sk#72] - -(137) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(138) CometFilter -Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Condition : ((((isnotnull(t_hour#76) AND isnotnull(t_minute#77)) AND (t_hour#76 = 11)) AND (t_minute#77 >= 30)) AND isnotnull(t_time_sk#75)) - -(139) CometProject -Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Arguments: [t_time_sk#75], [t_time_sk#75] - -(140) CometBroadcastExchange -Input [1]: [t_time_sk#75] -Arguments: [t_time_sk#75] - -(141) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#70, ss_store_sk#72] -Right output [1]: [t_time_sk#75] -Arguments: [ss_sold_time_sk#70], [t_time_sk#75], Inner, BuildRight - -(142) CometProject -Input [3]: [ss_sold_time_sk#70, ss_store_sk#72, t_time_sk#75] -Arguments: [ss_store_sk#72], [ss_store_sk#72] - -(143) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#78] - -(144) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#72] -Right output [1]: [s_store_sk#78] -Arguments: [ss_store_sk#72], [s_store_sk#78], Inner, BuildRight - -(145) CometProject -Input [2]: [ss_store_sk#72, s_store_sk#78] - -(146) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(147) CometExchange -Input [1]: [count#79] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(148) CometHashAggregate -Input [1]: [count#79] -Keys: [] -Functions [1]: [count(1)] - -(149) CometColumnarToRow [codegen id : 6] -Input [1]: [h11_30_to_12#80] - -(150) BroadcastExchange -Input [1]: [h11_30_to_12#80] -Arguments: IdentityBroadcastMode, [plan_id=13] - -(151) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(152) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(153) CometFilter -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Condition : ((isnotnull(ss_hdemo_sk#82) AND isnotnull(ss_sold_time_sk#81)) AND isnotnull(ss_store_sk#83)) - -(154) CometProject -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Arguments: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83], [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] - -(155) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#85] - -(156) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] -Right output [1]: [hd_demo_sk#85] -Arguments: [ss_hdemo_sk#82], [hd_demo_sk#85], Inner, BuildRight - -(157) CometProject -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, hd_demo_sk#85] -Arguments: [ss_sold_time_sk#81, ss_store_sk#83], [ss_sold_time_sk#81, ss_store_sk#83] - -(158) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(159) CometFilter -Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Condition : ((((isnotnull(t_hour#87) AND isnotnull(t_minute#88)) AND (t_hour#87 = 12)) AND (t_minute#88 < 30)) AND isnotnull(t_time_sk#86)) - -(160) CometProject -Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Arguments: [t_time_sk#86], [t_time_sk#86] - -(161) CometBroadcastExchange -Input [1]: [t_time_sk#86] -Arguments: [t_time_sk#86] - -(162) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#81, ss_store_sk#83] -Right output [1]: [t_time_sk#86] -Arguments: [ss_sold_time_sk#81], [t_time_sk#86], Inner, BuildRight - -(163) CometProject -Input [3]: [ss_sold_time_sk#81, ss_store_sk#83, t_time_sk#86] -Arguments: [ss_store_sk#83], [ss_store_sk#83] - -(164) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#89] - -(165) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#83] -Right output [1]: [s_store_sk#89] -Arguments: [ss_store_sk#83], [s_store_sk#89], Inner, BuildRight - -(166) CometProject -Input [2]: [ss_store_sk#83, s_store_sk#89] - -(167) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(168) CometExchange -Input [1]: [count#90] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(169) CometHashAggregate -Input [1]: [count#90] -Keys: [] -Functions [1]: [count(1)] - -(170) CometColumnarToRow [codegen id : 7] -Input [1]: [h12_to_12_30#91] - -(171) BroadcastExchange -Input [1]: [h12_to_12_30#91] -Arguments: IdentityBroadcastMode, [plan_id=15] - -(172) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/extended.txt deleted file mode 100644 index 1e1247665c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/extended.txt +++ /dev/null @@ -1,216 +0,0 @@ -BroadcastNestedLoopJoin -:- BroadcastNestedLoopJoin -: :- BroadcastNestedLoopJoin -: : :- BroadcastNestedLoopJoin -: : : :- BroadcastNestedLoopJoin -: : : : :- BroadcastNestedLoopJoin -: : : : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] -: : : : : : :- CometColumnarToRow -: : : : : : : +- CometHashAggregate -: : : : : : : +- CometExchange -: : : : : : : +- CometHashAggregate -: : : : : : : +- CometProject -: : : : : : : +- CometBroadcastHashJoin -: : : : : : : :- CometProject -: : : : : : : : +- CometBroadcastHashJoin -: : : : : : : : :- CometProject -: : : : : : : : : +- CometBroadcastHashJoin -: : : : : : : : : :- CometProject -: : : : : : : : : : +- CometFilter -: : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : : : : : +- CometBroadcastExchange -: : : : : : : : : +- CometProject -: : : : : : : : : +- CometFilter -: : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : : : : : +- CometBroadcastExchange -: : : : : : : : +- CometProject -: : : : : : : : +- CometFilter -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : : : : : +- CometBroadcastExchange -: : : : : : : +- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : : : : +- BroadcastExchange -: : : : : : +- CometColumnarToRow -: : : : : : +- CometHashAggregate -: : : : : : +- CometExchange -: : : : : : +- CometHashAggregate -: : : : : : +- CometProject -: : : : : : +- CometBroadcastHashJoin -: : : : : : :- CometProject -: : : : : : : +- CometBroadcastHashJoin -: : : : : : : :- CometProject -: : : : : : : : +- CometBroadcastHashJoin -: : : : : : : : :- CometProject -: : : : : : : : : +- CometFilter -: : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : : : : +- CometBroadcastExchange -: : : : : : : : +- CometProject -: : : : : : : : +- CometFilter -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : : : : +- CometBroadcastExchange -: : : : : : : +- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : : : : +- CometBroadcastExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : : : +- BroadcastExchange -: : : : : +- CometColumnarToRow -: : : : : +- CometHashAggregate -: : : : : +- CometExchange -: : : : : +- CometHashAggregate -: : : : : +- CometProject -: : : : : +- CometBroadcastHashJoin -: : : : : :- CometProject -: : : : : : +- CometBroadcastHashJoin -: : : : : : :- CometProject -: : : : : : : +- CometBroadcastHashJoin -: : : : : : : :- CometProject -: : : : : : : : +- CometFilter -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : : : +- CometBroadcastExchange -: : : : : : : +- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : : : +- CometBroadcastExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : : : +- CometBroadcastExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : : +- BroadcastExchange -: : : : +- CometColumnarToRow -: : : : +- CometHashAggregate -: : : : +- CometExchange -: : : : +- CometHashAggregate -: : : : +- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometBroadcastHashJoin -: : : : : :- CometProject -: : : : : : +- CometBroadcastHashJoin -: : : : : : :- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : : +- CometBroadcastExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : : +- CometBroadcastExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : +- BroadcastExchange -: : : +- CometColumnarToRow -: : : +- CometHashAggregate -: : : +- CometExchange -: : : +- CometHashAggregate -: : : +- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometBroadcastHashJoin -: : : : : :- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : +- CometBroadcastExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : +- BroadcastExchange -: : +- CometColumnarToRow -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: +- BroadcastExchange -: +- CometColumnarToRow -: +- CometHashAggregate -: +- CometExchange -: +- CometHashAggregate -: +- CometProject -: +- CometBroadcastHashJoin -: :- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: +- CometBroadcastExchange -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -+- BroadcastExchange - +- CometColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 192 out of 206 eligible operators (93%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index e3923fb3ea..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/simplified.txt +++ /dev/null @@ -1,195 +0,0 @@ -WholeStageCodegen (8) - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h8_30_to_9,count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_to_9_30,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_30_to_10,count(1)] - CometExchange #9 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #10 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_to_10_30,count(1)] - CometExchange #12 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #13 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_30_to_11,count(1)] - CometExchange #15 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #16 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_to_11_30,count(1)] - CometExchange #18 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #19 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_30_to_12,count(1)] - CometExchange #21 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #22 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #23 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h12_to_12_30,count(1)] - CometExchange #24 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #25 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/explain.txt deleted file mode 100644 index 56583c6d26..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/explain.txt +++ /dev/null @@ -1,202 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * CometColumnarToRow (26) - +- CometSort (25) - +- CometColumnarExchange (24) - +- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometColumnarExchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * CometColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (8) - : : +- * Filter (7) - : : +- * ColumnarToRow (6) - : : +- Scan parquet spark_catalog.default.store_sales (5) - : +- ReusedExchange (11) - +- BroadcastExchange (17) - +- * CometColumnarToRow (16) - +- CometFilter (15) - +- CometNativeScan parquet spark_catalog.default.store (14) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Books ,Electronics ,Sports ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (computers ,stereo ,football )) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Men ,Jewelry ,Women ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (shirts ,birdal ,dresses ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Arguments: [i_item_sk#1, i_brand#5, i_class#6, i_category#7], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) AS i_class#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) AS i_category#7] - -(4) CometColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7] - -(5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] - -(7) Filter [codegen id : 1] -Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Condition : (isnotnull(ss_item_sk#8) AND isnotnull(ss_store_sk#9)) - -(8) BroadcastExchange -Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#8] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Input [8]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7, ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] - -(11) ReusedExchange [Reuses operator id: 35] -Output [2]: [d_date_sk#13, d_moy#14] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#14] -Input [8]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11, d_date_sk#13, d_moy#14] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#15, s_store_name#16, s_company_name#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [s_store_sk#15, s_store_name#16, s_company_name#17] -Condition : isnotnull(s_store_sk#15) - -(16) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#15, s_store_name#16, s_company_name#17] - -(17) BroadcastExchange -Input [3]: [s_store_sk#15, s_store_name#16, s_company_name#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#9] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [7]: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#14, s_store_name#16, s_company_name#17] -Input [9]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#14, s_store_sk#15, s_store_name#16, s_company_name#17] - -(20) HashAggregate [codegen id : 4] -Input [7]: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#14, s_store_name#16, s_company_name#17] -Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#10))] -Aggregate Attributes [1]: [sum#18] -Results [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum#19] - -(21) CometColumnarExchange -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum#19] -Arguments: hashpartitioning(i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum#19] - -(23) HashAggregate [codegen id : 5] -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum#19] -Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14] -Functions [1]: [sum(UnscaledValue(ss_sales_price#10))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#10))#20] -Results [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#20,17,2) AS _w0#22] - -(24) CometColumnarExchange -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#16, s_company_name#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22] -Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#16 ASC NULLS FIRST, s_company_name#17 ASC NULLS FIRST] - -(26) CometColumnarToRow [codegen id : 6] -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22] - -(27) Window -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_category#7, i_brand#5, s_store_name#16, s_company_name#17, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#7, i_brand#5, s_store_name#16, s_company_name#17] - -(28) Filter [codegen id : 7] -Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22, avg_monthly_sales#23] -Condition : CASE WHEN NOT (avg_monthly_sales#23 = 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END - -(29) Project [codegen id : 7] -Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, avg_monthly_sales#23] -Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22, avg_monthly_sales#23] - -(30) TakeOrderedAndProject -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, avg_monthly_sales#23] -Arguments: 100, [(sum_sales#21 - avg_monthly_sales#23) ASC NULLS FIRST, s_store_name#16 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, avg_monthly_sales#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometNativeScan parquet spark_catalog.default.date_dim (31) - - -(31) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#24, d_moy#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [d_date_sk#13, d_year#24, d_moy#14] -Condition : ((isnotnull(d_year#24) AND (d_year#24 = 1999)) AND isnotnull(d_date_sk#13)) - -(33) CometProject -Input [3]: [d_date_sk#13, d_year#24, d_moy#14] -Arguments: [d_date_sk#13, d_moy#14], [d_date_sk#13, d_moy#14] - -(34) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_moy#14] - -(35) BroadcastExchange -Input [2]: [d_date_sk#13, d_moy#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/simplified.txt deleted file mode 100644 index f87ef33db4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/simplified.txt +++ /dev/null @@ -1,53 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (7) - Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometNativeScan 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/q89.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/explain.txt deleted file mode 100644 index 47ce0d4a91..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,198 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) - +- CometSort (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Books ,Electronics ,Sports ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (computers ,stereo ,football )) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Men ,Jewelry ,Women ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (shirts ,birdal ,dresses ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Arguments: [i_item_sk#1, i_brand#5, i_class#6, i_category#7], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) AS i_class#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) AS i_category#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Condition : (isnotnull(ss_item_sk#8) AND isnotnull(ss_store_sk#9)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Arguments: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] - -(7) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7] -Right output [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Arguments: [i_item_sk#1], [ss_item_sk#8], Inner, BuildRight - -(8) CometProject -Input [8]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7, ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Arguments: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11], [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 1999)) AND isnotnull(d_date_sk#13)) - -(11) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13, d_moy#15], [d_date_sk#13, d_moy#15] - -(12) CometBroadcastExchange -Input [2]: [d_date_sk#13, d_moy#15] -Arguments: [d_date_sk#13, d_moy#15] - -(13) CometBroadcastHashJoin -Left output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Right output [2]: [d_date_sk#13, d_moy#15] -Arguments: [ss_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(14) CometProject -Input [8]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11, d_date_sk#13, d_moy#15] -Arguments: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15], [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Condition : isnotnull(s_store_sk#16) - -(17) CometBroadcastExchange -Input [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Arguments: [s_store_sk#16, s_store_name#17, s_company_name#18] - -(18) CometBroadcastHashJoin -Left output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15] -Right output [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Arguments: [ss_store_sk#9], [s_store_sk#16], Inner, BuildRight - -(19) CometProject -Input [9]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15, s_store_sk#16, s_store_name#17, s_company_name#18] -Arguments: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#15, s_store_name#17, s_company_name#18], [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#15, s_store_name#17, s_company_name#18] - -(20) CometHashAggregate -Input [7]: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#15, s_store_name#17, s_company_name#18] -Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#10))] - -(21) CometExchange -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum#19] -Arguments: hashpartitioning(i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum#19] -Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15] -Functions [1]: [sum(UnscaledValue(ss_sales_price#10))] - -(23) CometExchange -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST, s_company_name#18 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 1] -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] - -(26) Window -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] - -(27) Filter [codegen id : 2] -Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] -Condition : CASE WHEN NOT (avg_monthly_sales#22 = 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END - -(28) Project [codegen id : 2] -Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] -Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] - -(29) TakeOrderedAndProject -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] -Arguments: 100, [(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 1999)) AND isnotnull(d_date_sk#13)) - -(32) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13, d_moy#15], [d_date_sk#13, d_moy#15] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_moy#15] - -(34) BroadcastExchange -Input [2]: [d_date_sk#13, d_moy#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/simplified.txt deleted file mode 100644 index 61bfd1d960..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (2) - Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] - CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - 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 [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,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] 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/q89/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/explain.txt deleted file mode 100644 index 47ce0d4a91..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/explain.txt +++ /dev/null @@ -1,198 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) - +- CometSort (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Books ,Electronics ,Sports ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (computers ,stereo ,football )) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Men ,Jewelry ,Women ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (shirts ,birdal ,dresses ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Arguments: [i_item_sk#1, i_brand#5, i_class#6, i_category#7], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) AS i_class#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) AS i_category#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Condition : (isnotnull(ss_item_sk#8) AND isnotnull(ss_store_sk#9)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Arguments: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] - -(7) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7] -Right output [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Arguments: [i_item_sk#1], [ss_item_sk#8], Inner, BuildRight - -(8) CometProject -Input [8]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7, ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Arguments: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11], [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 1999)) AND isnotnull(d_date_sk#13)) - -(11) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13, d_moy#15], [d_date_sk#13, d_moy#15] - -(12) CometBroadcastExchange -Input [2]: [d_date_sk#13, d_moy#15] -Arguments: [d_date_sk#13, d_moy#15] - -(13) CometBroadcastHashJoin -Left output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Right output [2]: [d_date_sk#13, d_moy#15] -Arguments: [ss_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(14) CometProject -Input [8]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11, d_date_sk#13, d_moy#15] -Arguments: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15], [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Condition : isnotnull(s_store_sk#16) - -(17) CometBroadcastExchange -Input [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Arguments: [s_store_sk#16, s_store_name#17, s_company_name#18] - -(18) CometBroadcastHashJoin -Left output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15] -Right output [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Arguments: [ss_store_sk#9], [s_store_sk#16], Inner, BuildRight - -(19) CometProject -Input [9]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15, s_store_sk#16, s_store_name#17, s_company_name#18] -Arguments: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#15, s_store_name#17, s_company_name#18], [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#15, s_store_name#17, s_company_name#18] - -(20) CometHashAggregate -Input [7]: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#15, s_store_name#17, s_company_name#18] -Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#10))] - -(21) CometExchange -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum#19] -Arguments: hashpartitioning(i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum#19] -Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15] -Functions [1]: [sum(UnscaledValue(ss_sales_price#10))] - -(23) CometExchange -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST, s_company_name#18 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 1] -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] - -(26) Window -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] - -(27) Filter [codegen id : 2] -Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] -Condition : CASE WHEN NOT (avg_monthly_sales#22 = 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END - -(28) Project [codegen id : 2] -Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] -Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] - -(29) TakeOrderedAndProject -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] -Arguments: 100, [(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 1999)) AND isnotnull(d_date_sk#13)) - -(32) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13, d_moy#15], [d_date_sk#13, d_moy#15] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_moy#15] - -(34) BroadcastExchange -Input [2]: [d_date_sk#13, d_moy#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/extended.txt deleted file mode 100644 index 825b1ed81c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 61bfd1d960..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (2) - Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] - CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - 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 [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,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_datafusion/explain.txt deleted file mode 100644 index da194f2825..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_datafusion/explain.txt +++ /dev/null @@ -1,283 +0,0 @@ -== Physical Plan == -* Project (4) -+- * CometColumnarToRow (3) - +- CometFilter (2) - +- CometNativeScan parquet spark_catalog.default.reason (1) - - -(1) CometNativeScan parquet spark_catalog.default.reason -Output [1]: [r_reason_sk#1] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk), EqualTo(r_reason_sk,1)] -ReadSchema: struct - -(2) CometFilter -Input [1]: [r_reason_sk#1] -Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) - -(3) CometColumnarToRow [codegen id : 1] -Input [1]: [r_reason_sk#1] - -(4) Project [codegen id : 1] -Output [5]: [CASE WHEN (Subquery scalar-subquery#2, [id=#3].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket1#4, CASE WHEN (Subquery scalar-subquery#5, [id=#6].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_net_paid) END AS bucket2#7, CASE WHEN (Subquery scalar-subquery#8, [id=#9].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_net_paid) END AS bucket3#10, CASE WHEN (Subquery scalar-subquery#11, [id=#12].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_net_paid) END AS bucket4#13, CASE WHEN (Subquery scalar-subquery#14, [id=#15].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_net_paid) END AS bucket5#16] -Input [1]: [r_reason_sk#1] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] -* CometColumnarToRow (12) -+- CometProject (11) - +- CometHashAggregate (10) - +- CometExchange (9) - +- CometHashAggregate (8) - +- CometProject (7) - +- CometFilter (6) - +- CometNativeScan parquet spark_catalog.default.store_sales (5) - - -(5) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] -ReadSchema: struct - -(6) CometFilter -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Condition : ((isnotnull(ss_quantity#17) AND (ss_quantity#17 >= 1)) AND (ss_quantity#17 <= 20)) - -(7) CometProject -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Arguments: [ss_ext_discount_amt#18, ss_net_paid#19], [ss_ext_discount_amt#18, ss_net_paid#19] - -(8) CometHashAggregate -Input [2]: [ss_ext_discount_amt#18, ss_net_paid#19] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#18)), partial_avg(UnscaledValue(ss_net_paid#19))] - -(9) CometExchange -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometHashAggregate -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(UnscaledValue(ss_net_paid#19))] - -(11) CometProject -Input [3]: [count(1)#26, avg(ss_ext_discount_amt)#27, avg(ss_net_paid)#28] -Arguments: [mergedValue#29], [named_struct(count(1), count(1)#26, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#27, avg(ss_net_paid), avg(ss_net_paid)#28) AS mergedValue#29] - -(12) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#29] - -Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] - -Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] - -Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] -* CometColumnarToRow (20) -+- CometProject (19) - +- CometHashAggregate (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometProject (15) - +- CometFilter (14) - +- CometNativeScan parquet spark_catalog.default.store_sales (13) - - -(13) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Condition : ((isnotnull(ss_quantity#30) AND (ss_quantity#30 >= 21)) AND (ss_quantity#30 <= 40)) - -(15) CometProject -Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Arguments: [ss_ext_discount_amt#31, ss_net_paid#32], [ss_ext_discount_amt#31, ss_net_paid#32] - -(16) CometHashAggregate -Input [2]: [ss_ext_discount_amt#31, ss_net_paid#32] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#31)), partial_avg(UnscaledValue(ss_net_paid#32))] - -(17) CometExchange -Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(18) CometHashAggregate -Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#31)), avg(UnscaledValue(ss_net_paid#32))] - -(19) CometProject -Input [3]: [count(1)#39, avg(ss_ext_discount_amt)#40, avg(ss_net_paid)#41] -Arguments: [mergedValue#42], [named_struct(count(1), count(1)#39, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#40, avg(ss_net_paid), avg(ss_net_paid)#41) AS mergedValue#42] - -(20) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#42] - -Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] - -Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] - -Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* CometColumnarToRow (28) -+- CometProject (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometFilter (22) - +- CometNativeScan parquet spark_catalog.default.store_sales (21) - - -(21) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] -ReadSchema: struct - -(22) CometFilter -Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Condition : ((isnotnull(ss_quantity#43) AND (ss_quantity#43 >= 41)) AND (ss_quantity#43 <= 60)) - -(23) CometProject -Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Arguments: [ss_ext_discount_amt#44, ss_net_paid#45], [ss_ext_discount_amt#44, ss_net_paid#45] - -(24) CometHashAggregate -Input [2]: [ss_ext_discount_amt#44, ss_net_paid#45] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#44)), partial_avg(UnscaledValue(ss_net_paid#45))] - -(25) CometExchange -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(26) CometHashAggregate -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#44)), avg(UnscaledValue(ss_net_paid#45))] - -(27) CometProject -Input [3]: [count(1)#52, avg(ss_ext_discount_amt)#53, avg(ss_net_paid)#54] -Arguments: [mergedValue#55], [named_struct(count(1), count(1)#52, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#53, avg(ss_net_paid), avg(ss_net_paid)#54) AS mergedValue#55] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#55] - -Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] - -Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] - -Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* CometColumnarToRow (36) -+- CometProject (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometFilter (30) - +- CometNativeScan parquet spark_catalog.default.store_sales (29) - - -(29) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_quantity#56) AND (ss_quantity#56 >= 61)) AND (ss_quantity#56 <= 80)) - -(31) CometProject -Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Arguments: [ss_ext_discount_amt#57, ss_net_paid#58], [ss_ext_discount_amt#57, ss_net_paid#58] - -(32) CometHashAggregate -Input [2]: [ss_ext_discount_amt#57, ss_net_paid#58] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#57)), partial_avg(UnscaledValue(ss_net_paid#58))] - -(33) CometExchange -Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(34) CometHashAggregate -Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#57)), avg(UnscaledValue(ss_net_paid#58))] - -(35) CometProject -Input [3]: [count(1)#65, avg(ss_ext_discount_amt)#66, avg(ss_net_paid)#67] -Arguments: [mergedValue#68], [named_struct(count(1), count(1)#65, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#66, avg(ss_net_paid), avg(ss_net_paid)#67) AS mergedValue#68] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#68] - -Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] - -Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] - -Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* CometColumnarToRow (44) -+- CometProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometFilter (38) - +- CometNativeScan parquet spark_catalog.default.store_sales (37) - - -(37) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] -ReadSchema: struct - -(38) CometFilter -Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Condition : ((isnotnull(ss_quantity#69) AND (ss_quantity#69 >= 81)) AND (ss_quantity#69 <= 100)) - -(39) CometProject -Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Arguments: [ss_ext_discount_amt#70, ss_net_paid#71], [ss_ext_discount_amt#70, ss_net_paid#71] - -(40) CometHashAggregate -Input [2]: [ss_ext_discount_amt#70, ss_net_paid#71] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#70)), partial_avg(UnscaledValue(ss_net_paid#71))] - -(41) CometExchange -Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(42) CometHashAggregate -Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#70)), avg(UnscaledValue(ss_net_paid#71))] - -(43) CometProject -Input [3]: [count(1)#78, avg(ss_ext_discount_amt)#79, avg(ss_net_paid)#80] -Arguments: [mergedValue#81], [named_struct(count(1), count(1)#78, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#79, avg(ss_net_paid), avg(ss_net_paid)#80) AS mergedValue#81] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#81] - -Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] - -Subquery:15 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_datafusion/simplified.txt deleted file mode 100644 index 9593d6ebd1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_datafusion/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -WholeStageCodegen (1) - Project - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #1 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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 - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #2 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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 - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #3 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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 - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #4 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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 - Subquery #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #5 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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 - CometColumnarToRow - InputAdapter - CometFilter [r_reason_sk] - CometNativeScan parquet spark_catalog.default.reason [r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/explain.txt deleted file mode 100644 index b3f32555de..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,283 +0,0 @@ -== Physical Plan == -* Project (4) -+- * CometColumnarToRow (3) - +- CometFilter (2) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (1) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Output [1]: [r_reason_sk#1] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk), EqualTo(r_reason_sk,1)] -ReadSchema: struct - -(2) CometFilter -Input [1]: [r_reason_sk#1] -Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) - -(3) CometColumnarToRow [codegen id : 1] -Input [1]: [r_reason_sk#1] - -(4) Project [codegen id : 1] -Output [5]: [CASE WHEN (Subquery scalar-subquery#2, [id=#3].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket1#4, CASE WHEN (Subquery scalar-subquery#5, [id=#6].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_net_paid) END AS bucket2#7, CASE WHEN (Subquery scalar-subquery#8, [id=#9].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_net_paid) END AS bucket3#10, CASE WHEN (Subquery scalar-subquery#11, [id=#12].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_net_paid) END AS bucket4#13, CASE WHEN (Subquery scalar-subquery#14, [id=#15].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_net_paid) END AS bucket5#16] -Input [1]: [r_reason_sk#1] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] -* CometColumnarToRow (12) -+- CometProject (11) - +- CometHashAggregate (10) - +- CometExchange (9) - +- CometHashAggregate (8) - +- CometProject (7) - +- CometFilter (6) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] -ReadSchema: struct - -(6) CometFilter -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Condition : ((isnotnull(ss_quantity#17) AND (ss_quantity#17 >= 1)) AND (ss_quantity#17 <= 20)) - -(7) CometProject -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Arguments: [ss_ext_discount_amt#18, ss_net_paid#19], [ss_ext_discount_amt#18, ss_net_paid#19] - -(8) CometHashAggregate -Input [2]: [ss_ext_discount_amt#18, ss_net_paid#19] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#18)), partial_avg(UnscaledValue(ss_net_paid#19))] - -(9) CometExchange -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometHashAggregate -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(UnscaledValue(ss_net_paid#19))] - -(11) CometProject -Input [3]: [count(1)#26, avg(ss_ext_discount_amt)#27, avg(ss_net_paid)#28] -Arguments: [mergedValue#29], [named_struct(count(1), count(1)#26, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#27, avg(ss_net_paid), avg(ss_net_paid)#28) AS mergedValue#29] - -(12) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#29] - -Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] - -Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] - -Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] -* CometColumnarToRow (20) -+- CometProject (19) - +- CometHashAggregate (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometProject (15) - +- CometFilter (14) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Condition : ((isnotnull(ss_quantity#30) AND (ss_quantity#30 >= 21)) AND (ss_quantity#30 <= 40)) - -(15) CometProject -Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Arguments: [ss_ext_discount_amt#31, ss_net_paid#32], [ss_ext_discount_amt#31, ss_net_paid#32] - -(16) CometHashAggregate -Input [2]: [ss_ext_discount_amt#31, ss_net_paid#32] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#31)), partial_avg(UnscaledValue(ss_net_paid#32))] - -(17) CometExchange -Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(18) CometHashAggregate -Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#31)), avg(UnscaledValue(ss_net_paid#32))] - -(19) CometProject -Input [3]: [count(1)#39, avg(ss_ext_discount_amt)#40, avg(ss_net_paid)#41] -Arguments: [mergedValue#42], [named_struct(count(1), count(1)#39, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#40, avg(ss_net_paid), avg(ss_net_paid)#41) AS mergedValue#42] - -(20) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#42] - -Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] - -Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] - -Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* CometColumnarToRow (28) -+- CometProject (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] -ReadSchema: struct - -(22) CometFilter -Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Condition : ((isnotnull(ss_quantity#43) AND (ss_quantity#43 >= 41)) AND (ss_quantity#43 <= 60)) - -(23) CometProject -Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Arguments: [ss_ext_discount_amt#44, ss_net_paid#45], [ss_ext_discount_amt#44, ss_net_paid#45] - -(24) CometHashAggregate -Input [2]: [ss_ext_discount_amt#44, ss_net_paid#45] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#44)), partial_avg(UnscaledValue(ss_net_paid#45))] - -(25) CometExchange -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(26) CometHashAggregate -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#44)), avg(UnscaledValue(ss_net_paid#45))] - -(27) CometProject -Input [3]: [count(1)#52, avg(ss_ext_discount_amt)#53, avg(ss_net_paid)#54] -Arguments: [mergedValue#55], [named_struct(count(1), count(1)#52, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#53, avg(ss_net_paid), avg(ss_net_paid)#54) AS mergedValue#55] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#55] - -Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] - -Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] - -Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* CometColumnarToRow (36) -+- CometProject (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (29) - - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_quantity#56) AND (ss_quantity#56 >= 61)) AND (ss_quantity#56 <= 80)) - -(31) CometProject -Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Arguments: [ss_ext_discount_amt#57, ss_net_paid#58], [ss_ext_discount_amt#57, ss_net_paid#58] - -(32) CometHashAggregate -Input [2]: [ss_ext_discount_amt#57, ss_net_paid#58] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#57)), partial_avg(UnscaledValue(ss_net_paid#58))] - -(33) CometExchange -Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(34) CometHashAggregate -Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#57)), avg(UnscaledValue(ss_net_paid#58))] - -(35) CometProject -Input [3]: [count(1)#65, avg(ss_ext_discount_amt)#66, avg(ss_net_paid)#67] -Arguments: [mergedValue#68], [named_struct(count(1), count(1)#65, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#66, avg(ss_net_paid), avg(ss_net_paid)#67) AS mergedValue#68] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#68] - -Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] - -Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] - -Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* CometColumnarToRow (44) -+- CometProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (37) - - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] -ReadSchema: struct - -(38) CometFilter -Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Condition : ((isnotnull(ss_quantity#69) AND (ss_quantity#69 >= 81)) AND (ss_quantity#69 <= 100)) - -(39) CometProject -Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Arguments: [ss_ext_discount_amt#70, ss_net_paid#71], [ss_ext_discount_amt#70, ss_net_paid#71] - -(40) CometHashAggregate -Input [2]: [ss_ext_discount_amt#70, ss_net_paid#71] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#70)), partial_avg(UnscaledValue(ss_net_paid#71))] - -(41) CometExchange -Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(42) CometHashAggregate -Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#70)), avg(UnscaledValue(ss_net_paid#71))] - -(43) CometProject -Input [3]: [count(1)#78, avg(ss_ext_discount_amt)#79, avg(ss_net_paid)#80] -Arguments: [mergedValue#81], [named_struct(count(1), count(1)#78, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#79, avg(ss_net_paid), avg(ss_net_paid)#80) AS mergedValue#81] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#81] - -Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] - -Subquery:15 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/simplified.txt deleted file mode 100644 index 558f5f4b36..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -WholeStageCodegen (1) - Project - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #1 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #2 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #3 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #4 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #5 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometColumnarToRow - InputAdapter - CometFilter [r_reason_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.reason [r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/explain.txt deleted file mode 100644 index b3f32555de..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/explain.txt +++ /dev/null @@ -1,283 +0,0 @@ -== Physical Plan == -* Project (4) -+- * CometColumnarToRow (3) - +- CometFilter (2) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (1) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Output [1]: [r_reason_sk#1] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk), EqualTo(r_reason_sk,1)] -ReadSchema: struct - -(2) CometFilter -Input [1]: [r_reason_sk#1] -Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) - -(3) CometColumnarToRow [codegen id : 1] -Input [1]: [r_reason_sk#1] - -(4) Project [codegen id : 1] -Output [5]: [CASE WHEN (Subquery scalar-subquery#2, [id=#3].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket1#4, CASE WHEN (Subquery scalar-subquery#5, [id=#6].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_net_paid) END AS bucket2#7, CASE WHEN (Subquery scalar-subquery#8, [id=#9].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_net_paid) END AS bucket3#10, CASE WHEN (Subquery scalar-subquery#11, [id=#12].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_net_paid) END AS bucket4#13, CASE WHEN (Subquery scalar-subquery#14, [id=#15].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_net_paid) END AS bucket5#16] -Input [1]: [r_reason_sk#1] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] -* CometColumnarToRow (12) -+- CometProject (11) - +- CometHashAggregate (10) - +- CometExchange (9) - +- CometHashAggregate (8) - +- CometProject (7) - +- CometFilter (6) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] -ReadSchema: struct - -(6) CometFilter -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Condition : ((isnotnull(ss_quantity#17) AND (ss_quantity#17 >= 1)) AND (ss_quantity#17 <= 20)) - -(7) CometProject -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Arguments: [ss_ext_discount_amt#18, ss_net_paid#19], [ss_ext_discount_amt#18, ss_net_paid#19] - -(8) CometHashAggregate -Input [2]: [ss_ext_discount_amt#18, ss_net_paid#19] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#18)), partial_avg(UnscaledValue(ss_net_paid#19))] - -(9) CometExchange -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometHashAggregate -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(UnscaledValue(ss_net_paid#19))] - -(11) CometProject -Input [3]: [count(1)#26, avg(ss_ext_discount_amt)#27, avg(ss_net_paid)#28] -Arguments: [mergedValue#29], [named_struct(count(1), count(1)#26, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#27, avg(ss_net_paid), avg(ss_net_paid)#28) AS mergedValue#29] - -(12) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#29] - -Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] - -Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] - -Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] -* CometColumnarToRow (20) -+- CometProject (19) - +- CometHashAggregate (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometProject (15) - +- CometFilter (14) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Condition : ((isnotnull(ss_quantity#30) AND (ss_quantity#30 >= 21)) AND (ss_quantity#30 <= 40)) - -(15) CometProject -Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Arguments: [ss_ext_discount_amt#31, ss_net_paid#32], [ss_ext_discount_amt#31, ss_net_paid#32] - -(16) CometHashAggregate -Input [2]: [ss_ext_discount_amt#31, ss_net_paid#32] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#31)), partial_avg(UnscaledValue(ss_net_paid#32))] - -(17) CometExchange -Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(18) CometHashAggregate -Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#31)), avg(UnscaledValue(ss_net_paid#32))] - -(19) CometProject -Input [3]: [count(1)#39, avg(ss_ext_discount_amt)#40, avg(ss_net_paid)#41] -Arguments: [mergedValue#42], [named_struct(count(1), count(1)#39, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#40, avg(ss_net_paid), avg(ss_net_paid)#41) AS mergedValue#42] - -(20) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#42] - -Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] - -Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] - -Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* CometColumnarToRow (28) -+- CometProject (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] -ReadSchema: struct - -(22) CometFilter -Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Condition : ((isnotnull(ss_quantity#43) AND (ss_quantity#43 >= 41)) AND (ss_quantity#43 <= 60)) - -(23) CometProject -Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Arguments: [ss_ext_discount_amt#44, ss_net_paid#45], [ss_ext_discount_amt#44, ss_net_paid#45] - -(24) CometHashAggregate -Input [2]: [ss_ext_discount_amt#44, ss_net_paid#45] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#44)), partial_avg(UnscaledValue(ss_net_paid#45))] - -(25) CometExchange -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(26) CometHashAggregate -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#44)), avg(UnscaledValue(ss_net_paid#45))] - -(27) CometProject -Input [3]: [count(1)#52, avg(ss_ext_discount_amt)#53, avg(ss_net_paid)#54] -Arguments: [mergedValue#55], [named_struct(count(1), count(1)#52, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#53, avg(ss_net_paid), avg(ss_net_paid)#54) AS mergedValue#55] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#55] - -Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] - -Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] - -Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* CometColumnarToRow (36) -+- CometProject (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (29) - - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_quantity#56) AND (ss_quantity#56 >= 61)) AND (ss_quantity#56 <= 80)) - -(31) CometProject -Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Arguments: [ss_ext_discount_amt#57, ss_net_paid#58], [ss_ext_discount_amt#57, ss_net_paid#58] - -(32) CometHashAggregate -Input [2]: [ss_ext_discount_amt#57, ss_net_paid#58] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#57)), partial_avg(UnscaledValue(ss_net_paid#58))] - -(33) CometExchange -Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(34) CometHashAggregate -Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#57)), avg(UnscaledValue(ss_net_paid#58))] - -(35) CometProject -Input [3]: [count(1)#65, avg(ss_ext_discount_amt)#66, avg(ss_net_paid)#67] -Arguments: [mergedValue#68], [named_struct(count(1), count(1)#65, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#66, avg(ss_net_paid), avg(ss_net_paid)#67) AS mergedValue#68] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#68] - -Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] - -Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] - -Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* CometColumnarToRow (44) -+- CometProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (37) - - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] -ReadSchema: struct - -(38) CometFilter -Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Condition : ((isnotnull(ss_quantity#69) AND (ss_quantity#69 >= 81)) AND (ss_quantity#69 <= 100)) - -(39) CometProject -Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Arguments: [ss_ext_discount_amt#70, ss_net_paid#71], [ss_ext_discount_amt#70, ss_net_paid#71] - -(40) CometHashAggregate -Input [2]: [ss_ext_discount_amt#70, ss_net_paid#71] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#70)), partial_avg(UnscaledValue(ss_net_paid#71))] - -(41) CometExchange -Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(42) CometHashAggregate -Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#70)), avg(UnscaledValue(ss_net_paid#71))] - -(43) CometProject -Input [3]: [count(1)#78, avg(ss_ext_discount_amt)#79, avg(ss_net_paid)#80] -Arguments: [mergedValue#81], [named_struct(count(1), count(1)#78, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#79, avg(ss_net_paid), avg(ss_net_paid)#80) AS mergedValue#81] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#81] - -Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] - -Subquery:15 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/extended.txt deleted file mode 100644 index ed71033b26..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/extended.txt +++ /dev/null @@ -1,61 +0,0 @@ - Project [COMET: ] -: :- Subquery -: : +- CometColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: +- ReusedSubquery -+- CometColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason - -Comet accelerated 37 out of 53 eligible operators (69%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 558f5f4b36..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q9/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -WholeStageCodegen (1) - Project - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #1 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #2 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #3 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #4 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #5 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometColumnarToRow - InputAdapter - CometFilter [r_reason_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.reason [r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_datafusion/explain.txt deleted file mode 100644 index 73b54f439b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_datafusion/explain.txt +++ /dev/null @@ -1,260 +0,0 @@ -== Physical Plan == -* Project (47) -+- * BroadcastNestedLoopJoin Inner BuildRight (46) - :- * CometColumnarToRow (25) - : +- CometHashAggregate (24) - : +- CometExchange (23) - : +- CometHashAggregate (22) - : +- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometProject (15) - : : +- CometBroadcastHashJoin (14) - : : :- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (4) - : : +- CometBroadcastExchange (13) - : : +- CometProject (12) - : : +- CometFilter (11) - : : +- CometNativeScan parquet spark_catalog.default.time_dim (10) - : +- CometBroadcastExchange (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometNativeScan parquet spark_catalog.default.web_page (16) - +- BroadcastExchange (45) - +- * CometColumnarToRow (44) - +- CometHashAggregate (43) - +- CometExchange (42) - +- CometHashAggregate (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometNativeScan parquet spark_catalog.default.web_sales (26) - : : +- ReusedExchange (29) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometNativeScan parquet spark_catalog.default.time_dim (32) - +- ReusedExchange (38) - - -(1) CometNativeScan parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3)) - -(3) CometProject -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Arguments: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] - -(4) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#5, hd_dep_count#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,6), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 6)) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ws_ship_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, hd_demo_sk#5] -Arguments: [ws_sold_time_sk#1, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_web_page_sk#3] - -(10) CometNativeScan parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#7, t_hour#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,8), LessThanOrEqual(t_hour,9), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [t_time_sk#7, t_hour#8] -Condition : (((isnotnull(t_hour#8) AND (t_hour#8 >= 8)) AND (t_hour#8 <= 9)) AND isnotnull(t_time_sk#7)) - -(12) CometProject -Input [2]: [t_time_sk#7, t_hour#8] -Arguments: [t_time_sk#7], [t_time_sk#7] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#7] -Arguments: [t_time_sk#7] - -(14) CometBroadcastHashJoin -Left output [2]: [ws_sold_time_sk#1, ws_web_page_sk#3] -Right output [1]: [t_time_sk#7] -Arguments: [ws_sold_time_sk#1], [t_time_sk#7], Inner, BuildRight - -(15) CometProject -Input [3]: [ws_sold_time_sk#1, ws_web_page_sk#3, t_time_sk#7] -Arguments: [ws_web_page_sk#3], [ws_web_page_sk#3] - -(16) CometNativeScan parquet spark_catalog.default.web_page -Output [2]: [wp_web_page_sk#9, wp_char_count#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_char_count), GreaterThanOrEqual(wp_char_count,5000), LessThanOrEqual(wp_char_count,5200), IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [wp_web_page_sk#9, wp_char_count#10] -Condition : (((isnotnull(wp_char_count#10) AND (wp_char_count#10 >= 5000)) AND (wp_char_count#10 <= 5200)) AND isnotnull(wp_web_page_sk#9)) - -(18) CometProject -Input [2]: [wp_web_page_sk#9, wp_char_count#10] -Arguments: [wp_web_page_sk#9], [wp_web_page_sk#9] - -(19) CometBroadcastExchange -Input [1]: [wp_web_page_sk#9] -Arguments: [wp_web_page_sk#9] - -(20) CometBroadcastHashJoin -Left output [1]: [ws_web_page_sk#3] -Right output [1]: [wp_web_page_sk#9] -Arguments: [ws_web_page_sk#3], [wp_web_page_sk#9], Inner, BuildRight - -(21) CometProject -Input [2]: [ws_web_page_sk#3, wp_web_page_sk#9] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#11] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#11] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 2] -Input [1]: [amc#12] - -(26) CometNativeScan parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Condition : ((isnotnull(ws_ship_hdemo_sk#14) AND isnotnull(ws_sold_time_sk#13)) AND isnotnull(ws_web_page_sk#15)) - -(28) CometProject -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Arguments: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] - -(29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#17] - -(30) CometBroadcastHashJoin -Left output [3]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] -Right output [1]: [hd_demo_sk#17] -Arguments: [ws_ship_hdemo_sk#14], [hd_demo_sk#17], Inner, BuildRight - -(31) CometProject -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, hd_demo_sk#17] -Arguments: [ws_sold_time_sk#13, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_web_page_sk#15] - -(32) CometNativeScan parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#18, t_hour#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [t_time_sk#18, t_hour#19] -Condition : (((isnotnull(t_hour#19) AND (t_hour#19 >= 19)) AND (t_hour#19 <= 20)) AND isnotnull(t_time_sk#18)) - -(34) CometProject -Input [2]: [t_time_sk#18, t_hour#19] -Arguments: [t_time_sk#18], [t_time_sk#18] - -(35) CometBroadcastExchange -Input [1]: [t_time_sk#18] -Arguments: [t_time_sk#18] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_sold_time_sk#13, ws_web_page_sk#15] -Right output [1]: [t_time_sk#18] -Arguments: [ws_sold_time_sk#13], [t_time_sk#18], Inner, BuildRight - -(37) CometProject -Input [3]: [ws_sold_time_sk#13, ws_web_page_sk#15, t_time_sk#18] -Arguments: [ws_web_page_sk#15], [ws_web_page_sk#15] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [wp_web_page_sk#20] - -(39) CometBroadcastHashJoin -Left output [1]: [ws_web_page_sk#15] -Right output [1]: [wp_web_page_sk#20] -Arguments: [ws_web_page_sk#15], [wp_web_page_sk#20], Inner, BuildRight - -(40) CometProject -Input [2]: [ws_web_page_sk#15, wp_web_page_sk#20] - -(41) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(42) CometExchange -Input [1]: [count#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(43) CometHashAggregate -Input [1]: [count#21] -Keys: [] -Functions [1]: [count(1)] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [pmc#22] - -(45) BroadcastExchange -Input [1]: [pmc#22] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(46) BroadcastNestedLoopJoin [codegen id : 2] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 2] -Output [1]: [(cast(amc#12 as decimal(15,4)) / cast(pmc#22 as decimal(15,4))) AS am_pm_ratio#23] -Input [2]: [amc#12, pmc#22] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_datafusion/simplified.txt deleted file mode 100644 index a13072cd91..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_datafusion/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (2) - Project [amc,pmc] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [amc,count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_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_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange [wp_web_page_sk] #4 - CometProject [wp_web_page_sk] - CometFilter [wp_web_page_sk,wp_char_count] - CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [pmc,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_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_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometNativeScan 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 [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometNativeScan 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/q90.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_iceberg_compat/explain.txt deleted file mode 100644 index 08ae744b5a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,260 +0,0 @@ -== Physical Plan == -* Project (47) -+- * BroadcastNestedLoopJoin Inner BuildRight (46) - :- * CometColumnarToRow (25) - : +- CometHashAggregate (24) - : +- CometExchange (23) - : +- CometHashAggregate (22) - : +- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometProject (15) - : : +- CometBroadcastHashJoin (14) - : : :- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (4) - : : +- CometBroadcastExchange (13) - : : +- CometProject (12) - : : +- CometFilter (11) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (10) - : +- CometBroadcastExchange (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (16) - +- BroadcastExchange (45) - +- * CometColumnarToRow (44) - +- CometHashAggregate (43) - +- CometExchange (42) - +- CometHashAggregate (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (26) - : : +- ReusedExchange (29) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (32) - +- ReusedExchange (38) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3)) - -(3) CometProject -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Arguments: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#5, hd_dep_count#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,6), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 6)) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ws_ship_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, hd_demo_sk#5] -Arguments: [ws_sold_time_sk#1, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_web_page_sk#3] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#7, t_hour#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,8), LessThanOrEqual(t_hour,9), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [t_time_sk#7, t_hour#8] -Condition : (((isnotnull(t_hour#8) AND (t_hour#8 >= 8)) AND (t_hour#8 <= 9)) AND isnotnull(t_time_sk#7)) - -(12) CometProject -Input [2]: [t_time_sk#7, t_hour#8] -Arguments: [t_time_sk#7], [t_time_sk#7] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#7] -Arguments: [t_time_sk#7] - -(14) CometBroadcastHashJoin -Left output [2]: [ws_sold_time_sk#1, ws_web_page_sk#3] -Right output [1]: [t_time_sk#7] -Arguments: [ws_sold_time_sk#1], [t_time_sk#7], Inner, BuildRight - -(15) CometProject -Input [3]: [ws_sold_time_sk#1, ws_web_page_sk#3, t_time_sk#7] -Arguments: [ws_web_page_sk#3], [ws_web_page_sk#3] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [2]: [wp_web_page_sk#9, wp_char_count#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_char_count), GreaterThanOrEqual(wp_char_count,5000), LessThanOrEqual(wp_char_count,5200), IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [wp_web_page_sk#9, wp_char_count#10] -Condition : (((isnotnull(wp_char_count#10) AND (wp_char_count#10 >= 5000)) AND (wp_char_count#10 <= 5200)) AND isnotnull(wp_web_page_sk#9)) - -(18) CometProject -Input [2]: [wp_web_page_sk#9, wp_char_count#10] -Arguments: [wp_web_page_sk#9], [wp_web_page_sk#9] - -(19) CometBroadcastExchange -Input [1]: [wp_web_page_sk#9] -Arguments: [wp_web_page_sk#9] - -(20) CometBroadcastHashJoin -Left output [1]: [ws_web_page_sk#3] -Right output [1]: [wp_web_page_sk#9] -Arguments: [ws_web_page_sk#3], [wp_web_page_sk#9], Inner, BuildRight - -(21) CometProject -Input [2]: [ws_web_page_sk#3, wp_web_page_sk#9] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#11] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#11] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 2] -Input [1]: [amc#12] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Condition : ((isnotnull(ws_ship_hdemo_sk#14) AND isnotnull(ws_sold_time_sk#13)) AND isnotnull(ws_web_page_sk#15)) - -(28) CometProject -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Arguments: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] - -(29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#17] - -(30) CometBroadcastHashJoin -Left output [3]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] -Right output [1]: [hd_demo_sk#17] -Arguments: [ws_ship_hdemo_sk#14], [hd_demo_sk#17], Inner, BuildRight - -(31) CometProject -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, hd_demo_sk#17] -Arguments: [ws_sold_time_sk#13, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_web_page_sk#15] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#18, t_hour#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [t_time_sk#18, t_hour#19] -Condition : (((isnotnull(t_hour#19) AND (t_hour#19 >= 19)) AND (t_hour#19 <= 20)) AND isnotnull(t_time_sk#18)) - -(34) CometProject -Input [2]: [t_time_sk#18, t_hour#19] -Arguments: [t_time_sk#18], [t_time_sk#18] - -(35) CometBroadcastExchange -Input [1]: [t_time_sk#18] -Arguments: [t_time_sk#18] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_sold_time_sk#13, ws_web_page_sk#15] -Right output [1]: [t_time_sk#18] -Arguments: [ws_sold_time_sk#13], [t_time_sk#18], Inner, BuildRight - -(37) CometProject -Input [3]: [ws_sold_time_sk#13, ws_web_page_sk#15, t_time_sk#18] -Arguments: [ws_web_page_sk#15], [ws_web_page_sk#15] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [wp_web_page_sk#20] - -(39) CometBroadcastHashJoin -Left output [1]: [ws_web_page_sk#15] -Right output [1]: [wp_web_page_sk#20] -Arguments: [ws_web_page_sk#15], [wp_web_page_sk#20], Inner, BuildRight - -(40) CometProject -Input [2]: [ws_web_page_sk#15, wp_web_page_sk#20] - -(41) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(42) CometExchange -Input [1]: [count#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(43) CometHashAggregate -Input [1]: [count#21] -Keys: [] -Functions [1]: [count(1)] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [pmc#22] - -(45) BroadcastExchange -Input [1]: [pmc#22] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(46) BroadcastNestedLoopJoin [codegen id : 2] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 2] -Output [1]: [(cast(amc#12 as decimal(15,4)) / cast(pmc#22 as decimal(15,4))) AS am_pm_ratio#23] -Input [2]: [amc#12, pmc#22] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_iceberg_compat/simplified.txt deleted file mode 100644 index 0991e4e0dc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (2) - Project [amc,pmc] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [amc,count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_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_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange [wp_web_page_sk] #4 - CometProject [wp_web_page_sk] - CometFilter [wp_web_page_sk,wp_char_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [pmc,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_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_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometScan [native_iceberg_compat] 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/q90/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/explain.txt deleted file mode 100644 index 08ae744b5a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/explain.txt +++ /dev/null @@ -1,260 +0,0 @@ -== Physical Plan == -* Project (47) -+- * BroadcastNestedLoopJoin Inner BuildRight (46) - :- * CometColumnarToRow (25) - : +- CometHashAggregate (24) - : +- CometExchange (23) - : +- CometHashAggregate (22) - : +- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometProject (15) - : : +- CometBroadcastHashJoin (14) - : : :- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (4) - : : +- CometBroadcastExchange (13) - : : +- CometProject (12) - : : +- CometFilter (11) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (10) - : +- CometBroadcastExchange (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (16) - +- BroadcastExchange (45) - +- * CometColumnarToRow (44) - +- CometHashAggregate (43) - +- CometExchange (42) - +- CometHashAggregate (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (26) - : : +- ReusedExchange (29) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (32) - +- ReusedExchange (38) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3)) - -(3) CometProject -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Arguments: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#5, hd_dep_count#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,6), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 6)) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ws_ship_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, hd_demo_sk#5] -Arguments: [ws_sold_time_sk#1, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_web_page_sk#3] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#7, t_hour#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,8), LessThanOrEqual(t_hour,9), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [t_time_sk#7, t_hour#8] -Condition : (((isnotnull(t_hour#8) AND (t_hour#8 >= 8)) AND (t_hour#8 <= 9)) AND isnotnull(t_time_sk#7)) - -(12) CometProject -Input [2]: [t_time_sk#7, t_hour#8] -Arguments: [t_time_sk#7], [t_time_sk#7] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#7] -Arguments: [t_time_sk#7] - -(14) CometBroadcastHashJoin -Left output [2]: [ws_sold_time_sk#1, ws_web_page_sk#3] -Right output [1]: [t_time_sk#7] -Arguments: [ws_sold_time_sk#1], [t_time_sk#7], Inner, BuildRight - -(15) CometProject -Input [3]: [ws_sold_time_sk#1, ws_web_page_sk#3, t_time_sk#7] -Arguments: [ws_web_page_sk#3], [ws_web_page_sk#3] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [2]: [wp_web_page_sk#9, wp_char_count#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_char_count), GreaterThanOrEqual(wp_char_count,5000), LessThanOrEqual(wp_char_count,5200), IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [wp_web_page_sk#9, wp_char_count#10] -Condition : (((isnotnull(wp_char_count#10) AND (wp_char_count#10 >= 5000)) AND (wp_char_count#10 <= 5200)) AND isnotnull(wp_web_page_sk#9)) - -(18) CometProject -Input [2]: [wp_web_page_sk#9, wp_char_count#10] -Arguments: [wp_web_page_sk#9], [wp_web_page_sk#9] - -(19) CometBroadcastExchange -Input [1]: [wp_web_page_sk#9] -Arguments: [wp_web_page_sk#9] - -(20) CometBroadcastHashJoin -Left output [1]: [ws_web_page_sk#3] -Right output [1]: [wp_web_page_sk#9] -Arguments: [ws_web_page_sk#3], [wp_web_page_sk#9], Inner, BuildRight - -(21) CometProject -Input [2]: [ws_web_page_sk#3, wp_web_page_sk#9] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#11] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#11] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 2] -Input [1]: [amc#12] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Condition : ((isnotnull(ws_ship_hdemo_sk#14) AND isnotnull(ws_sold_time_sk#13)) AND isnotnull(ws_web_page_sk#15)) - -(28) CometProject -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Arguments: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] - -(29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#17] - -(30) CometBroadcastHashJoin -Left output [3]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] -Right output [1]: [hd_demo_sk#17] -Arguments: [ws_ship_hdemo_sk#14], [hd_demo_sk#17], Inner, BuildRight - -(31) CometProject -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, hd_demo_sk#17] -Arguments: [ws_sold_time_sk#13, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_web_page_sk#15] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#18, t_hour#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [t_time_sk#18, t_hour#19] -Condition : (((isnotnull(t_hour#19) AND (t_hour#19 >= 19)) AND (t_hour#19 <= 20)) AND isnotnull(t_time_sk#18)) - -(34) CometProject -Input [2]: [t_time_sk#18, t_hour#19] -Arguments: [t_time_sk#18], [t_time_sk#18] - -(35) CometBroadcastExchange -Input [1]: [t_time_sk#18] -Arguments: [t_time_sk#18] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_sold_time_sk#13, ws_web_page_sk#15] -Right output [1]: [t_time_sk#18] -Arguments: [ws_sold_time_sk#13], [t_time_sk#18], Inner, BuildRight - -(37) CometProject -Input [3]: [ws_sold_time_sk#13, ws_web_page_sk#15, t_time_sk#18] -Arguments: [ws_web_page_sk#15], [ws_web_page_sk#15] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [wp_web_page_sk#20] - -(39) CometBroadcastHashJoin -Left output [1]: [ws_web_page_sk#15] -Right output [1]: [wp_web_page_sk#20] -Arguments: [ws_web_page_sk#15], [wp_web_page_sk#20], Inner, BuildRight - -(40) CometProject -Input [2]: [ws_web_page_sk#15, wp_web_page_sk#20] - -(41) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(42) CometExchange -Input [1]: [count#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(43) CometHashAggregate -Input [1]: [count#21] -Keys: [] -Functions [1]: [count(1)] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [pmc#22] - -(45) BroadcastExchange -Input [1]: [pmc#22] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(46) BroadcastNestedLoopJoin [codegen id : 2] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 2] -Output [1]: [(cast(amc#12 as decimal(15,4)) / cast(pmc#22 as decimal(15,4))) AS am_pm_ratio#23] -Input [2]: [amc#12, pmc#22] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/extended.txt deleted file mode 100644 index aa6c577ed7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/extended.txt +++ /dev/null @@ -1,55 +0,0 @@ -Project -+- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- BroadcastExchange - +- CometColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - -Comet accelerated 48 out of 51 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 0991e4e0dc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (2) - Project [amc,pmc] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [amc,count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_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_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange [wp_web_page_sk] #4 - CometProject [wp_web_page_sk] - CometFilter [wp_web_page_sk,wp_char_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [pmc,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_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_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/explain.txt deleted file mode 100644 index b52840adb9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/explain.txt +++ /dev/null @@ -1,299 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometSort (46) - +- CometColumnarExchange (45) - +- * HashAggregate (44) - +- * CometColumnarToRow (43) - +- CometColumnarExchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (19) - : : : +- * BroadcastHashJoin Inner BuildRight (18) - : : : :- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Project (10) - : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : :- * CometColumnarToRow (4) - : : : : : : +- CometProject (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometNativeScan parquet spark_catalog.default.call_center (1) - : : : : : +- BroadcastExchange (8) - : : : : : +- * Filter (7) - : : : : : +- * ColumnarToRow (6) - : : : : : +- Scan parquet spark_catalog.default.catalog_returns (5) - : : : : +- ReusedExchange (11) - : : : +- BroadcastExchange (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometFilter (15) - : : : +- CometNativeScan parquet spark_catalog.default.customer (14) - : : +- BroadcastExchange (24) - : : +- * CometColumnarToRow (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (20) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometNativeScan parquet spark_catalog.default.customer_demographics (27) - +- BroadcastExchange (38) - +- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometNativeScan parquet spark_catalog.default.household_demographics (34) - - -(1) CometNativeScan parquet spark_catalog.default.call_center -Output [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Condition : isnotnull(cc_call_center_sk#1) - -(3) CometProject -Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Arguments: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4], [cc_call_center_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cc_call_center_id#2, 16, true, false, true) AS cc_call_center_id#5, cc_name#3, cc_manager#4] - -(4) CometColumnarToRow [codegen id : 7] -Input [4]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4] - -(5) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#9), dynamicpruningexpression(cr_returned_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] - -(7) Filter [codegen id : 1] -Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Condition : (isnotnull(cr_call_center_sk#7) AND isnotnull(cr_returning_customer_sk#6)) - -(8) BroadcastExchange -Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cc_call_center_sk#1] -Right keys [1]: [cr_call_center_sk#7] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 7] -Output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9] -Input [8]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] - -(11) ReusedExchange [Reuses operator id: 52] -Output [1]: [d_date_sk#11] - -(12) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cr_returned_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 7] -Output [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8] -Input [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9, d_date_sk#11] - -(14) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#12, c_current_cdemo_sk#13, c_current_hdemo_sk#14, c_current_addr_sk#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [c_customer_sk#12, c_current_cdemo_sk#13, c_current_hdemo_sk#14, c_current_addr_sk#15] -Condition : (((isnotnull(c_customer_sk#12) AND isnotnull(c_current_addr_sk#15)) AND isnotnull(c_current_cdemo_sk#13)) AND isnotnull(c_current_hdemo_sk#14)) - -(16) CometColumnarToRow [codegen id : 3] -Input [4]: [c_customer_sk#12, c_current_cdemo_sk#13, c_current_hdemo_sk#14, c_current_addr_sk#15] - -(17) BroadcastExchange -Input [4]: [c_customer_sk#12, c_current_cdemo_sk#13, c_current_hdemo_sk#14, c_current_addr_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cr_returning_customer_sk#6] -Right keys [1]: [c_customer_sk#12] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 7] -Output [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#13, c_current_hdemo_sk#14, c_current_addr_sk#15] -Input [9]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, c_customer_sk#12, c_current_cdemo_sk#13, c_current_hdemo_sk#14, c_current_addr_sk#15] - -(20) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_gmt_offset#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-7.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [ca_address_sk#16, ca_gmt_offset#17] -Condition : ((isnotnull(ca_gmt_offset#17) AND (ca_gmt_offset#17 = -7.00)) AND isnotnull(ca_address_sk#16)) - -(22) CometProject -Input [2]: [ca_address_sk#16, ca_gmt_offset#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(23) CometColumnarToRow [codegen id : 4] -Input [1]: [ca_address_sk#16] - -(24) BroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(25) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#15] -Right keys [1]: [ca_address_sk#16] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 7] -Output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#13, c_current_hdemo_sk#14] -Input [8]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#13, c_current_hdemo_sk#14, c_current_addr_sk#15, ca_address_sk#16] - -(27) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(28) CometFilter -Input [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#20, 20, true, false, true) = Unknown )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = W) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#20, 20, true, false, true) = Advanced Degree ))) AND isnotnull(cd_demo_sk#18)) - -(29) CometProject -Input [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] -Arguments: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22], [cd_demo_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) AS cd_marital_status#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#20, 20, true, false, true) AS cd_education_status#22] - -(30) CometColumnarToRow [codegen id : 5] -Input [3]: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] - -(31) BroadcastExchange -Input [3]: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#13] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 7] -Output [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#14, cd_marital_status#21, cd_education_status#22] -Input [9]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#13, c_current_hdemo_sk#14, cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] - -(34) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#23, hd_buy_potential#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [hd_demo_sk#23, hd_buy_potential#24] -Condition : (StartsWith(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#24, 15, true, false, true), Unknown) AND isnotnull(hd_demo_sk#23)) - -(36) CometProject -Input [2]: [hd_demo_sk#23, hd_buy_potential#24] -Arguments: [hd_demo_sk#23], [hd_demo_sk#23] - -(37) CometColumnarToRow [codegen id : 6] -Input [1]: [hd_demo_sk#23] - -(38) BroadcastExchange -Input [1]: [hd_demo_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_hdemo_sk#14] -Right keys [1]: [hd_demo_sk#23] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 7] -Output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#21, cd_education_status#22] -Input [8]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#14, cd_marital_status#21, cd_education_status#22, hd_demo_sk#23] - -(41) HashAggregate [codegen id : 7] -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#21, cd_education_status#22] -Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#21, cd_education_status#22] -Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#8))] -Aggregate Attributes [1]: [sum#25] -Results [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#21, cd_education_status#22, sum#26] - -(42) CometColumnarExchange -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#21, cd_education_status#22, sum#26] -Arguments: hashpartitioning(cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#21, cd_education_status#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) CometColumnarToRow [codegen id : 8] -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#21, cd_education_status#22, sum#26] - -(44) HashAggregate [codegen id : 8] -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#21, cd_education_status#22, sum#26] -Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#21, cd_education_status#22] -Functions [1]: [sum(UnscaledValue(cr_net_loss#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#8))#27] -Results [4]: [cc_call_center_id#5 AS Call_Center#28, cc_name#3 AS Call_Center_Name#29, cc_manager#4 AS Manager#30, MakeDecimal(sum(UnscaledValue(cr_net_loss#8))#27,17,2) AS Returns_Loss#31] - -(45) CometColumnarExchange -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(46) CometSort -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31], [Returns_Loss#31 DESC NULLS LAST] - -(47) CometColumnarToRow [codegen id : 9] -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = cr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometNativeScan parquet spark_catalog.default.date_dim (48) - - -(48) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#32, d_moy#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#11, d_year#32, d_moy#33] -Condition : ((((isnotnull(d_year#32) AND isnotnull(d_moy#33)) AND (d_year#32 = 1998)) AND (d_moy#33 = 11)) AND isnotnull(d_date_sk#11)) - -(50) CometProject -Input [3]: [d_date_sk#11, d_year#32, d_moy#33] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(52) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/simplified.txt deleted file mode 100644 index 75fea0ec7f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/simplified.txt +++ /dev/null @@ -1,77 +0,0 @@ -WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] - CometColumnarExchange [Returns_Loss] #1 - WholeStageCodegen (8) - HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] [sum(UnscaledValue(cr_net_loss)),Call_Center,Call_Center_Name,Manager,Returns_Loss,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 - WholeStageCodegen (7) - HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] [sum,sum] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - BroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] - BroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] - CometColumnarToRow - InputAdapter - CometProject [cc_call_center_id] [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [cr_call_center_sk,cr_returning_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometNativeScan 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/q91.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/explain.txt deleted file mode 100644 index cf532d2b41..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,280 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (43) -+- CometSort (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (19) - : : : +- CometBroadcastHashJoin (18) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometProject (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (4) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : : +- CometBroadcastExchange (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (15) - : : +- CometBroadcastExchange (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Condition : isnotnull(cc_call_center_sk#1) - -(3) CometProject -Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Arguments: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4], [cc_call_center_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cc_call_center_id#2, 16, true, false, true) AS cc_call_center_id#5, cc_name#3, cc_manager#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#9), dynamicpruningexpression(cr_returned_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Condition : (isnotnull(cr_call_center_sk#7) AND isnotnull(cr_returning_customer_sk#6)) - -(6) CometBroadcastExchange -Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Arguments: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [4]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4] -Right output [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Arguments: [cc_call_center_sk#1], [cr_call_center_sk#7], Inner, BuildRight - -(8) CometProject -Input [8]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#13)) AND (d_year#12 = 1998)) AND (d_moy#13 = 11)) AND isnotnull(d_date_sk#11)) - -(11) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(13) CometBroadcastHashJoin -Left output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [cr_returned_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(14) CometProject -Input [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9, d_date_sk#11] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Condition : (((isnotnull(c_customer_sk#14) AND isnotnull(c_current_addr_sk#17)) AND isnotnull(c_current_cdemo_sk#15)) AND isnotnull(c_current_hdemo_sk#16)) - -(17) CometBroadcastExchange -Input [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Arguments: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] - -(18) CometBroadcastHashJoin -Left output [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8] -Right output [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Arguments: [cr_returning_customer_sk#6], [c_customer_sk#14], Inner, BuildRight - -(19) CometProject -Input [9]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_gmt_offset#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-7.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [ca_address_sk#18, ca_gmt_offset#19] -Condition : ((isnotnull(ca_gmt_offset#19) AND (ca_gmt_offset#19 = -7.00)) AND isnotnull(ca_address_sk#18)) - -(22) CometProject -Input [2]: [ca_address_sk#18, ca_gmt_offset#19] -Arguments: [ca_address_sk#18], [ca_address_sk#18] - -(23) CometBroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: [ca_address_sk#18] - -(24) CometBroadcastHashJoin -Left output [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Right output [1]: [ca_address_sk#18] -Arguments: [c_current_addr_sk#17], [ca_address_sk#18], Inner, BuildRight - -(25) CometProject -Input [8]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17, ca_address_sk#18] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#21, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#22, 20, true, false, true) = Unknown )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#21, 1, true, false, true) = W) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#22, 20, true, false, true) = Advanced Degree ))) AND isnotnull(cd_demo_sk#20)) - -(28) CometProject -Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Arguments: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24], [cd_demo_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#21, 1, true, false, true) AS cd_marital_status#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#22, 20, true, false, true) AS cd_education_status#24] - -(29) CometBroadcastExchange -Input [3]: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] -Arguments: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] - -(30) CometBroadcastHashJoin -Left output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16] -Right output [3]: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] -Arguments: [c_current_cdemo_sk#15], [cd_demo_sk#20], Inner, BuildRight - -(31) CometProject -Input [9]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#25, hd_buy_potential#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [hd_demo_sk#25, hd_buy_potential#26] -Condition : (StartsWith(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#26, 15, true, false, true), Unknown) AND isnotnull(hd_demo_sk#25)) - -(34) CometProject -Input [2]: [hd_demo_sk#25, hd_buy_potential#26] -Arguments: [hd_demo_sk#25], [hd_demo_sk#25] - -(35) CometBroadcastExchange -Input [1]: [hd_demo_sk#25] -Arguments: [hd_demo_sk#25] - -(36) CometBroadcastHashJoin -Left output [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24] -Right output [1]: [hd_demo_sk#25] -Arguments: [c_current_hdemo_sk#16], [hd_demo_sk#25], Inner, BuildRight - -(37) CometProject -Input [8]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24, hd_demo_sk#25] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#23, cd_education_status#24], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#23, cd_education_status#24] - -(38) CometHashAggregate -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#23, cd_education_status#24] -Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24] -Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#8))] - -(39) CometExchange -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, sum#27] -Arguments: hashpartitioning(cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(40) CometHashAggregate -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, sum#27] -Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24] -Functions [1]: [sum(UnscaledValue(cr_net_loss#8))] - -(41) CometExchange -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(42) CometSort -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31], [Returns_Loss#31 DESC NULLS LAST] - -(43) CometColumnarToRow [codegen id : 1] -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = cr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) - - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#13)) AND (d_year#12 = 1998)) AND (d_moy#13 = 11)) AND isnotnull(d_date_sk#11)) - -(46) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(48) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/simplified.txt deleted file mode 100644 index 3e9b8945da..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,53 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] - CometExchange [Returns_Loss] #1 - CometHashAggregate [cd_marital_status,cd_education_status,sum] [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,sum(UnscaledValue(cr_net_loss))] - CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 - CometHashAggregate [cr_net_loss] [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - 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 [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 [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 [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 [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,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] - CometProject [cc_call_center_id] [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk] #7 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [hd_demo_sk] #9 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan [native_iceberg_compat] 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/q91/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/explain.txt deleted file mode 100644 index cf532d2b41..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/explain.txt +++ /dev/null @@ -1,280 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (43) -+- CometSort (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (19) - : : : +- CometBroadcastHashJoin (18) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometProject (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (4) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : : +- CometBroadcastExchange (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (15) - : : +- CometBroadcastExchange (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Condition : isnotnull(cc_call_center_sk#1) - -(3) CometProject -Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Arguments: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4], [cc_call_center_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cc_call_center_id#2, 16, true, false, true) AS cc_call_center_id#5, cc_name#3, cc_manager#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#9), dynamicpruningexpression(cr_returned_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Condition : (isnotnull(cr_call_center_sk#7) AND isnotnull(cr_returning_customer_sk#6)) - -(6) CometBroadcastExchange -Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Arguments: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [4]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4] -Right output [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Arguments: [cc_call_center_sk#1], [cr_call_center_sk#7], Inner, BuildRight - -(8) CometProject -Input [8]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#13)) AND (d_year#12 = 1998)) AND (d_moy#13 = 11)) AND isnotnull(d_date_sk#11)) - -(11) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(13) CometBroadcastHashJoin -Left output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [cr_returned_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(14) CometProject -Input [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9, d_date_sk#11] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Condition : (((isnotnull(c_customer_sk#14) AND isnotnull(c_current_addr_sk#17)) AND isnotnull(c_current_cdemo_sk#15)) AND isnotnull(c_current_hdemo_sk#16)) - -(17) CometBroadcastExchange -Input [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Arguments: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] - -(18) CometBroadcastHashJoin -Left output [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8] -Right output [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Arguments: [cr_returning_customer_sk#6], [c_customer_sk#14], Inner, BuildRight - -(19) CometProject -Input [9]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_gmt_offset#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-7.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [ca_address_sk#18, ca_gmt_offset#19] -Condition : ((isnotnull(ca_gmt_offset#19) AND (ca_gmt_offset#19 = -7.00)) AND isnotnull(ca_address_sk#18)) - -(22) CometProject -Input [2]: [ca_address_sk#18, ca_gmt_offset#19] -Arguments: [ca_address_sk#18], [ca_address_sk#18] - -(23) CometBroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: [ca_address_sk#18] - -(24) CometBroadcastHashJoin -Left output [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Right output [1]: [ca_address_sk#18] -Arguments: [c_current_addr_sk#17], [ca_address_sk#18], Inner, BuildRight - -(25) CometProject -Input [8]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17, ca_address_sk#18] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#21, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#22, 20, true, false, true) = Unknown )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#21, 1, true, false, true) = W) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#22, 20, true, false, true) = Advanced Degree ))) AND isnotnull(cd_demo_sk#20)) - -(28) CometProject -Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Arguments: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24], [cd_demo_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#21, 1, true, false, true) AS cd_marital_status#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#22, 20, true, false, true) AS cd_education_status#24] - -(29) CometBroadcastExchange -Input [3]: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] -Arguments: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] - -(30) CometBroadcastHashJoin -Left output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16] -Right output [3]: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] -Arguments: [c_current_cdemo_sk#15], [cd_demo_sk#20], Inner, BuildRight - -(31) CometProject -Input [9]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#25, hd_buy_potential#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [hd_demo_sk#25, hd_buy_potential#26] -Condition : (StartsWith(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#26, 15, true, false, true), Unknown) AND isnotnull(hd_demo_sk#25)) - -(34) CometProject -Input [2]: [hd_demo_sk#25, hd_buy_potential#26] -Arguments: [hd_demo_sk#25], [hd_demo_sk#25] - -(35) CometBroadcastExchange -Input [1]: [hd_demo_sk#25] -Arguments: [hd_demo_sk#25] - -(36) CometBroadcastHashJoin -Left output [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24] -Right output [1]: [hd_demo_sk#25] -Arguments: [c_current_hdemo_sk#16], [hd_demo_sk#25], Inner, BuildRight - -(37) CometProject -Input [8]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24, hd_demo_sk#25] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#23, cd_education_status#24], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#23, cd_education_status#24] - -(38) CometHashAggregate -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#23, cd_education_status#24] -Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24] -Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#8))] - -(39) CometExchange -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, sum#27] -Arguments: hashpartitioning(cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(40) CometHashAggregate -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, sum#27] -Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24] -Functions [1]: [sum(UnscaledValue(cr_net_loss#8))] - -(41) CometExchange -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(42) CometSort -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31], [Returns_Loss#31 DESC NULLS LAST] - -(43) CometColumnarToRow [codegen id : 1] -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = cr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) - - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#13)) AND (d_year#12 = 1998)) AND (d_moy#13 = 11)) AND isnotnull(d_date_sk#11)) - -(46) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(48) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/extended.txt deleted file mode 100644 index a1c6b73d37..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/extended.txt +++ /dev/null @@ -1,51 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 3e9b8945da..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt +++ /dev/null @@ -1,53 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] - CometExchange [Returns_Loss] #1 - CometHashAggregate [cd_marital_status,cd_education_status,sum] [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,sum(UnscaledValue(cr_net_loss))] - CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 - CometHashAggregate [cr_net_loss] [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - 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 [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 [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 [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 [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,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] - CometProject [cc_call_center_id] [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk] #7 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [hd_demo_sk] #9 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/explain.txt deleted file mode 100644 index d96384bc72..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -* HashAggregate (31) -+- * CometColumnarToRow (30) - +- CometColumnarExchange (29) - +- * HashAggregate (28) - +- * Project (27) - +- * BroadcastHashJoin Inner BuildRight (26) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : +- BroadcastExchange (8) - : : +- * CometColumnarToRow (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometNativeScan parquet spark_catalog.default.item (4) - : +- BroadcastExchange (22) - : +- * Filter (21) - : +- * HashAggregate (20) - : +- * CometColumnarToRow (19) - : +- CometColumnarExchange (18) - : +- * HashAggregate (17) - : +- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet spark_catalog.default.web_sales (11) - : +- ReusedExchange (14) - +- ReusedExchange (25) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_ext_discount_amt)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] - -(3) Filter [codegen id : 6] -Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Condition : (isnotnull(ws_item_sk#1) AND isnotnull(ws_ext_discount_amt#2)) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_manufact_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,350), IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [i_item_sk#5, i_manufact_id#6] -Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 350)) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [2]: [i_item_sk#5, i_manufact_id#6] -Arguments: [i_item_sk#5], [i_item_sk#5] - -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [i_item_sk#5] - -(8) BroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 6] -Output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] -Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] - -(11) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] - -(13) Filter [codegen id : 3] -Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Condition : isnotnull(ws_item_sk#7) - -(14) ReusedExchange [Reuses operator id: 36] -Output [1]: [d_date_sk#10] - -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(16) Project [codegen id : 3] -Output [2]: [ws_item_sk#7, ws_ext_discount_amt#8] -Input [4]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9, d_date_sk#10] - -(17) HashAggregate [codegen id : 3] -Input [2]: [ws_item_sk#7, ws_ext_discount_amt#8] -Keys [1]: [ws_item_sk#7] -Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#8))] -Aggregate Attributes [2]: [sum#11, count#12] -Results [3]: [ws_item_sk#7, sum#13, count#14] - -(18) CometColumnarExchange -Input [3]: [ws_item_sk#7, sum#13, count#14] -Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(19) CometColumnarToRow [codegen id : 4] -Input [3]: [ws_item_sk#7, sum#13, count#14] - -(20) HashAggregate [codegen id : 4] -Input [3]: [ws_item_sk#7, sum#13, count#14] -Keys [1]: [ws_item_sk#7] -Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))] -Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))#15] -Results [2]: [(1.3 * cast((avg(UnscaledValue(ws_ext_discount_amt#8))#15 / 100.0) as decimal(11,6))) AS (1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] - -(21) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] -Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#16) - -(22) BroadcastExchange -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] - -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [i_item_sk#5] -Right keys [1]: [ws_item_sk#7] -Join type: Inner -Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#16) - -(24) Project [codegen id : 6] -Output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] -Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] - -(25) ReusedExchange [Reuses operator id: 36] -Output [1]: [d_date_sk#17] - -(26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 6] -Output [1]: [ws_ext_discount_amt#2] -Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#17] - -(28) HashAggregate [codegen id : 6] -Input [1]: [ws_ext_discount_amt#2] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#18] -Results [1]: [sum#19] - -(29) CometColumnarExchange -Input [1]: [sum#19] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 7] -Input [1]: [sum#19] - -(31) HashAggregate [codegen id : 7] -Input [1]: [sum#19] -Keys: [] -Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#20] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#20,17,2) AS Excess Discount Amount #21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.date_dim (32) - - -(32) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#17, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 2000-01-27)) AND (d_date#22 <= 2000-04-26)) AND isnotnull(d_date_sk#17)) - -(34) CometProject -Input [2]: [d_date_sk#17, d_date#22] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] - -(36) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/simplified.txt deleted file mode 100644 index 17df728966..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -WholeStageCodegen (7) - HashAggregate [sum] [sum(UnscaledValue(ws_ext_discount_amt)),Excess Discount Amount ,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (6) - HashAggregate [ws_ext_discount_amt] [sum,sum] - Project [ws_ext_discount_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_discount_amt,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk,ws_ext_discount_amt,(1.3 * avg(ws_ext_discount_amt))] - Project [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk,ws_ext_discount_amt] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Filter [(1.3 * avg(ws_ext_discount_amt))] - HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk] #5 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,ws_ext_discount_amt] [sum,count,sum,count] - Project [ws_item_sk,ws_ext_discount_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/explain.txt deleted file mode 100644 index cdc8dbc3bc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (30) -+- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : +- CometBroadcastExchange (21) - : +- CometFilter (20) - : +- CometHashAggregate (19) - : +- CometExchange (18) - : +- CometHashAggregate (17) - : +- CometProject (16) - : +- CometBroadcastHashJoin (15) - : :- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (9) - : +- CometBroadcastExchange (14) - : +- CometProject (13) - : +- CometFilter (12) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (11) - +- ReusedExchange (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_ext_discount_amt)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Condition : (isnotnull(ws_item_sk#1) AND isnotnull(ws_ext_discount_amt#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_manufact_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,350), IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_manufact_id#6] -Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 350)) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [2]: [i_item_sk#5, i_manufact_id#6] -Arguments: [i_item_sk#5], [i_item_sk#5] - -(6) CometBroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: [i_item_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Right output [1]: [i_item_sk#5] -Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] -Arguments: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5], [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Condition : isnotnull(ws_item_sk#7) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) - -(13) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(15) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [ws_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(16) CometProject -Input [4]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9, d_date_sk#11] -Arguments: [ws_item_sk#7, ws_ext_discount_amt#8], [ws_item_sk#7, ws_ext_discount_amt#8] - -(17) CometHashAggregate -Input [2]: [ws_item_sk#7, ws_ext_discount_amt#8] -Keys [1]: [ws_item_sk#7] -Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#8))] - -(18) CometExchange -Input [3]: [ws_item_sk#7, sum#13, count#14] -Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [3]: [ws_item_sk#7, sum#13, count#14] -Keys [1]: [ws_item_sk#7] -Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))] - -(20) CometFilter -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#15) - -(21) CometBroadcastExchange -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Arguments: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] - -(22) CometBroadcastHashJoin -Left output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] -Right output [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Arguments: [i_item_sk#5], [ws_item_sk#7], Inner, (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#15), BuildRight - -(23) CometProject -Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Arguments: [ws_ext_discount_amt#2, ws_sold_date_sk#3], [ws_ext_discount_amt#2, ws_sold_date_sk#3] - -(24) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] -Right output [1]: [d_date_sk#16] -Arguments: [ws_sold_date_sk#3], [d_date_sk#16], Inner, BuildRight - -(26) CometProject -Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#16] -Arguments: [ws_ext_discount_amt#2], [ws_ext_discount_amt#2] - -(27) CometHashAggregate -Input [1]: [ws_ext_discount_amt#2] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] - -(28) CometExchange -Input [1]: [sum#17] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [1]: [sum#17] -Keys: [] -Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [Excess Discount Amount #18] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#16, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#19 <= 2000-04-26)) AND isnotnull(d_date_sk#16)) - -(33) CometProject -Input [2]: [d_date_sk#16, d_date#19] -Arguments: [d_date_sk#16], [d_date_sk#16] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#16] - -(35) BroadcastExchange -Input [1]: [d_date_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 9 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/simplified.txt deleted file mode 100644 index 4ec5755df7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [Excess Discount Amount ,sum(UnscaledValue(ws_ext_discount_amt))] - CometExchange #1 - CometHashAggregate [ws_ext_discount_amt] [sum] - CometProject [ws_ext_discount_amt] - CometBroadcastHashJoin [ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] - CometProject [ws_ext_discount_amt,ws_sold_date_sk] - CometBroadcastHashJoin [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk,(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] - CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk] #3 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] #4 - CometFilter [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] - CometHashAggregate [sum,count] [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,avg(UnscaledValue(ws_ext_discount_amt))] - CometExchange [ws_item_sk] #5 - CometHashAggregate [ws_ext_discount_amt] [ws_item_sk,sum,count] - CometProject [ws_item_sk,ws_ext_discount_amt] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/explain.txt deleted file mode 100644 index cdc8dbc3bc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (30) -+- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : +- CometBroadcastExchange (21) - : +- CometFilter (20) - : +- CometHashAggregate (19) - : +- CometExchange (18) - : +- CometHashAggregate (17) - : +- CometProject (16) - : +- CometBroadcastHashJoin (15) - : :- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (9) - : +- CometBroadcastExchange (14) - : +- CometProject (13) - : +- CometFilter (12) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (11) - +- ReusedExchange (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_ext_discount_amt)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Condition : (isnotnull(ws_item_sk#1) AND isnotnull(ws_ext_discount_amt#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_manufact_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,350), IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_manufact_id#6] -Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 350)) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [2]: [i_item_sk#5, i_manufact_id#6] -Arguments: [i_item_sk#5], [i_item_sk#5] - -(6) CometBroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: [i_item_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Right output [1]: [i_item_sk#5] -Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] -Arguments: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5], [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Condition : isnotnull(ws_item_sk#7) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) - -(13) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(15) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [ws_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(16) CometProject -Input [4]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9, d_date_sk#11] -Arguments: [ws_item_sk#7, ws_ext_discount_amt#8], [ws_item_sk#7, ws_ext_discount_amt#8] - -(17) CometHashAggregate -Input [2]: [ws_item_sk#7, ws_ext_discount_amt#8] -Keys [1]: [ws_item_sk#7] -Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#8))] - -(18) CometExchange -Input [3]: [ws_item_sk#7, sum#13, count#14] -Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [3]: [ws_item_sk#7, sum#13, count#14] -Keys [1]: [ws_item_sk#7] -Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))] - -(20) CometFilter -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#15) - -(21) CometBroadcastExchange -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Arguments: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] - -(22) CometBroadcastHashJoin -Left output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] -Right output [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Arguments: [i_item_sk#5], [ws_item_sk#7], Inner, (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#15), BuildRight - -(23) CometProject -Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Arguments: [ws_ext_discount_amt#2, ws_sold_date_sk#3], [ws_ext_discount_amt#2, ws_sold_date_sk#3] - -(24) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] -Right output [1]: [d_date_sk#16] -Arguments: [ws_sold_date_sk#3], [d_date_sk#16], Inner, BuildRight - -(26) CometProject -Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#16] -Arguments: [ws_ext_discount_amt#2], [ws_ext_discount_amt#2] - -(27) CometHashAggregate -Input [1]: [ws_ext_discount_amt#2] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] - -(28) CometExchange -Input [1]: [sum#17] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [1]: [sum#17] -Keys: [] -Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [Excess Discount Amount #18] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#16, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#19 <= 2000-04-26)) AND isnotnull(d_date_sk#16)) - -(33) CometProject -Input [2]: [d_date_sk#16, d_date#19] -Arguments: [d_date_sk#16], [d_date_sk#16] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#16] - -(35) BroadcastExchange -Input [1]: [d_date_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 9 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/extended.txt deleted file mode 100644 index 20df9a8047..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 4ec5755df7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [Excess Discount Amount ,sum(UnscaledValue(ws_ext_discount_amt))] - CometExchange #1 - CometHashAggregate [ws_ext_discount_amt] [sum] - CometProject [ws_ext_discount_amt] - CometBroadcastHashJoin [ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] - CometProject [ws_ext_discount_amt,ws_sold_date_sk] - CometBroadcastHashJoin [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk,(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] - CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk] #3 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] #4 - CometFilter [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] - CometHashAggregate [sum,count] [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,avg(UnscaledValue(ws_ext_discount_amt))] - CometExchange [ws_item_sk] #5 - CometHashAggregate [ws_ext_discount_amt] [ws_item_sk,sum,count] - CometProject [ws_item_sk,ws_ext_discount_amt] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_datafusion/explain.txt deleted file mode 100644 index 3f38adcd39..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_datafusion/explain.txt +++ /dev/null @@ -1,124 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (22) -+- CometTakeOrderedAndProject (21) - +- CometHashAggregate (20) - +- CometExchange (19) - +- CometHashAggregate (18) - +- CometProject (17) - +- CometBroadcastHashJoin (16) - :- CometProject (11) - : +- CometSortMergeJoin (10) - : :- CometSort (4) - : : +- CometExchange (3) - : : +- CometProject (2) - : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : +- CometSort (9) - : +- CometExchange (8) - : +- CometProject (7) - : +- CometFilter (6) - : +- CometNativeScan parquet spark_catalog.default.store_returns (5) - +- CometBroadcastExchange (15) - +- CometProject (14) - +- CometFilter (13) - +- CometNativeScan parquet spark_catalog.default.reason (12) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -ReadSchema: struct - -(2) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5], [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] - -(3) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#3 ASC NULLS FIRST] - -(5) CometNativeScan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_reason_sk)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Condition : ((isnotnull(sr_item_sk#7) AND isnotnull(sr_ticket_number#9)) AND isnotnull(sr_reason_sk#8)) - -(7) CometProject -Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] - -(8) CometExchange -Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: hashpartitioning(sr_item_sk#7, sr_ticket_number#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7 ASC NULLS FIRST, sr_ticket_number#9 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Right output [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [ss_item_sk#1, ss_ticket_number#3], [sr_item_sk#7, sr_ticket_number#9], Inner - -(11) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10], [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] - -(12) CometNativeScan parquet spark_catalog.default.reason -Output [2]: [r_reason_sk#12, r_reason_desc#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [r_reason_sk#12, r_reason_desc#13] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, r_reason_desc#13, 100, true, false, true) = reason 28 ) AND isnotnull(r_reason_sk#12)) - -(14) CometProject -Input [2]: [r_reason_sk#12, r_reason_desc#13] -Arguments: [r_reason_sk#12], [r_reason_sk#12] - -(15) CometBroadcastExchange -Input [1]: [r_reason_sk#12] -Arguments: [r_reason_sk#12] - -(16) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] -Right output [1]: [r_reason_sk#12] -Arguments: [sr_reason_sk#8], [r_reason_sk#12], Inner, BuildRight - -(17) CometProject -Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10, r_reason_sk#12] -Arguments: [ss_customer_sk#2, act_sales#14], [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#10) THEN (cast((ss_quantity#4 - sr_return_quantity#10) as decimal(10,0)) * ss_sales_price#5) ELSE (cast(ss_quantity#4 as decimal(10,0)) * ss_sales_price#5) END AS act_sales#14] - -(18) CometHashAggregate -Input [2]: [ss_customer_sk#2, act_sales#14] -Keys [1]: [ss_customer_sk#2] -Functions [1]: [partial_sum(act_sales#14)] - -(19) CometExchange -Input [3]: [ss_customer_sk#2, sum#15, isEmpty#16] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometHashAggregate -Input [3]: [ss_customer_sk#2, sum#15, isEmpty#16] -Keys [1]: [ss_customer_sk#2] -Functions [1]: [sum(act_sales#14)] - -(21) CometTakeOrderedAndProject -Input [2]: [ss_customer_sk#2, sumsales#17] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sumsales#17 ASC NULLS FIRST,ss_customer_sk#2 ASC NULLS FIRST], output=[ss_customer_sk#2,sumsales#17]), [ss_customer_sk#2, sumsales#17], 100, 0, [sumsales#17 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#17] - -(22) CometColumnarToRow [codegen id : 1] -Input [2]: [ss_customer_sk#2, sumsales#17] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_datafusion/simplified.txt deleted file mode 100644 index c0b74010df..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_datafusion/simplified.txt +++ /dev/null @@ -1,24 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ss_customer_sk,sumsales] - CometHashAggregate [sum,isEmpty] [ss_customer_sk,sumsales,sum(act_sales)] - CometExchange [ss_customer_sk] #1 - CometHashAggregate [act_sales] [ss_customer_sk,sum,isEmpty] - CometProject [sr_return_quantity,ss_quantity,ss_sales_price] [ss_customer_sk,act_sales] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity,r_reason_sk] - CometProject [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometExchange [ss_item_sk,ss_ticket_number] #2 - CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometSort [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometExchange [sr_item_sk,sr_ticket_number] #3 - CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometBroadcastExchange [r_reason_sk] #4 - CometProject [r_reason_sk] - CometFilter [r_reason_sk,r_reason_desc] - CometNativeScan 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/q93.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_iceberg_compat/explain.txt deleted file mode 100644 index 799d860c95..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,124 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (22) -+- CometTakeOrderedAndProject (21) - +- CometHashAggregate (20) - +- CometExchange (19) - +- CometHashAggregate (18) - +- CometProject (17) - +- CometBroadcastHashJoin (16) - :- CometProject (11) - : +- CometSortMergeJoin (10) - : :- CometSort (4) - : : +- CometExchange (3) - : : +- CometProject (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometSort (9) - : +- CometExchange (8) - : +- CometProject (7) - : +- CometFilter (6) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - +- CometBroadcastExchange (15) - +- CometProject (14) - +- CometFilter (13) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (12) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -ReadSchema: struct - -(2) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5], [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] - -(3) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#3 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_reason_sk)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Condition : ((isnotnull(sr_item_sk#7) AND isnotnull(sr_ticket_number#9)) AND isnotnull(sr_reason_sk#8)) - -(7) CometProject -Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] - -(8) CometExchange -Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: hashpartitioning(sr_item_sk#7, sr_ticket_number#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7 ASC NULLS FIRST, sr_ticket_number#9 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Right output [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [ss_item_sk#1, ss_ticket_number#3], [sr_item_sk#7, sr_ticket_number#9], Inner - -(11) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10], [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Output [2]: [r_reason_sk#12, r_reason_desc#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [r_reason_sk#12, r_reason_desc#13] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, r_reason_desc#13, 100, true, false, true) = reason 28 ) AND isnotnull(r_reason_sk#12)) - -(14) CometProject -Input [2]: [r_reason_sk#12, r_reason_desc#13] -Arguments: [r_reason_sk#12], [r_reason_sk#12] - -(15) CometBroadcastExchange -Input [1]: [r_reason_sk#12] -Arguments: [r_reason_sk#12] - -(16) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] -Right output [1]: [r_reason_sk#12] -Arguments: [sr_reason_sk#8], [r_reason_sk#12], Inner, BuildRight - -(17) CometProject -Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10, r_reason_sk#12] -Arguments: [ss_customer_sk#2, act_sales#14], [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#10) THEN (cast((ss_quantity#4 - sr_return_quantity#10) as decimal(10,0)) * ss_sales_price#5) ELSE (cast(ss_quantity#4 as decimal(10,0)) * ss_sales_price#5) END AS act_sales#14] - -(18) CometHashAggregate -Input [2]: [ss_customer_sk#2, act_sales#14] -Keys [1]: [ss_customer_sk#2] -Functions [1]: [partial_sum(act_sales#14)] - -(19) CometExchange -Input [3]: [ss_customer_sk#2, sum#15, isEmpty#16] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometHashAggregate -Input [3]: [ss_customer_sk#2, sum#15, isEmpty#16] -Keys [1]: [ss_customer_sk#2] -Functions [1]: [sum(act_sales#14)] - -(21) CometTakeOrderedAndProject -Input [2]: [ss_customer_sk#2, sumsales#17] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sumsales#17 ASC NULLS FIRST,ss_customer_sk#2 ASC NULLS FIRST], output=[ss_customer_sk#2,sumsales#17]), [ss_customer_sk#2, sumsales#17], 100, 0, [sumsales#17 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#17] - -(22) CometColumnarToRow [codegen id : 1] -Input [2]: [ss_customer_sk#2, sumsales#17] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_iceberg_compat/simplified.txt deleted file mode 100644 index d1de4f3475..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,24 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ss_customer_sk,sumsales] - CometHashAggregate [sum,isEmpty] [ss_customer_sk,sumsales,sum(act_sales)] - CometExchange [ss_customer_sk] #1 - CometHashAggregate [act_sales] [ss_customer_sk,sum,isEmpty] - CometProject [sr_return_quantity,ss_quantity,ss_sales_price] [ss_customer_sk,act_sales] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity,r_reason_sk] - CometProject [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometExchange [ss_item_sk,ss_ticket_number] #2 - CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometSort [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometExchange [sr_item_sk,sr_ticket_number] #3 - CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometBroadcastExchange [r_reason_sk] #4 - CometProject [r_reason_sk] - CometFilter [r_reason_sk,r_reason_desc] - CometScan [native_iceberg_compat] 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/q93/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/explain.txt deleted file mode 100644 index 799d860c95..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/explain.txt +++ /dev/null @@ -1,124 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (22) -+- CometTakeOrderedAndProject (21) - +- CometHashAggregate (20) - +- CometExchange (19) - +- CometHashAggregate (18) - +- CometProject (17) - +- CometBroadcastHashJoin (16) - :- CometProject (11) - : +- CometSortMergeJoin (10) - : :- CometSort (4) - : : +- CometExchange (3) - : : +- CometProject (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometSort (9) - : +- CometExchange (8) - : +- CometProject (7) - : +- CometFilter (6) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - +- CometBroadcastExchange (15) - +- CometProject (14) - +- CometFilter (13) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (12) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -ReadSchema: struct - -(2) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5], [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] - -(3) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#3 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_reason_sk)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Condition : ((isnotnull(sr_item_sk#7) AND isnotnull(sr_ticket_number#9)) AND isnotnull(sr_reason_sk#8)) - -(7) CometProject -Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] - -(8) CometExchange -Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: hashpartitioning(sr_item_sk#7, sr_ticket_number#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7 ASC NULLS FIRST, sr_ticket_number#9 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Right output [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [ss_item_sk#1, ss_ticket_number#3], [sr_item_sk#7, sr_ticket_number#9], Inner - -(11) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10], [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Output [2]: [r_reason_sk#12, r_reason_desc#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [r_reason_sk#12, r_reason_desc#13] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, r_reason_desc#13, 100, true, false, true) = reason 28 ) AND isnotnull(r_reason_sk#12)) - -(14) CometProject -Input [2]: [r_reason_sk#12, r_reason_desc#13] -Arguments: [r_reason_sk#12], [r_reason_sk#12] - -(15) CometBroadcastExchange -Input [1]: [r_reason_sk#12] -Arguments: [r_reason_sk#12] - -(16) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] -Right output [1]: [r_reason_sk#12] -Arguments: [sr_reason_sk#8], [r_reason_sk#12], Inner, BuildRight - -(17) CometProject -Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10, r_reason_sk#12] -Arguments: [ss_customer_sk#2, act_sales#14], [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#10) THEN (cast((ss_quantity#4 - sr_return_quantity#10) as decimal(10,0)) * ss_sales_price#5) ELSE (cast(ss_quantity#4 as decimal(10,0)) * ss_sales_price#5) END AS act_sales#14] - -(18) CometHashAggregate -Input [2]: [ss_customer_sk#2, act_sales#14] -Keys [1]: [ss_customer_sk#2] -Functions [1]: [partial_sum(act_sales#14)] - -(19) CometExchange -Input [3]: [ss_customer_sk#2, sum#15, isEmpty#16] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometHashAggregate -Input [3]: [ss_customer_sk#2, sum#15, isEmpty#16] -Keys [1]: [ss_customer_sk#2] -Functions [1]: [sum(act_sales#14)] - -(21) CometTakeOrderedAndProject -Input [2]: [ss_customer_sk#2, sumsales#17] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sumsales#17 ASC NULLS FIRST,ss_customer_sk#2 ASC NULLS FIRST], output=[ss_customer_sk#2,sumsales#17]), [ss_customer_sk#2, sumsales#17], 100, 0, [sumsales#17 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#17] - -(22) CometColumnarToRow [codegen id : 1] -Input [2]: [ss_customer_sk#2, sumsales#17] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/extended.txt deleted file mode 100644 index 335f2765d7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/extended.txt +++ /dev/null @@ -1,24 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason - -Comet accelerated 21 out of 21 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d1de4f3475..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q93/simplified.txt +++ /dev/null @@ -1,24 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ss_customer_sk,sumsales] - CometHashAggregate [sum,isEmpty] [ss_customer_sk,sumsales,sum(act_sales)] - CometExchange [ss_customer_sk] #1 - CometHashAggregate [act_sales] [ss_customer_sk,sum,isEmpty] - CometProject [sr_return_quantity,ss_quantity,ss_sales_price] [ss_customer_sk,act_sales] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity,r_reason_sk] - CometProject [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometExchange [ss_item_sk,ss_ticket_number] #2 - CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometSort [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometExchange [sr_item_sk,sr_ticket_number] #3 - CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometBroadcastExchange [r_reason_sk] #4 - CometProject [r_reason_sk] - CometFilter [r_reason_sk,r_reason_desc] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_datafusion/explain.txt deleted file mode 100644 index 1caeb8ca76..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_datafusion/explain.txt +++ /dev/null @@ -1,235 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometColumnarExchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * CometColumnarToRow (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometNativeScan parquet spark_catalog.default.web_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometNativeScan parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometNativeScan parquet spark_catalog.default.web_site (29) - - -(1) CometNativeScan parquet spark_catalog.default.web_sales -Output [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) - -(3) CometProject -Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(4) CometExchange -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.web_sales -Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -ReadSchema: struct - -(7) CometProject -Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] -Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_warehouse_sk#9, ws_order_number#10] - -(8) CometExchange -Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_order_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_order_number#5], [ws_order_number#10], LeftSemi, NOT (ws_warehouse_sk#4 = ws_warehouse_sk#9) - -(11) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(12) CometNativeScan parquet spark_catalog.default.web_returns -Output [2]: [wr_order_number#12, wr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -ReadSchema: struct - -(13) CometProject -Input [2]: [wr_order_number#12, wr_returned_date_sk#13] -Arguments: [wr_order_number#12], [wr_order_number#12] - -(14) CometExchange -Input [1]: [wr_order_number#12] -Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(15) CometSort -Input [1]: [wr_order_number#12] -Arguments: [wr_order_number#12], [wr_order_number#12 ASC NULLS FIRST] - -(16) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [wr_order_number#12] -Arguments: [ws_order_number#5], [wr_order_number#12], LeftAnti - -(17) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-01)) AND (d_date#15 <= 1999-04-02)) AND isnotnull(d_date_sk#14)) - -(19) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(20) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(21) CometBroadcastHashJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [d_date_sk#14] -Arguments: [ws_ship_date_sk#1], [d_date_sk#14], Inner, BuildRight - -(22) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#14] -Arguments: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(23) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#17, 2, true, false, true) = IL) AND isnotnull(ca_address_sk#16)) - -(25) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(26) CometBroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: [ca_address_sk#16] - -(27) CometBroadcastHashJoin -Left output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [ca_address_sk#16] -Arguments: [ws_ship_addr_sk#2], [ca_address_sk#16], Inner, BuildRight - -(28) CometProject -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#16] -Arguments: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(29) CometNativeScan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#18, web_company_name#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [web_site_sk#18, web_company_name#19] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_company_name#19, 50, true, false, true) = pri ) AND isnotnull(web_site_sk#18)) - -(31) CometProject -Input [2]: [web_site_sk#18, web_company_name#19] -Arguments: [web_site_sk#18], [web_site_sk#18] - -(32) CometBroadcastExchange -Input [1]: [web_site_sk#18] -Arguments: [web_site_sk#18] - -(33) CometBroadcastHashJoin -Left output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [web_site_sk#18] -Arguments: [ws_web_site_sk#3], [web_site_sk#18], Inner, BuildRight - -(34) CometProject -Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#18] -Arguments: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(35) CometHashAggregate -Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Keys [1]: [ws_order_number#5] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] - -(36) CometColumnarToRow [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] - -(37) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] -Keys [1]: [ws_order_number#5] -Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23] -Results [3]: [ws_order_number#5, sum#20, sum#21] - -(38) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23, count(ws_order_number#5)#24] -Results [3]: [sum#20, sum#21, count#25] - -(39) CometColumnarExchange -Input [3]: [sum#20, sum#21, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(40) CometHashAggregate -Input [3]: [sum#20, sum#21, count#25] -Keys: [] -Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] - -(41) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #26, total shipping cost #27, total net profit #28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_datafusion/simplified.txt deleted file mode 100644 index feab73bcd6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_datafusion/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometSortMergeJoin [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_order_number,ws_warehouse_sk] - CometSort [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] - CometExchange [ws_order_number] #2 - 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,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometNativeScan 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] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometNativeScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometNativeScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometNativeScan 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/q94.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_iceberg_compat/explain.txt deleted file mode 100644 index 8164e345a6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,235 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometColumnarExchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * CometColumnarToRow (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (29) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) - -(3) CometProject -Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(4) CometExchange -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -ReadSchema: struct - -(7) CometProject -Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] -Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_warehouse_sk#9, ws_order_number#10] - -(8) CometExchange -Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_order_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_order_number#5], [ws_order_number#10], LeftSemi, NOT (ws_warehouse_sk#4 = ws_warehouse_sk#9) - -(11) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [2]: [wr_order_number#12, wr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -ReadSchema: struct - -(13) CometProject -Input [2]: [wr_order_number#12, wr_returned_date_sk#13] -Arguments: [wr_order_number#12], [wr_order_number#12] - -(14) CometExchange -Input [1]: [wr_order_number#12] -Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(15) CometSort -Input [1]: [wr_order_number#12] -Arguments: [wr_order_number#12], [wr_order_number#12 ASC NULLS FIRST] - -(16) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [wr_order_number#12] -Arguments: [ws_order_number#5], [wr_order_number#12], LeftAnti - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-01)) AND (d_date#15 <= 1999-04-02)) AND isnotnull(d_date_sk#14)) - -(19) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(20) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(21) CometBroadcastHashJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [d_date_sk#14] -Arguments: [ws_ship_date_sk#1], [d_date_sk#14], Inner, BuildRight - -(22) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#14] -Arguments: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#17, 2, true, false, true) = IL) AND isnotnull(ca_address_sk#16)) - -(25) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(26) CometBroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: [ca_address_sk#16] - -(27) CometBroadcastHashJoin -Left output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [ca_address_sk#16] -Arguments: [ws_ship_addr_sk#2], [ca_address_sk#16], Inner, BuildRight - -(28) CometProject -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#16] -Arguments: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#18, web_company_name#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [web_site_sk#18, web_company_name#19] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_company_name#19, 50, true, false, true) = pri ) AND isnotnull(web_site_sk#18)) - -(31) CometProject -Input [2]: [web_site_sk#18, web_company_name#19] -Arguments: [web_site_sk#18], [web_site_sk#18] - -(32) CometBroadcastExchange -Input [1]: [web_site_sk#18] -Arguments: [web_site_sk#18] - -(33) CometBroadcastHashJoin -Left output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [web_site_sk#18] -Arguments: [ws_web_site_sk#3], [web_site_sk#18], Inner, BuildRight - -(34) CometProject -Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#18] -Arguments: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(35) CometHashAggregate -Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Keys [1]: [ws_order_number#5] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] - -(36) CometColumnarToRow [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] - -(37) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] -Keys [1]: [ws_order_number#5] -Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23] -Results [3]: [ws_order_number#5, sum#20, sum#21] - -(38) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23, count(ws_order_number#5)#24] -Results [3]: [sum#20, sum#21, count#25] - -(39) CometColumnarExchange -Input [3]: [sum#20, sum#21, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(40) CometHashAggregate -Input [3]: [sum#20, sum#21, count#25] -Keys: [] -Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] - -(41) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #26, total shipping cost #27, total net profit #28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_iceberg_compat/simplified.txt deleted file mode 100644 index 629178d106..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometSortMergeJoin [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_order_number,ws_warehouse_sk] - CometSort [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] - CometExchange [ws_order_number] #2 - 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,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometScan [native_iceberg_compat] 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/q94/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/explain.txt deleted file mode 100644 index 8164e345a6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/explain.txt +++ /dev/null @@ -1,235 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometColumnarExchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * CometColumnarToRow (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (29) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) - -(3) CometProject -Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(4) CometExchange -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -ReadSchema: struct - -(7) CometProject -Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] -Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_warehouse_sk#9, ws_order_number#10] - -(8) CometExchange -Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_order_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_order_number#5], [ws_order_number#10], LeftSemi, NOT (ws_warehouse_sk#4 = ws_warehouse_sk#9) - -(11) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [2]: [wr_order_number#12, wr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -ReadSchema: struct - -(13) CometProject -Input [2]: [wr_order_number#12, wr_returned_date_sk#13] -Arguments: [wr_order_number#12], [wr_order_number#12] - -(14) CometExchange -Input [1]: [wr_order_number#12] -Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(15) CometSort -Input [1]: [wr_order_number#12] -Arguments: [wr_order_number#12], [wr_order_number#12 ASC NULLS FIRST] - -(16) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [wr_order_number#12] -Arguments: [ws_order_number#5], [wr_order_number#12], LeftAnti - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-01)) AND (d_date#15 <= 1999-04-02)) AND isnotnull(d_date_sk#14)) - -(19) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(20) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(21) CometBroadcastHashJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [d_date_sk#14] -Arguments: [ws_ship_date_sk#1], [d_date_sk#14], Inner, BuildRight - -(22) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#14] -Arguments: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#17, 2, true, false, true) = IL) AND isnotnull(ca_address_sk#16)) - -(25) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(26) CometBroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: [ca_address_sk#16] - -(27) CometBroadcastHashJoin -Left output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [ca_address_sk#16] -Arguments: [ws_ship_addr_sk#2], [ca_address_sk#16], Inner, BuildRight - -(28) CometProject -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#16] -Arguments: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#18, web_company_name#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [web_site_sk#18, web_company_name#19] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_company_name#19, 50, true, false, true) = pri ) AND isnotnull(web_site_sk#18)) - -(31) CometProject -Input [2]: [web_site_sk#18, web_company_name#19] -Arguments: [web_site_sk#18], [web_site_sk#18] - -(32) CometBroadcastExchange -Input [1]: [web_site_sk#18] -Arguments: [web_site_sk#18] - -(33) CometBroadcastHashJoin -Left output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [web_site_sk#18] -Arguments: [ws_web_site_sk#3], [web_site_sk#18], Inner, BuildRight - -(34) CometProject -Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#18] -Arguments: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(35) CometHashAggregate -Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Keys [1]: [ws_order_number#5] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] - -(36) CometColumnarToRow [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] - -(37) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] -Keys [1]: [ws_order_number#5] -Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23] -Results [3]: [ws_order_number#5, sum#20, sum#21] - -(38) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23, count(ws_order_number#5)#24] -Results [3]: [sum#20, sum#21, count#25] - -(39) CometColumnarExchange -Input [3]: [sum#20, sum#21, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(40) CometHashAggregate -Input [3]: [sum#20, sum#21, count#25] -Keys: [] -Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] - -(41) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #26, total shipping cost #27, total net profit #28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/extended.txt deleted file mode 100644 index eac4939621..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/extended.txt +++ /dev/null @@ -1,43 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - -Comet accelerated 37 out of 39 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 629178d106..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q94/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometSortMergeJoin [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_order_number,ws_warehouse_sk] - CometSort [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] - CometExchange [ws_order_number] #2 - 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,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_datafusion/explain.txt deleted file mode 100644 index 39f45f7a74..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_datafusion/explain.txt +++ /dev/null @@ -1,302 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (54) -+- CometHashAggregate (53) - +- CometColumnarExchange (52) - +- * HashAggregate (51) - +- * HashAggregate (50) - +- * CometColumnarToRow (49) - +- CometHashAggregate (48) - +- CometProject (47) - +- CometBroadcastHashJoin (46) - :- CometProject (41) - : +- CometBroadcastHashJoin (40) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometSortMergeJoin (29) - : : : :- CometSortMergeJoin (15) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales (1) - : : : : +- CometProject (14) - : : : : +- CometSortMergeJoin (13) - : : : : :- CometSort (10) - : : : : : +- CometExchange (9) - : : : : : +- CometProject (8) - : : : : : +- CometFilter (7) - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales (6) - : : : : +- CometSort (12) - : : : : +- ReusedExchange (11) - : : : +- CometProject (28) - : : : +- CometSortMergeJoin (27) - : : : :- CometSort (20) - : : : : +- CometExchange (19) - : : : : +- CometProject (18) - : : : : +- CometFilter (17) - : : : : +- CometNativeScan parquet spark_catalog.default.web_returns (16) - : : : +- CometProject (26) - : : : +- CometSortMergeJoin (25) - : : : :- CometSort (22) - : : : : +- ReusedExchange (21) - : : : +- CometSort (24) - : : : +- ReusedExchange (23) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (30) - : +- CometBroadcastExchange (39) - : +- CometProject (38) - : +- CometFilter (37) - : +- CometNativeScan parquet spark_catalog.default.customer_address (36) - +- CometBroadcastExchange (45) - +- CometProject (44) - +- CometFilter (43) - +- CometNativeScan parquet spark_catalog.default.web_site (42) - - -(1) CometNativeScan parquet spark_catalog.default.web_sales -Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) - -(3) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(4) CometExchange -Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.web_sales -Output [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Condition : (isnotnull(ws_order_number#9) AND isnotnull(ws_warehouse_sk#8)) - -(8) CometProject -Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_warehouse_sk#8, ws_order_number#9] - -(9) CometExchange -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: hashpartitioning(ws_order_number#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_order_number#9 ASC NULLS FIRST] - -(11) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#11, ws_order_number#12] - -(12) CometSort -Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_warehouse_sk#11, ws_order_number#12], [ws_order_number#12 ASC NULLS FIRST] - -(13) CometSortMergeJoin -Left output [2]: [ws_warehouse_sk#8, ws_order_number#9] -Right output [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#12], Inner, NOT (ws_warehouse_sk#8 = ws_warehouse_sk#11) - -(14) CometProject -Input [4]: [ws_warehouse_sk#8, ws_order_number#9, ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#9] - -(15) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [ws_order_number#9] -Arguments: [ws_order_number#4], [ws_order_number#9], LeftSemi - -(16) CometNativeScan parquet spark_catalog.default.web_returns -Output [2]: [wr_order_number#13, wr_returned_date_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [wr_order_number#13, wr_returned_date_sk#14] -Condition : isnotnull(wr_order_number#13) - -(18) CometProject -Input [2]: [wr_order_number#13, wr_returned_date_sk#14] -Arguments: [wr_order_number#13], [wr_order_number#13] - -(19) CometExchange -Input [1]: [wr_order_number#13] -Arguments: hashpartitioning(wr_order_number#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [1]: [wr_order_number#13] -Arguments: [wr_order_number#13], [wr_order_number#13 ASC NULLS FIRST] - -(21) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#15, ws_order_number#16] - -(22) CometSort -Input [2]: [ws_warehouse_sk#15, ws_order_number#16] -Arguments: [ws_warehouse_sk#15, ws_order_number#16], [ws_order_number#16 ASC NULLS FIRST] - -(23) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#17, ws_order_number#18] - -(24) CometSort -Input [2]: [ws_warehouse_sk#17, ws_order_number#18] -Arguments: [ws_warehouse_sk#17, ws_order_number#18], [ws_order_number#18 ASC NULLS FIRST] - -(25) CometSortMergeJoin -Left output [2]: [ws_warehouse_sk#15, ws_order_number#16] -Right output [2]: [ws_warehouse_sk#17, ws_order_number#18] -Arguments: [ws_order_number#16], [ws_order_number#18], Inner, NOT (ws_warehouse_sk#15 = ws_warehouse_sk#17) - -(26) CometProject -Input [4]: [ws_warehouse_sk#15, ws_order_number#16, ws_warehouse_sk#17, ws_order_number#18] -Arguments: [ws_order_number#16], [ws_order_number#16] - -(27) CometSortMergeJoin -Left output [1]: [wr_order_number#13] -Right output [1]: [ws_order_number#16] -Arguments: [wr_order_number#13], [ws_order_number#16], Inner - -(28) CometProject -Input [2]: [wr_order_number#13, ws_order_number#16] -Arguments: [wr_order_number#13], [wr_order_number#13] - -(29) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [wr_order_number#13] -Arguments: [ws_order_number#4], [wr_order_number#13], LeftSemi - -(30) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_date#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [d_date_sk#19, d_date#20] -Condition : (((isnotnull(d_date#20) AND (d_date#20 >= 1999-02-01)) AND (d_date#20 <= 1999-04-02)) AND isnotnull(d_date_sk#19)) - -(32) CometProject -Input [2]: [d_date_sk#19, d_date#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(33) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(34) CometBroadcastHashJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [d_date_sk#19] -Arguments: [ws_ship_date_sk#1], [d_date_sk#19], Inner, BuildRight - -(35) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#19] -Arguments: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(36) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#21, ca_state#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [ca_address_sk#21, ca_state#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#22, 2, true, false, true) = IL) AND isnotnull(ca_address_sk#21)) - -(38) CometProject -Input [2]: [ca_address_sk#21, ca_state#22] -Arguments: [ca_address_sk#21], [ca_address_sk#21] - -(39) CometBroadcastExchange -Input [1]: [ca_address_sk#21] -Arguments: [ca_address_sk#21] - -(40) CometBroadcastHashJoin -Left output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [ca_address_sk#21] -Arguments: [ws_ship_addr_sk#2], [ca_address_sk#21], Inner, BuildRight - -(41) CometProject -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#21] -Arguments: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(42) CometNativeScan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#23, web_company_name#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(43) CometFilter -Input [2]: [web_site_sk#23, web_company_name#24] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_company_name#24, 50, true, false, true) = pri ) AND isnotnull(web_site_sk#23)) - -(44) CometProject -Input [2]: [web_site_sk#23, web_company_name#24] -Arguments: [web_site_sk#23], [web_site_sk#23] - -(45) CometBroadcastExchange -Input [1]: [web_site_sk#23] -Arguments: [web_site_sk#23] - -(46) CometBroadcastHashJoin -Left output [4]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [web_site_sk#23] -Arguments: [ws_web_site_sk#3], [web_site_sk#23], Inner, BuildRight - -(47) CometProject -Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#23] -Arguments: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(48) CometHashAggregate -Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Keys [1]: [ws_order_number#4] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] - -(49) CometColumnarToRow [codegen id : 1] -Input [3]: [ws_order_number#4, sum#25, sum#26] - -(50) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#4, sum#25, sum#26] -Keys [1]: [ws_order_number#4] -Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28] -Results [3]: [ws_order_number#4, sum#25, sum#26] - -(51) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#4, sum#25, sum#26] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28, count(ws_order_number#4)#29] -Results [3]: [sum#25, sum#26, count#30] - -(52) CometColumnarExchange -Input [3]: [sum#25, sum#26, count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(53) CometHashAggregate -Input [3]: [sum#25, sum#26, count#30] -Keys: [] -Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] - -(54) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #31, total shipping cost #32, total net profit #33] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_datafusion/simplified.txt deleted file mode 100644 index 15f29c507c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_datafusion/simplified.txt +++ /dev/null @@ -1,58 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number] - CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometExchange [ws_order_number] #2 - 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,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometNativeScan 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] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometProject [wr_order_number] - CometSortMergeJoin [wr_order_number,ws_order_number] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometFilter [wr_order_number,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometNativeScan 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.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_iceberg_compat/explain.txt deleted file mode 100644 index 24e37a8483..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,302 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (54) -+- CometHashAggregate (53) - +- CometColumnarExchange (52) - +- * HashAggregate (51) - +- * HashAggregate (50) - +- * CometColumnarToRow (49) - +- CometHashAggregate (48) - +- CometProject (47) - +- CometBroadcastHashJoin (46) - :- CometProject (41) - : +- CometBroadcastHashJoin (40) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometSortMergeJoin (29) - : : : :- CometSortMergeJoin (15) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometProject (14) - : : : : +- CometSortMergeJoin (13) - : : : : :- CometSort (10) - : : : : : +- CometExchange (9) - : : : : : +- CometProject (8) - : : : : : +- CometFilter (7) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (6) - : : : : +- CometSort (12) - : : : : +- ReusedExchange (11) - : : : +- CometProject (28) - : : : +- CometSortMergeJoin (27) - : : : :- CometSort (20) - : : : : +- CometExchange (19) - : : : : +- CometProject (18) - : : : : +- CometFilter (17) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (16) - : : : +- CometProject (26) - : : : +- CometSortMergeJoin (25) - : : : :- CometSort (22) - : : : : +- ReusedExchange (21) - : : : +- CometSort (24) - : : : +- ReusedExchange (23) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - : +- CometBroadcastExchange (39) - : +- CometProject (38) - : +- CometFilter (37) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (36) - +- CometBroadcastExchange (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (42) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) - -(3) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(4) CometExchange -Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Condition : (isnotnull(ws_order_number#9) AND isnotnull(ws_warehouse_sk#8)) - -(8) CometProject -Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_warehouse_sk#8, ws_order_number#9] - -(9) CometExchange -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: hashpartitioning(ws_order_number#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_order_number#9 ASC NULLS FIRST] - -(11) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#11, ws_order_number#12] - -(12) CometSort -Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_warehouse_sk#11, ws_order_number#12], [ws_order_number#12 ASC NULLS FIRST] - -(13) CometSortMergeJoin -Left output [2]: [ws_warehouse_sk#8, ws_order_number#9] -Right output [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#12], Inner, NOT (ws_warehouse_sk#8 = ws_warehouse_sk#11) - -(14) CometProject -Input [4]: [ws_warehouse_sk#8, ws_order_number#9, ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#9] - -(15) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [ws_order_number#9] -Arguments: [ws_order_number#4], [ws_order_number#9], LeftSemi - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [2]: [wr_order_number#13, wr_returned_date_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [wr_order_number#13, wr_returned_date_sk#14] -Condition : isnotnull(wr_order_number#13) - -(18) CometProject -Input [2]: [wr_order_number#13, wr_returned_date_sk#14] -Arguments: [wr_order_number#13], [wr_order_number#13] - -(19) CometExchange -Input [1]: [wr_order_number#13] -Arguments: hashpartitioning(wr_order_number#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [1]: [wr_order_number#13] -Arguments: [wr_order_number#13], [wr_order_number#13 ASC NULLS FIRST] - -(21) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#15, ws_order_number#16] - -(22) CometSort -Input [2]: [ws_warehouse_sk#15, ws_order_number#16] -Arguments: [ws_warehouse_sk#15, ws_order_number#16], [ws_order_number#16 ASC NULLS FIRST] - -(23) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#17, ws_order_number#18] - -(24) CometSort -Input [2]: [ws_warehouse_sk#17, ws_order_number#18] -Arguments: [ws_warehouse_sk#17, ws_order_number#18], [ws_order_number#18 ASC NULLS FIRST] - -(25) CometSortMergeJoin -Left output [2]: [ws_warehouse_sk#15, ws_order_number#16] -Right output [2]: [ws_warehouse_sk#17, ws_order_number#18] -Arguments: [ws_order_number#16], [ws_order_number#18], Inner, NOT (ws_warehouse_sk#15 = ws_warehouse_sk#17) - -(26) CometProject -Input [4]: [ws_warehouse_sk#15, ws_order_number#16, ws_warehouse_sk#17, ws_order_number#18] -Arguments: [ws_order_number#16], [ws_order_number#16] - -(27) CometSortMergeJoin -Left output [1]: [wr_order_number#13] -Right output [1]: [ws_order_number#16] -Arguments: [wr_order_number#13], [ws_order_number#16], Inner - -(28) CometProject -Input [2]: [wr_order_number#13, ws_order_number#16] -Arguments: [wr_order_number#13], [wr_order_number#13] - -(29) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [wr_order_number#13] -Arguments: [ws_order_number#4], [wr_order_number#13], LeftSemi - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_date#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [d_date_sk#19, d_date#20] -Condition : (((isnotnull(d_date#20) AND (d_date#20 >= 1999-02-01)) AND (d_date#20 <= 1999-04-02)) AND isnotnull(d_date_sk#19)) - -(32) CometProject -Input [2]: [d_date_sk#19, d_date#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(33) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(34) CometBroadcastHashJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [d_date_sk#19] -Arguments: [ws_ship_date_sk#1], [d_date_sk#19], Inner, BuildRight - -(35) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#19] -Arguments: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#21, ca_state#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [ca_address_sk#21, ca_state#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#22, 2, true, false, true) = IL) AND isnotnull(ca_address_sk#21)) - -(38) CometProject -Input [2]: [ca_address_sk#21, ca_state#22] -Arguments: [ca_address_sk#21], [ca_address_sk#21] - -(39) CometBroadcastExchange -Input [1]: [ca_address_sk#21] -Arguments: [ca_address_sk#21] - -(40) CometBroadcastHashJoin -Left output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [ca_address_sk#21] -Arguments: [ws_ship_addr_sk#2], [ca_address_sk#21], Inner, BuildRight - -(41) CometProject -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#21] -Arguments: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#23, web_company_name#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(43) CometFilter -Input [2]: [web_site_sk#23, web_company_name#24] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_company_name#24, 50, true, false, true) = pri ) AND isnotnull(web_site_sk#23)) - -(44) CometProject -Input [2]: [web_site_sk#23, web_company_name#24] -Arguments: [web_site_sk#23], [web_site_sk#23] - -(45) CometBroadcastExchange -Input [1]: [web_site_sk#23] -Arguments: [web_site_sk#23] - -(46) CometBroadcastHashJoin -Left output [4]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [web_site_sk#23] -Arguments: [ws_web_site_sk#3], [web_site_sk#23], Inner, BuildRight - -(47) CometProject -Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#23] -Arguments: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(48) CometHashAggregate -Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Keys [1]: [ws_order_number#4] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] - -(49) CometColumnarToRow [codegen id : 1] -Input [3]: [ws_order_number#4, sum#25, sum#26] - -(50) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#4, sum#25, sum#26] -Keys [1]: [ws_order_number#4] -Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28] -Results [3]: [ws_order_number#4, sum#25, sum#26] - -(51) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#4, sum#25, sum#26] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28, count(ws_order_number#4)#29] -Results [3]: [sum#25, sum#26, count#30] - -(52) CometColumnarExchange -Input [3]: [sum#25, sum#26, count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(53) CometHashAggregate -Input [3]: [sum#25, sum#26, count#30] -Keys: [] -Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] - -(54) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #31, total shipping cost #32, total net profit #33] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_iceberg_compat/simplified.txt deleted file mode 100644 index 6c1b1bccfa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,58 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number] - CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometExchange [ws_order_number] #2 - 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,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometProject [wr_order_number] - CometSortMergeJoin [wr_order_number,ws_order_number] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometFilter [wr_order_number,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometScan [native_iceberg_compat] 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/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/explain.txt deleted file mode 100644 index 24e37a8483..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/explain.txt +++ /dev/null @@ -1,302 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (54) -+- CometHashAggregate (53) - +- CometColumnarExchange (52) - +- * HashAggregate (51) - +- * HashAggregate (50) - +- * CometColumnarToRow (49) - +- CometHashAggregate (48) - +- CometProject (47) - +- CometBroadcastHashJoin (46) - :- CometProject (41) - : +- CometBroadcastHashJoin (40) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometSortMergeJoin (29) - : : : :- CometSortMergeJoin (15) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometProject (14) - : : : : +- CometSortMergeJoin (13) - : : : : :- CometSort (10) - : : : : : +- CometExchange (9) - : : : : : +- CometProject (8) - : : : : : +- CometFilter (7) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (6) - : : : : +- CometSort (12) - : : : : +- ReusedExchange (11) - : : : +- CometProject (28) - : : : +- CometSortMergeJoin (27) - : : : :- CometSort (20) - : : : : +- CometExchange (19) - : : : : +- CometProject (18) - : : : : +- CometFilter (17) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (16) - : : : +- CometProject (26) - : : : +- CometSortMergeJoin (25) - : : : :- CometSort (22) - : : : : +- ReusedExchange (21) - : : : +- CometSort (24) - : : : +- ReusedExchange (23) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - : +- CometBroadcastExchange (39) - : +- CometProject (38) - : +- CometFilter (37) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (36) - +- CometBroadcastExchange (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (42) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) - -(3) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(4) CometExchange -Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Condition : (isnotnull(ws_order_number#9) AND isnotnull(ws_warehouse_sk#8)) - -(8) CometProject -Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_warehouse_sk#8, ws_order_number#9] - -(9) CometExchange -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: hashpartitioning(ws_order_number#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_order_number#9 ASC NULLS FIRST] - -(11) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#11, ws_order_number#12] - -(12) CometSort -Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_warehouse_sk#11, ws_order_number#12], [ws_order_number#12 ASC NULLS FIRST] - -(13) CometSortMergeJoin -Left output [2]: [ws_warehouse_sk#8, ws_order_number#9] -Right output [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#12], Inner, NOT (ws_warehouse_sk#8 = ws_warehouse_sk#11) - -(14) CometProject -Input [4]: [ws_warehouse_sk#8, ws_order_number#9, ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#9] - -(15) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [ws_order_number#9] -Arguments: [ws_order_number#4], [ws_order_number#9], LeftSemi - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [2]: [wr_order_number#13, wr_returned_date_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [wr_order_number#13, wr_returned_date_sk#14] -Condition : isnotnull(wr_order_number#13) - -(18) CometProject -Input [2]: [wr_order_number#13, wr_returned_date_sk#14] -Arguments: [wr_order_number#13], [wr_order_number#13] - -(19) CometExchange -Input [1]: [wr_order_number#13] -Arguments: hashpartitioning(wr_order_number#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [1]: [wr_order_number#13] -Arguments: [wr_order_number#13], [wr_order_number#13 ASC NULLS FIRST] - -(21) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#15, ws_order_number#16] - -(22) CometSort -Input [2]: [ws_warehouse_sk#15, ws_order_number#16] -Arguments: [ws_warehouse_sk#15, ws_order_number#16], [ws_order_number#16 ASC NULLS FIRST] - -(23) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#17, ws_order_number#18] - -(24) CometSort -Input [2]: [ws_warehouse_sk#17, ws_order_number#18] -Arguments: [ws_warehouse_sk#17, ws_order_number#18], [ws_order_number#18 ASC NULLS FIRST] - -(25) CometSortMergeJoin -Left output [2]: [ws_warehouse_sk#15, ws_order_number#16] -Right output [2]: [ws_warehouse_sk#17, ws_order_number#18] -Arguments: [ws_order_number#16], [ws_order_number#18], Inner, NOT (ws_warehouse_sk#15 = ws_warehouse_sk#17) - -(26) CometProject -Input [4]: [ws_warehouse_sk#15, ws_order_number#16, ws_warehouse_sk#17, ws_order_number#18] -Arguments: [ws_order_number#16], [ws_order_number#16] - -(27) CometSortMergeJoin -Left output [1]: [wr_order_number#13] -Right output [1]: [ws_order_number#16] -Arguments: [wr_order_number#13], [ws_order_number#16], Inner - -(28) CometProject -Input [2]: [wr_order_number#13, ws_order_number#16] -Arguments: [wr_order_number#13], [wr_order_number#13] - -(29) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [wr_order_number#13] -Arguments: [ws_order_number#4], [wr_order_number#13], LeftSemi - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_date#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [d_date_sk#19, d_date#20] -Condition : (((isnotnull(d_date#20) AND (d_date#20 >= 1999-02-01)) AND (d_date#20 <= 1999-04-02)) AND isnotnull(d_date_sk#19)) - -(32) CometProject -Input [2]: [d_date_sk#19, d_date#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(33) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(34) CometBroadcastHashJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [d_date_sk#19] -Arguments: [ws_ship_date_sk#1], [d_date_sk#19], Inner, BuildRight - -(35) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#19] -Arguments: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#21, ca_state#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [ca_address_sk#21, ca_state#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#22, 2, true, false, true) = IL) AND isnotnull(ca_address_sk#21)) - -(38) CometProject -Input [2]: [ca_address_sk#21, ca_state#22] -Arguments: [ca_address_sk#21], [ca_address_sk#21] - -(39) CometBroadcastExchange -Input [1]: [ca_address_sk#21] -Arguments: [ca_address_sk#21] - -(40) CometBroadcastHashJoin -Left output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [ca_address_sk#21] -Arguments: [ws_ship_addr_sk#2], [ca_address_sk#21], Inner, BuildRight - -(41) CometProject -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#21] -Arguments: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#23, web_company_name#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(43) CometFilter -Input [2]: [web_site_sk#23, web_company_name#24] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_company_name#24, 50, true, false, true) = pri ) AND isnotnull(web_site_sk#23)) - -(44) CometProject -Input [2]: [web_site_sk#23, web_company_name#24] -Arguments: [web_site_sk#23], [web_site_sk#23] - -(45) CometBroadcastExchange -Input [1]: [web_site_sk#23] -Arguments: [web_site_sk#23] - -(46) CometBroadcastHashJoin -Left output [4]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [web_site_sk#23] -Arguments: [ws_web_site_sk#3], [web_site_sk#23], Inner, BuildRight - -(47) CometProject -Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#23] -Arguments: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(48) CometHashAggregate -Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Keys [1]: [ws_order_number#4] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] - -(49) CometColumnarToRow [codegen id : 1] -Input [3]: [ws_order_number#4, sum#25, sum#26] - -(50) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#4, sum#25, sum#26] -Keys [1]: [ws_order_number#4] -Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28] -Results [3]: [ws_order_number#4, sum#25, sum#26] - -(51) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#4, sum#25, sum#26] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28, count(ws_order_number#4)#29] -Results [3]: [sum#25, sum#26, count#30] - -(52) CometColumnarExchange -Input [3]: [sum#25, sum#26, count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(53) CometHashAggregate -Input [3]: [sum#25, sum#26, count#30] -Keys: [] -Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] - -(54) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #31, total shipping cost #32, total net profit #33] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/extended.txt deleted file mode 100644 index 6ff8eba58f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometProject - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- CometProject - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - -Comet accelerated 59 out of 61 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 6c1b1bccfa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q95/simplified.txt +++ /dev/null @@ -1,58 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number] - CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometExchange [ws_order_number] #2 - 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,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometProject [wr_order_number] - CometSortMergeJoin [wr_order_number,ws_order_number] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometFilter [wr_order_number,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_datafusion/explain.txt deleted file mode 100644 index db701aae34..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_datafusion/explain.txt +++ /dev/null @@ -1,143 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometNativeScan parquet spark_catalog.default.household_demographics (4) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometNativeScan parquet spark_catalog.default.time_dim (10) - +- CometBroadcastExchange (19) - +- CometProject (18) - +- CometFilter (17) - +- CometNativeScan parquet spark_catalog.default.store (16) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(3) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(4) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#5, hd_dep_count#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,7), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 7)) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] -Arguments: [ss_sold_time_sk#1, ss_store_sk#3], [ss_sold_time_sk#1, ss_store_sk#3] - -(10) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,20), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Condition : ((((isnotnull(t_hour#8) AND isnotnull(t_minute#9)) AND (t_hour#8 = 20)) AND (t_minute#9 >= 30)) AND isnotnull(t_time_sk#7)) - -(12) CometProject -Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Arguments: [t_time_sk#7], [t_time_sk#7] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#7] -Arguments: [t_time_sk#7] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Right output [1]: [t_time_sk#7] -Arguments: [ss_sold_time_sk#1], [t_time_sk#7], Inner, BuildRight - -(15) CometProject -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#7] -Arguments: [ss_store_sk#3], [ss_store_sk#3] - -(16) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_store_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [s_store_sk#10, s_store_name#11] -Condition : ((isnotnull(s_store_name#11) AND (s_store_name#11 = ese)) AND isnotnull(s_store_sk#10)) - -(18) CometProject -Input [2]: [s_store_sk#10, s_store_name#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(19) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(20) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#3] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(21) CometProject -Input [2]: [ss_store_sk#3, s_store_sk#10] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#12] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 1] -Input [1]: [count(1)#13] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_datafusion/simplified.txt deleted file mode 100644 index b40f41659f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_datafusion/simplified.txt +++ /dev/null @@ -1,27 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometNativeScan 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/q96.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_iceberg_compat/explain.txt deleted file mode 100644 index f90d8b12b9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,143 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (4) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (10) - +- CometBroadcastExchange (19) - +- CometProject (18) - +- CometFilter (17) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (16) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(3) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#5, hd_dep_count#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,7), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 7)) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] -Arguments: [ss_sold_time_sk#1, ss_store_sk#3], [ss_sold_time_sk#1, ss_store_sk#3] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,20), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Condition : ((((isnotnull(t_hour#8) AND isnotnull(t_minute#9)) AND (t_hour#8 = 20)) AND (t_minute#9 >= 30)) AND isnotnull(t_time_sk#7)) - -(12) CometProject -Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Arguments: [t_time_sk#7], [t_time_sk#7] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#7] -Arguments: [t_time_sk#7] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Right output [1]: [t_time_sk#7] -Arguments: [ss_sold_time_sk#1], [t_time_sk#7], Inner, BuildRight - -(15) CometProject -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#7] -Arguments: [ss_store_sk#3], [ss_store_sk#3] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_store_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [s_store_sk#10, s_store_name#11] -Condition : ((isnotnull(s_store_name#11) AND (s_store_name#11 = ese)) AND isnotnull(s_store_sk#10)) - -(18) CometProject -Input [2]: [s_store_sk#10, s_store_name#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(19) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(20) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#3] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(21) CometProject -Input [2]: [ss_store_sk#3, s_store_sk#10] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#12] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 1] -Input [1]: [count(1)#13] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_iceberg_compat/simplified.txt deleted file mode 100644 index bbbd07245e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,27 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometScan [native_iceberg_compat] 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/q96/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/explain.txt deleted file mode 100644 index f90d8b12b9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/explain.txt +++ /dev/null @@ -1,143 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (4) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (10) - +- CometBroadcastExchange (19) - +- CometProject (18) - +- CometFilter (17) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (16) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(3) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#5, hd_dep_count#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,7), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 7)) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] -Arguments: [ss_sold_time_sk#1, ss_store_sk#3], [ss_sold_time_sk#1, ss_store_sk#3] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,20), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Condition : ((((isnotnull(t_hour#8) AND isnotnull(t_minute#9)) AND (t_hour#8 = 20)) AND (t_minute#9 >= 30)) AND isnotnull(t_time_sk#7)) - -(12) CometProject -Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Arguments: [t_time_sk#7], [t_time_sk#7] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#7] -Arguments: [t_time_sk#7] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Right output [1]: [t_time_sk#7] -Arguments: [ss_sold_time_sk#1], [t_time_sk#7], Inner, BuildRight - -(15) CometProject -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#7] -Arguments: [ss_store_sk#3], [ss_store_sk#3] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_store_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [s_store_sk#10, s_store_name#11] -Condition : ((isnotnull(s_store_name#11) AND (s_store_name#11 = ese)) AND isnotnull(s_store_sk#10)) - -(18) CometProject -Input [2]: [s_store_sk#10, s_store_name#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(19) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(20) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#3] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(21) CometProject -Input [2]: [ss_store_sk#3, s_store_sk#10] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#12] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 1] -Input [1]: [count(1)#13] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/extended.txt deleted file mode 100644 index 0f623c9021..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/extended.txt +++ /dev/null @@ -1,27 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 24 out of 24 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index bbbd07245e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q96/simplified.txt +++ /dev/null @@ -1,27 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/explain.txt deleted file mode 100644 index a7f1cb451d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/explain.txt +++ /dev/null @@ -1,174 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (24) -+- CometHashAggregate (23) - +- CometExchange (22) - +- CometHashAggregate (21) - +- CometProject (20) - +- CometSortMergeJoin (19) - :- CometSort (9) - : +- CometHashAggregate (8) - : +- CometColumnarExchange (7) - : +- * HashAggregate (6) - : +- * Project (5) - : +- * BroadcastHashJoin Inner BuildRight (4) - : :- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.store_sales (1) - : +- ReusedExchange (3) - +- CometSort (18) - +- CometHashAggregate (17) - +- CometColumnarExchange (16) - +- * HashAggregate (15) - +- * Project (14) - +- * BroadcastHashJoin Inner BuildRight (13) - :- * ColumnarToRow (11) - : +- Scan parquet spark_catalog.default.catalog_sales (10) - +- ReusedExchange (12) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 2] -Input [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] - -(3) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#5] - -(4) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(5) Project [codegen id : 2] -Output [2]: [ss_item_sk#1, ss_customer_sk#2] -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#5] - -(6) HashAggregate [codegen id : 2] -Input [2]: [ss_item_sk#1, ss_customer_sk#2] -Keys [2]: [ss_customer_sk#2, ss_item_sk#1] -Functions: [] -Aggregate Attributes: [] -Results [2]: [ss_customer_sk#2, ss_item_sk#1] - -(7) CometColumnarExchange -Input [2]: [ss_customer_sk#2, ss_item_sk#1] -Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(8) CometHashAggregate -Input [2]: [ss_customer_sk#2, ss_item_sk#1] -Keys [2]: [ss_customer_sk#2, ss_item_sk#1] -Functions: [] - -(9) CometSort -Input [2]: [customer_sk#6, item_sk#7] -Arguments: [customer_sk#6, item_sk#7], [customer_sk#6 ASC NULLS FIRST, item_sk#7 ASC NULLS FIRST] - -(10) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#4)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 4] -Input [3]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10] - -(12) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#11] - -(13) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#10] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 4] -Output [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Input [4]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10, d_date_sk#11] - -(15) HashAggregate [codegen id : 4] -Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Keys [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Functions: [] -Aggregate Attributes: [] -Results [2]: [cs_bill_customer_sk#8, cs_item_sk#9] - -(16) CometColumnarExchange -Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Arguments: hashpartitioning(cs_bill_customer_sk#8, cs_item_sk#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(17) CometHashAggregate -Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Keys [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Functions: [] - -(18) CometSort -Input [2]: [customer_sk#12, item_sk#13] -Arguments: [customer_sk#12, item_sk#13], [customer_sk#12 ASC NULLS FIRST, item_sk#13 ASC NULLS FIRST] - -(19) CometSortMergeJoin -Left output [2]: [customer_sk#6, item_sk#7] -Right output [2]: [customer_sk#12, item_sk#13] -Arguments: [customer_sk#6, item_sk#7], [customer_sk#12, item_sk#13], FullOuter - -(20) CometProject -Input [4]: [customer_sk#6, item_sk#7, customer_sk#12, item_sk#13] -Arguments: [customer_sk#6, customer_sk#12], [customer_sk#6, customer_sk#12] - -(21) CometHashAggregate -Input [2]: [customer_sk#6, customer_sk#12] -Keys: [] -Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#12)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#12)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#12)) THEN 1 ELSE 0 END)] - -(22) CometExchange -Input [3]: [sum#14, sum#15, sum#16] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(23) CometHashAggregate -Input [3]: [sum#14, sum#15, sum#16] -Keys: [] -Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#12)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#12)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#12)) THEN 1 ELSE 0 END)] - -(24) CometColumnarToRow [codegen id : 5] -Input [3]: [store_only#17, catalog_only#18, store_and_catalog#19] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometNativeScan parquet spark_catalog.default.date_dim (25) - - -(25) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#20] -Condition : (((isnotnull(d_month_seq#20) AND (d_month_seq#20 >= 1200)) AND (d_month_seq#20 <= 1211)) AND isnotnull(d_date_sk#5)) - -(27) CometProject -Input [2]: [d_date_sk#5, d_month_seq#20] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(29) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/simplified.txt deleted file mode 100644 index dac440b122..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,sum] [store_only,catalog_only,store_and_catalog,sum(CASE WHEN (isnotnull(customer_sk) AND isnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnotnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END)] - CometExchange #1 - CometHashAggregate [customer_sk,customer_sk] [sum,sum,sum] - CometProject [customer_sk,customer_sk] - CometSortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] - CometSort [customer_sk,item_sk] - CometHashAggregate [customer_sk,item_sk,ss_customer_sk,ss_item_sk] - CometColumnarExchange [ss_customer_sk,ss_item_sk] #2 - WholeStageCodegen (2) - HashAggregate [ss_customer_sk,ss_item_sk] - Project [ss_item_sk,ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 - CometSort [customer_sk,item_sk] - CometHashAggregate [customer_sk,item_sk,cs_bill_customer_sk,cs_item_sk] - CometColumnarExchange [cs_bill_customer_sk,cs_item_sk] #4 - WholeStageCodegen (4) - HashAggregate [cs_bill_customer_sk,cs_item_sk] - Project [cs_bill_customer_sk,cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/explain.txt deleted file mode 100644 index 420f37fccc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,179 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometSortMergeJoin (20) - :- CometSort (11) - : +- CometHashAggregate (10) - : +- CometExchange (9) - : +- CometHashAggregate (8) - : +- CometProject (7) - : +- CometBroadcastHashJoin (6) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (5) - : +- CometProject (4) - : +- CometFilter (3) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (2) - +- CometSort (19) - +- CometHashAggregate (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometProject (15) - +- CometBroadcastHashJoin (14) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (12) - +- ReusedExchange (13) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -ReadSchema: struct - -(2) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(3) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(4) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(5) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(6) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(7) CometProject -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2], [ss_item_sk#1, ss_customer_sk#2] - -(8) CometHashAggregate -Input [2]: [ss_item_sk#1, ss_customer_sk#2] -Keys [2]: [ss_customer_sk#2, ss_item_sk#1] -Functions: [] - -(9) CometExchange -Input [2]: [ss_customer_sk#2, ss_item_sk#1] -Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometHashAggregate -Input [2]: [ss_customer_sk#2, ss_item_sk#1] -Keys [2]: [ss_customer_sk#2, ss_item_sk#1] -Functions: [] - -(11) CometSort -Input [2]: [customer_sk#7, item_sk#8] -Arguments: [customer_sk#7, item_sk#8], [customer_sk#7 ASC NULLS FIRST, item_sk#8 ASC NULLS FIRST] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#11), dynamicpruningexpression(cs_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(13) ReusedExchange [Reuses operator id: 5] -Output [1]: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [3]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11] -Right output [1]: [d_date_sk#13] -Arguments: [cs_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [4]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11, d_date_sk#13] -Arguments: [cs_bill_customer_sk#9, cs_item_sk#10], [cs_bill_customer_sk#9, cs_item_sk#10] - -(16) CometHashAggregate -Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Keys [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Functions: [] - -(17) CometExchange -Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Arguments: hashpartitioning(cs_bill_customer_sk#9, cs_item_sk#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(18) CometHashAggregate -Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Keys [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Functions: [] - -(19) CometSort -Input [2]: [customer_sk#14, item_sk#15] -Arguments: [customer_sk#14, item_sk#15], [customer_sk#14 ASC NULLS FIRST, item_sk#15 ASC NULLS FIRST] - -(20) CometSortMergeJoin -Left output [2]: [customer_sk#7, item_sk#8] -Right output [2]: [customer_sk#14, item_sk#15] -Arguments: [customer_sk#7, item_sk#8], [customer_sk#14, item_sk#15], FullOuter - -(21) CometProject -Input [4]: [customer_sk#7, item_sk#8, customer_sk#14, item_sk#15] -Arguments: [customer_sk#7, customer_sk#14], [customer_sk#7, customer_sk#14] - -(22) CometHashAggregate -Input [2]: [customer_sk#7, customer_sk#14] -Keys: [] -Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)] - -(23) CometExchange -Input [3]: [sum#16, sum#17, sum#18] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(24) CometHashAggregate -Input [3]: [sum#16, sum#17, sum#18] -Keys: [] -Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)] - -(25) CometColumnarToRow [codegen id : 1] -Input [3]: [store_only#19, catalog_only#20, store_and_catalog#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(28) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(30) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 12 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/simplified.txt deleted file mode 100644 index a7de047b25..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,sum] [store_only,catalog_only,store_and_catalog,sum(CASE WHEN (isnotnull(customer_sk) AND isnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnotnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END)] - CometExchange #1 - CometHashAggregate [customer_sk,customer_sk] [sum,sum,sum] - CometProject [customer_sk,customer_sk] - CometSortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] - CometSort [customer_sk,item_sk] - CometHashAggregate [customer_sk,item_sk,ss_customer_sk,ss_item_sk] - CometExchange [ss_customer_sk,ss_item_sk] #2 - CometHashAggregate [ss_customer_sk,ss_item_sk] - CometProject [ss_item_sk,ss_customer_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometSort [customer_sk,item_sk] - CometHashAggregate [customer_sk,item_sk,cs_bill_customer_sk,cs_item_sk] - CometExchange [cs_bill_customer_sk,cs_item_sk] #5 - CometHashAggregate [cs_bill_customer_sk,cs_item_sk] - CometProject [cs_bill_customer_sk,cs_item_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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/q97/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/explain.txt deleted file mode 100644 index 420f37fccc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/explain.txt +++ /dev/null @@ -1,179 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometSortMergeJoin (20) - :- CometSort (11) - : +- CometHashAggregate (10) - : +- CometExchange (9) - : +- CometHashAggregate (8) - : +- CometProject (7) - : +- CometBroadcastHashJoin (6) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (5) - : +- CometProject (4) - : +- CometFilter (3) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (2) - +- CometSort (19) - +- CometHashAggregate (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometProject (15) - +- CometBroadcastHashJoin (14) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (12) - +- ReusedExchange (13) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -ReadSchema: struct - -(2) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(3) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(4) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(5) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(6) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(7) CometProject -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2], [ss_item_sk#1, ss_customer_sk#2] - -(8) CometHashAggregate -Input [2]: [ss_item_sk#1, ss_customer_sk#2] -Keys [2]: [ss_customer_sk#2, ss_item_sk#1] -Functions: [] - -(9) CometExchange -Input [2]: [ss_customer_sk#2, ss_item_sk#1] -Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometHashAggregate -Input [2]: [ss_customer_sk#2, ss_item_sk#1] -Keys [2]: [ss_customer_sk#2, ss_item_sk#1] -Functions: [] - -(11) CometSort -Input [2]: [customer_sk#7, item_sk#8] -Arguments: [customer_sk#7, item_sk#8], [customer_sk#7 ASC NULLS FIRST, item_sk#8 ASC NULLS FIRST] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#11), dynamicpruningexpression(cs_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(13) ReusedExchange [Reuses operator id: 5] -Output [1]: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [3]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11] -Right output [1]: [d_date_sk#13] -Arguments: [cs_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [4]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11, d_date_sk#13] -Arguments: [cs_bill_customer_sk#9, cs_item_sk#10], [cs_bill_customer_sk#9, cs_item_sk#10] - -(16) CometHashAggregate -Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Keys [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Functions: [] - -(17) CometExchange -Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Arguments: hashpartitioning(cs_bill_customer_sk#9, cs_item_sk#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(18) CometHashAggregate -Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Keys [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Functions: [] - -(19) CometSort -Input [2]: [customer_sk#14, item_sk#15] -Arguments: [customer_sk#14, item_sk#15], [customer_sk#14 ASC NULLS FIRST, item_sk#15 ASC NULLS FIRST] - -(20) CometSortMergeJoin -Left output [2]: [customer_sk#7, item_sk#8] -Right output [2]: [customer_sk#14, item_sk#15] -Arguments: [customer_sk#7, item_sk#8], [customer_sk#14, item_sk#15], FullOuter - -(21) CometProject -Input [4]: [customer_sk#7, item_sk#8, customer_sk#14, item_sk#15] -Arguments: [customer_sk#7, customer_sk#14], [customer_sk#7, customer_sk#14] - -(22) CometHashAggregate -Input [2]: [customer_sk#7, customer_sk#14] -Keys: [] -Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)] - -(23) CometExchange -Input [3]: [sum#16, sum#17, sum#18] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(24) CometHashAggregate -Input [3]: [sum#16, sum#17, sum#18] -Keys: [] -Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)] - -(25) CometColumnarToRow [codegen id : 1] -Input [3]: [store_only#19, catalog_only#20, store_and_catalog#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(28) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(30) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 12 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/extended.txt deleted file mode 100644 index e1aefb8382..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index a7de047b25..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,sum] [store_only,catalog_only,store_and_catalog,sum(CASE WHEN (isnotnull(customer_sk) AND isnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnotnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END)] - CometExchange #1 - CometHashAggregate [customer_sk,customer_sk] [sum,sum,sum] - CometProject [customer_sk,customer_sk] - CometSortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] - CometSort [customer_sk,item_sk] - CometHashAggregate [customer_sk,item_sk,ss_customer_sk,ss_item_sk] - CometExchange [ss_customer_sk,ss_item_sk] #2 - CometHashAggregate [ss_customer_sk,ss_item_sk] - CometProject [ss_item_sk,ss_customer_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometSort [customer_sk,item_sk] - CometHashAggregate [customer_sk,item_sk,cs_bill_customer_sk,cs_item_sk] - CometExchange [cs_bill_customer_sk,cs_item_sk] #5 - CometHashAggregate [cs_bill_customer_sk,cs_item_sk] - CometProject [cs_bill_customer_sk,cs_item_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/explain.txt deleted file mode 100644 index 8263680b2b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/explain.txt +++ /dev/null @@ -1,177 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometProject (25) - +- CometSort (24) - +- CometColumnarExchange (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.store_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.item (4) - +- ReusedExchange (11) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(7) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(8) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(11) ReusedExchange [Reuses operator id: 31] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] - -(14) HashAggregate [codegen id : 3] -Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(15) CometColumnarExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(17) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#17] -Results [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#17,17,2) AS _w0#19, i_item_id#11] - -(18) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(22) Project [codegen id : 6] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] - -(23) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(24) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST] - -(25) CometProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -(26) CometColumnarToRow [codegen id : 7] -Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometNativeScan parquet spark_catalog.default.date_dim (27) - - -(27) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(29) CometProject -Input [2]: [d_date_sk#14, d_date#22] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(31) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/simplified.txt deleted file mode 100644 index 3f64b57ffa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/simplified.txt +++ /dev/null @@ -1,48 +0,0 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (6) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/explain.txt deleted file mode 100644 index 4cc725ef2b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,178 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometProject (25) - +- CometSort (24) - +- CometColumnarExchange (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] - -(23) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] - -(25) CometProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -(26) CometColumnarToRow [codegen id : 3] -Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(29) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(31) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/simplified.txt deleted file mode 100644 index af05a7d8c6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,40 +0,0 @@ -WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #2 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/explain.txt deleted file mode 100644 index 4cc725ef2b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/explain.txt +++ /dev/null @@ -1,178 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometProject (25) - +- CometSort (24) - +- CometColumnarExchange (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] - -(23) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] - -(25) CometProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -(26) CometColumnarToRow [codegen id : 3] -Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(29) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(31) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/extended.txt deleted file mode 100644 index 4c972848e7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/extended.txt +++ /dev/null @@ -1,34 +0,0 @@ -CometColumnarToRow -+- CometProject - +- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index af05a7d8c6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/simplified.txt +++ /dev/null @@ -1,40 +0,0 @@ -WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #2 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/explain.txt deleted file mode 100644 index 3447b6efa3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometNativeScan parquet spark_catalog.default.warehouse (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometNativeScan parquet spark_catalog.default.ship_mode (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.call_center (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometNativeScan parquet spark_catalog.default.date_dim (19) - - -(1) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_ship_mode_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Condition : (((isnotnull(cs_warehouse_sk#4) AND isnotnull(cs_ship_mode_sk#3)) AND isnotnull(cs_call_center_sk#2)) AND isnotnull(cs_ship_date_sk#1)) - -(3) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [cs_warehouse_sk#4], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7], [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] - -(8) CometNativeScan parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#8, sm_type#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Condition : isnotnull(sm_ship_mode_sk#8) - -(10) CometProject -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Arguments: [sm_ship_mode_sk#8, sm_type#10], [sm_ship_mode_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, sm_type#9, 30, true, false, true) AS sm_type#10] - -(11) CometBroadcastExchange -Input [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [sm_ship_mode_sk#8, sm_type#10] - -(12) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] -Right output [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [cs_ship_mode_sk#3], [sm_ship_mode_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#10] -Arguments: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10], [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10] - -(14) CometNativeScan parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#11, cc_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#11, cc_name#12] -Condition : isnotnull(cc_call_center_sk#11) - -(16) CometBroadcastExchange -Input [2]: [cc_call_center_sk#11, cc_name#12] -Arguments: [cc_call_center_sk#11, cc_name#12] - -(17) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10] -Right output [2]: [cc_call_center_sk#11, cc_name#12] -Arguments: [cs_call_center_sk#2], [cc_call_center_sk#11], Inner, BuildRight - -(18) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_call_center_sk#11, cc_name#12] -Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12], [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12] - -(19) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_month_seq#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#13, d_month_seq#14] -Condition : (((isnotnull(d_month_seq#14) AND (d_month_seq#14 >= 1200)) AND (d_month_seq#14 <= 1211)) AND isnotnull(d_date_sk#13)) - -(21) CometProject -Input [2]: [d_date_sk#13, d_month_seq#14] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(23) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12] -Right output [1]: [d_date_sk#13] -Arguments: [cs_ship_date_sk#1], [d_date_sk#13], Inner, BuildRight - -(24) CometProject -Input [6]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12, d_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, _groupingexpression#15], [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#15] - -(25) CometHashAggregate -Input [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, sm_type#10, cc_name#12] -Functions [5]: [partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [8]: [_groupingexpression#15, sm_type#10, cc_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Arguments: hashpartitioning(_groupingexpression#15, sm_type#10, cc_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [8]: [_groupingexpression#15, sm_type#10, cc_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Keys [3]: [_groupingexpression#15, sm_type#10, cc_name#12] -Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,cc_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,cc_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, cc_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - -(29) CometColumnarToRow [codegen id : 1] -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/simplified.txt deleted file mode 100644 index 98a215dbc6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_datafusion/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 30) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 60) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 90) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [_groupingexpression,sm_type,cc_name] #1 - CometHashAggregate [cs_ship_date_sk,cs_sold_date_sk] [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum] - CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] - 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_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_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_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] - CometNativeScan 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 [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [cc_call_center_sk,cc_name] #4 - CometFilter [cc_call_center_sk,cc_name] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan 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/q99.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_iceberg_compat/explain.txt deleted file mode 100644 index 7e57135c10..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_ship_mode_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Condition : (((isnotnull(cs_warehouse_sk#4) AND isnotnull(cs_ship_mode_sk#3)) AND isnotnull(cs_call_center_sk#2)) AND isnotnull(cs_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [cs_warehouse_sk#4], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7], [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#8, sm_type#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Condition : isnotnull(sm_ship_mode_sk#8) - -(10) CometProject -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Arguments: [sm_ship_mode_sk#8, sm_type#10], [sm_ship_mode_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, sm_type#9, 30, true, false, true) AS sm_type#10] - -(11) CometBroadcastExchange -Input [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [sm_ship_mode_sk#8, sm_type#10] - -(12) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] -Right output [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [cs_ship_mode_sk#3], [sm_ship_mode_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#10] -Arguments: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10], [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#11, cc_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#11, cc_name#12] -Condition : isnotnull(cc_call_center_sk#11) - -(16) CometBroadcastExchange -Input [2]: [cc_call_center_sk#11, cc_name#12] -Arguments: [cc_call_center_sk#11, cc_name#12] - -(17) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10] -Right output [2]: [cc_call_center_sk#11, cc_name#12] -Arguments: [cs_call_center_sk#2], [cc_call_center_sk#11], Inner, BuildRight - -(18) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_call_center_sk#11, cc_name#12] -Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12], [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_month_seq#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#13, d_month_seq#14] -Condition : (((isnotnull(d_month_seq#14) AND (d_month_seq#14 >= 1200)) AND (d_month_seq#14 <= 1211)) AND isnotnull(d_date_sk#13)) - -(21) CometProject -Input [2]: [d_date_sk#13, d_month_seq#14] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(23) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12] -Right output [1]: [d_date_sk#13] -Arguments: [cs_ship_date_sk#1], [d_date_sk#13], Inner, BuildRight - -(24) CometProject -Input [6]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12, d_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, _groupingexpression#15], [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#15] - -(25) CometHashAggregate -Input [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, sm_type#10, cc_name#12] -Functions [5]: [partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [8]: [_groupingexpression#15, sm_type#10, cc_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Arguments: hashpartitioning(_groupingexpression#15, sm_type#10, cc_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [8]: [_groupingexpression#15, sm_type#10, cc_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Keys [3]: [_groupingexpression#15, sm_type#10, cc_name#12] -Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,cc_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,cc_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, cc_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - -(29) CometColumnarToRow [codegen id : 1] -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_iceberg_compat/simplified.txt deleted file mode 100644 index 22896ee566..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 30) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 60) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 90) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [_groupingexpression,sm_type,cc_name] #1 - CometHashAggregate [cs_ship_date_sk,cs_sold_date_sk] [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum] - CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] - 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_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_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_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 [native_iceberg_compat] 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 [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [cc_call_center_sk,cc_name] #4 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] 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/q99/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/explain.txt deleted file mode 100644 index 7e57135c10..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_ship_mode_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Condition : (((isnotnull(cs_warehouse_sk#4) AND isnotnull(cs_ship_mode_sk#3)) AND isnotnull(cs_call_center_sk#2)) AND isnotnull(cs_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [cs_warehouse_sk#4], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7], [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#8, sm_type#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Condition : isnotnull(sm_ship_mode_sk#8) - -(10) CometProject -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Arguments: [sm_ship_mode_sk#8, sm_type#10], [sm_ship_mode_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, sm_type#9, 30, true, false, true) AS sm_type#10] - -(11) CometBroadcastExchange -Input [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [sm_ship_mode_sk#8, sm_type#10] - -(12) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] -Right output [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [cs_ship_mode_sk#3], [sm_ship_mode_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#10] -Arguments: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10], [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#11, cc_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#11, cc_name#12] -Condition : isnotnull(cc_call_center_sk#11) - -(16) CometBroadcastExchange -Input [2]: [cc_call_center_sk#11, cc_name#12] -Arguments: [cc_call_center_sk#11, cc_name#12] - -(17) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10] -Right output [2]: [cc_call_center_sk#11, cc_name#12] -Arguments: [cs_call_center_sk#2], [cc_call_center_sk#11], Inner, BuildRight - -(18) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_call_center_sk#11, cc_name#12] -Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12], [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_month_seq#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#13, d_month_seq#14] -Condition : (((isnotnull(d_month_seq#14) AND (d_month_seq#14 >= 1200)) AND (d_month_seq#14 <= 1211)) AND isnotnull(d_date_sk#13)) - -(21) CometProject -Input [2]: [d_date_sk#13, d_month_seq#14] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(23) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12] -Right output [1]: [d_date_sk#13] -Arguments: [cs_ship_date_sk#1], [d_date_sk#13], Inner, BuildRight - -(24) CometProject -Input [6]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12, d_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, _groupingexpression#15], [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#15] - -(25) CometHashAggregate -Input [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, sm_type#10, cc_name#12] -Functions [5]: [partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [8]: [_groupingexpression#15, sm_type#10, cc_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Arguments: hashpartitioning(_groupingexpression#15, sm_type#10, cc_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [8]: [_groupingexpression#15, sm_type#10, cc_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Keys [3]: [_groupingexpression#15, sm_type#10, cc_name#12] -Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,cc_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,cc_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, cc_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - -(29) CometColumnarToRow [codegen id : 1] -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/extended.txt deleted file mode 100644 index 86cf2fc2f3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 28 out of 28 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 22896ee566..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q99/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 30) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 60) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 90) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [_groupingexpression,sm_type,cc_name] #1 - CometHashAggregate [cs_ship_date_sk,cs_sold_date_sk] [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum] - CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] - 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_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_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_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 [native_iceberg_compat] 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 [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [cc_call_center_sk,cc_name] #4 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/explain.txt deleted file mode 100644 index d1964eee2a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/explain.txt +++ /dev/null @@ -1,291 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (44) -+- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Filter (11) - : : : +- * HashAggregate (10) - : : : +- * CometColumnarToRow (9) - : : : +- CometColumnarExchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_returns (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (27) - : : +- * Filter (26) - : : +- * HashAggregate (25) - : : +- * CometColumnarToRow (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * HashAggregate (21) - : : +- * CometColumnarToRow (20) - : : +- CometColumnarExchange (19) - : : +- * HashAggregate (18) - : : +- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Filter (14) - : : : +- * ColumnarToRow (13) - : : : +- Scan parquet spark_catalog.default.store_returns (12) - : : +- ReusedExchange (15) - : +- BroadcastExchange (34) - : +- * CometColumnarToRow (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometNativeScan parquet spark_catalog.default.store (30) - +- BroadcastExchange (41) - +- * CometColumnarToRow (40) - +- CometProject (39) - +- CometFilter (38) - +- CometNativeScan parquet spark_catalog.default.customer (37) - - -(1) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk), IsNotNull(sr_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 2] -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] - -(3) Filter [codegen id : 2] -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [sr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 2] -Output [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] -Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] - -(7) HashAggregate [codegen id : 2] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] -Aggregate Attributes [1]: [sum#7] -Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] - -(8) CometColumnarExchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(9) CometColumnarToRow [codegen id : 9] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] - -(10) HashAggregate [codegen id : 9] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#9] -Results [3]: [sr_customer_sk#1 AS ctr_customer_sk#10, sr_store_sk#2 AS ctr_store_sk#11, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#9,17,2) AS ctr_total_return#12] - -(11) Filter [codegen id : 9] -Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12] -Condition : isnotnull(ctr_total_return#12) - -(12) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#16), dynamicpruningexpression(sr_returned_date_sk#16 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(13) ColumnarToRow [codegen id : 4] -Input [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] - -(14) Filter [codegen id : 4] -Input [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] -Condition : isnotnull(sr_store_sk#14) - -(15) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#17] - -(16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sr_returned_date_sk#16] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 4] -Output [3]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15] -Input [5]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16, d_date_sk#17] - -(18) HashAggregate [codegen id : 4] -Input [3]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15] -Keys [2]: [sr_customer_sk#13, sr_store_sk#14] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#15))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] - -(19) CometColumnarExchange -Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] -Arguments: hashpartitioning(sr_customer_sk#13, sr_store_sk#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(20) CometColumnarToRow [codegen id : 5] -Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] - -(21) HashAggregate [codegen id : 5] -Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] -Keys [2]: [sr_customer_sk#13, sr_store_sk#14] -Functions [1]: [sum(UnscaledValue(sr_return_amt#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#15))#9] -Results [2]: [sr_store_sk#14 AS ctr_store_sk#20, MakeDecimal(sum(UnscaledValue(sr_return_amt#15))#9,17,2) AS ctr_total_return#21] - -(22) HashAggregate [codegen id : 5] -Input [2]: [ctr_store_sk#20, ctr_total_return#21] -Keys [1]: [ctr_store_sk#20] -Functions [1]: [partial_avg(ctr_total_return#21)] -Aggregate Attributes [2]: [sum#22, count#23] -Results [3]: [ctr_store_sk#20, sum#24, count#25] - -(23) CometColumnarExchange -Input [3]: [ctr_store_sk#20, sum#24, count#25] -Arguments: hashpartitioning(ctr_store_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometColumnarToRow [codegen id : 6] -Input [3]: [ctr_store_sk#20, sum#24, count#25] - -(25) HashAggregate [codegen id : 6] -Input [3]: [ctr_store_sk#20, sum#24, count#25] -Keys [1]: [ctr_store_sk#20] -Functions [1]: [avg(ctr_total_return#21)] -Aggregate Attributes [1]: [avg(ctr_total_return#21)#26] -Results [2]: [(avg(ctr_total_return#21)#26 * 1.2) AS (avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] - -(26) Filter [codegen id : 6] -Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#27) - -(27) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=4] - -(28) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ctr_store_sk#11] -Right keys [1]: [ctr_store_sk#20] -Join type: Inner -Join condition: (cast(ctr_total_return#12 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#27) - -(29) Project [codegen id : 9] -Output [2]: [ctr_customer_sk#10, ctr_store_sk#11] -Input [5]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12, (avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] - -(30) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#28, s_state#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [s_store_sk#28, s_state#29] -Condition : ((isnotnull(s_state#29) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#29, 2)) = TN)) AND isnotnull(s_store_sk#28)) - -(32) CometProject -Input [2]: [s_store_sk#28, s_state#29] -Arguments: [s_store_sk#28], [s_store_sk#28] - -(33) CometColumnarToRow [codegen id : 7] -Input [1]: [s_store_sk#28] - -(34) BroadcastExchange -Input [1]: [s_store_sk#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ctr_store_sk#11] -Right keys [1]: [s_store_sk#28] -Join type: Inner -Join condition: None - -(36) Project [codegen id : 9] -Output [1]: [ctr_customer_sk#10] -Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, s_store_sk#28] - -(37) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#30, c_customer_id#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(38) CometFilter -Input [2]: [c_customer_sk#30, c_customer_id#31] -Condition : isnotnull(c_customer_sk#30) - -(39) CometProject -Input [2]: [c_customer_sk#30, c_customer_id#31] -Arguments: [c_customer_sk#30, c_customer_id#32], [c_customer_sk#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#31, 16)) AS c_customer_id#32] - -(40) CometColumnarToRow [codegen id : 8] -Input [2]: [c_customer_sk#30, c_customer_id#32] - -(41) BroadcastExchange -Input [2]: [c_customer_sk#30, c_customer_id#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(42) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ctr_customer_sk#10] -Right keys [1]: [c_customer_sk#30] -Join type: Inner -Join condition: None - -(43) Project [codegen id : 9] -Output [1]: [c_customer_id#32] -Input [3]: [ctr_customer_sk#10, c_customer_sk#30, c_customer_id#32] - -(44) TakeOrderedAndProject -Input [1]: [c_customer_id#32] -Arguments: 100, [c_customer_id#32 ASC NULLS FIRST], [c_customer_id#32] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometNativeScan parquet spark_catalog.default.date_dim (45) - - -(45) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [d_date_sk#6, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 2000)) AND isnotnull(d_date_sk#6)) - -(47) CometProject -Input [2]: [d_date_sk#6, d_year#33] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(49) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 12 Hosting Expression = sr_returned_date_sk#16 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/simplified.txt deleted file mode 100644 index ed85c142aa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/simplified.txt +++ /dev/null @@ -1,72 +0,0 @@ -TakeOrderedAndProject [c_customer_id] - WholeStageCodegen (9) - Project [c_customer_id] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk] - BroadcastHashJoin [ctr_store_sk,s_store_sk] - Project [ctr_customer_sk,ctr_store_sk] - BroadcastHashJoin [ctr_store_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [sr_customer_sk,sr_store_sk] #1 - WholeStageCodegen (2) - HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] - Project [sr_customer_sk,sr_store_sk,sr_return_amt] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_store_sk,sr_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (6) - Filter [(avg(ctr_total_return) * 1.2)] - HashAggregate [ctr_store_sk,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ctr_store_sk] #4 - WholeStageCodegen (5) - HashAggregate [ctr_store_sk,ctr_total_return] [sum,count,sum,count] - HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_store_sk,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [sr_customer_sk,sr_store_sk] #5 - WholeStageCodegen (4) - HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] - Project [sr_customer_sk,sr_store_sk,sr_return_amt] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id] [c_customer_sk,c_customer_id] - CometFilter [c_customer_sk,c_customer_id] - CometNativeScan 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/q1.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/explain.txt deleted file mode 100644 index a37054da2d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,269 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometTakeOrderedAndProject (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometFilter (12) - : : : +- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (25) - : : +- CometFilter (24) - : : +- CometHashAggregate (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometHashAggregate (20) - : : +- CometExchange (19) - : : +- CometHashAggregate (18) - : : +- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (13) - : : +- ReusedExchange (15) - : +- CometBroadcastExchange (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (28) - +- CometBroadcastExchange (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (34) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk), IsNotNull(sr_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [sr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] -Arguments: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3], [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] - -(9) CometHashAggregate -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] - -(10) CometExchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] - -(12) CometFilter -Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] -Condition : isnotnull(ctr_total_return#11) - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#16)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#13) - -(15) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#17] - -(16) CometBroadcastHashJoin -Left output [4]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15] -Right output [1]: [d_date_sk#17] -Arguments: [sr_returned_date_sk#15], [d_date_sk#17], Inner, BuildRight - -(17) CometProject -Input [5]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15, d_date_sk#17] -Arguments: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14], [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14] - -(18) CometHashAggregate -Input [3]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14] -Keys [2]: [sr_customer_sk#12, sr_store_sk#13] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#14))] - -(19) CometExchange -Input [3]: [sr_customer_sk#12, sr_store_sk#13, sum#18] -Arguments: hashpartitioning(sr_customer_sk#12, sr_store_sk#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometHashAggregate -Input [3]: [sr_customer_sk#12, sr_store_sk#13, sum#18] -Keys [2]: [sr_customer_sk#12, sr_store_sk#13] -Functions [1]: [sum(UnscaledValue(sr_return_amt#14))] - -(21) CometHashAggregate -Input [2]: [ctr_store_sk#19, ctr_total_return#20] -Keys [1]: [ctr_store_sk#19] -Functions [1]: [partial_avg(ctr_total_return#20)] - -(22) CometExchange -Input [3]: [ctr_store_sk#19, sum#21, count#22] -Arguments: hashpartitioning(ctr_store_sk#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(23) CometHashAggregate -Input [3]: [ctr_store_sk#19, sum#21, count#22] -Keys [1]: [ctr_store_sk#19] -Functions [1]: [avg(ctr_total_return#20)] - -(24) CometFilter -Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) - -(25) CometBroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] -Arguments: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] - -(26) CometBroadcastHashJoin -Left output [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] -Right output [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] -Arguments: [ctr_store_sk#10], [ctr_store_sk#19], Inner, (cast(ctr_total_return#11 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#23), BuildRight - -(27) CometProject -Input [5]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11, (avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] -Arguments: [ctr_customer_sk#9, ctr_store_sk#10], [ctr_customer_sk#9, ctr_store_sk#10] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_state#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] -ReadSchema: struct - -(29) CometFilter -Input [2]: [s_store_sk#24, s_state#25] -Condition : ((isnotnull(s_state#25) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#25, 2)) = TN)) AND isnotnull(s_store_sk#24)) - -(30) CometProject -Input [2]: [s_store_sk#24, s_state#25] -Arguments: [s_store_sk#24], [s_store_sk#24] - -(31) CometBroadcastExchange -Input [1]: [s_store_sk#24] -Arguments: [s_store_sk#24] - -(32) CometBroadcastHashJoin -Left output [2]: [ctr_customer_sk#9, ctr_store_sk#10] -Right output [1]: [s_store_sk#24] -Arguments: [ctr_store_sk#10], [s_store_sk#24], Inner, BuildRight - -(33) CometProject -Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, s_store_sk#24] -Arguments: [ctr_customer_sk#9], [ctr_customer_sk#9] - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#26, c_customer_id#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [c_customer_sk#26, c_customer_id#27] -Condition : isnotnull(c_customer_sk#26) - -(36) CometProject -Input [2]: [c_customer_sk#26, c_customer_id#27] -Arguments: [c_customer_sk#26, c_customer_id#28], [c_customer_sk#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#27, 16)) AS c_customer_id#28] - -(37) CometBroadcastExchange -Input [2]: [c_customer_sk#26, c_customer_id#28] -Arguments: [c_customer_sk#26, c_customer_id#28] - -(38) CometBroadcastHashJoin -Left output [1]: [ctr_customer_sk#9] -Right output [2]: [c_customer_sk#26, c_customer_id#28] -Arguments: [ctr_customer_sk#9], [c_customer_sk#26], Inner, BuildRight - -(39) CometProject -Input [3]: [ctr_customer_sk#9, c_customer_sk#26, c_customer_id#28] -Arguments: [c_customer_id#28], [c_customer_id#28] - -(40) CometTakeOrderedAndProject -Input [1]: [c_customer_id#28] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#28 ASC NULLS FIRST], output=[c_customer_id#28]), [c_customer_id#28], 100, 0, [c_customer_id#28 ASC NULLS FIRST], [c_customer_id#28] - -(41) CometColumnarToRow [codegen id : 1] -Input [1]: [c_customer_id#28] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) - - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(44) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(45) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(46) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 13 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/simplified.txt deleted file mode 100644 index 3e1d6243f8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id] - CometProject [c_customer_id] - CometBroadcastHashJoin [ctr_customer_sk,c_customer_sk,c_customer_id] - CometProject [ctr_customer_sk] - CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,s_store_sk] - CometProject [ctr_customer_sk,ctr_store_sk] - CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_store_sk] - CometFilter [ctr_customer_sk,ctr_store_sk,ctr_total_return] - CometHashAggregate [sum] [ctr_customer_sk,ctr_store_sk,ctr_total_return,sr_customer_sk,sr_store_sk,sum(UnscaledValue(sr_return_amt))] - CometExchange [sr_customer_sk,sr_store_sk] #1 - CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] - CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_store_sk] #4 - CometFilter [(avg(ctr_total_return) * 1.2),ctr_store_sk] - CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_store_sk,avg(ctr_total_return)] - CometExchange [ctr_store_sk] #5 - CometHashAggregate [ctr_total_return] [ctr_store_sk,sum,count] - CometHashAggregate [sr_customer_sk,sum] [ctr_store_sk,ctr_total_return,sr_store_sk,sum(UnscaledValue(sr_return_amt))] - CometExchange [sr_customer_sk,sr_store_sk] #6 - CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] - CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - 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 [native_iceberg_compat] 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 - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [c_customer_sk,c_customer_id] #8 - CometProject [c_customer_id] [c_customer_sk,c_customer_id] - CometFilter [c_customer_sk,c_customer_id] - CometScan [native_iceberg_compat] 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/q1/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/explain.txt deleted file mode 100644 index a37054da2d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/explain.txt +++ /dev/null @@ -1,269 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometTakeOrderedAndProject (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometFilter (12) - : : : +- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (25) - : : +- CometFilter (24) - : : +- CometHashAggregate (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometHashAggregate (20) - : : +- CometExchange (19) - : : +- CometHashAggregate (18) - : : +- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (13) - : : +- ReusedExchange (15) - : +- CometBroadcastExchange (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (28) - +- CometBroadcastExchange (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (34) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk), IsNotNull(sr_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [sr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] -Arguments: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3], [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] - -(9) CometHashAggregate -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] - -(10) CometExchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] - -(12) CometFilter -Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] -Condition : isnotnull(ctr_total_return#11) - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#16)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#13) - -(15) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#17] - -(16) CometBroadcastHashJoin -Left output [4]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15] -Right output [1]: [d_date_sk#17] -Arguments: [sr_returned_date_sk#15], [d_date_sk#17], Inner, BuildRight - -(17) CometProject -Input [5]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15, d_date_sk#17] -Arguments: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14], [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14] - -(18) CometHashAggregate -Input [3]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14] -Keys [2]: [sr_customer_sk#12, sr_store_sk#13] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#14))] - -(19) CometExchange -Input [3]: [sr_customer_sk#12, sr_store_sk#13, sum#18] -Arguments: hashpartitioning(sr_customer_sk#12, sr_store_sk#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometHashAggregate -Input [3]: [sr_customer_sk#12, sr_store_sk#13, sum#18] -Keys [2]: [sr_customer_sk#12, sr_store_sk#13] -Functions [1]: [sum(UnscaledValue(sr_return_amt#14))] - -(21) CometHashAggregate -Input [2]: [ctr_store_sk#19, ctr_total_return#20] -Keys [1]: [ctr_store_sk#19] -Functions [1]: [partial_avg(ctr_total_return#20)] - -(22) CometExchange -Input [3]: [ctr_store_sk#19, sum#21, count#22] -Arguments: hashpartitioning(ctr_store_sk#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(23) CometHashAggregate -Input [3]: [ctr_store_sk#19, sum#21, count#22] -Keys [1]: [ctr_store_sk#19] -Functions [1]: [avg(ctr_total_return#20)] - -(24) CometFilter -Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) - -(25) CometBroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] -Arguments: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] - -(26) CometBroadcastHashJoin -Left output [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] -Right output [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] -Arguments: [ctr_store_sk#10], [ctr_store_sk#19], Inner, (cast(ctr_total_return#11 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#23), BuildRight - -(27) CometProject -Input [5]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11, (avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] -Arguments: [ctr_customer_sk#9, ctr_store_sk#10], [ctr_customer_sk#9, ctr_store_sk#10] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_state#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] -ReadSchema: struct - -(29) CometFilter -Input [2]: [s_store_sk#24, s_state#25] -Condition : ((isnotnull(s_state#25) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#25, 2)) = TN)) AND isnotnull(s_store_sk#24)) - -(30) CometProject -Input [2]: [s_store_sk#24, s_state#25] -Arguments: [s_store_sk#24], [s_store_sk#24] - -(31) CometBroadcastExchange -Input [1]: [s_store_sk#24] -Arguments: [s_store_sk#24] - -(32) CometBroadcastHashJoin -Left output [2]: [ctr_customer_sk#9, ctr_store_sk#10] -Right output [1]: [s_store_sk#24] -Arguments: [ctr_store_sk#10], [s_store_sk#24], Inner, BuildRight - -(33) CometProject -Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, s_store_sk#24] -Arguments: [ctr_customer_sk#9], [ctr_customer_sk#9] - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#26, c_customer_id#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [c_customer_sk#26, c_customer_id#27] -Condition : isnotnull(c_customer_sk#26) - -(36) CometProject -Input [2]: [c_customer_sk#26, c_customer_id#27] -Arguments: [c_customer_sk#26, c_customer_id#28], [c_customer_sk#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#27, 16)) AS c_customer_id#28] - -(37) CometBroadcastExchange -Input [2]: [c_customer_sk#26, c_customer_id#28] -Arguments: [c_customer_sk#26, c_customer_id#28] - -(38) CometBroadcastHashJoin -Left output [1]: [ctr_customer_sk#9] -Right output [2]: [c_customer_sk#26, c_customer_id#28] -Arguments: [ctr_customer_sk#9], [c_customer_sk#26], Inner, BuildRight - -(39) CometProject -Input [3]: [ctr_customer_sk#9, c_customer_sk#26, c_customer_id#28] -Arguments: [c_customer_id#28], [c_customer_id#28] - -(40) CometTakeOrderedAndProject -Input [1]: [c_customer_id#28] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#28 ASC NULLS FIRST], output=[c_customer_id#28]), [c_customer_id#28], 100, 0, [c_customer_id#28 ASC NULLS FIRST], [c_customer_id#28] - -(41) CometColumnarToRow [codegen id : 1] -Input [1]: [c_customer_id#28] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) - - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(44) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(45) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(46) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 13 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/extended.txt deleted file mode 100644 index 8aa14c43dd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/extended.txt +++ /dev/null @@ -1,53 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 46 out of 49 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 3e1d6243f8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id] - CometProject [c_customer_id] - CometBroadcastHashJoin [ctr_customer_sk,c_customer_sk,c_customer_id] - CometProject [ctr_customer_sk] - CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,s_store_sk] - CometProject [ctr_customer_sk,ctr_store_sk] - CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_store_sk] - CometFilter [ctr_customer_sk,ctr_store_sk,ctr_total_return] - CometHashAggregate [sum] [ctr_customer_sk,ctr_store_sk,ctr_total_return,sr_customer_sk,sr_store_sk,sum(UnscaledValue(sr_return_amt))] - CometExchange [sr_customer_sk,sr_store_sk] #1 - CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] - CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_store_sk] #4 - CometFilter [(avg(ctr_total_return) * 1.2),ctr_store_sk] - CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_store_sk,avg(ctr_total_return)] - CometExchange [ctr_store_sk] #5 - CometHashAggregate [ctr_total_return] [ctr_store_sk,sum,count] - CometHashAggregate [sr_customer_sk,sum] [ctr_store_sk,ctr_total_return,sr_store_sk,sum(UnscaledValue(sr_return_amt))] - CometExchange [sr_customer_sk,sr_store_sk] #6 - CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] - CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - 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 [native_iceberg_compat] 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 - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [c_customer_sk,c_customer_id] #8 - CometProject [c_customer_id] [c_customer_sk,c_customer_id] - CometFilter [c_customer_sk,c_customer_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/explain.txt deleted file mode 100644 index 2c4b495826..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/explain.txt +++ /dev/null @@ -1,295 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (45) -+- * HashAggregate (44) - +- * CometColumnarToRow (43) - +- CometColumnarExchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometNativeScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (38) - +- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometNativeScan parquet spark_catalog.default.customer_demographics (34) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(4) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] - -(6) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#9] - -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None - -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#6] -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] - -(9) BroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#6] -Join type: LeftSemi -Join condition: None - -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] - -(13) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#12] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#10] -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] - -(16) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(17) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#10] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] - -(20) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#15] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#13] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] - -(23) BroadcastExchange -Input [1]: [cs_ship_customer_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#13] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(25) Filter [codegen id : 9] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(26) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(27) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_county#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [ca_address_sk#16, ca_county#17] -Condition : (ca_county#17 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#16)) - -(29) CometProject -Input [2]: [ca_address_sk#16, ca_county#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(30) CometColumnarToRow [codegen id : 7] -Input [1]: [ca_address_sk#16] - -(31) BroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#16] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 9] -Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#16] - -(34) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(35) CometFilter -Input [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Condition : isnotnull(cd_demo_sk#18) - -(36) CometProject -Input [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#19, 1)) AS cd_gender#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) AS cd_marital_status#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) AS cd_education_status#29, cd_purchase_estimate#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_credit_rating#23, 10)) AS cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(37) CometColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(38) BroadcastExchange -Input [9]: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 9] -Output [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(41) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#31] -Results [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] - -(42) CometColumnarExchange -Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] -Arguments: hashpartitioning(cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) CometColumnarToRow [codegen id : 10] -Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] - -(44) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] -Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#33] -Results [14]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, count(1)#33 AS cnt1#34, cd_purchase_estimate#22, count(1)#33 AS cnt2#35, cd_credit_rating#30, count(1)#33 AS cnt3#36, cd_dep_count#24, count(1)#33 AS cnt4#37, cd_dep_employed_count#25, count(1)#33 AS cnt5#38, cd_dep_college_count#26, count(1)#33 AS cnt6#39] - -(45) TakeOrderedAndProject -Input [14]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cnt1#34, cd_purchase_estimate#22, cnt2#35, cd_credit_rating#30, cnt3#36, cd_dep_count#24, cnt4#37, cd_dep_employed_count#25, cnt5#38, cd_dep_college_count#26, cnt6#39] -Arguments: 100, [cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_education_status#29 ASC NULLS FIRST, cd_purchase_estimate#22 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [cd_gender#27, cd_marital_status#28, cd_education_status#29, cnt1#34, cd_purchase_estimate#22, cnt2#35, cd_credit_rating#30, cnt3#36, cd_dep_count#24, cnt4#37, cd_dep_employed_count#25, cnt5#38, cd_dep_college_count#26, cnt6#39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (50) -+- * CometColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometNativeScan parquet spark_catalog.default.date_dim (46) - - -(46) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#40, d_moy#41] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_date_sk#9, d_year#40, d_moy#41] -Condition : (((((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2002)) AND (d_moy#41 >= 1)) AND (d_moy#41 <= 4)) AND isnotnull(d_date_sk#9)) - -(48) CometProject -Input [3]: [d_date_sk#9, d_year#40, d_moy#41] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(49) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(50) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/simplified.txt deleted file mode 100644 index 0cc108b0b0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/simplified.txt +++ /dev/null @@ -1,77 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (10) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [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] - 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] - CometNativeScan 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/q10.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/explain.txt deleted file mode 100644 index e253b8ca7b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * Filter (27) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2002)) AND (d_moy#11 >= 1)) AND (d_moy#11 <= 4)) AND isnotnull(d_date_sk#9)) - -(6) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: [d_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#9] -Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -Arguments: [ss_customer_sk#6], [ss_customer_sk#6] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: [ss_customer_sk#6] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Right output [1]: [ss_customer_sk#6] -Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#15] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Right output [1]: [d_date_sk#15] -Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] -Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#12] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#12] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#19] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#16] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#16] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(27) Filter [codegen id : 5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(28) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_county#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [ca_address_sk#20, ca_county#21] -Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#20)) - -(31) CometProject -Input [2]: [ca_address_sk#20, ca_county#21] -Arguments: [ca_address_sk#20], [ca_address_sk#20] - -(32) CometColumnarToRow [codegen id : 3] -Input [1]: [ca_address_sk#20] - -(33) BroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(34) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 5] -Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(37) CometFilter -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Condition : isnotnull(cd_demo_sk#22) - -(38) CometProject -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_demo_sk#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#23, 1)) AS cd_gender#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#24, 1)) AS cd_marital_status#32, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#25, 20)) AS cd_education_status#33, cd_purchase_estimate#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_credit_rating#27, 10)) AS cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(39) CometColumnarToRow [codegen id : 4] -Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(40) BroadcastExchange -Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(41) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#22] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 5] -Output [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(43) HashAggregate [codegen id : 5] -Input [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#35] -Results [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] - -(44) CometColumnarExchange -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] -Arguments: hashpartitioning(cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 6] -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] - -(46) HashAggregate [codegen id : 6] -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] -Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#37] -Results [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, count(1)#37 AS cnt1#38, cd_purchase_estimate#26, count(1)#37 AS cnt2#39, cd_credit_rating#34, count(1)#37 AS cnt3#40, cd_dep_count#28, count(1)#37 AS cnt4#41, cd_dep_employed_count#29, count(1)#37 AS cnt5#42, cd_dep_college_count#30, count(1)#37 AS cnt6#43] - -(47) TakeOrderedAndProject -Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#38, cd_purchase_estimate#26, cnt2#39, cd_credit_rating#34, cnt3#40, cd_dep_count#28, cnt4#41, cd_dep_employed_count#29, cnt5#42, cd_dep_college_count#30, cnt6#43] -Arguments: 100, [cd_gender#31 ASC NULLS FIRST, cd_marital_status#32 ASC NULLS FIRST, cd_education_status#33 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#34 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#38, cd_purchase_estimate#26, cnt2#39, cd_credit_rating#34, cnt3#40, cd_dep_count#28, cnt4#41, cd_dep_employed_count#29, cnt5#42, cd_dep_college_count#30, cnt6#43] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2002)) AND (d_moy#11 >= 1)) AND (d_moy#11 <= 4)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(52) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/simplified.txt deleted file mode 100644 index e7193f87e1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (6) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [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] - 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 [native_iceberg_compat] 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/q10/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/explain.txt deleted file mode 100644 index e253b8ca7b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * Filter (27) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2002)) AND (d_moy#11 >= 1)) AND (d_moy#11 <= 4)) AND isnotnull(d_date_sk#9)) - -(6) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: [d_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#9] -Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -Arguments: [ss_customer_sk#6], [ss_customer_sk#6] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: [ss_customer_sk#6] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Right output [1]: [ss_customer_sk#6] -Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#15] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Right output [1]: [d_date_sk#15] -Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] -Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#12] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#12] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#19] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#16] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#16] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(27) Filter [codegen id : 5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(28) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_county#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [ca_address_sk#20, ca_county#21] -Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#20)) - -(31) CometProject -Input [2]: [ca_address_sk#20, ca_county#21] -Arguments: [ca_address_sk#20], [ca_address_sk#20] - -(32) CometColumnarToRow [codegen id : 3] -Input [1]: [ca_address_sk#20] - -(33) BroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(34) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 5] -Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(37) CometFilter -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Condition : isnotnull(cd_demo_sk#22) - -(38) CometProject -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_demo_sk#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#23, 1)) AS cd_gender#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#24, 1)) AS cd_marital_status#32, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#25, 20)) AS cd_education_status#33, cd_purchase_estimate#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_credit_rating#27, 10)) AS cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(39) CometColumnarToRow [codegen id : 4] -Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(40) BroadcastExchange -Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(41) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#22] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 5] -Output [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(43) HashAggregate [codegen id : 5] -Input [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#35] -Results [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] - -(44) CometColumnarExchange -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] -Arguments: hashpartitioning(cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 6] -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] - -(46) HashAggregate [codegen id : 6] -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#36] -Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#37] -Results [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, count(1)#37 AS cnt1#38, cd_purchase_estimate#26, count(1)#37 AS cnt2#39, cd_credit_rating#34, count(1)#37 AS cnt3#40, cd_dep_count#28, count(1)#37 AS cnt4#41, cd_dep_employed_count#29, count(1)#37 AS cnt5#42, cd_dep_college_count#30, count(1)#37 AS cnt6#43] - -(47) TakeOrderedAndProject -Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#38, cd_purchase_estimate#26, cnt2#39, cd_credit_rating#34, cnt3#40, cd_dep_count#28, cnt4#41, cd_dep_employed_count#29, cnt5#42, cd_dep_college_count#30, cnt6#43] -Arguments: 100, [cd_gender#31 ASC NULLS FIRST, cd_marital_status#32 ASC NULLS FIRST, cd_education_status#33 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#34 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#38, cd_purchase_estimate#26, cnt2#39, cd_credit_rating#34, cnt3#40, cd_dep_count#28, cnt4#41, cd_dep_employed_count#29, cnt5#42, cd_dep_college_count#30, cnt6#43] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2002)) AND (d_moy#11 >= 1)) AND (d_moy#11 <= 4)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(52) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/extended.txt deleted file mode 100644 index a6f33d6f7e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/extended.txt +++ /dev/null @@ -1,63 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index e7193f87e1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (6) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [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] - 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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/explain.txt deleted file mode 100644 index 0b7aa0c883..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/explain.txt +++ /dev/null @@ -1,518 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (80) -+- * Project (79) - +- * BroadcastHashJoin Inner BuildRight (78) - :- * Project (59) - : +- * BroadcastHashJoin Inner BuildRight (58) - : :- * Project (38) - : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- * HashAggregate (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * Project (28) - : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (26) - : : : +- * Filter (25) - : : : +- * ColumnarToRow (24) - : : : +- Scan parquet spark_catalog.default.store_sales (23) - : : +- ReusedExchange (29) - : +- BroadcastExchange (57) - : +- * Filter (56) - : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) - : +- CometColumnarExchange (53) - : +- * HashAggregate (52) - : +- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Project (48) - : : +- * BroadcastHashJoin Inner BuildRight (47) - : : :- * CometColumnarToRow (42) - : : : +- CometProject (41) - : : : +- CometFilter (40) - : : : +- CometNativeScan parquet spark_catalog.default.customer (39) - : : +- BroadcastExchange (46) - : : +- * Filter (45) - : : +- * ColumnarToRow (44) - : : +- Scan parquet spark_catalog.default.web_sales (43) - : +- ReusedExchange (49) - +- BroadcastExchange (77) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- CometColumnarExchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * Project (69) - : +- * BroadcastHashJoin Inner BuildRight (68) - : :- * CometColumnarToRow (63) - : : +- CometProject (62) - : : +- CometFilter (61) - : : +- CometNativeScan parquet spark_catalog.default.customer (60) - : +- BroadcastExchange (67) - : +- * Filter (66) - : +- * ColumnarToRow (65) - : +- Scan parquet spark_catalog.default.web_sales (64) - +- ReusedExchange (70) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)) AS c_customer_id#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#3, 20)) AS c_first_name#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#4, 30)) AS c_last_name#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#5, 1)) AS c_preferred_cust_flag#12, c_birth_country#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#7, 13)) AS c_login#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#8, 50)) AS c_email_address#14] - -(4) CometColumnarToRow [codegen id : 3] -Input [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] - -(5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(7) Filter [codegen id : 1] -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#15) - -(8) BroadcastExchange -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#15] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(11) ReusedExchange [Reuses operator id: 84] -Output [2]: [d_date_sk#20, d_year#21] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#18] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] - -(14) HashAggregate [codegen id : 3] -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] -Aggregate Attributes [1]: [sum#22] -Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] - -(15) CometColumnarExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 16] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] - -(17) HashAggregate [codegen id : 16] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24] -Results [2]: [c_customer_id#9 AS customer_id#25, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24,18,2) AS year_total#26] - -(18) Filter [codegen id : 16] -Input [2]: [customer_id#25, year_total#26] -Condition : (isnotnull(year_total#26) AND (year_total#26 > 0.00)) - -(19) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(20) CometFilter -Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] -Condition : (isnotnull(c_customer_sk#27) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#28, 16)))) - -(21) CometProject -Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] -Arguments: [c_customer_sk#27, c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40], [c_customer_sk#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#28, 16)) AS c_customer_id#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#29, 20)) AS c_first_name#36, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#30, 30)) AS c_last_name#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#31, 1)) AS c_preferred_cust_flag#38, c_birth_country#32, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#33, 13)) AS c_login#39, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#34, 50)) AS c_email_address#40] - -(22) CometColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#27, c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40] - -(23) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#41, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#44), dynamicpruningexpression(ss_sold_date_sk#44 IN dynamicpruning#45)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#41, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44] - -(25) Filter [codegen id : 4] -Input [4]: [ss_customer_sk#41, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44] -Condition : isnotnull(ss_customer_sk#41) - -(26) BroadcastExchange -Input [4]: [ss_customer_sk#41, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#27] -Right keys [1]: [ss_customer_sk#41] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [10]: [c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44] -Input [12]: [c_customer_sk#27, c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, ss_customer_sk#41, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44] - -(29) ReusedExchange [Reuses operator id: 88] -Output [2]: [d_date_sk#46, d_year#47] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#44] -Right keys [1]: [d_date_sk#46] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [10]: [c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, ss_ext_discount_amt#42, ss_ext_list_price#43, d_year#47] -Input [12]: [c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44, d_date_sk#46, d_year#47] - -(32) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, ss_ext_discount_amt#42, ss_ext_list_price#43, d_year#47] -Keys [8]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#43 - ss_ext_discount_amt#42)))] -Aggregate Attributes [1]: [sum#48] -Results [9]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, sum#49] - -(33) CometColumnarExchange -Input [9]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, sum#49] -Arguments: hashpartitioning(c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 7] -Input [9]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, sum#49] - -(35) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, sum#49] -Keys [8]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#43 - ss_ext_discount_amt#42)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#43 - ss_ext_discount_amt#42)))#24] -Results [3]: [c_customer_id#35 AS customer_id#50, c_preferred_cust_flag#38 AS customer_preferred_cust_flag#51, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#43 - ss_ext_discount_amt#42)))#24,18,2) AS year_total#52] - -(36) BroadcastExchange -Input [3]: [customer_id#50, customer_preferred_cust_flag#51, year_total#52] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(37) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#50] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 16] -Output [4]: [customer_id#25, year_total#26, customer_preferred_cust_flag#51, year_total#52] -Input [5]: [customer_id#25, year_total#26, customer_id#50, customer_preferred_cust_flag#51, year_total#52] - -(39) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [8]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60] -Condition : (isnotnull(c_customer_sk#53) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#54, 16)))) - -(41) CometProject -Input [8]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60] -Arguments: [c_customer_sk#53, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66], [c_customer_sk#53, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#54, 16)) AS c_customer_id#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#55, 20)) AS c_first_name#62, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#56, 30)) AS c_last_name#63, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#57, 1)) AS c_preferred_cust_flag#64, c_birth_country#58, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#59, 13)) AS c_login#65, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#60, 50)) AS c_email_address#66] - -(42) CometColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#53, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66] - -(43) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#70), dynamicpruningexpression(ws_sold_date_sk#70 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(44) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] - -(45) Filter [codegen id : 8] -Input [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] -Condition : isnotnull(ws_bill_customer_sk#67) - -(46) BroadcastExchange -Input [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -(47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#53] -Right keys [1]: [ws_bill_customer_sk#67] -Join type: Inner -Join condition: None - -(48) Project [codegen id : 10] -Output [10]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] -Input [12]: [c_customer_sk#53, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] - -(49) ReusedExchange [Reuses operator id: 84] -Output [2]: [d_date_sk#71, d_year#72] - -(50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#70] -Right keys [1]: [d_date_sk#71] -Join type: Inner -Join condition: None - -(51) Project [codegen id : 10] -Output [10]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, d_year#72] -Input [12]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70, d_date_sk#71, d_year#72] - -(52) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, d_year#72] -Keys [8]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#72] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#69 - ws_ext_discount_amt#68)))] -Aggregate Attributes [1]: [sum#73] -Results [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#72, sum#74] - -(53) CometColumnarExchange -Input [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#72, sum#74] -Arguments: hashpartitioning(c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#72, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(54) CometColumnarToRow [codegen id : 11] -Input [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#72, sum#74] - -(55) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#72, sum#74] -Keys [8]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#72] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#69 - ws_ext_discount_amt#68)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#69 - ws_ext_discount_amt#68)))#75] -Results [2]: [c_customer_id#61 AS customer_id#76, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#69 - ws_ext_discount_amt#68)))#75,18,2) AS year_total#77] - -(56) Filter [codegen id : 11] -Input [2]: [customer_id#76, year_total#77] -Condition : (isnotnull(year_total#77) AND (year_total#77 > 0.00)) - -(57) BroadcastExchange -Input [2]: [customer_id#76, year_total#77] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(58) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#76] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 16] -Output [5]: [customer_id#25, year_total#26, customer_preferred_cust_flag#51, year_total#52, year_total#77] -Input [6]: [customer_id#25, year_total#26, customer_preferred_cust_flag#51, year_total#52, customer_id#76, year_total#77] - -(60) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(61) CometFilter -Input [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] -Condition : (isnotnull(c_customer_sk#78) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#79, 16)))) - -(62) CometProject -Input [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] -Arguments: [c_customer_sk#78, c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91], [c_customer_sk#78, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#79, 16)) AS c_customer_id#86, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#80, 20)) AS c_first_name#87, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#81, 30)) AS c_last_name#88, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#82, 1)) AS c_preferred_cust_flag#89, c_birth_country#83, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#84, 13)) AS c_login#90, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#85, 50)) AS c_email_address#91] - -(63) CometColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#78, c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91] - -(64) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#92, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#95), dynamicpruningexpression(ws_sold_date_sk#95 IN dynamicpruning#45)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(65) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#92, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] - -(66) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#92, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] -Condition : isnotnull(ws_bill_customer_sk#92) - -(67) BroadcastExchange -Input [4]: [ws_bill_customer_sk#92, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(68) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#78] -Right keys [1]: [ws_bill_customer_sk#92] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 14] -Output [10]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] -Input [12]: [c_customer_sk#78, c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_bill_customer_sk#92, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] - -(70) ReusedExchange [Reuses operator id: 88] -Output [2]: [d_date_sk#96, d_year#97] - -(71) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#95] -Right keys [1]: [d_date_sk#96] -Join type: Inner -Join condition: None - -(72) Project [codegen id : 14] -Output [10]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_ext_discount_amt#93, ws_ext_list_price#94, d_year#97] -Input [12]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95, d_date_sk#96, d_year#97] - -(73) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_ext_discount_amt#93, ws_ext_list_price#94, d_year#97] -Keys [8]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#97] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#94 - ws_ext_discount_amt#93)))] -Aggregate Attributes [1]: [sum#98] -Results [9]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#97, sum#99] - -(74) CometColumnarExchange -Input [9]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#97, sum#99] -Arguments: hashpartitioning(c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#97, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(75) CometColumnarToRow [codegen id : 15] -Input [9]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#97, sum#99] - -(76) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#97, sum#99] -Keys [8]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#97] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#94 - ws_ext_discount_amt#93)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#94 - ws_ext_discount_amt#93)))#75] -Results [2]: [c_customer_id#86 AS customer_id#100, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#94 - ws_ext_discount_amt#93)))#75,18,2) AS year_total#101] - -(77) BroadcastExchange -Input [2]: [customer_id#100, year_total#101] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] - -(78) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#100] -Join type: Inner -Join condition: (CASE WHEN (year_total#77 > 0.00) THEN (year_total#101 / year_total#77) END > CASE WHEN (year_total#26 > 0.00) THEN (year_total#52 / year_total#26) END) - -(79) Project [codegen id : 16] -Output [1]: [customer_preferred_cust_flag#51] -Input [7]: [customer_id#25, year_total#26, customer_preferred_cust_flag#51, year_total#52, year_total#77, customer_id#100, year_total#101] - -(80) TakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#51] -Arguments: 100, [customer_preferred_cust_flag#51 ASC NULLS FIRST], [customer_preferred_cust_flag#51] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (84) -+- * CometColumnarToRow (83) - +- CometFilter (82) - +- CometNativeScan parquet spark_catalog.default.date_dim (81) - - -(81) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(82) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(83) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#20, d_year#21] - -(84) BroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#44 IN dynamicpruning#45 -BroadcastExchange (88) -+- * CometColumnarToRow (87) - +- CometFilter (86) - +- CometNativeScan parquet spark_catalog.default.date_dim (85) - - -(85) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#46, d_year#47] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(86) CometFilter -Input [2]: [d_date_sk#46, d_year#47] -Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2002)) AND isnotnull(d_date_sk#46)) - -(87) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#46, d_year#47] - -(88) BroadcastExchange -Input [2]: [d_date_sk#46, d_year#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] - -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#70 IN dynamicpruning#19 - -Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#95 IN dynamicpruning#45 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/simplified.txt deleted file mode 100644 index 6c9e276c01..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/simplified.txt +++ /dev/null @@ -1,131 +0,0 @@ -TakeOrderedAndProject [customer_preferred_cust_flag] - WholeStageCodegen (16) - Project [customer_preferred_cust_flag] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - Project [customer_id,year_total,customer_preferred_cust_flag,year_total] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_preferred_cust_flag,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - WholeStageCodegen (6) - HashAggregate [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] [sum,sum] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) - HashAggregate [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] [sum,sum] - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) - HashAggregate [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] [sum,sum] - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/explain.txt deleted file mode 100644 index b6edfa8800..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,475 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (73) -+- CometTakeOrderedAndProject (72) - +- CometProject (71) - +- CometBroadcastHashJoin (70) - :- CometProject (54) - : +- CometBroadcastHashJoin (53) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (52) - : +- CometFilter (51) - : +- CometHashAggregate (50) - : +- CometExchange (49) - : +- CometHashAggregate (48) - : +- CometProject (47) - : +- CometBroadcastHashJoin (46) - : :- CometProject (44) - : : +- CometBroadcastHashJoin (43) - : : :- CometProject (39) - : : : +- CometFilter (38) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) - : : +- CometBroadcastExchange (42) - : : +- CometFilter (41) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) - : +- ReusedExchange (45) - +- CometBroadcastExchange (69) - +- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometProject (65) - +- CometBroadcastHashJoin (64) - :- CometProject (62) - : +- CometBroadcastHashJoin (61) - : :- CometProject (57) - : : +- CometFilter (56) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (55) - : +- CometBroadcastExchange (60) - : +- CometFilter (59) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (58) - +- ReusedExchange (63) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)) AS c_customer_id#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#3, 20)) AS c_first_name#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#4, 30)) AS c_last_name#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#5, 1)) AS c_preferred_cust_flag#12, c_birth_country#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#7, 13)) AS c_login#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#8, 50)) AS c_email_address#14] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#15) - -(6) CometBroadcastExchange -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(7) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_sk#1], [ss_customer_sk#15], Inner, BuildRight - -(8) CometProject -Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: [d_date_sk#20, d_year#21] - -(12) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Right output [2]: [d_date_sk#20, d_year#21] -Arguments: [ss_sold_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(13) CometProject -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] - -(14) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(15) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(17) CometFilter -Input [2]: [customer_id#23, year_total#24] -Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#26, 16)))) - -(20) CometProject -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Arguments: [c_customer_sk#25, c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38], [c_customer_sk#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#26, 16)) AS c_customer_id#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#27, 20)) AS c_first_name#34, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#28, 30)) AS c_last_name#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#29, 1)) AS c_preferred_cust_flag#36, c_birth_country#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#31, 13)) AS c_login#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#32, 50)) AS c_email_address#38] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#42), dynamicpruningexpression(ss_sold_date_sk#42 IN dynamicpruning#43)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Condition : isnotnull(ss_customer_sk#39) - -(23) CometBroadcastExchange -Input [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Arguments: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] - -(24) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#25, c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] -Right output [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Arguments: [c_customer_sk#25], [ss_customer_sk#39], Inner, BuildRight - -(25) CometProject -Input [12]: [c_customer_sk#25, c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Arguments: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42], [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] - -(29) CometBroadcastHashJoin -Left output [10]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [ss_sold_date_sk#42], [d_date_sk#44], Inner, BuildRight - -(30) CometProject -Input [12]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42, d_date_sk#44, d_year#45] -Arguments: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45], [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45] - -(31) CometHashAggregate -Input [10]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45] -Keys [8]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#41 - ss_ext_discount_amt#40)))] - -(32) CometExchange -Input [9]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, sum#46] -Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [9]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, sum#46] -Keys [8]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#41 - ss_ext_discount_amt#40)))] - -(34) CometBroadcastExchange -Input [3]: [customer_id#47, customer_preferred_cust_flag#48, year_total#49] -Arguments: [customer_id#47, customer_preferred_cust_flag#48, year_total#49] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#23, year_total#24] -Right output [3]: [customer_id#47, customer_preferred_cust_flag#48, year_total#49] -Arguments: [customer_id#23], [customer_id#47], Inner, BuildRight - -(36) CometProject -Input [5]: [customer_id#23, year_total#24, customer_id#47, customer_preferred_cust_flag#48, year_total#49] -Arguments: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49], [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#50, c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(38) CometFilter -Input [8]: [c_customer_sk#50, c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57] -Condition : (isnotnull(c_customer_sk#50) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#51, 16)))) - -(39) CometProject -Input [8]: [c_customer_sk#50, c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57] -Arguments: [c_customer_sk#50, c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63], [c_customer_sk#50, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#51, 16)) AS c_customer_id#58, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#52, 20)) AS c_first_name#59, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#53, 30)) AS c_last_name#60, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#54, 1)) AS c_preferred_cust_flag#61, c_birth_country#55, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#56, 13)) AS c_login#62, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#57, 50)) AS c_email_address#63] - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#67), dynamicpruningexpression(ws_sold_date_sk#67 IN dynamicpruning#68)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(41) CometFilter -Input [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] -Condition : isnotnull(ws_bill_customer_sk#64) - -(42) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] -Arguments: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] - -(43) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#50, c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63] -Right output [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] -Arguments: [c_customer_sk#50], [ws_bill_customer_sk#64], Inner, BuildRight - -(44) CometProject -Input [12]: [c_customer_sk#50, c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] -Arguments: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67], [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] - -(45) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#69, d_year#70] - -(46) CometBroadcastHashJoin -Left output [10]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] -Right output [2]: [d_date_sk#69, d_year#70] -Arguments: [ws_sold_date_sk#67], [d_date_sk#69], Inner, BuildRight - -(47) CometProject -Input [12]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67, d_date_sk#69, d_year#70] -Arguments: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#70], [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#70] - -(48) CometHashAggregate -Input [10]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#70] -Keys [8]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, d_year#70] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#66 - ws_ext_discount_amt#65)))] - -(49) CometExchange -Input [9]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, d_year#70, sum#71] -Arguments: hashpartitioning(c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, d_year#70, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(50) CometHashAggregate -Input [9]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, d_year#70, sum#71] -Keys [8]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, d_year#70] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#66 - ws_ext_discount_amt#65)))] - -(51) CometFilter -Input [2]: [customer_id#72, year_total#73] -Condition : (isnotnull(year_total#73) AND (year_total#73 > 0.00)) - -(52) CometBroadcastExchange -Input [2]: [customer_id#72, year_total#73] -Arguments: [customer_id#72, year_total#73] - -(53) CometBroadcastHashJoin -Left output [4]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49] -Right output [2]: [customer_id#72, year_total#73] -Arguments: [customer_id#23], [customer_id#72], Inner, BuildRight - -(54) CometProject -Input [6]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, customer_id#72, year_total#73] -Arguments: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73], [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(56) CometFilter -Input [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] -Condition : (isnotnull(c_customer_sk#74) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#75, 16)))) - -(57) CometProject -Input [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] -Arguments: [c_customer_sk#74, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87], [c_customer_sk#74, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#75, 16)) AS c_customer_id#82, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#76, 20)) AS c_first_name#83, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#77, 30)) AS c_last_name#84, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#78, 1)) AS c_preferred_cust_flag#85, c_birth_country#79, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#80, 13)) AS c_login#86, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#81, 50)) AS c_email_address#87] - -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#91), dynamicpruningexpression(ws_sold_date_sk#91 IN dynamicpruning#92)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(59) CometFilter -Input [4]: [ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] -Condition : isnotnull(ws_bill_customer_sk#88) - -(60) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] -Arguments: [ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] - -(61) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#74, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87] -Right output [4]: [ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] -Arguments: [c_customer_sk#74], [ws_bill_customer_sk#88], Inner, BuildRight - -(62) CometProject -Input [12]: [c_customer_sk#74, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] -Arguments: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91], [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] - -(63) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#93, d_year#94] - -(64) CometBroadcastHashJoin -Left output [10]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] -Right output [2]: [d_date_sk#93, d_year#94] -Arguments: [ws_sold_date_sk#91], [d_date_sk#93], Inner, BuildRight - -(65) CometProject -Input [12]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91, d_date_sk#93, d_year#94] -Arguments: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, d_year#94], [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, d_year#94] - -(66) CometHashAggregate -Input [10]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, d_year#94] -Keys [8]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, d_year#94] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#90 - ws_ext_discount_amt#89)))] - -(67) CometExchange -Input [9]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, d_year#94, sum#95] -Arguments: hashpartitioning(c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, d_year#94, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(68) CometHashAggregate -Input [9]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, d_year#94, sum#95] -Keys [8]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, d_year#94] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#90 - ws_ext_discount_amt#89)))] - -(69) CometBroadcastExchange -Input [2]: [customer_id#96, year_total#97] -Arguments: [customer_id#96, year_total#97] - -(70) CometBroadcastHashJoin -Left output [5]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73] -Right output [2]: [customer_id#96, year_total#97] -Arguments: [customer_id#23], [customer_id#96], Inner, (CASE WHEN (year_total#73 > 0.00) THEN (year_total#97 / year_total#73) END > CASE WHEN (year_total#24 > 0.00) THEN (year_total#49 / year_total#24) END), BuildRight - -(71) CometProject -Input [7]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73, customer_id#96, year_total#97] -Arguments: [customer_preferred_cust_flag#48], [customer_preferred_cust_flag#48] - -(72) CometTakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#48] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_preferred_cust_flag#48 ASC NULLS FIRST], output=[customer_preferred_cust_flag#48]), [customer_preferred_cust_flag#48], 100, 0, [customer_preferred_cust_flag#48 ASC NULLS FIRST], [customer_preferred_cust_flag#48] - -(73) CometColumnarToRow [codegen id : 1] -Input [1]: [customer_preferred_cust_flag#48] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (77) -+- * CometColumnarToRow (76) - +- CometFilter (75) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (74) - - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(75) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(76) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#20, d_year#21] - -(77) BroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#42 IN dynamicpruning#43 -BroadcastExchange (81) -+- * CometColumnarToRow (80) - +- CometFilter (79) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (78) - - -(78) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(79) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) - -(80) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#44, d_year#45] - -(81) BroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#67 IN dynamicpruning#19 - -Subquery:4 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#91 IN dynamicpruning#43 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/simplified.txt deleted file mode 100644 index 4599ecf539..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,91 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_preferred_cust_flag] - CometProject [customer_preferred_cust_flag] - CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_preferred_cust_flag,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_preferred_cust_flag,year_total] #5 - CometHashAggregate [c_first_name,c_last_name,d_year,c_birth_country,c_login,c_email_address,sum] [customer_id,customer_preferred_cust_flag,year_total,c_customer_id,c_preferred_cust_flag,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/explain.txt deleted file mode 100644 index b6edfa8800..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/explain.txt +++ /dev/null @@ -1,475 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (73) -+- CometTakeOrderedAndProject (72) - +- CometProject (71) - +- CometBroadcastHashJoin (70) - :- CometProject (54) - : +- CometBroadcastHashJoin (53) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (52) - : +- CometFilter (51) - : +- CometHashAggregate (50) - : +- CometExchange (49) - : +- CometHashAggregate (48) - : +- CometProject (47) - : +- CometBroadcastHashJoin (46) - : :- CometProject (44) - : : +- CometBroadcastHashJoin (43) - : : :- CometProject (39) - : : : +- CometFilter (38) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) - : : +- CometBroadcastExchange (42) - : : +- CometFilter (41) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) - : +- ReusedExchange (45) - +- CometBroadcastExchange (69) - +- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometProject (65) - +- CometBroadcastHashJoin (64) - :- CometProject (62) - : +- CometBroadcastHashJoin (61) - : :- CometProject (57) - : : +- CometFilter (56) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (55) - : +- CometBroadcastExchange (60) - : +- CometFilter (59) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (58) - +- ReusedExchange (63) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)) AS c_customer_id#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#3, 20)) AS c_first_name#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#4, 30)) AS c_last_name#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#5, 1)) AS c_preferred_cust_flag#12, c_birth_country#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#7, 13)) AS c_login#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#8, 50)) AS c_email_address#14] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#15) - -(6) CometBroadcastExchange -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(7) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_sk#1], [ss_customer_sk#15], Inner, BuildRight - -(8) CometProject -Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: [d_date_sk#20, d_year#21] - -(12) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Right output [2]: [d_date_sk#20, d_year#21] -Arguments: [ss_sold_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(13) CometProject -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] - -(14) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(15) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(17) CometFilter -Input [2]: [customer_id#23, year_total#24] -Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#26, 16)))) - -(20) CometProject -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Arguments: [c_customer_sk#25, c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38], [c_customer_sk#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#26, 16)) AS c_customer_id#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#27, 20)) AS c_first_name#34, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#28, 30)) AS c_last_name#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#29, 1)) AS c_preferred_cust_flag#36, c_birth_country#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#31, 13)) AS c_login#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#32, 50)) AS c_email_address#38] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#42), dynamicpruningexpression(ss_sold_date_sk#42 IN dynamicpruning#43)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Condition : isnotnull(ss_customer_sk#39) - -(23) CometBroadcastExchange -Input [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Arguments: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] - -(24) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#25, c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] -Right output [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Arguments: [c_customer_sk#25], [ss_customer_sk#39], Inner, BuildRight - -(25) CometProject -Input [12]: [c_customer_sk#25, c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Arguments: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42], [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] - -(29) CometBroadcastHashJoin -Left output [10]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [ss_sold_date_sk#42], [d_date_sk#44], Inner, BuildRight - -(30) CometProject -Input [12]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42, d_date_sk#44, d_year#45] -Arguments: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45], [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45] - -(31) CometHashAggregate -Input [10]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45] -Keys [8]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#41 - ss_ext_discount_amt#40)))] - -(32) CometExchange -Input [9]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, sum#46] -Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [9]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, sum#46] -Keys [8]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#41 - ss_ext_discount_amt#40)))] - -(34) CometBroadcastExchange -Input [3]: [customer_id#47, customer_preferred_cust_flag#48, year_total#49] -Arguments: [customer_id#47, customer_preferred_cust_flag#48, year_total#49] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#23, year_total#24] -Right output [3]: [customer_id#47, customer_preferred_cust_flag#48, year_total#49] -Arguments: [customer_id#23], [customer_id#47], Inner, BuildRight - -(36) CometProject -Input [5]: [customer_id#23, year_total#24, customer_id#47, customer_preferred_cust_flag#48, year_total#49] -Arguments: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49], [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#50, c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(38) CometFilter -Input [8]: [c_customer_sk#50, c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57] -Condition : (isnotnull(c_customer_sk#50) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#51, 16)))) - -(39) CometProject -Input [8]: [c_customer_sk#50, c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57] -Arguments: [c_customer_sk#50, c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63], [c_customer_sk#50, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#51, 16)) AS c_customer_id#58, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#52, 20)) AS c_first_name#59, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#53, 30)) AS c_last_name#60, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#54, 1)) AS c_preferred_cust_flag#61, c_birth_country#55, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#56, 13)) AS c_login#62, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#57, 50)) AS c_email_address#63] - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#67), dynamicpruningexpression(ws_sold_date_sk#67 IN dynamicpruning#68)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(41) CometFilter -Input [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] -Condition : isnotnull(ws_bill_customer_sk#64) - -(42) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] -Arguments: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] - -(43) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#50, c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63] -Right output [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] -Arguments: [c_customer_sk#50], [ws_bill_customer_sk#64], Inner, BuildRight - -(44) CometProject -Input [12]: [c_customer_sk#50, c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] -Arguments: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67], [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] - -(45) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#69, d_year#70] - -(46) CometBroadcastHashJoin -Left output [10]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] -Right output [2]: [d_date_sk#69, d_year#70] -Arguments: [ws_sold_date_sk#67], [d_date_sk#69], Inner, BuildRight - -(47) CometProject -Input [12]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67, d_date_sk#69, d_year#70] -Arguments: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#70], [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#70] - -(48) CometHashAggregate -Input [10]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#70] -Keys [8]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, d_year#70] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#66 - ws_ext_discount_amt#65)))] - -(49) CometExchange -Input [9]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, d_year#70, sum#71] -Arguments: hashpartitioning(c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, d_year#70, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(50) CometHashAggregate -Input [9]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, d_year#70, sum#71] -Keys [8]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, d_year#70] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#66 - ws_ext_discount_amt#65)))] - -(51) CometFilter -Input [2]: [customer_id#72, year_total#73] -Condition : (isnotnull(year_total#73) AND (year_total#73 > 0.00)) - -(52) CometBroadcastExchange -Input [2]: [customer_id#72, year_total#73] -Arguments: [customer_id#72, year_total#73] - -(53) CometBroadcastHashJoin -Left output [4]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49] -Right output [2]: [customer_id#72, year_total#73] -Arguments: [customer_id#23], [customer_id#72], Inner, BuildRight - -(54) CometProject -Input [6]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, customer_id#72, year_total#73] -Arguments: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73], [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(56) CometFilter -Input [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] -Condition : (isnotnull(c_customer_sk#74) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#75, 16)))) - -(57) CometProject -Input [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] -Arguments: [c_customer_sk#74, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87], [c_customer_sk#74, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#75, 16)) AS c_customer_id#82, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#76, 20)) AS c_first_name#83, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#77, 30)) AS c_last_name#84, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#78, 1)) AS c_preferred_cust_flag#85, c_birth_country#79, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#80, 13)) AS c_login#86, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#81, 50)) AS c_email_address#87] - -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#91), dynamicpruningexpression(ws_sold_date_sk#91 IN dynamicpruning#92)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(59) CometFilter -Input [4]: [ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] -Condition : isnotnull(ws_bill_customer_sk#88) - -(60) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] -Arguments: [ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] - -(61) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#74, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87] -Right output [4]: [ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] -Arguments: [c_customer_sk#74], [ws_bill_customer_sk#88], Inner, BuildRight - -(62) CometProject -Input [12]: [c_customer_sk#74, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] -Arguments: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91], [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] - -(63) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#93, d_year#94] - -(64) CometBroadcastHashJoin -Left output [10]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] -Right output [2]: [d_date_sk#93, d_year#94] -Arguments: [ws_sold_date_sk#91], [d_date_sk#93], Inner, BuildRight - -(65) CometProject -Input [12]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91, d_date_sk#93, d_year#94] -Arguments: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, d_year#94], [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, d_year#94] - -(66) CometHashAggregate -Input [10]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, d_year#94] -Keys [8]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, d_year#94] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#90 - ws_ext_discount_amt#89)))] - -(67) CometExchange -Input [9]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, d_year#94, sum#95] -Arguments: hashpartitioning(c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, d_year#94, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(68) CometHashAggregate -Input [9]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, d_year#94, sum#95] -Keys [8]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, d_year#94] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#90 - ws_ext_discount_amt#89)))] - -(69) CometBroadcastExchange -Input [2]: [customer_id#96, year_total#97] -Arguments: [customer_id#96, year_total#97] - -(70) CometBroadcastHashJoin -Left output [5]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73] -Right output [2]: [customer_id#96, year_total#97] -Arguments: [customer_id#23], [customer_id#96], Inner, (CASE WHEN (year_total#73 > 0.00) THEN (year_total#97 / year_total#73) END > CASE WHEN (year_total#24 > 0.00) THEN (year_total#49 / year_total#24) END), BuildRight - -(71) CometProject -Input [7]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73, customer_id#96, year_total#97] -Arguments: [customer_preferred_cust_flag#48], [customer_preferred_cust_flag#48] - -(72) CometTakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#48] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_preferred_cust_flag#48 ASC NULLS FIRST], output=[customer_preferred_cust_flag#48]), [customer_preferred_cust_flag#48], 100, 0, [customer_preferred_cust_flag#48 ASC NULLS FIRST], [customer_preferred_cust_flag#48] - -(73) CometColumnarToRow [codegen id : 1] -Input [1]: [customer_preferred_cust_flag#48] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (77) -+- * CometColumnarToRow (76) - +- CometFilter (75) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (74) - - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(75) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(76) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#20, d_year#21] - -(77) BroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#42 IN dynamicpruning#43 -BroadcastExchange (81) -+- * CometColumnarToRow (80) - +- CometFilter (79) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (78) - - -(78) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(79) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) - -(80) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#44, d_year#45] - -(81) BroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#67 IN dynamicpruning#19 - -Subquery:4 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#91 IN dynamicpruning#43 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/extended.txt deleted file mode 100644 index 781eae9054..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/extended.txt +++ /dev/null @@ -1,91 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 80 out of 86 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 4599ecf539..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt +++ /dev/null @@ -1,91 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_preferred_cust_flag] - CometProject [customer_preferred_cust_flag] - CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_preferred_cust_flag,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_preferred_cust_flag,year_total] #5 - CometHashAggregate [c_first_name,c_last_name,d_year,c_birth_country,c_login,c_email_address,sum] [customer_id,customer_preferred_cust_flag,year_total,c_customer_id,c_preferred_cust_flag,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/explain.txt deleted file mode 100644 index 409078c363..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/explain.txt +++ /dev/null @@ -1,163 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.web_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.item (4) - +- ReusedExchange (11) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] - -(7) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(8) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(11) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] - -(14) HashAggregate [codegen id : 3] -Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(15) CometColumnarExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(17) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#17] -Results [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#17,17,2) AS _w0#19, i_item_id#11] - -(18) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(22) Project [codegen id : 6] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] - -(23) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometNativeScan parquet spark_catalog.default.date_dim (24) - - -(24) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#22] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/simplified.txt deleted file mode 100644 index b7b0a89774..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/explain.txt deleted file mode 100644 index 8811d04e59..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ws_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/simplified.txt deleted file mode 100644 index fb83fd2f9a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt deleted file mode 100644 index 8811d04e59..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ws_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/extended.txt deleted file mode 100644 index 3f41c97ff5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index fb83fd2f9a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/explain.txt deleted file mode 100644 index 613d0cb7b2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/explain.txt +++ /dev/null @@ -1,241 +0,0 @@ -== Physical Plan == -* HashAggregate (36) -+- * CometColumnarToRow (35) - +- CometColumnarExchange (34) - +- * HashAggregate (33) - +- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * CometColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.store (4) - : : : +- BroadcastExchange (14) - : : : +- * CometColumnarToRow (13) - : : : +- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (10) - : : +- ReusedExchange (17) - : +- BroadcastExchange (24) - : +- * CometColumnarToRow (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometNativeScan parquet spark_catalog.default.customer_demographics (20) - +- BroadcastExchange (30) - +- * CometColumnarToRow (29) - +- CometFilter (28) - +- CometNativeScan parquet spark_catalog.default.household_demographics (27) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00))), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00)))] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] - -(3) Filter [codegen id : 6] -Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Condition : (((((isnotnull(ss_store_sk#4) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_hdemo_sk#2)) AND ((((ss_net_profit#9 >= 100.00) AND (ss_net_profit#9 <= 200.00)) OR ((ss_net_profit#9 >= 150.00) AND (ss_net_profit#9 <= 300.00))) OR ((ss_net_profit#9 >= 50.00) AND (ss_net_profit#9 <= 250.00)))) AND ((((ss_sales_price#6 >= 100.00) AND (ss_sales_price#6 <= 150.00)) OR ((ss_sales_price#6 >= 50.00) AND (ss_sales_price#6 <= 100.00))) OR ((ss_sales_price#6 >= 150.00) AND (ss_sales_price#6 <= 200.00)))) - -(4) CometNativeScan parquet spark_catalog.default.store -Output [1]: [s_store_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [1]: [s_store_sk#12] -Condition : isnotnull(s_store_sk#12) - -(6) CometColumnarToRow [codegen id : 1] -Input [1]: [s_store_sk#12] - -(7) BroadcastExchange -Input [1]: [s_store_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#12] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 6] -Output [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, s_store_sk#12] - -(10) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Condition : (((isnotnull(ca_country#15) AND (ca_country#15 = United States)) AND isnotnull(ca_address_sk#13)) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#14, 2)) IN (TX,OH) OR static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#14, 2)) IN (OR,NM,KY)) OR static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#14, 2)) IN (VA,TX,MS))) - -(12) CometProject -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Arguments: [ca_address_sk#13, ca_state#16], [ca_address_sk#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#14, 2)) AS ca_state#16] - -(13) CometColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#13, ca_state#16] - -(14) BroadcastExchange -Input [2]: [ca_address_sk#13, ca_state#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#13] -Join type: Inner -Join condition: ((((ca_state#16 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#16 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#16 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))) - -(16) Project [codegen id : 6] -Output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] -Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#13, ca_state#16] - -(17) ReusedExchange [Reuses operator id: 41] -Output [1]: [d_date_sk#17] - -(18) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#10] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 6] -Output [6]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10, d_date_sk#17] - -(20) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] -Condition : (isnotnull(cd_demo_sk#18) AND ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#19, 1)) = M) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#20, 20)) = Advanced Degree )) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#19, 1)) = S) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#20, 20)) = College ))) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#19, 1)) = W) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#20, 20)) = 2 yr Degree )))) - -(22) CometProject -Input [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] -Arguments: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22], [cd_demo_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#19, 1)) AS cd_marital_status#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#20, 20)) AS cd_education_status#22] - -(23) CometColumnarToRow [codegen id : 4] -Input [3]: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] - -(24) BroadcastExchange -Input [3]: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_cdemo_sk#1] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: ((((((cd_marital_status#21 = M) AND (cd_education_status#22 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#21 = S) AND (cd_education_status#22 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#21 = W) AND (cd_education_status#22 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) - -(26) Project [codegen id : 6] -Output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#21, cd_education_status#22] -Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] - -(27) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#23, hd_dep_count#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), Or(EqualTo(hd_dep_count,3),EqualTo(hd_dep_count,1))] -ReadSchema: struct - -(28) CometFilter -Input [2]: [hd_demo_sk#23, hd_dep_count#24] -Condition : (isnotnull(hd_demo_sk#23) AND ((hd_dep_count#24 = 3) OR (hd_dep_count#24 = 1))) - -(29) CometColumnarToRow [codegen id : 5] -Input [2]: [hd_demo_sk#23, hd_dep_count#24] - -(30) BroadcastExchange -Input [2]: [hd_demo_sk#23, hd_dep_count#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#23] -Join type: Inner -Join condition: (((((((cd_marital_status#21 = M) AND (cd_education_status#22 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#24 = 3)) OR (((((cd_marital_status#21 = S) AND (cd_education_status#22 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#24 = 1))) OR (((((cd_marital_status#21 = W) AND (cd_education_status#22 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#24 = 1))) - -(32) Project [codegen id : 6] -Output [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#21, cd_education_status#22, hd_demo_sk#23, hd_dep_count#24] - -(33) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Keys: [] -Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sales_price#7)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#8)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#8))] -Aggregate Attributes [7]: [sum#25, count#26, sum#27, count#28, sum#29, count#30, sum#31] -Results [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] - -(34) CometColumnarExchange -Input [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(35) CometColumnarToRow [codegen id : 7] -Input [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] - -(36) HashAggregate [codegen id : 7] -Input [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] -Keys: [] -Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] -Aggregate Attributes [4]: [avg(ss_quantity#5)#39, avg(UnscaledValue(ss_ext_sales_price#7))#40, avg(UnscaledValue(ss_ext_wholesale_cost#8))#41, sum(UnscaledValue(ss_ext_wholesale_cost#8))#42] -Results [4]: [avg(ss_quantity#5)#39 AS avg(ss_quantity)#43, cast((avg(UnscaledValue(ss_ext_sales_price#7))#40 / 100.0) as decimal(11,6)) AS avg(ss_ext_sales_price)#44, cast((avg(UnscaledValue(ss_ext_wholesale_cost#8))#41 / 100.0) as decimal(11,6)) AS avg(ss_ext_wholesale_cost)#45, MakeDecimal(sum(UnscaledValue(ss_ext_wholesale_cost#8))#42,17,2) AS sum(ss_ext_wholesale_cost)#46] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (41) -+- * CometColumnarToRow (40) - +- CometProject (39) - +- CometFilter (38) - +- CometNativeScan parquet spark_catalog.default.date_dim (37) - - -(37) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_year#47] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(38) CometFilter -Input [2]: [d_date_sk#17, d_year#47] -Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2001)) AND isnotnull(d_date_sk#17)) - -(39) CometProject -Input [2]: [d_date_sk#17, d_year#47] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(40) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] - -(41) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/simplified.txt deleted file mode 100644 index a33ae5a161..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (7) - HashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost)),avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (6) - HashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum,sum,count,sum,count,sum,count,sum] - Project [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] - Project [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] - Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] - Project [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] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometFilter [hd_demo_sk,hd_dep_count] - CometNativeScan 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/q13.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/explain.txt deleted file mode 100644 index 027d679419..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,231 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (34) -+- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (7) - : : : : +- CometBroadcastHashJoin (6) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (20) - +- CometBroadcastExchange (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00))), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00)))] -ReadSchema: struct - -(2) CometFilter -Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Condition : (((((isnotnull(ss_store_sk#4) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_hdemo_sk#2)) AND ((((ss_net_profit#9 >= 100.00) AND (ss_net_profit#9 <= 200.00)) OR ((ss_net_profit#9 >= 150.00) AND (ss_net_profit#9 <= 300.00))) OR ((ss_net_profit#9 >= 50.00) AND (ss_net_profit#9 <= 250.00)))) AND ((((ss_sales_price#6 >= 100.00) AND (ss_sales_price#6 <= 150.00)) OR ((ss_sales_price#6 >= 50.00) AND (ss_sales_price#6 <= 100.00))) OR ((ss_sales_price#6 >= 150.00) AND (ss_sales_price#6 <= 200.00)))) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [s_store_sk#12] -Condition : isnotnull(s_store_sk#12) - -(5) CometBroadcastExchange -Input [1]: [s_store_sk#12] -Arguments: [s_store_sk#12] - -(6) CometBroadcastHashJoin -Left output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Right output [1]: [s_store_sk#12] -Arguments: [ss_store_sk#4], [s_store_sk#12], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, s_store_sk#12] -Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Condition : (((isnotnull(ca_country#15) AND (ca_country#15 = United States)) AND isnotnull(ca_address_sk#13)) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#14, 2)) IN (TX,OH) OR static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#14, 2)) IN (OR,NM,KY)) OR static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#14, 2)) IN (VA,TX,MS))) - -(10) CometProject -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Arguments: [ca_address_sk#13, ca_state#16], [ca_address_sk#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#14, 2)) AS ca_state#16] - -(11) CometBroadcastExchange -Input [2]: [ca_address_sk#13, ca_state#16] -Arguments: [ca_address_sk#13, ca_state#16] - -(12) CometBroadcastHashJoin -Left output [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Right output [2]: [ca_address_sk#13, ca_state#16] -Arguments: [ss_addr_sk#3], [ca_address_sk#13], Inner, ((((ca_state#16 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#16 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#16 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))), BuildRight - -(13) CometProject -Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#13, ca_state#16] -Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_year#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#17, d_year#18] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(16) CometProject -Input [2]: [d_date_sk#17, d_year#18] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: [d_date_sk#17] - -(18) CometBroadcastHashJoin -Left output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] -Right output [1]: [d_date_sk#17] -Arguments: [ss_sold_date_sk#10], [d_date_sk#17], Inner, BuildRight - -(19) CometProject -Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10, d_date_sk#17] -Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Condition : (isnotnull(cd_demo_sk#19) AND ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) = M) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) = Advanced Degree )) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) = S) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) = College ))) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) = W) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) = 2 yr Degree )))) - -(22) CometProject -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) AS cd_marital_status#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) AS cd_education_status#23] - -(23) CometBroadcastExchange -Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] - -(24) CometBroadcastHashJoin -Left output [6]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Right output [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [ss_cdemo_sk#1], [cd_demo_sk#19], Inner, ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))), BuildRight - -(25) CometProject -Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23], [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#24, hd_dep_count#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), Or(EqualTo(hd_dep_count,3),EqualTo(hd_dep_count,1))] -ReadSchema: struct - -(27) CometFilter -Input [2]: [hd_demo_sk#24, hd_dep_count#25] -Condition : (isnotnull(hd_demo_sk#24) AND ((hd_dep_count#25 = 3) OR (hd_dep_count#25 = 1))) - -(28) CometBroadcastExchange -Input [2]: [hd_demo_sk#24, hd_dep_count#25] -Arguments: [hd_demo_sk#24, hd_dep_count#25] - -(29) CometBroadcastHashJoin -Left output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23] -Right output [2]: [hd_demo_sk#24, hd_dep_count#25] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#24], Inner, (((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#25 = 3)) OR (((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#25 = 1))) OR (((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#25 = 1))), BuildRight - -(30) CometProject -Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23, hd_demo_sk#24, hd_dep_count#25] -Arguments: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] - -(31) CometHashAggregate -Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Keys: [] -Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sales_price#7)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#8)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#8))] - -(32) CometExchange -Input [7]: [sum#26, count#27, sum#28, count#29, sum#30, count#31, sum#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(33) CometHashAggregate -Input [7]: [sum#26, count#27, sum#28, count#29, sum#30, count#31, sum#32] -Keys: [] -Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] - -(34) CometColumnarToRow [codegen id : 1] -Input [4]: [avg(ss_quantity)#33, avg(ss_ext_sales_price)#34, avg(ss_ext_wholesale_cost)#35, sum(ss_ext_wholesale_cost)#36] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (39) -+- * CometColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (35) - - -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_year#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(36) CometFilter -Input [2]: [d_date_sk#17, d_year#18] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(37) CometProject -Input [2]: [d_date_sk#17, d_year#18] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(38) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] - -(39) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/simplified.txt deleted file mode 100644 index 8ef882a435..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,44 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost))] - CometExchange #1 - CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum] - CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - 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,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_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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk] #3 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [hd_demo_sk,hd_dep_count] #7 - CometFilter [hd_demo_sk,hd_dep_count] - CometScan [native_iceberg_compat] 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/q13/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/explain.txt deleted file mode 100644 index 027d679419..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/explain.txt +++ /dev/null @@ -1,231 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (34) -+- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (7) - : : : : +- CometBroadcastHashJoin (6) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (20) - +- CometBroadcastExchange (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00))), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00)))] -ReadSchema: struct - -(2) CometFilter -Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Condition : (((((isnotnull(ss_store_sk#4) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_hdemo_sk#2)) AND ((((ss_net_profit#9 >= 100.00) AND (ss_net_profit#9 <= 200.00)) OR ((ss_net_profit#9 >= 150.00) AND (ss_net_profit#9 <= 300.00))) OR ((ss_net_profit#9 >= 50.00) AND (ss_net_profit#9 <= 250.00)))) AND ((((ss_sales_price#6 >= 100.00) AND (ss_sales_price#6 <= 150.00)) OR ((ss_sales_price#6 >= 50.00) AND (ss_sales_price#6 <= 100.00))) OR ((ss_sales_price#6 >= 150.00) AND (ss_sales_price#6 <= 200.00)))) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [s_store_sk#12] -Condition : isnotnull(s_store_sk#12) - -(5) CometBroadcastExchange -Input [1]: [s_store_sk#12] -Arguments: [s_store_sk#12] - -(6) CometBroadcastHashJoin -Left output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Right output [1]: [s_store_sk#12] -Arguments: [ss_store_sk#4], [s_store_sk#12], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, s_store_sk#12] -Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Condition : (((isnotnull(ca_country#15) AND (ca_country#15 = United States)) AND isnotnull(ca_address_sk#13)) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#14, 2)) IN (TX,OH) OR static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#14, 2)) IN (OR,NM,KY)) OR static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#14, 2)) IN (VA,TX,MS))) - -(10) CometProject -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Arguments: [ca_address_sk#13, ca_state#16], [ca_address_sk#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#14, 2)) AS ca_state#16] - -(11) CometBroadcastExchange -Input [2]: [ca_address_sk#13, ca_state#16] -Arguments: [ca_address_sk#13, ca_state#16] - -(12) CometBroadcastHashJoin -Left output [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Right output [2]: [ca_address_sk#13, ca_state#16] -Arguments: [ss_addr_sk#3], [ca_address_sk#13], Inner, ((((ca_state#16 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#16 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#16 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))), BuildRight - -(13) CometProject -Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#13, ca_state#16] -Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_year#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#17, d_year#18] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(16) CometProject -Input [2]: [d_date_sk#17, d_year#18] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: [d_date_sk#17] - -(18) CometBroadcastHashJoin -Left output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] -Right output [1]: [d_date_sk#17] -Arguments: [ss_sold_date_sk#10], [d_date_sk#17], Inner, BuildRight - -(19) CometProject -Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10, d_date_sk#17] -Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Condition : (isnotnull(cd_demo_sk#19) AND ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) = M) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) = Advanced Degree )) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) = S) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) = College ))) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) = W) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) = 2 yr Degree )))) - -(22) CometProject -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) AS cd_marital_status#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) AS cd_education_status#23] - -(23) CometBroadcastExchange -Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] - -(24) CometBroadcastHashJoin -Left output [6]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Right output [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [ss_cdemo_sk#1], [cd_demo_sk#19], Inner, ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))), BuildRight - -(25) CometProject -Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23], [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#24, hd_dep_count#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), Or(EqualTo(hd_dep_count,3),EqualTo(hd_dep_count,1))] -ReadSchema: struct - -(27) CometFilter -Input [2]: [hd_demo_sk#24, hd_dep_count#25] -Condition : (isnotnull(hd_demo_sk#24) AND ((hd_dep_count#25 = 3) OR (hd_dep_count#25 = 1))) - -(28) CometBroadcastExchange -Input [2]: [hd_demo_sk#24, hd_dep_count#25] -Arguments: [hd_demo_sk#24, hd_dep_count#25] - -(29) CometBroadcastHashJoin -Left output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23] -Right output [2]: [hd_demo_sk#24, hd_dep_count#25] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#24], Inner, (((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#25 = 3)) OR (((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#25 = 1))) OR (((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#25 = 1))), BuildRight - -(30) CometProject -Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23, hd_demo_sk#24, hd_dep_count#25] -Arguments: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] - -(31) CometHashAggregate -Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Keys: [] -Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sales_price#7)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#8)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#8))] - -(32) CometExchange -Input [7]: [sum#26, count#27, sum#28, count#29, sum#30, count#31, sum#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(33) CometHashAggregate -Input [7]: [sum#26, count#27, sum#28, count#29, sum#30, count#31, sum#32] -Keys: [] -Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] - -(34) CometColumnarToRow [codegen id : 1] -Input [4]: [avg(ss_quantity)#33, avg(ss_ext_sales_price)#34, avg(ss_ext_wholesale_cost)#35, sum(ss_ext_wholesale_cost)#36] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (39) -+- * CometColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (35) - - -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_year#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(36) CometFilter -Input [2]: [d_date_sk#17, d_year#18] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(37) CometProject -Input [2]: [d_date_sk#17, d_year#18] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(38) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] - -(39) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/extended.txt deleted file mode 100644 index 08e9beb692..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - -Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 8ef882a435..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt +++ /dev/null @@ -1,44 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost))] - CometExchange #1 - CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum] - CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - 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,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_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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk] #3 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [hd_demo_sk,hd_dep_count] #7 - CometFilter [hd_demo_sk,hd_dep_count] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/explain.txt deleted file mode 100644 index f7377ccf8a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/explain.txt +++ /dev/null @@ -1,822 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (110) -+- * HashAggregate (109) - +- * CometColumnarToRow (108) - +- CometColumnarExchange (107) - +- * HashAggregate (106) - +- * Expand (105) - +- Union (104) - :- * Project (69) - : +- * Filter (68) - : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) - : +- CometColumnarExchange (65) - : +- * HashAggregate (64) - : +- * Project (63) - : +- * BroadcastHashJoin Inner BuildRight (62) - : :- * Project (60) - : : +- * BroadcastHashJoin Inner BuildRight (59) - : : :- * BroadcastHashJoin LeftSemi BuildRight (52) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * CometColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (48) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * CometColumnarToRow (36) - : : : : +- CometHashAggregate (35) - : : : : +- CometColumnarExchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * CometColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * CometColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * Filter (39) - : : : : : +- * ColumnarToRow (38) - : : : : : +- Scan parquet spark_catalog.default.web_sales (37) - : : : : +- ReusedExchange (40) - : : : +- ReusedExchange (43) - : : +- BroadcastExchange (58) - : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : :- * CometColumnarToRow (55) - : : : +- CometFilter (54) - : : : +- CometNativeScan parquet spark_catalog.default.item (53) - : : +- ReusedExchange (56) - : +- ReusedExchange (61) - :- * Project (86) - : +- * Filter (85) - : +- * HashAggregate (84) - : +- * CometColumnarToRow (83) - : +- CometColumnarExchange (82) - : +- * HashAggregate (81) - : +- * Project (80) - : +- * BroadcastHashJoin Inner BuildRight (79) - : :- * Project (77) - : : +- * BroadcastHashJoin Inner BuildRight (76) - : : :- * BroadcastHashJoin LeftSemi BuildRight (74) - : : : :- * Filter (72) - : : : : +- * ColumnarToRow (71) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (70) - : : : +- ReusedExchange (73) - : : +- ReusedExchange (75) - : +- ReusedExchange (78) - +- * Project (103) - +- * Filter (102) - +- * HashAggregate (101) - +- * CometColumnarToRow (100) - +- CometColumnarExchange (99) - +- * HashAggregate (98) - +- * Project (97) - +- * BroadcastHashJoin Inner BuildRight (96) - :- * Project (94) - : +- * BroadcastHashJoin Inner BuildRight (93) - : :- * BroadcastHashJoin LeftSemi BuildRight (91) - : : :- * Filter (89) - : : : +- * ColumnarToRow (88) - : : : +- Scan parquet spark_catalog.default.web_sales (87) - : : +- ReusedExchange (90) - : +- ReusedExchange (92) - +- ReusedExchange (95) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(6) CometColumnarToRow [codegen id : 11] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(10) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(11) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(12) CometColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(16) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) - -(18) CometColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(19) BroadcastExchange -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#19] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(22) ReusedExchange [Reuses operator id: 140] -Output [1]: [d_date_sk#23] - -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#23] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#23] - -(25) BroadcastExchange -Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] -Join type: LeftSemi -Join condition: None - -(27) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None - -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(30) ReusedExchange [Reuses operator id: 140] -Output [1]: [d_date_sk#24] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#25, i_class_id#15 AS class_id#26, i_category_id#16 AS category_id#27] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#24] - -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#25, class_id#26, category_id#27] - -(34) CometColumnarExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(35) CometHashAggregate -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] - -(36) CometColumnarToRow [codegen id : 10] -Input [3]: [brand_id#25, class_id#26, category_id#27] - -(37) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] - -(39) Filter [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Condition : isnotnull(ws_item_sk#28) - -(40) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#28] -Right keys [1]: [i_item_sk#30] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] -Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(43) ReusedExchange [Reuses operator id: 140] -Output [1]: [d_date_sk#34] - -(44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#29] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 9] -Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] - -(46) BroadcastExchange -Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] - -(47) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] -Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] -Join type: LeftSemi -Join condition: None - -(48) BroadcastExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] - -(49) BroadcastHashJoin [codegen id : 11] -Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#25, class_id#26, category_id#27] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#35] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] - -(51) BroadcastExchange -Input [1]: [ss_item_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(52) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(53) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(54) CometFilter -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Condition : isnotnull(i_item_sk#36) - -(55) CometColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(56) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#35] - -(57) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#36] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(58) BroadcastExchange -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(59) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#36] -Join type: Inner -Join condition: None - -(60) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(61) ReusedExchange [Reuses operator id: 135] -Output [1]: [d_date_sk#40] - -(62) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] - -(64) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] -Results [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(65) CometColumnarExchange -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Arguments: hashpartitioning(i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(66) CometColumnarToRow [codegen id : 26] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(67) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] -Results [5]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#49, count(1)#48 AS number_sales#50] - -(68) Filter [codegen id : 26] -Input [5]: [i_brand_id#37, i_class_id#38, i_category_id#39, sales#49, number_sales#50] -Condition : (isnotnull(sales#49) AND (cast(sales#49 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(69) Project [codegen id : 26] -Output [6]: [sales#49, number_sales#50, store AS channel#53, i_brand_id#37 AS i_brand_id#54, i_class_id#38 AS i_class_id#55, i_category_id#39 AS i_category_id#56] -Input [5]: [i_brand_id#37, i_class_id#38, i_category_id#39, sales#49, number_sales#50] - -(70) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(71) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] - -(72) Filter [codegen id : 51] -Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Condition : isnotnull(cs_item_sk#57) - -(73) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#61] - -(74) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#57] -Right keys [1]: [ss_item_sk#61] -Join type: LeftSemi -Join condition: None - -(75) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] - -(76) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#57] -Right keys [1]: [i_item_sk#62] -Join type: Inner -Join condition: None - -(77) Project [codegen id : 51] -Output [6]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65] -Input [8]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] - -(78) ReusedExchange [Reuses operator id: 135] -Output [1]: [d_date_sk#66] - -(79) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#60] -Right keys [1]: [d_date_sk#66] -Join type: Inner -Join condition: None - -(80) Project [codegen id : 51] -Output [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] -Input [7]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] - -(81) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [partial_sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), partial_count(1)] -Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] -Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] - -(82) CometColumnarExchange -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(83) CometColumnarToRow [codegen id : 52] -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] - -(84) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), count(1)] -Aggregate Attributes [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#73, count(1)#74] -Results [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#73 AS sales#75, count(1)#74 AS number_sales#76] - -(85) Filter [codegen id : 52] -Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#75, number_sales#76] -Condition : (isnotnull(sales#75) AND (cast(sales#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(86) Project [codegen id : 52] -Output [6]: [sales#75, number_sales#76, catalog AS channel#77, i_brand_id#63, i_class_id#64, i_category_id#65] -Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#75, number_sales#76] - -(87) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(88) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] - -(89) Filter [codegen id : 77] -Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_item_sk#78) - -(90) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#82] - -(91) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#78] -Right keys [1]: [ss_item_sk#82] -Join type: LeftSemi -Join condition: None - -(92) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#83, i_brand_id#84, i_class_id#85, i_category_id#86] - -(93) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#78] -Right keys [1]: [i_item_sk#83] -Join type: Inner -Join condition: None - -(94) Project [codegen id : 77] -Output [6]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#84, i_class_id#85, i_category_id#86] -Input [8]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_item_sk#83, i_brand_id#84, i_class_id#85, i_category_id#86] - -(95) ReusedExchange [Reuses operator id: 135] -Output [1]: [d_date_sk#87] - -(96) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#81] -Right keys [1]: [d_date_sk#87] -Join type: Inner -Join condition: None - -(97) Project [codegen id : 77] -Output [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#84, i_class_id#85, i_category_id#86] -Input [7]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#84, i_class_id#85, i_category_id#86, d_date_sk#87] - -(98) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#84, i_class_id#85, i_category_id#86] -Keys [3]: [i_brand_id#84, i_class_id#85, i_category_id#86] -Functions [2]: [partial_sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), partial_count(1)] -Aggregate Attributes [3]: [sum#88, isEmpty#89, count#90] -Results [6]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum#91, isEmpty#92, count#93] - -(99) CometColumnarExchange -Input [6]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum#91, isEmpty#92, count#93] -Arguments: hashpartitioning(i_brand_id#84, i_class_id#85, i_category_id#86, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(100) CometColumnarToRow [codegen id : 78] -Input [6]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum#91, isEmpty#92, count#93] - -(101) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum#91, isEmpty#92, count#93] -Keys [3]: [i_brand_id#84, i_class_id#85, i_category_id#86] -Functions [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), count(1)] -Aggregate Attributes [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#94, count(1)#95] -Results [5]: [i_brand_id#84, i_class_id#85, i_category_id#86, sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#94 AS sales#96, count(1)#95 AS number_sales#97] - -(102) Filter [codegen id : 78] -Input [5]: [i_brand_id#84, i_class_id#85, i_category_id#86, sales#96, number_sales#97] -Condition : (isnotnull(sales#96) AND (cast(sales#96 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(103) Project [codegen id : 78] -Output [6]: [sales#96, number_sales#97, web AS channel#98, i_brand_id#84, i_class_id#85, i_category_id#86] -Input [5]: [i_brand_id#84, i_class_id#85, i_category_id#86, sales#96, number_sales#97] - -(104) Union - -(105) Expand [codegen id : 79] -Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: [[sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, null, 1], [sales#49, number_sales#50, channel#53, i_brand_id#54, null, null, 3], [sales#49, number_sales#50, channel#53, null, null, null, 7], [sales#49, number_sales#50, null, null, null, null, 15]], [sales#49, number_sales#50, channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103] - -(106) HashAggregate [codegen id : 79] -Input [7]: [sales#49, number_sales#50, channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103] -Keys [5]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103] -Functions [2]: [partial_sum(sales#49), partial_sum(number_sales#50)] -Aggregate Attributes [3]: [sum#104, isEmpty#105, sum#106] -Results [8]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103, sum#107, isEmpty#108, sum#109] - -(107) CometColumnarExchange -Input [8]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103, sum#107, isEmpty#108, sum#109] -Arguments: hashpartitioning(channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(108) CometColumnarToRow [codegen id : 80] -Input [8]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103, sum#107, isEmpty#108, sum#109] - -(109) HashAggregate [codegen id : 80] -Input [8]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103, sum#107, isEmpty#108, sum#109] -Keys [5]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, spark_grouping_id#103] -Functions [2]: [sum(sales#49), sum(number_sales#50)] -Aggregate Attributes [2]: [sum(sales#49)#110, sum(number_sales#50)#111] -Results [6]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, sum(sales#49)#110 AS sum(sales)#112, sum(number_sales#50)#111 AS sum(number_sales)#113] - -(110) TakeOrderedAndProject -Input [6]: [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, sum(sales)#112, sum(number_sales)#113] -Arguments: 100, [channel#99 ASC NULLS FIRST, i_brand_id#100 ASC NULLS FIRST, i_class_id#101 ASC NULLS FIRST, i_category_id#102 ASC NULLS FIRST], [channel#99, i_brand_id#100, i_class_id#101, i_category_id#102, sum(sales)#112, sum(number_sales)#113] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* HashAggregate (130) -+- * CometColumnarToRow (129) - +- CometColumnarExchange (128) - +- * HashAggregate (127) - +- Union (126) - :- * Project (115) - : +- * BroadcastHashJoin Inner BuildRight (114) - : :- * ColumnarToRow (112) - : : +- Scan parquet spark_catalog.default.store_sales (111) - : +- ReusedExchange (113) - :- * Project (120) - : +- * BroadcastHashJoin Inner BuildRight (119) - : :- * ColumnarToRow (117) - : : +- Scan parquet spark_catalog.default.catalog_sales (116) - : +- ReusedExchange (118) - +- * Project (125) - +- * BroadcastHashJoin Inner BuildRight (124) - :- * ColumnarToRow (122) - : +- Scan parquet spark_catalog.default.web_sales (121) - +- ReusedExchange (123) - - -(111) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#114, ss_list_price#115, ss_sold_date_sk#116] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#116), dynamicpruningexpression(ss_sold_date_sk#116 IN dynamicpruning#12)] -ReadSchema: struct - -(112) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#114, ss_list_price#115, ss_sold_date_sk#116] - -(113) ReusedExchange [Reuses operator id: 140] -Output [1]: [d_date_sk#117] - -(114) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#116] -Right keys [1]: [d_date_sk#117] -Join type: Inner -Join condition: None - -(115) Project [codegen id : 2] -Output [2]: [ss_quantity#114 AS quantity#118, ss_list_price#115 AS list_price#119] -Input [4]: [ss_quantity#114, ss_list_price#115, ss_sold_date_sk#116, d_date_sk#117] - -(116) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#120, cs_list_price#121, cs_sold_date_sk#122] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#122), dynamicpruningexpression(cs_sold_date_sk#122 IN dynamicpruning#12)] -ReadSchema: struct - -(117) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#120, cs_list_price#121, cs_sold_date_sk#122] - -(118) ReusedExchange [Reuses operator id: 140] -Output [1]: [d_date_sk#123] - -(119) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#122] -Right keys [1]: [d_date_sk#123] -Join type: Inner -Join condition: None - -(120) Project [codegen id : 4] -Output [2]: [cs_quantity#120 AS quantity#124, cs_list_price#121 AS list_price#125] -Input [4]: [cs_quantity#120, cs_list_price#121, cs_sold_date_sk#122, d_date_sk#123] - -(121) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#126, ws_list_price#127, ws_sold_date_sk#128] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#128), dynamicpruningexpression(ws_sold_date_sk#128 IN dynamicpruning#12)] -ReadSchema: struct - -(122) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#126, ws_list_price#127, ws_sold_date_sk#128] - -(123) ReusedExchange [Reuses operator id: 140] -Output [1]: [d_date_sk#129] - -(124) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#128] -Right keys [1]: [d_date_sk#129] -Join type: Inner -Join condition: None - -(125) Project [codegen id : 6] -Output [2]: [ws_quantity#126 AS quantity#130, ws_list_price#127 AS list_price#131] -Input [4]: [ws_quantity#126, ws_list_price#127, ws_sold_date_sk#128, d_date_sk#129] - -(126) Union - -(127) HashAggregate [codegen id : 7] -Input [2]: [quantity#118, list_price#119] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#118 as decimal(10,0)) * list_price#119))] -Aggregate Attributes [2]: [sum#132, count#133] -Results [2]: [sum#134, count#135] - -(128) CometColumnarExchange -Input [2]: [sum#134, count#135] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(129) CometColumnarToRow [codegen id : 8] -Input [2]: [sum#134, count#135] - -(130) HashAggregate [codegen id : 8] -Input [2]: [sum#134, count#135] -Keys: [] -Functions [1]: [avg((cast(quantity#118 as decimal(10,0)) * list_price#119))] -Aggregate Attributes [1]: [avg((cast(quantity#118 as decimal(10,0)) * list_price#119))#136] -Results [1]: [avg((cast(quantity#118 as decimal(10,0)) * list_price#119))#136 AS average_sales#137] - -Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#116 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 116 Hosting Expression = cs_sold_date_sk#122 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 121 Hosting Expression = ws_sold_date_sk#128 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (135) -+- * CometColumnarToRow (134) - +- CometProject (133) - +- CometFilter (132) - +- CometNativeScan parquet spark_catalog.default.date_dim (131) - - -(131) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#40, d_year#138, d_moy#139] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(132) CometFilter -Input [3]: [d_date_sk#40, d_year#138, d_moy#139] -Condition : ((((isnotnull(d_year#138) AND isnotnull(d_moy#139)) AND (d_year#138 = 2001)) AND (d_moy#139 = 11)) AND isnotnull(d_date_sk#40)) - -(133) CometProject -Input [3]: [d_date_sk#40, d_year#138, d_moy#139] -Arguments: [d_date_sk#40], [d_date_sk#40] - -(134) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#40] - -(135) BroadcastExchange -Input [1]: [d_date_sk#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (140) -+- * CometColumnarToRow (139) - +- CometProject (138) - +- CometFilter (137) - +- CometNativeScan parquet spark_catalog.default.date_dim (136) - - -(136) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#140] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(137) CometFilter -Input [2]: [d_date_sk#24, d_year#140] -Condition : (((isnotnull(d_year#140) AND (d_year#140 >= 1999)) AND (d_year#140 <= 2001)) AND isnotnull(d_date_sk#24)) - -(138) CometProject -Input [2]: [d_date_sk#24, d_year#140] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(139) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#24] - -(140) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] - -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:8 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 85 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] - -Subquery:10 Hosting operator id = 70 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 - -Subquery:11 Hosting operator id = 102 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] - -Subquery:12 Hosting operator id = 87 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/simplified.txt deleted file mode 100644 index c989fe9a81..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/simplified.txt +++ /dev/null @@ -1,220 +0,0 @@ -TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - WholeStageCodegen (80) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - WholeStageCodegen (79) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - InputAdapter - Union - WholeStageCodegen (26) - Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] - Filter [sales] - Subquery #3 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #13 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #7 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #7 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #7 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (52) - Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (51) - HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (78) - Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (77) - HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/explain.txt deleted file mode 100644 index 22f1896b63..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,754 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (102) -+- CometTakeOrderedAndProject (101) - +- CometHashAggregate (100) - +- CometExchange (99) - +- CometHashAggregate (98) - +- CometExpand (97) - +- CometUnion (96) - :- CometProject (65) - : +- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometBroadcastHashJoin (47) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (46) - : : : +- CometProject (45) - : : : +- CometBroadcastHashJoin (44) - : : : :- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (43) - : : : +- CometBroadcastHashJoin (42) - : : : :- CometHashAggregate (32) - : : : : +- CometExchange (31) - : : : : +- CometHashAggregate (30) - : : : : +- CometProject (29) - : : : : +- CometBroadcastHashJoin (28) - : : : : :- CometProject (26) - : : : : : +- CometBroadcastHashJoin (25) - : : : : : :- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : +- CometBroadcastExchange (24) - : : : : : +- CometBroadcastHashJoin (23) - : : : : : :- CometFilter (8) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : +- CometBroadcastExchange (22) - : : : : : +- CometProject (21) - : : : : : +- CometBroadcastHashJoin (20) - : : : : : :- CometProject (15) - : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : :- CometFilter (10) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : +- CometBroadcastExchange (13) - : : : : : : +- CometFilter (12) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : +- CometBroadcastExchange (19) - : : : : : +- CometProject (18) - : : : : : +- CometFilter (17) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : +- ReusedExchange (27) - : : : +- CometBroadcastExchange (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (34) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometFilter (49) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : +- ReusedExchange (50) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - :- CometProject (80) - : +- CometFilter (79) - : +- CometHashAggregate (78) - : +- CometExchange (77) - : +- CometHashAggregate (76) - : +- CometProject (75) - : +- CometBroadcastHashJoin (74) - : :- CometProject (72) - : : +- CometBroadcastHashJoin (71) - : : :- CometBroadcastHashJoin (69) - : : : :- CometFilter (67) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (66) - : : : +- ReusedExchange (68) - : : +- ReusedExchange (70) - : +- ReusedExchange (73) - +- CometProject (95) - +- CometFilter (94) - +- CometHashAggregate (93) - +- CometExchange (92) - +- CometHashAggregate (91) - +- CometProject (90) - +- CometBroadcastHashJoin (89) - :- CometProject (87) - : +- CometBroadcastHashJoin (86) - : :- CometBroadcastHashJoin (84) - : : :- CometFilter (82) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (81) - : : +- ReusedExchange (83) - : +- ReusedExchange (85) - +- ReusedExchange (88) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : isnotnull(i_item_sk#39) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2001)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#49, number_sales#50] -Condition : (isnotnull(sales#49) AND (cast(sales#49 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(65) CometProject -Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#49, number_sales#50] -Arguments: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56], [sales#49, number_sales#50, store AS channel#53, i_brand_id#40 AS i_brand_id#54, i_class_id#41 AS i_class_id#55, i_category_id#42 AS i_category_id#56] - -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(67) CometFilter -Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Condition : isnotnull(cs_item_sk#57) - -(68) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#62] - -(69) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Right output [1]: [ss_item_sk#62] -Arguments: [cs_item_sk#57], [ss_item_sk#62], LeftSemi, BuildRight - -(70) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66] - -(71) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Right output [4]: [i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66] -Arguments: [cs_item_sk#57], [i_item_sk#63], Inner, BuildRight - -(72) CometProject -Input [8]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66] -Arguments: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66], [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66] - -(73) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#67] - -(74) CometBroadcastHashJoin -Left output [6]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66] -Right output [1]: [d_date_sk#67] -Arguments: [cs_sold_date_sk#60], [d_date_sk#67], Inner, BuildRight - -(75) CometProject -Input [7]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66, d_date_sk#67] -Arguments: [cs_quantity#58, cs_list_price#59, i_brand_id#64, i_class_id#65, i_category_id#66], [cs_quantity#58, cs_list_price#59, i_brand_id#64, i_class_id#65, i_category_id#66] - -(76) CometHashAggregate -Input [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#64, i_class_id#65, i_category_id#66] -Keys [3]: [i_brand_id#64, i_class_id#65, i_category_id#66] -Functions [2]: [partial_sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), partial_count(1)] - -(77) CometExchange -Input [6]: [i_brand_id#64, i_class_id#65, i_category_id#66, sum#68, isEmpty#69, count#70] -Arguments: hashpartitioning(i_brand_id#64, i_class_id#65, i_category_id#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(78) CometHashAggregate -Input [6]: [i_brand_id#64, i_class_id#65, i_category_id#66, sum#68, isEmpty#69, count#70] -Keys [3]: [i_brand_id#64, i_class_id#65, i_category_id#66] -Functions [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), count(1)] - -(79) CometFilter -Input [5]: [i_brand_id#64, i_class_id#65, i_category_id#66, sales#71, number_sales#72] -Condition : (isnotnull(sales#71) AND (cast(sales#71 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(80) CometProject -Input [5]: [i_brand_id#64, i_class_id#65, i_category_id#66, sales#71, number_sales#72] -Arguments: [sales#71, number_sales#72, channel#73, i_brand_id#64, i_class_id#65, i_category_id#66], [sales#71, number_sales#72, catalog AS channel#73, i_brand_id#64, i_class_id#65, i_category_id#66] - -(81) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#77), dynamicpruningexpression(ws_sold_date_sk#77 IN dynamicpruning#78)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(82) CometFilter -Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] -Condition : isnotnull(ws_item_sk#74) - -(83) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#79] - -(84) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] -Right output [1]: [ss_item_sk#79] -Arguments: [ws_item_sk#74], [ss_item_sk#79], LeftSemi, BuildRight - -(85) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] - -(86) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] -Right output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] -Arguments: [ws_item_sk#74], [i_item_sk#80], Inner, BuildRight - -(87) CometProject -Input [8]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] -Arguments: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#81, i_class_id#82, i_category_id#83], [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#81, i_class_id#82, i_category_id#83] - -(88) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#84] - -(89) CometBroadcastHashJoin -Left output [6]: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#81, i_class_id#82, i_category_id#83] -Right output [1]: [d_date_sk#84] -Arguments: [ws_sold_date_sk#77], [d_date_sk#84], Inner, BuildRight - -(90) CometProject -Input [7]: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#81, i_class_id#82, i_category_id#83, d_date_sk#84] -Arguments: [ws_quantity#75, ws_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83], [ws_quantity#75, ws_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] - -(91) CometHashAggregate -Input [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [partial_sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), partial_count(1)] - -(92) CometExchange -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#85, isEmpty#86, count#87] -Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(93) CometHashAggregate -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#85, isEmpty#86, count#87] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), count(1)] - -(94) CometFilter -Input [5]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#88, number_sales#89] -Condition : (isnotnull(sales#88) AND (cast(sales#88 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(95) CometProject -Input [5]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#88, number_sales#89] -Arguments: [sales#88, number_sales#89, channel#90, i_brand_id#81, i_class_id#82, i_category_id#83], [sales#88, number_sales#89, web AS channel#90, i_brand_id#81, i_class_id#82, i_category_id#83] - -(96) CometUnion -Child 0 Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Child 1 Input [6]: [sales#71, number_sales#72, channel#73, i_brand_id#64, i_class_id#65, i_category_id#66] -Child 2 Input [6]: [sales#88, number_sales#89, channel#90, i_brand_id#81, i_class_id#82, i_category_id#83] - -(97) CometExpand -Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: [[sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, null, 1], [sales#49, number_sales#50, channel#53, i_brand_id#54, null, null, 3], [sales#49, number_sales#50, channel#53, null, null, null, 7], [sales#49, number_sales#50, null, null, null, null, 15]], [sales#49, number_sales#50, channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95] - -(98) CometHashAggregate -Input [7]: [sales#49, number_sales#50, channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95] -Keys [5]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95] -Functions [2]: [partial_sum(sales#49), partial_sum(number_sales#50)] - -(99) CometExchange -Input [8]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95, sum#96, isEmpty#97, sum#98] -Arguments: hashpartitioning(channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(100) CometHashAggregate -Input [8]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95, sum#96, isEmpty#97, sum#98] -Keys [5]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95] -Functions [2]: [sum(sales#49), sum(number_sales#50)] - -(101) CometTakeOrderedAndProject -Input [6]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#91 ASC NULLS FIRST,i_brand_id#92 ASC NULLS FIRST,i_class_id#93 ASC NULLS FIRST,i_category_id#94 ASC NULLS FIRST], output=[channel#91,i_brand_id#92,i_class_id#93,i_category_id#94,sum(sales)#99,sum(number_sales)#100]), [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100], 100, 0, [channel#91 ASC NULLS FIRST, i_brand_id#92 ASC NULLS FIRST, i_class_id#93 ASC NULLS FIRST, i_category_id#94 ASC NULLS FIRST], [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100] - -(102) CometColumnarToRow [codegen id : 1] -Input [6]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* CometColumnarToRow (119) -+- CometHashAggregate (118) - +- CometExchange (117) - +- CometHashAggregate (116) - +- CometUnion (115) - :- CometProject (106) - : +- CometBroadcastHashJoin (105) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (103) - : +- ReusedExchange (104) - :- CometProject (110) - : +- CometBroadcastHashJoin (109) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (107) - : +- ReusedExchange (108) - +- CometProject (114) - +- CometBroadcastHashJoin (113) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (111) - +- ReusedExchange (112) - - -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#101, ss_list_price#102, ss_sold_date_sk#103] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#103), dynamicpruningexpression(ss_sold_date_sk#103 IN dynamicpruning#104)] -ReadSchema: struct - -(104) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#105] - -(105) CometBroadcastHashJoin -Left output [3]: [ss_quantity#101, ss_list_price#102, ss_sold_date_sk#103] -Right output [1]: [d_date_sk#105] -Arguments: [ss_sold_date_sk#103], [d_date_sk#105], Inner, BuildRight - -(106) CometProject -Input [4]: [ss_quantity#101, ss_list_price#102, ss_sold_date_sk#103, d_date_sk#105] -Arguments: [quantity#106, list_price#107], [ss_quantity#101 AS quantity#106, ss_list_price#102 AS list_price#107] - -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#108, cs_list_price#109, cs_sold_date_sk#110] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#110), dynamicpruningexpression(cs_sold_date_sk#110 IN dynamicpruning#111)] -ReadSchema: struct - -(108) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#112] - -(109) CometBroadcastHashJoin -Left output [3]: [cs_quantity#108, cs_list_price#109, cs_sold_date_sk#110] -Right output [1]: [d_date_sk#112] -Arguments: [cs_sold_date_sk#110], [d_date_sk#112], Inner, BuildRight - -(110) CometProject -Input [4]: [cs_quantity#108, cs_list_price#109, cs_sold_date_sk#110, d_date_sk#112] -Arguments: [quantity#113, list_price#114], [cs_quantity#108 AS quantity#113, cs_list_price#109 AS list_price#114] - -(111) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#115, ws_list_price#116, ws_sold_date_sk#117] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#117), dynamicpruningexpression(ws_sold_date_sk#117 IN dynamicpruning#118)] -ReadSchema: struct - -(112) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#119] - -(113) CometBroadcastHashJoin -Left output [3]: [ws_quantity#115, ws_list_price#116, ws_sold_date_sk#117] -Right output [1]: [d_date_sk#119] -Arguments: [ws_sold_date_sk#117], [d_date_sk#119], Inner, BuildRight - -(114) CometProject -Input [4]: [ws_quantity#115, ws_list_price#116, ws_sold_date_sk#117, d_date_sk#119] -Arguments: [quantity#120, list_price#121], [ws_quantity#115 AS quantity#120, ws_list_price#116 AS list_price#121] - -(115) CometUnion -Child 0 Input [2]: [quantity#106, list_price#107] -Child 1 Input [2]: [quantity#113, list_price#114] -Child 2 Input [2]: [quantity#120, list_price#121] - -(116) CometHashAggregate -Input [2]: [quantity#106, list_price#107] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#106 as decimal(10,0)) * list_price#107))] - -(117) CometExchange -Input [2]: [sum#122, count#123] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(118) CometHashAggregate -Input [2]: [sum#122, count#123] -Keys: [] -Functions [1]: [avg((cast(quantity#106 as decimal(10,0)) * list_price#107))] - -(119) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#124] - -Subquery:2 Hosting operator id = 103 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 107 Hosting Expression = cs_sold_date_sk#110 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 111 Hosting Expression = ws_sold_date_sk#117 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (124) -+- * CometColumnarToRow (123) - +- CometProject (122) - +- CometFilter (121) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (120) - - -(120) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(121) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2001)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(122) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(123) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(124) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (129) -+- * CometColumnarToRow (128) - +- CometProject (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) - - -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#125] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#26, d_year#125] -Condition : (((isnotnull(d_year#125) AND (d_year#125 >= 1999)) AND (d_year#125 <= 2001)) AND isnotnull(d_date_sk#26)) - -(127) CometProject -Input [2]: [d_date_sk#26, d_year#125] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(128) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(129) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 79 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] - -Subquery:10 Hosting operator id = 66 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 - -Subquery:11 Hosting operator id = 94 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] - -Subquery:12 Hosting operator id = 81 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 79c782f2ca..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,149 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales),sum(sales),sum(number_salesL)] - CometExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - CometHashAggregate [sales,number_sales] [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] - CometExpand [channel,i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] - CometUnion [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometProject [i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #15 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #2 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk] #4 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #5 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #6 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #11 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #11 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #12 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - ReusedExchange [d_date_sk] #11 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #4 - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [cs_quantity,cs_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk] - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #4 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - ReusedExchange [d_date_sk] #14 - CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #17 - CometHashAggregate [ws_quantity,ws_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk] - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #4 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - ReusedExchange [d_date_sk] #14 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/explain.txt deleted file mode 100644 index 22f1896b63..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/explain.txt +++ /dev/null @@ -1,754 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (102) -+- CometTakeOrderedAndProject (101) - +- CometHashAggregate (100) - +- CometExchange (99) - +- CometHashAggregate (98) - +- CometExpand (97) - +- CometUnion (96) - :- CometProject (65) - : +- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometBroadcastHashJoin (47) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (46) - : : : +- CometProject (45) - : : : +- CometBroadcastHashJoin (44) - : : : :- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (43) - : : : +- CometBroadcastHashJoin (42) - : : : :- CometHashAggregate (32) - : : : : +- CometExchange (31) - : : : : +- CometHashAggregate (30) - : : : : +- CometProject (29) - : : : : +- CometBroadcastHashJoin (28) - : : : : :- CometProject (26) - : : : : : +- CometBroadcastHashJoin (25) - : : : : : :- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : +- CometBroadcastExchange (24) - : : : : : +- CometBroadcastHashJoin (23) - : : : : : :- CometFilter (8) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : +- CometBroadcastExchange (22) - : : : : : +- CometProject (21) - : : : : : +- CometBroadcastHashJoin (20) - : : : : : :- CometProject (15) - : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : :- CometFilter (10) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : +- CometBroadcastExchange (13) - : : : : : : +- CometFilter (12) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : +- CometBroadcastExchange (19) - : : : : : +- CometProject (18) - : : : : : +- CometFilter (17) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : +- ReusedExchange (27) - : : : +- CometBroadcastExchange (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (34) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometFilter (49) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : +- ReusedExchange (50) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - :- CometProject (80) - : +- CometFilter (79) - : +- CometHashAggregate (78) - : +- CometExchange (77) - : +- CometHashAggregate (76) - : +- CometProject (75) - : +- CometBroadcastHashJoin (74) - : :- CometProject (72) - : : +- CometBroadcastHashJoin (71) - : : :- CometBroadcastHashJoin (69) - : : : :- CometFilter (67) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (66) - : : : +- ReusedExchange (68) - : : +- ReusedExchange (70) - : +- ReusedExchange (73) - +- CometProject (95) - +- CometFilter (94) - +- CometHashAggregate (93) - +- CometExchange (92) - +- CometHashAggregate (91) - +- CometProject (90) - +- CometBroadcastHashJoin (89) - :- CometProject (87) - : +- CometBroadcastHashJoin (86) - : :- CometBroadcastHashJoin (84) - : : :- CometFilter (82) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (81) - : : +- ReusedExchange (83) - : +- ReusedExchange (85) - +- ReusedExchange (88) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : isnotnull(i_item_sk#39) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2001)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#49, number_sales#50] -Condition : (isnotnull(sales#49) AND (cast(sales#49 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(65) CometProject -Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#49, number_sales#50] -Arguments: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56], [sales#49, number_sales#50, store AS channel#53, i_brand_id#40 AS i_brand_id#54, i_class_id#41 AS i_class_id#55, i_category_id#42 AS i_category_id#56] - -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(67) CometFilter -Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Condition : isnotnull(cs_item_sk#57) - -(68) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#62] - -(69) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Right output [1]: [ss_item_sk#62] -Arguments: [cs_item_sk#57], [ss_item_sk#62], LeftSemi, BuildRight - -(70) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66] - -(71) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Right output [4]: [i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66] -Arguments: [cs_item_sk#57], [i_item_sk#63], Inner, BuildRight - -(72) CometProject -Input [8]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_item_sk#63, i_brand_id#64, i_class_id#65, i_category_id#66] -Arguments: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66], [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66] - -(73) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#67] - -(74) CometBroadcastHashJoin -Left output [6]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66] -Right output [1]: [d_date_sk#67] -Arguments: [cs_sold_date_sk#60], [d_date_sk#67], Inner, BuildRight - -(75) CometProject -Input [7]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#64, i_class_id#65, i_category_id#66, d_date_sk#67] -Arguments: [cs_quantity#58, cs_list_price#59, i_brand_id#64, i_class_id#65, i_category_id#66], [cs_quantity#58, cs_list_price#59, i_brand_id#64, i_class_id#65, i_category_id#66] - -(76) CometHashAggregate -Input [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#64, i_class_id#65, i_category_id#66] -Keys [3]: [i_brand_id#64, i_class_id#65, i_category_id#66] -Functions [2]: [partial_sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), partial_count(1)] - -(77) CometExchange -Input [6]: [i_brand_id#64, i_class_id#65, i_category_id#66, sum#68, isEmpty#69, count#70] -Arguments: hashpartitioning(i_brand_id#64, i_class_id#65, i_category_id#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(78) CometHashAggregate -Input [6]: [i_brand_id#64, i_class_id#65, i_category_id#66, sum#68, isEmpty#69, count#70] -Keys [3]: [i_brand_id#64, i_class_id#65, i_category_id#66] -Functions [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), count(1)] - -(79) CometFilter -Input [5]: [i_brand_id#64, i_class_id#65, i_category_id#66, sales#71, number_sales#72] -Condition : (isnotnull(sales#71) AND (cast(sales#71 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(80) CometProject -Input [5]: [i_brand_id#64, i_class_id#65, i_category_id#66, sales#71, number_sales#72] -Arguments: [sales#71, number_sales#72, channel#73, i_brand_id#64, i_class_id#65, i_category_id#66], [sales#71, number_sales#72, catalog AS channel#73, i_brand_id#64, i_class_id#65, i_category_id#66] - -(81) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#77), dynamicpruningexpression(ws_sold_date_sk#77 IN dynamicpruning#78)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(82) CometFilter -Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] -Condition : isnotnull(ws_item_sk#74) - -(83) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#79] - -(84) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] -Right output [1]: [ss_item_sk#79] -Arguments: [ws_item_sk#74], [ss_item_sk#79], LeftSemi, BuildRight - -(85) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] - -(86) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] -Right output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] -Arguments: [ws_item_sk#74], [i_item_sk#80], Inner, BuildRight - -(87) CometProject -Input [8]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] -Arguments: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#81, i_class_id#82, i_category_id#83], [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#81, i_class_id#82, i_category_id#83] - -(88) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#84] - -(89) CometBroadcastHashJoin -Left output [6]: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#81, i_class_id#82, i_category_id#83] -Right output [1]: [d_date_sk#84] -Arguments: [ws_sold_date_sk#77], [d_date_sk#84], Inner, BuildRight - -(90) CometProject -Input [7]: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#81, i_class_id#82, i_category_id#83, d_date_sk#84] -Arguments: [ws_quantity#75, ws_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83], [ws_quantity#75, ws_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] - -(91) CometHashAggregate -Input [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [partial_sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), partial_count(1)] - -(92) CometExchange -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#85, isEmpty#86, count#87] -Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(93) CometHashAggregate -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#85, isEmpty#86, count#87] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), count(1)] - -(94) CometFilter -Input [5]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#88, number_sales#89] -Condition : (isnotnull(sales#88) AND (cast(sales#88 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(95) CometProject -Input [5]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#88, number_sales#89] -Arguments: [sales#88, number_sales#89, channel#90, i_brand_id#81, i_class_id#82, i_category_id#83], [sales#88, number_sales#89, web AS channel#90, i_brand_id#81, i_class_id#82, i_category_id#83] - -(96) CometUnion -Child 0 Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Child 1 Input [6]: [sales#71, number_sales#72, channel#73, i_brand_id#64, i_class_id#65, i_category_id#66] -Child 2 Input [6]: [sales#88, number_sales#89, channel#90, i_brand_id#81, i_class_id#82, i_category_id#83] - -(97) CometExpand -Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: [[sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, null, 1], [sales#49, number_sales#50, channel#53, i_brand_id#54, null, null, 3], [sales#49, number_sales#50, channel#53, null, null, null, 7], [sales#49, number_sales#50, null, null, null, null, 15]], [sales#49, number_sales#50, channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95] - -(98) CometHashAggregate -Input [7]: [sales#49, number_sales#50, channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95] -Keys [5]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95] -Functions [2]: [partial_sum(sales#49), partial_sum(number_sales#50)] - -(99) CometExchange -Input [8]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95, sum#96, isEmpty#97, sum#98] -Arguments: hashpartitioning(channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(100) CometHashAggregate -Input [8]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95, sum#96, isEmpty#97, sum#98] -Keys [5]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, spark_grouping_id#95] -Functions [2]: [sum(sales#49), sum(number_sales#50)] - -(101) CometTakeOrderedAndProject -Input [6]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#91 ASC NULLS FIRST,i_brand_id#92 ASC NULLS FIRST,i_class_id#93 ASC NULLS FIRST,i_category_id#94 ASC NULLS FIRST], output=[channel#91,i_brand_id#92,i_class_id#93,i_category_id#94,sum(sales)#99,sum(number_sales)#100]), [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100], 100, 0, [channel#91 ASC NULLS FIRST, i_brand_id#92 ASC NULLS FIRST, i_class_id#93 ASC NULLS FIRST, i_category_id#94 ASC NULLS FIRST], [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100] - -(102) CometColumnarToRow [codegen id : 1] -Input [6]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum(sales)#99, sum(number_sales)#100] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* CometColumnarToRow (119) -+- CometHashAggregate (118) - +- CometExchange (117) - +- CometHashAggregate (116) - +- CometUnion (115) - :- CometProject (106) - : +- CometBroadcastHashJoin (105) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (103) - : +- ReusedExchange (104) - :- CometProject (110) - : +- CometBroadcastHashJoin (109) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (107) - : +- ReusedExchange (108) - +- CometProject (114) - +- CometBroadcastHashJoin (113) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (111) - +- ReusedExchange (112) - - -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#101, ss_list_price#102, ss_sold_date_sk#103] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#103), dynamicpruningexpression(ss_sold_date_sk#103 IN dynamicpruning#104)] -ReadSchema: struct - -(104) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#105] - -(105) CometBroadcastHashJoin -Left output [3]: [ss_quantity#101, ss_list_price#102, ss_sold_date_sk#103] -Right output [1]: [d_date_sk#105] -Arguments: [ss_sold_date_sk#103], [d_date_sk#105], Inner, BuildRight - -(106) CometProject -Input [4]: [ss_quantity#101, ss_list_price#102, ss_sold_date_sk#103, d_date_sk#105] -Arguments: [quantity#106, list_price#107], [ss_quantity#101 AS quantity#106, ss_list_price#102 AS list_price#107] - -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#108, cs_list_price#109, cs_sold_date_sk#110] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#110), dynamicpruningexpression(cs_sold_date_sk#110 IN dynamicpruning#111)] -ReadSchema: struct - -(108) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#112] - -(109) CometBroadcastHashJoin -Left output [3]: [cs_quantity#108, cs_list_price#109, cs_sold_date_sk#110] -Right output [1]: [d_date_sk#112] -Arguments: [cs_sold_date_sk#110], [d_date_sk#112], Inner, BuildRight - -(110) CometProject -Input [4]: [cs_quantity#108, cs_list_price#109, cs_sold_date_sk#110, d_date_sk#112] -Arguments: [quantity#113, list_price#114], [cs_quantity#108 AS quantity#113, cs_list_price#109 AS list_price#114] - -(111) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#115, ws_list_price#116, ws_sold_date_sk#117] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#117), dynamicpruningexpression(ws_sold_date_sk#117 IN dynamicpruning#118)] -ReadSchema: struct - -(112) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#119] - -(113) CometBroadcastHashJoin -Left output [3]: [ws_quantity#115, ws_list_price#116, ws_sold_date_sk#117] -Right output [1]: [d_date_sk#119] -Arguments: [ws_sold_date_sk#117], [d_date_sk#119], Inner, BuildRight - -(114) CometProject -Input [4]: [ws_quantity#115, ws_list_price#116, ws_sold_date_sk#117, d_date_sk#119] -Arguments: [quantity#120, list_price#121], [ws_quantity#115 AS quantity#120, ws_list_price#116 AS list_price#121] - -(115) CometUnion -Child 0 Input [2]: [quantity#106, list_price#107] -Child 1 Input [2]: [quantity#113, list_price#114] -Child 2 Input [2]: [quantity#120, list_price#121] - -(116) CometHashAggregate -Input [2]: [quantity#106, list_price#107] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#106 as decimal(10,0)) * list_price#107))] - -(117) CometExchange -Input [2]: [sum#122, count#123] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(118) CometHashAggregate -Input [2]: [sum#122, count#123] -Keys: [] -Functions [1]: [avg((cast(quantity#106 as decimal(10,0)) * list_price#107))] - -(119) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#124] - -Subquery:2 Hosting operator id = 103 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 107 Hosting Expression = cs_sold_date_sk#110 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 111 Hosting Expression = ws_sold_date_sk#117 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (124) -+- * CometColumnarToRow (123) - +- CometProject (122) - +- CometFilter (121) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (120) - - -(120) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(121) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2001)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(122) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(123) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(124) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (129) -+- * CometColumnarToRow (128) - +- CometProject (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) - - -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#125] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#26, d_year#125] -Condition : (((isnotnull(d_year#125) AND (d_year#125 >= 1999)) AND (d_year#125 <= 2001)) AND isnotnull(d_date_sk#26)) - -(127) CometProject -Input [2]: [d_date_sk#26, d_year#125] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(128) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(129) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 79 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] - -Subquery:10 Hosting operator id = 66 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 - -Subquery:11 Hosting operator id = 94 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] - -Subquery:12 Hosting operator id = 81 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/extended.txt deleted file mode 100644 index 4af04a7846..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/extended.txt +++ /dev/null @@ -1,469 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 424 out of 458 eligible operators (92%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 79c782f2ca..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt +++ /dev/null @@ -1,149 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales),sum(sales),sum(number_salesL)] - CometExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - CometHashAggregate [sales,number_sales] [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] - CometExpand [channel,i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] - CometUnion [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometProject [i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #15 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #2 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk] #4 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #5 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #6 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #11 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #11 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #12 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - ReusedExchange [d_date_sk] #11 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #4 - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [cs_quantity,cs_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk] - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #4 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - ReusedExchange [d_date_sk] #14 - CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #17 - CometHashAggregate [ws_quantity,ws_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk] - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #4 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - ReusedExchange [d_date_sk] #14 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/explain.txt deleted file mode 100644 index 7630396f0b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/explain.txt +++ /dev/null @@ -1,773 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (87) -+- * BroadcastHashJoin Inner BuildRight (86) - :- * Filter (68) - : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) - : +- CometColumnarExchange (65) - : +- * HashAggregate (64) - : +- * Project (63) - : +- * BroadcastHashJoin Inner BuildRight (62) - : :- * Project (60) - : : +- * BroadcastHashJoin Inner BuildRight (59) - : : :- * BroadcastHashJoin LeftSemi BuildRight (52) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * CometColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (48) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * CometColumnarToRow (36) - : : : : +- CometHashAggregate (35) - : : : : +- CometColumnarExchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * CometColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * CometColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * Filter (39) - : : : : : +- * ColumnarToRow (38) - : : : : : +- Scan parquet spark_catalog.default.web_sales (37) - : : : : +- ReusedExchange (40) - : : : +- ReusedExchange (43) - : : +- BroadcastExchange (58) - : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : :- * CometColumnarToRow (55) - : : : +- CometFilter (54) - : : : +- CometNativeScan parquet spark_catalog.default.item (53) - : : +- ReusedExchange (56) - : +- ReusedExchange (61) - +- BroadcastExchange (85) - +- * Filter (84) - +- * HashAggregate (83) - +- * CometColumnarToRow (82) - +- CometColumnarExchange (81) - +- * HashAggregate (80) - +- * Project (79) - +- * BroadcastHashJoin Inner BuildRight (78) - :- * Project (76) - : +- * BroadcastHashJoin Inner BuildRight (75) - : :- * BroadcastHashJoin LeftSemi BuildRight (73) - : : :- * Filter (71) - : : : +- * ColumnarToRow (70) - : : : +- Scan parquet spark_catalog.default.store_sales (69) - : : +- ReusedExchange (72) - : +- ReusedExchange (74) - +- ReusedExchange (77) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(6) CometColumnarToRow [codegen id : 11] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(10) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(11) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(12) CometColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(16) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) - -(18) CometColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(19) BroadcastExchange -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#19] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(22) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#23] - -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#23] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#23] - -(25) BroadcastExchange -Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] -Join type: LeftSemi -Join condition: None - -(27) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None - -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(30) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#24] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#25, i_class_id#15 AS class_id#26, i_category_id#16 AS category_id#27] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#24] - -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#25, class_id#26, category_id#27] - -(34) CometColumnarExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(35) CometHashAggregate -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] - -(36) CometColumnarToRow [codegen id : 10] -Input [3]: [brand_id#25, class_id#26, category_id#27] - -(37) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] - -(39) Filter [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Condition : isnotnull(ws_item_sk#28) - -(40) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#28] -Right keys [1]: [i_item_sk#30] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] -Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(43) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#34] - -(44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#29] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 9] -Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] - -(46) BroadcastExchange -Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] - -(47) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] -Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] -Join type: LeftSemi -Join condition: None - -(48) BroadcastExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] - -(49) BroadcastHashJoin [codegen id : 11] -Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#25, class_id#26, category_id#27] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#35] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] - -(51) BroadcastExchange -Input [1]: [ss_item_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(52) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(53) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(54) CometFilter -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Condition : (((isnotnull(i_item_sk#36) AND isnotnull(i_brand_id#37)) AND isnotnull(i_class_id#38)) AND isnotnull(i_category_id#39)) - -(55) CometColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(56) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#35] - -(57) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#36] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(58) BroadcastExchange -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(59) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#36] -Join type: Inner -Join condition: None - -(60) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(61) ReusedExchange [Reuses operator id: 112] -Output [1]: [d_date_sk#40] - -(62) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] - -(64) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] -Results [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(65) CometColumnarExchange -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Arguments: hashpartitioning(i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(66) CometColumnarToRow [codegen id : 52] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(67) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] -Results [6]: [store AS channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51] - -(68) Filter [codegen id : 52] -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51] -Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(69) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_sold_date_sk#57 IN dynamicpruning#58)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(70) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] - -(71) Filter [codegen id : 50] -Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -Condition : isnotnull(ss_item_sk#54) - -(72) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#59] - -(73) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [ss_item_sk#59] -Join type: LeftSemi -Join condition: None - -(74) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] - -(75) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [i_item_sk#60] -Join type: Inner -Join condition: None - -(76) Project [codegen id : 50] -Output [6]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] -Input [8]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] - -(77) ReusedExchange [Reuses operator id: 126] -Output [1]: [d_date_sk#64] - -(78) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#57] -Right keys [1]: [d_date_sk#64] -Join type: Inner -Join condition: None - -(79) Project [codegen id : 50] -Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] -Input [7]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] - -(80) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)] -Aggregate Attributes [3]: [sum#65, isEmpty#66, count#67] -Results [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] - -(81) CometColumnarExchange -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] -Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(82) CometColumnarToRow [codegen id : 51] -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] - -(83) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#71, count(1)#72] -Results [6]: [store AS channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#71 AS sales#74, count(1)#72 AS number_sales#75] - -(84) Filter [codegen id : 51] -Input [6]: [channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] -Condition : (isnotnull(sales#74) AND (cast(sales#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(85) BroadcastExchange -Input [6]: [channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] - -(86) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Right keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Join type: Inner -Join condition: None - -(87) TakeOrderedAndProject -Input [12]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51, channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] -Arguments: 100, [i_brand_id#37 ASC NULLS FIRST, i_class_id#38 ASC NULLS FIRST, i_category_id#39 ASC NULLS FIRST], [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51, channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* HashAggregate (107) -+- * CometColumnarToRow (106) - +- CometColumnarExchange (105) - +- * HashAggregate (104) - +- Union (103) - :- * Project (92) - : +- * BroadcastHashJoin Inner BuildRight (91) - : :- * ColumnarToRow (89) - : : +- Scan parquet spark_catalog.default.store_sales (88) - : +- ReusedExchange (90) - :- * Project (97) - : +- * BroadcastHashJoin Inner BuildRight (96) - : :- * ColumnarToRow (94) - : : +- Scan parquet spark_catalog.default.catalog_sales (93) - : +- ReusedExchange (95) - +- * Project (102) - +- * BroadcastHashJoin Inner BuildRight (101) - :- * ColumnarToRow (99) - : +- Scan parquet spark_catalog.default.web_sales (98) - +- ReusedExchange (100) - - -(88) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#12)] -ReadSchema: struct - -(89) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] - -(90) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#79] - -(91) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#78] -Right keys [1]: [d_date_sk#79] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 2] -Output [2]: [ss_quantity#76 AS quantity#80, ss_list_price#77 AS list_price#81] -Input [4]: [ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78, d_date_sk#79] - -(93) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#12)] -ReadSchema: struct - -(94) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] - -(95) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#85] - -(96) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#84] -Right keys [1]: [d_date_sk#85] -Join type: Inner -Join condition: None - -(97) Project [codegen id : 4] -Output [2]: [cs_quantity#82 AS quantity#86, cs_list_price#83 AS list_price#87] -Input [4]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84, d_date_sk#85] - -(98) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#90), dynamicpruningexpression(ws_sold_date_sk#90 IN dynamicpruning#12)] -ReadSchema: struct - -(99) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] - -(100) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#91] - -(101) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#90] -Right keys [1]: [d_date_sk#91] -Join type: Inner -Join condition: None - -(102) Project [codegen id : 6] -Output [2]: [ws_quantity#88 AS quantity#92, ws_list_price#89 AS list_price#93] -Input [4]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90, d_date_sk#91] - -(103) Union - -(104) HashAggregate [codegen id : 7] -Input [2]: [quantity#80, list_price#81] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] -Aggregate Attributes [2]: [sum#94, count#95] -Results [2]: [sum#96, count#97] - -(105) CometColumnarExchange -Input [2]: [sum#96, count#97] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(106) CometColumnarToRow [codegen id : 8] -Input [2]: [sum#96, count#97] - -(107) HashAggregate [codegen id : 8] -Input [2]: [sum#96, count#97] -Keys: [] -Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] -Aggregate Attributes [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))#98] -Results [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))#98 AS average_sales#99] - -Subquery:2 Hosting operator id = 88 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 93 Hosting Expression = cs_sold_date_sk#84 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 98 Hosting Expression = ws_sold_date_sk#90 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (112) -+- * CometColumnarToRow (111) - +- CometProject (110) - +- CometFilter (109) - +- CometNativeScan parquet spark_catalog.default.date_dim (108) - - -(108) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#40, d_week_seq#100] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#101), IsNotNull(d_date_sk)] -ReadSchema: struct - -(109) CometFilter -Input [2]: [d_date_sk#40, d_week_seq#100] -Condition : ((isnotnull(d_week_seq#100) AND (d_week_seq#100 = ReusedSubquery Subquery scalar-subquery#101, [id=#102])) AND isnotnull(d_date_sk#40)) - -(110) CometProject -Input [2]: [d_date_sk#40, d_week_seq#100] -Arguments: [d_date_sk#40], [d_date_sk#40] - -(111) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#40] - -(112) BroadcastExchange -Input [1]: [d_date_sk#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:6 Hosting operator id = 109 Hosting Expression = ReusedSubquery Subquery scalar-subquery#101, [id=#102] - -Subquery:7 Hosting operator id = 108 Hosting Expression = Subquery scalar-subquery#101, [id=#102] -* CometColumnarToRow (116) -+- CometProject (115) - +- CometFilter (114) - +- CometNativeScan parquet spark_catalog.default.date_dim (113) - - -(113) CometNativeScan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] -ReadSchema: struct - -(114) CometFilter -Input [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] -Condition : (((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND isnotnull(d_dom#106)) AND (d_year#104 = 2000)) AND (d_moy#105 = 12)) AND (d_dom#106 = 11)) - -(115) CometProject -Input [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] -Arguments: [d_week_seq#103], [d_week_seq#103] - -(116) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#103] - -Subquery:8 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometNativeScan parquet spark_catalog.default.date_dim (117) - - -(117) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#107] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#24, d_year#107] -Condition : (((isnotnull(d_year#107) AND (d_year#107 >= 1999)) AND (d_year#107 <= 2001)) AND isnotnull(d_date_sk#24)) - -(119) CometProject -Input [2]: [d_date_sk#24, d_year#107] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#24] - -(121) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:9 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:10 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 - -Subquery:11 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:12 Hosting operator id = 69 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 -BroadcastExchange (126) -+- * CometColumnarToRow (125) - +- CometProject (124) - +- CometFilter (123) - +- CometNativeScan parquet spark_catalog.default.date_dim (122) - - -(122) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#64, d_week_seq#108] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#109), IsNotNull(d_date_sk)] -ReadSchema: struct - -(123) CometFilter -Input [2]: [d_date_sk#64, d_week_seq#108] -Condition : ((isnotnull(d_week_seq#108) AND (d_week_seq#108 = ReusedSubquery Subquery scalar-subquery#109, [id=#110])) AND isnotnull(d_date_sk#64)) - -(124) CometProject -Input [2]: [d_date_sk#64, d_week_seq#108] -Arguments: [d_date_sk#64], [d_date_sk#64] - -(125) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#64] - -(126) BroadcastExchange -Input [1]: [d_date_sk#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] - -Subquery:13 Hosting operator id = 123 Hosting Expression = ReusedSubquery Subquery scalar-subquery#109, [id=#110] - -Subquery:14 Hosting operator id = 122 Hosting Expression = Subquery scalar-subquery#109, [id=#110] -* CometColumnarToRow (130) -+- CometProject (129) - +- CometFilter (128) - +- CometNativeScan parquet spark_catalog.default.date_dim (127) - - -(127) CometNativeScan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#111, d_year#112, d_moy#113, d_dom#114] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] -ReadSchema: struct - -(128) CometFilter -Input [4]: [d_week_seq#111, d_year#112, d_moy#113, d_dom#114] -Condition : (((((isnotnull(d_year#112) AND isnotnull(d_moy#113)) AND isnotnull(d_dom#114)) AND (d_year#112 = 1999)) AND (d_moy#113 = 12)) AND (d_dom#114 = 11)) - -(129) CometProject -Input [4]: [d_week_seq#111, d_year#112, d_moy#113, d_dom#114] -Arguments: [d_week_seq#111], [d_week_seq#111] - -(130) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#111] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/simplified.txt deleted file mode 100644 index cc02a716ea..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/simplified.txt +++ /dev/null @@ -1,208 +0,0 @@ -TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (52) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Filter [sales] - Subquery #4 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #12 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (51) - Filter [sales] - ReusedSubquery [average_sales] #4 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (50) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #15 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/explain.txt deleted file mode 100644 index ccc69246c8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,751 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (85) -+- CometTakeOrderedAndProject (84) - +- CometBroadcastHashJoin (83) - :- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometBroadcastHashJoin (47) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (46) - : : : +- CometProject (45) - : : : +- CometBroadcastHashJoin (44) - : : : :- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (43) - : : : +- CometBroadcastHashJoin (42) - : : : :- CometHashAggregate (32) - : : : : +- CometExchange (31) - : : : : +- CometHashAggregate (30) - : : : : +- CometProject (29) - : : : : +- CometBroadcastHashJoin (28) - : : : : :- CometProject (26) - : : : : : +- CometBroadcastHashJoin (25) - : : : : : :- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : +- CometBroadcastExchange (24) - : : : : : +- CometBroadcastHashJoin (23) - : : : : : :- CometFilter (8) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : +- CometBroadcastExchange (22) - : : : : : +- CometProject (21) - : : : : : +- CometBroadcastHashJoin (20) - : : : : : :- CometProject (15) - : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : :- CometFilter (10) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : +- CometBroadcastExchange (13) - : : : : : : +- CometFilter (12) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : +- CometBroadcastExchange (19) - : : : : : +- CometProject (18) - : : : : : +- CometFilter (17) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : +- ReusedExchange (27) - : : : +- CometBroadcastExchange (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (34) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometFilter (49) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : +- ReusedExchange (50) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometBroadcastExchange (82) - +- CometFilter (81) - +- CometHashAggregate (80) - +- CometExchange (79) - +- CometHashAggregate (78) - +- CometProject (77) - +- CometBroadcastHashJoin (76) - :- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometBroadcastHashJoin (68) - : : :- CometFilter (66) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (65) - : : +- ReusedExchange (67) - : +- ReusedExchange (69) - +- CometBroadcastExchange (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42)) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#45), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Condition : (isnotnull(sales#51) AND (cast(sales#51 as decimal(32,6)) > cast(Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Condition : isnotnull(ss_item_sk#55) - -(67) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#60] - -(68) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [1]: [ss_item_sk#60] -Arguments: [ss_item_sk#55], [ss_item_sk#60], LeftSemi, BuildRight - -(69) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] - -(70) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -Arguments: [ss_item_sk#55], [i_item_sk#61], Inner, BuildRight - -(71) CometProject -Input [8]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -Arguments: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#67), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#66] -Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = ReusedSubquery Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) - -(74) CometProject -Input [2]: [d_date_sk#65, d_week_seq#66] -Arguments: [d_date_sk#65], [d_date_sk#65] - -(75) CometBroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: [d_date_sk#65] - -(76) CometBroadcastHashJoin -Left output [6]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(77) CometProject -Input [7]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] -Arguments: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] - -(78) CometHashAggregate -Input [5]: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [partial_sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), partial_count(1)] - -(79) CometExchange -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(80) CometHashAggregate -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), count(1)] - -(81) CometFilter -Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(82) CometBroadcastExchange -Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -(83) CometBroadcastHashJoin -Left output [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Right output [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#62, i_class_id#63, i_category_id#64], Inner, BuildRight - -(84) CometTakeOrderedAndProject -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#72,i_brand_id#62,i_class_id#63,i_category_id#64,sales#73,number_sales#74]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74], 100, 0, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -(85) CometColumnarToRow [codegen id : 1] -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* CometColumnarToRow (102) -+- CometHashAggregate (101) - +- CometExchange (100) - +- CometHashAggregate (99) - +- CometUnion (98) - :- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (86) - : +- ReusedExchange (87) - :- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (90) - : +- ReusedExchange (91) - +- CometProject (97) - +- CometBroadcastHashJoin (96) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (94) - +- ReusedExchange (95) - - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#78)] -ReadSchema: struct - -(87) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#79] - -(88) CometBroadcastHashJoin -Left output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -Right output [1]: [d_date_sk#79] -Arguments: [ss_sold_date_sk#77], [d_date_sk#79], Inner, BuildRight - -(89) CometProject -Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#79] -Arguments: [quantity#80, list_price#81], [ss_quantity#75 AS quantity#80, ss_list_price#76 AS list_price#81] - -(90) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#85)] -ReadSchema: struct - -(91) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#86] - -(92) CometBroadcastHashJoin -Left output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] -Right output [1]: [d_date_sk#86] -Arguments: [cs_sold_date_sk#84], [d_date_sk#86], Inner, BuildRight - -(93) CometProject -Input [4]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84, d_date_sk#86] -Arguments: [quantity#87, list_price#88], [cs_quantity#82 AS quantity#87, cs_list_price#83 AS list_price#88] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#91), dynamicpruningexpression(ws_sold_date_sk#91 IN dynamicpruning#92)] -ReadSchema: struct - -(95) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#93] - -(96) CometBroadcastHashJoin -Left output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] -Right output [1]: [d_date_sk#93] -Arguments: [ws_sold_date_sk#91], [d_date_sk#93], Inner, BuildRight - -(97) CometProject -Input [4]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91, d_date_sk#93] -Arguments: [quantity#94, list_price#95], [ws_quantity#89 AS quantity#94, ws_list_price#90 AS list_price#95] - -(98) CometUnion -Child 0 Input [2]: [quantity#80, list_price#81] -Child 1 Input [2]: [quantity#87, list_price#88] -Child 2 Input [2]: [quantity#94, list_price#95] - -(99) CometHashAggregate -Input [2]: [quantity#80, list_price#81] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] - -(100) CometExchange -Input [2]: [sum#96, count#97] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(101) CometHashAggregate -Input [2]: [sum#96, count#97] -Keys: [] -Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] - -(102) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#98] - -Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#84 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (107) -+- * CometColumnarToRow (106) - +- CometProject (105) - +- CometFilter (104) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) - - -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#45), IsNotNull(d_date_sk)] -ReadSchema: struct - -(104) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(105) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(106) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(107) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:6 Hosting operator id = 104 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] - -Subquery:7 Hosting operator id = 103 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) -+- CometProject (110) - +- CometFilter (109) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) - - -(108) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] -ReadSchema: struct - -(109) CometFilter -Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d_dom#102)) AND (d_year#100 = 2000)) AND (d_moy#101 = 12)) AND (d_dom#102 = 11)) - -(110) CometProject -Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Arguments: [d_week_seq#99], [d_week_seq#99] - -(111) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#99] - -Subquery:8 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (116) -+- * CometColumnarToRow (115) - +- CometProject (114) - +- CometFilter (113) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) - - -(112) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#103] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(113) CometFilter -Input [2]: [d_date_sk#26, d_year#103] -Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1999)) AND (d_year#103 <= 2001)) AND isnotnull(d_date_sk#26)) - -(114) CometProject -Input [2]: [d_date_sk#26, d_year#103] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(115) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(116) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:9 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:10 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:11 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] - -Subquery:12 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] - -Subquery:13 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] - -Subquery:14 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#67), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#66] -Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = ReusedSubquery Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) - -(119) CometProject -Input [2]: [d_date_sk#65, d_week_seq#66] -Arguments: [d_date_sk#65], [d_date_sk#65] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#65] - -(121) BroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:15 Hosting operator id = 118 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] - -Subquery:16 Hosting operator id = 117 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* CometColumnarToRow (125) -+- CometProject (124) - +- CometFilter (123) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) - - -(122) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] -ReadSchema: struct - -(123) CometFilter -Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d_dom#107)) AND (d_year#105 = 1999)) AND (d_moy#106 = 12)) AND (d_dom#107 = 11)) - -(124) CometProject -Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Arguments: [d_week_seq#104], [d_week_seq#104] - -(125) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#104] - -Subquery:17 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] - -Subquery:18 Hosting operator id = 72 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/simplified.txt deleted file mode 100644 index 6b5264b76d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,157 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #14 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #1 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometBroadcastExchange [ss_item_sk] #3 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #4 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #5 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #10 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #11 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #3 - CometBroadcastExchange [d_date_sk] #13 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #4 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - ReusedExchange [ss_item_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastExchange [d_date_sk] #18 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/explain.txt deleted file mode 100644 index ccc69246c8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/explain.txt +++ /dev/null @@ -1,751 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (85) -+- CometTakeOrderedAndProject (84) - +- CometBroadcastHashJoin (83) - :- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometBroadcastHashJoin (47) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (46) - : : : +- CometProject (45) - : : : +- CometBroadcastHashJoin (44) - : : : :- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (43) - : : : +- CometBroadcastHashJoin (42) - : : : :- CometHashAggregate (32) - : : : : +- CometExchange (31) - : : : : +- CometHashAggregate (30) - : : : : +- CometProject (29) - : : : : +- CometBroadcastHashJoin (28) - : : : : :- CometProject (26) - : : : : : +- CometBroadcastHashJoin (25) - : : : : : :- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : +- CometBroadcastExchange (24) - : : : : : +- CometBroadcastHashJoin (23) - : : : : : :- CometFilter (8) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : +- CometBroadcastExchange (22) - : : : : : +- CometProject (21) - : : : : : +- CometBroadcastHashJoin (20) - : : : : : :- CometProject (15) - : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : :- CometFilter (10) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : +- CometBroadcastExchange (13) - : : : : : : +- CometFilter (12) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : +- CometBroadcastExchange (19) - : : : : : +- CometProject (18) - : : : : : +- CometFilter (17) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : +- ReusedExchange (27) - : : : +- CometBroadcastExchange (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (34) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometFilter (49) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : +- ReusedExchange (50) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometBroadcastExchange (82) - +- CometFilter (81) - +- CometHashAggregate (80) - +- CometExchange (79) - +- CometHashAggregate (78) - +- CometProject (77) - +- CometBroadcastHashJoin (76) - :- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometBroadcastHashJoin (68) - : : :- CometFilter (66) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (65) - : : +- ReusedExchange (67) - : +- ReusedExchange (69) - +- CometBroadcastExchange (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42)) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#45), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Condition : (isnotnull(sales#51) AND (cast(sales#51 as decimal(32,6)) > cast(Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Condition : isnotnull(ss_item_sk#55) - -(67) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#60] - -(68) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [1]: [ss_item_sk#60] -Arguments: [ss_item_sk#55], [ss_item_sk#60], LeftSemi, BuildRight - -(69) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] - -(70) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -Arguments: [ss_item_sk#55], [i_item_sk#61], Inner, BuildRight - -(71) CometProject -Input [8]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -Arguments: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#67), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#66] -Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = ReusedSubquery Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) - -(74) CometProject -Input [2]: [d_date_sk#65, d_week_seq#66] -Arguments: [d_date_sk#65], [d_date_sk#65] - -(75) CometBroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: [d_date_sk#65] - -(76) CometBroadcastHashJoin -Left output [6]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(77) CometProject -Input [7]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] -Arguments: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] - -(78) CometHashAggregate -Input [5]: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [partial_sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), partial_count(1)] - -(79) CometExchange -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(80) CometHashAggregate -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), count(1)] - -(81) CometFilter -Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(82) CometBroadcastExchange -Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -(83) CometBroadcastHashJoin -Left output [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Right output [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#62, i_class_id#63, i_category_id#64], Inner, BuildRight - -(84) CometTakeOrderedAndProject -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#72,i_brand_id#62,i_class_id#63,i_category_id#64,sales#73,number_sales#74]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74], 100, 0, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -(85) CometColumnarToRow [codegen id : 1] -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* CometColumnarToRow (102) -+- CometHashAggregate (101) - +- CometExchange (100) - +- CometHashAggregate (99) - +- CometUnion (98) - :- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (86) - : +- ReusedExchange (87) - :- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (90) - : +- ReusedExchange (91) - +- CometProject (97) - +- CometBroadcastHashJoin (96) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (94) - +- ReusedExchange (95) - - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#78)] -ReadSchema: struct - -(87) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#79] - -(88) CometBroadcastHashJoin -Left output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -Right output [1]: [d_date_sk#79] -Arguments: [ss_sold_date_sk#77], [d_date_sk#79], Inner, BuildRight - -(89) CometProject -Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#79] -Arguments: [quantity#80, list_price#81], [ss_quantity#75 AS quantity#80, ss_list_price#76 AS list_price#81] - -(90) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#85)] -ReadSchema: struct - -(91) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#86] - -(92) CometBroadcastHashJoin -Left output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] -Right output [1]: [d_date_sk#86] -Arguments: [cs_sold_date_sk#84], [d_date_sk#86], Inner, BuildRight - -(93) CometProject -Input [4]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84, d_date_sk#86] -Arguments: [quantity#87, list_price#88], [cs_quantity#82 AS quantity#87, cs_list_price#83 AS list_price#88] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#91), dynamicpruningexpression(ws_sold_date_sk#91 IN dynamicpruning#92)] -ReadSchema: struct - -(95) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#93] - -(96) CometBroadcastHashJoin -Left output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] -Right output [1]: [d_date_sk#93] -Arguments: [ws_sold_date_sk#91], [d_date_sk#93], Inner, BuildRight - -(97) CometProject -Input [4]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91, d_date_sk#93] -Arguments: [quantity#94, list_price#95], [ws_quantity#89 AS quantity#94, ws_list_price#90 AS list_price#95] - -(98) CometUnion -Child 0 Input [2]: [quantity#80, list_price#81] -Child 1 Input [2]: [quantity#87, list_price#88] -Child 2 Input [2]: [quantity#94, list_price#95] - -(99) CometHashAggregate -Input [2]: [quantity#80, list_price#81] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] - -(100) CometExchange -Input [2]: [sum#96, count#97] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(101) CometHashAggregate -Input [2]: [sum#96, count#97] -Keys: [] -Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] - -(102) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#98] - -Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#84 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (107) -+- * CometColumnarToRow (106) - +- CometProject (105) - +- CometFilter (104) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) - - -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#45), IsNotNull(d_date_sk)] -ReadSchema: struct - -(104) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(105) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(106) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(107) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:6 Hosting operator id = 104 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] - -Subquery:7 Hosting operator id = 103 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) -+- CometProject (110) - +- CometFilter (109) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) - - -(108) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] -ReadSchema: struct - -(109) CometFilter -Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d_dom#102)) AND (d_year#100 = 2000)) AND (d_moy#101 = 12)) AND (d_dom#102 = 11)) - -(110) CometProject -Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Arguments: [d_week_seq#99], [d_week_seq#99] - -(111) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#99] - -Subquery:8 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (116) -+- * CometColumnarToRow (115) - +- CometProject (114) - +- CometFilter (113) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) - - -(112) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#103] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(113) CometFilter -Input [2]: [d_date_sk#26, d_year#103] -Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1999)) AND (d_year#103 <= 2001)) AND isnotnull(d_date_sk#26)) - -(114) CometProject -Input [2]: [d_date_sk#26, d_year#103] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(115) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(116) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:9 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:10 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:11 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] - -Subquery:12 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] - -Subquery:13 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] - -Subquery:14 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#67), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#66] -Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = ReusedSubquery Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) - -(119) CometProject -Input [2]: [d_date_sk#65, d_week_seq#66] -Arguments: [d_date_sk#65], [d_date_sk#65] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#65] - -(121) BroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:15 Hosting operator id = 118 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] - -Subquery:16 Hosting operator id = 117 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* CometColumnarToRow (125) -+- CometProject (124) - +- CometFilter (123) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) - - -(122) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] -ReadSchema: struct - -(123) CometFilter -Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d_dom#107)) AND (d_year#105 = 1999)) AND (d_moy#106 = 12)) AND (d_dom#107 = 11)) - -(124) CometProject -Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Arguments: [d_week_seq#104], [d_week_seq#104] - -(125) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#104] - -Subquery:17 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] - -Subquery:18 Hosting operator id = 72 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/extended.txt deleted file mode 100644 index b32dd95954..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/extended.txt +++ /dev/null @@ -1,343 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometBroadcastHashJoin - :- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- ReusedSubquery - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- ReusedSubquery - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- ReusedSubquery - -Comet accelerated 298 out of 331 eligible operators (90%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 6b5264b76d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt +++ /dev/null @@ -1,157 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #14 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #1 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometBroadcastExchange [ss_item_sk] #3 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #4 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #5 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #10 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #11 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #3 - CometBroadcastExchange [d_date_sk] #13 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #4 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - ReusedExchange [ss_item_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastExchange [d_date_sk] #18 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/explain.txt deleted file mode 100644 index ddc9c5e253..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/explain.txt +++ /dev/null @@ -1,173 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (24) -+- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometColumnarExchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : +- BroadcastExchange (7) - : : +- * CometColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometNativeScan parquet spark_catalog.default.customer (4) - : +- BroadcastExchange (14) - : +- * CometColumnarToRow (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometNativeScan parquet spark_catalog.default.customer_address (10) - +- ReusedExchange (17) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] - -(3) Filter [codegen id : 4] -Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_bill_customer_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#5, c_current_addr_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] -Condition : (isnotnull(c_customer_sk#5) AND isnotnull(c_current_addr_sk#6)) - -(6) CometColumnarToRow [codegen id : 1] -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] - -(7) BroadcastExchange -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#5] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 4] -Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] -Input [5]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3, c_customer_sk#5, c_current_addr_sk#6] - -(10) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Condition : isnotnull(ca_address_sk#7) - -(12) CometProject -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Arguments: [ca_address_sk#7, ca_state#10, ca_zip#11], [ca_address_sk#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#8, 2)) AS ca_state#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#9, 10)) AS ca_zip#11] - -(13) CometColumnarToRow [codegen id : 2] -Input [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] - -(14) BroadcastExchange -Input [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [c_current_addr_sk#6] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: ((substr(ca_zip#11, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#10 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)) - -(16) Project [codegen id : 4] -Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11] -Input [6]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, ca_address_sk#7, ca_state#10, ca_zip#11] - -(17) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#12] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [2]: [cs_sales_price#2, ca_zip#11] -Input [4]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11, d_date_sk#12] - -(20) HashAggregate [codegen id : 4] -Input [2]: [cs_sales_price#2, ca_zip#11] -Keys [1]: [ca_zip#11] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum#13] -Results [2]: [ca_zip#11, sum#14] - -(21) CometColumnarExchange -Input [2]: [ca_zip#11, sum#14] -Arguments: hashpartitioning(ca_zip#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [2]: [ca_zip#11, sum#14] - -(23) HashAggregate [codegen id : 5] -Input [2]: [ca_zip#11, sum#14] -Keys [1]: [ca_zip#11] -Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#15] -Results [2]: [ca_zip#11, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#15,17,2) AS sum(cs_sales_price)#16] - -(24) TakeOrderedAndProject -Input [2]: [ca_zip#11, sum(cs_sales_price)#16] -Arguments: 100, [ca_zip#11 ASC NULLS FIRST], [ca_zip#11, sum(cs_sales_price)#16] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometNativeScan parquet spark_catalog.default.date_dim (25) - - -(25) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#12, d_year#17, d_qoy#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [3]: [d_date_sk#12, d_year#17, d_qoy#18] -Condition : ((((isnotnull(d_qoy#18) AND isnotnull(d_year#17)) AND (d_qoy#18 = 2)) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#12)) - -(27) CometProject -Input [3]: [d_date_sk#12, d_year#17, d_qoy#18] -Arguments: [d_date_sk#12], [d_date_sk#12] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#12] - -(29) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/simplified.txt deleted file mode 100644 index f31442dcfe..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - WholeStageCodegen (5) - HashAggregate [ca_zip,sum] [sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price),sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_zip] #1 - WholeStageCodegen (4) - HashAggregate [ca_zip,cs_sales_price] [sum,sum] - Project [cs_sales_price,ca_zip] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sales_price,cs_sold_date_sk,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] - Project [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip] - CometFilter [ca_address_sk,ca_state,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - InputAdapter - ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/explain.txt deleted file mode 100644 index 8a4fb53ece..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,173 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (13) - : +- CometBroadcastHashJoin (12) - : :- CometProject (7) - : : +- CometBroadcastHashJoin (6) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : +- CometBroadcastExchange (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (3) - : +- CometBroadcastExchange (11) - : +- CometProject (10) - : +- CometFilter (9) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - +- CometBroadcastExchange (17) - +- CometProject (16) - +- CometFilter (15) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_bill_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#5, c_current_addr_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] -Condition : (isnotnull(c_customer_sk#5) AND isnotnull(c_current_addr_sk#6)) - -(5) CometBroadcastExchange -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] -Arguments: [c_customer_sk#5, c_current_addr_sk#6] - -(6) CometBroadcastHashJoin -Left output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Right output [2]: [c_customer_sk#5, c_current_addr_sk#6] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#5], Inner, BuildRight - -(7) CometProject -Input [5]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3, c_customer_sk#5, c_current_addr_sk#6] -Arguments: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6], [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Condition : isnotnull(ca_address_sk#7) - -(10) CometProject -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Arguments: [ca_address_sk#7, ca_state#10, ca_zip#11], [ca_address_sk#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#8, 2)) AS ca_state#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#9, 10)) AS ca_zip#11] - -(11) CometBroadcastExchange -Input [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] -Arguments: [ca_address_sk#7, ca_state#10, ca_zip#11] - -(12) CometBroadcastHashJoin -Left output [3]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] -Right output [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] -Arguments: [c_current_addr_sk#6], [ca_address_sk#7], Inner, ((substr(ca_zip#11, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#10 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)), BuildRight - -(13) CometProject -Input [6]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, ca_address_sk#7, ca_state#10, ca_zip#11] -Arguments: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11], [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2)) AND (d_year#13 = 2001)) AND isnotnull(d_date_sk#12)) - -(16) CometProject -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Arguments: [d_date_sk#12], [d_date_sk#12] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: [d_date_sk#12] - -(18) CometBroadcastHashJoin -Left output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11] -Right output [1]: [d_date_sk#12] -Arguments: [cs_sold_date_sk#3], [d_date_sk#12], Inner, BuildRight - -(19) CometProject -Input [4]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11, d_date_sk#12] -Arguments: [cs_sales_price#2, ca_zip#11], [cs_sales_price#2, ca_zip#11] - -(20) CometHashAggregate -Input [2]: [cs_sales_price#2, ca_zip#11] -Keys [1]: [ca_zip#11] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] - -(21) CometExchange -Input [2]: [ca_zip#11, sum#15] -Arguments: hashpartitioning(ca_zip#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [2]: [ca_zip#11, sum#15] -Keys [1]: [ca_zip#11] -Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] - -(23) CometTakeOrderedAndProject -Input [2]: [ca_zip#11, sum(cs_sales_price)#16] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_zip#11 ASC NULLS FIRST], output=[ca_zip#11,sum(cs_sales_price)#16]), [ca_zip#11, sum(cs_sales_price)#16], 100, 0, [ca_zip#11 ASC NULLS FIRST], [ca_zip#11, sum(cs_sales_price)#16] - -(24) CometColumnarToRow [codegen id : 1] -Input [2]: [ca_zip#11, sum(cs_sales_price)#16] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2)) AND (d_year#13 = 2001)) AND isnotnull(d_date_sk#12)) - -(27) CometProject -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Arguments: [d_date_sk#12], [d_date_sk#12] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#12] - -(29) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/simplified.txt deleted file mode 100644 index c39b96efe3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,34 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - CometHashAggregate [sum] [ca_zip,sum(cs_sales_price),sum(UnscaledValue(cs_sales_price))] - CometExchange [ca_zip] #1 - CometHashAggregate [cs_sales_price] [ca_zip,sum] - CometProject [cs_sales_price,ca_zip] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] - CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip] - CometFilter [ca_address_sk,ca_state,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] 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/q15/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/explain.txt deleted file mode 100644 index 8a4fb53ece..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/explain.txt +++ /dev/null @@ -1,173 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (13) - : +- CometBroadcastHashJoin (12) - : :- CometProject (7) - : : +- CometBroadcastHashJoin (6) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : +- CometBroadcastExchange (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (3) - : +- CometBroadcastExchange (11) - : +- CometProject (10) - : +- CometFilter (9) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - +- CometBroadcastExchange (17) - +- CometProject (16) - +- CometFilter (15) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_bill_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#5, c_current_addr_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] -Condition : (isnotnull(c_customer_sk#5) AND isnotnull(c_current_addr_sk#6)) - -(5) CometBroadcastExchange -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] -Arguments: [c_customer_sk#5, c_current_addr_sk#6] - -(6) CometBroadcastHashJoin -Left output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Right output [2]: [c_customer_sk#5, c_current_addr_sk#6] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#5], Inner, BuildRight - -(7) CometProject -Input [5]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3, c_customer_sk#5, c_current_addr_sk#6] -Arguments: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6], [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Condition : isnotnull(ca_address_sk#7) - -(10) CometProject -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Arguments: [ca_address_sk#7, ca_state#10, ca_zip#11], [ca_address_sk#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#8, 2)) AS ca_state#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#9, 10)) AS ca_zip#11] - -(11) CometBroadcastExchange -Input [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] -Arguments: [ca_address_sk#7, ca_state#10, ca_zip#11] - -(12) CometBroadcastHashJoin -Left output [3]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] -Right output [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] -Arguments: [c_current_addr_sk#6], [ca_address_sk#7], Inner, ((substr(ca_zip#11, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#10 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)), BuildRight - -(13) CometProject -Input [6]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, ca_address_sk#7, ca_state#10, ca_zip#11] -Arguments: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11], [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2)) AND (d_year#13 = 2001)) AND isnotnull(d_date_sk#12)) - -(16) CometProject -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Arguments: [d_date_sk#12], [d_date_sk#12] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: [d_date_sk#12] - -(18) CometBroadcastHashJoin -Left output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11] -Right output [1]: [d_date_sk#12] -Arguments: [cs_sold_date_sk#3], [d_date_sk#12], Inner, BuildRight - -(19) CometProject -Input [4]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11, d_date_sk#12] -Arguments: [cs_sales_price#2, ca_zip#11], [cs_sales_price#2, ca_zip#11] - -(20) CometHashAggregate -Input [2]: [cs_sales_price#2, ca_zip#11] -Keys [1]: [ca_zip#11] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] - -(21) CometExchange -Input [2]: [ca_zip#11, sum#15] -Arguments: hashpartitioning(ca_zip#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [2]: [ca_zip#11, sum#15] -Keys [1]: [ca_zip#11] -Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] - -(23) CometTakeOrderedAndProject -Input [2]: [ca_zip#11, sum(cs_sales_price)#16] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_zip#11 ASC NULLS FIRST], output=[ca_zip#11,sum(cs_sales_price)#16]), [ca_zip#11, sum(cs_sales_price)#16], 100, 0, [ca_zip#11 ASC NULLS FIRST], [ca_zip#11, sum(cs_sales_price)#16] - -(24) CometColumnarToRow [codegen id : 1] -Input [2]: [ca_zip#11, sum(cs_sales_price)#16] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2)) AND (d_year#13 = 2001)) AND isnotnull(d_date_sk#12)) - -(27) CometProject -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Arguments: [d_date_sk#12], [d_date_sk#12] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#12] - -(29) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/extended.txt deleted file mode 100644 index 6de0c64850..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/extended.txt +++ /dev/null @@ -1,32 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index c39b96efe3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt +++ /dev/null @@ -1,34 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - CometHashAggregate [sum] [ca_zip,sum(cs_sales_price),sum(UnscaledValue(cs_sales_price))] - CometExchange [ca_zip] #1 - CometHashAggregate [cs_sales_price] [ca_zip,sum] - CometProject [cs_sales_price,ca_zip] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] - CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip] - CometFilter [ca_address_sk,ca_state,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_datafusion/explain.txt deleted file mode 100644 index 2c1be26c56..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_datafusion/explain.txt +++ /dev/null @@ -1,235 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometColumnarExchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * CometColumnarToRow (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometNativeScan parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometNativeScan parquet spark_catalog.default.call_center (29) - - -(1) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) - -(3) CometProject -Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(4) CometExchange -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -ReadSchema: struct - -(7) CometProject -Input [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] -Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_warehouse_sk#9, cs_order_number#10] - -(8) CometExchange -Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_order_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_order_number#5], [cs_order_number#10], LeftSemi, NOT (cs_warehouse_sk#4 = cs_warehouse_sk#9) - -(11) CometProject -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(12) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [2]: [cr_order_number#12, cr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -ReadSchema: struct - -(13) CometProject -Input [2]: [cr_order_number#12, cr_returned_date_sk#13] -Arguments: [cr_order_number#12], [cr_order_number#12] - -(14) CometExchange -Input [1]: [cr_order_number#12] -Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(15) CometSort -Input [1]: [cr_order_number#12] -Arguments: [cr_order_number#12], [cr_order_number#12 ASC NULLS FIRST] - -(16) CometSortMergeJoin -Left output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [cr_order_number#12] -Arguments: [cs_order_number#5], [cr_order_number#12], LeftAnti - -(17) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2002-02-01)) AND (d_date#15 <= 2002-04-02)) AND isnotnull(d_date_sk#14)) - -(19) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(20) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(21) CometBroadcastHashJoin -Left output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [d_date_sk#14] -Arguments: [cs_ship_date_sk#1], [d_date_sk#14], Inner, BuildRight - -(22) CometProject -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#14] -Arguments: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(23) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((isnotnull(ca_state#17) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#17, 2)) = GA)) AND isnotnull(ca_address_sk#16)) - -(25) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(26) CometBroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: [ca_address_sk#16] - -(27) CometBroadcastHashJoin -Left output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [ca_address_sk#16] -Arguments: [cs_ship_addr_sk#2], [ca_address_sk#16], Inner, BuildRight - -(28) CometProject -Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] -Arguments: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(29) CometNativeScan parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#18, cc_county#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [cc_call_center_sk#18, cc_county#19] -Condition : ((isnotnull(cc_county#19) AND (cc_county#19 = Williamson County)) AND isnotnull(cc_call_center_sk#18)) - -(31) CometProject -Input [2]: [cc_call_center_sk#18, cc_county#19] -Arguments: [cc_call_center_sk#18], [cc_call_center_sk#18] - -(32) CometBroadcastExchange -Input [1]: [cc_call_center_sk#18] -Arguments: [cc_call_center_sk#18] - -(33) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [cc_call_center_sk#18] -Arguments: [cs_call_center_sk#3], [cc_call_center_sk#18], Inner, BuildRight - -(34) CometProject -Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#18] -Arguments: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(35) CometHashAggregate -Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Keys [1]: [cs_order_number#5] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] - -(36) CometColumnarToRow [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] - -(37) HashAggregate [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] -Keys [1]: [cs_order_number#5] -Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23] -Results [3]: [cs_order_number#5, sum#20, sum#21] - -(38) HashAggregate [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23, count(cs_order_number#5)#24] -Results [3]: [sum#20, sum#21, count#25] - -(39) CometColumnarExchange -Input [3]: [sum#20, sum#21, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(40) CometHashAggregate -Input [3]: [sum#20, sum#21, count#25] -Keys: [] -Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] - -(41) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #26, total shipping cost #27, total net profit #28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_datafusion/simplified.txt deleted file mode 100644 index 429d83d08c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_datafusion/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(cs_order_number),sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [cs_ext_ship_cost,cs_net_profit] [cs_order_number,sum,sum] - CometProject [cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cc_call_center_sk] - CometProject [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,ca_address_sk] - CometProject [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,d_date_sk] - CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cr_order_number] - CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometSortMergeJoin [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_order_number,cs_warehouse_sk] - CometSort [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] - CometExchange [cs_order_number] #2 - 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,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - CometNativeScan 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] - CometSort [cs_warehouse_sk,cs_order_number] - CometExchange [cs_order_number] #3 - CometProject [cs_warehouse_sk,cs_order_number] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] - CometSort [cr_order_number] - CometExchange [cr_order_number] #4 - CometProject [cr_order_number] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cc_call_center_sk] #7 - CometProject [cc_call_center_sk] - CometFilter [cc_call_center_sk,cc_county] - CometNativeScan 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/q16.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/explain.txt deleted file mode 100644 index 7a4c0fa400..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,235 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometColumnarExchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * CometColumnarToRow (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (29) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) - -(3) CometProject -Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(4) CometExchange -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -ReadSchema: struct - -(7) CometProject -Input [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] -Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_warehouse_sk#9, cs_order_number#10] - -(8) CometExchange -Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_order_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_order_number#5], [cs_order_number#10], LeftSemi, NOT (cs_warehouse_sk#4 = cs_warehouse_sk#9) - -(11) CometProject -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [2]: [cr_order_number#12, cr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -ReadSchema: struct - -(13) CometProject -Input [2]: [cr_order_number#12, cr_returned_date_sk#13] -Arguments: [cr_order_number#12], [cr_order_number#12] - -(14) CometExchange -Input [1]: [cr_order_number#12] -Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(15) CometSort -Input [1]: [cr_order_number#12] -Arguments: [cr_order_number#12], [cr_order_number#12 ASC NULLS FIRST] - -(16) CometSortMergeJoin -Left output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [cr_order_number#12] -Arguments: [cs_order_number#5], [cr_order_number#12], LeftAnti - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2002-02-01)) AND (d_date#15 <= 2002-04-02)) AND isnotnull(d_date_sk#14)) - -(19) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(20) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(21) CometBroadcastHashJoin -Left output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [d_date_sk#14] -Arguments: [cs_ship_date_sk#1], [d_date_sk#14], Inner, BuildRight - -(22) CometProject -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#14] -Arguments: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((isnotnull(ca_state#17) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#17, 2)) = GA)) AND isnotnull(ca_address_sk#16)) - -(25) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(26) CometBroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: [ca_address_sk#16] - -(27) CometBroadcastHashJoin -Left output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [ca_address_sk#16] -Arguments: [cs_ship_addr_sk#2], [ca_address_sk#16], Inner, BuildRight - -(28) CometProject -Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] -Arguments: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#18, cc_county#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [cc_call_center_sk#18, cc_county#19] -Condition : ((isnotnull(cc_county#19) AND (cc_county#19 = Williamson County)) AND isnotnull(cc_call_center_sk#18)) - -(31) CometProject -Input [2]: [cc_call_center_sk#18, cc_county#19] -Arguments: [cc_call_center_sk#18], [cc_call_center_sk#18] - -(32) CometBroadcastExchange -Input [1]: [cc_call_center_sk#18] -Arguments: [cc_call_center_sk#18] - -(33) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [cc_call_center_sk#18] -Arguments: [cs_call_center_sk#3], [cc_call_center_sk#18], Inner, BuildRight - -(34) CometProject -Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#18] -Arguments: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(35) CometHashAggregate -Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Keys [1]: [cs_order_number#5] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] - -(36) CometColumnarToRow [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] - -(37) HashAggregate [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] -Keys [1]: [cs_order_number#5] -Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23] -Results [3]: [cs_order_number#5, sum#20, sum#21] - -(38) HashAggregate [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23, count(cs_order_number#5)#24] -Results [3]: [sum#20, sum#21, count#25] - -(39) CometColumnarExchange -Input [3]: [sum#20, sum#21, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(40) CometHashAggregate -Input [3]: [sum#20, sum#21, count#25] -Keys: [] -Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] - -(41) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #26, total shipping cost #27, total net profit #28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/simplified.txt deleted file mode 100644 index 8427aa49a9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(cs_order_number),sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [cs_ext_ship_cost,cs_net_profit] [cs_order_number,sum,sum] - CometProject [cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cc_call_center_sk] - CometProject [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,ca_address_sk] - CometProject [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,d_date_sk] - CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cr_order_number] - CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometSortMergeJoin [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_order_number,cs_warehouse_sk] - CometSort [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] - CometExchange [cs_order_number] #2 - 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,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [cs_warehouse_sk,cs_order_number] - CometExchange [cs_order_number] #3 - CometProject [cs_warehouse_sk,cs_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] - CometSort [cr_order_number] - CometExchange [cr_order_number] #4 - CometProject [cr_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cc_call_center_sk] #7 - CometProject [cc_call_center_sk] - CometFilter [cc_call_center_sk,cc_county] - CometScan [native_iceberg_compat] 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/q16/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/explain.txt deleted file mode 100644 index 7a4c0fa400..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/explain.txt +++ /dev/null @@ -1,235 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometColumnarExchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * CometColumnarToRow (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (29) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) - -(3) CometProject -Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(4) CometExchange -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -ReadSchema: struct - -(7) CometProject -Input [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] -Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_warehouse_sk#9, cs_order_number#10] - -(8) CometExchange -Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_order_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_order_number#5], [cs_order_number#10], LeftSemi, NOT (cs_warehouse_sk#4 = cs_warehouse_sk#9) - -(11) CometProject -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [2]: [cr_order_number#12, cr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -ReadSchema: struct - -(13) CometProject -Input [2]: [cr_order_number#12, cr_returned_date_sk#13] -Arguments: [cr_order_number#12], [cr_order_number#12] - -(14) CometExchange -Input [1]: [cr_order_number#12] -Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(15) CometSort -Input [1]: [cr_order_number#12] -Arguments: [cr_order_number#12], [cr_order_number#12 ASC NULLS FIRST] - -(16) CometSortMergeJoin -Left output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [cr_order_number#12] -Arguments: [cs_order_number#5], [cr_order_number#12], LeftAnti - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2002-02-01)) AND (d_date#15 <= 2002-04-02)) AND isnotnull(d_date_sk#14)) - -(19) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(20) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(21) CometBroadcastHashJoin -Left output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [d_date_sk#14] -Arguments: [cs_ship_date_sk#1], [d_date_sk#14], Inner, BuildRight - -(22) CometProject -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#14] -Arguments: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((isnotnull(ca_state#17) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#17, 2)) = GA)) AND isnotnull(ca_address_sk#16)) - -(25) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(26) CometBroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: [ca_address_sk#16] - -(27) CometBroadcastHashJoin -Left output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [ca_address_sk#16] -Arguments: [cs_ship_addr_sk#2], [ca_address_sk#16], Inner, BuildRight - -(28) CometProject -Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] -Arguments: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#18, cc_county#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [cc_call_center_sk#18, cc_county#19] -Condition : ((isnotnull(cc_county#19) AND (cc_county#19 = Williamson County)) AND isnotnull(cc_call_center_sk#18)) - -(31) CometProject -Input [2]: [cc_call_center_sk#18, cc_county#19] -Arguments: [cc_call_center_sk#18], [cc_call_center_sk#18] - -(32) CometBroadcastExchange -Input [1]: [cc_call_center_sk#18] -Arguments: [cc_call_center_sk#18] - -(33) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [cc_call_center_sk#18] -Arguments: [cs_call_center_sk#3], [cc_call_center_sk#18], Inner, BuildRight - -(34) CometProject -Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#18] -Arguments: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(35) CometHashAggregate -Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Keys [1]: [cs_order_number#5] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] - -(36) CometColumnarToRow [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] - -(37) HashAggregate [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] -Keys [1]: [cs_order_number#5] -Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23] -Results [3]: [cs_order_number#5, sum#20, sum#21] - -(38) HashAggregate [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23, count(cs_order_number#5)#24] -Results [3]: [sum#20, sum#21, count#25] - -(39) CometColumnarExchange -Input [3]: [sum#20, sum#21, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(40) CometHashAggregate -Input [3]: [sum#20, sum#21, count#25] -Keys: [] -Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] - -(41) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #26, total shipping cost #27, total net profit #28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/extended.txt deleted file mode 100644 index 2ad029e444..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/extended.txt +++ /dev/null @@ -1,43 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - -Comet accelerated 37 out of 39 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 8427aa49a9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(cs_order_number),sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [cs_ext_ship_cost,cs_net_profit] [cs_order_number,sum,sum] - CometProject [cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cc_call_center_sk] - CometProject [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,ca_address_sk] - CometProject [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,d_date_sk] - CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cr_order_number] - CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometSortMergeJoin [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_order_number,cs_warehouse_sk] - CometSort [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] - CometExchange [cs_order_number] #2 - 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,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [cs_warehouse_sk,cs_order_number] - CometExchange [cs_order_number] #3 - CometProject [cs_warehouse_sk,cs_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] - CometSort [cr_order_number] - CometExchange [cr_order_number] #4 - CometProject [cr_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cc_call_center_sk] #7 - CometProject [cc_call_center_sk] - CometFilter [cc_call_center_sk,cc_county] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/explain.txt deleted file mode 100644 index f6ba10374e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/explain.txt +++ /dev/null @@ -1,312 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (18) - : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet spark_catalog.default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * Filter (12) - : : : : : +- * ColumnarToRow (11) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (10) - : : : : +- ReusedExchange (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.store (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.item (32) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 8] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] - -(3) Filter [codegen id : 8] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(4) Scan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(6) Filter [codegen id : 1] -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(7) BroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 8] -Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] -Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(10) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(12) Filter [codegen id : 2] -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(13) BroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] -Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(16) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#18] - -(17) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#18] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 8] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#18] - -(19) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#19] - -(20) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [sr_returned_date_sk#12] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] - -(22) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#20] - -(23) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 8] -Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#20] - -(25) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#21, s_state#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [s_store_sk#21, s_state#22] -Condition : isnotnull(s_store_sk#21) - -(27) CometProject -Input [2]: [s_store_sk#21, s_state#22] -Arguments: [s_store_sk#21, s_state#23], [s_store_sk#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#22, 2)) AS s_state#23] - -(28) CometColumnarToRow [codegen id : 6] -Input [2]: [s_store_sk#21, s_state#23] - -(29) BroadcastExchange -Input [2]: [s_store_sk#21, s_state#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(30) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#21] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 8] -Output [5]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23] -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#21, s_state#23] - -(32) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] -Condition : isnotnull(i_item_sk#24) - -(34) CometProject -Input [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] -Arguments: [i_item_sk#24, i_item_id#27, i_item_desc#26], [i_item_sk#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#25, 16)) AS i_item_id#27, i_item_desc#26] - -(35) CometColumnarToRow [codegen id : 7] -Input [3]: [i_item_sk#24, i_item_id#27, i_item_desc#26] - -(36) BroadcastExchange -Input [3]: [i_item_sk#24, i_item_id#27, i_item_desc#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(37) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#24] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 8] -Output [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_id#27, i_item_desc#26] -Input [8]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_sk#24, i_item_id#27, i_item_desc#26] - -(39) HashAggregate [codegen id : 8] -Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_id#27, i_item_desc#26] -Keys [3]: [i_item_id#27, i_item_desc#26, s_state#23] -Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partial_stddev_samp(cast(ss_quantity#5 as double)), partial_count(sr_return_quantity#11), partial_avg(sr_return_quantity#11), partial_stddev_samp(cast(sr_return_quantity#11 as double)), partial_count(cs_quantity#16), partial_avg(cs_quantity#16), partial_stddev_samp(cast(cs_quantity#16 as double))] -Aggregate Attributes [18]: [count#28, sum#29, count#30, n#31, avg#32, m2#33, count#34, sum#35, count#36, n#37, avg#38, m2#39, count#40, sum#41, count#42, n#43, avg#44, m2#45] -Results [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count#48, n#49, avg#50, m2#51, count#52, sum#53, count#54, n#55, avg#56, m2#57, count#58, sum#59, count#60, n#61, avg#62, m2#63] - -(40) CometColumnarExchange -Input [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count#48, n#49, avg#50, m2#51, count#52, sum#53, count#54, n#55, avg#56, m2#57, count#58, sum#59, count#60, n#61, avg#62, m2#63] -Arguments: hashpartitioning(i_item_id#27, i_item_desc#26, s_state#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(41) CometColumnarToRow [codegen id : 9] -Input [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count#48, n#49, avg#50, m2#51, count#52, sum#53, count#54, n#55, avg#56, m2#57, count#58, sum#59, count#60, n#61, avg#62, m2#63] - -(42) HashAggregate [codegen id : 9] -Input [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count#48, n#49, avg#50, m2#51, count#52, sum#53, count#54, n#55, avg#56, m2#57, count#58, sum#59, count#60, n#61, avg#62, m2#63] -Keys [3]: [i_item_id#27, i_item_desc#26, s_state#23] -Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#11), avg(sr_return_quantity#11), stddev_samp(cast(sr_return_quantity#11 as double)), count(cs_quantity#16), avg(cs_quantity#16), stddev_samp(cast(cs_quantity#16 as double))] -Aggregate Attributes [9]: [count(ss_quantity#5)#64, avg(ss_quantity#5)#65, stddev_samp(cast(ss_quantity#5 as double))#66, count(sr_return_quantity#11)#67, avg(sr_return_quantity#11)#68, stddev_samp(cast(sr_return_quantity#11 as double))#69, count(cs_quantity#16)#70, avg(cs_quantity#16)#71, stddev_samp(cast(cs_quantity#16 as double))#72] -Results [15]: [i_item_id#27, i_item_desc#26, s_state#23, count(ss_quantity#5)#64 AS store_sales_quantitycount#73, avg(ss_quantity#5)#65 AS store_sales_quantityave#74, stddev_samp(cast(ss_quantity#5 as double))#66 AS store_sales_quantitystdev#75, (stddev_samp(cast(ss_quantity#5 as double))#66 / avg(ss_quantity#5)#65) AS store_sales_quantitycov#76, count(sr_return_quantity#11)#67 AS as_store_returns_quantitycount#77, avg(sr_return_quantity#11)#68 AS as_store_returns_quantityave#78, stddev_samp(cast(sr_return_quantity#11 as double))#69 AS as_store_returns_quantitystdev#79, (stddev_samp(cast(sr_return_quantity#11 as double))#69 / avg(sr_return_quantity#11)#68) AS store_returns_quantitycov#80, count(cs_quantity#16)#70 AS catalog_sales_quantitycount#81, avg(cs_quantity#16)#71 AS catalog_sales_quantityave#82, (stddev_samp(cast(cs_quantity#16 as double))#72 / avg(cs_quantity#16)#71) AS catalog_sales_quantitystdev#83, (stddev_samp(cast(cs_quantity#16 as double))#72 / avg(cs_quantity#16)#71) AS catalog_sales_quantitycov#84] - -(43) TakeOrderedAndProject -Input [15]: [i_item_id#27, i_item_desc#26, s_state#23, store_sales_quantitycount#73, store_sales_quantityave#74, store_sales_quantitystdev#75, store_sales_quantitycov#76, as_store_returns_quantitycount#77, as_store_returns_quantityave#78, as_store_returns_quantitystdev#79, store_returns_quantitycov#80, catalog_sales_quantitycount#81, catalog_sales_quantityave#82, catalog_sales_quantitystdev#83, catalog_sales_quantitycov#84] -Arguments: 100, [i_item_id#27 ASC NULLS FIRST, i_item_desc#26 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#27, i_item_desc#26, s_state#23, store_sales_quantitycount#73, store_sales_quantityave#74, store_sales_quantitystdev#75, store_sales_quantitycov#76, as_store_returns_quantitycount#77, as_store_returns_quantityave#78, as_store_returns_quantitystdev#79, store_returns_quantitycov#80, catalog_sales_quantitycount#81, catalog_sales_quantityave#82, catalog_sales_quantitystdev#83, catalog_sales_quantitycov#84] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometNativeScan parquet spark_catalog.default.date_dim (44) - - -(44) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_quarter_name#85] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_quarter_name), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [2]: [d_date_sk#18, d_quarter_name#85] -Condition : ((isnotnull(d_quarter_name#85) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(d_quarter_name#85, 6)) = 2001Q1)) AND isnotnull(d_date_sk#18)) - -(46) CometProject -Input [2]: [d_date_sk#18, d_quarter_name#85] -Arguments: [d_date_sk#18], [d_date_sk#18] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#18] - -(48) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometNativeScan parquet spark_catalog.default.date_dim (49) - - -(49) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_quarter_name#86] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [2]: [d_date_sk#19, d_quarter_name#86] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(d_quarter_name#86, 6)) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#19)) - -(51) CometProject -Input [2]: [d_date_sk#19, d_quarter_name#86] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(53) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/simplified.txt deleted file mode 100644 index c63dd716a1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/simplified.txt +++ /dev/null @@ -1,79 +0,0 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] - WholeStageCodegen (9) - HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(ss_quantity),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(sr_return_quantity),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cs_quantity),stddev_samp(cast(cs_quantity as double)),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,s_state] #1 - WholeStageCodegen (8) - HashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] - Project [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [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] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Filter [cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometNativeScan 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/q17.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/explain.txt deleted file mode 100644 index e6d049b949..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,319 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometProject (18) - : : : : +- CometBroadcastHashJoin (17) - : : : : :- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (8) - : : : : +- CometBroadcastExchange (16) - : : : : +- CometProject (15) - : : : : +- CometFilter (14) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : : : +- CometBroadcastExchange (22) - : : : +- CometProject (21) - : : : +- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - : : +- ReusedExchange (25) - : +- CometBroadcastExchange (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (28) - +- CometBroadcastExchange (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (34) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(5) CometBroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(6) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(10) CometBroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(11) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] -Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight - -(12) CometProject -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_quarter_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_quarter_name), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#19, d_quarter_name#20] -Condition : ((isnotnull(d_quarter_name#20) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(d_quarter_name#20, 6)) = 2001Q1)) AND isnotnull(d_date_sk#19)) - -(15) CometProject -Input [2]: [d_date_sk#19, d_quarter_name#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#21, d_quarter_name#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#21, d_quarter_name#22] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(d_quarter_name#22, 6)) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#21)) - -(21) CometProject -Input [2]: [d_date_sk#21, d_quarter_name#22] -Arguments: [d_date_sk#21], [d_date_sk#21] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: [d_date_sk#21] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#21] -Arguments: [sr_returned_date_sk#12], [d_date_sk#21], Inner, BuildRight - -(24) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#21] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] - -(25) ReusedExchange [Reuses operator id: 22] -Output [1]: [d_date_sk#23] - -(26) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#23] -Arguments: [cs_sold_date_sk#17], [d_date_sk#23], Inner, BuildRight - -(27) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#23] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_state#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(29) CometFilter -Input [2]: [s_store_sk#24, s_state#25] -Condition : isnotnull(s_store_sk#24) - -(30) CometProject -Input [2]: [s_store_sk#24, s_state#25] -Arguments: [s_store_sk#24, s_state#26], [s_store_sk#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#25, 2)) AS s_state#26] - -(31) CometBroadcastExchange -Input [2]: [s_store_sk#24, s_state#26] -Arguments: [s_store_sk#24, s_state#26] - -(32) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Right output [2]: [s_store_sk#24, s_state#26] -Arguments: [ss_store_sk#3], [s_store_sk#24], Inner, BuildRight - -(33) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#24, s_state#26] -Arguments: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26], [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26] - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#27, i_item_id#28, i_item_desc#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [i_item_sk#27, i_item_id#28, i_item_desc#29] -Condition : isnotnull(i_item_sk#27) - -(36) CometProject -Input [3]: [i_item_sk#27, i_item_id#28, i_item_desc#29] -Arguments: [i_item_sk#27, i_item_id#30, i_item_desc#29], [i_item_sk#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#28, 16)) AS i_item_id#30, i_item_desc#29] - -(37) CometBroadcastExchange -Input [3]: [i_item_sk#27, i_item_id#30, i_item_desc#29] -Arguments: [i_item_sk#27, i_item_id#30, i_item_desc#29] - -(38) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26] -Right output [3]: [i_item_sk#27, i_item_id#30, i_item_desc#29] -Arguments: [ss_item_sk#1], [i_item_sk#27], Inner, BuildRight - -(39) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_sk#27, i_item_id#30, i_item_desc#29] -Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#30, i_item_desc#29], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#30, i_item_desc#29] - -(40) CometHashAggregate -Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#30, i_item_desc#29] -Keys [3]: [i_item_id#30, i_item_desc#29, s_state#26] -Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partial_stddev_samp(cast(ss_quantity#5 as double)), partial_count(sr_return_quantity#11), partial_avg(sr_return_quantity#11), partial_stddev_samp(cast(sr_return_quantity#11 as double)), partial_count(cs_quantity#16), partial_avg(cs_quantity#16), partial_stddev_samp(cast(cs_quantity#16 as double))] - -(41) CometExchange -Input [21]: [i_item_id#30, i_item_desc#29, s_state#26, count#31, sum#32, count#33, n#34, avg#35, m2#36, count#37, sum#38, count#39, n#40, avg#41, m2#42, count#43, sum#44, count#45, n#46, avg#47, m2#48] -Arguments: hashpartitioning(i_item_id#30, i_item_desc#29, s_state#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(42) CometHashAggregate -Input [21]: [i_item_id#30, i_item_desc#29, s_state#26, count#31, sum#32, count#33, n#34, avg#35, m2#36, count#37, sum#38, count#39, n#40, avg#41, m2#42, count#43, sum#44, count#45, n#46, avg#47, m2#48] -Keys [3]: [i_item_id#30, i_item_desc#29, s_state#26] -Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#11), avg(sr_return_quantity#11), stddev_samp(cast(sr_return_quantity#11 as double)), count(cs_quantity#16), avg(cs_quantity#16), stddev_samp(cast(cs_quantity#16 as double))] - -(43) CometTakeOrderedAndProject -Input [15]: [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#30 ASC NULLS FIRST,i_item_desc#29 ASC NULLS FIRST,s_state#26 ASC NULLS FIRST], output=[i_item_id#30,i_item_desc#29,s_state#26,store_sales_quantitycount#49,store_sales_quantityave#50,store_sales_quantitystdev#51,store_sales_quantitycov#52,as_store_returns_quantitycount#53,as_store_returns_quantityave#54,as_store_returns_quantitystdev#55,store_returns_quantitycov#56,catalog_sales_quantitycount#57,catalog_sales_quantityave#58,catalog_sales_quantitystdev#59,catalog_sales_quantitycov#60]), [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60], 100, 0, [i_item_id#30 ASC NULLS FIRST, i_item_desc#29 ASC NULLS FIRST, s_state#26 ASC NULLS FIRST], [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60] - -(44) CometColumnarToRow [codegen id : 1] -Input [15]: [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_quarter_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_quarter_name), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [d_date_sk#19, d_quarter_name#20] -Condition : ((isnotnull(d_quarter_name#20) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(d_quarter_name#20, 6)) = 2001Q1)) AND isnotnull(d_date_sk#19)) - -(47) CometProject -Input [2]: [d_date_sk#19, d_quarter_name#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(49) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) - - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#21, d_quarter_name#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [2]: [d_date_sk#21, d_quarter_name#22] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(d_quarter_name#22, 6)) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#21)) - -(52) CometProject -Input [2]: [d_date_sk#21, d_quarter_name#22] -Arguments: [d_date_sk#21], [d_date_sk#21] - -(53) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#21] - -(54) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/simplified.txt deleted file mode 100644 index 1d48f96b09..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,63 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] - CometHashAggregate [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count(ss_quantity),avg(ss_quantity),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(sr_return_quantity),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cs_quantity),stddev_samp(cast(cs_quantity as double))] - CometExchange [i_item_id,i_item_desc,s_state] #1 - CometHashAggregate [ss_quantity,sr_return_quantity,cs_quantity] [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] - CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - 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_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 [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 [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_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 [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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - 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 [native_iceberg_compat] 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 [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan [native_iceberg_compat] 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/q17/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/explain.txt deleted file mode 100644 index e6d049b949..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/explain.txt +++ /dev/null @@ -1,319 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometProject (18) - : : : : +- CometBroadcastHashJoin (17) - : : : : :- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (8) - : : : : +- CometBroadcastExchange (16) - : : : : +- CometProject (15) - : : : : +- CometFilter (14) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : : : +- CometBroadcastExchange (22) - : : : +- CometProject (21) - : : : +- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - : : +- ReusedExchange (25) - : +- CometBroadcastExchange (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (28) - +- CometBroadcastExchange (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (34) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(5) CometBroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(6) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(10) CometBroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(11) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] -Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight - -(12) CometProject -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_quarter_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_quarter_name), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#19, d_quarter_name#20] -Condition : ((isnotnull(d_quarter_name#20) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(d_quarter_name#20, 6)) = 2001Q1)) AND isnotnull(d_date_sk#19)) - -(15) CometProject -Input [2]: [d_date_sk#19, d_quarter_name#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#21, d_quarter_name#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#21, d_quarter_name#22] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(d_quarter_name#22, 6)) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#21)) - -(21) CometProject -Input [2]: [d_date_sk#21, d_quarter_name#22] -Arguments: [d_date_sk#21], [d_date_sk#21] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: [d_date_sk#21] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#21] -Arguments: [sr_returned_date_sk#12], [d_date_sk#21], Inner, BuildRight - -(24) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#21] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] - -(25) ReusedExchange [Reuses operator id: 22] -Output [1]: [d_date_sk#23] - -(26) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#23] -Arguments: [cs_sold_date_sk#17], [d_date_sk#23], Inner, BuildRight - -(27) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#23] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_state#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(29) CometFilter -Input [2]: [s_store_sk#24, s_state#25] -Condition : isnotnull(s_store_sk#24) - -(30) CometProject -Input [2]: [s_store_sk#24, s_state#25] -Arguments: [s_store_sk#24, s_state#26], [s_store_sk#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#25, 2)) AS s_state#26] - -(31) CometBroadcastExchange -Input [2]: [s_store_sk#24, s_state#26] -Arguments: [s_store_sk#24, s_state#26] - -(32) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Right output [2]: [s_store_sk#24, s_state#26] -Arguments: [ss_store_sk#3], [s_store_sk#24], Inner, BuildRight - -(33) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#24, s_state#26] -Arguments: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26], [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26] - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#27, i_item_id#28, i_item_desc#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [i_item_sk#27, i_item_id#28, i_item_desc#29] -Condition : isnotnull(i_item_sk#27) - -(36) CometProject -Input [3]: [i_item_sk#27, i_item_id#28, i_item_desc#29] -Arguments: [i_item_sk#27, i_item_id#30, i_item_desc#29], [i_item_sk#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#28, 16)) AS i_item_id#30, i_item_desc#29] - -(37) CometBroadcastExchange -Input [3]: [i_item_sk#27, i_item_id#30, i_item_desc#29] -Arguments: [i_item_sk#27, i_item_id#30, i_item_desc#29] - -(38) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26] -Right output [3]: [i_item_sk#27, i_item_id#30, i_item_desc#29] -Arguments: [ss_item_sk#1], [i_item_sk#27], Inner, BuildRight - -(39) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_sk#27, i_item_id#30, i_item_desc#29] -Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#30, i_item_desc#29], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#30, i_item_desc#29] - -(40) CometHashAggregate -Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#30, i_item_desc#29] -Keys [3]: [i_item_id#30, i_item_desc#29, s_state#26] -Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partial_stddev_samp(cast(ss_quantity#5 as double)), partial_count(sr_return_quantity#11), partial_avg(sr_return_quantity#11), partial_stddev_samp(cast(sr_return_quantity#11 as double)), partial_count(cs_quantity#16), partial_avg(cs_quantity#16), partial_stddev_samp(cast(cs_quantity#16 as double))] - -(41) CometExchange -Input [21]: [i_item_id#30, i_item_desc#29, s_state#26, count#31, sum#32, count#33, n#34, avg#35, m2#36, count#37, sum#38, count#39, n#40, avg#41, m2#42, count#43, sum#44, count#45, n#46, avg#47, m2#48] -Arguments: hashpartitioning(i_item_id#30, i_item_desc#29, s_state#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(42) CometHashAggregate -Input [21]: [i_item_id#30, i_item_desc#29, s_state#26, count#31, sum#32, count#33, n#34, avg#35, m2#36, count#37, sum#38, count#39, n#40, avg#41, m2#42, count#43, sum#44, count#45, n#46, avg#47, m2#48] -Keys [3]: [i_item_id#30, i_item_desc#29, s_state#26] -Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#11), avg(sr_return_quantity#11), stddev_samp(cast(sr_return_quantity#11 as double)), count(cs_quantity#16), avg(cs_quantity#16), stddev_samp(cast(cs_quantity#16 as double))] - -(43) CometTakeOrderedAndProject -Input [15]: [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#30 ASC NULLS FIRST,i_item_desc#29 ASC NULLS FIRST,s_state#26 ASC NULLS FIRST], output=[i_item_id#30,i_item_desc#29,s_state#26,store_sales_quantitycount#49,store_sales_quantityave#50,store_sales_quantitystdev#51,store_sales_quantitycov#52,as_store_returns_quantitycount#53,as_store_returns_quantityave#54,as_store_returns_quantitystdev#55,store_returns_quantitycov#56,catalog_sales_quantitycount#57,catalog_sales_quantityave#58,catalog_sales_quantitystdev#59,catalog_sales_quantitycov#60]), [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60], 100, 0, [i_item_id#30 ASC NULLS FIRST, i_item_desc#29 ASC NULLS FIRST, s_state#26 ASC NULLS FIRST], [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60] - -(44) CometColumnarToRow [codegen id : 1] -Input [15]: [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_quarter_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_quarter_name), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [d_date_sk#19, d_quarter_name#20] -Condition : ((isnotnull(d_quarter_name#20) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(d_quarter_name#20, 6)) = 2001Q1)) AND isnotnull(d_date_sk#19)) - -(47) CometProject -Input [2]: [d_date_sk#19, d_quarter_name#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(49) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) - - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#21, d_quarter_name#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [2]: [d_date_sk#21, d_quarter_name#22] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(d_quarter_name#22, 6)) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#21)) - -(52) CometProject -Input [2]: [d_date_sk#21, d_quarter_name#22] -Arguments: [d_date_sk#21], [d_date_sk#21] - -(53) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#21] - -(54) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/extended.txt deleted file mode 100644 index 4e1a4f3ec0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/extended.txt +++ /dev/null @@ -1,62 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 1d48f96b09..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt +++ /dev/null @@ -1,63 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] - CometHashAggregate [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count(ss_quantity),avg(ss_quantity),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(sr_return_quantity),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cs_quantity),stddev_samp(cast(cs_quantity as double))] - CometExchange [i_item_id,i_item_desc,s_state] #1 - CometHashAggregate [ss_quantity,sr_return_quantity,cs_quantity] [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] - CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - 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_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 [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 [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_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 [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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - 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 [native_iceberg_compat] 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 [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/explain.txt deleted file mode 100644 index f807104667..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/explain.txt +++ /dev/null @@ -1,295 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (46) -+- * HashAggregate (45) - +- * CometColumnarToRow (44) - +- CometColumnarExchange (43) - +- * HashAggregate (42) - +- * Expand (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * Project (23) - : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : :- * Project (17) - : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : :- * Project (10) - : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : :- * Filter (3) - : : : : : : +- * ColumnarToRow (2) - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : : +- BroadcastExchange (8) - : : : : : +- * CometColumnarToRow (7) - : : : : : +- CometProject (6) - : : : : : +- CometFilter (5) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) - : : : : +- BroadcastExchange (15) - : : : : +- * CometColumnarToRow (14) - : : : : +- CometProject (13) - : : : : +- CometFilter (12) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (11) - : : : +- BroadcastExchange (21) - : : : +- * CometColumnarToRow (20) - : : : +- CometFilter (19) - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (18) - : : +- BroadcastExchange (28) - : : +- * CometColumnarToRow (27) - : : +- CometProject (26) - : : +- CometFilter (25) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (24) - : +- ReusedExchange (31) - +- BroadcastExchange (38) - +- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometNativeScan parquet spark_catalog.default.item (34) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 7] -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] - -(3) Filter [codegen id : 7] -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) - -(4) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#12, 1)) = F)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#13, 20)) = Unknown )) AND isnotnull(cd_demo_sk#11)) - -(6) CometProject -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] - -(7) CometColumnarToRow [codegen id : 1] -Input [2]: [cd_demo_sk#11, cd_dep_count#14] - -(8) BroadcastExchange -Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 7] -Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] - -(11) CometNativeScan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [In(c_birth_month, [1,12,2,6,8,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(12) CometFilter -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Condition : (((c_birth_month#18 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) - -(13) CometProject -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(14) CometColumnarToRow [codegen id : 2] -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(15) BroadcastExchange -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(18) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(19) CometFilter -Input [1]: [cd_demo_sk#20] -Condition : isnotnull(cd_demo_sk#20) - -(20) CometColumnarToRow [codegen id : 3] -Input [1]: [cd_demo_sk#20] - -(21) BroadcastExchange -Input [1]: [cd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#16] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None - -(23) Project [codegen id : 7] -Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] - -(24) CometNativeScan parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#23, 2)) IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#21)) - -(26) CometProject -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#23, 2)) AS ca_state#25, ca_country#24] - -(27) CometColumnarToRow [codegen id : 4] -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(28) BroadcastExchange -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(29) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#17] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None - -(30) Project [codegen id : 7] -Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(31) ReusedExchange [Reuses operator id: 51] -Output [1]: [d_date_sk#26] - -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#26] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] - -(34) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#27, i_item_id#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [i_item_sk#27, i_item_id#28] -Condition : isnotnull(i_item_sk#27) - -(36) CometProject -Input [2]: [i_item_sk#27, i_item_id#28] -Arguments: [i_item_sk#27, i_item_id#29], [i_item_sk#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#28, 16)) AS i_item_id#29] - -(37) CometColumnarToRow [codegen id : 6] -Input [2]: [i_item_sk#27, i_item_id#29] - -(38) BroadcastExchange -Input [2]: [i_item_sk#27, i_item_id#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#27] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 7] -Output [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, ca_county#22] -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#27, i_item_id#29] - -(41) Expand [codegen id : 7] -Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, ca_county#22] -Arguments: [[cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, ca_county#22, 0], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, null, 1], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, null, null, 3], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, null, null, null, 7], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, null, null, null, null, 15]], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] - -(42) HashAggregate [codegen id : 7] -Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] -Keys [5]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] -Functions [7]: [partial_avg(cast(cs_quantity#4 as decimal(12,2))), partial_avg(cast(cs_list_price#5 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#7 as decimal(12,2))), partial_avg(cast(cs_sales_price#6 as decimal(12,2))), partial_avg(cast(cs_net_profit#8 as decimal(12,2))), partial_avg(cast(c_birth_year#19 as decimal(12,2))), partial_avg(cast(cd_dep_count#14 as decimal(12,2)))] -Aggregate Attributes [14]: [sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42, sum#43, count#44, sum#45, count#46, sum#47, count#48] -Results [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] - -(43) CometColumnarExchange -Input [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Arguments: hashpartitioning(i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(44) CometColumnarToRow [codegen id : 8] -Input [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] - -(45) HashAggregate [codegen id : 8] -Input [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Keys [5]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] -Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_price#5 as decimal(12,2))), avg(cast(cs_coupon_amt#7 as decimal(12,2))), avg(cast(cs_sales_price#6 as decimal(12,2))), avg(cast(cs_net_profit#8 as decimal(12,2))), avg(cast(c_birth_year#19 as decimal(12,2))), avg(cast(cd_dep_count#14 as decimal(12,2)))] -Aggregate Attributes [7]: [avg(cast(cs_quantity#4 as decimal(12,2)))#63, avg(cast(cs_list_price#5 as decimal(12,2)))#64, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#65, avg(cast(cs_sales_price#6 as decimal(12,2)))#66, avg(cast(cs_net_profit#8 as decimal(12,2)))#67, avg(cast(c_birth_year#19 as decimal(12,2)))#68, avg(cast(cd_dep_count#14 as decimal(12,2)))#69] -Results [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, avg(cast(cs_quantity#4 as decimal(12,2)))#63 AS agg1#70, avg(cast(cs_list_price#5 as decimal(12,2)))#64 AS agg2#71, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#65 AS agg3#72, avg(cast(cs_sales_price#6 as decimal(12,2)))#66 AS agg4#73, avg(cast(cs_net_profit#8 as decimal(12,2)))#67 AS agg5#74, avg(cast(c_birth_year#19 as decimal(12,2)))#68 AS agg6#75, avg(cast(cd_dep_count#14 as decimal(12,2)))#69 AS agg7#76] - -(46) TakeOrderedAndProject -Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#70, agg2#71, agg3#72, agg4#73, agg5#74, agg6#75, agg7#76] -Arguments: 100, [ca_country#31 ASC NULLS FIRST, ca_state#32 ASC NULLS FIRST, ca_county#33 ASC NULLS FIRST, i_item_id#30 ASC NULLS FIRST], [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#70, agg2#71, agg3#72, agg4#73, agg5#74, agg6#75, agg7#76] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (51) -+- * CometColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometNativeScan parquet spark_catalog.default.date_dim (47) - - -(47) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#77] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(48) CometFilter -Input [2]: [d_date_sk#26, d_year#77] -Condition : ((isnotnull(d_year#77) AND (d_year#77 = 1998)) AND isnotnull(d_date_sk#26)) - -(49) CometProject -Input [2]: [d_date_sk#26, d_year#77] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(50) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(51) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/simplified.txt deleted file mode 100644 index 77a45c46cd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - WholeStageCodegen (8) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2))),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 - WholeStageCodegen (7) - HashAggregate [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] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Expand [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] - Project [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] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [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] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan 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/q18.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/explain.txt deleted file mode 100644 index 674e83325e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,280 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (43) -+- CometTakeOrderedAndProject (42) - +- CometHashAggregate (41) - +- CometExchange (40) - +- CometHashAggregate (39) - +- CometExpand (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (19) - : : : +- CometBroadcastHashJoin (18) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometProject (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : : : +- CometBroadcastExchange (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (15) - : : +- CometBroadcastExchange (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#12, 1)) = F)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#13, 20)) = Unknown )) AND isnotnull(cd_demo_sk#11)) - -(5) CometProject -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] - -(6) CometBroadcastExchange -Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14] - -(7) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Right output [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight - -(8) CometProject -Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [In(c_birth_month, [1,12,2,6,8,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Condition : (((c_birth_month#18 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) - -(11) CometProject -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(12) CometBroadcastExchange -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(13) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight - -(14) CometProject -Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [cd_demo_sk#20] -Condition : isnotnull(cd_demo_sk#20) - -(17) CometBroadcastExchange -Input [1]: [cd_demo_sk#20] -Arguments: [cd_demo_sk#20] - -(18) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Right output [1]: [cd_demo_sk#20] -Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight - -(19) CometProject -Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#23, 2)) IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#21)) - -(22) CometProject -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#23, 2)) AS ca_state#25, ca_country#24] - -(23) CometBroadcastExchange -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(24) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -Right output [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight - -(25) CometProject -Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 1998)) AND isnotnull(d_date_sk#26)) - -(28) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(29) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] - -(30) CometBroadcastHashJoin -Left output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [1]: [d_date_sk#26] -Arguments: [cs_sold_date_sk#9], [d_date_sk#26], Inner, BuildRight - -(31) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#28, i_item_id#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [i_item_sk#28, i_item_id#29] -Condition : isnotnull(i_item_sk#28) - -(34) CometProject -Input [2]: [i_item_sk#28, i_item_id#29] -Arguments: [i_item_sk#28, i_item_id#30], [i_item_sk#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#29, 16)) AS i_item_id#30] - -(35) CometBroadcastExchange -Input [2]: [i_item_sk#28, i_item_id#30] -Arguments: [i_item_sk#28, i_item_id#30] - -(36) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [2]: [i_item_sk#28, i_item_id#30] -Arguments: [cs_item_sk#3], [i_item_sk#28], Inner, BuildRight - -(37) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#28, i_item_id#30] -Arguments: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22] - -(38) CometExpand -Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22] -Arguments: [[cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22, 0], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, null, 1], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, null, null, 3], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, null, null, null, 7], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, null, null, null, null, 15]], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] - -(39) CometHashAggregate -Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] -Keys [5]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] -Functions [7]: [partial_avg(cast(cs_quantity#4 as decimal(12,2))), partial_avg(cast(cs_list_price#5 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#7 as decimal(12,2))), partial_avg(cast(cs_sales_price#6 as decimal(12,2))), partial_avg(cast(cs_net_profit#8 as decimal(12,2))), partial_avg(cast(c_birth_year#19 as decimal(12,2))), partial_avg(cast(cd_dep_count#14 as decimal(12,2)))] - -(40) CometExchange -Input [19]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35, sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49] -Arguments: hashpartitioning(i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(41) CometHashAggregate -Input [19]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35, sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49] -Keys [5]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] -Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_price#5 as decimal(12,2))), avg(cast(cs_coupon_amt#7 as decimal(12,2))), avg(cast(cs_sales_price#6 as decimal(12,2))), avg(cast(cs_net_profit#8 as decimal(12,2))), avg(cast(c_birth_year#19 as decimal(12,2))), avg(cast(cd_dep_count#14 as decimal(12,2)))] - -(42) CometTakeOrderedAndProject -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#32 ASC NULLS FIRST,ca_state#33 ASC NULLS FIRST,ca_county#34 ASC NULLS FIRST,i_item_id#31 ASC NULLS FIRST], output=[i_item_id#31,ca_country#32,ca_state#33,ca_county#34,agg1#50,agg2#51,agg3#52,agg4#53,agg5#54,agg6#55,agg7#56]), [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56], 100, 0, [ca_country#32 ASC NULLS FIRST, ca_state#33 ASC NULLS FIRST, ca_county#34 ASC NULLS FIRST, i_item_id#31 ASC NULLS FIRST], [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56] - -(43) CometColumnarToRow [codegen id : 1] -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) - - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 1998)) AND isnotnull(d_date_sk#26)) - -(46) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(48) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/simplified.txt deleted file mode 100644 index 6c2b8b2e4f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,53 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometHashAggregate [spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2)))] - CometExchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 - CometHashAggregate [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,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,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,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_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 [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 [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,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_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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - 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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange [cd_demo_sk] #5 - CometFilter [cd_demo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #8 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] 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/q18/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/explain.txt deleted file mode 100644 index 674e83325e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/explain.txt +++ /dev/null @@ -1,280 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (43) -+- CometTakeOrderedAndProject (42) - +- CometHashAggregate (41) - +- CometExchange (40) - +- CometHashAggregate (39) - +- CometExpand (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (19) - : : : +- CometBroadcastHashJoin (18) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometProject (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : : : +- CometBroadcastExchange (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (15) - : : +- CometBroadcastExchange (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#12, 1)) = F)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#13, 20)) = Unknown )) AND isnotnull(cd_demo_sk#11)) - -(5) CometProject -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] - -(6) CometBroadcastExchange -Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14] - -(7) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Right output [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight - -(8) CometProject -Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [In(c_birth_month, [1,12,2,6,8,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Condition : (((c_birth_month#18 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) - -(11) CometProject -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(12) CometBroadcastExchange -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(13) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight - -(14) CometProject -Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [cd_demo_sk#20] -Condition : isnotnull(cd_demo_sk#20) - -(17) CometBroadcastExchange -Input [1]: [cd_demo_sk#20] -Arguments: [cd_demo_sk#20] - -(18) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Right output [1]: [cd_demo_sk#20] -Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight - -(19) CometProject -Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#23, 2)) IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#21)) - -(22) CometProject -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#23, 2)) AS ca_state#25, ca_country#24] - -(23) CometBroadcastExchange -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(24) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -Right output [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight - -(25) CometProject -Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 1998)) AND isnotnull(d_date_sk#26)) - -(28) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(29) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] - -(30) CometBroadcastHashJoin -Left output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [1]: [d_date_sk#26] -Arguments: [cs_sold_date_sk#9], [d_date_sk#26], Inner, BuildRight - -(31) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#28, i_item_id#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [i_item_sk#28, i_item_id#29] -Condition : isnotnull(i_item_sk#28) - -(34) CometProject -Input [2]: [i_item_sk#28, i_item_id#29] -Arguments: [i_item_sk#28, i_item_id#30], [i_item_sk#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#29, 16)) AS i_item_id#30] - -(35) CometBroadcastExchange -Input [2]: [i_item_sk#28, i_item_id#30] -Arguments: [i_item_sk#28, i_item_id#30] - -(36) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [2]: [i_item_sk#28, i_item_id#30] -Arguments: [cs_item_sk#3], [i_item_sk#28], Inner, BuildRight - -(37) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#28, i_item_id#30] -Arguments: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22] - -(38) CometExpand -Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22] -Arguments: [[cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22, 0], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, null, 1], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, null, null, 3], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, null, null, null, 7], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, null, null, null, null, 15]], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] - -(39) CometHashAggregate -Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] -Keys [5]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] -Functions [7]: [partial_avg(cast(cs_quantity#4 as decimal(12,2))), partial_avg(cast(cs_list_price#5 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#7 as decimal(12,2))), partial_avg(cast(cs_sales_price#6 as decimal(12,2))), partial_avg(cast(cs_net_profit#8 as decimal(12,2))), partial_avg(cast(c_birth_year#19 as decimal(12,2))), partial_avg(cast(cd_dep_count#14 as decimal(12,2)))] - -(40) CometExchange -Input [19]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35, sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49] -Arguments: hashpartitioning(i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(41) CometHashAggregate -Input [19]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35, sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49] -Keys [5]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] -Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_price#5 as decimal(12,2))), avg(cast(cs_coupon_amt#7 as decimal(12,2))), avg(cast(cs_sales_price#6 as decimal(12,2))), avg(cast(cs_net_profit#8 as decimal(12,2))), avg(cast(c_birth_year#19 as decimal(12,2))), avg(cast(cd_dep_count#14 as decimal(12,2)))] - -(42) CometTakeOrderedAndProject -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#32 ASC NULLS FIRST,ca_state#33 ASC NULLS FIRST,ca_county#34 ASC NULLS FIRST,i_item_id#31 ASC NULLS FIRST], output=[i_item_id#31,ca_country#32,ca_state#33,ca_county#34,agg1#50,agg2#51,agg3#52,agg4#53,agg5#54,agg6#55,agg7#56]), [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56], 100, 0, [ca_country#32 ASC NULLS FIRST, ca_state#33 ASC NULLS FIRST, ca_county#34 ASC NULLS FIRST, i_item_id#31 ASC NULLS FIRST], [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56] - -(43) CometColumnarToRow [codegen id : 1] -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) - - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 1998)) AND isnotnull(d_date_sk#26)) - -(46) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(48) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/extended.txt deleted file mode 100644 index 0b554c7e7c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/extended.txt +++ /dev/null @@ -1,51 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 6c2b8b2e4f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt +++ /dev/null @@ -1,53 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometHashAggregate [spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2)))] - CometExchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 - CometHashAggregate [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,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,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,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_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 [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 [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,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_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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - 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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange [cd_demo_sk] #5 - CometFilter [cd_demo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #8 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/explain.txt deleted file mode 100644 index 0e34c62d0c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (36) -+- CometTakeOrderedAndProject (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometNativeScan parquet spark_catalog.default.item (9) - : : +- CometBroadcastExchange (17) - : : +- CometFilter (16) - : : +- CometNativeScan parquet spark_catalog.default.customer (15) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometNativeScan parquet spark_catalog.default.customer_address (20) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometFilter (27) - +- CometNativeScan parquet spark_catalog.default.store (26) - - -(1) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1998)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1], [d_date_sk#1] - -(4) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_item_sk#4) AND isnotnull(ss_customer_sk#5)) AND isnotnull(ss_store_sk#6)) - -(6) CometBroadcastExchange -Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] - -(7) CometBroadcastHashJoin -Left output [1]: [d_date_sk#1] -Right output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [d_date_sk#1], [ss_sold_date_sk#8], Inner, BuildRight - -(8) CometProject -Input [6]: [d_date_sk#1, ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7], [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] - -(9) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Condition : ((isnotnull(i_manager_id#14) AND (i_manager_id#14 = 8)) AND isnotnull(i_item_sk#9)) - -(11) CometProject -Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Arguments: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [i_item_sk#9, i_brand_id#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#11, 50)) AS i_brand#15, i_manufact_id#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_manufact#13, 50)) AS i_manufact#16] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] -Right output [5]: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [ss_item_sk#4], [i_item_sk#9], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(15) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(c_current_addr_sk#18)) - -(17) CometBroadcastExchange -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [c_customer_sk#17, c_current_addr_sk#18] - -(18) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Right output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_customer_sk#5], [c_customer_sk#17], Inner, BuildRight - -(19) CometProject -Input [9]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18] - -(20) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_zip#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [ca_address_sk#19, ca_zip#20] -Condition : (isnotnull(ca_address_sk#19) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#20, 10)))) - -(22) CometProject -Input [2]: [ca_address_sk#19, ca_zip#20] -Arguments: [ca_address_sk#19, ca_zip#21], [ca_address_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#20, 10)) AS ca_zip#21] - -(23) CometBroadcastExchange -Input [2]: [ca_address_sk#19, ca_zip#21] -Arguments: [ca_address_sk#19, ca_zip#21] - -(24) CometBroadcastHashJoin -Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18] -Right output [2]: [ca_address_sk#19, ca_zip#21] -Arguments: [c_current_addr_sk#18], [ca_address_sk#19], Inner, BuildRight - -(25) CometProject -Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18, ca_address_sk#19, ca_zip#21] -Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21] - -(26) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#22, s_zip#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [s_store_sk#22, s_zip#23] -Condition : (isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#23, 10))) AND isnotnull(s_store_sk#22)) - -(28) CometProject -Input [2]: [s_store_sk#22, s_zip#23] -Arguments: [s_store_sk#22, s_zip#24], [s_store_sk#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#23, 10)) AS s_zip#24] - -(29) CometBroadcastExchange -Input [2]: [s_store_sk#22, s_zip#24] -Arguments: [s_store_sk#22, s_zip#24] - -(30) CometBroadcastHashJoin -Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21] -Right output [2]: [s_store_sk#22, s_zip#24] -Arguments: [ss_store_sk#6], [s_store_sk#22], Inner, NOT (substr(ca_zip#21, 1, 5) = substr(s_zip#24, 1, 5)), BuildRight - -(31) CometProject -Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21, s_store_sk#22, s_zip#24] -Arguments: [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(32) CometHashAggregate -Input [5]: [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Keys [4]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] - -(33) CometExchange -Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] -Arguments: hashpartitioning(i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(34) CometHashAggregate -Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] -Keys [4]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] - -(35) CometTakeOrderedAndProject -Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#28 DESC NULLS LAST,brand#27 ASC NULLS FIRST,brand_id#26 ASC NULLS FIRST,i_manufact_id#12 ASC NULLS FIRST,i_manufact#16 ASC NULLS FIRST], output=[brand_id#26,brand#27,i_manufact_id#12,i_manufact#16,ext_price#28]), [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28], 100, 0, [ext_price#28 DESC NULLS LAST, brand#27 ASC NULLS FIRST, brand_id#26 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#16 ASC NULLS FIRST], [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] - -(36) CometColumnarToRow [codegen id : 1] -Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/simplified.txt deleted file mode 100644 index 675500cd0b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_datafusion/simplified.txt +++ /dev/null @@ -1,38 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,i_manufact_id,i_manufact,ext_price] - CometHashAggregate [sum] [brand_id,brand,i_manufact_id,i_manufact,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 - CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] - CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - 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 [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,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,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_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 - CometProject [i_brand,i_manufact] [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_zip] #5 - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [s_store_sk,s_zip] #6 - CometProject [s_zip] [s_store_sk,s_zip] - CometFilter [s_store_sk,s_zip] - CometNativeScan 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/q19.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/explain.txt deleted file mode 100644 index 056ed8dd1f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (36) -+- CometTakeOrderedAndProject (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - : : +- CometBroadcastExchange (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (15) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1998)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1], [d_date_sk#1] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_item_sk#4) AND isnotnull(ss_customer_sk#5)) AND isnotnull(ss_store_sk#6)) - -(6) CometBroadcastExchange -Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] - -(7) CometBroadcastHashJoin -Left output [1]: [d_date_sk#1] -Right output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [d_date_sk#1], [ss_sold_date_sk#8], Inner, BuildRight - -(8) CometProject -Input [6]: [d_date_sk#1, ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7], [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Condition : ((isnotnull(i_manager_id#14) AND (i_manager_id#14 = 8)) AND isnotnull(i_item_sk#9)) - -(11) CometProject -Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Arguments: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [i_item_sk#9, i_brand_id#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#11, 50)) AS i_brand#15, i_manufact_id#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_manufact#13, 50)) AS i_manufact#16] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] -Right output [5]: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [ss_item_sk#4], [i_item_sk#9], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(c_current_addr_sk#18)) - -(17) CometBroadcastExchange -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [c_customer_sk#17, c_current_addr_sk#18] - -(18) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Right output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_customer_sk#5], [c_customer_sk#17], Inner, BuildRight - -(19) CometProject -Input [9]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_zip#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [ca_address_sk#19, ca_zip#20] -Condition : (isnotnull(ca_address_sk#19) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#20, 10)))) - -(22) CometProject -Input [2]: [ca_address_sk#19, ca_zip#20] -Arguments: [ca_address_sk#19, ca_zip#21], [ca_address_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#20, 10)) AS ca_zip#21] - -(23) CometBroadcastExchange -Input [2]: [ca_address_sk#19, ca_zip#21] -Arguments: [ca_address_sk#19, ca_zip#21] - -(24) CometBroadcastHashJoin -Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18] -Right output [2]: [ca_address_sk#19, ca_zip#21] -Arguments: [c_current_addr_sk#18], [ca_address_sk#19], Inner, BuildRight - -(25) CometProject -Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18, ca_address_sk#19, ca_zip#21] -Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#22, s_zip#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [s_store_sk#22, s_zip#23] -Condition : (isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#23, 10))) AND isnotnull(s_store_sk#22)) - -(28) CometProject -Input [2]: [s_store_sk#22, s_zip#23] -Arguments: [s_store_sk#22, s_zip#24], [s_store_sk#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#23, 10)) AS s_zip#24] - -(29) CometBroadcastExchange -Input [2]: [s_store_sk#22, s_zip#24] -Arguments: [s_store_sk#22, s_zip#24] - -(30) CometBroadcastHashJoin -Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21] -Right output [2]: [s_store_sk#22, s_zip#24] -Arguments: [ss_store_sk#6], [s_store_sk#22], Inner, NOT (substr(ca_zip#21, 1, 5) = substr(s_zip#24, 1, 5)), BuildRight - -(31) CometProject -Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21, s_store_sk#22, s_zip#24] -Arguments: [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(32) CometHashAggregate -Input [5]: [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Keys [4]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] - -(33) CometExchange -Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] -Arguments: hashpartitioning(i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(34) CometHashAggregate -Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] -Keys [4]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] - -(35) CometTakeOrderedAndProject -Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#28 DESC NULLS LAST,brand#27 ASC NULLS FIRST,brand_id#26 ASC NULLS FIRST,i_manufact_id#12 ASC NULLS FIRST,i_manufact#16 ASC NULLS FIRST], output=[brand_id#26,brand#27,i_manufact_id#12,i_manufact#16,ext_price#28]), [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28], 100, 0, [ext_price#28 DESC NULLS LAST, brand#27 ASC NULLS FIRST, brand_id#26 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#16 ASC NULLS FIRST], [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] - -(36) CometColumnarToRow [codegen id : 1] -Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/simplified.txt deleted file mode 100644 index 93ab89c142..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,38 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,i_manufact_id,i_manufact,ext_price] - CometHashAggregate [sum] [brand_id,brand,i_manufact_id,i_manufact,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 - CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] - CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - 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 [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,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,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_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 - CometProject [i_brand,i_manufact] [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_zip] #5 - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [s_store_sk,s_zip] #6 - CometProject [s_zip] [s_store_sk,s_zip] - CometFilter [s_store_sk,s_zip] - CometScan [native_iceberg_compat] 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/q19/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/explain.txt deleted file mode 100644 index 056ed8dd1f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (36) -+- CometTakeOrderedAndProject (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - : : +- CometBroadcastExchange (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (15) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1998)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1], [d_date_sk#1] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_item_sk#4) AND isnotnull(ss_customer_sk#5)) AND isnotnull(ss_store_sk#6)) - -(6) CometBroadcastExchange -Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] - -(7) CometBroadcastHashJoin -Left output [1]: [d_date_sk#1] -Right output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [d_date_sk#1], [ss_sold_date_sk#8], Inner, BuildRight - -(8) CometProject -Input [6]: [d_date_sk#1, ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7], [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Condition : ((isnotnull(i_manager_id#14) AND (i_manager_id#14 = 8)) AND isnotnull(i_item_sk#9)) - -(11) CometProject -Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Arguments: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [i_item_sk#9, i_brand_id#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#11, 50)) AS i_brand#15, i_manufact_id#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_manufact#13, 50)) AS i_manufact#16] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] -Right output [5]: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [ss_item_sk#4], [i_item_sk#9], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(c_current_addr_sk#18)) - -(17) CometBroadcastExchange -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [c_customer_sk#17, c_current_addr_sk#18] - -(18) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Right output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_customer_sk#5], [c_customer_sk#17], Inner, BuildRight - -(19) CometProject -Input [9]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_zip#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [ca_address_sk#19, ca_zip#20] -Condition : (isnotnull(ca_address_sk#19) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#20, 10)))) - -(22) CometProject -Input [2]: [ca_address_sk#19, ca_zip#20] -Arguments: [ca_address_sk#19, ca_zip#21], [ca_address_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#20, 10)) AS ca_zip#21] - -(23) CometBroadcastExchange -Input [2]: [ca_address_sk#19, ca_zip#21] -Arguments: [ca_address_sk#19, ca_zip#21] - -(24) CometBroadcastHashJoin -Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18] -Right output [2]: [ca_address_sk#19, ca_zip#21] -Arguments: [c_current_addr_sk#18], [ca_address_sk#19], Inner, BuildRight - -(25) CometProject -Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18, ca_address_sk#19, ca_zip#21] -Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#22, s_zip#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [s_store_sk#22, s_zip#23] -Condition : (isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#23, 10))) AND isnotnull(s_store_sk#22)) - -(28) CometProject -Input [2]: [s_store_sk#22, s_zip#23] -Arguments: [s_store_sk#22, s_zip#24], [s_store_sk#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#23, 10)) AS s_zip#24] - -(29) CometBroadcastExchange -Input [2]: [s_store_sk#22, s_zip#24] -Arguments: [s_store_sk#22, s_zip#24] - -(30) CometBroadcastHashJoin -Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21] -Right output [2]: [s_store_sk#22, s_zip#24] -Arguments: [ss_store_sk#6], [s_store_sk#22], Inner, NOT (substr(ca_zip#21, 1, 5) = substr(s_zip#24, 1, 5)), BuildRight - -(31) CometProject -Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21, s_store_sk#22, s_zip#24] -Arguments: [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(32) CometHashAggregate -Input [5]: [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Keys [4]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] - -(33) CometExchange -Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] -Arguments: hashpartitioning(i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(34) CometHashAggregate -Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] -Keys [4]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] - -(35) CometTakeOrderedAndProject -Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#28 DESC NULLS LAST,brand#27 ASC NULLS FIRST,brand_id#26 ASC NULLS FIRST,i_manufact_id#12 ASC NULLS FIRST,i_manufact#16 ASC NULLS FIRST], output=[brand_id#26,brand#27,i_manufact_id#12,i_manufact#16,ext_price#28]), [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28], 100, 0, [ext_price#28 DESC NULLS LAST, brand#27 ASC NULLS FIRST, brand_id#26 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#16 ASC NULLS FIRST], [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] - -(36) CometColumnarToRow [codegen id : 1] -Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/extended.txt deleted file mode 100644 index a8cfde2d31..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/extended.txt +++ /dev/null @@ -1,38 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 35 out of 35 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 93ab89c142..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt +++ /dev/null @@ -1,38 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,i_manufact_id,i_manufact,ext_price] - CometHashAggregate [sum] [brand_id,brand,i_manufact_id,i_manufact,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 - CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] - CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - 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 [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,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,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_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 - CometProject [i_brand,i_manufact] [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_zip] #5 - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [s_store_sk,s_zip] #6 - CometProject [s_zip] [s_store_sk,s_zip] - CometFilter [s_store_sk,s_zip] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_datafusion/explain.txt deleted file mode 100644 index 7ade9d6af8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_datafusion/explain.txt +++ /dev/null @@ -1,193 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (34) -+- CometSort (33) - +- CometExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometHashAggregate (14) - : : +- CometExchange (13) - : : +- CometHashAggregate (12) - : : +- CometProject (11) - : : +- CometBroadcastHashJoin (10) - : : :- CometUnion (5) - : : : :- CometProject (2) - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales (1) - : : : +- CometProject (4) - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales (3) - : : +- CometBroadcastExchange (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (6) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometNativeScan parquet spark_catalog.default.date_dim (15) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometBroadcastHashJoin (27) - :- CometHashAggregate (22) - : +- ReusedExchange (21) - +- CometBroadcastExchange (26) - +- CometProject (25) - +- CometFilter (24) - +- CometNativeScan parquet spark_catalog.default.date_dim (23) - - -(1) CometNativeScan parquet spark_catalog.default.web_sales -Output [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ws_sold_date_sk#2)] -ReadSchema: struct - -(2) CometProject -Input [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] -Arguments: [sold_date_sk#3, sales_price#4], [ws_sold_date_sk#2 AS sold_date_sk#3, ws_ext_sales_price#1 AS sales_price#4] - -(3) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(cs_sold_date_sk#6)] -ReadSchema: struct - -(4) CometProject -Input [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] -Arguments: [sold_date_sk#7, sales_price#8], [cs_sold_date_sk#6 AS sold_date_sk#7, cs_ext_sales_price#5 AS sales_price#8] - -(5) CometUnion -Child 0 Input [2]: [sold_date_sk#3, sales_price#4] -Child 1 Input [2]: [sold_date_sk#7, sales_price#8] - -(6) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Condition : (isnotnull(d_date_sk#9) AND isnotnull(d_week_seq#10)) - -(8) CometProject -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Arguments: [d_date_sk#9, d_week_seq#10, d_day_name#12], [d_date_sk#9, d_week_seq#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(d_day_name#11, 9)) AS d_day_name#12] - -(9) CometBroadcastExchange -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [d_date_sk#9, d_week_seq#10, d_day_name#12] - -(10) CometBroadcastHashJoin -Left output [2]: [sold_date_sk#3, sales_price#4] -Right output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [sold_date_sk#3], [d_date_sk#9], Inner, BuildRight - -(11) CometProject -Input [5]: [sold_date_sk#3, sales_price#4, d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [sales_price#4, d_week_seq#10, d_day_name#12], [sales_price#4, d_week_seq#10, d_day_name#12] - -(12) CometHashAggregate -Input [3]: [sales_price#4, d_week_seq#10, d_day_name#12] -Keys [1]: [d_week_seq#10] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(13) CometExchange -Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(14) CometHashAggregate -Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Keys [1]: [d_week_seq#10] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(15) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_week_seq)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [d_week_seq#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_week_seq#20)) - -(17) CometProject -Input [2]: [d_week_seq#20, d_year#21] -Arguments: [d_week_seq#20], [d_week_seq#20] - -(18) CometBroadcastExchange -Input [1]: [d_week_seq#20] -Arguments: [d_week_seq#20] - -(19) CometBroadcastHashJoin -Left output [8]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28] -Right output [1]: [d_week_seq#20] -Arguments: [d_week_seq#10], [d_week_seq#20], Inner, BuildRight - -(20) CometProject -Input [9]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28, d_week_seq#20] -Arguments: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36], [d_week_seq#10 AS d_week_seq1#29, sun_sales#22 AS sun_sales1#30, mon_sales#23 AS mon_sales1#31, tue_sales#24 AS tue_sales1#32, wed_sales#25 AS wed_sales1#33, thu_sales#26 AS thu_sales1#34, fri_sales#27 AS fri_sales1#35, sat_sales#28 AS sat_sales1#36] - -(21) ReusedExchange [Reuses operator id: 13] -Output [8]: [d_week_seq#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] - -(22) CometHashAggregate -Input [8]: [d_week_seq#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] -Keys [1]: [d_week_seq#37] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#45 = Sunday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Monday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Tuesday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Wednesday) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Thursday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Friday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Saturday ) THEN sales_price#46 END))] - -(23) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#47, d_year#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [d_week_seq#47, d_year#48] -Condition : ((isnotnull(d_year#48) AND (d_year#48 = 2002)) AND isnotnull(d_week_seq#47)) - -(25) CometProject -Input [2]: [d_week_seq#47, d_year#48] -Arguments: [d_week_seq#47], [d_week_seq#47] - -(26) CometBroadcastExchange -Input [1]: [d_week_seq#47] -Arguments: [d_week_seq#47] - -(27) CometBroadcastHashJoin -Left output [8]: [d_week_seq#37, sun_sales#49, mon_sales#50, tue_sales#51, wed_sales#52, thu_sales#53, fri_sales#54, sat_sales#55] -Right output [1]: [d_week_seq#47] -Arguments: [d_week_seq#37], [d_week_seq#47], Inner, BuildRight - -(28) CometProject -Input [9]: [d_week_seq#37, sun_sales#49, mon_sales#50, tue_sales#51, wed_sales#52, thu_sales#53, fri_sales#54, sat_sales#55, d_week_seq#47] -Arguments: [d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63], [d_week_seq#37 AS d_week_seq2#56, sun_sales#49 AS sun_sales2#57, mon_sales#50 AS mon_sales2#58, tue_sales#51 AS tue_sales2#59, wed_sales#52 AS wed_sales2#60, thu_sales#53 AS thu_sales2#61, fri_sales#54 AS fri_sales2#62, sat_sales#55 AS sat_sales2#63] - -(29) CometBroadcastExchange -Input [8]: [d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63] -Arguments: [d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63] - -(30) CometBroadcastHashJoin -Left output [8]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36] -Right output [8]: [d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63] -Arguments: [d_week_seq1#29], [(d_week_seq2#56 - 53)], Inner, BuildRight - -(31) CometProject -Input [16]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36, d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63] -Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70], [d_week_seq1#29, round((sun_sales1#30 / sun_sales2#57), 2) AS round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1#31 / mon_sales2#58), 2) AS round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1#32 / tue_sales2#59), 2) AS round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1#33 / wed_sales2#60), 2) AS round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1#34 / thu_sales2#61), 2) AS round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1#35 / fri_sales2#62), 2) AS round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1#36 / sat_sales2#63), 2) AS round((sat_sales1 / sat_sales2), 2)#70] - -(32) CometExchange -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70] -Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometSort -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70] -Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70], [d_week_seq1#29 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 1] -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_datafusion/simplified.txt deleted file mode 100644 index e4b6e81639..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_datafusion/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] - CometExchange [d_week_seq1] #1 - CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] - CometBroadcastHashJoin [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] - CometExchange [d_week_seq] #2 - CometHashAggregate [d_day_name,sales_price] [d_week_seq,sum,sum,sum,sum,sum,sum,sum] - CometProject [sales_price,d_week_seq,d_day_name] - 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] - CometNativeScan 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] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] - CometBroadcastExchange [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] - ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 - CometBroadcastExchange [d_week_seq] #6 - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometNativeScan 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/q2.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/explain.txt deleted file mode 100644 index a7a7ef7b7c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,193 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (34) -+- CometSort (33) - +- CometExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometHashAggregate (14) - : : +- CometExchange (13) - : : +- CometHashAggregate (12) - : : +- CometProject (11) - : : +- CometBroadcastHashJoin (10) - : : :- CometUnion (5) - : : : :- CometProject (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometProject (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (3) - : : +- CometBroadcastExchange (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (6) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (15) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometBroadcastHashJoin (27) - :- CometHashAggregate (22) - : +- ReusedExchange (21) - +- CometBroadcastExchange (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (23) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#2)] -ReadSchema: struct - -(2) CometProject -Input [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] -Arguments: [sold_date_sk#3, sales_price#4], [ws_sold_date_sk#2 AS sold_date_sk#3, ws_ext_sales_price#1 AS sales_price#4] - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#6)] -ReadSchema: struct - -(4) CometProject -Input [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] -Arguments: [sold_date_sk#7, sales_price#8], [cs_sold_date_sk#6 AS sold_date_sk#7, cs_ext_sales_price#5 AS sales_price#8] - -(5) CometUnion -Child 0 Input [2]: [sold_date_sk#3, sales_price#4] -Child 1 Input [2]: [sold_date_sk#7, sales_price#8] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Condition : (isnotnull(d_date_sk#9) AND isnotnull(d_week_seq#10)) - -(8) CometProject -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Arguments: [d_date_sk#9, d_week_seq#10, d_day_name#12], [d_date_sk#9, d_week_seq#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(d_day_name#11, 9)) AS d_day_name#12] - -(9) CometBroadcastExchange -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [d_date_sk#9, d_week_seq#10, d_day_name#12] - -(10) CometBroadcastHashJoin -Left output [2]: [sold_date_sk#3, sales_price#4] -Right output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [sold_date_sk#3], [d_date_sk#9], Inner, BuildRight - -(11) CometProject -Input [5]: [sold_date_sk#3, sales_price#4, d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [sales_price#4, d_week_seq#10, d_day_name#12], [sales_price#4, d_week_seq#10, d_day_name#12] - -(12) CometHashAggregate -Input [3]: [sales_price#4, d_week_seq#10, d_day_name#12] -Keys [1]: [d_week_seq#10] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(13) CometExchange -Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(14) CometHashAggregate -Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Keys [1]: [d_week_seq#10] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_week_seq)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [d_week_seq#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_week_seq#20)) - -(17) CometProject -Input [2]: [d_week_seq#20, d_year#21] -Arguments: [d_week_seq#20], [d_week_seq#20] - -(18) CometBroadcastExchange -Input [1]: [d_week_seq#20] -Arguments: [d_week_seq#20] - -(19) CometBroadcastHashJoin -Left output [8]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28] -Right output [1]: [d_week_seq#20] -Arguments: [d_week_seq#10], [d_week_seq#20], Inner, BuildRight - -(20) CometProject -Input [9]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28, d_week_seq#20] -Arguments: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36], [d_week_seq#10 AS d_week_seq1#29, sun_sales#22 AS sun_sales1#30, mon_sales#23 AS mon_sales1#31, tue_sales#24 AS tue_sales1#32, wed_sales#25 AS wed_sales1#33, thu_sales#26 AS thu_sales1#34, fri_sales#27 AS fri_sales1#35, sat_sales#28 AS sat_sales1#36] - -(21) ReusedExchange [Reuses operator id: 13] -Output [8]: [d_week_seq#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] - -(22) CometHashAggregate -Input [8]: [d_week_seq#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] -Keys [1]: [d_week_seq#37] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#45 = Sunday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Monday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Tuesday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Wednesday) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Thursday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Friday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Saturday ) THEN sales_price#46 END))] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#47, d_year#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [d_week_seq#47, d_year#48] -Condition : ((isnotnull(d_year#48) AND (d_year#48 = 2002)) AND isnotnull(d_week_seq#47)) - -(25) CometProject -Input [2]: [d_week_seq#47, d_year#48] -Arguments: [d_week_seq#47], [d_week_seq#47] - -(26) CometBroadcastExchange -Input [1]: [d_week_seq#47] -Arguments: [d_week_seq#47] - -(27) CometBroadcastHashJoin -Left output [8]: [d_week_seq#37, sun_sales#49, mon_sales#50, tue_sales#51, wed_sales#52, thu_sales#53, fri_sales#54, sat_sales#55] -Right output [1]: [d_week_seq#47] -Arguments: [d_week_seq#37], [d_week_seq#47], Inner, BuildRight - -(28) CometProject -Input [9]: [d_week_seq#37, sun_sales#49, mon_sales#50, tue_sales#51, wed_sales#52, thu_sales#53, fri_sales#54, sat_sales#55, d_week_seq#47] -Arguments: [d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63], [d_week_seq#37 AS d_week_seq2#56, sun_sales#49 AS sun_sales2#57, mon_sales#50 AS mon_sales2#58, tue_sales#51 AS tue_sales2#59, wed_sales#52 AS wed_sales2#60, thu_sales#53 AS thu_sales2#61, fri_sales#54 AS fri_sales2#62, sat_sales#55 AS sat_sales2#63] - -(29) CometBroadcastExchange -Input [8]: [d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63] -Arguments: [d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63] - -(30) CometBroadcastHashJoin -Left output [8]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36] -Right output [8]: [d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63] -Arguments: [d_week_seq1#29], [(d_week_seq2#56 - 53)], Inner, BuildRight - -(31) CometProject -Input [16]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36, d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63] -Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70], [d_week_seq1#29, round((sun_sales1#30 / sun_sales2#57), 2) AS round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1#31 / mon_sales2#58), 2) AS round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1#32 / tue_sales2#59), 2) AS round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1#33 / wed_sales2#60), 2) AS round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1#34 / thu_sales2#61), 2) AS round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1#35 / fri_sales2#62), 2) AS round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1#36 / sat_sales2#63), 2) AS round((sat_sales1 / sat_sales2), 2)#70] - -(32) CometExchange -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70] -Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometSort -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70] -Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70], [d_week_seq1#29 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 1] -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/simplified.txt deleted file mode 100644 index 852c5fca0f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] - CometExchange [d_week_seq1] #1 - CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] - CometBroadcastHashJoin [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] - CometExchange [d_week_seq] #2 - CometHashAggregate [d_day_name,sales_price] [d_week_seq,sum,sum,sum,sum,sum,sum,sum] - CometProject [sales_price,d_week_seq,d_day_name] - 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 [native_iceberg_compat] 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year] - CometBroadcastExchange [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] - ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 - CometBroadcastExchange [d_week_seq] #6 - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometScan [native_iceberg_compat] 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/q2/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/explain.txt deleted file mode 100644 index a7a7ef7b7c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/explain.txt +++ /dev/null @@ -1,193 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (34) -+- CometSort (33) - +- CometExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometHashAggregate (14) - : : +- CometExchange (13) - : : +- CometHashAggregate (12) - : : +- CometProject (11) - : : +- CometBroadcastHashJoin (10) - : : :- CometUnion (5) - : : : :- CometProject (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometProject (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (3) - : : +- CometBroadcastExchange (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (6) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (15) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometBroadcastHashJoin (27) - :- CometHashAggregate (22) - : +- ReusedExchange (21) - +- CometBroadcastExchange (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (23) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#2)] -ReadSchema: struct - -(2) CometProject -Input [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] -Arguments: [sold_date_sk#3, sales_price#4], [ws_sold_date_sk#2 AS sold_date_sk#3, ws_ext_sales_price#1 AS sales_price#4] - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#6)] -ReadSchema: struct - -(4) CometProject -Input [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] -Arguments: [sold_date_sk#7, sales_price#8], [cs_sold_date_sk#6 AS sold_date_sk#7, cs_ext_sales_price#5 AS sales_price#8] - -(5) CometUnion -Child 0 Input [2]: [sold_date_sk#3, sales_price#4] -Child 1 Input [2]: [sold_date_sk#7, sales_price#8] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Condition : (isnotnull(d_date_sk#9) AND isnotnull(d_week_seq#10)) - -(8) CometProject -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Arguments: [d_date_sk#9, d_week_seq#10, d_day_name#12], [d_date_sk#9, d_week_seq#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(d_day_name#11, 9)) AS d_day_name#12] - -(9) CometBroadcastExchange -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [d_date_sk#9, d_week_seq#10, d_day_name#12] - -(10) CometBroadcastHashJoin -Left output [2]: [sold_date_sk#3, sales_price#4] -Right output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [sold_date_sk#3], [d_date_sk#9], Inner, BuildRight - -(11) CometProject -Input [5]: [sold_date_sk#3, sales_price#4, d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [sales_price#4, d_week_seq#10, d_day_name#12], [sales_price#4, d_week_seq#10, d_day_name#12] - -(12) CometHashAggregate -Input [3]: [sales_price#4, d_week_seq#10, d_day_name#12] -Keys [1]: [d_week_seq#10] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(13) CometExchange -Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(14) CometHashAggregate -Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Keys [1]: [d_week_seq#10] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_week_seq)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [d_week_seq#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_week_seq#20)) - -(17) CometProject -Input [2]: [d_week_seq#20, d_year#21] -Arguments: [d_week_seq#20], [d_week_seq#20] - -(18) CometBroadcastExchange -Input [1]: [d_week_seq#20] -Arguments: [d_week_seq#20] - -(19) CometBroadcastHashJoin -Left output [8]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28] -Right output [1]: [d_week_seq#20] -Arguments: [d_week_seq#10], [d_week_seq#20], Inner, BuildRight - -(20) CometProject -Input [9]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28, d_week_seq#20] -Arguments: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36], [d_week_seq#10 AS d_week_seq1#29, sun_sales#22 AS sun_sales1#30, mon_sales#23 AS mon_sales1#31, tue_sales#24 AS tue_sales1#32, wed_sales#25 AS wed_sales1#33, thu_sales#26 AS thu_sales1#34, fri_sales#27 AS fri_sales1#35, sat_sales#28 AS sat_sales1#36] - -(21) ReusedExchange [Reuses operator id: 13] -Output [8]: [d_week_seq#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] - -(22) CometHashAggregate -Input [8]: [d_week_seq#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] -Keys [1]: [d_week_seq#37] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#45 = Sunday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Monday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Tuesday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Wednesday) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Thursday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Friday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Saturday ) THEN sales_price#46 END))] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#47, d_year#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [d_week_seq#47, d_year#48] -Condition : ((isnotnull(d_year#48) AND (d_year#48 = 2002)) AND isnotnull(d_week_seq#47)) - -(25) CometProject -Input [2]: [d_week_seq#47, d_year#48] -Arguments: [d_week_seq#47], [d_week_seq#47] - -(26) CometBroadcastExchange -Input [1]: [d_week_seq#47] -Arguments: [d_week_seq#47] - -(27) CometBroadcastHashJoin -Left output [8]: [d_week_seq#37, sun_sales#49, mon_sales#50, tue_sales#51, wed_sales#52, thu_sales#53, fri_sales#54, sat_sales#55] -Right output [1]: [d_week_seq#47] -Arguments: [d_week_seq#37], [d_week_seq#47], Inner, BuildRight - -(28) CometProject -Input [9]: [d_week_seq#37, sun_sales#49, mon_sales#50, tue_sales#51, wed_sales#52, thu_sales#53, fri_sales#54, sat_sales#55, d_week_seq#47] -Arguments: [d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63], [d_week_seq#37 AS d_week_seq2#56, sun_sales#49 AS sun_sales2#57, mon_sales#50 AS mon_sales2#58, tue_sales#51 AS tue_sales2#59, wed_sales#52 AS wed_sales2#60, thu_sales#53 AS thu_sales2#61, fri_sales#54 AS fri_sales2#62, sat_sales#55 AS sat_sales2#63] - -(29) CometBroadcastExchange -Input [8]: [d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63] -Arguments: [d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63] - -(30) CometBroadcastHashJoin -Left output [8]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36] -Right output [8]: [d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63] -Arguments: [d_week_seq1#29], [(d_week_seq2#56 - 53)], Inner, BuildRight - -(31) CometProject -Input [16]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36, d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63] -Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70], [d_week_seq1#29, round((sun_sales1#30 / sun_sales2#57), 2) AS round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1#31 / mon_sales2#58), 2) AS round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1#32 / tue_sales2#59), 2) AS round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1#33 / wed_sales2#60), 2) AS round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1#34 / thu_sales2#61), 2) AS round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1#35 / fri_sales2#62), 2) AS round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1#36 / sat_sales2#63), 2) AS round((sat_sales1 / sat_sales2), 2)#70] - -(32) CometExchange -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70] -Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometSort -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70] -Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70], [d_week_seq1#29 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 1] -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/extended.txt deleted file mode 100644 index 15f5db847d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/extended.txt +++ /dev/null @@ -1,48 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometProject - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 45 out of 45 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 852c5fca0f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] - CometExchange [d_week_seq1] #1 - CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] - CometBroadcastHashJoin [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] - CometExchange [d_week_seq] #2 - CometHashAggregate [d_day_name,sales_price] [d_week_seq,sum,sum,sum,sum,sum,sum,sum] - CometProject [sales_price,d_week_seq,d_day_name] - 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 [native_iceberg_compat] 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year] - CometBroadcastExchange [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] - ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 - CometBroadcastExchange [d_week_seq] #6 - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/explain.txt deleted file mode 100644 index aa781d42a5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/explain.txt +++ /dev/null @@ -1,163 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.item (4) - +- ReusedExchange (11) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] - -(7) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(8) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(11) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] - -(14) HashAggregate [codegen id : 3] -Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(15) CometColumnarExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(17) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#17] -Results [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#17,17,2) AS _w0#19, i_item_id#11] - -(18) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(22) Project [codegen id : 6] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] - -(23) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometNativeScan parquet spark_catalog.default.date_dim (24) - - -(24) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#22] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/simplified.txt deleted file mode 100644 index fb0ed62abe..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/explain.txt deleted file mode 100644 index e6612e7574..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [cs_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/simplified.txt deleted file mode 100644 index 2958d060fe..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt deleted file mode 100644 index e6612e7574..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [cs_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/extended.txt deleted file mode 100644 index cd52b2cd12..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 2958d060fe..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/explain.txt deleted file mode 100644 index fbabcc7275..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/explain.txt +++ /dev/null @@ -1,173 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (25) -+- * Filter (24) - +- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometColumnarExchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.inventory (1) - : : +- BroadcastExchange (7) - : : +- * CometColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometNativeScan parquet spark_catalog.default.warehouse (4) - : +- BroadcastExchange (14) - : +- * CometColumnarToRow (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometNativeScan parquet spark_catalog.default.item (10) - +- ReusedExchange (17) - - -(1) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_warehouse_sk), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_warehouse_sk#2) AND isnotnull(inv_item_sk#1)) - -(4) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(6) CometColumnarToRow [codegen id : 1] -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] - -(7) BroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#6] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 4] -Output [4]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] -Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_sk#6, w_warehouse_name#7] - -(10) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Condition : (((isnotnull(i_current_price#10) AND (i_current_price#10 >= 0.99)) AND (i_current_price#10 <= 1.49)) AND isnotnull(i_item_sk#8)) - -(12) CometProject -Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Arguments: [i_item_sk#8, i_item_id#11], [i_item_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#9, 16)) AS i_item_id#11] - -(13) CometColumnarToRow [codegen id : 2] -Input [2]: [i_item_sk#8, i_item_id#11] - -(14) BroadcastExchange -Input [2]: [i_item_sk#8, i_item_id#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#8] -Join type: Inner -Join condition: None - -(16) Project [codegen id : 4] -Output [4]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11] -Input [6]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_sk#8, i_item_id#11] - -(17) ReusedExchange [Reuses operator id: 29] -Output [2]: [d_date_sk#12, d_date#13] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] -Input [6]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11, d_date_sk#12, d_date#13] - -(20) HashAggregate [codegen id : 4] -Input [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] -Keys [2]: [w_warehouse_name#7, i_item_id#11] -Functions [2]: [partial_sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), partial_sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] -Aggregate Attributes [2]: [sum#14, sum#15] -Results [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] - -(21) CometColumnarExchange -Input [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] -Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] - -(23) HashAggregate [codegen id : 5] -Input [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] -Keys [2]: [w_warehouse_name#7, i_item_id#11] -Functions [2]: [sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] -Aggregate Attributes [2]: [sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18, sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#19] -Results [4]: [w_warehouse_name#7, i_item_id#11, sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18 AS inv_before#20, sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#19 AS inv_after#21] - -(24) Filter [codegen id : 5] -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#20, inv_after#21] -Condition : (CASE WHEN (inv_before#20 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#21 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#20 as double)))))) >= 0.666667) END AND CASE WHEN (inv_before#20 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#21 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#20 as double)))))) <= 1.5) END) - -(25) TakeOrderedAndProject -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#20, inv_after#21] -Arguments: 100, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#11, inv_before#20, inv_after#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometFilter (27) - +- CometNativeScan parquet spark_catalog.default.date_dim (26) - - -(26) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_date#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) - -(28) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_date#13] - -(29) BroadcastExchange -Input [2]: [d_date_sk#12, d_date#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/simplified.txt deleted file mode 100644 index 94925f8911..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] - WholeStageCodegen (5) - Filter [inv_before,inv_after] - HashAggregate [w_warehouse_name,i_item_id,sum,sum] [sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),inv_before,inv_after,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,i_item_id] #1 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] - Project [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Filter [inv_warehouse_sk,inv_item_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - InputAdapter - ReusedExchange [d_date_sk,d_date] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/explain.txt deleted file mode 100644 index 69e7bad682..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometFilter (22) - +- CometHashAggregate (21) - +- CometExchange (20) - +- CometHashAggregate (19) - +- CometProject (18) - +- CometBroadcastHashJoin (17) - :- CometProject (13) - : +- CometBroadcastHashJoin (12) - : :- CometProject (7) - : : +- CometBroadcastHashJoin (6) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : +- CometBroadcastExchange (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : +- CometBroadcastExchange (11) - : +- CometProject (10) - : +- CometFilter (9) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (8) - +- CometBroadcastExchange (16) - +- CometFilter (15) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_warehouse_sk), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_warehouse_sk#2) AND isnotnull(inv_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7], [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Condition : (((isnotnull(i_current_price#10) AND (i_current_price#10 >= 0.99)) AND (i_current_price#10 <= 1.49)) AND isnotnull(i_item_sk#8)) - -(10) CometProject -Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Arguments: [i_item_sk#8, i_item_id#11], [i_item_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#9, 16)) AS i_item_id#11] - -(11) CometBroadcastExchange -Input [2]: [i_item_sk#8, i_item_id#11] -Arguments: [i_item_sk#8, i_item_id#11] - -(12) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] -Right output [2]: [i_item_sk#8, i_item_id#11] -Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight - -(13) CometProject -Input [6]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_sk#8, i_item_id#11] -Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11], [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_date#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_date#13] -Arguments: [d_date_sk#12, d_date#13] - -(17) CometBroadcastHashJoin -Left output [4]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11] -Right output [2]: [d_date_sk#12, d_date#13] -Arguments: [inv_date_sk#4], [d_date_sk#12], Inner, BuildRight - -(18) CometProject -Input [6]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11, d_date_sk#12, d_date#13] -Arguments: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13], [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] - -(19) CometHashAggregate -Input [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] -Keys [2]: [w_warehouse_name#7, i_item_id#11] -Functions [2]: [partial_sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), partial_sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] - -(20) CometExchange -Input [4]: [w_warehouse_name#7, i_item_id#11, sum#14, sum#15] -Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [4]: [w_warehouse_name#7, i_item_id#11, sum#14, sum#15] -Keys [2]: [w_warehouse_name#7, i_item_id#11] -Functions [2]: [sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] - -(22) CometFilter -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] -Condition : (CASE WHEN (inv_before#16 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#17 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#16 as double)))))) >= 0.666667) END AND CASE WHEN (inv_before#16 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#17 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#16 as double)))))) <= 1.5) END) - -(23) CometTakeOrderedAndProject -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#7 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST], output=[w_warehouse_name#7,i_item_id#11,inv_before#16,inv_after#17]), [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17], 100, 0, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] - -(24) CometColumnarToRow [codegen id : 1] -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_date#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) - -(27) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_date#13] - -(28) BroadcastExchange -Input [2]: [d_date_sk#12, d_date#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/simplified.txt deleted file mode 100644 index 1c2e80c991..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,33 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] - CometFilter [w_warehouse_name,i_item_id,inv_before,inv_after] - CometHashAggregate [sum,sum] [w_warehouse_name,i_item_id,inv_before,inv_after,sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END)] - CometExchange [w_warehouse_name,i_item_id] #1 - CometHashAggregate [d_date,inv_quantity_on_hand] [w_warehouse_name,i_item_id,sum,sum] - CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - 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,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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_id] #4 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange [d_date_sk,d_date] #5 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/explain.txt deleted file mode 100644 index 69e7bad682..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometFilter (22) - +- CometHashAggregate (21) - +- CometExchange (20) - +- CometHashAggregate (19) - +- CometProject (18) - +- CometBroadcastHashJoin (17) - :- CometProject (13) - : +- CometBroadcastHashJoin (12) - : :- CometProject (7) - : : +- CometBroadcastHashJoin (6) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : +- CometBroadcastExchange (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : +- CometBroadcastExchange (11) - : +- CometProject (10) - : +- CometFilter (9) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (8) - +- CometBroadcastExchange (16) - +- CometFilter (15) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_warehouse_sk), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_warehouse_sk#2) AND isnotnull(inv_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7], [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Condition : (((isnotnull(i_current_price#10) AND (i_current_price#10 >= 0.99)) AND (i_current_price#10 <= 1.49)) AND isnotnull(i_item_sk#8)) - -(10) CometProject -Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Arguments: [i_item_sk#8, i_item_id#11], [i_item_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#9, 16)) AS i_item_id#11] - -(11) CometBroadcastExchange -Input [2]: [i_item_sk#8, i_item_id#11] -Arguments: [i_item_sk#8, i_item_id#11] - -(12) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] -Right output [2]: [i_item_sk#8, i_item_id#11] -Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight - -(13) CometProject -Input [6]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_sk#8, i_item_id#11] -Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11], [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_date#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_date#13] -Arguments: [d_date_sk#12, d_date#13] - -(17) CometBroadcastHashJoin -Left output [4]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11] -Right output [2]: [d_date_sk#12, d_date#13] -Arguments: [inv_date_sk#4], [d_date_sk#12], Inner, BuildRight - -(18) CometProject -Input [6]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11, d_date_sk#12, d_date#13] -Arguments: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13], [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] - -(19) CometHashAggregate -Input [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] -Keys [2]: [w_warehouse_name#7, i_item_id#11] -Functions [2]: [partial_sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), partial_sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] - -(20) CometExchange -Input [4]: [w_warehouse_name#7, i_item_id#11, sum#14, sum#15] -Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [4]: [w_warehouse_name#7, i_item_id#11, sum#14, sum#15] -Keys [2]: [w_warehouse_name#7, i_item_id#11] -Functions [2]: [sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] - -(22) CometFilter -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] -Condition : (CASE WHEN (inv_before#16 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#17 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#16 as double)))))) >= 0.666667) END AND CASE WHEN (inv_before#16 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#17 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#16 as double)))))) <= 1.5) END) - -(23) CometTakeOrderedAndProject -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#7 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST], output=[w_warehouse_name#7,i_item_id#11,inv_before#16,inv_after#17]), [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17], 100, 0, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] - -(24) CometColumnarToRow [codegen id : 1] -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_date#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) - -(27) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_date#13] - -(28) BroadcastExchange -Input [2]: [d_date_sk#12, d_date#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/extended.txt deleted file mode 100644 index ef02334507..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 1c2e80c991..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt +++ /dev/null @@ -1,33 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] - CometFilter [w_warehouse_name,i_item_id,inv_before,inv_after] - CometHashAggregate [sum,sum] [w_warehouse_name,i_item_id,inv_before,inv_after,sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END)] - CometExchange [w_warehouse_name,i_item_id] #1 - CometHashAggregate [d_date,inv_quantity_on_hand] [w_warehouse_name,i_item_id,sum,sum] - CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - 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,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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_id] #4 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange [d_date_sk,d_date] #5 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/explain.txt deleted file mode 100644 index 4de342c4d8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/explain.txt +++ /dev/null @@ -1,178 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (25) -+- * HashAggregate (24) - +- * CometColumnarToRow (23) - +- CometColumnarExchange (22) - +- * HashAggregate (21) - +- * Expand (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.inventory (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) - : +- CometProject (9) - : +- CometFilter (8) - : +- CometNativeScan parquet spark_catalog.default.item (7) - +- BroadcastExchange (17) - +- * CometColumnarToRow (16) - +- CometFilter (15) - +- CometNativeScan parquet spark_catalog.default.warehouse (14) - - -(1) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(4) ReusedExchange [Reuses operator id: 30] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(8) CometFilter -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Condition : isnotnull(i_item_sk#7) - -(9) CometProject -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#8, 50)) AS i_brand#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#11, 50)) AS i_product_name#15] - -(10) CometColumnarToRow [codegen id : 2] -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(11) BroadcastExchange -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(14) CometNativeScan parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(15) CometFilter -Input [1]: [w_warehouse_sk#16] -Condition : isnotnull(w_warehouse_sk#16) - -(16) CometColumnarToRow [codegen id : 3] -Input [1]: [w_warehouse_sk#16] - -(17) BroadcastExchange -Input [1]: [w_warehouse_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#16] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, i_category#14] -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15, w_warehouse_sk#16] - -(20) Expand [codegen id : 4] -Input [5]: [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, i_category#14] -Arguments: [[inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, i_category#14, 0], [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, null, 1], [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, null, null, 3], [inv_quantity_on_hand#3, i_product_name#15, null, null, null, 7], [inv_quantity_on_hand#3, null, null, null, null, 15]], [inv_quantity_on_hand#3, i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] - -(21) HashAggregate [codegen id : 4] -Input [6]: [inv_quantity_on_hand#3, i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] -Keys [5]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] -Functions [1]: [partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [sum#22, count#23] -Results [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, sum#24, count#25] - -(22) CometColumnarExchange -Input [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, sum#24, count#25] -Arguments: hashpartitioning(i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 5] -Input [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, sum#24, count#25] - -(24) HashAggregate [codegen id : 5] -Input [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, sum#24, count#25] -Keys [5]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#26] -Results [5]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, avg(inv_quantity_on_hand#3)#26 AS qoh#27] - -(25) TakeOrderedAndProject -Input [5]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, qoh#27] -Arguments: 100, [qoh#27 ASC NULLS FIRST, i_product_name#17 ASC NULLS FIRST, i_brand#18 ASC NULLS FIRST, i_class#19 ASC NULLS FIRST, i_category#20 ASC NULLS FIRST], [i_product_name#17, i_brand#18, i_class#19, i_category#20, qoh#27] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometNativeScan parquet spark_catalog.default.date_dim (26) - - -(26) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#28] -Condition : (((isnotnull(d_month_seq#28) AND (d_month_seq#28 >= 1200)) AND (d_month_seq#28 <= 1211)) AND isnotnull(d_date_sk#6)) - -(28) CometProject -Input [2]: [d_date_sk#6, d_month_seq#28] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(30) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/simplified.txt deleted file mode 100644 index 092e187177..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/simplified.txt +++ /dev/null @@ -1,44 +0,0 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (5) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/explain.txt deleted file mode 100644 index c45ad5f2fa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,178 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometTakeOrderedAndProject (24) - +- CometHashAggregate (23) - +- CometExchange (22) - +- CometHashAggregate (21) - +- CometExpand (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [inv_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] -Arguments: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3], [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Condition : isnotnull(i_item_sk#8) - -(11) CometProject -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16], [i_item_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#9, 50)) AS i_brand#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#10, 50)) AS i_class#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#11, 50)) AS i_category#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#12, 50)) AS i_product_name#16] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(13) CometBroadcastHashJoin -Left output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -Right output [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight - -(14) CometProject -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [w_warehouse_sk#17] -Condition : isnotnull(w_warehouse_sk#17) - -(17) CometBroadcastExchange -Input [1]: [w_warehouse_sk#17] -Arguments: [w_warehouse_sk#17] - -(18) CometBroadcastHashJoin -Left output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Right output [1]: [w_warehouse_sk#17] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#17], Inner, BuildRight - -(19) CometProject -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16, w_warehouse_sk#17] -Arguments: [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15], [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15] - -(20) CometExpand -Input [5]: [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15] -Arguments: [[inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15, 0], [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, null, 1], [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, null, null, 3], [inv_quantity_on_hand#3, i_product_name#16, null, null, null, 7], [inv_quantity_on_hand#3, null, null, null, null, 15]], [inv_quantity_on_hand#3, i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] - -(21) CometHashAggregate -Input [6]: [inv_quantity_on_hand#3, i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] -Keys [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] -Functions [1]: [partial_avg(inv_quantity_on_hand#3)] - -(22) CometExchange -Input [7]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22, sum#23, count#24] -Arguments: hashpartitioning(i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [7]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22, sum#23, count#24] -Keys [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] -Functions [1]: [avg(inv_quantity_on_hand#3)] - -(24) CometTakeOrderedAndProject -Input [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#25 ASC NULLS FIRST,i_product_name#18 ASC NULLS FIRST,i_brand#19 ASC NULLS FIRST,i_class#20 ASC NULLS FIRST,i_category#21 ASC NULLS FIRST], output=[i_product_name#18,i_brand#19,i_class#20,i_category#21,qoh#25]), [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25], 100, 0, [qoh#25 ASC NULLS FIRST, i_product_name#18 ASC NULLS FIRST, i_brand#19 ASC NULLS FIRST, i_class#20 ASC NULLS FIRST, i_category#21 ASC NULLS FIRST], [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] - -(25) CometColumnarToRow [codegen id : 1] -Input [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6)) - -(28) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(30) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/simplified.txt deleted file mode 100644 index 9119ee749d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] - CometHashAggregate [spark_grouping_id,sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(inv_quantity_on_hand)] - CometExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - CometHashAggregate [inv_quantity_on_hand] [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] - 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,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,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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange [w_warehouse_sk] #5 - CometFilter [w_warehouse_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/explain.txt deleted file mode 100644 index c45ad5f2fa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/explain.txt +++ /dev/null @@ -1,178 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometTakeOrderedAndProject (24) - +- CometHashAggregate (23) - +- CometExchange (22) - +- CometHashAggregate (21) - +- CometExpand (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [inv_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] -Arguments: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3], [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Condition : isnotnull(i_item_sk#8) - -(11) CometProject -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16], [i_item_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#9, 50)) AS i_brand#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#10, 50)) AS i_class#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#11, 50)) AS i_category#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#12, 50)) AS i_product_name#16] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(13) CometBroadcastHashJoin -Left output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -Right output [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight - -(14) CometProject -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [w_warehouse_sk#17] -Condition : isnotnull(w_warehouse_sk#17) - -(17) CometBroadcastExchange -Input [1]: [w_warehouse_sk#17] -Arguments: [w_warehouse_sk#17] - -(18) CometBroadcastHashJoin -Left output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Right output [1]: [w_warehouse_sk#17] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#17], Inner, BuildRight - -(19) CometProject -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16, w_warehouse_sk#17] -Arguments: [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15], [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15] - -(20) CometExpand -Input [5]: [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15] -Arguments: [[inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15, 0], [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, null, 1], [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, null, null, 3], [inv_quantity_on_hand#3, i_product_name#16, null, null, null, 7], [inv_quantity_on_hand#3, null, null, null, null, 15]], [inv_quantity_on_hand#3, i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] - -(21) CometHashAggregate -Input [6]: [inv_quantity_on_hand#3, i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] -Keys [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] -Functions [1]: [partial_avg(inv_quantity_on_hand#3)] - -(22) CometExchange -Input [7]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22, sum#23, count#24] -Arguments: hashpartitioning(i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [7]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22, sum#23, count#24] -Keys [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] -Functions [1]: [avg(inv_quantity_on_hand#3)] - -(24) CometTakeOrderedAndProject -Input [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#25 ASC NULLS FIRST,i_product_name#18 ASC NULLS FIRST,i_brand#19 ASC NULLS FIRST,i_class#20 ASC NULLS FIRST,i_category#21 ASC NULLS FIRST], output=[i_product_name#18,i_brand#19,i_class#20,i_category#21,qoh#25]), [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25], 100, 0, [qoh#25 ASC NULLS FIRST, i_product_name#18 ASC NULLS FIRST, i_brand#19 ASC NULLS FIRST, i_class#20 ASC NULLS FIRST, i_category#21 ASC NULLS FIRST], [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] - -(25) CometColumnarToRow [codegen id : 1] -Input [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6)) - -(28) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(30) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/extended.txt deleted file mode 100644 index 7369619d2f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/extended.txt +++ /dev/null @@ -1,33 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - -Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 9119ee749d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] - CometHashAggregate [spark_grouping_id,sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(inv_quantity_on_hand)] - CometExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - CometHashAggregate [inv_quantity_on_hand] [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] - 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,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,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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange [w_warehouse_sk] #5 - CometFilter [w_warehouse_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/explain.txt deleted file mode 100644 index 029c7fd3df..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/explain.txt +++ /dev/null @@ -1,602 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (69) -+- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometUnion (65) - :- CometProject (47) - : +- CometBroadcastHashJoin (46) - : :- CometProject (41) - : : +- CometSortMergeJoin (40) - : : :- CometSort (25) - : : : +- CometColumnarExchange (24) - : : : +- * Project (23) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : : : :- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : +- BroadcastExchange (21) - : : : +- * Project (20) - : : : +- * Filter (19) - : : : +- * HashAggregate (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometColumnarExchange (16) - : : : +- * HashAggregate (15) - : : : +- * Project (14) - : : : +- * BroadcastHashJoin Inner BuildRight (13) - : : : :- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * Filter (5) - : : : : : +- * ColumnarToRow (4) - : : : : : +- Scan parquet spark_catalog.default.store_sales (3) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (12) - : : : +- * CometColumnarToRow (11) - : : : +- CometFilter (10) - : : : +- CometNativeScan parquet spark_catalog.default.item (9) - : : +- CometSort (39) - : : +- CometProject (38) - : : +- CometFilter (37) - : : +- CometHashAggregate (36) - : : +- CometExchange (35) - : : +- CometHashAggregate (34) - : : +- CometProject (33) - : : +- CometBroadcastHashJoin (32) - : : :- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometNativeScan parquet spark_catalog.default.store_sales (26) - : : +- CometBroadcastExchange (31) - : : +- CometFilter (30) - : : +- CometNativeScan parquet spark_catalog.default.customer (29) - : +- CometBroadcastExchange (45) - : +- CometProject (44) - : +- CometFilter (43) - : +- CometNativeScan parquet spark_catalog.default.date_dim (42) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometSortMergeJoin (60) - : :- CometSort (54) - : : +- CometColumnarExchange (53) - : : +- * Project (52) - : : +- * BroadcastHashJoin LeftSemi BuildRight (51) - : : :- * ColumnarToRow (49) - : : : +- Scan parquet spark_catalog.default.web_sales (48) - : : +- ReusedExchange (50) - : +- CometSort (59) - : +- CometProject (58) - : +- CometFilter (57) - : +- CometHashAggregate (56) - : +- ReusedExchange (55) - +- ReusedExchange (62) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(3) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(4) ColumnarToRow [codegen id : 3] -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] - -(5) Filter [codegen id : 3] -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_item_sk#7) - -(6) ReusedExchange [Reuses operator id: 79] -Output [2]: [d_date_sk#10, d_date#11] - -(7) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(8) Project [codegen id : 3] -Output [2]: [ss_item_sk#7, d_date#11] -Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] - -(9) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#12, i_item_desc#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [i_item_sk#12, i_item_desc#13] -Condition : isnotnull(i_item_sk#12) - -(11) CometColumnarToRow [codegen id : 2] -Input [2]: [i_item_sk#12, i_item_desc#13] - -(12) BroadcastExchange -Input [2]: [i_item_sk#12, i_item_desc#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#7] -Right keys [1]: [i_item_sk#12] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 3] -Output [3]: [d_date#11, i_item_sk#12, substr(i_item_desc#13, 1, 30) AS _groupingexpression#14] -Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#12, i_item_desc#13] - -(15) HashAggregate [codegen id : 3] -Input [3]: [d_date#11, i_item_sk#12, _groupingexpression#14] -Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#15] -Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] - -(16) CometColumnarExchange -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(17) CometColumnarToRow [codegen id : 4] -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] - -(18) HashAggregate [codegen id : 4] -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#17] -Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] - -(19) Filter [codegen id : 4] -Input [2]: [item_sk#18, cnt#19] -Condition : (cnt#19 > 4) - -(20) Project [codegen id : 4] -Output [1]: [item_sk#18] -Input [2]: [item_sk#18, cnt#19] - -(21) BroadcastExchange -Input [1]: [item_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(22) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [item_sk#18] -Join type: LeftSemi -Join condition: None - -(23) Project [codegen id : 5] -Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(24) CometColumnarExchange -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] - -(26) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_customer_sk#20) - -(28) CometProject -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -Arguments: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22], [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] - -(29) CometNativeScan parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(30) CometFilter -Input [1]: [c_customer_sk#24] -Condition : isnotnull(c_customer_sk#24) - -(31) CometBroadcastExchange -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24] - -(32) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] -Right output [1]: [c_customer_sk#24] -Arguments: [ss_customer_sk#20], [c_customer_sk#24], Inner, BuildRight - -(33) CometProject -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -Arguments: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24], [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] - -(34) CometHashAggregate -Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -Keys [1]: [c_customer_sk#24] -Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] - -(35) CometExchange -Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] -Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(36) CometHashAggregate -Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] -Keys [1]: [c_customer_sk#24] -Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] - -(37) CometFilter -Input [2]: [c_customer_sk#24, ssales#27] -Condition : (isnotnull(ssales#27) AND (cast(ssales#27 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#28, [id=#29]))) - -(38) CometProject -Input [2]: [c_customer_sk#24, ssales#27] -Arguments: [c_customer_sk#24], [c_customer_sk#24] - -(39) CometSort -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24], [c_customer_sk#24 ASC NULLS FIRST] - -(40) CometSortMergeJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [c_customer_sk#24] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#24], LeftSemi - -(41) CometProject -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(42) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#30, d_year#31, d_moy#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [3]: [d_date_sk#30, d_year#31, d_moy#32] -Condition : ((((isnotnull(d_year#31) AND isnotnull(d_moy#32)) AND (d_year#31 = 2000)) AND (d_moy#32 = 2)) AND isnotnull(d_date_sk#30)) - -(44) CometProject -Input [3]: [d_date_sk#30, d_year#31, d_moy#32] -Arguments: [d_date_sk#30], [d_date_sk#30] - -(45) CometBroadcastExchange -Input [1]: [d_date_sk#30] -Arguments: [d_date_sk#30] - -(46) CometBroadcastHashJoin -Left output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [d_date_sk#30] -Arguments: [cs_sold_date_sk#5], [d_date_sk#30], Inner, BuildRight - -(47) CometProject -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#30] -Arguments: [sales#33], [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#33] - -(48) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#34, ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#6)] -ReadSchema: struct - -(49) ColumnarToRow [codegen id : 10] -Input [5]: [ws_item_sk#34, ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] - -(50) ReusedExchange [Reuses operator id: 21] -Output [1]: [item_sk#39] - -(51) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [item_sk#39] -Join type: LeftSemi -Join condition: None - -(52) Project [codegen id : 10] -Output [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -Input [5]: [ws_item_sk#34, ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] - -(53) CometColumnarExchange -Input [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -Arguments: hashpartitioning(ws_bill_customer_sk#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(54) CometSort -Input [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -Arguments: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38], [ws_bill_customer_sk#35 ASC NULLS FIRST] - -(55) ReusedExchange [Reuses operator id: 35] -Output [3]: [c_customer_sk#40, sum#41, isEmpty#42] - -(56) CometHashAggregate -Input [3]: [c_customer_sk#40, sum#41, isEmpty#42] -Keys [1]: [c_customer_sk#40] -Functions [1]: [sum((cast(ss_quantity#43 as decimal(10,0)) * ss_sales_price#44))] - -(57) CometFilter -Input [2]: [c_customer_sk#40, ssales#45] -Condition : (isnotnull(ssales#45) AND (cast(ssales#45 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#28, [id=#29]))) - -(58) CometProject -Input [2]: [c_customer_sk#40, ssales#45] -Arguments: [c_customer_sk#40], [c_customer_sk#40] - -(59) CometSort -Input [1]: [c_customer_sk#40] -Arguments: [c_customer_sk#40], [c_customer_sk#40 ASC NULLS FIRST] - -(60) CometSortMergeJoin -Left output [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -Right output [1]: [c_customer_sk#40] -Arguments: [ws_bill_customer_sk#35], [c_customer_sk#40], LeftSemi - -(61) CometProject -Input [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -Arguments: [ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38], [ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] - -(62) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#46] - -(63) CometBroadcastHashJoin -Left output [3]: [ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -Right output [1]: [d_date_sk#46] -Arguments: [ws_sold_date_sk#38], [d_date_sk#46], Inner, BuildRight - -(64) CometProject -Input [4]: [ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38, d_date_sk#46] -Arguments: [sales#47], [(cast(ws_quantity#36 as decimal(10,0)) * ws_list_price#37) AS sales#47] - -(65) CometUnion -Child 0 Input [1]: [sales#33] -Child 1 Input [1]: [sales#47] - -(66) CometHashAggregate -Input [1]: [sales#33] -Keys: [] -Functions [1]: [partial_sum(sales#33)] - -(67) CometExchange -Input [2]: [sum#48, isEmpty#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(68) CometHashAggregate -Input [2]: [sum#48, isEmpty#49] -Keys: [] -Functions [1]: [sum(sales#33)] - -(69) CometColumnarToRow [codegen id : 11] -Input [1]: [sum(sales)#50] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (74) -+- * CometColumnarToRow (73) - +- CometProject (72) - +- CometFilter (71) - +- CometNativeScan parquet spark_catalog.default.date_dim (70) - - -(70) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#30, d_year#31, d_moy#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(71) CometFilter -Input [3]: [d_date_sk#30, d_year#31, d_moy#32] -Condition : ((((isnotnull(d_year#31) AND isnotnull(d_moy#32)) AND (d_year#31 = 2000)) AND (d_moy#32 = 2)) AND isnotnull(d_date_sk#30)) - -(72) CometProject -Input [3]: [d_date_sk#30, d_year#31, d_moy#32] -Arguments: [d_date_sk#30], [d_date_sk#30] - -(73) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#30] - -(74) BroadcastExchange -Input [1]: [d_date_sk#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (79) -+- * CometColumnarToRow (78) - +- CometProject (77) - +- CometFilter (76) - +- CometNativeScan parquet spark_catalog.default.date_dim (75) - - -(75) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(76) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#51] -Condition : (d_year#51 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(77) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#51] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(78) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_date#11] - -(79) BroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:3 Hosting operator id = 37 Hosting Expression = Subquery scalar-subquery#28, [id=#29] -* HashAggregate (99) -+- * CometColumnarToRow (98) - +- CometColumnarExchange (97) - +- * HashAggregate (96) - +- * HashAggregate (95) - +- * CometColumnarToRow (94) - +- CometColumnarExchange (93) - +- * HashAggregate (92) - +- * Project (91) - +- * BroadcastHashJoin Inner BuildRight (90) - :- * Project (88) - : +- * BroadcastHashJoin Inner BuildRight (87) - : :- * Filter (82) - : : +- * ColumnarToRow (81) - : : +- Scan parquet spark_catalog.default.store_sales (80) - : +- BroadcastExchange (86) - : +- * CometColumnarToRow (85) - : +- CometFilter (84) - : +- CometNativeScan parquet spark_catalog.default.customer (83) - +- ReusedExchange (89) - - -(80) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#55), dynamicpruningexpression(ss_sold_date_sk#55 IN dynamicpruning#56)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(81) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55] - -(82) Filter [codegen id : 3] -Input [4]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55] -Condition : isnotnull(ss_customer_sk#52) - -(83) CometNativeScan parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#57] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(84) CometFilter -Input [1]: [c_customer_sk#57] -Condition : isnotnull(c_customer_sk#57) - -(85) CometColumnarToRow [codegen id : 1] -Input [1]: [c_customer_sk#57] - -(86) BroadcastExchange -Input [1]: [c_customer_sk#57] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -(87) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#52] -Right keys [1]: [c_customer_sk#57] -Join type: Inner -Join condition: None - -(88) Project [codegen id : 3] -Output [4]: [ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57] -Input [5]: [ss_customer_sk#52, ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57] - -(89) ReusedExchange [Reuses operator id: 104] -Output [1]: [d_date_sk#58] - -(90) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#55] -Right keys [1]: [d_date_sk#58] -Join type: Inner -Join condition: None - -(91) Project [codegen id : 3] -Output [3]: [ss_quantity#53, ss_sales_price#54, c_customer_sk#57] -Input [5]: [ss_quantity#53, ss_sales_price#54, ss_sold_date_sk#55, c_customer_sk#57, d_date_sk#58] - -(92) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#53, ss_sales_price#54, c_customer_sk#57] -Keys [1]: [c_customer_sk#57] -Functions [1]: [partial_sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))] -Aggregate Attributes [2]: [sum#59, isEmpty#60] -Results [3]: [c_customer_sk#57, sum#61, isEmpty#62] - -(93) CometColumnarExchange -Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] -Arguments: hashpartitioning(c_customer_sk#57, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(94) CometColumnarToRow [codegen id : 4] -Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] - -(95) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#57, sum#61, isEmpty#62] -Keys [1]: [c_customer_sk#57] -Functions [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))#63] -Results [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))#63 AS csales#64] - -(96) HashAggregate [codegen id : 4] -Input [1]: [csales#64] -Keys: [] -Functions [1]: [partial_max(csales#64)] -Aggregate Attributes [1]: [max#65] -Results [1]: [max#66] - -(97) CometColumnarExchange -Input [1]: [max#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(98) CometColumnarToRow [codegen id : 5] -Input [1]: [max#66] - -(99) HashAggregate [codegen id : 5] -Input [1]: [max#66] -Keys: [] -Functions [1]: [max(csales#64)] -Aggregate Attributes [1]: [max(csales#64)#67] -Results [1]: [max(csales#64)#67 AS tpcds_cmax#68] - -Subquery:4 Hosting operator id = 80 Hosting Expression = ss_sold_date_sk#55 IN dynamicpruning#56 -BroadcastExchange (104) -+- * CometColumnarToRow (103) - +- CometProject (102) - +- CometFilter (101) - +- CometNativeScan parquet spark_catalog.default.date_dim (100) - - -(100) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#58, d_year#69] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(101) CometFilter -Input [2]: [d_date_sk#58, d_year#69] -Condition : (d_year#69 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#58)) - -(102) CometProject -Input [2]: [d_date_sk#58, d_year#69] -Arguments: [d_date_sk#58], [d_date_sk#58] - -(103) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#58] - -(104) BroadcastExchange -Input [1]: [d_date_sk#58] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:5 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#6 - -Subquery:6 Hosting operator id = 57 Hosting Expression = ReusedSubquery Subquery scalar-subquery#28, [id=#29] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/simplified.txt deleted file mode 100644 index d42a6ba29e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/simplified.txt +++ /dev/null @@ -1,142 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,isEmpty] [sum(sales),sum(sales)] - CometExchange #1 - CometHashAggregate [sales] [sum,isEmpty] - CometUnion [sales] - CometProject [cs_quantity,cs_list_price] [sales] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometColumnarExchange [cs_bill_customer_sk] #2 - WholeStageCodegen (5) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [item_sk] - Filter [cnt] - HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 - WholeStageCodegen (3) - HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (5) - HashAggregate [max] [max(csales),tpcds_cmax,max] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (4) - HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_sk] #11 - WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #12 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] - InputAdapter - ReusedExchange [d_date_sk] #12 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #8 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,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,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #9 - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [ws_quantity,ws_list_price] [sales] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometColumnarExchange [ws_bill_customer_sk] #15 - WholeStageCodegen (10) - Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - BroadcastHashJoin [ws_item_sk,item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #8 - ReusedExchange [d_date_sk] #14 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/explain.txt deleted file mode 100644 index ce21e141e4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,576 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (67) -+- CometHashAggregate (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometUnion (63) - :- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (40) - : : +- CometSortMergeJoin (39) - : : :- CometSort (24) - : : : +- CometExchange (23) - : : : +- CometProject (22) - : : : +- CometBroadcastHashJoin (21) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometBroadcastExchange (20) - : : : +- CometProject (19) - : : : +- CometFilter (18) - : : : +- CometHashAggregate (17) - : : : +- CometExchange (16) - : : : +- CometHashAggregate (15) - : : : +- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometFilter (3) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (2) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometFilter (11) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (10) - : : +- CometSort (38) - : : +- CometProject (37) - : : +- CometFilter (36) - : : +- CometHashAggregate (35) - : : +- CometExchange (34) - : : +- CometHashAggregate (33) - : : +- CometProject (32) - : : +- CometBroadcastHashJoin (31) - : : :- CometProject (27) - : : : +- CometFilter (26) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (25) - : : +- CometBroadcastExchange (30) - : : +- CometFilter (29) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (28) - : +- CometBroadcastExchange (44) - : +- CometProject (43) - : +- CometFilter (42) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - +- CometProject (62) - +- CometBroadcastHashJoin (61) - :- CometProject (59) - : +- CometSortMergeJoin (58) - : :- CometSort (52) - : : +- CometExchange (51) - : : +- CometProject (50) - : : +- CometBroadcastHashJoin (49) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : : +- ReusedExchange (48) - : +- CometSort (57) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometHashAggregate (54) - : +- ReusedExchange (53) - +- ReusedExchange (60) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(2) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(3) CometFilter -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_item_sk#7) - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(6) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(7) CometBroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: [d_date_sk#10, d_date#11] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Right output [2]: [d_date_sk#10, d_date#11] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] -Arguments: [ss_item_sk#7, d_date#11], [ss_item_sk#7, d_date#11] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#13, i_item_desc#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [i_item_sk#13, i_item_desc#14] -Condition : isnotnull(i_item_sk#13) - -(12) CometBroadcastExchange -Input [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [i_item_sk#13, i_item_desc#14] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, d_date#11] -Right output [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [ss_item_sk#7], [i_item_sk#13], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#13, i_item_desc#14] -Arguments: [d_date#11, i_item_sk#13, _groupingexpression#15], [d_date#11, i_item_sk#13, substr(i_item_desc#14, 1, 30) AS _groupingexpression#15] - -(15) CometHashAggregate -Input [3]: [d_date#11, i_item_sk#13, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [partial_count(1)] - -(16) CometExchange -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [count(1)] - -(18) CometFilter -Input [2]: [item_sk#17, cnt#18] -Condition : (cnt#18 > 4) - -(19) CometProject -Input [2]: [item_sk#17, cnt#18] -Arguments: [item_sk#17], [item_sk#17] - -(20) CometBroadcastExchange -Input [1]: [item_sk#17] -Arguments: [item_sk#17] - -(21) CometBroadcastHashJoin -Left output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [item_sk#17] -Arguments: [cs_item_sk#2], [item_sk#17], LeftSemi, BuildRight - -(22) CometProject -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(23) CometExchange -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#19) - -(27) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Arguments: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21], [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(29) CometFilter -Input [1]: [c_customer_sk#23] -Condition : isnotnull(c_customer_sk#23) - -(30) CometBroadcastExchange -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23] - -(31) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] -Right output [1]: [c_customer_sk#23] -Arguments: [ss_customer_sk#19], [c_customer_sk#23], Inner, BuildRight - -(32) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Arguments: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23], [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] - -(33) CometHashAggregate -Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Keys [1]: [c_customer_sk#23] -Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(34) CometExchange -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(35) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(36) CometFilter -Input [2]: [c_customer_sk#23, ssales#26] -Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#27, [id=#28]))) - -(37) CometProject -Input [2]: [c_customer_sk#23, ssales#26] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(38) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(39) CometSortMergeJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [c_customer_sk#23] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#23], LeftSemi - -(40) CometProject -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#29, d_year#30, d_moy#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Condition : ((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2000)) AND (d_moy#31 = 2)) AND isnotnull(d_date_sk#29)) - -(43) CometProject -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Arguments: [d_date_sk#29], [d_date_sk#29] - -(44) CometBroadcastExchange -Input [1]: [d_date_sk#29] -Arguments: [d_date_sk#29] - -(45) CometBroadcastHashJoin -Left output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [d_date_sk#29] -Arguments: [cs_sold_date_sk#5], [d_date_sk#29], Inner, BuildRight - -(46) CometProject -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#29] -Arguments: [sales#32], [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#32] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_sold_date_sk#37 IN dynamicpruning#38)] -ReadSchema: struct - -(48) ReusedExchange [Reuses operator id: 20] -Output [1]: [item_sk#39] - -(49) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Right output [1]: [item_sk#39] -Arguments: [ws_item_sk#33], [item_sk#39], LeftSemi, BuildRight - -(50) CometProject -Input [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] - -(51) CometExchange -Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: hashpartitioning(ws_bill_customer_sk#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(52) CometSort -Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_bill_customer_sk#34 ASC NULLS FIRST] - -(53) ReusedExchange [Reuses operator id: 34] -Output [3]: [c_customer_sk#40, sum#41, isEmpty#42] - -(54) CometHashAggregate -Input [3]: [c_customer_sk#40, sum#41, isEmpty#42] -Keys [1]: [c_customer_sk#40] -Functions [1]: [sum((cast(ss_quantity#43 as decimal(10,0)) * ss_sales_price#44))] - -(55) CometFilter -Input [2]: [c_customer_sk#40, ssales#45] -Condition : (isnotnull(ssales#45) AND (cast(ssales#45 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) - -(56) CometProject -Input [2]: [c_customer_sk#40, ssales#45] -Arguments: [c_customer_sk#40], [c_customer_sk#40] - -(57) CometSort -Input [1]: [c_customer_sk#40] -Arguments: [c_customer_sk#40], [c_customer_sk#40 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Right output [1]: [c_customer_sk#40] -Arguments: [ws_bill_customer_sk#34], [c_customer_sk#40], LeftSemi - -(59) CometProject -Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] - -(60) ReusedExchange [Reuses operator id: 44] -Output [1]: [d_date_sk#46] - -(61) CometBroadcastHashJoin -Left output [3]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Right output [1]: [d_date_sk#46] -Arguments: [ws_sold_date_sk#37], [d_date_sk#46], Inner, BuildRight - -(62) CometProject -Input [4]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37, d_date_sk#46] -Arguments: [sales#47], [(cast(ws_quantity#35 as decimal(10,0)) * ws_list_price#36) AS sales#47] - -(63) CometUnion -Child 0 Input [1]: [sales#32] -Child 1 Input [1]: [sales#47] - -(64) CometHashAggregate -Input [1]: [sales#32] -Keys: [] -Functions [1]: [partial_sum(sales#32)] - -(65) CometExchange -Input [2]: [sum#48, isEmpty#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(66) CometHashAggregate -Input [2]: [sum#48, isEmpty#49] -Keys: [] -Functions [1]: [sum(sales#32)] - -(67) CometColumnarToRow [codegen id : 1] -Input [1]: [sum(sales)#50] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (72) -+- * CometColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) - - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#29, d_year#30, d_moy#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Condition : ((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2000)) AND (d_moy#31 = 2)) AND isnotnull(d_date_sk#29)) - -(70) CometProject -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Arguments: [d_date_sk#29], [d_date_sk#29] - -(71) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#29] - -(72) BroadcastExchange -Input [1]: [d_date_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 2 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (77) -+- * CometColumnarToRow (76) - +- CometProject (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(75) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(76) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_date#11] - -(77) BroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:3 Hosting operator id = 36 Hosting Expression = Subquery scalar-subquery#27, [id=#28] -* CometColumnarToRow (95) -+- CometHashAggregate (94) - +- CometExchange (93) - +- CometHashAggregate (92) - +- CometHashAggregate (91) - +- CometExchange (90) - +- CometHashAggregate (89) - +- CometProject (88) - +- CometBroadcastHashJoin (87) - :- CometProject (82) - : +- CometBroadcastHashJoin (81) - : :- CometFilter (79) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (78) - : +- ReusedExchange (80) - +- CometBroadcastExchange (86) - +- CometProject (85) - +- CometFilter (84) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (83) - - -(78) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#54), dynamicpruningexpression(ss_sold_date_sk#54 IN dynamicpruning#55)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(79) CometFilter -Input [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] -Condition : isnotnull(ss_customer_sk#51) - -(80) ReusedExchange [Reuses operator id: 30] -Output [1]: [c_customer_sk#56] - -(81) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] -Right output [1]: [c_customer_sk#56] -Arguments: [ss_customer_sk#51], [c_customer_sk#56], Inner, BuildRight - -(82) CometProject -Input [5]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56] -Arguments: [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56], [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56] - -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#57, d_year#58] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(84) CometFilter -Input [2]: [d_date_sk#57, d_year#58] -Condition : (d_year#58 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#57)) - -(85) CometProject -Input [2]: [d_date_sk#57, d_year#58] -Arguments: [d_date_sk#57], [d_date_sk#57] - -(86) CometBroadcastExchange -Input [1]: [d_date_sk#57] -Arguments: [d_date_sk#57] - -(87) CometBroadcastHashJoin -Left output [4]: [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56] -Right output [1]: [d_date_sk#57] -Arguments: [ss_sold_date_sk#54], [d_date_sk#57], Inner, BuildRight - -(88) CometProject -Input [5]: [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56, d_date_sk#57] -Arguments: [ss_quantity#52, ss_sales_price#53, c_customer_sk#56], [ss_quantity#52, ss_sales_price#53, c_customer_sk#56] - -(89) CometHashAggregate -Input [3]: [ss_quantity#52, ss_sales_price#53, c_customer_sk#56] -Keys [1]: [c_customer_sk#56] -Functions [1]: [partial_sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))] - -(90) CometExchange -Input [3]: [c_customer_sk#56, sum#59, isEmpty#60] -Arguments: hashpartitioning(c_customer_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(91) CometHashAggregate -Input [3]: [c_customer_sk#56, sum#59, isEmpty#60] -Keys [1]: [c_customer_sk#56] -Functions [1]: [sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))] - -(92) CometHashAggregate -Input [1]: [csales#61] -Keys: [] -Functions [1]: [partial_max(csales#61)] - -(93) CometExchange -Input [1]: [max#62] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(94) CometHashAggregate -Input [1]: [max#62] -Keys: [] -Functions [1]: [max(csales#61)] - -(95) CometColumnarToRow [codegen id : 1] -Input [1]: [tpcds_cmax#63] - -Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#54 IN dynamicpruning#55 -BroadcastExchange (100) -+- * CometColumnarToRow (99) - +- CometProject (98) - +- CometFilter (97) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (96) - - -(96) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#57, d_year#58] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(97) CometFilter -Input [2]: [d_date_sk#57, d_year#58] -Condition : (d_year#58 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#57)) - -(98) CometProject -Input [2]: [d_date_sk#57, d_year#58] -Arguments: [d_date_sk#57], [d_date_sk#57] - -(99) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#57] - -(100) BroadcastExchange -Input [1]: [d_date_sk#57] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] - -Subquery:5 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#6 - -Subquery:6 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 8f1bddf6c0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,116 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,isEmpty] [sum(sales),sum(sales)] - CometExchange #1 - CometHashAggregate [sales] [sum,isEmpty] - CometUnion [sales] - CometProject [cs_quantity,cs_list_price] [sales] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometExchange [cs_bill_customer_sk] #2 - CometProject [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [item_sk] #4 - CometProject [item_sk] - CometFilter [item_sk,cnt] - CometHashAggregate [_groupingexpression,d_date,count] [item_sk,cnt,i_item_sk,count(1)] - CometExchange [_groupingexpression,i_item_sk,d_date] #5 - CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] - CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] - CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [i_item_sk,i_item_desc] #8 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [max] [tpcds_cmax,max(csales)] - CometExchange #11 - CometHashAggregate [csales] [max] - CometHashAggregate [c_customer_sk,sum,isEmpty] [csales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #12 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,c_customer_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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #9 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,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,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #10 - CometFilter [c_customer_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [d_date_sk] #15 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [ws_quantity,ws_list_price] [sales] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometExchange [ws_bill_customer_sk] #16 - CometProject [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,item_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #9 - ReusedExchange [d_date_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/explain.txt deleted file mode 100644 index ce21e141e4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/explain.txt +++ /dev/null @@ -1,576 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (67) -+- CometHashAggregate (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometUnion (63) - :- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (40) - : : +- CometSortMergeJoin (39) - : : :- CometSort (24) - : : : +- CometExchange (23) - : : : +- CometProject (22) - : : : +- CometBroadcastHashJoin (21) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometBroadcastExchange (20) - : : : +- CometProject (19) - : : : +- CometFilter (18) - : : : +- CometHashAggregate (17) - : : : +- CometExchange (16) - : : : +- CometHashAggregate (15) - : : : +- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometFilter (3) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (2) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometFilter (11) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (10) - : : +- CometSort (38) - : : +- CometProject (37) - : : +- CometFilter (36) - : : +- CometHashAggregate (35) - : : +- CometExchange (34) - : : +- CometHashAggregate (33) - : : +- CometProject (32) - : : +- CometBroadcastHashJoin (31) - : : :- CometProject (27) - : : : +- CometFilter (26) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (25) - : : +- CometBroadcastExchange (30) - : : +- CometFilter (29) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (28) - : +- CometBroadcastExchange (44) - : +- CometProject (43) - : +- CometFilter (42) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - +- CometProject (62) - +- CometBroadcastHashJoin (61) - :- CometProject (59) - : +- CometSortMergeJoin (58) - : :- CometSort (52) - : : +- CometExchange (51) - : : +- CometProject (50) - : : +- CometBroadcastHashJoin (49) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : : +- ReusedExchange (48) - : +- CometSort (57) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometHashAggregate (54) - : +- ReusedExchange (53) - +- ReusedExchange (60) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(2) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(3) CometFilter -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_item_sk#7) - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(6) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(7) CometBroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: [d_date_sk#10, d_date#11] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Right output [2]: [d_date_sk#10, d_date#11] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] -Arguments: [ss_item_sk#7, d_date#11], [ss_item_sk#7, d_date#11] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#13, i_item_desc#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [i_item_sk#13, i_item_desc#14] -Condition : isnotnull(i_item_sk#13) - -(12) CometBroadcastExchange -Input [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [i_item_sk#13, i_item_desc#14] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, d_date#11] -Right output [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [ss_item_sk#7], [i_item_sk#13], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#13, i_item_desc#14] -Arguments: [d_date#11, i_item_sk#13, _groupingexpression#15], [d_date#11, i_item_sk#13, substr(i_item_desc#14, 1, 30) AS _groupingexpression#15] - -(15) CometHashAggregate -Input [3]: [d_date#11, i_item_sk#13, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [partial_count(1)] - -(16) CometExchange -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [count(1)] - -(18) CometFilter -Input [2]: [item_sk#17, cnt#18] -Condition : (cnt#18 > 4) - -(19) CometProject -Input [2]: [item_sk#17, cnt#18] -Arguments: [item_sk#17], [item_sk#17] - -(20) CometBroadcastExchange -Input [1]: [item_sk#17] -Arguments: [item_sk#17] - -(21) CometBroadcastHashJoin -Left output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [item_sk#17] -Arguments: [cs_item_sk#2], [item_sk#17], LeftSemi, BuildRight - -(22) CometProject -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(23) CometExchange -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#19) - -(27) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Arguments: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21], [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(29) CometFilter -Input [1]: [c_customer_sk#23] -Condition : isnotnull(c_customer_sk#23) - -(30) CometBroadcastExchange -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23] - -(31) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] -Right output [1]: [c_customer_sk#23] -Arguments: [ss_customer_sk#19], [c_customer_sk#23], Inner, BuildRight - -(32) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Arguments: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23], [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] - -(33) CometHashAggregate -Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Keys [1]: [c_customer_sk#23] -Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(34) CometExchange -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(35) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(36) CometFilter -Input [2]: [c_customer_sk#23, ssales#26] -Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#27, [id=#28]))) - -(37) CometProject -Input [2]: [c_customer_sk#23, ssales#26] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(38) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(39) CometSortMergeJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [c_customer_sk#23] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#23], LeftSemi - -(40) CometProject -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#29, d_year#30, d_moy#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Condition : ((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2000)) AND (d_moy#31 = 2)) AND isnotnull(d_date_sk#29)) - -(43) CometProject -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Arguments: [d_date_sk#29], [d_date_sk#29] - -(44) CometBroadcastExchange -Input [1]: [d_date_sk#29] -Arguments: [d_date_sk#29] - -(45) CometBroadcastHashJoin -Left output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [d_date_sk#29] -Arguments: [cs_sold_date_sk#5], [d_date_sk#29], Inner, BuildRight - -(46) CometProject -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#29] -Arguments: [sales#32], [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#32] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_sold_date_sk#37 IN dynamicpruning#38)] -ReadSchema: struct - -(48) ReusedExchange [Reuses operator id: 20] -Output [1]: [item_sk#39] - -(49) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Right output [1]: [item_sk#39] -Arguments: [ws_item_sk#33], [item_sk#39], LeftSemi, BuildRight - -(50) CometProject -Input [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] - -(51) CometExchange -Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: hashpartitioning(ws_bill_customer_sk#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(52) CometSort -Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_bill_customer_sk#34 ASC NULLS FIRST] - -(53) ReusedExchange [Reuses operator id: 34] -Output [3]: [c_customer_sk#40, sum#41, isEmpty#42] - -(54) CometHashAggregate -Input [3]: [c_customer_sk#40, sum#41, isEmpty#42] -Keys [1]: [c_customer_sk#40] -Functions [1]: [sum((cast(ss_quantity#43 as decimal(10,0)) * ss_sales_price#44))] - -(55) CometFilter -Input [2]: [c_customer_sk#40, ssales#45] -Condition : (isnotnull(ssales#45) AND (cast(ssales#45 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) - -(56) CometProject -Input [2]: [c_customer_sk#40, ssales#45] -Arguments: [c_customer_sk#40], [c_customer_sk#40] - -(57) CometSort -Input [1]: [c_customer_sk#40] -Arguments: [c_customer_sk#40], [c_customer_sk#40 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Right output [1]: [c_customer_sk#40] -Arguments: [ws_bill_customer_sk#34], [c_customer_sk#40], LeftSemi - -(59) CometProject -Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] - -(60) ReusedExchange [Reuses operator id: 44] -Output [1]: [d_date_sk#46] - -(61) CometBroadcastHashJoin -Left output [3]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Right output [1]: [d_date_sk#46] -Arguments: [ws_sold_date_sk#37], [d_date_sk#46], Inner, BuildRight - -(62) CometProject -Input [4]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37, d_date_sk#46] -Arguments: [sales#47], [(cast(ws_quantity#35 as decimal(10,0)) * ws_list_price#36) AS sales#47] - -(63) CometUnion -Child 0 Input [1]: [sales#32] -Child 1 Input [1]: [sales#47] - -(64) CometHashAggregate -Input [1]: [sales#32] -Keys: [] -Functions [1]: [partial_sum(sales#32)] - -(65) CometExchange -Input [2]: [sum#48, isEmpty#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(66) CometHashAggregate -Input [2]: [sum#48, isEmpty#49] -Keys: [] -Functions [1]: [sum(sales#32)] - -(67) CometColumnarToRow [codegen id : 1] -Input [1]: [sum(sales)#50] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (72) -+- * CometColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) - - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#29, d_year#30, d_moy#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Condition : ((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2000)) AND (d_moy#31 = 2)) AND isnotnull(d_date_sk#29)) - -(70) CometProject -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Arguments: [d_date_sk#29], [d_date_sk#29] - -(71) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#29] - -(72) BroadcastExchange -Input [1]: [d_date_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 2 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (77) -+- * CometColumnarToRow (76) - +- CometProject (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(75) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(76) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_date#11] - -(77) BroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:3 Hosting operator id = 36 Hosting Expression = Subquery scalar-subquery#27, [id=#28] -* CometColumnarToRow (95) -+- CometHashAggregate (94) - +- CometExchange (93) - +- CometHashAggregate (92) - +- CometHashAggregate (91) - +- CometExchange (90) - +- CometHashAggregate (89) - +- CometProject (88) - +- CometBroadcastHashJoin (87) - :- CometProject (82) - : +- CometBroadcastHashJoin (81) - : :- CometFilter (79) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (78) - : +- ReusedExchange (80) - +- CometBroadcastExchange (86) - +- CometProject (85) - +- CometFilter (84) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (83) - - -(78) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#54), dynamicpruningexpression(ss_sold_date_sk#54 IN dynamicpruning#55)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(79) CometFilter -Input [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] -Condition : isnotnull(ss_customer_sk#51) - -(80) ReusedExchange [Reuses operator id: 30] -Output [1]: [c_customer_sk#56] - -(81) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54] -Right output [1]: [c_customer_sk#56] -Arguments: [ss_customer_sk#51], [c_customer_sk#56], Inner, BuildRight - -(82) CometProject -Input [5]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56] -Arguments: [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56], [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56] - -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#57, d_year#58] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(84) CometFilter -Input [2]: [d_date_sk#57, d_year#58] -Condition : (d_year#58 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#57)) - -(85) CometProject -Input [2]: [d_date_sk#57, d_year#58] -Arguments: [d_date_sk#57], [d_date_sk#57] - -(86) CometBroadcastExchange -Input [1]: [d_date_sk#57] -Arguments: [d_date_sk#57] - -(87) CometBroadcastHashJoin -Left output [4]: [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56] -Right output [1]: [d_date_sk#57] -Arguments: [ss_sold_date_sk#54], [d_date_sk#57], Inner, BuildRight - -(88) CometProject -Input [5]: [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56, d_date_sk#57] -Arguments: [ss_quantity#52, ss_sales_price#53, c_customer_sk#56], [ss_quantity#52, ss_sales_price#53, c_customer_sk#56] - -(89) CometHashAggregate -Input [3]: [ss_quantity#52, ss_sales_price#53, c_customer_sk#56] -Keys [1]: [c_customer_sk#56] -Functions [1]: [partial_sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))] - -(90) CometExchange -Input [3]: [c_customer_sk#56, sum#59, isEmpty#60] -Arguments: hashpartitioning(c_customer_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(91) CometHashAggregate -Input [3]: [c_customer_sk#56, sum#59, isEmpty#60] -Keys [1]: [c_customer_sk#56] -Functions [1]: [sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))] - -(92) CometHashAggregate -Input [1]: [csales#61] -Keys: [] -Functions [1]: [partial_max(csales#61)] - -(93) CometExchange -Input [1]: [max#62] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(94) CometHashAggregate -Input [1]: [max#62] -Keys: [] -Functions [1]: [max(csales#61)] - -(95) CometColumnarToRow [codegen id : 1] -Input [1]: [tpcds_cmax#63] - -Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#54 IN dynamicpruning#55 -BroadcastExchange (100) -+- * CometColumnarToRow (99) - +- CometProject (98) - +- CometFilter (97) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (96) - - -(96) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#57, d_year#58] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(97) CometFilter -Input [2]: [d_date_sk#57, d_year#58] -Condition : (d_year#58 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#57)) - -(98) CometProject -Input [2]: [d_date_sk#57, d_year#58] -Arguments: [d_date_sk#57], [d_date_sk#57] - -(99) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#57] - -(100) BroadcastExchange -Input [1]: [d_date_sk#57] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] - -Subquery:5 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#6 - -Subquery:6 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/extended.txt deleted file mode 100644 index a84226b077..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/extended.txt +++ /dev/null @@ -1,146 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometSort - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometSort - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 127 out of 138 eligible operators (92%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 8f1bddf6c0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt +++ /dev/null @@ -1,116 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,isEmpty] [sum(sales),sum(sales)] - CometExchange #1 - CometHashAggregate [sales] [sum,isEmpty] - CometUnion [sales] - CometProject [cs_quantity,cs_list_price] [sales] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometExchange [cs_bill_customer_sk] #2 - CometProject [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [item_sk] #4 - CometProject [item_sk] - CometFilter [item_sk,cnt] - CometHashAggregate [_groupingexpression,d_date,count] [item_sk,cnt,i_item_sk,count(1)] - CometExchange [_groupingexpression,i_item_sk,d_date] #5 - CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] - CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] - CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [i_item_sk,i_item_desc] #8 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [max] [tpcds_cmax,max(csales)] - CometExchange #11 - CometHashAggregate [csales] [max] - CometHashAggregate [c_customer_sk,sum,isEmpty] [csales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #12 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,c_customer_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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #9 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,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,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #10 - CometFilter [c_customer_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [d_date_sk] #15 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [ws_quantity,ws_list_price] [sales] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometExchange [ws_bill_customer_sk] #16 - CometProject [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,item_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #9 - ReusedExchange [d_date_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/explain.txt deleted file mode 100644 index ff47906a58..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/explain.txt +++ /dev/null @@ -1,718 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (90) -+- CometTakeOrderedAndProject (89) - +- CometUnion (88) - :- CometHashAggregate (64) - : +- CometExchange (63) - : +- CometHashAggregate (62) - : +- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (55) - : : +- CometBroadcastHashJoin (54) - : : :- CometSortMergeJoin (41) - : : : :- CometSort (26) - : : : : +- CometColumnarExchange (25) - : : : : +- * Project (24) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (22) - : : : : +- * Project (21) - : : : : +- * Filter (20) - : : : : +- * HashAggregate (19) - : : : : +- * CometColumnarToRow (18) - : : : : +- CometColumnarExchange (17) - : : : : +- * HashAggregate (16) - : : : : +- * Project (15) - : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : :- * Project (9) - : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : :- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : : : +- ReusedExchange (7) - : : : : +- BroadcastExchange (13) - : : : : +- * CometColumnarToRow (12) - : : : : +- CometFilter (11) - : : : : +- CometNativeScan parquet spark_catalog.default.item (10) - : : : +- CometSort (40) - : : : +- CometProject (39) - : : : +- CometFilter (38) - : : : +- CometHashAggregate (37) - : : : +- CometExchange (36) - : : : +- CometHashAggregate (35) - : : : +- CometProject (34) - : : : +- CometBroadcastHashJoin (33) - : : : :- CometProject (29) - : : : : +- CometFilter (28) - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (27) - : : : +- CometBroadcastExchange (32) - : : : +- CometFilter (31) - : : : +- CometNativeScan parquet spark_catalog.default.customer (30) - : : +- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometSortMergeJoin (51) - : : :- CometSort (45) - : : : +- CometExchange (44) - : : : +- CometFilter (43) - : : : +- CometNativeScan parquet spark_catalog.default.customer (42) - : : +- CometSort (50) - : : +- CometProject (49) - : : +- CometFilter (48) - : : +- CometHashAggregate (47) - : : +- ReusedExchange (46) - : +- CometBroadcastExchange (59) - : +- CometProject (58) - : +- CometFilter (57) - : +- CometNativeScan parquet spark_catalog.default.date_dim (56) - +- CometHashAggregate (87) - +- CometExchange (86) - +- CometHashAggregate (85) - +- CometProject (84) - +- CometBroadcastHashJoin (83) - :- CometProject (81) - : +- CometBroadcastHashJoin (80) - : :- CometSortMergeJoin (78) - : : :- CometSort (72) - : : : +- CometColumnarExchange (71) - : : : +- * Project (70) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (69) - : : : :- * Filter (67) - : : : : +- * ColumnarToRow (66) - : : : : +- Scan parquet spark_catalog.default.web_sales (65) - : : : +- ReusedExchange (68) - : : +- CometSort (77) - : : +- CometProject (76) - : : +- CometFilter (75) - : : +- CometHashAggregate (74) - : : +- ReusedExchange (73) - : +- ReusedExchange (79) - +- ReusedExchange (82) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(3) Filter [codegen id : 5] -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_bill_customer_sk#1) - -(4) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 3] -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] - -(6) Filter [codegen id : 3] -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_item_sk#7) - -(7) ReusedExchange [Reuses operator id: 100] -Output [2]: [d_date_sk#10, d_date#11] - -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 3] -Output [2]: [ss_item_sk#7, d_date#11] -Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] - -(10) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#12, i_item_desc#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [i_item_sk#12, i_item_desc#13] -Condition : isnotnull(i_item_sk#12) - -(12) CometColumnarToRow [codegen id : 2] -Input [2]: [i_item_sk#12, i_item_desc#13] - -(13) BroadcastExchange -Input [2]: [i_item_sk#12, i_item_desc#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(14) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#7] -Right keys [1]: [i_item_sk#12] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 3] -Output [3]: [d_date#11, i_item_sk#12, substr(i_item_desc#13, 1, 30) AS _groupingexpression#14] -Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#12, i_item_desc#13] - -(16) HashAggregate [codegen id : 3] -Input [3]: [d_date#11, i_item_sk#12, _groupingexpression#14] -Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#15] -Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] - -(17) CometColumnarExchange -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(18) CometColumnarToRow [codegen id : 4] -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] - -(19) HashAggregate [codegen id : 4] -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#17] -Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] - -(20) Filter [codegen id : 4] -Input [2]: [item_sk#18, cnt#19] -Condition : (cnt#19 > 4) - -(21) Project [codegen id : 4] -Output [1]: [item_sk#18] -Input [2]: [item_sk#18, cnt#19] - -(22) BroadcastExchange -Input [1]: [item_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(23) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [item_sk#18] -Join type: LeftSemi -Join condition: None - -(24) Project [codegen id : 5] -Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(25) CometColumnarExchange -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(26) CometSort -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] - -(27) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(28) CometFilter -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_customer_sk#20) - -(29) CometProject -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -Arguments: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22], [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] - -(30) CometNativeScan parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(31) CometFilter -Input [1]: [c_customer_sk#24] -Condition : isnotnull(c_customer_sk#24) - -(32) CometBroadcastExchange -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24] - -(33) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] -Right output [1]: [c_customer_sk#24] -Arguments: [ss_customer_sk#20], [c_customer_sk#24], Inner, BuildRight - -(34) CometProject -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -Arguments: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24], [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] - -(35) CometHashAggregate -Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -Keys [1]: [c_customer_sk#24] -Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] - -(36) CometExchange -Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] -Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(37) CometHashAggregate -Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] -Keys [1]: [c_customer_sk#24] -Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] - -(38) CometFilter -Input [2]: [c_customer_sk#24, ssales#27] -Condition : (isnotnull(ssales#27) AND (cast(ssales#27 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#28, [id=#29]))) - -(39) CometProject -Input [2]: [c_customer_sk#24, ssales#27] -Arguments: [c_customer_sk#24], [c_customer_sk#24] - -(40) CometSort -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24], [c_customer_sk#24 ASC NULLS FIRST] - -(41) CometSortMergeJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [c_customer_sk#24] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#24], LeftSemi - -(42) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(43) CometFilter -Input [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Condition : isnotnull(c_customer_sk#30) - -(44) CometExchange -Input [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(45) CometSort -Input [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Arguments: [c_customer_sk#30, c_first_name#31, c_last_name#32], [c_customer_sk#30 ASC NULLS FIRST] - -(46) ReusedExchange [Reuses operator id: 36] -Output [3]: [c_customer_sk#24, sum#25, isEmpty#26] - -(47) CometHashAggregate -Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] -Keys [1]: [c_customer_sk#24] -Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] - -(48) CometFilter -Input [2]: [c_customer_sk#24, ssales#27] -Condition : (isnotnull(ssales#27) AND (cast(ssales#27 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#28, [id=#29]))) - -(49) CometProject -Input [2]: [c_customer_sk#24, ssales#27] -Arguments: [c_customer_sk#24], [c_customer_sk#24] - -(50) CometSort -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24], [c_customer_sk#24 ASC NULLS FIRST] - -(51) CometSortMergeJoin -Left output [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Right output [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#30], [c_customer_sk#24], LeftSemi - -(52) CometProject -Input [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Arguments: [c_customer_sk#30, c_first_name#33, c_last_name#34], [c_customer_sk#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#31, 20)) AS c_first_name#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#32, 30)) AS c_last_name#34] - -(53) CometBroadcastExchange -Input [3]: [c_customer_sk#30, c_first_name#33, c_last_name#34] -Arguments: [c_customer_sk#30, c_first_name#33, c_last_name#34] - -(54) CometBroadcastHashJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [3]: [c_customer_sk#30, c_first_name#33, c_last_name#34] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#30], Inner, BuildRight - -(55) CometProject -Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#30, c_first_name#33, c_last_name#34] -Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#33, c_last_name#34], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#33, c_last_name#34] - -(56) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#35, d_year#36, d_moy#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(57) CometFilter -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] -Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2000)) AND (d_moy#37 = 2)) AND isnotnull(d_date_sk#35)) - -(58) CometProject -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] -Arguments: [d_date_sk#35], [d_date_sk#35] - -(59) CometBroadcastExchange -Input [1]: [d_date_sk#35] -Arguments: [d_date_sk#35] - -(60) CometBroadcastHashJoin -Left output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#33, c_last_name#34] -Right output [1]: [d_date_sk#35] -Arguments: [cs_sold_date_sk#5], [d_date_sk#35], Inner, BuildRight - -(61) CometProject -Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#33, c_last_name#34, d_date_sk#35] -Arguments: [cs_quantity#3, cs_list_price#4, c_first_name#33, c_last_name#34], [cs_quantity#3, cs_list_price#4, c_first_name#33, c_last_name#34] - -(62) CometHashAggregate -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#33, c_last_name#34] -Keys [2]: [c_last_name#34, c_first_name#33] -Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] - -(63) CometExchange -Input [4]: [c_last_name#34, c_first_name#33, sum#38, isEmpty#39] -Arguments: hashpartitioning(c_last_name#34, c_first_name#33, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(64) CometHashAggregate -Input [4]: [c_last_name#34, c_first_name#33, sum#38, isEmpty#39] -Keys [2]: [c_last_name#34, c_first_name#33] -Functions [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] - -(65) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#40, ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(66) ColumnarToRow [codegen id : 10] -Input [5]: [ws_item_sk#40, ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] - -(67) Filter [codegen id : 10] -Input [5]: [ws_item_sk#40, ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_bill_customer_sk#41) - -(68) ReusedExchange [Reuses operator id: 22] -Output [1]: [item_sk#45] - -(69) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [item_sk#45] -Join type: LeftSemi -Join condition: None - -(70) Project [codegen id : 10] -Output [4]: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -Input [5]: [ws_item_sk#40, ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] - -(71) CometColumnarExchange -Input [4]: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -Arguments: hashpartitioning(ws_bill_customer_sk#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(72) CometSort -Input [4]: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -Arguments: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44], [ws_bill_customer_sk#41 ASC NULLS FIRST] - -(73) ReusedExchange [Reuses operator id: 36] -Output [3]: [c_customer_sk#46, sum#47, isEmpty#48] - -(74) CometHashAggregate -Input [3]: [c_customer_sk#46, sum#47, isEmpty#48] -Keys [1]: [c_customer_sk#46] -Functions [1]: [sum((cast(ss_quantity#49 as decimal(10,0)) * ss_sales_price#50))] - -(75) CometFilter -Input [2]: [c_customer_sk#46, ssales#51] -Condition : (isnotnull(ssales#51) AND (cast(ssales#51 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#28, [id=#29]))) - -(76) CometProject -Input [2]: [c_customer_sk#46, ssales#51] -Arguments: [c_customer_sk#46], [c_customer_sk#46] - -(77) CometSort -Input [1]: [c_customer_sk#46] -Arguments: [c_customer_sk#46], [c_customer_sk#46 ASC NULLS FIRST] - -(78) CometSortMergeJoin -Left output [4]: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -Right output [1]: [c_customer_sk#46] -Arguments: [ws_bill_customer_sk#41], [c_customer_sk#46], LeftSemi - -(79) ReusedExchange [Reuses operator id: 53] -Output [3]: [c_customer_sk#52, c_first_name#53, c_last_name#54] - -(80) CometBroadcastHashJoin -Left output [4]: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -Right output [3]: [c_customer_sk#52, c_first_name#53, c_last_name#54] -Arguments: [ws_bill_customer_sk#41], [c_customer_sk#52], Inner, BuildRight - -(81) CometProject -Input [7]: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44, c_customer_sk#52, c_first_name#53, c_last_name#54] -Arguments: [ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44, c_first_name#53, c_last_name#54], [ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44, c_first_name#53, c_last_name#54] - -(82) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#55] - -(83) CometBroadcastHashJoin -Left output [5]: [ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44, c_first_name#53, c_last_name#54] -Right output [1]: [d_date_sk#55] -Arguments: [ws_sold_date_sk#44], [d_date_sk#55], Inner, BuildRight - -(84) CometProject -Input [6]: [ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44, c_first_name#53, c_last_name#54, d_date_sk#55] -Arguments: [ws_quantity#42, ws_list_price#43, c_first_name#53, c_last_name#54], [ws_quantity#42, ws_list_price#43, c_first_name#53, c_last_name#54] - -(85) CometHashAggregate -Input [4]: [ws_quantity#42, ws_list_price#43, c_first_name#53, c_last_name#54] -Keys [2]: [c_last_name#54, c_first_name#53] -Functions [1]: [partial_sum((cast(ws_quantity#42 as decimal(10,0)) * ws_list_price#43))] - -(86) CometExchange -Input [4]: [c_last_name#54, c_first_name#53, sum#56, isEmpty#57] -Arguments: hashpartitioning(c_last_name#54, c_first_name#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(87) CometHashAggregate -Input [4]: [c_last_name#54, c_first_name#53, sum#56, isEmpty#57] -Keys [2]: [c_last_name#54, c_first_name#53] -Functions [1]: [sum((cast(ws_quantity#42 as decimal(10,0)) * ws_list_price#43))] - -(88) CometUnion -Child 0 Input [3]: [c_last_name#34, c_first_name#33, sales#58] -Child 1 Input [3]: [c_last_name#54, c_first_name#53, sales#59] - -(89) CometTakeOrderedAndProject -Input [3]: [c_last_name#34, c_first_name#33, sales#58] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#34 ASC NULLS FIRST,c_first_name#33 ASC NULLS FIRST,sales#58 ASC NULLS FIRST], output=[c_last_name#34,c_first_name#33,sales#58]), [c_last_name#34, c_first_name#33, sales#58], 100, 0, [c_last_name#34 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, sales#58 ASC NULLS FIRST], [c_last_name#34, c_first_name#33, sales#58] - -(90) CometColumnarToRow [codegen id : 11] -Input [3]: [c_last_name#34, c_first_name#33, sales#58] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (95) -+- * CometColumnarToRow (94) - +- CometProject (93) - +- CometFilter (92) - +- CometNativeScan parquet spark_catalog.default.date_dim (91) - - -(91) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#35, d_year#36, d_moy#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(92) CometFilter -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] -Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2000)) AND (d_moy#37 = 2)) AND isnotnull(d_date_sk#35)) - -(93) CometProject -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] -Arguments: [d_date_sk#35], [d_date_sk#35] - -(94) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#35] - -(95) BroadcastExchange -Input [1]: [d_date_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (100) -+- * CometColumnarToRow (99) - +- CometProject (98) - +- CometFilter (97) - +- CometNativeScan parquet spark_catalog.default.date_dim (96) - - -(96) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#60] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(97) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#60] -Condition : (d_year#60 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(98) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#60] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(99) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_date#11] - -(100) BroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -Subquery:3 Hosting operator id = 38 Hosting Expression = Subquery scalar-subquery#28, [id=#29] -* HashAggregate (120) -+- * CometColumnarToRow (119) - +- CometColumnarExchange (118) - +- * HashAggregate (117) - +- * HashAggregate (116) - +- * CometColumnarToRow (115) - +- CometColumnarExchange (114) - +- * HashAggregate (113) - +- * Project (112) - +- * BroadcastHashJoin Inner BuildRight (111) - :- * Project (109) - : +- * BroadcastHashJoin Inner BuildRight (108) - : :- * Filter (103) - : : +- * ColumnarToRow (102) - : : +- Scan parquet spark_catalog.default.store_sales (101) - : +- BroadcastExchange (107) - : +- * CometColumnarToRow (106) - : +- CometFilter (105) - : +- CometNativeScan parquet spark_catalog.default.customer (104) - +- ReusedExchange (110) - - -(101) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#64), dynamicpruningexpression(ss_sold_date_sk#64 IN dynamicpruning#65)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(102) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64] - -(103) Filter [codegen id : 3] -Input [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64] -Condition : isnotnull(ss_customer_sk#61) - -(104) CometNativeScan parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(105) CometFilter -Input [1]: [c_customer_sk#66] -Condition : isnotnull(c_customer_sk#66) - -(106) CometColumnarToRow [codegen id : 1] -Input [1]: [c_customer_sk#66] - -(107) BroadcastExchange -Input [1]: [c_customer_sk#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -(108) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#61] -Right keys [1]: [c_customer_sk#66] -Join type: Inner -Join condition: None - -(109) Project [codegen id : 3] -Output [4]: [ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, c_customer_sk#66] -Input [5]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, c_customer_sk#66] - -(110) ReusedExchange [Reuses operator id: 125] -Output [1]: [d_date_sk#67] - -(111) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#64] -Right keys [1]: [d_date_sk#67] -Join type: Inner -Join condition: None - -(112) Project [codegen id : 3] -Output [3]: [ss_quantity#62, ss_sales_price#63, c_customer_sk#66] -Input [5]: [ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, c_customer_sk#66, d_date_sk#67] - -(113) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#62, ss_sales_price#63, c_customer_sk#66] -Keys [1]: [c_customer_sk#66] -Functions [1]: [partial_sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))] -Aggregate Attributes [2]: [sum#68, isEmpty#69] -Results [3]: [c_customer_sk#66, sum#70, isEmpty#71] - -(114) CometColumnarExchange -Input [3]: [c_customer_sk#66, sum#70, isEmpty#71] -Arguments: hashpartitioning(c_customer_sk#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(115) CometColumnarToRow [codegen id : 4] -Input [3]: [c_customer_sk#66, sum#70, isEmpty#71] - -(116) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#66, sum#70, isEmpty#71] -Keys [1]: [c_customer_sk#66] -Functions [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))#72] -Results [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))#72 AS csales#73] - -(117) HashAggregate [codegen id : 4] -Input [1]: [csales#73] -Keys: [] -Functions [1]: [partial_max(csales#73)] -Aggregate Attributes [1]: [max#74] -Results [1]: [max#75] - -(118) CometColumnarExchange -Input [1]: [max#75] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(119) CometColumnarToRow [codegen id : 5] -Input [1]: [max#75] - -(120) HashAggregate [codegen id : 5] -Input [1]: [max#75] -Keys: [] -Functions [1]: [max(csales#73)] -Aggregate Attributes [1]: [max(csales#73)#76] -Results [1]: [max(csales#73)#76 AS tpcds_cmax#77] - -Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#64 IN dynamicpruning#65 -BroadcastExchange (125) -+- * CometColumnarToRow (124) - +- CometProject (123) - +- CometFilter (122) - +- CometNativeScan parquet spark_catalog.default.date_dim (121) - - -(121) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#67, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(122) CometFilter -Input [2]: [d_date_sk#67, d_year#78] -Condition : (d_year#78 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#67)) - -(123) CometProject -Input [2]: [d_date_sk#67, d_year#78] -Arguments: [d_date_sk#67], [d_date_sk#67] - -(124) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#67] - -(125) BroadcastExchange -Input [1]: [d_date_sk#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] - -Subquery:5 Hosting operator id = 48 Hosting Expression = ReusedSubquery Subquery scalar-subquery#28, [id=#29] - -Subquery:6 Hosting operator id = 65 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 - -Subquery:7 Hosting operator id = 75 Hosting Expression = ReusedSubquery Subquery scalar-subquery#28, [id=#29] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/simplified.txt deleted file mode 100644 index 41f01311f8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/simplified.txt +++ /dev/null @@ -1,164 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,sales] - CometUnion [c_last_name,c_first_name,sales] - CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price))] - CometExchange [c_last_name,c_first_name] #1 - CometHashAggregate [cs_quantity,cs_list_price] [c_last_name,c_first_name,sum,isEmpty] - CometProject [cs_quantity,cs_list_price,c_first_name,c_last_name] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometColumnarExchange [cs_bill_customer_sk] #2 - WholeStageCodegen (5) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [item_sk] - Filter [cnt] - HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 - WholeStageCodegen (3) - HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (5) - HashAggregate [max] [max(csales),tpcds_cmax,max] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (4) - HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_sk] #11 - WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #12 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] - InputAdapter - ReusedExchange [d_date_sk] #12 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #8 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,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,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #9 - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #14 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [c_customer_sk,c_first_name,c_last_name,c_customer_sk] - CometSort [c_customer_sk,c_first_name,c_last_name] - CometExchange [c_customer_sk] #15 - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #8 - CometBroadcastExchange [d_date_sk] #16 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price))] - CometExchange [c_last_name,c_first_name] #17 - CometHashAggregate [ws_quantity,ws_list_price] [c_last_name,c_first_name,sum,isEmpty] - CometProject [ws_quantity,ws_list_price,c_first_name,c_last_name] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometColumnarExchange [ws_bill_customer_sk] #18 - WholeStageCodegen (10) - Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - BroadcastHashJoin [ws_item_sk,item_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #8 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #14 - ReusedExchange [d_date_sk] #16 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/explain.txt deleted file mode 100644 index c1131ab0a1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,692 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (88) -+- CometTakeOrderedAndProject (87) - +- CometUnion (86) - :- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometSortMergeJoin (40) - : : : :- CometSort (25) - : : : : +- CometExchange (24) - : : : : +- CometProject (23) - : : : : +- CometBroadcastHashJoin (22) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometBroadcastExchange (21) - : : : : +- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometHashAggregate (18) - : : : : +- CometExchange (17) - : : : : +- CometHashAggregate (16) - : : : : +- CometProject (15) - : : : : +- CometBroadcastHashJoin (14) - : : : : :- CometProject (10) - : : : : : +- CometBroadcastHashJoin (9) - : : : : : :- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : : +- CometBroadcastExchange (8) - : : : : : +- CometProject (7) - : : : : : +- CometFilter (6) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (5) - : : : : +- CometBroadcastExchange (13) - : : : : +- CometFilter (12) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : +- CometSort (39) - : : : +- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometHashAggregate (36) - : : : +- CometExchange (35) - : : : +- CometHashAggregate (34) - : : : +- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometProject (28) - : : : : +- CometFilter (27) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (26) - : : : +- CometBroadcastExchange (31) - : : : +- CometFilter (30) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (29) - : : +- CometBroadcastExchange (52) - : : +- CometProject (51) - : : +- CometSortMergeJoin (50) - : : :- CometSort (44) - : : : +- CometExchange (43) - : : : +- CometFilter (42) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (41) - : : +- CometSort (49) - : : +- CometProject (48) - : : +- CometFilter (47) - : : +- CometHashAggregate (46) - : : +- ReusedExchange (45) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometHashAggregate (85) - +- CometExchange (84) - +- CometHashAggregate (83) - +- CometProject (82) - +- CometBroadcastHashJoin (81) - :- CometProject (79) - : +- CometBroadcastHashJoin (78) - : :- CometSortMergeJoin (76) - : : :- CometSort (70) - : : : +- CometExchange (69) - : : : +- CometProject (68) - : : : +- CometBroadcastHashJoin (67) - : : : :- CometFilter (65) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (64) - : : : +- ReusedExchange (66) - : : +- CometSort (75) - : : +- CometProject (74) - : : +- CometFilter (73) - : : +- CometHashAggregate (72) - : : +- ReusedExchange (71) - : +- ReusedExchange (77) - +- ReusedExchange (80) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_bill_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_item_sk#7) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(7) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(8) CometBroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: [d_date_sk#10, d_date#11] - -(9) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Right output [2]: [d_date_sk#10, d_date#11] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(10) CometProject -Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] -Arguments: [ss_item_sk#7, d_date#11], [ss_item_sk#7, d_date#11] - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#13, i_item_desc#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [2]: [i_item_sk#13, i_item_desc#14] -Condition : isnotnull(i_item_sk#13) - -(13) CometBroadcastExchange -Input [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [i_item_sk#13, i_item_desc#14] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, d_date#11] -Right output [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [ss_item_sk#7], [i_item_sk#13], Inner, BuildRight - -(15) CometProject -Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#13, i_item_desc#14] -Arguments: [d_date#11, i_item_sk#13, _groupingexpression#15], [d_date#11, i_item_sk#13, substr(i_item_desc#14, 1, 30) AS _groupingexpression#15] - -(16) CometHashAggregate -Input [3]: [d_date#11, i_item_sk#13, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [partial_count(1)] - -(17) CometExchange -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [count(1)] - -(19) CometFilter -Input [2]: [item_sk#17, cnt#18] -Condition : (cnt#18 > 4) - -(20) CometProject -Input [2]: [item_sk#17, cnt#18] -Arguments: [item_sk#17], [item_sk#17] - -(21) CometBroadcastExchange -Input [1]: [item_sk#17] -Arguments: [item_sk#17] - -(22) CometBroadcastHashJoin -Left output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [item_sk#17] -Arguments: [cs_item_sk#2], [item_sk#17], LeftSemi, BuildRight - -(23) CometProject -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(24) CometExchange -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(25) CometSort -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#19) - -(28) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Arguments: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21], [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(30) CometFilter -Input [1]: [c_customer_sk#23] -Condition : isnotnull(c_customer_sk#23) - -(31) CometBroadcastExchange -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23] - -(32) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] -Right output [1]: [c_customer_sk#23] -Arguments: [ss_customer_sk#19], [c_customer_sk#23], Inner, BuildRight - -(33) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Arguments: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23], [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] - -(34) CometHashAggregate -Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Keys [1]: [c_customer_sk#23] -Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(35) CometExchange -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(36) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(37) CometFilter -Input [2]: [c_customer_sk#23, ssales#26] -Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#27, [id=#28]))) - -(38) CometProject -Input [2]: [c_customer_sk#23, ssales#26] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(39) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(40) CometSortMergeJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [c_customer_sk#23] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#23], LeftSemi - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Condition : isnotnull(c_customer_sk#29) - -(43) CometExchange -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(44) CometSort -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Arguments: [c_customer_sk#29, c_first_name#30, c_last_name#31], [c_customer_sk#29 ASC NULLS FIRST] - -(45) ReusedExchange [Reuses operator id: 35] -Output [3]: [c_customer_sk#23, sum#24, isEmpty#25] - -(46) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(47) CometFilter -Input [2]: [c_customer_sk#23, ssales#26] -Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) - -(48) CometProject -Input [2]: [c_customer_sk#23, ssales#26] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(49) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(50) CometSortMergeJoin -Left output [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Right output [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#29], [c_customer_sk#23], LeftSemi - -(51) CometProject -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Arguments: [c_customer_sk#29, c_first_name#32, c_last_name#33], [c_customer_sk#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#30, 20)) AS c_first_name#32, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#31, 30)) AS c_last_name#33] - -(52) CometBroadcastExchange -Input [3]: [c_customer_sk#29, c_first_name#32, c_last_name#33] -Arguments: [c_customer_sk#29, c_first_name#32, c_last_name#33] - -(53) CometBroadcastHashJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [3]: [c_customer_sk#29, c_first_name#32, c_last_name#33] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#29], Inner, BuildRight - -(54) CometProject -Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#29, c_first_name#32, c_last_name#33] -Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#35, d_moy#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Condition : ((((isnotnull(d_year#35) AND isnotnull(d_moy#36)) AND (d_year#35 = 2000)) AND (d_moy#36 = 2)) AND isnotnull(d_date_sk#34)) - -(57) CometProject -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Arguments: [d_date_sk#34], [d_date_sk#34] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#34] -Arguments: [d_date_sk#34] - -(59) CometBroadcastHashJoin -Left output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33] -Right output [1]: [d_date_sk#34] -Arguments: [cs_sold_date_sk#5], [d_date_sk#34], Inner, BuildRight - -(60) CometProject -Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33, d_date_sk#34] -Arguments: [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33], [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33] - -(61) CometHashAggregate -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33] -Keys [2]: [c_last_name#33, c_first_name#32] -Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] - -(62) CometExchange -Input [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] -Arguments: hashpartitioning(c_last_name#33, c_first_name#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(63) CometHashAggregate -Input [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] -Keys [2]: [c_last_name#33, c_first_name#32] -Functions [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#43), dynamicpruningexpression(ws_sold_date_sk#43 IN dynamicpruning#44)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(65) CometFilter -Input [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Condition : isnotnull(ws_bill_customer_sk#40) - -(66) ReusedExchange [Reuses operator id: 21] -Output [1]: [item_sk#45] - -(67) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Right output [1]: [item_sk#45] -Arguments: [ws_item_sk#39], [item_sk#45], LeftSemi, BuildRight - -(68) CometProject -Input [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Arguments: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43], [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] - -(69) CometExchange -Input [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Arguments: hashpartitioning(ws_bill_customer_sk#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(70) CometSort -Input [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Arguments: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43], [ws_bill_customer_sk#40 ASC NULLS FIRST] - -(71) ReusedExchange [Reuses operator id: 35] -Output [3]: [c_customer_sk#46, sum#47, isEmpty#48] - -(72) CometHashAggregate -Input [3]: [c_customer_sk#46, sum#47, isEmpty#48] -Keys [1]: [c_customer_sk#46] -Functions [1]: [sum((cast(ss_quantity#49 as decimal(10,0)) * ss_sales_price#50))] - -(73) CometFilter -Input [2]: [c_customer_sk#46, ssales#51] -Condition : (isnotnull(ssales#51) AND (cast(ssales#51 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) - -(74) CometProject -Input [2]: [c_customer_sk#46, ssales#51] -Arguments: [c_customer_sk#46], [c_customer_sk#46] - -(75) CometSort -Input [1]: [c_customer_sk#46] -Arguments: [c_customer_sk#46], [c_customer_sk#46 ASC NULLS FIRST] - -(76) CometSortMergeJoin -Left output [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Right output [1]: [c_customer_sk#46] -Arguments: [ws_bill_customer_sk#40], [c_customer_sk#46], LeftSemi - -(77) ReusedExchange [Reuses operator id: 52] -Output [3]: [c_customer_sk#52, c_first_name#53, c_last_name#54] - -(78) CometBroadcastHashJoin -Left output [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Right output [3]: [c_customer_sk#52, c_first_name#53, c_last_name#54] -Arguments: [ws_bill_customer_sk#40], [c_customer_sk#52], Inner, BuildRight - -(79) CometProject -Input [7]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_customer_sk#52, c_first_name#53, c_last_name#54] -Arguments: [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#53, c_last_name#54], [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#53, c_last_name#54] - -(80) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#55] - -(81) CometBroadcastHashJoin -Left output [5]: [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#53, c_last_name#54] -Right output [1]: [d_date_sk#55] -Arguments: [ws_sold_date_sk#43], [d_date_sk#55], Inner, BuildRight - -(82) CometProject -Input [6]: [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#53, c_last_name#54, d_date_sk#55] -Arguments: [ws_quantity#41, ws_list_price#42, c_first_name#53, c_last_name#54], [ws_quantity#41, ws_list_price#42, c_first_name#53, c_last_name#54] - -(83) CometHashAggregate -Input [4]: [ws_quantity#41, ws_list_price#42, c_first_name#53, c_last_name#54] -Keys [2]: [c_last_name#54, c_first_name#53] -Functions [1]: [partial_sum((cast(ws_quantity#41 as decimal(10,0)) * ws_list_price#42))] - -(84) CometExchange -Input [4]: [c_last_name#54, c_first_name#53, sum#56, isEmpty#57] -Arguments: hashpartitioning(c_last_name#54, c_first_name#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(85) CometHashAggregate -Input [4]: [c_last_name#54, c_first_name#53, sum#56, isEmpty#57] -Keys [2]: [c_last_name#54, c_first_name#53] -Functions [1]: [sum((cast(ws_quantity#41 as decimal(10,0)) * ws_list_price#42))] - -(86) CometUnion -Child 0 Input [3]: [c_last_name#33, c_first_name#32, sales#58] -Child 1 Input [3]: [c_last_name#54, c_first_name#53, sales#59] - -(87) CometTakeOrderedAndProject -Input [3]: [c_last_name#33, c_first_name#32, sales#58] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#33 ASC NULLS FIRST,c_first_name#32 ASC NULLS FIRST,sales#58 ASC NULLS FIRST], output=[c_last_name#33,c_first_name#32,sales#58]), [c_last_name#33, c_first_name#32, sales#58], 100, 0, [c_last_name#33 ASC NULLS FIRST, c_first_name#32 ASC NULLS FIRST, sales#58 ASC NULLS FIRST], [c_last_name#33, c_first_name#32, sales#58] - -(88) CometColumnarToRow [codegen id : 1] -Input [3]: [c_last_name#33, c_first_name#32, sales#58] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (93) -+- * CometColumnarToRow (92) - +- CometProject (91) - +- CometFilter (90) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (89) - - -(89) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#35, d_moy#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(90) CometFilter -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Condition : ((((isnotnull(d_year#35) AND isnotnull(d_moy#36)) AND (d_year#35 = 2000)) AND (d_moy#36 = 2)) AND isnotnull(d_date_sk#34)) - -(91) CometProject -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Arguments: [d_date_sk#34], [d_date_sk#34] - -(92) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#34] - -(93) BroadcastExchange -Input [1]: [d_date_sk#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (98) -+- * CometColumnarToRow (97) - +- CometProject (96) - +- CometFilter (95) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (94) - - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(95) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(96) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(97) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_date#11] - -(98) BroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:3 Hosting operator id = 37 Hosting Expression = Subquery scalar-subquery#27, [id=#28] -* CometColumnarToRow (116) -+- CometHashAggregate (115) - +- CometExchange (114) - +- CometHashAggregate (113) - +- CometHashAggregate (112) - +- CometExchange (111) - +- CometHashAggregate (110) - +- CometProject (109) - +- CometBroadcastHashJoin (108) - :- CometProject (103) - : +- CometBroadcastHashJoin (102) - : :- CometFilter (100) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (99) - : +- ReusedExchange (101) - +- CometBroadcastExchange (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) - - -(99) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#60, ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#63), dynamicpruningexpression(ss_sold_date_sk#63 IN dynamicpruning#64)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(100) CometFilter -Input [4]: [ss_customer_sk#60, ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63] -Condition : isnotnull(ss_customer_sk#60) - -(101) ReusedExchange [Reuses operator id: 31] -Output [1]: [c_customer_sk#65] - -(102) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#60, ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63] -Right output [1]: [c_customer_sk#65] -Arguments: [ss_customer_sk#60], [c_customer_sk#65], Inner, BuildRight - -(103) CometProject -Input [5]: [ss_customer_sk#60, ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63, c_customer_sk#65] -Arguments: [ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63, c_customer_sk#65], [ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63, c_customer_sk#65] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#66, d_year#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [d_date_sk#66, d_year#67] -Condition : (d_year#67 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#66)) - -(106) CometProject -Input [2]: [d_date_sk#66, d_year#67] -Arguments: [d_date_sk#66], [d_date_sk#66] - -(107) CometBroadcastExchange -Input [1]: [d_date_sk#66] -Arguments: [d_date_sk#66] - -(108) CometBroadcastHashJoin -Left output [4]: [ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63, c_customer_sk#65] -Right output [1]: [d_date_sk#66] -Arguments: [ss_sold_date_sk#63], [d_date_sk#66], Inner, BuildRight - -(109) CometProject -Input [5]: [ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63, c_customer_sk#65, d_date_sk#66] -Arguments: [ss_quantity#61, ss_sales_price#62, c_customer_sk#65], [ss_quantity#61, ss_sales_price#62, c_customer_sk#65] - -(110) CometHashAggregate -Input [3]: [ss_quantity#61, ss_sales_price#62, c_customer_sk#65] -Keys [1]: [c_customer_sk#65] -Functions [1]: [partial_sum((cast(ss_quantity#61 as decimal(10,0)) * ss_sales_price#62))] - -(111) CometExchange -Input [3]: [c_customer_sk#65, sum#68, isEmpty#69] -Arguments: hashpartitioning(c_customer_sk#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(112) CometHashAggregate -Input [3]: [c_customer_sk#65, sum#68, isEmpty#69] -Keys [1]: [c_customer_sk#65] -Functions [1]: [sum((cast(ss_quantity#61 as decimal(10,0)) * ss_sales_price#62))] - -(113) CometHashAggregate -Input [1]: [csales#70] -Keys: [] -Functions [1]: [partial_max(csales#70)] - -(114) CometExchange -Input [1]: [max#71] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(115) CometHashAggregate -Input [1]: [max#71] -Keys: [] -Functions [1]: [max(csales#70)] - -(116) CometColumnarToRow [codegen id : 1] -Input [1]: [tpcds_cmax#72] - -Subquery:4 Hosting operator id = 99 Hosting Expression = ss_sold_date_sk#63 IN dynamicpruning#64 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#66, d_year#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#66, d_year#67] -Condition : (d_year#67 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#66)) - -(119) CometProject -Input [2]: [d_date_sk#66, d_year#67] -Arguments: [d_date_sk#66], [d_date_sk#66] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#66] - -(121) BroadcastExchange -Input [1]: [d_date_sk#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -Subquery:5 Hosting operator id = 47 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] - -Subquery:6 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#43 IN dynamicpruning#6 - -Subquery:7 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/simplified.txt deleted file mode 100644 index 92563114ae..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,138 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,sales] - CometUnion [c_last_name,c_first_name,sales] - CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price))] - CometExchange [c_last_name,c_first_name] #1 - CometHashAggregate [cs_quantity,cs_list_price] [c_last_name,c_first_name,sum,isEmpty] - CometProject [cs_quantity,cs_list_price,c_first_name,c_last_name] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometExchange [cs_bill_customer_sk] #2 - CometProject [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,item_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [item_sk] #4 - CometProject [item_sk] - CometFilter [item_sk,cnt] - CometHashAggregate [_groupingexpression,d_date,count] [item_sk,cnt,i_item_sk,count(1)] - CometExchange [_groupingexpression,i_item_sk,d_date] #5 - CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] - CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] - CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [i_item_sk,i_item_desc] #8 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [max] [tpcds_cmax,max(csales)] - CometExchange #11 - CometHashAggregate [csales] [max] - CometHashAggregate [c_customer_sk,sum,isEmpty] [csales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #12 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,c_customer_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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #9 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,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,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #10 - CometFilter [c_customer_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #15 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [c_customer_sk,c_first_name,c_last_name,c_customer_sk] - CometSort [c_customer_sk,c_first_name,c_last_name] - CometExchange [c_customer_sk] #16 - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #9 - CometBroadcastExchange [d_date_sk] #17 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price))] - CometExchange [c_last_name,c_first_name] #18 - CometHashAggregate [ws_quantity,ws_list_price] [c_last_name,c_first_name,sum,isEmpty] - CometProject [ws_quantity,ws_list_price,c_first_name,c_last_name] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometExchange [ws_bill_customer_sk] #19 - CometProject [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #9 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 - ReusedExchange [d_date_sk] #17 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt deleted file mode 100644 index c1131ab0a1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt +++ /dev/null @@ -1,692 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (88) -+- CometTakeOrderedAndProject (87) - +- CometUnion (86) - :- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometSortMergeJoin (40) - : : : :- CometSort (25) - : : : : +- CometExchange (24) - : : : : +- CometProject (23) - : : : : +- CometBroadcastHashJoin (22) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometBroadcastExchange (21) - : : : : +- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometHashAggregate (18) - : : : : +- CometExchange (17) - : : : : +- CometHashAggregate (16) - : : : : +- CometProject (15) - : : : : +- CometBroadcastHashJoin (14) - : : : : :- CometProject (10) - : : : : : +- CometBroadcastHashJoin (9) - : : : : : :- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : : +- CometBroadcastExchange (8) - : : : : : +- CometProject (7) - : : : : : +- CometFilter (6) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (5) - : : : : +- CometBroadcastExchange (13) - : : : : +- CometFilter (12) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : +- CometSort (39) - : : : +- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometHashAggregate (36) - : : : +- CometExchange (35) - : : : +- CometHashAggregate (34) - : : : +- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometProject (28) - : : : : +- CometFilter (27) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (26) - : : : +- CometBroadcastExchange (31) - : : : +- CometFilter (30) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (29) - : : +- CometBroadcastExchange (52) - : : +- CometProject (51) - : : +- CometSortMergeJoin (50) - : : :- CometSort (44) - : : : +- CometExchange (43) - : : : +- CometFilter (42) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (41) - : : +- CometSort (49) - : : +- CometProject (48) - : : +- CometFilter (47) - : : +- CometHashAggregate (46) - : : +- ReusedExchange (45) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometHashAggregate (85) - +- CometExchange (84) - +- CometHashAggregate (83) - +- CometProject (82) - +- CometBroadcastHashJoin (81) - :- CometProject (79) - : +- CometBroadcastHashJoin (78) - : :- CometSortMergeJoin (76) - : : :- CometSort (70) - : : : +- CometExchange (69) - : : : +- CometProject (68) - : : : +- CometBroadcastHashJoin (67) - : : : :- CometFilter (65) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (64) - : : : +- ReusedExchange (66) - : : +- CometSort (75) - : : +- CometProject (74) - : : +- CometFilter (73) - : : +- CometHashAggregate (72) - : : +- ReusedExchange (71) - : +- ReusedExchange (77) - +- ReusedExchange (80) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_bill_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_item_sk#7) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(7) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(8) CometBroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: [d_date_sk#10, d_date#11] - -(9) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Right output [2]: [d_date_sk#10, d_date#11] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(10) CometProject -Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] -Arguments: [ss_item_sk#7, d_date#11], [ss_item_sk#7, d_date#11] - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#13, i_item_desc#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [2]: [i_item_sk#13, i_item_desc#14] -Condition : isnotnull(i_item_sk#13) - -(13) CometBroadcastExchange -Input [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [i_item_sk#13, i_item_desc#14] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, d_date#11] -Right output [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [ss_item_sk#7], [i_item_sk#13], Inner, BuildRight - -(15) CometProject -Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#13, i_item_desc#14] -Arguments: [d_date#11, i_item_sk#13, _groupingexpression#15], [d_date#11, i_item_sk#13, substr(i_item_desc#14, 1, 30) AS _groupingexpression#15] - -(16) CometHashAggregate -Input [3]: [d_date#11, i_item_sk#13, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [partial_count(1)] - -(17) CometExchange -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [count(1)] - -(19) CometFilter -Input [2]: [item_sk#17, cnt#18] -Condition : (cnt#18 > 4) - -(20) CometProject -Input [2]: [item_sk#17, cnt#18] -Arguments: [item_sk#17], [item_sk#17] - -(21) CometBroadcastExchange -Input [1]: [item_sk#17] -Arguments: [item_sk#17] - -(22) CometBroadcastHashJoin -Left output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [item_sk#17] -Arguments: [cs_item_sk#2], [item_sk#17], LeftSemi, BuildRight - -(23) CometProject -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(24) CometExchange -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(25) CometSort -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#19) - -(28) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Arguments: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21], [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(30) CometFilter -Input [1]: [c_customer_sk#23] -Condition : isnotnull(c_customer_sk#23) - -(31) CometBroadcastExchange -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23] - -(32) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] -Right output [1]: [c_customer_sk#23] -Arguments: [ss_customer_sk#19], [c_customer_sk#23], Inner, BuildRight - -(33) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Arguments: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23], [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] - -(34) CometHashAggregate -Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Keys [1]: [c_customer_sk#23] -Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(35) CometExchange -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(36) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(37) CometFilter -Input [2]: [c_customer_sk#23, ssales#26] -Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#27, [id=#28]))) - -(38) CometProject -Input [2]: [c_customer_sk#23, ssales#26] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(39) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(40) CometSortMergeJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [c_customer_sk#23] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#23], LeftSemi - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Condition : isnotnull(c_customer_sk#29) - -(43) CometExchange -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(44) CometSort -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Arguments: [c_customer_sk#29, c_first_name#30, c_last_name#31], [c_customer_sk#29 ASC NULLS FIRST] - -(45) ReusedExchange [Reuses operator id: 35] -Output [3]: [c_customer_sk#23, sum#24, isEmpty#25] - -(46) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(47) CometFilter -Input [2]: [c_customer_sk#23, ssales#26] -Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) - -(48) CometProject -Input [2]: [c_customer_sk#23, ssales#26] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(49) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(50) CometSortMergeJoin -Left output [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Right output [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#29], [c_customer_sk#23], LeftSemi - -(51) CometProject -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Arguments: [c_customer_sk#29, c_first_name#32, c_last_name#33], [c_customer_sk#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#30, 20)) AS c_first_name#32, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#31, 30)) AS c_last_name#33] - -(52) CometBroadcastExchange -Input [3]: [c_customer_sk#29, c_first_name#32, c_last_name#33] -Arguments: [c_customer_sk#29, c_first_name#32, c_last_name#33] - -(53) CometBroadcastHashJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [3]: [c_customer_sk#29, c_first_name#32, c_last_name#33] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#29], Inner, BuildRight - -(54) CometProject -Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#29, c_first_name#32, c_last_name#33] -Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#35, d_moy#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Condition : ((((isnotnull(d_year#35) AND isnotnull(d_moy#36)) AND (d_year#35 = 2000)) AND (d_moy#36 = 2)) AND isnotnull(d_date_sk#34)) - -(57) CometProject -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Arguments: [d_date_sk#34], [d_date_sk#34] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#34] -Arguments: [d_date_sk#34] - -(59) CometBroadcastHashJoin -Left output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33] -Right output [1]: [d_date_sk#34] -Arguments: [cs_sold_date_sk#5], [d_date_sk#34], Inner, BuildRight - -(60) CometProject -Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33, d_date_sk#34] -Arguments: [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33], [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33] - -(61) CometHashAggregate -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33] -Keys [2]: [c_last_name#33, c_first_name#32] -Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] - -(62) CometExchange -Input [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] -Arguments: hashpartitioning(c_last_name#33, c_first_name#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(63) CometHashAggregate -Input [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] -Keys [2]: [c_last_name#33, c_first_name#32] -Functions [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#43), dynamicpruningexpression(ws_sold_date_sk#43 IN dynamicpruning#44)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(65) CometFilter -Input [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Condition : isnotnull(ws_bill_customer_sk#40) - -(66) ReusedExchange [Reuses operator id: 21] -Output [1]: [item_sk#45] - -(67) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Right output [1]: [item_sk#45] -Arguments: [ws_item_sk#39], [item_sk#45], LeftSemi, BuildRight - -(68) CometProject -Input [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Arguments: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43], [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] - -(69) CometExchange -Input [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Arguments: hashpartitioning(ws_bill_customer_sk#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(70) CometSort -Input [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Arguments: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43], [ws_bill_customer_sk#40 ASC NULLS FIRST] - -(71) ReusedExchange [Reuses operator id: 35] -Output [3]: [c_customer_sk#46, sum#47, isEmpty#48] - -(72) CometHashAggregate -Input [3]: [c_customer_sk#46, sum#47, isEmpty#48] -Keys [1]: [c_customer_sk#46] -Functions [1]: [sum((cast(ss_quantity#49 as decimal(10,0)) * ss_sales_price#50))] - -(73) CometFilter -Input [2]: [c_customer_sk#46, ssales#51] -Condition : (isnotnull(ssales#51) AND (cast(ssales#51 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) - -(74) CometProject -Input [2]: [c_customer_sk#46, ssales#51] -Arguments: [c_customer_sk#46], [c_customer_sk#46] - -(75) CometSort -Input [1]: [c_customer_sk#46] -Arguments: [c_customer_sk#46], [c_customer_sk#46 ASC NULLS FIRST] - -(76) CometSortMergeJoin -Left output [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Right output [1]: [c_customer_sk#46] -Arguments: [ws_bill_customer_sk#40], [c_customer_sk#46], LeftSemi - -(77) ReusedExchange [Reuses operator id: 52] -Output [3]: [c_customer_sk#52, c_first_name#53, c_last_name#54] - -(78) CometBroadcastHashJoin -Left output [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Right output [3]: [c_customer_sk#52, c_first_name#53, c_last_name#54] -Arguments: [ws_bill_customer_sk#40], [c_customer_sk#52], Inner, BuildRight - -(79) CometProject -Input [7]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_customer_sk#52, c_first_name#53, c_last_name#54] -Arguments: [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#53, c_last_name#54], [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#53, c_last_name#54] - -(80) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#55] - -(81) CometBroadcastHashJoin -Left output [5]: [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#53, c_last_name#54] -Right output [1]: [d_date_sk#55] -Arguments: [ws_sold_date_sk#43], [d_date_sk#55], Inner, BuildRight - -(82) CometProject -Input [6]: [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#53, c_last_name#54, d_date_sk#55] -Arguments: [ws_quantity#41, ws_list_price#42, c_first_name#53, c_last_name#54], [ws_quantity#41, ws_list_price#42, c_first_name#53, c_last_name#54] - -(83) CometHashAggregate -Input [4]: [ws_quantity#41, ws_list_price#42, c_first_name#53, c_last_name#54] -Keys [2]: [c_last_name#54, c_first_name#53] -Functions [1]: [partial_sum((cast(ws_quantity#41 as decimal(10,0)) * ws_list_price#42))] - -(84) CometExchange -Input [4]: [c_last_name#54, c_first_name#53, sum#56, isEmpty#57] -Arguments: hashpartitioning(c_last_name#54, c_first_name#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(85) CometHashAggregate -Input [4]: [c_last_name#54, c_first_name#53, sum#56, isEmpty#57] -Keys [2]: [c_last_name#54, c_first_name#53] -Functions [1]: [sum((cast(ws_quantity#41 as decimal(10,0)) * ws_list_price#42))] - -(86) CometUnion -Child 0 Input [3]: [c_last_name#33, c_first_name#32, sales#58] -Child 1 Input [3]: [c_last_name#54, c_first_name#53, sales#59] - -(87) CometTakeOrderedAndProject -Input [3]: [c_last_name#33, c_first_name#32, sales#58] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#33 ASC NULLS FIRST,c_first_name#32 ASC NULLS FIRST,sales#58 ASC NULLS FIRST], output=[c_last_name#33,c_first_name#32,sales#58]), [c_last_name#33, c_first_name#32, sales#58], 100, 0, [c_last_name#33 ASC NULLS FIRST, c_first_name#32 ASC NULLS FIRST, sales#58 ASC NULLS FIRST], [c_last_name#33, c_first_name#32, sales#58] - -(88) CometColumnarToRow [codegen id : 1] -Input [3]: [c_last_name#33, c_first_name#32, sales#58] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (93) -+- * CometColumnarToRow (92) - +- CometProject (91) - +- CometFilter (90) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (89) - - -(89) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#35, d_moy#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(90) CometFilter -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Condition : ((((isnotnull(d_year#35) AND isnotnull(d_moy#36)) AND (d_year#35 = 2000)) AND (d_moy#36 = 2)) AND isnotnull(d_date_sk#34)) - -(91) CometProject -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Arguments: [d_date_sk#34], [d_date_sk#34] - -(92) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#34] - -(93) BroadcastExchange -Input [1]: [d_date_sk#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (98) -+- * CometColumnarToRow (97) - +- CometProject (96) - +- CometFilter (95) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (94) - - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(95) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(96) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(97) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_date#11] - -(98) BroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:3 Hosting operator id = 37 Hosting Expression = Subquery scalar-subquery#27, [id=#28] -* CometColumnarToRow (116) -+- CometHashAggregate (115) - +- CometExchange (114) - +- CometHashAggregate (113) - +- CometHashAggregate (112) - +- CometExchange (111) - +- CometHashAggregate (110) - +- CometProject (109) - +- CometBroadcastHashJoin (108) - :- CometProject (103) - : +- CometBroadcastHashJoin (102) - : :- CometFilter (100) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (99) - : +- ReusedExchange (101) - +- CometBroadcastExchange (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) - - -(99) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#60, ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#63), dynamicpruningexpression(ss_sold_date_sk#63 IN dynamicpruning#64)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(100) CometFilter -Input [4]: [ss_customer_sk#60, ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63] -Condition : isnotnull(ss_customer_sk#60) - -(101) ReusedExchange [Reuses operator id: 31] -Output [1]: [c_customer_sk#65] - -(102) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#60, ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63] -Right output [1]: [c_customer_sk#65] -Arguments: [ss_customer_sk#60], [c_customer_sk#65], Inner, BuildRight - -(103) CometProject -Input [5]: [ss_customer_sk#60, ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63, c_customer_sk#65] -Arguments: [ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63, c_customer_sk#65], [ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63, c_customer_sk#65] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#66, d_year#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [d_date_sk#66, d_year#67] -Condition : (d_year#67 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#66)) - -(106) CometProject -Input [2]: [d_date_sk#66, d_year#67] -Arguments: [d_date_sk#66], [d_date_sk#66] - -(107) CometBroadcastExchange -Input [1]: [d_date_sk#66] -Arguments: [d_date_sk#66] - -(108) CometBroadcastHashJoin -Left output [4]: [ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63, c_customer_sk#65] -Right output [1]: [d_date_sk#66] -Arguments: [ss_sold_date_sk#63], [d_date_sk#66], Inner, BuildRight - -(109) CometProject -Input [5]: [ss_quantity#61, ss_sales_price#62, ss_sold_date_sk#63, c_customer_sk#65, d_date_sk#66] -Arguments: [ss_quantity#61, ss_sales_price#62, c_customer_sk#65], [ss_quantity#61, ss_sales_price#62, c_customer_sk#65] - -(110) CometHashAggregate -Input [3]: [ss_quantity#61, ss_sales_price#62, c_customer_sk#65] -Keys [1]: [c_customer_sk#65] -Functions [1]: [partial_sum((cast(ss_quantity#61 as decimal(10,0)) * ss_sales_price#62))] - -(111) CometExchange -Input [3]: [c_customer_sk#65, sum#68, isEmpty#69] -Arguments: hashpartitioning(c_customer_sk#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(112) CometHashAggregate -Input [3]: [c_customer_sk#65, sum#68, isEmpty#69] -Keys [1]: [c_customer_sk#65] -Functions [1]: [sum((cast(ss_quantity#61 as decimal(10,0)) * ss_sales_price#62))] - -(113) CometHashAggregate -Input [1]: [csales#70] -Keys: [] -Functions [1]: [partial_max(csales#70)] - -(114) CometExchange -Input [1]: [max#71] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(115) CometHashAggregate -Input [1]: [max#71] -Keys: [] -Functions [1]: [max(csales#70)] - -(116) CometColumnarToRow [codegen id : 1] -Input [1]: [tpcds_cmax#72] - -Subquery:4 Hosting operator id = 99 Hosting Expression = ss_sold_date_sk#63 IN dynamicpruning#64 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#66, d_year#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#66, d_year#67] -Condition : (d_year#67 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#66)) - -(119) CometProject -Input [2]: [d_date_sk#66, d_year#67] -Arguments: [d_date_sk#66], [d_date_sk#66] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#66] - -(121) BroadcastExchange -Input [1]: [d_date_sk#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -Subquery:5 Hosting operator id = 47 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] - -Subquery:6 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#43 IN dynamicpruning#6 - -Subquery:7 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/extended.txt deleted file mode 100644 index cd91f5de49..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/extended.txt +++ /dev/null @@ -1,198 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometSort - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometSort - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometSort - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometSort - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 177 out of 190 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 92563114ae..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt +++ /dev/null @@ -1,138 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,sales] - CometUnion [c_last_name,c_first_name,sales] - CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price))] - CometExchange [c_last_name,c_first_name] #1 - CometHashAggregate [cs_quantity,cs_list_price] [c_last_name,c_first_name,sum,isEmpty] - CometProject [cs_quantity,cs_list_price,c_first_name,c_last_name] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometExchange [cs_bill_customer_sk] #2 - CometProject [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,item_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [item_sk] #4 - CometProject [item_sk] - CometFilter [item_sk,cnt] - CometHashAggregate [_groupingexpression,d_date,count] [item_sk,cnt,i_item_sk,count(1)] - CometExchange [_groupingexpression,i_item_sk,d_date] #5 - CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] - CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] - CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [i_item_sk,i_item_desc] #8 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [max] [tpcds_cmax,max(csales)] - CometExchange #11 - CometHashAggregate [csales] [max] - CometHashAggregate [c_customer_sk,sum,isEmpty] [csales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #12 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,c_customer_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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #9 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,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,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #10 - CometFilter [c_customer_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #15 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [c_customer_sk,c_first_name,c_last_name,c_customer_sk] - CometSort [c_customer_sk,c_first_name,c_last_name] - CometExchange [c_customer_sk] #16 - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #9 - CometBroadcastExchange [d_date_sk] #17 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price))] - CometExchange [c_last_name,c_first_name] #18 - CometHashAggregate [ws_quantity,ws_list_price] [c_last_name,c_first_name,sum,isEmpty] - CometProject [ws_quantity,ws_list_price,c_first_name,c_last_name] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometExchange [ws_bill_customer_sk] #19 - CometProject [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #9 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 - ReusedExchange [d_date_sk] #17 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_datafusion/explain.txt deleted file mode 100644 index d6087012db..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_datafusion/explain.txt +++ /dev/null @@ -1,439 +0,0 @@ -== Physical Plan == -* Filter (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometNativeScan parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometNativeScan parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.customer_address (32) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometNativeScan parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#14, 10)))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#13, 2)) AS s_state#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#14, 10)) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((isnotnull(i_color#20) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#20, 20)) = pale )) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#19, 20)) AS i_size#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#20, 20)) AS i_color#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#21, 10)) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Condition : (isnotnull(c_customer_sk#26) AND isnotnull(c_birth_country#29)) - -(27) CometProject -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29], [c_customer_sk#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#27, 20)) AS c_first_name#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#28, 30)) AS c_last_name#31, c_birth_country#29] - -(28) CometBroadcastExchange -Input [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(31) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(32) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_state#32, ca_zip#33, ca_country#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Condition : (isnotnull(ca_country#34) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#33, 10)))) - -(34) CometProject -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Arguments: [ca_state#35, ca_zip#36, ca_country#34], [static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#32, 2)) AS ca_state#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#33, 10)) AS ca_zip#36, ca_country#34] - -(35) CometColumnarToRow [codegen id : 1] -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] - -(36) BroadcastExchange -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#29, s_zip#16] -Right keys [2]: [upper(ca_country#34), ca_zip#36] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#37] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(40) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [partial_sum(netpaid#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(44) CometColumnarExchange -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [sum(netpaid#40)] -Aggregate Attributes [1]: [sum(netpaid#40)#45] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] -Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (77) -+- * CometColumnarToRow (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (56) - : : : +- CometBroadcastHashJoin (55) - : : : :- CometProject (53) - : : : : +- CometSortMergeJoin (52) - : : : : :- CometSort (49) - : : : : : +- ReusedExchange (48) - : : : : +- CometSort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (54) - : : +- CometBroadcastExchange (60) - : : +- CometProject (59) - : : +- CometFilter (58) - : : +- CometNativeScan parquet spark_catalog.default.item (57) - : +- ReusedExchange (63) - +- ReusedExchange (67) - - -(48) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] - -(49) CometSort -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53], [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST] - -(50) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#54, sr_ticket_number#55] - -(51) CometSort -Input [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [sr_item_sk#54, sr_ticket_number#55], [sr_ticket_number#55 ASC NULLS FIRST, sr_item_sk#54 ASC NULLS FIRST] - -(52) CometSortMergeJoin -Left output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Right output [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_ticket_number#52, ss_item_sk#49], [sr_ticket_number#55, sr_item_sk#54], Inner - -(53) CometProject -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53], [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] - -(54) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] - -(55) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Right output [4]: [s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] -Arguments: [ss_store_sk#51], [s_store_sk#56], Inner, BuildRight - -(56) CometProject -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59], [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59] - -(57) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Condition : isnotnull(i_item_sk#60) - -(59) CometProject -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65], [i_item_sk#60, i_current_price#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#62, 20)) AS i_size#66, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#63, 20)) AS i_color#67, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#64, 10)) AS i_units#68, i_manager_id#65] - -(60) CometBroadcastExchange -Input [6]: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] - -(61) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59] -Right output [6]: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Arguments: [ss_item_sk#49], [i_item_sk#60], Inner, BuildRight - -(62) CometProject -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Arguments: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65], [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] - -(63) ReusedExchange [Reuses operator id: 28] -Output [4]: [c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] - -(64) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Right output [4]: [c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] -Arguments: [ss_customer_sk#50], [c_customer_sk#69], Inner, BuildRight - -(65) CometProject -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] -Arguments: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72], [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] - -(66) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] - -(67) ReusedExchange [Reuses operator id: 36] -Output [3]: [ca_state#73, ca_zip#74, ca_country#75] - -(68) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#72, s_zip#59] -Right keys [2]: [upper(ca_country#75), ca_zip#74] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 2] -Output [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] -Input [15]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72, ca_state#73, ca_zip#74, ca_country#75] - -(70) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] -Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#76] -Results [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] - -(71) CometColumnarExchange -Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] -Arguments: hashpartitioning(c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(72) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] - -(73) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] -Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#78] - -(74) HashAggregate [codegen id : 3] -Input [1]: [netpaid#78] -Keys: [] -Functions [1]: [partial_avg(netpaid#78)] -Aggregate Attributes [2]: [sum#79, count#80] -Results [2]: [sum#81, count#82] - -(75) CometColumnarExchange -Input [2]: [sum#81, count#82] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#81, count#82] - -(77) HashAggregate [codegen id : 4] -Input [2]: [sum#81, count#82] -Keys: [] -Functions [1]: [avg(netpaid#78)] -Aggregate Attributes [1]: [avg(netpaid#78)#83] -Results [1]: [(0.05 * avg(netpaid#78)#83) AS (0.05 * avg(netpaid))#84] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_datafusion/simplified.txt deleted file mode 100644 index af8d5ee7aa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_datafusion/simplified.txt +++ /dev/null @@ -1,94 +0,0 @@ -WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #6 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometNativeScan 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/q24a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/explain.txt deleted file mode 100644 index df3b1ff6b0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,439 +0,0 @@ -== Physical Plan == -* Filter (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#14, 10)))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#13, 2)) AS s_state#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#14, 10)) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((isnotnull(i_color#20) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#20, 20)) = pale )) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#19, 20)) AS i_size#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#20, 20)) AS i_color#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#21, 10)) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Condition : (isnotnull(c_customer_sk#26) AND isnotnull(c_birth_country#29)) - -(27) CometProject -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29], [c_customer_sk#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#27, 20)) AS c_first_name#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#28, 30)) AS c_last_name#31, c_birth_country#29] - -(28) CometBroadcastExchange -Input [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(31) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_state#32, ca_zip#33, ca_country#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Condition : (isnotnull(ca_country#34) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#33, 10)))) - -(34) CometProject -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Arguments: [ca_state#35, ca_zip#36, ca_country#34], [static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#32, 2)) AS ca_state#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#33, 10)) AS ca_zip#36, ca_country#34] - -(35) CometColumnarToRow [codegen id : 1] -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] - -(36) BroadcastExchange -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#29, s_zip#16] -Right keys [2]: [upper(ca_country#34), ca_zip#36] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#37] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(40) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [partial_sum(netpaid#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(44) CometColumnarExchange -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [sum(netpaid#40)] -Aggregate Attributes [1]: [sum(netpaid#40)#45] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] -Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (77) -+- * CometColumnarToRow (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (56) - : : : +- CometBroadcastHashJoin (55) - : : : :- CometProject (53) - : : : : +- CometSortMergeJoin (52) - : : : : :- CometSort (49) - : : : : : +- ReusedExchange (48) - : : : : +- CometSort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (54) - : : +- CometBroadcastExchange (60) - : : +- CometProject (59) - : : +- CometFilter (58) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (57) - : +- ReusedExchange (63) - +- ReusedExchange (67) - - -(48) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] - -(49) CometSort -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53], [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST] - -(50) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#54, sr_ticket_number#55] - -(51) CometSort -Input [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [sr_item_sk#54, sr_ticket_number#55], [sr_ticket_number#55 ASC NULLS FIRST, sr_item_sk#54 ASC NULLS FIRST] - -(52) CometSortMergeJoin -Left output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Right output [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_ticket_number#52, ss_item_sk#49], [sr_ticket_number#55, sr_item_sk#54], Inner - -(53) CometProject -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53], [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] - -(54) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] - -(55) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Right output [4]: [s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] -Arguments: [ss_store_sk#51], [s_store_sk#56], Inner, BuildRight - -(56) CometProject -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59], [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Condition : isnotnull(i_item_sk#60) - -(59) CometProject -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65], [i_item_sk#60, i_current_price#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#62, 20)) AS i_size#66, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#63, 20)) AS i_color#67, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#64, 10)) AS i_units#68, i_manager_id#65] - -(60) CometBroadcastExchange -Input [6]: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] - -(61) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59] -Right output [6]: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Arguments: [ss_item_sk#49], [i_item_sk#60], Inner, BuildRight - -(62) CometProject -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Arguments: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65], [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] - -(63) ReusedExchange [Reuses operator id: 28] -Output [4]: [c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] - -(64) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Right output [4]: [c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] -Arguments: [ss_customer_sk#50], [c_customer_sk#69], Inner, BuildRight - -(65) CometProject -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] -Arguments: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72], [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] - -(66) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] - -(67) ReusedExchange [Reuses operator id: 36] -Output [3]: [ca_state#73, ca_zip#74, ca_country#75] - -(68) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#72, s_zip#59] -Right keys [2]: [upper(ca_country#75), ca_zip#74] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 2] -Output [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] -Input [15]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72, ca_state#73, ca_zip#74, ca_country#75] - -(70) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] -Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#76] -Results [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] - -(71) CometColumnarExchange -Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] -Arguments: hashpartitioning(c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(72) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] - -(73) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] -Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#78] - -(74) HashAggregate [codegen id : 3] -Input [1]: [netpaid#78] -Keys: [] -Functions [1]: [partial_avg(netpaid#78)] -Aggregate Attributes [2]: [sum#79, count#80] -Results [2]: [sum#81, count#82] - -(75) CometColumnarExchange -Input [2]: [sum#81, count#82] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#81, count#82] - -(77) HashAggregate [codegen id : 4] -Input [2]: [sum#81, count#82] -Keys: [] -Functions [1]: [avg(netpaid#78)] -Aggregate Attributes [1]: [avg(netpaid#78)#83] -Results [1]: [(0.05 * avg(netpaid#78)#83) AS (0.05 * avg(netpaid))#84] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 440f477d0f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,94 +0,0 @@ -WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #6 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] 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/q24a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/explain.txt deleted file mode 100644 index df3b1ff6b0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/explain.txt +++ /dev/null @@ -1,439 +0,0 @@ -== Physical Plan == -* Filter (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#14, 10)))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#13, 2)) AS s_state#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#14, 10)) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((isnotnull(i_color#20) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#20, 20)) = pale )) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#19, 20)) AS i_size#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#20, 20)) AS i_color#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#21, 10)) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Condition : (isnotnull(c_customer_sk#26) AND isnotnull(c_birth_country#29)) - -(27) CometProject -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29], [c_customer_sk#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#27, 20)) AS c_first_name#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#28, 30)) AS c_last_name#31, c_birth_country#29] - -(28) CometBroadcastExchange -Input [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(31) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_state#32, ca_zip#33, ca_country#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Condition : (isnotnull(ca_country#34) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#33, 10)))) - -(34) CometProject -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Arguments: [ca_state#35, ca_zip#36, ca_country#34], [static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#32, 2)) AS ca_state#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#33, 10)) AS ca_zip#36, ca_country#34] - -(35) CometColumnarToRow [codegen id : 1] -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] - -(36) BroadcastExchange -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#29, s_zip#16] -Right keys [2]: [upper(ca_country#34), ca_zip#36] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#37] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(40) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [partial_sum(netpaid#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(44) CometColumnarExchange -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [sum(netpaid#40)] -Aggregate Attributes [1]: [sum(netpaid#40)#45] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] -Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (77) -+- * CometColumnarToRow (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (56) - : : : +- CometBroadcastHashJoin (55) - : : : :- CometProject (53) - : : : : +- CometSortMergeJoin (52) - : : : : :- CometSort (49) - : : : : : +- ReusedExchange (48) - : : : : +- CometSort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (54) - : : +- CometBroadcastExchange (60) - : : +- CometProject (59) - : : +- CometFilter (58) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (57) - : +- ReusedExchange (63) - +- ReusedExchange (67) - - -(48) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] - -(49) CometSort -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53], [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST] - -(50) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#54, sr_ticket_number#55] - -(51) CometSort -Input [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [sr_item_sk#54, sr_ticket_number#55], [sr_ticket_number#55 ASC NULLS FIRST, sr_item_sk#54 ASC NULLS FIRST] - -(52) CometSortMergeJoin -Left output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Right output [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_ticket_number#52, ss_item_sk#49], [sr_ticket_number#55, sr_item_sk#54], Inner - -(53) CometProject -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53], [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] - -(54) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] - -(55) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Right output [4]: [s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] -Arguments: [ss_store_sk#51], [s_store_sk#56], Inner, BuildRight - -(56) CometProject -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59], [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Condition : isnotnull(i_item_sk#60) - -(59) CometProject -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65], [i_item_sk#60, i_current_price#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#62, 20)) AS i_size#66, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#63, 20)) AS i_color#67, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#64, 10)) AS i_units#68, i_manager_id#65] - -(60) CometBroadcastExchange -Input [6]: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] - -(61) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59] -Right output [6]: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Arguments: [ss_item_sk#49], [i_item_sk#60], Inner, BuildRight - -(62) CometProject -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Arguments: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65], [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] - -(63) ReusedExchange [Reuses operator id: 28] -Output [4]: [c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] - -(64) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Right output [4]: [c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] -Arguments: [ss_customer_sk#50], [c_customer_sk#69], Inner, BuildRight - -(65) CometProject -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] -Arguments: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72], [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] - -(66) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] - -(67) ReusedExchange [Reuses operator id: 36] -Output [3]: [ca_state#73, ca_zip#74, ca_country#75] - -(68) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#72, s_zip#59] -Right keys [2]: [upper(ca_country#75), ca_zip#74] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 2] -Output [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] -Input [15]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72, ca_state#73, ca_zip#74, ca_country#75] - -(70) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] -Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#76] -Results [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] - -(71) CometColumnarExchange -Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] -Arguments: hashpartitioning(c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(72) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] - -(73) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] -Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#78] - -(74) HashAggregate [codegen id : 3] -Input [1]: [netpaid#78] -Keys: [] -Functions [1]: [partial_avg(netpaid#78)] -Aggregate Attributes [2]: [sum#79, count#80] -Results [2]: [sum#81, count#82] - -(75) CometColumnarExchange -Input [2]: [sum#81, count#82] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#81, count#82] - -(77) HashAggregate [codegen id : 4] -Input [2]: [sum#81, count#82] -Keys: [] -Functions [1]: [avg(netpaid#78)] -Aggregate Attributes [1]: [avg(netpaid#78)#83] -Results [1]: [(0.05 * avg(netpaid#78)#83) AS (0.05 * avg(netpaid))#84] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/extended.txt deleted file mode 100644 index 8777e81a37..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/extended.txt +++ /dev/null @@ -1,96 +0,0 @@ -Filter -: +- Subquery -: +- HashAggregate -: +- CometColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- HashAggregate -: +- CometColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- Project -: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometColumnarToRow -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometSortMergeJoin -: : : : : :- CometSort -: : : : : : +- CometExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : +- CometSort -: : : : : +- CometExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -: +- BroadcastExchange -: +- CometColumnarToRow -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 70 out of 86 eligible operators (81%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 440f477d0f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt +++ /dev/null @@ -1,94 +0,0 @@ -WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #6 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_datafusion/explain.txt deleted file mode 100644 index f4e4f60060..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_datafusion/explain.txt +++ /dev/null @@ -1,439 +0,0 @@ -== Physical Plan == -* Filter (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometNativeScan parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometNativeScan parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.customer_address (32) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometNativeScan parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#14, 10)))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#13, 2)) AS s_state#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#14, 10)) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((isnotnull(i_color#20) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#20, 20)) = chiffon )) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#19, 20)) AS i_size#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#20, 20)) AS i_color#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#21, 10)) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Condition : (isnotnull(c_customer_sk#26) AND isnotnull(c_birth_country#29)) - -(27) CometProject -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29], [c_customer_sk#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#27, 20)) AS c_first_name#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#28, 30)) AS c_last_name#31, c_birth_country#29] - -(28) CometBroadcastExchange -Input [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(31) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(32) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_state#32, ca_zip#33, ca_country#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Condition : (isnotnull(ca_country#34) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#33, 10)))) - -(34) CometProject -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Arguments: [ca_state#35, ca_zip#36, ca_country#34], [static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#32, 2)) AS ca_state#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#33, 10)) AS ca_zip#36, ca_country#34] - -(35) CometColumnarToRow [codegen id : 1] -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] - -(36) BroadcastExchange -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#29, s_zip#16] -Right keys [2]: [upper(ca_country#34), ca_zip#36] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#37] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(40) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [partial_sum(netpaid#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(44) CometColumnarExchange -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [sum(netpaid#40)] -Aggregate Attributes [1]: [sum(netpaid#40)#45] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] -Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (77) -+- * CometColumnarToRow (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (56) - : : : +- CometBroadcastHashJoin (55) - : : : :- CometProject (53) - : : : : +- CometSortMergeJoin (52) - : : : : :- CometSort (49) - : : : : : +- ReusedExchange (48) - : : : : +- CometSort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (54) - : : +- CometBroadcastExchange (60) - : : +- CometProject (59) - : : +- CometFilter (58) - : : +- CometNativeScan parquet spark_catalog.default.item (57) - : +- ReusedExchange (63) - +- ReusedExchange (67) - - -(48) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] - -(49) CometSort -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53], [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST] - -(50) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#54, sr_ticket_number#55] - -(51) CometSort -Input [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [sr_item_sk#54, sr_ticket_number#55], [sr_ticket_number#55 ASC NULLS FIRST, sr_item_sk#54 ASC NULLS FIRST] - -(52) CometSortMergeJoin -Left output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Right output [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_ticket_number#52, ss_item_sk#49], [sr_ticket_number#55, sr_item_sk#54], Inner - -(53) CometProject -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53], [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] - -(54) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] - -(55) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Right output [4]: [s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] -Arguments: [ss_store_sk#51], [s_store_sk#56], Inner, BuildRight - -(56) CometProject -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59], [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59] - -(57) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Condition : isnotnull(i_item_sk#60) - -(59) CometProject -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65], [i_item_sk#60, i_current_price#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#62, 20)) AS i_size#66, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#63, 20)) AS i_color#67, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#64, 10)) AS i_units#68, i_manager_id#65] - -(60) CometBroadcastExchange -Input [6]: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] - -(61) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59] -Right output [6]: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Arguments: [ss_item_sk#49], [i_item_sk#60], Inner, BuildRight - -(62) CometProject -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Arguments: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65], [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] - -(63) ReusedExchange [Reuses operator id: 28] -Output [4]: [c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] - -(64) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Right output [4]: [c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] -Arguments: [ss_customer_sk#50], [c_customer_sk#69], Inner, BuildRight - -(65) CometProject -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] -Arguments: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72], [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] - -(66) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] - -(67) ReusedExchange [Reuses operator id: 36] -Output [3]: [ca_state#73, ca_zip#74, ca_country#75] - -(68) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#72, s_zip#59] -Right keys [2]: [upper(ca_country#75), ca_zip#74] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 2] -Output [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] -Input [15]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72, ca_state#73, ca_zip#74, ca_country#75] - -(70) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] -Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#76] -Results [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] - -(71) CometColumnarExchange -Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] -Arguments: hashpartitioning(c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(72) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] - -(73) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] -Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#78] - -(74) HashAggregate [codegen id : 3] -Input [1]: [netpaid#78] -Keys: [] -Functions [1]: [partial_avg(netpaid#78)] -Aggregate Attributes [2]: [sum#79, count#80] -Results [2]: [sum#81, count#82] - -(75) CometColumnarExchange -Input [2]: [sum#81, count#82] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#81, count#82] - -(77) HashAggregate [codegen id : 4] -Input [2]: [sum#81, count#82] -Keys: [] -Functions [1]: [avg(netpaid#78)] -Aggregate Attributes [1]: [avg(netpaid#78)#83] -Results [1]: [(0.05 * avg(netpaid#78)#83) AS (0.05 * avg(netpaid))#84] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_datafusion/simplified.txt deleted file mode 100644 index af8d5ee7aa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_datafusion/simplified.txt +++ /dev/null @@ -1,94 +0,0 @@ -WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #6 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometNativeScan 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.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/explain.txt deleted file mode 100644 index 4e088146fe..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,439 +0,0 @@ -== Physical Plan == -* Filter (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#14, 10)))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#13, 2)) AS s_state#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#14, 10)) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((isnotnull(i_color#20) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#20, 20)) = chiffon )) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#19, 20)) AS i_size#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#20, 20)) AS i_color#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#21, 10)) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Condition : (isnotnull(c_customer_sk#26) AND isnotnull(c_birth_country#29)) - -(27) CometProject -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29], [c_customer_sk#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#27, 20)) AS c_first_name#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#28, 30)) AS c_last_name#31, c_birth_country#29] - -(28) CometBroadcastExchange -Input [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(31) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_state#32, ca_zip#33, ca_country#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Condition : (isnotnull(ca_country#34) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#33, 10)))) - -(34) CometProject -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Arguments: [ca_state#35, ca_zip#36, ca_country#34], [static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#32, 2)) AS ca_state#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#33, 10)) AS ca_zip#36, ca_country#34] - -(35) CometColumnarToRow [codegen id : 1] -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] - -(36) BroadcastExchange -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#29, s_zip#16] -Right keys [2]: [upper(ca_country#34), ca_zip#36] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#37] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(40) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [partial_sum(netpaid#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(44) CometColumnarExchange -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [sum(netpaid#40)] -Aggregate Attributes [1]: [sum(netpaid#40)#45] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] -Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (77) -+- * CometColumnarToRow (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (56) - : : : +- CometBroadcastHashJoin (55) - : : : :- CometProject (53) - : : : : +- CometSortMergeJoin (52) - : : : : :- CometSort (49) - : : : : : +- ReusedExchange (48) - : : : : +- CometSort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (54) - : : +- CometBroadcastExchange (60) - : : +- CometProject (59) - : : +- CometFilter (58) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (57) - : +- ReusedExchange (63) - +- ReusedExchange (67) - - -(48) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] - -(49) CometSort -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53], [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST] - -(50) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#54, sr_ticket_number#55] - -(51) CometSort -Input [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [sr_item_sk#54, sr_ticket_number#55], [sr_ticket_number#55 ASC NULLS FIRST, sr_item_sk#54 ASC NULLS FIRST] - -(52) CometSortMergeJoin -Left output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Right output [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_ticket_number#52, ss_item_sk#49], [sr_ticket_number#55, sr_item_sk#54], Inner - -(53) CometProject -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53], [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] - -(54) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] - -(55) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Right output [4]: [s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] -Arguments: [ss_store_sk#51], [s_store_sk#56], Inner, BuildRight - -(56) CometProject -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59], [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Condition : isnotnull(i_item_sk#60) - -(59) CometProject -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65], [i_item_sk#60, i_current_price#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#62, 20)) AS i_size#66, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#63, 20)) AS i_color#67, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#64, 10)) AS i_units#68, i_manager_id#65] - -(60) CometBroadcastExchange -Input [6]: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] - -(61) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59] -Right output [6]: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Arguments: [ss_item_sk#49], [i_item_sk#60], Inner, BuildRight - -(62) CometProject -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Arguments: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65], [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] - -(63) ReusedExchange [Reuses operator id: 28] -Output [4]: [c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] - -(64) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Right output [4]: [c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] -Arguments: [ss_customer_sk#50], [c_customer_sk#69], Inner, BuildRight - -(65) CometProject -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] -Arguments: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72], [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] - -(66) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] - -(67) ReusedExchange [Reuses operator id: 36] -Output [3]: [ca_state#73, ca_zip#74, ca_country#75] - -(68) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#72, s_zip#59] -Right keys [2]: [upper(ca_country#75), ca_zip#74] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 2] -Output [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] -Input [15]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72, ca_state#73, ca_zip#74, ca_country#75] - -(70) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] -Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#76] -Results [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] - -(71) CometColumnarExchange -Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] -Arguments: hashpartitioning(c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(72) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] - -(73) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] -Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#78] - -(74) HashAggregate [codegen id : 3] -Input [1]: [netpaid#78] -Keys: [] -Functions [1]: [partial_avg(netpaid#78)] -Aggregate Attributes [2]: [sum#79, count#80] -Results [2]: [sum#81, count#82] - -(75) CometColumnarExchange -Input [2]: [sum#81, count#82] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#81, count#82] - -(77) HashAggregate [codegen id : 4] -Input [2]: [sum#81, count#82] -Keys: [] -Functions [1]: [avg(netpaid#78)] -Aggregate Attributes [1]: [avg(netpaid#78)#83] -Results [1]: [(0.05 * avg(netpaid#78)#83) AS (0.05 * avg(netpaid))#84] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/simplified.txt deleted file mode 100644 index 440f477d0f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,94 +0,0 @@ -WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #6 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] 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/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/explain.txt deleted file mode 100644 index 4e088146fe..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/explain.txt +++ /dev/null @@ -1,439 +0,0 @@ -== Physical Plan == -* Filter (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#14, 10)))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#13, 2)) AS s_state#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#14, 10)) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((isnotnull(i_color#20) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#20, 20)) = chiffon )) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#19, 20)) AS i_size#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#20, 20)) AS i_color#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#21, 10)) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Condition : (isnotnull(c_customer_sk#26) AND isnotnull(c_birth_country#29)) - -(27) CometProject -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29], [c_customer_sk#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#27, 20)) AS c_first_name#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#28, 30)) AS c_last_name#31, c_birth_country#29] - -(28) CometBroadcastExchange -Input [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(31) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_state#32, ca_zip#33, ca_country#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Condition : (isnotnull(ca_country#34) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#33, 10)))) - -(34) CometProject -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Arguments: [ca_state#35, ca_zip#36, ca_country#34], [static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#32, 2)) AS ca_state#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#33, 10)) AS ca_zip#36, ca_country#34] - -(35) CometColumnarToRow [codegen id : 1] -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] - -(36) BroadcastExchange -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#29, s_zip#16] -Right keys [2]: [upper(ca_country#34), ca_zip#36] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#37] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(40) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [partial_sum(netpaid#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(44) CometColumnarExchange -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [sum(netpaid#40)] -Aggregate Attributes [1]: [sum(netpaid#40)#45] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] -Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (77) -+- * CometColumnarToRow (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (56) - : : : +- CometBroadcastHashJoin (55) - : : : :- CometProject (53) - : : : : +- CometSortMergeJoin (52) - : : : : :- CometSort (49) - : : : : : +- ReusedExchange (48) - : : : : +- CometSort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (54) - : : +- CometBroadcastExchange (60) - : : +- CometProject (59) - : : +- CometFilter (58) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (57) - : +- ReusedExchange (63) - +- ReusedExchange (67) - - -(48) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] - -(49) CometSort -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53], [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST] - -(50) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#54, sr_ticket_number#55] - -(51) CometSort -Input [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [sr_item_sk#54, sr_ticket_number#55], [sr_ticket_number#55 ASC NULLS FIRST, sr_item_sk#54 ASC NULLS FIRST] - -(52) CometSortMergeJoin -Left output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Right output [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_ticket_number#52, ss_item_sk#49], [sr_ticket_number#55, sr_item_sk#54], Inner - -(53) CometProject -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53], [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] - -(54) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] - -(55) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Right output [4]: [s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] -Arguments: [ss_store_sk#51], [s_store_sk#56], Inner, BuildRight - -(56) CometProject -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#56, s_store_name#57, s_state#58, s_zip#59] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59], [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Condition : isnotnull(i_item_sk#60) - -(59) CometProject -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65], [i_item_sk#60, i_current_price#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#62, 20)) AS i_size#66, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#63, 20)) AS i_color#67, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#64, 10)) AS i_units#68, i_manager_id#65] - -(60) CometBroadcastExchange -Input [6]: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] - -(61) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59] -Right output [6]: [i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Arguments: [ss_item_sk#49], [i_item_sk#60], Inner, BuildRight - -(62) CometProject -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_item_sk#60, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Arguments: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65], [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] - -(63) ReusedExchange [Reuses operator id: 28] -Output [4]: [c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] - -(64) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65] -Right output [4]: [c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] -Arguments: [ss_customer_sk#50], [c_customer_sk#69], Inner, BuildRight - -(65) CometProject -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_customer_sk#69, c_first_name#70, c_last_name#71, c_birth_country#72] -Arguments: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72], [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] - -(66) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72] - -(67) ReusedExchange [Reuses operator id: 36] -Output [3]: [ca_state#73, ca_zip#74, ca_country#75] - -(68) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#72, s_zip#59] -Right keys [2]: [upper(ca_country#75), ca_zip#74] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 2] -Output [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] -Input [15]: [ss_net_paid#53, s_store_name#57, s_state#58, s_zip#59, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, c_birth_country#72, ca_state#73, ca_zip#74, ca_country#75] - -(70) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#53, s_store_name#57, s_state#58, i_current_price#61, i_size#66, i_color#67, i_units#68, i_manager_id#65, c_first_name#70, c_last_name#71, ca_state#73] -Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#76] -Results [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] - -(71) CometColumnarExchange -Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] -Arguments: hashpartitioning(c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(72) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] - -(73) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66, sum#77] -Keys [10]: [c_last_name#71, c_first_name#70, s_store_name#57, ca_state#73, s_state#58, i_color#67, i_current_price#61, i_manager_id#65, i_units#68, i_size#66] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#78] - -(74) HashAggregate [codegen id : 3] -Input [1]: [netpaid#78] -Keys: [] -Functions [1]: [partial_avg(netpaid#78)] -Aggregate Attributes [2]: [sum#79, count#80] -Results [2]: [sum#81, count#82] - -(75) CometColumnarExchange -Input [2]: [sum#81, count#82] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#81, count#82] - -(77) HashAggregate [codegen id : 4] -Input [2]: [sum#81, count#82] -Keys: [] -Functions [1]: [avg(netpaid#78)] -Aggregate Attributes [1]: [avg(netpaid#78)#83] -Results [1]: [(0.05 * avg(netpaid#78)#83) AS (0.05 * avg(netpaid))#84] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/extended.txt deleted file mode 100644 index 8777e81a37..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/extended.txt +++ /dev/null @@ -1,96 +0,0 @@ -Filter -: +- Subquery -: +- HashAggregate -: +- CometColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- HashAggregate -: +- CometColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- Project -: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometColumnarToRow -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometSortMergeJoin -: : : : : :- CometSort -: : : : : : +- CometExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : +- CometSort -: : : : : +- CometExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -: +- BroadcastExchange -: +- CometColumnarToRow -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 70 out of 86 eligible operators (81%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 440f477d0f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt +++ /dev/null @@ -1,94 +0,0 @@ -WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #6 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/explain.txt deleted file mode 100644 index 36bfea5046..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/explain.txt +++ /dev/null @@ -1,312 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (18) - : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet spark_catalog.default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * Filter (12) - : : : : : +- * ColumnarToRow (11) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (10) - : : : : +- ReusedExchange (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.store (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.item (32) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 8] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] - -(3) Filter [codegen id : 8] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(4) Scan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] - -(6) Filter [codegen id : 1] -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(7) BroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 8] -Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] -Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] - -(10) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] - -(12) Filter [codegen id : 2] -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(13) BroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] -Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] - -(16) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#18] - -(17) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#18] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 8] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#18] - -(19) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#19] - -(20) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [sr_returned_date_sk#12] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#19] - -(22) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#20] - -(23) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 8] -Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#20] - -(25) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#21, s_store_id#22, s_store_name#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(26) CometFilter -Input [3]: [s_store_sk#21, s_store_id#22, s_store_name#23] -Condition : isnotnull(s_store_sk#21) - -(27) CometProject -Input [3]: [s_store_sk#21, s_store_id#22, s_store_name#23] -Arguments: [s_store_sk#21, s_store_id#24, s_store_name#23], [s_store_sk#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#22, 16)) AS s_store_id#24, s_store_name#23] - -(28) CometColumnarToRow [codegen id : 6] -Input [3]: [s_store_sk#21, s_store_id#24, s_store_name#23] - -(29) BroadcastExchange -Input [3]: [s_store_sk#21, s_store_id#24, s_store_name#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(30) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#21] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_sk#21, s_store_id#24, s_store_name#23] - -(32) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] -Condition : isnotnull(i_item_sk#25) - -(34) CometProject -Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] -Arguments: [i_item_sk#25, i_item_id#28, i_item_desc#27], [i_item_sk#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#26, 16)) AS i_item_id#28, i_item_desc#27] - -(35) CometColumnarToRow [codegen id : 7] -Input [3]: [i_item_sk#25, i_item_id#28, i_item_desc#27] - -(36) BroadcastExchange -Input [3]: [i_item_sk#25, i_item_id#28, i_item_desc#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(37) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#25] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 8] -Output [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23, i_item_id#28, i_item_desc#27] -Input [9]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23, i_item_sk#25, i_item_id#28, i_item_desc#27] - -(39) HashAggregate [codegen id : 8] -Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23, i_item_id#28, i_item_desc#27] -Keys [4]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23] -Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#11)), partial_sum(UnscaledValue(cs_net_profit#16))] -Aggregate Attributes [3]: [sum#29, sum#30, sum#31] -Results [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum#32, sum#33, sum#34] - -(40) CometColumnarExchange -Input [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum#32, sum#33, sum#34] -Arguments: hashpartitioning(i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(41) CometColumnarToRow [codegen id : 9] -Input [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum#32, sum#33, sum#34] - -(42) HashAggregate [codegen id : 9] -Input [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum#32, sum#33, sum#34] -Keys [4]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23] -Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#35, sum(UnscaledValue(sr_net_loss#11))#36, sum(UnscaledValue(cs_net_profit#16))#37] -Results [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#35,17,2) AS store_sales_profit#38, MakeDecimal(sum(UnscaledValue(sr_net_loss#11))#36,17,2) AS store_returns_loss#39, MakeDecimal(sum(UnscaledValue(cs_net_profit#16))#37,17,2) AS catalog_sales_profit#40] - -(43) TakeOrderedAndProject -Input [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, store_sales_profit#38, store_returns_loss#39, catalog_sales_profit#40] -Arguments: 100, [i_item_id#28 ASC NULLS FIRST, i_item_desc#27 ASC NULLS FIRST, s_store_id#24 ASC NULLS FIRST, s_store_name#23 ASC NULLS FIRST], [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, store_sales_profit#38, store_returns_loss#39, catalog_sales_profit#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometNativeScan parquet spark_catalog.default.date_dim (44) - - -(44) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#18, d_year#41, d_moy#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [3]: [d_date_sk#18, d_year#41, d_moy#42] -Condition : ((((isnotnull(d_moy#42) AND isnotnull(d_year#41)) AND (d_moy#42 = 4)) AND (d_year#41 = 2001)) AND isnotnull(d_date_sk#18)) - -(46) CometProject -Input [3]: [d_date_sk#18, d_year#41, d_moy#42] -Arguments: [d_date_sk#18], [d_date_sk#18] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#18] - -(48) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometNativeScan parquet spark_catalog.default.date_dim (49) - - -(49) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#43, d_moy#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#19, d_year#43, d_moy#44] -Condition : (((((isnotnull(d_moy#44) AND isnotnull(d_year#43)) AND (d_moy#44 >= 4)) AND (d_moy#44 <= 10)) AND (d_year#43 = 2001)) AND isnotnull(d_date_sk#19)) - -(51) CometProject -Input [3]: [d_date_sk#19, d_year#43, d_moy#44] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(53) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/simplified.txt deleted file mode 100644 index 288561a740..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/simplified.txt +++ /dev/null @@ -1,79 +0,0 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] - WholeStageCodegen (9) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit)),store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - WholeStageCodegen (8) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] [sum,sum,sum,sum,sum,sum] - Project [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [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] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Filter [cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometNativeScan 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/q25.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/explain.txt deleted file mode 100644 index f076c7886b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,319 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometProject (18) - : : : : +- CometBroadcastHashJoin (17) - : : : : :- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (8) - : : : : +- CometBroadcastExchange (16) - : : : : +- CometProject (15) - : : : : +- CometFilter (14) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : : : +- CometBroadcastExchange (22) - : : : +- CometProject (21) - : : : +- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - : : +- ReusedExchange (25) - : +- CometBroadcastExchange (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (28) - +- CometBroadcastExchange (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (34) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(5) CometBroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] - -(6) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(10) CometBroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] - -(11) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] -Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight - -(12) CometProject -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(15) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 4)) AND (d_moy#24 <= 10)) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(21) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#22] -Arguments: [sr_returned_date_sk#12], [d_date_sk#22], Inner, BuildRight - -(24) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#22] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] - -(25) ReusedExchange [Reuses operator id: 22] -Output [1]: [d_date_sk#25] - -(26) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#25] -Arguments: [cs_sold_date_sk#17], [d_date_sk#25], Inner, BuildRight - -(27) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#25] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(29) CometFilter -Input [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] -Condition : isnotnull(s_store_sk#26) - -(30) CometProject -Input [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] -Arguments: [s_store_sk#26, s_store_id#29, s_store_name#28], [s_store_sk#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#27, 16)) AS s_store_id#29, s_store_name#28] - -(31) CometBroadcastExchange -Input [3]: [s_store_sk#26, s_store_id#29, s_store_name#28] -Arguments: [s_store_sk#26, s_store_id#29, s_store_name#28] - -(32) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] -Right output [3]: [s_store_sk#26, s_store_id#29, s_store_name#28] -Arguments: [ss_store_sk#3], [s_store_sk#26], Inner, BuildRight - -(33) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_sk#26, s_store_id#29, s_store_name#28] -Arguments: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28], [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28] - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] -Condition : isnotnull(i_item_sk#30) - -(36) CometProject -Input [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] -Arguments: [i_item_sk#30, i_item_id#33, i_item_desc#32], [i_item_sk#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#31, 16)) AS i_item_id#33, i_item_desc#32] - -(37) CometBroadcastExchange -Input [3]: [i_item_sk#30, i_item_id#33, i_item_desc#32] -Arguments: [i_item_sk#30, i_item_id#33, i_item_desc#32] - -(38) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28] -Right output [3]: [i_item_sk#30, i_item_id#33, i_item_desc#32] -Arguments: [ss_item_sk#1], [i_item_sk#30], Inner, BuildRight - -(39) CometProject -Input [9]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_sk#30, i_item_id#33, i_item_desc#32] -Arguments: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_id#33, i_item_desc#32], [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_id#33, i_item_desc#32] - -(40) CometHashAggregate -Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_id#33, i_item_desc#32] -Keys [4]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28] -Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#11)), partial_sum(UnscaledValue(cs_net_profit#16))] - -(41) CometExchange -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, sum#34, sum#35, sum#36] -Arguments: hashpartitioning(i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(42) CometHashAggregate -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, sum#34, sum#35, sum#36] -Keys [4]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28] -Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] - -(43) CometTakeOrderedAndProject -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#33 ASC NULLS FIRST,i_item_desc#32 ASC NULLS FIRST,s_store_id#29 ASC NULLS FIRST,s_store_name#28 ASC NULLS FIRST], output=[i_item_id#33,i_item_desc#32,s_store_id#29,s_store_name#28,store_sales_profit#37,store_returns_loss#38,catalog_sales_profit#39]), [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39], 100, 0, [i_item_id#33 ASC NULLS FIRST, i_item_desc#32 ASC NULLS FIRST, s_store_id#29 ASC NULLS FIRST, s_store_name#28 ASC NULLS FIRST], [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] - -(44) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(47) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(49) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) - - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 4)) AND (d_moy#24 <= 10)) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(52) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(53) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(54) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/simplified.txt deleted file mode 100644 index dcad304452..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,63 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] - CometHashAggregate [sum,sum,sum] [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit))] - CometExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - CometHashAggregate [ss_net_profit,sr_net_loss,cs_net_profit] [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] - 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,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_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 [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 [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_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 [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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] 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 [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #8 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan [native_iceberg_compat] 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/q25/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt deleted file mode 100644 index f076c7886b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt +++ /dev/null @@ -1,319 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometProject (18) - : : : : +- CometBroadcastHashJoin (17) - : : : : :- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (8) - : : : : +- CometBroadcastExchange (16) - : : : : +- CometProject (15) - : : : : +- CometFilter (14) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : : : +- CometBroadcastExchange (22) - : : : +- CometProject (21) - : : : +- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - : : +- ReusedExchange (25) - : +- CometBroadcastExchange (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (28) - +- CometBroadcastExchange (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (34) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(5) CometBroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] - -(6) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(10) CometBroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] - -(11) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] -Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight - -(12) CometProject -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(15) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 4)) AND (d_moy#24 <= 10)) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(21) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#22] -Arguments: [sr_returned_date_sk#12], [d_date_sk#22], Inner, BuildRight - -(24) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#22] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] - -(25) ReusedExchange [Reuses operator id: 22] -Output [1]: [d_date_sk#25] - -(26) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#25] -Arguments: [cs_sold_date_sk#17], [d_date_sk#25], Inner, BuildRight - -(27) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#25] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(29) CometFilter -Input [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] -Condition : isnotnull(s_store_sk#26) - -(30) CometProject -Input [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] -Arguments: [s_store_sk#26, s_store_id#29, s_store_name#28], [s_store_sk#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#27, 16)) AS s_store_id#29, s_store_name#28] - -(31) CometBroadcastExchange -Input [3]: [s_store_sk#26, s_store_id#29, s_store_name#28] -Arguments: [s_store_sk#26, s_store_id#29, s_store_name#28] - -(32) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] -Right output [3]: [s_store_sk#26, s_store_id#29, s_store_name#28] -Arguments: [ss_store_sk#3], [s_store_sk#26], Inner, BuildRight - -(33) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_sk#26, s_store_id#29, s_store_name#28] -Arguments: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28], [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28] - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] -Condition : isnotnull(i_item_sk#30) - -(36) CometProject -Input [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] -Arguments: [i_item_sk#30, i_item_id#33, i_item_desc#32], [i_item_sk#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#31, 16)) AS i_item_id#33, i_item_desc#32] - -(37) CometBroadcastExchange -Input [3]: [i_item_sk#30, i_item_id#33, i_item_desc#32] -Arguments: [i_item_sk#30, i_item_id#33, i_item_desc#32] - -(38) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28] -Right output [3]: [i_item_sk#30, i_item_id#33, i_item_desc#32] -Arguments: [ss_item_sk#1], [i_item_sk#30], Inner, BuildRight - -(39) CometProject -Input [9]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_sk#30, i_item_id#33, i_item_desc#32] -Arguments: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_id#33, i_item_desc#32], [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_id#33, i_item_desc#32] - -(40) CometHashAggregate -Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_id#33, i_item_desc#32] -Keys [4]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28] -Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#11)), partial_sum(UnscaledValue(cs_net_profit#16))] - -(41) CometExchange -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, sum#34, sum#35, sum#36] -Arguments: hashpartitioning(i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(42) CometHashAggregate -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, sum#34, sum#35, sum#36] -Keys [4]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28] -Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] - -(43) CometTakeOrderedAndProject -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#33 ASC NULLS FIRST,i_item_desc#32 ASC NULLS FIRST,s_store_id#29 ASC NULLS FIRST,s_store_name#28 ASC NULLS FIRST], output=[i_item_id#33,i_item_desc#32,s_store_id#29,s_store_name#28,store_sales_profit#37,store_returns_loss#38,catalog_sales_profit#39]), [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39], 100, 0, [i_item_id#33 ASC NULLS FIRST, i_item_desc#32 ASC NULLS FIRST, s_store_id#29 ASC NULLS FIRST, s_store_name#28 ASC NULLS FIRST], [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] - -(44) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(47) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(49) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) - - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 4)) AND (d_moy#24 <= 10)) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(52) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(53) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(54) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/extended.txt deleted file mode 100644 index 4e1a4f3ec0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/extended.txt +++ /dev/null @@ -1,62 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index dcad304452..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt +++ /dev/null @@ -1,63 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] - CometHashAggregate [sum,sum,sum] [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit))] - CometExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - CometHashAggregate [ss_net_profit,sr_net_loss,cs_net_profit] [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] - 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,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_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 [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 [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_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 [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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] 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 [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #8 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/explain.txt deleted file mode 100644 index 5e1cd8b012..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- * CometColumnarToRow (30) - +- CometColumnarExchange (29) - +- * HashAggregate (28) - +- * Project (27) - +- * BroadcastHashJoin Inner BuildRight (26) - :- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.item (14) - +- BroadcastExchange (25) - +- * CometColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometNativeScan parquet spark_catalog.default.promotion (21) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] - -(3) Filter [codegen id : 5] -Input [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Condition : ((isnotnull(cs_bill_cdemo_sk#1) AND isnotnull(cs_item_sk#2)) AND isnotnull(cs_promo_sk#3)) - -(4) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#11, 1)) = M)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#12, 1)) = S)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#13, 20)) = College )) AND isnotnull(cd_demo_sk#10)) - -(6) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [cd_demo_sk#10] - -(8) BroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_bill_cdemo_sk#1] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 5] -Output [7]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Input [9]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, cd_demo_sk#10] - -(11) ReusedExchange [Reuses operator id: 37] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [6]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] -Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, d_date_sk#14] - -(14) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#15, i_item_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [i_item_sk#15, i_item_id#16] -Condition : isnotnull(i_item_sk#15) - -(16) CometProject -Input [2]: [i_item_sk#15, i_item_id#16] -Arguments: [i_item_sk#15, i_item_id#17], [i_item_sk#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#16, 16)) AS i_item_id#17] - -(17) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#15, i_item_id#17] - -(18) BroadcastExchange -Input [2]: [i_item_sk#15, i_item_id#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [6]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] -Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_sk#15, i_item_id#17] - -(21) CometNativeScan parquet spark_catalog.default.promotion -Output [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] -Condition : (((static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_email#19, 1)) = N) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_event#20, 1)) = N)) AND isnotnull(p_promo_sk#18)) - -(23) CometProject -Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] -Arguments: [p_promo_sk#18], [p_promo_sk#18] - -(24) CometColumnarToRow [codegen id : 4] -Input [1]: [p_promo_sk#18] - -(25) BroadcastExchange -Input [1]: [p_promo_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_promo_sk#3] -Right keys [1]: [p_promo_sk#18] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 5] -Output [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] -Input [7]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17, p_promo_sk#18] - -(28) HashAggregate [codegen id : 5] -Input [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] -Keys [1]: [i_item_id#17] -Functions [4]: [partial_avg(cs_quantity#4), partial_avg(UnscaledValue(cs_list_price#5)), partial_avg(UnscaledValue(cs_coupon_amt#7)), partial_avg(UnscaledValue(cs_sales_price#6))] -Aggregate Attributes [8]: [sum#21, count#22, sum#23, count#24, sum#25, count#26, sum#27, count#28] -Results [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] - -(29) CometColumnarExchange -Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] -Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 6] -Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] - -(31) HashAggregate [codegen id : 6] -Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] -Keys [1]: [i_item_id#17] -Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(UnscaledValue(cs_coupon_amt#7)), avg(UnscaledValue(cs_sales_price#6))] -Aggregate Attributes [4]: [avg(cs_quantity#4)#37, avg(UnscaledValue(cs_list_price#5))#38, avg(UnscaledValue(cs_coupon_amt#7))#39, avg(UnscaledValue(cs_sales_price#6))#40] -Results [5]: [i_item_id#17, avg(cs_quantity#4)#37 AS agg1#41, cast((avg(UnscaledValue(cs_list_price#5))#38 / 100.0) as decimal(11,6)) AS agg2#42, cast((avg(UnscaledValue(cs_coupon_amt#7))#39 / 100.0) as decimal(11,6)) AS agg3#43, cast((avg(UnscaledValue(cs_sales_price#6))#40 / 100.0) as decimal(11,6)) AS agg4#44] - -(32) TakeOrderedAndProject -Input [5]: [i_item_id#17, agg1#41, agg2#42, agg3#43, agg4#44] -Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#41, agg2#42, agg3#43, agg4#44] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometNativeScan parquet spark_catalog.default.date_dim (33) - - -(33) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [d_date_sk#14, d_year#45] -Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2000)) AND isnotnull(d_date_sk#14)) - -(35) CometProject -Input [2]: [d_date_sk#14, d_year#45] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(37) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/simplified.txt deleted file mode 100644 index 7eb9c67611..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) - HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometNativeScan 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/q26.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/explain.txt deleted file mode 100644 index 06138ff307..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,212 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (31) -+- CometTakeOrderedAndProject (30) - +- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Condition : ((isnotnull(cs_bill_cdemo_sk#1) AND isnotnull(cs_item_sk#2)) AND isnotnull(cs_promo_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#11, 1)) = M)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#12, 1)) = S)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#13, 20)) = College )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [cs_bill_cdemo_sk#1], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, cd_demo_sk#10] -Arguments: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8], [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [cs_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, d_date_sk#14] -Arguments: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7], [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#16, i_item_id#17] -Condition : isnotnull(i_item_sk#16) - -(17) CometProject -Input [2]: [i_item_sk#16, i_item_id#17] -Arguments: [i_item_sk#16, i_item_id#18], [i_item_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#17, 16)) AS i_item_id#18] - -(18) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_id#18] -Arguments: [i_item_sk#16, i_item_id#18] - -(19) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] -Right output [2]: [i_item_sk#16, i_item_id#18] -Arguments: [cs_item_sk#2], [i_item_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_sk#16, i_item_id#18] -Arguments: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18], [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Condition : (((static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_email#20, 1)) = N) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_event#21, 1)) = N)) AND isnotnull(p_promo_sk#19)) - -(23) CometProject -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Arguments: [p_promo_sk#19], [p_promo_sk#19] - -(24) CometBroadcastExchange -Input [1]: [p_promo_sk#19] -Arguments: [p_promo_sk#19] - -(25) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] -Right output [1]: [p_promo_sk#19] -Arguments: [cs_promo_sk#3], [p_promo_sk#19], Inner, BuildRight - -(26) CometProject -Input [7]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18, p_promo_sk#19] -Arguments: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] - -(27) CometHashAggregate -Input [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [4]: [partial_avg(cs_quantity#4), partial_avg(UnscaledValue(cs_list_price#5)), partial_avg(UnscaledValue(cs_coupon_amt#7)), partial_avg(UnscaledValue(cs_sales_price#6))] - -(28) CometExchange -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Keys [1]: [i_item_id#18] -Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(UnscaledValue(cs_coupon_amt#7)), avg(UnscaledValue(cs_sales_price#6))] - -(30) CometTakeOrderedAndProject -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#18 ASC NULLS FIRST], output=[i_item_id#18,agg1#30,agg2#31,agg3#32,agg4#33]), [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33], 100, 0, [i_item_id#18 ASC NULLS FIRST], [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(34) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(36) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/simplified.txt deleted file mode 100644 index 0b56a47547..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,agg1,agg2,agg3,agg4,avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price))] - CometExchange [i_item_id] #1 - CometHashAggregate [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [i_item_id,sum,count,sum,count,sum,count,sum,count] - CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - 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,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_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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [p_promo_sk] #6 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometScan [native_iceberg_compat] 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/q26/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/explain.txt deleted file mode 100644 index 06138ff307..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/explain.txt +++ /dev/null @@ -1,212 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (31) -+- CometTakeOrderedAndProject (30) - +- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Condition : ((isnotnull(cs_bill_cdemo_sk#1) AND isnotnull(cs_item_sk#2)) AND isnotnull(cs_promo_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#11, 1)) = M)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#12, 1)) = S)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#13, 20)) = College )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [cs_bill_cdemo_sk#1], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, cd_demo_sk#10] -Arguments: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8], [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [cs_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, d_date_sk#14] -Arguments: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7], [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#16, i_item_id#17] -Condition : isnotnull(i_item_sk#16) - -(17) CometProject -Input [2]: [i_item_sk#16, i_item_id#17] -Arguments: [i_item_sk#16, i_item_id#18], [i_item_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#17, 16)) AS i_item_id#18] - -(18) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_id#18] -Arguments: [i_item_sk#16, i_item_id#18] - -(19) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] -Right output [2]: [i_item_sk#16, i_item_id#18] -Arguments: [cs_item_sk#2], [i_item_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_sk#16, i_item_id#18] -Arguments: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18], [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Condition : (((static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_email#20, 1)) = N) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_event#21, 1)) = N)) AND isnotnull(p_promo_sk#19)) - -(23) CometProject -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Arguments: [p_promo_sk#19], [p_promo_sk#19] - -(24) CometBroadcastExchange -Input [1]: [p_promo_sk#19] -Arguments: [p_promo_sk#19] - -(25) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] -Right output [1]: [p_promo_sk#19] -Arguments: [cs_promo_sk#3], [p_promo_sk#19], Inner, BuildRight - -(26) CometProject -Input [7]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18, p_promo_sk#19] -Arguments: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] - -(27) CometHashAggregate -Input [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [4]: [partial_avg(cs_quantity#4), partial_avg(UnscaledValue(cs_list_price#5)), partial_avg(UnscaledValue(cs_coupon_amt#7)), partial_avg(UnscaledValue(cs_sales_price#6))] - -(28) CometExchange -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Keys [1]: [i_item_id#18] -Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(UnscaledValue(cs_coupon_amt#7)), avg(UnscaledValue(cs_sales_price#6))] - -(30) CometTakeOrderedAndProject -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#18 ASC NULLS FIRST], output=[i_item_id#18,agg1#30,agg2#31,agg3#32,agg4#33]), [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33], 100, 0, [i_item_id#18 ASC NULLS FIRST], [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(34) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(36) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/extended.txt deleted file mode 100644 index dbea7afe34..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/extended.txt +++ /dev/null @@ -1,39 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 0b56a47547..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,agg1,agg2,agg3,agg4,avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price))] - CometExchange [i_item_id] #1 - CometHashAggregate [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [i_item_id,sum,count,sum,count,sum,count,sum,count] - CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - 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,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_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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [p_promo_sk] #6 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/explain.txt deleted file mode 100644 index f3815e677f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (33) -+- * HashAggregate (32) - +- * CometColumnarToRow (31) - +- CometColumnarExchange (30) - +- * HashAggregate (29) - +- * Expand (28) - +- * Project (27) - +- * BroadcastHashJoin Inner BuildRight (26) - :- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.store (14) - +- BroadcastExchange (25) - +- * CometColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometNativeScan parquet spark_catalog.default.item (21) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(3) Filter [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) - -(4) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#11, 1)) = M)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#12, 1)) = S)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#13, 20)) = College )) AND isnotnull(cd_demo_sk#10)) - -(6) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [cd_demo_sk#10] - -(8) BroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 5] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] - -(11) ReusedExchange [Reuses operator id: 38] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#15, s_state#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#15, s_state#16] -Condition : ((isnotnull(s_state#16) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#16, 2)) = TN)) AND isnotnull(s_store_sk#15)) - -(16) CometProject -Input [2]: [s_store_sk#15, s_state#16] -Arguments: [s_store_sk#15, s_state#17], [s_store_sk#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#16, 2)) AS s_state#17] - -(17) CometColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#15, s_state#17] - -(18) BroadcastExchange -Input [2]: [s_store_sk#15, s_state#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15, s_state#17] - -(21) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#18, i_item_id#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_item_sk#18, i_item_id#19] -Condition : isnotnull(i_item_sk#18) - -(23) CometProject -Input [2]: [i_item_sk#18, i_item_id#19] -Arguments: [i_item_sk#18, i_item_id#20], [i_item_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#19, 16)) AS i_item_id#20] - -(24) CometColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#18, i_item_id#20] - -(25) BroadcastExchange -Input [2]: [i_item_sk#18, i_item_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#18] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 5] -Output [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#17] -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17, i_item_sk#18, i_item_id#20] - -(28) Expand [codegen id : 5] -Input [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#17] -Arguments: [[ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#17, 0], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, null, 1], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, null, null, 3]], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#22, spark_grouping_id#23] - -(29) HashAggregate [codegen id : 5] -Input [7]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#22, spark_grouping_id#23] -Keys [3]: [i_item_id#21, s_state#22, spark_grouping_id#23] -Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [8]: [sum#24, count#25, sum#26, count#27, sum#28, count#29, sum#30, count#31] -Results [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] - -(30) CometColumnarExchange -Input [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] -Arguments: hashpartitioning(i_item_id#21, s_state#22, spark_grouping_id#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(31) CometColumnarToRow [codegen id : 6] -Input [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] - -(32) HashAggregate [codegen id : 6] -Input [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] -Keys [3]: [i_item_id#21, s_state#22, spark_grouping_id#23] -Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [4]: [avg(ss_quantity#4)#40, avg(UnscaledValue(ss_list_price#5))#41, avg(UnscaledValue(ss_coupon_amt#7))#42, avg(UnscaledValue(ss_sales_price#6))#43] -Results [7]: [i_item_id#21, s_state#22, cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint) AS g_state#44, avg(ss_quantity#4)#40 AS agg1#45, cast((avg(UnscaledValue(ss_list_price#5))#41 / 100.0) as decimal(11,6)) AS agg2#46, cast((avg(UnscaledValue(ss_coupon_amt#7))#42 / 100.0) as decimal(11,6)) AS agg3#47, cast((avg(UnscaledValue(ss_sales_price#6))#43 / 100.0) as decimal(11,6)) AS agg4#48] - -(33) TakeOrderedAndProject -Input [7]: [i_item_id#21, s_state#22, g_state#44, agg1#45, agg2#46, agg3#47, agg4#48] -Arguments: 100, [i_item_id#21 ASC NULLS FIRST, s_state#22 ASC NULLS FIRST], [i_item_id#21, s_state#22, g_state#44, agg1#45, agg2#46, agg3#47, agg4#48] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometNativeScan parquet spark_catalog.default.date_dim (34) - - -(34) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [d_date_sk#14, d_year#49] -Condition : ((isnotnull(d_year#49) AND (d_year#49 = 2002)) AND isnotnull(d_date_sk#14)) - -(36) CometProject -Input [2]: [d_date_sk#14, d_year#49] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(37) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(38) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/simplified.txt deleted file mode 100644 index dcb4953707..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/simplified.txt +++ /dev/null @@ -1,55 +0,0 @@ -TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) - HashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,s_state,spark_grouping_id] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Expand [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Filter [ss_cdemo_sk,ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan 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/q27.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/explain.txt deleted file mode 100644 index 8dc9e12f34..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (32) -+- CometTakeOrderedAndProject (31) - +- CometHashAggregate (30) - +- CometExchange (29) - +- CometHashAggregate (28) - +- CometExpand (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#11, 1)) = M)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#12, 1)) = S)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#13, 20)) = College )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#16, s_state#17] -Condition : ((isnotnull(s_state#17) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#17, 2)) = TN)) AND isnotnull(s_store_sk#16)) - -(17) CometProject -Input [2]: [s_store_sk#16, s_state#17] -Arguments: [s_store_sk#16, s_state#18], [s_store_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#17, 2)) AS s_state#18] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_state#18] -Arguments: [s_store_sk#16, s_state#18] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Right output [2]: [s_store_sk#16, s_state#18] -Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16, s_state#18] -Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_item_id#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_item_sk#19, i_item_id#20] -Condition : isnotnull(i_item_sk#19) - -(23) CometProject -Input [2]: [i_item_sk#19, i_item_id#20] -Arguments: [i_item_sk#19, i_item_id#21], [i_item_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#20, 16)) AS i_item_id#21] - -(24) CometBroadcastExchange -Input [2]: [i_item_sk#19, i_item_id#21] -Arguments: [i_item_sk#19, i_item_id#21] - -(25) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] -Right output [2]: [i_item_sk#19, i_item_id#21] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(26) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18, i_item_sk#19, i_item_id#21] -Arguments: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18] - -(27) CometExpand -Input [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18] -Arguments: [[ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18, 0], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, null, 1], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, null, null, 3]], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#22, s_state#23, spark_grouping_id#24] - -(28) CometHashAggregate -Input [7]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#22, s_state#23, spark_grouping_id#24] -Keys [3]: [i_item_id#22, s_state#23, spark_grouping_id#24] -Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] - -(29) CometExchange -Input [11]: [i_item_id#22, s_state#23, spark_grouping_id#24, sum#25, count#26, sum#27, count#28, sum#29, count#30, sum#31, count#32] -Arguments: hashpartitioning(i_item_id#22, s_state#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(30) CometHashAggregate -Input [11]: [i_item_id#22, s_state#23, spark_grouping_id#24, sum#25, count#26, sum#27, count#28, sum#29, count#30, sum#31, count#32] -Keys [3]: [i_item_id#22, s_state#23, spark_grouping_id#24] -Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] - -(31) CometTakeOrderedAndProject -Input [7]: [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#22 ASC NULLS FIRST,s_state#23 ASC NULLS FIRST], output=[i_item_id#22,s_state#23,g_state#33,agg1#34,agg2#35,agg3#36,agg4#37]), [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37], 100, 0, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37] - -(32) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (33) - - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_sk#14)) - -(35) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(37) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/simplified.txt deleted file mode 100644 index bdae0cc477..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,42 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,spark_grouping_id,avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price))] - CometExchange [i_item_id,s_state,spark_grouping_id] #1 - CometHashAggregate [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] - 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,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_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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_state] #5 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] 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/q27/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/explain.txt deleted file mode 100644 index 8dc9e12f34..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (32) -+- CometTakeOrderedAndProject (31) - +- CometHashAggregate (30) - +- CometExchange (29) - +- CometHashAggregate (28) - +- CometExpand (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#11, 1)) = M)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#12, 1)) = S)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#13, 20)) = College )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#16, s_state#17] -Condition : ((isnotnull(s_state#17) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#17, 2)) = TN)) AND isnotnull(s_store_sk#16)) - -(17) CometProject -Input [2]: [s_store_sk#16, s_state#17] -Arguments: [s_store_sk#16, s_state#18], [s_store_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#17, 2)) AS s_state#18] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_state#18] -Arguments: [s_store_sk#16, s_state#18] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Right output [2]: [s_store_sk#16, s_state#18] -Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16, s_state#18] -Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_item_id#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_item_sk#19, i_item_id#20] -Condition : isnotnull(i_item_sk#19) - -(23) CometProject -Input [2]: [i_item_sk#19, i_item_id#20] -Arguments: [i_item_sk#19, i_item_id#21], [i_item_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#20, 16)) AS i_item_id#21] - -(24) CometBroadcastExchange -Input [2]: [i_item_sk#19, i_item_id#21] -Arguments: [i_item_sk#19, i_item_id#21] - -(25) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] -Right output [2]: [i_item_sk#19, i_item_id#21] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(26) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18, i_item_sk#19, i_item_id#21] -Arguments: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18] - -(27) CometExpand -Input [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18] -Arguments: [[ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18, 0], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, null, 1], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, null, null, 3]], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#22, s_state#23, spark_grouping_id#24] - -(28) CometHashAggregate -Input [7]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#22, s_state#23, spark_grouping_id#24] -Keys [3]: [i_item_id#22, s_state#23, spark_grouping_id#24] -Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] - -(29) CometExchange -Input [11]: [i_item_id#22, s_state#23, spark_grouping_id#24, sum#25, count#26, sum#27, count#28, sum#29, count#30, sum#31, count#32] -Arguments: hashpartitioning(i_item_id#22, s_state#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(30) CometHashAggregate -Input [11]: [i_item_id#22, s_state#23, spark_grouping_id#24, sum#25, count#26, sum#27, count#28, sum#29, count#30, sum#31, count#32] -Keys [3]: [i_item_id#22, s_state#23, spark_grouping_id#24] -Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] - -(31) CometTakeOrderedAndProject -Input [7]: [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#22 ASC NULLS FIRST,s_state#23 ASC NULLS FIRST], output=[i_item_id#22,s_state#23,g_state#33,agg1#34,agg2#35,agg3#36,agg4#37]), [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37], 100, 0, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37] - -(32) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (33) - - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_sk#14)) - -(35) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(37) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/extended.txt deleted file mode 100644 index f550f3855c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/extended.txt +++ /dev/null @@ -1,40 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index bdae0cc477..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt +++ /dev/null @@ -1,42 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,spark_grouping_id,avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price))] - CometExchange [i_item_id,s_state,spark_grouping_id] #1 - CometHashAggregate [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] - 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,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_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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_state] #5 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_datafusion/explain.txt deleted file mode 100644 index 985274408b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_datafusion/explain.txt +++ /dev/null @@ -1,437 +0,0 @@ -== Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (76) -:- * BroadcastNestedLoopJoin Inner BuildRight (63) -: :- * BroadcastNestedLoopJoin Inner BuildRight (50) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (37) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (24) -: : : : :- * CometColumnarToRow (11) -: : : : : +- CometHashAggregate (10) -: : : : : +- CometColumnarExchange (9) -: : : : : +- * HashAggregate (8) -: : : : : +- * HashAggregate (7) -: : : : : +- * CometColumnarToRow (6) -: : : : : +- CometExchange (5) -: : : : : +- CometHashAggregate (4) -: : : : : +- CometProject (3) -: : : : : +- CometFilter (2) -: : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) -: : : : +- BroadcastExchange (23) -: : : : +- * CometColumnarToRow (22) -: : : : +- CometHashAggregate (21) -: : : : +- CometColumnarExchange (20) -: : : : +- * HashAggregate (19) -: : : : +- * HashAggregate (18) -: : : : +- * CometColumnarToRow (17) -: : : : +- CometExchange (16) -: : : : +- CometHashAggregate (15) -: : : : +- CometProject (14) -: : : : +- CometFilter (13) -: : : : +- CometNativeScan parquet spark_catalog.default.store_sales (12) -: : : +- BroadcastExchange (36) -: : : +- * CometColumnarToRow (35) -: : : +- CometHashAggregate (34) -: : : +- CometColumnarExchange (33) -: : : +- * HashAggregate (32) -: : : +- * HashAggregate (31) -: : : +- * CometColumnarToRow (30) -: : : +- CometExchange (29) -: : : +- CometHashAggregate (28) -: : : +- CometProject (27) -: : : +- CometFilter (26) -: : : +- CometNativeScan parquet spark_catalog.default.store_sales (25) -: : +- BroadcastExchange (49) -: : +- * CometColumnarToRow (48) -: : +- CometHashAggregate (47) -: : +- CometColumnarExchange (46) -: : +- * HashAggregate (45) -: : +- * HashAggregate (44) -: : +- * CometColumnarToRow (43) -: : +- CometExchange (42) -: : +- CometHashAggregate (41) -: : +- CometProject (40) -: : +- CometFilter (39) -: : +- CometNativeScan parquet spark_catalog.default.store_sales (38) -: +- BroadcastExchange (62) -: +- * CometColumnarToRow (61) -: +- CometHashAggregate (60) -: +- CometColumnarExchange (59) -: +- * HashAggregate (58) -: +- * HashAggregate (57) -: +- * CometColumnarToRow (56) -: +- CometExchange (55) -: +- CometHashAggregate (54) -: +- CometProject (53) -: +- CometFilter (52) -: +- CometNativeScan parquet spark_catalog.default.store_sales (51) -+- BroadcastExchange (75) - +- * CometColumnarToRow (74) - +- CometHashAggregate (73) - +- CometColumnarExchange (72) - +- * HashAggregate (71) - +- * HashAggregate (70) - +- * CometColumnarToRow (69) - +- CometExchange (68) - +- CometHashAggregate (67) - +- CometProject (66) - +- CometFilter (65) - +- CometNativeScan parquet spark_catalog.default.store_sales (64) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,0), LessThanOrEqual(ss_quantity,5), Or(Or(And(GreaterThanOrEqual(ss_list_price,8.00),LessThanOrEqual(ss_list_price,18.00)),And(GreaterThanOrEqual(ss_coupon_amt,459.00),LessThanOrEqual(ss_coupon_amt,1459.00))),And(GreaterThanOrEqual(ss_wholesale_cost,57.00),LessThanOrEqual(ss_wholesale_cost,77.00)))] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 0)) AND (ss_quantity#1 <= 5)) AND ((((ss_list_price#3 >= 8.00) AND (ss_list_price#3 <= 18.00)) OR ((ss_coupon_amt#4 >= 459.00) AND (ss_coupon_amt#4 <= 1459.00))) OR ((ss_wholesale_cost#2 >= 57.00) AND (ss_wholesale_cost#2 <= 77.00)))) - -(3) CometProject -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Arguments: [ss_list_price#3], [ss_list_price#3] - -(4) CometHashAggregate -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] - -(5) CometExchange -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(6) CometColumnarToRow [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] - -(7) HashAggregate [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10] -Results [4]: [ss_list_price#3, sum#6, count#7, count#8] - -(8) HashAggregate [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10, count(ss_list_price#3)#11] -Results [4]: [sum#6, count#7, count#8, count#12] - -(9) CometColumnarExchange -Input [4]: [sum#6, count#7, count#8, count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(10) CometHashAggregate -Input [4]: [sum#6, count#7, count#8, count#12] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] - -(11) CometColumnarToRow [codegen id : 12] -Input [3]: [B1_LP#13, B1_CNT#14, B1_CNTD#15] - -(12) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] -ReadSchema: struct - -(13) CometFilter -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Condition : (((isnotnull(ss_quantity#16) AND (ss_quantity#16 >= 6)) AND (ss_quantity#16 <= 10)) AND ((((ss_list_price#18 >= 90.00) AND (ss_list_price#18 <= 100.00)) OR ((ss_coupon_amt#19 >= 2323.00) AND (ss_coupon_amt#19 <= 3323.00))) OR ((ss_wholesale_cost#17 >= 31.00) AND (ss_wholesale_cost#17 <= 51.00)))) - -(14) CometProject -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Arguments: [ss_list_price#18], [ss_list_price#18] - -(15) CometHashAggregate -Input [1]: [ss_list_price#18] -Keys [1]: [ss_list_price#18] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_list_price#18)] - -(16) CometExchange -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(17) CometColumnarToRow [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] - -(18) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Keys [1]: [ss_list_price#18] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25] -Results [4]: [ss_list_price#18, sum#21, count#22, count#23] - -(19) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18), partial_count(distinct ss_list_price#18)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25, count(ss_list_price#18)#26] -Results [4]: [sum#21, count#22, count#23, count#27] - -(20) CometColumnarExchange -Input [4]: [sum#21, count#22, count#23, count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(21) CometHashAggregate -Input [4]: [sum#21, count#22, count#23, count#27] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#18)), count(ss_list_price#18), count(distinct ss_list_price#18)] - -(22) CometColumnarToRow [codegen id : 3] -Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] - -(23) BroadcastExchange -Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] -Arguments: IdentityBroadcastMode, [plan_id=5] - -(24) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(25) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] -ReadSchema: struct - -(26) CometFilter -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Condition : (((isnotnull(ss_quantity#31) AND (ss_quantity#31 >= 11)) AND (ss_quantity#31 <= 15)) AND ((((ss_list_price#33 >= 142.00) AND (ss_list_price#33 <= 152.00)) OR ((ss_coupon_amt#34 >= 12214.00) AND (ss_coupon_amt#34 <= 13214.00))) OR ((ss_wholesale_cost#32 >= 79.00) AND (ss_wholesale_cost#32 <= 99.00)))) - -(27) CometProject -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Arguments: [ss_list_price#33], [ss_list_price#33] - -(28) CometHashAggregate -Input [1]: [ss_list_price#33] -Keys [1]: [ss_list_price#33] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_list_price#33)] - -(29) CometExchange -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(30) CometColumnarToRow [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] - -(31) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Keys [1]: [ss_list_price#33] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40] -Results [4]: [ss_list_price#33, sum#36, count#37, count#38] - -(32) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33), partial_count(distinct ss_list_price#33)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40, count(ss_list_price#33)#41] -Results [4]: [sum#36, count#37, count#38, count#42] - -(33) CometColumnarExchange -Input [4]: [sum#36, count#37, count#38, count#42] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(34) CometHashAggregate -Input [4]: [sum#36, count#37, count#38, count#42] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#33)), count(ss_list_price#33), count(distinct ss_list_price#33)] - -(35) CometColumnarToRow [codegen id : 5] -Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] - -(36) BroadcastExchange -Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] -Arguments: IdentityBroadcastMode, [plan_id=8] - -(37) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(38) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] -ReadSchema: struct - -(39) CometFilter -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Condition : (((isnotnull(ss_quantity#46) AND (ss_quantity#46 >= 16)) AND (ss_quantity#46 <= 20)) AND ((((ss_list_price#48 >= 135.00) AND (ss_list_price#48 <= 145.00)) OR ((ss_coupon_amt#49 >= 6071.00) AND (ss_coupon_amt#49 <= 7071.00))) OR ((ss_wholesale_cost#47 >= 38.00) AND (ss_wholesale_cost#47 <= 58.00)))) - -(40) CometProject -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Arguments: [ss_list_price#48], [ss_list_price#48] - -(41) CometHashAggregate -Input [1]: [ss_list_price#48] -Keys [1]: [ss_list_price#48] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_list_price#48)] - -(42) CometExchange -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(43) CometColumnarToRow [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] - -(44) HashAggregate [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Keys [1]: [ss_list_price#48] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55] -Results [4]: [ss_list_price#48, sum#51, count#52, count#53] - -(45) HashAggregate [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48), partial_count(distinct ss_list_price#48)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55, count(ss_list_price#48)#56] -Results [4]: [sum#51, count#52, count#53, count#57] - -(46) CometColumnarExchange -Input [4]: [sum#51, count#52, count#53, count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(47) CometHashAggregate -Input [4]: [sum#51, count#52, count#53, count#57] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#48)), count(ss_list_price#48), count(distinct ss_list_price#48)] - -(48) CometColumnarToRow [codegen id : 7] -Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] - -(49) BroadcastExchange -Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] -Arguments: IdentityBroadcastMode, [plan_id=11] - -(50) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(51) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] -ReadSchema: struct - -(52) CometFilter -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Condition : (((isnotnull(ss_quantity#61) AND (ss_quantity#61 >= 21)) AND (ss_quantity#61 <= 25)) AND ((((ss_list_price#63 >= 122.00) AND (ss_list_price#63 <= 132.00)) OR ((ss_coupon_amt#64 >= 836.00) AND (ss_coupon_amt#64 <= 1836.00))) OR ((ss_wholesale_cost#62 >= 17.00) AND (ss_wholesale_cost#62 <= 37.00)))) - -(53) CometProject -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Arguments: [ss_list_price#63], [ss_list_price#63] - -(54) CometHashAggregate -Input [1]: [ss_list_price#63] -Keys [1]: [ss_list_price#63] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_list_price#63)] - -(55) CometExchange -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(56) CometColumnarToRow [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] - -(57) HashAggregate [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Keys [1]: [ss_list_price#63] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70] -Results [4]: [ss_list_price#63, sum#66, count#67, count#68] - -(58) HashAggregate [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63), partial_count(distinct ss_list_price#63)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70, count(ss_list_price#63)#71] -Results [4]: [sum#66, count#67, count#68, count#72] - -(59) CometColumnarExchange -Input [4]: [sum#66, count#67, count#68, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(60) CometHashAggregate -Input [4]: [sum#66, count#67, count#68, count#72] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#63)), count(ss_list_price#63), count(distinct ss_list_price#63)] - -(61) CometColumnarToRow [codegen id : 9] -Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] - -(62) BroadcastExchange -Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] -Arguments: IdentityBroadcastMode, [plan_id=14] - -(63) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(64) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] -ReadSchema: struct - -(65) CometFilter -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Condition : (((isnotnull(ss_quantity#76) AND (ss_quantity#76 >= 26)) AND (ss_quantity#76 <= 30)) AND ((((ss_list_price#78 >= 154.00) AND (ss_list_price#78 <= 164.00)) OR ((ss_coupon_amt#79 >= 7326.00) AND (ss_coupon_amt#79 <= 8326.00))) OR ((ss_wholesale_cost#77 >= 7.00) AND (ss_wholesale_cost#77 <= 27.00)))) - -(66) CometProject -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Arguments: [ss_list_price#78], [ss_list_price#78] - -(67) CometHashAggregate -Input [1]: [ss_list_price#78] -Keys [1]: [ss_list_price#78] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_list_price#78)] - -(68) CometExchange -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(69) CometColumnarToRow [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] - -(70) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Keys [1]: [ss_list_price#78] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85] -Results [4]: [ss_list_price#78, sum#81, count#82, count#83] - -(71) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78), partial_count(distinct ss_list_price#78)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85, count(ss_list_price#78)#86] -Results [4]: [sum#81, count#82, count#83, count#87] - -(72) CometColumnarExchange -Input [4]: [sum#81, count#82, count#83, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(73) CometHashAggregate -Input [4]: [sum#81, count#82, count#83, count#87] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#78)), count(ss_list_price#78), count(distinct ss_list_price#78)] - -(74) CometColumnarToRow [codegen id : 11] -Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] - -(75) BroadcastExchange -Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] -Arguments: IdentityBroadcastMode, [plan_id=17] - -(76) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_datafusion/simplified.txt deleted file mode 100644 index a8540a4abb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_datafusion/simplified.txt +++ /dev/null @@ -1,105 +0,0 @@ -WholeStageCodegen (12) - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B1_LP,B1_CNT,B1_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #2 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B2_LP,B2_CNT,B2_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #4 - WholeStageCodegen (2) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #5 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B3_LP,B3_CNT,B3_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #7 - WholeStageCodegen (4) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #8 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B4_LP,B4_CNT,B4_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #10 - WholeStageCodegen (6) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #11 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B5_LP,B5_CNT,B5_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #13 - WholeStageCodegen (8) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #14 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B6_LP,B6_CNT,B6_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #16 - WholeStageCodegen (10) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #17 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan 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/q28.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_iceberg_compat/explain.txt deleted file mode 100644 index 98ac957fc7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,437 +0,0 @@ -== Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (76) -:- * BroadcastNestedLoopJoin Inner BuildRight (63) -: :- * BroadcastNestedLoopJoin Inner BuildRight (50) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (37) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (24) -: : : : :- * CometColumnarToRow (11) -: : : : : +- CometHashAggregate (10) -: : : : : +- CometColumnarExchange (9) -: : : : : +- * HashAggregate (8) -: : : : : +- * HashAggregate (7) -: : : : : +- * CometColumnarToRow (6) -: : : : : +- CometExchange (5) -: : : : : +- CometHashAggregate (4) -: : : : : +- CometProject (3) -: : : : : +- CometFilter (2) -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) -: : : : +- BroadcastExchange (23) -: : : : +- * CometColumnarToRow (22) -: : : : +- CometHashAggregate (21) -: : : : +- CometColumnarExchange (20) -: : : : +- * HashAggregate (19) -: : : : +- * HashAggregate (18) -: : : : +- * CometColumnarToRow (17) -: : : : +- CometExchange (16) -: : : : +- CometHashAggregate (15) -: : : : +- CometProject (14) -: : : : +- CometFilter (13) -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (12) -: : : +- BroadcastExchange (36) -: : : +- * CometColumnarToRow (35) -: : : +- CometHashAggregate (34) -: : : +- CometColumnarExchange (33) -: : : +- * HashAggregate (32) -: : : +- * HashAggregate (31) -: : : +- * CometColumnarToRow (30) -: : : +- CometExchange (29) -: : : +- CometHashAggregate (28) -: : : +- CometProject (27) -: : : +- CometFilter (26) -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (25) -: : +- BroadcastExchange (49) -: : +- * CometColumnarToRow (48) -: : +- CometHashAggregate (47) -: : +- CometColumnarExchange (46) -: : +- * HashAggregate (45) -: : +- * HashAggregate (44) -: : +- * CometColumnarToRow (43) -: : +- CometExchange (42) -: : +- CometHashAggregate (41) -: : +- CometProject (40) -: : +- CometFilter (39) -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (38) -: +- BroadcastExchange (62) -: +- * CometColumnarToRow (61) -: +- CometHashAggregate (60) -: +- CometColumnarExchange (59) -: +- * HashAggregate (58) -: +- * HashAggregate (57) -: +- * CometColumnarToRow (56) -: +- CometExchange (55) -: +- CometHashAggregate (54) -: +- CometProject (53) -: +- CometFilter (52) -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (51) -+- BroadcastExchange (75) - +- * CometColumnarToRow (74) - +- CometHashAggregate (73) - +- CometColumnarExchange (72) - +- * HashAggregate (71) - +- * HashAggregate (70) - +- * CometColumnarToRow (69) - +- CometExchange (68) - +- CometHashAggregate (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (64) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,0), LessThanOrEqual(ss_quantity,5), Or(Or(And(GreaterThanOrEqual(ss_list_price,8.00),LessThanOrEqual(ss_list_price,18.00)),And(GreaterThanOrEqual(ss_coupon_amt,459.00),LessThanOrEqual(ss_coupon_amt,1459.00))),And(GreaterThanOrEqual(ss_wholesale_cost,57.00),LessThanOrEqual(ss_wholesale_cost,77.00)))] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 0)) AND (ss_quantity#1 <= 5)) AND ((((ss_list_price#3 >= 8.00) AND (ss_list_price#3 <= 18.00)) OR ((ss_coupon_amt#4 >= 459.00) AND (ss_coupon_amt#4 <= 1459.00))) OR ((ss_wholesale_cost#2 >= 57.00) AND (ss_wholesale_cost#2 <= 77.00)))) - -(3) CometProject -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Arguments: [ss_list_price#3], [ss_list_price#3] - -(4) CometHashAggregate -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] - -(5) CometExchange -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(6) CometColumnarToRow [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] - -(7) HashAggregate [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10] -Results [4]: [ss_list_price#3, sum#6, count#7, count#8] - -(8) HashAggregate [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10, count(ss_list_price#3)#11] -Results [4]: [sum#6, count#7, count#8, count#12] - -(9) CometColumnarExchange -Input [4]: [sum#6, count#7, count#8, count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(10) CometHashAggregate -Input [4]: [sum#6, count#7, count#8, count#12] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] - -(11) CometColumnarToRow [codegen id : 12] -Input [3]: [B1_LP#13, B1_CNT#14, B1_CNTD#15] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] -ReadSchema: struct - -(13) CometFilter -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Condition : (((isnotnull(ss_quantity#16) AND (ss_quantity#16 >= 6)) AND (ss_quantity#16 <= 10)) AND ((((ss_list_price#18 >= 90.00) AND (ss_list_price#18 <= 100.00)) OR ((ss_coupon_amt#19 >= 2323.00) AND (ss_coupon_amt#19 <= 3323.00))) OR ((ss_wholesale_cost#17 >= 31.00) AND (ss_wholesale_cost#17 <= 51.00)))) - -(14) CometProject -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Arguments: [ss_list_price#18], [ss_list_price#18] - -(15) CometHashAggregate -Input [1]: [ss_list_price#18] -Keys [1]: [ss_list_price#18] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_list_price#18)] - -(16) CometExchange -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(17) CometColumnarToRow [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] - -(18) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Keys [1]: [ss_list_price#18] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25] -Results [4]: [ss_list_price#18, sum#21, count#22, count#23] - -(19) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18), partial_count(distinct ss_list_price#18)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25, count(ss_list_price#18)#26] -Results [4]: [sum#21, count#22, count#23, count#27] - -(20) CometColumnarExchange -Input [4]: [sum#21, count#22, count#23, count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(21) CometHashAggregate -Input [4]: [sum#21, count#22, count#23, count#27] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#18)), count(ss_list_price#18), count(distinct ss_list_price#18)] - -(22) CometColumnarToRow [codegen id : 3] -Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] - -(23) BroadcastExchange -Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] -Arguments: IdentityBroadcastMode, [plan_id=5] - -(24) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] -ReadSchema: struct - -(26) CometFilter -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Condition : (((isnotnull(ss_quantity#31) AND (ss_quantity#31 >= 11)) AND (ss_quantity#31 <= 15)) AND ((((ss_list_price#33 >= 142.00) AND (ss_list_price#33 <= 152.00)) OR ((ss_coupon_amt#34 >= 12214.00) AND (ss_coupon_amt#34 <= 13214.00))) OR ((ss_wholesale_cost#32 >= 79.00) AND (ss_wholesale_cost#32 <= 99.00)))) - -(27) CometProject -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Arguments: [ss_list_price#33], [ss_list_price#33] - -(28) CometHashAggregate -Input [1]: [ss_list_price#33] -Keys [1]: [ss_list_price#33] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_list_price#33)] - -(29) CometExchange -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(30) CometColumnarToRow [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] - -(31) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Keys [1]: [ss_list_price#33] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40] -Results [4]: [ss_list_price#33, sum#36, count#37, count#38] - -(32) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33), partial_count(distinct ss_list_price#33)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40, count(ss_list_price#33)#41] -Results [4]: [sum#36, count#37, count#38, count#42] - -(33) CometColumnarExchange -Input [4]: [sum#36, count#37, count#38, count#42] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(34) CometHashAggregate -Input [4]: [sum#36, count#37, count#38, count#42] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#33)), count(ss_list_price#33), count(distinct ss_list_price#33)] - -(35) CometColumnarToRow [codegen id : 5] -Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] - -(36) BroadcastExchange -Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] -Arguments: IdentityBroadcastMode, [plan_id=8] - -(37) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] -ReadSchema: struct - -(39) CometFilter -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Condition : (((isnotnull(ss_quantity#46) AND (ss_quantity#46 >= 16)) AND (ss_quantity#46 <= 20)) AND ((((ss_list_price#48 >= 135.00) AND (ss_list_price#48 <= 145.00)) OR ((ss_coupon_amt#49 >= 6071.00) AND (ss_coupon_amt#49 <= 7071.00))) OR ((ss_wholesale_cost#47 >= 38.00) AND (ss_wholesale_cost#47 <= 58.00)))) - -(40) CometProject -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Arguments: [ss_list_price#48], [ss_list_price#48] - -(41) CometHashAggregate -Input [1]: [ss_list_price#48] -Keys [1]: [ss_list_price#48] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_list_price#48)] - -(42) CometExchange -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(43) CometColumnarToRow [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] - -(44) HashAggregate [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Keys [1]: [ss_list_price#48] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55] -Results [4]: [ss_list_price#48, sum#51, count#52, count#53] - -(45) HashAggregate [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48), partial_count(distinct ss_list_price#48)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55, count(ss_list_price#48)#56] -Results [4]: [sum#51, count#52, count#53, count#57] - -(46) CometColumnarExchange -Input [4]: [sum#51, count#52, count#53, count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(47) CometHashAggregate -Input [4]: [sum#51, count#52, count#53, count#57] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#48)), count(ss_list_price#48), count(distinct ss_list_price#48)] - -(48) CometColumnarToRow [codegen id : 7] -Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] - -(49) BroadcastExchange -Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] -Arguments: IdentityBroadcastMode, [plan_id=11] - -(50) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] -ReadSchema: struct - -(52) CometFilter -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Condition : (((isnotnull(ss_quantity#61) AND (ss_quantity#61 >= 21)) AND (ss_quantity#61 <= 25)) AND ((((ss_list_price#63 >= 122.00) AND (ss_list_price#63 <= 132.00)) OR ((ss_coupon_amt#64 >= 836.00) AND (ss_coupon_amt#64 <= 1836.00))) OR ((ss_wholesale_cost#62 >= 17.00) AND (ss_wholesale_cost#62 <= 37.00)))) - -(53) CometProject -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Arguments: [ss_list_price#63], [ss_list_price#63] - -(54) CometHashAggregate -Input [1]: [ss_list_price#63] -Keys [1]: [ss_list_price#63] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_list_price#63)] - -(55) CometExchange -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(56) CometColumnarToRow [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] - -(57) HashAggregate [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Keys [1]: [ss_list_price#63] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70] -Results [4]: [ss_list_price#63, sum#66, count#67, count#68] - -(58) HashAggregate [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63), partial_count(distinct ss_list_price#63)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70, count(ss_list_price#63)#71] -Results [4]: [sum#66, count#67, count#68, count#72] - -(59) CometColumnarExchange -Input [4]: [sum#66, count#67, count#68, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(60) CometHashAggregate -Input [4]: [sum#66, count#67, count#68, count#72] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#63)), count(ss_list_price#63), count(distinct ss_list_price#63)] - -(61) CometColumnarToRow [codegen id : 9] -Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] - -(62) BroadcastExchange -Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] -Arguments: IdentityBroadcastMode, [plan_id=14] - -(63) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] -ReadSchema: struct - -(65) CometFilter -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Condition : (((isnotnull(ss_quantity#76) AND (ss_quantity#76 >= 26)) AND (ss_quantity#76 <= 30)) AND ((((ss_list_price#78 >= 154.00) AND (ss_list_price#78 <= 164.00)) OR ((ss_coupon_amt#79 >= 7326.00) AND (ss_coupon_amt#79 <= 8326.00))) OR ((ss_wholesale_cost#77 >= 7.00) AND (ss_wholesale_cost#77 <= 27.00)))) - -(66) CometProject -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Arguments: [ss_list_price#78], [ss_list_price#78] - -(67) CometHashAggregate -Input [1]: [ss_list_price#78] -Keys [1]: [ss_list_price#78] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_list_price#78)] - -(68) CometExchange -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(69) CometColumnarToRow [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] - -(70) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Keys [1]: [ss_list_price#78] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85] -Results [4]: [ss_list_price#78, sum#81, count#82, count#83] - -(71) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78), partial_count(distinct ss_list_price#78)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85, count(ss_list_price#78)#86] -Results [4]: [sum#81, count#82, count#83, count#87] - -(72) CometColumnarExchange -Input [4]: [sum#81, count#82, count#83, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(73) CometHashAggregate -Input [4]: [sum#81, count#82, count#83, count#87] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#78)), count(ss_list_price#78), count(distinct ss_list_price#78)] - -(74) CometColumnarToRow [codegen id : 11] -Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] - -(75) BroadcastExchange -Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] -Arguments: IdentityBroadcastMode, [plan_id=17] - -(76) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_iceberg_compat/simplified.txt deleted file mode 100644 index ce476affda..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,105 +0,0 @@ -WholeStageCodegen (12) - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B1_LP,B1_CNT,B1_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #2 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B2_LP,B2_CNT,B2_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #4 - WholeStageCodegen (2) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #5 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B3_LP,B3_CNT,B3_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #7 - WholeStageCodegen (4) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #8 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B4_LP,B4_CNT,B4_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #10 - WholeStageCodegen (6) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #11 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B5_LP,B5_CNT,B5_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #13 - WholeStageCodegen (8) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #14 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B6_LP,B6_CNT,B6_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #16 - WholeStageCodegen (10) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #17 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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/q28/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/explain.txt deleted file mode 100644 index 98ac957fc7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/explain.txt +++ /dev/null @@ -1,437 +0,0 @@ -== Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (76) -:- * BroadcastNestedLoopJoin Inner BuildRight (63) -: :- * BroadcastNestedLoopJoin Inner BuildRight (50) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (37) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (24) -: : : : :- * CometColumnarToRow (11) -: : : : : +- CometHashAggregate (10) -: : : : : +- CometColumnarExchange (9) -: : : : : +- * HashAggregate (8) -: : : : : +- * HashAggregate (7) -: : : : : +- * CometColumnarToRow (6) -: : : : : +- CometExchange (5) -: : : : : +- CometHashAggregate (4) -: : : : : +- CometProject (3) -: : : : : +- CometFilter (2) -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) -: : : : +- BroadcastExchange (23) -: : : : +- * CometColumnarToRow (22) -: : : : +- CometHashAggregate (21) -: : : : +- CometColumnarExchange (20) -: : : : +- * HashAggregate (19) -: : : : +- * HashAggregate (18) -: : : : +- * CometColumnarToRow (17) -: : : : +- CometExchange (16) -: : : : +- CometHashAggregate (15) -: : : : +- CometProject (14) -: : : : +- CometFilter (13) -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (12) -: : : +- BroadcastExchange (36) -: : : +- * CometColumnarToRow (35) -: : : +- CometHashAggregate (34) -: : : +- CometColumnarExchange (33) -: : : +- * HashAggregate (32) -: : : +- * HashAggregate (31) -: : : +- * CometColumnarToRow (30) -: : : +- CometExchange (29) -: : : +- CometHashAggregate (28) -: : : +- CometProject (27) -: : : +- CometFilter (26) -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (25) -: : +- BroadcastExchange (49) -: : +- * CometColumnarToRow (48) -: : +- CometHashAggregate (47) -: : +- CometColumnarExchange (46) -: : +- * HashAggregate (45) -: : +- * HashAggregate (44) -: : +- * CometColumnarToRow (43) -: : +- CometExchange (42) -: : +- CometHashAggregate (41) -: : +- CometProject (40) -: : +- CometFilter (39) -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (38) -: +- BroadcastExchange (62) -: +- * CometColumnarToRow (61) -: +- CometHashAggregate (60) -: +- CometColumnarExchange (59) -: +- * HashAggregate (58) -: +- * HashAggregate (57) -: +- * CometColumnarToRow (56) -: +- CometExchange (55) -: +- CometHashAggregate (54) -: +- CometProject (53) -: +- CometFilter (52) -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (51) -+- BroadcastExchange (75) - +- * CometColumnarToRow (74) - +- CometHashAggregate (73) - +- CometColumnarExchange (72) - +- * HashAggregate (71) - +- * HashAggregate (70) - +- * CometColumnarToRow (69) - +- CometExchange (68) - +- CometHashAggregate (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (64) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,0), LessThanOrEqual(ss_quantity,5), Or(Or(And(GreaterThanOrEqual(ss_list_price,8.00),LessThanOrEqual(ss_list_price,18.00)),And(GreaterThanOrEqual(ss_coupon_amt,459.00),LessThanOrEqual(ss_coupon_amt,1459.00))),And(GreaterThanOrEqual(ss_wholesale_cost,57.00),LessThanOrEqual(ss_wholesale_cost,77.00)))] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 0)) AND (ss_quantity#1 <= 5)) AND ((((ss_list_price#3 >= 8.00) AND (ss_list_price#3 <= 18.00)) OR ((ss_coupon_amt#4 >= 459.00) AND (ss_coupon_amt#4 <= 1459.00))) OR ((ss_wholesale_cost#2 >= 57.00) AND (ss_wholesale_cost#2 <= 77.00)))) - -(3) CometProject -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Arguments: [ss_list_price#3], [ss_list_price#3] - -(4) CometHashAggregate -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] - -(5) CometExchange -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(6) CometColumnarToRow [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] - -(7) HashAggregate [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10] -Results [4]: [ss_list_price#3, sum#6, count#7, count#8] - -(8) HashAggregate [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10, count(ss_list_price#3)#11] -Results [4]: [sum#6, count#7, count#8, count#12] - -(9) CometColumnarExchange -Input [4]: [sum#6, count#7, count#8, count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(10) CometHashAggregate -Input [4]: [sum#6, count#7, count#8, count#12] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] - -(11) CometColumnarToRow [codegen id : 12] -Input [3]: [B1_LP#13, B1_CNT#14, B1_CNTD#15] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] -ReadSchema: struct - -(13) CometFilter -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Condition : (((isnotnull(ss_quantity#16) AND (ss_quantity#16 >= 6)) AND (ss_quantity#16 <= 10)) AND ((((ss_list_price#18 >= 90.00) AND (ss_list_price#18 <= 100.00)) OR ((ss_coupon_amt#19 >= 2323.00) AND (ss_coupon_amt#19 <= 3323.00))) OR ((ss_wholesale_cost#17 >= 31.00) AND (ss_wholesale_cost#17 <= 51.00)))) - -(14) CometProject -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Arguments: [ss_list_price#18], [ss_list_price#18] - -(15) CometHashAggregate -Input [1]: [ss_list_price#18] -Keys [1]: [ss_list_price#18] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_list_price#18)] - -(16) CometExchange -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(17) CometColumnarToRow [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] - -(18) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Keys [1]: [ss_list_price#18] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25] -Results [4]: [ss_list_price#18, sum#21, count#22, count#23] - -(19) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18), partial_count(distinct ss_list_price#18)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25, count(ss_list_price#18)#26] -Results [4]: [sum#21, count#22, count#23, count#27] - -(20) CometColumnarExchange -Input [4]: [sum#21, count#22, count#23, count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(21) CometHashAggregate -Input [4]: [sum#21, count#22, count#23, count#27] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#18)), count(ss_list_price#18), count(distinct ss_list_price#18)] - -(22) CometColumnarToRow [codegen id : 3] -Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] - -(23) BroadcastExchange -Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] -Arguments: IdentityBroadcastMode, [plan_id=5] - -(24) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] -ReadSchema: struct - -(26) CometFilter -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Condition : (((isnotnull(ss_quantity#31) AND (ss_quantity#31 >= 11)) AND (ss_quantity#31 <= 15)) AND ((((ss_list_price#33 >= 142.00) AND (ss_list_price#33 <= 152.00)) OR ((ss_coupon_amt#34 >= 12214.00) AND (ss_coupon_amt#34 <= 13214.00))) OR ((ss_wholesale_cost#32 >= 79.00) AND (ss_wholesale_cost#32 <= 99.00)))) - -(27) CometProject -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Arguments: [ss_list_price#33], [ss_list_price#33] - -(28) CometHashAggregate -Input [1]: [ss_list_price#33] -Keys [1]: [ss_list_price#33] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_list_price#33)] - -(29) CometExchange -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(30) CometColumnarToRow [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] - -(31) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Keys [1]: [ss_list_price#33] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40] -Results [4]: [ss_list_price#33, sum#36, count#37, count#38] - -(32) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33), partial_count(distinct ss_list_price#33)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40, count(ss_list_price#33)#41] -Results [4]: [sum#36, count#37, count#38, count#42] - -(33) CometColumnarExchange -Input [4]: [sum#36, count#37, count#38, count#42] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(34) CometHashAggregate -Input [4]: [sum#36, count#37, count#38, count#42] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#33)), count(ss_list_price#33), count(distinct ss_list_price#33)] - -(35) CometColumnarToRow [codegen id : 5] -Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] - -(36) BroadcastExchange -Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] -Arguments: IdentityBroadcastMode, [plan_id=8] - -(37) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] -ReadSchema: struct - -(39) CometFilter -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Condition : (((isnotnull(ss_quantity#46) AND (ss_quantity#46 >= 16)) AND (ss_quantity#46 <= 20)) AND ((((ss_list_price#48 >= 135.00) AND (ss_list_price#48 <= 145.00)) OR ((ss_coupon_amt#49 >= 6071.00) AND (ss_coupon_amt#49 <= 7071.00))) OR ((ss_wholesale_cost#47 >= 38.00) AND (ss_wholesale_cost#47 <= 58.00)))) - -(40) CometProject -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Arguments: [ss_list_price#48], [ss_list_price#48] - -(41) CometHashAggregate -Input [1]: [ss_list_price#48] -Keys [1]: [ss_list_price#48] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_list_price#48)] - -(42) CometExchange -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(43) CometColumnarToRow [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] - -(44) HashAggregate [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Keys [1]: [ss_list_price#48] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55] -Results [4]: [ss_list_price#48, sum#51, count#52, count#53] - -(45) HashAggregate [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48), partial_count(distinct ss_list_price#48)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55, count(ss_list_price#48)#56] -Results [4]: [sum#51, count#52, count#53, count#57] - -(46) CometColumnarExchange -Input [4]: [sum#51, count#52, count#53, count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(47) CometHashAggregate -Input [4]: [sum#51, count#52, count#53, count#57] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#48)), count(ss_list_price#48), count(distinct ss_list_price#48)] - -(48) CometColumnarToRow [codegen id : 7] -Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] - -(49) BroadcastExchange -Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] -Arguments: IdentityBroadcastMode, [plan_id=11] - -(50) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] -ReadSchema: struct - -(52) CometFilter -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Condition : (((isnotnull(ss_quantity#61) AND (ss_quantity#61 >= 21)) AND (ss_quantity#61 <= 25)) AND ((((ss_list_price#63 >= 122.00) AND (ss_list_price#63 <= 132.00)) OR ((ss_coupon_amt#64 >= 836.00) AND (ss_coupon_amt#64 <= 1836.00))) OR ((ss_wholesale_cost#62 >= 17.00) AND (ss_wholesale_cost#62 <= 37.00)))) - -(53) CometProject -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Arguments: [ss_list_price#63], [ss_list_price#63] - -(54) CometHashAggregate -Input [1]: [ss_list_price#63] -Keys [1]: [ss_list_price#63] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_list_price#63)] - -(55) CometExchange -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(56) CometColumnarToRow [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] - -(57) HashAggregate [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Keys [1]: [ss_list_price#63] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70] -Results [4]: [ss_list_price#63, sum#66, count#67, count#68] - -(58) HashAggregate [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63), partial_count(distinct ss_list_price#63)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70, count(ss_list_price#63)#71] -Results [4]: [sum#66, count#67, count#68, count#72] - -(59) CometColumnarExchange -Input [4]: [sum#66, count#67, count#68, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(60) CometHashAggregate -Input [4]: [sum#66, count#67, count#68, count#72] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#63)), count(ss_list_price#63), count(distinct ss_list_price#63)] - -(61) CometColumnarToRow [codegen id : 9] -Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] - -(62) BroadcastExchange -Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] -Arguments: IdentityBroadcastMode, [plan_id=14] - -(63) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] -ReadSchema: struct - -(65) CometFilter -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Condition : (((isnotnull(ss_quantity#76) AND (ss_quantity#76 >= 26)) AND (ss_quantity#76 <= 30)) AND ((((ss_list_price#78 >= 154.00) AND (ss_list_price#78 <= 164.00)) OR ((ss_coupon_amt#79 >= 7326.00) AND (ss_coupon_amt#79 <= 8326.00))) OR ((ss_wholesale_cost#77 >= 7.00) AND (ss_wholesale_cost#77 <= 27.00)))) - -(66) CometProject -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Arguments: [ss_list_price#78], [ss_list_price#78] - -(67) CometHashAggregate -Input [1]: [ss_list_price#78] -Keys [1]: [ss_list_price#78] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_list_price#78)] - -(68) CometExchange -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(69) CometColumnarToRow [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] - -(70) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Keys [1]: [ss_list_price#78] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85] -Results [4]: [ss_list_price#78, sum#81, count#82, count#83] - -(71) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78), partial_count(distinct ss_list_price#78)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85, count(ss_list_price#78)#86] -Results [4]: [sum#81, count#82, count#83, count#87] - -(72) CometColumnarExchange -Input [4]: [sum#81, count#82, count#83, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(73) CometHashAggregate -Input [4]: [sum#81, count#82, count#83, count#87] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#78)), count(ss_list_price#78), count(distinct ss_list_price#78)] - -(74) CometColumnarToRow [codegen id : 11] -Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] - -(75) BroadcastExchange -Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] -Arguments: IdentityBroadcastMode, [plan_id=17] - -(76) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/extended.txt deleted file mode 100644 index 809265b4d5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/extended.txt +++ /dev/null @@ -1,78 +0,0 @@ -BroadcastNestedLoopJoin -:- BroadcastNestedLoopJoin -: :- BroadcastNestedLoopJoin -: : :- BroadcastNestedLoopJoin -: : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] -: : : : :- CometColumnarToRow -: : : : : +- CometHashAggregate -: : : : : +- CometColumnarExchange -: : : : : +- HashAggregate -: : : : : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : : : : +- CometColumnarToRow -: : : : : +- CometExchange -: : : : : +- CometHashAggregate -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : +- BroadcastExchange -: : : : +- CometColumnarToRow -: : : : +- CometHashAggregate -: : : : +- CometColumnarExchange -: : : : +- HashAggregate -: : : : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : : : +- CometColumnarToRow -: : : : +- CometExchange -: : : : +- CometHashAggregate -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : +- BroadcastExchange -: : : +- CometColumnarToRow -: : : +- CometHashAggregate -: : : +- CometColumnarExchange -: : : +- HashAggregate -: : : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : : +- CometColumnarToRow -: : : +- CometExchange -: : : +- CometHashAggregate -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : +- BroadcastExchange -: : +- CometColumnarToRow -: : +- CometHashAggregate -: : +- CometColumnarExchange -: : +- HashAggregate -: : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : +- CometColumnarToRow -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: +- BroadcastExchange -: +- CometColumnarToRow -: +- CometHashAggregate -: +- CometColumnarExchange -: +- HashAggregate -: +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: +- CometColumnarToRow -: +- CometExchange -: +- CometHashAggregate -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -+- BroadcastExchange - +- CometColumnarToRow - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - -Comet accelerated 42 out of 64 eligible operators (65%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index ce476affda..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt +++ /dev/null @@ -1,105 +0,0 @@ -WholeStageCodegen (12) - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B1_LP,B1_CNT,B1_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #2 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B2_LP,B2_CNT,B2_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #4 - WholeStageCodegen (2) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #5 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B3_LP,B3_CNT,B3_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #7 - WholeStageCodegen (4) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #8 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B4_LP,B4_CNT,B4_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #10 - WholeStageCodegen (6) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #11 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B5_LP,B5_CNT,B5_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #13 - WholeStageCodegen (8) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #14 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B6_LP,B6_CNT,B6_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #16 - WholeStageCodegen (10) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #17 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/explain.txt deleted file mode 100644 index f8b79c2e07..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/explain.txt +++ /dev/null @@ -1,340 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (18) - : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet spark_catalog.default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * Filter (12) - : : : : : +- * ColumnarToRow (11) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (10) - : : : : +- ReusedExchange (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.store (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.item (32) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 8] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] - -(3) Filter [codegen id : 8] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(4) Scan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(6) Filter [codegen id : 1] -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(7) BroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 8] -Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] -Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(10) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(12) Filter [codegen id : 2] -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(13) BroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] -Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(16) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#19] - -(17) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 8] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] - -(19) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#20] - -(20) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [sr_returned_date_sk#12] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#20] - -(22) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#21] - -(23) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#21] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 8] -Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#21] - -(25) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(26) CometFilter -Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] -Condition : isnotnull(s_store_sk#22) - -(27) CometProject -Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] -Arguments: [s_store_sk#22, s_store_id#25, s_store_name#24], [s_store_sk#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#23, 16)) AS s_store_id#25, s_store_name#24] - -(28) CometColumnarToRow [codegen id : 6] -Input [3]: [s_store_sk#22, s_store_id#25, s_store_name#24] - -(29) BroadcastExchange -Input [3]: [s_store_sk#22, s_store_id#25, s_store_name#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(30) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#22] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#22, s_store_id#25, s_store_name#24] - -(32) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] -Condition : isnotnull(i_item_sk#26) - -(34) CometProject -Input [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] -Arguments: [i_item_sk#26, i_item_id#29, i_item_desc#28], [i_item_sk#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#27, 16)) AS i_item_id#29, i_item_desc#28] - -(35) CometColumnarToRow [codegen id : 7] -Input [3]: [i_item_sk#26, i_item_id#29, i_item_desc#28] - -(36) BroadcastExchange -Input [3]: [i_item_sk#26, i_item_id#29, i_item_desc#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(37) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#26] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 8] -Output [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24, i_item_id#29, i_item_desc#28] -Input [9]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24, i_item_sk#26, i_item_id#29, i_item_desc#28] - -(39) HashAggregate [codegen id : 8] -Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24, i_item_id#29, i_item_desc#28] -Keys [4]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24] -Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#11), partial_sum(cs_quantity#16)] -Aggregate Attributes [3]: [sum#30, sum#31, sum#32] -Results [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum#33, sum#34, sum#35] - -(40) CometColumnarExchange -Input [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum#33, sum#34, sum#35] -Arguments: hashpartitioning(i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(41) CometColumnarToRow [codegen id : 9] -Input [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum#33, sum#34, sum#35] - -(42) HashAggregate [codegen id : 9] -Input [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum#33, sum#34, sum#35] -Keys [4]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24] -Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity#16)] -Aggregate Attributes [3]: [sum(ss_quantity#5)#36, sum(sr_return_quantity#11)#37, sum(cs_quantity#16)#38] -Results [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum(ss_quantity#5)#36 AS store_sales_quantity#39, sum(sr_return_quantity#11)#37 AS store_returns_quantity#40, sum(cs_quantity#16)#38 AS catalog_sales_quantity#41] - -(43) TakeOrderedAndProject -Input [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, store_sales_quantity#39, store_returns_quantity#40, catalog_sales_quantity#41] -Arguments: 100, [i_item_id#29 ASC NULLS FIRST, i_item_desc#28 ASC NULLS FIRST, s_store_id#25 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST], [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, store_sales_quantity#39, store_returns_quantity#40, catalog_sales_quantity#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometNativeScan parquet spark_catalog.default.date_dim (44) - - -(44) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#42, d_moy#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [3]: [d_date_sk#19, d_year#42, d_moy#43] -Condition : ((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 = 9)) AND (d_year#42 = 1999)) AND isnotnull(d_date_sk#19)) - -(46) CometProject -Input [3]: [d_date_sk#19, d_year#42, d_moy#43] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(48) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometNativeScan parquet spark_catalog.default.date_dim (49) - - -(49) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#20, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#20, d_year#44, d_moy#45] -Condition : (((((isnotnull(d_moy#45) AND isnotnull(d_year#44)) AND (d_moy#45 >= 9)) AND (d_moy#45 <= 12)) AND (d_year#44 = 1999)) AND isnotnull(d_date_sk#20)) - -(51) CometProject -Input [3]: [d_date_sk#20, d_year#44, d_moy#45] -Arguments: [d_date_sk#20], [d_date_sk#20] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#20] - -(53) BroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 -BroadcastExchange (58) -+- * CometColumnarToRow (57) - +- CometProject (56) - +- CometFilter (55) - +- CometNativeScan parquet spark_catalog.default.date_dim (54) - - -(54) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#21, d_year#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(55) CometFilter -Input [2]: [d_date_sk#21, d_year#46] -Condition : (d_year#46 IN (1999,2000,2001) AND isnotnull(d_date_sk#21)) - -(56) CometProject -Input [2]: [d_date_sk#21, d_year#46] -Arguments: [d_date_sk#21], [d_date_sk#21] - -(57) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#21] - -(58) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/simplified.txt deleted file mode 100644 index 54bdbeeb5a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/simplified.txt +++ /dev/null @@ -1,86 +0,0 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] - WholeStageCodegen (9) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity),store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - WholeStageCodegen (8) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] [sum,sum,sum,sum,sum,sum] - Project [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [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] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Filter [cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometNativeScan 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/q29.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/explain.txt deleted file mode 100644 index a885286877..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,366 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometTakeOrderedAndProject (46) - +- CometHashAggregate (45) - +- CometExchange (44) - +- CometHashAggregate (43) - +- CometProject (42) - +- CometBroadcastHashJoin (41) - :- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometProject (18) - : : : : +- CometBroadcastHashJoin (17) - : : : : :- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (8) - : : : : +- CometBroadcastExchange (16) - : : : : +- CometProject (15) - : : : : +- CometFilter (14) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : : : +- CometBroadcastExchange (22) - : : : +- CometProject (21) - : : : +- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - : : +- CometBroadcastExchange (28) - : : +- CometProject (27) - : : +- CometFilter (26) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - : +- CometBroadcastExchange (34) - : +- CometProject (33) - : +- CometFilter (32) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (31) - +- CometBroadcastExchange (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (37) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(5) CometBroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(6) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(10) CometBroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(11) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] -Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight - -(12) CometProject -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9)) AND (d_year#20 = 1999)) AND isnotnull(d_date_sk#19)) - -(15) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 9)) AND (d_moy#24 <= 12)) AND (d_year#23 = 1999)) AND isnotnull(d_date_sk#22)) - -(21) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#22] -Arguments: [sr_returned_date_sk#12], [d_date_sk#22], Inner, BuildRight - -(24) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#22] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) - -(27) CometProject -Input [2]: [d_date_sk#25, d_year#26] -Arguments: [d_date_sk#25], [d_date_sk#25] - -(28) CometBroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: [d_date_sk#25] - -(29) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#25] -Arguments: [cs_sold_date_sk#17], [d_date_sk#25], Inner, BuildRight - -(30) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#25] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] -Condition : isnotnull(s_store_sk#27) - -(33) CometProject -Input [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] -Arguments: [s_store_sk#27, s_store_id#30, s_store_name#29], [s_store_sk#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#28, 16)) AS s_store_id#30, s_store_name#29] - -(34) CometBroadcastExchange -Input [3]: [s_store_sk#27, s_store_id#30, s_store_name#29] -Arguments: [s_store_sk#27, s_store_id#30, s_store_name#29] - -(35) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Right output [3]: [s_store_sk#27, s_store_id#30, s_store_name#29] -Arguments: [ss_store_sk#3], [s_store_sk#27], Inner, BuildRight - -(36) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#27, s_store_id#30, s_store_name#29] -Arguments: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29], [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(38) CometFilter -Input [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Condition : isnotnull(i_item_sk#31) - -(39) CometProject -Input [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Arguments: [i_item_sk#31, i_item_id#34, i_item_desc#33], [i_item_sk#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#32, 16)) AS i_item_id#34, i_item_desc#33] - -(40) CometBroadcastExchange -Input [3]: [i_item_sk#31, i_item_id#34, i_item_desc#33] -Arguments: [i_item_sk#31, i_item_id#34, i_item_desc#33] - -(41) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29] -Right output [3]: [i_item_sk#31, i_item_id#34, i_item_desc#33] -Arguments: [ss_item_sk#1], [i_item_sk#31], Inner, BuildRight - -(42) CometProject -Input [9]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_sk#31, i_item_id#34, i_item_desc#33] -Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_id#34, i_item_desc#33], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_id#34, i_item_desc#33] - -(43) CometHashAggregate -Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_id#34, i_item_desc#33] -Keys [4]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29] -Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#11), partial_sum(cs_quantity#16)] - -(44) CometExchange -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, sum#35, sum#36, sum#37] -Arguments: hashpartitioning(i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(45) CometHashAggregate -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, sum#35, sum#36, sum#37] -Keys [4]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29] -Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity#16)] - -(46) CometTakeOrderedAndProject -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#34 ASC NULLS FIRST,i_item_desc#33 ASC NULLS FIRST,s_store_id#30 ASC NULLS FIRST,s_store_name#29 ASC NULLS FIRST], output=[i_item_id#34,i_item_desc#33,s_store_id#30,s_store_name#29,store_sales_quantity#38,store_returns_quantity#39,catalog_sales_quantity#40]), [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40], 100, 0, [i_item_id#34 ASC NULLS FIRST, i_item_desc#33 ASC NULLS FIRST, s_store_id#30 ASC NULLS FIRST, s_store_name#29 ASC NULLS FIRST], [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40] - -(47) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9)) AND (d_year#20 = 1999)) AND isnotnull(d_date_sk#19)) - -(50) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(52) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 9)) AND (d_moy#24 <= 12)) AND (d_year#23 = 1999)) AND isnotnull(d_date_sk#22)) - -(55) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(56) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(57) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 -BroadcastExchange (62) -+- * CometColumnarToRow (61) - +- CometProject (60) - +- CometFilter (59) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) - - -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(59) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) - -(60) CometProject -Input [2]: [d_date_sk#25, d_year#26] -Arguments: [d_date_sk#25], [d_date_sk#25] - -(61) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#25] - -(62) BroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/simplified.txt deleted file mode 100644 index 15b992d3a4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] - CometHashAggregate [sum,sum,sum] [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity)] - CometExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - CometHashAggregate [ss_quantity,sr_return_quantity,cs_quantity] [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] - CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - 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_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 [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 [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_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 [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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #8 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #9 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan [native_iceberg_compat] 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/q29/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/explain.txt deleted file mode 100644 index a885286877..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/explain.txt +++ /dev/null @@ -1,366 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometTakeOrderedAndProject (46) - +- CometHashAggregate (45) - +- CometExchange (44) - +- CometHashAggregate (43) - +- CometProject (42) - +- CometBroadcastHashJoin (41) - :- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometProject (18) - : : : : +- CometBroadcastHashJoin (17) - : : : : :- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (8) - : : : : +- CometBroadcastExchange (16) - : : : : +- CometProject (15) - : : : : +- CometFilter (14) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : : : +- CometBroadcastExchange (22) - : : : +- CometProject (21) - : : : +- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - : : +- CometBroadcastExchange (28) - : : +- CometProject (27) - : : +- CometFilter (26) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - : +- CometBroadcastExchange (34) - : +- CometProject (33) - : +- CometFilter (32) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (31) - +- CometBroadcastExchange (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (37) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(5) CometBroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(6) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(10) CometBroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(11) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] -Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight - -(12) CometProject -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9)) AND (d_year#20 = 1999)) AND isnotnull(d_date_sk#19)) - -(15) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 9)) AND (d_moy#24 <= 12)) AND (d_year#23 = 1999)) AND isnotnull(d_date_sk#22)) - -(21) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#22] -Arguments: [sr_returned_date_sk#12], [d_date_sk#22], Inner, BuildRight - -(24) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#22] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) - -(27) CometProject -Input [2]: [d_date_sk#25, d_year#26] -Arguments: [d_date_sk#25], [d_date_sk#25] - -(28) CometBroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: [d_date_sk#25] - -(29) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#25] -Arguments: [cs_sold_date_sk#17], [d_date_sk#25], Inner, BuildRight - -(30) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#25] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] -Condition : isnotnull(s_store_sk#27) - -(33) CometProject -Input [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] -Arguments: [s_store_sk#27, s_store_id#30, s_store_name#29], [s_store_sk#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#28, 16)) AS s_store_id#30, s_store_name#29] - -(34) CometBroadcastExchange -Input [3]: [s_store_sk#27, s_store_id#30, s_store_name#29] -Arguments: [s_store_sk#27, s_store_id#30, s_store_name#29] - -(35) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Right output [3]: [s_store_sk#27, s_store_id#30, s_store_name#29] -Arguments: [ss_store_sk#3], [s_store_sk#27], Inner, BuildRight - -(36) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#27, s_store_id#30, s_store_name#29] -Arguments: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29], [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(38) CometFilter -Input [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Condition : isnotnull(i_item_sk#31) - -(39) CometProject -Input [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Arguments: [i_item_sk#31, i_item_id#34, i_item_desc#33], [i_item_sk#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#32, 16)) AS i_item_id#34, i_item_desc#33] - -(40) CometBroadcastExchange -Input [3]: [i_item_sk#31, i_item_id#34, i_item_desc#33] -Arguments: [i_item_sk#31, i_item_id#34, i_item_desc#33] - -(41) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29] -Right output [3]: [i_item_sk#31, i_item_id#34, i_item_desc#33] -Arguments: [ss_item_sk#1], [i_item_sk#31], Inner, BuildRight - -(42) CometProject -Input [9]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_sk#31, i_item_id#34, i_item_desc#33] -Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_id#34, i_item_desc#33], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_id#34, i_item_desc#33] - -(43) CometHashAggregate -Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_id#34, i_item_desc#33] -Keys [4]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29] -Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#11), partial_sum(cs_quantity#16)] - -(44) CometExchange -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, sum#35, sum#36, sum#37] -Arguments: hashpartitioning(i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(45) CometHashAggregate -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, sum#35, sum#36, sum#37] -Keys [4]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29] -Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity#16)] - -(46) CometTakeOrderedAndProject -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#34 ASC NULLS FIRST,i_item_desc#33 ASC NULLS FIRST,s_store_id#30 ASC NULLS FIRST,s_store_name#29 ASC NULLS FIRST], output=[i_item_id#34,i_item_desc#33,s_store_id#30,s_store_name#29,store_sales_quantity#38,store_returns_quantity#39,catalog_sales_quantity#40]), [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40], 100, 0, [i_item_id#34 ASC NULLS FIRST, i_item_desc#33 ASC NULLS FIRST, s_store_id#30 ASC NULLS FIRST, s_store_name#29 ASC NULLS FIRST], [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40] - -(47) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9)) AND (d_year#20 = 1999)) AND isnotnull(d_date_sk#19)) - -(50) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(52) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 9)) AND (d_moy#24 <= 12)) AND (d_year#23 = 1999)) AND isnotnull(d_date_sk#22)) - -(55) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(56) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(57) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 -BroadcastExchange (62) -+- * CometColumnarToRow (61) - +- CometProject (60) - +- CometFilter (59) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) - - -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(59) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) - -(60) CometProject -Input [2]: [d_date_sk#25, d_year#26] -Arguments: [d_date_sk#25], [d_date_sk#25] - -(61) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#25] - -(62) BroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/extended.txt deleted file mode 100644 index fe03fef8a7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/extended.txt +++ /dev/null @@ -1,67 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 15b992d3a4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] - CometHashAggregate [sum,sum,sum] [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity)] - CometExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - CometHashAggregate [ss_quantity,sr_return_quantity,cs_quantity] [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] - CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - 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_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 [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 [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_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 [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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #8 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #9 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_datafusion/explain.txt deleted file mode 100644 index 0fe8bceddd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_datafusion/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometNativeScan parquet spark_catalog.default.item (9) - - -(1) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((isnotnull(d_moy#3) AND (d_moy#3 = 11)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometNativeScan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,128), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Condition : ((isnotnull(i_manufact_id#10) AND (i_manufact_id#10 = 128)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#9, 50)) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,sum_agg#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,sum_agg#15]), [d_year#2, brand_id#13, brand#14, sum_agg#15], 100, 0, [d_year#2 ASC NULLS FIRST, sum_agg#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, sum_agg#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_datafusion/simplified.txt deleted file mode 100644 index c19e6caf52..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_datafusion/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,sum_agg] - CometHashAggregate [sum] [d_year,brand_id,brand,sum_agg,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] - CometNativeScan 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/q3.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/explain.txt deleted file mode 100644 index 2d4f57a9f6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((isnotnull(d_moy#3) AND (d_moy#3 = 11)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,128), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Condition : ((isnotnull(i_manufact_id#10) AND (i_manufact_id#10 = 128)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#9, 50)) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,sum_agg#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,sum_agg#15]), [d_year#2, brand_id#13, brand#14, sum_agg#15], 100, 0, [d_year#2 ASC NULLS FIRST, sum_agg#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, sum_agg#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/simplified.txt deleted file mode 100644 index 7bdad4b94f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,sum_agg] - CometHashAggregate [sum] [d_year,brand_id,brand,sum_agg,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] - CometScan [native_iceberg_compat] 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/q3/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/explain.txt deleted file mode 100644 index 2d4f57a9f6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((isnotnull(d_moy#3) AND (d_moy#3 = 11)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,128), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Condition : ((isnotnull(i_manufact_id#10) AND (i_manufact_id#10 = 128)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#9, 50)) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,sum_agg#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,sum_agg#15]), [d_year#2, brand_id#13, brand#14, sum_agg#15], 100, 0, [d_year#2 ASC NULLS FIRST, sum_agg#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, sum_agg#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/extended.txt deleted file mode 100644 index a292badf5d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 7bdad4b94f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,sum_agg] - CometHashAggregate [sum] [d_year,brand_id,brand,sum_agg,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/explain.txt deleted file mode 100644 index 87bbeb050a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/explain.txt +++ /dev/null @@ -1,346 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (54) -+- * Project (53) - +- * BroadcastHashJoin Inner BuildRight (52) - :- * Project (46) - : +- * BroadcastHashJoin Inner BuildRight (45) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- * HashAggregate (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.web_returns (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (11) - : : : +- * CometColumnarToRow (10) - : : : +- CometProject (9) - : : : +- CometFilter (8) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) - : : +- BroadcastExchange (37) - : : +- * Filter (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * HashAggregate (31) - : : +- * CometColumnarToRow (30) - : : +- CometColumnarExchange (29) - : : +- * HashAggregate (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (24) - : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : :- * Filter (21) - : : : : +- * ColumnarToRow (20) - : : : : +- Scan parquet spark_catalog.default.web_returns (19) - : : : +- ReusedExchange (22) - : : +- ReusedExchange (25) - : +- BroadcastExchange (44) - : +- * CometColumnarToRow (43) - : +- CometProject (42) - : +- CometFilter (41) - : +- CometNativeScan parquet spark_catalog.default.customer (40) - +- BroadcastExchange (51) - +- * CometColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometNativeScan parquet spark_catalog.default.customer_address (47) - - -(1) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_returning_addr_sk), IsNotNull(wr_returning_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] - -(3) Filter [codegen id : 3] -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Condition : (isnotnull(wr_returning_addr_sk#2) AND isnotnull(wr_returning_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [wr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_state#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_state#8] -Condition : (isnotnull(ca_address_sk#7) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#8, 2)))) - -(9) CometProject -Input [2]: [ca_address_sk#7, ca_state#8] -Arguments: [ca_address_sk#7, ca_state#9], [ca_address_sk#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#8, 2)) AS ca_state#9] - -(10) CometColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#7, ca_state#9] - -(11) BroadcastExchange -Input [2]: [ca_address_sk#7, ca_state#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [wr_returning_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#7, ca_state#9] - -(14) HashAggregate [codegen id : 3] -Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] -Keys [2]: [wr_returning_customer_sk#1, ca_state#9] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum#10] -Results [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] - -(15) CometColumnarExchange -Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 11] -Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] - -(17) HashAggregate [codegen id : 11] -Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] -Keys [2]: [wr_returning_customer_sk#1, ca_state#9] -Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#12] -Results [3]: [wr_returning_customer_sk#1 AS ctr_customer_sk#13, ca_state#9 AS ctr_state#14, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#12,17,2) AS ctr_total_return#15] - -(18) Filter [codegen id : 11] -Input [3]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15] -Condition : isnotnull(ctr_total_return#15) - -(19) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#19), dynamicpruningexpression(wr_returned_date_sk#19 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_returning_addr_sk)] -ReadSchema: struct - -(20) ColumnarToRow [codegen id : 6] -Input [4]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19] - -(21) Filter [codegen id : 6] -Input [4]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19] -Condition : isnotnull(wr_returning_addr_sk#17) - -(22) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#20] - -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returned_date_sk#19] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18] -Input [5]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19, d_date_sk#20] - -(25) ReusedExchange [Reuses operator id: 11] -Output [2]: [ca_address_sk#21, ca_state#22] - -(26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returning_addr_sk#17] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#22] -Input [5]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, ca_address_sk#21, ca_state#22] - -(28) HashAggregate [codegen id : 6] -Input [3]: [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#22] -Keys [2]: [wr_returning_customer_sk#16, ca_state#22] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#18))] -Aggregate Attributes [1]: [sum#23] -Results [3]: [wr_returning_customer_sk#16, ca_state#22, sum#24] - -(29) CometColumnarExchange -Input [3]: [wr_returning_customer_sk#16, ca_state#22, sum#24] -Arguments: hashpartitioning(wr_returning_customer_sk#16, ca_state#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(30) CometColumnarToRow [codegen id : 7] -Input [3]: [wr_returning_customer_sk#16, ca_state#22, sum#24] - -(31) HashAggregate [codegen id : 7] -Input [3]: [wr_returning_customer_sk#16, ca_state#22, sum#24] -Keys [2]: [wr_returning_customer_sk#16, ca_state#22] -Functions [1]: [sum(UnscaledValue(wr_return_amt#18))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#18))#12] -Results [2]: [ca_state#22 AS ctr_state#25, MakeDecimal(sum(UnscaledValue(wr_return_amt#18))#12,17,2) AS ctr_total_return#26] - -(32) HashAggregate [codegen id : 7] -Input [2]: [ctr_state#25, ctr_total_return#26] -Keys [1]: [ctr_state#25] -Functions [1]: [partial_avg(ctr_total_return#26)] -Aggregate Attributes [2]: [sum#27, count#28] -Results [3]: [ctr_state#25, sum#29, count#30] - -(33) CometColumnarExchange -Input [3]: [ctr_state#25, sum#29, count#30] -Arguments: hashpartitioning(ctr_state#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 8] -Input [3]: [ctr_state#25, sum#29, count#30] - -(35) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#25, sum#29, count#30] -Keys [1]: [ctr_state#25] -Functions [1]: [avg(ctr_total_return#26)] -Aggregate Attributes [1]: [avg(ctr_total_return#26)#31] -Results [2]: [(avg(ctr_total_return#26)#31 * 1.2) AS (avg(ctr_total_return) * 1.2)#32, ctr_state#25] - -(36) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#32) - -(37) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] - -(38) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_state#14] -Right keys [1]: [ctr_state#25] -Join type: Inner -Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#32) - -(39) Project [codegen id : 11] -Output [2]: [ctr_customer_sk#13, ctr_total_return#15] -Input [5]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#32, ctr_state#25] - -(40) CometNativeScan parquet spark_catalog.default.customer -Output [14]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(41) CometFilter -Input [14]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46] -Condition : (isnotnull(c_customer_sk#33) AND isnotnull(c_current_addr_sk#35)) - -(42) CometProject -Input [14]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46] -Arguments: [c_customer_sk#33, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46], [c_customer_sk#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#34, 16)) AS c_customer_id#47, c_current_addr_sk#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#36, 10)) AS c_salutation#48, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#37, 20)) AS c_first_name#49, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#38, 30)) AS c_last_name#50, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#39, 1)) AS c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#44, 13)) AS c_login#52, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#45, 50)) AS c_email_address#53, c_last_review_date#46] - -(43) CometColumnarToRow [codegen id : 9] -Input [14]: [c_customer_sk#33, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46] - -(44) BroadcastExchange -Input [14]: [c_customer_sk#33, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(45) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_customer_sk#13] -Right keys [1]: [c_customer_sk#33] -Join type: Inner -Join condition: None - -(46) Project [codegen id : 11] -Output [14]: [ctr_total_return#15, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46] -Input [16]: [ctr_customer_sk#13, ctr_total_return#15, c_customer_sk#33, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46] - -(47) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#54, ca_state#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(48) CometFilter -Input [2]: [ca_address_sk#54, ca_state#55] -Condition : ((isnotnull(ca_state#55) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#55, 2)) = GA)) AND isnotnull(ca_address_sk#54)) - -(49) CometProject -Input [2]: [ca_address_sk#54, ca_state#55] -Arguments: [ca_address_sk#54], [ca_address_sk#54] - -(50) CometColumnarToRow [codegen id : 10] -Input [1]: [ca_address_sk#54] - -(51) BroadcastExchange -Input [1]: [ca_address_sk#54] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(52) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#35] -Right keys [1]: [ca_address_sk#54] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 11] -Output [13]: [c_customer_id#47, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46, ctr_total_return#15] -Input [15]: [ctr_total_return#15, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46, ca_address_sk#54] - -(54) TakeOrderedAndProject -Input [13]: [c_customer_id#47, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46, ctr_total_return#15] -Arguments: 100, [c_customer_id#47 ASC NULLS FIRST, c_salutation#48 ASC NULLS FIRST, c_first_name#49 ASC NULLS FIRST, c_last_name#50 ASC NULLS FIRST, c_preferred_cust_flag#51 ASC NULLS FIRST, c_birth_day#40 ASC NULLS FIRST, c_birth_month#41 ASC NULLS FIRST, c_birth_year#42 ASC NULLS FIRST, c_birth_country#43 ASC NULLS FIRST, c_login#52 ASC NULLS FIRST, c_email_address#53 ASC NULLS FIRST, c_last_review_date#46 ASC NULLS FIRST, ctr_total_return#15 ASC NULLS FIRST], [c_customer_id#47, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46, ctr_total_return#15] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (59) -+- * CometColumnarToRow (58) - +- CometProject (57) - +- CometFilter (56) - +- CometNativeScan parquet spark_catalog.default.date_dim (55) - - -(55) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date_sk#6, d_year#56] -Condition : ((isnotnull(d_year#56) AND (d_year#56 = 2002)) AND isnotnull(d_date_sk#6)) - -(57) CometProject -Input [2]: [d_date_sk#6, d_year#56] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(58) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(59) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 19 Hosting Expression = wr_returned_date_sk#19 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/simplified.txt deleted file mode 100644 index 72b6fbd1ba..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/simplified.txt +++ /dev/null @@ -1,86 +0,0 @@ -TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - WholeStageCodegen (11) - Project [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (3) - HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] - Project [wr_returning_customer_sk,wr_return_amt,ca_state] - BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] - Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Filter [wr_returning_addr_sk,wr_returning_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (8) - Filter [(avg(ctr_total_return) * 1.2)] - HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ctr_state] #5 - WholeStageCodegen (7) - HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] - HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wr_returning_customer_sk,ca_state] #6 - WholeStageCodegen (6) - HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] - Project [wr_returning_customer_sk,wr_return_amt,ca_state] - BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] - Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Filter [wr_returning_addr_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [ca_address_sk,ca_state] #3 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [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] - 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] - CometNativeScan 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 #8 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan 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/q30.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/explain.txt deleted file mode 100644 index dac625d856..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,318 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (50) -+- CometTakeOrderedAndProject (49) - +- CometProject (48) - +- CometBroadcastHashJoin (47) - :- CometProject (42) - : +- CometBroadcastHashJoin (41) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometFilter (18) - : : : +- CometHashAggregate (17) - : : : +- CometExchange (16) - : : : +- CometHashAggregate (15) - : : : +- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : : +- CometBroadcastExchange (34) - : : +- CometFilter (33) - : : +- CometHashAggregate (32) - : : +- CometExchange (31) - : : +- CometHashAggregate (30) - : : +- CometHashAggregate (29) - : : +- CometExchange (28) - : : +- CometHashAggregate (27) - : : +- CometProject (26) - : : +- CometBroadcastHashJoin (25) - : : :- CometProject (23) - : : : +- CometBroadcastHashJoin (22) - : : : :- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (19) - : : : +- ReusedExchange (21) - : : +- ReusedExchange (24) - : +- CometBroadcastExchange (40) - : +- CometProject (39) - : +- CometFilter (38) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) - +- CometBroadcastExchange (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (43) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_returning_addr_sk), IsNotNull(wr_returning_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Condition : (isnotnull(wr_returning_addr_sk#2) AND isnotnull(wr_returning_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [wr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6] -Arguments: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3], [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#8, ca_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#8, ca_state#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#9, 2)))) - -(11) CometProject -Input [2]: [ca_address_sk#8, ca_state#9] -Arguments: [ca_address_sk#8, ca_state#10], [ca_address_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#9, 2)) AS ca_state#10] - -(12) CometBroadcastExchange -Input [2]: [ca_address_sk#8, ca_state#10] -Arguments: [ca_address_sk#8, ca_state#10] - -(13) CometBroadcastHashJoin -Left output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] -Right output [2]: [ca_address_sk#8, ca_state#10] -Arguments: [wr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight - -(14) CometProject -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#8, ca_state#10] -Arguments: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10], [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] - -(15) CometHashAggregate -Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] -Keys [2]: [wr_returning_customer_sk#1, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] - -(16) CometExchange -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#11] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#11] -Keys [2]: [wr_returning_customer_sk#1, ca_state#10] -Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] - -(18) CometFilter -Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Condition : isnotnull(ctr_total_return#14) - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#18), dynamicpruningexpression(wr_returned_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(wr_returning_addr_sk)] -ReadSchema: struct - -(20) CometFilter -Input [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] -Condition : isnotnull(wr_returning_addr_sk#16) - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#20] - -(22) CometBroadcastHashJoin -Left output [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] -Right output [1]: [d_date_sk#20] -Arguments: [wr_returned_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(23) CometProject -Input [5]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18, d_date_sk#20] -Arguments: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17], [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17] - -(24) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#21, ca_state#22] - -(25) CometBroadcastHashJoin -Left output [3]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17] -Right output [2]: [ca_address_sk#21, ca_state#22] -Arguments: [wr_returning_addr_sk#16], [ca_address_sk#21], Inner, BuildRight - -(26) CometProject -Input [5]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, ca_address_sk#21, ca_state#22] -Arguments: [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#22], [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#22] - -(27) CometHashAggregate -Input [3]: [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#22] -Keys [2]: [wr_returning_customer_sk#15, ca_state#22] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#17))] - -(28) CometExchange -Input [3]: [wr_returning_customer_sk#15, ca_state#22, sum#23] -Arguments: hashpartitioning(wr_returning_customer_sk#15, ca_state#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [3]: [wr_returning_customer_sk#15, ca_state#22, sum#23] -Keys [2]: [wr_returning_customer_sk#15, ca_state#22] -Functions [1]: [sum(UnscaledValue(wr_return_amt#17))] - -(30) CometHashAggregate -Input [2]: [ctr_state#24, ctr_total_return#25] -Keys [1]: [ctr_state#24] -Functions [1]: [partial_avg(ctr_total_return#25)] - -(31) CometExchange -Input [3]: [ctr_state#24, sum#26, count#27] -Arguments: hashpartitioning(ctr_state#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometHashAggregate -Input [3]: [ctr_state#24, sum#26, count#27] -Keys [1]: [ctr_state#24] -Functions [1]: [avg(ctr_total_return#25)] - -(33) CometFilter -Input [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#28) - -(34) CometBroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] -Arguments: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] - -(35) CometBroadcastHashJoin -Left output [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Right output [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] -Arguments: [ctr_state#13], [ctr_state#24], Inner, (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#28), BuildRight - -(36) CometProject -Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#28, ctr_state#24] -Arguments: [ctr_customer_sk#12, ctr_total_return#14], [ctr_customer_sk#12, ctr_total_return#14] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [14]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(38) CometFilter -Input [14]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42] -Condition : (isnotnull(c_customer_sk#29) AND isnotnull(c_current_addr_sk#31)) - -(39) CometProject -Input [14]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42] -Arguments: [c_customer_sk#29, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42], [c_customer_sk#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#30, 16)) AS c_customer_id#43, c_current_addr_sk#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#32, 10)) AS c_salutation#44, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#33, 20)) AS c_first_name#45, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#34, 30)) AS c_last_name#46, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#35, 1)) AS c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#40, 13)) AS c_login#48, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#41, 50)) AS c_email_address#49, c_last_review_date#42] - -(40) CometBroadcastExchange -Input [14]: [c_customer_sk#29, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] -Arguments: [c_customer_sk#29, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] - -(41) CometBroadcastHashJoin -Left output [2]: [ctr_customer_sk#12, ctr_total_return#14] -Right output [14]: [c_customer_sk#29, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] -Arguments: [ctr_customer_sk#12], [c_customer_sk#29], Inner, BuildRight - -(42) CometProject -Input [16]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#29, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] -Arguments: [ctr_total_return#14, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42], [ctr_total_return#14, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#50, ca_state#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(44) CometFilter -Input [2]: [ca_address_sk#50, ca_state#51] -Condition : ((isnotnull(ca_state#51) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#51, 2)) = GA)) AND isnotnull(ca_address_sk#50)) - -(45) CometProject -Input [2]: [ca_address_sk#50, ca_state#51] -Arguments: [ca_address_sk#50], [ca_address_sk#50] - -(46) CometBroadcastExchange -Input [1]: [ca_address_sk#50] -Arguments: [ca_address_sk#50] - -(47) CometBroadcastHashJoin -Left output [14]: [ctr_total_return#14, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] -Right output [1]: [ca_address_sk#50] -Arguments: [c_current_addr_sk#31], [ca_address_sk#50], Inner, BuildRight - -(48) CometProject -Input [15]: [ctr_total_return#14, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ca_address_sk#50] -Arguments: [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14], [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14] - -(49) CometTakeOrderedAndProject -Input [13]: [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#43 ASC NULLS FIRST,c_salutation#44 ASC NULLS FIRST,c_first_name#45 ASC NULLS FIRST,c_last_name#46 ASC NULLS FIRST,c_preferred_cust_flag#47 ASC NULLS FIRST,c_birth_day#36 ASC NULLS FIRST,c_birth_month#37 ASC NULLS FIRST,c_birth_year#38 ASC NULLS FIRST,c_birth_country#39 ASC NULLS FIRST,c_login#48 ASC NULLS FIRST,c_email_address#49 ASC NULLS FIRST,c_last_review_date#42 ASC NULLS FIRST,ctr_total_return#14 ASC NULLS FIRST], output=[c_customer_id#43,c_salutation#44,c_first_name#45,c_last_name#46,c_preferred_cust_flag#47,c_birth_day#36,c_birth_month#37,c_birth_year#38,c_birth_country#39,c_login#48,c_email_address#49,c_last_review_date#42,ctr_total_return#14]), [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14], 100, 0, [c_customer_id#43 ASC NULLS FIRST, c_salutation#44 ASC NULLS FIRST, c_first_name#45 ASC NULLS FIRST, c_last_name#46 ASC NULLS FIRST, c_preferred_cust_flag#47 ASC NULLS FIRST, c_birth_day#36 ASC NULLS FIRST, c_birth_month#37 ASC NULLS FIRST, c_birth_year#38 ASC NULLS FIRST, c_birth_country#39 ASC NULLS FIRST, c_login#48 ASC NULLS FIRST, c_email_address#49 ASC NULLS FIRST, c_last_review_date#42 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14] - -(50) CometColumnarToRow [codegen id : 1] -Input [13]: [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) - - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(52) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) - -(53) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(54) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(55) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 19 Hosting Expression = wr_returned_date_sk#18 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/simplified.txt deleted file mode 100644 index 0227534fa5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - CometBroadcastHashJoin [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ca_address_sk] - CometProject [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - CometBroadcastHashJoin [ctr_customer_sk,ctr_total_return,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] - CometProject [ctr_customer_sk,ctr_total_return] - CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] - CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] - CometHashAggregate [sum] [ctr_customer_sk,ctr_state,ctr_total_return,wr_returning_customer_sk,ca_state,sum(UnscaledValue(wr_return_amt))] - CometExchange [wr_returning_customer_sk,ca_state] #1 - CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] - CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 - CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] - CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_state,avg(ctr_total_return)] - CometExchange [ctr_state] #6 - CometHashAggregate [ctr_total_return] [ctr_state,sum,count] - CometHashAggregate [wr_returning_customer_sk,sum] [ctr_state,ctr_total_return,ca_state,sum(UnscaledValue(wr_return_amt))] - CometExchange [wr_returning_customer_sk,ca_state] #7 - CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] - CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 - ReusedExchange [ca_address_sk,ca_state] #4 - CometBroadcastExchange [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] #8 - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [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] - 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 [native_iceberg_compat] 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] - CometBroadcastExchange [ca_address_sk] #9 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] 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/q30/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/explain.txt deleted file mode 100644 index dac625d856..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/explain.txt +++ /dev/null @@ -1,318 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (50) -+- CometTakeOrderedAndProject (49) - +- CometProject (48) - +- CometBroadcastHashJoin (47) - :- CometProject (42) - : +- CometBroadcastHashJoin (41) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometFilter (18) - : : : +- CometHashAggregate (17) - : : : +- CometExchange (16) - : : : +- CometHashAggregate (15) - : : : +- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : : +- CometBroadcastExchange (34) - : : +- CometFilter (33) - : : +- CometHashAggregate (32) - : : +- CometExchange (31) - : : +- CometHashAggregate (30) - : : +- CometHashAggregate (29) - : : +- CometExchange (28) - : : +- CometHashAggregate (27) - : : +- CometProject (26) - : : +- CometBroadcastHashJoin (25) - : : :- CometProject (23) - : : : +- CometBroadcastHashJoin (22) - : : : :- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (19) - : : : +- ReusedExchange (21) - : : +- ReusedExchange (24) - : +- CometBroadcastExchange (40) - : +- CometProject (39) - : +- CometFilter (38) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) - +- CometBroadcastExchange (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (43) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_returning_addr_sk), IsNotNull(wr_returning_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Condition : (isnotnull(wr_returning_addr_sk#2) AND isnotnull(wr_returning_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [wr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6] -Arguments: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3], [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#8, ca_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#8, ca_state#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#9, 2)))) - -(11) CometProject -Input [2]: [ca_address_sk#8, ca_state#9] -Arguments: [ca_address_sk#8, ca_state#10], [ca_address_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#9, 2)) AS ca_state#10] - -(12) CometBroadcastExchange -Input [2]: [ca_address_sk#8, ca_state#10] -Arguments: [ca_address_sk#8, ca_state#10] - -(13) CometBroadcastHashJoin -Left output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] -Right output [2]: [ca_address_sk#8, ca_state#10] -Arguments: [wr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight - -(14) CometProject -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#8, ca_state#10] -Arguments: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10], [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] - -(15) CometHashAggregate -Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] -Keys [2]: [wr_returning_customer_sk#1, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] - -(16) CometExchange -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#11] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#11] -Keys [2]: [wr_returning_customer_sk#1, ca_state#10] -Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] - -(18) CometFilter -Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Condition : isnotnull(ctr_total_return#14) - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#18), dynamicpruningexpression(wr_returned_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(wr_returning_addr_sk)] -ReadSchema: struct - -(20) CometFilter -Input [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] -Condition : isnotnull(wr_returning_addr_sk#16) - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#20] - -(22) CometBroadcastHashJoin -Left output [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] -Right output [1]: [d_date_sk#20] -Arguments: [wr_returned_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(23) CometProject -Input [5]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18, d_date_sk#20] -Arguments: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17], [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17] - -(24) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#21, ca_state#22] - -(25) CometBroadcastHashJoin -Left output [3]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17] -Right output [2]: [ca_address_sk#21, ca_state#22] -Arguments: [wr_returning_addr_sk#16], [ca_address_sk#21], Inner, BuildRight - -(26) CometProject -Input [5]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, ca_address_sk#21, ca_state#22] -Arguments: [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#22], [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#22] - -(27) CometHashAggregate -Input [3]: [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#22] -Keys [2]: [wr_returning_customer_sk#15, ca_state#22] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#17))] - -(28) CometExchange -Input [3]: [wr_returning_customer_sk#15, ca_state#22, sum#23] -Arguments: hashpartitioning(wr_returning_customer_sk#15, ca_state#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [3]: [wr_returning_customer_sk#15, ca_state#22, sum#23] -Keys [2]: [wr_returning_customer_sk#15, ca_state#22] -Functions [1]: [sum(UnscaledValue(wr_return_amt#17))] - -(30) CometHashAggregate -Input [2]: [ctr_state#24, ctr_total_return#25] -Keys [1]: [ctr_state#24] -Functions [1]: [partial_avg(ctr_total_return#25)] - -(31) CometExchange -Input [3]: [ctr_state#24, sum#26, count#27] -Arguments: hashpartitioning(ctr_state#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometHashAggregate -Input [3]: [ctr_state#24, sum#26, count#27] -Keys [1]: [ctr_state#24] -Functions [1]: [avg(ctr_total_return#25)] - -(33) CometFilter -Input [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#28) - -(34) CometBroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] -Arguments: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] - -(35) CometBroadcastHashJoin -Left output [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Right output [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] -Arguments: [ctr_state#13], [ctr_state#24], Inner, (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#28), BuildRight - -(36) CometProject -Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#28, ctr_state#24] -Arguments: [ctr_customer_sk#12, ctr_total_return#14], [ctr_customer_sk#12, ctr_total_return#14] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [14]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(38) CometFilter -Input [14]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42] -Condition : (isnotnull(c_customer_sk#29) AND isnotnull(c_current_addr_sk#31)) - -(39) CometProject -Input [14]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42] -Arguments: [c_customer_sk#29, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42], [c_customer_sk#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#30, 16)) AS c_customer_id#43, c_current_addr_sk#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#32, 10)) AS c_salutation#44, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#33, 20)) AS c_first_name#45, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#34, 30)) AS c_last_name#46, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#35, 1)) AS c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#40, 13)) AS c_login#48, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#41, 50)) AS c_email_address#49, c_last_review_date#42] - -(40) CometBroadcastExchange -Input [14]: [c_customer_sk#29, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] -Arguments: [c_customer_sk#29, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] - -(41) CometBroadcastHashJoin -Left output [2]: [ctr_customer_sk#12, ctr_total_return#14] -Right output [14]: [c_customer_sk#29, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] -Arguments: [ctr_customer_sk#12], [c_customer_sk#29], Inner, BuildRight - -(42) CometProject -Input [16]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#29, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] -Arguments: [ctr_total_return#14, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42], [ctr_total_return#14, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#50, ca_state#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(44) CometFilter -Input [2]: [ca_address_sk#50, ca_state#51] -Condition : ((isnotnull(ca_state#51) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#51, 2)) = GA)) AND isnotnull(ca_address_sk#50)) - -(45) CometProject -Input [2]: [ca_address_sk#50, ca_state#51] -Arguments: [ca_address_sk#50], [ca_address_sk#50] - -(46) CometBroadcastExchange -Input [1]: [ca_address_sk#50] -Arguments: [ca_address_sk#50] - -(47) CometBroadcastHashJoin -Left output [14]: [ctr_total_return#14, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] -Right output [1]: [ca_address_sk#50] -Arguments: [c_current_addr_sk#31], [ca_address_sk#50], Inner, BuildRight - -(48) CometProject -Input [15]: [ctr_total_return#14, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ca_address_sk#50] -Arguments: [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14], [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14] - -(49) CometTakeOrderedAndProject -Input [13]: [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#43 ASC NULLS FIRST,c_salutation#44 ASC NULLS FIRST,c_first_name#45 ASC NULLS FIRST,c_last_name#46 ASC NULLS FIRST,c_preferred_cust_flag#47 ASC NULLS FIRST,c_birth_day#36 ASC NULLS FIRST,c_birth_month#37 ASC NULLS FIRST,c_birth_year#38 ASC NULLS FIRST,c_birth_country#39 ASC NULLS FIRST,c_login#48 ASC NULLS FIRST,c_email_address#49 ASC NULLS FIRST,c_last_review_date#42 ASC NULLS FIRST,ctr_total_return#14 ASC NULLS FIRST], output=[c_customer_id#43,c_salutation#44,c_first_name#45,c_last_name#46,c_preferred_cust_flag#47,c_birth_day#36,c_birth_month#37,c_birth_year#38,c_birth_country#39,c_login#48,c_email_address#49,c_last_review_date#42,ctr_total_return#14]), [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14], 100, 0, [c_customer_id#43 ASC NULLS FIRST, c_salutation#44 ASC NULLS FIRST, c_first_name#45 ASC NULLS FIRST, c_last_name#46 ASC NULLS FIRST, c_preferred_cust_flag#47 ASC NULLS FIRST, c_birth_day#36 ASC NULLS FIRST, c_birth_month#37 ASC NULLS FIRST, c_birth_year#38 ASC NULLS FIRST, c_birth_country#39 ASC NULLS FIRST, c_login#48 ASC NULLS FIRST, c_email_address#49 ASC NULLS FIRST, c_last_review_date#42 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14] - -(50) CometColumnarToRow [codegen id : 1] -Input [13]: [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) - - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(52) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) - -(53) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(54) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(55) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 19 Hosting Expression = wr_returned_date_sk#18 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/extended.txt deleted file mode 100644 index 7ebd69d340..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 0227534fa5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - CometBroadcastHashJoin [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ca_address_sk] - CometProject [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - CometBroadcastHashJoin [ctr_customer_sk,ctr_total_return,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] - CometProject [ctr_customer_sk,ctr_total_return] - CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] - CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] - CometHashAggregate [sum] [ctr_customer_sk,ctr_state,ctr_total_return,wr_returning_customer_sk,ca_state,sum(UnscaledValue(wr_return_amt))] - CometExchange [wr_returning_customer_sk,ca_state] #1 - CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] - CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 - CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] - CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_state,avg(ctr_total_return)] - CometExchange [ctr_state] #6 - CometHashAggregate [ctr_total_return] [ctr_state,sum,count] - CometHashAggregate [wr_returning_customer_sk,sum] [ctr_state,ctr_total_return,ca_state,sum(UnscaledValue(wr_return_amt))] - CometExchange [wr_returning_customer_sk,ca_state] #7 - CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] - CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 - ReusedExchange [ca_address_sk,ca_state] #4 - CometBroadcastExchange [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] #8 - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [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] - 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 [native_iceberg_compat] 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] - CometBroadcastExchange [ca_address_sk] #9 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/explain.txt deleted file mode 100644 index 9ce8c738e8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/explain.txt +++ /dev/null @@ -1,644 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (97) -+- CometSort (96) - +- CometColumnarExchange (95) - +- * Project (94) - +- * BroadcastHashJoin Inner BuildRight (93) - :- * Project (78) - : +- * BroadcastHashJoin Inner BuildRight (77) - : :- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (47) - : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : :- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * HashAggregate (16) - : : : : : +- * CometColumnarToRow (15) - : : : : : +- CometColumnarExchange (14) - : : : : : +- * HashAggregate (13) - : : : : : +- * Project (12) - : : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : : :- * Project (6) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : +- ReusedExchange (4) - : : : : : +- BroadcastExchange (10) - : : : : : +- * CometColumnarToRow (9) - : : : : : +- CometFilter (8) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) - : : : : +- BroadcastExchange (30) - : : : : +- * HashAggregate (29) - : : : : +- * CometColumnarToRow (28) - : : : : +- CometColumnarExchange (27) - : : : : +- * HashAggregate (26) - : : : : +- * Project (25) - : : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : : :- * Project (22) - : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : :- * Filter (19) - : : : : : : +- * ColumnarToRow (18) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (17) - : : : : : +- ReusedExchange (20) - : : : : +- ReusedExchange (23) - : : : +- BroadcastExchange (45) - : : : +- * HashAggregate (44) - : : : +- * CometColumnarToRow (43) - : : : +- CometColumnarExchange (42) - : : : +- * HashAggregate (41) - : : : +- * Project (40) - : : : +- * BroadcastHashJoin Inner BuildRight (39) - : : : :- * Project (37) - : : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : : :- * Filter (34) - : : : : : +- * ColumnarToRow (33) - : : : : : +- Scan parquet spark_catalog.default.store_sales (32) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- BroadcastExchange (61) - : : +- * HashAggregate (60) - : : +- * CometColumnarToRow (59) - : : +- CometColumnarExchange (58) - : : +- * HashAggregate (57) - : : +- * Project (56) - : : +- * BroadcastHashJoin Inner BuildRight (55) - : : :- * Project (53) - : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : :- * Filter (50) - : : : : +- * ColumnarToRow (49) - : : : : +- Scan parquet spark_catalog.default.web_sales (48) - : : : +- ReusedExchange (51) - : : +- ReusedExchange (54) - : +- BroadcastExchange (76) - : +- * HashAggregate (75) - : +- * CometColumnarToRow (74) - : +- CometColumnarExchange (73) - : +- * HashAggregate (72) - : +- * Project (71) - : +- * BroadcastHashJoin Inner BuildRight (70) - : :- * Project (68) - : : +- * BroadcastHashJoin Inner BuildRight (67) - : : :- * Filter (65) - : : : +- * ColumnarToRow (64) - : : : +- Scan parquet spark_catalog.default.web_sales (63) - : : +- ReusedExchange (66) - : +- ReusedExchange (69) - +- BroadcastExchange (92) - +- * HashAggregate (91) - +- * CometColumnarToRow (90) - +- CometColumnarExchange (89) - +- * HashAggregate (88) - +- * Project (87) - +- * BroadcastHashJoin Inner BuildRight (86) - :- * Project (84) - : +- * BroadcastHashJoin Inner BuildRight (83) - : :- * Filter (81) - : : +- * ColumnarToRow (80) - : : +- Scan parquet spark_catalog.default.web_sales (79) - : +- ReusedExchange (82) - +- ReusedExchange (85) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_addr_sk#1) - -(4) ReusedExchange [Reuses operator id: 101] -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6, d_qoy#7] - -(7) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#8, ca_county#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [ca_address_sk#8, ca_county#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_county#9)) - -(9) CometColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#8, ca_county#9] - -(10) BroadcastExchange -Input [2]: [ca_address_sk#8, ca_county#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_addr_sk#1] -Right keys [1]: [ca_address_sk#8] -Join type: Inner -Join condition: None - -(12) Project [codegen id : 3] -Output [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_address_sk#8, ca_county#9] - -(13) HashAggregate [codegen id : 3] -Input [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] -Keys [3]: [ca_county#9, d_qoy#7, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#10] -Results [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] - -(14) CometColumnarExchange -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] -Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(15) CometColumnarToRow [codegen id : 24] -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] - -(16) HashAggregate [codegen id : 24] -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] -Keys [3]: [ca_county#9, d_qoy#7, d_year#6] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#12] -Results [3]: [ca_county#9, d_year#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS store_sales#13] - -(17) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#16), dynamicpruningexpression(ss_sold_date_sk#16 IN dynamicpruning#17)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 6] -Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] - -(19) Filter [codegen id : 6] -Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] -Condition : isnotnull(ss_addr_sk#14) - -(20) ReusedExchange [Reuses operator id: 105] -Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#16] -Right keys [1]: [d_date_sk#18] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [4]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20] -Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16, d_date_sk#18, d_year#19, d_qoy#20] - -(23) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#21, ca_county#22] - -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#14] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] -Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_address_sk#21, ca_county#22] - -(26) HashAggregate [codegen id : 6] -Input [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] -Keys [3]: [ca_county#22, d_qoy#20, d_year#19] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#15))] -Aggregate Attributes [1]: [sum#23] -Results [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] - -(27) CometColumnarExchange -Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] -Arguments: hashpartitioning(ca_county#22, d_qoy#20, d_year#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometColumnarToRow [codegen id : 7] -Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] - -(29) HashAggregate [codegen id : 7] -Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] -Keys [3]: [ca_county#22, d_qoy#20, d_year#19] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#15))#12] -Results [2]: [ca_county#22, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#15))#12,17,2) AS store_sales#25] - -(30) BroadcastExchange -Input [2]: [ca_county#22, store_sales#25] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#9] -Right keys [1]: [ca_county#22] -Join type: Inner -Join condition: None - -(32) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#29)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(33) ColumnarToRow [codegen id : 10] -Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] - -(34) Filter [codegen id : 10] -Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -Condition : isnotnull(ss_addr_sk#26) - -(35) ReusedExchange [Reuses operator id: 109] -Output [3]: [d_date_sk#30, d_year#31, d_qoy#32] - -(36) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#28] -Right keys [1]: [d_date_sk#30] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 10] -Output [4]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32] -Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#30, d_year#31, d_qoy#32] - -(38) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#33, ca_county#34] - -(39) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_addr_sk#26] -Right keys [1]: [ca_address_sk#33] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 10] -Output [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] -Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_address_sk#33, ca_county#34] - -(41) HashAggregate [codegen id : 10] -Input [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] -Keys [3]: [ca_county#34, d_qoy#32, d_year#31] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum#35] -Results [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] - -(42) CometColumnarExchange -Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] -Arguments: hashpartitioning(ca_county#34, d_qoy#32, d_year#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(43) CometColumnarToRow [codegen id : 11] -Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] - -(44) HashAggregate [codegen id : 11] -Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] -Keys [3]: [ca_county#34, d_qoy#32, d_year#31] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#12] -Results [2]: [ca_county#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#12,17,2) AS store_sales#37] - -(45) BroadcastExchange -Input [2]: [ca_county#34, store_sales#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] - -(46) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#22] -Right keys [1]: [ca_county#34] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 24] -Output [5]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37] -Input [7]: [ca_county#9, d_year#6, store_sales#13, ca_county#22, store_sales#25, ca_county#34, store_sales#37] - -(48) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(49) ColumnarToRow [codegen id : 14] -Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] - -(50) Filter [codegen id : 14] -Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -Condition : isnotnull(ws_bill_addr_sk#38) - -(51) ReusedExchange [Reuses operator id: 101] -Output [3]: [d_date_sk#41, d_year#42, d_qoy#43] - -(52) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#41] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 14] -Output [4]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#42, d_qoy#43] -Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41, d_year#42, d_qoy#43] - -(54) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#44, ca_county#45] - -(55) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#38] -Right keys [1]: [ca_address_sk#44] -Join type: Inner -Join condition: None - -(56) Project [codegen id : 14] -Output [4]: [ws_ext_sales_price#39, d_year#42, d_qoy#43, ca_county#45] -Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#42, d_qoy#43, ca_address_sk#44, ca_county#45] - -(57) HashAggregate [codegen id : 14] -Input [4]: [ws_ext_sales_price#39, d_year#42, d_qoy#43, ca_county#45] -Keys [3]: [ca_county#45, d_qoy#43, d_year#42] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum#46] -Results [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] - -(58) CometColumnarExchange -Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] -Arguments: hashpartitioning(ca_county#45, d_qoy#43, d_year#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(59) CometColumnarToRow [codegen id : 15] -Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] - -(60) HashAggregate [codegen id : 15] -Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] -Keys [3]: [ca_county#45, d_qoy#43, d_year#42] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] -Results [2]: [ca_county#45, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS web_sales#49] - -(61) BroadcastExchange -Input [2]: [ca_county#45, web_sales#49] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(62) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#9] -Right keys [1]: [ca_county#45] -Join type: Inner -Join condition: None - -(63) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#17)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(64) ColumnarToRow [codegen id : 18] -Input [3]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] - -(65) Filter [codegen id : 18] -Input [3]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] -Condition : isnotnull(ws_bill_addr_sk#50) - -(66) ReusedExchange [Reuses operator id: 105] -Output [3]: [d_date_sk#53, d_year#54, d_qoy#55] - -(67) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#52] -Right keys [1]: [d_date_sk#53] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 18] -Output [4]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, d_year#54, d_qoy#55] -Input [6]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52, d_date_sk#53, d_year#54, d_qoy#55] - -(69) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#56, ca_county#57] - -(70) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_bill_addr_sk#50] -Right keys [1]: [ca_address_sk#56] -Join type: Inner -Join condition: None - -(71) Project [codegen id : 18] -Output [4]: [ws_ext_sales_price#51, d_year#54, d_qoy#55, ca_county#57] -Input [6]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, d_year#54, d_qoy#55, ca_address_sk#56, ca_county#57] - -(72) HashAggregate [codegen id : 18] -Input [4]: [ws_ext_sales_price#51, d_year#54, d_qoy#55, ca_county#57] -Keys [3]: [ca_county#57, d_qoy#55, d_year#54] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#51))] -Aggregate Attributes [1]: [sum#58] -Results [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] - -(73) CometColumnarExchange -Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] -Arguments: hashpartitioning(ca_county#57, d_qoy#55, d_year#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(74) CometColumnarToRow [codegen id : 19] -Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] - -(75) HashAggregate [codegen id : 19] -Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] -Keys [3]: [ca_county#57, d_qoy#55, d_year#54] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#51))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#51))#48] -Results [2]: [ca_county#57, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#51))#48,17,2) AS web_sales#60] - -(76) BroadcastExchange -Input [2]: [ca_county#57, web_sales#60] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=10] - -(77) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#45] -Right keys [1]: [ca_county#57] -Join type: Inner -Join condition: (CASE WHEN (web_sales#49 > 0.00) THEN (web_sales#60 / web_sales#49) END > CASE WHEN (store_sales#13 > 0.00) THEN (store_sales#25 / store_sales#13) END) - -(78) Project [codegen id : 24] -Output [8]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#45, web_sales#49, web_sales#60] -Input [9]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#45, web_sales#49, ca_county#57, web_sales#60] - -(79) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#29)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(80) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] - -(81) Filter [codegen id : 22] -Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_bill_addr_sk#61) - -(82) ReusedExchange [Reuses operator id: 109] -Output [3]: [d_date_sk#64, d_year#65, d_qoy#66] - -(83) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#63] -Right keys [1]: [d_date_sk#64] -Join type: Inner -Join condition: None - -(84) Project [codegen id : 22] -Output [4]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#65, d_qoy#66] -Input [6]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63, d_date_sk#64, d_year#65, d_qoy#66] - -(85) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#67, ca_county#68] - -(86) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_bill_addr_sk#61] -Right keys [1]: [ca_address_sk#67] -Join type: Inner -Join condition: None - -(87) Project [codegen id : 22] -Output [4]: [ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_county#68] -Input [6]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_address_sk#67, ca_county#68] - -(88) HashAggregate [codegen id : 22] -Input [4]: [ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_county#68] -Keys [3]: [ca_county#68, d_qoy#66, d_year#65] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#62))] -Aggregate Attributes [1]: [sum#69] -Results [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] - -(89) CometColumnarExchange -Input [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] -Arguments: hashpartitioning(ca_county#68, d_qoy#66, d_year#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(90) CometColumnarToRow [codegen id : 23] -Input [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] - -(91) HashAggregate [codegen id : 23] -Input [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] -Keys [3]: [ca_county#68, d_qoy#66, d_year#65] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#62))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#62))#48] -Results [2]: [ca_county#68, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#62))#48,17,2) AS web_sales#71] - -(92) BroadcastExchange -Input [2]: [ca_county#68, web_sales#71] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=12] - -(93) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#45] -Right keys [1]: [ca_county#68] -Join type: Inner -Join condition: (CASE WHEN (web_sales#60 > 0.00) THEN (web_sales#71 / web_sales#60) END > CASE WHEN (store_sales#25 > 0.00) THEN (store_sales#37 / store_sales#25) END) - -(94) Project [codegen id : 24] -Output [6]: [ca_county#9, d_year#6, (web_sales#60 / web_sales#49) AS web_q1_q2_increase#72, (store_sales#25 / store_sales#13) AS store_q1_q2_increase#73, (web_sales#71 / web_sales#60) AS web_q2_q3_increase#74, (store_sales#37 / store_sales#25) AS store_q2_q3_increase#75] -Input [10]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#45, web_sales#49, web_sales#60, ca_county#68, web_sales#71] - -(95) CometColumnarExchange -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(96) CometSort -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75] -Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75], [ca_county#9 ASC NULLS FIRST] - -(97) CometColumnarToRow [codegen id : 25] -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (101) -+- * CometColumnarToRow (100) - +- CometFilter (99) - +- CometNativeScan parquet spark_catalog.default.date_dim (98) - - -(98) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(99) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) - -(100) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] - -(101) BroadcastExchange -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=14] - -Subquery:2 Hosting operator id = 17 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#17 -BroadcastExchange (105) -+- * CometColumnarToRow (104) - +- CometFilter (103) - +- CometNativeScan parquet spark_catalog.default.date_dim (102) - - -(102) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(103) CometFilter -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] -Condition : ((((isnotnull(d_qoy#20) AND isnotnull(d_year#19)) AND (d_qoy#20 = 2)) AND (d_year#19 = 2000)) AND isnotnull(d_date_sk#18)) - -(104) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] - -(105) BroadcastExchange -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] - -Subquery:3 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 -BroadcastExchange (109) -+- * CometColumnarToRow (108) - +- CometFilter (107) - +- CometNativeScan parquet spark_catalog.default.date_dim (106) - - -(106) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#30, d_year#31, d_qoy#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(107) CometFilter -Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] -Condition : ((((isnotnull(d_qoy#32) AND isnotnull(d_year#31)) AND (d_qoy#32 = 3)) AND (d_year#31 = 2000)) AND isnotnull(d_date_sk#30)) - -(108) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] - -(109) BroadcastExchange -Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] - -Subquery:4 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#4 - -Subquery:5 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#17 - -Subquery:6 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#29 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/simplified.txt deleted file mode 100644 index f3a0572081..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/simplified.txt +++ /dev/null @@ -1,166 +0,0 @@ -WholeStageCodegen (25) - CometColumnarToRow - InputAdapter - CometSort [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] - CometColumnarExchange [ca_county] #1 - WholeStageCodegen (24) - Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] - Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - Project [ca_county,d_year,store_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - BroadcastHashJoin [ca_county,ca_county] - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #2 - WholeStageCodegen (3) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_county] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #6 - WholeStageCodegen (6) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #7 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #9 - WholeStageCodegen (10) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #10 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #12 - WholeStageCodegen (14) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (19) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #14 - WholeStageCodegen (18) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #7 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (23) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #16 - WholeStageCodegen (22) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/explain.txt deleted file mode 100644 index 83d717e295..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,593 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (90) -+- CometSort (89) - +- CometExchange (88) - +- CometProject (87) - +- CometBroadcastHashJoin (86) - :- CometProject (73) - : +- CometBroadcastHashJoin (72) - : :- CometBroadcastHashJoin (59) - : : :- CometProject (46) - : : : +- CometBroadcastHashJoin (45) - : : : :- CometBroadcastHashJoin (30) - : : : : :- CometHashAggregate (15) - : : : : : +- CometExchange (14) - : : : : : +- CometHashAggregate (13) - : : : : : +- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - : : : : +- CometBroadcastExchange (29) - : : : : +- CometHashAggregate (28) - : : : : +- CometExchange (27) - : : : : +- CometHashAggregate (26) - : : : : +- CometProject (25) - : : : : +- CometBroadcastHashJoin (24) - : : : : :- CometProject (22) - : : : : : +- CometBroadcastHashJoin (21) - : : : : : :- CometFilter (17) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (16) - : : : : : +- CometBroadcastExchange (20) - : : : : : +- CometFilter (19) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (18) - : : : : +- ReusedExchange (23) - : : : +- CometBroadcastExchange (44) - : : : +- CometHashAggregate (43) - : : : +- CometExchange (42) - : : : +- CometHashAggregate (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (32) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (31) - : : : : +- CometBroadcastExchange (35) - : : : : +- CometFilter (34) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (33) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (58) - : : +- CometHashAggregate (57) - : : +- CometExchange (56) - : : +- CometHashAggregate (55) - : : +- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometFilter (48) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : : : +- ReusedExchange (49) - : : +- ReusedExchange (52) - : +- CometBroadcastExchange (71) - : +- CometHashAggregate (70) - : +- CometExchange (69) - : +- CometHashAggregate (68) - : +- CometProject (67) - : +- CometBroadcastHashJoin (66) - : :- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometFilter (61) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (60) - : : +- ReusedExchange (62) - : +- ReusedExchange (65) - +- CometBroadcastExchange (85) - +- CometHashAggregate (84) - +- CometExchange (83) - +- CometHashAggregate (82) - +- CometProject (81) - +- CometBroadcastHashJoin (80) - :- CometProject (78) - : +- CometBroadcastHashJoin (77) - : :- CometFilter (75) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (74) - : +- ReusedExchange (76) - +- ReusedExchange (79) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_addr_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) - -(5) CometBroadcastExchange -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [d_date_sk#5, d_year#6, d_qoy#7] - -(6) CometBroadcastHashJoin -Left output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(7) CometProject -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7], [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#8, ca_county#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [ca_address_sk#8, ca_county#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_county#9)) - -(10) CometBroadcastExchange -Input [2]: [ca_address_sk#8, ca_county#9] -Arguments: [ca_address_sk#8, ca_county#9] - -(11) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] -Right output [2]: [ca_address_sk#8, ca_county#9] -Arguments: [ss_addr_sk#1], [ca_address_sk#8], Inner, BuildRight - -(12) CometProject -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_address_sk#8, ca_county#9] -Arguments: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9], [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] - -(13) CometHashAggregate -Input [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] -Keys [3]: [ca_county#9, d_qoy#7, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(14) CometExchange -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10] -Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(15) CometHashAggregate -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10] -Keys [3]: [ca_county#9, d_qoy#7, d_year#6] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(17) CometFilter -Input [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] -Condition : isnotnull(ss_addr_sk#11) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(19) CometFilter -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) - -(20) CometBroadcastExchange -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Arguments: [d_date_sk#15, d_year#16, d_qoy#17] - -(21) CometBroadcastHashJoin -Left output [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] -Right output [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(22) CometProject -Input [6]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_year#16, d_qoy#17] -Arguments: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17], [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17] - -(23) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#18, ca_county#19] - -(24) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17] -Right output [2]: [ca_address_sk#18, ca_county#19] -Arguments: [ss_addr_sk#11], [ca_address_sk#18], Inner, BuildRight - -(25) CometProject -Input [6]: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_address_sk#18, ca_county#19] -Arguments: [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19], [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19] - -(26) CometHashAggregate -Input [4]: [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19] -Keys [3]: [ca_county#19, d_qoy#17, d_year#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#12))] - -(27) CometExchange -Input [4]: [ca_county#19, d_qoy#17, d_year#16, sum#20] -Arguments: hashpartitioning(ca_county#19, d_qoy#17, d_year#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [4]: [ca_county#19, d_qoy#17, d_year#16, sum#20] -Keys [3]: [ca_county#19, d_qoy#17, d_year#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#12))] - -(29) CometBroadcastExchange -Input [2]: [ca_county#19, store_sales#21] -Arguments: [ca_county#19, store_sales#21] - -(30) CometBroadcastHashJoin -Left output [3]: [ca_county#9, d_year#6, store_sales#22] -Right output [2]: [ca_county#19, store_sales#21] -Arguments: [ca_county#9], [ca_county#19], Inner, BuildRight - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_addr_sk#23) - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) - -(35) CometBroadcastExchange -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Arguments: [d_date_sk#27, d_year#28, d_qoy#29] - -(36) CometBroadcastHashJoin -Left output [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] -Right output [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight - -(37) CometProject -Input [6]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_year#28, d_qoy#29] -Arguments: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29], [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29] - -(38) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#30, ca_county#31] - -(39) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29] -Right output [2]: [ca_address_sk#30, ca_county#31] -Arguments: [ss_addr_sk#23], [ca_address_sk#30], Inner, BuildRight - -(40) CometProject -Input [6]: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_address_sk#30, ca_county#31] -Arguments: [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31], [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31] - -(41) CometHashAggregate -Input [4]: [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31] -Keys [3]: [ca_county#31, d_qoy#29, d_year#28] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#24))] - -(42) CometExchange -Input [4]: [ca_county#31, d_qoy#29, d_year#28, sum#32] -Arguments: hashpartitioning(ca_county#31, d_qoy#29, d_year#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(43) CometHashAggregate -Input [4]: [ca_county#31, d_qoy#29, d_year#28, sum#32] -Keys [3]: [ca_county#31, d_qoy#29, d_year#28] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#24))] - -(44) CometBroadcastExchange -Input [2]: [ca_county#31, store_sales#33] -Arguments: [ca_county#31, store_sales#33] - -(45) CometBroadcastHashJoin -Left output [5]: [ca_county#9, d_year#6, store_sales#22, ca_county#19, store_sales#21] -Right output [2]: [ca_county#31, store_sales#33] -Arguments: [ca_county#19], [ca_county#31], Inner, BuildRight - -(46) CometProject -Input [7]: [ca_county#9, d_year#6, store_sales#22, ca_county#19, store_sales#21, ca_county#31, store_sales#33] -Arguments: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33], [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#37)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(48) CometFilter -Input [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_bill_addr_sk#34) - -(49) ReusedExchange [Reuses operator id: 5] -Output [3]: [d_date_sk#38, d_year#39, d_qoy#40] - -(50) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Right output [3]: [d_date_sk#38, d_year#39, d_qoy#40] -Arguments: [ws_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight - -(51) CometProject -Input [6]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#38, d_year#39, d_qoy#40] -Arguments: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40], [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40] - -(52) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#41, ca_county#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40] -Right output [2]: [ca_address_sk#41, ca_county#42] -Arguments: [ws_bill_addr_sk#34], [ca_address_sk#41], Inner, BuildRight - -(54) CometProject -Input [6]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_address_sk#41, ca_county#42] -Arguments: [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42], [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42] - -(55) CometHashAggregate -Input [4]: [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42] -Keys [3]: [ca_county#42, d_qoy#40, d_year#39] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] - -(56) CometExchange -Input [4]: [ca_county#42, d_qoy#40, d_year#39, sum#43] -Arguments: hashpartitioning(ca_county#42, d_qoy#40, d_year#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(57) CometHashAggregate -Input [4]: [ca_county#42, d_qoy#40, d_year#39, sum#43] -Keys [3]: [ca_county#42, d_qoy#40, d_year#39] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] - -(58) CometBroadcastExchange -Input [2]: [ca_county#42, web_sales#44] -Arguments: [ca_county#42, web_sales#44] - -(59) CometBroadcastHashJoin -Left output [5]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33] -Right output [2]: [ca_county#42, web_sales#44] -Arguments: [ca_county#9], [ca_county#42], Inner, BuildRight - -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#48)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(61) CometFilter -Input [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] -Condition : isnotnull(ws_bill_addr_sk#45) - -(62) ReusedExchange [Reuses operator id: 20] -Output [3]: [d_date_sk#49, d_year#50, d_qoy#51] - -(63) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] -Right output [3]: [d_date_sk#49, d_year#50, d_qoy#51] -Arguments: [ws_sold_date_sk#47], [d_date_sk#49], Inner, BuildRight - -(64) CometProject -Input [6]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47, d_date_sk#49, d_year#50, d_qoy#51] -Arguments: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51], [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51] - -(65) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#52, ca_county#53] - -(66) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51] -Right output [2]: [ca_address_sk#52, ca_county#53] -Arguments: [ws_bill_addr_sk#45], [ca_address_sk#52], Inner, BuildRight - -(67) CometProject -Input [6]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_address_sk#52, ca_county#53] -Arguments: [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53], [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53] - -(68) CometHashAggregate -Input [4]: [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53] -Keys [3]: [ca_county#53, d_qoy#51, d_year#50] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#46))] - -(69) CometExchange -Input [4]: [ca_county#53, d_qoy#51, d_year#50, sum#54] -Arguments: hashpartitioning(ca_county#53, d_qoy#51, d_year#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(70) CometHashAggregate -Input [4]: [ca_county#53, d_qoy#51, d_year#50, sum#54] -Keys [3]: [ca_county#53, d_qoy#51, d_year#50] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#46))] - -(71) CometBroadcastExchange -Input [2]: [ca_county#53, web_sales#55] -Arguments: [ca_county#53, web_sales#55] - -(72) CometBroadcastHashJoin -Left output [7]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44] -Right output [2]: [ca_county#53, web_sales#55] -Arguments: [ca_county#42], [ca_county#53], Inner, (CASE WHEN (web_sales#44 > 0.00) THEN (web_sales#55 / web_sales#44) END > CASE WHEN (store_sales#22 > 0.00) THEN (store_sales#21 / store_sales#22) END), BuildRight - -(73) CometProject -Input [9]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, ca_county#53, web_sales#55] -Arguments: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55], [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55] - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(75) CometFilter -Input [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] -Condition : isnotnull(ws_bill_addr_sk#56) - -(76) ReusedExchange [Reuses operator id: 35] -Output [3]: [d_date_sk#60, d_year#61, d_qoy#62] - -(77) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] -Right output [3]: [d_date_sk#60, d_year#61, d_qoy#62] -Arguments: [ws_sold_date_sk#58], [d_date_sk#60], Inner, BuildRight - -(78) CometProject -Input [6]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58, d_date_sk#60, d_year#61, d_qoy#62] -Arguments: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62], [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62] - -(79) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#63, ca_county#64] - -(80) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62] -Right output [2]: [ca_address_sk#63, ca_county#64] -Arguments: [ws_bill_addr_sk#56], [ca_address_sk#63], Inner, BuildRight - -(81) CometProject -Input [6]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_address_sk#63, ca_county#64] -Arguments: [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64], [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64] - -(82) CometHashAggregate -Input [4]: [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64] -Keys [3]: [ca_county#64, d_qoy#62, d_year#61] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#57))] - -(83) CometExchange -Input [4]: [ca_county#64, d_qoy#62, d_year#61, sum#65] -Arguments: hashpartitioning(ca_county#64, d_qoy#62, d_year#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(84) CometHashAggregate -Input [4]: [ca_county#64, d_qoy#62, d_year#61, sum#65] -Keys [3]: [ca_county#64, d_qoy#62, d_year#61] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#57))] - -(85) CometBroadcastExchange -Input [2]: [ca_county#64, web_sales#66] -Arguments: [ca_county#64, web_sales#66] - -(86) CometBroadcastHashJoin -Left output [8]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55] -Right output [2]: [ca_county#64, web_sales#66] -Arguments: [ca_county#42], [ca_county#64], Inner, (CASE WHEN (web_sales#55 > 0.00) THEN (web_sales#66 / web_sales#55) END > CASE WHEN (store_sales#21 > 0.00) THEN (store_sales#33 / store_sales#21) END), BuildRight - -(87) CometProject -Input [10]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55, ca_county#64, web_sales#66] -Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9, d_year#6, (web_sales#55 / web_sales#44) AS web_q1_q2_increase#67, (store_sales#21 / store_sales#22) AS store_q1_q2_increase#68, (web_sales#66 / web_sales#55) AS web_q2_q3_increase#69, (store_sales#33 / store_sales#21) AS store_q2_q3_increase#70] - -(88) CometExchange -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(89) CometSort -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] -Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9 ASC NULLS FIRST] - -(90) CometColumnarToRow [codegen id : 1] -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (94) -+- * CometColumnarToRow (93) - +- CometFilter (92) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (91) - - -(91) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(92) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) - -(93) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] - -(94) BroadcastExchange -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (98) -+- * CometColumnarToRow (97) - +- CometFilter (96) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (95) - - -(95) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(96) CometFilter -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) - -(97) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] - -(98) BroadcastExchange -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -Subquery:3 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 -BroadcastExchange (102) -+- * CometColumnarToRow (101) - +- CometFilter (100) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (99) - - -(99) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(100) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) - -(101) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] - -(102) BroadcastExchange -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:4 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#4 - -Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#14 - -Subquery:6 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#26 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/simplified.txt deleted file mode 100644 index 2f97384320..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,116 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] - CometExchange [ca_county] #1 - CometProject [web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] - CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales,ca_county,web_sales] - CometProject [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,ca_county,web_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales] - CometProject [ca_county,d_year,store_sales,store_sales,store_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales,ca_county,store_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales] - CometHashAggregate [d_qoy,sum] [ca_county,d_year,store_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #2 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [ca_address_sk,ca_county] #5 - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange [ca_county,store_sales] #6 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,store_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #7 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,store_sales] #10 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,store_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #11 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,web_sales] #14 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #15 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_year,d_qoy] #4 - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,web_sales] #16 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #17 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [d_date_sk,d_year,d_qoy] #9 - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,web_sales] #18 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #19 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [d_date_sk,d_year,d_qoy] #13 - ReusedExchange [ca_address_sk,ca_county] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/explain.txt deleted file mode 100644 index 83d717e295..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/explain.txt +++ /dev/null @@ -1,593 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (90) -+- CometSort (89) - +- CometExchange (88) - +- CometProject (87) - +- CometBroadcastHashJoin (86) - :- CometProject (73) - : +- CometBroadcastHashJoin (72) - : :- CometBroadcastHashJoin (59) - : : :- CometProject (46) - : : : +- CometBroadcastHashJoin (45) - : : : :- CometBroadcastHashJoin (30) - : : : : :- CometHashAggregate (15) - : : : : : +- CometExchange (14) - : : : : : +- CometHashAggregate (13) - : : : : : +- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - : : : : +- CometBroadcastExchange (29) - : : : : +- CometHashAggregate (28) - : : : : +- CometExchange (27) - : : : : +- CometHashAggregate (26) - : : : : +- CometProject (25) - : : : : +- CometBroadcastHashJoin (24) - : : : : :- CometProject (22) - : : : : : +- CometBroadcastHashJoin (21) - : : : : : :- CometFilter (17) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (16) - : : : : : +- CometBroadcastExchange (20) - : : : : : +- CometFilter (19) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (18) - : : : : +- ReusedExchange (23) - : : : +- CometBroadcastExchange (44) - : : : +- CometHashAggregate (43) - : : : +- CometExchange (42) - : : : +- CometHashAggregate (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (32) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (31) - : : : : +- CometBroadcastExchange (35) - : : : : +- CometFilter (34) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (33) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (58) - : : +- CometHashAggregate (57) - : : +- CometExchange (56) - : : +- CometHashAggregate (55) - : : +- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometFilter (48) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : : : +- ReusedExchange (49) - : : +- ReusedExchange (52) - : +- CometBroadcastExchange (71) - : +- CometHashAggregate (70) - : +- CometExchange (69) - : +- CometHashAggregate (68) - : +- CometProject (67) - : +- CometBroadcastHashJoin (66) - : :- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometFilter (61) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (60) - : : +- ReusedExchange (62) - : +- ReusedExchange (65) - +- CometBroadcastExchange (85) - +- CometHashAggregate (84) - +- CometExchange (83) - +- CometHashAggregate (82) - +- CometProject (81) - +- CometBroadcastHashJoin (80) - :- CometProject (78) - : +- CometBroadcastHashJoin (77) - : :- CometFilter (75) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (74) - : +- ReusedExchange (76) - +- ReusedExchange (79) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_addr_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) - -(5) CometBroadcastExchange -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [d_date_sk#5, d_year#6, d_qoy#7] - -(6) CometBroadcastHashJoin -Left output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(7) CometProject -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7], [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#8, ca_county#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [ca_address_sk#8, ca_county#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_county#9)) - -(10) CometBroadcastExchange -Input [2]: [ca_address_sk#8, ca_county#9] -Arguments: [ca_address_sk#8, ca_county#9] - -(11) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] -Right output [2]: [ca_address_sk#8, ca_county#9] -Arguments: [ss_addr_sk#1], [ca_address_sk#8], Inner, BuildRight - -(12) CometProject -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_address_sk#8, ca_county#9] -Arguments: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9], [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] - -(13) CometHashAggregate -Input [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] -Keys [3]: [ca_county#9, d_qoy#7, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(14) CometExchange -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10] -Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(15) CometHashAggregate -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10] -Keys [3]: [ca_county#9, d_qoy#7, d_year#6] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(17) CometFilter -Input [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] -Condition : isnotnull(ss_addr_sk#11) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(19) CometFilter -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) - -(20) CometBroadcastExchange -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Arguments: [d_date_sk#15, d_year#16, d_qoy#17] - -(21) CometBroadcastHashJoin -Left output [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] -Right output [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(22) CometProject -Input [6]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_year#16, d_qoy#17] -Arguments: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17], [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17] - -(23) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#18, ca_county#19] - -(24) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17] -Right output [2]: [ca_address_sk#18, ca_county#19] -Arguments: [ss_addr_sk#11], [ca_address_sk#18], Inner, BuildRight - -(25) CometProject -Input [6]: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_address_sk#18, ca_county#19] -Arguments: [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19], [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19] - -(26) CometHashAggregate -Input [4]: [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19] -Keys [3]: [ca_county#19, d_qoy#17, d_year#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#12))] - -(27) CometExchange -Input [4]: [ca_county#19, d_qoy#17, d_year#16, sum#20] -Arguments: hashpartitioning(ca_county#19, d_qoy#17, d_year#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [4]: [ca_county#19, d_qoy#17, d_year#16, sum#20] -Keys [3]: [ca_county#19, d_qoy#17, d_year#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#12))] - -(29) CometBroadcastExchange -Input [2]: [ca_county#19, store_sales#21] -Arguments: [ca_county#19, store_sales#21] - -(30) CometBroadcastHashJoin -Left output [3]: [ca_county#9, d_year#6, store_sales#22] -Right output [2]: [ca_county#19, store_sales#21] -Arguments: [ca_county#9], [ca_county#19], Inner, BuildRight - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_addr_sk#23) - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) - -(35) CometBroadcastExchange -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Arguments: [d_date_sk#27, d_year#28, d_qoy#29] - -(36) CometBroadcastHashJoin -Left output [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] -Right output [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight - -(37) CometProject -Input [6]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_year#28, d_qoy#29] -Arguments: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29], [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29] - -(38) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#30, ca_county#31] - -(39) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29] -Right output [2]: [ca_address_sk#30, ca_county#31] -Arguments: [ss_addr_sk#23], [ca_address_sk#30], Inner, BuildRight - -(40) CometProject -Input [6]: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_address_sk#30, ca_county#31] -Arguments: [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31], [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31] - -(41) CometHashAggregate -Input [4]: [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31] -Keys [3]: [ca_county#31, d_qoy#29, d_year#28] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#24))] - -(42) CometExchange -Input [4]: [ca_county#31, d_qoy#29, d_year#28, sum#32] -Arguments: hashpartitioning(ca_county#31, d_qoy#29, d_year#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(43) CometHashAggregate -Input [4]: [ca_county#31, d_qoy#29, d_year#28, sum#32] -Keys [3]: [ca_county#31, d_qoy#29, d_year#28] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#24))] - -(44) CometBroadcastExchange -Input [2]: [ca_county#31, store_sales#33] -Arguments: [ca_county#31, store_sales#33] - -(45) CometBroadcastHashJoin -Left output [5]: [ca_county#9, d_year#6, store_sales#22, ca_county#19, store_sales#21] -Right output [2]: [ca_county#31, store_sales#33] -Arguments: [ca_county#19], [ca_county#31], Inner, BuildRight - -(46) CometProject -Input [7]: [ca_county#9, d_year#6, store_sales#22, ca_county#19, store_sales#21, ca_county#31, store_sales#33] -Arguments: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33], [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#37)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(48) CometFilter -Input [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_bill_addr_sk#34) - -(49) ReusedExchange [Reuses operator id: 5] -Output [3]: [d_date_sk#38, d_year#39, d_qoy#40] - -(50) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Right output [3]: [d_date_sk#38, d_year#39, d_qoy#40] -Arguments: [ws_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight - -(51) CometProject -Input [6]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#38, d_year#39, d_qoy#40] -Arguments: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40], [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40] - -(52) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#41, ca_county#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40] -Right output [2]: [ca_address_sk#41, ca_county#42] -Arguments: [ws_bill_addr_sk#34], [ca_address_sk#41], Inner, BuildRight - -(54) CometProject -Input [6]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_address_sk#41, ca_county#42] -Arguments: [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42], [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42] - -(55) CometHashAggregate -Input [4]: [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42] -Keys [3]: [ca_county#42, d_qoy#40, d_year#39] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] - -(56) CometExchange -Input [4]: [ca_county#42, d_qoy#40, d_year#39, sum#43] -Arguments: hashpartitioning(ca_county#42, d_qoy#40, d_year#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(57) CometHashAggregate -Input [4]: [ca_county#42, d_qoy#40, d_year#39, sum#43] -Keys [3]: [ca_county#42, d_qoy#40, d_year#39] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] - -(58) CometBroadcastExchange -Input [2]: [ca_county#42, web_sales#44] -Arguments: [ca_county#42, web_sales#44] - -(59) CometBroadcastHashJoin -Left output [5]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33] -Right output [2]: [ca_county#42, web_sales#44] -Arguments: [ca_county#9], [ca_county#42], Inner, BuildRight - -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#48)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(61) CometFilter -Input [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] -Condition : isnotnull(ws_bill_addr_sk#45) - -(62) ReusedExchange [Reuses operator id: 20] -Output [3]: [d_date_sk#49, d_year#50, d_qoy#51] - -(63) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] -Right output [3]: [d_date_sk#49, d_year#50, d_qoy#51] -Arguments: [ws_sold_date_sk#47], [d_date_sk#49], Inner, BuildRight - -(64) CometProject -Input [6]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47, d_date_sk#49, d_year#50, d_qoy#51] -Arguments: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51], [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51] - -(65) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#52, ca_county#53] - -(66) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51] -Right output [2]: [ca_address_sk#52, ca_county#53] -Arguments: [ws_bill_addr_sk#45], [ca_address_sk#52], Inner, BuildRight - -(67) CometProject -Input [6]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_address_sk#52, ca_county#53] -Arguments: [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53], [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53] - -(68) CometHashAggregate -Input [4]: [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53] -Keys [3]: [ca_county#53, d_qoy#51, d_year#50] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#46))] - -(69) CometExchange -Input [4]: [ca_county#53, d_qoy#51, d_year#50, sum#54] -Arguments: hashpartitioning(ca_county#53, d_qoy#51, d_year#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(70) CometHashAggregate -Input [4]: [ca_county#53, d_qoy#51, d_year#50, sum#54] -Keys [3]: [ca_county#53, d_qoy#51, d_year#50] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#46))] - -(71) CometBroadcastExchange -Input [2]: [ca_county#53, web_sales#55] -Arguments: [ca_county#53, web_sales#55] - -(72) CometBroadcastHashJoin -Left output [7]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44] -Right output [2]: [ca_county#53, web_sales#55] -Arguments: [ca_county#42], [ca_county#53], Inner, (CASE WHEN (web_sales#44 > 0.00) THEN (web_sales#55 / web_sales#44) END > CASE WHEN (store_sales#22 > 0.00) THEN (store_sales#21 / store_sales#22) END), BuildRight - -(73) CometProject -Input [9]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, ca_county#53, web_sales#55] -Arguments: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55], [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55] - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(75) CometFilter -Input [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] -Condition : isnotnull(ws_bill_addr_sk#56) - -(76) ReusedExchange [Reuses operator id: 35] -Output [3]: [d_date_sk#60, d_year#61, d_qoy#62] - -(77) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] -Right output [3]: [d_date_sk#60, d_year#61, d_qoy#62] -Arguments: [ws_sold_date_sk#58], [d_date_sk#60], Inner, BuildRight - -(78) CometProject -Input [6]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58, d_date_sk#60, d_year#61, d_qoy#62] -Arguments: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62], [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62] - -(79) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#63, ca_county#64] - -(80) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62] -Right output [2]: [ca_address_sk#63, ca_county#64] -Arguments: [ws_bill_addr_sk#56], [ca_address_sk#63], Inner, BuildRight - -(81) CometProject -Input [6]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_address_sk#63, ca_county#64] -Arguments: [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64], [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64] - -(82) CometHashAggregate -Input [4]: [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64] -Keys [3]: [ca_county#64, d_qoy#62, d_year#61] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#57))] - -(83) CometExchange -Input [4]: [ca_county#64, d_qoy#62, d_year#61, sum#65] -Arguments: hashpartitioning(ca_county#64, d_qoy#62, d_year#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(84) CometHashAggregate -Input [4]: [ca_county#64, d_qoy#62, d_year#61, sum#65] -Keys [3]: [ca_county#64, d_qoy#62, d_year#61] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#57))] - -(85) CometBroadcastExchange -Input [2]: [ca_county#64, web_sales#66] -Arguments: [ca_county#64, web_sales#66] - -(86) CometBroadcastHashJoin -Left output [8]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55] -Right output [2]: [ca_county#64, web_sales#66] -Arguments: [ca_county#42], [ca_county#64], Inner, (CASE WHEN (web_sales#55 > 0.00) THEN (web_sales#66 / web_sales#55) END > CASE WHEN (store_sales#21 > 0.00) THEN (store_sales#33 / store_sales#21) END), BuildRight - -(87) CometProject -Input [10]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55, ca_county#64, web_sales#66] -Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9, d_year#6, (web_sales#55 / web_sales#44) AS web_q1_q2_increase#67, (store_sales#21 / store_sales#22) AS store_q1_q2_increase#68, (web_sales#66 / web_sales#55) AS web_q2_q3_increase#69, (store_sales#33 / store_sales#21) AS store_q2_q3_increase#70] - -(88) CometExchange -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(89) CometSort -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] -Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9 ASC NULLS FIRST] - -(90) CometColumnarToRow [codegen id : 1] -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (94) -+- * CometColumnarToRow (93) - +- CometFilter (92) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (91) - - -(91) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(92) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) - -(93) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] - -(94) BroadcastExchange -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (98) -+- * CometColumnarToRow (97) - +- CometFilter (96) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (95) - - -(95) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(96) CometFilter -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) - -(97) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] - -(98) BroadcastExchange -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -Subquery:3 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 -BroadcastExchange (102) -+- * CometColumnarToRow (101) - +- CometFilter (100) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (99) - - -(99) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(100) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) - -(101) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] - -(102) BroadcastExchange -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:4 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#4 - -Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#14 - -Subquery:6 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#26 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/extended.txt deleted file mode 100644 index b4e7aa4db6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/extended.txt +++ /dev/null @@ -1,126 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : : +- CometBroadcastExchange - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 111 out of 120 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 2f97384320..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt +++ /dev/null @@ -1,116 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] - CometExchange [ca_county] #1 - CometProject [web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] - CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales,ca_county,web_sales] - CometProject [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,ca_county,web_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales] - CometProject [ca_county,d_year,store_sales,store_sales,store_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales,ca_county,store_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales] - CometHashAggregate [d_qoy,sum] [ca_county,d_year,store_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #2 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [ca_address_sk,ca_county] #5 - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange [ca_county,store_sales] #6 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,store_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #7 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,store_sales] #10 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,store_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #11 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,web_sales] #14 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #15 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_year,d_qoy] #4 - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,web_sales] #16 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #17 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [d_date_sk,d_year,d_qoy] #9 - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,web_sales] #18 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #19 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [d_date_sk,d_year,d_qoy] #13 - ReusedExchange [ca_address_sk,ca_county] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/explain.txt deleted file mode 100644 index df3987b0a0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -* HashAggregate (31) -+- * CometColumnarToRow (30) - +- CometColumnarExchange (29) - +- * HashAggregate (28) - +- * Project (27) - +- * BroadcastHashJoin Inner BuildRight (26) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : +- BroadcastExchange (8) - : : +- * CometColumnarToRow (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometNativeScan parquet spark_catalog.default.item (4) - : +- BroadcastExchange (22) - : +- * Filter (21) - : +- * HashAggregate (20) - : +- * CometColumnarToRow (19) - : +- CometColumnarExchange (18) - : +- * HashAggregate (17) - : +- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet spark_catalog.default.catalog_sales (11) - : +- ReusedExchange (14) - +- ReusedExchange (25) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_ext_discount_amt)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] - -(3) Filter [codegen id : 6] -Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Condition : (isnotnull(cs_item_sk#1) AND isnotnull(cs_ext_discount_amt#2)) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_manufact_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,977), IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [i_item_sk#5, i_manufact_id#6] -Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 977)) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [2]: [i_item_sk#5, i_manufact_id#6] -Arguments: [i_item_sk#5], [i_item_sk#5] - -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [i_item_sk#5] - -(8) BroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 6] -Output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] -Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] - -(11) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] - -(13) Filter [codegen id : 3] -Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Condition : isnotnull(cs_item_sk#7) - -(14) ReusedExchange [Reuses operator id: 36] -Output [1]: [d_date_sk#10] - -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(16) Project [codegen id : 3] -Output [2]: [cs_item_sk#7, cs_ext_discount_amt#8] -Input [4]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9, d_date_sk#10] - -(17) HashAggregate [codegen id : 3] -Input [2]: [cs_item_sk#7, cs_ext_discount_amt#8] -Keys [1]: [cs_item_sk#7] -Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#8))] -Aggregate Attributes [2]: [sum#11, count#12] -Results [3]: [cs_item_sk#7, sum#13, count#14] - -(18) CometColumnarExchange -Input [3]: [cs_item_sk#7, sum#13, count#14] -Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(19) CometColumnarToRow [codegen id : 4] -Input [3]: [cs_item_sk#7, sum#13, count#14] - -(20) HashAggregate [codegen id : 4] -Input [3]: [cs_item_sk#7, sum#13, count#14] -Keys [1]: [cs_item_sk#7] -Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))] -Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))#15] -Results [2]: [(1.3 * cast((avg(UnscaledValue(cs_ext_discount_amt#8))#15 / 100.0) as decimal(11,6))) AS (1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] - -(21) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] -Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#16) - -(22) BroadcastExchange -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] - -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [i_item_sk#5] -Right keys [1]: [cs_item_sk#7] -Join type: Inner -Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#16) - -(24) Project [codegen id : 6] -Output [2]: [cs_ext_discount_amt#2, cs_sold_date_sk#3] -Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] - -(25) ReusedExchange [Reuses operator id: 36] -Output [1]: [d_date_sk#17] - -(26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 6] -Output [1]: [cs_ext_discount_amt#2] -Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#17] - -(28) HashAggregate [codegen id : 6] -Input [1]: [cs_ext_discount_amt#2] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#18] -Results [1]: [sum#19] - -(29) CometColumnarExchange -Input [1]: [sum#19] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 7] -Input [1]: [sum#19] - -(31) HashAggregate [codegen id : 7] -Input [1]: [sum#19] -Keys: [] -Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))#20] -Results [1]: [MakeDecimal(sum(UnscaledValue(cs_ext_discount_amt#2))#20,17,2) AS excess discount amount#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.date_dim (32) - - -(32) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#17, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 2000-01-27)) AND (d_date#22 <= 2000-04-26)) AND isnotnull(d_date_sk#17)) - -(34) CometProject -Input [2]: [d_date_sk#17, d_date#22] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] - -(36) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/simplified.txt deleted file mode 100644 index 14c21b32a5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -WholeStageCodegen (7) - HashAggregate [sum] [sum(UnscaledValue(cs_ext_discount_amt)),excess discount amount,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (6) - HashAggregate [cs_ext_discount_amt] [sum,sum] - Project [cs_ext_discount_amt] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_discount_amt,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk,cs_ext_discount_amt,(1.3 * avg(cs_ext_discount_amt))] - Project [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk,cs_ext_discount_amt] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Filter [(1.3 * avg(cs_ext_discount_amt))] - HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_item_sk] #5 - WholeStageCodegen (3) - HashAggregate [cs_item_sk,cs_ext_discount_amt] [sum,count,sum,count] - Project [cs_item_sk,cs_ext_discount_amt] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/explain.txt deleted file mode 100644 index d6c10008ec..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (30) -+- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : +- CometBroadcastExchange (21) - : +- CometFilter (20) - : +- CometHashAggregate (19) - : +- CometExchange (18) - : +- CometHashAggregate (17) - : +- CometProject (16) - : +- CometBroadcastHashJoin (15) - : :- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : +- CometBroadcastExchange (14) - : +- CometProject (13) - : +- CometFilter (12) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (11) - +- ReusedExchange (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_ext_discount_amt)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Condition : (isnotnull(cs_item_sk#1) AND isnotnull(cs_ext_discount_amt#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_manufact_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,977), IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_manufact_id#6] -Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 977)) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [2]: [i_item_sk#5, i_manufact_id#6] -Arguments: [i_item_sk#5], [i_item_sk#5] - -(6) CometBroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: [i_item_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Right output [1]: [i_item_sk#5] -Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] -Arguments: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5], [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Condition : isnotnull(cs_item_sk#7) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) - -(13) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(15) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(16) CometProject -Input [4]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9, d_date_sk#11] -Arguments: [cs_item_sk#7, cs_ext_discount_amt#8], [cs_item_sk#7, cs_ext_discount_amt#8] - -(17) CometHashAggregate -Input [2]: [cs_item_sk#7, cs_ext_discount_amt#8] -Keys [1]: [cs_item_sk#7] -Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#8))] - -(18) CometExchange -Input [3]: [cs_item_sk#7, sum#13, count#14] -Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [3]: [cs_item_sk#7, sum#13, count#14] -Keys [1]: [cs_item_sk#7] -Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))] - -(20) CometFilter -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#15) - -(21) CometBroadcastExchange -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Arguments: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] - -(22) CometBroadcastHashJoin -Left output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] -Right output [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Arguments: [i_item_sk#5], [cs_item_sk#7], Inner, (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#15), BuildRight - -(23) CometProject -Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Arguments: [cs_ext_discount_amt#2, cs_sold_date_sk#3], [cs_ext_discount_amt#2, cs_sold_date_sk#3] - -(24) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#16] - -(25) CometBroadcastHashJoin -Left output [2]: [cs_ext_discount_amt#2, cs_sold_date_sk#3] -Right output [1]: [d_date_sk#16] -Arguments: [cs_sold_date_sk#3], [d_date_sk#16], Inner, BuildRight - -(26) CometProject -Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#16] -Arguments: [cs_ext_discount_amt#2], [cs_ext_discount_amt#2] - -(27) CometHashAggregate -Input [1]: [cs_ext_discount_amt#2] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))] - -(28) CometExchange -Input [1]: [sum#17] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [1]: [sum#17] -Keys: [] -Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [excess discount amount#18] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#16, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#19 <= 2000-04-26)) AND isnotnull(d_date_sk#16)) - -(33) CometProject -Input [2]: [d_date_sk#16, d_date#19] -Arguments: [d_date_sk#16], [d_date_sk#16] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#16] - -(35) BroadcastExchange -Input [1]: [d_date_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/simplified.txt deleted file mode 100644 index 07619fc999..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [excess discount amount,sum(UnscaledValue(cs_ext_discount_amt))] - CometExchange #1 - CometHashAggregate [cs_ext_discount_amt] [sum] - CometProject [cs_ext_discount_amt] - CometBroadcastHashJoin [cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] - CometProject [cs_ext_discount_amt,cs_sold_date_sk] - CometBroadcastHashJoin [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk,(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] - CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk] #3 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] #4 - CometFilter [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] - CometHashAggregate [sum,count] [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,avg(UnscaledValue(cs_ext_discount_amt))] - CometExchange [cs_item_sk] #5 - CometHashAggregate [cs_ext_discount_amt] [cs_item_sk,sum,count] - CometProject [cs_item_sk,cs_ext_discount_amt] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/explain.txt deleted file mode 100644 index d6c10008ec..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (30) -+- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : +- CometBroadcastExchange (21) - : +- CometFilter (20) - : +- CometHashAggregate (19) - : +- CometExchange (18) - : +- CometHashAggregate (17) - : +- CometProject (16) - : +- CometBroadcastHashJoin (15) - : :- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : +- CometBroadcastExchange (14) - : +- CometProject (13) - : +- CometFilter (12) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (11) - +- ReusedExchange (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_ext_discount_amt)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Condition : (isnotnull(cs_item_sk#1) AND isnotnull(cs_ext_discount_amt#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_manufact_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,977), IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_manufact_id#6] -Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 977)) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [2]: [i_item_sk#5, i_manufact_id#6] -Arguments: [i_item_sk#5], [i_item_sk#5] - -(6) CometBroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: [i_item_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Right output [1]: [i_item_sk#5] -Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] -Arguments: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5], [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Condition : isnotnull(cs_item_sk#7) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) - -(13) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(15) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(16) CometProject -Input [4]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9, d_date_sk#11] -Arguments: [cs_item_sk#7, cs_ext_discount_amt#8], [cs_item_sk#7, cs_ext_discount_amt#8] - -(17) CometHashAggregate -Input [2]: [cs_item_sk#7, cs_ext_discount_amt#8] -Keys [1]: [cs_item_sk#7] -Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#8))] - -(18) CometExchange -Input [3]: [cs_item_sk#7, sum#13, count#14] -Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [3]: [cs_item_sk#7, sum#13, count#14] -Keys [1]: [cs_item_sk#7] -Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))] - -(20) CometFilter -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#15) - -(21) CometBroadcastExchange -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Arguments: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] - -(22) CometBroadcastHashJoin -Left output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] -Right output [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Arguments: [i_item_sk#5], [cs_item_sk#7], Inner, (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#15), BuildRight - -(23) CometProject -Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Arguments: [cs_ext_discount_amt#2, cs_sold_date_sk#3], [cs_ext_discount_amt#2, cs_sold_date_sk#3] - -(24) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#16] - -(25) CometBroadcastHashJoin -Left output [2]: [cs_ext_discount_amt#2, cs_sold_date_sk#3] -Right output [1]: [d_date_sk#16] -Arguments: [cs_sold_date_sk#3], [d_date_sk#16], Inner, BuildRight - -(26) CometProject -Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#16] -Arguments: [cs_ext_discount_amt#2], [cs_ext_discount_amt#2] - -(27) CometHashAggregate -Input [1]: [cs_ext_discount_amt#2] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))] - -(28) CometExchange -Input [1]: [sum#17] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [1]: [sum#17] -Keys: [] -Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [excess discount amount#18] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#16, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#19 <= 2000-04-26)) AND isnotnull(d_date_sk#16)) - -(33) CometProject -Input [2]: [d_date_sk#16, d_date#19] -Arguments: [d_date_sk#16], [d_date_sk#16] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#16] - -(35) BroadcastExchange -Input [1]: [d_date_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/extended.txt deleted file mode 100644 index 943bb147ff..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 07619fc999..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [excess discount amount,sum(UnscaledValue(cs_ext_discount_amt))] - CometExchange #1 - CometHashAggregate [cs_ext_discount_amt] [sum] - CometProject [cs_ext_discount_amt] - CometBroadcastHashJoin [cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] - CometProject [cs_ext_discount_amt,cs_sold_date_sk] - CometBroadcastHashJoin [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk,(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] - CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk] #3 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] #4 - CometFilter [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] - CometHashAggregate [sum,count] [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,avg(UnscaledValue(cs_ext_discount_amt))] - CometExchange [cs_item_sk] #5 - CometHashAggregate [cs_ext_discount_amt] [cs_item_sk,sum,count] - CometProject [cs_item_sk,cs_ext_discount_amt] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/explain.txt deleted file mode 100644 index a4e15583d4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/explain.txt +++ /dev/null @@ -1,416 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (66) -+- * HashAggregate (65) - +- * CometColumnarToRow (64) - +- CometColumnarExchange (63) - +- * HashAggregate (62) - +- Union (61) - :- * HashAggregate (28) - : +- * CometColumnarToRow (27) - : +- CometColumnarExchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (22) - : +- * CometColumnarToRow (21) - : +- CometBroadcastHashJoin (20) - : :- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.item (14) - : +- CometBroadcastExchange (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometNativeScan parquet spark_catalog.default.item (16) - :- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- CometColumnarExchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (34) - : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : :- * Filter (31) - : : : : +- * ColumnarToRow (30) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) - +- * HashAggregate (60) - +- * CometColumnarToRow (59) - +- CometColumnarExchange (58) - +- * HashAggregate (57) - +- * Project (56) - +- * BroadcastHashJoin Inner BuildRight (55) - :- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) - : :- * Project (50) - : : +- * BroadcastHashJoin Inner BuildRight (49) - : : :- * Filter (47) - : : : +- * ColumnarToRow (46) - : : : +- Scan parquet spark_catalog.default.web_sales (45) - : : +- ReusedExchange (48) - : +- ReusedExchange (51) - +- ReusedExchange (54) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(4) ReusedExchange [Reuses operator id: 71] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_gmt_offset#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isnotnull(ca_address_sk#7)) - -(9) CometProject -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Arguments: [ca_address_sk#7], [ca_address_sk#7] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [ca_address_sk#7] - -(11) BroadcastExchange -Input [1]: [ca_address_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] - -(14) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#9, i_manufact_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [i_item_sk#9, i_manufact_id#10] -Condition : isnotnull(i_item_sk#9) - -(16) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_category#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [i_category#11, i_manufact_id#12] -Condition : (isnotnull(i_category#11) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#11, 50)) = Electronics )) - -(18) CometProject -Input [2]: [i_category#11, i_manufact_id#12] -Arguments: [i_manufact_id#12], [i_manufact_id#12] - -(19) CometBroadcastExchange -Input [1]: [i_manufact_id#12] -Arguments: [i_manufact_id#12] - -(20) CometBroadcastHashJoin -Left output [2]: [i_item_sk#9, i_manufact_id#10] -Right output [1]: [i_manufact_id#12] -Arguments: [i_manufact_id#10], [i_manufact_id#12], LeftSemi, BuildRight - -(21) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#9, i_manufact_id#10] - -(22) BroadcastExchange -Input [2]: [i_item_sk#9, i_manufact_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#9] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 4] -Output [2]: [ss_ext_sales_price#3, i_manufact_id#10] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_manufact_id#10] - -(25) HashAggregate [codegen id : 4] -Input [2]: [ss_ext_sales_price#3, i_manufact_id#10] -Keys [1]: [i_manufact_id#10] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum#13] -Results [2]: [i_manufact_id#10, sum#14] - -(26) CometColumnarExchange -Input [2]: [i_manufact_id#10, sum#14] -Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(27) CometColumnarToRow [codegen id : 5] -Input [2]: [i_manufact_id#10, sum#14] - -(28) HashAggregate [codegen id : 5] -Input [2]: [i_manufact_id#10, sum#14] -Keys [1]: [i_manufact_id#10] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#15] -Results [2]: [i_manufact_id#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#15,17,2) AS total_sales#16] - -(29) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(30) ColumnarToRow [codegen id : 9] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] - -(31) Filter [codegen id : 9] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) - -(32) ReusedExchange [Reuses operator id: 71] -Output [1]: [d_date_sk#21] - -(33) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#20] -Right keys [1]: [d_date_sk#21] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 9] -Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] -Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#21] - -(35) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#22] - -(36) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_bill_addr_sk#17] -Right keys [1]: [ca_address_sk#22] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 9] -Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#22] - -(38) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#23, i_manufact_id#24] - -(39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#23] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 9] -Output [2]: [cs_ext_sales_price#19, i_manufact_id#24] -Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#23, i_manufact_id#24] - -(41) HashAggregate [codegen id : 9] -Input [2]: [cs_ext_sales_price#19, i_manufact_id#24] -Keys [1]: [i_manufact_id#24] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum#25] -Results [2]: [i_manufact_id#24, sum#26] - -(42) CometColumnarExchange -Input [2]: [i_manufact_id#24, sum#26] -Arguments: hashpartitioning(i_manufact_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(43) CometColumnarToRow [codegen id : 10] -Input [2]: [i_manufact_id#24, sum#26] - -(44) HashAggregate [codegen id : 10] -Input [2]: [i_manufact_id#24, sum#26] -Keys [1]: [i_manufact_id#24] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#19))#27] -Results [2]: [i_manufact_id#24, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#19))#27,17,2) AS total_sales#28] - -(45) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#32), dynamicpruningexpression(ws_sold_date_sk#32 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 14] -Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32] - -(47) Filter [codegen id : 14] -Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32] -Condition : (isnotnull(ws_bill_addr_sk#30) AND isnotnull(ws_item_sk#29)) - -(48) ReusedExchange [Reuses operator id: 71] -Output [1]: [d_date_sk#33] - -(49) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#32] -Right keys [1]: [d_date_sk#33] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 14] -Output [3]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31] -Input [5]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32, d_date_sk#33] - -(51) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#34] - -(52) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#30] -Right keys [1]: [ca_address_sk#34] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 14] -Output [2]: [ws_item_sk#29, ws_ext_sales_price#31] -Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ca_address_sk#34] - -(54) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#35, i_manufact_id#36] - -(55) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_item_sk#29] -Right keys [1]: [i_item_sk#35] -Join type: Inner -Join condition: None - -(56) Project [codegen id : 14] -Output [2]: [ws_ext_sales_price#31, i_manufact_id#36] -Input [4]: [ws_item_sk#29, ws_ext_sales_price#31, i_item_sk#35, i_manufact_id#36] - -(57) HashAggregate [codegen id : 14] -Input [2]: [ws_ext_sales_price#31, i_manufact_id#36] -Keys [1]: [i_manufact_id#36] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#31))] -Aggregate Attributes [1]: [sum#37] -Results [2]: [i_manufact_id#36, sum#38] - -(58) CometColumnarExchange -Input [2]: [i_manufact_id#36, sum#38] -Arguments: hashpartitioning(i_manufact_id#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(59) CometColumnarToRow [codegen id : 15] -Input [2]: [i_manufact_id#36, sum#38] - -(60) HashAggregate [codegen id : 15] -Input [2]: [i_manufact_id#36, sum#38] -Keys [1]: [i_manufact_id#36] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#31))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#31))#39] -Results [2]: [i_manufact_id#36, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#31))#39,17,2) AS total_sales#40] - -(61) Union - -(62) HashAggregate [codegen id : 16] -Input [2]: [i_manufact_id#10, total_sales#16] -Keys [1]: [i_manufact_id#10] -Functions [1]: [partial_sum(total_sales#16)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [3]: [i_manufact_id#10, sum#43, isEmpty#44] - -(63) CometColumnarExchange -Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] -Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(64) CometColumnarToRow [codegen id : 17] -Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] - -(65) HashAggregate [codegen id : 17] -Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] -Keys [1]: [i_manufact_id#10] -Functions [1]: [sum(total_sales#16)] -Aggregate Attributes [1]: [sum(total_sales#16)#45] -Results [2]: [i_manufact_id#10, sum(total_sales#16)#45 AS total_sales#46] - -(66) TakeOrderedAndProject -Input [2]: [i_manufact_id#10, total_sales#46] -Arguments: 100, [total_sales#46 ASC NULLS FIRST], [i_manufact_id#10, total_sales#46] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometProject (69) - +- CometFilter (68) - +- CometNativeScan parquet spark_catalog.default.date_dim (67) - - -(67) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#47, d_moy#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] -ReadSchema: struct - -(68) CometFilter -Input [3]: [d_date_sk#6, d_year#47, d_moy#48] -Condition : ((((isnotnull(d_year#47) AND isnotnull(d_moy#48)) AND (d_year#47 = 1998)) AND (d_moy#48 = 5)) AND isnotnull(d_date_sk#6)) - -(69) CometProject -Input [3]: [d_date_sk#6, d_year#47, d_moy#48] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(70) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(71) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#32 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/simplified.txt deleted file mode 100644 index fd1c1d0ef2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/simplified.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject [total_sales,i_manufact_id] - WholeStageCodegen (17) - HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id] #1 - WholeStageCodegen (16) - HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id] #2 - WholeStageCodegen (4) - HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] - CometFilter [i_item_sk,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [i_manufact_id] #6 - CometProject [i_manufact_id] - CometFilter [i_category,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_category,i_manufact_id] - WholeStageCodegen (10) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id] #7 - WholeStageCodegen (9) - HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_addr_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 - WholeStageCodegen (15) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id] #8 - WholeStageCodegen (14) - HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/explain.txt deleted file mode 100644 index b7f7807f53..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,381 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (61) -+- CometTakeOrderedAndProject (60) - +- CometHashAggregate (59) - +- CometExchange (58) - +- CometHashAggregate (57) - +- CometUnion (56) - :- CometHashAggregate (27) - : +- CometExchange (26) - : +- CometHashAggregate (25) - : +- CometProject (24) - : +- CometBroadcastHashJoin (23) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : +- CometBroadcastExchange (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometFilter (29) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (28) - : : : +- ReusedExchange (30) - : : +- ReusedExchange (33) - : +- ReusedExchange (36) - +- CometHashAggregate (55) - +- CometExchange (54) - +- CometHashAggregate (53) - +- CometProject (52) - +- CometBroadcastHashJoin (51) - :- CometProject (49) - : +- CometBroadcastHashJoin (48) - : :- CometProject (46) - : : +- CometBroadcastHashJoin (45) - : : :- CometFilter (43) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : : +- ReusedExchange (44) - : +- ReusedExchange (47) - +- ReusedExchange (50) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(11) CometProject -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Arguments: [ca_address_sk#9], [ca_address_sk#9] - -(12) CometBroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: [ca_address_sk#9] - -(13) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Right output [1]: [ca_address_sk#9] -Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] -Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#11, i_manufact_id#12] -Condition : isnotnull(i_item_sk#11) - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_category#13, i_manufact_id#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [i_category#13, i_manufact_id#14] -Condition : (isnotnull(i_category#13) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#13, 50)) = Electronics )) - -(19) CometProject -Input [2]: [i_category#13, i_manufact_id#14] -Arguments: [i_manufact_id#14], [i_manufact_id#14] - -(20) CometBroadcastExchange -Input [1]: [i_manufact_id#14] -Arguments: [i_manufact_id#14] - -(21) CometBroadcastHashJoin -Left output [2]: [i_item_sk#11, i_manufact_id#12] -Right output [1]: [i_manufact_id#14] -Arguments: [i_manufact_id#12], [i_manufact_id#14], LeftSemi, BuildRight - -(22) CometBroadcastExchange -Input [2]: [i_item_sk#11, i_manufact_id#12] -Arguments: [i_item_sk#11, i_manufact_id#12] - -(23) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Right output [2]: [i_item_sk#11, i_manufact_id#12] -Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight - -(24) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_manufact_id#12] -Arguments: [ss_ext_sales_price#3, i_manufact_id#12], [ss_ext_sales_price#3, i_manufact_id#12] - -(25) CometHashAggregate -Input [2]: [ss_ext_sales_price#3, i_manufact_id#12] -Keys [1]: [i_manufact_id#12] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(26) CometExchange -Input [2]: [i_manufact_id#12, sum#15] -Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [2]: [i_manufact_id#12, sum#15] -Keys [1]: [i_manufact_id#12] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#20)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(29) CometFilter -Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] -Condition : (isnotnull(cs_bill_addr_sk#16) AND isnotnull(cs_item_sk#17)) - -(30) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#21] - -(31) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] -Right output [1]: [d_date_sk#21] -Arguments: [cs_sold_date_sk#19], [d_date_sk#21], Inner, BuildRight - -(32) CometProject -Input [5]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19, d_date_sk#21] -Arguments: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18], [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] - -(33) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#22] - -(34) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] -Right output [1]: [ca_address_sk#22] -Arguments: [cs_bill_addr_sk#16], [ca_address_sk#22], Inner, BuildRight - -(35) CometProject -Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, ca_address_sk#22] -Arguments: [cs_item_sk#17, cs_ext_sales_price#18], [cs_item_sk#17, cs_ext_sales_price#18] - -(36) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#23, i_manufact_id#24] - -(37) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_ext_sales_price#18] -Right output [2]: [i_item_sk#23, i_manufact_id#24] -Arguments: [cs_item_sk#17], [i_item_sk#23], Inner, BuildRight - -(38) CometProject -Input [4]: [cs_item_sk#17, cs_ext_sales_price#18, i_item_sk#23, i_manufact_id#24] -Arguments: [cs_ext_sales_price#18, i_manufact_id#24], [cs_ext_sales_price#18, i_manufact_id#24] - -(39) CometHashAggregate -Input [2]: [cs_ext_sales_price#18, i_manufact_id#24] -Keys [1]: [i_manufact_id#24] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#18))] - -(40) CometExchange -Input [2]: [i_manufact_id#24, sum#25] -Arguments: hashpartitioning(i_manufact_id#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(41) CometHashAggregate -Input [2]: [i_manufact_id#24, sum#25] -Keys [1]: [i_manufact_id#24] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#18))] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(43) CometFilter -Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Condition : (isnotnull(ws_bill_addr_sk#27) AND isnotnull(ws_item_sk#26)) - -(44) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#31] - -(45) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Right output [1]: [d_date_sk#31] -Arguments: [ws_sold_date_sk#29], [d_date_sk#31], Inner, BuildRight - -(46) CometProject -Input [5]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29, d_date_sk#31] -Arguments: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28], [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] - -(47) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#32] - -(48) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] -Right output [1]: [ca_address_sk#32] -Arguments: [ws_bill_addr_sk#27], [ca_address_sk#32], Inner, BuildRight - -(49) CometProject -Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ca_address_sk#32] -Arguments: [ws_item_sk#26, ws_ext_sales_price#28], [ws_item_sk#26, ws_ext_sales_price#28] - -(50) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#33, i_manufact_id#34] - -(51) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#26, ws_ext_sales_price#28] -Right output [2]: [i_item_sk#33, i_manufact_id#34] -Arguments: [ws_item_sk#26], [i_item_sk#33], Inner, BuildRight - -(52) CometProject -Input [4]: [ws_item_sk#26, ws_ext_sales_price#28, i_item_sk#33, i_manufact_id#34] -Arguments: [ws_ext_sales_price#28, i_manufact_id#34], [ws_ext_sales_price#28, i_manufact_id#34] - -(53) CometHashAggregate -Input [2]: [ws_ext_sales_price#28, i_manufact_id#34] -Keys [1]: [i_manufact_id#34] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#28))] - -(54) CometExchange -Input [2]: [i_manufact_id#34, sum#35] -Arguments: hashpartitioning(i_manufact_id#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(55) CometHashAggregate -Input [2]: [i_manufact_id#34, sum#35] -Keys [1]: [i_manufact_id#34] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#28))] - -(56) CometUnion -Child 0 Input [2]: [i_manufact_id#12, total_sales#36] -Child 1 Input [2]: [i_manufact_id#24, total_sales#37] -Child 2 Input [2]: [i_manufact_id#34, total_sales#38] - -(57) CometHashAggregate -Input [2]: [i_manufact_id#12, total_sales#36] -Keys [1]: [i_manufact_id#12] -Functions [1]: [partial_sum(total_sales#36)] - -(58) CometExchange -Input [3]: [i_manufact_id#12, sum#39, isEmpty#40] -Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(59) CometHashAggregate -Input [3]: [i_manufact_id#12, sum#39, isEmpty#40] -Keys [1]: [i_manufact_id#12] -Functions [1]: [sum(total_sales#36)] - -(60) CometTakeOrderedAndProject -Input [2]: [i_manufact_id#12, total_sales#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#41 ASC NULLS FIRST], output=[i_manufact_id#12,total_sales#41]), [i_manufact_id#12, total_sales#41], 100, 0, [total_sales#41 ASC NULLS FIRST], [i_manufact_id#12, total_sales#41] - -(61) CometColumnarToRow [codegen id : 1] -Input [2]: [i_manufact_id#12, total_sales#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (66) -+- * CometColumnarToRow (65) - +- CometProject (64) - +- CometFilter (63) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (62) - - -(62) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] -ReadSchema: struct - -(63) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6)) - -(64) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(65) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(66) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/simplified.txt deleted file mode 100644 index b4b84c2ac6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_manufact_id,total_sales] - CometHashAggregate [sum,isEmpty] [i_manufact_id,total_sales,sum(total_sales)] - CometExchange [i_manufact_id] #1 - CometHashAggregate [total_sales] [i_manufact_id,sum,isEmpty] - CometUnion [i_manufact_id,total_sales] - CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_manufact_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_manufact_id,sum] - CometProject [ss_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [i_manufact_id] #7 - CometProject [i_manufact_id] - CometFilter [i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact_id] - CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_manufact_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_manufact_id,sum] - CometProject [cs_ext_sales_price,i_manufact_id] - 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,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_manufact_id] #6 - CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_manufact_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_manufact_id,sum] - CometProject [ws_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [ws_item_sk,ws_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt deleted file mode 100644 index b7f7807f53..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt +++ /dev/null @@ -1,381 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (61) -+- CometTakeOrderedAndProject (60) - +- CometHashAggregate (59) - +- CometExchange (58) - +- CometHashAggregate (57) - +- CometUnion (56) - :- CometHashAggregate (27) - : +- CometExchange (26) - : +- CometHashAggregate (25) - : +- CometProject (24) - : +- CometBroadcastHashJoin (23) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : +- CometBroadcastExchange (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometFilter (29) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (28) - : : : +- ReusedExchange (30) - : : +- ReusedExchange (33) - : +- ReusedExchange (36) - +- CometHashAggregate (55) - +- CometExchange (54) - +- CometHashAggregate (53) - +- CometProject (52) - +- CometBroadcastHashJoin (51) - :- CometProject (49) - : +- CometBroadcastHashJoin (48) - : :- CometProject (46) - : : +- CometBroadcastHashJoin (45) - : : :- CometFilter (43) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : : +- ReusedExchange (44) - : +- ReusedExchange (47) - +- ReusedExchange (50) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(11) CometProject -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Arguments: [ca_address_sk#9], [ca_address_sk#9] - -(12) CometBroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: [ca_address_sk#9] - -(13) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Right output [1]: [ca_address_sk#9] -Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] -Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#11, i_manufact_id#12] -Condition : isnotnull(i_item_sk#11) - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_category#13, i_manufact_id#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [i_category#13, i_manufact_id#14] -Condition : (isnotnull(i_category#13) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#13, 50)) = Electronics )) - -(19) CometProject -Input [2]: [i_category#13, i_manufact_id#14] -Arguments: [i_manufact_id#14], [i_manufact_id#14] - -(20) CometBroadcastExchange -Input [1]: [i_manufact_id#14] -Arguments: [i_manufact_id#14] - -(21) CometBroadcastHashJoin -Left output [2]: [i_item_sk#11, i_manufact_id#12] -Right output [1]: [i_manufact_id#14] -Arguments: [i_manufact_id#12], [i_manufact_id#14], LeftSemi, BuildRight - -(22) CometBroadcastExchange -Input [2]: [i_item_sk#11, i_manufact_id#12] -Arguments: [i_item_sk#11, i_manufact_id#12] - -(23) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Right output [2]: [i_item_sk#11, i_manufact_id#12] -Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight - -(24) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_manufact_id#12] -Arguments: [ss_ext_sales_price#3, i_manufact_id#12], [ss_ext_sales_price#3, i_manufact_id#12] - -(25) CometHashAggregate -Input [2]: [ss_ext_sales_price#3, i_manufact_id#12] -Keys [1]: [i_manufact_id#12] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(26) CometExchange -Input [2]: [i_manufact_id#12, sum#15] -Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [2]: [i_manufact_id#12, sum#15] -Keys [1]: [i_manufact_id#12] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#20)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(29) CometFilter -Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] -Condition : (isnotnull(cs_bill_addr_sk#16) AND isnotnull(cs_item_sk#17)) - -(30) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#21] - -(31) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] -Right output [1]: [d_date_sk#21] -Arguments: [cs_sold_date_sk#19], [d_date_sk#21], Inner, BuildRight - -(32) CometProject -Input [5]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19, d_date_sk#21] -Arguments: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18], [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] - -(33) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#22] - -(34) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] -Right output [1]: [ca_address_sk#22] -Arguments: [cs_bill_addr_sk#16], [ca_address_sk#22], Inner, BuildRight - -(35) CometProject -Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, ca_address_sk#22] -Arguments: [cs_item_sk#17, cs_ext_sales_price#18], [cs_item_sk#17, cs_ext_sales_price#18] - -(36) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#23, i_manufact_id#24] - -(37) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_ext_sales_price#18] -Right output [2]: [i_item_sk#23, i_manufact_id#24] -Arguments: [cs_item_sk#17], [i_item_sk#23], Inner, BuildRight - -(38) CometProject -Input [4]: [cs_item_sk#17, cs_ext_sales_price#18, i_item_sk#23, i_manufact_id#24] -Arguments: [cs_ext_sales_price#18, i_manufact_id#24], [cs_ext_sales_price#18, i_manufact_id#24] - -(39) CometHashAggregate -Input [2]: [cs_ext_sales_price#18, i_manufact_id#24] -Keys [1]: [i_manufact_id#24] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#18))] - -(40) CometExchange -Input [2]: [i_manufact_id#24, sum#25] -Arguments: hashpartitioning(i_manufact_id#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(41) CometHashAggregate -Input [2]: [i_manufact_id#24, sum#25] -Keys [1]: [i_manufact_id#24] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#18))] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(43) CometFilter -Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Condition : (isnotnull(ws_bill_addr_sk#27) AND isnotnull(ws_item_sk#26)) - -(44) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#31] - -(45) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Right output [1]: [d_date_sk#31] -Arguments: [ws_sold_date_sk#29], [d_date_sk#31], Inner, BuildRight - -(46) CometProject -Input [5]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29, d_date_sk#31] -Arguments: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28], [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] - -(47) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#32] - -(48) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] -Right output [1]: [ca_address_sk#32] -Arguments: [ws_bill_addr_sk#27], [ca_address_sk#32], Inner, BuildRight - -(49) CometProject -Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ca_address_sk#32] -Arguments: [ws_item_sk#26, ws_ext_sales_price#28], [ws_item_sk#26, ws_ext_sales_price#28] - -(50) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#33, i_manufact_id#34] - -(51) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#26, ws_ext_sales_price#28] -Right output [2]: [i_item_sk#33, i_manufact_id#34] -Arguments: [ws_item_sk#26], [i_item_sk#33], Inner, BuildRight - -(52) CometProject -Input [4]: [ws_item_sk#26, ws_ext_sales_price#28, i_item_sk#33, i_manufact_id#34] -Arguments: [ws_ext_sales_price#28, i_manufact_id#34], [ws_ext_sales_price#28, i_manufact_id#34] - -(53) CometHashAggregate -Input [2]: [ws_ext_sales_price#28, i_manufact_id#34] -Keys [1]: [i_manufact_id#34] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#28))] - -(54) CometExchange -Input [2]: [i_manufact_id#34, sum#35] -Arguments: hashpartitioning(i_manufact_id#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(55) CometHashAggregate -Input [2]: [i_manufact_id#34, sum#35] -Keys [1]: [i_manufact_id#34] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#28))] - -(56) CometUnion -Child 0 Input [2]: [i_manufact_id#12, total_sales#36] -Child 1 Input [2]: [i_manufact_id#24, total_sales#37] -Child 2 Input [2]: [i_manufact_id#34, total_sales#38] - -(57) CometHashAggregate -Input [2]: [i_manufact_id#12, total_sales#36] -Keys [1]: [i_manufact_id#12] -Functions [1]: [partial_sum(total_sales#36)] - -(58) CometExchange -Input [3]: [i_manufact_id#12, sum#39, isEmpty#40] -Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(59) CometHashAggregate -Input [3]: [i_manufact_id#12, sum#39, isEmpty#40] -Keys [1]: [i_manufact_id#12] -Functions [1]: [sum(total_sales#36)] - -(60) CometTakeOrderedAndProject -Input [2]: [i_manufact_id#12, total_sales#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#41 ASC NULLS FIRST], output=[i_manufact_id#12,total_sales#41]), [i_manufact_id#12, total_sales#41], 100, 0, [total_sales#41 ASC NULLS FIRST], [i_manufact_id#12, total_sales#41] - -(61) CometColumnarToRow [codegen id : 1] -Input [2]: [i_manufact_id#12, total_sales#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (66) -+- * CometColumnarToRow (65) - +- CometProject (64) - +- CometFilter (63) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (62) - - -(62) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] -ReadSchema: struct - -(63) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6)) - -(64) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(65) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(66) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/extended.txt deleted file mode 100644 index 063fdb6faf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/extended.txt +++ /dev/null @@ -1,97 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 89 out of 93 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index b4b84c2ac6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_manufact_id,total_sales] - CometHashAggregate [sum,isEmpty] [i_manufact_id,total_sales,sum(total_sales)] - CometExchange [i_manufact_id] #1 - CometHashAggregate [total_sales] [i_manufact_id,sum,isEmpty] - CometUnion [i_manufact_id,total_sales] - CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_manufact_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_manufact_id,sum] - CometProject [ss_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [i_manufact_id] #7 - CometProject [i_manufact_id] - CometFilter [i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact_id] - CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_manufact_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_manufact_id,sum] - CometProject [cs_ext_sales_price,i_manufact_id] - 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,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_manufact_id] #6 - CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_manufact_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_manufact_id,sum] - CometProject [ws_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [ws_item_sk,ws_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/explain.txt deleted file mode 100644 index 7d332cb2e7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/explain.txt +++ /dev/null @@ -1,231 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (35) -+- CometSort (34) - +- CometColumnarExchange (33) - +- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * Filter (25) - : +- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (30) - +- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometNativeScan parquet spark_catalog.default.customer (26) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 40] -Output [1]: [d_date_sk#7] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_county#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [s_store_sk#8, s_county#9] -Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND isnotnull(s_store_sk#8)) - -(9) CometProject -Input [2]: [s_store_sk#8, s_county#9] -Arguments: [s_store_sk#8], [s_store_sk#8] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#8] - -(11) BroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] - -(14) CometNativeScan parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : ((((isnotnull(hd_vehicle_count#13) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#11, 15)) = >10000 ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#11, 15)) = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#12 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#13 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#10)) - -(16) CometProject -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Arguments: [hd_demo_sk#10], [hd_demo_sk#10] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#10] - -(18) BroadcastExchange -Input [1]: [hd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#10] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [2]: [ss_customer_sk#1, ss_ticket_number#4] -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] - -(21) HashAggregate [codegen id : 4] -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#14] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(22) CometColumnarExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(24) HashAggregate [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#16] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] - -(25) Filter [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) - -(26) CometNativeScan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(27) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(28) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#19, 10)) AS c_salutation#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#20, 20)) AS c_first_name#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#21, 30)) AS c_last_name#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#22, 1)) AS c_preferred_cust_flag#26] - -(29) CometColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(30) BroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#18] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 6] -Output [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(33) CometColumnarExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(34) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST] - -(35) CometColumnarToRow [codegen id : 7] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (40) -+- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometNativeScan parquet spark_catalog.default.date_dim (36) - - -(36) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#27, d_dom#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [d_date_sk#7, d_year#27, d_dom#28] -Condition : (((((d_dom#28 >= 1) AND (d_dom#28 <= 3)) OR ((d_dom#28 >= 25) AND (d_dom#28 <= 28))) AND d_year#27 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(38) CometProject -Input [3]: [d_date_sk#7, d_year#27, d_dom#28] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(39) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(40) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/simplified.txt deleted file mode 100644 index 10af5ceadc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/simplified.txt +++ /dev/null @@ -1,59 +0,0 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometNativeScan 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/q34.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/explain.txt deleted file mode 100644 index d8ee4f981e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometSort (32) - +- CometExchange (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (24) - : +- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_county#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#10, s_county#11] -Condition : ((isnotnull(s_county#11) AND (s_county#11 = Williamson County)) AND isnotnull(s_store_sk#10)) - -(11) CometProject -Input [2]: [s_store_sk#10, s_county#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : ((((isnotnull(hd_vehicle_count#15) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#13, 15)) = >10000 ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#13, 15)) = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#14 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#15 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#12)) - -(17) CometProject -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#12], [hd_demo_sk#12] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#12] -Arguments: [hd_demo_sk#12] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Right output [1]: [hd_demo_sk#12] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight - -(20) CometProject -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] -Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] - -(21) CometHashAggregate -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] - -(22) CometExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] - -(24) CometFilter -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(27) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#19, 10)) AS c_salutation#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#20, 20)) AS c_first_name#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#21, 30)) AS c_last_name#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#22, 1)) AS c_preferred_cust_flag#26] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(29) CometBroadcastHashJoin -Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Right output [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -(31) CometExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(32) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST] - -(33) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(36) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(37) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(38) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/simplified.txt deleted file mode 100644 index 47d0e35dd2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 - CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] - CometExchange [ss_ticket_number,ss_customer_sk] #2 - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - 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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/explain.txt deleted file mode 100644 index d8ee4f981e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometSort (32) - +- CometExchange (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (24) - : +- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_county#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#10, s_county#11] -Condition : ((isnotnull(s_county#11) AND (s_county#11 = Williamson County)) AND isnotnull(s_store_sk#10)) - -(11) CometProject -Input [2]: [s_store_sk#10, s_county#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : ((((isnotnull(hd_vehicle_count#15) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#13, 15)) = >10000 ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#13, 15)) = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#14 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#15 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#12)) - -(17) CometProject -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#12], [hd_demo_sk#12] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#12] -Arguments: [hd_demo_sk#12] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Right output [1]: [hd_demo_sk#12] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight - -(20) CometProject -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] -Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] - -(21) CometHashAggregate -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] - -(22) CometExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] - -(24) CometFilter -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(27) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#19, 10)) AS c_salutation#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#20, 20)) AS c_first_name#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#21, 30)) AS c_last_name#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#22, 1)) AS c_preferred_cust_flag#26] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(29) CometBroadcastHashJoin -Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Right output [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -(31) CometExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(32) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST] - -(33) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(36) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(37) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(38) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/extended.txt deleted file mode 100644 index 3d9d23d2cc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/extended.txt +++ /dev/null @@ -1,41 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 47d0e35dd2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 - CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] - CometExchange [ss_ticket_number,ss_customer_sk] #2 - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - 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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/explain.txt deleted file mode 100644 index 7d1dadfda2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/explain.txt +++ /dev/null @@ -1,295 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (45) -+- * HashAggregate (44) - +- * CometColumnarToRow (43) - +- CometColumnarExchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometNativeScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (38) - +- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometNativeScan parquet spark_catalog.default.customer_demographics (34) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(4) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] - -(6) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#9] - -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None - -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#6] -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] - -(9) BroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#6] -Join type: LeftSemi -Join condition: None - -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] - -(13) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#12] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#10] -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] - -(16) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(17) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#10] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] - -(20) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#15] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#13] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] - -(23) BroadcastExchange -Input [1]: [cs_ship_customer_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#13] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(25) Filter [codegen id : 9] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(26) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(27) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : isnotnull(ca_address_sk#16) - -(29) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16, ca_state#18], [ca_address_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#17, 2)) AS ca_state#18] - -(30) CometColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#16, ca_state#18] - -(31) BroadcastExchange -Input [2]: [ca_address_sk#16, ca_state#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#16] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#18] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#16, ca_state#18] - -(34) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(35) CometFilter -Input [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Condition : isnotnull(cd_demo_sk#19) - -(36) CometProject -Input [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Arguments: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24], [cd_demo_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#20, 1)) AS cd_gender#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#21, 1)) AS cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] - -(37) CometColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] - -(38) BroadcastExchange -Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#19] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 9] -Output [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Input [8]: [c_current_cdemo_sk#4, ca_state#18, cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] - -(41) HashAggregate [codegen id : 9] -Input [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#22), partial_max(cd_dep_count#22), partial_avg(cd_dep_count#22), partial_min(cd_dep_employed_count#23), partial_max(cd_dep_employed_count#23), partial_avg(cd_dep_employed_count#23), partial_min(cd_dep_college_count#24), partial_max(cd_dep_college_count#24), partial_avg(cd_dep_college_count#24)] -Aggregate Attributes [13]: [count#27, min#28, max#29, sum#30, count#31, min#32, max#33, sum#34, count#35, min#36, max#37, sum#38, count#39] -Results [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, min#41, max#42, sum#43, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52] - -(42) CometColumnarExchange -Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, min#41, max#42, sum#43, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52] -Arguments: hashpartitioning(ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) CometColumnarToRow [codegen id : 10] -Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, min#41, max#42, sum#43, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52] - -(44) HashAggregate [codegen id : 10] -Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, min#41, max#42, sum#43, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52] -Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Functions [10]: [count(1), min(cd_dep_count#22), max(cd_dep_count#22), avg(cd_dep_count#22), min(cd_dep_employed_count#23), max(cd_dep_employed_count#23), avg(cd_dep_employed_count#23), min(cd_dep_college_count#24), max(cd_dep_college_count#24), avg(cd_dep_college_count#24)] -Aggregate Attributes [10]: [count(1)#53, min(cd_dep_count#22)#54, max(cd_dep_count#22)#55, avg(cd_dep_count#22)#56, min(cd_dep_employed_count#23)#57, max(cd_dep_employed_count#23)#58, avg(cd_dep_employed_count#23)#59, min(cd_dep_college_count#24)#60, max(cd_dep_college_count#24)#61, avg(cd_dep_college_count#24)#62] -Results [18]: [ca_state#18, cd_gender#25, cd_marital_status#26, count(1)#53 AS cnt1#63, min(cd_dep_count#22)#54 AS min(cd_dep_count)#64, max(cd_dep_count#22)#55 AS max(cd_dep_count)#65, avg(cd_dep_count#22)#56 AS avg(cd_dep_count)#66, cd_dep_employed_count#23, count(1)#53 AS cnt2#67, min(cd_dep_employed_count#23)#57 AS min(cd_dep_employed_count)#68, max(cd_dep_employed_count#23)#58 AS max(cd_dep_employed_count)#69, avg(cd_dep_employed_count#23)#59 AS avg(cd_dep_employed_count)#70, cd_dep_college_count#24, count(1)#53 AS cnt3#71, min(cd_dep_college_count#24)#60 AS min(cd_dep_college_count)#72, max(cd_dep_college_count#24)#61 AS max(cd_dep_college_count)#73, avg(cd_dep_college_count#24)#62 AS avg(cd_dep_college_count)#74, cd_dep_count#22] - -(45) TakeOrderedAndProject -Input [18]: [ca_state#18, cd_gender#25, cd_marital_status#26, cnt1#63, min(cd_dep_count)#64, max(cd_dep_count)#65, avg(cd_dep_count)#66, cd_dep_employed_count#23, cnt2#67, min(cd_dep_employed_count)#68, max(cd_dep_employed_count)#69, avg(cd_dep_employed_count)#70, cd_dep_college_count#24, cnt3#71, min(cd_dep_college_count)#72, max(cd_dep_college_count)#73, avg(cd_dep_college_count)#74, cd_dep_count#22] -Arguments: 100, [ca_state#18 ASC NULLS FIRST, cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_dep_count#22 ASC NULLS FIRST, cd_dep_employed_count#23 ASC NULLS FIRST, cd_dep_college_count#24 ASC NULLS FIRST], [ca_state#18, cd_gender#25, cd_marital_status#26, cnt1#63, min(cd_dep_count)#64, max(cd_dep_count)#65, avg(cd_dep_count)#66, cd_dep_employed_count#23, cnt2#67, min(cd_dep_employed_count)#68, max(cd_dep_employed_count)#69, avg(cd_dep_employed_count)#70, cd_dep_college_count#24, cnt3#71, min(cd_dep_college_count)#72, max(cd_dep_college_count)#73, avg(cd_dep_college_count)#74] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (50) -+- * CometColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometNativeScan parquet spark_catalog.default.date_dim (46) - - -(46) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#75, d_qoy#76] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_date_sk#9, d_year#75, d_qoy#76] -Condition : ((((isnotnull(d_year#75) AND isnotnull(d_qoy#76)) AND (d_year#75 = 2002)) AND (d_qoy#76 < 4)) AND isnotnull(d_date_sk#9)) - -(48) CometProject -Input [3]: [d_date_sk#9, d_year#75, d_qoy#76] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(49) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(50) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/simplified.txt deleted file mode 100644 index 48bc5637d0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/simplified.txt +++ /dev/null @@ -1,77 +0,0 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (10) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan 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/q35.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/explain.txt deleted file mode 100644 index 4db51f2231..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * Filter (27) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(6) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: [d_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#9] -Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -Arguments: [ss_customer_sk#6], [ss_customer_sk#6] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: [ss_customer_sk#6] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Right output [1]: [ss_customer_sk#6] -Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#15] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Right output [1]: [d_date_sk#15] -Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] -Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#12] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#12] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#19] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#16] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#16] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(27) Filter [codegen id : 5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(28) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : isnotnull(ca_address_sk#20) - -(31) CometProject -Input [2]: [ca_address_sk#20, ca_state#21] -Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#21, 2)) AS ca_state#22] - -(32) CometColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#20, ca_state#22] - -(33) BroadcastExchange -Input [2]: [ca_address_sk#20, ca_state#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(34) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, ca_state#22] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#22] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(37) CometFilter -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#23) - -(38) CometProject -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#24, 1)) AS cd_gender#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#25, 1)) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(39) CometColumnarToRow [codegen id : 4] -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(40) BroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(41) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#23] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 5] -Output [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [8]: [c_current_cdemo_sk#4, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(43) HashAggregate [codegen id : 5] -Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#26), partial_max(cd_dep_count#26), partial_avg(cd_dep_count#26), partial_min(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_avg(cd_dep_employed_count#27), partial_min(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_avg(cd_dep_college_count#28)] -Aggregate Attributes [13]: [count#31, min#32, max#33, sum#34, count#35, min#36, max#37, sum#38, count#39, min#40, max#41, sum#42, count#43] -Results [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] - -(44) CometColumnarExchange -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] -Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 6] -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] - -(46) HashAggregate [codegen id : 6] -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [count(1), min(cd_dep_count#26), max(cd_dep_count#26), avg(cd_dep_count#26), min(cd_dep_employed_count#27), max(cd_dep_employed_count#27), avg(cd_dep_employed_count#27), min(cd_dep_college_count#28), max(cd_dep_college_count#28), avg(cd_dep_college_count#28)] -Aggregate Attributes [10]: [count(1)#57, min(cd_dep_count#26)#58, max(cd_dep_count#26)#59, avg(cd_dep_count#26)#60, min(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, avg(cd_dep_employed_count#27)#63, min(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, avg(cd_dep_college_count#28)#66] -Results [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, count(1)#57 AS cnt1#67, min(cd_dep_count#26)#58 AS min(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, avg(cd_dep_count#26)#60 AS avg(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, min(cd_dep_employed_count#27)#61 AS min(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, avg(cd_dep_employed_count#27)#63 AS avg(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, min(cd_dep_college_count#28)#64 AS min(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, avg(cd_dep_college_count#28)#66 AS avg(cd_dep_college_count)#78, cd_dep_count#26] - -(47) TakeOrderedAndProject -Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78, cd_dep_count#26] -Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#22, cd_gender#29, cd_marital_status#30, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(52) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/simplified.txt deleted file mode 100644 index f1fe09fb46..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (6) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] 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/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/explain.txt deleted file mode 100644 index 4db51f2231..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * Filter (27) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(6) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: [d_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#9] -Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -Arguments: [ss_customer_sk#6], [ss_customer_sk#6] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: [ss_customer_sk#6] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Right output [1]: [ss_customer_sk#6] -Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#15] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Right output [1]: [d_date_sk#15] -Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] -Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#12] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#12] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#19] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#16] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#16] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(27) Filter [codegen id : 5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(28) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : isnotnull(ca_address_sk#20) - -(31) CometProject -Input [2]: [ca_address_sk#20, ca_state#21] -Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#21, 2)) AS ca_state#22] - -(32) CometColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#20, ca_state#22] - -(33) BroadcastExchange -Input [2]: [ca_address_sk#20, ca_state#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(34) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, ca_state#22] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#22] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(37) CometFilter -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#23) - -(38) CometProject -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#24, 1)) AS cd_gender#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#25, 1)) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(39) CometColumnarToRow [codegen id : 4] -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(40) BroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(41) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#23] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 5] -Output [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [8]: [c_current_cdemo_sk#4, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(43) HashAggregate [codegen id : 5] -Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#26), partial_max(cd_dep_count#26), partial_avg(cd_dep_count#26), partial_min(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_avg(cd_dep_employed_count#27), partial_min(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_avg(cd_dep_college_count#28)] -Aggregate Attributes [13]: [count#31, min#32, max#33, sum#34, count#35, min#36, max#37, sum#38, count#39, min#40, max#41, sum#42, count#43] -Results [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] - -(44) CometColumnarExchange -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] -Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 6] -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] - -(46) HashAggregate [codegen id : 6] -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, min#45, max#46, sum#47, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [count(1), min(cd_dep_count#26), max(cd_dep_count#26), avg(cd_dep_count#26), min(cd_dep_employed_count#27), max(cd_dep_employed_count#27), avg(cd_dep_employed_count#27), min(cd_dep_college_count#28), max(cd_dep_college_count#28), avg(cd_dep_college_count#28)] -Aggregate Attributes [10]: [count(1)#57, min(cd_dep_count#26)#58, max(cd_dep_count#26)#59, avg(cd_dep_count#26)#60, min(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, avg(cd_dep_employed_count#27)#63, min(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, avg(cd_dep_college_count#28)#66] -Results [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, count(1)#57 AS cnt1#67, min(cd_dep_count#26)#58 AS min(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, avg(cd_dep_count#26)#60 AS avg(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, min(cd_dep_employed_count#27)#61 AS min(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, avg(cd_dep_employed_count#27)#63 AS avg(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, min(cd_dep_college_count#28)#64 AS min(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, avg(cd_dep_college_count#28)#66 AS avg(cd_dep_college_count)#78, cd_dep_count#26] - -(47) TakeOrderedAndProject -Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78, cd_dep_count#26] -Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#22, cd_gender#29, cd_marital_status#30, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(52) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/extended.txt deleted file mode 100644 index a6f33d6f7e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/extended.txt +++ /dev/null @@ -1,63 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index f1fe09fb46..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (6) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/explain.txt deleted file mode 100644 index d3f2114c78..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/explain.txt +++ /dev/null @@ -1,207 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (31) -+- * Project (30) - +- Window (29) - +- * CometColumnarToRow (28) - +- CometSort (27) - +- CometColumnarExchange (26) - +- * HashAggregate (25) - +- * CometColumnarToRow (24) - +- CometColumnarExchange (23) - +- * HashAggregate (22) - +- * Expand (21) - +- * Project (20) - +- * BroadcastHashJoin Inner BuildRight (19) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) - : +- CometProject (9) - : +- CometFilter (8) - : +- CometNativeScan parquet spark_catalog.default.item (7) - +- BroadcastExchange (18) - +- * CometColumnarToRow (17) - +- CometProject (16) - +- CometFilter (15) - +- CometNativeScan parquet spark_catalog.default.store (14) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) - -(4) ReusedExchange [Reuses operator id: 36] -Output [1]: [d_date_sk#7] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] - -(7) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Condition : isnotnull(i_item_sk#8) - -(9) CometProject -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Arguments: [i_item_sk#8, i_class#11, i_category#12], [i_item_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#12] - -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#8, i_class#11, i_category#12] - -(11) BroadcastExchange -Input [3]: [i_item_sk#8, i_class#11, i_category#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#8] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#8, i_class#11, i_category#12] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#13, s_state#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#13, s_state#14] -Condition : ((isnotnull(s_state#14) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#14, 2)) = TN)) AND isnotnull(s_store_sk#13)) - -(16) CometProject -Input [2]: [s_store_sk#13, s_state#14] -Arguments: [s_store_sk#13], [s_store_sk#13] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#13] - -(18) BroadcastExchange -Input [1]: [s_store_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#13] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#12, i_class#11] -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12, s_store_sk#13] - -(21) Expand [codegen id : 4] -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#12, i_class#11] -Arguments: [[ss_ext_sales_price#3, ss_net_profit#4, i_category#12, i_class#11, 0], [ss_ext_sales_price#3, ss_net_profit#4, i_category#12, null, 1], [ss_ext_sales_price#3, ss_net_profit#4, null, null, 3]], [ss_ext_sales_price#3, ss_net_profit#4, i_category#15, i_class#16, spark_grouping_id#17] - -(22) HashAggregate [codegen id : 4] -Input [5]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#15, i_class#16, spark_grouping_id#17] -Keys [3]: [i_category#15, i_class#16, spark_grouping_id#17] -Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum#18, sum#19] -Results [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] - -(23) CometColumnarExchange -Input [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] -Arguments: hashpartitioning(i_category#15, i_class#16, spark_grouping_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometColumnarToRow [codegen id : 5] -Input [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] - -(25) HashAggregate [codegen id : 5] -Input [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] -Keys [3]: [i_category#15, i_class#16, spark_grouping_id#17] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#22, sum(UnscaledValue(ss_ext_sales_price#3))#23] -Results [7]: [(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#22,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#23,17,2)) AS gross_margin#24, i_category#15, i_class#16, (cast((shiftright(spark_grouping_id#17, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#17, 0) & 1) as tinyint)) AS lochierarchy#25, (MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#22,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#23,17,2)) AS _w0#26, (cast((shiftright(spark_grouping_id#17, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#17, 0) & 1) as tinyint)) AS _w1#27, CASE WHEN (cast((shiftright(spark_grouping_id#17, 0) & 1) as tinyint) = 0) THEN i_category#15 END AS _w2#28] - -(26) CometColumnarExchange -Input [7]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28] -Arguments: hashpartitioning(_w1#27, _w2#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometSort -Input [7]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28] -Arguments: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28], [_w1#27 ASC NULLS FIRST, _w2#28 ASC NULLS FIRST, _w0#26 ASC NULLS FIRST] - -(28) CometColumnarToRow [codegen id : 6] -Input [7]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28] - -(29) Window -Input [7]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28] -Arguments: [rank(_w0#26) windowspecdefinition(_w1#27, _w2#28, _w0#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#29], [_w1#27, _w2#28], [_w0#26 ASC NULLS FIRST] - -(30) Project [codegen id : 7] -Output [5]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, rank_within_parent#29] -Input [8]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28, rank_within_parent#29] - -(31) TakeOrderedAndProject -Input [5]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, rank_within_parent#29] -Arguments: 100, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#29 ASC NULLS FIRST], [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, rank_within_parent#29] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.date_dim (32) - - -(32) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#7, d_year#30] -Condition : ((isnotnull(d_year#30) AND (d_year#30 = 2001)) AND isnotnull(d_date_sk#7)) - -(34) CometProject -Input [2]: [d_date_sk#7, d_year#30] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(36) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/simplified.txt deleted file mode 100644 index d6f8548c89..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (7) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Expand [ss_ext_sales_price,ss_net_profit,i_category,i_class] - Project [ss_ext_sales_price,ss_net_profit,i_category,i_class] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan 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/q36.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/explain.txt deleted file mode 100644 index 634c01506e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,203 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- Window (28) - +- * CometColumnarToRow (27) - +- CometSort (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExpand (21) - +- CometProject (20) - +- CometBroadcastHashJoin (19) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - +- CometBroadcastExchange (18) - +- CometProject (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4], [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#9, i_class#10, i_category#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Condition : isnotnull(i_item_sk#9) - -(11) CometProject -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Arguments: [i_item_sk#9, i_class#12, i_category#13], [i_item_sk#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#10, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#11, 50)) AS i_category#13] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [i_item_sk#9, i_class#12, i_category#13] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Right output [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#9], Inner, BuildRight - -(14) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : ((isnotnull(s_state#15) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#15, 2)) = TN)) AND isnotnull(s_store_sk#14)) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14], [s_store_sk#14] - -(18) CometBroadcastExchange -Input [1]: [s_store_sk#14] -Arguments: [s_store_sk#14] - -(19) CometBroadcastHashJoin -Left output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -Right output [1]: [s_store_sk#14] -Arguments: [ss_store_sk#2], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13, s_store_sk#14] -Arguments: [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12], [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12] - -(21) CometExpand -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12] -Arguments: [[ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12, 0], [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, null, 1], [ss_ext_sales_price#3, ss_net_profit#4, null, null, 3]], [ss_ext_sales_price#3, ss_net_profit#4, i_category#16, i_class#17, spark_grouping_id#18] - -(22) CometHashAggregate -Input [5]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#16, i_class#17, spark_grouping_id#18] -Keys [3]: [i_category#16, i_class#17, spark_grouping_id#18] -Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(23) CometExchange -Input [5]: [i_category#16, i_class#17, spark_grouping_id#18, sum#19, sum#20] -Arguments: hashpartitioning(i_category#16, i_class#17, spark_grouping_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [5]: [i_category#16, i_class#17, spark_grouping_id#18, sum#19, sum#20] -Keys [3]: [i_category#16, i_class#17, spark_grouping_id#18] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] - -(25) CometExchange -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -Arguments: hashpartitioning(_w1#24, _w2#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(26) CometSort -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -Arguments: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25], [_w1#24 ASC NULLS FIRST, _w2#25 ASC NULLS FIRST, _w0#23 ASC NULLS FIRST] - -(27) CometColumnarToRow [codegen id : 1] -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] - -(28) Window -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -Arguments: [rank(_w0#23) windowspecdefinition(_w1#24, _w2#25, _w0#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#26], [_w1#24, _w2#25], [_w0#23 ASC NULLS FIRST] - -(29) Project [codegen id : 2] -Output [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] -Input [8]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25, rank_within_parent#26] - -(30) TakeOrderedAndProject -Input [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] -Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#26 ASC NULLS FIRST], [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(33) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(35) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/simplified.txt deleted file mode 100644 index 61782e3011..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,42 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (2) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometExchange [_w1,_w2] #1 - CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),spark_grouping_id] - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] - 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,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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #6 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] 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/q36/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt deleted file mode 100644 index 634c01506e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt +++ /dev/null @@ -1,203 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- Window (28) - +- * CometColumnarToRow (27) - +- CometSort (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExpand (21) - +- CometProject (20) - +- CometBroadcastHashJoin (19) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - +- CometBroadcastExchange (18) - +- CometProject (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4], [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#9, i_class#10, i_category#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Condition : isnotnull(i_item_sk#9) - -(11) CometProject -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Arguments: [i_item_sk#9, i_class#12, i_category#13], [i_item_sk#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#10, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#11, 50)) AS i_category#13] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [i_item_sk#9, i_class#12, i_category#13] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Right output [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#9], Inner, BuildRight - -(14) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : ((isnotnull(s_state#15) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#15, 2)) = TN)) AND isnotnull(s_store_sk#14)) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14], [s_store_sk#14] - -(18) CometBroadcastExchange -Input [1]: [s_store_sk#14] -Arguments: [s_store_sk#14] - -(19) CometBroadcastHashJoin -Left output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -Right output [1]: [s_store_sk#14] -Arguments: [ss_store_sk#2], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13, s_store_sk#14] -Arguments: [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12], [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12] - -(21) CometExpand -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12] -Arguments: [[ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12, 0], [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, null, 1], [ss_ext_sales_price#3, ss_net_profit#4, null, null, 3]], [ss_ext_sales_price#3, ss_net_profit#4, i_category#16, i_class#17, spark_grouping_id#18] - -(22) CometHashAggregate -Input [5]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#16, i_class#17, spark_grouping_id#18] -Keys [3]: [i_category#16, i_class#17, spark_grouping_id#18] -Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(23) CometExchange -Input [5]: [i_category#16, i_class#17, spark_grouping_id#18, sum#19, sum#20] -Arguments: hashpartitioning(i_category#16, i_class#17, spark_grouping_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [5]: [i_category#16, i_class#17, spark_grouping_id#18, sum#19, sum#20] -Keys [3]: [i_category#16, i_class#17, spark_grouping_id#18] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] - -(25) CometExchange -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -Arguments: hashpartitioning(_w1#24, _w2#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(26) CometSort -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -Arguments: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25], [_w1#24 ASC NULLS FIRST, _w2#25 ASC NULLS FIRST, _w0#23 ASC NULLS FIRST] - -(27) CometColumnarToRow [codegen id : 1] -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] - -(28) Window -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -Arguments: [rank(_w0#23) windowspecdefinition(_w1#24, _w2#25, _w0#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#26], [_w1#24, _w2#25], [_w0#23 ASC NULLS FIRST] - -(29) Project [codegen id : 2] -Output [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] -Input [8]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25, rank_within_parent#26] - -(30) TakeOrderedAndProject -Input [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] -Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#26 ASC NULLS FIRST], [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(33) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(35) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/extended.txt deleted file mode 100644 index 89c523a388..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/extended.txt +++ /dev/null @@ -1,38 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 61782e3011..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt +++ /dev/null @@ -1,42 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (2) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometExchange [_w1,_w2] #1 - CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),spark_grouping_id] - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] - 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,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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #6 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/explain.txt deleted file mode 100644 index 7fcb5f8109..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/explain.txt +++ /dev/null @@ -1,181 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometTakeOrderedAndProject (25) - +- CometHashAggregate (24) - +- CometColumnarExchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildLeft (20) - :- BroadcastExchange (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * CometColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (9) - : : +- * Project (8) - : : +- * Filter (7) - : : +- * ColumnarToRow (6) - : : +- Scan parquet spark_catalog.default.inventory (5) - : +- ReusedExchange (12) - +- * CometColumnarToRow (19) - +- CometProject (18) - +- CometFilter (17) - +- CometNativeScan parquet spark_catalog.default.catalog_sales (16) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,68.00), LessThanOrEqual(i_current_price,98.00), In(i_manufact_id, [677,694,808,940]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 68.00)) AND (i_current_price#4 <= 98.00)) AND i_manufact_id#5 IN (677,940,694,808)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#2, 16)) AS i_item_id#6, i_item_desc#3, i_current_price#4] - -(4) CometColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(5) Scan parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#9), dynamicpruningexpression(inv_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] - -(7) Filter [codegen id : 1] -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Condition : (((isnotnull(inv_quantity_on_hand#8) AND (inv_quantity_on_hand#8 >= 100)) AND (inv_quantity_on_hand#8 <= 500)) AND isnotnull(inv_item_sk#7)) - -(8) Project [codegen id : 1] -Output [2]: [inv_item_sk#7, inv_date_sk#9] -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] - -(9) BroadcastExchange -Input [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [inv_item_sk#7] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 3] -Output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] - -(12) ReusedExchange [Reuses operator id: 31] -Output [1]: [d_date_sk#11] - -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [inv_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 3] -Output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] - -(15) BroadcastExchange -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#12, cs_sold_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [cs_item_sk#12, cs_sold_date_sk#13] -Condition : isnotnull(cs_item_sk#12) - -(18) CometProject -Input [2]: [cs_item_sk#12, cs_sold_date_sk#13] -Arguments: [cs_item_sk#12], [cs_item_sk#12] - -(19) CometColumnarToRow -Input [1]: [cs_item_sk#12] - -(20) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#12] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 4] -Output [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, cs_item_sk#12] - -(22) HashAggregate [codegen id : 4] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -(23) CometColumnarExchange -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(25) CometTakeOrderedAndProject -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(26) CometColumnarToRow [codegen id : 5] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometNativeScan parquet spark_catalog.default.date_dim (27) - - -(27) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#11, d_date#14] -Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 2000-02-01)) AND (d_date#14 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) - -(29) CometProject -Input [2]: [d_date_sk#11, d_date#14] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(31) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/simplified.txt deleted file mode 100644 index 88a3895990..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_current_price] - Project [i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [i_item_sk,cs_item_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (3) - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [inv_item_sk,inv_date_sk] - Filter [inv_quantity_on_hand,inv_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #4 - CometColumnarToRow - InputAdapter - CometProject [cs_item_sk] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometNativeScan 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/q37.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/explain.txt deleted file mode 100644 index ff90c33aff..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometTakeOrderedAndProject (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (4) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - +- CometProject (19) - +- CometFilter (18) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (17) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,68.00), LessThanOrEqual(i_current_price,98.00), In(i_manufact_id, [677,694,808,940]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 68.00)) AND (i_current_price#4 <= 98.00)) AND i_manufact_id#5 IN (677,940,694,808)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#2, 16)) AS i_item_id#6, i_item_desc#3, i_current_price#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#9), dynamicpruningexpression(inv_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Condition : (((isnotnull(inv_quantity_on_hand#8) AND (inv_quantity_on_hand#8 >= 100)) AND (inv_quantity_on_hand#8 <= 500)) AND isnotnull(inv_item_sk#7)) - -(6) CometProject -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9], [inv_item_sk#7, inv_date_sk#9] - -(7) CometBroadcastExchange -Input [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1], [inv_item_sk#7], Inner, BuildRight - -(9) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#12 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) - -(12) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(14) CometBroadcastHashJoin -Left output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [inv_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(15) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(16) CometBroadcastExchange -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#13, cs_sold_date_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [cs_item_sk#13, cs_sold_date_sk#14] -Condition : isnotnull(cs_item_sk#13) - -(19) CometProject -Input [2]: [cs_item_sk#13, cs_sold_date_sk#14] -Arguments: [cs_item_sk#13], [cs_item_sk#13] - -(20) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [1]: [cs_item_sk#13] -Arguments: [i_item_sk#1], [cs_item_sk#13], Inner, BuildLeft - -(21) CometProject -Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, cs_item_sk#13] -Arguments: [i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(22) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(23) CometExchange -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(25) CometTakeOrderedAndProject -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(26) CometColumnarToRow [codegen id : 1] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#12 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) - -(29) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(31) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/simplified.txt deleted file mode 100644 index 7ab646a628..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometExchange [i_item_id,i_item_desc,i_current_price] #1 - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometProject [i_item_id,i_item_desc,i_current_price] - 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 [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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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/q37/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/explain.txt deleted file mode 100644 index ff90c33aff..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometTakeOrderedAndProject (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (4) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - +- CometProject (19) - +- CometFilter (18) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (17) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,68.00), LessThanOrEqual(i_current_price,98.00), In(i_manufact_id, [677,694,808,940]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 68.00)) AND (i_current_price#4 <= 98.00)) AND i_manufact_id#5 IN (677,940,694,808)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#2, 16)) AS i_item_id#6, i_item_desc#3, i_current_price#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#9), dynamicpruningexpression(inv_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Condition : (((isnotnull(inv_quantity_on_hand#8) AND (inv_quantity_on_hand#8 >= 100)) AND (inv_quantity_on_hand#8 <= 500)) AND isnotnull(inv_item_sk#7)) - -(6) CometProject -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9], [inv_item_sk#7, inv_date_sk#9] - -(7) CometBroadcastExchange -Input [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1], [inv_item_sk#7], Inner, BuildRight - -(9) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#12 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) - -(12) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(14) CometBroadcastHashJoin -Left output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [inv_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(15) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(16) CometBroadcastExchange -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#13, cs_sold_date_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [cs_item_sk#13, cs_sold_date_sk#14] -Condition : isnotnull(cs_item_sk#13) - -(19) CometProject -Input [2]: [cs_item_sk#13, cs_sold_date_sk#14] -Arguments: [cs_item_sk#13], [cs_item_sk#13] - -(20) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [1]: [cs_item_sk#13] -Arguments: [i_item_sk#1], [cs_item_sk#13], Inner, BuildLeft - -(21) CometProject -Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, cs_item_sk#13] -Arguments: [i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(22) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(23) CometExchange -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(25) CometTakeOrderedAndProject -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(26) CometColumnarToRow [codegen id : 1] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#12 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) - -(29) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(31) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/extended.txt deleted file mode 100644 index abe0937d1d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/extended.txt +++ /dev/null @@ -1,34 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 7ab646a628..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometExchange [i_item_id,i_item_desc,i_current_price] #1 - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometProject [i_item_id,i_item_desc,i_current_price] - 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 [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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/explain.txt deleted file mode 100644 index 89b70107e2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/explain.txt +++ /dev/null @@ -1,317 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (49) -+- CometHashAggregate (48) - +- CometExchange (47) - +- CometHashAggregate (46) - +- CometProject (45) - +- CometBroadcastHashJoin (44) - :- CometBroadcastHashJoin (30) - : :- CometHashAggregate (16) - : : +- CometColumnarExchange (15) - : : +- * HashAggregate (14) - : : +- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.customer (7) - : +- CometBroadcastExchange (29) - : +- CometHashAggregate (28) - : +- CometColumnarExchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet spark_catalog.default.catalog_sales (17) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - +- CometBroadcastExchange (43) - +- CometHashAggregate (42) - +- CometColumnarExchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Filter (33) - : : +- * ColumnarToRow (32) - : : +- Scan parquet spark_catalog.default.web_sales (31) - : +- ReusedExchange (34) - +- ReusedExchange (37) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sold_date_sk#2 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] - -(3) Filter [codegen id : 3] -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Condition : isnotnull(ss_customer_sk#1) - -(4) ReusedExchange [Reuses operator id: 54] -Output [2]: [d_date_sk#4, d_date#5] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#2] -Right keys [1]: [d_date_sk#4] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [2]: [ss_customer_sk#1, d_date#5] -Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] - -(7) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Condition : isnotnull(c_customer_sk#6) - -(9) CometProject -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Arguments: [c_customer_sk#6, c_first_name#9, c_last_name#10], [c_customer_sk#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#7, 20)) AS c_first_name#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#8, 30)) AS c_last_name#10] - -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] - -(11) BroadcastExchange -Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#6] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [3]: [c_last_name#10, c_first_name#9, d_date#5] -Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#6, c_first_name#9, c_last_name#10] - -(14) HashAggregate [codegen id : 3] -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#5] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#5] - -(15) CometColumnarExchange -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] -Arguments: hashpartitioning(c_last_name#10, c_first_name#9, d_date#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometHashAggregate -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#5] -Functions: [] - -(17) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#12), dynamicpruningexpression(cs_sold_date_sk#12 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 6] -Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] - -(19) Filter [codegen id : 6] -Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] -Condition : isnotnull(cs_bill_customer_sk#11) - -(20) ReusedExchange [Reuses operator id: 54] -Output [2]: [d_date_sk#13, d_date#14] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#12] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [2]: [cs_bill_customer_sk#11, d_date#14] -Input [4]: [cs_bill_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13, d_date#14] - -(23) ReusedExchange [Reuses operator id: 11] -Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] - -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_bill_customer_sk#11] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [3]: [c_last_name#17, c_first_name#16, d_date#14] -Input [5]: [cs_bill_customer_sk#11, d_date#14, c_customer_sk#15, c_first_name#16, c_last_name#17] - -(26) HashAggregate [codegen id : 6] -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#17, c_first_name#16, d_date#14] - -(27) CometColumnarExchange -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, d_date#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometHashAggregate -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] -Functions: [] - -(29) CometBroadcastExchange -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Arguments: [c_last_name#17, c_first_name#16, d_date#14] - -(30) CometBroadcastHashJoin -Left output [3]: [c_last_name#10, c_first_name#9, d_date#5] -Right output [3]: [c_last_name#17, c_first_name#16, d_date#14] -Arguments: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)], [coalesce(c_last_name#17, ), isnull(c_last_name#17), coalesce(c_first_name#16, ), isnull(c_first_name#16), coalesce(d_date#14, 1970-01-01), isnull(d_date#14)], LeftSemi, BuildRight - -(31) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_sold_date_sk#19 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(32) ColumnarToRow [codegen id : 9] -Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] - -(33) Filter [codegen id : 9] -Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] -Condition : isnotnull(ws_bill_customer_sk#18) - -(34) ReusedExchange [Reuses operator id: 54] -Output [2]: [d_date_sk#20, d_date#21] - -(35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#19] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(36) Project [codegen id : 9] -Output [2]: [ws_bill_customer_sk#18, d_date#21] -Input [4]: [ws_bill_customer_sk#18, ws_sold_date_sk#19, d_date_sk#20, d_date#21] - -(37) ReusedExchange [Reuses operator id: 11] -Output [3]: [c_customer_sk#22, c_first_name#23, c_last_name#24] - -(38) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_bill_customer_sk#18] -Right keys [1]: [c_customer_sk#22] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 9] -Output [3]: [c_last_name#24, c_first_name#23, d_date#21] -Input [5]: [ws_bill_customer_sk#18, d_date#21, c_customer_sk#22, c_first_name#23, c_last_name#24] - -(40) HashAggregate [codegen id : 9] -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#24, c_first_name#23, d_date#21] - -(41) CometColumnarExchange -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, d_date#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(42) CometHashAggregate -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] -Functions: [] - -(43) CometBroadcastExchange -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: [c_last_name#24, c_first_name#23, d_date#21] - -(44) CometBroadcastHashJoin -Left output [3]: [c_last_name#10, c_first_name#9, d_date#5] -Right output [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)], [coalesce(c_last_name#24, ), isnull(c_last_name#24), coalesce(c_first_name#23, ), isnull(c_first_name#23), coalesce(d_date#21, 1970-01-01), isnull(d_date#21)], LeftSemi, BuildRight - -(45) CometProject -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] - -(46) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(47) CometExchange -Input [1]: [count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(48) CometHashAggregate -Input [1]: [count#25] -Keys: [] -Functions [1]: [count(1)] - -(49) CometColumnarToRow [codegen id : 10] -Input [1]: [count(1)#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometNativeScan parquet spark_catalog.default.date_dim (50) - - -(50) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#27] -Condition : (((isnotnull(d_month_seq#27) AND (d_month_seq#27 >= 1200)) AND (d_month_seq#27 <= 1211)) AND isnotnull(d_date_sk#4)) - -(52) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#27] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(53) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] - -(54) BroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 17 Hosting Expression = cs_sold_date_sk#12 IN dynamicpruning#3 - -Subquery:3 Hosting operator id = 31 Hosting Expression = ws_sold_date_sk#19 IN dynamicpruning#3 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/simplified.txt deleted file mode 100644 index 547965d09c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/simplified.txt +++ /dev/null @@ -1,75 +0,0 @@ -WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometBroadcastExchange [c_last_name,c_first_name,d_date] #5 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 - CometBroadcastExchange [c_last_name,c_first_name,d_date] #7 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (9) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/explain.txt deleted file mode 100644 index 3079061df7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,308 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (48) -+- CometHashAggregate (47) - +- CometExchange (46) - +- CometHashAggregate (45) - +- CometProject (44) - +- CometBroadcastHashJoin (43) - :- CometBroadcastHashJoin (30) - : :- CometHashAggregate (17) - : : +- CometExchange (16) - : : +- CometHashAggregate (15) - : : +- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : +- CometBroadcastExchange (29) - : +- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometFilter (19) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - +- CometBroadcastExchange (42) - +- CometHashAggregate (41) - +- CometExchange (40) - +- CometHashAggregate (39) - +- CometProject (38) - +- CometBroadcastHashJoin (37) - :- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (31) - : +- ReusedExchange (33) - +- ReusedExchange (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sold_date_sk#2 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Condition : isnotnull(ss_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(5) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: [d_date_sk#4, d_date#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Right output [2]: [d_date_sk#4, d_date#5] -Arguments: [ss_sold_date_sk#2], [d_date_sk#4], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] -Arguments: [ss_customer_sk#1, d_date#5], [ss_customer_sk#1, d_date#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Condition : isnotnull(c_customer_sk#7) - -(11) CometProject -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11], [c_customer_sk#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#8, 20)) AS c_first_name#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#9, 30)) AS c_last_name#11] - -(12) CometBroadcastExchange -Input [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, d_date#5] -Right output [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [ss_customer_sk#1], [c_customer_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_last_name#11, c_first_name#10, d_date#5], [c_last_name#11, c_first_name#10, d_date#5] - -(15) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(16) CometExchange -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(c_last_name#11, c_first_name#10, d_date#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Condition : isnotnull(cs_bill_customer_sk#12) - -(20) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#15, d_date#16] - -(21) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_date#16] -Arguments: [cs_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(22) CometProject -Input [4]: [cs_bill_customer_sk#12, cs_sold_date_sk#13, d_date_sk#15, d_date#16] -Arguments: [cs_bill_customer_sk#12, d_date#16], [cs_bill_customer_sk#12, d_date#16] - -(23) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] - -(24) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, d_date#16] -Right output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [cs_bill_customer_sk#12], [c_customer_sk#17], Inner, BuildRight - -(25) CometProject -Input [5]: [cs_bill_customer_sk#12, d_date#16, c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [c_last_name#19, c_first_name#18, d_date#16], [c_last_name#19, c_first_name#18, d_date#16] - -(26) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(27) CometExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: hashpartitioning(c_last_name#19, c_first_name#18, d_date#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(29) CometBroadcastExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: [c_last_name#19, c_first_name#18, d_date#16] - -(30) CometBroadcastHashJoin -Left output [3]: [c_last_name#11, c_first_name#10, d_date#5] -Right output [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)], [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 1970-01-01), isnull(d_date#16)], LeftSemi, BuildRight - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#21), dynamicpruningexpression(ws_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Condition : isnotnull(ws_bill_customer_sk#20) - -(33) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#23, d_date#24] - -(34) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Right output [2]: [d_date_sk#23, d_date#24] -Arguments: [ws_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(35) CometProject -Input [4]: [ws_bill_customer_sk#20, ws_sold_date_sk#21, d_date_sk#23, d_date#24] -Arguments: [ws_bill_customer_sk#20, d_date#24], [ws_bill_customer_sk#20, d_date#24] - -(36) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] - -(37) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, d_date#24] -Right output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [ws_bill_customer_sk#20], [c_customer_sk#25], Inner, BuildRight - -(38) CometProject -Input [5]: [ws_bill_customer_sk#20, d_date#24, c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_last_name#27, c_first_name#26, d_date#24], [c_last_name#27, c_first_name#26, d_date#24] - -(39) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(40) CometExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: hashpartitioning(c_last_name#27, c_first_name#26, d_date#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(41) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(42) CometBroadcastExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: [c_last_name#27, c_first_name#26, d_date#24] - -(43) CometBroadcastHashJoin -Left output [3]: [c_last_name#11, c_first_name#10, d_date#5] -Right output [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)], [coalesce(c_last_name#27, ), isnull(c_last_name#27), coalesce(c_first_name#26, ), isnull(c_first_name#26), coalesce(d_date#24, 1970-01-01), isnull(d_date#24)], LeftSemi, BuildRight - -(44) CometProject -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] - -(45) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(46) CometExchange -Input [1]: [count#28] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(47) CometHashAggregate -Input [1]: [count#28] -Keys: [] -Functions [1]: [count(1)] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [count(1)#29] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(51) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(52) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] - -(53) BroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#3 - -Subquery:3 Hosting operator id = 31 Hosting Expression = ws_sold_date_sk#21 IN dynamicpruning#3 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/simplified.txt deleted file mode 100644 index 7ceccf5d2c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,60 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #2 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] - CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_customer_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #4 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometBroadcastExchange [c_last_name,c_first_name,d_date] #6 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #7 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] - CometFilter [cs_bill_customer_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometBroadcastExchange [c_last_name,c_first_name,d_date] #8 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #9 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_bill_customer_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/explain.txt deleted file mode 100644 index 3079061df7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/explain.txt +++ /dev/null @@ -1,308 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (48) -+- CometHashAggregate (47) - +- CometExchange (46) - +- CometHashAggregate (45) - +- CometProject (44) - +- CometBroadcastHashJoin (43) - :- CometBroadcastHashJoin (30) - : :- CometHashAggregate (17) - : : +- CometExchange (16) - : : +- CometHashAggregate (15) - : : +- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : +- CometBroadcastExchange (29) - : +- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometFilter (19) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - +- CometBroadcastExchange (42) - +- CometHashAggregate (41) - +- CometExchange (40) - +- CometHashAggregate (39) - +- CometProject (38) - +- CometBroadcastHashJoin (37) - :- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (31) - : +- ReusedExchange (33) - +- ReusedExchange (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sold_date_sk#2 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Condition : isnotnull(ss_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(5) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: [d_date_sk#4, d_date#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Right output [2]: [d_date_sk#4, d_date#5] -Arguments: [ss_sold_date_sk#2], [d_date_sk#4], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] -Arguments: [ss_customer_sk#1, d_date#5], [ss_customer_sk#1, d_date#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Condition : isnotnull(c_customer_sk#7) - -(11) CometProject -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11], [c_customer_sk#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#8, 20)) AS c_first_name#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#9, 30)) AS c_last_name#11] - -(12) CometBroadcastExchange -Input [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, d_date#5] -Right output [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [ss_customer_sk#1], [c_customer_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_last_name#11, c_first_name#10, d_date#5], [c_last_name#11, c_first_name#10, d_date#5] - -(15) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(16) CometExchange -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(c_last_name#11, c_first_name#10, d_date#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Condition : isnotnull(cs_bill_customer_sk#12) - -(20) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#15, d_date#16] - -(21) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_date#16] -Arguments: [cs_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(22) CometProject -Input [4]: [cs_bill_customer_sk#12, cs_sold_date_sk#13, d_date_sk#15, d_date#16] -Arguments: [cs_bill_customer_sk#12, d_date#16], [cs_bill_customer_sk#12, d_date#16] - -(23) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] - -(24) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, d_date#16] -Right output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [cs_bill_customer_sk#12], [c_customer_sk#17], Inner, BuildRight - -(25) CometProject -Input [5]: [cs_bill_customer_sk#12, d_date#16, c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [c_last_name#19, c_first_name#18, d_date#16], [c_last_name#19, c_first_name#18, d_date#16] - -(26) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(27) CometExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: hashpartitioning(c_last_name#19, c_first_name#18, d_date#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(29) CometBroadcastExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: [c_last_name#19, c_first_name#18, d_date#16] - -(30) CometBroadcastHashJoin -Left output [3]: [c_last_name#11, c_first_name#10, d_date#5] -Right output [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)], [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 1970-01-01), isnull(d_date#16)], LeftSemi, BuildRight - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#21), dynamicpruningexpression(ws_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Condition : isnotnull(ws_bill_customer_sk#20) - -(33) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#23, d_date#24] - -(34) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Right output [2]: [d_date_sk#23, d_date#24] -Arguments: [ws_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(35) CometProject -Input [4]: [ws_bill_customer_sk#20, ws_sold_date_sk#21, d_date_sk#23, d_date#24] -Arguments: [ws_bill_customer_sk#20, d_date#24], [ws_bill_customer_sk#20, d_date#24] - -(36) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] - -(37) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, d_date#24] -Right output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [ws_bill_customer_sk#20], [c_customer_sk#25], Inner, BuildRight - -(38) CometProject -Input [5]: [ws_bill_customer_sk#20, d_date#24, c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_last_name#27, c_first_name#26, d_date#24], [c_last_name#27, c_first_name#26, d_date#24] - -(39) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(40) CometExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: hashpartitioning(c_last_name#27, c_first_name#26, d_date#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(41) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(42) CometBroadcastExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: [c_last_name#27, c_first_name#26, d_date#24] - -(43) CometBroadcastHashJoin -Left output [3]: [c_last_name#11, c_first_name#10, d_date#5] -Right output [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)], [coalesce(c_last_name#27, ), isnull(c_last_name#27), coalesce(c_first_name#26, ), isnull(c_first_name#26), coalesce(d_date#24, 1970-01-01), isnull(d_date#24)], LeftSemi, BuildRight - -(44) CometProject -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] - -(45) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(46) CometExchange -Input [1]: [count#28] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(47) CometHashAggregate -Input [1]: [count#28] -Keys: [] -Functions [1]: [count(1)] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [count(1)#29] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(51) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(52) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] - -(53) BroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#3 - -Subquery:3 Hosting operator id = 31 Hosting Expression = ws_sold_date_sk#21 IN dynamicpruning#3 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/extended.txt deleted file mode 100644 index 55af64efc6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/extended.txt +++ /dev/null @@ -1,70 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 62 out of 66 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 7ceccf5d2c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt +++ /dev/null @@ -1,60 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #2 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] - CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_customer_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #4 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometBroadcastExchange [c_last_name,c_first_name,d_date] #6 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #7 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] - CometFilter [cs_bill_customer_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometBroadcastExchange [c_last_name,c_first_name,d_date] #8 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #9 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_bill_customer_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/explain.txt deleted file mode 100644 index 540361017b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/explain.txt +++ /dev/null @@ -1,330 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometSort (46) - +- CometColumnarExchange (45) - +- * BroadcastHashJoin Inner BuildRight (44) - :- * Project (24) - : +- * Filter (23) - : +- * HashAggregate (22) - : +- * CometColumnarToRow (21) - : +- CometColumnarExchange (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.inventory (1) - : : : +- BroadcastExchange (7) - : : : +- * CometColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : +- BroadcastExchange (13) - : : +- * CometColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) - : +- ReusedExchange (16) - +- BroadcastExchange (43) - +- * Project (42) - +- * Filter (41) - +- * HashAggregate (40) - +- * CometColumnarToRow (39) - +- CometColumnarExchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * Filter (27) - : : : +- * ColumnarToRow (26) - : : : +- Scan parquet spark_catalog.default.inventory (25) - : : +- ReusedExchange (28) - : +- ReusedExchange (31) - +- ReusedExchange (34) - - -(1) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [1]: [i_item_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [1]: [i_item_sk#6] -Condition : isnotnull(i_item_sk#6) - -(6) CometColumnarToRow [codegen id : 1] -Input [1]: [i_item_sk#6] - -(7) BroadcastExchange -Input [1]: [i_item_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 4] -Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] - -(10) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Condition : isnotnull(w_warehouse_sk#7) - -(12) CometColumnarToRow [codegen id : 2] -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] - -(13) BroadcastExchange -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#7] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] - -(16) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#9, d_moy#10] - -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#10] - -(19) HashAggregate [codegen id : 4] -Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [5]: [n#11, avg#12, m2#13, sum#14, count#15] -Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] - -(20) CometColumnarExchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(21) CometColumnarToRow [codegen id : 10] -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] - -(22) HashAggregate [codegen id : 10] -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#21, avg(inv_quantity_on_hand#3)#22] -Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stddev_samp(cast(inv_quantity_on_hand#3 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#3)#22 AS mean#24] - -(23) Filter [codegen id : 10] -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.0) END - -(24) Project [codegen id : 10] -Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN null ELSE (stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))) END AS cov#25] -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] - -(25) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#29), dynamicpruningexpression(inv_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(26) ColumnarToRow [codegen id : 8] -Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] - -(27) Filter [codegen id : 8] -Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -Condition : (isnotnull(inv_item_sk#26) AND isnotnull(inv_warehouse_sk#27)) - -(28) ReusedExchange [Reuses operator id: 7] -Output [1]: [i_item_sk#31] - -(29) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_item_sk#26] -Right keys [1]: [i_item_sk#31] -Join type: Inner -Join condition: None - -(30) Project [codegen id : 8] -Output [4]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] -Input [5]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] - -(31) ReusedExchange [Reuses operator id: 13] -Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] - -(32) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_warehouse_sk#27] -Right keys [1]: [w_warehouse_sk#32] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] -Input [6]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] - -(34) ReusedExchange [Reuses operator id: 57] -Output [2]: [d_date_sk#34, d_moy#35] - -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_date_sk#29] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None - -(36) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] -Input [7]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_date_sk#34, d_moy#35] - -(37) HashAggregate [codegen id : 8] -Input [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] -Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#28 as double)), partial_avg(inv_quantity_on_hand#28)] -Aggregate Attributes [5]: [n#36, avg#37, m2#38, sum#39, count#40] -Results [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] - -(38) CometColumnarExchange -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(39) CometColumnarToRow [codegen id : 9] -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] - -(40) HashAggregate [codegen id : 9] -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] -Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#46, avg(inv_quantity_on_hand#28)#22 AS mean#47] - -(41) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#47)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#46 / knownfloatingpointnormalized(normalizenanandzero(mean#47))))) > 1.0) END - -(42) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#47)) = 0.0) THEN null ELSE (stdev#46 / knownfloatingpointnormalized(normalizenanandzero(mean#47))) END AS cov#48] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] - -(43) BroadcastExchange -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] - -(44) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] -Right keys [2]: [i_item_sk#31, w_warehouse_sk#32] -Join type: Inner -Join condition: None - -(45) CometColumnarExchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#47 ASC NULLS FIRST, cov#48 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(46) CometSort -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#47 ASC NULLS FIRST, cov#48 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 11] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometNativeScan parquet spark_catalog.default.date_dim (48) - - -(48) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#49, d_moy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#49, d_moy#10] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#10)) AND (d_year#49 = 2001)) AND (d_moy#10 = 1)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#49, d_moy#10] -Arguments: [d_date_sk#9, d_moy#10], [d_date_sk#9, d_moy#10] - -(51) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#9, d_moy#10] - -(52) BroadcastExchange -Input [2]: [d_date_sk#9, d_moy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 25 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometNativeScan parquet spark_catalog.default.date_dim (53) - - -(53) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#50, d_moy#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#34, d_year#50, d_moy#35] -Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#35)) AND (d_year#50 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#34)) - -(55) CometProject -Input [3]: [d_date_sk#34, d_year#50, d_moy#35] -Arguments: [d_date_sk#34, d_moy#35], [d_date_sk#34, d_moy#35] - -(56) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#34, d_moy#35] - -(57) BroadcastExchange -Input [2]: [d_date_sk#34, d_moy#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/simplified.txt deleted file mode 100644 index 11dd59a97f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/simplified.txt +++ /dev/null @@ -1,84 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 - WholeStageCodegen (8) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [i_item_sk] #4 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - InputAdapter - ReusedExchange [d_date_sk,d_moy] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/explain.txt deleted file mode 100644 index 4e7b77d7ec..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,329 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometSort (46) - +- CometExchange (45) - +- CometBroadcastHashJoin (44) - :- CometProject (23) - : +- CometFilter (22) - : +- CometHashAggregate (21) - : +- CometExchange (20) - : +- CometHashAggregate (19) - : +- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (12) - : : +- CometBroadcastHashJoin (11) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : +- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - +- CometBroadcastExchange (43) - +- CometProject (42) - +- CometFilter (41) - +- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (28) - : : +- CometBroadcastHashJoin (27) - : : :- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (24) - : : +- ReusedExchange (26) - : +- ReusedExchange (29) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [i_item_sk#6] -Condition : isnotnull(i_item_sk#6) - -(5) CometBroadcastExchange -Input [1]: [i_item_sk#6] -Arguments: [i_item_sk#6] - -(6) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [i_item_sk#6] -Arguments: [inv_item_sk#1], [i_item_sk#6], Inner, BuildRight - -(7) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6], [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Condition : isnotnull(w_warehouse_sk#7) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [w_warehouse_sk#7, w_warehouse_name#8] - -(11) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Right output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#7], Inner, BuildRight - -(12) CometProject -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8], [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(15) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11] - -(17) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Right output [2]: [d_date_sk#9, d_moy#11] -Arguments: [inv_date_sk#4], [d_date_sk#9], Inner, BuildRight - -(18) CometProject -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#11] -Arguments: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11], [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] - -(19) CometHashAggregate -Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] - -(20) CometExchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] - -(22) CometFilter -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.0) END - -(23) CometProject -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19], [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN null ELSE (stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))) END AS cov#19] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#23), dynamicpruningexpression(inv_date_sk#23 IN dynamicpruning#24)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Condition : (isnotnull(inv_item_sk#20) AND isnotnull(inv_warehouse_sk#21)) - -(26) ReusedExchange [Reuses operator id: 5] -Output [1]: [i_item_sk#25] - -(27) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Right output [1]: [i_item_sk#25] -Arguments: [inv_item_sk#20], [i_item_sk#25], Inner, BuildRight - -(28) CometProject -Input [5]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Arguments: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25], [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] - -(29) ReusedExchange [Reuses operator id: 10] -Output [2]: [w_warehouse_sk#26, w_warehouse_name#27] - -(30) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Right output [2]: [w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_warehouse_sk#21], [w_warehouse_sk#26], Inner, BuildRight - -(31) CometProject -Input [6]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27], [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(34) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(35) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30] - -(36) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Right output [2]: [d_date_sk#28, d_moy#30] -Arguments: [inv_date_sk#23], [d_date_sk#28], Inner, BuildRight - -(37) CometProject -Input [7]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_date_sk#28, d_moy#30] -Arguments: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30], [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] - -(38) CometHashAggregate -Input [5]: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#22 as double)), partial_avg(inv_quantity_on_hand#22)] - -(39) CometExchange -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Arguments: hashpartitioning(w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(40) CometHashAggregate -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#22 as double)), avg(inv_quantity_on_hand#22)] - -(41) CometFilter -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#36, mean#37] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#37)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#36 / knownfloatingpointnormalized(normalizenanandzero(mean#37))))) > 1.0) END - -(42) CometProject -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#36, mean#37] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38], [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#37)) = 0.0) THEN null ELSE (stdev#36 / knownfloatingpointnormalized(normalizenanandzero(mean#37))) END AS cov#38] - -(43) CometBroadcastExchange -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] - -(44) CometBroadcastHashJoin -Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] -Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] -Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight - -(45) CometExchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#37 ASC NULLS FIRST, cov#38 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(46) CometSort -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#37 ASC NULLS FIRST, cov#38 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 1] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(51) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#9, d_moy#11] - -(52) BroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(55) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(56) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#28, d_moy#30] - -(57) BroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/simplified.txt deleted file mode 100644 index e54435ae10..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,65 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [i_item_sk] #4 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [d_date_sk,d_moy] #6 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov] #7 - CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_moy] #10 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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/q39a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/explain.txt deleted file mode 100644 index 4e7b77d7ec..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/explain.txt +++ /dev/null @@ -1,329 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometSort (46) - +- CometExchange (45) - +- CometBroadcastHashJoin (44) - :- CometProject (23) - : +- CometFilter (22) - : +- CometHashAggregate (21) - : +- CometExchange (20) - : +- CometHashAggregate (19) - : +- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (12) - : : +- CometBroadcastHashJoin (11) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : +- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - +- CometBroadcastExchange (43) - +- CometProject (42) - +- CometFilter (41) - +- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (28) - : : +- CometBroadcastHashJoin (27) - : : :- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (24) - : : +- ReusedExchange (26) - : +- ReusedExchange (29) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [i_item_sk#6] -Condition : isnotnull(i_item_sk#6) - -(5) CometBroadcastExchange -Input [1]: [i_item_sk#6] -Arguments: [i_item_sk#6] - -(6) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [i_item_sk#6] -Arguments: [inv_item_sk#1], [i_item_sk#6], Inner, BuildRight - -(7) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6], [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Condition : isnotnull(w_warehouse_sk#7) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [w_warehouse_sk#7, w_warehouse_name#8] - -(11) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Right output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#7], Inner, BuildRight - -(12) CometProject -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8], [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(15) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11] - -(17) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Right output [2]: [d_date_sk#9, d_moy#11] -Arguments: [inv_date_sk#4], [d_date_sk#9], Inner, BuildRight - -(18) CometProject -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#11] -Arguments: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11], [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] - -(19) CometHashAggregate -Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] - -(20) CometExchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] - -(22) CometFilter -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.0) END - -(23) CometProject -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19], [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN null ELSE (stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))) END AS cov#19] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#23), dynamicpruningexpression(inv_date_sk#23 IN dynamicpruning#24)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Condition : (isnotnull(inv_item_sk#20) AND isnotnull(inv_warehouse_sk#21)) - -(26) ReusedExchange [Reuses operator id: 5] -Output [1]: [i_item_sk#25] - -(27) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Right output [1]: [i_item_sk#25] -Arguments: [inv_item_sk#20], [i_item_sk#25], Inner, BuildRight - -(28) CometProject -Input [5]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Arguments: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25], [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] - -(29) ReusedExchange [Reuses operator id: 10] -Output [2]: [w_warehouse_sk#26, w_warehouse_name#27] - -(30) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Right output [2]: [w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_warehouse_sk#21], [w_warehouse_sk#26], Inner, BuildRight - -(31) CometProject -Input [6]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27], [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(34) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(35) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30] - -(36) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Right output [2]: [d_date_sk#28, d_moy#30] -Arguments: [inv_date_sk#23], [d_date_sk#28], Inner, BuildRight - -(37) CometProject -Input [7]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_date_sk#28, d_moy#30] -Arguments: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30], [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] - -(38) CometHashAggregate -Input [5]: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#22 as double)), partial_avg(inv_quantity_on_hand#22)] - -(39) CometExchange -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Arguments: hashpartitioning(w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(40) CometHashAggregate -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#22 as double)), avg(inv_quantity_on_hand#22)] - -(41) CometFilter -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#36, mean#37] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#37)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#36 / knownfloatingpointnormalized(normalizenanandzero(mean#37))))) > 1.0) END - -(42) CometProject -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#36, mean#37] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38], [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#37)) = 0.0) THEN null ELSE (stdev#36 / knownfloatingpointnormalized(normalizenanandzero(mean#37))) END AS cov#38] - -(43) CometBroadcastExchange -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] - -(44) CometBroadcastHashJoin -Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] -Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] -Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight - -(45) CometExchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#37 ASC NULLS FIRST, cov#38 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(46) CometSort -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#37 ASC NULLS FIRST, cov#38 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 1] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(51) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#9, d_moy#11] - -(52) BroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(55) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(56) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#28, d_moy#30] - -(57) BroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/extended.txt deleted file mode 100644 index 8ff1442cf8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index e54435ae10..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt +++ /dev/null @@ -1,65 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [i_item_sk] #4 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [d_date_sk,d_moy] #6 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov] #7 - CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_moy] #10 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/explain.txt deleted file mode 100644 index 1e019e0e20..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/explain.txt +++ /dev/null @@ -1,330 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometSort (46) - +- CometColumnarExchange (45) - +- * BroadcastHashJoin Inner BuildRight (44) - :- * Project (24) - : +- * Filter (23) - : +- * HashAggregate (22) - : +- * CometColumnarToRow (21) - : +- CometColumnarExchange (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.inventory (1) - : : : +- BroadcastExchange (7) - : : : +- * CometColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : +- BroadcastExchange (13) - : : +- * CometColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) - : +- ReusedExchange (16) - +- BroadcastExchange (43) - +- * Project (42) - +- * Filter (41) - +- * HashAggregate (40) - +- * CometColumnarToRow (39) - +- CometColumnarExchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * Filter (27) - : : : +- * ColumnarToRow (26) - : : : +- Scan parquet spark_catalog.default.inventory (25) - : : +- ReusedExchange (28) - : +- ReusedExchange (31) - +- ReusedExchange (34) - - -(1) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [1]: [i_item_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [1]: [i_item_sk#6] -Condition : isnotnull(i_item_sk#6) - -(6) CometColumnarToRow [codegen id : 1] -Input [1]: [i_item_sk#6] - -(7) BroadcastExchange -Input [1]: [i_item_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 4] -Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] - -(10) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Condition : isnotnull(w_warehouse_sk#7) - -(12) CometColumnarToRow [codegen id : 2] -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] - -(13) BroadcastExchange -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#7] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] - -(16) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#9, d_moy#10] - -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#10] - -(19) HashAggregate [codegen id : 4] -Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [5]: [n#11, avg#12, m2#13, sum#14, count#15] -Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] - -(20) CometColumnarExchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(21) CometColumnarToRow [codegen id : 10] -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] - -(22) HashAggregate [codegen id : 10] -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#21, avg(inv_quantity_on_hand#3)#22] -Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stddev_samp(cast(inv_quantity_on_hand#3 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#3)#22 AS mean#24] - -(23) Filter [codegen id : 10] -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] -Condition : (CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.0) END AND CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.5) END) - -(24) Project [codegen id : 10] -Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN null ELSE (stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))) END AS cov#25] -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] - -(25) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#29), dynamicpruningexpression(inv_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(26) ColumnarToRow [codegen id : 8] -Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] - -(27) Filter [codegen id : 8] -Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -Condition : (isnotnull(inv_item_sk#26) AND isnotnull(inv_warehouse_sk#27)) - -(28) ReusedExchange [Reuses operator id: 7] -Output [1]: [i_item_sk#31] - -(29) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_item_sk#26] -Right keys [1]: [i_item_sk#31] -Join type: Inner -Join condition: None - -(30) Project [codegen id : 8] -Output [4]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] -Input [5]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] - -(31) ReusedExchange [Reuses operator id: 13] -Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] - -(32) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_warehouse_sk#27] -Right keys [1]: [w_warehouse_sk#32] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] -Input [6]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] - -(34) ReusedExchange [Reuses operator id: 57] -Output [2]: [d_date_sk#34, d_moy#35] - -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_date_sk#29] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None - -(36) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] -Input [7]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_date_sk#34, d_moy#35] - -(37) HashAggregate [codegen id : 8] -Input [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] -Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#28 as double)), partial_avg(inv_quantity_on_hand#28)] -Aggregate Attributes [5]: [n#36, avg#37, m2#38, sum#39, count#40] -Results [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] - -(38) CometColumnarExchange -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(39) CometColumnarToRow [codegen id : 9] -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] - -(40) HashAggregate [codegen id : 9] -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] -Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#46, avg(inv_quantity_on_hand#28)#22 AS mean#47] - -(41) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#47)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#46 / knownfloatingpointnormalized(normalizenanandzero(mean#47))))) > 1.0) END - -(42) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#47)) = 0.0) THEN null ELSE (stdev#46 / knownfloatingpointnormalized(normalizenanandzero(mean#47))) END AS cov#48] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] - -(43) BroadcastExchange -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] - -(44) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] -Right keys [2]: [i_item_sk#31, w_warehouse_sk#32] -Join type: Inner -Join condition: None - -(45) CometColumnarExchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#47 ASC NULLS FIRST, cov#48 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(46) CometSort -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#47 ASC NULLS FIRST, cov#48 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 11] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometNativeScan parquet spark_catalog.default.date_dim (48) - - -(48) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#49, d_moy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#49, d_moy#10] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#10)) AND (d_year#49 = 2001)) AND (d_moy#10 = 1)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#49, d_moy#10] -Arguments: [d_date_sk#9, d_moy#10], [d_date_sk#9, d_moy#10] - -(51) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#9, d_moy#10] - -(52) BroadcastExchange -Input [2]: [d_date_sk#9, d_moy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 25 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometNativeScan parquet spark_catalog.default.date_dim (53) - - -(53) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#50, d_moy#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#34, d_year#50, d_moy#35] -Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#35)) AND (d_year#50 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#34)) - -(55) CometProject -Input [3]: [d_date_sk#34, d_year#50, d_moy#35] -Arguments: [d_date_sk#34, d_moy#35], [d_date_sk#34, d_moy#35] - -(56) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#34, d_moy#35] - -(57) BroadcastExchange -Input [2]: [d_date_sk#34, d_moy#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/simplified.txt deleted file mode 100644 index 11dd59a97f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/simplified.txt +++ /dev/null @@ -1,84 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 - WholeStageCodegen (8) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [i_item_sk] #4 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - InputAdapter - ReusedExchange [d_date_sk,d_moy] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/explain.txt deleted file mode 100644 index 7d8df40ba0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,329 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometSort (46) - +- CometExchange (45) - +- CometBroadcastHashJoin (44) - :- CometProject (23) - : +- CometFilter (22) - : +- CometHashAggregate (21) - : +- CometExchange (20) - : +- CometHashAggregate (19) - : +- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (12) - : : +- CometBroadcastHashJoin (11) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : +- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - +- CometBroadcastExchange (43) - +- CometProject (42) - +- CometFilter (41) - +- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (28) - : : +- CometBroadcastHashJoin (27) - : : :- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (24) - : : +- ReusedExchange (26) - : +- ReusedExchange (29) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [i_item_sk#6] -Condition : isnotnull(i_item_sk#6) - -(5) CometBroadcastExchange -Input [1]: [i_item_sk#6] -Arguments: [i_item_sk#6] - -(6) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [i_item_sk#6] -Arguments: [inv_item_sk#1], [i_item_sk#6], Inner, BuildRight - -(7) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6], [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Condition : isnotnull(w_warehouse_sk#7) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [w_warehouse_sk#7, w_warehouse_name#8] - -(11) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Right output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#7], Inner, BuildRight - -(12) CometProject -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8], [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(15) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11] - -(17) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Right output [2]: [d_date_sk#9, d_moy#11] -Arguments: [inv_date_sk#4], [d_date_sk#9], Inner, BuildRight - -(18) CometProject -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#11] -Arguments: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11], [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] - -(19) CometHashAggregate -Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] - -(20) CometExchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] - -(22) CometFilter -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Condition : (CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.0) END AND CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.5) END) - -(23) CometProject -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19], [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN null ELSE (stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))) END AS cov#19] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#23), dynamicpruningexpression(inv_date_sk#23 IN dynamicpruning#24)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Condition : (isnotnull(inv_item_sk#20) AND isnotnull(inv_warehouse_sk#21)) - -(26) ReusedExchange [Reuses operator id: 5] -Output [1]: [i_item_sk#25] - -(27) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Right output [1]: [i_item_sk#25] -Arguments: [inv_item_sk#20], [i_item_sk#25], Inner, BuildRight - -(28) CometProject -Input [5]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Arguments: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25], [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] - -(29) ReusedExchange [Reuses operator id: 10] -Output [2]: [w_warehouse_sk#26, w_warehouse_name#27] - -(30) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Right output [2]: [w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_warehouse_sk#21], [w_warehouse_sk#26], Inner, BuildRight - -(31) CometProject -Input [6]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27], [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(34) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(35) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30] - -(36) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Right output [2]: [d_date_sk#28, d_moy#30] -Arguments: [inv_date_sk#23], [d_date_sk#28], Inner, BuildRight - -(37) CometProject -Input [7]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_date_sk#28, d_moy#30] -Arguments: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30], [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] - -(38) CometHashAggregate -Input [5]: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#22 as double)), partial_avg(inv_quantity_on_hand#22)] - -(39) CometExchange -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Arguments: hashpartitioning(w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(40) CometHashAggregate -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#22 as double)), avg(inv_quantity_on_hand#22)] - -(41) CometFilter -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#36, mean#37] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#37)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#36 / knownfloatingpointnormalized(normalizenanandzero(mean#37))))) > 1.0) END - -(42) CometProject -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#36, mean#37] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38], [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#37)) = 0.0) THEN null ELSE (stdev#36 / knownfloatingpointnormalized(normalizenanandzero(mean#37))) END AS cov#38] - -(43) CometBroadcastExchange -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] - -(44) CometBroadcastHashJoin -Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] -Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] -Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight - -(45) CometExchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#37 ASC NULLS FIRST, cov#38 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(46) CometSort -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#37 ASC NULLS FIRST, cov#38 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 1] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(51) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#9, d_moy#11] - -(52) BroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(55) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(56) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#28, d_moy#30] - -(57) BroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/simplified.txt deleted file mode 100644 index e54435ae10..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,65 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [i_item_sk] #4 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [d_date_sk,d_moy] #6 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov] #7 - CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_moy] #10 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/explain.txt deleted file mode 100644 index 7d8df40ba0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/explain.txt +++ /dev/null @@ -1,329 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometSort (46) - +- CometExchange (45) - +- CometBroadcastHashJoin (44) - :- CometProject (23) - : +- CometFilter (22) - : +- CometHashAggregate (21) - : +- CometExchange (20) - : +- CometHashAggregate (19) - : +- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (12) - : : +- CometBroadcastHashJoin (11) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : +- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - +- CometBroadcastExchange (43) - +- CometProject (42) - +- CometFilter (41) - +- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (28) - : : +- CometBroadcastHashJoin (27) - : : :- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (24) - : : +- ReusedExchange (26) - : +- ReusedExchange (29) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [i_item_sk#6] -Condition : isnotnull(i_item_sk#6) - -(5) CometBroadcastExchange -Input [1]: [i_item_sk#6] -Arguments: [i_item_sk#6] - -(6) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [i_item_sk#6] -Arguments: [inv_item_sk#1], [i_item_sk#6], Inner, BuildRight - -(7) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6], [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Condition : isnotnull(w_warehouse_sk#7) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [w_warehouse_sk#7, w_warehouse_name#8] - -(11) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Right output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#7], Inner, BuildRight - -(12) CometProject -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8], [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(15) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11] - -(17) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Right output [2]: [d_date_sk#9, d_moy#11] -Arguments: [inv_date_sk#4], [d_date_sk#9], Inner, BuildRight - -(18) CometProject -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#11] -Arguments: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11], [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] - -(19) CometHashAggregate -Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] - -(20) CometExchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] - -(22) CometFilter -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Condition : (CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.0) END AND CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.5) END) - -(23) CometProject -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19], [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN null ELSE (stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))) END AS cov#19] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#23), dynamicpruningexpression(inv_date_sk#23 IN dynamicpruning#24)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Condition : (isnotnull(inv_item_sk#20) AND isnotnull(inv_warehouse_sk#21)) - -(26) ReusedExchange [Reuses operator id: 5] -Output [1]: [i_item_sk#25] - -(27) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Right output [1]: [i_item_sk#25] -Arguments: [inv_item_sk#20], [i_item_sk#25], Inner, BuildRight - -(28) CometProject -Input [5]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Arguments: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25], [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] - -(29) ReusedExchange [Reuses operator id: 10] -Output [2]: [w_warehouse_sk#26, w_warehouse_name#27] - -(30) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Right output [2]: [w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_warehouse_sk#21], [w_warehouse_sk#26], Inner, BuildRight - -(31) CometProject -Input [6]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27], [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(34) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(35) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30] - -(36) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Right output [2]: [d_date_sk#28, d_moy#30] -Arguments: [inv_date_sk#23], [d_date_sk#28], Inner, BuildRight - -(37) CometProject -Input [7]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_date_sk#28, d_moy#30] -Arguments: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30], [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] - -(38) CometHashAggregate -Input [5]: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#22 as double)), partial_avg(inv_quantity_on_hand#22)] - -(39) CometExchange -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Arguments: hashpartitioning(w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(40) CometHashAggregate -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#22 as double)), avg(inv_quantity_on_hand#22)] - -(41) CometFilter -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#36, mean#37] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#37)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#36 / knownfloatingpointnormalized(normalizenanandzero(mean#37))))) > 1.0) END - -(42) CometProject -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#36, mean#37] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38], [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#37)) = 0.0) THEN null ELSE (stdev#36 / knownfloatingpointnormalized(normalizenanandzero(mean#37))) END AS cov#38] - -(43) CometBroadcastExchange -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] - -(44) CometBroadcastHashJoin -Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] -Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] -Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight - -(45) CometExchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#37 ASC NULLS FIRST, cov#38 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(46) CometSort -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#37 ASC NULLS FIRST, cov#38 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 1] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(51) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#9, d_moy#11] - -(52) BroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(55) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(56) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#28, d_moy#30] - -(57) BroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/extended.txt deleted file mode 100644 index 8ff1442cf8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index e54435ae10..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt +++ /dev/null @@ -1,65 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [i_item_sk] #4 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [d_date_sk,d_moy] #6 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov] #7 - CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_moy] #10 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/explain.txt deleted file mode 100644 index 57160be074..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/explain.txt +++ /dev/null @@ -1,752 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (120) -+- * Project (119) - +- * BroadcastHashJoin Inner BuildRight (118) - :- * Project (99) - : +- * BroadcastHashJoin Inner BuildRight (98) - : :- * Project (78) - : : +- * BroadcastHashJoin Inner BuildRight (77) - : : :- * Project (58) - : : : +- * BroadcastHashJoin Inner BuildRight (57) - : : : :- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Filter (18) - : : : : : +- * HashAggregate (17) - : : : : : +- * CometColumnarToRow (16) - : : : : : +- CometColumnarExchange (15) - : : : : : +- * HashAggregate (14) - : : : : : +- * Project (13) - : : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * CometColumnarToRow (4) - : : : : : : : +- CometProject (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * Filter (7) - : : : : : : +- * ColumnarToRow (6) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (5) - : : : : : +- ReusedExchange (11) - : : : : +- BroadcastExchange (36) - : : : : +- * HashAggregate (35) - : : : : +- * CometColumnarToRow (34) - : : : : +- CometColumnarExchange (33) - : : : : +- * HashAggregate (32) - : : : : +- * Project (31) - : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : :- * Project (28) - : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : :- * CometColumnarToRow (22) - : : : : : : +- CometProject (21) - : : : : : : +- CometFilter (20) - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) - : : : : : +- BroadcastExchange (26) - : : : : : +- * Filter (25) - : : : : : +- * ColumnarToRow (24) - : : : : : +- Scan parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (29) - : : : +- BroadcastExchange (56) - : : : +- * Filter (55) - : : : +- * HashAggregate (54) - : : : +- * CometColumnarToRow (53) - : : : +- CometColumnarExchange (52) - : : : +- * HashAggregate (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * Project (47) - : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : :- * CometColumnarToRow (41) - : : : : : +- CometProject (40) - : : : : : +- CometFilter (39) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (38) - : : : : +- BroadcastExchange (45) - : : : : +- * Filter (44) - : : : : +- * ColumnarToRow (43) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (42) - : : : +- ReusedExchange (48) - : : +- BroadcastExchange (76) - : : +- * HashAggregate (75) - : : +- * CometColumnarToRow (74) - : : +- CometColumnarExchange (73) - : : +- * HashAggregate (72) - : : +- * Project (71) - : : +- * BroadcastHashJoin Inner BuildRight (70) - : : :- * Project (68) - : : : +- * BroadcastHashJoin Inner BuildRight (67) - : : : :- * CometColumnarToRow (62) - : : : : +- CometProject (61) - : : : : +- CometFilter (60) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (59) - : : : +- BroadcastExchange (66) - : : : +- * Filter (65) - : : : +- * ColumnarToRow (64) - : : : +- Scan parquet spark_catalog.default.catalog_sales (63) - : : +- ReusedExchange (69) - : +- BroadcastExchange (97) - : +- * Filter (96) - : +- * HashAggregate (95) - : +- * CometColumnarToRow (94) - : +- CometColumnarExchange (93) - : +- * HashAggregate (92) - : +- * Project (91) - : +- * BroadcastHashJoin Inner BuildRight (90) - : :- * Project (88) - : : +- * BroadcastHashJoin Inner BuildRight (87) - : : :- * CometColumnarToRow (82) - : : : +- CometProject (81) - : : : +- CometFilter (80) - : : : +- CometNativeScan parquet spark_catalog.default.customer (79) - : : +- BroadcastExchange (86) - : : +- * Filter (85) - : : +- * ColumnarToRow (84) - : : +- Scan parquet spark_catalog.default.web_sales (83) - : +- ReusedExchange (89) - +- BroadcastExchange (117) - +- * HashAggregate (116) - +- * CometColumnarToRow (115) - +- CometColumnarExchange (114) - +- * HashAggregate (113) - +- * Project (112) - +- * BroadcastHashJoin Inner BuildRight (111) - :- * Project (109) - : +- * BroadcastHashJoin Inner BuildRight (108) - : :- * CometColumnarToRow (103) - : : +- CometProject (102) - : : +- CometFilter (101) - : : +- CometNativeScan parquet spark_catalog.default.customer (100) - : +- BroadcastExchange (107) - : +- * Filter (106) - : +- * ColumnarToRow (105) - : +- Scan parquet spark_catalog.default.web_sales (104) - +- ReusedExchange (110) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)) AS c_customer_id#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#3, 20)) AS c_first_name#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#4, 30)) AS c_last_name#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#5, 1)) AS c_preferred_cust_flag#12, c_birth_country#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#7, 13)) AS c_login#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#8, 50)) AS c_email_address#14] - -(4) CometColumnarToRow [codegen id : 3] -Input [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] - -(5) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#20), dynamicpruningexpression(ss_sold_date_sk#20 IN dynamicpruning#21)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] - -(7) Filter [codegen id : 1] -Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Condition : isnotnull(ss_customer_sk#15) - -(8) BroadcastExchange -Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#15] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Input [14]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] - -(11) ReusedExchange [Reuses operator id: 124] -Output [2]: [d_date_sk#22, d_year#23] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#20] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] -Input [14]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20, d_date_sk#22, d_year#23] - -(14) HashAggregate [codegen id : 3] -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] -Functions [1]: [partial_sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] -Aggregate Attributes [2]: [sum#24, isEmpty#25] -Results [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#26, isEmpty#27] - -(15) CometColumnarExchange -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#26, isEmpty#27] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 24] -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#26, isEmpty#27] - -(17) HashAggregate [codegen id : 24] -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#26, isEmpty#27] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] -Functions [1]: [sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] -Aggregate Attributes [1]: [sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))#28] -Results [2]: [c_customer_id#9 AS customer_id#29, sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))#28 AS year_total#30] - -(18) Filter [codegen id : 24] -Input [2]: [customer_id#29, year_total#30] -Condition : (isnotnull(year_total#30) AND (year_total#30 > 0.000000)) - -(19) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_country#36, c_login#37, c_email_address#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(20) CometFilter -Input [8]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_country#36, c_login#37, c_email_address#38] -Condition : (isnotnull(c_customer_sk#31) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#32, 16)))) - -(21) CometProject -Input [8]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_country#36, c_login#37, c_email_address#38] -Arguments: [c_customer_sk#31, c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44], [c_customer_sk#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#32, 16)) AS c_customer_id#39, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#33, 20)) AS c_first_name#40, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#34, 30)) AS c_last_name#41, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#35, 1)) AS c_preferred_cust_flag#42, c_birth_country#36, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#37, 13)) AS c_login#43, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#38, 50)) AS c_email_address#44] - -(22) CometColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#31, c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44] - -(23) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_customer_sk#45, ss_ext_discount_amt#46, ss_ext_sales_price#47, ss_ext_wholesale_cost#48, ss_ext_list_price#49, ss_sold_date_sk#50] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#50), dynamicpruningexpression(ss_sold_date_sk#50 IN dynamicpruning#51)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [6]: [ss_customer_sk#45, ss_ext_discount_amt#46, ss_ext_sales_price#47, ss_ext_wholesale_cost#48, ss_ext_list_price#49, ss_sold_date_sk#50] - -(25) Filter [codegen id : 4] -Input [6]: [ss_customer_sk#45, ss_ext_discount_amt#46, ss_ext_sales_price#47, ss_ext_wholesale_cost#48, ss_ext_list_price#49, ss_sold_date_sk#50] -Condition : isnotnull(ss_customer_sk#45) - -(26) BroadcastExchange -Input [6]: [ss_customer_sk#45, ss_ext_discount_amt#46, ss_ext_sales_price#47, ss_ext_wholesale_cost#48, ss_ext_list_price#49, ss_sold_date_sk#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#31] -Right keys [1]: [ss_customer_sk#45] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [12]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, ss_ext_discount_amt#46, ss_ext_sales_price#47, ss_ext_wholesale_cost#48, ss_ext_list_price#49, ss_sold_date_sk#50] -Input [14]: [c_customer_sk#31, c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, ss_customer_sk#45, ss_ext_discount_amt#46, ss_ext_sales_price#47, ss_ext_wholesale_cost#48, ss_ext_list_price#49, ss_sold_date_sk#50] - -(29) ReusedExchange [Reuses operator id: 128] -Output [2]: [d_date_sk#52, d_year#53] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#50] -Right keys [1]: [d_date_sk#52] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [12]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, ss_ext_discount_amt#46, ss_ext_sales_price#47, ss_ext_wholesale_cost#48, ss_ext_list_price#49, d_year#53] -Input [14]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, ss_ext_discount_amt#46, ss_ext_sales_price#47, ss_ext_wholesale_cost#48, ss_ext_list_price#49, ss_sold_date_sk#50, d_date_sk#52, d_year#53] - -(32) HashAggregate [codegen id : 6] -Input [12]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, ss_ext_discount_amt#46, ss_ext_sales_price#47, ss_ext_wholesale_cost#48, ss_ext_list_price#49, d_year#53] -Keys [8]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, d_year#53] -Functions [1]: [partial_sum(((((ss_ext_list_price#49 - ss_ext_wholesale_cost#48) - ss_ext_discount_amt#46) + ss_ext_sales_price#47) / 2))] -Aggregate Attributes [2]: [sum#54, isEmpty#55] -Results [10]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, d_year#53, sum#56, isEmpty#57] - -(33) CometColumnarExchange -Input [10]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, d_year#53, sum#56, isEmpty#57] -Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, d_year#53, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 7] -Input [10]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, d_year#53, sum#56, isEmpty#57] - -(35) HashAggregate [codegen id : 7] -Input [10]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, d_year#53, sum#56, isEmpty#57] -Keys [8]: [c_customer_id#39, c_first_name#40, c_last_name#41, c_preferred_cust_flag#42, c_birth_country#36, c_login#43, c_email_address#44, d_year#53] -Functions [1]: [sum(((((ss_ext_list_price#49 - ss_ext_wholesale_cost#48) - ss_ext_discount_amt#46) + ss_ext_sales_price#47) / 2))] -Aggregate Attributes [1]: [sum(((((ss_ext_list_price#49 - ss_ext_wholesale_cost#48) - ss_ext_discount_amt#46) + ss_ext_sales_price#47) / 2))#28] -Results [8]: [c_customer_id#39 AS customer_id#58, c_first_name#40 AS customer_first_name#59, c_last_name#41 AS customer_last_name#60, c_preferred_cust_flag#42 AS customer_preferred_cust_flag#61, c_birth_country#36 AS customer_birth_country#62, c_login#43 AS customer_login#63, c_email_address#44 AS customer_email_address#64, sum(((((ss_ext_list_price#49 - ss_ext_wholesale_cost#48) - ss_ext_discount_amt#46) + ss_ext_sales_price#47) / 2))#28 AS year_total#65] - -(36) BroadcastExchange -Input [8]: [customer_id#58, customer_first_name#59, customer_last_name#60, customer_preferred_cust_flag#61, customer_birth_country#62, customer_login#63, customer_email_address#64, year_total#65] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(37) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#29] -Right keys [1]: [customer_id#58] -Join type: Inner -Join condition: None - -(38) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#66, c_customer_id#67, c_first_name#68, c_last_name#69, c_preferred_cust_flag#70, c_birth_country#71, c_login#72, c_email_address#73] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(39) CometFilter -Input [8]: [c_customer_sk#66, c_customer_id#67, c_first_name#68, c_last_name#69, c_preferred_cust_flag#70, c_birth_country#71, c_login#72, c_email_address#73] -Condition : (isnotnull(c_customer_sk#66) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#67, 16)))) - -(40) CometProject -Input [8]: [c_customer_sk#66, c_customer_id#67, c_first_name#68, c_last_name#69, c_preferred_cust_flag#70, c_birth_country#71, c_login#72, c_email_address#73] -Arguments: [c_customer_sk#66, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79], [c_customer_sk#66, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#67, 16)) AS c_customer_id#74, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#68, 20)) AS c_first_name#75, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#69, 30)) AS c_last_name#76, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#70, 1)) AS c_preferred_cust_flag#77, c_birth_country#71, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#72, 13)) AS c_login#78, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#73, 50)) AS c_email_address#79] - -(41) CometColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#66, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79] - -(42) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#80, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#85), dynamicpruningexpression(cs_sold_date_sk#85 IN dynamicpruning#21)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 8] -Input [6]: [cs_bill_customer_sk#80, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] - -(44) Filter [codegen id : 8] -Input [6]: [cs_bill_customer_sk#80, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] -Condition : isnotnull(cs_bill_customer_sk#80) - -(45) BroadcastExchange -Input [6]: [cs_bill_customer_sk#80, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -(46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#66] -Right keys [1]: [cs_bill_customer_sk#80] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 10] -Output [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] -Input [14]: [c_customer_sk#66, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, cs_bill_customer_sk#80, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85] - -(48) ReusedExchange [Reuses operator id: 124] -Output [2]: [d_date_sk#86, d_year#87] - -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#85] -Right keys [1]: [d_date_sk#86] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 10] -Output [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, d_year#87] -Input [14]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, cs_sold_date_sk#85, d_date_sk#86, d_year#87] - -(51) HashAggregate [codegen id : 10] -Input [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, cs_ext_discount_amt#81, cs_ext_sales_price#82, cs_ext_wholesale_cost#83, cs_ext_list_price#84, d_year#87] -Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, d_year#87] -Functions [1]: [partial_sum(((((cs_ext_list_price#84 - cs_ext_wholesale_cost#83) - cs_ext_discount_amt#81) + cs_ext_sales_price#82) / 2))] -Aggregate Attributes [2]: [sum#88, isEmpty#89] -Results [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, d_year#87, sum#90, isEmpty#91] - -(52) CometColumnarExchange -Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, d_year#87, sum#90, isEmpty#91] -Arguments: hashpartitioning(c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, d_year#87, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(53) CometColumnarToRow [codegen id : 11] -Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, d_year#87, sum#90, isEmpty#91] - -(54) HashAggregate [codegen id : 11] -Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, d_year#87, sum#90, isEmpty#91] -Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#71, c_login#78, c_email_address#79, d_year#87] -Functions [1]: [sum(((((cs_ext_list_price#84 - cs_ext_wholesale_cost#83) - cs_ext_discount_amt#81) + cs_ext_sales_price#82) / 2))] -Aggregate Attributes [1]: [sum(((((cs_ext_list_price#84 - cs_ext_wholesale_cost#83) - cs_ext_discount_amt#81) + cs_ext_sales_price#82) / 2))#92] -Results [2]: [c_customer_id#74 AS customer_id#93, sum(((((cs_ext_list_price#84 - cs_ext_wholesale_cost#83) - cs_ext_discount_amt#81) + cs_ext_sales_price#82) / 2))#92 AS year_total#94] - -(55) Filter [codegen id : 11] -Input [2]: [customer_id#93, year_total#94] -Condition : (isnotnull(year_total#94) AND (year_total#94 > 0.000000)) - -(56) BroadcastExchange -Input [2]: [customer_id#93, year_total#94] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(57) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#29] -Right keys [1]: [customer_id#93] -Join type: Inner -Join condition: None - -(58) Project [codegen id : 24] -Output [11]: [customer_id#29, year_total#30, customer_id#58, customer_first_name#59, customer_last_name#60, customer_preferred_cust_flag#61, customer_birth_country#62, customer_login#63, customer_email_address#64, year_total#65, year_total#94] -Input [12]: [customer_id#29, year_total#30, customer_id#58, customer_first_name#59, customer_last_name#60, customer_preferred_cust_flag#61, customer_birth_country#62, customer_login#63, customer_email_address#64, year_total#65, customer_id#93, year_total#94] - -(59) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#95, c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#100, c_login#101, c_email_address#102] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(60) CometFilter -Input [8]: [c_customer_sk#95, c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#100, c_login#101, c_email_address#102] -Condition : (isnotnull(c_customer_sk#95) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#96, 16)))) - -(61) CometProject -Input [8]: [c_customer_sk#95, c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#100, c_login#101, c_email_address#102] -Arguments: [c_customer_sk#95, c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108], [c_customer_sk#95, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#96, 16)) AS c_customer_id#103, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#97, 20)) AS c_first_name#104, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#98, 30)) AS c_last_name#105, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#99, 1)) AS c_preferred_cust_flag#106, c_birth_country#100, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#101, 13)) AS c_login#107, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#102, 50)) AS c_email_address#108] - -(62) CometColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#95, c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108] - -(63) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#109, cs_ext_discount_amt#110, cs_ext_sales_price#111, cs_ext_wholesale_cost#112, cs_ext_list_price#113, cs_sold_date_sk#114] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#114), dynamicpruningexpression(cs_sold_date_sk#114 IN dynamicpruning#51)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(64) ColumnarToRow [codegen id : 12] -Input [6]: [cs_bill_customer_sk#109, cs_ext_discount_amt#110, cs_ext_sales_price#111, cs_ext_wholesale_cost#112, cs_ext_list_price#113, cs_sold_date_sk#114] - -(65) Filter [codegen id : 12] -Input [6]: [cs_bill_customer_sk#109, cs_ext_discount_amt#110, cs_ext_sales_price#111, cs_ext_wholesale_cost#112, cs_ext_list_price#113, cs_sold_date_sk#114] -Condition : isnotnull(cs_bill_customer_sk#109) - -(66) BroadcastExchange -Input [6]: [cs_bill_customer_sk#109, cs_ext_discount_amt#110, cs_ext_sales_price#111, cs_ext_wholesale_cost#112, cs_ext_list_price#113, cs_sold_date_sk#114] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#95] -Right keys [1]: [cs_bill_customer_sk#109] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 14] -Output [12]: [c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, cs_ext_discount_amt#110, cs_ext_sales_price#111, cs_ext_wholesale_cost#112, cs_ext_list_price#113, cs_sold_date_sk#114] -Input [14]: [c_customer_sk#95, c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, cs_bill_customer_sk#109, cs_ext_discount_amt#110, cs_ext_sales_price#111, cs_ext_wholesale_cost#112, cs_ext_list_price#113, cs_sold_date_sk#114] - -(69) ReusedExchange [Reuses operator id: 128] -Output [2]: [d_date_sk#115, d_year#116] - -(70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [cs_sold_date_sk#114] -Right keys [1]: [d_date_sk#115] -Join type: Inner -Join condition: None - -(71) Project [codegen id : 14] -Output [12]: [c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, cs_ext_discount_amt#110, cs_ext_sales_price#111, cs_ext_wholesale_cost#112, cs_ext_list_price#113, d_year#116] -Input [14]: [c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, cs_ext_discount_amt#110, cs_ext_sales_price#111, cs_ext_wholesale_cost#112, cs_ext_list_price#113, cs_sold_date_sk#114, d_date_sk#115, d_year#116] - -(72) HashAggregate [codegen id : 14] -Input [12]: [c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, cs_ext_discount_amt#110, cs_ext_sales_price#111, cs_ext_wholesale_cost#112, cs_ext_list_price#113, d_year#116] -Keys [8]: [c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, d_year#116] -Functions [1]: [partial_sum(((((cs_ext_list_price#113 - cs_ext_wholesale_cost#112) - cs_ext_discount_amt#110) + cs_ext_sales_price#111) / 2))] -Aggregate Attributes [2]: [sum#117, isEmpty#118] -Results [10]: [c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, d_year#116, sum#119, isEmpty#120] - -(73) CometColumnarExchange -Input [10]: [c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, d_year#116, sum#119, isEmpty#120] -Arguments: hashpartitioning(c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, d_year#116, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(74) CometColumnarToRow [codegen id : 15] -Input [10]: [c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, d_year#116, sum#119, isEmpty#120] - -(75) HashAggregate [codegen id : 15] -Input [10]: [c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, d_year#116, sum#119, isEmpty#120] -Keys [8]: [c_customer_id#103, c_first_name#104, c_last_name#105, c_preferred_cust_flag#106, c_birth_country#100, c_login#107, c_email_address#108, d_year#116] -Functions [1]: [sum(((((cs_ext_list_price#113 - cs_ext_wholesale_cost#112) - cs_ext_discount_amt#110) + cs_ext_sales_price#111) / 2))] -Aggregate Attributes [1]: [sum(((((cs_ext_list_price#113 - cs_ext_wholesale_cost#112) - cs_ext_discount_amt#110) + cs_ext_sales_price#111) / 2))#92] -Results [2]: [c_customer_id#103 AS customer_id#121, sum(((((cs_ext_list_price#113 - cs_ext_wholesale_cost#112) - cs_ext_discount_amt#110) + cs_ext_sales_price#111) / 2))#92 AS year_total#122] - -(76) BroadcastExchange -Input [2]: [customer_id#121, year_total#122] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] - -(77) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#29] -Right keys [1]: [customer_id#121] -Join type: Inner -Join condition: (CASE WHEN (year_total#94 > 0.000000) THEN (year_total#122 / year_total#94) END > CASE WHEN (year_total#30 > 0.000000) THEN (year_total#65 / year_total#30) END) - -(78) Project [codegen id : 24] -Output [10]: [customer_id#29, customer_id#58, customer_first_name#59, customer_last_name#60, customer_preferred_cust_flag#61, customer_birth_country#62, customer_login#63, customer_email_address#64, year_total#94, year_total#122] -Input [13]: [customer_id#29, year_total#30, customer_id#58, customer_first_name#59, customer_last_name#60, customer_preferred_cust_flag#61, customer_birth_country#62, customer_login#63, customer_email_address#64, year_total#65, year_total#94, customer_id#121, year_total#122] - -(79) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#123, c_customer_id#124, c_first_name#125, c_last_name#126, c_preferred_cust_flag#127, c_birth_country#128, c_login#129, c_email_address#130] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(80) CometFilter -Input [8]: [c_customer_sk#123, c_customer_id#124, c_first_name#125, c_last_name#126, c_preferred_cust_flag#127, c_birth_country#128, c_login#129, c_email_address#130] -Condition : (isnotnull(c_customer_sk#123) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#124, 16)))) - -(81) CometProject -Input [8]: [c_customer_sk#123, c_customer_id#124, c_first_name#125, c_last_name#126, c_preferred_cust_flag#127, c_birth_country#128, c_login#129, c_email_address#130] -Arguments: [c_customer_sk#123, c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136], [c_customer_sk#123, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#124, 16)) AS c_customer_id#131, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#125, 20)) AS c_first_name#132, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#126, 30)) AS c_last_name#133, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#127, 1)) AS c_preferred_cust_flag#134, c_birth_country#128, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#129, 13)) AS c_login#135, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#130, 50)) AS c_email_address#136] - -(82) CometColumnarToRow [codegen id : 18] -Input [8]: [c_customer_sk#123, c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136] - -(83) Scan parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#137, ws_ext_discount_amt#138, ws_ext_sales_price#139, ws_ext_wholesale_cost#140, ws_ext_list_price#141, ws_sold_date_sk#142] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#142), dynamicpruningexpression(ws_sold_date_sk#142 IN dynamicpruning#21)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(84) ColumnarToRow [codegen id : 16] -Input [6]: [ws_bill_customer_sk#137, ws_ext_discount_amt#138, ws_ext_sales_price#139, ws_ext_wholesale_cost#140, ws_ext_list_price#141, ws_sold_date_sk#142] - -(85) Filter [codegen id : 16] -Input [6]: [ws_bill_customer_sk#137, ws_ext_discount_amt#138, ws_ext_sales_price#139, ws_ext_wholesale_cost#140, ws_ext_list_price#141, ws_sold_date_sk#142] -Condition : isnotnull(ws_bill_customer_sk#137) - -(86) BroadcastExchange -Input [6]: [ws_bill_customer_sk#137, ws_ext_discount_amt#138, ws_ext_sales_price#139, ws_ext_wholesale_cost#140, ws_ext_list_price#141, ws_sold_date_sk#142] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -(87) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_customer_sk#123] -Right keys [1]: [ws_bill_customer_sk#137] -Join type: Inner -Join condition: None - -(88) Project [codegen id : 18] -Output [12]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, ws_ext_discount_amt#138, ws_ext_sales_price#139, ws_ext_wholesale_cost#140, ws_ext_list_price#141, ws_sold_date_sk#142] -Input [14]: [c_customer_sk#123, c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, ws_bill_customer_sk#137, ws_ext_discount_amt#138, ws_ext_sales_price#139, ws_ext_wholesale_cost#140, ws_ext_list_price#141, ws_sold_date_sk#142] - -(89) ReusedExchange [Reuses operator id: 124] -Output [2]: [d_date_sk#143, d_year#144] - -(90) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#142] -Right keys [1]: [d_date_sk#143] -Join type: Inner -Join condition: None - -(91) Project [codegen id : 18] -Output [12]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, ws_ext_discount_amt#138, ws_ext_sales_price#139, ws_ext_wholesale_cost#140, ws_ext_list_price#141, d_year#144] -Input [14]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, ws_ext_discount_amt#138, ws_ext_sales_price#139, ws_ext_wholesale_cost#140, ws_ext_list_price#141, ws_sold_date_sk#142, d_date_sk#143, d_year#144] - -(92) HashAggregate [codegen id : 18] -Input [12]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, ws_ext_discount_amt#138, ws_ext_sales_price#139, ws_ext_wholesale_cost#140, ws_ext_list_price#141, d_year#144] -Keys [8]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, d_year#144] -Functions [1]: [partial_sum(((((ws_ext_list_price#141 - ws_ext_wholesale_cost#140) - ws_ext_discount_amt#138) + ws_ext_sales_price#139) / 2))] -Aggregate Attributes [2]: [sum#145, isEmpty#146] -Results [10]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, d_year#144, sum#147, isEmpty#148] - -(93) CometColumnarExchange -Input [10]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, d_year#144, sum#147, isEmpty#148] -Arguments: hashpartitioning(c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, d_year#144, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(94) CometColumnarToRow [codegen id : 19] -Input [10]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, d_year#144, sum#147, isEmpty#148] - -(95) HashAggregate [codegen id : 19] -Input [10]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, d_year#144, sum#147, isEmpty#148] -Keys [8]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#128, c_login#135, c_email_address#136, d_year#144] -Functions [1]: [sum(((((ws_ext_list_price#141 - ws_ext_wholesale_cost#140) - ws_ext_discount_amt#138) + ws_ext_sales_price#139) / 2))] -Aggregate Attributes [1]: [sum(((((ws_ext_list_price#141 - ws_ext_wholesale_cost#140) - ws_ext_discount_amt#138) + ws_ext_sales_price#139) / 2))#149] -Results [2]: [c_customer_id#131 AS customer_id#150, sum(((((ws_ext_list_price#141 - ws_ext_wholesale_cost#140) - ws_ext_discount_amt#138) + ws_ext_sales_price#139) / 2))#149 AS year_total#151] - -(96) Filter [codegen id : 19] -Input [2]: [customer_id#150, year_total#151] -Condition : (isnotnull(year_total#151) AND (year_total#151 > 0.000000)) - -(97) BroadcastExchange -Input [2]: [customer_id#150, year_total#151] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=14] - -(98) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#29] -Right keys [1]: [customer_id#150] -Join type: Inner -Join condition: None - -(99) Project [codegen id : 24] -Output [11]: [customer_id#29, customer_id#58, customer_first_name#59, customer_last_name#60, customer_preferred_cust_flag#61, customer_birth_country#62, customer_login#63, customer_email_address#64, year_total#94, year_total#122, year_total#151] -Input [12]: [customer_id#29, customer_id#58, customer_first_name#59, customer_last_name#60, customer_preferred_cust_flag#61, customer_birth_country#62, customer_login#63, customer_email_address#64, year_total#94, year_total#122, customer_id#150, year_total#151] - -(100) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#152, c_customer_id#153, c_first_name#154, c_last_name#155, c_preferred_cust_flag#156, c_birth_country#157, c_login#158, c_email_address#159] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(101) CometFilter -Input [8]: [c_customer_sk#152, c_customer_id#153, c_first_name#154, c_last_name#155, c_preferred_cust_flag#156, c_birth_country#157, c_login#158, c_email_address#159] -Condition : (isnotnull(c_customer_sk#152) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#153, 16)))) - -(102) CometProject -Input [8]: [c_customer_sk#152, c_customer_id#153, c_first_name#154, c_last_name#155, c_preferred_cust_flag#156, c_birth_country#157, c_login#158, c_email_address#159] -Arguments: [c_customer_sk#152, c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165], [c_customer_sk#152, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#153, 16)) AS c_customer_id#160, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#154, 20)) AS c_first_name#161, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#155, 30)) AS c_last_name#162, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#156, 1)) AS c_preferred_cust_flag#163, c_birth_country#157, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#158, 13)) AS c_login#164, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#159, 50)) AS c_email_address#165] - -(103) CometColumnarToRow [codegen id : 22] -Input [8]: [c_customer_sk#152, c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165] - -(104) Scan parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#166, ws_ext_discount_amt#167, ws_ext_sales_price#168, ws_ext_wholesale_cost#169, ws_ext_list_price#170, ws_sold_date_sk#171] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#171), dynamicpruningexpression(ws_sold_date_sk#171 IN dynamicpruning#51)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(105) ColumnarToRow [codegen id : 20] -Input [6]: [ws_bill_customer_sk#166, ws_ext_discount_amt#167, ws_ext_sales_price#168, ws_ext_wholesale_cost#169, ws_ext_list_price#170, ws_sold_date_sk#171] - -(106) Filter [codegen id : 20] -Input [6]: [ws_bill_customer_sk#166, ws_ext_discount_amt#167, ws_ext_sales_price#168, ws_ext_wholesale_cost#169, ws_ext_list_price#170, ws_sold_date_sk#171] -Condition : isnotnull(ws_bill_customer_sk#166) - -(107) BroadcastExchange -Input [6]: [ws_bill_customer_sk#166, ws_ext_discount_amt#167, ws_ext_sales_price#168, ws_ext_wholesale_cost#169, ws_ext_list_price#170, ws_sold_date_sk#171] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] - -(108) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [c_customer_sk#152] -Right keys [1]: [ws_bill_customer_sk#166] -Join type: Inner -Join condition: None - -(109) Project [codegen id : 22] -Output [12]: [c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, ws_ext_discount_amt#167, ws_ext_sales_price#168, ws_ext_wholesale_cost#169, ws_ext_list_price#170, ws_sold_date_sk#171] -Input [14]: [c_customer_sk#152, c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, ws_bill_customer_sk#166, ws_ext_discount_amt#167, ws_ext_sales_price#168, ws_ext_wholesale_cost#169, ws_ext_list_price#170, ws_sold_date_sk#171] - -(110) ReusedExchange [Reuses operator id: 128] -Output [2]: [d_date_sk#172, d_year#173] - -(111) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#171] -Right keys [1]: [d_date_sk#172] -Join type: Inner -Join condition: None - -(112) Project [codegen id : 22] -Output [12]: [c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, ws_ext_discount_amt#167, ws_ext_sales_price#168, ws_ext_wholesale_cost#169, ws_ext_list_price#170, d_year#173] -Input [14]: [c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, ws_ext_discount_amt#167, ws_ext_sales_price#168, ws_ext_wholesale_cost#169, ws_ext_list_price#170, ws_sold_date_sk#171, d_date_sk#172, d_year#173] - -(113) HashAggregate [codegen id : 22] -Input [12]: [c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, ws_ext_discount_amt#167, ws_ext_sales_price#168, ws_ext_wholesale_cost#169, ws_ext_list_price#170, d_year#173] -Keys [8]: [c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, d_year#173] -Functions [1]: [partial_sum(((((ws_ext_list_price#170 - ws_ext_wholesale_cost#169) - ws_ext_discount_amt#167) + ws_ext_sales_price#168) / 2))] -Aggregate Attributes [2]: [sum#174, isEmpty#175] -Results [10]: [c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, d_year#173, sum#176, isEmpty#177] - -(114) CometColumnarExchange -Input [10]: [c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, d_year#173, sum#176, isEmpty#177] -Arguments: hashpartitioning(c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, d_year#173, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(115) CometColumnarToRow [codegen id : 23] -Input [10]: [c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, d_year#173, sum#176, isEmpty#177] - -(116) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, d_year#173, sum#176, isEmpty#177] -Keys [8]: [c_customer_id#160, c_first_name#161, c_last_name#162, c_preferred_cust_flag#163, c_birth_country#157, c_login#164, c_email_address#165, d_year#173] -Functions [1]: [sum(((((ws_ext_list_price#170 - ws_ext_wholesale_cost#169) - ws_ext_discount_amt#167) + ws_ext_sales_price#168) / 2))] -Aggregate Attributes [1]: [sum(((((ws_ext_list_price#170 - ws_ext_wholesale_cost#169) - ws_ext_discount_amt#167) + ws_ext_sales_price#168) / 2))#149] -Results [2]: [c_customer_id#160 AS customer_id#178, sum(((((ws_ext_list_price#170 - ws_ext_wholesale_cost#169) - ws_ext_discount_amt#167) + ws_ext_sales_price#168) / 2))#149 AS year_total#179] - -(117) BroadcastExchange -Input [2]: [customer_id#178, year_total#179] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=17] - -(118) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#29] -Right keys [1]: [customer_id#178] -Join type: Inner -Join condition: (CASE WHEN (year_total#94 > 0.000000) THEN (year_total#122 / year_total#94) END > CASE WHEN (year_total#151 > 0.000000) THEN (year_total#179 / year_total#151) END) - -(119) Project [codegen id : 24] -Output [7]: [customer_id#58, customer_first_name#59, customer_last_name#60, customer_preferred_cust_flag#61, customer_birth_country#62, customer_login#63, customer_email_address#64] -Input [13]: [customer_id#29, customer_id#58, customer_first_name#59, customer_last_name#60, customer_preferred_cust_flag#61, customer_birth_country#62, customer_login#63, customer_email_address#64, year_total#94, year_total#122, year_total#151, customer_id#178, year_total#179] - -(120) TakeOrderedAndProject -Input [7]: [customer_id#58, customer_first_name#59, customer_last_name#60, customer_preferred_cust_flag#61, customer_birth_country#62, customer_login#63, customer_email_address#64] -Arguments: 100, [customer_id#58 ASC NULLS FIRST, customer_first_name#59 ASC NULLS FIRST, customer_last_name#60 ASC NULLS FIRST, customer_preferred_cust_flag#61 ASC NULLS FIRST, customer_birth_country#62 ASC NULLS FIRST, customer_login#63 ASC NULLS FIRST, customer_email_address#64 ASC NULLS FIRST], [customer_id#58, customer_first_name#59, customer_last_name#60, customer_preferred_cust_flag#61, customer_birth_country#62, customer_login#63, customer_email_address#64] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 -BroadcastExchange (124) -+- * CometColumnarToRow (123) - +- CometFilter (122) - +- CometNativeScan parquet spark_catalog.default.date_dim (121) - - -(121) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_year#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(122) CometFilter -Input [2]: [d_date_sk#22, d_year#23] -Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(123) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#22, d_year#23] - -(124) BroadcastExchange -Input [2]: [d_date_sk#22, d_year#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#50 IN dynamicpruning#51 -BroadcastExchange (128) -+- * CometColumnarToRow (127) - +- CometFilter (126) - +- CometNativeScan parquet spark_catalog.default.date_dim (125) - - -(125) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#52, d_year#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#52, d_year#53] -Condition : ((isnotnull(d_year#53) AND (d_year#53 = 2002)) AND isnotnull(d_date_sk#52)) - -(127) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#52, d_year#53] - -(128) BroadcastExchange -Input [2]: [d_date_sk#52, d_year#53] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] - -Subquery:3 Hosting operator id = 42 Hosting Expression = cs_sold_date_sk#85 IN dynamicpruning#21 - -Subquery:4 Hosting operator id = 63 Hosting Expression = cs_sold_date_sk#114 IN dynamicpruning#51 - -Subquery:5 Hosting operator id = 83 Hosting Expression = ws_sold_date_sk#142 IN dynamicpruning#21 - -Subquery:6 Hosting operator id = 104 Hosting Expression = ws_sold_date_sk#171 IN dynamicpruning#51 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/simplified.txt deleted file mode 100644 index 56bc32e144..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/simplified.txt +++ /dev/null @@ -1,191 +0,0 @@ -TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - WholeStageCodegen (24) - Project [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 - WholeStageCodegen (3) - HashAggregate [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] [sum,isEmpty,sum,isEmpty] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #5 - WholeStageCodegen (6) - HashAggregate [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] [sum,isEmpty,sum,isEmpty] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) - HashAggregate [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] [sum,isEmpty,sum,isEmpty] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) - HashAggregate [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] [sum,isEmpty,sum,isEmpty] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (19) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #15 - WholeStageCodegen (18) - HashAggregate [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] [sum,isEmpty,sum,isEmpty] - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (16) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (23) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #18 - WholeStageCodegen (22) - HashAggregate [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] [sum,isEmpty,sum,isEmpty] - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #19 - WholeStageCodegen (20) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/explain.txt deleted file mode 100644 index b2368c1fe0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,671 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (107) -+- CometTakeOrderedAndProject (106) - +- CometProject (105) - +- CometBroadcastHashJoin (104) - :- CometProject (88) - : +- CometBroadcastHashJoin (87) - : :- CometProject (70) - : : +- CometBroadcastHashJoin (69) - : : :- CometProject (53) - : : : +- CometBroadcastHashJoin (52) - : : : :- CometBroadcastHashJoin (35) - : : : : :- CometFilter (17) - : : : : : +- CometHashAggregate (16) - : : : : : +- CometExchange (15) - : : : : : +- CometHashAggregate (14) - : : : : : +- CometProject (13) - : : : : : +- CometBroadcastHashJoin (12) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometProject (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : : : +- CometBroadcastExchange (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : : : +- CometBroadcastExchange (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : : : +- CometBroadcastExchange (34) - : : : : +- CometHashAggregate (33) - : : : : +- CometExchange (32) - : : : : +- CometHashAggregate (31) - : : : : +- CometProject (30) - : : : : +- CometBroadcastHashJoin (29) - : : : : :- CometProject (25) - : : : : : +- CometBroadcastHashJoin (24) - : : : : : :- CometProject (20) - : : : : : : +- CometFilter (19) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : : : +- CometBroadcastExchange (23) - : : : : : +- CometFilter (22) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : : : +- CometBroadcastExchange (28) - : : : : +- CometFilter (27) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : : : +- CometBroadcastExchange (51) - : : : +- CometFilter (50) - : : : +- CometHashAggregate (49) - : : : +- CometExchange (48) - : : : +- CometHashAggregate (47) - : : : +- CometProject (46) - : : : +- CometBroadcastHashJoin (45) - : : : :- CometProject (43) - : : : : +- CometBroadcastHashJoin (42) - : : : : :- CometProject (38) - : : : : : +- CometFilter (37) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : : : +- CometBroadcastExchange (41) - : : : : +- CometFilter (40) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) - : : : +- ReusedExchange (44) - : : +- CometBroadcastExchange (68) - : : +- CometHashAggregate (67) - : : +- CometExchange (66) - : : +- CometHashAggregate (65) - : : +- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometProject (61) - : : : +- CometBroadcastHashJoin (60) - : : : :- CometProject (56) - : : : : +- CometFilter (55) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : : : +- CometBroadcastExchange (59) - : : : +- CometFilter (58) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (57) - : : +- ReusedExchange (62) - : +- CometBroadcastExchange (86) - : +- CometFilter (85) - : +- CometHashAggregate (84) - : +- CometExchange (83) - : +- CometHashAggregate (82) - : +- CometProject (81) - : +- CometBroadcastHashJoin (80) - : :- CometProject (78) - : : +- CometBroadcastHashJoin (77) - : : :- CometProject (73) - : : : +- CometFilter (72) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (71) - : : +- CometBroadcastExchange (76) - : : +- CometFilter (75) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (74) - : +- ReusedExchange (79) - +- CometBroadcastExchange (103) - +- CometHashAggregate (102) - +- CometExchange (101) - +- CometHashAggregate (100) - +- CometProject (99) - +- CometBroadcastHashJoin (98) - :- CometProject (96) - : +- CometBroadcastHashJoin (95) - : :- CometProject (91) - : : +- CometFilter (90) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (89) - : +- CometBroadcastExchange (94) - : +- CometFilter (93) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (92) - +- ReusedExchange (97) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)) AS c_customer_id#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#3, 20)) AS c_first_name#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#4, 30)) AS c_last_name#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#5, 1)) AS c_preferred_cust_flag#12, c_birth_country#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#7, 13)) AS c_login#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#8, 50)) AS c_email_address#14] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#20), dynamicpruningexpression(ss_sold_date_sk#20 IN dynamicpruning#21)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Condition : isnotnull(ss_customer_sk#15) - -(6) CometBroadcastExchange -Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Arguments: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] - -(7) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Right output [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Arguments: [c_customer_sk#1], [ss_customer_sk#15], Inner, BuildRight - -(8) CometProject -Input [14]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_year#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#22, d_year#23] -Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#22, d_year#23] -Arguments: [d_date_sk#22, d_year#23] - -(12) CometBroadcastHashJoin -Left output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Right output [2]: [d_date_sk#22, d_year#23] -Arguments: [ss_sold_date_sk#20], [d_date_sk#22], Inner, BuildRight - -(13) CometProject -Input [14]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20, d_date_sk#22, d_year#23] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] - -(14) CometHashAggregate -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] -Functions [1]: [partial_sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] - -(15) CometExchange -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#24, isEmpty#25] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] -Functions [1]: [sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] - -(17) CometFilter -Input [2]: [customer_id#26, year_total#27] -Condition : (isnotnull(year_total#27) AND (year_total#27 > 0.000000)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] -Condition : (isnotnull(c_customer_sk#28) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#29, 16)))) - -(20) CometProject -Input [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] -Arguments: [c_customer_sk#28, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41], [c_customer_sk#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#29, 16)) AS c_customer_id#36, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#30, 20)) AS c_first_name#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#31, 30)) AS c_last_name#38, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#32, 1)) AS c_preferred_cust_flag#39, c_birth_country#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#34, 13)) AS c_login#40, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#35, 50)) AS c_email_address#41] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_customer_sk#42, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, ss_sold_date_sk#47] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#47), dynamicpruningexpression(ss_sold_date_sk#47 IN dynamicpruning#48)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [6]: [ss_customer_sk#42, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, ss_sold_date_sk#47] -Condition : isnotnull(ss_customer_sk#42) - -(23) CometBroadcastExchange -Input [6]: [ss_customer_sk#42, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, ss_sold_date_sk#47] -Arguments: [ss_customer_sk#42, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, ss_sold_date_sk#47] - -(24) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#28, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41] -Right output [6]: [ss_customer_sk#42, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, ss_sold_date_sk#47] -Arguments: [c_customer_sk#28], [ss_customer_sk#42], Inner, BuildRight - -(25) CometProject -Input [14]: [c_customer_sk#28, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, ss_customer_sk#42, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, ss_sold_date_sk#47] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, ss_sold_date_sk#47], [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, ss_sold_date_sk#47] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#49, d_year#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#49, d_year#50] -Condition : ((isnotnull(d_year#50) AND (d_year#50 = 2002)) AND isnotnull(d_date_sk#49)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#49, d_year#50] -Arguments: [d_date_sk#49, d_year#50] - -(29) CometBroadcastHashJoin -Left output [12]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, ss_sold_date_sk#47] -Right output [2]: [d_date_sk#49, d_year#50] -Arguments: [ss_sold_date_sk#47], [d_date_sk#49], Inner, BuildRight - -(30) CometProject -Input [14]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, ss_sold_date_sk#47, d_date_sk#49, d_year#50] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, d_year#50], [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, d_year#50] - -(31) CometHashAggregate -Input [12]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, d_year#50] -Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, d_year#50] -Functions [1]: [partial_sum(((((ss_ext_list_price#46 - ss_ext_wholesale_cost#45) - ss_ext_discount_amt#43) + ss_ext_sales_price#44) / 2))] - -(32) CometExchange -Input [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, d_year#50, sum#51, isEmpty#52] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, d_year#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, d_year#50, sum#51, isEmpty#52] -Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, d_year#50] -Functions [1]: [sum(((((ss_ext_list_price#46 - ss_ext_wholesale_cost#45) - ss_ext_discount_amt#43) + ss_ext_sales_price#44) / 2))] - -(34) CometBroadcastExchange -Input [8]: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60] -Arguments: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#26, year_total#27] -Right output [8]: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60] -Arguments: [customer_id#26], [customer_id#53], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] -Condition : (isnotnull(c_customer_sk#61) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#62, 16)))) - -(38) CometProject -Input [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] -Arguments: [c_customer_sk#61, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74], [c_customer_sk#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#62, 16)) AS c_customer_id#69, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#63, 20)) AS c_first_name#70, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#64, 30)) AS c_last_name#71, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#65, 1)) AS c_preferred_cust_flag#72, c_birth_country#66, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#67, 13)) AS c_login#73, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#68, 50)) AS c_email_address#74] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#75, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, cs_sold_date_sk#80] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#80), dynamicpruningexpression(cs_sold_date_sk#80 IN dynamicpruning#81)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [6]: [cs_bill_customer_sk#75, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, cs_sold_date_sk#80] -Condition : isnotnull(cs_bill_customer_sk#75) - -(41) CometBroadcastExchange -Input [6]: [cs_bill_customer_sk#75, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, cs_sold_date_sk#80] -Arguments: [cs_bill_customer_sk#75, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, cs_sold_date_sk#80] - -(42) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#61, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74] -Right output [6]: [cs_bill_customer_sk#75, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, cs_sold_date_sk#80] -Arguments: [c_customer_sk#61], [cs_bill_customer_sk#75], Inner, BuildRight - -(43) CometProject -Input [14]: [c_customer_sk#61, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, cs_bill_customer_sk#75, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, cs_sold_date_sk#80] -Arguments: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, cs_sold_date_sk#80], [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, cs_sold_date_sk#80] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#82, d_year#83] - -(45) CometBroadcastHashJoin -Left output [12]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, cs_sold_date_sk#80] -Right output [2]: [d_date_sk#82, d_year#83] -Arguments: [cs_sold_date_sk#80], [d_date_sk#82], Inner, BuildRight - -(46) CometProject -Input [14]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, cs_sold_date_sk#80, d_date_sk#82, d_year#83] -Arguments: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, d_year#83], [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, d_year#83] - -(47) CometHashAggregate -Input [12]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, d_year#83] -Keys [8]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, d_year#83] -Functions [1]: [partial_sum(((((cs_ext_list_price#79 - cs_ext_wholesale_cost#78) - cs_ext_discount_amt#76) + cs_ext_sales_price#77) / 2))] - -(48) CometExchange -Input [10]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, d_year#83, sum#84, isEmpty#85] -Arguments: hashpartitioning(c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, d_year#83, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [10]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, d_year#83, sum#84, isEmpty#85] -Keys [8]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, d_year#83] -Functions [1]: [sum(((((cs_ext_list_price#79 - cs_ext_wholesale_cost#78) - cs_ext_discount_amt#76) + cs_ext_sales_price#77) / 2))] - -(50) CometFilter -Input [2]: [customer_id#86, year_total#87] -Condition : (isnotnull(year_total#87) AND (year_total#87 > 0.000000)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#86, year_total#87] -Arguments: [customer_id#86, year_total#87] - -(52) CometBroadcastHashJoin -Left output [10]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60] -Right output [2]: [customer_id#86, year_total#87] -Arguments: [customer_id#26], [customer_id#86], Inner, BuildRight - -(53) CometProject -Input [12]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, customer_id#86, year_total#87] -Arguments: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, year_total#87], [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, year_total#87] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#88, c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [8]: [c_customer_sk#88, c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95] -Condition : (isnotnull(c_customer_sk#88) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#89, 16)))) - -(56) CometProject -Input [8]: [c_customer_sk#88, c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95] -Arguments: [c_customer_sk#88, c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101], [c_customer_sk#88, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#89, 16)) AS c_customer_id#96, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#90, 20)) AS c_first_name#97, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#91, 30)) AS c_last_name#98, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#92, 1)) AS c_preferred_cust_flag#99, c_birth_country#93, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#94, 13)) AS c_login#100, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#95, 50)) AS c_email_address#101] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#102, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, cs_sold_date_sk#107] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#107), dynamicpruningexpression(cs_sold_date_sk#107 IN dynamicpruning#108)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [cs_bill_customer_sk#102, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, cs_sold_date_sk#107] -Condition : isnotnull(cs_bill_customer_sk#102) - -(59) CometBroadcastExchange -Input [6]: [cs_bill_customer_sk#102, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, cs_sold_date_sk#107] -Arguments: [cs_bill_customer_sk#102, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, cs_sold_date_sk#107] - -(60) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#88, c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101] -Right output [6]: [cs_bill_customer_sk#102, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, cs_sold_date_sk#107] -Arguments: [c_customer_sk#88], [cs_bill_customer_sk#102], Inner, BuildRight - -(61) CometProject -Input [14]: [c_customer_sk#88, c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, cs_bill_customer_sk#102, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, cs_sold_date_sk#107] -Arguments: [c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, cs_sold_date_sk#107], [c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, cs_sold_date_sk#107] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#109, d_year#110] - -(63) CometBroadcastHashJoin -Left output [12]: [c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, cs_sold_date_sk#107] -Right output [2]: [d_date_sk#109, d_year#110] -Arguments: [cs_sold_date_sk#107], [d_date_sk#109], Inner, BuildRight - -(64) CometProject -Input [14]: [c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, cs_sold_date_sk#107, d_date_sk#109, d_year#110] -Arguments: [c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, d_year#110], [c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, d_year#110] - -(65) CometHashAggregate -Input [12]: [c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, d_year#110] -Keys [8]: [c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, d_year#110] -Functions [1]: [partial_sum(((((cs_ext_list_price#106 - cs_ext_wholesale_cost#105) - cs_ext_discount_amt#103) + cs_ext_sales_price#104) / 2))] - -(66) CometExchange -Input [10]: [c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, d_year#110, sum#111, isEmpty#112] -Arguments: hashpartitioning(c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, d_year#110, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [10]: [c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, d_year#110, sum#111, isEmpty#112] -Keys [8]: [c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, d_year#110] -Functions [1]: [sum(((((cs_ext_list_price#106 - cs_ext_wholesale_cost#105) - cs_ext_discount_amt#103) + cs_ext_sales_price#104) / 2))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#113, year_total#114] -Arguments: [customer_id#113, year_total#114] - -(69) CometBroadcastHashJoin -Left output [11]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, year_total#87] -Right output [2]: [customer_id#113, year_total#114] -Arguments: [customer_id#26], [customer_id#113], Inner, (CASE WHEN (year_total#87 > 0.000000) THEN (year_total#114 / year_total#87) END > CASE WHEN (year_total#27 > 0.000000) THEN (year_total#60 / year_total#27) END), BuildRight - -(70) CometProject -Input [13]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, year_total#87, customer_id#113, year_total#114] -Arguments: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#87, year_total#114], [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#87, year_total#114] - -(71) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#115, c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(72) CometFilter -Input [8]: [c_customer_sk#115, c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122] -Condition : (isnotnull(c_customer_sk#115) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#116, 16)))) - -(73) CometProject -Input [8]: [c_customer_sk#115, c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122] -Arguments: [c_customer_sk#115, c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128], [c_customer_sk#115, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#116, 16)) AS c_customer_id#123, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#117, 20)) AS c_first_name#124, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#118, 30)) AS c_last_name#125, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#119, 1)) AS c_preferred_cust_flag#126, c_birth_country#120, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#121, 13)) AS c_login#127, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#122, 50)) AS c_email_address#128] - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#129, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, ws_sold_date_sk#134] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#134), dynamicpruningexpression(ws_sold_date_sk#134 IN dynamicpruning#135)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(75) CometFilter -Input [6]: [ws_bill_customer_sk#129, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, ws_sold_date_sk#134] -Condition : isnotnull(ws_bill_customer_sk#129) - -(76) CometBroadcastExchange -Input [6]: [ws_bill_customer_sk#129, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, ws_sold_date_sk#134] -Arguments: [ws_bill_customer_sk#129, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, ws_sold_date_sk#134] - -(77) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#115, c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128] -Right output [6]: [ws_bill_customer_sk#129, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, ws_sold_date_sk#134] -Arguments: [c_customer_sk#115], [ws_bill_customer_sk#129], Inner, BuildRight - -(78) CometProject -Input [14]: [c_customer_sk#115, c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, ws_bill_customer_sk#129, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, ws_sold_date_sk#134] -Arguments: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, ws_sold_date_sk#134], [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, ws_sold_date_sk#134] - -(79) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#136, d_year#137] - -(80) CometBroadcastHashJoin -Left output [12]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, ws_sold_date_sk#134] -Right output [2]: [d_date_sk#136, d_year#137] -Arguments: [ws_sold_date_sk#134], [d_date_sk#136], Inner, BuildRight - -(81) CometProject -Input [14]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, ws_sold_date_sk#134, d_date_sk#136, d_year#137] -Arguments: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, d_year#137], [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, d_year#137] - -(82) CometHashAggregate -Input [12]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, d_year#137] -Keys [8]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, d_year#137] -Functions [1]: [partial_sum(((((ws_ext_list_price#133 - ws_ext_wholesale_cost#132) - ws_ext_discount_amt#130) + ws_ext_sales_price#131) / 2))] - -(83) CometExchange -Input [10]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, d_year#137, sum#138, isEmpty#139] -Arguments: hashpartitioning(c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, d_year#137, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(84) CometHashAggregate -Input [10]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, d_year#137, sum#138, isEmpty#139] -Keys [8]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, d_year#137] -Functions [1]: [sum(((((ws_ext_list_price#133 - ws_ext_wholesale_cost#132) - ws_ext_discount_amt#130) + ws_ext_sales_price#131) / 2))] - -(85) CometFilter -Input [2]: [customer_id#140, year_total#141] -Condition : (isnotnull(year_total#141) AND (year_total#141 > 0.000000)) - -(86) CometBroadcastExchange -Input [2]: [customer_id#140, year_total#141] -Arguments: [customer_id#140, year_total#141] - -(87) CometBroadcastHashJoin -Left output [10]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#87, year_total#114] -Right output [2]: [customer_id#140, year_total#141] -Arguments: [customer_id#26], [customer_id#140], Inner, BuildRight - -(88) CometProject -Input [12]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#87, year_total#114, customer_id#140, year_total#141] -Arguments: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#87, year_total#114, year_total#141], [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#87, year_total#114, year_total#141] - -(89) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#142, c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(90) CometFilter -Input [8]: [c_customer_sk#142, c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149] -Condition : (isnotnull(c_customer_sk#142) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#143, 16)))) - -(91) CometProject -Input [8]: [c_customer_sk#142, c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149] -Arguments: [c_customer_sk#142, c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155], [c_customer_sk#142, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#143, 16)) AS c_customer_id#150, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#144, 20)) AS c_first_name#151, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#145, 30)) AS c_last_name#152, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#146, 1)) AS c_preferred_cust_flag#153, c_birth_country#147, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#148, 13)) AS c_login#154, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#149, 50)) AS c_email_address#155] - -(92) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#156, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, ws_sold_date_sk#161] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#161), dynamicpruningexpression(ws_sold_date_sk#161 IN dynamicpruning#162)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(93) CometFilter -Input [6]: [ws_bill_customer_sk#156, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, ws_sold_date_sk#161] -Condition : isnotnull(ws_bill_customer_sk#156) - -(94) CometBroadcastExchange -Input [6]: [ws_bill_customer_sk#156, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, ws_sold_date_sk#161] -Arguments: [ws_bill_customer_sk#156, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, ws_sold_date_sk#161] - -(95) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#142, c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155] -Right output [6]: [ws_bill_customer_sk#156, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, ws_sold_date_sk#161] -Arguments: [c_customer_sk#142], [ws_bill_customer_sk#156], Inner, BuildRight - -(96) CometProject -Input [14]: [c_customer_sk#142, c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, ws_bill_customer_sk#156, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, ws_sold_date_sk#161] -Arguments: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, ws_sold_date_sk#161], [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, ws_sold_date_sk#161] - -(97) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#163, d_year#164] - -(98) CometBroadcastHashJoin -Left output [12]: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, ws_sold_date_sk#161] -Right output [2]: [d_date_sk#163, d_year#164] -Arguments: [ws_sold_date_sk#161], [d_date_sk#163], Inner, BuildRight - -(99) CometProject -Input [14]: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, ws_sold_date_sk#161, d_date_sk#163, d_year#164] -Arguments: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, d_year#164], [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, d_year#164] - -(100) CometHashAggregate -Input [12]: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, d_year#164] -Keys [8]: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, d_year#164] -Functions [1]: [partial_sum(((((ws_ext_list_price#160 - ws_ext_wholesale_cost#159) - ws_ext_discount_amt#157) + ws_ext_sales_price#158) / 2))] - -(101) CometExchange -Input [10]: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, d_year#164, sum#165, isEmpty#166] -Arguments: hashpartitioning(c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, d_year#164, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(102) CometHashAggregate -Input [10]: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, d_year#164, sum#165, isEmpty#166] -Keys [8]: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, d_year#164] -Functions [1]: [sum(((((ws_ext_list_price#160 - ws_ext_wholesale_cost#159) - ws_ext_discount_amt#157) + ws_ext_sales_price#158) / 2))] - -(103) CometBroadcastExchange -Input [2]: [customer_id#167, year_total#168] -Arguments: [customer_id#167, year_total#168] - -(104) CometBroadcastHashJoin -Left output [11]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#87, year_total#114, year_total#141] -Right output [2]: [customer_id#167, year_total#168] -Arguments: [customer_id#26], [customer_id#167], Inner, (CASE WHEN (year_total#87 > 0.000000) THEN (year_total#114 / year_total#87) END > CASE WHEN (year_total#141 > 0.000000) THEN (year_total#168 / year_total#141) END), BuildRight - -(105) CometProject -Input [13]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#87, year_total#114, year_total#141, customer_id#167, year_total#168] -Arguments: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59], [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59] - -(106) CometTakeOrderedAndProject -Input [7]: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#53 ASC NULLS FIRST,customer_first_name#54 ASC NULLS FIRST,customer_last_name#55 ASC NULLS FIRST,customer_preferred_cust_flag#56 ASC NULLS FIRST,customer_birth_country#57 ASC NULLS FIRST,customer_login#58 ASC NULLS FIRST,customer_email_address#59 ASC NULLS FIRST], output=[customer_id#53,customer_first_name#54,customer_last_name#55,customer_preferred_cust_flag#56,customer_birth_country#57,customer_login#58,customer_email_address#59]), [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59], 100, 0, [customer_id#53 ASC NULLS FIRST, customer_first_name#54 ASC NULLS FIRST, customer_last_name#55 ASC NULLS FIRST, customer_preferred_cust_flag#56 ASC NULLS FIRST, customer_birth_country#57 ASC NULLS FIRST, customer_login#58 ASC NULLS FIRST, customer_email_address#59 ASC NULLS FIRST], [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59] - -(107) CometColumnarToRow [codegen id : 1] -Input [7]: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 -BroadcastExchange (111) -+- * CometColumnarToRow (110) - +- CometFilter (109) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) - - -(108) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_year#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(109) CometFilter -Input [2]: [d_date_sk#22, d_year#23] -Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(110) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#22, d_year#23] - -(111) BroadcastExchange -Input [2]: [d_date_sk#22, d_year#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#47 IN dynamicpruning#48 -BroadcastExchange (115) -+- * CometColumnarToRow (114) - +- CometFilter (113) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) - - -(112) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#49, d_year#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(113) CometFilter -Input [2]: [d_date_sk#49, d_year#50] -Condition : ((isnotnull(d_year#50) AND (d_year#50 = 2002)) AND isnotnull(d_date_sk#49)) - -(114) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#49, d_year#50] - -(115) BroadcastExchange -Input [2]: [d_date_sk#49, d_year#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -Subquery:3 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#80 IN dynamicpruning#21 - -Subquery:4 Hosting operator id = 57 Hosting Expression = cs_sold_date_sk#107 IN dynamicpruning#48 - -Subquery:5 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#134 IN dynamicpruning#21 - -Subquery:6 Hosting operator id = 92 Hosting Expression = ws_sold_date_sk#161 IN dynamicpruning#48 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/simplified.txt deleted file mode 100644 index 36be965b48..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,127 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - CometProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - CometBroadcastHashJoin [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total,customer_id,year_total] - CometProject [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] - CometBroadcastHashJoin [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,customer_id,year_total] - CometProject [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 - CometHashAggregate [ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total] #5 - CometHashAggregate [d_year,sum,isEmpty] [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #6 - CometHashAggregate [ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #16 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #17 - CometHashAggregate [ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #19 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #20 - CometHashAggregate [ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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/q4/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/explain.txt deleted file mode 100644 index b2368c1fe0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/explain.txt +++ /dev/null @@ -1,671 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (107) -+- CometTakeOrderedAndProject (106) - +- CometProject (105) - +- CometBroadcastHashJoin (104) - :- CometProject (88) - : +- CometBroadcastHashJoin (87) - : :- CometProject (70) - : : +- CometBroadcastHashJoin (69) - : : :- CometProject (53) - : : : +- CometBroadcastHashJoin (52) - : : : :- CometBroadcastHashJoin (35) - : : : : :- CometFilter (17) - : : : : : +- CometHashAggregate (16) - : : : : : +- CometExchange (15) - : : : : : +- CometHashAggregate (14) - : : : : : +- CometProject (13) - : : : : : +- CometBroadcastHashJoin (12) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometProject (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : : : +- CometBroadcastExchange (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : : : +- CometBroadcastExchange (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : : : +- CometBroadcastExchange (34) - : : : : +- CometHashAggregate (33) - : : : : +- CometExchange (32) - : : : : +- CometHashAggregate (31) - : : : : +- CometProject (30) - : : : : +- CometBroadcastHashJoin (29) - : : : : :- CometProject (25) - : : : : : +- CometBroadcastHashJoin (24) - : : : : : :- CometProject (20) - : : : : : : +- CometFilter (19) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : : : +- CometBroadcastExchange (23) - : : : : : +- CometFilter (22) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : : : +- CometBroadcastExchange (28) - : : : : +- CometFilter (27) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : : : +- CometBroadcastExchange (51) - : : : +- CometFilter (50) - : : : +- CometHashAggregate (49) - : : : +- CometExchange (48) - : : : +- CometHashAggregate (47) - : : : +- CometProject (46) - : : : +- CometBroadcastHashJoin (45) - : : : :- CometProject (43) - : : : : +- CometBroadcastHashJoin (42) - : : : : :- CometProject (38) - : : : : : +- CometFilter (37) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : : : +- CometBroadcastExchange (41) - : : : : +- CometFilter (40) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) - : : : +- ReusedExchange (44) - : : +- CometBroadcastExchange (68) - : : +- CometHashAggregate (67) - : : +- CometExchange (66) - : : +- CometHashAggregate (65) - : : +- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometProject (61) - : : : +- CometBroadcastHashJoin (60) - : : : :- CometProject (56) - : : : : +- CometFilter (55) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : : : +- CometBroadcastExchange (59) - : : : +- CometFilter (58) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (57) - : : +- ReusedExchange (62) - : +- CometBroadcastExchange (86) - : +- CometFilter (85) - : +- CometHashAggregate (84) - : +- CometExchange (83) - : +- CometHashAggregate (82) - : +- CometProject (81) - : +- CometBroadcastHashJoin (80) - : :- CometProject (78) - : : +- CometBroadcastHashJoin (77) - : : :- CometProject (73) - : : : +- CometFilter (72) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (71) - : : +- CometBroadcastExchange (76) - : : +- CometFilter (75) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (74) - : +- ReusedExchange (79) - +- CometBroadcastExchange (103) - +- CometHashAggregate (102) - +- CometExchange (101) - +- CometHashAggregate (100) - +- CometProject (99) - +- CometBroadcastHashJoin (98) - :- CometProject (96) - : +- CometBroadcastHashJoin (95) - : :- CometProject (91) - : : +- CometFilter (90) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (89) - : +- CometBroadcastExchange (94) - : +- CometFilter (93) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (92) - +- ReusedExchange (97) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)) AS c_customer_id#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#3, 20)) AS c_first_name#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#4, 30)) AS c_last_name#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#5, 1)) AS c_preferred_cust_flag#12, c_birth_country#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#7, 13)) AS c_login#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#8, 50)) AS c_email_address#14] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#20), dynamicpruningexpression(ss_sold_date_sk#20 IN dynamicpruning#21)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Condition : isnotnull(ss_customer_sk#15) - -(6) CometBroadcastExchange -Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Arguments: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] - -(7) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Right output [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Arguments: [c_customer_sk#1], [ss_customer_sk#15], Inner, BuildRight - -(8) CometProject -Input [14]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_year#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#22, d_year#23] -Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#22, d_year#23] -Arguments: [d_date_sk#22, d_year#23] - -(12) CometBroadcastHashJoin -Left output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Right output [2]: [d_date_sk#22, d_year#23] -Arguments: [ss_sold_date_sk#20], [d_date_sk#22], Inner, BuildRight - -(13) CometProject -Input [14]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20, d_date_sk#22, d_year#23] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] - -(14) CometHashAggregate -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] -Functions [1]: [partial_sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] - -(15) CometExchange -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#24, isEmpty#25] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] -Functions [1]: [sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] - -(17) CometFilter -Input [2]: [customer_id#26, year_total#27] -Condition : (isnotnull(year_total#27) AND (year_total#27 > 0.000000)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] -Condition : (isnotnull(c_customer_sk#28) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#29, 16)))) - -(20) CometProject -Input [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] -Arguments: [c_customer_sk#28, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41], [c_customer_sk#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#29, 16)) AS c_customer_id#36, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#30, 20)) AS c_first_name#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#31, 30)) AS c_last_name#38, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#32, 1)) AS c_preferred_cust_flag#39, c_birth_country#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#34, 13)) AS c_login#40, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#35, 50)) AS c_email_address#41] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_customer_sk#42, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, ss_sold_date_sk#47] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#47), dynamicpruningexpression(ss_sold_date_sk#47 IN dynamicpruning#48)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [6]: [ss_customer_sk#42, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, ss_sold_date_sk#47] -Condition : isnotnull(ss_customer_sk#42) - -(23) CometBroadcastExchange -Input [6]: [ss_customer_sk#42, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, ss_sold_date_sk#47] -Arguments: [ss_customer_sk#42, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, ss_sold_date_sk#47] - -(24) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#28, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41] -Right output [6]: [ss_customer_sk#42, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, ss_sold_date_sk#47] -Arguments: [c_customer_sk#28], [ss_customer_sk#42], Inner, BuildRight - -(25) CometProject -Input [14]: [c_customer_sk#28, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, ss_customer_sk#42, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, ss_sold_date_sk#47] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, ss_sold_date_sk#47], [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, ss_sold_date_sk#47] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#49, d_year#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#49, d_year#50] -Condition : ((isnotnull(d_year#50) AND (d_year#50 = 2002)) AND isnotnull(d_date_sk#49)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#49, d_year#50] -Arguments: [d_date_sk#49, d_year#50] - -(29) CometBroadcastHashJoin -Left output [12]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, ss_sold_date_sk#47] -Right output [2]: [d_date_sk#49, d_year#50] -Arguments: [ss_sold_date_sk#47], [d_date_sk#49], Inner, BuildRight - -(30) CometProject -Input [14]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, ss_sold_date_sk#47, d_date_sk#49, d_year#50] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, d_year#50], [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, d_year#50] - -(31) CometHashAggregate -Input [12]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, ss_ext_discount_amt#43, ss_ext_sales_price#44, ss_ext_wholesale_cost#45, ss_ext_list_price#46, d_year#50] -Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, d_year#50] -Functions [1]: [partial_sum(((((ss_ext_list_price#46 - ss_ext_wholesale_cost#45) - ss_ext_discount_amt#43) + ss_ext_sales_price#44) / 2))] - -(32) CometExchange -Input [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, d_year#50, sum#51, isEmpty#52] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, d_year#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, d_year#50, sum#51, isEmpty#52] -Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#33, c_login#40, c_email_address#41, d_year#50] -Functions [1]: [sum(((((ss_ext_list_price#46 - ss_ext_wholesale_cost#45) - ss_ext_discount_amt#43) + ss_ext_sales_price#44) / 2))] - -(34) CometBroadcastExchange -Input [8]: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60] -Arguments: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#26, year_total#27] -Right output [8]: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60] -Arguments: [customer_id#26], [customer_id#53], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] -Condition : (isnotnull(c_customer_sk#61) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#62, 16)))) - -(38) CometProject -Input [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] -Arguments: [c_customer_sk#61, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74], [c_customer_sk#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#62, 16)) AS c_customer_id#69, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#63, 20)) AS c_first_name#70, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#64, 30)) AS c_last_name#71, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#65, 1)) AS c_preferred_cust_flag#72, c_birth_country#66, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#67, 13)) AS c_login#73, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#68, 50)) AS c_email_address#74] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#75, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, cs_sold_date_sk#80] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#80), dynamicpruningexpression(cs_sold_date_sk#80 IN dynamicpruning#81)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [6]: [cs_bill_customer_sk#75, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, cs_sold_date_sk#80] -Condition : isnotnull(cs_bill_customer_sk#75) - -(41) CometBroadcastExchange -Input [6]: [cs_bill_customer_sk#75, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, cs_sold_date_sk#80] -Arguments: [cs_bill_customer_sk#75, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, cs_sold_date_sk#80] - -(42) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#61, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74] -Right output [6]: [cs_bill_customer_sk#75, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, cs_sold_date_sk#80] -Arguments: [c_customer_sk#61], [cs_bill_customer_sk#75], Inner, BuildRight - -(43) CometProject -Input [14]: [c_customer_sk#61, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, cs_bill_customer_sk#75, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, cs_sold_date_sk#80] -Arguments: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, cs_sold_date_sk#80], [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, cs_sold_date_sk#80] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#82, d_year#83] - -(45) CometBroadcastHashJoin -Left output [12]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, cs_sold_date_sk#80] -Right output [2]: [d_date_sk#82, d_year#83] -Arguments: [cs_sold_date_sk#80], [d_date_sk#82], Inner, BuildRight - -(46) CometProject -Input [14]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, cs_sold_date_sk#80, d_date_sk#82, d_year#83] -Arguments: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, d_year#83], [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, d_year#83] - -(47) CometHashAggregate -Input [12]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, cs_ext_discount_amt#76, cs_ext_sales_price#77, cs_ext_wholesale_cost#78, cs_ext_list_price#79, d_year#83] -Keys [8]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, d_year#83] -Functions [1]: [partial_sum(((((cs_ext_list_price#79 - cs_ext_wholesale_cost#78) - cs_ext_discount_amt#76) + cs_ext_sales_price#77) / 2))] - -(48) CometExchange -Input [10]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, d_year#83, sum#84, isEmpty#85] -Arguments: hashpartitioning(c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, d_year#83, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [10]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, d_year#83, sum#84, isEmpty#85] -Keys [8]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#66, c_login#73, c_email_address#74, d_year#83] -Functions [1]: [sum(((((cs_ext_list_price#79 - cs_ext_wholesale_cost#78) - cs_ext_discount_amt#76) + cs_ext_sales_price#77) / 2))] - -(50) CometFilter -Input [2]: [customer_id#86, year_total#87] -Condition : (isnotnull(year_total#87) AND (year_total#87 > 0.000000)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#86, year_total#87] -Arguments: [customer_id#86, year_total#87] - -(52) CometBroadcastHashJoin -Left output [10]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60] -Right output [2]: [customer_id#86, year_total#87] -Arguments: [customer_id#26], [customer_id#86], Inner, BuildRight - -(53) CometProject -Input [12]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, customer_id#86, year_total#87] -Arguments: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, year_total#87], [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, year_total#87] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#88, c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [8]: [c_customer_sk#88, c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95] -Condition : (isnotnull(c_customer_sk#88) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#89, 16)))) - -(56) CometProject -Input [8]: [c_customer_sk#88, c_customer_id#89, c_first_name#90, c_last_name#91, c_preferred_cust_flag#92, c_birth_country#93, c_login#94, c_email_address#95] -Arguments: [c_customer_sk#88, c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101], [c_customer_sk#88, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#89, 16)) AS c_customer_id#96, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#90, 20)) AS c_first_name#97, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#91, 30)) AS c_last_name#98, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#92, 1)) AS c_preferred_cust_flag#99, c_birth_country#93, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#94, 13)) AS c_login#100, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#95, 50)) AS c_email_address#101] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#102, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, cs_sold_date_sk#107] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#107), dynamicpruningexpression(cs_sold_date_sk#107 IN dynamicpruning#108)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [cs_bill_customer_sk#102, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, cs_sold_date_sk#107] -Condition : isnotnull(cs_bill_customer_sk#102) - -(59) CometBroadcastExchange -Input [6]: [cs_bill_customer_sk#102, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, cs_sold_date_sk#107] -Arguments: [cs_bill_customer_sk#102, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, cs_sold_date_sk#107] - -(60) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#88, c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101] -Right output [6]: [cs_bill_customer_sk#102, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, cs_sold_date_sk#107] -Arguments: [c_customer_sk#88], [cs_bill_customer_sk#102], Inner, BuildRight - -(61) CometProject -Input [14]: [c_customer_sk#88, c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, cs_bill_customer_sk#102, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, cs_sold_date_sk#107] -Arguments: [c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, cs_sold_date_sk#107], [c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, cs_sold_date_sk#107] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#109, d_year#110] - -(63) CometBroadcastHashJoin -Left output [12]: [c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, cs_sold_date_sk#107] -Right output [2]: [d_date_sk#109, d_year#110] -Arguments: [cs_sold_date_sk#107], [d_date_sk#109], Inner, BuildRight - -(64) CometProject -Input [14]: [c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, cs_sold_date_sk#107, d_date_sk#109, d_year#110] -Arguments: [c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, d_year#110], [c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, d_year#110] - -(65) CometHashAggregate -Input [12]: [c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, cs_ext_discount_amt#103, cs_ext_sales_price#104, cs_ext_wholesale_cost#105, cs_ext_list_price#106, d_year#110] -Keys [8]: [c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, d_year#110] -Functions [1]: [partial_sum(((((cs_ext_list_price#106 - cs_ext_wholesale_cost#105) - cs_ext_discount_amt#103) + cs_ext_sales_price#104) / 2))] - -(66) CometExchange -Input [10]: [c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, d_year#110, sum#111, isEmpty#112] -Arguments: hashpartitioning(c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, d_year#110, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [10]: [c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, d_year#110, sum#111, isEmpty#112] -Keys [8]: [c_customer_id#96, c_first_name#97, c_last_name#98, c_preferred_cust_flag#99, c_birth_country#93, c_login#100, c_email_address#101, d_year#110] -Functions [1]: [sum(((((cs_ext_list_price#106 - cs_ext_wholesale_cost#105) - cs_ext_discount_amt#103) + cs_ext_sales_price#104) / 2))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#113, year_total#114] -Arguments: [customer_id#113, year_total#114] - -(69) CometBroadcastHashJoin -Left output [11]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, year_total#87] -Right output [2]: [customer_id#113, year_total#114] -Arguments: [customer_id#26], [customer_id#113], Inner, (CASE WHEN (year_total#87 > 0.000000) THEN (year_total#114 / year_total#87) END > CASE WHEN (year_total#27 > 0.000000) THEN (year_total#60 / year_total#27) END), BuildRight - -(70) CometProject -Input [13]: [customer_id#26, year_total#27, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#60, year_total#87, customer_id#113, year_total#114] -Arguments: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#87, year_total#114], [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#87, year_total#114] - -(71) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#115, c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(72) CometFilter -Input [8]: [c_customer_sk#115, c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122] -Condition : (isnotnull(c_customer_sk#115) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#116, 16)))) - -(73) CometProject -Input [8]: [c_customer_sk#115, c_customer_id#116, c_first_name#117, c_last_name#118, c_preferred_cust_flag#119, c_birth_country#120, c_login#121, c_email_address#122] -Arguments: [c_customer_sk#115, c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128], [c_customer_sk#115, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#116, 16)) AS c_customer_id#123, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#117, 20)) AS c_first_name#124, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#118, 30)) AS c_last_name#125, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#119, 1)) AS c_preferred_cust_flag#126, c_birth_country#120, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#121, 13)) AS c_login#127, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#122, 50)) AS c_email_address#128] - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#129, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, ws_sold_date_sk#134] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#134), dynamicpruningexpression(ws_sold_date_sk#134 IN dynamicpruning#135)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(75) CometFilter -Input [6]: [ws_bill_customer_sk#129, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, ws_sold_date_sk#134] -Condition : isnotnull(ws_bill_customer_sk#129) - -(76) CometBroadcastExchange -Input [6]: [ws_bill_customer_sk#129, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, ws_sold_date_sk#134] -Arguments: [ws_bill_customer_sk#129, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, ws_sold_date_sk#134] - -(77) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#115, c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128] -Right output [6]: [ws_bill_customer_sk#129, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, ws_sold_date_sk#134] -Arguments: [c_customer_sk#115], [ws_bill_customer_sk#129], Inner, BuildRight - -(78) CometProject -Input [14]: [c_customer_sk#115, c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, ws_bill_customer_sk#129, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, ws_sold_date_sk#134] -Arguments: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, ws_sold_date_sk#134], [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, ws_sold_date_sk#134] - -(79) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#136, d_year#137] - -(80) CometBroadcastHashJoin -Left output [12]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, ws_sold_date_sk#134] -Right output [2]: [d_date_sk#136, d_year#137] -Arguments: [ws_sold_date_sk#134], [d_date_sk#136], Inner, BuildRight - -(81) CometProject -Input [14]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, ws_sold_date_sk#134, d_date_sk#136, d_year#137] -Arguments: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, d_year#137], [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, d_year#137] - -(82) CometHashAggregate -Input [12]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, ws_ext_discount_amt#130, ws_ext_sales_price#131, ws_ext_wholesale_cost#132, ws_ext_list_price#133, d_year#137] -Keys [8]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, d_year#137] -Functions [1]: [partial_sum(((((ws_ext_list_price#133 - ws_ext_wholesale_cost#132) - ws_ext_discount_amt#130) + ws_ext_sales_price#131) / 2))] - -(83) CometExchange -Input [10]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, d_year#137, sum#138, isEmpty#139] -Arguments: hashpartitioning(c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, d_year#137, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(84) CometHashAggregate -Input [10]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, d_year#137, sum#138, isEmpty#139] -Keys [8]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#120, c_login#127, c_email_address#128, d_year#137] -Functions [1]: [sum(((((ws_ext_list_price#133 - ws_ext_wholesale_cost#132) - ws_ext_discount_amt#130) + ws_ext_sales_price#131) / 2))] - -(85) CometFilter -Input [2]: [customer_id#140, year_total#141] -Condition : (isnotnull(year_total#141) AND (year_total#141 > 0.000000)) - -(86) CometBroadcastExchange -Input [2]: [customer_id#140, year_total#141] -Arguments: [customer_id#140, year_total#141] - -(87) CometBroadcastHashJoin -Left output [10]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#87, year_total#114] -Right output [2]: [customer_id#140, year_total#141] -Arguments: [customer_id#26], [customer_id#140], Inner, BuildRight - -(88) CometProject -Input [12]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#87, year_total#114, customer_id#140, year_total#141] -Arguments: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#87, year_total#114, year_total#141], [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#87, year_total#114, year_total#141] - -(89) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#142, c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(90) CometFilter -Input [8]: [c_customer_sk#142, c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149] -Condition : (isnotnull(c_customer_sk#142) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#143, 16)))) - -(91) CometProject -Input [8]: [c_customer_sk#142, c_customer_id#143, c_first_name#144, c_last_name#145, c_preferred_cust_flag#146, c_birth_country#147, c_login#148, c_email_address#149] -Arguments: [c_customer_sk#142, c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155], [c_customer_sk#142, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#143, 16)) AS c_customer_id#150, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#144, 20)) AS c_first_name#151, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#145, 30)) AS c_last_name#152, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#146, 1)) AS c_preferred_cust_flag#153, c_birth_country#147, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#148, 13)) AS c_login#154, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#149, 50)) AS c_email_address#155] - -(92) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#156, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, ws_sold_date_sk#161] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#161), dynamicpruningexpression(ws_sold_date_sk#161 IN dynamicpruning#162)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(93) CometFilter -Input [6]: [ws_bill_customer_sk#156, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, ws_sold_date_sk#161] -Condition : isnotnull(ws_bill_customer_sk#156) - -(94) CometBroadcastExchange -Input [6]: [ws_bill_customer_sk#156, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, ws_sold_date_sk#161] -Arguments: [ws_bill_customer_sk#156, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, ws_sold_date_sk#161] - -(95) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#142, c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155] -Right output [6]: [ws_bill_customer_sk#156, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, ws_sold_date_sk#161] -Arguments: [c_customer_sk#142], [ws_bill_customer_sk#156], Inner, BuildRight - -(96) CometProject -Input [14]: [c_customer_sk#142, c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, ws_bill_customer_sk#156, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, ws_sold_date_sk#161] -Arguments: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, ws_sold_date_sk#161], [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, ws_sold_date_sk#161] - -(97) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#163, d_year#164] - -(98) CometBroadcastHashJoin -Left output [12]: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, ws_sold_date_sk#161] -Right output [2]: [d_date_sk#163, d_year#164] -Arguments: [ws_sold_date_sk#161], [d_date_sk#163], Inner, BuildRight - -(99) CometProject -Input [14]: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, ws_sold_date_sk#161, d_date_sk#163, d_year#164] -Arguments: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, d_year#164], [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, d_year#164] - -(100) CometHashAggregate -Input [12]: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, ws_ext_discount_amt#157, ws_ext_sales_price#158, ws_ext_wholesale_cost#159, ws_ext_list_price#160, d_year#164] -Keys [8]: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, d_year#164] -Functions [1]: [partial_sum(((((ws_ext_list_price#160 - ws_ext_wholesale_cost#159) - ws_ext_discount_amt#157) + ws_ext_sales_price#158) / 2))] - -(101) CometExchange -Input [10]: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, d_year#164, sum#165, isEmpty#166] -Arguments: hashpartitioning(c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, d_year#164, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(102) CometHashAggregate -Input [10]: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, d_year#164, sum#165, isEmpty#166] -Keys [8]: [c_customer_id#150, c_first_name#151, c_last_name#152, c_preferred_cust_flag#153, c_birth_country#147, c_login#154, c_email_address#155, d_year#164] -Functions [1]: [sum(((((ws_ext_list_price#160 - ws_ext_wholesale_cost#159) - ws_ext_discount_amt#157) + ws_ext_sales_price#158) / 2))] - -(103) CometBroadcastExchange -Input [2]: [customer_id#167, year_total#168] -Arguments: [customer_id#167, year_total#168] - -(104) CometBroadcastHashJoin -Left output [11]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#87, year_total#114, year_total#141] -Right output [2]: [customer_id#167, year_total#168] -Arguments: [customer_id#26], [customer_id#167], Inner, (CASE WHEN (year_total#87 > 0.000000) THEN (year_total#114 / year_total#87) END > CASE WHEN (year_total#141 > 0.000000) THEN (year_total#168 / year_total#141) END), BuildRight - -(105) CometProject -Input [13]: [customer_id#26, customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59, year_total#87, year_total#114, year_total#141, customer_id#167, year_total#168] -Arguments: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59], [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59] - -(106) CometTakeOrderedAndProject -Input [7]: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#53 ASC NULLS FIRST,customer_first_name#54 ASC NULLS FIRST,customer_last_name#55 ASC NULLS FIRST,customer_preferred_cust_flag#56 ASC NULLS FIRST,customer_birth_country#57 ASC NULLS FIRST,customer_login#58 ASC NULLS FIRST,customer_email_address#59 ASC NULLS FIRST], output=[customer_id#53,customer_first_name#54,customer_last_name#55,customer_preferred_cust_flag#56,customer_birth_country#57,customer_login#58,customer_email_address#59]), [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59], 100, 0, [customer_id#53 ASC NULLS FIRST, customer_first_name#54 ASC NULLS FIRST, customer_last_name#55 ASC NULLS FIRST, customer_preferred_cust_flag#56 ASC NULLS FIRST, customer_birth_country#57 ASC NULLS FIRST, customer_login#58 ASC NULLS FIRST, customer_email_address#59 ASC NULLS FIRST], [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59] - -(107) CometColumnarToRow [codegen id : 1] -Input [7]: [customer_id#53, customer_first_name#54, customer_last_name#55, customer_preferred_cust_flag#56, customer_birth_country#57, customer_login#58, customer_email_address#59] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 -BroadcastExchange (111) -+- * CometColumnarToRow (110) - +- CometFilter (109) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) - - -(108) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_year#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(109) CometFilter -Input [2]: [d_date_sk#22, d_year#23] -Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(110) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#22, d_year#23] - -(111) BroadcastExchange -Input [2]: [d_date_sk#22, d_year#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#47 IN dynamicpruning#48 -BroadcastExchange (115) -+- * CometColumnarToRow (114) - +- CometFilter (113) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) - - -(112) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#49, d_year#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(113) CometFilter -Input [2]: [d_date_sk#49, d_year#50] -Condition : ((isnotnull(d_year#50) AND (d_year#50 = 2002)) AND isnotnull(d_date_sk#49)) - -(114) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#49, d_year#50] - -(115) BroadcastExchange -Input [2]: [d_date_sk#49, d_year#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -Subquery:3 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#80 IN dynamicpruning#21 - -Subquery:4 Hosting operator id = 57 Hosting Expression = cs_sold_date_sk#107 IN dynamicpruning#48 - -Subquery:5 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#134 IN dynamicpruning#21 - -Subquery:6 Hosting operator id = 92 Hosting Expression = ws_sold_date_sk#161 IN dynamicpruning#48 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/extended.txt deleted file mode 100644 index 1be0816ee1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/extended.txt +++ /dev/null @@ -1,131 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 118 out of 126 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 36be965b48..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt +++ /dev/null @@ -1,127 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - CometProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - CometBroadcastHashJoin [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total,customer_id,year_total] - CometProject [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] - CometBroadcastHashJoin [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,customer_id,year_total] - CometProject [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 - CometHashAggregate [ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total] #5 - CometHashAggregate [d_year,sum,isEmpty] [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #6 - CometHashAggregate [ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #16 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #17 - CometHashAggregate [ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #19 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #20 - CometHashAggregate [ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/explain.txt deleted file mode 100644 index 5f766cebdc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/explain.txt +++ /dev/null @@ -1,221 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (34) -+- CometTakeOrderedAndProject (33) - +- CometHashAggregate (32) - +- CometExchange (31) - +- CometHashAggregate (30) - +- CometProject (29) - +- CometBroadcastHashJoin (28) - :- CometProject (24) - : +- CometBroadcastHashJoin (23) - : :- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (12) - : : : +- CometSortMergeJoin (11) - : : : :- CometSort (5) - : : : : +- CometColumnarExchange (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : +- CometSort (10) - : : : +- CometExchange (9) - : : : +- CometProject (8) - : : : +- CometFilter (7) - : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (6) - : : +- CometBroadcastExchange (16) - : : +- CometProject (15) - : : +- CometFilter (14) - : : +- CometNativeScan parquet spark_catalog.default.warehouse (13) - : +- CometBroadcastExchange (22) - : +- CometProject (21) - : +- CometFilter (20) - : +- CometNativeScan parquet spark_catalog.default.item (19) - +- CometBroadcastExchange (27) - +- CometFilter (26) - +- CometNativeScan parquet spark_catalog.default.date_dim (25) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] - -(3) Filter [codegen id : 1] -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Condition : (isnotnull(cs_warehouse_sk#1) AND isnotnull(cs_item_sk#2)) - -(4) CometColumnarExchange -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_order_number#3, cs_item_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5], [cs_order_number#3 ASC NULLS FIRST, cs_item_sk#2 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Condition : (isnotnull(cr_order_number#8) AND isnotnull(cr_item_sk#7)) - -(8) CometProject -Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] - -(9) CometExchange -Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: hashpartitioning(cr_order_number#8, cr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_order_number#8 ASC NULLS FIRST, cr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Right output [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cs_order_number#3, cs_item_sk#2], [cr_order_number#8, cr_item_sk#7], LeftOuter - -(12) CometProject -Input [8]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5, cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9], [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] - -(13) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#11, w_state#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [w_warehouse_sk#11, w_state#12] -Condition : isnotnull(w_warehouse_sk#11) - -(15) CometProject -Input [2]: [w_warehouse_sk#11, w_state#12] -Arguments: [w_warehouse_sk#11, w_state#13], [w_warehouse_sk#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(w_state#12, 2)) AS w_state#13] - -(16) CometBroadcastExchange -Input [2]: [w_warehouse_sk#11, w_state#13] -Arguments: [w_warehouse_sk#11, w_state#13] - -(17) CometBroadcastHashJoin -Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] -Right output [2]: [w_warehouse_sk#11, w_state#13] -Arguments: [cs_warehouse_sk#1], [w_warehouse_sk#11], Inner, BuildRight - -(18) CometProject -Input [7]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_warehouse_sk#11, w_state#13] -Arguments: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13], [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13] - -(19) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Condition : (((isnotnull(i_current_price#16) AND (i_current_price#16 >= 0.99)) AND (i_current_price#16 <= 1.49)) AND isnotnull(i_item_sk#14)) - -(21) CometProject -Input [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Arguments: [i_item_sk#14, i_item_id#17], [i_item_sk#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#15, 16)) AS i_item_id#17] - -(22) CometBroadcastExchange -Input [2]: [i_item_sk#14, i_item_id#17] -Arguments: [i_item_sk#14, i_item_id#17] - -(23) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13] -Right output [2]: [i_item_sk#14, i_item_id#17] -Arguments: [cs_item_sk#2], [i_item_sk#14], Inner, BuildRight - -(24) CometProject -Input [7]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_sk#14, i_item_id#17] -Arguments: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17], [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17] - -(25) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#18, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) - -(27) CometBroadcastExchange -Input [2]: [d_date_sk#18, d_date#19] -Arguments: [d_date_sk#18, d_date#19] - -(28) CometBroadcastHashJoin -Left output [5]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17] -Right output [2]: [d_date_sk#18, d_date#19] -Arguments: [cs_sold_date_sk#5], [d_date_sk#18], Inner, BuildRight - -(29) CometProject -Input [7]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date_sk#18, d_date#19] -Arguments: [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19], [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19] - -(30) CometHashAggregate -Input [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19] -Keys [2]: [w_state#13, i_item_id#17] -Functions [2]: [partial_sum(CASE WHEN (d_date#19 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), partial_sum(CASE WHEN (d_date#19 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] - -(31) CometExchange -Input [6]: [w_state#13, i_item_id#17, sum#20, isEmpty#21, sum#22, isEmpty#23] -Arguments: hashpartitioning(w_state#13, i_item_id#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometHashAggregate -Input [6]: [w_state#13, i_item_id#17, sum#20, isEmpty#21, sum#22, isEmpty#23] -Keys [2]: [w_state#13, i_item_id#17] -Functions [2]: [sum(CASE WHEN (d_date#19 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), sum(CASE WHEN (d_date#19 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] - -(33) CometTakeOrderedAndProject -Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_state#13 ASC NULLS FIRST,i_item_id#17 ASC NULLS FIRST], output=[w_state#13,i_item_id#17,sales_before#24,sales_after#25]), [w_state#13, i_item_id#17, sales_before#24, sales_after#25], 100, 0, [w_state#13 ASC NULLS FIRST, i_item_id#17 ASC NULLS FIRST], [w_state#13, i_item_id#17, sales_before#24, sales_after#25] - -(34) CometColumnarToRow [codegen id : 2] -Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometFilter (36) - +- CometNativeScan parquet spark_catalog.default.date_dim (35) - - -(35) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(36) CometFilter -Input [2]: [d_date_sk#18, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) - -(37) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#18, d_date#19] - -(38) BroadcastExchange -Input [2]: [d_date_sk#18, d_date#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/simplified.txt deleted file mode 100644 index 13760fcb0b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] - CometHashAggregate [sum,isEmpty,sum,isEmpty] [w_state,i_item_id,sales_before,sales_after,sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END)] - CometExchange [w_state,i_item_id] #1 - CometHashAggregate [d_date,cs_sales_price,cr_refunded_cash] [w_state,i_item_id,sum,isEmpty,sum,isEmpty] - CometProject [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id,d_date_sk,d_date] - CometProject [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] - CometBroadcastHashJoin [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_warehouse_sk,w_state] - CometProject [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] - CometSortMergeJoin [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_refunded_cash] - CometSort [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - CometColumnarExchange [cs_order_number,cs_item_sk] #2 - WholeStageCodegen (1) - Filter [cs_warehouse_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] - CometExchange [cr_order_number,cr_item_sk] #4 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_state] #5 - CometProject [w_state] [w_warehouse_sk,w_state] - CometFilter [w_warehouse_sk,w_state] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometFilter [d_date_sk,d_date] - CometNativeScan 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/q40.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/explain.txt deleted file mode 100644 index ab95b81a9a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometTakeOrderedAndProject (32) - +- CometHashAggregate (31) - +- CometExchange (30) - +- CometHashAggregate (29) - +- CometProject (28) - +- CometBroadcastHashJoin (27) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometProject (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (5) - : : +- CometBroadcastExchange (15) - : : +- CometProject (14) - : : +- CometFilter (13) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (12) - : +- CometBroadcastExchange (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (18) - +- CometBroadcastExchange (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Condition : (isnotnull(cs_warehouse_sk#1) AND isnotnull(cs_item_sk#2)) - -(3) CometExchange -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_order_number#3, cs_item_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5], [cs_order_number#3 ASC NULLS FIRST, cs_item_sk#2 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Condition : (isnotnull(cr_order_number#8) AND isnotnull(cr_item_sk#7)) - -(7) CometProject -Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] - -(8) CometExchange -Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: hashpartitioning(cr_order_number#8, cr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_order_number#8 ASC NULLS FIRST, cr_item_sk#7 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Right output [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cs_order_number#3, cs_item_sk#2], [cr_order_number#8, cr_item_sk#7], LeftOuter - -(11) CometProject -Input [8]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5, cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9], [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#11, w_state#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [w_warehouse_sk#11, w_state#12] -Condition : isnotnull(w_warehouse_sk#11) - -(14) CometProject -Input [2]: [w_warehouse_sk#11, w_state#12] -Arguments: [w_warehouse_sk#11, w_state#13], [w_warehouse_sk#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(w_state#12, 2)) AS w_state#13] - -(15) CometBroadcastExchange -Input [2]: [w_warehouse_sk#11, w_state#13] -Arguments: [w_warehouse_sk#11, w_state#13] - -(16) CometBroadcastHashJoin -Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] -Right output [2]: [w_warehouse_sk#11, w_state#13] -Arguments: [cs_warehouse_sk#1], [w_warehouse_sk#11], Inner, BuildRight - -(17) CometProject -Input [7]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_warehouse_sk#11, w_state#13] -Arguments: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13], [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] -ReadSchema: struct - -(19) CometFilter -Input [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Condition : (((isnotnull(i_current_price#16) AND (i_current_price#16 >= 0.99)) AND (i_current_price#16 <= 1.49)) AND isnotnull(i_item_sk#14)) - -(20) CometProject -Input [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Arguments: [i_item_sk#14, i_item_id#17], [i_item_sk#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#15, 16)) AS i_item_id#17] - -(21) CometBroadcastExchange -Input [2]: [i_item_sk#14, i_item_id#17] -Arguments: [i_item_sk#14, i_item_id#17] - -(22) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13] -Right output [2]: [i_item_sk#14, i_item_id#17] -Arguments: [cs_item_sk#2], [i_item_sk#14], Inner, BuildRight - -(23) CometProject -Input [7]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_sk#14, i_item_id#17] -Arguments: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17], [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#18, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) - -(26) CometBroadcastExchange -Input [2]: [d_date_sk#18, d_date#19] -Arguments: [d_date_sk#18, d_date#19] - -(27) CometBroadcastHashJoin -Left output [5]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17] -Right output [2]: [d_date_sk#18, d_date#19] -Arguments: [cs_sold_date_sk#5], [d_date_sk#18], Inner, BuildRight - -(28) CometProject -Input [7]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date_sk#18, d_date#19] -Arguments: [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19], [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19] - -(29) CometHashAggregate -Input [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19] -Keys [2]: [w_state#13, i_item_id#17] -Functions [2]: [partial_sum(CASE WHEN (d_date#19 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), partial_sum(CASE WHEN (d_date#19 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] - -(30) CometExchange -Input [6]: [w_state#13, i_item_id#17, sum#20, isEmpty#21, sum#22, isEmpty#23] -Arguments: hashpartitioning(w_state#13, i_item_id#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(31) CometHashAggregate -Input [6]: [w_state#13, i_item_id#17, sum#20, isEmpty#21, sum#22, isEmpty#23] -Keys [2]: [w_state#13, i_item_id#17] -Functions [2]: [sum(CASE WHEN (d_date#19 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), sum(CASE WHEN (d_date#19 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] - -(32) CometTakeOrderedAndProject -Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_state#13 ASC NULLS FIRST,i_item_id#17 ASC NULLS FIRST], output=[w_state#13,i_item_id#17,sales_before#24,sales_after#25]), [w_state#13, i_item_id#17, sales_before#24, sales_after#25], 100, 0, [w_state#13 ASC NULLS FIRST, i_item_id#17 ASC NULLS FIRST], [w_state#13, i_item_id#17, sales_before#24, sales_after#25] - -(33) CometColumnarToRow [codegen id : 1] -Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [d_date_sk#18, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) - -(36) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#18, d_date#19] - -(37) BroadcastExchange -Input [2]: [d_date_sk#18, d_date#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/simplified.txt deleted file mode 100644 index 79e3556750..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,42 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] - CometHashAggregate [sum,isEmpty,sum,isEmpty] [w_state,i_item_id,sales_before,sales_after,sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END)] - CometExchange [w_state,i_item_id] #1 - CometHashAggregate [d_date,cs_sales_price,cr_refunded_cash] [w_state,i_item_id,sum,isEmpty,sum,isEmpty] - CometProject [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id,d_date_sk,d_date] - CometProject [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] - CometBroadcastHashJoin [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_warehouse_sk,w_state] - CometProject [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] - CometSortMergeJoin [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_refunded_cash] - CometSort [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - CometExchange [cs_order_number,cs_item_sk] #2 - CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] - CometExchange [cr_order_number,cr_item_sk] #4 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_state] #5 - CometProject [w_state] [w_warehouse_sk,w_state] - CometFilter [w_warehouse_sk,w_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q40/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/explain.txt deleted file mode 100644 index ab95b81a9a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometTakeOrderedAndProject (32) - +- CometHashAggregate (31) - +- CometExchange (30) - +- CometHashAggregate (29) - +- CometProject (28) - +- CometBroadcastHashJoin (27) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometProject (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (5) - : : +- CometBroadcastExchange (15) - : : +- CometProject (14) - : : +- CometFilter (13) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (12) - : +- CometBroadcastExchange (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (18) - +- CometBroadcastExchange (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Condition : (isnotnull(cs_warehouse_sk#1) AND isnotnull(cs_item_sk#2)) - -(3) CometExchange -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_order_number#3, cs_item_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5], [cs_order_number#3 ASC NULLS FIRST, cs_item_sk#2 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Condition : (isnotnull(cr_order_number#8) AND isnotnull(cr_item_sk#7)) - -(7) CometProject -Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] - -(8) CometExchange -Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: hashpartitioning(cr_order_number#8, cr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_order_number#8 ASC NULLS FIRST, cr_item_sk#7 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Right output [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cs_order_number#3, cs_item_sk#2], [cr_order_number#8, cr_item_sk#7], LeftOuter - -(11) CometProject -Input [8]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5, cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9], [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#11, w_state#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [w_warehouse_sk#11, w_state#12] -Condition : isnotnull(w_warehouse_sk#11) - -(14) CometProject -Input [2]: [w_warehouse_sk#11, w_state#12] -Arguments: [w_warehouse_sk#11, w_state#13], [w_warehouse_sk#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(w_state#12, 2)) AS w_state#13] - -(15) CometBroadcastExchange -Input [2]: [w_warehouse_sk#11, w_state#13] -Arguments: [w_warehouse_sk#11, w_state#13] - -(16) CometBroadcastHashJoin -Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] -Right output [2]: [w_warehouse_sk#11, w_state#13] -Arguments: [cs_warehouse_sk#1], [w_warehouse_sk#11], Inner, BuildRight - -(17) CometProject -Input [7]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_warehouse_sk#11, w_state#13] -Arguments: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13], [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] -ReadSchema: struct - -(19) CometFilter -Input [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Condition : (((isnotnull(i_current_price#16) AND (i_current_price#16 >= 0.99)) AND (i_current_price#16 <= 1.49)) AND isnotnull(i_item_sk#14)) - -(20) CometProject -Input [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Arguments: [i_item_sk#14, i_item_id#17], [i_item_sk#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#15, 16)) AS i_item_id#17] - -(21) CometBroadcastExchange -Input [2]: [i_item_sk#14, i_item_id#17] -Arguments: [i_item_sk#14, i_item_id#17] - -(22) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13] -Right output [2]: [i_item_sk#14, i_item_id#17] -Arguments: [cs_item_sk#2], [i_item_sk#14], Inner, BuildRight - -(23) CometProject -Input [7]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_sk#14, i_item_id#17] -Arguments: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17], [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#18, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) - -(26) CometBroadcastExchange -Input [2]: [d_date_sk#18, d_date#19] -Arguments: [d_date_sk#18, d_date#19] - -(27) CometBroadcastHashJoin -Left output [5]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17] -Right output [2]: [d_date_sk#18, d_date#19] -Arguments: [cs_sold_date_sk#5], [d_date_sk#18], Inner, BuildRight - -(28) CometProject -Input [7]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date_sk#18, d_date#19] -Arguments: [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19], [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19] - -(29) CometHashAggregate -Input [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19] -Keys [2]: [w_state#13, i_item_id#17] -Functions [2]: [partial_sum(CASE WHEN (d_date#19 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), partial_sum(CASE WHEN (d_date#19 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] - -(30) CometExchange -Input [6]: [w_state#13, i_item_id#17, sum#20, isEmpty#21, sum#22, isEmpty#23] -Arguments: hashpartitioning(w_state#13, i_item_id#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(31) CometHashAggregate -Input [6]: [w_state#13, i_item_id#17, sum#20, isEmpty#21, sum#22, isEmpty#23] -Keys [2]: [w_state#13, i_item_id#17] -Functions [2]: [sum(CASE WHEN (d_date#19 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), sum(CASE WHEN (d_date#19 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] - -(32) CometTakeOrderedAndProject -Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_state#13 ASC NULLS FIRST,i_item_id#17 ASC NULLS FIRST], output=[w_state#13,i_item_id#17,sales_before#24,sales_after#25]), [w_state#13, i_item_id#17, sales_before#24, sales_after#25], 100, 0, [w_state#13 ASC NULLS FIRST, i_item_id#17 ASC NULLS FIRST], [w_state#13, i_item_id#17, sales_before#24, sales_after#25] - -(33) CometColumnarToRow [codegen id : 1] -Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [d_date_sk#18, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) - -(36) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#18, d_date#19] - -(37) BroadcastExchange -Input [2]: [d_date_sk#18, d_date#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/extended.txt deleted file mode 100644 index 54a7b21486..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/extended.txt +++ /dev/null @@ -1,40 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 79e3556750..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt +++ /dev/null @@ -1,42 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] - CometHashAggregate [sum,isEmpty,sum,isEmpty] [w_state,i_item_id,sales_before,sales_after,sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END)] - CometExchange [w_state,i_item_id] #1 - CometHashAggregate [d_date,cs_sales_price,cr_refunded_cash] [w_state,i_item_id,sum,isEmpty,sum,isEmpty] - CometProject [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id,d_date_sk,d_date] - CometProject [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] - CometBroadcastHashJoin [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_warehouse_sk,w_state] - CometProject [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] - CometSortMergeJoin [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_refunded_cash] - CometSort [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - CometExchange [cs_order_number,cs_item_sk] #2 - CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] - CometExchange [cr_order_number,cr_item_sk] #4 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_state] #5 - CometProject [w_state] [w_warehouse_sk,w_state] - CometFilter [w_warehouse_sk,w_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q41.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_datafusion/explain.txt deleted file mode 100644 index 786423a455..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_datafusion/explain.txt +++ /dev/null @@ -1,107 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (3) - : +- CometFilter (2) - : +- CometNativeScan parquet spark_catalog.default.item (1) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometHashAggregate (9) - +- CometExchange (8) - +- CometHashAggregate (7) - +- CometProject (6) - +- CometFilter (5) - +- CometNativeScan parquet spark_catalog.default.item (4) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), GreaterThanOrEqual(i_manufact_id,738), LessThanOrEqual(i_manufact_id,778), IsNotNull(i_manufact)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Condition : (((isnotnull(i_manufact_id#1) AND (i_manufact_id#1 >= 738)) AND (i_manufact_id#1 <= 778)) AND isnotnull(i_manufact#2)) - -(3) CometProject -Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Arguments: [i_manufact#2, i_product_name#3], [i_manufact#2, i_product_name#3] - -(4) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(5) CometFilter -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] -Condition : (((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) = Women ) AND (((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = powder ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = khaki )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Ounce ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Oz ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = medium ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = extra large ))) OR ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = brown ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = honeydew )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Bunch ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Ton ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = N/A ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = small ))))) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) = Men ) AND (((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = floral ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = deep )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = N/A ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Dozen ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = petite ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = large ))) OR ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = light ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = cornflower )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Box ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Pound ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = medium ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = extra large )))))) OR (((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) = Women ) AND (((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = midnight ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = snow )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Pallet ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Gross ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = medium ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = extra large ))) OR ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = cyan ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = papaya )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Cup ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Dram ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = N/A ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = small ))))) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) = Men ) AND (((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = orange ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = frosted )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Each ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Tbl ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = petite ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = large ))) OR ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = forest ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = ghost )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Lb ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Bundle ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = medium ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = extra large ))))))) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_manufact#5, 50)))) - -(6) CometProject -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] -Arguments: [i_manufact#9], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_manufact#5, 50)) AS i_manufact#9] - -(7) CometHashAggregate -Input [1]: [i_manufact#9] -Keys [1]: [i_manufact#9] -Functions [1]: [partial_count(1)] - -(8) CometExchange -Input [2]: [i_manufact#9, count#10] -Arguments: hashpartitioning(i_manufact#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(9) CometHashAggregate -Input [2]: [i_manufact#9, count#10] -Keys [1]: [i_manufact#9] -Functions [1]: [count(1)] - -(10) CometFilter -Input [2]: [item_cnt#11, i_manufact#9] -Condition : (item_cnt#11 > 0) - -(11) CometProject -Input [2]: [item_cnt#11, i_manufact#9] -Arguments: [i_manufact#9], [i_manufact#9] - -(12) CometBroadcastExchange -Input [1]: [i_manufact#9] -Arguments: [i_manufact#9] - -(13) CometBroadcastHashJoin -Left output [2]: [i_manufact#2, i_product_name#3] -Right output [1]: [i_manufact#9] -Arguments: [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_manufact#2, 50))], [i_manufact#9], Inner, BuildRight - -(14) CometProject -Input [3]: [i_manufact#2, i_product_name#3, i_manufact#9] -Arguments: [i_product_name#12], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#3, 50)) AS i_product_name#12] - -(15) CometHashAggregate -Input [1]: [i_product_name#12] -Keys [1]: [i_product_name#12] -Functions: [] - -(16) CometExchange -Input [1]: [i_product_name#12] -Arguments: hashpartitioning(i_product_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(17) CometHashAggregate -Input [1]: [i_product_name#12] -Keys [1]: [i_product_name#12] -Functions: [] - -(18) CometTakeOrderedAndProject -Input [1]: [i_product_name#12] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#12 ASC NULLS FIRST], output=[i_product_name#12]), [i_product_name#12], 100, 0, [i_product_name#12 ASC NULLS FIRST], [i_product_name#12] - -(19) CometColumnarToRow [codegen id : 1] -Input [1]: [i_product_name#12] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_datafusion/simplified.txt deleted file mode 100644 index 2557f51684..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_datafusion/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name] - CometHashAggregate [i_product_name] - CometExchange [i_product_name] #1 - CometHashAggregate [i_product_name] - CometProject [i_product_name] [i_product_name] - CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - CometBroadcastExchange [i_manufact] #2 - CometProject [i_manufact] - CometFilter [item_cnt,i_manufact] - CometHashAggregate [count] [item_cnt,i_manufact,count(1)] - CometExchange [i_manufact] #3 - CometHashAggregate [i_manufact,count] - CometProject [i_manufact] [i_manufact] - CometFilter [i_category,i_manufact,i_size,i_color,i_units] - CometNativeScan 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/q41.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_iceberg_compat/explain.txt deleted file mode 100644 index 71f0293c69..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,107 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (3) - : +- CometFilter (2) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometHashAggregate (9) - +- CometExchange (8) - +- CometHashAggregate (7) - +- CometProject (6) - +- CometFilter (5) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), GreaterThanOrEqual(i_manufact_id,738), LessThanOrEqual(i_manufact_id,778), IsNotNull(i_manufact)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Condition : (((isnotnull(i_manufact_id#1) AND (i_manufact_id#1 >= 738)) AND (i_manufact_id#1 <= 778)) AND isnotnull(i_manufact#2)) - -(3) CometProject -Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Arguments: [i_manufact#2, i_product_name#3], [i_manufact#2, i_product_name#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(5) CometFilter -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] -Condition : (((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) = Women ) AND (((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = powder ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = khaki )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Ounce ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Oz ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = medium ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = extra large ))) OR ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = brown ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = honeydew )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Bunch ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Ton ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = N/A ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = small ))))) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) = Men ) AND (((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = floral ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = deep )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = N/A ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Dozen ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = petite ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = large ))) OR ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = light ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = cornflower )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Box ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Pound ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = medium ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = extra large )))))) OR (((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) = Women ) AND (((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = midnight ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = snow )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Pallet ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Gross ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = medium ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = extra large ))) OR ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = cyan ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = papaya )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Cup ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Dram ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = N/A ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = small ))))) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) = Men ) AND (((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = orange ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = frosted )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Each ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Tbl ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = petite ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = large ))) OR ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = forest ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = ghost )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Lb ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Bundle ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = medium ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = extra large ))))))) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_manufact#5, 50)))) - -(6) CometProject -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] -Arguments: [i_manufact#9], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_manufact#5, 50)) AS i_manufact#9] - -(7) CometHashAggregate -Input [1]: [i_manufact#9] -Keys [1]: [i_manufact#9] -Functions [1]: [partial_count(1)] - -(8) CometExchange -Input [2]: [i_manufact#9, count#10] -Arguments: hashpartitioning(i_manufact#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(9) CometHashAggregate -Input [2]: [i_manufact#9, count#10] -Keys [1]: [i_manufact#9] -Functions [1]: [count(1)] - -(10) CometFilter -Input [2]: [item_cnt#11, i_manufact#9] -Condition : (item_cnt#11 > 0) - -(11) CometProject -Input [2]: [item_cnt#11, i_manufact#9] -Arguments: [i_manufact#9], [i_manufact#9] - -(12) CometBroadcastExchange -Input [1]: [i_manufact#9] -Arguments: [i_manufact#9] - -(13) CometBroadcastHashJoin -Left output [2]: [i_manufact#2, i_product_name#3] -Right output [1]: [i_manufact#9] -Arguments: [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_manufact#2, 50))], [i_manufact#9], Inner, BuildRight - -(14) CometProject -Input [3]: [i_manufact#2, i_product_name#3, i_manufact#9] -Arguments: [i_product_name#12], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#3, 50)) AS i_product_name#12] - -(15) CometHashAggregate -Input [1]: [i_product_name#12] -Keys [1]: [i_product_name#12] -Functions: [] - -(16) CometExchange -Input [1]: [i_product_name#12] -Arguments: hashpartitioning(i_product_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(17) CometHashAggregate -Input [1]: [i_product_name#12] -Keys [1]: [i_product_name#12] -Functions: [] - -(18) CometTakeOrderedAndProject -Input [1]: [i_product_name#12] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#12 ASC NULLS FIRST], output=[i_product_name#12]), [i_product_name#12], 100, 0, [i_product_name#12 ASC NULLS FIRST], [i_product_name#12] - -(19) CometColumnarToRow [codegen id : 1] -Input [1]: [i_product_name#12] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_iceberg_compat/simplified.txt deleted file mode 100644 index 99c935aaa4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name] - CometHashAggregate [i_product_name] - CometExchange [i_product_name] #1 - CometHashAggregate [i_product_name] - CometProject [i_product_name] [i_product_name] - CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - CometBroadcastExchange [i_manufact] #2 - CometProject [i_manufact] - CometFilter [item_cnt,i_manufact] - CometHashAggregate [count] [item_cnt,i_manufact,count(1)] - CometExchange [i_manufact] #3 - CometHashAggregate [i_manufact,count] - CometProject [i_manufact] [i_manufact] - CometFilter [i_category,i_manufact,i_size,i_color,i_units] - CometScan [native_iceberg_compat] 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/q41/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/explain.txt deleted file mode 100644 index 71f0293c69..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/explain.txt +++ /dev/null @@ -1,107 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (3) - : +- CometFilter (2) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometHashAggregate (9) - +- CometExchange (8) - +- CometHashAggregate (7) - +- CometProject (6) - +- CometFilter (5) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), GreaterThanOrEqual(i_manufact_id,738), LessThanOrEqual(i_manufact_id,778), IsNotNull(i_manufact)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Condition : (((isnotnull(i_manufact_id#1) AND (i_manufact_id#1 >= 738)) AND (i_manufact_id#1 <= 778)) AND isnotnull(i_manufact#2)) - -(3) CometProject -Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Arguments: [i_manufact#2, i_product_name#3], [i_manufact#2, i_product_name#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(5) CometFilter -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] -Condition : (((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) = Women ) AND (((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = powder ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = khaki )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Ounce ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Oz ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = medium ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = extra large ))) OR ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = brown ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = honeydew )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Bunch ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Ton ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = N/A ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = small ))))) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) = Men ) AND (((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = floral ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = deep )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = N/A ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Dozen ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = petite ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = large ))) OR ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = light ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = cornflower )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Box ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Pound ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = medium ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = extra large )))))) OR (((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) = Women ) AND (((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = midnight ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = snow )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Pallet ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Gross ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = medium ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = extra large ))) OR ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = cyan ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = papaya )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Cup ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Dram ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = N/A ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = small ))))) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) = Men ) AND (((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = orange ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = frosted )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Each ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Tbl ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = petite ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = large ))) OR ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = forest ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#7, 20)) = ghost )) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Lb ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#8, 10)) = Bundle ))) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = medium ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#6, 20)) = extra large ))))))) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_manufact#5, 50)))) - -(6) CometProject -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] -Arguments: [i_manufact#9], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_manufact#5, 50)) AS i_manufact#9] - -(7) CometHashAggregate -Input [1]: [i_manufact#9] -Keys [1]: [i_manufact#9] -Functions [1]: [partial_count(1)] - -(8) CometExchange -Input [2]: [i_manufact#9, count#10] -Arguments: hashpartitioning(i_manufact#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(9) CometHashAggregate -Input [2]: [i_manufact#9, count#10] -Keys [1]: [i_manufact#9] -Functions [1]: [count(1)] - -(10) CometFilter -Input [2]: [item_cnt#11, i_manufact#9] -Condition : (item_cnt#11 > 0) - -(11) CometProject -Input [2]: [item_cnt#11, i_manufact#9] -Arguments: [i_manufact#9], [i_manufact#9] - -(12) CometBroadcastExchange -Input [1]: [i_manufact#9] -Arguments: [i_manufact#9] - -(13) CometBroadcastHashJoin -Left output [2]: [i_manufact#2, i_product_name#3] -Right output [1]: [i_manufact#9] -Arguments: [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_manufact#2, 50))], [i_manufact#9], Inner, BuildRight - -(14) CometProject -Input [3]: [i_manufact#2, i_product_name#3, i_manufact#9] -Arguments: [i_product_name#12], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#3, 50)) AS i_product_name#12] - -(15) CometHashAggregate -Input [1]: [i_product_name#12] -Keys [1]: [i_product_name#12] -Functions: [] - -(16) CometExchange -Input [1]: [i_product_name#12] -Arguments: hashpartitioning(i_product_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(17) CometHashAggregate -Input [1]: [i_product_name#12] -Keys [1]: [i_product_name#12] -Functions: [] - -(18) CometTakeOrderedAndProject -Input [1]: [i_product_name#12] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#12 ASC NULLS FIRST], output=[i_product_name#12]), [i_product_name#12], 100, 0, [i_product_name#12 ASC NULLS FIRST], [i_product_name#12] - -(19) CometColumnarToRow [codegen id : 1] -Input [1]: [i_product_name#12] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/extended.txt deleted file mode 100644 index 9d802b5033..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 99c935aaa4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name] - CometHashAggregate [i_product_name] - CometExchange [i_product_name] #1 - CometHashAggregate [i_product_name] - CometProject [i_product_name] [i_product_name] - CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - CometBroadcastExchange [i_manufact] #2 - CometProject [i_manufact] - CometFilter [item_cnt,i_manufact] - CometHashAggregate [count] [item_cnt,i_manufact,count(1)] - CometExchange [i_manufact] #3 - CometHashAggregate [i_manufact,count] - CometProject [i_manufact] [i_manufact] - CometFilter [i_category,i_manufact,i_size,i_color,i_units] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_datafusion/explain.txt deleted file mode 100644 index ec72e1b00f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_datafusion/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometNativeScan parquet spark_catalog.default.item (9) - - -(1) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometNativeScan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_category_id#8, i_category#11], [i_item_sk#7, i_category_id#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#9, 50)) AS i_category#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [i_item_sk#7, i_category_id#8, i_category#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11], [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11] -Keys [3]: [d_year#2, i_category_id#8, i_category#11] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] -Arguments: hashpartitioning(d_year#2, i_category_id#8, i_category#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] -Keys [3]: [d_year#2, i_category_id#8, i_category#11] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sum(ss_ext_sales_price)#13 DESC NULLS LAST,d_year#2 ASC NULLS FIRST,i_category_id#8 ASC NULLS FIRST,i_category#11 ASC NULLS FIRST], output=[d_year#2,i_category_id#8,i_category#11,sum(ss_ext_sales_price)#13]), [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13], 100, 0, [sum(ss_ext_sales_price)#13 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_datafusion/simplified.txt deleted file mode 100644 index 96ab34454d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_datafusion/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,i_category_id,i_category,sum(ss_ext_sales_price)] - CometHashAggregate [sum] [d_year,i_category_id,i_category,sum(ss_ext_sales_price),sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_category_id,i_category] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_category_id,i_category,sum] - CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 - CometProject [i_category] [i_item_sk,i_category_id,i_category] - CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] - CometNativeScan 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/q42.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/explain.txt deleted file mode 100644 index add87bddfb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_category_id#8, i_category#11], [i_item_sk#7, i_category_id#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#9, 50)) AS i_category#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [i_item_sk#7, i_category_id#8, i_category#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11], [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11] -Keys [3]: [d_year#2, i_category_id#8, i_category#11] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] -Arguments: hashpartitioning(d_year#2, i_category_id#8, i_category#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] -Keys [3]: [d_year#2, i_category_id#8, i_category#11] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sum(ss_ext_sales_price)#13 DESC NULLS LAST,d_year#2 ASC NULLS FIRST,i_category_id#8 ASC NULLS FIRST,i_category#11 ASC NULLS FIRST], output=[d_year#2,i_category_id#8,i_category#11,sum(ss_ext_sales_price)#13]), [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13], 100, 0, [sum(ss_ext_sales_price)#13 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/simplified.txt deleted file mode 100644 index 839937d40c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,i_category_id,i_category,sum(ss_ext_sales_price)] - CometHashAggregate [sum] [d_year,i_category_id,i_category,sum(ss_ext_sales_price),sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_category_id,i_category] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_category_id,i_category,sum] - CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 - CometProject [i_category] [i_item_sk,i_category_id,i_category] - CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] - CometScan [native_iceberg_compat] 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/q42/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/explain.txt deleted file mode 100644 index add87bddfb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_category_id#8, i_category#11], [i_item_sk#7, i_category_id#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#9, 50)) AS i_category#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [i_item_sk#7, i_category_id#8, i_category#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11], [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11] -Keys [3]: [d_year#2, i_category_id#8, i_category#11] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] -Arguments: hashpartitioning(d_year#2, i_category_id#8, i_category#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] -Keys [3]: [d_year#2, i_category_id#8, i_category#11] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sum(ss_ext_sales_price)#13 DESC NULLS LAST,d_year#2 ASC NULLS FIRST,i_category_id#8 ASC NULLS FIRST,i_category#11 ASC NULLS FIRST], output=[d_year#2,i_category_id#8,i_category#11,sum(ss_ext_sales_price)#13]), [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13], 100, 0, [sum(ss_ext_sales_price)#13 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/extended.txt deleted file mode 100644 index a292badf5d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 839937d40c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,i_category_id,i_category,sum(ss_ext_sales_price)] - CometHashAggregate [sum] [d_year,i_category_id,i_category,sum(ss_ext_sales_price),sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_category_id,i_category] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_category_id,i_category,sum] - CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 - CometProject [i_category] [i_item_sk,i_category_id,i_category] - CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_datafusion/explain.txt deleted file mode 100644 index 1a5c4758da..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_datafusion/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometNativeScan parquet spark_catalog.default.store (9) - - -(1) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Condition : ((isnotnull(d_year#2) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Arguments: [d_date_sk#1, d_day_name#4], [d_date_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(d_day_name#3, 9)) AS d_day_name#4] - -(4) CometNativeScan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_store_sk#5) - -(6) CometBroadcastExchange -Input [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_day_name#4] -Right output [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [d_date_sk#1], [ss_sold_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_day_name#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [d_day_name#4, ss_store_sk#5, ss_sales_price#6], [d_day_name#4, ss_store_sk#5, ss_sales_price#6] - -(9) CometNativeScan parquet spark_catalog.default.store -Output [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Condition : ((isnotnull(s_gmt_offset#11) AND (s_gmt_offset#11 = -5.00)) AND isnotnull(s_store_sk#8)) - -(11) CometProject -Input [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Arguments: [s_store_sk#8, s_store_id#12, s_store_name#10], [s_store_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#9, 16)) AS s_store_id#12, s_store_name#10] - -(12) CometBroadcastExchange -Input [3]: [s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [s_store_sk#8, s_store_id#12, s_store_name#10] - -(13) CometBroadcastHashJoin -Left output [3]: [d_day_name#4, ss_store_sk#5, ss_sales_price#6] -Right output [3]: [s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [ss_store_sk#5], [s_store_sk#8], Inner, BuildRight - -(14) CometProject -Input [6]: [d_day_name#4, ss_store_sk#5, ss_sales_price#6, s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10], [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10] - -(15) CometHashAggregate -Input [4]: [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10] -Keys [2]: [s_store_name#10, s_store_id#12] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))] - -(16) CometExchange -Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_name#10, s_store_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Keys [2]: [s_store_name#10, s_store_id#12] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))] - -(18) CometTakeOrderedAndProject -Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#10 ASC NULLS FIRST,s_store_id#12 ASC NULLS FIRST,sun_sales#20 ASC NULLS FIRST,mon_sales#21 ASC NULLS FIRST,tue_sales#22 ASC NULLS FIRST,wed_sales#23 ASC NULLS FIRST,thu_sales#24 ASC NULLS FIRST,fri_sales#25 ASC NULLS FIRST,sat_sales#26 ASC NULLS FIRST], output=[s_store_name#10,s_store_id#12,sun_sales#20,mon_sales#21,tue_sales#22,wed_sales#23,thu_sales#24,fri_sales#25,sat_sales#26]), [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26], 100, 0, [s_store_name#10 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sun_sales#20 ASC NULLS FIRST, mon_sales#21 ASC NULLS FIRST, tue_sales#22 ASC NULLS FIRST, wed_sales#23 ASC NULLS FIRST, thu_sales#24 ASC NULLS FIRST, fri_sales#25 ASC NULLS FIRST, sat_sales#26 ASC NULLS FIRST], [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] - -(19) CometColumnarToRow [codegen id : 1] -Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_datafusion/simplified.txt deleted file mode 100644 index c7064b3ad8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_datafusion/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - CometExchange [s_store_name,s_store_id] #1 - CometHashAggregate [d_day_name,ss_sales_price] [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] - CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - 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,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [d_day_name] [d_date_sk,d_day_name] - CometFilter [d_date_sk,d_year,d_day_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] - CometNativeScan 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/q43.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/explain.txt deleted file mode 100644 index ae3885cc8a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Condition : ((isnotnull(d_year#2) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Arguments: [d_date_sk#1, d_day_name#4], [d_date_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(d_day_name#3, 9)) AS d_day_name#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_store_sk#5) - -(6) CometBroadcastExchange -Input [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_day_name#4] -Right output [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [d_date_sk#1], [ss_sold_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_day_name#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [d_day_name#4, ss_store_sk#5, ss_sales_price#6], [d_day_name#4, ss_store_sk#5, ss_sales_price#6] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Condition : ((isnotnull(s_gmt_offset#11) AND (s_gmt_offset#11 = -5.00)) AND isnotnull(s_store_sk#8)) - -(11) CometProject -Input [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Arguments: [s_store_sk#8, s_store_id#12, s_store_name#10], [s_store_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#9, 16)) AS s_store_id#12, s_store_name#10] - -(12) CometBroadcastExchange -Input [3]: [s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [s_store_sk#8, s_store_id#12, s_store_name#10] - -(13) CometBroadcastHashJoin -Left output [3]: [d_day_name#4, ss_store_sk#5, ss_sales_price#6] -Right output [3]: [s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [ss_store_sk#5], [s_store_sk#8], Inner, BuildRight - -(14) CometProject -Input [6]: [d_day_name#4, ss_store_sk#5, ss_sales_price#6, s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10], [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10] - -(15) CometHashAggregate -Input [4]: [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10] -Keys [2]: [s_store_name#10, s_store_id#12] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))] - -(16) CometExchange -Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_name#10, s_store_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Keys [2]: [s_store_name#10, s_store_id#12] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))] - -(18) CometTakeOrderedAndProject -Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#10 ASC NULLS FIRST,s_store_id#12 ASC NULLS FIRST,sun_sales#20 ASC NULLS FIRST,mon_sales#21 ASC NULLS FIRST,tue_sales#22 ASC NULLS FIRST,wed_sales#23 ASC NULLS FIRST,thu_sales#24 ASC NULLS FIRST,fri_sales#25 ASC NULLS FIRST,sat_sales#26 ASC NULLS FIRST], output=[s_store_name#10,s_store_id#12,sun_sales#20,mon_sales#21,tue_sales#22,wed_sales#23,thu_sales#24,fri_sales#25,sat_sales#26]), [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26], 100, 0, [s_store_name#10 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sun_sales#20 ASC NULLS FIRST, mon_sales#21 ASC NULLS FIRST, tue_sales#22 ASC NULLS FIRST, wed_sales#23 ASC NULLS FIRST, thu_sales#24 ASC NULLS FIRST, fri_sales#25 ASC NULLS FIRST, sat_sales#26 ASC NULLS FIRST], [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] - -(19) CometColumnarToRow [codegen id : 1] -Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/simplified.txt deleted file mode 100644 index 59ad7611a5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - CometExchange [s_store_name,s_store_id] #1 - CometHashAggregate [d_day_name,ss_sales_price] [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] - CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - 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,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [d_day_name] [d_date_sk,d_day_name] - CometFilter [d_date_sk,d_year,d_day_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] - CometScan [native_iceberg_compat] 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/q43/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/explain.txt deleted file mode 100644 index ae3885cc8a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Condition : ((isnotnull(d_year#2) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Arguments: [d_date_sk#1, d_day_name#4], [d_date_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(d_day_name#3, 9)) AS d_day_name#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_store_sk#5) - -(6) CometBroadcastExchange -Input [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_day_name#4] -Right output [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [d_date_sk#1], [ss_sold_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_day_name#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [d_day_name#4, ss_store_sk#5, ss_sales_price#6], [d_day_name#4, ss_store_sk#5, ss_sales_price#6] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Condition : ((isnotnull(s_gmt_offset#11) AND (s_gmt_offset#11 = -5.00)) AND isnotnull(s_store_sk#8)) - -(11) CometProject -Input [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Arguments: [s_store_sk#8, s_store_id#12, s_store_name#10], [s_store_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#9, 16)) AS s_store_id#12, s_store_name#10] - -(12) CometBroadcastExchange -Input [3]: [s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [s_store_sk#8, s_store_id#12, s_store_name#10] - -(13) CometBroadcastHashJoin -Left output [3]: [d_day_name#4, ss_store_sk#5, ss_sales_price#6] -Right output [3]: [s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [ss_store_sk#5], [s_store_sk#8], Inner, BuildRight - -(14) CometProject -Input [6]: [d_day_name#4, ss_store_sk#5, ss_sales_price#6, s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10], [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10] - -(15) CometHashAggregate -Input [4]: [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10] -Keys [2]: [s_store_name#10, s_store_id#12] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))] - -(16) CometExchange -Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_name#10, s_store_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Keys [2]: [s_store_name#10, s_store_id#12] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))] - -(18) CometTakeOrderedAndProject -Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#10 ASC NULLS FIRST,s_store_id#12 ASC NULLS FIRST,sun_sales#20 ASC NULLS FIRST,mon_sales#21 ASC NULLS FIRST,tue_sales#22 ASC NULLS FIRST,wed_sales#23 ASC NULLS FIRST,thu_sales#24 ASC NULLS FIRST,fri_sales#25 ASC NULLS FIRST,sat_sales#26 ASC NULLS FIRST], output=[s_store_name#10,s_store_id#12,sun_sales#20,mon_sales#21,tue_sales#22,wed_sales#23,thu_sales#24,fri_sales#25,sat_sales#26]), [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26], 100, 0, [s_store_name#10 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sun_sales#20 ASC NULLS FIRST, mon_sales#21 ASC NULLS FIRST, tue_sales#22 ASC NULLS FIRST, wed_sales#23 ASC NULLS FIRST, thu_sales#24 ASC NULLS FIRST, fri_sales#25 ASC NULLS FIRST, sat_sales#26 ASC NULLS FIRST], [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] - -(19) CometColumnarToRow [codegen id : 1] -Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/extended.txt deleted file mode 100644 index cb125e2456..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 59ad7611a5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - CometExchange [s_store_name,s_store_id] #1 - CometHashAggregate [d_day_name,ss_sales_price] [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] - CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - 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,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [d_day_name] [d_date_sk,d_day_name] - CometFilter [d_date_sk,d_year,d_day_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_datafusion/explain.txt deleted file mode 100644 index 835c6f13f7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_datafusion/explain.txt +++ /dev/null @@ -1,290 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometTakeOrderedAndProject (46) - +- CometProject (45) - +- CometBroadcastHashJoin (44) - :- CometProject (42) - : +- CometBroadcastHashJoin (41) - : :- CometProject (36) - : : +- CometSortMergeJoin (35) - : : :- CometSort (19) - : : : +- CometColumnarExchange (18) - : : : +- * Project (17) - : : : +- * Filter (16) - : : : +- Window (15) - : : : +- WindowGroupLimit (14) - : : : +- * CometColumnarToRow (13) - : : : +- CometSort (12) - : : : +- CometColumnarExchange (11) - : : : +- WindowGroupLimit (10) - : : : +- * CometColumnarToRow (9) - : : : +- CometSort (8) - : : : +- CometFilter (7) - : : : +- CometHashAggregate (6) - : : : +- CometExchange (5) - : : : +- CometHashAggregate (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : +- CometSort (34) - : : +- CometColumnarExchange (33) - : : +- * Project (32) - : : +- * Filter (31) - : : +- Window (30) - : : +- WindowGroupLimit (29) - : : +- * CometColumnarToRow (28) - : : +- CometSort (27) - : : +- CometColumnarExchange (26) - : : +- WindowGroupLimit (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometFilter (22) - : : +- CometHashAggregate (21) - : : +- ReusedExchange (20) - : +- CometBroadcastExchange (40) - : +- CometProject (39) - : +- CometFilter (38) - : +- CometNativeScan parquet spark_catalog.default.item (37) - +- ReusedExchange (43) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4)) - -(3) CometProject -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Arguments: [ss_item_sk#1, ss_net_profit#3], [ss_item_sk#1, ss_net_profit#3] - -(4) CometHashAggregate -Input [2]: [ss_item_sk#1, ss_net_profit#3] -Keys [1]: [ss_item_sk#1] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] - -(5) CometExchange -Input [3]: [ss_item_sk#1, sum#5, count#6] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(6) CometHashAggregate -Input [3]: [ss_item_sk#1, sum#5, count#6] -Keys [1]: [ss_item_sk#1] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] - -(7) CometFilter -Input [2]: [item_sk#7, rank_col#8] -Condition : (isnotnull(rank_col#8) AND (cast(rank_col#8 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#9, [id=#10]))) - -(8) CometSort -Input [2]: [item_sk#7, rank_col#8] -Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] - -(9) CometColumnarToRow [codegen id : 1] -Input [2]: [item_sk#7, rank_col#8] - -(10) WindowGroupLimit -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank_col#8 ASC NULLS FIRST], rank(rank_col#8), 10, Partial - -(11) CometColumnarExchange -Input [2]: [item_sk#7, rank_col#8] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(12) CometSort -Input [2]: [item_sk#7, rank_col#8] -Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] - -(13) CometColumnarToRow [codegen id : 2] -Input [2]: [item_sk#7, rank_col#8] - -(14) WindowGroupLimit -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank_col#8 ASC NULLS FIRST], rank(rank_col#8), 10, Final - -(15) Window -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] - -(16) Filter [codegen id : 3] -Input [3]: [item_sk#7, rank_col#8, rnk#11] -Condition : ((rnk#11 < 11) AND isnotnull(item_sk#7)) - -(17) Project [codegen id : 3] -Output [2]: [item_sk#7, rnk#11] -Input [3]: [item_sk#7, rank_col#8, rnk#11] - -(18) CometColumnarExchange -Input [2]: [item_sk#7, rnk#11] -Arguments: hashpartitioning(rnk#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(19) CometSort -Input [2]: [item_sk#7, rnk#11] -Arguments: [item_sk#7, rnk#11], [rnk#11 ASC NULLS FIRST] - -(20) ReusedExchange [Reuses operator id: 5] -Output [3]: [ss_item_sk#12, sum#13, count#14] - -(21) CometHashAggregate -Input [3]: [ss_item_sk#12, sum#13, count#14] -Keys [1]: [ss_item_sk#12] -Functions [1]: [avg(UnscaledValue(ss_net_profit#15))] - -(22) CometFilter -Input [2]: [item_sk#16, rank_col#17] -Condition : (isnotnull(rank_col#17) AND (cast(rank_col#17 as decimal(13,7)) > (0.9 * ReusedSubquery Subquery scalar-subquery#9, [id=#10]))) - -(23) CometSort -Input [2]: [item_sk#16, rank_col#17] -Arguments: [item_sk#16, rank_col#17], [rank_col#17 DESC NULLS LAST] - -(24) CometColumnarToRow [codegen id : 4] -Input [2]: [item_sk#16, rank_col#17] - -(25) WindowGroupLimit -Input [2]: [item_sk#16, rank_col#17] -Arguments: [rank_col#17 DESC NULLS LAST], rank(rank_col#17), 10, Partial - -(26) CometColumnarExchange -Input [2]: [item_sk#16, rank_col#17] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometSort -Input [2]: [item_sk#16, rank_col#17] -Arguments: [item_sk#16, rank_col#17], [rank_col#17 DESC NULLS LAST] - -(28) CometColumnarToRow [codegen id : 5] -Input [2]: [item_sk#16, rank_col#17] - -(29) WindowGroupLimit -Input [2]: [item_sk#16, rank_col#17] -Arguments: [rank_col#17 DESC NULLS LAST], rank(rank_col#17), 10, Final - -(30) Window -Input [2]: [item_sk#16, rank_col#17] -Arguments: [rank(rank_col#17) windowspecdefinition(rank_col#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#18], [rank_col#17 DESC NULLS LAST] - -(31) Filter [codegen id : 6] -Input [3]: [item_sk#16, rank_col#17, rnk#18] -Condition : ((rnk#18 < 11) AND isnotnull(item_sk#16)) - -(32) Project [codegen id : 6] -Output [2]: [item_sk#16, rnk#18] -Input [3]: [item_sk#16, rank_col#17, rnk#18] - -(33) CometColumnarExchange -Input [2]: [item_sk#16, rnk#18] -Arguments: hashpartitioning(rnk#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(34) CometSort -Input [2]: [item_sk#16, rnk#18] -Arguments: [item_sk#16, rnk#18], [rnk#18 ASC NULLS FIRST] - -(35) CometSortMergeJoin -Left output [2]: [item_sk#7, rnk#11] -Right output [2]: [item_sk#16, rnk#18] -Arguments: [rnk#11], [rnk#18], Inner - -(36) CometProject -Input [4]: [item_sk#7, rnk#11, item_sk#16, rnk#18] -Arguments: [item_sk#7, rnk#11, item_sk#16], [item_sk#7, rnk#11, item_sk#16] - -(37) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_product_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(38) CometFilter -Input [2]: [i_item_sk#19, i_product_name#20] -Condition : isnotnull(i_item_sk#19) - -(39) CometProject -Input [2]: [i_item_sk#19, i_product_name#20] -Arguments: [i_item_sk#19, i_product_name#21], [i_item_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#20, 50)) AS i_product_name#21] - -(40) CometBroadcastExchange -Input [2]: [i_item_sk#19, i_product_name#21] -Arguments: [i_item_sk#19, i_product_name#21] - -(41) CometBroadcastHashJoin -Left output [3]: [item_sk#7, rnk#11, item_sk#16] -Right output [2]: [i_item_sk#19, i_product_name#21] -Arguments: [item_sk#7], [i_item_sk#19], Inner, BuildRight - -(42) CometProject -Input [5]: [item_sk#7, rnk#11, item_sk#16, i_item_sk#19, i_product_name#21] -Arguments: [rnk#11, item_sk#16, i_product_name#21], [rnk#11, item_sk#16, i_product_name#21] - -(43) ReusedExchange [Reuses operator id: 40] -Output [2]: [i_item_sk#22, i_product_name#23] - -(44) CometBroadcastHashJoin -Left output [3]: [rnk#11, item_sk#16, i_product_name#21] -Right output [2]: [i_item_sk#22, i_product_name#23] -Arguments: [item_sk#16], [i_item_sk#22], Inner, BuildRight - -(45) CometProject -Input [5]: [rnk#11, item_sk#16, i_product_name#21, i_item_sk#22, i_product_name#23] -Arguments: [rnk#11, best_performing#24, worst_performing#25], [rnk#11, i_product_name#21 AS best_performing#24, i_product_name#23 AS worst_performing#25] - -(46) CometTakeOrderedAndProject -Input [3]: [rnk#11, best_performing#24, worst_performing#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[rnk#11 ASC NULLS FIRST], output=[rnk#11,best_performing#24,worst_performing#25]), [rnk#11, best_performing#24, worst_performing#25], 100, 0, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#24, worst_performing#25] - -(47) CometColumnarToRow [codegen id : 7] -Input [3]: [rnk#11, best_performing#24, worst_performing#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* CometColumnarToRow (54) -+- CometHashAggregate (53) - +- CometExchange (52) - +- CometHashAggregate (51) - +- CometProject (50) - +- CometFilter (49) - +- CometNativeScan parquet spark_catalog.default.store_sales (48) - - -(48) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_addr_sk#26, ss_store_sk#27, ss_net_profit#28, ss_sold_date_sk#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [ss_addr_sk#26, ss_store_sk#27, ss_net_profit#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_store_sk#27) AND (ss_store_sk#27 = 4)) AND isnull(ss_addr_sk#26)) - -(50) CometProject -Input [4]: [ss_addr_sk#26, ss_store_sk#27, ss_net_profit#28, ss_sold_date_sk#29] -Arguments: [ss_store_sk#27, ss_net_profit#28], [ss_store_sk#27, ss_net_profit#28] - -(51) CometHashAggregate -Input [2]: [ss_store_sk#27, ss_net_profit#28] -Keys [1]: [ss_store_sk#27] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#28))] - -(52) CometExchange -Input [3]: [ss_store_sk#27, sum#30, count#31] -Arguments: hashpartitioning(ss_store_sk#27, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(53) CometHashAggregate -Input [3]: [ss_store_sk#27, sum#30, count#31] -Keys [1]: [ss_store_sk#27] -Functions [1]: [avg(UnscaledValue(ss_net_profit#28))] - -(54) CometColumnarToRow [codegen id : 1] -Input [1]: [rank_col#32] - -Subquery:2 Hosting operator id = 22 Hosting Expression = ReusedSubquery Subquery scalar-subquery#9, [id=#10] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_datafusion/simplified.txt deleted file mode 100644 index 02d8749188..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_datafusion/simplified.txt +++ /dev/null @@ -1,72 +0,0 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [rnk,best_performing,worst_performing] - CometProject [i_product_name,i_product_name] [rnk,best_performing,worst_performing] - CometBroadcastHashJoin [rnk,item_sk,i_product_name,i_item_sk,i_product_name] - CometProject [rnk,item_sk,i_product_name] - CometBroadcastHashJoin [item_sk,rnk,item_sk,i_item_sk,i_product_name] - CometProject [item_sk,rnk,item_sk] - CometSortMergeJoin [item_sk,rnk,item_sk,rnk] - CometSort [item_sk,rnk] - CometColumnarExchange [rnk] #1 - WholeStageCodegen (3) - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WindowGroupLimit [rank_col] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometColumnarExchange #2 - WindowGroupLimit [rank_col] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometFilter [item_sk,rank_col] - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_store_sk] #4 - CometHashAggregate [ss_net_profit] [ss_store_sk,sum,count] - CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_item_sk] #3 - CometHashAggregate [ss_net_profit] [ss_item_sk,sum,count] - CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometSort [item_sk,rnk] - CometColumnarExchange [rnk] #5 - WholeStageCodegen (6) - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WindowGroupLimit [rank_col] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometColumnarExchange #6 - WindowGroupLimit [rank_col] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometFilter [item_sk,rank_col] - ReusedSubquery [rank_col] #1 - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - ReusedExchange [ss_item_sk,sum,count] #3 - CometBroadcastExchange [i_item_sk,i_product_name] #7 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_product_name] - ReusedExchange [i_item_sk,i_product_name] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_iceberg_compat/explain.txt deleted file mode 100644 index 5f8d469668..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,290 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometTakeOrderedAndProject (46) - +- CometProject (45) - +- CometBroadcastHashJoin (44) - :- CometProject (42) - : +- CometBroadcastHashJoin (41) - : :- CometProject (36) - : : +- CometSortMergeJoin (35) - : : :- CometSort (19) - : : : +- CometColumnarExchange (18) - : : : +- * Project (17) - : : : +- * Filter (16) - : : : +- Window (15) - : : : +- WindowGroupLimit (14) - : : : +- * CometColumnarToRow (13) - : : : +- CometSort (12) - : : : +- CometColumnarExchange (11) - : : : +- WindowGroupLimit (10) - : : : +- * CometColumnarToRow (9) - : : : +- CometSort (8) - : : : +- CometFilter (7) - : : : +- CometHashAggregate (6) - : : : +- CometExchange (5) - : : : +- CometHashAggregate (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometSort (34) - : : +- CometColumnarExchange (33) - : : +- * Project (32) - : : +- * Filter (31) - : : +- Window (30) - : : +- WindowGroupLimit (29) - : : +- * CometColumnarToRow (28) - : : +- CometSort (27) - : : +- CometColumnarExchange (26) - : : +- WindowGroupLimit (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometFilter (22) - : : +- CometHashAggregate (21) - : : +- ReusedExchange (20) - : +- CometBroadcastExchange (40) - : +- CometProject (39) - : +- CometFilter (38) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (37) - +- ReusedExchange (43) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4)) - -(3) CometProject -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Arguments: [ss_item_sk#1, ss_net_profit#3], [ss_item_sk#1, ss_net_profit#3] - -(4) CometHashAggregate -Input [2]: [ss_item_sk#1, ss_net_profit#3] -Keys [1]: [ss_item_sk#1] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] - -(5) CometExchange -Input [3]: [ss_item_sk#1, sum#5, count#6] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(6) CometHashAggregate -Input [3]: [ss_item_sk#1, sum#5, count#6] -Keys [1]: [ss_item_sk#1] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] - -(7) CometFilter -Input [2]: [item_sk#7, rank_col#8] -Condition : (isnotnull(rank_col#8) AND (cast(rank_col#8 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#9, [id=#10]))) - -(8) CometSort -Input [2]: [item_sk#7, rank_col#8] -Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] - -(9) CometColumnarToRow [codegen id : 1] -Input [2]: [item_sk#7, rank_col#8] - -(10) WindowGroupLimit -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank_col#8 ASC NULLS FIRST], rank(rank_col#8), 10, Partial - -(11) CometColumnarExchange -Input [2]: [item_sk#7, rank_col#8] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(12) CometSort -Input [2]: [item_sk#7, rank_col#8] -Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] - -(13) CometColumnarToRow [codegen id : 2] -Input [2]: [item_sk#7, rank_col#8] - -(14) WindowGroupLimit -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank_col#8 ASC NULLS FIRST], rank(rank_col#8), 10, Final - -(15) Window -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] - -(16) Filter [codegen id : 3] -Input [3]: [item_sk#7, rank_col#8, rnk#11] -Condition : ((rnk#11 < 11) AND isnotnull(item_sk#7)) - -(17) Project [codegen id : 3] -Output [2]: [item_sk#7, rnk#11] -Input [3]: [item_sk#7, rank_col#8, rnk#11] - -(18) CometColumnarExchange -Input [2]: [item_sk#7, rnk#11] -Arguments: hashpartitioning(rnk#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(19) CometSort -Input [2]: [item_sk#7, rnk#11] -Arguments: [item_sk#7, rnk#11], [rnk#11 ASC NULLS FIRST] - -(20) ReusedExchange [Reuses operator id: 5] -Output [3]: [ss_item_sk#12, sum#13, count#14] - -(21) CometHashAggregate -Input [3]: [ss_item_sk#12, sum#13, count#14] -Keys [1]: [ss_item_sk#12] -Functions [1]: [avg(UnscaledValue(ss_net_profit#15))] - -(22) CometFilter -Input [2]: [item_sk#16, rank_col#17] -Condition : (isnotnull(rank_col#17) AND (cast(rank_col#17 as decimal(13,7)) > (0.9 * ReusedSubquery Subquery scalar-subquery#9, [id=#10]))) - -(23) CometSort -Input [2]: [item_sk#16, rank_col#17] -Arguments: [item_sk#16, rank_col#17], [rank_col#17 DESC NULLS LAST] - -(24) CometColumnarToRow [codegen id : 4] -Input [2]: [item_sk#16, rank_col#17] - -(25) WindowGroupLimit -Input [2]: [item_sk#16, rank_col#17] -Arguments: [rank_col#17 DESC NULLS LAST], rank(rank_col#17), 10, Partial - -(26) CometColumnarExchange -Input [2]: [item_sk#16, rank_col#17] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometSort -Input [2]: [item_sk#16, rank_col#17] -Arguments: [item_sk#16, rank_col#17], [rank_col#17 DESC NULLS LAST] - -(28) CometColumnarToRow [codegen id : 5] -Input [2]: [item_sk#16, rank_col#17] - -(29) WindowGroupLimit -Input [2]: [item_sk#16, rank_col#17] -Arguments: [rank_col#17 DESC NULLS LAST], rank(rank_col#17), 10, Final - -(30) Window -Input [2]: [item_sk#16, rank_col#17] -Arguments: [rank(rank_col#17) windowspecdefinition(rank_col#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#18], [rank_col#17 DESC NULLS LAST] - -(31) Filter [codegen id : 6] -Input [3]: [item_sk#16, rank_col#17, rnk#18] -Condition : ((rnk#18 < 11) AND isnotnull(item_sk#16)) - -(32) Project [codegen id : 6] -Output [2]: [item_sk#16, rnk#18] -Input [3]: [item_sk#16, rank_col#17, rnk#18] - -(33) CometColumnarExchange -Input [2]: [item_sk#16, rnk#18] -Arguments: hashpartitioning(rnk#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(34) CometSort -Input [2]: [item_sk#16, rnk#18] -Arguments: [item_sk#16, rnk#18], [rnk#18 ASC NULLS FIRST] - -(35) CometSortMergeJoin -Left output [2]: [item_sk#7, rnk#11] -Right output [2]: [item_sk#16, rnk#18] -Arguments: [rnk#11], [rnk#18], Inner - -(36) CometProject -Input [4]: [item_sk#7, rnk#11, item_sk#16, rnk#18] -Arguments: [item_sk#7, rnk#11, item_sk#16], [item_sk#7, rnk#11, item_sk#16] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_product_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(38) CometFilter -Input [2]: [i_item_sk#19, i_product_name#20] -Condition : isnotnull(i_item_sk#19) - -(39) CometProject -Input [2]: [i_item_sk#19, i_product_name#20] -Arguments: [i_item_sk#19, i_product_name#21], [i_item_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#20, 50)) AS i_product_name#21] - -(40) CometBroadcastExchange -Input [2]: [i_item_sk#19, i_product_name#21] -Arguments: [i_item_sk#19, i_product_name#21] - -(41) CometBroadcastHashJoin -Left output [3]: [item_sk#7, rnk#11, item_sk#16] -Right output [2]: [i_item_sk#19, i_product_name#21] -Arguments: [item_sk#7], [i_item_sk#19], Inner, BuildRight - -(42) CometProject -Input [5]: [item_sk#7, rnk#11, item_sk#16, i_item_sk#19, i_product_name#21] -Arguments: [rnk#11, item_sk#16, i_product_name#21], [rnk#11, item_sk#16, i_product_name#21] - -(43) ReusedExchange [Reuses operator id: 40] -Output [2]: [i_item_sk#22, i_product_name#23] - -(44) CometBroadcastHashJoin -Left output [3]: [rnk#11, item_sk#16, i_product_name#21] -Right output [2]: [i_item_sk#22, i_product_name#23] -Arguments: [item_sk#16], [i_item_sk#22], Inner, BuildRight - -(45) CometProject -Input [5]: [rnk#11, item_sk#16, i_product_name#21, i_item_sk#22, i_product_name#23] -Arguments: [rnk#11, best_performing#24, worst_performing#25], [rnk#11, i_product_name#21 AS best_performing#24, i_product_name#23 AS worst_performing#25] - -(46) CometTakeOrderedAndProject -Input [3]: [rnk#11, best_performing#24, worst_performing#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[rnk#11 ASC NULLS FIRST], output=[rnk#11,best_performing#24,worst_performing#25]), [rnk#11, best_performing#24, worst_performing#25], 100, 0, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#24, worst_performing#25] - -(47) CometColumnarToRow [codegen id : 7] -Input [3]: [rnk#11, best_performing#24, worst_performing#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* CometColumnarToRow (54) -+- CometHashAggregate (53) - +- CometExchange (52) - +- CometHashAggregate (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_addr_sk#26, ss_store_sk#27, ss_net_profit#28, ss_sold_date_sk#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [ss_addr_sk#26, ss_store_sk#27, ss_net_profit#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_store_sk#27) AND (ss_store_sk#27 = 4)) AND isnull(ss_addr_sk#26)) - -(50) CometProject -Input [4]: [ss_addr_sk#26, ss_store_sk#27, ss_net_profit#28, ss_sold_date_sk#29] -Arguments: [ss_store_sk#27, ss_net_profit#28], [ss_store_sk#27, ss_net_profit#28] - -(51) CometHashAggregate -Input [2]: [ss_store_sk#27, ss_net_profit#28] -Keys [1]: [ss_store_sk#27] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#28))] - -(52) CometExchange -Input [3]: [ss_store_sk#27, sum#30, count#31] -Arguments: hashpartitioning(ss_store_sk#27, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(53) CometHashAggregate -Input [3]: [ss_store_sk#27, sum#30, count#31] -Keys [1]: [ss_store_sk#27] -Functions [1]: [avg(UnscaledValue(ss_net_profit#28))] - -(54) CometColumnarToRow [codegen id : 1] -Input [1]: [rank_col#32] - -Subquery:2 Hosting operator id = 22 Hosting Expression = ReusedSubquery Subquery scalar-subquery#9, [id=#10] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_iceberg_compat/simplified.txt deleted file mode 100644 index dba03cff02..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,72 +0,0 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [rnk,best_performing,worst_performing] - CometProject [i_product_name,i_product_name] [rnk,best_performing,worst_performing] - CometBroadcastHashJoin [rnk,item_sk,i_product_name,i_item_sk,i_product_name] - CometProject [rnk,item_sk,i_product_name] - CometBroadcastHashJoin [item_sk,rnk,item_sk,i_item_sk,i_product_name] - CometProject [item_sk,rnk,item_sk] - CometSortMergeJoin [item_sk,rnk,item_sk,rnk] - CometSort [item_sk,rnk] - CometColumnarExchange [rnk] #1 - WholeStageCodegen (3) - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WindowGroupLimit [rank_col] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometColumnarExchange #2 - WindowGroupLimit [rank_col] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometFilter [item_sk,rank_col] - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_store_sk] #4 - CometHashAggregate [ss_net_profit] [ss_store_sk,sum,count] - CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_item_sk] #3 - CometHashAggregate [ss_net_profit] [ss_item_sk,sum,count] - CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometSort [item_sk,rnk] - CometColumnarExchange [rnk] #5 - WholeStageCodegen (6) - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WindowGroupLimit [rank_col] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometColumnarExchange #6 - WindowGroupLimit [rank_col] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometFilter [item_sk,rank_col] - ReusedSubquery [rank_col] #1 - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - ReusedExchange [ss_item_sk,sum,count] #3 - CometBroadcastExchange [i_item_sk,i_product_name] #7 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_product_name] - ReusedExchange [i_item_sk,i_product_name] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/explain.txt deleted file mode 100644 index 5f8d469668..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/explain.txt +++ /dev/null @@ -1,290 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometTakeOrderedAndProject (46) - +- CometProject (45) - +- CometBroadcastHashJoin (44) - :- CometProject (42) - : +- CometBroadcastHashJoin (41) - : :- CometProject (36) - : : +- CometSortMergeJoin (35) - : : :- CometSort (19) - : : : +- CometColumnarExchange (18) - : : : +- * Project (17) - : : : +- * Filter (16) - : : : +- Window (15) - : : : +- WindowGroupLimit (14) - : : : +- * CometColumnarToRow (13) - : : : +- CometSort (12) - : : : +- CometColumnarExchange (11) - : : : +- WindowGroupLimit (10) - : : : +- * CometColumnarToRow (9) - : : : +- CometSort (8) - : : : +- CometFilter (7) - : : : +- CometHashAggregate (6) - : : : +- CometExchange (5) - : : : +- CometHashAggregate (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometSort (34) - : : +- CometColumnarExchange (33) - : : +- * Project (32) - : : +- * Filter (31) - : : +- Window (30) - : : +- WindowGroupLimit (29) - : : +- * CometColumnarToRow (28) - : : +- CometSort (27) - : : +- CometColumnarExchange (26) - : : +- WindowGroupLimit (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometFilter (22) - : : +- CometHashAggregate (21) - : : +- ReusedExchange (20) - : +- CometBroadcastExchange (40) - : +- CometProject (39) - : +- CometFilter (38) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (37) - +- ReusedExchange (43) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4)) - -(3) CometProject -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Arguments: [ss_item_sk#1, ss_net_profit#3], [ss_item_sk#1, ss_net_profit#3] - -(4) CometHashAggregate -Input [2]: [ss_item_sk#1, ss_net_profit#3] -Keys [1]: [ss_item_sk#1] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] - -(5) CometExchange -Input [3]: [ss_item_sk#1, sum#5, count#6] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(6) CometHashAggregate -Input [3]: [ss_item_sk#1, sum#5, count#6] -Keys [1]: [ss_item_sk#1] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] - -(7) CometFilter -Input [2]: [item_sk#7, rank_col#8] -Condition : (isnotnull(rank_col#8) AND (cast(rank_col#8 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#9, [id=#10]))) - -(8) CometSort -Input [2]: [item_sk#7, rank_col#8] -Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] - -(9) CometColumnarToRow [codegen id : 1] -Input [2]: [item_sk#7, rank_col#8] - -(10) WindowGroupLimit -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank_col#8 ASC NULLS FIRST], rank(rank_col#8), 10, Partial - -(11) CometColumnarExchange -Input [2]: [item_sk#7, rank_col#8] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(12) CometSort -Input [2]: [item_sk#7, rank_col#8] -Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] - -(13) CometColumnarToRow [codegen id : 2] -Input [2]: [item_sk#7, rank_col#8] - -(14) WindowGroupLimit -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank_col#8 ASC NULLS FIRST], rank(rank_col#8), 10, Final - -(15) Window -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] - -(16) Filter [codegen id : 3] -Input [3]: [item_sk#7, rank_col#8, rnk#11] -Condition : ((rnk#11 < 11) AND isnotnull(item_sk#7)) - -(17) Project [codegen id : 3] -Output [2]: [item_sk#7, rnk#11] -Input [3]: [item_sk#7, rank_col#8, rnk#11] - -(18) CometColumnarExchange -Input [2]: [item_sk#7, rnk#11] -Arguments: hashpartitioning(rnk#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(19) CometSort -Input [2]: [item_sk#7, rnk#11] -Arguments: [item_sk#7, rnk#11], [rnk#11 ASC NULLS FIRST] - -(20) ReusedExchange [Reuses operator id: 5] -Output [3]: [ss_item_sk#12, sum#13, count#14] - -(21) CometHashAggregate -Input [3]: [ss_item_sk#12, sum#13, count#14] -Keys [1]: [ss_item_sk#12] -Functions [1]: [avg(UnscaledValue(ss_net_profit#15))] - -(22) CometFilter -Input [2]: [item_sk#16, rank_col#17] -Condition : (isnotnull(rank_col#17) AND (cast(rank_col#17 as decimal(13,7)) > (0.9 * ReusedSubquery Subquery scalar-subquery#9, [id=#10]))) - -(23) CometSort -Input [2]: [item_sk#16, rank_col#17] -Arguments: [item_sk#16, rank_col#17], [rank_col#17 DESC NULLS LAST] - -(24) CometColumnarToRow [codegen id : 4] -Input [2]: [item_sk#16, rank_col#17] - -(25) WindowGroupLimit -Input [2]: [item_sk#16, rank_col#17] -Arguments: [rank_col#17 DESC NULLS LAST], rank(rank_col#17), 10, Partial - -(26) CometColumnarExchange -Input [2]: [item_sk#16, rank_col#17] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometSort -Input [2]: [item_sk#16, rank_col#17] -Arguments: [item_sk#16, rank_col#17], [rank_col#17 DESC NULLS LAST] - -(28) CometColumnarToRow [codegen id : 5] -Input [2]: [item_sk#16, rank_col#17] - -(29) WindowGroupLimit -Input [2]: [item_sk#16, rank_col#17] -Arguments: [rank_col#17 DESC NULLS LAST], rank(rank_col#17), 10, Final - -(30) Window -Input [2]: [item_sk#16, rank_col#17] -Arguments: [rank(rank_col#17) windowspecdefinition(rank_col#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#18], [rank_col#17 DESC NULLS LAST] - -(31) Filter [codegen id : 6] -Input [3]: [item_sk#16, rank_col#17, rnk#18] -Condition : ((rnk#18 < 11) AND isnotnull(item_sk#16)) - -(32) Project [codegen id : 6] -Output [2]: [item_sk#16, rnk#18] -Input [3]: [item_sk#16, rank_col#17, rnk#18] - -(33) CometColumnarExchange -Input [2]: [item_sk#16, rnk#18] -Arguments: hashpartitioning(rnk#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(34) CometSort -Input [2]: [item_sk#16, rnk#18] -Arguments: [item_sk#16, rnk#18], [rnk#18 ASC NULLS FIRST] - -(35) CometSortMergeJoin -Left output [2]: [item_sk#7, rnk#11] -Right output [2]: [item_sk#16, rnk#18] -Arguments: [rnk#11], [rnk#18], Inner - -(36) CometProject -Input [4]: [item_sk#7, rnk#11, item_sk#16, rnk#18] -Arguments: [item_sk#7, rnk#11, item_sk#16], [item_sk#7, rnk#11, item_sk#16] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_product_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(38) CometFilter -Input [2]: [i_item_sk#19, i_product_name#20] -Condition : isnotnull(i_item_sk#19) - -(39) CometProject -Input [2]: [i_item_sk#19, i_product_name#20] -Arguments: [i_item_sk#19, i_product_name#21], [i_item_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#20, 50)) AS i_product_name#21] - -(40) CometBroadcastExchange -Input [2]: [i_item_sk#19, i_product_name#21] -Arguments: [i_item_sk#19, i_product_name#21] - -(41) CometBroadcastHashJoin -Left output [3]: [item_sk#7, rnk#11, item_sk#16] -Right output [2]: [i_item_sk#19, i_product_name#21] -Arguments: [item_sk#7], [i_item_sk#19], Inner, BuildRight - -(42) CometProject -Input [5]: [item_sk#7, rnk#11, item_sk#16, i_item_sk#19, i_product_name#21] -Arguments: [rnk#11, item_sk#16, i_product_name#21], [rnk#11, item_sk#16, i_product_name#21] - -(43) ReusedExchange [Reuses operator id: 40] -Output [2]: [i_item_sk#22, i_product_name#23] - -(44) CometBroadcastHashJoin -Left output [3]: [rnk#11, item_sk#16, i_product_name#21] -Right output [2]: [i_item_sk#22, i_product_name#23] -Arguments: [item_sk#16], [i_item_sk#22], Inner, BuildRight - -(45) CometProject -Input [5]: [rnk#11, item_sk#16, i_product_name#21, i_item_sk#22, i_product_name#23] -Arguments: [rnk#11, best_performing#24, worst_performing#25], [rnk#11, i_product_name#21 AS best_performing#24, i_product_name#23 AS worst_performing#25] - -(46) CometTakeOrderedAndProject -Input [3]: [rnk#11, best_performing#24, worst_performing#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[rnk#11 ASC NULLS FIRST], output=[rnk#11,best_performing#24,worst_performing#25]), [rnk#11, best_performing#24, worst_performing#25], 100, 0, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#24, worst_performing#25] - -(47) CometColumnarToRow [codegen id : 7] -Input [3]: [rnk#11, best_performing#24, worst_performing#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* CometColumnarToRow (54) -+- CometHashAggregate (53) - +- CometExchange (52) - +- CometHashAggregate (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_addr_sk#26, ss_store_sk#27, ss_net_profit#28, ss_sold_date_sk#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [ss_addr_sk#26, ss_store_sk#27, ss_net_profit#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_store_sk#27) AND (ss_store_sk#27 = 4)) AND isnull(ss_addr_sk#26)) - -(50) CometProject -Input [4]: [ss_addr_sk#26, ss_store_sk#27, ss_net_profit#28, ss_sold_date_sk#29] -Arguments: [ss_store_sk#27, ss_net_profit#28], [ss_store_sk#27, ss_net_profit#28] - -(51) CometHashAggregate -Input [2]: [ss_store_sk#27, ss_net_profit#28] -Keys [1]: [ss_store_sk#27] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#28))] - -(52) CometExchange -Input [3]: [ss_store_sk#27, sum#30, count#31] -Arguments: hashpartitioning(ss_store_sk#27, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(53) CometHashAggregate -Input [3]: [ss_store_sk#27, sum#30, count#31] -Keys [1]: [ss_store_sk#27] -Functions [1]: [avg(UnscaledValue(ss_net_profit#28))] - -(54) CometColumnarToRow [codegen id : 1] -Input [1]: [rank_col#32] - -Subquery:2 Hosting operator id = 22 Hosting Expression = ReusedSubquery Subquery scalar-subquery#9, [id=#10] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/extended.txt deleted file mode 100644 index b784a161f7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- Filter - : : : +- Window - : : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : : +- CometColumnarToRow - : : : +- CometSort - : : : +- CometColumnarExchange - : : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : : +- CometColumnarToRow - : : : +- CometSort - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- Filter - : : +- Window - : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 45 out of 57 eligible operators (78%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index dba03cff02..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt +++ /dev/null @@ -1,72 +0,0 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [rnk,best_performing,worst_performing] - CometProject [i_product_name,i_product_name] [rnk,best_performing,worst_performing] - CometBroadcastHashJoin [rnk,item_sk,i_product_name,i_item_sk,i_product_name] - CometProject [rnk,item_sk,i_product_name] - CometBroadcastHashJoin [item_sk,rnk,item_sk,i_item_sk,i_product_name] - CometProject [item_sk,rnk,item_sk] - CometSortMergeJoin [item_sk,rnk,item_sk,rnk] - CometSort [item_sk,rnk] - CometColumnarExchange [rnk] #1 - WholeStageCodegen (3) - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WindowGroupLimit [rank_col] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometColumnarExchange #2 - WindowGroupLimit [rank_col] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometFilter [item_sk,rank_col] - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_store_sk] #4 - CometHashAggregate [ss_net_profit] [ss_store_sk,sum,count] - CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_item_sk] #3 - CometHashAggregate [ss_net_profit] [ss_item_sk,sum,count] - CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometSort [item_sk,rnk] - CometColumnarExchange [rnk] #5 - WholeStageCodegen (6) - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WindowGroupLimit [rank_col] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometColumnarExchange #6 - WindowGroupLimit [rank_col] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometFilter [item_sk,rank_col] - ReusedSubquery [rank_col] #1 - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - ReusedExchange [ss_item_sk,sum,count] #3 - CometBroadcastExchange [i_item_sk,i_product_name] #7 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_product_name] - 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/explain.txt deleted file mode 100644 index 3a1a49b654..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/explain.txt +++ /dev/null @@ -1,256 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (39) -+- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * Filter (33) - +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (32) - :- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * CometColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (4) - : : : +- BroadcastExchange (14) - : : : +- * CometColumnarToRow (13) - : : : +- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (10) - : : +- ReusedExchange (17) - : +- BroadcastExchange (24) - : +- * CometColumnarToRow (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometNativeScan parquet spark_catalog.default.item (20) - +- BroadcastExchange (31) - +- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometNativeScan parquet spark_catalog.default.item (27) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#5), dynamicpruningexpression(ws_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] - -(3) Filter [codegen id : 6] -Input [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Condition : (isnotnull(ws_bill_customer_sk#3) AND isnotnull(ws_item_sk#2)) - -(4) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#7, c_current_addr_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Condition : (isnotnull(c_customer_sk#7) AND isnotnull(c_current_addr_sk#8)) - -(6) CometColumnarToRow [codegen id : 1] -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] - -(7) BroadcastExchange -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_bill_customer_sk#3] -Right keys [1]: [c_customer_sk#7] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 6] -Output [4]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] -Input [6]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5, c_customer_sk#7, c_current_addr_sk#8] - -(10) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Condition : isnotnull(ca_address_sk#9) - -(12) CometProject -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Arguments: [ca_address_sk#9, ca_city#10, ca_zip#12], [ca_address_sk#9, ca_city#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#11, 10)) AS ca_zip#12] - -(13) CometColumnarToRow [codegen id : 2] -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] - -(14) BroadcastExchange -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_current_addr_sk#8] -Right keys [1]: [ca_address_sk#9] -Join type: Inner -Join condition: None - -(16) Project [codegen id : 6] -Output [5]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12] -Input [7]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8, ca_address_sk#9, ca_city#10, ca_zip#12] - -(17) ReusedExchange [Reuses operator id: 44] -Output [1]: [d_date_sk#13] - -(18) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#5] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 6] -Output [4]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12] -Input [6]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12, d_date_sk#13] - -(20) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#14, i_item_id#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [i_item_sk#14, i_item_id#15] -Condition : isnotnull(i_item_sk#14) - -(22) CometProject -Input [2]: [i_item_sk#14, i_item_id#15] -Arguments: [i_item_sk#14, i_item_id#16], [i_item_sk#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#15, 16)) AS i_item_id#16] - -(23) CometColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#14, i_item_id#16] - -(24) BroadcastExchange -Input [2]: [i_item_sk#14, i_item_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#2] -Right keys [1]: [i_item_sk#14] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 6] -Output [4]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#16] -Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12, i_item_sk#14, i_item_id#16] - -(27) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#17, i_item_id#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_item_sk, [11,13,17,19,2,23,29,3,5,7])] -ReadSchema: struct - -(28) CometFilter -Input [2]: [i_item_sk#17, i_item_id#18] -Condition : i_item_sk#17 IN (2,3,5,7,11,13,17,19,23,29) - -(29) CometProject -Input [2]: [i_item_sk#17, i_item_id#18] -Arguments: [i_item_id#19], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#18, 16)) AS i_item_id#19] - -(30) CometColumnarToRow [codegen id : 5] -Input [1]: [i_item_id#19] - -(31) BroadcastExchange -Input [1]: [i_item_id#19] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [i_item_id#16] -Right keys [1]: [i_item_id#19] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(33) Filter [codegen id : 6] -Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#16, exists#1] -Condition : (substr(ca_zip#12, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) - -(34) Project [codegen id : 6] -Output [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] -Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#16, exists#1] - -(35) HashAggregate [codegen id : 6] -Input [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] -Keys [2]: [ca_zip#12, ca_city#10] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum#20] -Results [3]: [ca_zip#12, ca_city#10, sum#21] - -(36) CometColumnarExchange -Input [3]: [ca_zip#12, ca_city#10, sum#21] -Arguments: hashpartitioning(ca_zip#12, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(37) CometColumnarToRow [codegen id : 7] -Input [3]: [ca_zip#12, ca_city#10, sum#21] - -(38) HashAggregate [codegen id : 7] -Input [3]: [ca_zip#12, ca_city#10, sum#21] -Keys [2]: [ca_zip#12, ca_city#10] -Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#22] -Results [3]: [ca_zip#12, ca_city#10, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#22,17,2) AS sum(ws_sales_price)#23] - -(39) TakeOrderedAndProject -Input [3]: [ca_zip#12, ca_city#10, sum(ws_sales_price)#23] -Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip#12, ca_city#10, sum(ws_sales_price)#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometNativeScan parquet spark_catalog.default.date_dim (40) - - -(40) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#24, d_qoy#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(41) CometFilter -Input [3]: [d_date_sk#13, d_year#24, d_qoy#25] -Condition : ((((isnotnull(d_qoy#25) AND isnotnull(d_year#24)) AND (d_qoy#25 = 2)) AND (d_year#24 = 2001)) AND isnotnull(d_date_sk#13)) - -(42) CometProject -Input [3]: [d_date_sk#13, d_year#24, d_qoy#25] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(43) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(44) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/simplified.txt deleted file mode 100644 index 81ecde84b2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/simplified.txt +++ /dev/null @@ -1,64 +0,0 @@ -TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - WholeStageCodegen (7) - HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_zip,ca_city] #1 - WholeStageCodegen (6) - HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] - Project [ws_sales_price,ca_city,ca_zip] - Filter [ca_zip,exists] - BroadcastHashJoin [i_item_id,i_item_id] - Project [ws_sales_price,ca_city,ca_zip,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_sales_price,ca_city,ca_zip] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Filter [ws_bill_customer_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_zip] [ca_address_sk,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_city,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan 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/q45.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/explain.txt deleted file mode 100644 index cf57f11d1e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,259 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (39) -+- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * Filter (33) - +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (32) - :- * CometColumnarToRow (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (7) - : : : : +- CometBroadcastHashJoin (6) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometBroadcastExchange (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- BroadcastExchange (31) - +- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (27) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#5), dynamicpruningexpression(ws_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Condition : (isnotnull(ws_bill_customer_sk#3) AND isnotnull(ws_item_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#7, c_current_addr_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Condition : (isnotnull(c_customer_sk#7) AND isnotnull(c_current_addr_sk#8)) - -(5) CometBroadcastExchange -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Arguments: [c_customer_sk#7, c_current_addr_sk#8] - -(6) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Right output [2]: [c_customer_sk#7, c_current_addr_sk#8] -Arguments: [ws_bill_customer_sk#3], [c_customer_sk#7], Inner, BuildRight - -(7) CometProject -Input [6]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5, c_customer_sk#7, c_current_addr_sk#8] -Arguments: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8], [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Condition : isnotnull(ca_address_sk#9) - -(10) CometProject -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Arguments: [ca_address_sk#9, ca_city#10, ca_zip#12], [ca_address_sk#9, ca_city#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#11, 10)) AS ca_zip#12] - -(11) CometBroadcastExchange -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] -Arguments: [ca_address_sk#9, ca_city#10, ca_zip#12] - -(12) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] -Right output [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] -Arguments: [c_current_addr_sk#8], [ca_address_sk#9], Inner, BuildRight - -(13) CometProject -Input [7]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8, ca_address_sk#9, ca_city#10, ca_zip#12] -Arguments: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12], [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Condition : ((((isnotnull(d_qoy#15) AND isnotnull(d_year#14)) AND (d_qoy#15 = 2)) AND (d_year#14 = 2001)) AND isnotnull(d_date_sk#13)) - -(16) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(18) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight - -(19) CometProject -Input [6]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12, d_date_sk#13] -Arguments: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12], [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [i_item_sk#16, i_item_id#17] -Condition : isnotnull(i_item_sk#16) - -(22) CometProject -Input [2]: [i_item_sk#16, i_item_id#17] -Arguments: [i_item_sk#16, i_item_id#18], [i_item_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#17, 16)) AS i_item_id#18] - -(23) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_id#18] -Arguments: [i_item_sk#16, i_item_id#18] - -(24) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12] -Right output [2]: [i_item_sk#16, i_item_id#18] -Arguments: [ws_item_sk#2], [i_item_sk#16], Inner, BuildRight - -(25) CometProject -Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12, i_item_sk#16, i_item_id#18] -Arguments: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18], [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] - -(26) CometColumnarToRow [codegen id : 2] -Input [4]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_item_id#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_item_sk, [11,13,17,19,2,23,29,3,5,7])] -ReadSchema: struct - -(28) CometFilter -Input [2]: [i_item_sk#19, i_item_id#20] -Condition : i_item_sk#19 IN (2,3,5,7,11,13,17,19,23,29) - -(29) CometProject -Input [2]: [i_item_sk#19, i_item_id#20] -Arguments: [i_item_id#21], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#20, 16)) AS i_item_id#21] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [i_item_id#21] - -(31) BroadcastExchange -Input [1]: [i_item_id#21] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=1] - -(32) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [i_item_id#18] -Right keys [1]: [i_item_id#21] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(33) Filter [codegen id : 2] -Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] -Condition : (substr(ca_zip#12, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) - -(34) Project [codegen id : 2] -Output [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] -Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] - -(35) HashAggregate [codegen id : 2] -Input [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] -Keys [2]: [ca_zip#12, ca_city#10] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [ca_zip#12, ca_city#10, sum#23] - -(36) CometColumnarExchange -Input [3]: [ca_zip#12, ca_city#10, sum#23] -Arguments: hashpartitioning(ca_zip#12, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(37) CometColumnarToRow [codegen id : 3] -Input [3]: [ca_zip#12, ca_city#10, sum#23] - -(38) HashAggregate [codegen id : 3] -Input [3]: [ca_zip#12, ca_city#10, sum#23] -Keys [2]: [ca_zip#12, ca_city#10] -Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#24] -Results [3]: [ca_zip#12, ca_city#10, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#24,17,2) AS sum(ws_sales_price)#25] - -(39) TakeOrderedAndProject -Input [3]: [ca_zip#12, ca_city#10, sum(ws_sales_price)#25] -Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip#12, ca_city#10, sum(ws_sales_price)#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) - - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(41) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Condition : ((((isnotnull(d_qoy#15) AND isnotnull(d_year#14)) AND (d_qoy#15 = 2)) AND (d_year#14 = 2001)) AND isnotnull(d_date_sk#13)) - -(42) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(43) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(44) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/simplified.txt deleted file mode 100644 index d78fc716ab..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - WholeStageCodegen (3) - HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_zip,ca_city] #1 - WholeStageCodegen (2) - HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] - Project [ws_sales_price,ca_city,ca_zip] - Filter [ca_zip,exists] - BroadcastHashJoin [i_item_id,i_item_id] - CometColumnarToRow - InputAdapter - CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] - 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_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 [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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 - CometProject [ca_zip] [ca_address_sk,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_city,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] 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/q45/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/explain.txt deleted file mode 100644 index cf57f11d1e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/explain.txt +++ /dev/null @@ -1,259 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (39) -+- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * Filter (33) - +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (32) - :- * CometColumnarToRow (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (7) - : : : : +- CometBroadcastHashJoin (6) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometBroadcastExchange (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- BroadcastExchange (31) - +- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (27) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#5), dynamicpruningexpression(ws_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Condition : (isnotnull(ws_bill_customer_sk#3) AND isnotnull(ws_item_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#7, c_current_addr_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Condition : (isnotnull(c_customer_sk#7) AND isnotnull(c_current_addr_sk#8)) - -(5) CometBroadcastExchange -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Arguments: [c_customer_sk#7, c_current_addr_sk#8] - -(6) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Right output [2]: [c_customer_sk#7, c_current_addr_sk#8] -Arguments: [ws_bill_customer_sk#3], [c_customer_sk#7], Inner, BuildRight - -(7) CometProject -Input [6]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5, c_customer_sk#7, c_current_addr_sk#8] -Arguments: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8], [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Condition : isnotnull(ca_address_sk#9) - -(10) CometProject -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Arguments: [ca_address_sk#9, ca_city#10, ca_zip#12], [ca_address_sk#9, ca_city#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#11, 10)) AS ca_zip#12] - -(11) CometBroadcastExchange -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] -Arguments: [ca_address_sk#9, ca_city#10, ca_zip#12] - -(12) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] -Right output [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] -Arguments: [c_current_addr_sk#8], [ca_address_sk#9], Inner, BuildRight - -(13) CometProject -Input [7]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8, ca_address_sk#9, ca_city#10, ca_zip#12] -Arguments: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12], [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Condition : ((((isnotnull(d_qoy#15) AND isnotnull(d_year#14)) AND (d_qoy#15 = 2)) AND (d_year#14 = 2001)) AND isnotnull(d_date_sk#13)) - -(16) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(18) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight - -(19) CometProject -Input [6]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12, d_date_sk#13] -Arguments: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12], [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [i_item_sk#16, i_item_id#17] -Condition : isnotnull(i_item_sk#16) - -(22) CometProject -Input [2]: [i_item_sk#16, i_item_id#17] -Arguments: [i_item_sk#16, i_item_id#18], [i_item_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#17, 16)) AS i_item_id#18] - -(23) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_id#18] -Arguments: [i_item_sk#16, i_item_id#18] - -(24) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12] -Right output [2]: [i_item_sk#16, i_item_id#18] -Arguments: [ws_item_sk#2], [i_item_sk#16], Inner, BuildRight - -(25) CometProject -Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12, i_item_sk#16, i_item_id#18] -Arguments: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18], [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] - -(26) CometColumnarToRow [codegen id : 2] -Input [4]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_item_id#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_item_sk, [11,13,17,19,2,23,29,3,5,7])] -ReadSchema: struct - -(28) CometFilter -Input [2]: [i_item_sk#19, i_item_id#20] -Condition : i_item_sk#19 IN (2,3,5,7,11,13,17,19,23,29) - -(29) CometProject -Input [2]: [i_item_sk#19, i_item_id#20] -Arguments: [i_item_id#21], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#20, 16)) AS i_item_id#21] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [i_item_id#21] - -(31) BroadcastExchange -Input [1]: [i_item_id#21] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=1] - -(32) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [i_item_id#18] -Right keys [1]: [i_item_id#21] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(33) Filter [codegen id : 2] -Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] -Condition : (substr(ca_zip#12, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) - -(34) Project [codegen id : 2] -Output [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] -Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] - -(35) HashAggregate [codegen id : 2] -Input [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] -Keys [2]: [ca_zip#12, ca_city#10] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [ca_zip#12, ca_city#10, sum#23] - -(36) CometColumnarExchange -Input [3]: [ca_zip#12, ca_city#10, sum#23] -Arguments: hashpartitioning(ca_zip#12, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(37) CometColumnarToRow [codegen id : 3] -Input [3]: [ca_zip#12, ca_city#10, sum#23] - -(38) HashAggregate [codegen id : 3] -Input [3]: [ca_zip#12, ca_city#10, sum#23] -Keys [2]: [ca_zip#12, ca_city#10] -Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#24] -Results [3]: [ca_zip#12, ca_city#10, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#24,17,2) AS sum(ws_sales_price)#25] - -(39) TakeOrderedAndProject -Input [3]: [ca_zip#12, ca_city#10, sum(ws_sales_price)#25] -Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip#12, ca_city#10, sum(ws_sales_price)#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) - - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(41) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Condition : ((((isnotnull(d_qoy#15) AND isnotnull(d_year#14)) AND (d_qoy#15 = 2)) AND (d_year#14 = 2001)) AND isnotnull(d_date_sk#13)) - -(42) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(43) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(44) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/extended.txt deleted file mode 100644 index 4a57091056..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/extended.txt +++ /dev/null @@ -1,47 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- Filter - +- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 32 out of 41 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d78fc716ab..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - WholeStageCodegen (3) - HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_zip,ca_city] #1 - WholeStageCodegen (2) - HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] - Project [ws_sales_price,ca_city,ca_zip] - Filter [ca_zip,exists] - BroadcastHashJoin [i_item_id,i_item_id] - CometColumnarToRow - InputAdapter - CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] - 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_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 [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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 - CometProject [ca_zip] [ca_address_sk,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_city,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/explain.txt deleted file mode 100644 index 4630317250..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/explain.txt +++ /dev/null @@ -1,267 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (41) -+- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * HashAggregate (30) - : : +- * CometColumnarToRow (29) - : : +- CometColumnarExchange (28) - : : +- * HashAggregate (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Project (6) - : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : :- * Filter (3) - : : : : : : +- * ColumnarToRow (2) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : +- ReusedExchange (4) - : : : : +- BroadcastExchange (11) - : : : : +- * CometColumnarToRow (10) - : : : : +- CometProject (9) - : : : : +- CometFilter (8) - : : : : +- CometNativeScan parquet spark_catalog.default.store (7) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) - : : +- BroadcastExchange (24) - : : +- * CometColumnarToRow (23) - : : +- CometFilter (22) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (21) - : +- BroadcastExchange (35) - : +- * CometColumnarToRow (34) - : +- CometProject (33) - : +- CometFilter (32) - : +- CometNativeScan parquet spark_catalog.default.customer (31) - +- ReusedExchange (38) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] - -(3) Filter [codegen id : 5] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 46] -Output [1]: [d_date_sk#10] - -(5) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 5] -Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_city#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [s_store_sk#11, s_city#12] -Condition : (s_city#12 IN (Fairview,Midway) AND isnotnull(s_store_sk#11)) - -(9) CometProject -Input [2]: [s_store_sk#11, s_city#12] -Arguments: [s_store_sk#11], [s_store_sk#11] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#11] - -(11) BroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#11] - -(14) CometNativeScan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#13, hd_dep_count#14, hd_vehicle_count#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [hd_demo_sk#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : (((hd_dep_count#14 = 4) OR (hd_vehicle_count#15 = 3)) AND isnotnull(hd_demo_sk#13)) - -(16) CometProject -Input [3]: [hd_demo_sk#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#13], [hd_demo_sk#13] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#13] - -(18) BroadcastExchange -Input [1]: [hd_demo_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#13] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [5]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Input [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, hd_demo_sk#13] - -(21) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_city#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [ca_address_sk#16, ca_city#17] -Condition : (isnotnull(ca_address_sk#16) AND isnotnull(ca_city#17)) - -(23) CometColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#16, ca_city#17] - -(24) BroadcastExchange -Input [2]: [ca_address_sk#16, ca_city#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#16] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 5] -Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#17] -Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_address_sk#16, ca_city#17] - -(27) HashAggregate [codegen id : 5] -Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#17] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17] -Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum#18, sum#19] -Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] - -(28) CometColumnarExchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(29) CometColumnarToRow [codegen id : 8] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] - -(30) HashAggregate [codegen id : 8] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17] -Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#6))#22, sum(UnscaledValue(ss_net_profit#7))#23] -Results [5]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#17 AS bought_city#24, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#22,17,2) AS amt#25, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#23,17,2) AS profit#26] - -(31) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(32) CometFilter -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] -Condition : (isnotnull(c_customer_sk#27) AND isnotnull(c_current_addr_sk#28)) - -(33) CometProject -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] -Arguments: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32], [c_customer_sk#27, c_current_addr_sk#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#29, 20)) AS c_first_name#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#30, 30)) AS c_last_name#32] - -(34) CometColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] - -(35) BroadcastExchange -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(36) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#27] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 8] -Output [7]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#31, c_last_name#32] -Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#24, amt#25, profit#26, c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] - -(38) ReusedExchange [Reuses operator id: 24] -Output [2]: [ca_address_sk#33, ca_city#34] - -(39) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [c_current_addr_sk#28] -Right keys [1]: [ca_address_sk#33] -Join type: Inner -Join condition: NOT (ca_city#34 = bought_city#24) - -(40) Project [codegen id : 8] -Output [7]: [c_last_name#32, c_first_name#31, ca_city#34, bought_city#24, ss_ticket_number#5, amt#25, profit#26] -Input [9]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#31, c_last_name#32, ca_address_sk#33, ca_city#34] - -(41) TakeOrderedAndProject -Input [7]: [c_last_name#32, c_first_name#31, ca_city#34, bought_city#24, ss_ticket_number#5, amt#25, profit#26] -Arguments: 100, [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, ca_city#34 ASC NULLS FIRST, bought_city#24 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#32, c_first_name#31, ca_city#34, bought_city#24, ss_ticket_number#5, amt#25, profit#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometNativeScan parquet spark_catalog.default.date_dim (42) - - -(42) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#35, d_dow#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [3]: [d_date_sk#10, d_year#35, d_dow#36] -Condition : ((d_dow#36 IN (6,0) AND d_year#35 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(44) CometProject -Input [3]: [d_date_sk#10, d_year#35, d_dow#36] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(45) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(46) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/simplified.txt deleted file mode 100644 index b4b78689b1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/simplified.txt +++ /dev/null @@ -1,67 +0,0 @@ -TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - WholeStageCodegen (8) - Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] - Project [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - WholeStageCodegen (5) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_city] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - InputAdapter - ReusedExchange [ca_address_sk,ca_city] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/explain.txt deleted file mode 100644 index 08bba41c02..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,256 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (39) -+- CometTakeOrderedAndProject (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (34) - : +- CometBroadcastHashJoin (33) - : :- CometHashAggregate (28) - : : +- CometExchange (27) - : : +- CometHashAggregate (26) - : : +- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (20) - : : : +- CometBroadcastHashJoin (19) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometProject (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : : : +- CometBroadcastExchange (18) - : : : +- CometProject (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - : : +- CometBroadcastExchange (23) - : : +- CometFilter (22) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (21) - : +- CometBroadcastExchange (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (29) - +- ReusedExchange (35) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(5) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#13, s_city#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#13, s_city#14] -Condition : (s_city#14 IN (Fairview,Midway) AND isnotnull(s_store_sk#13)) - -(11) CometProject -Input [2]: [s_store_sk#13, s_city#14] -Arguments: [s_store_sk#13], [s_store_sk#13] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#13] -Arguments: [s_store_sk#13] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [1]: [s_store_sk#13] -Arguments: [ss_store_sk#4], [s_store_sk#13], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#13] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((hd_dep_count#16 = 4) OR (hd_vehicle_count#17 = 3)) AND isnotnull(hd_demo_sk#15)) - -(17) CometProject -Input [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] -Arguments: [hd_demo_sk#15], [hd_demo_sk#15] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#15] -Arguments: [hd_demo_sk#15] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [1]: [hd_demo_sk#15] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#15], Inner, BuildRight - -(20) CometProject -Input [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, hd_demo_sk#15] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_city#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [ca_address_sk#18, ca_city#19] -Condition : (isnotnull(ca_address_sk#18) AND isnotnull(ca_city#19)) - -(23) CometBroadcastExchange -Input [2]: [ca_address_sk#18, ca_city#19] -Arguments: [ca_address_sk#18, ca_city#19] - -(24) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [2]: [ca_address_sk#18, ca_city#19] -Arguments: [ss_addr_sk#3], [ca_address_sk#18], Inner, BuildRight - -(25) CometProject -Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_address_sk#18, ca_city#19] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19] - -(26) CometHashAggregate -Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19] -Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] - -(27) CometExchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#20, sum#21] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#20, sum#21] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19] -Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] -Condition : (isnotnull(c_customer_sk#22) AND isnotnull(c_current_addr_sk#23)) - -(31) CometProject -Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] -Arguments: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27], [c_customer_sk#22, c_current_addr_sk#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#24, 20)) AS c_first_name#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#25, 30)) AS c_last_name#27] - -(32) CometBroadcastExchange -Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Arguments: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] - -(33) CometBroadcastHashJoin -Left output [5]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, amt#29, profit#30] -Right output [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Arguments: [ss_customer_sk#1], [c_customer_sk#22], Inner, BuildRight - -(34) CometProject -Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, amt#29, profit#30, c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Arguments: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27], [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27] - -(35) ReusedExchange [Reuses operator id: 23] -Output [2]: [ca_address_sk#31, ca_city#32] - -(36) CometBroadcastHashJoin -Left output [7]: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Right output [2]: [ca_address_sk#31, ca_city#32] -Arguments: [c_current_addr_sk#23], [ca_address_sk#31], Inner, NOT (ca_city#32 = bought_city#28), BuildRight - -(37) CometProject -Input [9]: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27, ca_address_sk#31, ca_city#32] -Arguments: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30], [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] - -(38) CometTakeOrderedAndProject -Input [7]: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#27 ASC NULLS FIRST,c_first_name#26 ASC NULLS FIRST,ca_city#32 ASC NULLS FIRST,bought_city#28 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#27,c_first_name#26,ca_city#32,bought_city#28,ss_ticket_number#5,amt#29,profit#30]), [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30], 100, 0, [c_last_name#27 ASC NULLS FIRST, c_first_name#26 ASC NULLS FIRST, ca_city#32 ASC NULLS FIRST, bought_city#28 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] - -(39) CometColumnarToRow [codegen id : 1] -Input [7]: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) - - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(41) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(42) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(43) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(44) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/simplified.txt deleted file mode 100644 index e646cb620a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - CometBroadcastHashJoin [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] - CometProject [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometHashAggregate [ss_addr_sk,sum,sum] [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,ca_city,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] - CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - CometHashAggregate [ss_coupon_amt,ss_net_profit] [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] - CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - 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_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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [ca_address_sk,ca_city] #6 - CometFilter [ca_address_sk,ca_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - ReusedExchange [ca_address_sk,ca_city] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt deleted file mode 100644 index 08bba41c02..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt +++ /dev/null @@ -1,256 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (39) -+- CometTakeOrderedAndProject (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (34) - : +- CometBroadcastHashJoin (33) - : :- CometHashAggregate (28) - : : +- CometExchange (27) - : : +- CometHashAggregate (26) - : : +- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (20) - : : : +- CometBroadcastHashJoin (19) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometProject (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : : : +- CometBroadcastExchange (18) - : : : +- CometProject (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - : : +- CometBroadcastExchange (23) - : : +- CometFilter (22) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (21) - : +- CometBroadcastExchange (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (29) - +- ReusedExchange (35) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(5) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#13, s_city#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#13, s_city#14] -Condition : (s_city#14 IN (Fairview,Midway) AND isnotnull(s_store_sk#13)) - -(11) CometProject -Input [2]: [s_store_sk#13, s_city#14] -Arguments: [s_store_sk#13], [s_store_sk#13] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#13] -Arguments: [s_store_sk#13] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [1]: [s_store_sk#13] -Arguments: [ss_store_sk#4], [s_store_sk#13], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#13] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((hd_dep_count#16 = 4) OR (hd_vehicle_count#17 = 3)) AND isnotnull(hd_demo_sk#15)) - -(17) CometProject -Input [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] -Arguments: [hd_demo_sk#15], [hd_demo_sk#15] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#15] -Arguments: [hd_demo_sk#15] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [1]: [hd_demo_sk#15] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#15], Inner, BuildRight - -(20) CometProject -Input [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, hd_demo_sk#15] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_city#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [ca_address_sk#18, ca_city#19] -Condition : (isnotnull(ca_address_sk#18) AND isnotnull(ca_city#19)) - -(23) CometBroadcastExchange -Input [2]: [ca_address_sk#18, ca_city#19] -Arguments: [ca_address_sk#18, ca_city#19] - -(24) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [2]: [ca_address_sk#18, ca_city#19] -Arguments: [ss_addr_sk#3], [ca_address_sk#18], Inner, BuildRight - -(25) CometProject -Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_address_sk#18, ca_city#19] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19] - -(26) CometHashAggregate -Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19] -Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] - -(27) CometExchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#20, sum#21] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#20, sum#21] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19] -Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] -Condition : (isnotnull(c_customer_sk#22) AND isnotnull(c_current_addr_sk#23)) - -(31) CometProject -Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] -Arguments: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27], [c_customer_sk#22, c_current_addr_sk#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#24, 20)) AS c_first_name#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#25, 30)) AS c_last_name#27] - -(32) CometBroadcastExchange -Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Arguments: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] - -(33) CometBroadcastHashJoin -Left output [5]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, amt#29, profit#30] -Right output [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Arguments: [ss_customer_sk#1], [c_customer_sk#22], Inner, BuildRight - -(34) CometProject -Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, amt#29, profit#30, c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Arguments: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27], [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27] - -(35) ReusedExchange [Reuses operator id: 23] -Output [2]: [ca_address_sk#31, ca_city#32] - -(36) CometBroadcastHashJoin -Left output [7]: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Right output [2]: [ca_address_sk#31, ca_city#32] -Arguments: [c_current_addr_sk#23], [ca_address_sk#31], Inner, NOT (ca_city#32 = bought_city#28), BuildRight - -(37) CometProject -Input [9]: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27, ca_address_sk#31, ca_city#32] -Arguments: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30], [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] - -(38) CometTakeOrderedAndProject -Input [7]: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#27 ASC NULLS FIRST,c_first_name#26 ASC NULLS FIRST,ca_city#32 ASC NULLS FIRST,bought_city#28 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#27,c_first_name#26,ca_city#32,bought_city#28,ss_ticket_number#5,amt#29,profit#30]), [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30], 100, 0, [c_last_name#27 ASC NULLS FIRST, c_first_name#26 ASC NULLS FIRST, ca_city#32 ASC NULLS FIRST, bought_city#28 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] - -(39) CometColumnarToRow [codegen id : 1] -Input [7]: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) - - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(41) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(42) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(43) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(44) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/extended.txt deleted file mode 100644 index c842000e11..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/extended.txt +++ /dev/null @@ -1,49 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index e646cb620a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - CometBroadcastHashJoin [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] - CometProject [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometHashAggregate [ss_addr_sk,sum,sum] [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,ca_city,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] - CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - CometHashAggregate [ss_coupon_amt,ss_net_profit] [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] - CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - 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_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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [ca_address_sk,ca_city] #6 - CometFilter [ca_address_sk,ca_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - ReusedExchange [ca_address_sk,ca_city] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/explain.txt deleted file mode 100644 index 5956a3b4a6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/explain.txt +++ /dev/null @@ -1,304 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (51) -+- * Project (50) - +- * BroadcastHashJoin Inner BuildRight (49) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * CometColumnarToRow (26) - : : +- CometSort (25) - : : +- CometColumnarExchange (24) - : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- * HashAggregate (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.store (14) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- Window (38) - : +- * CometColumnarToRow (37) - : +- CometSort (36) - : +- CometColumnarExchange (35) - : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) - : +- ReusedExchange (32) - +- BroadcastExchange (48) - +- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- ReusedExchange (43) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)))) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) AS i_brand#4, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)) AS i_category#5] - -(4) CometColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#4, i_category#5] - -(5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(7) Filter [codegen id : 1] -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) - -(8) BroadcastExchange -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#6] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(11) ReusedExchange [Reuses operator id: 55] -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] -Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) - -(16) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] - -(17) BroadcastExchange -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#7] -Right keys [1]: [s_store_sk#14] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] - -(20) HashAggregate [codegen id : 4] -Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] -Aggregate Attributes [1]: [sum#17] -Results [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] - -(21) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] - -(23) HashAggregate [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#8))#19] -Results [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#19,17,2) AS sum_sales#20, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#19,17,2) AS _w0#21] - -(24) CometColumnarExchange -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) CometColumnarToRow [codegen id : 6] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] - -(27) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(28) Filter [codegen id : 7] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(29) Window -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] -Arguments: [avg(_w0#21) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(30) Filter [codegen id : 22] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] -Condition : ((isnotnull(avg_monthly_sales#23) AND (avg_monthly_sales#23 > 0.000000)) AND CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END) - -(31) Project [codegen id : 22] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] - -(32) ReusedExchange [Reuses operator id: 21] -Output [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum#30] - -(33) CometColumnarToRow [codegen id : 12] -Input [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum#30] - -(34) HashAggregate [codegen id : 12] -Input [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum#30] -Keys [6]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29] -Functions [1]: [sum(UnscaledValue(ss_sales_price#31))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#31))#19] -Results [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, MakeDecimal(sum(UnscaledValue(ss_sales_price#31))#19,17,2) AS sum_sales#32] - -(35) CometColumnarExchange -Input [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum_sales#32] -Arguments: hashpartitioning(i_category#24, i_brand#25, s_store_name#26, s_company_name#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(36) CometSort -Input [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum_sales#32] -Arguments: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum_sales#32], [i_category#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, s_store_name#26 ASC NULLS FIRST, s_company_name#27 ASC NULLS FIRST, d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST] - -(37) CometColumnarToRow [codegen id : 13] -Input [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum_sales#32] - -(38) Window -Input [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum_sales#32] -Arguments: [rank(d_year#28, d_moy#29) windowspecdefinition(i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#24, i_brand#25, s_store_name#26, s_company_name#27], [d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST] - -(39) Project [codegen id : 14] -Output [6]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, sum_sales#32, rn#33] -Input [8]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum_sales#32, rn#33] - -(40) BroadcastExchange -Input [6]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, sum_sales#32, rn#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] - -(41) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] -Right keys [5]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, (rn#33 + 1)] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 22] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#32] -Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, i_category#24, i_brand#25, s_store_name#26, s_company_name#27, sum_sales#32, rn#33] - -(43) ReusedExchange [Reuses operator id: 35] -Output [7]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38, d_moy#39, sum_sales#40] - -(44) CometSort -Input [7]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38, d_moy#39, sum_sales#40] -Arguments: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38, d_moy#39, sum_sales#40], [i_category#34 ASC NULLS FIRST, i_brand#35 ASC NULLS FIRST, s_store_name#36 ASC NULLS FIRST, s_company_name#37 ASC NULLS FIRST, d_year#38 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST] - -(45) CometColumnarToRow [codegen id : 20] -Input [7]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38, d_moy#39, sum_sales#40] - -(46) Window -Input [7]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38, d_moy#39, sum_sales#40] -Arguments: [rank(d_year#38, d_moy#39) windowspecdefinition(i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#34, i_brand#35, s_store_name#36, s_company_name#37], [d_year#38 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST] - -(47) Project [codegen id : 21] -Output [6]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, sum_sales#40, rn#41] -Input [8]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38, d_moy#39, sum_sales#40, rn#41] - -(48) BroadcastExchange -Input [6]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, sum_sales#40, rn#41] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] - -(49) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] -Right keys [5]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, (rn#41 - 1)] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 22] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, sum_sales#32 AS psum#42, sum_sales#40 AS nsum#43] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#32, i_category#34, i_brand#35, s_store_name#36, s_company_name#37, sum_sales#40, rn#41] - -(51) TakeOrderedAndProject -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] -Arguments: 100, [(sum_sales#20 - avg_monthly_sales#23) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometFilter (53) - +- CometNativeScan parquet spark_catalog.default.date_dim (52) - - -(52) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(54) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(55) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/simplified.txt deleted file mode 100644 index 017805e5b2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/simplified.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (22) - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (7) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/explain.txt deleted file mode 100644 index 5b6a94ab57..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)))) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) AS i_brand#4, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_item_sk#1], [ss_item_sk#6], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) - -(16) CometBroadcastExchange -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [s_store_sk#14, s_store_name#15, s_company_name#16] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] -Right output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [ss_store_sk#7], [s_store_sk#14], Inner, BuildRight - -(18) CometProject -Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16], [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] - -(19) CometHashAggregate -Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] - -(20) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] - -(22) CometExchange -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] - -(25) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(28) Filter [codegen id : 7] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] - -(30) ReusedExchange [Reuses operator id: 20] -Output [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] - -(31) CometHashAggregate -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] -Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] - -(32) CometExchange -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] -Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] -Arguments: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30], [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] - -(35) Window -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] -Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#31], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] -Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30, rn#31] - -(37) BroadcastExchange -Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#31 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30] -Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] - -(40) ReusedExchange [Reuses operator id: 32] -Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] - -(41) CometSort -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] -Arguments: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38], [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] - -(43) Window -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] -Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] -Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38, rn#39] - -(45) BroadcastExchange -Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#39 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#30 AS psum#40, sum_sales#38 AS nsum#41] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] - -(48) TakeOrderedAndProject -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/simplified.txt deleted file mode 100644 index 19124a9e20..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - 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 [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,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt deleted file mode 100644 index 5b6a94ab57..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)))) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) AS i_brand#4, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_item_sk#1], [ss_item_sk#6], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) - -(16) CometBroadcastExchange -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [s_store_sk#14, s_store_name#15, s_company_name#16] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] -Right output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [ss_store_sk#7], [s_store_sk#14], Inner, BuildRight - -(18) CometProject -Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16], [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] - -(19) CometHashAggregate -Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] - -(20) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] - -(22) CometExchange -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] - -(25) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(28) Filter [codegen id : 7] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] - -(30) ReusedExchange [Reuses operator id: 20] -Output [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] - -(31) CometHashAggregate -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] -Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] - -(32) CometExchange -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] -Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] -Arguments: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30], [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] - -(35) Window -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] -Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#31], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] -Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30, rn#31] - -(37) BroadcastExchange -Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#31 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30] -Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] - -(40) ReusedExchange [Reuses operator id: 32] -Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] - -(41) CometSort -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] -Arguments: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38], [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] - -(43) Window -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] -Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] -Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38, rn#39] - -(45) BroadcastExchange -Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#39 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#30 AS psum#40, sum_sales#38 AS nsum#41] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] - -(48) TakeOrderedAndProject -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/extended.txt deleted file mode 100644 index b50b570b4b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 19124a9e20..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - 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 [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,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/explain.txt deleted file mode 100644 index 3f674fa5c8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/explain.txt +++ /dev/null @@ -1,207 +0,0 @@ -== Physical Plan == -* HashAggregate (30) -+- * CometColumnarToRow (29) - +- CometColumnarExchange (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * CometColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.store (4) - : : +- BroadcastExchange (14) - : : +- * CometColumnarToRow (13) - : : +- CometProject (12) - : : +- CometFilter (11) - : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (10) - : +- BroadcastExchange (21) - : +- * CometColumnarToRow (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometNativeScan parquet spark_catalog.default.customer_address (17) - +- ReusedExchange (24) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_addr_sk), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ss_net_profit,0.00),LessThanOrEqual(ss_net_profit,2000.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,3000.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,25000.00)))] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] - -(3) Filter [codegen id : 5] -Input [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((((isnotnull(ss_store_sk#3) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_addr_sk#2)) AND ((((ss_sales_price#5 >= 100.00) AND (ss_sales_price#5 <= 150.00)) OR ((ss_sales_price#5 >= 50.00) AND (ss_sales_price#5 <= 100.00))) OR ((ss_sales_price#5 >= 150.00) AND (ss_sales_price#5 <= 200.00)))) AND ((((ss_net_profit#6 >= 0.00) AND (ss_net_profit#6 <= 2000.00)) OR ((ss_net_profit#6 >= 150.00) AND (ss_net_profit#6 <= 3000.00))) OR ((ss_net_profit#6 >= 50.00) AND (ss_net_profit#6 <= 25000.00)))) - -(4) CometNativeScan parquet spark_catalog.default.store -Output [1]: [s_store_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [1]: [s_store_sk#9] -Condition : isnotnull(s_store_sk#9) - -(6) CometColumnarToRow [codegen id : 1] -Input [1]: [s_store_sk#9] - -(7) BroadcastExchange -Input [1]: [s_store_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 5] -Output [6]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Input [8]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, s_store_sk#9] - -(10) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Condition : (isnotnull(cd_demo_sk#10) AND ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#11, 1)) = M) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#12, 20)) = 4 yr Degree )) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#11, 1)) = D) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#12, 20)) = 2 yr Degree ))) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#11, 1)) = S) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#12, 20)) = College )))) - -(12) CometProject -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Arguments: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14], [cd_demo_sk#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#11, 1)) AS cd_marital_status#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#12, 20)) AS cd_education_status#14] - -(13) CometColumnarToRow [codegen id : 2] -Input [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] - -(14) BroadcastExchange -Input [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#1] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: ((((((cd_marital_status#13 = M) AND (cd_education_status#14 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#13 = D) AND (cd_education_status#14 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#13 = S) AND (cd_education_status#14 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) - -(16) Project [codegen id : 5] -Output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] -Input [9]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] - -(17) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(18) CometFilter -Input [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Condition : (((isnotnull(ca_country#17) AND (ca_country#17 = United States)) AND isnotnull(ca_address_sk#15)) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#16, 2)) IN (CO,OH,TX) OR static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#16, 2)) IN (OR,MN,KY)) OR static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#16, 2)) IN (VA,CA,MS))) - -(19) CometProject -Input [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Arguments: [ca_address_sk#15, ca_state#18], [ca_address_sk#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#16, 2)) AS ca_state#18] - -(20) CometColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#15, ca_state#18] - -(21) BroadcastExchange -Input [2]: [ca_address_sk#15, ca_state#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(22) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#15] -Join type: Inner -Join condition: ((((ca_state#18 IN (CO,OH,TX) AND (ss_net_profit#6 >= 0.00)) AND (ss_net_profit#6 <= 2000.00)) OR ((ca_state#18 IN (OR,MN,KY) AND (ss_net_profit#6 >= 150.00)) AND (ss_net_profit#6 <= 3000.00))) OR ((ca_state#18 IN (VA,CA,MS) AND (ss_net_profit#6 >= 50.00)) AND (ss_net_profit#6 <= 25000.00))) - -(23) Project [codegen id : 5] -Output [2]: [ss_quantity#4, ss_sold_date_sk#7] -Input [6]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7, ca_address_sk#15, ca_state#18] - -(24) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#19] - -(25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 5] -Output [1]: [ss_quantity#4] -Input [3]: [ss_quantity#4, ss_sold_date_sk#7, d_date_sk#19] - -(27) HashAggregate [codegen id : 5] -Input [1]: [ss_quantity#4] -Keys: [] -Functions [1]: [partial_sum(ss_quantity#4)] -Aggregate Attributes [1]: [sum#20] -Results [1]: [sum#21] - -(28) CometColumnarExchange -Input [1]: [sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(29) CometColumnarToRow [codegen id : 6] -Input [1]: [sum#21] - -(30) HashAggregate [codegen id : 6] -Input [1]: [sum#21] -Keys: [] -Functions [1]: [sum(ss_quantity#4)] -Aggregate Attributes [1]: [sum(ss_quantity#4)#22] -Results [1]: [sum(ss_quantity#4)#22 AS sum(ss_quantity)#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometNativeScan parquet spark_catalog.default.date_dim (31) - - -(31) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_year#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#19, d_year#24] -Condition : ((isnotnull(d_year#24) AND (d_year#24 = 2001)) AND isnotnull(d_date_sk#19)) - -(33) CometProject -Input [2]: [d_date_sk#19, d_year#24] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(35) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/simplified.txt deleted file mode 100644 index 0cd7e900b0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (6) - HashAggregate [sum] [sum(ss_quantity),sum(ss_quantity),sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (5) - HashAggregate [ss_quantity] [sum,sum] - Project [ss_quantity] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sold_date_sk] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] - Project [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] - Project [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/explain.txt deleted file mode 100644 index e481658923..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,202 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometHashAggregate (28) - +- CometExchange (27) - +- CometHashAggregate (26) - +- CometProject (25) - +- CometBroadcastHashJoin (24) - :- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (8) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (14) - +- CometBroadcastExchange (23) - +- CometProject (22) - +- CometFilter (21) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (20) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_addr_sk), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ss_net_profit,0.00),LessThanOrEqual(ss_net_profit,2000.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,3000.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,25000.00)))] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((((isnotnull(ss_store_sk#3) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_addr_sk#2)) AND ((((ss_sales_price#5 >= 100.00) AND (ss_sales_price#5 <= 150.00)) OR ((ss_sales_price#5 >= 50.00) AND (ss_sales_price#5 <= 100.00))) OR ((ss_sales_price#5 >= 150.00) AND (ss_sales_price#5 <= 200.00)))) AND ((((ss_net_profit#6 >= 0.00) AND (ss_net_profit#6 <= 2000.00)) OR ((ss_net_profit#6 >= 150.00) AND (ss_net_profit#6 <= 3000.00))) OR ((ss_net_profit#6 >= 50.00) AND (ss_net_profit#6 <= 25000.00)))) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [s_store_sk#9] -Condition : isnotnull(s_store_sk#9) - -(5) CometBroadcastExchange -Input [1]: [s_store_sk#9] -Arguments: [s_store_sk#9] - -(6) CometBroadcastHashJoin -Left output [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [1]: [s_store_sk#9] -Arguments: [ss_store_sk#3], [s_store_sk#9], Inner, BuildRight - -(7) CometProject -Input [8]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, s_store_sk#9] -Arguments: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Condition : (isnotnull(cd_demo_sk#10) AND ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#11, 1)) = M) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#12, 20)) = 4 yr Degree )) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#11, 1)) = D) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#12, 20)) = 2 yr Degree ))) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#11, 1)) = S) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#12, 20)) = College )))) - -(10) CometProject -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Arguments: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14], [cd_demo_sk#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#11, 1)) AS cd_marital_status#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#12, 20)) AS cd_education_status#14] - -(11) CometBroadcastExchange -Input [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] -Arguments: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] - -(12) CometBroadcastHashJoin -Left output [6]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] -Arguments: [ss_cdemo_sk#1], [cd_demo_sk#10], Inner, ((((((cd_marital_status#13 = M) AND (cd_education_status#14 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#13 = D) AND (cd_education_status#14 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#13 = S) AND (cd_education_status#14 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))), BuildRight - -(13) CometProject -Input [9]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] -Arguments: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7], [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Condition : (((isnotnull(ca_country#17) AND (ca_country#17 = United States)) AND isnotnull(ca_address_sk#15)) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#16, 2)) IN (CO,OH,TX) OR static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#16, 2)) IN (OR,MN,KY)) OR static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#16, 2)) IN (VA,CA,MS))) - -(16) CometProject -Input [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Arguments: [ca_address_sk#15, ca_state#18], [ca_address_sk#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#16, 2)) AS ca_state#18] - -(17) CometBroadcastExchange -Input [2]: [ca_address_sk#15, ca_state#18] -Arguments: [ca_address_sk#15, ca_state#18] - -(18) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] -Right output [2]: [ca_address_sk#15, ca_state#18] -Arguments: [ss_addr_sk#2], [ca_address_sk#15], Inner, ((((ca_state#18 IN (CO,OH,TX) AND (ss_net_profit#6 >= 0.00)) AND (ss_net_profit#6 <= 2000.00)) OR ((ca_state#18 IN (OR,MN,KY) AND (ss_net_profit#6 >= 150.00)) AND (ss_net_profit#6 <= 3000.00))) OR ((ca_state#18 IN (VA,CA,MS) AND (ss_net_profit#6 >= 50.00)) AND (ss_net_profit#6 <= 25000.00))), BuildRight - -(19) CometProject -Input [6]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7, ca_address_sk#15, ca_state#18] -Arguments: [ss_quantity#4, ss_sold_date_sk#7], [ss_quantity#4, ss_sold_date_sk#7] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_year#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [d_date_sk#19, d_year#20] -Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(22) CometProject -Input [2]: [d_date_sk#19, d_year#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(23) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(24) CometBroadcastHashJoin -Left output [2]: [ss_quantity#4, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#7], [d_date_sk#19], Inner, BuildRight - -(25) CometProject -Input [3]: [ss_quantity#4, ss_sold_date_sk#7, d_date_sk#19] -Arguments: [ss_quantity#4], [ss_quantity#4] - -(26) CometHashAggregate -Input [1]: [ss_quantity#4] -Keys: [] -Functions [1]: [partial_sum(ss_quantity#4)] - -(27) CometExchange -Input [1]: [sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [1]: [sum#21] -Keys: [] -Functions [1]: [sum(ss_quantity#4)] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [sum(ss_quantity)#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_year#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [d_date_sk#19, d_year#20] -Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(32) CometProject -Input [2]: [d_date_sk#19, d_year#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(33) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(34) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/simplified.txt deleted file mode 100644 index e29e101c7e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,39 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [sum(ss_quantity),sum(ss_quantity)] - CometExchange #1 - CometHashAggregate [ss_quantity] [sum] - CometProject [ss_quantity] - CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] - CometProject [ss_quantity,ss_sold_date_sk] - 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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk] #3 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #4 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [ca_address_sk,ca_state] #5 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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/q48/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/explain.txt deleted file mode 100644 index e481658923..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/explain.txt +++ /dev/null @@ -1,202 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometHashAggregate (28) - +- CometExchange (27) - +- CometHashAggregate (26) - +- CometProject (25) - +- CometBroadcastHashJoin (24) - :- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (8) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (14) - +- CometBroadcastExchange (23) - +- CometProject (22) - +- CometFilter (21) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (20) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_addr_sk), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ss_net_profit,0.00),LessThanOrEqual(ss_net_profit,2000.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,3000.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,25000.00)))] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((((isnotnull(ss_store_sk#3) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_addr_sk#2)) AND ((((ss_sales_price#5 >= 100.00) AND (ss_sales_price#5 <= 150.00)) OR ((ss_sales_price#5 >= 50.00) AND (ss_sales_price#5 <= 100.00))) OR ((ss_sales_price#5 >= 150.00) AND (ss_sales_price#5 <= 200.00)))) AND ((((ss_net_profit#6 >= 0.00) AND (ss_net_profit#6 <= 2000.00)) OR ((ss_net_profit#6 >= 150.00) AND (ss_net_profit#6 <= 3000.00))) OR ((ss_net_profit#6 >= 50.00) AND (ss_net_profit#6 <= 25000.00)))) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [s_store_sk#9] -Condition : isnotnull(s_store_sk#9) - -(5) CometBroadcastExchange -Input [1]: [s_store_sk#9] -Arguments: [s_store_sk#9] - -(6) CometBroadcastHashJoin -Left output [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [1]: [s_store_sk#9] -Arguments: [ss_store_sk#3], [s_store_sk#9], Inner, BuildRight - -(7) CometProject -Input [8]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, s_store_sk#9] -Arguments: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Condition : (isnotnull(cd_demo_sk#10) AND ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#11, 1)) = M) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#12, 20)) = 4 yr Degree )) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#11, 1)) = D) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#12, 20)) = 2 yr Degree ))) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#11, 1)) = S) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#12, 20)) = College )))) - -(10) CometProject -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Arguments: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14], [cd_demo_sk#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#11, 1)) AS cd_marital_status#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#12, 20)) AS cd_education_status#14] - -(11) CometBroadcastExchange -Input [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] -Arguments: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] - -(12) CometBroadcastHashJoin -Left output [6]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] -Arguments: [ss_cdemo_sk#1], [cd_demo_sk#10], Inner, ((((((cd_marital_status#13 = M) AND (cd_education_status#14 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#13 = D) AND (cd_education_status#14 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#13 = S) AND (cd_education_status#14 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))), BuildRight - -(13) CometProject -Input [9]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] -Arguments: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7], [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Condition : (((isnotnull(ca_country#17) AND (ca_country#17 = United States)) AND isnotnull(ca_address_sk#15)) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#16, 2)) IN (CO,OH,TX) OR static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#16, 2)) IN (OR,MN,KY)) OR static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#16, 2)) IN (VA,CA,MS))) - -(16) CometProject -Input [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Arguments: [ca_address_sk#15, ca_state#18], [ca_address_sk#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#16, 2)) AS ca_state#18] - -(17) CometBroadcastExchange -Input [2]: [ca_address_sk#15, ca_state#18] -Arguments: [ca_address_sk#15, ca_state#18] - -(18) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] -Right output [2]: [ca_address_sk#15, ca_state#18] -Arguments: [ss_addr_sk#2], [ca_address_sk#15], Inner, ((((ca_state#18 IN (CO,OH,TX) AND (ss_net_profit#6 >= 0.00)) AND (ss_net_profit#6 <= 2000.00)) OR ((ca_state#18 IN (OR,MN,KY) AND (ss_net_profit#6 >= 150.00)) AND (ss_net_profit#6 <= 3000.00))) OR ((ca_state#18 IN (VA,CA,MS) AND (ss_net_profit#6 >= 50.00)) AND (ss_net_profit#6 <= 25000.00))), BuildRight - -(19) CometProject -Input [6]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7, ca_address_sk#15, ca_state#18] -Arguments: [ss_quantity#4, ss_sold_date_sk#7], [ss_quantity#4, ss_sold_date_sk#7] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_year#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [d_date_sk#19, d_year#20] -Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(22) CometProject -Input [2]: [d_date_sk#19, d_year#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(23) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(24) CometBroadcastHashJoin -Left output [2]: [ss_quantity#4, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#7], [d_date_sk#19], Inner, BuildRight - -(25) CometProject -Input [3]: [ss_quantity#4, ss_sold_date_sk#7, d_date_sk#19] -Arguments: [ss_quantity#4], [ss_quantity#4] - -(26) CometHashAggregate -Input [1]: [ss_quantity#4] -Keys: [] -Functions [1]: [partial_sum(ss_quantity#4)] - -(27) CometExchange -Input [1]: [sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [1]: [sum#21] -Keys: [] -Functions [1]: [sum(ss_quantity#4)] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [sum(ss_quantity)#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_year#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [d_date_sk#19, d_year#20] -Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(32) CometProject -Input [2]: [d_date_sk#19, d_year#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(33) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(34) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/extended.txt deleted file mode 100644 index 57fbc2d035..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index e29e101c7e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt +++ /dev/null @@ -1,39 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [sum(ss_quantity),sum(ss_quantity)] - CometExchange #1 - CometHashAggregate [ss_quantity] [sum] - CometProject [ss_quantity] - CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] - CometProject [ss_quantity,ss_sold_date_sk] - 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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk] #3 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #4 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [ca_address_sk,ca_state] #5 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/explain.txt deleted file mode 100644 index 0a43ebc390..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/explain.txt +++ /dev/null @@ -1,497 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (84) -+- CometTakeOrderedAndProject (83) - +- CometHashAggregate (82) - +- CometColumnarExchange (81) - +- * HashAggregate (80) - +- Union (79) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometColumnarExchange (19) - : +- * HashAggregate (18) - : +- * CometColumnarToRow (17) - : +- CometColumnarExchange (16) - : +- * HashAggregate (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildLeft (10) - : : :- BroadcastExchange (5) - : : : +- * Project (4) - : : : +- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : +- * CometColumnarToRow (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometNativeScan parquet spark_catalog.default.web_returns (6) - : +- ReusedExchange (12) - :- * Project (52) - : +- * Filter (51) - : +- Window (50) - : +- * Sort (49) - : +- Window (48) - : +- * CometColumnarToRow (47) - : +- CometSort (46) - : +- CometColumnarExchange (45) - : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- CometColumnarExchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildLeft (36) - : : :- BroadcastExchange (31) - : : : +- * Project (30) - : : : +- * Filter (29) - : : : +- * ColumnarToRow (28) - : : : +- Scan parquet spark_catalog.default.catalog_sales (27) - : : +- * CometColumnarToRow (35) - : : +- CometProject (34) - : : +- CometFilter (33) - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (32) - : +- ReusedExchange (38) - +- * Project (78) - +- * Filter (77) - +- Window (76) - +- * Sort (75) - +- Window (74) - +- * CometColumnarToRow (73) - +- CometSort (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * CometColumnarToRow (69) - +- CometColumnarExchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * BroadcastHashJoin Inner BuildRight (65) - :- * Project (63) - : +- * BroadcastHashJoin Inner BuildLeft (62) - : :- BroadcastExchange (57) - : : +- * Project (56) - : : +- * Filter (55) - : : +- * ColumnarToRow (54) - : : +- Scan parquet spark_catalog.default.store_sales (53) - : +- * CometColumnarToRow (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometNativeScan parquet spark_catalog.default.store_returns (58) - +- ReusedExchange (64) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] - -(3) Filter [codegen id : 1] -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND isnotnull(ws_quantity#3)) AND (ws_net_profit#5 > 1.00)) AND (ws_net_paid#4 > 0.00)) AND (ws_quantity#3 > 0)) AND isnotnull(ws_order_number#2)) AND isnotnull(ws_item_sk#1)) - -(4) Project [codegen id : 1] -Output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] - -(5) BroadcastExchange -Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=1] - -(6) CometNativeScan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) - -(8) CometProject -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(9) CometColumnarToRow -Input [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(10) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [ws_order_number#2, ws_item_sk#1] -Right keys [2]: [wr_order_number#9, wr_item_sk#8] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 3] -Output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(12) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#13] - -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#6] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 3] -Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] - -(15) HashAggregate [codegen id : 3] -Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Keys [1]: [ws_item_sk#1] -Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#14, sum#15, sum#16, isEmpty#17, sum#18, isEmpty#19] -Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] - -(16) CometColumnarExchange -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(17) CometColumnarToRow [codegen id : 4] -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] - -(18) HashAggregate [codegen id : 4] -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Keys [1]: [ws_item_sk#1] -Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] -Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] - -(19) CometColumnarExchange -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(20) CometSort -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [item#30, return_ratio#31, currency_ratio#32], [return_ratio#31 ASC NULLS FIRST] - -(21) CometColumnarToRow [codegen id : 5] -Input [3]: [item#30, return_ratio#31, currency_ratio#32] - -(22) Window -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] - -(23) Sort [codegen id : 6] -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 - -(24) Window -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] - -(25) Filter [codegen id : 7] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] -Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) - -(26) Project [codegen id : 7] -Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] - -(27) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(28) ColumnarToRow [codegen id : 8] -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] - -(29) Filter [codegen id : 8] -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) - -(30) Project [codegen id : 8] -Output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] - -(31) BroadcastExchange -Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=4] - -(32) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] -Condition : (((isnotnull(cr_return_amount#45) AND (cr_return_amount#45 > 10000.00)) AND isnotnull(cr_order_number#43)) AND isnotnull(cr_item_sk#42)) - -(34) CometProject -Input [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] -Arguments: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45], [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] - -(35) CometColumnarToRow -Input [4]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] - -(36) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [cs_order_number#37, cs_item_sk#36] -Right keys [2]: [cr_order_number#43, cr_item_sk#42] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 10] -Output [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45] -Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] - -(38) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#47] - -(39) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#41] -Right keys [1]: [d_date_sk#47] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 10] -Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] -Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45, d_date_sk#47] - -(41) HashAggregate [codegen id : 10] -Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] -Keys [1]: [cs_item_sk#36] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#44, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#48, sum#49, sum#50, isEmpty#51, sum#52, isEmpty#53] -Results [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] - -(42) CometColumnarExchange -Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(43) CometColumnarToRow [codegen id : 11] -Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] - -(44) HashAggregate [codegen id : 11] -Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Keys [1]: [cs_item_sk#36] -Functions [4]: [sum(coalesce(cr_return_quantity#44, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#44, 0))#60, sum(coalesce(cs_quantity#38, 0))#61, sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#62, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63] -Results [3]: [cs_item_sk#36 AS item#64, (cast(sum(coalesce(cr_return_quantity#44, 0))#60 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#61 as decimal(15,4))) AS return_ratio#65, (cast(sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#62 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63 as decimal(15,4))) AS currency_ratio#66] - -(45) CometColumnarExchange -Input [3]: [item#64, return_ratio#65, currency_ratio#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(46) CometSort -Input [3]: [item#64, return_ratio#65, currency_ratio#66] -Arguments: [item#64, return_ratio#65, currency_ratio#66], [return_ratio#65 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 12] -Input [3]: [item#64, return_ratio#65, currency_ratio#66] - -(48) Window -Input [3]: [item#64, return_ratio#65, currency_ratio#66] -Arguments: [rank(return_ratio#65) windowspecdefinition(return_ratio#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#67], [return_ratio#65 ASC NULLS FIRST] - -(49) Sort [codegen id : 13] -Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] -Arguments: [currency_ratio#66 ASC NULLS FIRST], false, 0 - -(50) Window -Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] -Arguments: [rank(currency_ratio#66) windowspecdefinition(currency_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#68], [currency_ratio#66 ASC NULLS FIRST] - -(51) Filter [codegen id : 14] -Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] -Condition : ((return_rank#67 <= 10) OR (currency_rank#68 <= 10)) - -(52) Project [codegen id : 14] -Output [5]: [catalog AS channel#69, item#64, return_ratio#65, return_rank#67, currency_rank#68] -Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] - -(53) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#75), dynamicpruningexpression(ss_sold_date_sk#75 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(54) ColumnarToRow [codegen id : 15] -Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] - -(55) Filter [codegen id : 15] -Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] -Condition : (((((((isnotnull(ss_net_profit#74) AND isnotnull(ss_net_paid#73)) AND isnotnull(ss_quantity#72)) AND (ss_net_profit#74 > 1.00)) AND (ss_net_paid#73 > 0.00)) AND (ss_quantity#72 > 0)) AND isnotnull(ss_ticket_number#71)) AND isnotnull(ss_item_sk#70)) - -(56) Project [codegen id : 15] -Output [5]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75] -Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] - -(57) BroadcastExchange -Input [5]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=7] - -(58) CometNativeScan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(59) CometFilter -Input [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] -Condition : (((isnotnull(sr_return_amt#79) AND (sr_return_amt#79 > 10000.00)) AND isnotnull(sr_ticket_number#77)) AND isnotnull(sr_item_sk#76)) - -(60) CometProject -Input [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] -Arguments: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79], [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] - -(61) CometColumnarToRow -Input [4]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] - -(62) BroadcastHashJoin [codegen id : 17] -Left keys [2]: [ss_ticket_number#71, ss_item_sk#70] -Right keys [2]: [sr_ticket_number#77, sr_item_sk#76] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 17] -Output [6]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79] -Input [9]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] - -(64) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#81] - -(65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#75] -Right keys [1]: [d_date_sk#81] -Join type: Inner -Join condition: None - -(66) Project [codegen id : 17] -Output [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] -Input [7]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79, d_date_sk#81] - -(67) HashAggregate [codegen id : 17] -Input [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] -Keys [1]: [ss_item_sk#70] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#78, 0)), partial_sum(coalesce(ss_quantity#72, 0)), partial_sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#82, sum#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Results [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] - -(68) CometColumnarExchange -Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] -Arguments: hashpartitioning(ss_item_sk#70, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(69) CometColumnarToRow [codegen id : 18] -Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] - -(70) HashAggregate [codegen id : 18] -Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] -Keys [1]: [ss_item_sk#70] -Functions [4]: [sum(coalesce(sr_return_quantity#78, 0)), sum(coalesce(ss_quantity#72, 0)), sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#78, 0))#94, sum(coalesce(ss_quantity#72, 0))#95, sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00))#96, sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97] -Results [3]: [ss_item_sk#70 AS item#98, (cast(sum(coalesce(sr_return_quantity#78, 0))#94 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#72, 0))#95 as decimal(15,4))) AS return_ratio#99, (cast(sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00))#96 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97 as decimal(15,4))) AS currency_ratio#100] - -(71) CometColumnarExchange -Input [3]: [item#98, return_ratio#99, currency_ratio#100] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(72) CometSort -Input [3]: [item#98, return_ratio#99, currency_ratio#100] -Arguments: [item#98, return_ratio#99, currency_ratio#100], [return_ratio#99 ASC NULLS FIRST] - -(73) CometColumnarToRow [codegen id : 19] -Input [3]: [item#98, return_ratio#99, currency_ratio#100] - -(74) Window -Input [3]: [item#98, return_ratio#99, currency_ratio#100] -Arguments: [rank(return_ratio#99) windowspecdefinition(return_ratio#99 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#101], [return_ratio#99 ASC NULLS FIRST] - -(75) Sort [codegen id : 20] -Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] -Arguments: [currency_ratio#100 ASC NULLS FIRST], false, 0 - -(76) Window -Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] -Arguments: [rank(currency_ratio#100) windowspecdefinition(currency_ratio#100 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#102], [currency_ratio#100 ASC NULLS FIRST] - -(77) Filter [codegen id : 21] -Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] -Condition : ((return_rank#101 <= 10) OR (currency_rank#102 <= 10)) - -(78) Project [codegen id : 21] -Output [5]: [store AS channel#103, item#98, return_ratio#99, return_rank#101, currency_rank#102] -Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] - -(79) Union - -(80) HashAggregate [codegen id : 22] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] - -(81) CometColumnarExchange -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(82) CometHashAggregate -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Functions: [] - -(83) CometTakeOrderedAndProject -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,return_rank#33 ASC NULLS FIRST,currency_rank#34 ASC NULLS FIRST], output=[channel#35,item#30,return_ratio#31,return_rank#33,currency_rank#34]), [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34], 100, 0, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] - -(84) CometColumnarToRow [codegen id : 23] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (89) -+- * CometColumnarToRow (88) - +- CometProject (87) - +- CometFilter (86) - +- CometNativeScan parquet spark_catalog.default.date_dim (85) - - -(85) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#104, d_moy#105] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(86) CometFilter -Input [3]: [d_date_sk#13, d_year#104, d_moy#105] -Condition : ((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND (d_year#104 = 2001)) AND (d_moy#105 = 12)) AND isnotnull(d_date_sk#13)) - -(87) CometProject -Input [3]: [d_date_sk#13, d_year#104, d_moy#105] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(88) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(89) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 - -Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#75 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/simplified.txt deleted file mode 100644 index 047269434d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/simplified.txt +++ /dev/null @@ -1,140 +0,0 @@ -WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (22) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (7) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (6) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #2 - WholeStageCodegen (4) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (14) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (13) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #6 - WholeStageCodegen (11) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_item_sk] #7 - WholeStageCodegen (10) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (21) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (20) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #9 - WholeStageCodegen (18) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk] #10 - WholeStageCodegen (17) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/explain.txt deleted file mode 100644 index 338f5f4201..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,462 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- * Project (49) - : +- * Filter (48) - : +- Window (47) - : +- * Sort (46) - : +- Window (45) - : +- * CometColumnarToRow (44) - : +- CometSort (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometBroadcastExchange (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometBroadcastHashJoin (57) - : :- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (59) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND isnotnull(ws_quantity#3)) AND (ws_net_profit#5 > 1.00)) AND (ws_net_paid#4 > 0.00)) AND (ws_quantity#3 > 0)) AND isnotnull(ws_order_number#2)) AND isnotnull(ws_item_sk#1)) - -(3) CometProject -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6], [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(4) CometBroadcastExchange -Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) - -(7) CometProject -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(8) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner, BuildLeft - -(9) CometProject -Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(12) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] - -(16) CometHashAggregate -Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Keys [1]: [ws_item_sk#1] -Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(17) CometExchange -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Keys [1]: [ws_item_sk#1] -Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(19) CometExchange -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] - -(21) CometColumnarToRow [codegen id : 1] -Input [3]: [item#22, return_ratio#23, currency_ratio#24] - -(22) Window -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] - -(23) Sort [codegen id : 2] -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 - -(24) Window -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] - -(25) Filter [codegen id : 3] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] -Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) - -(26) Project [codegen id : 3] -Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_sold_date_sk#33 IN dynamicpruning#34)] -PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(28) CometFilter -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) - -(29) CometProject -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(30) CometBroadcastExchange -Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(32) CometFilter -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) - -(33) CometProject -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] - -(34) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft - -(35) CometProject -Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] - -(36) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#40] - -(37) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -Right output [1]: [d_date_sk#40] -Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight - -(38) CometProject -Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] - -(39) CometHashAggregate -Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -Keys [1]: [cs_item_sk#28] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(40) CometExchange -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(41) CometHashAggregate -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Keys [1]: [cs_item_sk#28] -Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(42) CometExchange -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(43) CometSort -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] - -(44) CometColumnarToRow [codegen id : 4] -Input [3]: [item#47, return_ratio#48, currency_ratio#49] - -(45) Window -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] - -(46) Sort [codegen id : 5] -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 - -(47) Window -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] - -(48) Filter [codegen id : 6] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] -Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) - -(49) Project [codegen id : 6] -Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) - -(52) CometProject -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(53) CometBroadcastExchange -Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(55) CometFilter -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) - -(56) CometProject -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] - -(57) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft - -(58) CometProject -Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] - -(59) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#65] - -(60) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(61) CometProject -Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] - -(62) CometHashAggregate -Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -Keys [1]: [ss_item_sk#53] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(63) CometExchange -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(64) CometHashAggregate -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Keys [1]: [ss_item_sk#53] -Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(65) CometExchange -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(66) CometSort -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] - -(67) CometColumnarToRow [codegen id : 7] -Input [3]: [item#72, return_ratio#73, currency_ratio#74] - -(68) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] - -(69) Sort [codegen id : 8] -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 - -(70) Window -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] - -(71) Filter [codegen id : 9] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] -Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) - -(72) Project [codegen id : 9] -Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] - -(73) Union - -(74) HashAggregate [codegen id : 10] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(75) CometColumnarExchange -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometHashAggregate -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] - -(77) CometTakeOrderedAndProject -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(78) CometColumnarToRow [codegen id : 11] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) - - -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(80) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(81) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(82) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(83) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 - -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/simplified.txt deleted file mode 100644 index ca80833ee5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,110 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (10) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (3) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (2) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - 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_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_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - 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_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_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 (9) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (8) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - 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_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_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt deleted file mode 100644 index 338f5f4201..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt +++ /dev/null @@ -1,462 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- * Project (49) - : +- * Filter (48) - : +- Window (47) - : +- * Sort (46) - : +- Window (45) - : +- * CometColumnarToRow (44) - : +- CometSort (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometBroadcastExchange (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometBroadcastHashJoin (57) - : :- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (59) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND isnotnull(ws_quantity#3)) AND (ws_net_profit#5 > 1.00)) AND (ws_net_paid#4 > 0.00)) AND (ws_quantity#3 > 0)) AND isnotnull(ws_order_number#2)) AND isnotnull(ws_item_sk#1)) - -(3) CometProject -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6], [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(4) CometBroadcastExchange -Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) - -(7) CometProject -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(8) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner, BuildLeft - -(9) CometProject -Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(12) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] - -(16) CometHashAggregate -Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Keys [1]: [ws_item_sk#1] -Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(17) CometExchange -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Keys [1]: [ws_item_sk#1] -Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(19) CometExchange -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] - -(21) CometColumnarToRow [codegen id : 1] -Input [3]: [item#22, return_ratio#23, currency_ratio#24] - -(22) Window -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] - -(23) Sort [codegen id : 2] -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 - -(24) Window -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] - -(25) Filter [codegen id : 3] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] -Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) - -(26) Project [codegen id : 3] -Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_sold_date_sk#33 IN dynamicpruning#34)] -PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(28) CometFilter -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) - -(29) CometProject -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(30) CometBroadcastExchange -Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(32) CometFilter -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) - -(33) CometProject -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] - -(34) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft - -(35) CometProject -Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] - -(36) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#40] - -(37) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -Right output [1]: [d_date_sk#40] -Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight - -(38) CometProject -Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] - -(39) CometHashAggregate -Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -Keys [1]: [cs_item_sk#28] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(40) CometExchange -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(41) CometHashAggregate -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Keys [1]: [cs_item_sk#28] -Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(42) CometExchange -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(43) CometSort -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] - -(44) CometColumnarToRow [codegen id : 4] -Input [3]: [item#47, return_ratio#48, currency_ratio#49] - -(45) Window -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] - -(46) Sort [codegen id : 5] -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 - -(47) Window -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] - -(48) Filter [codegen id : 6] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] -Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) - -(49) Project [codegen id : 6] -Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) - -(52) CometProject -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(53) CometBroadcastExchange -Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(55) CometFilter -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) - -(56) CometProject -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] - -(57) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft - -(58) CometProject -Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] - -(59) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#65] - -(60) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(61) CometProject -Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] - -(62) CometHashAggregate -Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -Keys [1]: [ss_item_sk#53] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(63) CometExchange -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(64) CometHashAggregate -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Keys [1]: [ss_item_sk#53] -Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(65) CometExchange -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(66) CometSort -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] - -(67) CometColumnarToRow [codegen id : 7] -Input [3]: [item#72, return_ratio#73, currency_ratio#74] - -(68) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] - -(69) Sort [codegen id : 8] -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 - -(70) Window -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] - -(71) Filter [codegen id : 9] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] -Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) - -(72) Project [codegen id : 9] -Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] - -(73) Union - -(74) HashAggregate [codegen id : 10] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(75) CometColumnarExchange -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometHashAggregate -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] - -(77) CometTakeOrderedAndProject -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(78) CometColumnarToRow [codegen id : 11] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) - - -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(80) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(81) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(82) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(83) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 - -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/extended.txt deleted file mode 100644 index 75684a9669..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/extended.txt +++ /dev/null @@ -1,94 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index ca80833ee5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt +++ /dev/null @@ -1,110 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (10) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (3) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (2) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - 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_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_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - 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_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_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 (9) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (8) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - 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_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_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/explain.txt deleted file mode 100644 index 39114bd417..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/explain.txt +++ /dev/null @@ -1,499 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (81) -+- * HashAggregate (80) - +- * CometColumnarToRow (79) - +- CometColumnarExchange (78) - +- * HashAggregate (77) - +- * Expand (76) - +- Union (75) - :- * HashAggregate (23) - : +- * CometColumnarToRow (22) - : +- CometColumnarExchange (21) - : +- * HashAggregate (20) - : +- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- Union (9) - : : : :- * Project (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- * Project (8) - : : : +- * Filter (7) - : : : +- * ColumnarToRow (6) - : : : +- Scan parquet spark_catalog.default.store_returns (5) - : : +- ReusedExchange (10) - : +- BroadcastExchange (17) - : +- * CometColumnarToRow (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometNativeScan parquet spark_catalog.default.store (13) - :- * HashAggregate (46) - : +- * CometColumnarToRow (45) - : +- CometColumnarExchange (44) - : +- * HashAggregate (43) - : +- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- Union (32) - : : : :- * Project (27) - : : : : +- * Filter (26) - : : : : +- * ColumnarToRow (25) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (24) - : : : +- * Project (31) - : : : +- * Filter (30) - : : : +- * ColumnarToRow (29) - : : : +- Scan parquet spark_catalog.default.catalog_returns (28) - : : +- ReusedExchange (33) - : +- BroadcastExchange (40) - : +- * CometColumnarToRow (39) - : +- CometProject (38) - : +- CometFilter (37) - : +- CometNativeScan parquet spark_catalog.default.catalog_page (36) - +- * HashAggregate (74) - +- * CometColumnarToRow (73) - +- CometColumnarExchange (72) - +- * HashAggregate (71) - +- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * Project (63) - : +- * BroadcastHashJoin Inner BuildRight (62) - : :- Union (60) - : : :- * Project (50) - : : : +- * Filter (49) - : : : +- * ColumnarToRow (48) - : : : +- Scan parquet spark_catalog.default.web_sales (47) - : : +- * Project (59) - : : +- * BroadcastHashJoin Inner BuildLeft (58) - : : :- BroadcastExchange (53) - : : : +- * ColumnarToRow (52) - : : : +- Scan parquet spark_catalog.default.web_returns (51) - : : +- * CometColumnarToRow (57) - : : +- CometProject (56) - : : +- CometFilter (55) - : : +- CometNativeScan parquet spark_catalog.default.web_sales (54) - : +- ReusedExchange (61) - +- BroadcastExchange (68) - +- * CometColumnarToRow (67) - +- CometProject (66) - +- CometFilter (65) - +- CometNativeScan parquet spark_catalog.default.web_site (64) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 1] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(4) Project [codegen id : 1] -Output [6]: [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] - -(5) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 2] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] - -(7) Filter [codegen id : 2] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#12) - -(8) Project [codegen id : 2] -Output [6]: [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] - -(9) Union - -(10) ReusedExchange [Reuses operator id: 86] -Output [1]: [d_date_sk#22] - -(11) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [date_sk#7] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None - -(12) Project [codegen id : 5] -Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] - -(13) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#23, s_store_id#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [s_store_sk#23, s_store_id#24] -Condition : isnotnull(s_store_sk#23) - -(15) CometProject -Input [2]: [s_store_sk#23, s_store_id#24] -Arguments: [s_store_sk#23, s_store_id#25], [s_store_sk#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#24, 16)) AS s_store_id#25] - -(16) CometColumnarToRow [codegen id : 4] -Input [2]: [s_store_sk#23, s_store_id#25] - -(17) BroadcastExchange -Input [2]: [s_store_sk#23, s_store_id#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(18) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [store_sk#6] -Right keys [1]: [s_store_sk#23] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 5] -Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#25] - -(20) HashAggregate [codegen id : 5] -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -Keys [1]: [s_store_id#25] -Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum#26, sum#27, sum#28, sum#29] -Results [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] - -(21) CometColumnarExchange -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(22) CometColumnarToRow [codegen id : 6] -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] - -(23) HashAggregate [codegen id : 6] -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -Keys [1]: [s_store_id#25] -Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#34, sum(UnscaledValue(return_amt#10))#35, sum(UnscaledValue(profit#9))#36, sum(UnscaledValue(net_loss#11))#37] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#8))#34,17,2) AS sales#38, MakeDecimal(sum(UnscaledValue(return_amt#10))#35,17,2) AS returns#39, (MakeDecimal(sum(UnscaledValue(profit#9))#36,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#37,17,2)) AS profit#40, store channel AS channel#41, concat(store, s_store_id#25) AS id#42] - -(24) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_sold_date_sk#46 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cs_catalog_page_sk)] -ReadSchema: struct - -(25) ColumnarToRow [codegen id : 7] -Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] - -(26) Filter [codegen id : 7] -Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Condition : isnotnull(cs_catalog_page_sk#43) - -(27) Project [codegen id : 7] -Output [6]: [cs_catalog_page_sk#43 AS page_sk#47, cs_sold_date_sk#46 AS date_sk#48, cs_ext_sales_price#44 AS sales_price#49, cs_net_profit#45 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] -Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] - -(28) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cr_catalog_page_sk)] -ReadSchema: struct - -(29) ColumnarToRow [codegen id : 8] -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] - -(30) Filter [codegen id : 8] -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -Condition : isnotnull(cr_catalog_page_sk#53) - -(31) Project [codegen id : 8] -Output [6]: [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] - -(32) Union - -(33) ReusedExchange [Reuses operator id: 86] -Output [1]: [d_date_sk#63] - -(34) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [date_sk#48] -Right keys [1]: [d_date_sk#63] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 11] -Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] -Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] - -(36) CometNativeScan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Condition : isnotnull(cp_catalog_page_sk#64) - -(38) CometProject -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Arguments: [cp_catalog_page_sk#64, cp_catalog_page_id#66], [cp_catalog_page_sk#64, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#65, 16)) AS cp_catalog_page_id#66] - -(39) CometColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] - -(40) BroadcastExchange -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(41) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [page_sk#47] -Right keys [1]: [cp_catalog_page_sk#64] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 11] -Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] -Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#66] - -(43) HashAggregate [codegen id : 11] -Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] -Keys [1]: [cp_catalog_page_id#66] -Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] -Aggregate Attributes [4]: [sum#67, sum#68, sum#69, sum#70] -Results [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] - -(44) CometColumnarExchange -Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] -Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(45) CometColumnarToRow [codegen id : 12] -Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] - -(46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] -Keys [1]: [cp_catalog_page_id#66] -Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#75, sum(UnscaledValue(return_amt#51))#76, sum(UnscaledValue(profit#50))#77, sum(UnscaledValue(net_loss#52))#78] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#49))#75,17,2) AS sales#79, MakeDecimal(sum(UnscaledValue(return_amt#51))#76,17,2) AS returns#80, (MakeDecimal(sum(UnscaledValue(profit#50))#77,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#52))#78,17,2)) AS profit#81, catalog channel AS channel#82, concat(catalog_page, cp_catalog_page_id#66) AS id#83] - -(47) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_sold_date_sk#87 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] - -(49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] -Condition : isnotnull(ws_web_site_sk#84) - -(50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#84 AS wsr_web_site_sk#88, ws_sold_date_sk#87 AS date_sk#89, ws_ext_sales_price#85 AS sales_price#90, ws_net_profit#86 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] -Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] - -(51) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#5)] -ReadSchema: struct - -(52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] - -(53) BroadcastExchange -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [plan_id=5] - -(54) CometNativeScan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(55) CometFilter -Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) - -(56) CometProject -Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] - -(57) CometColumnarToRow -Input [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] - -(58) BroadcastHashJoin [codegen id : 15] -Left keys [2]: [wr_item_sk#94, wr_order_number#95] -Right keys [2]: [ws_item_sk#99, ws_order_number#101] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 15] -Output [6]: [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] -Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] - -(60) Union - -(61) ReusedExchange [Reuses operator id: 86] -Output [1]: [d_date_sk#109] - -(62) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [date_sk#89] -Right keys [1]: [d_date_sk#109] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 18] -Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] -Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] - -(64) CometNativeScan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#110, web_site_id#111] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(65) CometFilter -Input [2]: [web_site_sk#110, web_site_id#111] -Condition : isnotnull(web_site_sk#110) - -(66) CometProject -Input [2]: [web_site_sk#110, web_site_id#111] -Arguments: [web_site_sk#110, web_site_id#112], [web_site_sk#110, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#111, 16)) AS web_site_id#112] - -(67) CometColumnarToRow [codegen id : 17] -Input [2]: [web_site_sk#110, web_site_id#112] - -(68) BroadcastExchange -Input [2]: [web_site_sk#110, web_site_id#112] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(69) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [wsr_web_site_sk#88] -Right keys [1]: [web_site_sk#110] -Join type: Inner -Join condition: None - -(70) Project [codegen id : 18] -Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] -Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#112] - -(71) HashAggregate [codegen id : 18] -Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] -Keys [1]: [web_site_id#112] -Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum#113, sum#114, sum#115, sum#116] -Results [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] - -(72) CometColumnarExchange -Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] -Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(73) CometColumnarToRow [codegen id : 19] -Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] - -(74) HashAggregate [codegen id : 19] -Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] -Keys [1]: [web_site_id#112] -Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#121, sum(UnscaledValue(return_amt#92))#122, sum(UnscaledValue(profit#91))#123, sum(UnscaledValue(net_loss#93))#124] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#90))#121,17,2) AS sales#125, MakeDecimal(sum(UnscaledValue(return_amt#92))#122,17,2) AS returns#126, (MakeDecimal(sum(UnscaledValue(profit#91))#123,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#124,17,2)) AS profit#127, web channel AS channel#128, concat(web_site, web_site_id#112) AS id#129] - -(75) Union - -(76) Expand [codegen id : 20] -Input [5]: [sales#38, returns#39, profit#40, channel#41, id#42] -Arguments: [[sales#38, returns#39, profit#40, channel#41, id#42, 0], [sales#38, returns#39, profit#40, channel#41, null, 1], [sales#38, returns#39, profit#40, null, null, 3]], [sales#38, returns#39, profit#40, channel#130, id#131, spark_grouping_id#132] - -(77) HashAggregate [codegen id : 20] -Input [6]: [sales#38, returns#39, profit#40, channel#130, id#131, spark_grouping_id#132] -Keys [3]: [channel#130, id#131, spark_grouping_id#132] -Functions [3]: [partial_sum(sales#38), partial_sum(returns#39), partial_sum(profit#40)] -Aggregate Attributes [6]: [sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -Results [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] - -(78) CometColumnarExchange -Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] -Arguments: hashpartitioning(channel#130, id#131, spark_grouping_id#132, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(79) CometColumnarToRow [codegen id : 21] -Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] - -(80) HashAggregate [codegen id : 21] -Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] -Keys [3]: [channel#130, id#131, spark_grouping_id#132] -Functions [3]: [sum(sales#38), sum(returns#39), sum(profit#40)] -Aggregate Attributes [3]: [sum(sales#38)#145, sum(returns#39)#146, sum(profit#40)#147] -Results [5]: [channel#130, id#131, sum(sales#38)#145 AS sales#148, sum(returns#39)#146 AS returns#149, sum(profit#40)#147 AS profit#150] - -(81) TakeOrderedAndProject -Input [5]: [channel#130, id#131, sales#148, returns#149, profit#150] -Arguments: 100, [channel#130 ASC NULLS FIRST, id#131 ASC NULLS FIRST], [channel#130, id#131, sales#148, returns#149, profit#150] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (86) -+- * CometColumnarToRow (85) - +- CometProject (84) - +- CometFilter (83) - +- CometNativeScan parquet spark_catalog.default.date_dim (82) - - -(82) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#151] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] -ReadSchema: struct - -(83) CometFilter -Input [2]: [d_date_sk#22, d_date#151] -Condition : (((isnotnull(d_date#151) AND (d_date#151 >= 2000-08-23)) AND (d_date#151 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) - -(84) CometProject -Input [2]: [d_date_sk#22, d_date#151] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(85) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(86) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 28 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#87 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/simplified.txt deleted file mode 100644 index 9dc4fc19c7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/simplified.txt +++ /dev/null @@ -1,137 +0,0 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (21) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (20) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [s_store_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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_id] #2 - WholeStageCodegen (5) - HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,s_store_id] - BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (1) - Project [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - WholeStageCodegen (2) - Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] - Filter [sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (12) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cp_catalog_page_id] #5 - WholeStageCodegen (11) - HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (7) - Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] - Filter [cs_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (8) - Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] - Filter [cr_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (19) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [web_site_id] #7 - WholeStageCodegen (18) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (13) - Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] - Filter [ws_web_site_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (15) - Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (14) - ColumnarToRow - InputAdapter - Scan 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 - CometColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (17) - CometColumnarToRow - InputAdapter - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan 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/q5.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/explain.txt deleted file mode 100644 index 6f23a02597..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,452 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (71) -+- CometTakeOrderedAndProject (70) - +- CometHashAggregate (69) - +- CometExchange (68) - +- CometHashAggregate (67) - +- CometExpand (66) - +- CometUnion (65) - :- CometHashAggregate (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometUnion (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - :- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (32) - : : +- CometBroadcastHashJoin (31) - : : :- CometUnion (29) - : : : :- CometProject (25) - : : : : +- CometFilter (24) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) - : : : +- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) - : : +- ReusedExchange (30) - : +- CometBroadcastExchange (36) - : +- CometProject (35) - : +- CometFilter (34) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (55) - : +- CometBroadcastHashJoin (54) - : :- CometUnion (52) - : : :- CometProject (44) - : : : +- CometFilter (43) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : : +- CometProject (51) - : : +- CometBroadcastHashJoin (50) - : : :- CometBroadcastExchange (46) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) - : : +- CometProject (49) - : : +- CometFilter (48) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : +- ReusedExchange (53) - +- CometBroadcastExchange (59) - +- CometProject (58) - +- CometFilter (57) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Arguments: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11], [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#12) - -(6) CometProject -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Arguments: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21], [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] - -(7) CometUnion -Child 0 Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) - -(10) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(11) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(12) CometBroadcastHashJoin -Left output [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [1]: [d_date_sk#22] -Arguments: [date_sk#7], [d_date_sk#22], Inner, BuildRight - -(13) CometProject -Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] -Arguments: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11], [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_store_id#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#24, s_store_id#25] -Condition : isnotnull(s_store_sk#24) - -(16) CometProject -Input [2]: [s_store_sk#24, s_store_id#25] -Arguments: [s_store_sk#24, s_store_id#26], [s_store_sk#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#25, 16)) AS s_store_id#26] - -(17) CometBroadcastExchange -Input [2]: [s_store_sk#24, s_store_id#26] -Arguments: [s_store_sk#24, s_store_id#26] - -(18) CometBroadcastHashJoin -Left output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [2]: [s_store_sk#24, s_store_id#26] -Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight - -(19) CometProject -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#26] -Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] - -(20) CometHashAggregate -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] -Keys [1]: [s_store_id#26] -Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] - -(21) CometExchange -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Keys [1]: [s_store_id#26] -Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk)] -ReadSchema: struct - -(24) CometFilter -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : isnotnull(cs_catalog_page_sk#31) - -(25) CometProject -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cr_catalog_page_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Condition : isnotnull(cr_catalog_page_sk#42) - -(28) CometProject -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] - -(29) CometUnion -Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] - -(30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#52] - -(31) CometBroadcastHashJoin -Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [1]: [d_date_sk#52] -Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight - -(32) CometProject -Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] -Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Condition : isnotnull(cp_catalog_page_sk#53) - -(35) CometProject -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#54, 16)) AS cp_catalog_page_id#55] - -(36) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] - -(37) CometBroadcastHashJoin -Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight - -(38) CometProject -Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] - -(39) CometHashAggregate -Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] - -(40) CometExchange -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(41) CometHashAggregate -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] -PushedFilters: [IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(43) CometFilter -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_web_site_sk#60) - -(44) CometProject -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] -ReadSchema: struct - -(46) CometBroadcastExchange -Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(48) CometFilter -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) - -(49) CometProject -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] - -(50) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft - -(51) CometProject -Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] - -(52) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] - -(53) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#86] - -(54) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [1]: [d_date_sk#86] -Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight - -(55) CometProject -Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] -Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#87, web_site_id#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(57) CometFilter -Input [2]: [web_site_sk#87, web_site_id#88] -Condition : isnotnull(web_site_sk#87) - -(58) CometProject -Input [2]: [web_site_sk#87, web_site_id#88] -Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#88, 16)) AS web_site_id#89] - -(59) CometBroadcastExchange -Input [2]: [web_site_sk#87, web_site_id#89] -Arguments: [web_site_sk#87, web_site_id#89] - -(60) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [2]: [web_site_sk#87, web_site_id#89] -Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight - -(61) CometProject -Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] -Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] - -(62) CometHashAggregate -Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] -Keys [1]: [web_site_id#89] -Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] - -(63) CometExchange -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(64) CometHashAggregate -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Keys [1]: [web_site_id#89] -Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] - -(65) CometUnion -Child 0 Input [5]: [sales#94, returns#95, profit#96, channel#97, id#98] -Child 1 Input [5]: [sales#99, returns#100, profit#101, channel#102, id#103] -Child 2 Input [5]: [sales#104, returns#105, profit#106, channel#107, id#108] - -(66) CometExpand -Input [5]: [sales#94, returns#95, profit#96, channel#97, id#98] -Arguments: [[sales#94, returns#95, profit#96, channel#97, id#98, 0], [sales#94, returns#95, profit#96, channel#97, null, 1], [sales#94, returns#95, profit#96, null, null, 3]], [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] - -(67) CometHashAggregate -Input [6]: [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] -Keys [3]: [channel#109, id#110, spark_grouping_id#111] -Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] - -(68) CometExchange -Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Arguments: hashpartitioning(channel#109, id#110, spark_grouping_id#111, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(69) CometHashAggregate -Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Keys [3]: [channel#109, id#110, spark_grouping_id#111] -Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] - -(70) CometTakeOrderedAndProject -Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#109 ASC NULLS FIRST,id#110 ASC NULLS FIRST], output=[channel#109,id#110,sales#118,returns#119,profit#120]), [channel#109, id#110, sales#118, returns#119, profit#120], 100, 0, [channel#109 ASC NULLS FIRST, id#110 ASC NULLS FIRST], [channel#109, id#110, sales#118, returns#119, profit#120] - -(71) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) - -(74) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(75) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(76) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/simplified.txt deleted file mode 100644 index 6fff89a25c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,86 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id,spark_grouping_id] #1 - CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] - CometUnion [sales,returns,profit,channel,id] - CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),s_store_id] - CometExchange [s_store_id] #2 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - 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 [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,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),cp_catalog_page_id] - CometExchange [cp_catalog_page_id] #6 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - 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 [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,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [cp_catalog_page_sk,cp_catalog_page_id] #7 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),web_site_id] - CometExchange [web_site_id] #8 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - 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 [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,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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,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 [native_iceberg_compat] 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_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 [web_site_sk,web_site_id] #10 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] 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/q5/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/explain.txt deleted file mode 100644 index 6f23a02597..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/explain.txt +++ /dev/null @@ -1,452 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (71) -+- CometTakeOrderedAndProject (70) - +- CometHashAggregate (69) - +- CometExchange (68) - +- CometHashAggregate (67) - +- CometExpand (66) - +- CometUnion (65) - :- CometHashAggregate (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometUnion (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - :- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (32) - : : +- CometBroadcastHashJoin (31) - : : :- CometUnion (29) - : : : :- CometProject (25) - : : : : +- CometFilter (24) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) - : : : +- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) - : : +- ReusedExchange (30) - : +- CometBroadcastExchange (36) - : +- CometProject (35) - : +- CometFilter (34) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (55) - : +- CometBroadcastHashJoin (54) - : :- CometUnion (52) - : : :- CometProject (44) - : : : +- CometFilter (43) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : : +- CometProject (51) - : : +- CometBroadcastHashJoin (50) - : : :- CometBroadcastExchange (46) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) - : : +- CometProject (49) - : : +- CometFilter (48) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : +- ReusedExchange (53) - +- CometBroadcastExchange (59) - +- CometProject (58) - +- CometFilter (57) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Arguments: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11], [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#12) - -(6) CometProject -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Arguments: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21], [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] - -(7) CometUnion -Child 0 Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) - -(10) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(11) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(12) CometBroadcastHashJoin -Left output [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [1]: [d_date_sk#22] -Arguments: [date_sk#7], [d_date_sk#22], Inner, BuildRight - -(13) CometProject -Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] -Arguments: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11], [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_store_id#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#24, s_store_id#25] -Condition : isnotnull(s_store_sk#24) - -(16) CometProject -Input [2]: [s_store_sk#24, s_store_id#25] -Arguments: [s_store_sk#24, s_store_id#26], [s_store_sk#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#25, 16)) AS s_store_id#26] - -(17) CometBroadcastExchange -Input [2]: [s_store_sk#24, s_store_id#26] -Arguments: [s_store_sk#24, s_store_id#26] - -(18) CometBroadcastHashJoin -Left output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [2]: [s_store_sk#24, s_store_id#26] -Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight - -(19) CometProject -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#26] -Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] - -(20) CometHashAggregate -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] -Keys [1]: [s_store_id#26] -Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] - -(21) CometExchange -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Keys [1]: [s_store_id#26] -Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk)] -ReadSchema: struct - -(24) CometFilter -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : isnotnull(cs_catalog_page_sk#31) - -(25) CometProject -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cr_catalog_page_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Condition : isnotnull(cr_catalog_page_sk#42) - -(28) CometProject -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] - -(29) CometUnion -Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] - -(30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#52] - -(31) CometBroadcastHashJoin -Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [1]: [d_date_sk#52] -Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight - -(32) CometProject -Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] -Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Condition : isnotnull(cp_catalog_page_sk#53) - -(35) CometProject -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#54, 16)) AS cp_catalog_page_id#55] - -(36) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] - -(37) CometBroadcastHashJoin -Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight - -(38) CometProject -Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] - -(39) CometHashAggregate -Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] - -(40) CometExchange -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(41) CometHashAggregate -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] -PushedFilters: [IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(43) CometFilter -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_web_site_sk#60) - -(44) CometProject -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] -ReadSchema: struct - -(46) CometBroadcastExchange -Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(48) CometFilter -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) - -(49) CometProject -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] - -(50) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft - -(51) CometProject -Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] - -(52) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] - -(53) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#86] - -(54) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [1]: [d_date_sk#86] -Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight - -(55) CometProject -Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] -Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#87, web_site_id#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(57) CometFilter -Input [2]: [web_site_sk#87, web_site_id#88] -Condition : isnotnull(web_site_sk#87) - -(58) CometProject -Input [2]: [web_site_sk#87, web_site_id#88] -Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#88, 16)) AS web_site_id#89] - -(59) CometBroadcastExchange -Input [2]: [web_site_sk#87, web_site_id#89] -Arguments: [web_site_sk#87, web_site_id#89] - -(60) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [2]: [web_site_sk#87, web_site_id#89] -Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight - -(61) CometProject -Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] -Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] - -(62) CometHashAggregate -Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] -Keys [1]: [web_site_id#89] -Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] - -(63) CometExchange -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(64) CometHashAggregate -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Keys [1]: [web_site_id#89] -Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] - -(65) CometUnion -Child 0 Input [5]: [sales#94, returns#95, profit#96, channel#97, id#98] -Child 1 Input [5]: [sales#99, returns#100, profit#101, channel#102, id#103] -Child 2 Input [5]: [sales#104, returns#105, profit#106, channel#107, id#108] - -(66) CometExpand -Input [5]: [sales#94, returns#95, profit#96, channel#97, id#98] -Arguments: [[sales#94, returns#95, profit#96, channel#97, id#98, 0], [sales#94, returns#95, profit#96, channel#97, null, 1], [sales#94, returns#95, profit#96, null, null, 3]], [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] - -(67) CometHashAggregate -Input [6]: [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] -Keys [3]: [channel#109, id#110, spark_grouping_id#111] -Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] - -(68) CometExchange -Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Arguments: hashpartitioning(channel#109, id#110, spark_grouping_id#111, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(69) CometHashAggregate -Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Keys [3]: [channel#109, id#110, spark_grouping_id#111] -Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] - -(70) CometTakeOrderedAndProject -Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#109 ASC NULLS FIRST,id#110 ASC NULLS FIRST], output=[channel#109,id#110,sales#118,returns#119,profit#120]), [channel#109, id#110, sales#118, returns#119, profit#120], 100, 0, [channel#109 ASC NULLS FIRST, id#110 ASC NULLS FIRST], [channel#109, id#110, sales#118, returns#119, profit#120] - -(71) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) - -(74) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(75) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(76) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/extended.txt deleted file mode 100644 index d128acaf06..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/extended.txt +++ /dev/null @@ -1,90 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - -Comet accelerated 79 out of 86 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 6fff89a25c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt +++ /dev/null @@ -1,86 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id,spark_grouping_id] #1 - CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] - CometUnion [sales,returns,profit,channel,id] - CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),s_store_id] - CometExchange [s_store_id] #2 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - 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 [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,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),cp_catalog_page_id] - CometExchange [cp_catalog_page_id] #6 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - 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 [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,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [cp_catalog_page_sk,cp_catalog_page_id] #7 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),web_site_id] - CometExchange [web_site_id] #8 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - 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 [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,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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,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 [native_iceberg_compat] 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_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 [web_site_sk,web_site_id] #10 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/explain.txt deleted file mode 100644 index 4001dbe16b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (30) -+- * HashAggregate (29) - +- * CometColumnarToRow (28) - +- CometColumnarExchange (27) - +- * HashAggregate (26) - +- * Project (25) - +- * BroadcastHashJoin Inner BuildRight (24) - :- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet spark_catalog.default.store_returns (4) - : : +- BroadcastExchange (14) - : : +- * CometColumnarToRow (13) - : : +- CometProject (12) - : : +- CometFilter (11) - : : +- CometNativeScan parquet spark_catalog.default.store (10) - : +- BroadcastExchange (20) - : +- * CometColumnarToRow (19) - : +- CometFilter (18) - : +- CometNativeScan parquet spark_catalog.default.date_dim (17) - +- ReusedExchange (23) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#5)] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_store_sk#3)) - -(3) CometColumnarToRow [codegen id : 5] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] - -(4) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#9), dynamicpruningexpression(sr_returned_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk), IsNotNull(sr_customer_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] - -(6) Filter [codegen id : 1] -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : ((isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#6)) AND isnotnull(sr_customer_sk#7)) - -(7) BroadcastExchange -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(input[2, int, false], input[0, int, false], input[1, int, false]),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 5] -Left keys [3]: [ss_ticket_number#4, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [sr_ticket_number#8, sr_item_sk#6, sr_customer_sk#7] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 5] -Output [3]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9] -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] - -(10) CometNativeScan parquet spark_catalog.default.store -Output [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(11) CometFilter -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Condition : isnotnull(s_store_sk#11) - -(12) CometProject -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Arguments: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_street_type#16, 15)) AS s_street_type#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_suite_number#17, 10)) AS s_suite_number#23, s_city#18, s_county#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#20, 2)) AS s_state#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#21, 10)) AS s_zip#25] - -(13) CometColumnarToRow [codegen id : 2] -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(14) BroadcastExchange -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None - -(16) Project [codegen id : 5] -Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Input [14]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(17) CometNativeScan parquet spark_catalog.default.date_dim -Output [1]: [d_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) CometFilter -Input [1]: [d_date_sk#26] -Condition : isnotnull(d_date_sk#26) - -(19) CometColumnarToRow [codegen id : 3] -Input [1]: [d_date_sk#26] - -(20) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#26] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 5] -Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#26] - -(23) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#27] - -(24) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [sr_returned_date_sk#9] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 5] -Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#27] - -(26) HashAggregate [codegen id : 5] -Input [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Functions [5]: [partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] -Aggregate Attributes [5]: [sum#28, sum#29, sum#30, sum#31, sum#32] -Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#33, sum#34, sum#35, sum#36, sum#37] - -(27) CometColumnarExchange -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#33, sum#34, sum#35, sum#36, sum#37] -Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(28) CometColumnarToRow [codegen id : 6] -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#33, sum#34, sum#35, sum#36, sum#37] - -(29) HashAggregate [codegen id : 6] -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#33, sum#34, sum#35, sum#36, sum#37] -Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Functions [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] -Aggregate Attributes [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#38, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#39, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#40, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#41, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#42] -Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#38 AS 30 days #43, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#39 AS 31 - 60 days #44, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#40 AS 61 - 90 days #45, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#41 AS 91 - 120 days #46, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#42 AS >120 days #47] - -(30) TakeOrderedAndProject -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #43, 31 - 60 days #44, 61 - 90 days #45, 91 - 120 days #46, >120 days #47] -Arguments: 100, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRST, s_street_number#14 ASC NULLS FIRST, s_street_name#15 ASC NULLS FIRST, s_street_type#22 ASC NULLS FIRST, s_suite_number#23 ASC NULLS FIRST, s_city#18 ASC NULLS FIRST, s_county#19 ASC NULLS FIRST, s_state#24 ASC NULLS FIRST, s_zip#25 ASC NULLS FIRST], [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #43, 31 - 60 days #44, 61 - 90 days #45, 91 - 120 days #46, >120 days #47] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometNativeScan parquet spark_catalog.default.date_dim (31) - - -(31) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#48, d_moy#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [d_date_sk#27, d_year#48, d_moy#49] -Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#49)) AND (d_year#48 = 2001)) AND (d_moy#49 = 8)) AND isnotnull(d_date_sk#27)) - -(33) CometProject -Input [3]: [d_date_sk#27, d_year#48, d_moy#49] -Arguments: [d_date_sk#27], [d_date_sk#27] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#27] - -(35) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/simplified.txt deleted file mode 100644 index 186c041b09..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -TakeOrderedAndProject [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,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - WholeStageCodegen (6) - HashAggregate [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] [sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 30) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 60) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 90) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) > 120) THEN 1 ELSE 0 END),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [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] #1 - WholeStageCodegen (5) - HashAggregate [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] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [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] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [sr_ticket_number,sr_item_sk,sr_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_street_type,s_suite_number,s_state,s_zip] [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] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/explain.txt deleted file mode 100644 index d55e867987..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,203 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5)] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#9), dynamicpruningexpression(sr_returned_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk), IsNotNull(sr_customer_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : ((isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#6)) AND isnotnull(sr_customer_sk#7)) - -(5) CometBroadcastExchange -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] - -(6) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [ss_ticket_number#4, ss_item_sk#1, ss_customer_sk#2], [sr_ticket_number#8, sr_item_sk#6, sr_customer_sk#7], Inner, BuildRight - -(7) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9], [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(9) CometFilter -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Condition : isnotnull(s_store_sk#11) - -(10) CometProject -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Arguments: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_street_type#16, 15)) AS s_street_type#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_suite_number#17, 10)) AS s_suite_number#23, s_city#18, s_county#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#20, 2)) AS s_state#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#21, 10)) AS s_zip#25] - -(11) CometBroadcastExchange -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Arguments: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(12) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9] -Right output [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Arguments: [ss_store_sk#3], [s_store_sk#11], Inner, BuildRight - -(13) CometProject -Input [14]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [1]: [d_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [1]: [d_date_sk#26] -Condition : isnotnull(d_date_sk#26) - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] - -(17) CometBroadcastHashJoin -Left output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#5], [d_date_sk#26], Inner, BuildRight - -(18) CometProject -Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#26] -Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_moy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Condition : ((((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2001)) AND (d_moy#29 = 8)) AND isnotnull(d_date_sk#27)) - -(21) CometProject -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Arguments: [d_date_sk#27], [d_date_sk#27] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: [d_date_sk#27] - -(23) CometBroadcastHashJoin -Left output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Right output [1]: [d_date_sk#27] -Arguments: [sr_returned_date_sk#9], [d_date_sk#27], Inner, BuildRight - -(24) CometProject -Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#27] -Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(25) CometHashAggregate -Input [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Functions [5]: [partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#30, sum#31, sum#32, sum#33, sum#34] -Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#30, sum#31, sum#32, sum#33, sum#34] -Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Functions [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#12 ASC NULLS FIRST,s_company_id#13 ASC NULLS FIRST,s_street_number#14 ASC NULLS FIRST,s_street_name#15 ASC NULLS FIRST,s_street_type#22 ASC NULLS FIRST,s_suite_number#23 ASC NULLS FIRST,s_city#18 ASC NULLS FIRST,s_county#19 ASC NULLS FIRST,s_state#24 ASC NULLS FIRST,s_zip#25 ASC NULLS FIRST], output=[s_store_name#12,s_company_id#13,s_street_number#14,s_street_name#15,s_street_type#22,s_suite_number#23,s_city#18,s_county#19,s_state#24,s_zip#25,30 days #35,31 - 60 days #36,61 - 90 days #37,91 - 120 days #38,>120 days #39]), [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39], 100, 0, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRST, s_street_number#14 ASC NULLS FIRST, s_street_name#15 ASC NULLS FIRST, s_street_type#22 ASC NULLS FIRST, s_suite_number#23 ASC NULLS FIRST, s_city#18 ASC NULLS FIRST, s_county#19 ASC NULLS FIRST, s_state#24 ASC NULLS FIRST, s_zip#25 ASC NULLS FIRST], [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39] - -(29) CometColumnarToRow [codegen id : 1] -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_moy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Condition : ((((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2001)) AND (d_moy#29 = 8)) AND isnotnull(d_date_sk#27)) - -(32) CometProject -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Arguments: [d_date_sk#27], [d_date_sk#27] - -(33) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#27] - -(34) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/simplified.txt deleted file mode 100644 index 98a44eaf40..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,39 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [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,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [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,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 30) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 60) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 90) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [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] #1 - CometHashAggregate [sr_returned_date_sk,ss_sold_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,sum,sum,sum,sum,sum] - 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,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,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,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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 - CometProject [s_street_type,s_suite_number,s_state,s_zip] [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] - 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 [native_iceberg_compat] 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 [d_date_sk] #5 - CometFilter [d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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/q50/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/explain.txt deleted file mode 100644 index d55e867987..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/explain.txt +++ /dev/null @@ -1,203 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5)] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#9), dynamicpruningexpression(sr_returned_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk), IsNotNull(sr_customer_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : ((isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#6)) AND isnotnull(sr_customer_sk#7)) - -(5) CometBroadcastExchange -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] - -(6) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [ss_ticket_number#4, ss_item_sk#1, ss_customer_sk#2], [sr_ticket_number#8, sr_item_sk#6, sr_customer_sk#7], Inner, BuildRight - -(7) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9], [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(9) CometFilter -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Condition : isnotnull(s_store_sk#11) - -(10) CometProject -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Arguments: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_street_type#16, 15)) AS s_street_type#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_suite_number#17, 10)) AS s_suite_number#23, s_city#18, s_county#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#20, 2)) AS s_state#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#21, 10)) AS s_zip#25] - -(11) CometBroadcastExchange -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Arguments: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(12) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9] -Right output [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Arguments: [ss_store_sk#3], [s_store_sk#11], Inner, BuildRight - -(13) CometProject -Input [14]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [1]: [d_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [1]: [d_date_sk#26] -Condition : isnotnull(d_date_sk#26) - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] - -(17) CometBroadcastHashJoin -Left output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#5], [d_date_sk#26], Inner, BuildRight - -(18) CometProject -Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#26] -Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_moy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Condition : ((((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2001)) AND (d_moy#29 = 8)) AND isnotnull(d_date_sk#27)) - -(21) CometProject -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Arguments: [d_date_sk#27], [d_date_sk#27] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: [d_date_sk#27] - -(23) CometBroadcastHashJoin -Left output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Right output [1]: [d_date_sk#27] -Arguments: [sr_returned_date_sk#9], [d_date_sk#27], Inner, BuildRight - -(24) CometProject -Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#27] -Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(25) CometHashAggregate -Input [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Functions [5]: [partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#30, sum#31, sum#32, sum#33, sum#34] -Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#30, sum#31, sum#32, sum#33, sum#34] -Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Functions [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#12 ASC NULLS FIRST,s_company_id#13 ASC NULLS FIRST,s_street_number#14 ASC NULLS FIRST,s_street_name#15 ASC NULLS FIRST,s_street_type#22 ASC NULLS FIRST,s_suite_number#23 ASC NULLS FIRST,s_city#18 ASC NULLS FIRST,s_county#19 ASC NULLS FIRST,s_state#24 ASC NULLS FIRST,s_zip#25 ASC NULLS FIRST], output=[s_store_name#12,s_company_id#13,s_street_number#14,s_street_name#15,s_street_type#22,s_suite_number#23,s_city#18,s_county#19,s_state#24,s_zip#25,30 days #35,31 - 60 days #36,61 - 90 days #37,91 - 120 days #38,>120 days #39]), [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39], 100, 0, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRST, s_street_number#14 ASC NULLS FIRST, s_street_name#15 ASC NULLS FIRST, s_street_type#22 ASC NULLS FIRST, s_suite_number#23 ASC NULLS FIRST, s_city#18 ASC NULLS FIRST, s_county#19 ASC NULLS FIRST, s_state#24 ASC NULLS FIRST, s_zip#25 ASC NULLS FIRST], [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39] - -(29) CometColumnarToRow [codegen id : 1] -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_moy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Condition : ((((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2001)) AND (d_moy#29 = 8)) AND isnotnull(d_date_sk#27)) - -(32) CometProject -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Arguments: [d_date_sk#27], [d_date_sk#27] - -(33) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#27] - -(34) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/extended.txt deleted file mode 100644 index 0fbba7d1e8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 98a44eaf40..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt +++ /dev/null @@ -1,39 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [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,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [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,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 30) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 60) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 90) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [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] #1 - CometHashAggregate [sr_returned_date_sk,ss_sold_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,sum,sum,sum,sum,sum] - 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,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,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,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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 - CometProject [s_street_type,s_suite_number,s_state,s_zip] [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] - 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 [native_iceberg_compat] 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 [d_date_sk] #5 - CometFilter [d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/explain.txt deleted file mode 100644 index 9af40029c8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/explain.txt +++ /dev/null @@ -1,264 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (42) -+- * Filter (41) - +- Window (40) - +- * CometColumnarToRow (39) - +- CometSort (38) - +- CometExchange (37) - +- CometProject (36) - +- CometSortMergeJoin (35) - :- CometSort (17) - : +- CometColumnarExchange (16) - : +- * Project (15) - : +- Window (14) - : +- * CometColumnarToRow (13) - : +- CometSort (12) - : +- CometColumnarExchange (11) - : +- * HashAggregate (10) - : +- * CometColumnarToRow (9) - : +- CometColumnarExchange (8) - : +- * HashAggregate (7) - : +- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.web_sales (1) - : +- ReusedExchange (4) - +- CometSort (34) - +- CometColumnarExchange (33) - +- * Project (32) - +- Window (31) - +- * CometColumnarToRow (30) - +- CometSort (29) - +- CometColumnarExchange (28) - +- * HashAggregate (27) - +- * CometColumnarToRow (26) - +- CometColumnarExchange (25) - +- * HashAggregate (24) - +- * Project (23) - +- * BroadcastHashJoin Inner BuildRight (22) - :- * Filter (20) - : +- * ColumnarToRow (19) - : +- Scan parquet spark_catalog.default.store_sales (18) - +- ReusedExchange (21) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] - -(3) Filter [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(4) ReusedExchange [Reuses operator id: 47] -Output [2]: [d_date_sk#5, d_date#6] - -(5) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 2] -Output [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] - -(7) HashAggregate [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum#7] -Results [3]: [ws_item_sk#1, d_date#6, sum#8] - -(8) CometColumnarExchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(9) CometColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, d_date#6, sum#8] - -(10) HashAggregate [codegen id : 3] -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#9] -Results [4]: [ws_item_sk#1 AS item_sk#10, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#9,17,2) AS _w0#11, ws_item_sk#1] - -(11) CometColumnarExchange -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(12) CometSort -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(13) CometColumnarToRow [codegen id : 4] -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] - -(14) Window -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: [sum(_w0#11) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(15) Project [codegen id : 5] -Output [3]: [item_sk#10, d_date#6, cume_sales#12] -Input [5]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1, cume_sales#12] - -(16) CometColumnarExchange -Input [3]: [item_sk#10, d_date#6, cume_sales#12] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(17) CometSort -Input [3]: [item_sk#10, d_date#6, cume_sales#12] -Arguments: [item_sk#10, d_date#6, cume_sales#12], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(18) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_sold_date_sk#15 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 7] -Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] - -(20) Filter [codegen id : 7] -Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -Condition : isnotnull(ss_item_sk#13) - -(21) ReusedExchange [Reuses operator id: 47] -Output [2]: [d_date_sk#16, d_date#17] - -(22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#15] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None - -(23) Project [codegen id : 7] -Output [3]: [ss_item_sk#13, ss_sales_price#14, d_date#17] -Input [5]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#16, d_date#17] - -(24) HashAggregate [codegen id : 7] -Input [3]: [ss_item_sk#13, ss_sales_price#14, d_date#17] -Keys [2]: [ss_item_sk#13, d_date#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [ss_item_sk#13, d_date#17, sum#19] - -(25) CometColumnarExchange -Input [3]: [ss_item_sk#13, d_date#17, sum#19] -Arguments: hashpartitioning(ss_item_sk#13, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(26) CometColumnarToRow [codegen id : 8] -Input [3]: [ss_item_sk#13, d_date#17, sum#19] - -(27) HashAggregate [codegen id : 8] -Input [3]: [ss_item_sk#13, d_date#17, sum#19] -Keys [2]: [ss_item_sk#13, d_date#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#14))#20] -Results [4]: [ss_item_sk#13 AS item_sk#21, d_date#17, MakeDecimal(sum(UnscaledValue(ss_sales_price#14))#20,17,2) AS _w0#22, ss_item_sk#13] - -(28) CometColumnarExchange -Input [4]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13] -Arguments: hashpartitioning(ss_item_sk#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(29) CometSort -Input [4]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13] -Arguments: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13], [ss_item_sk#13 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] - -(30) CometColumnarToRow [codegen id : 9] -Input [4]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13] - -(31) Window -Input [4]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13] -Arguments: [sum(_w0#22) windowspecdefinition(ss_item_sk#13, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#23], [ss_item_sk#13], [d_date#17 ASC NULLS FIRST] - -(32) Project [codegen id : 10] -Output [3]: [item_sk#21, d_date#17, cume_sales#23] -Input [5]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13, cume_sales#23] - -(33) CometColumnarExchange -Input [3]: [item_sk#21, d_date#17, cume_sales#23] -Arguments: hashpartitioning(item_sk#21, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(34) CometSort -Input [3]: [item_sk#21, d_date#17, cume_sales#23] -Arguments: [item_sk#21, d_date#17, cume_sales#23], [item_sk#21 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] - -(35) CometSortMergeJoin -Left output [3]: [item_sk#10, d_date#6, cume_sales#12] -Right output [3]: [item_sk#21, d_date#17, cume_sales#23] -Arguments: [item_sk#10, d_date#6], [item_sk#21, d_date#17], FullOuter - -(36) CometProject -Input [6]: [item_sk#10, d_date#6, cume_sales#12, item_sk#21, d_date#17, cume_sales#23] -Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#21 END AS item_sk#24, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#17 END AS d_date#25, cume_sales#12 AS web_sales#26, cume_sales#23 AS store_sales#27] - -(37) CometExchange -Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] -Arguments: hashpartitioning(item_sk#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(38) CometSort -Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] -Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27], [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST] - -(39) CometColumnarToRow [codegen id : 11] -Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] - -(40) Window -Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] -Arguments: [max(web_sales#26) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#28, max(store_sales#27) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#29], [item_sk#24], [d_date#25 ASC NULLS FIRST] - -(41) Filter [codegen id : 12] -Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] -Condition : ((isnotnull(web_cumulative#28) AND isnotnull(store_cumulative#29)) AND (web_cumulative#28 > store_cumulative#29)) - -(42) TakeOrderedAndProject -Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] -Arguments: 100, [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST], [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (47) -+- * CometColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometNativeScan parquet spark_catalog.default.date_dim (43) - - -(43) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(44) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#30] -Condition : (((isnotnull(d_month_seq#30) AND (d_month_seq#30 >= 1200)) AND (d_month_seq#30 <= 1211)) AND isnotnull(d_date_sk#5)) - -(45) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#30] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(46) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] - -(47) BroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/simplified.txt deleted file mode 100644 index 51d3f11ae6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (12) - Filter [web_cumulative,store_cumulative] - InputAdapter - Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (5) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ws_item_sk] - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk,d_date] #4 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #6 - WholeStageCodegen (10) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ss_item_sk] - CometColumnarExchange [ss_item_sk] #7 - WholeStageCodegen (8) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk,d_date] #8 - WholeStageCodegen (7) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/explain.txt deleted file mode 100644 index 683cc25d24..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,257 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (41) -+- * Filter (40) - +- Window (39) - +- * CometColumnarToRow (38) - +- CometSort (37) - +- CometExchange (36) - +- CometProject (35) - +- CometSortMergeJoin (34) - :- CometSort (18) - : +- CometColumnarExchange (17) - : +- * Project (16) - : +- Window (15) - : +- * CometColumnarToRow (14) - : +- CometSort (13) - : +- CometExchange (12) - : +- CometHashAggregate (11) - : +- CometExchange (10) - : +- CometHashAggregate (9) - : +- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- CometSort (33) - +- CometColumnarExchange (32) - +- * Project (31) - +- Window (30) - +- * CometColumnarToRow (29) - +- CometSort (28) - +- CometExchange (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometBroadcastHashJoin (22) - :- CometFilter (20) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (19) - +- ReusedExchange (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: [d_date_sk#5, d_date#6] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Right output [2]: [d_date_sk#5, d_date#6] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] -Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] - -(9) CometHashAggregate -Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] - -(10) CometExchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] - -(12) CometExchange -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(13) CometSort -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(14) CometColumnarToRow [codegen id : 1] -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] - -(15) Window -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: [sum(_w0#10) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(16) Project [codegen id : 2] -Output [3]: [item_sk#9, d_date#6, cume_sales#11] -Input [5]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1, cume_sales#11] - -(17) CometColumnarExchange -Input [3]: [item_sk#9, d_date#6, cume_sales#11] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(18) CometSort -Input [3]: [item_sk#9, d_date#6, cume_sales#11] -Arguments: [item_sk#9, d_date#6, cume_sales#11], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#14), dynamicpruningexpression(ss_sold_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] -Condition : isnotnull(ss_item_sk#12) - -(21) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#16, d_date#17] - -(22) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] -Right output [2]: [d_date_sk#16, d_date#17] -Arguments: [ss_sold_date_sk#14], [d_date_sk#16], Inner, BuildRight - -(23) CometProject -Input [5]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14, d_date_sk#16, d_date#17] -Arguments: [ss_item_sk#12, ss_sales_price#13, d_date#17], [ss_item_sk#12, ss_sales_price#13, d_date#17] - -(24) CometHashAggregate -Input [3]: [ss_item_sk#12, ss_sales_price#13, d_date#17] -Keys [2]: [ss_item_sk#12, d_date#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#13))] - -(25) CometExchange -Input [3]: [ss_item_sk#12, d_date#17, sum#18] -Arguments: hashpartitioning(ss_item_sk#12, d_date#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(26) CometHashAggregate -Input [3]: [ss_item_sk#12, d_date#17, sum#18] -Keys [2]: [ss_item_sk#12, d_date#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#13))] - -(27) CometExchange -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -Arguments: hashpartitioning(ss_item_sk#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(28) CometSort -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -Arguments: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12], [ss_item_sk#12 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] - -(29) CometColumnarToRow [codegen id : 3] -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] - -(30) Window -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -Arguments: [sum(_w0#20) windowspecdefinition(ss_item_sk#12, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#21], [ss_item_sk#12], [d_date#17 ASC NULLS FIRST] - -(31) Project [codegen id : 4] -Output [3]: [item_sk#19, d_date#17, cume_sales#21] -Input [5]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12, cume_sales#21] - -(32) CometColumnarExchange -Input [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: hashpartitioning(item_sk#19, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(33) CometSort -Input [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: [item_sk#19, d_date#17, cume_sales#21], [item_sk#19 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] - -(34) CometSortMergeJoin -Left output [3]: [item_sk#9, d_date#6, cume_sales#11] -Right output [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: [item_sk#9, d_date#6], [item_sk#19, d_date#17], FullOuter - -(35) CometProject -Input [6]: [item_sk#9, d_date#6, cume_sales#11, item_sk#19, d_date#17, cume_sales#21] -Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#19 END AS item_sk#22, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#17 END AS d_date#23, cume_sales#11 AS web_sales#24, cume_sales#21 AS store_sales#25] - -(36) CometExchange -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: hashpartitioning(item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(37) CometSort -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST] - -(38) CometColumnarToRow [codegen id : 5] -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] - -(39) Window -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: [max(web_sales#24) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#26, max(store_sales#25) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#27], [item_sk#22], [d_date#23 ASC NULLS FIRST] - -(40) Filter [codegen id : 6] -Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] -Condition : ((isnotnull(web_cumulative#26) AND isnotnull(store_cumulative#27)) AND (web_cumulative#26 > store_cumulative#27)) - -(41) TakeOrderedAndProject -Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] -Arguments: 100, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) - - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) - -(44) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(45) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] - -(46) BroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/simplified.txt deleted file mode 100644 index a5af2e114b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,62 +0,0 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (6) - Filter [web_cumulative,store_cumulative] - InputAdapter - Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (2) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ws_item_sk] - CometExchange [ws_item_sk] #3 - CometHashAggregate [sum] [item_sk,d_date,_w0,ws_item_sk,sum(UnscaledValue(ws_sales_price))] - CometExchange [ws_item_sk,d_date] #4 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #6 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #7 - WholeStageCodegen (4) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ss_item_sk] - CometExchange [ss_item_sk] #8 - CometHashAggregate [sum] [item_sk,d_date,_w0,ss_item_sk,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_item_sk,d_date] #9 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - 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 [native_iceberg_compat] 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/q51/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt deleted file mode 100644 index 683cc25d24..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt +++ /dev/null @@ -1,257 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (41) -+- * Filter (40) - +- Window (39) - +- * CometColumnarToRow (38) - +- CometSort (37) - +- CometExchange (36) - +- CometProject (35) - +- CometSortMergeJoin (34) - :- CometSort (18) - : +- CometColumnarExchange (17) - : +- * Project (16) - : +- Window (15) - : +- * CometColumnarToRow (14) - : +- CometSort (13) - : +- CometExchange (12) - : +- CometHashAggregate (11) - : +- CometExchange (10) - : +- CometHashAggregate (9) - : +- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- CometSort (33) - +- CometColumnarExchange (32) - +- * Project (31) - +- Window (30) - +- * CometColumnarToRow (29) - +- CometSort (28) - +- CometExchange (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometBroadcastHashJoin (22) - :- CometFilter (20) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (19) - +- ReusedExchange (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: [d_date_sk#5, d_date#6] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Right output [2]: [d_date_sk#5, d_date#6] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] -Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] - -(9) CometHashAggregate -Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] - -(10) CometExchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] - -(12) CometExchange -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(13) CometSort -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(14) CometColumnarToRow [codegen id : 1] -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] - -(15) Window -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: [sum(_w0#10) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(16) Project [codegen id : 2] -Output [3]: [item_sk#9, d_date#6, cume_sales#11] -Input [5]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1, cume_sales#11] - -(17) CometColumnarExchange -Input [3]: [item_sk#9, d_date#6, cume_sales#11] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(18) CometSort -Input [3]: [item_sk#9, d_date#6, cume_sales#11] -Arguments: [item_sk#9, d_date#6, cume_sales#11], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#14), dynamicpruningexpression(ss_sold_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] -Condition : isnotnull(ss_item_sk#12) - -(21) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#16, d_date#17] - -(22) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] -Right output [2]: [d_date_sk#16, d_date#17] -Arguments: [ss_sold_date_sk#14], [d_date_sk#16], Inner, BuildRight - -(23) CometProject -Input [5]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14, d_date_sk#16, d_date#17] -Arguments: [ss_item_sk#12, ss_sales_price#13, d_date#17], [ss_item_sk#12, ss_sales_price#13, d_date#17] - -(24) CometHashAggregate -Input [3]: [ss_item_sk#12, ss_sales_price#13, d_date#17] -Keys [2]: [ss_item_sk#12, d_date#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#13))] - -(25) CometExchange -Input [3]: [ss_item_sk#12, d_date#17, sum#18] -Arguments: hashpartitioning(ss_item_sk#12, d_date#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(26) CometHashAggregate -Input [3]: [ss_item_sk#12, d_date#17, sum#18] -Keys [2]: [ss_item_sk#12, d_date#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#13))] - -(27) CometExchange -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -Arguments: hashpartitioning(ss_item_sk#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(28) CometSort -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -Arguments: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12], [ss_item_sk#12 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] - -(29) CometColumnarToRow [codegen id : 3] -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] - -(30) Window -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -Arguments: [sum(_w0#20) windowspecdefinition(ss_item_sk#12, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#21], [ss_item_sk#12], [d_date#17 ASC NULLS FIRST] - -(31) Project [codegen id : 4] -Output [3]: [item_sk#19, d_date#17, cume_sales#21] -Input [5]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12, cume_sales#21] - -(32) CometColumnarExchange -Input [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: hashpartitioning(item_sk#19, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(33) CometSort -Input [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: [item_sk#19, d_date#17, cume_sales#21], [item_sk#19 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] - -(34) CometSortMergeJoin -Left output [3]: [item_sk#9, d_date#6, cume_sales#11] -Right output [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: [item_sk#9, d_date#6], [item_sk#19, d_date#17], FullOuter - -(35) CometProject -Input [6]: [item_sk#9, d_date#6, cume_sales#11, item_sk#19, d_date#17, cume_sales#21] -Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#19 END AS item_sk#22, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#17 END AS d_date#23, cume_sales#11 AS web_sales#24, cume_sales#21 AS store_sales#25] - -(36) CometExchange -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: hashpartitioning(item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(37) CometSort -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST] - -(38) CometColumnarToRow [codegen id : 5] -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] - -(39) Window -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: [max(web_sales#24) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#26, max(store_sales#25) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#27], [item_sk#22], [d_date#23 ASC NULLS FIRST] - -(40) Filter [codegen id : 6] -Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] -Condition : ((isnotnull(web_cumulative#26) AND isnotnull(store_cumulative#27)) AND (web_cumulative#26 > store_cumulative#27)) - -(41) TakeOrderedAndProject -Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] -Arguments: 100, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) - - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) - -(44) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(45) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] - -(46) BroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/extended.txt deleted file mode 100644 index 66c5717cc7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/extended.txt +++ /dev/null @@ -1,53 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometColumnarExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 37 out of 47 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index a5af2e114b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt +++ /dev/null @@ -1,62 +0,0 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (6) - Filter [web_cumulative,store_cumulative] - InputAdapter - Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (2) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ws_item_sk] - CometExchange [ws_item_sk] #3 - CometHashAggregate [sum] [item_sk,d_date,_w0,ws_item_sk,sum(UnscaledValue(ws_sales_price))] - CometExchange [ws_item_sk,d_date] #4 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #6 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #7 - WholeStageCodegen (4) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ss_item_sk] - CometExchange [ss_item_sk] #8 - CometHashAggregate [sum] [item_sk,d_date,_w0,ss_item_sk,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_item_sk,d_date] #9 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - 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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_datafusion/explain.txt deleted file mode 100644 index 9de09937ac..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_datafusion/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometNativeScan parquet spark_catalog.default.item (9) - - -(1) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometNativeScan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#9, 50)) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,ext_price#15]), [d_year#2, brand_id#13, brand#14, ext_price#15], 100, 0, [d_year#2 ASC NULLS FIRST, ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, ext_price#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_datafusion/simplified.txt deleted file mode 100644 index cee223d027..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_datafusion/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,ext_price] - CometHashAggregate [sum] [d_year,brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometNativeScan 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/q52.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/explain.txt deleted file mode 100644 index 68a8ab88fa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#9, 50)) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,ext_price#15]), [d_year#2, brand_id#13, brand#14, ext_price#15], 100, 0, [d_year#2 ASC NULLS FIRST, ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, ext_price#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/simplified.txt deleted file mode 100644 index 218062c3ef..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,ext_price] - CometHashAggregate [sum] [d_year,brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan [native_iceberg_compat] 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/q52/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/explain.txt deleted file mode 100644 index 68a8ab88fa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#9, 50)) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,ext_price#15]), [d_year#2, brand_id#13, brand#14, ext_price#15], 100, 0, [d_year#2 ASC NULLS FIRST, ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, ext_price#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/extended.txt deleted file mode 100644 index a292badf5d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 218062c3ef..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,ext_price] - CometHashAggregate [sum] [d_year,brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/explain.txt deleted file mode 100644 index 8f5f268b10..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/explain.txt +++ /dev/null @@ -1,202 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * CometColumnarToRow (26) - +- CometSort (25) - +- CometColumnarExchange (24) - +- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometColumnarExchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * CometColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (8) - : : +- * Filter (7) - : : +- * ColumnarToRow (6) - : : +- Scan parquet spark_catalog.default.store_sales (5) - : +- ReusedExchange (11) - +- BroadcastExchange (17) - +- * CometColumnarToRow (16) - +- CometFilter (15) - +- CometNativeScan parquet spark_catalog.default.store (14) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) IN (Books ,Children ,Electronics ) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#3, 50)) IN (personal ,portable ,reference ,self-help )) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) IN (Women ,Music ,Men ) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#3, 50)) IN (accessories ,classical ,fragrances ,pants )) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_manufact_id#5], [i_item_sk#1, i_manufact_id#5] - -(4) CometColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#1, i_manufact_id#5] - -(5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(7) Filter [codegen id : 1] -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) - -(8) BroadcastExchange -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#10] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Input [6]: [i_item_sk#1, i_manufact_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(11) ReusedExchange [Reuses operator id: 35] -Output [2]: [d_date_sk#15, d_qoy#16] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#13] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#16] -Input [6]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_qoy#16] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [1]: [s_store_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [1]: [s_store_sk#17] -Condition : isnotnull(s_store_sk#17) - -(16) CometColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#17] - -(17) BroadcastExchange -Input [1]: [s_store_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#11] -Right keys [1]: [s_store_sk#17] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#16] -Input [5]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#16, s_store_sk#17] - -(20) HashAggregate [codegen id : 4] -Input [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#16] -Keys [2]: [i_manufact_id#5, d_qoy#16] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [i_manufact_id#5, d_qoy#16, sum#19] - -(21) CometColumnarExchange -Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] -Arguments: hashpartitioning(i_manufact_id#5, d_qoy#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] - -(23) HashAggregate [codegen id : 5] -Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] -Keys [2]: [i_manufact_id#5, d_qoy#16] -Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#20] -Results [3]: [i_manufact_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS _w0#22] - -(24) CometColumnarExchange -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: [i_manufact_id#5, sum_sales#21, _w0#22], [i_manufact_id#5 ASC NULLS FIRST] - -(26) CometColumnarToRow [codegen id : 6] -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] - -(27) Window -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#23], [i_manufact_id#5] - -(28) Filter [codegen id : 7] -Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] -Condition : CASE WHEN (avg_quarterly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_quarterly_sales#23)) / avg_quarterly_sales#23) > 0.1000000000000000) ELSE false END - -(29) Project [codegen id : 7] -Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] -Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] - -(30) TakeOrderedAndProject -Input [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] -Arguments: 100, [avg_quarterly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometNativeScan parquet spark_catalog.default.date_dim (31) - - -(31) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] -Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(33) CometProject -Input [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] -Arguments: [d_date_sk#15, d_qoy#16], [d_date_sk#15, d_qoy#16] - -(34) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_qoy#16] - -(35) BroadcastExchange -Input [2]: [d_date_sk#15, d_qoy#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/simplified.txt deleted file mode 100644 index 63c3e1a17a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/simplified.txt +++ /dev/null @@ -1,53 +0,0 @@ -TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (7) - Project [i_manufact_id,sum_sales,avg_quarterly_sales] - Filter [avg_quarterly_sales,sum_sales] - InputAdapter - Window [_w0,i_manufact_id] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_manufact_id,sum_sales,_w0] - CometColumnarExchange [i_manufact_id] #1 - WholeStageCodegen (5) - HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id,d_qoy] #2 - WholeStageCodegen (4) - HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] - Project [i_manufact_id,ss_sales_price,d_qoy] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_sk,i_manufact_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_qoy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/explain.txt deleted file mode 100644 index 5b68f4c00c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,198 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) - +- CometSort (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) IN (Books ,Children ,Electronics ) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#3, 50)) IN (personal ,portable ,reference ,self-help )) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) IN (Women ,Music ,Men ) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#3, 50)) IN (accessories ,classical ,fragrances ,pants )) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_manufact_id#5], [i_item_sk#1, i_manufact_id#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(7) CometBroadcastHashJoin -Left output [2]: [i_item_sk#1, i_manufact_id#5] -Right output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_item_sk#1], [ss_item_sk#10], Inner, BuildRight - -(8) CometProject -Input [6]: [i_item_sk#1, i_manufact_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13], [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(11) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] - -(12) CometBroadcastExchange -Input [2]: [d_date_sk#15, d_qoy#17] -Arguments: [d_date_sk#15, d_qoy#17] - -(13) CometBroadcastHashJoin -Left output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_qoy#17] -Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(14) CometProject -Input [6]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_qoy#17] -Arguments: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17], [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [s_store_sk#18] -Condition : isnotnull(s_store_sk#18) - -(17) CometBroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: [s_store_sk#18] - -(18) CometBroadcastHashJoin -Left output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17] -Right output [1]: [s_store_sk#18] -Arguments: [ss_store_sk#11], [s_store_sk#18], Inner, BuildRight - -(19) CometProject -Input [5]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17, s_store_sk#18] -Arguments: [i_manufact_id#5, ss_sales_price#12, d_qoy#17], [i_manufact_id#5, ss_sales_price#12, d_qoy#17] - -(20) CometHashAggregate -Input [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#17] -Keys [2]: [i_manufact_id#5, d_qoy#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] - -(21) CometExchange -Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] -Arguments: hashpartitioning(i_manufact_id#5, d_qoy#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] -Keys [2]: [i_manufact_id#5, d_qoy#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] - -(23) CometExchange -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: [i_manufact_id#5, sum_sales#20, _w0#21], [i_manufact_id#5 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 1] -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] - -(26) Window -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#22], [i_manufact_id#5] - -(27) Filter [codegen id : 2] -Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] -Condition : CASE WHEN (avg_quarterly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_quarterly_sales#22)) / avg_quarterly_sales#22) > 0.1000000000000000) ELSE false END - -(28) Project [codegen id : 2] -Output [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] -Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] - -(29) TakeOrderedAndProject -Input [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] -Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(32) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_qoy#17] - -(34) BroadcastExchange -Input [2]: [d_date_sk#15, d_qoy#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/simplified.txt deleted file mode 100644 index 0c123eb728..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (2) - Project [i_manufact_id,sum_sales,avg_quarterly_sales] - Filter [avg_quarterly_sales,sum_sales] - InputAdapter - Window [_w0,i_manufact_id] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_manufact_id,sum_sales,_w0] - CometExchange [i_manufact_id] #1 - CometHashAggregate [d_qoy,sum] [i_manufact_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_manufact_id,d_qoy] #2 - CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] - CometProject [i_manufact_id,ss_sales_price,d_qoy] - 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 [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,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_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [d_date_sk,d_qoy] #5 - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt deleted file mode 100644 index 5b68f4c00c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt +++ /dev/null @@ -1,198 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) - +- CometSort (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) IN (Books ,Children ,Electronics ) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#3, 50)) IN (personal ,portable ,reference ,self-help )) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) IN (Women ,Music ,Men ) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#3, 50)) IN (accessories ,classical ,fragrances ,pants )) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_manufact_id#5], [i_item_sk#1, i_manufact_id#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(7) CometBroadcastHashJoin -Left output [2]: [i_item_sk#1, i_manufact_id#5] -Right output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_item_sk#1], [ss_item_sk#10], Inner, BuildRight - -(8) CometProject -Input [6]: [i_item_sk#1, i_manufact_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13], [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(11) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] - -(12) CometBroadcastExchange -Input [2]: [d_date_sk#15, d_qoy#17] -Arguments: [d_date_sk#15, d_qoy#17] - -(13) CometBroadcastHashJoin -Left output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_qoy#17] -Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(14) CometProject -Input [6]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_qoy#17] -Arguments: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17], [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [s_store_sk#18] -Condition : isnotnull(s_store_sk#18) - -(17) CometBroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: [s_store_sk#18] - -(18) CometBroadcastHashJoin -Left output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17] -Right output [1]: [s_store_sk#18] -Arguments: [ss_store_sk#11], [s_store_sk#18], Inner, BuildRight - -(19) CometProject -Input [5]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17, s_store_sk#18] -Arguments: [i_manufact_id#5, ss_sales_price#12, d_qoy#17], [i_manufact_id#5, ss_sales_price#12, d_qoy#17] - -(20) CometHashAggregate -Input [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#17] -Keys [2]: [i_manufact_id#5, d_qoy#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] - -(21) CometExchange -Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] -Arguments: hashpartitioning(i_manufact_id#5, d_qoy#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] -Keys [2]: [i_manufact_id#5, d_qoy#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] - -(23) CometExchange -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: [i_manufact_id#5, sum_sales#20, _w0#21], [i_manufact_id#5 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 1] -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] - -(26) Window -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#22], [i_manufact_id#5] - -(27) Filter [codegen id : 2] -Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] -Condition : CASE WHEN (avg_quarterly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_quarterly_sales#22)) / avg_quarterly_sales#22) > 0.1000000000000000) ELSE false END - -(28) Project [codegen id : 2] -Output [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] -Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] - -(29) TakeOrderedAndProject -Input [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] -Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(32) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_qoy#17] - -(34) BroadcastExchange -Input [2]: [d_date_sk#15, d_qoy#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/extended.txt deleted file mode 100644 index 825b1ed81c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 0c123eb728..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (2) - Project [i_manufact_id,sum_sales,avg_quarterly_sales] - Filter [avg_quarterly_sales,sum_sales] - InputAdapter - Window [_w0,i_manufact_id] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_manufact_id,sum_sales,_w0] - CometExchange [i_manufact_id] #1 - CometHashAggregate [d_qoy,sum] [i_manufact_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_manufact_id,d_qoy] #2 - CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] - CometProject [i_manufact_id,ss_sales_price,d_qoy] - 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 [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,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_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [d_date_sk,d_qoy] #5 - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/explain.txt deleted file mode 100644 index e362f08727..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/explain.txt +++ /dev/null @@ -1,501 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (61) -+- * HashAggregate (60) - +- * CometColumnarToRow (59) - +- CometColumnarExchange (58) - +- * HashAggregate (57) - +- * HashAggregate (56) - +- * CometColumnarToRow (55) - +- CometColumnarExchange (54) - +- * HashAggregate (53) - +- * Project (52) - +- * BroadcastHashJoin Inner BuildRight (51) - :- * Project (49) - : +- * BroadcastHashJoin Inner BuildRight (48) - : :- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * CometColumnarToRow (29) - : : : : +- CometHashAggregate (28) - : : : : +- CometColumnarExchange (27) - : : : : +- * HashAggregate (26) - : : : : +- * Project (25) - : : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : : :- * Project (19) - : : : : : +- * BroadcastHashJoin Inner BuildRight (18) - : : : : : :- * Project (16) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : : : : :- Union (9) - : : : : : : : :- * Project (4) - : : : : : : : : +- * Filter (3) - : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : +- * Project (8) - : : : : : : : +- * Filter (7) - : : : : : : : +- * ColumnarToRow (6) - : : : : : : : +- Scan parquet spark_catalog.default.web_sales (5) - : : : : : : +- BroadcastExchange (14) - : : : : : : +- * CometColumnarToRow (13) - : : : : : : +- CometProject (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) - : : : : : +- ReusedExchange (17) - : : : : +- BroadcastExchange (23) - : : : : +- * CometColumnarToRow (22) - : : : : +- CometFilter (21) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (20) - : : : +- BroadcastExchange (33) - : : : +- * Filter (32) - : : : +- * ColumnarToRow (31) - : : : +- Scan parquet spark_catalog.default.store_sales (30) - : : +- BroadcastExchange (40) - : : +- * CometColumnarToRow (39) - : : +- CometProject (38) - : : +- CometFilter (37) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (36) - : +- BroadcastExchange (47) - : +- * CometColumnarToRow (46) - : +- CometProject (45) - : +- CometFilter (44) - : +- CometNativeScan parquet spark_catalog.default.store (43) - +- ReusedExchange (50) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] - -(3) Filter [codegen id : 1] -Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_bill_customer_sk#1)) - -(4) Project [codegen id : 1] -Output [3]: [cs_sold_date_sk#3 AS sold_date_sk#5, cs_bill_customer_sk#1 AS customer_sk#6, cs_item_sk#2 AS item_sk#7] -Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] - -(5) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#10), dynamicpruningexpression(ws_sold_date_sk#10 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 2] -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] - -(7) Filter [codegen id : 2] -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Condition : (isnotnull(ws_item_sk#8) AND isnotnull(ws_bill_customer_sk#9)) - -(8) Project [codegen id : 2] -Output [3]: [ws_sold_date_sk#10 AS sold_date_sk#11, ws_bill_customer_sk#9 AS customer_sk#12, ws_item_sk#8 AS item_sk#13] -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] - -(9) Union - -(10) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_class#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), IsNotNull(i_item_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Condition : ((((isnotnull(i_category#16) AND isnotnull(i_class#15)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#16, 50)) = Women )) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#15, 50)) = maternity )) AND isnotnull(i_item_sk#14)) - -(12) CometProject -Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Arguments: [i_item_sk#14], [i_item_sk#14] - -(13) CometColumnarToRow [codegen id : 3] -Input [1]: [i_item_sk#14] - -(14) BroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(15) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [item_sk#7] -Right keys [1]: [i_item_sk#14] -Join type: Inner -Join condition: None - -(16) Project [codegen id : 6] -Output [2]: [sold_date_sk#5, customer_sk#6] -Input [4]: [sold_date_sk#5, customer_sk#6, item_sk#7, i_item_sk#14] - -(17) ReusedExchange [Reuses operator id: 66] -Output [1]: [d_date_sk#17] - -(18) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sold_date_sk#5] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 6] -Output [1]: [customer_sk#6] -Input [3]: [sold_date_sk#5, customer_sk#6, d_date_sk#17] - -(20) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#18, c_current_addr_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Condition : (isnotnull(c_customer_sk#18) AND isnotnull(c_current_addr_sk#19)) - -(22) CometColumnarToRow [codegen id : 5] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] - -(23) BroadcastExchange -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [customer_sk#6] -Right keys [1]: [c_customer_sk#18] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [2]: [c_customer_sk#18, c_current_addr_sk#19] -Input [3]: [customer_sk#6, c_customer_sk#18, c_current_addr_sk#19] - -(26) HashAggregate [codegen id : 6] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Keys [2]: [c_customer_sk#18, c_current_addr_sk#19] -Functions: [] -Aggregate Attributes: [] -Results [2]: [c_customer_sk#18, c_current_addr_sk#19] - -(27) CometColumnarExchange -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Arguments: hashpartitioning(c_customer_sk#18, c_current_addr_sk#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometHashAggregate -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Keys [2]: [c_customer_sk#18, c_current_addr_sk#19] -Functions: [] - -(29) CometColumnarToRow [codegen id : 11] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] - -(30) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 7] -Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] - -(32) Filter [codegen id : 7] -Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#20) - -(33) BroadcastExchange -Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - -(34) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_customer_sk#18] -Right keys [1]: [ss_customer_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 11] -Output [4]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22] -Input [5]: [c_customer_sk#18, c_current_addr_sk#19, ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] - -(36) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#24, ca_county#25, ca_state#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] -Condition : ((isnotnull(ca_address_sk#24) AND isnotnull(ca_county#25)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#26, 2)))) - -(38) CometProject -Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] -Arguments: [ca_address_sk#24, ca_county#25, ca_state#27], [ca_address_sk#24, ca_county#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#26, 2)) AS ca_state#27] - -(39) CometColumnarToRow [codegen id : 8] -Input [3]: [ca_address_sk#24, ca_county#25, ca_state#27] - -(40) BroadcastExchange -Input [3]: [ca_address_sk#24, ca_county#25, ca_state#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(41) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#19] -Right keys [1]: [ca_address_sk#24] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 11] -Output [5]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#27] -Input [7]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_address_sk#24, ca_county#25, ca_state#27] - -(43) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_county#28, s_state#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county)] -ReadSchema: struct - -(44) CometFilter -Input [2]: [s_county#28, s_state#29] -Condition : (isnotnull(s_county#28) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#29, 2)))) - -(45) CometProject -Input [2]: [s_county#28, s_state#29] -Arguments: [s_county#28, s_state#30], [s_county#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#29, 2)) AS s_state#30] - -(46) CometColumnarToRow [codegen id : 9] -Input [2]: [s_county#28, s_state#30] - -(47) BroadcastExchange -Input [2]: [s_county#28, s_state#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true]),false), [plan_id=6] - -(48) BroadcastHashJoin [codegen id : 11] -Left keys [2]: [ca_county#25, ca_state#27] -Right keys [2]: [s_county#28, s_state#30] -Join type: Inner -Join condition: None - -(49) Project [codegen id : 11] -Output [3]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22] -Input [7]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#27, s_county#28, s_state#30] - -(50) ReusedExchange [Reuses operator id: 71] -Output [1]: [d_date_sk#31] - -(51) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#22] -Right keys [1]: [d_date_sk#31] -Join type: Inner -Join condition: None - -(52) Project [codegen id : 11] -Output [2]: [c_customer_sk#18, ss_ext_sales_price#21] -Input [4]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, d_date_sk#31] - -(53) HashAggregate [codegen id : 11] -Input [2]: [c_customer_sk#18, ss_ext_sales_price#21] -Keys [1]: [c_customer_sk#18] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#21))] -Aggregate Attributes [1]: [sum#32] -Results [2]: [c_customer_sk#18, sum#33] - -(54) CometColumnarExchange -Input [2]: [c_customer_sk#18, sum#33] -Arguments: hashpartitioning(c_customer_sk#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(55) CometColumnarToRow [codegen id : 12] -Input [2]: [c_customer_sk#18, sum#33] - -(56) HashAggregate [codegen id : 12] -Input [2]: [c_customer_sk#18, sum#33] -Keys [1]: [c_customer_sk#18] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#21))#34] -Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#21))#34,17,2) / 50) as int) AS segment#35] - -(57) HashAggregate [codegen id : 12] -Input [1]: [segment#35] -Keys [1]: [segment#35] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#36] -Results [2]: [segment#35, count#37] - -(58) CometColumnarExchange -Input [2]: [segment#35, count#37] -Arguments: hashpartitioning(segment#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(59) CometColumnarToRow [codegen id : 13] -Input [2]: [segment#35, count#37] - -(60) HashAggregate [codegen id : 13] -Input [2]: [segment#35, count#37] -Keys [1]: [segment#35] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#38] -Results [3]: [segment#35, count(1)#38 AS num_customers#39, (segment#35 * 50) AS segment_base#40] - -(61) TakeOrderedAndProject -Input [3]: [segment#35, num_customers#39, segment_base#40] -Arguments: 100, [segment#35 ASC NULLS FIRST, num_customers#39 ASC NULLS FIRST], [segment#35, num_customers#39, segment_base#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (66) -+- * CometColumnarToRow (65) - +- CometProject (64) - +- CometFilter (63) - +- CometNativeScan parquet spark_catalog.default.date_dim (62) - - -(62) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#41, d_moy#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(63) CometFilter -Input [3]: [d_date_sk#17, d_year#41, d_moy#42] -Condition : ((((isnotnull(d_moy#42) AND isnotnull(d_year#41)) AND (d_moy#42 = 12)) AND (d_year#41 = 1998)) AND isnotnull(d_date_sk#17)) - -(64) CometProject -Input [3]: [d_date_sk#17, d_year#41, d_moy#42] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(65) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] - -(66) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 - -Subquery:3 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometProject (69) - +- CometFilter (68) - +- CometNativeScan parquet spark_catalog.default.date_dim (67) - - -(67) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#31, d_month_seq#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,ScalarSubquery#44), LessThanOrEqual(d_month_seq,ScalarSubquery#45), IsNotNull(d_date_sk)] -ReadSchema: struct - -(68) CometFilter -Input [2]: [d_date_sk#31, d_month_seq#43] -Condition : (((isnotnull(d_month_seq#43) AND (d_month_seq#43 >= ReusedSubquery Subquery scalar-subquery#44, [id=#46])) AND (d_month_seq#43 <= ReusedSubquery Subquery scalar-subquery#45, [id=#47])) AND isnotnull(d_date_sk#31)) - -(69) CometProject -Input [2]: [d_date_sk#31, d_month_seq#43] -Arguments: [d_date_sk#31], [d_date_sk#31] - -(70) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#31] - -(71) BroadcastExchange -Input [1]: [d_date_sk#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] - -Subquery:4 Hosting operator id = 68 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#46] - -Subquery:5 Hosting operator id = 68 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#47] - -Subquery:6 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#44, [id=#46] -* CometColumnarToRow (78) -+- CometHashAggregate (77) - +- CometExchange (76) - +- CometHashAggregate (75) - +- CometProject (74) - +- CometFilter (73) - +- CometNativeScan parquet spark_catalog.default.date_dim (72) - - -(72) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#48, d_year#49, d_moy#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] -ReadSchema: struct - -(73) CometFilter -Input [3]: [d_month_seq#48, d_year#49, d_moy#50] -Condition : (((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 1998)) AND (d_moy#50 = 12)) - -(74) CometProject -Input [3]: [d_month_seq#48, d_year#49, d_moy#50] -Arguments: [(d_month_seq + 1)#51], [(d_month_seq#48 + 1) AS (d_month_seq + 1)#51] - -(75) CometHashAggregate -Input [1]: [(d_month_seq + 1)#51] -Keys [1]: [(d_month_seq + 1)#51] -Functions: [] - -(76) CometExchange -Input [1]: [(d_month_seq + 1)#51] -Arguments: hashpartitioning((d_month_seq + 1)#51, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(77) CometHashAggregate -Input [1]: [(d_month_seq + 1)#51] -Keys [1]: [(d_month_seq + 1)#51] -Functions: [] - -(78) CometColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 1)#51] - -Subquery:7 Hosting operator id = 67 Hosting Expression = Subquery scalar-subquery#45, [id=#47] -* CometColumnarToRow (85) -+- CometHashAggregate (84) - +- CometExchange (83) - +- CometHashAggregate (82) - +- CometProject (81) - +- CometFilter (80) - +- CometNativeScan parquet spark_catalog.default.date_dim (79) - - -(79) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#52, d_year#53, d_moy#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] -ReadSchema: struct - -(80) CometFilter -Input [3]: [d_month_seq#52, d_year#53, d_moy#54] -Condition : (((isnotnull(d_year#53) AND isnotnull(d_moy#54)) AND (d_year#53 = 1998)) AND (d_moy#54 = 12)) - -(81) CometProject -Input [3]: [d_month_seq#52, d_year#53, d_moy#54] -Arguments: [(d_month_seq + 3)#55], [(d_month_seq#52 + 3) AS (d_month_seq + 3)#55] - -(82) CometHashAggregate -Input [1]: [(d_month_seq + 3)#55] -Keys [1]: [(d_month_seq + 3)#55] -Functions: [] - -(83) CometExchange -Input [1]: [(d_month_seq + 3)#55] -Arguments: hashpartitioning((d_month_seq + 3)#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(84) CometHashAggregate -Input [1]: [(d_month_seq + 3)#55] -Keys [1]: [(d_month_seq + 3)#55] -Functions: [] - -(85) CometColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 3)#55] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/simplified.txt deleted file mode 100644 index 6e6245b520..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/simplified.txt +++ /dev/null @@ -1,129 +0,0 @@ -TakeOrderedAndProject [segment,num_customers,segment_base] - WholeStageCodegen (13) - HashAggregate [segment,count] [count(1),num_customers,segment_base,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [segment] #1 - WholeStageCodegen (12) - HashAggregate [segment] [count,count] - HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_sk] #2 - WholeStageCodegen (11) - HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] - Project [c_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ca_county,ca_state,s_county,s_state] - Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometHashAggregate [c_customer_sk,c_current_addr_sk] - CometColumnarExchange [c_customer_sk,c_current_addr_sk] #3 - WholeStageCodegen (6) - HashAggregate [c_customer_sk,c_current_addr_sk] - Project [c_customer_sk,c_current_addr_sk] - BroadcastHashJoin [customer_sk,c_customer_sk] - Project [customer_sk] - BroadcastHashJoin [sold_date_sk,d_date_sk] - Project [sold_date_sk,customer_sk] - BroadcastHashJoin [item_sk,i_item_sk] - InputAdapter - Union - WholeStageCodegen (1) - Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] - Filter [cs_item_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (2) - Project [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] - Filter [ws_item_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [(d_month_seq + 1)] #3 - ReusedSubquery [(d_month_seq + 3)] #4 - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometExchange [(d_month_seq + 1)] #9 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #10 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_county,ca_state] - CometFilter [ca_address_sk,ca_county,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_county,s_state] - CometFilter [s_county,s_state] - CometNativeScan parquet spark_catalog.default.store [s_county,s_state] - InputAdapter - ReusedExchange [d_date_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/explain.txt deleted file mode 100644 index 9eb7865391..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,496 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (58) -+- CometTakeOrderedAndProject (57) - +- CometHashAggregate (56) - +- CometExchange (55) - +- CometHashAggregate (54) - +- CometHashAggregate (53) - +- CometExchange (52) - +- CometHashAggregate (51) - +- CometProject (50) - +- CometBroadcastHashJoin (49) - :- CometProject (44) - : +- CometBroadcastHashJoin (43) - : :- CometProject (38) - : : +- CometBroadcastHashJoin (37) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometHashAggregate (27) - : : : : +- CometExchange (26) - : : : : +- CometHashAggregate (25) - : : : : +- CometProject (24) - : : : : +- CometBroadcastHashJoin (23) - : : : : :- CometProject (19) - : : : : : +- CometBroadcastHashJoin (18) - : : : : : :- CometProject (13) - : : : : : : +- CometBroadcastHashJoin (12) - : : : : : : :- CometUnion (7) - : : : : : : : :- CometProject (3) - : : : : : : : : +- CometFilter (2) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : +- CometProject (6) - : : : : : : : +- CometFilter (5) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (4) - : : : : : : +- CometBroadcastExchange (11) - : : : : : : +- CometProject (10) - : : : : : : +- CometFilter (9) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (8) - : : : : : +- CometBroadcastExchange (17) - : : : : : +- CometProject (16) - : : : : : +- CometFilter (15) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - : : : : +- CometBroadcastExchange (22) - : : : : +- CometFilter (21) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (20) - : : : +- CometBroadcastExchange (30) - : : : +- CometFilter (29) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (28) - : : +- CometBroadcastExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (33) - : +- CometBroadcastExchange (42) - : +- CometProject (41) - : +- CometFilter (40) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (39) - +- CometBroadcastExchange (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_bill_customer_sk#1)) - -(3) CometProject -Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Arguments: [sold_date_sk#5, customer_sk#6, item_sk#7], [cs_sold_date_sk#3 AS sold_date_sk#5, cs_bill_customer_sk#1 AS customer_sk#6, cs_item_sk#2 AS item_sk#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#10), dynamicpruningexpression(ws_sold_date_sk#10 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Condition : (isnotnull(ws_item_sk#8) AND isnotnull(ws_bill_customer_sk#9)) - -(6) CometProject -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Arguments: [sold_date_sk#11, customer_sk#12, item_sk#13], [ws_sold_date_sk#10 AS sold_date_sk#11, ws_bill_customer_sk#9 AS customer_sk#12, ws_item_sk#8 AS item_sk#13] - -(7) CometUnion -Child 0 Input [3]: [sold_date_sk#5, customer_sk#6, item_sk#7] -Child 1 Input [3]: [sold_date_sk#11, customer_sk#12, item_sk#13] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_class#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), IsNotNull(i_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Condition : ((((isnotnull(i_category#16) AND isnotnull(i_class#15)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#16, 50)) = Women )) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#15, 50)) = maternity )) AND isnotnull(i_item_sk#14)) - -(10) CometProject -Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Arguments: [i_item_sk#14], [i_item_sk#14] - -(11) CometBroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: [i_item_sk#14] - -(12) CometBroadcastHashJoin -Left output [3]: [sold_date_sk#5, customer_sk#6, item_sk#7] -Right output [1]: [i_item_sk#14] -Arguments: [item_sk#7], [i_item_sk#14], Inner, BuildRight - -(13) CometProject -Input [4]: [sold_date_sk#5, customer_sk#6, item_sk#7, i_item_sk#14] -Arguments: [sold_date_sk#5, customer_sk#6], [sold_date_sk#5, customer_sk#6] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12)) AND (d_year#18 = 1998)) AND isnotnull(d_date_sk#17)) - -(16) CometProject -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: [d_date_sk#17] - -(18) CometBroadcastHashJoin -Left output [2]: [sold_date_sk#5, customer_sk#6] -Right output [1]: [d_date_sk#17] -Arguments: [sold_date_sk#5], [d_date_sk#17], Inner, BuildRight - -(19) CometProject -Input [3]: [sold_date_sk#5, customer_sk#6, d_date_sk#17] -Arguments: [customer_sk#6], [customer_sk#6] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#20, c_current_addr_sk#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Condition : (isnotnull(c_customer_sk#20) AND isnotnull(c_current_addr_sk#21)) - -(22) CometBroadcastExchange -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Arguments: [c_customer_sk#20, c_current_addr_sk#21] - -(23) CometBroadcastHashJoin -Left output [1]: [customer_sk#6] -Right output [2]: [c_customer_sk#20, c_current_addr_sk#21] -Arguments: [customer_sk#6], [c_customer_sk#20], Inner, BuildRight - -(24) CometProject -Input [3]: [customer_sk#6, c_customer_sk#20, c_current_addr_sk#21] -Arguments: [c_customer_sk#20, c_current_addr_sk#21], [c_customer_sk#20, c_current_addr_sk#21] - -(25) CometHashAggregate -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Keys [2]: [c_customer_sk#20, c_current_addr_sk#21] -Functions: [] - -(26) CometExchange -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Arguments: hashpartitioning(c_customer_sk#20, c_current_addr_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Keys [2]: [c_customer_sk#20, c_current_addr_sk#21] -Functions: [] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#24), dynamicpruningexpression(ss_sold_date_sk#24 IN dynamicpruning#25)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(29) CometFilter -Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Condition : isnotnull(ss_customer_sk#22) - -(30) CometBroadcastExchange -Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Arguments: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] - -(31) CometBroadcastHashJoin -Left output [2]: [c_customer_sk#20, c_current_addr_sk#21] -Right output [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Arguments: [c_customer_sk#20], [ss_customer_sk#22], Inner, BuildRight - -(32) CometProject -Input [5]: [c_customer_sk#20, c_current_addr_sk#21, ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Arguments: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24], [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#26, ca_county#27, ca_state#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct - -(34) CometFilter -Input [3]: [ca_address_sk#26, ca_county#27, ca_state#28] -Condition : ((isnotnull(ca_address_sk#26) AND isnotnull(ca_county#27)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#28, 2)))) - -(35) CometProject -Input [3]: [ca_address_sk#26, ca_county#27, ca_state#28] -Arguments: [ca_address_sk#26, ca_county#27, ca_state#29], [ca_address_sk#26, ca_county#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#28, 2)) AS ca_state#29] - -(36) CometBroadcastExchange -Input [3]: [ca_address_sk#26, ca_county#27, ca_state#29] -Arguments: [ca_address_sk#26, ca_county#27, ca_state#29] - -(37) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24] -Right output [3]: [ca_address_sk#26, ca_county#27, ca_state#29] -Arguments: [c_current_addr_sk#21], [ca_address_sk#26], Inner, BuildRight - -(38) CometProject -Input [7]: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_address_sk#26, ca_county#27, ca_state#29] -Arguments: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29], [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_county#30, s_state#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county)] -ReadSchema: struct - -(40) CometFilter -Input [2]: [s_county#30, s_state#31] -Condition : (isnotnull(s_county#30) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#31, 2)))) - -(41) CometProject -Input [2]: [s_county#30, s_state#31] -Arguments: [s_county#30, s_state#32], [s_county#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#31, 2)) AS s_state#32] - -(42) CometBroadcastExchange -Input [2]: [s_county#30, s_state#32] -Arguments: [s_county#30, s_state#32] - -(43) CometBroadcastHashJoin -Left output [5]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29] -Right output [2]: [s_county#30, s_state#32] -Arguments: [ca_county#27, ca_state#29], [s_county#30, s_state#32], Inner, BuildRight - -(44) CometProject -Input [7]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29, s_county#30, s_state#32] -Arguments: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24], [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_month_seq#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,ScalarSubquery#35), LessThanOrEqual(d_month_seq,ScalarSubquery#36), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [d_date_sk#33, d_month_seq#34] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= ReusedSubquery Subquery scalar-subquery#35, [id=#37])) AND (d_month_seq#34 <= ReusedSubquery Subquery scalar-subquery#36, [id=#38])) AND isnotnull(d_date_sk#33)) - -(47) CometProject -Input [2]: [d_date_sk#33, d_month_seq#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(48) CometBroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: [d_date_sk#33] - -(49) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] -Right output [1]: [d_date_sk#33] -Arguments: [ss_sold_date_sk#24], [d_date_sk#33], Inner, BuildRight - -(50) CometProject -Input [4]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, d_date_sk#33] -Arguments: [c_customer_sk#20, ss_ext_sales_price#23], [c_customer_sk#20, ss_ext_sales_price#23] - -(51) CometHashAggregate -Input [2]: [c_customer_sk#20, ss_ext_sales_price#23] -Keys [1]: [c_customer_sk#20] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#23))] - -(52) CometExchange -Input [2]: [c_customer_sk#20, sum#39] -Arguments: hashpartitioning(c_customer_sk#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(53) CometHashAggregate -Input [2]: [c_customer_sk#20, sum#39] -Keys [1]: [c_customer_sk#20] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#23))] - -(54) CometHashAggregate -Input [1]: [segment#40] -Keys [1]: [segment#40] -Functions [1]: [partial_count(1)] - -(55) CometExchange -Input [2]: [segment#40, count#41] -Arguments: hashpartitioning(segment#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(56) CometHashAggregate -Input [2]: [segment#40, count#41] -Keys [1]: [segment#40] -Functions [1]: [count(1)] - -(57) CometTakeOrderedAndProject -Input [3]: [segment#40, num_customers#42, segment_base#43] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[segment#40 ASC NULLS FIRST,num_customers#42 ASC NULLS FIRST], output=[segment#40,num_customers#42,segment_base#43]), [segment#40, num_customers#42, segment_base#43], 100, 0, [segment#40 ASC NULLS FIRST, num_customers#42 ASC NULLS FIRST], [segment#40, num_customers#42, segment_base#43] - -(58) CometColumnarToRow [codegen id : 1] -Input [3]: [segment#40, num_customers#42, segment_base#43] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (63) -+- * CometColumnarToRow (62) - +- CometProject (61) - +- CometFilter (60) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (59) - - -(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(60) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12)) AND (d_year#18 = 1998)) AND isnotnull(d_date_sk#17)) - -(61) CometProject -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(62) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] - -(63) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 - -Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 -BroadcastExchange (68) -+- * CometColumnarToRow (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (64) - - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_month_seq#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,ScalarSubquery#35), LessThanOrEqual(d_month_seq,ScalarSubquery#36), IsNotNull(d_date_sk)] -ReadSchema: struct - -(65) CometFilter -Input [2]: [d_date_sk#33, d_month_seq#34] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= ReusedSubquery Subquery scalar-subquery#35, [id=#37])) AND (d_month_seq#34 <= ReusedSubquery Subquery scalar-subquery#36, [id=#38])) AND isnotnull(d_date_sk#33)) - -(66) CometProject -Input [2]: [d_date_sk#33, d_month_seq#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(67) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#33] - -(68) BroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:4 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#37] - -Subquery:5 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#36, [id=#38] - -Subquery:6 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#35, [id=#37] -* CometColumnarToRow (75) -+- CometHashAggregate (74) - +- CometExchange (73) - +- CometHashAggregate (72) - +- CometProject (71) - +- CometFilter (70) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (69) - - -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#44, d_year#45, d_moy#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] -ReadSchema: struct - -(70) CometFilter -Input [3]: [d_month_seq#44, d_year#45, d_moy#46] -Condition : (((isnotnull(d_year#45) AND isnotnull(d_moy#46)) AND (d_year#45 = 1998)) AND (d_moy#46 = 12)) - -(71) CometProject -Input [3]: [d_month_seq#44, d_year#45, d_moy#46] -Arguments: [(d_month_seq + 1)#47], [(d_month_seq#44 + 1) AS (d_month_seq + 1)#47] - -(72) CometHashAggregate -Input [1]: [(d_month_seq + 1)#47] -Keys [1]: [(d_month_seq + 1)#47] -Functions: [] - -(73) CometExchange -Input [1]: [(d_month_seq + 1)#47] -Arguments: hashpartitioning((d_month_seq + 1)#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(74) CometHashAggregate -Input [1]: [(d_month_seq + 1)#47] -Keys [1]: [(d_month_seq + 1)#47] -Functions: [] - -(75) CometColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 1)#47] - -Subquery:7 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#36, [id=#38] -* CometColumnarToRow (82) -+- CometHashAggregate (81) - +- CometExchange (80) - +- CometHashAggregate (79) - +- CometProject (78) - +- CometFilter (77) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) - - -(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#48, d_year#49, d_moy#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] -ReadSchema: struct - -(77) CometFilter -Input [3]: [d_month_seq#48, d_year#49, d_moy#50] -Condition : (((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 1998)) AND (d_moy#50 = 12)) - -(78) CometProject -Input [3]: [d_month_seq#48, d_year#49, d_moy#50] -Arguments: [(d_month_seq + 3)#51], [(d_month_seq#48 + 3) AS (d_month_seq + 3)#51] - -(79) CometHashAggregate -Input [1]: [(d_month_seq + 3)#51] -Keys [1]: [(d_month_seq + 3)#51] -Functions: [] - -(80) CometExchange -Input [1]: [(d_month_seq + 3)#51] -Arguments: hashpartitioning((d_month_seq + 3)#51, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(81) CometHashAggregate -Input [1]: [(d_month_seq + 3)#51] -Keys [1]: [(d_month_seq + 3)#51] -Functions: [] - -(82) CometColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 3)#51] - -Subquery:8 Hosting operator id = 46 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#37] - -Subquery:9 Hosting operator id = 46 Hosting Expression = ReusedSubquery Subquery scalar-subquery#36, [id=#38] - -Subquery:10 Hosting operator id = 45 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#37] - -Subquery:11 Hosting operator id = 45 Hosting Expression = ReusedSubquery Subquery scalar-subquery#36, [id=#38] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/simplified.txt deleted file mode 100644 index ce1acded44..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,103 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [segment,num_customers,segment_base] - CometHashAggregate [count] [segment,num_customers,segment_base,count(1)] - CometExchange [segment] #1 - CometHashAggregate [segment,count] - CometHashAggregate [c_customer_sk,sum] [segment,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [c_customer_sk] #2 - CometHashAggregate [ss_ext_sales_price] [c_customer_sk,sum] - CometProject [c_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state,s_county,s_state] - CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] - CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk,ca_address_sk,ca_county,ca_state] - CometProject [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometHashAggregate [c_customer_sk,c_current_addr_sk] - CometExchange [c_customer_sk,c_current_addr_sk] #3 - CometHashAggregate [c_customer_sk,c_current_addr_sk] - CometProject [c_customer_sk,c_current_addr_sk] - CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] - CometProject [customer_sk] - CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] - CometProject [sold_date_sk,customer_sk] - 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_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [i_item_sk] #5 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] #8 - CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [(d_month_seq + 1)] #3 - ReusedSubquery [(d_month_seq + 3)] #4 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometExchange [(d_month_seq + 1)] #10 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #11 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state] #12 - CometProject [ca_state] [ca_address_sk,ca_county,ca_state] - CometFilter [ca_address_sk,ca_county,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] - CometBroadcastExchange [s_county,s_state] #13 - CometProject [s_state] [s_county,s_state] - CometFilter [s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_county,s_state] - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [(d_month_seq + 1)] #3 - ReusedSubquery [(d_month_seq + 3)] #4 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - ReusedSubquery [(d_month_seq + 1)] #3 - ReusedSubquery [(d_month_seq + 3)] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/explain.txt deleted file mode 100644 index 9eb7865391..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/explain.txt +++ /dev/null @@ -1,496 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (58) -+- CometTakeOrderedAndProject (57) - +- CometHashAggregate (56) - +- CometExchange (55) - +- CometHashAggregate (54) - +- CometHashAggregate (53) - +- CometExchange (52) - +- CometHashAggregate (51) - +- CometProject (50) - +- CometBroadcastHashJoin (49) - :- CometProject (44) - : +- CometBroadcastHashJoin (43) - : :- CometProject (38) - : : +- CometBroadcastHashJoin (37) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometHashAggregate (27) - : : : : +- CometExchange (26) - : : : : +- CometHashAggregate (25) - : : : : +- CometProject (24) - : : : : +- CometBroadcastHashJoin (23) - : : : : :- CometProject (19) - : : : : : +- CometBroadcastHashJoin (18) - : : : : : :- CometProject (13) - : : : : : : +- CometBroadcastHashJoin (12) - : : : : : : :- CometUnion (7) - : : : : : : : :- CometProject (3) - : : : : : : : : +- CometFilter (2) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : +- CometProject (6) - : : : : : : : +- CometFilter (5) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (4) - : : : : : : +- CometBroadcastExchange (11) - : : : : : : +- CometProject (10) - : : : : : : +- CometFilter (9) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (8) - : : : : : +- CometBroadcastExchange (17) - : : : : : +- CometProject (16) - : : : : : +- CometFilter (15) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - : : : : +- CometBroadcastExchange (22) - : : : : +- CometFilter (21) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (20) - : : : +- CometBroadcastExchange (30) - : : : +- CometFilter (29) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (28) - : : +- CometBroadcastExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (33) - : +- CometBroadcastExchange (42) - : +- CometProject (41) - : +- CometFilter (40) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (39) - +- CometBroadcastExchange (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_bill_customer_sk#1)) - -(3) CometProject -Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Arguments: [sold_date_sk#5, customer_sk#6, item_sk#7], [cs_sold_date_sk#3 AS sold_date_sk#5, cs_bill_customer_sk#1 AS customer_sk#6, cs_item_sk#2 AS item_sk#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#10), dynamicpruningexpression(ws_sold_date_sk#10 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Condition : (isnotnull(ws_item_sk#8) AND isnotnull(ws_bill_customer_sk#9)) - -(6) CometProject -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Arguments: [sold_date_sk#11, customer_sk#12, item_sk#13], [ws_sold_date_sk#10 AS sold_date_sk#11, ws_bill_customer_sk#9 AS customer_sk#12, ws_item_sk#8 AS item_sk#13] - -(7) CometUnion -Child 0 Input [3]: [sold_date_sk#5, customer_sk#6, item_sk#7] -Child 1 Input [3]: [sold_date_sk#11, customer_sk#12, item_sk#13] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_class#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), IsNotNull(i_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Condition : ((((isnotnull(i_category#16) AND isnotnull(i_class#15)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#16, 50)) = Women )) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#15, 50)) = maternity )) AND isnotnull(i_item_sk#14)) - -(10) CometProject -Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Arguments: [i_item_sk#14], [i_item_sk#14] - -(11) CometBroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: [i_item_sk#14] - -(12) CometBroadcastHashJoin -Left output [3]: [sold_date_sk#5, customer_sk#6, item_sk#7] -Right output [1]: [i_item_sk#14] -Arguments: [item_sk#7], [i_item_sk#14], Inner, BuildRight - -(13) CometProject -Input [4]: [sold_date_sk#5, customer_sk#6, item_sk#7, i_item_sk#14] -Arguments: [sold_date_sk#5, customer_sk#6], [sold_date_sk#5, customer_sk#6] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12)) AND (d_year#18 = 1998)) AND isnotnull(d_date_sk#17)) - -(16) CometProject -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: [d_date_sk#17] - -(18) CometBroadcastHashJoin -Left output [2]: [sold_date_sk#5, customer_sk#6] -Right output [1]: [d_date_sk#17] -Arguments: [sold_date_sk#5], [d_date_sk#17], Inner, BuildRight - -(19) CometProject -Input [3]: [sold_date_sk#5, customer_sk#6, d_date_sk#17] -Arguments: [customer_sk#6], [customer_sk#6] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#20, c_current_addr_sk#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Condition : (isnotnull(c_customer_sk#20) AND isnotnull(c_current_addr_sk#21)) - -(22) CometBroadcastExchange -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Arguments: [c_customer_sk#20, c_current_addr_sk#21] - -(23) CometBroadcastHashJoin -Left output [1]: [customer_sk#6] -Right output [2]: [c_customer_sk#20, c_current_addr_sk#21] -Arguments: [customer_sk#6], [c_customer_sk#20], Inner, BuildRight - -(24) CometProject -Input [3]: [customer_sk#6, c_customer_sk#20, c_current_addr_sk#21] -Arguments: [c_customer_sk#20, c_current_addr_sk#21], [c_customer_sk#20, c_current_addr_sk#21] - -(25) CometHashAggregate -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Keys [2]: [c_customer_sk#20, c_current_addr_sk#21] -Functions: [] - -(26) CometExchange -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Arguments: hashpartitioning(c_customer_sk#20, c_current_addr_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Keys [2]: [c_customer_sk#20, c_current_addr_sk#21] -Functions: [] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#24), dynamicpruningexpression(ss_sold_date_sk#24 IN dynamicpruning#25)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(29) CometFilter -Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Condition : isnotnull(ss_customer_sk#22) - -(30) CometBroadcastExchange -Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Arguments: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] - -(31) CometBroadcastHashJoin -Left output [2]: [c_customer_sk#20, c_current_addr_sk#21] -Right output [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Arguments: [c_customer_sk#20], [ss_customer_sk#22], Inner, BuildRight - -(32) CometProject -Input [5]: [c_customer_sk#20, c_current_addr_sk#21, ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Arguments: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24], [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#26, ca_county#27, ca_state#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct - -(34) CometFilter -Input [3]: [ca_address_sk#26, ca_county#27, ca_state#28] -Condition : ((isnotnull(ca_address_sk#26) AND isnotnull(ca_county#27)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#28, 2)))) - -(35) CometProject -Input [3]: [ca_address_sk#26, ca_county#27, ca_state#28] -Arguments: [ca_address_sk#26, ca_county#27, ca_state#29], [ca_address_sk#26, ca_county#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#28, 2)) AS ca_state#29] - -(36) CometBroadcastExchange -Input [3]: [ca_address_sk#26, ca_county#27, ca_state#29] -Arguments: [ca_address_sk#26, ca_county#27, ca_state#29] - -(37) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24] -Right output [3]: [ca_address_sk#26, ca_county#27, ca_state#29] -Arguments: [c_current_addr_sk#21], [ca_address_sk#26], Inner, BuildRight - -(38) CometProject -Input [7]: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_address_sk#26, ca_county#27, ca_state#29] -Arguments: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29], [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_county#30, s_state#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county)] -ReadSchema: struct - -(40) CometFilter -Input [2]: [s_county#30, s_state#31] -Condition : (isnotnull(s_county#30) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#31, 2)))) - -(41) CometProject -Input [2]: [s_county#30, s_state#31] -Arguments: [s_county#30, s_state#32], [s_county#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#31, 2)) AS s_state#32] - -(42) CometBroadcastExchange -Input [2]: [s_county#30, s_state#32] -Arguments: [s_county#30, s_state#32] - -(43) CometBroadcastHashJoin -Left output [5]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29] -Right output [2]: [s_county#30, s_state#32] -Arguments: [ca_county#27, ca_state#29], [s_county#30, s_state#32], Inner, BuildRight - -(44) CometProject -Input [7]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29, s_county#30, s_state#32] -Arguments: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24], [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_month_seq#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,ScalarSubquery#35), LessThanOrEqual(d_month_seq,ScalarSubquery#36), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [d_date_sk#33, d_month_seq#34] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= ReusedSubquery Subquery scalar-subquery#35, [id=#37])) AND (d_month_seq#34 <= ReusedSubquery Subquery scalar-subquery#36, [id=#38])) AND isnotnull(d_date_sk#33)) - -(47) CometProject -Input [2]: [d_date_sk#33, d_month_seq#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(48) CometBroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: [d_date_sk#33] - -(49) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] -Right output [1]: [d_date_sk#33] -Arguments: [ss_sold_date_sk#24], [d_date_sk#33], Inner, BuildRight - -(50) CometProject -Input [4]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, d_date_sk#33] -Arguments: [c_customer_sk#20, ss_ext_sales_price#23], [c_customer_sk#20, ss_ext_sales_price#23] - -(51) CometHashAggregate -Input [2]: [c_customer_sk#20, ss_ext_sales_price#23] -Keys [1]: [c_customer_sk#20] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#23))] - -(52) CometExchange -Input [2]: [c_customer_sk#20, sum#39] -Arguments: hashpartitioning(c_customer_sk#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(53) CometHashAggregate -Input [2]: [c_customer_sk#20, sum#39] -Keys [1]: [c_customer_sk#20] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#23))] - -(54) CometHashAggregate -Input [1]: [segment#40] -Keys [1]: [segment#40] -Functions [1]: [partial_count(1)] - -(55) CometExchange -Input [2]: [segment#40, count#41] -Arguments: hashpartitioning(segment#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(56) CometHashAggregate -Input [2]: [segment#40, count#41] -Keys [1]: [segment#40] -Functions [1]: [count(1)] - -(57) CometTakeOrderedAndProject -Input [3]: [segment#40, num_customers#42, segment_base#43] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[segment#40 ASC NULLS FIRST,num_customers#42 ASC NULLS FIRST], output=[segment#40,num_customers#42,segment_base#43]), [segment#40, num_customers#42, segment_base#43], 100, 0, [segment#40 ASC NULLS FIRST, num_customers#42 ASC NULLS FIRST], [segment#40, num_customers#42, segment_base#43] - -(58) CometColumnarToRow [codegen id : 1] -Input [3]: [segment#40, num_customers#42, segment_base#43] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (63) -+- * CometColumnarToRow (62) - +- CometProject (61) - +- CometFilter (60) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (59) - - -(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(60) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12)) AND (d_year#18 = 1998)) AND isnotnull(d_date_sk#17)) - -(61) CometProject -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(62) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] - -(63) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 - -Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 -BroadcastExchange (68) -+- * CometColumnarToRow (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (64) - - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_month_seq#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,ScalarSubquery#35), LessThanOrEqual(d_month_seq,ScalarSubquery#36), IsNotNull(d_date_sk)] -ReadSchema: struct - -(65) CometFilter -Input [2]: [d_date_sk#33, d_month_seq#34] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= ReusedSubquery Subquery scalar-subquery#35, [id=#37])) AND (d_month_seq#34 <= ReusedSubquery Subquery scalar-subquery#36, [id=#38])) AND isnotnull(d_date_sk#33)) - -(66) CometProject -Input [2]: [d_date_sk#33, d_month_seq#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(67) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#33] - -(68) BroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:4 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#37] - -Subquery:5 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#36, [id=#38] - -Subquery:6 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#35, [id=#37] -* CometColumnarToRow (75) -+- CometHashAggregate (74) - +- CometExchange (73) - +- CometHashAggregate (72) - +- CometProject (71) - +- CometFilter (70) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (69) - - -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#44, d_year#45, d_moy#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] -ReadSchema: struct - -(70) CometFilter -Input [3]: [d_month_seq#44, d_year#45, d_moy#46] -Condition : (((isnotnull(d_year#45) AND isnotnull(d_moy#46)) AND (d_year#45 = 1998)) AND (d_moy#46 = 12)) - -(71) CometProject -Input [3]: [d_month_seq#44, d_year#45, d_moy#46] -Arguments: [(d_month_seq + 1)#47], [(d_month_seq#44 + 1) AS (d_month_seq + 1)#47] - -(72) CometHashAggregate -Input [1]: [(d_month_seq + 1)#47] -Keys [1]: [(d_month_seq + 1)#47] -Functions: [] - -(73) CometExchange -Input [1]: [(d_month_seq + 1)#47] -Arguments: hashpartitioning((d_month_seq + 1)#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(74) CometHashAggregate -Input [1]: [(d_month_seq + 1)#47] -Keys [1]: [(d_month_seq + 1)#47] -Functions: [] - -(75) CometColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 1)#47] - -Subquery:7 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#36, [id=#38] -* CometColumnarToRow (82) -+- CometHashAggregate (81) - +- CometExchange (80) - +- CometHashAggregate (79) - +- CometProject (78) - +- CometFilter (77) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) - - -(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#48, d_year#49, d_moy#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] -ReadSchema: struct - -(77) CometFilter -Input [3]: [d_month_seq#48, d_year#49, d_moy#50] -Condition : (((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 1998)) AND (d_moy#50 = 12)) - -(78) CometProject -Input [3]: [d_month_seq#48, d_year#49, d_moy#50] -Arguments: [(d_month_seq + 3)#51], [(d_month_seq#48 + 3) AS (d_month_seq + 3)#51] - -(79) CometHashAggregate -Input [1]: [(d_month_seq + 3)#51] -Keys [1]: [(d_month_seq + 3)#51] -Functions: [] - -(80) CometExchange -Input [1]: [(d_month_seq + 3)#51] -Arguments: hashpartitioning((d_month_seq + 3)#51, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(81) CometHashAggregate -Input [1]: [(d_month_seq + 3)#51] -Keys [1]: [(d_month_seq + 3)#51] -Functions: [] - -(82) CometColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 3)#51] - -Subquery:8 Hosting operator id = 46 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#37] - -Subquery:9 Hosting operator id = 46 Hosting Expression = ReusedSubquery Subquery scalar-subquery#36, [id=#38] - -Subquery:10 Hosting operator id = 45 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#37] - -Subquery:11 Hosting operator id = 45 Hosting Expression = ReusedSubquery Subquery scalar-subquery#36, [id=#38] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/extended.txt deleted file mode 100644 index a49bbabd10..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/extended.txt +++ /dev/null @@ -1,95 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometUnion - : : : : : : : :- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : :- ReusedSubquery - : : : : +- ReusedSubquery - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : :- ReusedSubquery - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- ReusedSubquery - +- ReusedSubquery - -Comet accelerated 75 out of 88 eligible operators (85%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index ce1acded44..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt +++ /dev/null @@ -1,103 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [segment,num_customers,segment_base] - CometHashAggregate [count] [segment,num_customers,segment_base,count(1)] - CometExchange [segment] #1 - CometHashAggregate [segment,count] - CometHashAggregate [c_customer_sk,sum] [segment,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [c_customer_sk] #2 - CometHashAggregate [ss_ext_sales_price] [c_customer_sk,sum] - CometProject [c_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state,s_county,s_state] - CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] - CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk,ca_address_sk,ca_county,ca_state] - CometProject [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometHashAggregate [c_customer_sk,c_current_addr_sk] - CometExchange [c_customer_sk,c_current_addr_sk] #3 - CometHashAggregate [c_customer_sk,c_current_addr_sk] - CometProject [c_customer_sk,c_current_addr_sk] - CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] - CometProject [customer_sk] - CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] - CometProject [sold_date_sk,customer_sk] - 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_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [i_item_sk] #5 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] #8 - CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [(d_month_seq + 1)] #3 - ReusedSubquery [(d_month_seq + 3)] #4 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometExchange [(d_month_seq + 1)] #10 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #11 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state] #12 - CometProject [ca_state] [ca_address_sk,ca_county,ca_state] - CometFilter [ca_address_sk,ca_county,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] - CometBroadcastExchange [s_county,s_state] #13 - CometProject [s_state] [s_county,s_state] - CometFilter [s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_county,s_state] - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [(d_month_seq + 1)] #3 - ReusedSubquery [(d_month_seq + 3)] #4 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - ReusedSubquery [(d_month_seq + 1)] #3 - ReusedSubquery [(d_month_seq + 3)] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_datafusion/explain.txt deleted file mode 100644 index d2f7dbb904..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_datafusion/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometNativeScan parquet spark_catalog.default.item (9) - - -(1) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1999)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1], [d_date_sk#1] - -(4) CometNativeScan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [1]: [d_date_sk#1] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [4]: [d_date_sk#1, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5], [ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,28), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 28)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#9, 50)) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [3]: [ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [2]: [i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [3]: [i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [i_brand#11, i_brand_id#8, sum#12] -Keys [2]: [i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [3]: [brand_id#13, brand#14, ext_price#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[brand_id#13,brand#14,ext_price#15]), [brand_id#13, brand#14, ext_price#15], 100, 0, [ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [brand_id#13, brand#14, ext_price#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [3]: [brand_id#13, brand#14, ext_price#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_datafusion/simplified.txt deleted file mode 100644 index a2adacc531..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_datafusion/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,ext_price] - CometHashAggregate [sum] [brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,sum] - CometProject [ss_ext_sales_price,i_brand_id,i_brand] - 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_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometNativeScan 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/q55.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/explain.txt deleted file mode 100644 index 08e1bae2ff..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1999)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1], [d_date_sk#1] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [1]: [d_date_sk#1] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [4]: [d_date_sk#1, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5], [ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,28), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 28)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#9, 50)) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [3]: [ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [2]: [i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [3]: [i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [i_brand#11, i_brand_id#8, sum#12] -Keys [2]: [i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [3]: [brand_id#13, brand#14, ext_price#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[brand_id#13,brand#14,ext_price#15]), [brand_id#13, brand#14, ext_price#15], 100, 0, [ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [brand_id#13, brand#14, ext_price#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [3]: [brand_id#13, brand#14, ext_price#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/simplified.txt deleted file mode 100644 index dd7658cbac..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,ext_price] - CometHashAggregate [sum] [brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,sum] - CometProject [ss_ext_sales_price,i_brand_id,i_brand] - 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_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan [native_iceberg_compat] 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/q55/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/explain.txt deleted file mode 100644 index 08e1bae2ff..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1999)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1], [d_date_sk#1] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [1]: [d_date_sk#1] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [4]: [d_date_sk#1, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5], [ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,28), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 28)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#9, 50)) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [3]: [ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [2]: [i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [3]: [i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [i_brand#11, i_brand_id#8, sum#12] -Keys [2]: [i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [3]: [brand_id#13, brand#14, ext_price#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[brand_id#13,brand#14,ext_price#15]), [brand_id#13, brand#14, ext_price#15], 100, 0, [ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [brand_id#13, brand#14, ext_price#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [3]: [brand_id#13, brand#14, ext_price#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/extended.txt deleted file mode 100644 index a292badf5d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index dd7658cbac..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,ext_price] - CometHashAggregate [sum] [brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,sum] - CometProject [ss_ext_sales_price,i_brand_id,i_brand] - 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_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/explain.txt deleted file mode 100644 index dab8a2806c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/explain.txt +++ /dev/null @@ -1,420 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (67) -+- * HashAggregate (66) - +- * CometColumnarToRow (65) - +- CometColumnarExchange (64) - +- * HashAggregate (63) - +- Union (62) - :- * HashAggregate (29) - : +- * CometColumnarToRow (28) - : +- CometColumnarExchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * CometColumnarToRow (22) - : +- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.item (14) - : +- CometBroadcastExchange (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometNativeScan parquet spark_catalog.default.item (16) - :- * HashAggregate (45) - : +- * CometColumnarToRow (44) - : +- CometColumnarExchange (43) - : +- * HashAggregate (42) - : +- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (38) - : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * Filter (32) - : : : : +- * ColumnarToRow (31) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (30) - : : : +- ReusedExchange (33) - : : +- ReusedExchange (36) - : +- ReusedExchange (39) - +- * HashAggregate (61) - +- * CometColumnarToRow (60) - +- CometColumnarExchange (59) - +- * HashAggregate (58) - +- * Project (57) - +- * BroadcastHashJoin Inner BuildRight (56) - :- * Project (54) - : +- * BroadcastHashJoin Inner BuildRight (53) - : :- * Project (51) - : : +- * BroadcastHashJoin Inner BuildRight (50) - : : :- * Filter (48) - : : : +- * ColumnarToRow (47) - : : : +- Scan parquet spark_catalog.default.web_sales (46) - : : +- ReusedExchange (49) - : +- ReusedExchange (52) - +- ReusedExchange (55) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(4) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_gmt_offset#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isnotnull(ca_address_sk#7)) - -(9) CometProject -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Arguments: [ca_address_sk#7], [ca_address_sk#7] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [ca_address_sk#7] - -(11) BroadcastExchange -Input [1]: [ca_address_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] - -(14) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#9, i_item_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [i_item_sk#9, i_item_id#10] -Condition : isnotnull(i_item_sk#9) - -(16) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_id#11, i_color#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(17) CometFilter -Input [2]: [i_item_id#11, i_color#12] -Condition : static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#12, 20)) IN (slate ,blanched ,burnished ) - -(18) CometProject -Input [2]: [i_item_id#11, i_color#12] -Arguments: [i_item_id#13], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#11, 16)) AS i_item_id#13] - -(19) CometBroadcastExchange -Input [1]: [i_item_id#13] -Arguments: [i_item_id#13] - -(20) CometBroadcastHashJoin -Left output [2]: [i_item_sk#9, i_item_id#10] -Right output [1]: [i_item_id#13] -Arguments: [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#10, 16))], [i_item_id#13], LeftSemi, BuildRight - -(21) CometProject -Input [2]: [i_item_sk#9, i_item_id#10] -Arguments: [i_item_sk#9, i_item_id#14], [i_item_sk#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#10, 16)) AS i_item_id#14] - -(22) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#9, i_item_id#14] - -(23) BroadcastExchange -Input [2]: [i_item_sk#9, i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#9] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 4] -Output [2]: [ss_ext_sales_price#3, i_item_id#14] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_item_id#14] - -(26) HashAggregate [codegen id : 4] -Input [2]: [ss_ext_sales_price#3, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum#15] -Results [2]: [i_item_id#14, sum#16] - -(27) CometColumnarExchange -Input [2]: [i_item_id#14, sum#16] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometColumnarToRow [codegen id : 5] -Input [2]: [i_item_id#14, sum#16] - -(29) HashAggregate [codegen id : 5] -Input [2]: [i_item_id#14, sum#16] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#17] -Results [2]: [i_item_id#14 AS i_item_id#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#17,17,2) AS total_sales#19] - -(30) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 9] -Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] - -(32) Filter [codegen id : 9] -Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -Condition : (isnotnull(cs_bill_addr_sk#20) AND isnotnull(cs_item_sk#21)) - -(33) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#24] - -(34) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 9] -Output [3]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22] -Input [5]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#24] - -(36) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#25] - -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_bill_addr_sk#20] -Right keys [1]: [ca_address_sk#25] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 9] -Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] -Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, ca_address_sk#25] - -(39) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#26, i_item_id#27] - -(40) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#21] -Right keys [1]: [i_item_sk#26] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 9] -Output [2]: [cs_ext_sales_price#22, i_item_id#27] -Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#26, i_item_id#27] - -(42) HashAggregate [codegen id : 9] -Input [2]: [cs_ext_sales_price#22, i_item_id#27] -Keys [1]: [i_item_id#27] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#27, sum#29] - -(43) CometColumnarExchange -Input [2]: [i_item_id#27, sum#29] -Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(44) CometColumnarToRow [codegen id : 10] -Input [2]: [i_item_id#27, sum#29] - -(45) HashAggregate [codegen id : 10] -Input [2]: [i_item_id#27, sum#29] -Keys [1]: [i_item_id#27] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] -Results [2]: [i_item_id#27, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS total_sales#31] - -(46) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(47) ColumnarToRow [codegen id : 14] -Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] - -(48) Filter [codegen id : 14] -Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] -Condition : (isnotnull(ws_bill_addr_sk#33) AND isnotnull(ws_item_sk#32)) - -(49) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#36] - -(50) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#36] -Join type: Inner -Join condition: None - -(51) Project [codegen id : 14] -Output [3]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34] -Input [5]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35, d_date_sk#36] - -(52) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#37] - -(53) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#33] -Right keys [1]: [ca_address_sk#37] -Join type: Inner -Join condition: None - -(54) Project [codegen id : 14] -Output [2]: [ws_item_sk#32, ws_ext_sales_price#34] -Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ca_address_sk#37] - -(55) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#38, i_item_id#39] - -(56) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_item_sk#32] -Right keys [1]: [i_item_sk#38] -Join type: Inner -Join condition: None - -(57) Project [codegen id : 14] -Output [2]: [ws_ext_sales_price#34, i_item_id#39] -Input [4]: [ws_item_sk#32, ws_ext_sales_price#34, i_item_sk#38, i_item_id#39] - -(58) HashAggregate [codegen id : 14] -Input [2]: [ws_ext_sales_price#34, i_item_id#39] -Keys [1]: [i_item_id#39] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#34))] -Aggregate Attributes [1]: [sum#40] -Results [2]: [i_item_id#39, sum#41] - -(59) CometColumnarExchange -Input [2]: [i_item_id#39, sum#41] -Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(60) CometColumnarToRow [codegen id : 15] -Input [2]: [i_item_id#39, sum#41] - -(61) HashAggregate [codegen id : 15] -Input [2]: [i_item_id#39, sum#41] -Keys [1]: [i_item_id#39] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#34))#42] -Results [2]: [i_item_id#39, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#34))#42,17,2) AS total_sales#43] - -(62) Union - -(63) HashAggregate [codegen id : 16] -Input [2]: [i_item_id#18, total_sales#19] -Keys [1]: [i_item_id#18] -Functions [1]: [partial_sum(total_sales#19)] -Aggregate Attributes [2]: [sum#44, isEmpty#45] -Results [3]: [i_item_id#18, sum#46, isEmpty#47] - -(64) CometColumnarExchange -Input [3]: [i_item_id#18, sum#46, isEmpty#47] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(65) CometColumnarToRow [codegen id : 17] -Input [3]: [i_item_id#18, sum#46, isEmpty#47] - -(66) HashAggregate [codegen id : 17] -Input [3]: [i_item_id#18, sum#46, isEmpty#47] -Keys [1]: [i_item_id#18] -Functions [1]: [sum(total_sales#19)] -Aggregate Attributes [1]: [sum(total_sales#19)#48] -Results [2]: [i_item_id#18, sum(total_sales#19)#48 AS total_sales#49] - -(67) TakeOrderedAndProject -Input [2]: [i_item_id#18, total_sales#49] -Arguments: 100, [total_sales#49 ASC NULLS FIRST], [i_item_id#18, total_sales#49] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (72) -+- * CometColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometNativeScan parquet spark_catalog.default.date_dim (68) - - -(68) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#50, d_moy#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [3]: [d_date_sk#6, d_year#50, d_moy#51] -Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 2001)) AND (d_moy#51 = 2)) AND isnotnull(d_date_sk#6)) - -(70) CometProject -Input [3]: [d_date_sk#6, d_year#50, d_moy#51] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(71) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(72) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/simplified.txt deleted file mode 100644 index 81d90bdfb2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/simplified.txt +++ /dev/null @@ -1,106 +0,0 @@ -TakeOrderedAndProject [total_sales,i_item_id] - WholeStageCodegen (17) - HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (16) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),i_item_id,total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #6 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_color] - CometNativeScan parquet spark_catalog.default.item [i_item_id,i_color] - WholeStageCodegen (10) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #7 - WholeStageCodegen (9) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_addr_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (15) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #8 - WholeStageCodegen (14) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/explain.txt deleted file mode 100644 index 16d7564ea5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,385 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (62) -+- CometTakeOrderedAndProject (61) - +- CometHashAggregate (60) - +- CometExchange (59) - +- CometHashAggregate (58) - +- CometUnion (57) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- CometHashAggregate (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometFilter (30) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- CometHashAggregate (56) - +- CometExchange (55) - +- CometHashAggregate (54) - +- CometProject (53) - +- CometBroadcastHashJoin (52) - :- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometFilter (44) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (43) - : : +- ReusedExchange (45) - : +- ReusedExchange (48) - +- ReusedExchange (51) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001)) AND (d_moy#8 = 2)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(11) CometProject -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Arguments: [ca_address_sk#9], [ca_address_sk#9] - -(12) CometBroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: [ca_address_sk#9] - -(13) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Right output [1]: [ca_address_sk#9] -Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] -Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#11, i_item_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#11, i_item_id#12] -Condition : isnotnull(i_item_sk#11) - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_id#13, i_color#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(18) CometFilter -Input [2]: [i_item_id#13, i_color#14] -Condition : static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#14, 20)) IN (slate ,blanched ,burnished ) - -(19) CometProject -Input [2]: [i_item_id#13, i_color#14] -Arguments: [i_item_id#15], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#13, 16)) AS i_item_id#15] - -(20) CometBroadcastExchange -Input [1]: [i_item_id#15] -Arguments: [i_item_id#15] - -(21) CometBroadcastHashJoin -Left output [2]: [i_item_sk#11, i_item_id#12] -Right output [1]: [i_item_id#15] -Arguments: [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#12, 16))], [i_item_id#15], LeftSemi, BuildRight - -(22) CometProject -Input [2]: [i_item_sk#11, i_item_id#12] -Arguments: [i_item_sk#11, i_item_id#16], [i_item_sk#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#12, 16)) AS i_item_id#16] - -(23) CometBroadcastExchange -Input [2]: [i_item_sk#11, i_item_id#16] -Arguments: [i_item_sk#11, i_item_id#16] - -(24) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Right output [2]: [i_item_sk#11, i_item_id#16] -Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight - -(25) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_item_id#16] -Arguments: [ss_ext_sales_price#3, i_item_id#16], [ss_ext_sales_price#3, i_item_id#16] - -(26) CometHashAggregate -Input [2]: [ss_ext_sales_price#3, i_item_id#16] -Keys [1]: [i_item_id#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(27) CometExchange -Input [2]: [i_item_id#16, sum#17] -Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [2]: [i_item_id#16, sum#17] -Keys [1]: [i_item_id#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Condition : (isnotnull(cs_bill_addr_sk#18) AND isnotnull(cs_item_sk#19)) - -(31) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#23] - -(32) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Right output [1]: [d_date_sk#23] -Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(33) CometProject -Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] -Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] - -(34) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#24] - -(35) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -Right output [1]: [ca_address_sk#24] -Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner, BuildRight - -(36) CometProject -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] -Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] - -(37) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#25, i_item_id#26] - -(38) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] -Right output [2]: [i_item_sk#25, i_item_id#26] -Arguments: [cs_item_sk#19], [i_item_sk#25], Inner, BuildRight - -(39) CometProject -Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_item_id#26] -Arguments: [cs_ext_sales_price#20, i_item_id#26], [cs_ext_sales_price#20, i_item_id#26] - -(40) CometHashAggregate -Input [2]: [cs_ext_sales_price#20, i_item_id#26] -Keys [1]: [i_item_id#26] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] - -(41) CometExchange -Input [2]: [i_item_id#26, sum#27] -Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(42) CometHashAggregate -Input [2]: [i_item_id#26, sum#27] -Keys [1]: [i_item_id#26] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(44) CometFilter -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Condition : (isnotnull(ws_bill_addr_sk#29) AND isnotnull(ws_item_sk#28)) - -(45) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(46) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [ws_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(47) CometProject -Input [5]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31, d_date_sk#33] -Arguments: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30], [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] - -(48) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#34] - -(49) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] -Right output [1]: [ca_address_sk#34] -Arguments: [ws_bill_addr_sk#29], [ca_address_sk#34], Inner, BuildRight - -(50) CometProject -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ca_address_sk#34] -Arguments: [ws_item_sk#28, ws_ext_sales_price#30], [ws_item_sk#28, ws_ext_sales_price#30] - -(51) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#35, i_item_id#36] - -(52) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#28, ws_ext_sales_price#30] -Right output [2]: [i_item_sk#35, i_item_id#36] -Arguments: [ws_item_sk#28], [i_item_sk#35], Inner, BuildRight - -(53) CometProject -Input [4]: [ws_item_sk#28, ws_ext_sales_price#30, i_item_sk#35, i_item_id#36] -Arguments: [ws_ext_sales_price#30, i_item_id#36], [ws_ext_sales_price#30, i_item_id#36] - -(54) CometHashAggregate -Input [2]: [ws_ext_sales_price#30, i_item_id#36] -Keys [1]: [i_item_id#36] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#30))] - -(55) CometExchange -Input [2]: [i_item_id#36, sum#37] -Arguments: hashpartitioning(i_item_id#36, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(56) CometHashAggregate -Input [2]: [i_item_id#36, sum#37] -Keys [1]: [i_item_id#36] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#30))] - -(57) CometUnion -Child 0 Input [2]: [i_item_id#38, total_sales#39] -Child 1 Input [2]: [i_item_id#26, total_sales#40] -Child 2 Input [2]: [i_item_id#36, total_sales#41] - -(58) CometHashAggregate -Input [2]: [i_item_id#38, total_sales#39] -Keys [1]: [i_item_id#38] -Functions [1]: [partial_sum(total_sales#39)] - -(59) CometExchange -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(60) CometHashAggregate -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Keys [1]: [i_item_id#38] -Functions [1]: [sum(total_sales#39)] - -(61) CometTakeOrderedAndProject -Input [2]: [i_item_id#38, total_sales#44] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#44 ASC NULLS FIRST], output=[i_item_id#38,total_sales#44]), [i_item_id#38, total_sales#44], 100, 0, [total_sales#44 ASC NULLS FIRST], [i_item_id#38, total_sales#44] - -(62) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_id#38, total_sales#44] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (67) -+- * CometColumnarToRow (66) - +- CometProject (65) - +- CometFilter (64) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) - - -(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(64) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001)) AND (d_moy#8 = 2)) AND isnotnull(d_date_sk#6)) - -(65) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(66) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(67) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/simplified.txt deleted file mode 100644 index 19eca782cd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,total_sales] - CometHashAggregate [sum,isEmpty] [i_item_id,total_sales,sum(total_sales)] - CometExchange [i_item_id] #1 - CometHashAggregate [total_sales] [i_item_id,sum,isEmpty] - CometUnion [i_item_id,total_sales] - CometHashAggregate [sum] [i_item_id,total_sales,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #7 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_color] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_color] - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - 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,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/explain.txt deleted file mode 100644 index 16d7564ea5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/explain.txt +++ /dev/null @@ -1,385 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (62) -+- CometTakeOrderedAndProject (61) - +- CometHashAggregate (60) - +- CometExchange (59) - +- CometHashAggregate (58) - +- CometUnion (57) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- CometHashAggregate (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometFilter (30) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- CometHashAggregate (56) - +- CometExchange (55) - +- CometHashAggregate (54) - +- CometProject (53) - +- CometBroadcastHashJoin (52) - :- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometFilter (44) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (43) - : : +- ReusedExchange (45) - : +- ReusedExchange (48) - +- ReusedExchange (51) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001)) AND (d_moy#8 = 2)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(11) CometProject -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Arguments: [ca_address_sk#9], [ca_address_sk#9] - -(12) CometBroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: [ca_address_sk#9] - -(13) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Right output [1]: [ca_address_sk#9] -Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] -Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#11, i_item_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#11, i_item_id#12] -Condition : isnotnull(i_item_sk#11) - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_id#13, i_color#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(18) CometFilter -Input [2]: [i_item_id#13, i_color#14] -Condition : static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#14, 20)) IN (slate ,blanched ,burnished ) - -(19) CometProject -Input [2]: [i_item_id#13, i_color#14] -Arguments: [i_item_id#15], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#13, 16)) AS i_item_id#15] - -(20) CometBroadcastExchange -Input [1]: [i_item_id#15] -Arguments: [i_item_id#15] - -(21) CometBroadcastHashJoin -Left output [2]: [i_item_sk#11, i_item_id#12] -Right output [1]: [i_item_id#15] -Arguments: [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#12, 16))], [i_item_id#15], LeftSemi, BuildRight - -(22) CometProject -Input [2]: [i_item_sk#11, i_item_id#12] -Arguments: [i_item_sk#11, i_item_id#16], [i_item_sk#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#12, 16)) AS i_item_id#16] - -(23) CometBroadcastExchange -Input [2]: [i_item_sk#11, i_item_id#16] -Arguments: [i_item_sk#11, i_item_id#16] - -(24) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Right output [2]: [i_item_sk#11, i_item_id#16] -Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight - -(25) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_item_id#16] -Arguments: [ss_ext_sales_price#3, i_item_id#16], [ss_ext_sales_price#3, i_item_id#16] - -(26) CometHashAggregate -Input [2]: [ss_ext_sales_price#3, i_item_id#16] -Keys [1]: [i_item_id#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(27) CometExchange -Input [2]: [i_item_id#16, sum#17] -Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [2]: [i_item_id#16, sum#17] -Keys [1]: [i_item_id#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Condition : (isnotnull(cs_bill_addr_sk#18) AND isnotnull(cs_item_sk#19)) - -(31) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#23] - -(32) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Right output [1]: [d_date_sk#23] -Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(33) CometProject -Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] -Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] - -(34) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#24] - -(35) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -Right output [1]: [ca_address_sk#24] -Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner, BuildRight - -(36) CometProject -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] -Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] - -(37) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#25, i_item_id#26] - -(38) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] -Right output [2]: [i_item_sk#25, i_item_id#26] -Arguments: [cs_item_sk#19], [i_item_sk#25], Inner, BuildRight - -(39) CometProject -Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_item_id#26] -Arguments: [cs_ext_sales_price#20, i_item_id#26], [cs_ext_sales_price#20, i_item_id#26] - -(40) CometHashAggregate -Input [2]: [cs_ext_sales_price#20, i_item_id#26] -Keys [1]: [i_item_id#26] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] - -(41) CometExchange -Input [2]: [i_item_id#26, sum#27] -Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(42) CometHashAggregate -Input [2]: [i_item_id#26, sum#27] -Keys [1]: [i_item_id#26] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(44) CometFilter -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Condition : (isnotnull(ws_bill_addr_sk#29) AND isnotnull(ws_item_sk#28)) - -(45) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(46) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [ws_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(47) CometProject -Input [5]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31, d_date_sk#33] -Arguments: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30], [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] - -(48) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#34] - -(49) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] -Right output [1]: [ca_address_sk#34] -Arguments: [ws_bill_addr_sk#29], [ca_address_sk#34], Inner, BuildRight - -(50) CometProject -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ca_address_sk#34] -Arguments: [ws_item_sk#28, ws_ext_sales_price#30], [ws_item_sk#28, ws_ext_sales_price#30] - -(51) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#35, i_item_id#36] - -(52) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#28, ws_ext_sales_price#30] -Right output [2]: [i_item_sk#35, i_item_id#36] -Arguments: [ws_item_sk#28], [i_item_sk#35], Inner, BuildRight - -(53) CometProject -Input [4]: [ws_item_sk#28, ws_ext_sales_price#30, i_item_sk#35, i_item_id#36] -Arguments: [ws_ext_sales_price#30, i_item_id#36], [ws_ext_sales_price#30, i_item_id#36] - -(54) CometHashAggregate -Input [2]: [ws_ext_sales_price#30, i_item_id#36] -Keys [1]: [i_item_id#36] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#30))] - -(55) CometExchange -Input [2]: [i_item_id#36, sum#37] -Arguments: hashpartitioning(i_item_id#36, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(56) CometHashAggregate -Input [2]: [i_item_id#36, sum#37] -Keys [1]: [i_item_id#36] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#30))] - -(57) CometUnion -Child 0 Input [2]: [i_item_id#38, total_sales#39] -Child 1 Input [2]: [i_item_id#26, total_sales#40] -Child 2 Input [2]: [i_item_id#36, total_sales#41] - -(58) CometHashAggregate -Input [2]: [i_item_id#38, total_sales#39] -Keys [1]: [i_item_id#38] -Functions [1]: [partial_sum(total_sales#39)] - -(59) CometExchange -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(60) CometHashAggregate -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Keys [1]: [i_item_id#38] -Functions [1]: [sum(total_sales#39)] - -(61) CometTakeOrderedAndProject -Input [2]: [i_item_id#38, total_sales#44] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#44 ASC NULLS FIRST], output=[i_item_id#38,total_sales#44]), [i_item_id#38, total_sales#44], 100, 0, [total_sales#44 ASC NULLS FIRST], [i_item_id#38, total_sales#44] - -(62) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_id#38, total_sales#44] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (67) -+- * CometColumnarToRow (66) - +- CometProject (65) - +- CometFilter (64) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) - - -(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(64) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001)) AND (d_moy#8 = 2)) AND isnotnull(d_date_sk#6)) - -(65) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(66) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(67) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/extended.txt deleted file mode 100644 index 36b9a8306d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/extended.txt +++ /dev/null @@ -1,100 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 19eca782cd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,total_sales] - CometHashAggregate [sum,isEmpty] [i_item_id,total_sales,sum(total_sales)] - CometExchange [i_item_id] #1 - CometHashAggregate [total_sales] [i_item_id,sum,isEmpty] - CometUnion [i_item_id,total_sales] - CometHashAggregate [sum] [i_item_id,total_sales,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #7 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_color] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_color] - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - 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,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/explain.txt deleted file mode 100644 index 9d35d600a6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/explain.txt +++ /dev/null @@ -1,304 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (51) -+- * Project (50) - +- * BroadcastHashJoin Inner BuildRight (49) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * CometColumnarToRow (26) - : : +- CometSort (25) - : : +- CometColumnarExchange (24) - : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- * HashAggregate (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- Window (38) - : +- * CometColumnarToRow (37) - : +- CometSort (36) - : +- CometColumnarExchange (35) - : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) - : +- ReusedExchange (32) - +- BroadcastExchange (48) - +- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- ReusedExchange (43) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)))) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) AS i_brand#4, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)) AS i_category#5] - -(4) CometColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#4, i_category#5] - -(5) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(7) Filter [codegen id : 1] -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) - -(8) BroadcastExchange -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#7] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(11) ReusedExchange [Reuses operator id: 55] -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] - -(14) CometNativeScan parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#14, cc_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#14, cc_name#15] -Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) - -(16) CometColumnarToRow [codegen id : 3] -Input [2]: [cc_call_center_sk#14, cc_name#15] - -(17) BroadcastExchange -Input [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_call_center_sk#6] -Right keys [1]: [cc_call_center_sk#14] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] - -(20) HashAggregate [codegen id : 4] -Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] -Aggregate Attributes [1]: [sum#16] -Results [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] - -(21) CometColumnarExchange -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] - -(23) HashAggregate [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#8))#18] -Results [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#18,17,2) AS sum_sales#19, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#18,17,2) AS _w0#20] - -(24) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) CometColumnarToRow [codegen id : 6] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] - -(27) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(28) Filter [codegen id : 7] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(29) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] -Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(30) Filter [codegen id : 22] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] -Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) - -(31) Project [codegen id : 22] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] - -(32) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum#28] - -(33) CometColumnarToRow [codegen id : 12] -Input [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum#28] - -(34) HashAggregate [codegen id : 12] -Input [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum#28] -Keys [5]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27] -Functions [1]: [sum(UnscaledValue(cs_sales_price#29))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#29))#18] -Results [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(cs_sales_price#29))#18,17,2) AS sum_sales#30] - -(35) CometColumnarExchange -Input [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sales#30] -Arguments: hashpartitioning(i_category#23, i_brand#24, cc_name#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(36) CometSort -Input [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sales#30] -Arguments: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sales#30], [i_category#23 ASC NULLS FIRST, i_brand#24 ASC NULLS FIRST, cc_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] - -(37) CometColumnarToRow [codegen id : 13] -Input [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sales#30] - -(38) Window -Input [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sales#30] -Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#23, i_brand#24, cc_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#31], [i_category#23, i_brand#24, cc_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] - -(39) Project [codegen id : 14] -Output [5]: [i_category#23, i_brand#24, cc_name#25, sum_sales#30, rn#31] -Input [7]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sales#30, rn#31] - -(40) BroadcastExchange -Input [5]: [i_category#23, i_brand#24, cc_name#25, sum_sales#30, rn#31] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] - -(41) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] -Right keys [4]: [i_category#23, i_brand#24, cc_name#25, (rn#31 + 1)] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 22] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#30] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#23, i_brand#24, cc_name#25, sum_sales#30, rn#31] - -(43) ReusedExchange [Reuses operator id: 35] -Output [6]: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sales#37] - -(44) CometSort -Input [6]: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sales#37] -Arguments: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sales#37], [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, cc_name#34 ASC NULLS FIRST, d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST] - -(45) CometColumnarToRow [codegen id : 20] -Input [6]: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sales#37] - -(46) Window -Input [6]: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sales#37] -Arguments: [rank(d_year#35, d_moy#36) windowspecdefinition(i_category#32, i_brand#33, cc_name#34, d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#32, i_brand#33, cc_name#34], [d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST] - -(47) Project [codegen id : 21] -Output [5]: [i_category#32, i_brand#33, cc_name#34, sum_sales#37, rn#38] -Input [7]: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sales#37, rn#38] - -(48) BroadcastExchange -Input [5]: [i_category#32, i_brand#33, cc_name#34, sum_sales#37, rn#38] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] - -(49) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] -Right keys [4]: [i_category#32, i_brand#33, cc_name#34, (rn#38 - 1)] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 22] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#30 AS psum#39, sum_sales#37 AS nsum#40] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#30, i_category#32, i_brand#33, cc_name#34, sum_sales#37, rn#38] - -(51) TakeOrderedAndProject -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] -Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometFilter (53) - +- CometNativeScan parquet spark_catalog.default.date_dim (52) - - -(52) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(54) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(55) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/simplified.txt deleted file mode 100644 index 4f9ac35f4d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/simplified.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (22) - Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (7) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [cs_item_sk,cs_call_center_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk,cc_name] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/explain.txt deleted file mode 100644 index 7fec2cf2e8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)))) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) AS i_brand#4, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) - -(6) CometBroadcastExchange -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_item_sk#1], [cs_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#14, cc_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#14, cc_name#15] -Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) - -(16) CometBroadcastExchange -Input [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cc_call_center_sk#14, cc_name#15] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] -Right output [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cs_call_center_sk#6], [cc_call_center_sk#14], Inner, BuildRight - -(18) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] -Arguments: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15], [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] - -(19) CometHashAggregate -Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] - -(20) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] - -(22) CometExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] - -(25) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(28) Filter [codegen id : 7] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] - -(30) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] - -(31) CometHashAggregate -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] -Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] - -(32) CometExchange -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] -Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] -Arguments: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28], [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] - -(35) Window -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#29], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] -Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28, rn#29] - -(37) BroadcastExchange -Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#29 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] - -(40) ReusedExchange [Reuses operator id: 32] -Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] - -(41) CometSort -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] -Arguments: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35], [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] - -(43) Window -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] -Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] -Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35, rn#36] - -(45) BroadcastExchange -Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#36 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#28 AS psum#37, sum_sales#35 AS nsum#38] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28, i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] - -(48) TakeOrderedAndProject -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/simplified.txt deleted file mode 100644 index b854e818db..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - 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 [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,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt deleted file mode 100644 index 7fec2cf2e8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)))) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) AS i_brand#4, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) - -(6) CometBroadcastExchange -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_item_sk#1], [cs_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#14, cc_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#14, cc_name#15] -Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) - -(16) CometBroadcastExchange -Input [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cc_call_center_sk#14, cc_name#15] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] -Right output [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cs_call_center_sk#6], [cc_call_center_sk#14], Inner, BuildRight - -(18) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] -Arguments: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15], [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] - -(19) CometHashAggregate -Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] - -(20) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] - -(22) CometExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] - -(25) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(28) Filter [codegen id : 7] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] - -(30) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] - -(31) CometHashAggregate -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] -Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] - -(32) CometExchange -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] -Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] -Arguments: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28], [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] - -(35) Window -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#29], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] -Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28, rn#29] - -(37) BroadcastExchange -Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#29 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] - -(40) ReusedExchange [Reuses operator id: 32] -Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] - -(41) CometSort -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] -Arguments: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35], [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] - -(43) Window -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] -Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] -Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35, rn#36] - -(45) BroadcastExchange -Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#36 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#28 AS psum#37, sum_sales#35 AS nsum#38] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28, i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] - -(48) TakeOrderedAndProject -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/extended.txt deleted file mode 100644 index de7d26cd06..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index b854e818db..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - 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 [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,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/explain.txt deleted file mode 100644 index 6ea099c1da..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/explain.txt +++ /dev/null @@ -1,398 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (53) -+- * Project (52) - +- * BroadcastHashJoin Inner BuildRight (51) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Filter (18) - : : +- * HashAggregate (17) - : : +- * CometColumnarToRow (16) - : : +- CometColumnarExchange (15) - : : +- * HashAggregate (14) - : : +- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (33) - : +- * Filter (32) - : +- * HashAggregate (31) - : +- * CometColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- * HashAggregate (28) - : +- * Project (27) - : +- * BroadcastHashJoin Inner BuildRight (26) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Filter (21) - : : : +- * ColumnarToRow (20) - : : : +- Scan parquet spark_catalog.default.catalog_sales (19) - : : +- ReusedExchange (22) - : +- ReusedExchange (25) - +- BroadcastExchange (50) - +- * Filter (49) - +- * HashAggregate (48) - +- * CometColumnarToRow (47) - +- CometColumnarExchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Filter (38) - : : +- * ColumnarToRow (37) - : : +- Scan parquet spark_catalog.default.web_sales (36) - : +- ReusedExchange (39) - +- ReusedExchange (42) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_item_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [i_item_sk#5, i_item_id#6] -Condition : (isnotnull(i_item_sk#5) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)))) - -(6) CometProject -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#7] - -(7) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_sk#5, i_item_id#7] - -(8) BroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7] -Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#7] - -(11) ReusedExchange [Reuses operator id: 63] -Output [1]: [d_date_sk#8] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#8] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [2]: [ss_ext_sales_price#2, i_item_id#7] -Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7, d_date_sk#8] - -(14) HashAggregate [codegen id : 3] -Input [2]: [ss_ext_sales_price#2, i_item_id#7] -Keys [1]: [i_item_id#7] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#9] -Results [2]: [i_item_id#7, sum#10] - -(15) CometColumnarExchange -Input [2]: [i_item_id#7, sum#10] -Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 12] -Input [2]: [i_item_id#7, sum#10] - -(17) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#7, sum#10] -Keys [1]: [i_item_id#7] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#11] -Results [2]: [i_item_id#7 AS item_id#12, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#11,17,2) AS ss_item_rev#13] - -(18) Filter [codegen id : 12] -Input [2]: [item_id#12, ss_item_rev#13] -Condition : isnotnull(ss_item_rev#13) - -(19) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(20) ColumnarToRow [codegen id : 6] -Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] - -(21) Filter [codegen id : 6] -Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#14) - -(22) ReusedExchange [Reuses operator id: 8] -Output [2]: [i_item_sk#17, i_item_id#18] - -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_item_sk#14] -Right keys [1]: [i_item_sk#17] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 6] -Output [3]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#18] -Input [5]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_sk#17, i_item_id#18] - -(25) ReusedExchange [Reuses operator id: 63] -Output [1]: [d_date_sk#19] - -(26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 6] -Output [2]: [cs_ext_sales_price#15, i_item_id#18] -Input [4]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#18, d_date_sk#19] - -(28) HashAggregate [codegen id : 6] -Input [2]: [cs_ext_sales_price#15, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#15))] -Aggregate Attributes [1]: [sum#20] -Results [2]: [i_item_id#18, sum#21] - -(29) CometColumnarExchange -Input [2]: [i_item_id#18, sum#21] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(30) CometColumnarToRow [codegen id : 7] -Input [2]: [i_item_id#18, sum#21] - -(31) HashAggregate [codegen id : 7] -Input [2]: [i_item_id#18, sum#21] -Keys [1]: [i_item_id#18] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#15))#22] -Results [2]: [i_item_id#18 AS item_id#23, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#15))#22,17,2) AS cs_item_rev#24] - -(32) Filter [codegen id : 7] -Input [2]: [item_id#23, cs_item_rev#24] -Condition : isnotnull(cs_item_rev#24) - -(33) BroadcastExchange -Input [2]: [item_id#23, cs_item_rev#24] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] - -(34) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [item_id#12] -Right keys [1]: [item_id#23] -Join type: Inner -Join condition: ((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * cs_item_rev#24)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) - -(35) Project [codegen id : 12] -Output [3]: [item_id#12, ss_item_rev#13, cs_item_rev#24] -Input [4]: [item_id#12, ss_item_rev#13, item_id#23, cs_item_rev#24] - -(36) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(37) ColumnarToRow [codegen id : 10] -Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] - -(38) Filter [codegen id : 10] -Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#25) - -(39) ReusedExchange [Reuses operator id: 8] -Output [2]: [i_item_sk#28, i_item_id#29] - -(40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_item_sk#25] -Right keys [1]: [i_item_sk#28] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 10] -Output [3]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#29] -Input [5]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_sk#28, i_item_id#29] - -(42) ReusedExchange [Reuses operator id: 63] -Output [1]: [d_date_sk#30] - -(43) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#30] -Join type: Inner -Join condition: None - -(44) Project [codegen id : 10] -Output [2]: [ws_ext_sales_price#26, i_item_id#29] -Input [4]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#29, d_date_sk#30] - -(45) HashAggregate [codegen id : 10] -Input [2]: [ws_ext_sales_price#26, i_item_id#29] -Keys [1]: [i_item_id#29] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#31] -Results [2]: [i_item_id#29, sum#32] - -(46) CometColumnarExchange -Input [2]: [i_item_id#29, sum#32] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(47) CometColumnarToRow [codegen id : 11] -Input [2]: [i_item_id#29, sum#32] - -(48) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#29, sum#32] -Keys [1]: [i_item_id#29] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#26))#33] -Results [2]: [i_item_id#29 AS item_id#34, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#26))#33,17,2) AS ws_item_rev#35] - -(49) Filter [codegen id : 11] -Input [2]: [item_id#34, ws_item_rev#35] -Condition : isnotnull(ws_item_rev#35) - -(50) BroadcastExchange -Input [2]: [item_id#34, ws_item_rev#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] - -(51) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [item_id#12] -Right keys [1]: [item_id#34] -Join type: Inner -Join condition: ((((((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * ws_item_rev#35)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * ws_item_rev#35))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ws_item_rev#35))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ws_item_rev#35))) AND (cast(ws_item_rev#35 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(ws_item_rev#35 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) AND (cast(ws_item_rev#35 as decimal(19,3)) >= (0.9 * cs_item_rev#24))) AND (cast(ws_item_rev#35 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) - -(52) Project [codegen id : 12] -Output [8]: [item_id#12, ss_item_rev#13, (((ss_item_rev#13 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35)) / 3) * 100) AS ss_dev#36, cs_item_rev#24, (((cs_item_rev#24 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35)) / 3) * 100) AS cs_dev#37, ws_item_rev#35, (((ws_item_rev#35 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35)) / 3) * 100) AS ws_dev#38, (((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#35) / 3) AS average#39] -Input [5]: [item_id#12, ss_item_rev#13, cs_item_rev#24, item_id#34, ws_item_rev#35] - -(53) TakeOrderedAndProject -Input [8]: [item_id#12, ss_item_rev#13, ss_dev#36, cs_item_rev#24, cs_dev#37, ws_item_rev#35, ws_dev#38, average#39] -Arguments: 100, [item_id#12 ASC NULLS FIRST, ss_item_rev#13 ASC NULLS FIRST], [item_id#12, ss_item_rev#13, ss_dev#36, cs_item_rev#24, cs_dev#37, ws_item_rev#35, ws_dev#38, average#39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (63) -+- * CometColumnarToRow (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometFilter (55) - : +- CometNativeScan parquet spark_catalog.default.date_dim (54) - +- CometBroadcastExchange (59) - +- CometProject (58) - +- CometFilter (57) - +- CometNativeScan parquet spark_catalog.default.date_dim (56) - - -(54) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(55) CometFilter -Input [2]: [d_date_sk#8, d_date#40] -Condition : isnotnull(d_date_sk#8) - -(56) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date#41, d_week_seq#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#43)] -ReadSchema: struct - -(57) CometFilter -Input [2]: [d_date#41, d_week_seq#42] -Condition : (isnotnull(d_week_seq#42) AND (d_week_seq#42 = ReusedSubquery Subquery scalar-subquery#43, [id=#44])) - -(58) CometProject -Input [2]: [d_date#41, d_week_seq#42] -Arguments: [d_date#41], [d_date#41] - -(59) CometBroadcastExchange -Input [1]: [d_date#41] -Arguments: [d_date#41] - -(60) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#40] -Right output [1]: [d_date#41] -Arguments: [d_date#40], [d_date#41], LeftSemi, BuildRight - -(61) CometProject -Input [2]: [d_date_sk#8, d_date#40] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(62) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(63) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 57 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44] - -Subquery:3 Hosting operator id = 56 Hosting Expression = Subquery scalar-subquery#43, [id=#44] -* CometColumnarToRow (67) -+- CometProject (66) - +- CometFilter (65) - +- CometNativeScan parquet spark_catalog.default.date_dim (64) - - -(64) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date#45, d_week_seq#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] -ReadSchema: struct - -(65) CometFilter -Input [2]: [d_date#45, d_week_seq#46] -Condition : (isnotnull(d_date#45) AND (d_date#45 = 2000-01-03)) - -(66) CometProject -Input [2]: [d_date#45, d_week_seq#46] -Arguments: [d_week_seq#46], [d_week_seq#46] - -(67) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#46] - -Subquery:4 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#4 - -Subquery:5 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/simplified.txt deleted file mode 100644 index fd841da602..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/simplified.txt +++ /dev/null @@ -1,98 +0,0 @@ -TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - WholeStageCodegen (12) - Project [item_id,ss_item_rev,cs_item_rev,ws_item_rev] - BroadcastHashJoin [item_id,item_id,ss_item_rev,ws_item_rev,cs_item_rev] - Project [item_id,ss_item_rev,cs_item_rev] - BroadcastHashJoin [item_id,item_id,ss_item_rev,cs_item_rev] - Filter [ss_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (3) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometFilter [d_date,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - Filter [cs_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #6 - WholeStageCodegen (6) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #4 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) - Filter [ws_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #8 - WholeStageCodegen (10) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #4 - InputAdapter - ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/explain.txt deleted file mode 100644 index 3c368afc65..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,383 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (52) -+- CometTakeOrderedAndProject (51) - +- CometProject (50) - +- CometBroadcastHashJoin (49) - :- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometFilter (20) - : : +- CometHashAggregate (19) - : : +- CometExchange (18) - : : +- CometHashAggregate (17) - : : +- CometProject (16) - : : +- CometBroadcastHashJoin (15) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (14) - : : +- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- ReusedExchange (11) - : +- CometBroadcastExchange (33) - : +- CometFilter (32) - : +- CometHashAggregate (31) - : +- CometExchange (30) - : +- CometHashAggregate (29) - : +- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (21) - : : +- ReusedExchange (23) - : +- ReusedExchange (26) - +- CometBroadcastExchange (48) - +- CometFilter (47) - +- CometHashAggregate (46) - +- CometExchange (45) - +- CometHashAggregate (44) - +- CometProject (43) - +- CometBroadcastHashJoin (42) - :- CometProject (40) - : +- CometBroadcastHashJoin (39) - : :- CometFilter (37) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (36) - : +- ReusedExchange (38) - +- ReusedExchange (41) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_item_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_item_id#6] -Condition : (isnotnull(i_item_sk#5) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)))) - -(5) CometProject -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#7] - -(6) CometBroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#7] -Arguments: [i_item_sk#5, i_item_id#7] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [2]: [i_item_sk#5, i_item_id#7] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#7] -Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(11) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date#10] - -(12) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#10] -Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight - -(13) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: [d_date_sk#8] - -(15) CometBroadcastHashJoin -Left output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7] -Right output [1]: [d_date_sk#8] -Arguments: [ss_sold_date_sk#3], [d_date_sk#8], Inner, BuildRight - -(16) CometProject -Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7, d_date_sk#8] -Arguments: [ss_ext_sales_price#2, i_item_id#7], [ss_ext_sales_price#2, i_item_id#7] - -(17) CometHashAggregate -Input [2]: [ss_ext_sales_price#2, i_item_id#7] -Keys [1]: [i_item_id#7] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(18) CometExchange -Input [2]: [i_item_id#7, sum#11] -Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [2]: [i_item_id#7, sum#11] -Keys [1]: [i_item_id#7] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(20) CometFilter -Input [2]: [item_id#12, ss_item_rev#13] -Condition : isnotnull(ss_item_rev#13) - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#14) - -(23) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#18, i_item_id#19] - -(24) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Right output [2]: [i_item_sk#18, i_item_id#19] -Arguments: [cs_item_sk#14], [i_item_sk#18], Inner, BuildRight - -(25) CometProject -Input [5]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_sk#18, i_item_id#19] -Arguments: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19], [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19] - -(26) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#20] - -(27) CometBroadcastHashJoin -Left output [3]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19] -Right output [1]: [d_date_sk#20] -Arguments: [cs_sold_date_sk#16], [d_date_sk#20], Inner, BuildRight - -(28) CometProject -Input [4]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19, d_date_sk#20] -Arguments: [cs_ext_sales_price#15, i_item_id#19], [cs_ext_sales_price#15, i_item_id#19] - -(29) CometHashAggregate -Input [2]: [cs_ext_sales_price#15, i_item_id#19] -Keys [1]: [i_item_id#19] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#15))] - -(30) CometExchange -Input [2]: [i_item_id#19, sum#21] -Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(31) CometHashAggregate -Input [2]: [i_item_id#19, sum#21] -Keys [1]: [i_item_id#19] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#15))] - -(32) CometFilter -Input [2]: [item_id#22, cs_item_rev#23] -Condition : isnotnull(cs_item_rev#23) - -(33) CometBroadcastExchange -Input [2]: [item_id#22, cs_item_rev#23] -Arguments: [item_id#22, cs_item_rev#23] - -(34) CometBroadcastHashJoin -Left output [2]: [item_id#12, ss_item_rev#13] -Right output [2]: [item_id#22, cs_item_rev#23] -Arguments: [item_id#12], [item_id#22], Inner, ((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * cs_item_rev#23)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * cs_item_rev#23))) AND (cast(cs_item_rev#23 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(cs_item_rev#23 as decimal(20,3)) <= (1.1 * ss_item_rev#13))), BuildRight - -(35) CometProject -Input [4]: [item_id#12, ss_item_rev#13, item_id#22, cs_item_rev#23] -Arguments: [item_id#12, ss_item_rev#13, cs_item_rev#23], [item_id#12, ss_item_rev#13, cs_item_rev#23] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#26), dynamicpruningexpression(ws_sold_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26] -Condition : isnotnull(ws_item_sk#24) - -(38) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#28, i_item_id#29] - -(39) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26] -Right output [2]: [i_item_sk#28, i_item_id#29] -Arguments: [ws_item_sk#24], [i_item_sk#28], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_sk#28, i_item_id#29] -Arguments: [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29], [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29] - -(41) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#30] - -(42) CometBroadcastHashJoin -Left output [3]: [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29] -Right output [1]: [d_date_sk#30] -Arguments: [ws_sold_date_sk#26], [d_date_sk#30], Inner, BuildRight - -(43) CometProject -Input [4]: [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29, d_date_sk#30] -Arguments: [ws_ext_sales_price#25, i_item_id#29], [ws_ext_sales_price#25, i_item_id#29] - -(44) CometHashAggregate -Input [2]: [ws_ext_sales_price#25, i_item_id#29] -Keys [1]: [i_item_id#29] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#25))] - -(45) CometExchange -Input [2]: [i_item_id#29, sum#31] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(46) CometHashAggregate -Input [2]: [i_item_id#29, sum#31] -Keys [1]: [i_item_id#29] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#25))] - -(47) CometFilter -Input [2]: [item_id#32, ws_item_rev#33] -Condition : isnotnull(ws_item_rev#33) - -(48) CometBroadcastExchange -Input [2]: [item_id#32, ws_item_rev#33] -Arguments: [item_id#32, ws_item_rev#33] - -(49) CometBroadcastHashJoin -Left output [3]: [item_id#12, ss_item_rev#13, cs_item_rev#23] -Right output [2]: [item_id#32, ws_item_rev#33] -Arguments: [item_id#12], [item_id#32], Inner, ((((((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * ws_item_rev#33)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * ws_item_rev#33))) AND (cast(cs_item_rev#23 as decimal(19,3)) >= (0.9 * ws_item_rev#33))) AND (cast(cs_item_rev#23 as decimal(20,3)) <= (1.1 * ws_item_rev#33))) AND (cast(ws_item_rev#33 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(ws_item_rev#33 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) AND (cast(ws_item_rev#33 as decimal(19,3)) >= (0.9 * cs_item_rev#23))) AND (cast(ws_item_rev#33 as decimal(20,3)) <= (1.1 * cs_item_rev#23))), BuildRight - -(50) CometProject -Input [5]: [item_id#12, ss_item_rev#13, cs_item_rev#23, item_id#32, ws_item_rev#33] -Arguments: [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37], [item_id#12, ss_item_rev#13, (((ss_item_rev#13 / ((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33)) / 3) * 100) AS ss_dev#34, cs_item_rev#23, (((cs_item_rev#23 / ((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33)) / 3) * 100) AS cs_dev#35, ws_item_rev#33, (((ws_item_rev#33 / ((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33)) / 3) * 100) AS ws_dev#36, (((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33) / 3) AS average#37] - -(51) CometTakeOrderedAndProject -Input [8]: [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_id#12 ASC NULLS FIRST,ss_item_rev#13 ASC NULLS FIRST], output=[item_id#12,ss_item_rev#13,ss_dev#34,cs_item_rev#23,cs_dev#35,ws_item_rev#33,ws_dev#36,average#37]), [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37], 100, 0, [item_id#12 ASC NULLS FIRST, ss_item_rev#13 ASC NULLS FIRST], [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37] - -(52) CometColumnarToRow [codegen id : 1] -Input [8]: [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (62) -+- * CometColumnarToRow (61) - +- CometProject (60) - +- CometBroadcastHashJoin (59) - :- CometFilter (54) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - +- CometBroadcastExchange (58) - +- CometProject (57) - +- CometFilter (56) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#10, d_week_seq#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#39)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date#10, d_week_seq#38] -Condition : (isnotnull(d_week_seq#38) AND (d_week_seq#38 = ReusedSubquery Subquery scalar-subquery#39, [id=#40])) - -(57) CometProject -Input [2]: [d_date#10, d_week_seq#38] -Arguments: [d_date#10], [d_date#10] - -(58) CometBroadcastExchange -Input [1]: [d_date#10] -Arguments: [d_date#10] - -(59) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#10] -Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight - -(60) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(61) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(62) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#39, [id=#40] - -Subquery:3 Hosting operator id = 55 Hosting Expression = Subquery scalar-subquery#39, [id=#40] -* CometColumnarToRow (66) -+- CometProject (65) - +- CometFilter (64) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) - - -(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#41, d_week_seq#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] -ReadSchema: struct - -(64) CometFilter -Input [2]: [d_date#41, d_week_seq#42] -Condition : (isnotnull(d_date#41) AND (d_date#41 = 2000-01-03)) - -(65) CometProject -Input [2]: [d_date#41, d_week_seq#42] -Arguments: [d_week_seq#42], [d_week_seq#42] - -(66) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#42] - -Subquery:4 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#4 - -Subquery:5 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#26 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/simplified.txt deleted file mode 100644 index 9d8bcdb40a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,77 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - CometProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - CometBroadcastHashJoin [item_id,ss_item_rev,cs_item_rev,item_id,ws_item_rev] - CometProject [item_id,ss_item_rev,cs_item_rev] - CometBroadcastHashJoin [item_id,ss_item_rev,item_id,cs_item_rev] - CometFilter [item_id,ss_item_rev] - CometHashAggregate [sum] [item_id,ss_item_rev,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id] #1 - CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] - CometProject [ss_ext_sales_price,i_item_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometFilter [d_date,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [i_item_sk,i_item_id] #4 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #3 - CometBroadcastExchange [item_id,cs_item_rev] #6 - CometFilter [item_id,cs_item_rev] - CometHashAggregate [sum] [item_id,cs_item_rev,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id] #7 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - 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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [item_id,ws_item_rev] #8 - CometFilter [item_id,ws_item_rev] - CometHashAggregate [sum] [item_id,ws_item_rev,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - 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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/explain.txt deleted file mode 100644 index 3c368afc65..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/explain.txt +++ /dev/null @@ -1,383 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (52) -+- CometTakeOrderedAndProject (51) - +- CometProject (50) - +- CometBroadcastHashJoin (49) - :- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometFilter (20) - : : +- CometHashAggregate (19) - : : +- CometExchange (18) - : : +- CometHashAggregate (17) - : : +- CometProject (16) - : : +- CometBroadcastHashJoin (15) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (14) - : : +- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- ReusedExchange (11) - : +- CometBroadcastExchange (33) - : +- CometFilter (32) - : +- CometHashAggregate (31) - : +- CometExchange (30) - : +- CometHashAggregate (29) - : +- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (21) - : : +- ReusedExchange (23) - : +- ReusedExchange (26) - +- CometBroadcastExchange (48) - +- CometFilter (47) - +- CometHashAggregate (46) - +- CometExchange (45) - +- CometHashAggregate (44) - +- CometProject (43) - +- CometBroadcastHashJoin (42) - :- CometProject (40) - : +- CometBroadcastHashJoin (39) - : :- CometFilter (37) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (36) - : +- ReusedExchange (38) - +- ReusedExchange (41) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_item_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_item_id#6] -Condition : (isnotnull(i_item_sk#5) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)))) - -(5) CometProject -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#7] - -(6) CometBroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#7] -Arguments: [i_item_sk#5, i_item_id#7] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [2]: [i_item_sk#5, i_item_id#7] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#7] -Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(11) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date#10] - -(12) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#10] -Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight - -(13) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: [d_date_sk#8] - -(15) CometBroadcastHashJoin -Left output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7] -Right output [1]: [d_date_sk#8] -Arguments: [ss_sold_date_sk#3], [d_date_sk#8], Inner, BuildRight - -(16) CometProject -Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7, d_date_sk#8] -Arguments: [ss_ext_sales_price#2, i_item_id#7], [ss_ext_sales_price#2, i_item_id#7] - -(17) CometHashAggregate -Input [2]: [ss_ext_sales_price#2, i_item_id#7] -Keys [1]: [i_item_id#7] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(18) CometExchange -Input [2]: [i_item_id#7, sum#11] -Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [2]: [i_item_id#7, sum#11] -Keys [1]: [i_item_id#7] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(20) CometFilter -Input [2]: [item_id#12, ss_item_rev#13] -Condition : isnotnull(ss_item_rev#13) - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#14) - -(23) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#18, i_item_id#19] - -(24) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Right output [2]: [i_item_sk#18, i_item_id#19] -Arguments: [cs_item_sk#14], [i_item_sk#18], Inner, BuildRight - -(25) CometProject -Input [5]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_sk#18, i_item_id#19] -Arguments: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19], [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19] - -(26) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#20] - -(27) CometBroadcastHashJoin -Left output [3]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19] -Right output [1]: [d_date_sk#20] -Arguments: [cs_sold_date_sk#16], [d_date_sk#20], Inner, BuildRight - -(28) CometProject -Input [4]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19, d_date_sk#20] -Arguments: [cs_ext_sales_price#15, i_item_id#19], [cs_ext_sales_price#15, i_item_id#19] - -(29) CometHashAggregate -Input [2]: [cs_ext_sales_price#15, i_item_id#19] -Keys [1]: [i_item_id#19] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#15))] - -(30) CometExchange -Input [2]: [i_item_id#19, sum#21] -Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(31) CometHashAggregate -Input [2]: [i_item_id#19, sum#21] -Keys [1]: [i_item_id#19] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#15))] - -(32) CometFilter -Input [2]: [item_id#22, cs_item_rev#23] -Condition : isnotnull(cs_item_rev#23) - -(33) CometBroadcastExchange -Input [2]: [item_id#22, cs_item_rev#23] -Arguments: [item_id#22, cs_item_rev#23] - -(34) CometBroadcastHashJoin -Left output [2]: [item_id#12, ss_item_rev#13] -Right output [2]: [item_id#22, cs_item_rev#23] -Arguments: [item_id#12], [item_id#22], Inner, ((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * cs_item_rev#23)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * cs_item_rev#23))) AND (cast(cs_item_rev#23 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(cs_item_rev#23 as decimal(20,3)) <= (1.1 * ss_item_rev#13))), BuildRight - -(35) CometProject -Input [4]: [item_id#12, ss_item_rev#13, item_id#22, cs_item_rev#23] -Arguments: [item_id#12, ss_item_rev#13, cs_item_rev#23], [item_id#12, ss_item_rev#13, cs_item_rev#23] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#26), dynamicpruningexpression(ws_sold_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26] -Condition : isnotnull(ws_item_sk#24) - -(38) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#28, i_item_id#29] - -(39) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26] -Right output [2]: [i_item_sk#28, i_item_id#29] -Arguments: [ws_item_sk#24], [i_item_sk#28], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_sk#28, i_item_id#29] -Arguments: [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29], [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29] - -(41) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#30] - -(42) CometBroadcastHashJoin -Left output [3]: [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29] -Right output [1]: [d_date_sk#30] -Arguments: [ws_sold_date_sk#26], [d_date_sk#30], Inner, BuildRight - -(43) CometProject -Input [4]: [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29, d_date_sk#30] -Arguments: [ws_ext_sales_price#25, i_item_id#29], [ws_ext_sales_price#25, i_item_id#29] - -(44) CometHashAggregate -Input [2]: [ws_ext_sales_price#25, i_item_id#29] -Keys [1]: [i_item_id#29] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#25))] - -(45) CometExchange -Input [2]: [i_item_id#29, sum#31] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(46) CometHashAggregate -Input [2]: [i_item_id#29, sum#31] -Keys [1]: [i_item_id#29] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#25))] - -(47) CometFilter -Input [2]: [item_id#32, ws_item_rev#33] -Condition : isnotnull(ws_item_rev#33) - -(48) CometBroadcastExchange -Input [2]: [item_id#32, ws_item_rev#33] -Arguments: [item_id#32, ws_item_rev#33] - -(49) CometBroadcastHashJoin -Left output [3]: [item_id#12, ss_item_rev#13, cs_item_rev#23] -Right output [2]: [item_id#32, ws_item_rev#33] -Arguments: [item_id#12], [item_id#32], Inner, ((((((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * ws_item_rev#33)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * ws_item_rev#33))) AND (cast(cs_item_rev#23 as decimal(19,3)) >= (0.9 * ws_item_rev#33))) AND (cast(cs_item_rev#23 as decimal(20,3)) <= (1.1 * ws_item_rev#33))) AND (cast(ws_item_rev#33 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(ws_item_rev#33 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) AND (cast(ws_item_rev#33 as decimal(19,3)) >= (0.9 * cs_item_rev#23))) AND (cast(ws_item_rev#33 as decimal(20,3)) <= (1.1 * cs_item_rev#23))), BuildRight - -(50) CometProject -Input [5]: [item_id#12, ss_item_rev#13, cs_item_rev#23, item_id#32, ws_item_rev#33] -Arguments: [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37], [item_id#12, ss_item_rev#13, (((ss_item_rev#13 / ((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33)) / 3) * 100) AS ss_dev#34, cs_item_rev#23, (((cs_item_rev#23 / ((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33)) / 3) * 100) AS cs_dev#35, ws_item_rev#33, (((ws_item_rev#33 / ((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33)) / 3) * 100) AS ws_dev#36, (((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33) / 3) AS average#37] - -(51) CometTakeOrderedAndProject -Input [8]: [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_id#12 ASC NULLS FIRST,ss_item_rev#13 ASC NULLS FIRST], output=[item_id#12,ss_item_rev#13,ss_dev#34,cs_item_rev#23,cs_dev#35,ws_item_rev#33,ws_dev#36,average#37]), [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37], 100, 0, [item_id#12 ASC NULLS FIRST, ss_item_rev#13 ASC NULLS FIRST], [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37] - -(52) CometColumnarToRow [codegen id : 1] -Input [8]: [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (62) -+- * CometColumnarToRow (61) - +- CometProject (60) - +- CometBroadcastHashJoin (59) - :- CometFilter (54) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - +- CometBroadcastExchange (58) - +- CometProject (57) - +- CometFilter (56) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#10, d_week_seq#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#39)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date#10, d_week_seq#38] -Condition : (isnotnull(d_week_seq#38) AND (d_week_seq#38 = ReusedSubquery Subquery scalar-subquery#39, [id=#40])) - -(57) CometProject -Input [2]: [d_date#10, d_week_seq#38] -Arguments: [d_date#10], [d_date#10] - -(58) CometBroadcastExchange -Input [1]: [d_date#10] -Arguments: [d_date#10] - -(59) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#10] -Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight - -(60) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(61) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(62) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#39, [id=#40] - -Subquery:3 Hosting operator id = 55 Hosting Expression = Subquery scalar-subquery#39, [id=#40] -* CometColumnarToRow (66) -+- CometProject (65) - +- CometFilter (64) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) - - -(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#41, d_week_seq#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] -ReadSchema: struct - -(64) CometFilter -Input [2]: [d_date#41, d_week_seq#42] -Condition : (isnotnull(d_date#41) AND (d_date#41 = 2000-01-03)) - -(65) CometProject -Input [2]: [d_date#41, d_week_seq#42] -Arguments: [d_week_seq#42], [d_week_seq#42] - -(66) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#42] - -Subquery:4 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#4 - -Subquery:5 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#26 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/extended.txt deleted file mode 100644 index b4255cb7a1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/extended.txt +++ /dev/null @@ -1,116 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- ReusedSubquery - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Subquery - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 96 out of 108 eligible operators (88%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 9d8bcdb40a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt +++ /dev/null @@ -1,77 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - CometProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - CometBroadcastHashJoin [item_id,ss_item_rev,cs_item_rev,item_id,ws_item_rev] - CometProject [item_id,ss_item_rev,cs_item_rev] - CometBroadcastHashJoin [item_id,ss_item_rev,item_id,cs_item_rev] - CometFilter [item_id,ss_item_rev] - CometHashAggregate [sum] [item_id,ss_item_rev,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id] #1 - CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] - CometProject [ss_ext_sales_price,i_item_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometFilter [d_date,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [i_item_sk,i_item_id] #4 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #3 - CometBroadcastExchange [item_id,cs_item_rev] #6 - CometFilter [item_id,cs_item_rev] - CometHashAggregate [sum] [item_id,cs_item_rev,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id] #7 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - 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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [item_id,ws_item_rev] #8 - CometFilter [item_id,ws_item_rev] - CometHashAggregate [sum] [item_id,ws_item_rev,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - 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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_datafusion/explain.txt deleted file mode 100644 index 9270d963a1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_datafusion/explain.txt +++ /dev/null @@ -1,239 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (42) -+- CometTakeOrderedAndProject (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometNativeScan parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (15) - : : +- CometProject (14) - : : +- CometFilter (13) - : : +- CometNativeScan parquet spark_catalog.default.store (12) - : +- CometBroadcastExchange (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometNativeScan parquet spark_catalog.default.date_dim (18) - +- CometBroadcastExchange (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometHashAggregate (25) - : : +- ReusedExchange (24) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometNativeScan parquet spark_catalog.default.store (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.date_dim (32) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#3)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Condition : (isnotnull(d_date_sk#4) AND isnotnull(d_week_seq#5)) - -(5) CometProject -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Arguments: [d_date_sk#4, d_week_seq#5, d_day_name#7], [d_date_sk#4, d_week_seq#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(d_day_name#6, 9)) AS d_day_name#7] - -(6) CometBroadcastExchange -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [d_date_sk#4, d_week_seq#5, d_day_name#7] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Right output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [ss_sold_date_sk#3], [d_date_sk#4], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3, d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7], [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7] - -(9) CometHashAggregate -Input [4]: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7] -Keys [2]: [d_week_seq#5, ss_store_sk#1] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] - -(10) CometExchange -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] -Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] -Keys [2]: [d_week_seq#5, ss_store_sk#1] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] - -(12) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(13) CometFilter -Input [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Condition : (isnotnull(s_store_sk#15) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#16, 16)))) - -(14) CometProject -Input [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Arguments: [s_store_sk#15, s_store_id#18, s_store_name#17], [s_store_sk#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#16, 16)) AS s_store_id#18, s_store_name#17] - -(15) CometBroadcastExchange -Input [3]: [s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [s_store_sk#15, s_store_id#18, s_store_name#17] - -(16) CometBroadcastHashJoin -Left output [9]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25] -Right output [3]: [s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [ss_store_sk#1], [s_store_sk#15], Inner, BuildRight - -(17) CometProject -Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17], [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17] - -(18) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#26, d_week_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_week_seq)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [d_month_seq#26, d_week_seq#27] -Condition : (((isnotnull(d_month_seq#26) AND (d_month_seq#26 >= 1212)) AND (d_month_seq#26 <= 1223)) AND isnotnull(d_week_seq#27)) - -(20) CometProject -Input [2]: [d_month_seq#26, d_week_seq#27] -Arguments: [d_week_seq#27], [d_week_seq#27] - -(21) CometBroadcastExchange -Input [1]: [d_week_seq#27] -Arguments: [d_week_seq#27] - -(22) CometBroadcastHashJoin -Left output [10]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17] -Right output [1]: [d_week_seq#27] -Arguments: [d_week_seq#5], [d_week_seq#27], Inner, BuildRight - -(23) CometProject -Input [11]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17, d_week_seq#27] -Arguments: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37], [s_store_name#17 AS s_store_name1#28, d_week_seq#5 AS d_week_seq1#29, s_store_id#18 AS s_store_id1#30, sun_sales#19 AS sun_sales1#31, mon_sales#20 AS mon_sales1#32, tue_sales#21 AS tue_sales1#33, wed_sales#22 AS wed_sales1#34, thu_sales#23 AS thu_sales1#35, fri_sales#24 AS fri_sales1#36, sat_sales#25 AS sat_sales1#37] - -(24) ReusedExchange [Reuses operator id: 10] -Output [9]: [d_week_seq#38, ss_store_sk#39, sum#40, sum#41, sum#42, sum#43, sum#44, sum#45, sum#46] - -(25) CometHashAggregate -Input [9]: [d_week_seq#38, ss_store_sk#39, sum#40, sum#41, sum#42, sum#43, sum#44, sum#45, sum#46] -Keys [2]: [d_week_seq#38, ss_store_sk#39] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#47 = Sunday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Monday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Tuesday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Wednesday) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Thursday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Friday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Saturday ) THEN ss_sales_price#48 END))] - -(26) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#49, s_store_id#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [s_store_sk#49, s_store_id#50] -Condition : (isnotnull(s_store_sk#49) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#50, 16)))) - -(28) CometProject -Input [2]: [s_store_sk#49, s_store_id#50] -Arguments: [s_store_sk#49, s_store_id#51], [s_store_sk#49, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#50, 16)) AS s_store_id#51] - -(29) CometBroadcastExchange -Input [2]: [s_store_sk#49, s_store_id#51] -Arguments: [s_store_sk#49, s_store_id#51] - -(30) CometBroadcastHashJoin -Left output [9]: [d_week_seq#38, ss_store_sk#39, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58] -Right output [2]: [s_store_sk#49, s_store_id#51] -Arguments: [ss_store_sk#39], [s_store_sk#49], Inner, BuildRight - -(31) CometProject -Input [11]: [d_week_seq#38, ss_store_sk#39, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_sk#49, s_store_id#51] -Arguments: [d_week_seq#38, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_id#51], [d_week_seq#38, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_id#51] - -(32) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#59, d_week_seq#60] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1224), LessThanOrEqual(d_month_seq,1235), IsNotNull(d_week_seq)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_month_seq#59, d_week_seq#60] -Condition : (((isnotnull(d_month_seq#59) AND (d_month_seq#59 >= 1224)) AND (d_month_seq#59 <= 1235)) AND isnotnull(d_week_seq#60)) - -(34) CometProject -Input [2]: [d_month_seq#59, d_week_seq#60] -Arguments: [d_week_seq#60], [d_week_seq#60] - -(35) CometBroadcastExchange -Input [1]: [d_week_seq#60] -Arguments: [d_week_seq#60] - -(36) CometBroadcastHashJoin -Left output [9]: [d_week_seq#38, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_id#51] -Right output [1]: [d_week_seq#60] -Arguments: [d_week_seq#38], [d_week_seq#60], Inner, BuildRight - -(37) CometProject -Input [10]: [d_week_seq#38, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_id#51, d_week_seq#60] -Arguments: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69], [d_week_seq#38 AS d_week_seq2#61, s_store_id#51 AS s_store_id2#62, sun_sales#52 AS sun_sales2#63, mon_sales#53 AS mon_sales2#64, tue_sales#54 AS tue_sales2#65, wed_sales#55 AS wed_sales2#66, thu_sales#56 AS thu_sales2#67, fri_sales#57 AS fri_sales2#68, sat_sales#58 AS sat_sales2#69] - -(38) CometBroadcastExchange -Input [9]: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] -Arguments: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] - -(39) CometBroadcastHashJoin -Left output [10]: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37] -Right output [9]: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] -Arguments: [s_store_id1#30, d_week_seq1#29], [s_store_id2#62, (d_week_seq2#61 - 52)], Inner, BuildRight - -(40) CometProject -Input [19]: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37, d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] -Arguments: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1#31 / sun_sales2#63) AS (sun_sales1 / sun_sales2)#70, (mon_sales1#32 / mon_sales2#64) AS (mon_sales1 / mon_sales2)#71, (tue_sales1#33 / tue_sales2#65) AS (tue_sales1 / tue_sales2)#72, (wed_sales1#34 / wed_sales2#66) AS (wed_sales1 / wed_sales2)#73, (thu_sales1#35 / thu_sales2#67) AS (thu_sales1 / thu_sales2)#74, (fri_sales1#36 / fri_sales2#68) AS (fri_sales1 / fri_sales2)#75, (sat_sales1#37 / sat_sales2#69) AS (sat_sales1 / sat_sales2)#76] - -(41) CometTakeOrderedAndProject -Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name1#28 ASC NULLS FIRST,s_store_id1#30 ASC NULLS FIRST,d_week_seq1#29 ASC NULLS FIRST], output=[s_store_name1#28,s_store_id1#30,d_week_seq1#29,(sun_sales1 / sun_sales2)#70,(mon_sales1 / mon_sales2)#71,(tue_sales1 / tue_sales2)#72,(wed_sales1 / wed_sales2)#73,(thu_sales1 / thu_sales2)#74,(fri_sales1 / fri_sales2)#75,(sat_sales1 / sat_sales2)#76]), [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76], 100, 0, [s_store_name1#28 ASC NULLS FIRST, s_store_id1#30 ASC NULLS FIRST, d_week_seq1#29 ASC NULLS FIRST], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] - -(42) CometColumnarToRow [codegen id : 1] -Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_datafusion/simplified.txt deleted file mode 100644 index 72823b14c1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_datafusion/simplified.txt +++ /dev/null @@ -1,44 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometBroadcastHashJoin [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometProject [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id,s_store_name] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - CometExchange [d_week_seq,ss_store_sk] #1 - CometHashAggregate [d_day_name,ss_sales_price] [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] - CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] - CometBroadcastExchange [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 - CometBroadcastExchange [s_store_sk,s_store_id] #6 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [d_week_seq] #7 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/explain.txt deleted file mode 100644 index 8fb48d458e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,239 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (42) -+- CometTakeOrderedAndProject (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (15) - : : +- CometProject (14) - : : +- CometFilter (13) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (12) - : +- CometBroadcastExchange (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (18) - +- CometBroadcastExchange (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometHashAggregate (25) - : : +- ReusedExchange (24) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Condition : (isnotnull(d_date_sk#4) AND isnotnull(d_week_seq#5)) - -(5) CometProject -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Arguments: [d_date_sk#4, d_week_seq#5, d_day_name#7], [d_date_sk#4, d_week_seq#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(d_day_name#6, 9)) AS d_day_name#7] - -(6) CometBroadcastExchange -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [d_date_sk#4, d_week_seq#5, d_day_name#7] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Right output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [ss_sold_date_sk#3], [d_date_sk#4], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3, d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7], [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7] - -(9) CometHashAggregate -Input [4]: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7] -Keys [2]: [d_week_seq#5, ss_store_sk#1] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] - -(10) CometExchange -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] -Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] -Keys [2]: [d_week_seq#5, ss_store_sk#1] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(13) CometFilter -Input [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Condition : (isnotnull(s_store_sk#15) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#16, 16)))) - -(14) CometProject -Input [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Arguments: [s_store_sk#15, s_store_id#18, s_store_name#17], [s_store_sk#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#16, 16)) AS s_store_id#18, s_store_name#17] - -(15) CometBroadcastExchange -Input [3]: [s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [s_store_sk#15, s_store_id#18, s_store_name#17] - -(16) CometBroadcastHashJoin -Left output [9]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25] -Right output [3]: [s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [ss_store_sk#1], [s_store_sk#15], Inner, BuildRight - -(17) CometProject -Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17], [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#26, d_week_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_week_seq)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [d_month_seq#26, d_week_seq#27] -Condition : (((isnotnull(d_month_seq#26) AND (d_month_seq#26 >= 1212)) AND (d_month_seq#26 <= 1223)) AND isnotnull(d_week_seq#27)) - -(20) CometProject -Input [2]: [d_month_seq#26, d_week_seq#27] -Arguments: [d_week_seq#27], [d_week_seq#27] - -(21) CometBroadcastExchange -Input [1]: [d_week_seq#27] -Arguments: [d_week_seq#27] - -(22) CometBroadcastHashJoin -Left output [10]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17] -Right output [1]: [d_week_seq#27] -Arguments: [d_week_seq#5], [d_week_seq#27], Inner, BuildRight - -(23) CometProject -Input [11]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17, d_week_seq#27] -Arguments: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37], [s_store_name#17 AS s_store_name1#28, d_week_seq#5 AS d_week_seq1#29, s_store_id#18 AS s_store_id1#30, sun_sales#19 AS sun_sales1#31, mon_sales#20 AS mon_sales1#32, tue_sales#21 AS tue_sales1#33, wed_sales#22 AS wed_sales1#34, thu_sales#23 AS thu_sales1#35, fri_sales#24 AS fri_sales1#36, sat_sales#25 AS sat_sales1#37] - -(24) ReusedExchange [Reuses operator id: 10] -Output [9]: [d_week_seq#38, ss_store_sk#39, sum#40, sum#41, sum#42, sum#43, sum#44, sum#45, sum#46] - -(25) CometHashAggregate -Input [9]: [d_week_seq#38, ss_store_sk#39, sum#40, sum#41, sum#42, sum#43, sum#44, sum#45, sum#46] -Keys [2]: [d_week_seq#38, ss_store_sk#39] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#47 = Sunday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Monday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Tuesday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Wednesday) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Thursday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Friday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Saturday ) THEN ss_sales_price#48 END))] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#49, s_store_id#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [s_store_sk#49, s_store_id#50] -Condition : (isnotnull(s_store_sk#49) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#50, 16)))) - -(28) CometProject -Input [2]: [s_store_sk#49, s_store_id#50] -Arguments: [s_store_sk#49, s_store_id#51], [s_store_sk#49, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#50, 16)) AS s_store_id#51] - -(29) CometBroadcastExchange -Input [2]: [s_store_sk#49, s_store_id#51] -Arguments: [s_store_sk#49, s_store_id#51] - -(30) CometBroadcastHashJoin -Left output [9]: [d_week_seq#38, ss_store_sk#39, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58] -Right output [2]: [s_store_sk#49, s_store_id#51] -Arguments: [ss_store_sk#39], [s_store_sk#49], Inner, BuildRight - -(31) CometProject -Input [11]: [d_week_seq#38, ss_store_sk#39, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_sk#49, s_store_id#51] -Arguments: [d_week_seq#38, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_id#51], [d_week_seq#38, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_id#51] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#59, d_week_seq#60] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1224), LessThanOrEqual(d_month_seq,1235), IsNotNull(d_week_seq)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_month_seq#59, d_week_seq#60] -Condition : (((isnotnull(d_month_seq#59) AND (d_month_seq#59 >= 1224)) AND (d_month_seq#59 <= 1235)) AND isnotnull(d_week_seq#60)) - -(34) CometProject -Input [2]: [d_month_seq#59, d_week_seq#60] -Arguments: [d_week_seq#60], [d_week_seq#60] - -(35) CometBroadcastExchange -Input [1]: [d_week_seq#60] -Arguments: [d_week_seq#60] - -(36) CometBroadcastHashJoin -Left output [9]: [d_week_seq#38, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_id#51] -Right output [1]: [d_week_seq#60] -Arguments: [d_week_seq#38], [d_week_seq#60], Inner, BuildRight - -(37) CometProject -Input [10]: [d_week_seq#38, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_id#51, d_week_seq#60] -Arguments: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69], [d_week_seq#38 AS d_week_seq2#61, s_store_id#51 AS s_store_id2#62, sun_sales#52 AS sun_sales2#63, mon_sales#53 AS mon_sales2#64, tue_sales#54 AS tue_sales2#65, wed_sales#55 AS wed_sales2#66, thu_sales#56 AS thu_sales2#67, fri_sales#57 AS fri_sales2#68, sat_sales#58 AS sat_sales2#69] - -(38) CometBroadcastExchange -Input [9]: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] -Arguments: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] - -(39) CometBroadcastHashJoin -Left output [10]: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37] -Right output [9]: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] -Arguments: [s_store_id1#30, d_week_seq1#29], [s_store_id2#62, (d_week_seq2#61 - 52)], Inner, BuildRight - -(40) CometProject -Input [19]: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37, d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] -Arguments: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1#31 / sun_sales2#63) AS (sun_sales1 / sun_sales2)#70, (mon_sales1#32 / mon_sales2#64) AS (mon_sales1 / mon_sales2)#71, (tue_sales1#33 / tue_sales2#65) AS (tue_sales1 / tue_sales2)#72, (wed_sales1#34 / wed_sales2#66) AS (wed_sales1 / wed_sales2)#73, (thu_sales1#35 / thu_sales2#67) AS (thu_sales1 / thu_sales2)#74, (fri_sales1#36 / fri_sales2#68) AS (fri_sales1 / fri_sales2)#75, (sat_sales1#37 / sat_sales2#69) AS (sat_sales1 / sat_sales2)#76] - -(41) CometTakeOrderedAndProject -Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name1#28 ASC NULLS FIRST,s_store_id1#30 ASC NULLS FIRST,d_week_seq1#29 ASC NULLS FIRST], output=[s_store_name1#28,s_store_id1#30,d_week_seq1#29,(sun_sales1 / sun_sales2)#70,(mon_sales1 / mon_sales2)#71,(tue_sales1 / tue_sales2)#72,(wed_sales1 / wed_sales2)#73,(thu_sales1 / thu_sales2)#74,(fri_sales1 / fri_sales2)#75,(sat_sales1 / sat_sales2)#76]), [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76], 100, 0, [s_store_name1#28 ASC NULLS FIRST, s_store_id1#30 ASC NULLS FIRST, d_week_seq1#29 ASC NULLS FIRST], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] - -(42) CometColumnarToRow [codegen id : 1] -Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/simplified.txt deleted file mode 100644 index bca9c6463d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,44 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometBroadcastHashJoin [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometProject [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id,s_store_name] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - CometExchange [d_week_seq,ss_store_sk] #1 - CometHashAggregate [d_day_name,ss_sales_price] [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] - CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] - CometBroadcastExchange [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 - CometBroadcastExchange [s_store_sk,s_store_id] #6 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [d_week_seq] #7 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/explain.txt deleted file mode 100644 index 8fb48d458e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/explain.txt +++ /dev/null @@ -1,239 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (42) -+- CometTakeOrderedAndProject (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (15) - : : +- CometProject (14) - : : +- CometFilter (13) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (12) - : +- CometBroadcastExchange (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (18) - +- CometBroadcastExchange (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometHashAggregate (25) - : : +- ReusedExchange (24) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Condition : (isnotnull(d_date_sk#4) AND isnotnull(d_week_seq#5)) - -(5) CometProject -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Arguments: [d_date_sk#4, d_week_seq#5, d_day_name#7], [d_date_sk#4, d_week_seq#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(d_day_name#6, 9)) AS d_day_name#7] - -(6) CometBroadcastExchange -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [d_date_sk#4, d_week_seq#5, d_day_name#7] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Right output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [ss_sold_date_sk#3], [d_date_sk#4], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3, d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7], [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7] - -(9) CometHashAggregate -Input [4]: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7] -Keys [2]: [d_week_seq#5, ss_store_sk#1] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] - -(10) CometExchange -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] -Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] -Keys [2]: [d_week_seq#5, ss_store_sk#1] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(13) CometFilter -Input [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Condition : (isnotnull(s_store_sk#15) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#16, 16)))) - -(14) CometProject -Input [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Arguments: [s_store_sk#15, s_store_id#18, s_store_name#17], [s_store_sk#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#16, 16)) AS s_store_id#18, s_store_name#17] - -(15) CometBroadcastExchange -Input [3]: [s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [s_store_sk#15, s_store_id#18, s_store_name#17] - -(16) CometBroadcastHashJoin -Left output [9]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25] -Right output [3]: [s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [ss_store_sk#1], [s_store_sk#15], Inner, BuildRight - -(17) CometProject -Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17], [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#26, d_week_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_week_seq)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [d_month_seq#26, d_week_seq#27] -Condition : (((isnotnull(d_month_seq#26) AND (d_month_seq#26 >= 1212)) AND (d_month_seq#26 <= 1223)) AND isnotnull(d_week_seq#27)) - -(20) CometProject -Input [2]: [d_month_seq#26, d_week_seq#27] -Arguments: [d_week_seq#27], [d_week_seq#27] - -(21) CometBroadcastExchange -Input [1]: [d_week_seq#27] -Arguments: [d_week_seq#27] - -(22) CometBroadcastHashJoin -Left output [10]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17] -Right output [1]: [d_week_seq#27] -Arguments: [d_week_seq#5], [d_week_seq#27], Inner, BuildRight - -(23) CometProject -Input [11]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17, d_week_seq#27] -Arguments: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37], [s_store_name#17 AS s_store_name1#28, d_week_seq#5 AS d_week_seq1#29, s_store_id#18 AS s_store_id1#30, sun_sales#19 AS sun_sales1#31, mon_sales#20 AS mon_sales1#32, tue_sales#21 AS tue_sales1#33, wed_sales#22 AS wed_sales1#34, thu_sales#23 AS thu_sales1#35, fri_sales#24 AS fri_sales1#36, sat_sales#25 AS sat_sales1#37] - -(24) ReusedExchange [Reuses operator id: 10] -Output [9]: [d_week_seq#38, ss_store_sk#39, sum#40, sum#41, sum#42, sum#43, sum#44, sum#45, sum#46] - -(25) CometHashAggregate -Input [9]: [d_week_seq#38, ss_store_sk#39, sum#40, sum#41, sum#42, sum#43, sum#44, sum#45, sum#46] -Keys [2]: [d_week_seq#38, ss_store_sk#39] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#47 = Sunday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Monday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Tuesday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Wednesday) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Thursday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Friday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Saturday ) THEN ss_sales_price#48 END))] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#49, s_store_id#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [s_store_sk#49, s_store_id#50] -Condition : (isnotnull(s_store_sk#49) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#50, 16)))) - -(28) CometProject -Input [2]: [s_store_sk#49, s_store_id#50] -Arguments: [s_store_sk#49, s_store_id#51], [s_store_sk#49, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#50, 16)) AS s_store_id#51] - -(29) CometBroadcastExchange -Input [2]: [s_store_sk#49, s_store_id#51] -Arguments: [s_store_sk#49, s_store_id#51] - -(30) CometBroadcastHashJoin -Left output [9]: [d_week_seq#38, ss_store_sk#39, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58] -Right output [2]: [s_store_sk#49, s_store_id#51] -Arguments: [ss_store_sk#39], [s_store_sk#49], Inner, BuildRight - -(31) CometProject -Input [11]: [d_week_seq#38, ss_store_sk#39, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_sk#49, s_store_id#51] -Arguments: [d_week_seq#38, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_id#51], [d_week_seq#38, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_id#51] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#59, d_week_seq#60] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1224), LessThanOrEqual(d_month_seq,1235), IsNotNull(d_week_seq)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_month_seq#59, d_week_seq#60] -Condition : (((isnotnull(d_month_seq#59) AND (d_month_seq#59 >= 1224)) AND (d_month_seq#59 <= 1235)) AND isnotnull(d_week_seq#60)) - -(34) CometProject -Input [2]: [d_month_seq#59, d_week_seq#60] -Arguments: [d_week_seq#60], [d_week_seq#60] - -(35) CometBroadcastExchange -Input [1]: [d_week_seq#60] -Arguments: [d_week_seq#60] - -(36) CometBroadcastHashJoin -Left output [9]: [d_week_seq#38, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_id#51] -Right output [1]: [d_week_seq#60] -Arguments: [d_week_seq#38], [d_week_seq#60], Inner, BuildRight - -(37) CometProject -Input [10]: [d_week_seq#38, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_id#51, d_week_seq#60] -Arguments: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69], [d_week_seq#38 AS d_week_seq2#61, s_store_id#51 AS s_store_id2#62, sun_sales#52 AS sun_sales2#63, mon_sales#53 AS mon_sales2#64, tue_sales#54 AS tue_sales2#65, wed_sales#55 AS wed_sales2#66, thu_sales#56 AS thu_sales2#67, fri_sales#57 AS fri_sales2#68, sat_sales#58 AS sat_sales2#69] - -(38) CometBroadcastExchange -Input [9]: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] -Arguments: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] - -(39) CometBroadcastHashJoin -Left output [10]: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37] -Right output [9]: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] -Arguments: [s_store_id1#30, d_week_seq1#29], [s_store_id2#62, (d_week_seq2#61 - 52)], Inner, BuildRight - -(40) CometProject -Input [19]: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37, d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] -Arguments: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1#31 / sun_sales2#63) AS (sun_sales1 / sun_sales2)#70, (mon_sales1#32 / mon_sales2#64) AS (mon_sales1 / mon_sales2)#71, (tue_sales1#33 / tue_sales2#65) AS (tue_sales1 / tue_sales2)#72, (wed_sales1#34 / wed_sales2#66) AS (wed_sales1 / wed_sales2)#73, (thu_sales1#35 / thu_sales2#67) AS (thu_sales1 / thu_sales2)#74, (fri_sales1#36 / fri_sales2#68) AS (fri_sales1 / fri_sales2)#75, (sat_sales1#37 / sat_sales2#69) AS (sat_sales1 / sat_sales2)#76] - -(41) CometTakeOrderedAndProject -Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name1#28 ASC NULLS FIRST,s_store_id1#30 ASC NULLS FIRST,d_week_seq1#29 ASC NULLS FIRST], output=[s_store_name1#28,s_store_id1#30,d_week_seq1#29,(sun_sales1 / sun_sales2)#70,(mon_sales1 / mon_sales2)#71,(tue_sales1 / tue_sales2)#72,(wed_sales1 / wed_sales2)#73,(thu_sales1 / thu_sales2)#74,(fri_sales1 / fri_sales2)#75,(sat_sales1 / sat_sales2)#76]), [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76], 100, 0, [s_store_name1#28 ASC NULLS FIRST, s_store_id1#30 ASC NULLS FIRST, d_week_seq1#29 ASC NULLS FIRST], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] - -(42) CometColumnarToRow [codegen id : 1] -Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/extended.txt deleted file mode 100644 index 3d50a2131e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/extended.txt +++ /dev/null @@ -1,53 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 50 out of 50 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index bca9c6463d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt +++ /dev/null @@ -1,44 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometBroadcastHashJoin [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometProject [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id,s_store_name] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - CometExchange [d_week_seq,ss_store_sk] #1 - CometHashAggregate [d_day_name,ss_sales_price] [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] - CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] - CometBroadcastExchange [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 - CometBroadcastExchange [s_store_sk,s_store_id] #6 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [d_week_seq] #7 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/explain.txt deleted file mode 100644 index 48c6b7db44..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (40) -+- * Filter (39) - +- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * CometColumnarToRow (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.customer (4) - : : +- BroadcastExchange (13) - : : +- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet spark_catalog.default.store_sales (10) - : +- ReusedExchange (16) - +- BroadcastExchange (32) - +- * CometColumnarToRow (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (20) - : +- CometNativeScan parquet spark_catalog.default.item (19) - +- CometBroadcastExchange (28) - +- CometFilter (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometFilter (22) - +- CometNativeScan parquet spark_catalog.default.item (21) - - -(1) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#1, ca_state#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ca_address_sk#1, ca_state#2] -Condition : isnotnull(ca_address_sk#1) - -(3) CometProject -Input [2]: [ca_address_sk#1, ca_state#2] -Arguments: [ca_address_sk#1, ca_state#3], [ca_address_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#2, 2)) AS ca_state#3] - -(4) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_customer_sk#4)) - -(6) CometBroadcastExchange -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [c_customer_sk#4, c_current_addr_sk#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#1, ca_state#3] -Right output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] - -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ca_state#3, c_customer_sk#4] - -(10) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 1] -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(12) Filter [codegen id : 1] -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) - -(13) BroadcastExchange -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [c_customer_sk#4] -Right keys [1]: [ss_customer_sk#7] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(16) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#10] - -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [2]: [ca_state#3, ss_item_sk#6] -Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] - -(19) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#11, i_current_price#12, i_category#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_category), IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [i_item_sk#11, i_current_price#12, i_category#13] -Condition : ((isnotnull(i_current_price#12) AND isnotnull(i_category#13)) AND isnotnull(i_item_sk#11)) - -(21) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_current_price#14, i_category#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_current_price#14, i_category#15] -Condition : isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#15, 50))) - -(23) CometProject -Input [2]: [i_current_price#14, i_category#15] -Arguments: [i_category#16, i_current_price#14], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#15, 50)) AS i_category#16, i_current_price#14] - -(24) CometHashAggregate -Input [2]: [i_category#16, i_current_price#14] -Keys [1]: [i_category#16] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#14))] - -(25) CometExchange -Input [3]: [i_category#16, sum#17, count#18] -Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(26) CometHashAggregate -Input [3]: [i_category#16, sum#17, count#18] -Keys [1]: [i_category#16] -Functions [1]: [avg(UnscaledValue(i_current_price#14))] - -(27) CometFilter -Input [2]: [avg(i_current_price)#19, i_category#16] -Condition : isnotnull(avg(i_current_price)#19) - -(28) CometBroadcastExchange -Input [2]: [avg(i_current_price)#19, i_category#16] -Arguments: [avg(i_current_price)#19, i_category#16] - -(29) CometBroadcastHashJoin -Left output [3]: [i_item_sk#11, i_current_price#12, i_category#13] -Right output [2]: [avg(i_current_price)#19, i_category#16] -Arguments: [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#13, 50))], [i_category#16], Inner, (cast(i_current_price#12 as decimal(14,7)) > (1.2 * avg(i_current_price)#19)), BuildRight - -(30) CometProject -Input [5]: [i_item_sk#11, i_current_price#12, i_category#13, avg(i_current_price)#19, i_category#16] -Arguments: [i_item_sk#11], [i_item_sk#11] - -(31) CometColumnarToRow [codegen id : 3] -Input [1]: [i_item_sk#11] - -(32) BroadcastExchange -Input [1]: [i_item_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(33) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#6] -Right keys [1]: [i_item_sk#11] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 4] -Output [1]: [ca_state#3] -Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#11] - -(35) HashAggregate [codegen id : 4] -Input [1]: [ca_state#3] -Keys [1]: [ca_state#3] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#20] -Results [2]: [ca_state#3, count#21] - -(36) CometColumnarExchange -Input [2]: [ca_state#3, count#21] -Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(37) CometColumnarToRow [codegen id : 5] -Input [2]: [ca_state#3, count#21] - -(38) HashAggregate [codegen id : 5] -Input [2]: [ca_state#3, count#21] -Keys [1]: [ca_state#3] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#22] -Results [2]: [ca_state#3 AS state#23, count(1)#22 AS cnt#24] - -(39) Filter [codegen id : 5] -Input [2]: [state#23, cnt#24] -Condition : (cnt#24 >= 10) - -(40) TakeOrderedAndProject -Input [2]: [state#23, cnt#24] -Arguments: 100, [cnt#24 ASC NULLS FIRST], [state#23, cnt#24] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometNativeScan parquet spark_catalog.default.date_dim (41) - - -(41) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), EqualTo(d_month_seq,ScalarSubquery#26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#25] -Condition : ((isnotnull(d_month_seq#25) AND (d_month_seq#25 = ReusedSubquery Subquery scalar-subquery#26, [id=#27])) AND isnotnull(d_date_sk#10)) - -(43) CometProject -Input [2]: [d_date_sk#10, d_month_seq#25] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(45) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 42 Hosting Expression = ReusedSubquery Subquery scalar-subquery#26, [id=#27] - -Subquery:3 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#26, [id=#27] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometNativeScan parquet spark_catalog.default.date_dim (46) - - -(46) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_month_seq#28, d_year#29, d_moy#30] -Condition : (((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2000)) AND (d_moy#30 = 1)) - -(48) CometProject -Input [3]: [d_month_seq#28, d_year#29, d_moy#30] -Arguments: [d_month_seq#28], [d_month_seq#28] - -(49) CometHashAggregate -Input [1]: [d_month_seq#28] -Keys [1]: [d_month_seq#28] -Functions: [] - -(50) CometExchange -Input [1]: [d_month_seq#28] -Arguments: hashpartitioning(d_month_seq#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(51) CometHashAggregate -Input [1]: [d_month_seq#28] -Keys [1]: [d_month_seq#28] -Functions: [] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#28] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/simplified.txt deleted file mode 100644 index 6b96848c69..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/simplified.txt +++ /dev/null @@ -1,70 +0,0 @@ -TakeOrderedAndProject [cnt,state] - WholeStageCodegen (5) - Filter [cnt] - HashAggregate [ca_state,count] [count(1),state,cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state] #1 - WholeStageCodegen (4) - HashAggregate [ca_state] [count,count] - Project [ca_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ca_state,ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ca_state,ss_item_sk,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_customer_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #7 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #8 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_category,i_current_price] - CometFilter [i_current_price,i_category] - CometNativeScan 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/q6.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/explain.txt deleted file mode 100644 index 7871dbc31e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,310 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometFilter (38) - +- CometHashAggregate (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (4) - : : +- CometBroadcastExchange (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (9) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- CometBroadcastExchange (29) - +- CometFilter (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (22) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#1, ca_state#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ca_address_sk#1, ca_state#2] -Condition : isnotnull(ca_address_sk#1) - -(3) CometProject -Input [2]: [ca_address_sk#1, ca_state#2] -Arguments: [ca_address_sk#1, ca_state#3], [ca_address_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#2, 2)) AS ca_state#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_customer_sk#4)) - -(6) CometBroadcastExchange -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [c_customer_sk#4, c_current_addr_sk#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#1, ca_state#3] -Right output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) - -(11) CometBroadcastExchange -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(12) CometBroadcastHashJoin -Left output [2]: [ca_state#3, c_customer_sk#4] -Right output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [c_customer_sk#4], [ss_customer_sk#7], Inner, BuildRight - -(13) CometProject -Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8], [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), EqualTo(d_month_seq,ScalarSubquery#12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(16) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(18) CometBroadcastHashJoin -Left output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(19) CometProject -Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ca_state#3, ss_item_sk#6], [ca_state#3, ss_item_sk#6] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_category), IsNotNull(i_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Condition : ((isnotnull(i_current_price#15) AND isnotnull(i_category#16)) AND isnotnull(i_item_sk#14)) - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_current_price#17, i_category#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(23) CometFilter -Input [2]: [i_current_price#17, i_category#18] -Condition : isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#18, 50))) - -(24) CometProject -Input [2]: [i_current_price#17, i_category#18] -Arguments: [i_category#19, i_current_price#17], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#18, 50)) AS i_category#19, i_current_price#17] - -(25) CometHashAggregate -Input [2]: [i_category#19, i_current_price#17] -Keys [1]: [i_category#19] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#17))] - -(26) CometExchange -Input [3]: [i_category#19, sum#20, count#21] -Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [3]: [i_category#19, sum#20, count#21] -Keys [1]: [i_category#19] -Functions [1]: [avg(UnscaledValue(i_current_price#17))] - -(28) CometFilter -Input [2]: [avg(i_current_price)#22, i_category#19] -Condition : isnotnull(avg(i_current_price)#22) - -(29) CometBroadcastExchange -Input [2]: [avg(i_current_price)#22, i_category#19] -Arguments: [avg(i_current_price)#22, i_category#19] - -(30) CometBroadcastHashJoin -Left output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Right output [2]: [avg(i_current_price)#22, i_category#19] -Arguments: [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#16, 50))], [i_category#19], Inner, (cast(i_current_price#15 as decimal(14,7)) > (1.2 * avg(i_current_price)#22)), BuildRight - -(31) CometProject -Input [5]: [i_item_sk#14, i_current_price#15, i_category#16, avg(i_current_price)#22, i_category#19] -Arguments: [i_item_sk#14], [i_item_sk#14] - -(32) CometBroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: [i_item_sk#14] - -(33) CometBroadcastHashJoin -Left output [2]: [ca_state#3, ss_item_sk#6] -Right output [1]: [i_item_sk#14] -Arguments: [ss_item_sk#6], [i_item_sk#14], Inner, BuildRight - -(34) CometProject -Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#14] -Arguments: [ca_state#3], [ca_state#3] - -(35) CometHashAggregate -Input [1]: [ca_state#3] -Keys [1]: [ca_state#3] -Functions [1]: [partial_count(1)] - -(36) CometExchange -Input [2]: [ca_state#3, count#23] -Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(37) CometHashAggregate -Input [2]: [ca_state#3, count#23] -Keys [1]: [ca_state#3] -Functions [1]: [count(1)] - -(38) CometFilter -Input [2]: [state#24, cnt#25] -Condition : (cnt#25 >= 10) - -(39) CometTakeOrderedAndProject -Input [2]: [state#24, cnt#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#25 ASC NULLS FIRST], output=[state#24,cnt#25]), [state#24, cnt#25], 100, 0, [cnt#25 ASC NULLS FIRST], [state#24, cnt#25] - -(40) CometColumnarToRow [codegen id : 1] -Input [2]: [state#24, cnt#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), EqualTo(d_month_seq,ScalarSubquery#12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(43) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(45) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 42 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] - -Subquery:3 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) - - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#26, d_year#27, d_moy#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Condition : (((isnotnull(d_year#27) AND isnotnull(d_moy#28)) AND (d_year#27 = 2000)) AND (d_moy#28 = 1)) - -(48) CometProject -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Arguments: [d_month_seq#26], [d_month_seq#26] - -(49) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] -Functions: [] - -(50) CometExchange -Input [1]: [d_month_seq#26] -Arguments: hashpartitioning(d_month_seq#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(51) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] -Functions: [] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#26] - -Subquery:4 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] - -Subquery:5 Hosting operator id = 14 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/simplified.txt deleted file mode 100644 index 010720bc0e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,63 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [state,cnt] - CometFilter [state,cnt] - CometHashAggregate [count] [state,cnt,ca_state,count(1)] - CometExchange [ca_state] #1 - CometHashAggregate [ca_state,count] - CometProject [ca_state] - CometBroadcastHashJoin [ca_state,ss_item_sk,i_item_sk] - CometProject [ca_state,ss_item_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 [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,ca_state,c_customer_sk,c_current_addr_sk] - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometBroadcastExchange [i_item_sk] #7 - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #8 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #9 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_category,i_current_price] - CometFilter [i_current_price,i_category] - CometScan [native_iceberg_compat] 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/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/explain.txt deleted file mode 100644 index 7871dbc31e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/explain.txt +++ /dev/null @@ -1,310 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometFilter (38) - +- CometHashAggregate (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (4) - : : +- CometBroadcastExchange (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (9) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- CometBroadcastExchange (29) - +- CometFilter (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (22) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#1, ca_state#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ca_address_sk#1, ca_state#2] -Condition : isnotnull(ca_address_sk#1) - -(3) CometProject -Input [2]: [ca_address_sk#1, ca_state#2] -Arguments: [ca_address_sk#1, ca_state#3], [ca_address_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#2, 2)) AS ca_state#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_customer_sk#4)) - -(6) CometBroadcastExchange -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [c_customer_sk#4, c_current_addr_sk#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#1, ca_state#3] -Right output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) - -(11) CometBroadcastExchange -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(12) CometBroadcastHashJoin -Left output [2]: [ca_state#3, c_customer_sk#4] -Right output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [c_customer_sk#4], [ss_customer_sk#7], Inner, BuildRight - -(13) CometProject -Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8], [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), EqualTo(d_month_seq,ScalarSubquery#12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(16) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(18) CometBroadcastHashJoin -Left output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(19) CometProject -Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ca_state#3, ss_item_sk#6], [ca_state#3, ss_item_sk#6] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_category), IsNotNull(i_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Condition : ((isnotnull(i_current_price#15) AND isnotnull(i_category#16)) AND isnotnull(i_item_sk#14)) - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_current_price#17, i_category#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(23) CometFilter -Input [2]: [i_current_price#17, i_category#18] -Condition : isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#18, 50))) - -(24) CometProject -Input [2]: [i_current_price#17, i_category#18] -Arguments: [i_category#19, i_current_price#17], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#18, 50)) AS i_category#19, i_current_price#17] - -(25) CometHashAggregate -Input [2]: [i_category#19, i_current_price#17] -Keys [1]: [i_category#19] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#17))] - -(26) CometExchange -Input [3]: [i_category#19, sum#20, count#21] -Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [3]: [i_category#19, sum#20, count#21] -Keys [1]: [i_category#19] -Functions [1]: [avg(UnscaledValue(i_current_price#17))] - -(28) CometFilter -Input [2]: [avg(i_current_price)#22, i_category#19] -Condition : isnotnull(avg(i_current_price)#22) - -(29) CometBroadcastExchange -Input [2]: [avg(i_current_price)#22, i_category#19] -Arguments: [avg(i_current_price)#22, i_category#19] - -(30) CometBroadcastHashJoin -Left output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Right output [2]: [avg(i_current_price)#22, i_category#19] -Arguments: [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#16, 50))], [i_category#19], Inner, (cast(i_current_price#15 as decimal(14,7)) > (1.2 * avg(i_current_price)#22)), BuildRight - -(31) CometProject -Input [5]: [i_item_sk#14, i_current_price#15, i_category#16, avg(i_current_price)#22, i_category#19] -Arguments: [i_item_sk#14], [i_item_sk#14] - -(32) CometBroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: [i_item_sk#14] - -(33) CometBroadcastHashJoin -Left output [2]: [ca_state#3, ss_item_sk#6] -Right output [1]: [i_item_sk#14] -Arguments: [ss_item_sk#6], [i_item_sk#14], Inner, BuildRight - -(34) CometProject -Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#14] -Arguments: [ca_state#3], [ca_state#3] - -(35) CometHashAggregate -Input [1]: [ca_state#3] -Keys [1]: [ca_state#3] -Functions [1]: [partial_count(1)] - -(36) CometExchange -Input [2]: [ca_state#3, count#23] -Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(37) CometHashAggregate -Input [2]: [ca_state#3, count#23] -Keys [1]: [ca_state#3] -Functions [1]: [count(1)] - -(38) CometFilter -Input [2]: [state#24, cnt#25] -Condition : (cnt#25 >= 10) - -(39) CometTakeOrderedAndProject -Input [2]: [state#24, cnt#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#25 ASC NULLS FIRST], output=[state#24,cnt#25]), [state#24, cnt#25], 100, 0, [cnt#25 ASC NULLS FIRST], [state#24, cnt#25] - -(40) CometColumnarToRow [codegen id : 1] -Input [2]: [state#24, cnt#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), EqualTo(d_month_seq,ScalarSubquery#12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(43) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(45) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 42 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] - -Subquery:3 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) - - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#26, d_year#27, d_moy#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Condition : (((isnotnull(d_year#27) AND isnotnull(d_moy#28)) AND (d_year#27 = 2000)) AND (d_moy#28 = 1)) - -(48) CometProject -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Arguments: [d_month_seq#26], [d_month_seq#26] - -(49) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] -Functions: [] - -(50) CometExchange -Input [1]: [d_month_seq#26] -Arguments: hashpartitioning(d_month_seq#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(51) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] -Functions: [] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#26] - -Subquery:4 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] - -Subquery:5 Hosting operator id = 14 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/extended.txt deleted file mode 100644 index 2d049f149f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/extended.txt +++ /dev/null @@ -1,59 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 48 out of 54 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 010720bc0e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt +++ /dev/null @@ -1,63 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [state,cnt] - CometFilter [state,cnt] - CometHashAggregate [count] [state,cnt,ca_state,count(1)] - CometExchange [ca_state] #1 - CometHashAggregate [ca_state,count] - CometProject [ca_state] - CometBroadcastHashJoin [ca_state,ss_item_sk,i_item_sk] - CometProject [ca_state,ss_item_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 [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,ca_state,c_customer_sk,c_current_addr_sk] - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometBroadcastExchange [i_item_sk] #7 - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #8 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #9 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_category,i_current_price] - CometFilter [i_current_price,i_category] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/explain.txt deleted file mode 100644 index 38d1c84a80..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/explain.txt +++ /dev/null @@ -1,421 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (67) -+- * HashAggregate (66) - +- * CometColumnarToRow (65) - +- CometColumnarExchange (64) - +- * HashAggregate (63) - +- Union (62) - :- * HashAggregate (29) - : +- * CometColumnarToRow (28) - : +- CometColumnarExchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * CometColumnarToRow (22) - : +- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.item (14) - : +- CometBroadcastExchange (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometNativeScan parquet spark_catalog.default.item (16) - :- * HashAggregate (45) - : +- * CometColumnarToRow (44) - : +- CometColumnarExchange (43) - : +- * HashAggregate (42) - : +- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (38) - : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * Filter (32) - : : : : +- * ColumnarToRow (31) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (30) - : : : +- ReusedExchange (33) - : : +- ReusedExchange (36) - : +- ReusedExchange (39) - +- * HashAggregate (61) - +- * CometColumnarToRow (60) - +- CometColumnarExchange (59) - +- * HashAggregate (58) - +- * Project (57) - +- * BroadcastHashJoin Inner BuildRight (56) - :- * Project (54) - : +- * BroadcastHashJoin Inner BuildRight (53) - : :- * Project (51) - : : +- * BroadcastHashJoin Inner BuildRight (50) - : : :- * Filter (48) - : : : +- * ColumnarToRow (47) - : : : +- Scan parquet spark_catalog.default.web_sales (46) - : : +- ReusedExchange (49) - : +- ReusedExchange (52) - +- ReusedExchange (55) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(4) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_gmt_offset#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isnotnull(ca_address_sk#7)) - -(9) CometProject -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Arguments: [ca_address_sk#7], [ca_address_sk#7] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [ca_address_sk#7] - -(11) BroadcastExchange -Input [1]: [ca_address_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] - -(14) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#9, i_item_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [i_item_sk#9, i_item_id#10] -Condition : isnotnull(i_item_sk#9) - -(16) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_id#11, i_category#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [i_item_id#11, i_category#12] -Condition : (isnotnull(i_category#12) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#12, 50)) = Music )) - -(18) CometProject -Input [2]: [i_item_id#11, i_category#12] -Arguments: [i_item_id#13], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#11, 16)) AS i_item_id#13] - -(19) CometBroadcastExchange -Input [1]: [i_item_id#13] -Arguments: [i_item_id#13] - -(20) CometBroadcastHashJoin -Left output [2]: [i_item_sk#9, i_item_id#10] -Right output [1]: [i_item_id#13] -Arguments: [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#10, 16))], [i_item_id#13], LeftSemi, BuildRight - -(21) CometProject -Input [2]: [i_item_sk#9, i_item_id#10] -Arguments: [i_item_sk#9, i_item_id#14], [i_item_sk#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#10, 16)) AS i_item_id#14] - -(22) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#9, i_item_id#14] - -(23) BroadcastExchange -Input [2]: [i_item_sk#9, i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#9] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 4] -Output [2]: [ss_ext_sales_price#3, i_item_id#14] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_item_id#14] - -(26) HashAggregate [codegen id : 4] -Input [2]: [ss_ext_sales_price#3, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum#15] -Results [2]: [i_item_id#14, sum#16] - -(27) CometColumnarExchange -Input [2]: [i_item_id#14, sum#16] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometColumnarToRow [codegen id : 5] -Input [2]: [i_item_id#14, sum#16] - -(29) HashAggregate [codegen id : 5] -Input [2]: [i_item_id#14, sum#16] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#17] -Results [2]: [i_item_id#14 AS i_item_id#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#17,17,2) AS total_sales#19] - -(30) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 9] -Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] - -(32) Filter [codegen id : 9] -Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -Condition : (isnotnull(cs_bill_addr_sk#20) AND isnotnull(cs_item_sk#21)) - -(33) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#24] - -(34) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 9] -Output [3]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22] -Input [5]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#24] - -(36) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#25] - -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_bill_addr_sk#20] -Right keys [1]: [ca_address_sk#25] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 9] -Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] -Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, ca_address_sk#25] - -(39) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#26, i_item_id#27] - -(40) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#21] -Right keys [1]: [i_item_sk#26] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 9] -Output [2]: [cs_ext_sales_price#22, i_item_id#27] -Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#26, i_item_id#27] - -(42) HashAggregate [codegen id : 9] -Input [2]: [cs_ext_sales_price#22, i_item_id#27] -Keys [1]: [i_item_id#27] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#27, sum#29] - -(43) CometColumnarExchange -Input [2]: [i_item_id#27, sum#29] -Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(44) CometColumnarToRow [codegen id : 10] -Input [2]: [i_item_id#27, sum#29] - -(45) HashAggregate [codegen id : 10] -Input [2]: [i_item_id#27, sum#29] -Keys [1]: [i_item_id#27] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] -Results [2]: [i_item_id#27, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS total_sales#31] - -(46) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(47) ColumnarToRow [codegen id : 14] -Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] - -(48) Filter [codegen id : 14] -Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] -Condition : (isnotnull(ws_bill_addr_sk#33) AND isnotnull(ws_item_sk#32)) - -(49) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#36] - -(50) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#36] -Join type: Inner -Join condition: None - -(51) Project [codegen id : 14] -Output [3]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34] -Input [5]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35, d_date_sk#36] - -(52) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#37] - -(53) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#33] -Right keys [1]: [ca_address_sk#37] -Join type: Inner -Join condition: None - -(54) Project [codegen id : 14] -Output [2]: [ws_item_sk#32, ws_ext_sales_price#34] -Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ca_address_sk#37] - -(55) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#38, i_item_id#39] - -(56) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_item_sk#32] -Right keys [1]: [i_item_sk#38] -Join type: Inner -Join condition: None - -(57) Project [codegen id : 14] -Output [2]: [ws_ext_sales_price#34, i_item_id#39] -Input [4]: [ws_item_sk#32, ws_ext_sales_price#34, i_item_sk#38, i_item_id#39] - -(58) HashAggregate [codegen id : 14] -Input [2]: [ws_ext_sales_price#34, i_item_id#39] -Keys [1]: [i_item_id#39] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#34))] -Aggregate Attributes [1]: [sum#40] -Results [2]: [i_item_id#39, sum#41] - -(59) CometColumnarExchange -Input [2]: [i_item_id#39, sum#41] -Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(60) CometColumnarToRow [codegen id : 15] -Input [2]: [i_item_id#39, sum#41] - -(61) HashAggregate [codegen id : 15] -Input [2]: [i_item_id#39, sum#41] -Keys [1]: [i_item_id#39] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#34))#42] -Results [2]: [i_item_id#39, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#34))#42,17,2) AS total_sales#43] - -(62) Union - -(63) HashAggregate [codegen id : 16] -Input [2]: [i_item_id#18, total_sales#19] -Keys [1]: [i_item_id#18] -Functions [1]: [partial_sum(total_sales#19)] -Aggregate Attributes [2]: [sum#44, isEmpty#45] -Results [3]: [i_item_id#18, sum#46, isEmpty#47] - -(64) CometColumnarExchange -Input [3]: [i_item_id#18, sum#46, isEmpty#47] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(65) CometColumnarToRow [codegen id : 17] -Input [3]: [i_item_id#18, sum#46, isEmpty#47] - -(66) HashAggregate [codegen id : 17] -Input [3]: [i_item_id#18, sum#46, isEmpty#47] -Keys [1]: [i_item_id#18] -Functions [1]: [sum(total_sales#19)] -Aggregate Attributes [1]: [sum(total_sales#19)#48] -Results [2]: [i_item_id#18, sum(total_sales#19)#48 AS total_sales#49] - -(67) TakeOrderedAndProject -Input [2]: [i_item_id#18, total_sales#49] -Arguments: 100, [i_item_id#18 ASC NULLS FIRST, total_sales#49 ASC NULLS FIRST], [i_item_id#18, total_sales#49] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (72) -+- * CometColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometNativeScan parquet spark_catalog.default.date_dim (68) - - -(68) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#50, d_moy#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [3]: [d_date_sk#6, d_year#50, d_moy#51] -Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 1998)) AND (d_moy#51 = 9)) AND isnotnull(d_date_sk#6)) - -(70) CometProject -Input [3]: [d_date_sk#6, d_year#50, d_moy#51] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(71) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(72) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/simplified.txt deleted file mode 100644 index 4c05038b34..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/simplified.txt +++ /dev/null @@ -1,106 +0,0 @@ -TakeOrderedAndProject [i_item_id,total_sales] - WholeStageCodegen (17) - HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (16) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),i_item_id,total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #6 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_id,i_category] - WholeStageCodegen (10) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #7 - WholeStageCodegen (9) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_addr_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (15) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #8 - WholeStageCodegen (14) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/explain.txt deleted file mode 100644 index 1777ebee23..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,386 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (62) -+- CometTakeOrderedAndProject (61) - +- CometHashAggregate (60) - +- CometExchange (59) - +- CometHashAggregate (58) - +- CometUnion (57) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- CometHashAggregate (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometFilter (30) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- CometHashAggregate (56) - +- CometExchange (55) - +- CometHashAggregate (54) - +- CometProject (53) - +- CometBroadcastHashJoin (52) - :- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometFilter (44) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (43) - : : +- ReusedExchange (45) - : +- ReusedExchange (48) - +- ReusedExchange (51) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 9)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(11) CometProject -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Arguments: [ca_address_sk#9], [ca_address_sk#9] - -(12) CometBroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: [ca_address_sk#9] - -(13) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Right output [1]: [ca_address_sk#9] -Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] -Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#11, i_item_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#11, i_item_id#12] -Condition : isnotnull(i_item_sk#11) - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_id#13, i_category#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [i_item_id#13, i_category#14] -Condition : (isnotnull(i_category#14) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#14, 50)) = Music )) - -(19) CometProject -Input [2]: [i_item_id#13, i_category#14] -Arguments: [i_item_id#15], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#13, 16)) AS i_item_id#15] - -(20) CometBroadcastExchange -Input [1]: [i_item_id#15] -Arguments: [i_item_id#15] - -(21) CometBroadcastHashJoin -Left output [2]: [i_item_sk#11, i_item_id#12] -Right output [1]: [i_item_id#15] -Arguments: [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#12, 16))], [i_item_id#15], LeftSemi, BuildRight - -(22) CometProject -Input [2]: [i_item_sk#11, i_item_id#12] -Arguments: [i_item_sk#11, i_item_id#16], [i_item_sk#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#12, 16)) AS i_item_id#16] - -(23) CometBroadcastExchange -Input [2]: [i_item_sk#11, i_item_id#16] -Arguments: [i_item_sk#11, i_item_id#16] - -(24) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Right output [2]: [i_item_sk#11, i_item_id#16] -Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight - -(25) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_item_id#16] -Arguments: [ss_ext_sales_price#3, i_item_id#16], [ss_ext_sales_price#3, i_item_id#16] - -(26) CometHashAggregate -Input [2]: [ss_ext_sales_price#3, i_item_id#16] -Keys [1]: [i_item_id#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(27) CometExchange -Input [2]: [i_item_id#16, sum#17] -Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [2]: [i_item_id#16, sum#17] -Keys [1]: [i_item_id#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Condition : (isnotnull(cs_bill_addr_sk#18) AND isnotnull(cs_item_sk#19)) - -(31) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#23] - -(32) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Right output [1]: [d_date_sk#23] -Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(33) CometProject -Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] -Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] - -(34) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#24] - -(35) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -Right output [1]: [ca_address_sk#24] -Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner, BuildRight - -(36) CometProject -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] -Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] - -(37) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#25, i_item_id#26] - -(38) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] -Right output [2]: [i_item_sk#25, i_item_id#26] -Arguments: [cs_item_sk#19], [i_item_sk#25], Inner, BuildRight - -(39) CometProject -Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_item_id#26] -Arguments: [cs_ext_sales_price#20, i_item_id#26], [cs_ext_sales_price#20, i_item_id#26] - -(40) CometHashAggregate -Input [2]: [cs_ext_sales_price#20, i_item_id#26] -Keys [1]: [i_item_id#26] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] - -(41) CometExchange -Input [2]: [i_item_id#26, sum#27] -Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(42) CometHashAggregate -Input [2]: [i_item_id#26, sum#27] -Keys [1]: [i_item_id#26] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(44) CometFilter -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Condition : (isnotnull(ws_bill_addr_sk#29) AND isnotnull(ws_item_sk#28)) - -(45) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(46) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [ws_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(47) CometProject -Input [5]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31, d_date_sk#33] -Arguments: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30], [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] - -(48) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#34] - -(49) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] -Right output [1]: [ca_address_sk#34] -Arguments: [ws_bill_addr_sk#29], [ca_address_sk#34], Inner, BuildRight - -(50) CometProject -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ca_address_sk#34] -Arguments: [ws_item_sk#28, ws_ext_sales_price#30], [ws_item_sk#28, ws_ext_sales_price#30] - -(51) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#35, i_item_id#36] - -(52) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#28, ws_ext_sales_price#30] -Right output [2]: [i_item_sk#35, i_item_id#36] -Arguments: [ws_item_sk#28], [i_item_sk#35], Inner, BuildRight - -(53) CometProject -Input [4]: [ws_item_sk#28, ws_ext_sales_price#30, i_item_sk#35, i_item_id#36] -Arguments: [ws_ext_sales_price#30, i_item_id#36], [ws_ext_sales_price#30, i_item_id#36] - -(54) CometHashAggregate -Input [2]: [ws_ext_sales_price#30, i_item_id#36] -Keys [1]: [i_item_id#36] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#30))] - -(55) CometExchange -Input [2]: [i_item_id#36, sum#37] -Arguments: hashpartitioning(i_item_id#36, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(56) CometHashAggregate -Input [2]: [i_item_id#36, sum#37] -Keys [1]: [i_item_id#36] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#30))] - -(57) CometUnion -Child 0 Input [2]: [i_item_id#38, total_sales#39] -Child 1 Input [2]: [i_item_id#26, total_sales#40] -Child 2 Input [2]: [i_item_id#36, total_sales#41] - -(58) CometHashAggregate -Input [2]: [i_item_id#38, total_sales#39] -Keys [1]: [i_item_id#38] -Functions [1]: [partial_sum(total_sales#39)] - -(59) CometExchange -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(60) CometHashAggregate -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Keys [1]: [i_item_id#38] -Functions [1]: [sum(total_sales#39)] - -(61) CometTakeOrderedAndProject -Input [2]: [i_item_id#38, total_sales#44] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#38 ASC NULLS FIRST,total_sales#44 ASC NULLS FIRST], output=[i_item_id#38,total_sales#44]), [i_item_id#38, total_sales#44], 100, 0, [i_item_id#38 ASC NULLS FIRST, total_sales#44 ASC NULLS FIRST], [i_item_id#38, total_sales#44] - -(62) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_id#38, total_sales#44] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (67) -+- * CometColumnarToRow (66) - +- CometProject (65) - +- CometFilter (64) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) - - -(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] -ReadSchema: struct - -(64) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 9)) AND isnotnull(d_date_sk#6)) - -(65) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(66) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(67) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/simplified.txt deleted file mode 100644 index 086da3b776..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,total_sales] - CometHashAggregate [sum,isEmpty] [i_item_id,total_sales,sum(total_sales)] - CometExchange [i_item_id] #1 - CometHashAggregate [total_sales] [i_item_id,sum,isEmpty] - CometUnion [i_item_id,total_sales] - CometHashAggregate [sum] [i_item_id,total_sales,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #7 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_category] - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - 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,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt deleted file mode 100644 index 1777ebee23..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt +++ /dev/null @@ -1,386 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (62) -+- CometTakeOrderedAndProject (61) - +- CometHashAggregate (60) - +- CometExchange (59) - +- CometHashAggregate (58) - +- CometUnion (57) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- CometHashAggregate (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometFilter (30) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- CometHashAggregate (56) - +- CometExchange (55) - +- CometHashAggregate (54) - +- CometProject (53) - +- CometBroadcastHashJoin (52) - :- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometFilter (44) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (43) - : : +- ReusedExchange (45) - : +- ReusedExchange (48) - +- ReusedExchange (51) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 9)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(11) CometProject -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Arguments: [ca_address_sk#9], [ca_address_sk#9] - -(12) CometBroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: [ca_address_sk#9] - -(13) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Right output [1]: [ca_address_sk#9] -Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] -Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#11, i_item_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#11, i_item_id#12] -Condition : isnotnull(i_item_sk#11) - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_id#13, i_category#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [i_item_id#13, i_category#14] -Condition : (isnotnull(i_category#14) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#14, 50)) = Music )) - -(19) CometProject -Input [2]: [i_item_id#13, i_category#14] -Arguments: [i_item_id#15], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#13, 16)) AS i_item_id#15] - -(20) CometBroadcastExchange -Input [1]: [i_item_id#15] -Arguments: [i_item_id#15] - -(21) CometBroadcastHashJoin -Left output [2]: [i_item_sk#11, i_item_id#12] -Right output [1]: [i_item_id#15] -Arguments: [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#12, 16))], [i_item_id#15], LeftSemi, BuildRight - -(22) CometProject -Input [2]: [i_item_sk#11, i_item_id#12] -Arguments: [i_item_sk#11, i_item_id#16], [i_item_sk#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#12, 16)) AS i_item_id#16] - -(23) CometBroadcastExchange -Input [2]: [i_item_sk#11, i_item_id#16] -Arguments: [i_item_sk#11, i_item_id#16] - -(24) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Right output [2]: [i_item_sk#11, i_item_id#16] -Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight - -(25) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_item_id#16] -Arguments: [ss_ext_sales_price#3, i_item_id#16], [ss_ext_sales_price#3, i_item_id#16] - -(26) CometHashAggregate -Input [2]: [ss_ext_sales_price#3, i_item_id#16] -Keys [1]: [i_item_id#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(27) CometExchange -Input [2]: [i_item_id#16, sum#17] -Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [2]: [i_item_id#16, sum#17] -Keys [1]: [i_item_id#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Condition : (isnotnull(cs_bill_addr_sk#18) AND isnotnull(cs_item_sk#19)) - -(31) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#23] - -(32) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Right output [1]: [d_date_sk#23] -Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(33) CometProject -Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] -Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] - -(34) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#24] - -(35) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -Right output [1]: [ca_address_sk#24] -Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner, BuildRight - -(36) CometProject -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] -Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] - -(37) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#25, i_item_id#26] - -(38) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] -Right output [2]: [i_item_sk#25, i_item_id#26] -Arguments: [cs_item_sk#19], [i_item_sk#25], Inner, BuildRight - -(39) CometProject -Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_item_id#26] -Arguments: [cs_ext_sales_price#20, i_item_id#26], [cs_ext_sales_price#20, i_item_id#26] - -(40) CometHashAggregate -Input [2]: [cs_ext_sales_price#20, i_item_id#26] -Keys [1]: [i_item_id#26] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] - -(41) CometExchange -Input [2]: [i_item_id#26, sum#27] -Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(42) CometHashAggregate -Input [2]: [i_item_id#26, sum#27] -Keys [1]: [i_item_id#26] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(44) CometFilter -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Condition : (isnotnull(ws_bill_addr_sk#29) AND isnotnull(ws_item_sk#28)) - -(45) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(46) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [ws_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(47) CometProject -Input [5]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31, d_date_sk#33] -Arguments: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30], [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] - -(48) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#34] - -(49) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] -Right output [1]: [ca_address_sk#34] -Arguments: [ws_bill_addr_sk#29], [ca_address_sk#34], Inner, BuildRight - -(50) CometProject -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ca_address_sk#34] -Arguments: [ws_item_sk#28, ws_ext_sales_price#30], [ws_item_sk#28, ws_ext_sales_price#30] - -(51) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#35, i_item_id#36] - -(52) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#28, ws_ext_sales_price#30] -Right output [2]: [i_item_sk#35, i_item_id#36] -Arguments: [ws_item_sk#28], [i_item_sk#35], Inner, BuildRight - -(53) CometProject -Input [4]: [ws_item_sk#28, ws_ext_sales_price#30, i_item_sk#35, i_item_id#36] -Arguments: [ws_ext_sales_price#30, i_item_id#36], [ws_ext_sales_price#30, i_item_id#36] - -(54) CometHashAggregate -Input [2]: [ws_ext_sales_price#30, i_item_id#36] -Keys [1]: [i_item_id#36] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#30))] - -(55) CometExchange -Input [2]: [i_item_id#36, sum#37] -Arguments: hashpartitioning(i_item_id#36, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(56) CometHashAggregate -Input [2]: [i_item_id#36, sum#37] -Keys [1]: [i_item_id#36] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#30))] - -(57) CometUnion -Child 0 Input [2]: [i_item_id#38, total_sales#39] -Child 1 Input [2]: [i_item_id#26, total_sales#40] -Child 2 Input [2]: [i_item_id#36, total_sales#41] - -(58) CometHashAggregate -Input [2]: [i_item_id#38, total_sales#39] -Keys [1]: [i_item_id#38] -Functions [1]: [partial_sum(total_sales#39)] - -(59) CometExchange -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(60) CometHashAggregate -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Keys [1]: [i_item_id#38] -Functions [1]: [sum(total_sales#39)] - -(61) CometTakeOrderedAndProject -Input [2]: [i_item_id#38, total_sales#44] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#38 ASC NULLS FIRST,total_sales#44 ASC NULLS FIRST], output=[i_item_id#38,total_sales#44]), [i_item_id#38, total_sales#44], 100, 0, [i_item_id#38 ASC NULLS FIRST, total_sales#44 ASC NULLS FIRST], [i_item_id#38, total_sales#44] - -(62) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_id#38, total_sales#44] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (67) -+- * CometColumnarToRow (66) - +- CometProject (65) - +- CometFilter (64) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) - - -(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] -ReadSchema: struct - -(64) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 9)) AND isnotnull(d_date_sk#6)) - -(65) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(66) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(67) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/extended.txt deleted file mode 100644 index 36b9a8306d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/extended.txt +++ /dev/null @@ -1,100 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 086da3b776..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,total_sales] - CometHashAggregate [sum,isEmpty] [i_item_id,total_sales,sum(total_sales)] - CometExchange [i_item_id] #1 - CometHashAggregate [total_sales] [i_item_id,sum,isEmpty] - CometUnion [i_item_id,total_sales] - CometHashAggregate [sum] [i_item_id,total_sales,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #7 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_category] - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - 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,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/explain.txt deleted file mode 100644 index 7eae15cdc2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/explain.txt +++ /dev/null @@ -1,425 +0,0 @@ -== Physical Plan == -* Project (69) -+- * BroadcastNestedLoopJoin Inner BuildRight (68) - :- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- CometColumnarExchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (33) - : : +- * BroadcastHashJoin Inner BuildRight (32) - : : :- * Project (26) - : : : +- * BroadcastHashJoin Inner BuildRight (25) - : : : :- * Project (20) - : : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : : :- * Project (17) - : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * CometColumnarToRow (7) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometNativeScan parquet spark_catalog.default.store (4) - : : : : : +- BroadcastExchange (15) - : : : : : +- * CometColumnarToRow (14) - : : : : : +- CometProject (13) - : : : : : +- CometFilter (12) - : : : : : +- CometNativeScan parquet spark_catalog.default.promotion (11) - : : : : +- ReusedExchange (18) - : : : +- BroadcastExchange (24) - : : : +- * CometColumnarToRow (23) - : : : +- CometFilter (22) - : : : +- CometNativeScan parquet spark_catalog.default.customer (21) - : : +- BroadcastExchange (31) - : : +- * CometColumnarToRow (30) - : : +- CometProject (29) - : : +- CometFilter (28) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (27) - : +- BroadcastExchange (38) - : +- * CometColumnarToRow (37) - : +- CometProject (36) - : +- CometFilter (35) - : +- CometNativeScan parquet spark_catalog.default.item (34) - +- BroadcastExchange (67) - +- * HashAggregate (66) - +- * CometColumnarToRow (65) - +- CometColumnarExchange (64) - +- * HashAggregate (63) - +- * Project (62) - +- * BroadcastHashJoin Inner BuildRight (61) - :- * Project (59) - : +- * BroadcastHashJoin Inner BuildRight (58) - : :- * Project (56) - : : +- * BroadcastHashJoin Inner BuildRight (55) - : : :- * Project (53) - : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : :- * Project (50) - : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : :- * Filter (47) - : : : : : +- * ColumnarToRow (46) - : : : : : +- Scan parquet spark_catalog.default.store_sales (45) - : : : : +- ReusedExchange (48) - : : : +- ReusedExchange (51) - : : +- ReusedExchange (54) - : +- ReusedExchange (57) - +- ReusedExchange (60) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 7] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(3) Filter [codegen id : 7] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_store_sk#3) AND isnotnull(ss_promo_sk#4)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1)) - -(4) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_gmt_offset#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [s_store_sk#8, s_gmt_offset#9] -Condition : ((isnotnull(s_gmt_offset#9) AND (s_gmt_offset#9 = -5.00)) AND isnotnull(s_store_sk#8)) - -(6) CometProject -Input [2]: [s_store_sk#8, s_gmt_offset#9] -Arguments: [s_store_sk#8], [s_store_sk#8] - -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [s_store_sk#8] - -(8) BroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 7] -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, s_store_sk#8] - -(11) CometNativeScan parquet spark_catalog.default.promotion -Output [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Condition : ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_dmail#11, 1)) = Y) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_email#12, 1)) = Y)) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_tv#13, 1)) = Y)) AND isnotnull(p_promo_sk#10)) - -(13) CometProject -Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Arguments: [p_promo_sk#10], [p_promo_sk#10] - -(14) CometColumnarToRow [codegen id : 2] -Input [1]: [p_promo_sk#10] - -(15) BroadcastExchange -Input [1]: [p_promo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_promo_sk#4] -Right keys [1]: [p_promo_sk#10] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 7] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, p_promo_sk#10] - -(18) ReusedExchange [Reuses operator id: 74] -Output [1]: [d_date_sk#14] - -(19) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 7] -Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#14] - -(21) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#15, c_current_addr_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [c_customer_sk#15, c_current_addr_sk#16] -Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_current_addr_sk#16)) - -(23) CometColumnarToRow [codegen id : 4] -Input [2]: [c_customer_sk#15, c_current_addr_sk#16] - -(24) BroadcastExchange -Input [2]: [c_customer_sk#15, c_current_addr_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(25) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 7] -Output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#16] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#15, c_current_addr_sk#16] - -(27) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#17, ca_gmt_offset#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [ca_address_sk#17, ca_gmt_offset#18] -Condition : ((isnotnull(ca_gmt_offset#18) AND (ca_gmt_offset#18 = -5.00)) AND isnotnull(ca_address_sk#17)) - -(29) CometProject -Input [2]: [ca_address_sk#17, ca_gmt_offset#18] -Arguments: [ca_address_sk#17], [ca_address_sk#17] - -(30) CometColumnarToRow [codegen id : 5] -Input [1]: [ca_address_sk#17] - -(31) BroadcastExchange -Input [1]: [ca_address_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#16] -Right keys [1]: [ca_address_sk#17] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 7] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#5] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#16, ca_address_sk#17] - -(34) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_category#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), IsNotNull(i_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [i_item_sk#19, i_category#20] -Condition : ((isnotnull(i_category#20) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#20, 50)) = Jewelry )) AND isnotnull(i_item_sk#19)) - -(36) CometProject -Input [2]: [i_item_sk#19, i_category#20] -Arguments: [i_item_sk#19], [i_item_sk#19] - -(37) CometColumnarToRow [codegen id : 6] -Input [1]: [i_item_sk#19] - -(38) BroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#19] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 7] -Output [1]: [ss_ext_sales_price#5] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#19] - -(41) HashAggregate [codegen id : 7] -Input [1]: [ss_ext_sales_price#5] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum#21] -Results [1]: [sum#22] - -(42) CometColumnarExchange -Input [1]: [sum#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) CometColumnarToRow [codegen id : 15] -Input [1]: [sum#22] - -(44) HashAggregate [codegen id : 15] -Input [1]: [sum#22] -Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#23] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#23,17,2) AS promotions#24] - -(45) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 13] -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] - -(47) Filter [codegen id : 13] -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_store_sk#27) AND isnotnull(ss_customer_sk#26)) AND isnotnull(ss_item_sk#25)) - -(48) ReusedExchange [Reuses operator id: 8] -Output [1]: [s_store_sk#30] - -(49) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#27] -Right keys [1]: [s_store_sk#30] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 13] -Output [4]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] -Input [6]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, s_store_sk#30] - -(51) ReusedExchange [Reuses operator id: 74] -Output [1]: [d_date_sk#31] - -(52) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_date_sk#29] -Right keys [1]: [d_date_sk#31] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 13] -Output [3]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#31] - -(54) ReusedExchange [Reuses operator id: 24] -Output [2]: [c_customer_sk#32, c_current_addr_sk#33] - -(55) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#32] -Join type: Inner -Join condition: None - -(56) Project [codegen id : 13] -Output [3]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#33] -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, c_customer_sk#32, c_current_addr_sk#33] - -(57) ReusedExchange [Reuses operator id: 31] -Output [1]: [ca_address_sk#34] - -(58) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [c_current_addr_sk#33] -Right keys [1]: [ca_address_sk#34] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 13] -Output [2]: [ss_item_sk#25, ss_ext_sales_price#28] -Input [4]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#33, ca_address_sk#34] - -(60) ReusedExchange [Reuses operator id: 38] -Output [1]: [i_item_sk#35] - -(61) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_item_sk#25] -Right keys [1]: [i_item_sk#35] -Join type: Inner -Join condition: None - -(62) Project [codegen id : 13] -Output [1]: [ss_ext_sales_price#28] -Input [3]: [ss_item_sk#25, ss_ext_sales_price#28, i_item_sk#35] - -(63) HashAggregate [codegen id : 13] -Input [1]: [ss_ext_sales_price#28] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#28))] -Aggregate Attributes [1]: [sum#36] -Results [1]: [sum#37] - -(64) CometColumnarExchange -Input [1]: [sum#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(65) CometColumnarToRow [codegen id : 14] -Input [1]: [sum#37] - -(66) HashAggregate [codegen id : 14] -Input [1]: [sum#37] -Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#28))#38] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#28))#38,17,2) AS total#39] - -(67) BroadcastExchange -Input [1]: [total#39] -Arguments: IdentityBroadcastMode, [plan_id=8] - -(68) BroadcastNestedLoopJoin [codegen id : 15] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 15] -Output [3]: [promotions#24, total#39, ((cast(promotions#24 as decimal(15,4)) / cast(total#39 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] -Input [2]: [promotions#24, total#39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (74) -+- * CometColumnarToRow (73) - +- CometProject (72) - +- CometFilter (71) - +- CometNativeScan parquet spark_catalog.default.date_dim (70) - - -(70) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#14, d_year#41, d_moy#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(71) CometFilter -Input [3]: [d_date_sk#14, d_year#41, d_moy#42] -Condition : ((((isnotnull(d_year#41) AND isnotnull(d_moy#42)) AND (d_year#41 = 1998)) AND (d_moy#42 = 11)) AND isnotnull(d_date_sk#14)) - -(72) CometProject -Input [3]: [d_date_sk#14, d_year#41, d_moy#42] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(73) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(74) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 45 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/simplified.txt deleted file mode 100644 index f5bacac9c6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/simplified.txt +++ /dev/null @@ -1,108 +0,0 @@ -WholeStageCodegen (15) - Project [promotions,total] - BroadcastNestedLoopJoin - HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (7) - HashAggregate [ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_gmt_offset] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (14) - HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (13) - HashAggregate [ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk,ss_customer_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [s_store_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [c_customer_sk,c_current_addr_sk] #5 - InputAdapter - ReusedExchange [ca_address_sk] #6 - InputAdapter - ReusedExchange [i_item_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/explain.txt deleted file mode 100644 index 9465cc9c1d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,397 +0,0 @@ -== Physical Plan == -* Project (65) -+- * BroadcastNestedLoopJoin Inner BuildRight (64) - :- * CometColumnarToRow (41) - : +- CometHashAggregate (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometBroadcastHashJoin (19) - : : : : :- CometProject (14) - : : : : : +- CometBroadcastHashJoin (13) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (6) - : : : : : : +- CometProject (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (3) - : : : : : +- CometBroadcastExchange (12) - : : : : : +- CometProject (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (9) - : : : : +- CometBroadcastExchange (18) - : : : : +- CometProject (17) - : : : : +- CometFilter (16) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (15) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (21) - : : +- CometBroadcastExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (26) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) - +- BroadcastExchange (63) - +- * CometColumnarToRow (62) - +- CometHashAggregate (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometProject (55) - : +- CometBroadcastHashJoin (54) - : :- CometProject (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometProject (49) - : : : +- CometBroadcastHashJoin (48) - : : : :- CometProject (46) - : : : : +- CometBroadcastHashJoin (45) - : : : : :- CometFilter (43) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (42) - : : : : +- ReusedExchange (44) - : : : +- ReusedExchange (47) - : : +- ReusedExchange (50) - : +- ReusedExchange (53) - +- ReusedExchange (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_store_sk#3) AND isnotnull(ss_promo_sk#4)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_gmt_offset#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [s_store_sk#8, s_gmt_offset#9] -Condition : ((isnotnull(s_gmt_offset#9) AND (s_gmt_offset#9 = -5.00)) AND isnotnull(s_store_sk#8)) - -(5) CometProject -Input [2]: [s_store_sk#8, s_gmt_offset#9] -Arguments: [s_store_sk#8], [s_store_sk#8] - -(6) CometBroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: [s_store_sk#8] - -(7) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [s_store_sk#8] -Arguments: [ss_store_sk#3], [s_store_sk#8], Inner, BuildRight - -(8) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, s_store_sk#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6], [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Condition : ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_dmail#11, 1)) = Y) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_email#12, 1)) = Y)) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_tv#13, 1)) = Y)) AND isnotnull(p_promo_sk#10)) - -(11) CometProject -Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Arguments: [p_promo_sk#10], [p_promo_sk#10] - -(12) CometBroadcastExchange -Input [1]: [p_promo_sk#10] -Arguments: [p_promo_sk#10] - -(13) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [p_promo_sk#10] -Arguments: [ss_promo_sk#4], [p_promo_sk#10], Inner, BuildRight - -(14) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, p_promo_sk#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6], [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#14, d_year#15, d_moy#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1998)) AND (d_moy#16 = 11)) AND isnotnull(d_date_sk#14)) - -(17) CometProject -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(18) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(19) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#6], [d_date_sk#14], Inner, BuildRight - -(20) CometProject -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5], [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(c_current_addr_sk#18)) - -(23) CometBroadcastExchange -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [c_customer_sk#17, c_current_addr_sk#18] - -(24) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] -Right output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_customer_sk#2], [c_customer_sk#17], Inner, BuildRight - -(25) CometProject -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18], [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_gmt_offset#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [ca_address_sk#19, ca_gmt_offset#20] -Condition : ((isnotnull(ca_gmt_offset#20) AND (ca_gmt_offset#20 = -5.00)) AND isnotnull(ca_address_sk#19)) - -(28) CometProject -Input [2]: [ca_address_sk#19, ca_gmt_offset#20] -Arguments: [ca_address_sk#19], [ca_address_sk#19] - -(29) CometBroadcastExchange -Input [1]: [ca_address_sk#19] -Arguments: [ca_address_sk#19] - -(30) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18] -Right output [1]: [ca_address_sk#19] -Arguments: [c_current_addr_sk#18], [ca_address_sk#19], Inner, BuildRight - -(31) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18, ca_address_sk#19] -Arguments: [ss_item_sk#1, ss_ext_sales_price#5], [ss_item_sk#1, ss_ext_sales_price#5] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#21, i_category#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [i_item_sk#21, i_category#22] -Condition : ((isnotnull(i_category#22) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#22, 50)) = Jewelry )) AND isnotnull(i_item_sk#21)) - -(34) CometProject -Input [2]: [i_item_sk#21, i_category#22] -Arguments: [i_item_sk#21], [i_item_sk#21] - -(35) CometBroadcastExchange -Input [1]: [i_item_sk#21] -Arguments: [i_item_sk#21] - -(36) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#5] -Right output [1]: [i_item_sk#21] -Arguments: [ss_item_sk#1], [i_item_sk#21], Inner, BuildRight - -(37) CometProject -Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#21] -Arguments: [ss_ext_sales_price#5], [ss_ext_sales_price#5] - -(38) CometHashAggregate -Input [1]: [ss_ext_sales_price#5] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(39) CometExchange -Input [1]: [sum#23] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(40) CometHashAggregate -Input [1]: [sum#23] -Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(41) CometColumnarToRow [codegen id : 2] -Input [1]: [promotions#24] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(43) CometFilter -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_store_sk#27) AND isnotnull(ss_customer_sk#26)) AND isnotnull(ss_item_sk#25)) - -(44) ReusedExchange [Reuses operator id: 6] -Output [1]: [s_store_sk#31] - -(45) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Right output [1]: [s_store_sk#31] -Arguments: [ss_store_sk#27], [s_store_sk#31], Inner, BuildRight - -(46) CometProject -Input [6]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, s_store_sk#31] -Arguments: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29], [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] - -(47) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#32] - -(48) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] -Right output [1]: [d_date_sk#32] -Arguments: [ss_sold_date_sk#29], [d_date_sk#32], Inner, BuildRight - -(49) CometProject -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#32] -Arguments: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28], [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] - -(50) ReusedExchange [Reuses operator id: 23] -Output [2]: [c_customer_sk#33, c_current_addr_sk#34] - -(51) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] -Right output [2]: [c_customer_sk#33, c_current_addr_sk#34] -Arguments: [ss_customer_sk#26], [c_customer_sk#33], Inner, BuildRight - -(52) CometProject -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, c_customer_sk#33, c_current_addr_sk#34] -Arguments: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34], [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] - -(53) ReusedExchange [Reuses operator id: 29] -Output [1]: [ca_address_sk#35] - -(54) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] -Right output [1]: [ca_address_sk#35] -Arguments: [c_current_addr_sk#34], [ca_address_sk#35], Inner, BuildRight - -(55) CometProject -Input [4]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34, ca_address_sk#35] -Arguments: [ss_item_sk#25, ss_ext_sales_price#28], [ss_item_sk#25, ss_ext_sales_price#28] - -(56) ReusedExchange [Reuses operator id: 35] -Output [1]: [i_item_sk#36] - -(57) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#25, ss_ext_sales_price#28] -Right output [1]: [i_item_sk#36] -Arguments: [ss_item_sk#25], [i_item_sk#36], Inner, BuildRight - -(58) CometProject -Input [3]: [ss_item_sk#25, ss_ext_sales_price#28, i_item_sk#36] -Arguments: [ss_ext_sales_price#28], [ss_ext_sales_price#28] - -(59) CometHashAggregate -Input [1]: [ss_ext_sales_price#28] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#28))] - -(60) CometExchange -Input [1]: [sum#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(61) CometHashAggregate -Input [1]: [sum#37] -Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] - -(62) CometColumnarToRow [codegen id : 1] -Input [1]: [total#38] - -(63) BroadcastExchange -Input [1]: [total#38] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(64) BroadcastNestedLoopJoin [codegen id : 2] -Join type: Inner -Join condition: None - -(65) Project [codegen id : 2] -Output [3]: [promotions#24, total#38, ((cast(promotions#24 as decimal(15,4)) / cast(total#38 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#39] -Input [2]: [promotions#24, total#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (70) -+- * CometColumnarToRow (69) - +- CometProject (68) - +- CometFilter (67) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) - - -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#14, d_year#15, d_moy#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(67) CometFilter -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1998)) AND (d_moy#16 = 11)) AND isnotnull(d_date_sk#14)) - -(68) CometProject -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(69) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(70) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 42 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/simplified.txt deleted file mode 100644 index 6ebe25b753..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,79 +0,0 @@ -WholeStageCodegen (2) - Project [promotions,total] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [promotions,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange #1 - CometHashAggregate [ss_ext_sales_price] [sum] - CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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_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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk] #3 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - CometBroadcastExchange [p_promo_sk] #4 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk] #7 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_category] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [total,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange #10 - CometHashAggregate [ss_ext_sales_price] [sum] - CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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_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_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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #5 - ReusedExchange [c_customer_sk,c_current_addr_sk] #6 - ReusedExchange [ca_address_sk] #7 - ReusedExchange [i_item_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/explain.txt deleted file mode 100644 index 9465cc9c1d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/explain.txt +++ /dev/null @@ -1,397 +0,0 @@ -== Physical Plan == -* Project (65) -+- * BroadcastNestedLoopJoin Inner BuildRight (64) - :- * CometColumnarToRow (41) - : +- CometHashAggregate (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometBroadcastHashJoin (19) - : : : : :- CometProject (14) - : : : : : +- CometBroadcastHashJoin (13) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (6) - : : : : : : +- CometProject (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (3) - : : : : : +- CometBroadcastExchange (12) - : : : : : +- CometProject (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (9) - : : : : +- CometBroadcastExchange (18) - : : : : +- CometProject (17) - : : : : +- CometFilter (16) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (15) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (21) - : : +- CometBroadcastExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (26) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) - +- BroadcastExchange (63) - +- * CometColumnarToRow (62) - +- CometHashAggregate (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometProject (55) - : +- CometBroadcastHashJoin (54) - : :- CometProject (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometProject (49) - : : : +- CometBroadcastHashJoin (48) - : : : :- CometProject (46) - : : : : +- CometBroadcastHashJoin (45) - : : : : :- CometFilter (43) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (42) - : : : : +- ReusedExchange (44) - : : : +- ReusedExchange (47) - : : +- ReusedExchange (50) - : +- ReusedExchange (53) - +- ReusedExchange (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_store_sk#3) AND isnotnull(ss_promo_sk#4)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_gmt_offset#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [s_store_sk#8, s_gmt_offset#9] -Condition : ((isnotnull(s_gmt_offset#9) AND (s_gmt_offset#9 = -5.00)) AND isnotnull(s_store_sk#8)) - -(5) CometProject -Input [2]: [s_store_sk#8, s_gmt_offset#9] -Arguments: [s_store_sk#8], [s_store_sk#8] - -(6) CometBroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: [s_store_sk#8] - -(7) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [s_store_sk#8] -Arguments: [ss_store_sk#3], [s_store_sk#8], Inner, BuildRight - -(8) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, s_store_sk#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6], [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Condition : ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_dmail#11, 1)) = Y) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_email#12, 1)) = Y)) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_tv#13, 1)) = Y)) AND isnotnull(p_promo_sk#10)) - -(11) CometProject -Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Arguments: [p_promo_sk#10], [p_promo_sk#10] - -(12) CometBroadcastExchange -Input [1]: [p_promo_sk#10] -Arguments: [p_promo_sk#10] - -(13) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [p_promo_sk#10] -Arguments: [ss_promo_sk#4], [p_promo_sk#10], Inner, BuildRight - -(14) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, p_promo_sk#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6], [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#14, d_year#15, d_moy#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1998)) AND (d_moy#16 = 11)) AND isnotnull(d_date_sk#14)) - -(17) CometProject -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(18) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(19) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#6], [d_date_sk#14], Inner, BuildRight - -(20) CometProject -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5], [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(c_current_addr_sk#18)) - -(23) CometBroadcastExchange -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [c_customer_sk#17, c_current_addr_sk#18] - -(24) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] -Right output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_customer_sk#2], [c_customer_sk#17], Inner, BuildRight - -(25) CometProject -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18], [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_gmt_offset#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [ca_address_sk#19, ca_gmt_offset#20] -Condition : ((isnotnull(ca_gmt_offset#20) AND (ca_gmt_offset#20 = -5.00)) AND isnotnull(ca_address_sk#19)) - -(28) CometProject -Input [2]: [ca_address_sk#19, ca_gmt_offset#20] -Arguments: [ca_address_sk#19], [ca_address_sk#19] - -(29) CometBroadcastExchange -Input [1]: [ca_address_sk#19] -Arguments: [ca_address_sk#19] - -(30) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18] -Right output [1]: [ca_address_sk#19] -Arguments: [c_current_addr_sk#18], [ca_address_sk#19], Inner, BuildRight - -(31) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18, ca_address_sk#19] -Arguments: [ss_item_sk#1, ss_ext_sales_price#5], [ss_item_sk#1, ss_ext_sales_price#5] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#21, i_category#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [i_item_sk#21, i_category#22] -Condition : ((isnotnull(i_category#22) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#22, 50)) = Jewelry )) AND isnotnull(i_item_sk#21)) - -(34) CometProject -Input [2]: [i_item_sk#21, i_category#22] -Arguments: [i_item_sk#21], [i_item_sk#21] - -(35) CometBroadcastExchange -Input [1]: [i_item_sk#21] -Arguments: [i_item_sk#21] - -(36) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#5] -Right output [1]: [i_item_sk#21] -Arguments: [ss_item_sk#1], [i_item_sk#21], Inner, BuildRight - -(37) CometProject -Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#21] -Arguments: [ss_ext_sales_price#5], [ss_ext_sales_price#5] - -(38) CometHashAggregate -Input [1]: [ss_ext_sales_price#5] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(39) CometExchange -Input [1]: [sum#23] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(40) CometHashAggregate -Input [1]: [sum#23] -Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(41) CometColumnarToRow [codegen id : 2] -Input [1]: [promotions#24] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(43) CometFilter -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_store_sk#27) AND isnotnull(ss_customer_sk#26)) AND isnotnull(ss_item_sk#25)) - -(44) ReusedExchange [Reuses operator id: 6] -Output [1]: [s_store_sk#31] - -(45) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Right output [1]: [s_store_sk#31] -Arguments: [ss_store_sk#27], [s_store_sk#31], Inner, BuildRight - -(46) CometProject -Input [6]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, s_store_sk#31] -Arguments: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29], [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] - -(47) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#32] - -(48) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] -Right output [1]: [d_date_sk#32] -Arguments: [ss_sold_date_sk#29], [d_date_sk#32], Inner, BuildRight - -(49) CometProject -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#32] -Arguments: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28], [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] - -(50) ReusedExchange [Reuses operator id: 23] -Output [2]: [c_customer_sk#33, c_current_addr_sk#34] - -(51) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] -Right output [2]: [c_customer_sk#33, c_current_addr_sk#34] -Arguments: [ss_customer_sk#26], [c_customer_sk#33], Inner, BuildRight - -(52) CometProject -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, c_customer_sk#33, c_current_addr_sk#34] -Arguments: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34], [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] - -(53) ReusedExchange [Reuses operator id: 29] -Output [1]: [ca_address_sk#35] - -(54) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] -Right output [1]: [ca_address_sk#35] -Arguments: [c_current_addr_sk#34], [ca_address_sk#35], Inner, BuildRight - -(55) CometProject -Input [4]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34, ca_address_sk#35] -Arguments: [ss_item_sk#25, ss_ext_sales_price#28], [ss_item_sk#25, ss_ext_sales_price#28] - -(56) ReusedExchange [Reuses operator id: 35] -Output [1]: [i_item_sk#36] - -(57) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#25, ss_ext_sales_price#28] -Right output [1]: [i_item_sk#36] -Arguments: [ss_item_sk#25], [i_item_sk#36], Inner, BuildRight - -(58) CometProject -Input [3]: [ss_item_sk#25, ss_ext_sales_price#28, i_item_sk#36] -Arguments: [ss_ext_sales_price#28], [ss_ext_sales_price#28] - -(59) CometHashAggregate -Input [1]: [ss_ext_sales_price#28] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#28))] - -(60) CometExchange -Input [1]: [sum#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(61) CometHashAggregate -Input [1]: [sum#37] -Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] - -(62) CometColumnarToRow [codegen id : 1] -Input [1]: [total#38] - -(63) BroadcastExchange -Input [1]: [total#38] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(64) BroadcastNestedLoopJoin [codegen id : 2] -Join type: Inner -Join condition: None - -(65) Project [codegen id : 2] -Output [3]: [promotions#24, total#38, ((cast(promotions#24 as decimal(15,4)) / cast(total#38 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#39] -Input [2]: [promotions#24, total#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (70) -+- * CometColumnarToRow (69) - +- CometProject (68) - +- CometFilter (67) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) - - -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#14, d_year#15, d_moy#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(67) CometFilter -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1998)) AND (d_moy#16 = 11)) AND isnotnull(d_date_sk#14)) - -(68) CometProject -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(69) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(70) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 42 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/extended.txt deleted file mode 100644 index 7bae7d5724..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/extended.txt +++ /dev/null @@ -1,88 +0,0 @@ -Project -+- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 77 out of 83 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 6ebe25b753..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt +++ /dev/null @@ -1,79 +0,0 @@ -WholeStageCodegen (2) - Project [promotions,total] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [promotions,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange #1 - CometHashAggregate [ss_ext_sales_price] [sum] - CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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_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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk] #3 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - CometBroadcastExchange [p_promo_sk] #4 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk] #7 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_category] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [total,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange #10 - CometHashAggregate [ss_ext_sales_price] [sum] - CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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_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_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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #5 - ReusedExchange [c_customer_sk,c_current_addr_sk] #6 - ReusedExchange [ca_address_sk] #7 - ReusedExchange [i_item_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/explain.txt deleted file mode 100644 index 9c710b15fb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometNativeScan parquet spark_catalog.default.warehouse (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometNativeScan parquet spark_catalog.default.ship_mode (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.web_site (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometNativeScan parquet spark_catalog.default.date_dim (19) - - -(1) CometNativeScan parquet spark_catalog.default.web_sales -Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_ship_mode_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Condition : (((isnotnull(ws_warehouse_sk#4) AND isnotnull(ws_ship_mode_sk#3)) AND isnotnull(ws_web_site_sk#2)) AND isnotnull(ws_ship_date_sk#1)) - -(3) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [ws_warehouse_sk#4], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7], [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] - -(8) CometNativeScan parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#8, sm_type#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Condition : isnotnull(sm_ship_mode_sk#8) - -(10) CometProject -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Arguments: [sm_ship_mode_sk#8, sm_type#10], [sm_ship_mode_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(sm_type#9, 30)) AS sm_type#10] - -(11) CometBroadcastExchange -Input [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [sm_ship_mode_sk#8, sm_type#10] - -(12) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] -Right output [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [ws_ship_mode_sk#3], [sm_ship_mode_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#10] -Arguments: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10], [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10] - -(14) CometNativeScan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#11, web_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [web_site_sk#11, web_name#12] -Condition : isnotnull(web_site_sk#11) - -(16) CometBroadcastExchange -Input [2]: [web_site_sk#11, web_name#12] -Arguments: [web_site_sk#11, web_name#12] - -(17) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10] -Right output [2]: [web_site_sk#11, web_name#12] -Arguments: [ws_web_site_sk#2], [web_site_sk#11], Inner, BuildRight - -(18) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_site_sk#11, web_name#12] -Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12], [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12] - -(19) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_month_seq#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#13, d_month_seq#14] -Condition : (((isnotnull(d_month_seq#14) AND (d_month_seq#14 >= 1200)) AND (d_month_seq#14 <= 1211)) AND isnotnull(d_date_sk#13)) - -(21) CometProject -Input [2]: [d_date_sk#13, d_month_seq#14] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(23) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12] -Right output [1]: [d_date_sk#13] -Arguments: [ws_ship_date_sk#1], [d_date_sk#13], Inner, BuildRight - -(24) CometProject -Input [6]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12, d_date_sk#13] -Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, _groupingexpression#15], [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#15] - -(25) CometHashAggregate -Input [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, sm_type#10, web_name#12] -Functions [5]: [partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [8]: [_groupingexpression#15, sm_type#10, web_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Arguments: hashpartitioning(_groupingexpression#15, sm_type#10, web_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [8]: [_groupingexpression#15, sm_type#10, web_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Keys [3]: [_groupingexpression#15, sm_type#10, web_name#12] -Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,web_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,web_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, web_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - -(29) CometColumnarToRow [codegen id : 1] -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/simplified.txt deleted file mode 100644 index de80c17349..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_datafusion/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 30) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 60) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 90) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [_groupingexpression,sm_type,web_name] #1 - CometHashAggregate [ws_ship_date_sk,ws_sold_date_sk] [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum] - CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] - 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_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_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_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] - CometNativeScan 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 [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [web_site_sk,web_name] #4 - CometFilter [web_site_sk,web_name] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan 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/q62.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_iceberg_compat/explain.txt deleted file mode 100644 index fe73a4b466..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_ship_mode_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Condition : (((isnotnull(ws_warehouse_sk#4) AND isnotnull(ws_ship_mode_sk#3)) AND isnotnull(ws_web_site_sk#2)) AND isnotnull(ws_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [ws_warehouse_sk#4], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7], [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#8, sm_type#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Condition : isnotnull(sm_ship_mode_sk#8) - -(10) CometProject -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Arguments: [sm_ship_mode_sk#8, sm_type#10], [sm_ship_mode_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(sm_type#9, 30)) AS sm_type#10] - -(11) CometBroadcastExchange -Input [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [sm_ship_mode_sk#8, sm_type#10] - -(12) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] -Right output [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [ws_ship_mode_sk#3], [sm_ship_mode_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#10] -Arguments: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10], [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#11, web_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [web_site_sk#11, web_name#12] -Condition : isnotnull(web_site_sk#11) - -(16) CometBroadcastExchange -Input [2]: [web_site_sk#11, web_name#12] -Arguments: [web_site_sk#11, web_name#12] - -(17) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10] -Right output [2]: [web_site_sk#11, web_name#12] -Arguments: [ws_web_site_sk#2], [web_site_sk#11], Inner, BuildRight - -(18) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_site_sk#11, web_name#12] -Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12], [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_month_seq#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#13, d_month_seq#14] -Condition : (((isnotnull(d_month_seq#14) AND (d_month_seq#14 >= 1200)) AND (d_month_seq#14 <= 1211)) AND isnotnull(d_date_sk#13)) - -(21) CometProject -Input [2]: [d_date_sk#13, d_month_seq#14] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(23) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12] -Right output [1]: [d_date_sk#13] -Arguments: [ws_ship_date_sk#1], [d_date_sk#13], Inner, BuildRight - -(24) CometProject -Input [6]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12, d_date_sk#13] -Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, _groupingexpression#15], [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#15] - -(25) CometHashAggregate -Input [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, sm_type#10, web_name#12] -Functions [5]: [partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [8]: [_groupingexpression#15, sm_type#10, web_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Arguments: hashpartitioning(_groupingexpression#15, sm_type#10, web_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [8]: [_groupingexpression#15, sm_type#10, web_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Keys [3]: [_groupingexpression#15, sm_type#10, web_name#12] -Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,web_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,web_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, web_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - -(29) CometColumnarToRow [codegen id : 1] -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_iceberg_compat/simplified.txt deleted file mode 100644 index dee2e434c7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 30) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 60) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 90) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [_groupingexpression,sm_type,web_name] #1 - CometHashAggregate [ws_ship_date_sk,ws_sold_date_sk] [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum] - CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] - 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_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_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_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 [native_iceberg_compat] 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 [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [web_site_sk,web_name] #4 - CometFilter [web_site_sk,web_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] 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/q62/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/explain.txt deleted file mode 100644 index fe73a4b466..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_ship_mode_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Condition : (((isnotnull(ws_warehouse_sk#4) AND isnotnull(ws_ship_mode_sk#3)) AND isnotnull(ws_web_site_sk#2)) AND isnotnull(ws_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [ws_warehouse_sk#4], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7], [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#8, sm_type#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Condition : isnotnull(sm_ship_mode_sk#8) - -(10) CometProject -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Arguments: [sm_ship_mode_sk#8, sm_type#10], [sm_ship_mode_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(sm_type#9, 30)) AS sm_type#10] - -(11) CometBroadcastExchange -Input [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [sm_ship_mode_sk#8, sm_type#10] - -(12) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] -Right output [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [ws_ship_mode_sk#3], [sm_ship_mode_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#10] -Arguments: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10], [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#11, web_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [web_site_sk#11, web_name#12] -Condition : isnotnull(web_site_sk#11) - -(16) CometBroadcastExchange -Input [2]: [web_site_sk#11, web_name#12] -Arguments: [web_site_sk#11, web_name#12] - -(17) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10] -Right output [2]: [web_site_sk#11, web_name#12] -Arguments: [ws_web_site_sk#2], [web_site_sk#11], Inner, BuildRight - -(18) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_site_sk#11, web_name#12] -Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12], [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_month_seq#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#13, d_month_seq#14] -Condition : (((isnotnull(d_month_seq#14) AND (d_month_seq#14 >= 1200)) AND (d_month_seq#14 <= 1211)) AND isnotnull(d_date_sk#13)) - -(21) CometProject -Input [2]: [d_date_sk#13, d_month_seq#14] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(23) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12] -Right output [1]: [d_date_sk#13] -Arguments: [ws_ship_date_sk#1], [d_date_sk#13], Inner, BuildRight - -(24) CometProject -Input [6]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12, d_date_sk#13] -Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, _groupingexpression#15], [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#15] - -(25) CometHashAggregate -Input [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, sm_type#10, web_name#12] -Functions [5]: [partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [8]: [_groupingexpression#15, sm_type#10, web_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Arguments: hashpartitioning(_groupingexpression#15, sm_type#10, web_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [8]: [_groupingexpression#15, sm_type#10, web_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Keys [3]: [_groupingexpression#15, sm_type#10, web_name#12] -Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,web_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,web_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, web_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - -(29) CometColumnarToRow [codegen id : 1] -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/extended.txt deleted file mode 100644 index a5fd7359a3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 28 out of 28 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index dee2e434c7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 30) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 60) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 90) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [_groupingexpression,sm_type,web_name] #1 - CometHashAggregate [ws_ship_date_sk,ws_sold_date_sk] [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum] - CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] - 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_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_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_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 [native_iceberg_compat] 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 [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [web_site_sk,web_name] #4 - CometFilter [web_site_sk,web_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/explain.txt deleted file mode 100644 index 03bd5bf65c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/explain.txt +++ /dev/null @@ -1,202 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * CometColumnarToRow (26) - +- CometSort (25) - +- CometColumnarExchange (24) - +- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometColumnarExchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * CometColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (8) - : : +- * Filter (7) - : : +- * ColumnarToRow (6) - : : +- Scan parquet spark_catalog.default.store_sales (5) - : +- ReusedExchange (11) - +- BroadcastExchange (17) - +- * CometColumnarToRow (16) - +- CometFilter (15) - +- CometNativeScan parquet spark_catalog.default.store (14) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) IN (Books ,Children ,Electronics ) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#3, 50)) IN (personal ,portable ,refernece ,self-help )) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) IN (Women ,Music ,Men ) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#3, 50)) IN (accessories ,classical ,fragrances ,pants )) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Arguments: [i_item_sk#1, i_manager_id#5], [i_item_sk#1, i_manager_id#5] - -(4) CometColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#1, i_manager_id#5] - -(5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(7) Filter [codegen id : 1] -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) - -(8) BroadcastExchange -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#10] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Input [6]: [i_item_sk#1, i_manager_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(11) ReusedExchange [Reuses operator id: 35] -Output [2]: [d_date_sk#15, d_moy#16] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#13] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#16] -Input [6]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_moy#16] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [1]: [s_store_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [1]: [s_store_sk#17] -Condition : isnotnull(s_store_sk#17) - -(16) CometColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#17] - -(17) BroadcastExchange -Input [1]: [s_store_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#11] -Right keys [1]: [s_store_sk#17] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [3]: [i_manager_id#5, ss_sales_price#12, d_moy#16] -Input [5]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#16, s_store_sk#17] - -(20) HashAggregate [codegen id : 4] -Input [3]: [i_manager_id#5, ss_sales_price#12, d_moy#16] -Keys [2]: [i_manager_id#5, d_moy#16] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [i_manager_id#5, d_moy#16, sum#19] - -(21) CometColumnarExchange -Input [3]: [i_manager_id#5, d_moy#16, sum#19] -Arguments: hashpartitioning(i_manager_id#5, d_moy#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [3]: [i_manager_id#5, d_moy#16, sum#19] - -(23) HashAggregate [codegen id : 5] -Input [3]: [i_manager_id#5, d_moy#16, sum#19] -Keys [2]: [i_manager_id#5, d_moy#16] -Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#20] -Results [3]: [i_manager_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS _w0#22] - -(24) CometColumnarExchange -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: [i_manager_id#5, sum_sales#21, _w0#22], [i_manager_id#5 ASC NULLS FIRST] - -(26) CometColumnarToRow [codegen id : 6] -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] - -(27) Window -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_manager_id#5] - -(28) Filter [codegen id : 7] -Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] -Condition : CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) ELSE false END - -(29) Project [codegen id : 7] -Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] -Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] - -(30) TakeOrderedAndProject -Input [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] -Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometNativeScan parquet spark_catalog.default.date_dim (31) - - -(31) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] -Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(33) CometProject -Input [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] -Arguments: [d_date_sk#15, d_moy#16], [d_date_sk#15, d_moy#16] - -(34) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_moy#16] - -(35) BroadcastExchange -Input [2]: [d_date_sk#15, d_moy#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/simplified.txt deleted file mode 100644 index 2933149226..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/simplified.txt +++ /dev/null @@ -1,53 +0,0 @@ -TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (7) - Project [i_manager_id,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_manager_id] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_manager_id,sum_sales,_w0] - CometColumnarExchange [i_manager_id] #1 - WholeStageCodegen (5) - HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manager_id,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] - Project [i_manager_id,ss_sales_price,d_moy] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_sk,i_manager_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/explain.txt deleted file mode 100644 index 608bf20b3e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,198 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) - +- CometSort (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) IN (Books ,Children ,Electronics ) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#3, 50)) IN (personal ,portable ,refernece ,self-help )) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) IN (Women ,Music ,Men ) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#3, 50)) IN (accessories ,classical ,fragrances ,pants )) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Arguments: [i_item_sk#1, i_manager_id#5], [i_item_sk#1, i_manager_id#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(7) CometBroadcastHashJoin -Left output [2]: [i_item_sk#1, i_manager_id#5] -Right output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_item_sk#1], [ss_item_sk#10], Inner, BuildRight - -(8) CometProject -Input [6]: [i_item_sk#1, i_manager_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13], [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(11) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] - -(12) CometBroadcastExchange -Input [2]: [d_date_sk#15, d_moy#17] -Arguments: [d_date_sk#15, d_moy#17] - -(13) CometBroadcastHashJoin -Left output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_moy#17] -Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(14) CometProject -Input [6]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_moy#17] -Arguments: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17], [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [s_store_sk#18] -Condition : isnotnull(s_store_sk#18) - -(17) CometBroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: [s_store_sk#18] - -(18) CometBroadcastHashJoin -Left output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17] -Right output [1]: [s_store_sk#18] -Arguments: [ss_store_sk#11], [s_store_sk#18], Inner, BuildRight - -(19) CometProject -Input [5]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17, s_store_sk#18] -Arguments: [i_manager_id#5, ss_sales_price#12, d_moy#17], [i_manager_id#5, ss_sales_price#12, d_moy#17] - -(20) CometHashAggregate -Input [3]: [i_manager_id#5, ss_sales_price#12, d_moy#17] -Keys [2]: [i_manager_id#5, d_moy#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] - -(21) CometExchange -Input [3]: [i_manager_id#5, d_moy#17, sum#19] -Arguments: hashpartitioning(i_manager_id#5, d_moy#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [3]: [i_manager_id#5, d_moy#17, sum#19] -Keys [2]: [i_manager_id#5, d_moy#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] - -(23) CometExchange -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: [i_manager_id#5, sum_sales#20, _w0#21], [i_manager_id#5 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 1] -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] - -(26) Window -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_manager_id#5] - -(27) Filter [codegen id : 2] -Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] -Condition : CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) ELSE false END - -(28) Project [codegen id : 2] -Output [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] -Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] - -(29) TakeOrderedAndProject -Input [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] -Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(32) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_moy#17] - -(34) BroadcastExchange -Input [2]: [d_date_sk#15, d_moy#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/simplified.txt deleted file mode 100644 index 02166879a9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (2) - Project [i_manager_id,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_manager_id] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_manager_id,sum_sales,_w0] - CometExchange [i_manager_id] #1 - CometHashAggregate [d_moy,sum] [i_manager_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_manager_id,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] - CometProject [i_manager_id,ss_sales_price,d_moy] - 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 [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,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_item_sk,i_brand,i_class,i_category,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt deleted file mode 100644 index 608bf20b3e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt +++ /dev/null @@ -1,198 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) - +- CometSort (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) IN (Books ,Children ,Electronics ) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#3, 50)) IN (personal ,portable ,refernece ,self-help )) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) IN (Women ,Music ,Men ) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#3, 50)) IN (accessories ,classical ,fragrances ,pants )) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Arguments: [i_item_sk#1, i_manager_id#5], [i_item_sk#1, i_manager_id#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(7) CometBroadcastHashJoin -Left output [2]: [i_item_sk#1, i_manager_id#5] -Right output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_item_sk#1], [ss_item_sk#10], Inner, BuildRight - -(8) CometProject -Input [6]: [i_item_sk#1, i_manager_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13], [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(11) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] - -(12) CometBroadcastExchange -Input [2]: [d_date_sk#15, d_moy#17] -Arguments: [d_date_sk#15, d_moy#17] - -(13) CometBroadcastHashJoin -Left output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_moy#17] -Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(14) CometProject -Input [6]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_moy#17] -Arguments: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17], [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [s_store_sk#18] -Condition : isnotnull(s_store_sk#18) - -(17) CometBroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: [s_store_sk#18] - -(18) CometBroadcastHashJoin -Left output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17] -Right output [1]: [s_store_sk#18] -Arguments: [ss_store_sk#11], [s_store_sk#18], Inner, BuildRight - -(19) CometProject -Input [5]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17, s_store_sk#18] -Arguments: [i_manager_id#5, ss_sales_price#12, d_moy#17], [i_manager_id#5, ss_sales_price#12, d_moy#17] - -(20) CometHashAggregate -Input [3]: [i_manager_id#5, ss_sales_price#12, d_moy#17] -Keys [2]: [i_manager_id#5, d_moy#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] - -(21) CometExchange -Input [3]: [i_manager_id#5, d_moy#17, sum#19] -Arguments: hashpartitioning(i_manager_id#5, d_moy#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [3]: [i_manager_id#5, d_moy#17, sum#19] -Keys [2]: [i_manager_id#5, d_moy#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] - -(23) CometExchange -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: [i_manager_id#5, sum_sales#20, _w0#21], [i_manager_id#5 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 1] -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] - -(26) Window -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_manager_id#5] - -(27) Filter [codegen id : 2] -Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] -Condition : CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) ELSE false END - -(28) Project [codegen id : 2] -Output [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] -Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] - -(29) TakeOrderedAndProject -Input [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] -Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(32) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_moy#17] - -(34) BroadcastExchange -Input [2]: [d_date_sk#15, d_moy#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/extended.txt deleted file mode 100644 index 825b1ed81c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 02166879a9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (2) - Project [i_manager_id,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_manager_id] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_manager_id,sum_sales,_w0] - CometExchange [i_manager_id] #1 - CometHashAggregate [d_moy,sum] [i_manager_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_manager_id,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] - CometProject [i_manager_id,ss_sales_price,d_moy] - 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 [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,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_item_sk,i_brand,i_class,i_category,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/explain.txt deleted file mode 100644 index ff9f485dd9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/explain.txt +++ /dev/null @@ -1,1029 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (180) -+- CometSort (179) - +- CometExchange (178) - +- CometProject (177) - +- CometSortMergeJoin (176) - :- CometSort (105) - : +- CometExchange (104) - : +- CometHashAggregate (103) - : +- CometHashAggregate (102) - : +- CometProject (101) - : +- CometBroadcastHashJoin (100) - : :- CometProject (95) - : : +- CometBroadcastHashJoin (94) - : : :- CometProject (92) - : : : +- CometBroadcastHashJoin (91) - : : : :- CometProject (87) - : : : : +- CometBroadcastHashJoin (86) - : : : : :- CometProject (84) - : : : : : +- CometBroadcastHashJoin (83) - : : : : : :- CometProject (78) - : : : : : : +- CometBroadcastHashJoin (77) - : : : : : : :- CometProject (75) - : : : : : : : +- CometBroadcastHashJoin (74) - : : : : : : : :- CometProject (70) - : : : : : : : : +- CometBroadcastHashJoin (69) - : : : : : : : : :- CometProject (65) - : : : : : : : : : +- CometBroadcastHashJoin (64) - : : : : : : : : : :- CometProject (62) - : : : : : : : : : : +- CometBroadcastHashJoin (61) - : : : : : : : : : : :- CometProject (56) - : : : : : : : : : : : +- CometBroadcastHashJoin (55) - : : : : : : : : : : : :- CometProject (53) - : : : : : : : : : : : : +- CometBroadcastHashJoin (52) - : : : : : : : : : : : : :- CometProject (48) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (47) - : : : : : : : : : : : : : :- CometProject (43) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (42) - : : : : : : : : : : : : : : :- CometProject (37) - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (36) - : : : : : : : : : : : : : : : :- CometProject (32) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (31) - : : : : : : : : : : : : : : : : :- CometSort (12) - : : : : : : : : : : : : : : : : : +- CometColumnarExchange (11) - : : : : : : : : : : : : : : : : : +- * Project (10) - : : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : : : : : : : : : : : : : : :- BroadcastExchange (4) - : : : : : : : : : : : : : : : : : : +- * Filter (3) - : : : : : : : : : : : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (8) - : : : : : : : : : : : : : : : : : +- CometProject (7) - : : : : : : : : : : : : : : : : : +- CometFilter (6) - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (5) - : : : : : : : : : : : : : : : : +- CometSort (30) - : : : : : : : : : : : : : : : : +- CometProject (29) - : : : : : : : : : : : : : : : : +- CometFilter (28) - : : : : : : : : : : : : : : : : +- CometHashAggregate (27) - : : : : : : : : : : : : : : : : +- CometExchange (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometProject (24) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (23) - : : : : : : : : : : : : : : : : :- CometSort (17) - : : : : : : : : : : : : : : : : : +- CometExchange (16) - : : : : : : : : : : : : : : : : : +- CometProject (15) - : : : : : : : : : : : : : : : : : +- CometFilter (14) - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : : : : : : : : : : : +- CometSort (22) - : : : : : : : : : : : : : : : : +- CometExchange (21) - : : : : : : : : : : : : : : : : +- CometProject (20) - : : : : : : : : : : : : : : : : +- CometFilter (19) - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (18) - : : : : : : : : : : : : : : : +- CometBroadcastExchange (35) - : : : : : : : : : : : : : : : +- CometFilter (34) - : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (33) - : : : : : : : : : : : : : : +- CometBroadcastExchange (41) - : : : : : : : : : : : : : : +- CometProject (40) - : : : : : : : : : : : : : : +- CometFilter (39) - : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store (38) - : : : : : : : : : : : : : +- CometBroadcastExchange (46) - : : : : : : : : : : : : : +- CometFilter (45) - : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer (44) - : : : : : : : : : : : : +- CometBroadcastExchange (51) - : : : : : : : : : : : : +- CometFilter (50) - : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (49) - : : : : : : : : : : : +- ReusedExchange (54) - : : : : : : : : : : +- CometBroadcastExchange (60) - : : : : : : : : : : +- CometProject (59) - : : : : : : : : : : +- CometFilter (58) - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (57) - : : : : : : : : : +- ReusedExchange (63) - : : : : : : : : +- CometBroadcastExchange (68) - : : : : : : : : +- CometFilter (67) - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.promotion (66) - : : : : : : : +- CometBroadcastExchange (73) - : : : : : : : +- CometFilter (72) - : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (71) - : : : : : : +- ReusedExchange (76) - : : : : : +- CometBroadcastExchange (82) - : : : : : +- CometProject (81) - : : : : : +- CometFilter (80) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address (79) - : : : : +- ReusedExchange (85) - : : : +- CometBroadcastExchange (90) - : : : +- CometFilter (89) - : : : +- CometNativeScan parquet spark_catalog.default.income_band (88) - : : +- ReusedExchange (93) - : +- CometBroadcastExchange (99) - : +- CometProject (98) - : +- CometFilter (97) - : +- CometNativeScan parquet spark_catalog.default.item (96) - +- CometSort (175) - +- CometExchange (174) - +- CometHashAggregate (173) - +- CometHashAggregate (172) - +- CometProject (171) - +- CometBroadcastHashJoin (170) - :- CometProject (168) - : +- CometBroadcastHashJoin (167) - : :- CometProject (165) - : : +- CometBroadcastHashJoin (164) - : : :- CometProject (162) - : : : +- CometBroadcastHashJoin (161) - : : : :- CometProject (159) - : : : : +- CometBroadcastHashJoin (158) - : : : : :- CometProject (156) - : : : : : +- CometBroadcastHashJoin (155) - : : : : : :- CometProject (153) - : : : : : : +- CometBroadcastHashJoin (152) - : : : : : : :- CometProject (150) - : : : : : : : +- CometBroadcastHashJoin (149) - : : : : : : : :- CometProject (147) - : : : : : : : : +- CometBroadcastHashJoin (146) - : : : : : : : : :- CometProject (144) - : : : : : : : : : +- CometBroadcastHashJoin (143) - : : : : : : : : : :- CometProject (141) - : : : : : : : : : : +- CometBroadcastHashJoin (140) - : : : : : : : : : : :- CometProject (138) - : : : : : : : : : : : +- CometBroadcastHashJoin (137) - : : : : : : : : : : : :- CometProject (135) - : : : : : : : : : : : : +- CometBroadcastHashJoin (134) - : : : : : : : : : : : : :- CometProject (132) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (131) - : : : : : : : : : : : : : :- CometProject (129) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (128) - : : : : : : : : : : : : : : :- CometProject (124) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (123) - : : : : : : : : : : : : : : : :- CometSort (117) - : : : : : : : : : : : : : : : : +- CometColumnarExchange (116) - : : : : : : : : : : : : : : : : +- * Project (115) - : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (114) - : : : : : : : : : : : : : : : : :- BroadcastExchange (109) - : : : : : : : : : : : : : : : : : +- * Filter (108) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (107) - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (106) - : : : : : : : : : : : : : : : : +- * CometColumnarToRow (113) - : : : : : : : : : : : : : : : : +- CometProject (112) - : : : : : : : : : : : : : : : : +- CometFilter (111) - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (110) - : : : : : : : : : : : : : : : +- CometSort (122) - : : : : : : : : : : : : : : : +- CometProject (121) - : : : : : : : : : : : : : : : +- CometFilter (120) - : : : : : : : : : : : : : : : +- CometHashAggregate (119) - : : : : : : : : : : : : : : : +- ReusedExchange (118) - : : : : : : : : : : : : : : +- CometBroadcastExchange (127) - : : : : : : : : : : : : : : +- CometFilter (126) - : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (125) - : : : : : : : : : : : : : +- ReusedExchange (130) - : : : : : : : : : : : : +- ReusedExchange (133) - : : : : : : : : : : : +- ReusedExchange (136) - : : : : : : : : : : +- ReusedExchange (139) - : : : : : : : : : +- ReusedExchange (142) - : : : : : : : : +- ReusedExchange (145) - : : : : : : : +- ReusedExchange (148) - : : : : : : +- ReusedExchange (151) - : : : : : +- ReusedExchange (154) - : : : : +- ReusedExchange (157) - : : : +- ReusedExchange (160) - : : +- ReusedExchange (163) - : +- ReusedExchange (166) - +- ReusedExchange (169) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(3) Filter [codegen id : 1] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) - -(4) BroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=1] - -(5) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) - -(7) CometProject -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] - -(8) CometColumnarToRow -Input [2]: [sr_item_sk#14, sr_ticket_number#15] - -(9) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [ss_item_sk#1, ss_ticket_number#8] -Right keys [2]: [sr_item_sk#14, sr_ticket_number#15] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 2] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] - -(11) CometColumnarExchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(12) CometSort -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] - -(13) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) - -(15) CometProject -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] - -(16) CometExchange -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(17) CometSort -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] - -(18) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(19) CometFilter -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) - -(20) CometProject -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(21) CometExchange -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(22) CometSort -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] - -(23) CometSortMergeJoin -Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner - -(24) CometProject -Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(25) CometHashAggregate -Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Keys [1]: [cs_item_sk#17] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(26) CometExchange -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(27) CometHashAggregate -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Keys [1]: [cs_item_sk#17] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(28) CometFilter -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(29) CometProject -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Arguments: [cs_item_sk#17], [cs_item_sk#17] - -(30) CometSort -Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] - -(31) CometSortMergeJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [1]: [cs_item_sk#17] -Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner - -(32) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(33) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(35) CometBroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: [d_date_sk#32, d_year#33] - -(36) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight - -(37) CometProject -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] - -(38) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name)] -ReadSchema: struct - -(39) CometFilter -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#36, 10)))) - -(40) CometProject -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37], [s_store_sk#34, s_store_name#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#36, 10)) AS s_zip#37] - -(41) CometBroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37] - -(42) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight - -(43) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] - -(44) CometNativeScan parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(45) CometFilter -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Condition : (((((isnotnull(c_customer_sk#38) AND isnotnull(c_first_sales_date_sk#43)) AND isnotnull(c_first_shipto_date_sk#42)) AND isnotnull(c_current_cdemo_sk#39)) AND isnotnull(c_current_hdemo_sk#40)) AND isnotnull(c_current_addr_sk#41)) - -(46) CometBroadcastExchange -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(47) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -Right output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_customer_sk#2], [c_customer_sk#38], Inner, BuildRight - -(48) CometProject -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(49) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : isnotnull(d_date_sk#44) - -(51) CometBroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] - -(52) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [c_first_sales_date_sk#43], [d_date_sk#44], Inner, BuildRight - -(53) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43, d_date_sk#44, d_year#45] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] - -(54) ReusedExchange [Reuses operator id: 51] -Output [2]: [d_date_sk#46, d_year#47] - -(55) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [c_first_shipto_date_sk#42], [d_date_sk#46], Inner, BuildRight - -(56) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45, d_date_sk#46, d_year#47] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(57) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#48, cd_marital_status#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(58) CometFilter -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#49, 1)))) - -(59) CometProject -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Arguments: [cd_demo_sk#48, cd_marital_status#50], [cd_demo_sk#48, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#49, 1)) AS cd_marital_status#50] - -(60) CometBroadcastExchange -Input [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [cd_demo_sk#48, cd_marital_status#50] - -(61) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_cdemo_sk#3], [cd_demo_sk#48], Inner, BuildRight - -(62) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] - -(63) ReusedExchange [Reuses operator id: 60] -Output [2]: [cd_demo_sk#51, cd_marital_status#52] - -(64) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -Right output [2]: [cd_demo_sk#51, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#39], [cd_demo_sk#51], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(65) CometProject -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50, cd_demo_sk#51, cd_marital_status#52] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(66) CometNativeScan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(67) CometFilter -Input [1]: [p_promo_sk#53] -Condition : isnotnull(p_promo_sk#53) - -(68) CometBroadcastExchange -Input [1]: [p_promo_sk#53] -Arguments: [p_promo_sk#53] - -(69) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [1]: [p_promo_sk#53] -Arguments: [ss_promo_sk#7], [p_promo_sk#53], Inner, BuildRight - -(70) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, p_promo_sk#53] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(71) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(72) CometFilter -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Condition : (isnotnull(hd_demo_sk#54) AND isnotnull(hd_income_band_sk#55)) - -(73) CometBroadcastExchange -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [hd_demo_sk#54, hd_income_band_sk#55] - -(74) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_hdemo_sk#4], [hd_demo_sk#54], Inner, BuildRight - -(75) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] - -(76) ReusedExchange [Reuses operator id: 73] -Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] - -(77) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -Right output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [c_current_hdemo_sk#40], [hd_demo_sk#56], Inner, BuildRight - -(78) CometProject -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] - -(79) CometNativeScan parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(80) CometFilter -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Condition : isnotnull(ca_address_sk#58) - -(81) CometProject -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ca_address_sk#58, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_number#59, 10)) AS ca_street_number#63, ca_street_name#60, ca_city#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#62, 10)) AS ca_zip#64] - -(82) CometBroadcastExchange -Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(83) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -Right output [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_addr_sk#5], [ca_address_sk#58], Inner, BuildRight - -(84) CometProject -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(85) ReusedExchange [Reuses operator id: 82] -Output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(86) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Right output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [c_current_addr_sk#41], [ca_address_sk#65], Inner, BuildRight - -(87) CometProject -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(88) CometNativeScan parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(89) CometFilter -Input [1]: [ib_income_band_sk#70] -Condition : isnotnull(ib_income_band_sk#70) - -(90) CometBroadcastExchange -Input [1]: [ib_income_band_sk#70] -Arguments: [ib_income_band_sk#70] - -(91) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#70] -Arguments: [hd_income_band_sk#55], [ib_income_band_sk#70], Inner, BuildRight - -(92) CometProject -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#70] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(93) ReusedExchange [Reuses operator id: 90] -Output [1]: [ib_income_band_sk#71] - -(94) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#71] -Arguments: [hd_income_band_sk#57], [ib_income_band_sk#71], Inner, BuildRight - -(95) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#71] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(96) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(97) CometFilter -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Condition : ((((((isnotnull(i_current_price#73) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#74, 20)) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#73 >= 64.00)) AND (i_current_price#73 <= 74.00)) AND (i_current_price#73 >= 65.00)) AND (i_current_price#73 <= 79.00)) AND isnotnull(i_item_sk#72)) - -(98) CometProject -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Arguments: [i_item_sk#72, i_product_name#76], [i_item_sk#72, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#75, 50)) AS i_product_name#76] - -(99) CometBroadcastExchange -Input [2]: [i_item_sk#72, i_product_name#76] -Arguments: [i_item_sk#72, i_product_name#76] - -(100) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [2]: [i_item_sk#72, i_product_name#76] -Arguments: [ss_item_sk#1], [i_item_sk#72], Inner, BuildRight - -(101) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] - -(102) CometHashAggregate -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] - -(103) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#77, sum#78, sum#79, sum#80] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] - -(104) CometExchange -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: hashpartitioning(item_sk#82, store_name#83, store_zip#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(105) CometSort -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97], [item_sk#82 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, store_zip#84 ASC NULLS FIRST] - -(106) Scan parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#109), dynamicpruningexpression(ss_sold_date_sk#109 IN dynamicpruning#110)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(107) ColumnarToRow [codegen id : 3] -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(108) Filter [codegen id : 3] -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Condition : (((((((isnotnull(ss_item_sk#98) AND isnotnull(ss_ticket_number#105)) AND isnotnull(ss_store_sk#103)) AND isnotnull(ss_customer_sk#99)) AND isnotnull(ss_cdemo_sk#100)) AND isnotnull(ss_promo_sk#104)) AND isnotnull(ss_hdemo_sk#101)) AND isnotnull(ss_addr_sk#102)) - -(109) BroadcastExchange -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=7] - -(110) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(111) CometFilter -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) - -(112) CometProject -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] - -(113) CometColumnarToRow -Input [2]: [sr_item_sk#111, sr_ticket_number#112] - -(114) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [ss_item_sk#98, ss_ticket_number#105] -Right keys [2]: [sr_item_sk#111, sr_ticket_number#112] -Join type: Inner -Join condition: None - -(115) Project [codegen id : 4] -Output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, sr_item_sk#111, sr_ticket_number#112] - -(116) CometColumnarExchange -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: hashpartitioning(ss_item_sk#98, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(117) CometSort -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98 ASC NULLS FIRST] - -(118) ReusedExchange [Reuses operator id: 26] -Output [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] - -(119) CometHashAggregate -Input [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] -Keys [1]: [cs_item_sk#114] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#118)), sum(((cr_refunded_cash#119 + cr_reversed_charge#120) + cr_store_credit#121))] - -(120) CometFilter -Input [3]: [cs_item_sk#114, sale#122, refund#123] -Condition : ((isnotnull(sale#122) AND isnotnull(refund#123)) AND (cast(sale#122 as decimal(21,2)) > (2 * refund#123))) - -(121) CometProject -Input [3]: [cs_item_sk#114, sale#122, refund#123] -Arguments: [cs_item_sk#114], [cs_item_sk#114] - -(122) CometSort -Input [1]: [cs_item_sk#114] -Arguments: [cs_item_sk#114], [cs_item_sk#114 ASC NULLS FIRST] - -(123) CometSortMergeJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [1]: [cs_item_sk#114] -Arguments: [ss_item_sk#98], [cs_item_sk#114], Inner - -(124) CometProject -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, cs_item_sk#114] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(125) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#124, d_year#125] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#124, d_year#125] -Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) - -(127) CometBroadcastExchange -Input [2]: [d_date_sk#124, d_year#125] -Arguments: [d_date_sk#124, d_year#125] - -(128) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [d_date_sk#124, d_year#125] -Arguments: [ss_sold_date_sk#109], [d_date_sk#124], Inner, BuildRight - -(129) CometProject -Input [13]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, d_date_sk#124, d_year#125] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125] - -(130) ReusedExchange [Reuses operator id: 41] -Output [3]: [s_store_sk#126, s_store_name#127, s_zip#128] - -(131) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125] -Right output [3]: [s_store_sk#126, s_store_name#127, s_zip#128] -Arguments: [ss_store_sk#103], [s_store_sk#126], Inner, BuildRight - -(132) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_sk#126, s_store_name#127, s_zip#128] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128] - -(133) ReusedExchange [Reuses operator id: 46] -Output [6]: [c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] - -(134) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128] -Right output [6]: [c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] -Arguments: [ss_customer_sk#99], [c_customer_sk#129], Inner, BuildRight - -(135) CometProject -Input [18]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] - -(136) ReusedExchange [Reuses operator id: 51] -Output [2]: [d_date_sk#135, d_year#136] - -(137) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] -Right output [2]: [d_date_sk#135, d_year#136] -Arguments: [c_first_sales_date_sk#134], [d_date_sk#135], Inner, BuildRight - -(138) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134, d_date_sk#135, d_year#136] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136] - -(139) ReusedExchange [Reuses operator id: 51] -Output [2]: [d_date_sk#137, d_year#138] - -(140) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136] -Right output [2]: [d_date_sk#137, d_year#138] -Arguments: [c_first_shipto_date_sk#133], [d_date_sk#137], Inner, BuildRight - -(141) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136, d_date_sk#137, d_year#138] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] - -(142) ReusedExchange [Reuses operator id: 60] -Output [2]: [cd_demo_sk#139, cd_marital_status#140] - -(143) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] -Right output [2]: [cd_demo_sk#139, cd_marital_status#140] -Arguments: [ss_cdemo_sk#100], [cd_demo_sk#139], Inner, BuildRight - -(144) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_demo_sk#139, cd_marital_status#140] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140] - -(145) ReusedExchange [Reuses operator id: 60] -Output [2]: [cd_demo_sk#141, cd_marital_status#142] - -(146) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140] -Right output [2]: [cd_demo_sk#141, cd_marital_status#142] -Arguments: [c_current_cdemo_sk#130], [cd_demo_sk#141], Inner, NOT (cd_marital_status#140 = cd_marital_status#142), BuildRight - -(147) CometProject -Input [18]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140, cd_demo_sk#141, cd_marital_status#142] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] - -(148) ReusedExchange [Reuses operator id: 68] -Output [1]: [p_promo_sk#143] - -(149) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] -Right output [1]: [p_promo_sk#143] -Arguments: [ss_promo_sk#104], [p_promo_sk#143], Inner, BuildRight - -(150) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, p_promo_sk#143] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] - -(151) ReusedExchange [Reuses operator id: 73] -Output [2]: [hd_demo_sk#144, hd_income_band_sk#145] - -(152) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] -Right output [2]: [hd_demo_sk#144, hd_income_band_sk#145] -Arguments: [ss_hdemo_sk#101], [hd_demo_sk#144], Inner, BuildRight - -(153) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_demo_sk#144, hd_income_band_sk#145] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145] - -(154) ReusedExchange [Reuses operator id: 73] -Output [2]: [hd_demo_sk#146, hd_income_band_sk#147] - -(155) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145] -Right output [2]: [hd_demo_sk#146, hd_income_band_sk#147] -Arguments: [c_current_hdemo_sk#131], [hd_demo_sk#146], Inner, BuildRight - -(156) CometProject -Input [15]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_demo_sk#146, hd_income_band_sk#147] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147] - -(157) ReusedExchange [Reuses operator id: 82] -Output [5]: [ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] - -(158) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147] -Right output [5]: [ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] -Arguments: [ss_addr_sk#102], [ca_address_sk#148], Inner, BuildRight - -(159) CometProject -Input [18]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] - -(160) ReusedExchange [Reuses operator id: 82] -Output [5]: [ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(161) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] -Right output [5]: [ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Arguments: [c_current_addr_sk#132], [ca_address_sk#153], Inner, BuildRight - -(162) CometProject -Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(163) ReusedExchange [Reuses operator id: 90] -Output [1]: [ib_income_band_sk#158] - -(164) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Right output [1]: [ib_income_band_sk#158] -Arguments: [hd_income_band_sk#145], [ib_income_band_sk#158], Inner, BuildRight - -(165) CometProject -Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, ib_income_band_sk#158] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(166) ReusedExchange [Reuses operator id: 90] -Output [1]: [ib_income_band_sk#159] - -(167) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Right output [1]: [ib_income_band_sk#159] -Arguments: [hd_income_band_sk#147], [ib_income_band_sk#159], Inner, BuildRight - -(168) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, ib_income_band_sk#159] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(169) ReusedExchange [Reuses operator id: 99] -Output [2]: [i_item_sk#160, i_product_name#161] - -(170) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Right output [2]: [i_item_sk#160, i_product_name#161] -Arguments: [ss_item_sk#98], [i_item_sk#160], Inner, BuildRight - -(171) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] -Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] - -(172) CometHashAggregate -Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] -Keys [15]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#106)), partial_sum(UnscaledValue(ss_list_price#107)), partial_sum(UnscaledValue(ss_coupon_amt#108))] - -(173) CometHashAggregate -Input [19]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138, count#77, sum#162, sum#163, sum#164] -Keys [15]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#106)), sum(UnscaledValue(ss_list_price#107)), sum(UnscaledValue(ss_coupon_amt#108))] - -(174) CometExchange -Input [8]: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: hashpartitioning(item_sk#165, store_name#166, store_zip#167, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(175) CometSort -Input [8]: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172], [item_sk#165 ASC NULLS FIRST, store_name#166 ASC NULLS FIRST, store_zip#167 ASC NULLS FIRST] - -(176) CometSortMergeJoin -Left output [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Right output [8]: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#165, store_name#166, store_zip#167], Inner, (cnt#169 <= cnt#94) - -(177) CometProject -Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] - -(178) CometExchange -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#169 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(179) CometSort -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#169 ASC NULLS FIRST] - -(180) CometColumnarToRow [codegen id : 5] -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (184) -+- * CometColumnarToRow (183) - +- CometFilter (182) - +- CometNativeScan parquet spark_catalog.default.date_dim (181) - - -(181) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(182) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(183) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#33] - -(184) BroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] - -Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 -BroadcastExchange (188) -+- * CometColumnarToRow (187) - +- CometFilter (186) - +- CometNativeScan parquet spark_catalog.default.date_dim (185) - - -(185) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#124, d_year#125] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(186) CometFilter -Input [2]: [d_date_sk#124, d_year#125] -Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) - -(187) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#124, d_year#125] - -(188) BroadcastExchange -Input [2]: [d_date_sk#124, d_year#125] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/simplified.txt deleted file mode 100644 index f954616bb5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/simplified.txt +++ /dev/null @@ -1,206 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometExchange [product_name,store_name,cnt] #1 - CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometColumnarExchange [ss_item_sk] #3 - WholeStageCodegen (2) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometNativeScan 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] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [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] #11 - 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] - CometNativeScan 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] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk] - ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometColumnarExchange [ss_item_sk] #20 - WholeStageCodegen (4) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #21 - WholeStageCodegen (3) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet 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 #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - CometBroadcastExchange [d_date_sk,d_year] #23 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/explain.txt deleted file mode 100644 index 806525a4e2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,1011 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (176) -+- CometSort (175) - +- CometExchange (174) - +- CometProject (173) - +- CometSortMergeJoin (172) - :- CometSort (103) - : +- CometExchange (102) - : +- CometHashAggregate (101) - : +- CometHashAggregate (100) - : +- CometProject (99) - : +- CometBroadcastHashJoin (98) - : :- CometProject (93) - : : +- CometBroadcastHashJoin (92) - : : :- CometProject (90) - : : : +- CometBroadcastHashJoin (89) - : : : :- CometProject (85) - : : : : +- CometBroadcastHashJoin (84) - : : : : :- CometProject (82) - : : : : : +- CometBroadcastHashJoin (81) - : : : : : :- CometProject (76) - : : : : : : +- CometBroadcastHashJoin (75) - : : : : : : :- CometProject (73) - : : : : : : : +- CometBroadcastHashJoin (72) - : : : : : : : :- CometProject (68) - : : : : : : : : +- CometBroadcastHashJoin (67) - : : : : : : : : :- CometProject (63) - : : : : : : : : : +- CometBroadcastHashJoin (62) - : : : : : : : : : :- CometProject (60) - : : : : : : : : : : +- CometBroadcastHashJoin (59) - : : : : : : : : : : :- CometProject (54) - : : : : : : : : : : : +- CometBroadcastHashJoin (53) - : : : : : : : : : : : :- CometProject (51) - : : : : : : : : : : : : +- CometBroadcastHashJoin (50) - : : : : : : : : : : : : :- CometProject (46) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (45) - : : : : : : : : : : : : : :- CometProject (41) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (40) - : : : : : : : : : : : : : : :- CometProject (35) - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (34) - : : : : : : : : : : : : : : : :- CometProject (30) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) - : : : : : : : : : : : : : : : : :- CometSort (10) - : : : : : : : : : : : : : : : : : +- CometExchange (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- CometSort (28) - : : : : : : : : : : : : : : : : +- CometProject (27) - : : : : : : : : : : : : : : : : +- CometFilter (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometExchange (24) - : : : : : : : : : : : : : : : : +- CometHashAggregate (23) - : : : : : : : : : : : : : : : : +- CometProject (22) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) - : : : : : : : : : : : : : : : : :- CometSort (15) - : : : : : : : : : : : : : : : : : +- CometExchange (14) - : : : : : : : : : : : : : : : : : +- CometProject (13) - : : : : : : : : : : : : : : : : : +- CometFilter (12) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (11) - : : : : : : : : : : : : : : : : +- CometSort (20) - : : : : : : : : : : : : : : : : +- CometExchange (19) - : : : : : : : : : : : : : : : : +- CometProject (18) - : : : : : : : : : : : : : : : : +- CometFilter (17) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : : : : : : : : : : : : : : : +- CometBroadcastExchange (33) - : : : : : : : : : : : : : : : +- CometFilter (32) - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - : : : : : : : : : : : : : : +- CometBroadcastExchange (39) - : : : : : : : : : : : : : : +- CometProject (38) - : : : : : : : : : : : : : : +- CometFilter (37) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (36) - : : : : : : : : : : : : : +- CometBroadcastExchange (44) - : : : : : : : : : : : : : +- CometFilter (43) - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (42) - : : : : : : : : : : : : +- CometBroadcastExchange (49) - : : : : : : : : : : : : +- CometFilter (48) - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) - : : : : : : : : : : : +- ReusedExchange (52) - : : : : : : : : : : +- CometBroadcastExchange (58) - : : : : : : : : : : +- CometProject (57) - : : : : : : : : : : +- CometFilter (56) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (55) - : : : : : : : : : +- ReusedExchange (61) - : : : : : : : : +- CometBroadcastExchange (66) - : : : : : : : : +- CometFilter (65) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (64) - : : : : : : : +- CometBroadcastExchange (71) - : : : : : : : +- CometFilter (70) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (69) - : : : : : : +- ReusedExchange (74) - : : : : : +- CometBroadcastExchange (80) - : : : : : +- CometProject (79) - : : : : : +- CometFilter (78) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (77) - : : : : +- ReusedExchange (83) - : : : +- CometBroadcastExchange (88) - : : : +- CometFilter (87) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (86) - : : +- ReusedExchange (91) - : +- CometBroadcastExchange (97) - : +- CometProject (96) - : +- CometFilter (95) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (94) - +- CometSort (171) - +- CometExchange (170) - +- CometHashAggregate (169) - +- CometHashAggregate (168) - +- CometProject (167) - +- CometBroadcastHashJoin (166) - :- CometProject (164) - : +- CometBroadcastHashJoin (163) - : :- CometProject (161) - : : +- CometBroadcastHashJoin (160) - : : :- CometProject (158) - : : : +- CometBroadcastHashJoin (157) - : : : :- CometProject (155) - : : : : +- CometBroadcastHashJoin (154) - : : : : :- CometProject (152) - : : : : : +- CometBroadcastHashJoin (151) - : : : : : :- CometProject (149) - : : : : : : +- CometBroadcastHashJoin (148) - : : : : : : :- CometProject (146) - : : : : : : : +- CometBroadcastHashJoin (145) - : : : : : : : :- CometProject (143) - : : : : : : : : +- CometBroadcastHashJoin (142) - : : : : : : : : :- CometProject (140) - : : : : : : : : : +- CometBroadcastHashJoin (139) - : : : : : : : : : :- CometProject (137) - : : : : : : : : : : +- CometBroadcastHashJoin (136) - : : : : : : : : : : :- CometProject (134) - : : : : : : : : : : : +- CometBroadcastHashJoin (133) - : : : : : : : : : : : :- CometProject (131) - : : : : : : : : : : : : +- CometBroadcastHashJoin (130) - : : : : : : : : : : : : :- CometProject (128) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (127) - : : : : : : : : : : : : : :- CometProject (125) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (124) - : : : : : : : : : : : : : : :- CometProject (120) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (119) - : : : : : : : : : : : : : : : :- CometSort (113) - : : : : : : : : : : : : : : : : +- CometExchange (112) - : : : : : : : : : : : : : : : : +- CometProject (111) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (110) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (106) - : : : : : : : : : : : : : : : : : +- CometFilter (105) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (104) - : : : : : : : : : : : : : : : : +- CometProject (109) - : : : : : : : : : : : : : : : : +- CometFilter (108) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (107) - : : : : : : : : : : : : : : : +- CometSort (118) - : : : : : : : : : : : : : : : +- CometProject (117) - : : : : : : : : : : : : : : : +- CometFilter (116) - : : : : : : : : : : : : : : : +- CometHashAggregate (115) - : : : : : : : : : : : : : : : +- ReusedExchange (114) - : : : : : : : : : : : : : : +- CometBroadcastExchange (123) - : : : : : : : : : : : : : : +- CometFilter (122) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (121) - : : : : : : : : : : : : : +- ReusedExchange (126) - : : : : : : : : : : : : +- ReusedExchange (129) - : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : +- ReusedExchange (141) - : : : : : : : +- ReusedExchange (144) - : : : : : : +- ReusedExchange (147) - : : : : : +- ReusedExchange (150) - : : : : +- ReusedExchange (153) - : : : +- ReusedExchange (156) - : : +- ReusedExchange (159) - : +- ReusedExchange (162) - +- ReusedExchange (165) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) CometFilter -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) - -(3) CometBroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) - -(6) CometProject -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] - -(7) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner, BuildLeft - -(8) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(9) CometExchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometSort -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) - -(13) CometProject -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] - -(14) CometExchange -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(15) CometSort -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(17) CometFilter -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) - -(18) CometProject -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(19) CometExchange -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner - -(22) CometProject -Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(23) CometHashAggregate -Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Keys [1]: [cs_item_sk#17] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(24) CometExchange -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometHashAggregate -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Keys [1]: [cs_item_sk#17] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(26) CometFilter -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(27) CometProject -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Arguments: [cs_item_sk#17], [cs_item_sk#17] - -(28) CometSort -Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] - -(29) CometSortMergeJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [1]: [cs_item_sk#17] -Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner - -(30) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(33) CometBroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: [d_date_sk#32, d_year#33] - -(34) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight - -(35) CometProject -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#36, 10)))) - -(38) CometProject -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37], [s_store_sk#34, s_store_name#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#36, 10)) AS s_zip#37] - -(39) CometBroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37] - -(40) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight - -(41) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(43) CometFilter -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Condition : (((((isnotnull(c_customer_sk#38) AND isnotnull(c_first_sales_date_sk#43)) AND isnotnull(c_first_shipto_date_sk#42)) AND isnotnull(c_current_cdemo_sk#39)) AND isnotnull(c_current_hdemo_sk#40)) AND isnotnull(c_current_addr_sk#41)) - -(44) CometBroadcastExchange -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(45) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -Right output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_customer_sk#2], [c_customer_sk#38], Inner, BuildRight - -(46) CometProject -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(48) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : isnotnull(d_date_sk#44) - -(49) CometBroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] - -(50) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [c_first_sales_date_sk#43], [d_date_sk#44], Inner, BuildRight - -(51) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43, d_date_sk#44, d_year#45] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] - -(52) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#46, d_year#47] - -(53) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [c_first_shipto_date_sk#42], [d_date_sk#46], Inner, BuildRight - -(54) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45, d_date_sk#46, d_year#47] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#48, cd_marital_status#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#49, 1)))) - -(57) CometProject -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Arguments: [cd_demo_sk#48, cd_marital_status#50], [cd_demo_sk#48, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#49, 1)) AS cd_marital_status#50] - -(58) CometBroadcastExchange -Input [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [cd_demo_sk#48, cd_marital_status#50] - -(59) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_cdemo_sk#3], [cd_demo_sk#48], Inner, BuildRight - -(60) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] - -(61) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#51, cd_marital_status#52] - -(62) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -Right output [2]: [cd_demo_sk#51, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#39], [cd_demo_sk#51], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(63) CometProject -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50, cd_demo_sk#51, cd_marital_status#52] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(65) CometFilter -Input [1]: [p_promo_sk#53] -Condition : isnotnull(p_promo_sk#53) - -(66) CometBroadcastExchange -Input [1]: [p_promo_sk#53] -Arguments: [p_promo_sk#53] - -(67) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [1]: [p_promo_sk#53] -Arguments: [ss_promo_sk#7], [p_promo_sk#53], Inner, BuildRight - -(68) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, p_promo_sk#53] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(70) CometFilter -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Condition : (isnotnull(hd_demo_sk#54) AND isnotnull(hd_income_band_sk#55)) - -(71) CometBroadcastExchange -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [hd_demo_sk#54, hd_income_band_sk#55] - -(72) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_hdemo_sk#4], [hd_demo_sk#54], Inner, BuildRight - -(73) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] - -(74) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] - -(75) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -Right output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [c_current_hdemo_sk#40], [hd_demo_sk#56], Inner, BuildRight - -(76) CometProject -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(78) CometFilter -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Condition : isnotnull(ca_address_sk#58) - -(79) CometProject -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ca_address_sk#58, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_number#59, 10)) AS ca_street_number#63, ca_street_name#60, ca_city#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#62, 10)) AS ca_zip#64] - -(80) CometBroadcastExchange -Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(81) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -Right output [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_addr_sk#5], [ca_address_sk#58], Inner, BuildRight - -(82) CometProject -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(83) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(84) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Right output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [c_current_addr_sk#41], [ca_address_sk#65], Inner, BuildRight - -(85) CometProject -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(87) CometFilter -Input [1]: [ib_income_band_sk#70] -Condition : isnotnull(ib_income_band_sk#70) - -(88) CometBroadcastExchange -Input [1]: [ib_income_band_sk#70] -Arguments: [ib_income_band_sk#70] - -(89) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#70] -Arguments: [hd_income_band_sk#55], [ib_income_band_sk#70], Inner, BuildRight - -(90) CometProject -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#70] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(91) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#71] - -(92) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#71] -Arguments: [hd_income_band_sk#57], [ib_income_band_sk#71], Inner, BuildRight - -(93) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#71] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(95) CometFilter -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Condition : ((((((isnotnull(i_current_price#73) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#74, 20)) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#73 >= 64.00)) AND (i_current_price#73 <= 74.00)) AND (i_current_price#73 >= 65.00)) AND (i_current_price#73 <= 79.00)) AND isnotnull(i_item_sk#72)) - -(96) CometProject -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Arguments: [i_item_sk#72, i_product_name#76], [i_item_sk#72, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#75, 50)) AS i_product_name#76] - -(97) CometBroadcastExchange -Input [2]: [i_item_sk#72, i_product_name#76] -Arguments: [i_item_sk#72, i_product_name#76] - -(98) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [2]: [i_item_sk#72, i_product_name#76] -Arguments: [ss_item_sk#1], [i_item_sk#72], Inner, BuildRight - -(99) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] - -(100) CometHashAggregate -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] - -(101) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#77, sum#78, sum#79, sum#80] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] - -(102) CometExchange -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: hashpartitioning(item_sk#82, store_name#83, store_zip#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(103) CometSort -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97], [item_sk#82 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, store_zip#84 ASC NULLS FIRST] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#109), dynamicpruningexpression(ss_sold_date_sk#109 IN dynamicpruning#110)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(105) CometFilter -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Condition : (((((((isnotnull(ss_item_sk#98) AND isnotnull(ss_ticket_number#105)) AND isnotnull(ss_store_sk#103)) AND isnotnull(ss_customer_sk#99)) AND isnotnull(ss_cdemo_sk#100)) AND isnotnull(ss_promo_sk#104)) AND isnotnull(ss_hdemo_sk#101)) AND isnotnull(ss_addr_sk#102)) - -(106) CometBroadcastExchange -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(108) CometFilter -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) - -(109) CometProject -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] - -(110) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_ticket_number#105], [sr_item_sk#111, sr_ticket_number#112], Inner, BuildLeft - -(111) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(112) CometExchange -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: hashpartitioning(ss_item_sk#98, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(113) CometSort -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98 ASC NULLS FIRST] - -(114) ReusedExchange [Reuses operator id: 24] -Output [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] - -(115) CometHashAggregate -Input [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] -Keys [1]: [cs_item_sk#114] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#118)), sum(((cr_refunded_cash#119 + cr_reversed_charge#120) + cr_store_credit#121))] - -(116) CometFilter -Input [3]: [cs_item_sk#114, sale#122, refund#123] -Condition : ((isnotnull(sale#122) AND isnotnull(refund#123)) AND (cast(sale#122 as decimal(21,2)) > (2 * refund#123))) - -(117) CometProject -Input [3]: [cs_item_sk#114, sale#122, refund#123] -Arguments: [cs_item_sk#114], [cs_item_sk#114] - -(118) CometSort -Input [1]: [cs_item_sk#114] -Arguments: [cs_item_sk#114], [cs_item_sk#114 ASC NULLS FIRST] - -(119) CometSortMergeJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [1]: [cs_item_sk#114] -Arguments: [ss_item_sk#98], [cs_item_sk#114], Inner - -(120) CometProject -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, cs_item_sk#114] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(121) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#124, d_year#125] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(122) CometFilter -Input [2]: [d_date_sk#124, d_year#125] -Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) - -(123) CometBroadcastExchange -Input [2]: [d_date_sk#124, d_year#125] -Arguments: [d_date_sk#124, d_year#125] - -(124) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [d_date_sk#124, d_year#125] -Arguments: [ss_sold_date_sk#109], [d_date_sk#124], Inner, BuildRight - -(125) CometProject -Input [13]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, d_date_sk#124, d_year#125] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125] - -(126) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_store_sk#126, s_store_name#127, s_zip#128] - -(127) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125] -Right output [3]: [s_store_sk#126, s_store_name#127, s_zip#128] -Arguments: [ss_store_sk#103], [s_store_sk#126], Inner, BuildRight - -(128) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_sk#126, s_store_name#127, s_zip#128] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128] - -(129) ReusedExchange [Reuses operator id: 44] -Output [6]: [c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] - -(130) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128] -Right output [6]: [c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] -Arguments: [ss_customer_sk#99], [c_customer_sk#129], Inner, BuildRight - -(131) CometProject -Input [18]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] - -(132) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#135, d_year#136] - -(133) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] -Right output [2]: [d_date_sk#135, d_year#136] -Arguments: [c_first_sales_date_sk#134], [d_date_sk#135], Inner, BuildRight - -(134) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134, d_date_sk#135, d_year#136] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136] - -(135) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#137, d_year#138] - -(136) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136] -Right output [2]: [d_date_sk#137, d_year#138] -Arguments: [c_first_shipto_date_sk#133], [d_date_sk#137], Inner, BuildRight - -(137) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136, d_date_sk#137, d_year#138] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] - -(138) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#139, cd_marital_status#140] - -(139) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] -Right output [2]: [cd_demo_sk#139, cd_marital_status#140] -Arguments: [ss_cdemo_sk#100], [cd_demo_sk#139], Inner, BuildRight - -(140) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_demo_sk#139, cd_marital_status#140] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140] - -(141) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#141, cd_marital_status#142] - -(142) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140] -Right output [2]: [cd_demo_sk#141, cd_marital_status#142] -Arguments: [c_current_cdemo_sk#130], [cd_demo_sk#141], Inner, NOT (cd_marital_status#140 = cd_marital_status#142), BuildRight - -(143) CometProject -Input [18]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140, cd_demo_sk#141, cd_marital_status#142] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] - -(144) ReusedExchange [Reuses operator id: 66] -Output [1]: [p_promo_sk#143] - -(145) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] -Right output [1]: [p_promo_sk#143] -Arguments: [ss_promo_sk#104], [p_promo_sk#143], Inner, BuildRight - -(146) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, p_promo_sk#143] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] - -(147) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#144, hd_income_band_sk#145] - -(148) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] -Right output [2]: [hd_demo_sk#144, hd_income_band_sk#145] -Arguments: [ss_hdemo_sk#101], [hd_demo_sk#144], Inner, BuildRight - -(149) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_demo_sk#144, hd_income_band_sk#145] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145] - -(150) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#146, hd_income_band_sk#147] - -(151) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145] -Right output [2]: [hd_demo_sk#146, hd_income_band_sk#147] -Arguments: [c_current_hdemo_sk#131], [hd_demo_sk#146], Inner, BuildRight - -(152) CometProject -Input [15]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_demo_sk#146, hd_income_band_sk#147] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147] - -(153) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] - -(154) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147] -Right output [5]: [ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] -Arguments: [ss_addr_sk#102], [ca_address_sk#148], Inner, BuildRight - -(155) CometProject -Input [18]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] - -(156) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(157) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] -Right output [5]: [ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Arguments: [c_current_addr_sk#132], [ca_address_sk#153], Inner, BuildRight - -(158) CometProject -Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(159) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#158] - -(160) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Right output [1]: [ib_income_band_sk#158] -Arguments: [hd_income_band_sk#145], [ib_income_band_sk#158], Inner, BuildRight - -(161) CometProject -Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, ib_income_band_sk#158] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(162) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#159] - -(163) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Right output [1]: [ib_income_band_sk#159] -Arguments: [hd_income_band_sk#147], [ib_income_band_sk#159], Inner, BuildRight - -(164) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, ib_income_band_sk#159] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(165) ReusedExchange [Reuses operator id: 97] -Output [2]: [i_item_sk#160, i_product_name#161] - -(166) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Right output [2]: [i_item_sk#160, i_product_name#161] -Arguments: [ss_item_sk#98], [i_item_sk#160], Inner, BuildRight - -(167) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] -Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] - -(168) CometHashAggregate -Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] -Keys [15]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#106)), partial_sum(UnscaledValue(ss_list_price#107)), partial_sum(UnscaledValue(ss_coupon_amt#108))] - -(169) CometHashAggregate -Input [19]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138, count#77, sum#162, sum#163, sum#164] -Keys [15]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#106)), sum(UnscaledValue(ss_list_price#107)), sum(UnscaledValue(ss_coupon_amt#108))] - -(170) CometExchange -Input [8]: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: hashpartitioning(item_sk#165, store_name#166, store_zip#167, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(171) CometSort -Input [8]: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172], [item_sk#165 ASC NULLS FIRST, store_name#166 ASC NULLS FIRST, store_zip#167 ASC NULLS FIRST] - -(172) CometSortMergeJoin -Left output [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Right output [8]: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#165, store_name#166, store_zip#167], Inner, (cnt#169 <= cnt#94) - -(173) CometProject -Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] - -(174) CometExchange -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#169 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(175) CometSort -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#169 ASC NULLS FIRST] - -(176) CometColumnarToRow [codegen id : 1] -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (180) -+- * CometColumnarToRow (179) - +- CometFilter (178) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) - - -(177) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(178) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(179) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#33] - -(180) BroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 -BroadcastExchange (184) -+- * CometColumnarToRow (183) - +- CometFilter (182) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) - - -(181) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#124, d_year#125] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(182) CometFilter -Input [2]: [d_date_sk#124, d_year#125] -Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) - -(183) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#124, d_year#125] - -(184) BroadcastExchange -Input [2]: [d_date_sk#124, d_year#125] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/simplified.txt deleted file mode 100644 index 43527978d8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,192 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometExchange [product_name,store_name,cnt] #1 - CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #3 - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [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] #11 - 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 [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk] - ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #20 - 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_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] #21 - 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 [native_iceberg_compat] 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 #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - CometBroadcastExchange [d_date_sk,d_year] #23 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt deleted file mode 100644 index 806525a4e2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt +++ /dev/null @@ -1,1011 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (176) -+- CometSort (175) - +- CometExchange (174) - +- CometProject (173) - +- CometSortMergeJoin (172) - :- CometSort (103) - : +- CometExchange (102) - : +- CometHashAggregate (101) - : +- CometHashAggregate (100) - : +- CometProject (99) - : +- CometBroadcastHashJoin (98) - : :- CometProject (93) - : : +- CometBroadcastHashJoin (92) - : : :- CometProject (90) - : : : +- CometBroadcastHashJoin (89) - : : : :- CometProject (85) - : : : : +- CometBroadcastHashJoin (84) - : : : : :- CometProject (82) - : : : : : +- CometBroadcastHashJoin (81) - : : : : : :- CometProject (76) - : : : : : : +- CometBroadcastHashJoin (75) - : : : : : : :- CometProject (73) - : : : : : : : +- CometBroadcastHashJoin (72) - : : : : : : : :- CometProject (68) - : : : : : : : : +- CometBroadcastHashJoin (67) - : : : : : : : : :- CometProject (63) - : : : : : : : : : +- CometBroadcastHashJoin (62) - : : : : : : : : : :- CometProject (60) - : : : : : : : : : : +- CometBroadcastHashJoin (59) - : : : : : : : : : : :- CometProject (54) - : : : : : : : : : : : +- CometBroadcastHashJoin (53) - : : : : : : : : : : : :- CometProject (51) - : : : : : : : : : : : : +- CometBroadcastHashJoin (50) - : : : : : : : : : : : : :- CometProject (46) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (45) - : : : : : : : : : : : : : :- CometProject (41) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (40) - : : : : : : : : : : : : : : :- CometProject (35) - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (34) - : : : : : : : : : : : : : : : :- CometProject (30) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) - : : : : : : : : : : : : : : : : :- CometSort (10) - : : : : : : : : : : : : : : : : : +- CometExchange (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- CometSort (28) - : : : : : : : : : : : : : : : : +- CometProject (27) - : : : : : : : : : : : : : : : : +- CometFilter (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometExchange (24) - : : : : : : : : : : : : : : : : +- CometHashAggregate (23) - : : : : : : : : : : : : : : : : +- CometProject (22) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) - : : : : : : : : : : : : : : : : :- CometSort (15) - : : : : : : : : : : : : : : : : : +- CometExchange (14) - : : : : : : : : : : : : : : : : : +- CometProject (13) - : : : : : : : : : : : : : : : : : +- CometFilter (12) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (11) - : : : : : : : : : : : : : : : : +- CometSort (20) - : : : : : : : : : : : : : : : : +- CometExchange (19) - : : : : : : : : : : : : : : : : +- CometProject (18) - : : : : : : : : : : : : : : : : +- CometFilter (17) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : : : : : : : : : : : : : : : +- CometBroadcastExchange (33) - : : : : : : : : : : : : : : : +- CometFilter (32) - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - : : : : : : : : : : : : : : +- CometBroadcastExchange (39) - : : : : : : : : : : : : : : +- CometProject (38) - : : : : : : : : : : : : : : +- CometFilter (37) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (36) - : : : : : : : : : : : : : +- CometBroadcastExchange (44) - : : : : : : : : : : : : : +- CometFilter (43) - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (42) - : : : : : : : : : : : : +- CometBroadcastExchange (49) - : : : : : : : : : : : : +- CometFilter (48) - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) - : : : : : : : : : : : +- ReusedExchange (52) - : : : : : : : : : : +- CometBroadcastExchange (58) - : : : : : : : : : : +- CometProject (57) - : : : : : : : : : : +- CometFilter (56) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (55) - : : : : : : : : : +- ReusedExchange (61) - : : : : : : : : +- CometBroadcastExchange (66) - : : : : : : : : +- CometFilter (65) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (64) - : : : : : : : +- CometBroadcastExchange (71) - : : : : : : : +- CometFilter (70) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (69) - : : : : : : +- ReusedExchange (74) - : : : : : +- CometBroadcastExchange (80) - : : : : : +- CometProject (79) - : : : : : +- CometFilter (78) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (77) - : : : : +- ReusedExchange (83) - : : : +- CometBroadcastExchange (88) - : : : +- CometFilter (87) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (86) - : : +- ReusedExchange (91) - : +- CometBroadcastExchange (97) - : +- CometProject (96) - : +- CometFilter (95) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (94) - +- CometSort (171) - +- CometExchange (170) - +- CometHashAggregate (169) - +- CometHashAggregate (168) - +- CometProject (167) - +- CometBroadcastHashJoin (166) - :- CometProject (164) - : +- CometBroadcastHashJoin (163) - : :- CometProject (161) - : : +- CometBroadcastHashJoin (160) - : : :- CometProject (158) - : : : +- CometBroadcastHashJoin (157) - : : : :- CometProject (155) - : : : : +- CometBroadcastHashJoin (154) - : : : : :- CometProject (152) - : : : : : +- CometBroadcastHashJoin (151) - : : : : : :- CometProject (149) - : : : : : : +- CometBroadcastHashJoin (148) - : : : : : : :- CometProject (146) - : : : : : : : +- CometBroadcastHashJoin (145) - : : : : : : : :- CometProject (143) - : : : : : : : : +- CometBroadcastHashJoin (142) - : : : : : : : : :- CometProject (140) - : : : : : : : : : +- CometBroadcastHashJoin (139) - : : : : : : : : : :- CometProject (137) - : : : : : : : : : : +- CometBroadcastHashJoin (136) - : : : : : : : : : : :- CometProject (134) - : : : : : : : : : : : +- CometBroadcastHashJoin (133) - : : : : : : : : : : : :- CometProject (131) - : : : : : : : : : : : : +- CometBroadcastHashJoin (130) - : : : : : : : : : : : : :- CometProject (128) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (127) - : : : : : : : : : : : : : :- CometProject (125) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (124) - : : : : : : : : : : : : : : :- CometProject (120) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (119) - : : : : : : : : : : : : : : : :- CometSort (113) - : : : : : : : : : : : : : : : : +- CometExchange (112) - : : : : : : : : : : : : : : : : +- CometProject (111) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (110) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (106) - : : : : : : : : : : : : : : : : : +- CometFilter (105) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (104) - : : : : : : : : : : : : : : : : +- CometProject (109) - : : : : : : : : : : : : : : : : +- CometFilter (108) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (107) - : : : : : : : : : : : : : : : +- CometSort (118) - : : : : : : : : : : : : : : : +- CometProject (117) - : : : : : : : : : : : : : : : +- CometFilter (116) - : : : : : : : : : : : : : : : +- CometHashAggregate (115) - : : : : : : : : : : : : : : : +- ReusedExchange (114) - : : : : : : : : : : : : : : +- CometBroadcastExchange (123) - : : : : : : : : : : : : : : +- CometFilter (122) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (121) - : : : : : : : : : : : : : +- ReusedExchange (126) - : : : : : : : : : : : : +- ReusedExchange (129) - : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : +- ReusedExchange (141) - : : : : : : : +- ReusedExchange (144) - : : : : : : +- ReusedExchange (147) - : : : : : +- ReusedExchange (150) - : : : : +- ReusedExchange (153) - : : : +- ReusedExchange (156) - : : +- ReusedExchange (159) - : +- ReusedExchange (162) - +- ReusedExchange (165) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) CometFilter -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) - -(3) CometBroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) - -(6) CometProject -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] - -(7) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner, BuildLeft - -(8) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(9) CometExchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometSort -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) - -(13) CometProject -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] - -(14) CometExchange -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(15) CometSort -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(17) CometFilter -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) - -(18) CometProject -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(19) CometExchange -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner - -(22) CometProject -Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(23) CometHashAggregate -Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Keys [1]: [cs_item_sk#17] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(24) CometExchange -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometHashAggregate -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Keys [1]: [cs_item_sk#17] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(26) CometFilter -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(27) CometProject -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Arguments: [cs_item_sk#17], [cs_item_sk#17] - -(28) CometSort -Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] - -(29) CometSortMergeJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [1]: [cs_item_sk#17] -Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner - -(30) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(33) CometBroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: [d_date_sk#32, d_year#33] - -(34) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight - -(35) CometProject -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#36, 10)))) - -(38) CometProject -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37], [s_store_sk#34, s_store_name#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#36, 10)) AS s_zip#37] - -(39) CometBroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37] - -(40) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight - -(41) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(43) CometFilter -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Condition : (((((isnotnull(c_customer_sk#38) AND isnotnull(c_first_sales_date_sk#43)) AND isnotnull(c_first_shipto_date_sk#42)) AND isnotnull(c_current_cdemo_sk#39)) AND isnotnull(c_current_hdemo_sk#40)) AND isnotnull(c_current_addr_sk#41)) - -(44) CometBroadcastExchange -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(45) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -Right output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_customer_sk#2], [c_customer_sk#38], Inner, BuildRight - -(46) CometProject -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(48) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : isnotnull(d_date_sk#44) - -(49) CometBroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] - -(50) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [c_first_sales_date_sk#43], [d_date_sk#44], Inner, BuildRight - -(51) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43, d_date_sk#44, d_year#45] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] - -(52) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#46, d_year#47] - -(53) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [c_first_shipto_date_sk#42], [d_date_sk#46], Inner, BuildRight - -(54) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45, d_date_sk#46, d_year#47] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#48, cd_marital_status#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#49, 1)))) - -(57) CometProject -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Arguments: [cd_demo_sk#48, cd_marital_status#50], [cd_demo_sk#48, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#49, 1)) AS cd_marital_status#50] - -(58) CometBroadcastExchange -Input [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [cd_demo_sk#48, cd_marital_status#50] - -(59) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_cdemo_sk#3], [cd_demo_sk#48], Inner, BuildRight - -(60) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] - -(61) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#51, cd_marital_status#52] - -(62) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -Right output [2]: [cd_demo_sk#51, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#39], [cd_demo_sk#51], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(63) CometProject -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50, cd_demo_sk#51, cd_marital_status#52] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(65) CometFilter -Input [1]: [p_promo_sk#53] -Condition : isnotnull(p_promo_sk#53) - -(66) CometBroadcastExchange -Input [1]: [p_promo_sk#53] -Arguments: [p_promo_sk#53] - -(67) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [1]: [p_promo_sk#53] -Arguments: [ss_promo_sk#7], [p_promo_sk#53], Inner, BuildRight - -(68) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, p_promo_sk#53] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(70) CometFilter -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Condition : (isnotnull(hd_demo_sk#54) AND isnotnull(hd_income_band_sk#55)) - -(71) CometBroadcastExchange -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [hd_demo_sk#54, hd_income_band_sk#55] - -(72) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_hdemo_sk#4], [hd_demo_sk#54], Inner, BuildRight - -(73) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] - -(74) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] - -(75) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -Right output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [c_current_hdemo_sk#40], [hd_demo_sk#56], Inner, BuildRight - -(76) CometProject -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(78) CometFilter -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Condition : isnotnull(ca_address_sk#58) - -(79) CometProject -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ca_address_sk#58, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_number#59, 10)) AS ca_street_number#63, ca_street_name#60, ca_city#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#62, 10)) AS ca_zip#64] - -(80) CometBroadcastExchange -Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(81) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -Right output [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_addr_sk#5], [ca_address_sk#58], Inner, BuildRight - -(82) CometProject -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(83) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(84) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Right output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [c_current_addr_sk#41], [ca_address_sk#65], Inner, BuildRight - -(85) CometProject -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(87) CometFilter -Input [1]: [ib_income_band_sk#70] -Condition : isnotnull(ib_income_band_sk#70) - -(88) CometBroadcastExchange -Input [1]: [ib_income_band_sk#70] -Arguments: [ib_income_band_sk#70] - -(89) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#70] -Arguments: [hd_income_band_sk#55], [ib_income_band_sk#70], Inner, BuildRight - -(90) CometProject -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#70] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(91) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#71] - -(92) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#71] -Arguments: [hd_income_band_sk#57], [ib_income_band_sk#71], Inner, BuildRight - -(93) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#71] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(95) CometFilter -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Condition : ((((((isnotnull(i_current_price#73) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#74, 20)) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#73 >= 64.00)) AND (i_current_price#73 <= 74.00)) AND (i_current_price#73 >= 65.00)) AND (i_current_price#73 <= 79.00)) AND isnotnull(i_item_sk#72)) - -(96) CometProject -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Arguments: [i_item_sk#72, i_product_name#76], [i_item_sk#72, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#75, 50)) AS i_product_name#76] - -(97) CometBroadcastExchange -Input [2]: [i_item_sk#72, i_product_name#76] -Arguments: [i_item_sk#72, i_product_name#76] - -(98) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [2]: [i_item_sk#72, i_product_name#76] -Arguments: [ss_item_sk#1], [i_item_sk#72], Inner, BuildRight - -(99) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] - -(100) CometHashAggregate -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] - -(101) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#77, sum#78, sum#79, sum#80] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] - -(102) CometExchange -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: hashpartitioning(item_sk#82, store_name#83, store_zip#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(103) CometSort -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97], [item_sk#82 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, store_zip#84 ASC NULLS FIRST] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#109), dynamicpruningexpression(ss_sold_date_sk#109 IN dynamicpruning#110)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(105) CometFilter -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Condition : (((((((isnotnull(ss_item_sk#98) AND isnotnull(ss_ticket_number#105)) AND isnotnull(ss_store_sk#103)) AND isnotnull(ss_customer_sk#99)) AND isnotnull(ss_cdemo_sk#100)) AND isnotnull(ss_promo_sk#104)) AND isnotnull(ss_hdemo_sk#101)) AND isnotnull(ss_addr_sk#102)) - -(106) CometBroadcastExchange -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(108) CometFilter -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) - -(109) CometProject -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] - -(110) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_ticket_number#105], [sr_item_sk#111, sr_ticket_number#112], Inner, BuildLeft - -(111) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(112) CometExchange -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: hashpartitioning(ss_item_sk#98, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(113) CometSort -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98 ASC NULLS FIRST] - -(114) ReusedExchange [Reuses operator id: 24] -Output [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] - -(115) CometHashAggregate -Input [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] -Keys [1]: [cs_item_sk#114] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#118)), sum(((cr_refunded_cash#119 + cr_reversed_charge#120) + cr_store_credit#121))] - -(116) CometFilter -Input [3]: [cs_item_sk#114, sale#122, refund#123] -Condition : ((isnotnull(sale#122) AND isnotnull(refund#123)) AND (cast(sale#122 as decimal(21,2)) > (2 * refund#123))) - -(117) CometProject -Input [3]: [cs_item_sk#114, sale#122, refund#123] -Arguments: [cs_item_sk#114], [cs_item_sk#114] - -(118) CometSort -Input [1]: [cs_item_sk#114] -Arguments: [cs_item_sk#114], [cs_item_sk#114 ASC NULLS FIRST] - -(119) CometSortMergeJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [1]: [cs_item_sk#114] -Arguments: [ss_item_sk#98], [cs_item_sk#114], Inner - -(120) CometProject -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, cs_item_sk#114] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(121) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#124, d_year#125] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(122) CometFilter -Input [2]: [d_date_sk#124, d_year#125] -Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) - -(123) CometBroadcastExchange -Input [2]: [d_date_sk#124, d_year#125] -Arguments: [d_date_sk#124, d_year#125] - -(124) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [d_date_sk#124, d_year#125] -Arguments: [ss_sold_date_sk#109], [d_date_sk#124], Inner, BuildRight - -(125) CometProject -Input [13]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, d_date_sk#124, d_year#125] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125] - -(126) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_store_sk#126, s_store_name#127, s_zip#128] - -(127) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125] -Right output [3]: [s_store_sk#126, s_store_name#127, s_zip#128] -Arguments: [ss_store_sk#103], [s_store_sk#126], Inner, BuildRight - -(128) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_sk#126, s_store_name#127, s_zip#128] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128] - -(129) ReusedExchange [Reuses operator id: 44] -Output [6]: [c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] - -(130) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128] -Right output [6]: [c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] -Arguments: [ss_customer_sk#99], [c_customer_sk#129], Inner, BuildRight - -(131) CometProject -Input [18]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] - -(132) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#135, d_year#136] - -(133) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] -Right output [2]: [d_date_sk#135, d_year#136] -Arguments: [c_first_sales_date_sk#134], [d_date_sk#135], Inner, BuildRight - -(134) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134, d_date_sk#135, d_year#136] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136] - -(135) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#137, d_year#138] - -(136) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136] -Right output [2]: [d_date_sk#137, d_year#138] -Arguments: [c_first_shipto_date_sk#133], [d_date_sk#137], Inner, BuildRight - -(137) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136, d_date_sk#137, d_year#138] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] - -(138) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#139, cd_marital_status#140] - -(139) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] -Right output [2]: [cd_demo_sk#139, cd_marital_status#140] -Arguments: [ss_cdemo_sk#100], [cd_demo_sk#139], Inner, BuildRight - -(140) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_demo_sk#139, cd_marital_status#140] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140] - -(141) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#141, cd_marital_status#142] - -(142) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140] -Right output [2]: [cd_demo_sk#141, cd_marital_status#142] -Arguments: [c_current_cdemo_sk#130], [cd_demo_sk#141], Inner, NOT (cd_marital_status#140 = cd_marital_status#142), BuildRight - -(143) CometProject -Input [18]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140, cd_demo_sk#141, cd_marital_status#142] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] - -(144) ReusedExchange [Reuses operator id: 66] -Output [1]: [p_promo_sk#143] - -(145) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] -Right output [1]: [p_promo_sk#143] -Arguments: [ss_promo_sk#104], [p_promo_sk#143], Inner, BuildRight - -(146) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, p_promo_sk#143] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] - -(147) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#144, hd_income_band_sk#145] - -(148) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] -Right output [2]: [hd_demo_sk#144, hd_income_band_sk#145] -Arguments: [ss_hdemo_sk#101], [hd_demo_sk#144], Inner, BuildRight - -(149) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_demo_sk#144, hd_income_band_sk#145] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145] - -(150) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#146, hd_income_band_sk#147] - -(151) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145] -Right output [2]: [hd_demo_sk#146, hd_income_band_sk#147] -Arguments: [c_current_hdemo_sk#131], [hd_demo_sk#146], Inner, BuildRight - -(152) CometProject -Input [15]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_demo_sk#146, hd_income_band_sk#147] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147] - -(153) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] - -(154) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147] -Right output [5]: [ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] -Arguments: [ss_addr_sk#102], [ca_address_sk#148], Inner, BuildRight - -(155) CometProject -Input [18]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] - -(156) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(157) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] -Right output [5]: [ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Arguments: [c_current_addr_sk#132], [ca_address_sk#153], Inner, BuildRight - -(158) CometProject -Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(159) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#158] - -(160) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Right output [1]: [ib_income_band_sk#158] -Arguments: [hd_income_band_sk#145], [ib_income_band_sk#158], Inner, BuildRight - -(161) CometProject -Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, ib_income_band_sk#158] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(162) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#159] - -(163) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Right output [1]: [ib_income_band_sk#159] -Arguments: [hd_income_band_sk#147], [ib_income_band_sk#159], Inner, BuildRight - -(164) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, ib_income_band_sk#159] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(165) ReusedExchange [Reuses operator id: 97] -Output [2]: [i_item_sk#160, i_product_name#161] - -(166) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Right output [2]: [i_item_sk#160, i_product_name#161] -Arguments: [ss_item_sk#98], [i_item_sk#160], Inner, BuildRight - -(167) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] -Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] - -(168) CometHashAggregate -Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] -Keys [15]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#106)), partial_sum(UnscaledValue(ss_list_price#107)), partial_sum(UnscaledValue(ss_coupon_amt#108))] - -(169) CometHashAggregate -Input [19]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138, count#77, sum#162, sum#163, sum#164] -Keys [15]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#106)), sum(UnscaledValue(ss_list_price#107)), sum(UnscaledValue(ss_coupon_amt#108))] - -(170) CometExchange -Input [8]: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: hashpartitioning(item_sk#165, store_name#166, store_zip#167, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(171) CometSort -Input [8]: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172], [item_sk#165 ASC NULLS FIRST, store_name#166 ASC NULLS FIRST, store_zip#167 ASC NULLS FIRST] - -(172) CometSortMergeJoin -Left output [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Right output [8]: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#165, store_name#166, store_zip#167], Inner, (cnt#169 <= cnt#94) - -(173) CometProject -Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] - -(174) CometExchange -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#169 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(175) CometSort -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#169 ASC NULLS FIRST] - -(176) CometColumnarToRow [codegen id : 1] -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (180) -+- * CometColumnarToRow (179) - +- CometFilter (178) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) - - -(177) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(178) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(179) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#33] - -(180) BroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 -BroadcastExchange (184) -+- * CometColumnarToRow (183) - +- CometFilter (182) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) - - -(181) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#124, d_year#125] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(182) CometFilter -Input [2]: [d_date_sk#124, d_year#125] -Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) - -(183) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#124, d_year#125] - -(184) BroadcastExchange -Input [2]: [d_date_sk#124, d_year#125] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/extended.txt deleted file mode 100644 index 059acab385..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/extended.txt +++ /dev/null @@ -1,247 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 43527978d8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt +++ /dev/null @@ -1,192 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometExchange [product_name,store_name,cnt] #1 - CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #3 - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [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] #11 - 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 [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk] - ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #20 - 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_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] #21 - 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 [native_iceberg_compat] 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 #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - CometBroadcastExchange [d_date_sk,d_year] #23 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/explain.txt deleted file mode 100644 index dcf67d7592..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/explain.txt +++ /dev/null @@ -1,286 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * CometColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.store (1) - : : +- BroadcastExchange (15) - : : +- * Filter (14) - : : +- * HashAggregate (13) - : : +- * CometColumnarToRow (12) - : : +- CometColumnarExchange (11) - : : +- * HashAggregate (10) - : : +- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : +- ReusedExchange (7) - : +- BroadcastExchange (22) - : +- * CometColumnarToRow (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometNativeScan parquet spark_catalog.default.item (18) - +- BroadcastExchange (40) - +- * Filter (39) - +- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * HashAggregate (34) - +- * CometColumnarToRow (33) - +- CometColumnarExchange (32) - +- * HashAggregate (31) - +- * Project (30) - +- * BroadcastHashJoin Inner BuildRight (29) - :- * Filter (27) - : +- * ColumnarToRow (26) - : +- Scan parquet spark_catalog.default.store_sales (25) - +- ReusedExchange (28) - - -(1) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#1, s_store_name#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [s_store_sk#1, s_store_name#2] -Condition : isnotnull(s_store_sk#1) - -(3) CometColumnarToRow [codegen id : 9] -Input [2]: [s_store_sk#1, s_store_name#2] - -(4) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] - -(6) Filter [codegen id : 2] -Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3)) - -(7) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#8] - -(8) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 2] -Output [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] -Input [5]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] - -(10) HashAggregate [codegen id : 2] -Input [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] -Keys [2]: [ss_store_sk#4, ss_item_sk#3] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#5))] -Aggregate Attributes [1]: [sum#9] -Results [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] - -(11) CometColumnarExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(12) CometColumnarToRow [codegen id : 3] -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] - -(13) HashAggregate [codegen id : 3] -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -Keys [2]: [ss_store_sk#4, ss_item_sk#3] -Functions [1]: [sum(UnscaledValue(ss_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#5))#11] -Results [3]: [ss_store_sk#4, ss_item_sk#3, MakeDecimal(sum(UnscaledValue(ss_sales_price#5))#11,17,2) AS revenue#12] - -(14) Filter [codegen id : 3] -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] -Condition : isnotnull(revenue#12) - -(15) BroadcastExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [s_store_sk#1] -Right keys [1]: [ss_store_sk#4] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 9] -Output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] -Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] - -(18) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(19) CometFilter -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Condition : isnotnull(i_item_sk#13) - -(20) CometProject -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Arguments: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18], [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#17, 50)) AS i_brand#18] - -(21) CometColumnarToRow [codegen id : 4] -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] - -(22) BroadcastExchange -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(23) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#3] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 9] -Output [7]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] -Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12, i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] - -(25) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(26) ColumnarToRow [codegen id : 6] -Input [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] - -(27) Filter [codegen id : 6] -Input [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_store_sk#20) - -(28) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#23] - -(29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#22] -Right keys [1]: [d_date_sk#23] -Join type: Inner -Join condition: None - -(30) Project [codegen id : 6] -Output [3]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21] -Input [5]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22, d_date_sk#23] - -(31) HashAggregate [codegen id : 6] -Input [3]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21] -Keys [2]: [ss_store_sk#20, ss_item_sk#19] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#21))] -Aggregate Attributes [1]: [sum#24] -Results [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] - -(32) CometColumnarExchange -Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] -Arguments: hashpartitioning(ss_store_sk#20, ss_item_sk#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(33) CometColumnarToRow [codegen id : 7] -Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] - -(34) HashAggregate [codegen id : 7] -Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] -Keys [2]: [ss_store_sk#20, ss_item_sk#19] -Functions [1]: [sum(UnscaledValue(ss_sales_price#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#21))#26] -Results [2]: [ss_store_sk#20, MakeDecimal(sum(UnscaledValue(ss_sales_price#21))#26,17,2) AS revenue#27] - -(35) HashAggregate [codegen id : 7] -Input [2]: [ss_store_sk#20, revenue#27] -Keys [1]: [ss_store_sk#20] -Functions [1]: [partial_avg(revenue#27)] -Aggregate Attributes [2]: [sum#28, count#29] -Results [3]: [ss_store_sk#20, sum#30, count#31] - -(36) CometColumnarExchange -Input [3]: [ss_store_sk#20, sum#30, count#31] -Arguments: hashpartitioning(ss_store_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(37) CometColumnarToRow [codegen id : 8] -Input [3]: [ss_store_sk#20, sum#30, count#31] - -(38) HashAggregate [codegen id : 8] -Input [3]: [ss_store_sk#20, sum#30, count#31] -Keys [1]: [ss_store_sk#20] -Functions [1]: [avg(revenue#27)] -Aggregate Attributes [1]: [avg(revenue#27)#32] -Results [2]: [ss_store_sk#20, avg(revenue#27)#32 AS ave#33] - -(39) Filter [codegen id : 8] -Input [2]: [ss_store_sk#20, ave#33] -Condition : isnotnull(ave#33) - -(40) BroadcastExchange -Input [2]: [ss_store_sk#20, ave#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [ss_store_sk#20] -Join type: Inner -Join condition: (cast(revenue#12 as decimal(23,7)) <= (0.1 * ave#33)) - -(42) Project [codegen id : 9] -Output [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#18] -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18, ss_store_sk#20, ave#33] - -(43) TakeOrderedAndProject -Input [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#18] -Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#14 ASC NULLS FIRST], [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#18] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometNativeScan parquet spark_catalog.default.date_dim (44) - - -(44) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [2]: [d_date_sk#8, d_month_seq#34] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1176)) AND (d_month_seq#34 <= 1187)) AND isnotnull(d_date_sk#8)) - -(46) CometProject -Input [2]: [d_date_sk#8, d_month_seq#34] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(48) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/simplified.txt deleted file mode 100644 index 2695c9fb89..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - WholeStageCodegen (9) - Project [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] - Project [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [s_store_name,ss_store_sk,ss_item_sk,revenue] - BroadcastHashJoin [s_store_sk,ss_store_sk] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name] - InputAdapter - BroadcastExchange #1 - WholeStageCodegen (3) - Filter [revenue] - HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_store_sk,ss_item_sk] #2 - WholeStageCodegen (2) - HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_brand] [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) - Filter [ave] - HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_store_sk] #6 - WholeStageCodegen (7) - HashAggregate [ss_store_sk,revenue] [sum,count,sum,count] - HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_store_sk,ss_item_sk] #7 - WholeStageCodegen (6) - HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/explain.txt deleted file mode 100644 index 6dc60931a8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,264 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometProject (38) - +- CometBroadcastHashJoin (37) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (1) - : : +- CometBroadcastExchange (15) - : : +- CometFilter (14) - : : +- CometHashAggregate (13) - : : +- CometExchange (12) - : : +- CometHashAggregate (11) - : : +- CometProject (10) - : : +- CometBroadcastHashJoin (9) - : : :- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : +- CometBroadcastExchange (8) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (5) - : +- CometBroadcastExchange (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (18) - +- CometBroadcastExchange (36) - +- CometFilter (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometHashAggregate (31) - +- CometExchange (30) - +- CometHashAggregate (29) - +- CometProject (28) - +- CometBroadcastHashJoin (27) - :- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (24) - +- ReusedExchange (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#1, s_store_name#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [s_store_sk#1, s_store_name#2] -Condition : isnotnull(s_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [d_date_sk#8, d_month_seq#9] -Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_month_seq#9 <= 1187)) AND isnotnull(d_date_sk#8)) - -(7) CometProject -Input [2]: [d_date_sk#8, d_month_seq#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(8) CometBroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: [d_date_sk#8] - -(9) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [d_date_sk#8] -Arguments: [ss_sold_date_sk#6], [d_date_sk#8], Inner, BuildRight - -(10) CometProject -Input [5]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] -Arguments: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5], [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] - -(11) CometHashAggregate -Input [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] -Keys [2]: [ss_store_sk#4, ss_item_sk#3] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#5))] - -(12) CometExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(13) CometHashAggregate -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -Keys [2]: [ss_store_sk#4, ss_item_sk#3] -Functions [1]: [sum(UnscaledValue(ss_sales_price#5))] - -(14) CometFilter -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] -Condition : isnotnull(revenue#11) - -(15) CometBroadcastExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] -Arguments: [ss_store_sk#4, ss_item_sk#3, revenue#11] - -(16) CometBroadcastHashJoin -Left output [2]: [s_store_sk#1, s_store_name#2] -Right output [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] -Arguments: [s_store_sk#1], [ss_store_sk#4], Inner, BuildRight - -(17) CometProject -Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] -Arguments: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11], [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(19) CometFilter -Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -Condition : isnotnull(i_item_sk#12) - -(20) CometProject -Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -Arguments: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17], [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#16, 50)) AS i_brand#17] - -(21) CometBroadcastExchange -Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(22) CometBroadcastHashJoin -Left output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] -Right output [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: [ss_item_sk#3], [i_item_sk#12], Inner, BuildRight - -(23) CometProject -Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11, i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17], [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#21), dynamicpruningexpression(ss_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] -Condition : isnotnull(ss_store_sk#19) - -(26) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#23] - -(27) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] -Right output [1]: [d_date_sk#23] -Arguments: [ss_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(28) CometProject -Input [5]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21, d_date_sk#23] -Arguments: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20], [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] - -(29) CometHashAggregate -Input [3]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] -Keys [2]: [ss_store_sk#19, ss_item_sk#18] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#20))] - -(30) CometExchange -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#24] -Arguments: hashpartitioning(ss_store_sk#19, ss_item_sk#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(31) CometHashAggregate -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#24] -Keys [2]: [ss_store_sk#19, ss_item_sk#18] -Functions [1]: [sum(UnscaledValue(ss_sales_price#20))] - -(32) CometHashAggregate -Input [2]: [ss_store_sk#19, revenue#25] -Keys [1]: [ss_store_sk#19] -Functions [1]: [partial_avg(revenue#25)] - -(33) CometExchange -Input [3]: [ss_store_sk#19, sum#26, count#27] -Arguments: hashpartitioning(ss_store_sk#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(34) CometHashAggregate -Input [3]: [ss_store_sk#19, sum#26, count#27] -Keys [1]: [ss_store_sk#19] -Functions [1]: [avg(revenue#25)] - -(35) CometFilter -Input [2]: [ss_store_sk#19, ave#28] -Condition : isnotnull(ave#28) - -(36) CometBroadcastExchange -Input [2]: [ss_store_sk#19, ave#28] -Arguments: [ss_store_sk#19, ave#28] - -(37) CometBroadcastHashJoin -Left output [7]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Right output [2]: [ss_store_sk#19, ave#28] -Arguments: [ss_store_sk#4], [ss_store_sk#19], Inner, (cast(revenue#11 as decimal(23,7)) <= (0.1 * ave#28)), BuildRight - -(38) CometProject -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17, ss_store_sk#19, ave#28] -Arguments: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(39) CometTakeOrderedAndProject -Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#2 ASC NULLS FIRST,i_item_desc#13 ASC NULLS FIRST], output=[s_store_name#2,i_item_desc#13,revenue#11,i_current_price#14,i_wholesale_cost#15,i_brand#17]), [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17], 100, 0, [s_store_name#2 ASC NULLS FIRST, i_item_desc#13 ASC NULLS FIRST], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(40) CometColumnarToRow [codegen id : 1] -Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#8, d_month_seq#9] -Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_month_seq#9 <= 1187)) AND isnotnull(d_date_sk#8)) - -(43) CometProject -Input [2]: [d_date_sk#8, d_month_seq#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(45) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/simplified.txt deleted file mode 100644 index cd1c84b6f2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,51 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - CometProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - CometBroadcastHashJoin [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand,ss_store_sk,ave] - CometProject [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometBroadcastHashJoin [s_store_name,ss_store_sk,ss_item_sk,revenue,i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometProject [s_store_name,ss_store_sk,ss_item_sk,revenue] - CometBroadcastHashJoin [s_store_sk,s_store_name,ss_store_sk,ss_item_sk,revenue] - CometFilter [s_store_sk,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name] - CometBroadcastExchange [ss_store_sk,ss_item_sk,revenue] #1 - CometFilter [ss_store_sk,ss_item_sk,revenue] - CometHashAggregate [sum] [ss_store_sk,ss_item_sk,revenue,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_store_sk,ss_item_sk] #2 - CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] - CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] #5 - CometProject [i_brand] [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometBroadcastExchange [ss_store_sk,ave] #6 - CometFilter [ss_store_sk,ave] - CometHashAggregate [sum,count] [ss_store_sk,ave,avg(revenue)] - CometExchange [ss_store_sk] #7 - CometHashAggregate [revenue] [ss_store_sk,sum,count] - CometHashAggregate [ss_item_sk,sum] [ss_store_sk,revenue,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_store_sk,ss_item_sk] #8 - CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] - CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - 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 [native_iceberg_compat] 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/q65/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/explain.txt deleted file mode 100644 index 6dc60931a8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/explain.txt +++ /dev/null @@ -1,264 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometProject (38) - +- CometBroadcastHashJoin (37) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (1) - : : +- CometBroadcastExchange (15) - : : +- CometFilter (14) - : : +- CometHashAggregate (13) - : : +- CometExchange (12) - : : +- CometHashAggregate (11) - : : +- CometProject (10) - : : +- CometBroadcastHashJoin (9) - : : :- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : +- CometBroadcastExchange (8) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (5) - : +- CometBroadcastExchange (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (18) - +- CometBroadcastExchange (36) - +- CometFilter (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometHashAggregate (31) - +- CometExchange (30) - +- CometHashAggregate (29) - +- CometProject (28) - +- CometBroadcastHashJoin (27) - :- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (24) - +- ReusedExchange (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#1, s_store_name#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [s_store_sk#1, s_store_name#2] -Condition : isnotnull(s_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [d_date_sk#8, d_month_seq#9] -Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_month_seq#9 <= 1187)) AND isnotnull(d_date_sk#8)) - -(7) CometProject -Input [2]: [d_date_sk#8, d_month_seq#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(8) CometBroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: [d_date_sk#8] - -(9) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [d_date_sk#8] -Arguments: [ss_sold_date_sk#6], [d_date_sk#8], Inner, BuildRight - -(10) CometProject -Input [5]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] -Arguments: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5], [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] - -(11) CometHashAggregate -Input [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] -Keys [2]: [ss_store_sk#4, ss_item_sk#3] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#5))] - -(12) CometExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(13) CometHashAggregate -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -Keys [2]: [ss_store_sk#4, ss_item_sk#3] -Functions [1]: [sum(UnscaledValue(ss_sales_price#5))] - -(14) CometFilter -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] -Condition : isnotnull(revenue#11) - -(15) CometBroadcastExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] -Arguments: [ss_store_sk#4, ss_item_sk#3, revenue#11] - -(16) CometBroadcastHashJoin -Left output [2]: [s_store_sk#1, s_store_name#2] -Right output [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] -Arguments: [s_store_sk#1], [ss_store_sk#4], Inner, BuildRight - -(17) CometProject -Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] -Arguments: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11], [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(19) CometFilter -Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -Condition : isnotnull(i_item_sk#12) - -(20) CometProject -Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -Arguments: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17], [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#16, 50)) AS i_brand#17] - -(21) CometBroadcastExchange -Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(22) CometBroadcastHashJoin -Left output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] -Right output [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: [ss_item_sk#3], [i_item_sk#12], Inner, BuildRight - -(23) CometProject -Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11, i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17], [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#21), dynamicpruningexpression(ss_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] -Condition : isnotnull(ss_store_sk#19) - -(26) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#23] - -(27) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] -Right output [1]: [d_date_sk#23] -Arguments: [ss_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(28) CometProject -Input [5]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21, d_date_sk#23] -Arguments: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20], [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] - -(29) CometHashAggregate -Input [3]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] -Keys [2]: [ss_store_sk#19, ss_item_sk#18] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#20))] - -(30) CometExchange -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#24] -Arguments: hashpartitioning(ss_store_sk#19, ss_item_sk#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(31) CometHashAggregate -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#24] -Keys [2]: [ss_store_sk#19, ss_item_sk#18] -Functions [1]: [sum(UnscaledValue(ss_sales_price#20))] - -(32) CometHashAggregate -Input [2]: [ss_store_sk#19, revenue#25] -Keys [1]: [ss_store_sk#19] -Functions [1]: [partial_avg(revenue#25)] - -(33) CometExchange -Input [3]: [ss_store_sk#19, sum#26, count#27] -Arguments: hashpartitioning(ss_store_sk#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(34) CometHashAggregate -Input [3]: [ss_store_sk#19, sum#26, count#27] -Keys [1]: [ss_store_sk#19] -Functions [1]: [avg(revenue#25)] - -(35) CometFilter -Input [2]: [ss_store_sk#19, ave#28] -Condition : isnotnull(ave#28) - -(36) CometBroadcastExchange -Input [2]: [ss_store_sk#19, ave#28] -Arguments: [ss_store_sk#19, ave#28] - -(37) CometBroadcastHashJoin -Left output [7]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Right output [2]: [ss_store_sk#19, ave#28] -Arguments: [ss_store_sk#4], [ss_store_sk#19], Inner, (cast(revenue#11 as decimal(23,7)) <= (0.1 * ave#28)), BuildRight - -(38) CometProject -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17, ss_store_sk#19, ave#28] -Arguments: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(39) CometTakeOrderedAndProject -Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#2 ASC NULLS FIRST,i_item_desc#13 ASC NULLS FIRST], output=[s_store_name#2,i_item_desc#13,revenue#11,i_current_price#14,i_wholesale_cost#15,i_brand#17]), [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17], 100, 0, [s_store_name#2 ASC NULLS FIRST, i_item_desc#13 ASC NULLS FIRST], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(40) CometColumnarToRow [codegen id : 1] -Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#8, d_month_seq#9] -Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_month_seq#9 <= 1187)) AND isnotnull(d_date_sk#8)) - -(43) CometProject -Input [2]: [d_date_sk#8, d_month_seq#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(45) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/extended.txt deleted file mode 100644 index 1fc09bd2e3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/extended.txt +++ /dev/null @@ -1,52 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 45 out of 48 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index cd1c84b6f2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt +++ /dev/null @@ -1,51 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - CometProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - CometBroadcastHashJoin [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand,ss_store_sk,ave] - CometProject [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometBroadcastHashJoin [s_store_name,ss_store_sk,ss_item_sk,revenue,i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometProject [s_store_name,ss_store_sk,ss_item_sk,revenue] - CometBroadcastHashJoin [s_store_sk,s_store_name,ss_store_sk,ss_item_sk,revenue] - CometFilter [s_store_sk,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name] - CometBroadcastExchange [ss_store_sk,ss_item_sk,revenue] #1 - CometFilter [ss_store_sk,ss_item_sk,revenue] - CometHashAggregate [sum] [ss_store_sk,ss_item_sk,revenue,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_store_sk,ss_item_sk] #2 - CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] - CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] #5 - CometProject [i_brand] [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometBroadcastExchange [ss_store_sk,ave] #6 - CometFilter [ss_store_sk,ave] - CometHashAggregate [sum,count] [ss_store_sk,ave,avg(revenue)] - CometExchange [ss_store_sk] #7 - CometHashAggregate [revenue] [ss_store_sk,sum,count] - CometHashAggregate [ss_item_sk,sum] [ss_store_sk,revenue,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_store_sk,ss_item_sk] #8 - CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] - CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - 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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/explain.txt deleted file mode 100644 index b4e8604910..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/explain.txt +++ /dev/null @@ -1,349 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (56) -+- * HashAggregate (55) - +- * CometColumnarToRow (54) - +- CometColumnarExchange (53) - +- * HashAggregate (52) - +- Union (51) - :- * HashAggregate (31) - : +- * CometColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- * HashAggregate (28) - : +- * Project (27) - : +- * BroadcastHashJoin Inner BuildRight (26) - : :- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.warehouse (4) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (18) - : : +- * CometColumnarToRow (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.time_dim (14) - : +- BroadcastExchange (25) - : +- * CometColumnarToRow (24) - : +- CometProject (23) - : +- CometFilter (22) - : +- CometNativeScan parquet spark_catalog.default.ship_mode (21) - +- * HashAggregate (50) - +- * CometColumnarToRow (49) - +- CometColumnarExchange (48) - +- * HashAggregate (47) - +- * Project (46) - +- * BroadcastHashJoin Inner BuildRight (45) - :- * Project (43) - : +- * BroadcastHashJoin Inner BuildRight (42) - : :- * Project (40) - : : +- * BroadcastHashJoin Inner BuildRight (39) - : : :- * Project (37) - : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : :- * Filter (34) - : : : : +- * ColumnarToRow (33) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (32) - : : : +- ReusedExchange (35) - : : +- ReusedExchange (38) - : +- ReusedExchange (41) - +- ReusedExchange (44) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_ship_mode_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] - -(3) Filter [codegen id : 5] -Input [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Condition : ((isnotnull(ws_warehouse_sk#3) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_ship_mode_sk#2)) - -(4) CometNativeScan parquet spark_catalog.default.warehouse -Output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(5) CometFilter -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Condition : isnotnull(w_warehouse_sk#9) - -(6) CometProject -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Arguments: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15], [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(w_state#14, 2)) AS w_state#16, w_country#15] - -(7) CometColumnarToRow [codegen id : 1] -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] - -(8) BroadcastExchange -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_warehouse_sk#3] -Right keys [1]: [w_warehouse_sk#9] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 5] -Output [12]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] - -(11) ReusedExchange [Reuses operator id: 60] -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_sold_date_sk#7] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [13]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Input [15]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_date_sk#17, d_year#18, d_moy#19] - -(14) CometNativeScan parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#20, t_time#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_time), GreaterThanOrEqual(t_time,30838), LessThanOrEqual(t_time,59638), IsNotNull(t_time_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [t_time_sk#20, t_time#21] -Condition : (((isnotnull(t_time#21) AND (t_time#21 >= 30838)) AND (t_time#21 <= 59638)) AND isnotnull(t_time_sk#20)) - -(16) CometProject -Input [2]: [t_time_sk#20, t_time#21] -Arguments: [t_time_sk#20], [t_time_sk#20] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [t_time_sk#20] - -(18) BroadcastExchange -Input [1]: [t_time_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_sold_time_sk#1] -Right keys [1]: [t_time_sk#20] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [12]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, t_time_sk#20] - -(21) CometNativeScan parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(sm_carrier#23, 20)) IN (DHL ,BARIAN ) AND isnotnull(sm_ship_mode_sk#22)) - -(23) CometProject -Input [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Arguments: [sm_ship_mode_sk#22], [sm_ship_mode_sk#22] - -(24) CometColumnarToRow [codegen id : 4] -Input [1]: [sm_ship_mode_sk#22] - -(25) BroadcastExchange -Input [1]: [sm_ship_mode_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_ship_mode_sk#2] -Right keys [1]: [sm_ship_mode_sk#22] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 5] -Output [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Input [13]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, sm_ship_mode_sk#22] - -(28) HashAggregate [codegen id : 5] -Input [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] -Functions [24]: [partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] -Aggregate Attributes [48]: [sum#24, isEmpty#25, sum#26, isEmpty#27, sum#28, isEmpty#29, sum#30, isEmpty#31, sum#32, isEmpty#33, sum#34, isEmpty#35, sum#36, isEmpty#37, sum#38, isEmpty#39, sum#40, isEmpty#41, sum#42, isEmpty#43, sum#44, isEmpty#45, sum#46, isEmpty#47, sum#48, isEmpty#49, sum#50, isEmpty#51, sum#52, isEmpty#53, sum#54, isEmpty#55, sum#56, isEmpty#57, sum#58, isEmpty#59, sum#60, isEmpty#61, sum#62, isEmpty#63, sum#64, isEmpty#65, sum#66, isEmpty#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Results [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#72, isEmpty#73, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] - -(29) CometColumnarExchange -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#72, isEmpty#73, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 6] -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#72, isEmpty#73, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] - -(31) HashAggregate [codegen id : 6] -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#72, isEmpty#73, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] -Functions [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#120, sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#121, sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#122, sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#123, sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#124, sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#125, sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#126, sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#127, sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#128, sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#129, sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#130, sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#131, sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#132, sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#133, sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#134, sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#135, sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#136, sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#137, sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#138, sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#139, sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#140, sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#141, sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#142, sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#143] -Results [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, DHL,BARIAN AS ship_carriers#144, d_year#18 AS year#145, sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#120 AS jan_sales#146, sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#121 AS feb_sales#147, sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#122 AS mar_sales#148, sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#123 AS apr_sales#149, sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#124 AS may_sales#150, sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#125 AS jun_sales#151, sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#126 AS jul_sales#152, sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#127 AS aug_sales#153, sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#128 AS sep_sales#154, sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#129 AS oct_sales#155, sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#130 AS nov_sales#156, sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#131 AS dec_sales#157, sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#132 AS jan_net#158, sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#133 AS feb_net#159, sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#134 AS mar_net#160, sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#135 AS apr_net#161, sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#136 AS may_net#162, sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#137 AS jun_net#163, sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#138 AS jul_net#164, sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#139 AS aug_net#165, sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#140 AS sep_net#166, sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#141 AS oct_net#167, sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#142 AS nov_net#168, sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#143 AS dec_net#169] - -(32) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_warehouse_sk#172, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#176), dynamicpruningexpression(cs_sold_date_sk#176 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] -ReadSchema: struct - -(33) ColumnarToRow [codegen id : 11] -Input [7]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_warehouse_sk#172, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176] - -(34) Filter [codegen id : 11] -Input [7]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_warehouse_sk#172, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176] -Condition : ((isnotnull(cs_warehouse_sk#172) AND isnotnull(cs_sold_time_sk#170)) AND isnotnull(cs_ship_mode_sk#171)) - -(35) ReusedExchange [Reuses operator id: 8] -Output [7]: [w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] - -(36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_warehouse_sk#172] -Right keys [1]: [w_warehouse_sk#177] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 11] -Output [12]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] -Input [14]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_warehouse_sk#172, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176, w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] - -(38) ReusedExchange [Reuses operator id: 60] -Output [3]: [d_date_sk#184, d_year#185, d_moy#186] - -(39) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#176] -Right keys [1]: [d_date_sk#184] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 11] -Output [13]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -Input [15]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_date_sk#184, d_year#185, d_moy#186] - -(41) ReusedExchange [Reuses operator id: 18] -Output [1]: [t_time_sk#187] - -(42) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_time_sk#170] -Right keys [1]: [t_time_sk#187] -Join type: Inner -Join condition: None - -(43) Project [codegen id : 11] -Output [12]: [cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -Input [14]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, t_time_sk#187] - -(44) ReusedExchange [Reuses operator id: 25] -Output [1]: [sm_ship_mode_sk#188] - -(45) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_ship_mode_sk#171] -Right keys [1]: [sm_ship_mode_sk#188] -Join type: Inner -Join condition: None - -(46) Project [codegen id : 11] -Output [11]: [cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -Input [13]: [cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, sm_ship_mode_sk#188] - -(47) HashAggregate [codegen id : 11] -Input [11]: [cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] -Functions [24]: [partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)] -Aggregate Attributes [48]: [sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236] -Results [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] - -(48) CometColumnarExchange -Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] -Arguments: hashpartitioning(w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(49) CometColumnarToRow [codegen id : 12] -Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] - -(50) HashAggregate [codegen id : 12] -Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] -Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] -Functions [24]: [sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#286, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#287, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#288, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#289, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#301, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#302, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#303, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#304, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#305, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#306, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#307, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#308] -Results [32]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, DHL,BARIAN AS ship_carriers#309, d_year#185 AS year#310, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#285 AS jan_sales#311, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#286 AS feb_sales#312, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#287 AS mar_sales#313, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#288 AS apr_sales#314, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#289 AS may_sales#315, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#290 AS jun_sales#316, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#291 AS jul_sales#317, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#292 AS aug_sales#318, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#293 AS sep_sales#319, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#294 AS oct_sales#320, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#295 AS nov_sales#321, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#296 AS dec_sales#322, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#297 AS jan_net#323, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#298 AS feb_net#324, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#299 AS mar_net#325, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#300 AS apr_net#326, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#301 AS may_net#327, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#302 AS jun_net#328, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#303 AS jul_net#329, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#304 AS aug_net#330, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#305 AS sep_net#331, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#306 AS oct_net#332, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#307 AS nov_net#333, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#308 AS dec_net#334] - -(51) Union - -(52) HashAggregate [codegen id : 13] -Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, jan_sales#146, feb_sales#147, mar_sales#148, apr_sales#149, may_sales#150, jun_sales#151, jul_sales#152, aug_sales#153, sep_sales#154, oct_sales#155, nov_sales#156, dec_sales#157, jan_net#158, feb_net#159, mar_net#160, apr_net#161, may_net#162, jun_net#163, jul_net#164, aug_net#165, sep_net#166, oct_net#167, nov_net#168, dec_net#169] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145] -Functions [36]: [partial_sum(jan_sales#146), partial_sum(feb_sales#147), partial_sum(mar_sales#148), partial_sum(apr_sales#149), partial_sum(may_sales#150), partial_sum(jun_sales#151), partial_sum(jul_sales#152), partial_sum(aug_sales#153), partial_sum(sep_sales#154), partial_sum(oct_sales#155), partial_sum(nov_sales#156), partial_sum(dec_sales#157), partial_sum((jan_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#157 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#158), partial_sum(feb_net#159), partial_sum(mar_net#160), partial_sum(apr_net#161), partial_sum(may_net#162), partial_sum(jun_net#163), partial_sum(jul_net#164), partial_sum(aug_net#165), partial_sum(sep_net#166), partial_sum(oct_net#167), partial_sum(nov_net#168), partial_sum(dec_net#169)] -Aggregate Attributes [72]: [sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358, sum#359, isEmpty#360, sum#361, isEmpty#362, sum#363, isEmpty#364, sum#365, isEmpty#366, sum#367, isEmpty#368, sum#369, isEmpty#370, sum#371, isEmpty#372, sum#373, isEmpty#374, sum#375, isEmpty#376, sum#377, isEmpty#378, sum#379, isEmpty#380, sum#381, isEmpty#382, sum#383, isEmpty#384, sum#385, isEmpty#386, sum#387, isEmpty#388, sum#389, isEmpty#390, sum#391, isEmpty#392, sum#393, isEmpty#394, sum#395, isEmpty#396, sum#397, isEmpty#398, sum#399, isEmpty#400, sum#401, isEmpty#402, sum#403, isEmpty#404, sum#405, isEmpty#406] -Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] - -(53) CometColumnarExchange -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(54) CometColumnarToRow [codegen id : 14] -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] - -(55) HashAggregate [codegen id : 14] -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145] -Functions [36]: [sum(jan_sales#146), sum(feb_sales#147), sum(mar_sales#148), sum(apr_sales#149), sum(may_sales#150), sum(jun_sales#151), sum(jul_sales#152), sum(aug_sales#153), sum(sep_sales#154), sum(oct_sales#155), sum(nov_sales#156), sum(dec_sales#157), sum((jan_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#157 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#158), sum(feb_net#159), sum(mar_net#160), sum(apr_net#161), sum(may_net#162), sum(jun_net#163), sum(jul_net#164), sum(aug_net#165), sum(sep_net#166), sum(oct_net#167), sum(nov_net#168), sum(dec_net#169)] -Aggregate Attributes [36]: [sum(jan_sales#146)#479, sum(feb_sales#147)#480, sum(mar_sales#148)#481, sum(apr_sales#149)#482, sum(may_sales#150)#483, sum(jun_sales#151)#484, sum(jul_sales#152)#485, sum(aug_sales#153)#486, sum(sep_sales#154)#487, sum(oct_sales#155)#488, sum(nov_sales#156)#489, sum(dec_sales#157)#490, sum((jan_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#491, sum((feb_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#492, sum((mar_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#493, sum((apr_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#494, sum((may_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#495, sum((jun_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#496, sum((jul_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#497, sum((aug_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#498, sum((sep_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#499, sum((oct_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#500, sum((nov_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#501, sum((dec_sales#157 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#502, sum(jan_net#158)#503, sum(feb_net#159)#504, sum(mar_net#160)#505, sum(apr_net#161)#506, sum(may_net#162)#507, sum(jun_net#163)#508, sum(jul_net#164)#509, sum(aug_net#165)#510, sum(sep_net#166)#511, sum(oct_net#167)#512, sum(nov_net#168)#513, sum(dec_net#169)#514] -Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum(jan_sales#146)#479 AS jan_sales#515, sum(feb_sales#147)#480 AS feb_sales#516, sum(mar_sales#148)#481 AS mar_sales#517, sum(apr_sales#149)#482 AS apr_sales#518, sum(may_sales#150)#483 AS may_sales#519, sum(jun_sales#151)#484 AS jun_sales#520, sum(jul_sales#152)#485 AS jul_sales#521, sum(aug_sales#153)#486 AS aug_sales#522, sum(sep_sales#154)#487 AS sep_sales#523, sum(oct_sales#155)#488 AS oct_sales#524, sum(nov_sales#156)#489 AS nov_sales#525, sum(dec_sales#157)#490 AS dec_sales#526, sum((jan_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#491 AS jan_sales_per_sq_foot#527, sum((feb_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#492 AS feb_sales_per_sq_foot#528, sum((mar_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#493 AS mar_sales_per_sq_foot#529, sum((apr_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#494 AS apr_sales_per_sq_foot#530, sum((may_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#495 AS may_sales_per_sq_foot#531, sum((jun_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#496 AS jun_sales_per_sq_foot#532, sum((jul_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#497 AS jul_sales_per_sq_foot#533, sum((aug_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#498 AS aug_sales_per_sq_foot#534, sum((sep_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#499 AS sep_sales_per_sq_foot#535, sum((oct_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#500 AS oct_sales_per_sq_foot#536, sum((nov_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#501 AS nov_sales_per_sq_foot#537, sum((dec_sales#157 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#502 AS dec_sales_per_sq_foot#538, sum(jan_net#158)#503 AS jan_net#539, sum(feb_net#159)#504 AS feb_net#540, sum(mar_net#160)#505 AS mar_net#541, sum(apr_net#161)#506 AS apr_net#542, sum(may_net#162)#507 AS may_net#543, sum(jun_net#163)#508 AS jun_net#544, sum(jul_net#164)#509 AS jul_net#545, sum(aug_net#165)#510 AS aug_net#546, sum(sep_net#166)#511 AS sep_net#547, sum(oct_net#167)#512 AS oct_net#548, sum(nov_net#168)#513 AS nov_net#549, sum(dec_net#169)#514 AS dec_net#550] - -(56) TakeOrderedAndProject -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, jan_sales#515, feb_sales#516, mar_sales#517, apr_sales#518, may_sales#519, jun_sales#520, jul_sales#521, aug_sales#522, sep_sales#523, oct_sales#524, nov_sales#525, dec_sales#526, jan_sales_per_sq_foot#527, feb_sales_per_sq_foot#528, mar_sales_per_sq_foot#529, apr_sales_per_sq_foot#530, may_sales_per_sq_foot#531, jun_sales_per_sq_foot#532, jul_sales_per_sq_foot#533, aug_sales_per_sq_foot#534, sep_sales_per_sq_foot#535, oct_sales_per_sq_foot#536, nov_sales_per_sq_foot#537, dec_sales_per_sq_foot#538, jan_net#539, feb_net#540, mar_net#541, apr_net#542, may_net#543, jun_net#544, jul_net#545, aug_net#546, sep_net#547, oct_net#548, nov_net#549, dec_net#550] -Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, jan_sales#515, feb_sales#516, mar_sales#517, apr_sales#518, may_sales#519, jun_sales#520, jul_sales#521, aug_sales#522, sep_sales#523, oct_sales#524, nov_sales#525, dec_sales#526, jan_sales_per_sq_foot#527, feb_sales_per_sq_foot#528, mar_sales_per_sq_foot#529, apr_sales_per_sq_foot#530, may_sales_per_sq_foot#531, jun_sales_per_sq_foot#532, jul_sales_per_sq_foot#533, aug_sales_per_sq_foot#534, sep_sales_per_sq_foot#535, oct_sales_per_sq_foot#536, nov_sales_per_sq_foot#537, dec_sales_per_sq_foot#538, jan_net#539, feb_net#540, mar_net#541, apr_net#542, may_net#543, jun_net#544, jul_net#545, aug_net#546, sep_net#547, oct_net#548, nov_net#549, dec_net#550] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (60) -+- * CometColumnarToRow (59) - +- CometFilter (58) - +- CometNativeScan parquet spark_catalog.default.date_dim (57) - - -(57) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(59) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] - -(60) BroadcastExchange -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#176 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/simplified.txt deleted file mode 100644 index 3a1f053d60..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - WholeStageCodegen (14) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,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,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum((jan_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((feb_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((mar_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((apr_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((may_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jun_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jul_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((aug_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((sep_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((oct_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((nov_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((dec_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net),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_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,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,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - WholeStageCodegen (13) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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,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,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] - InputAdapter - Union - WholeStageCodegen (6) - 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 (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - WholeStageCodegen (5) - HashAggregate [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] [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,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] - Project [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] - BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] - Project [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] - BroadcastHashJoin [ws_sold_time_sk,t_time_sk] - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - Filter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [w_state] [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_time_sk,t_time] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [sm_ship_mode_sk] - CometFilter [sm_ship_mode_sk,sm_carrier] - CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] - WholeStageCodegen (12) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #7 - WholeStageCodegen (11) - HashAggregate [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] [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,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] - Project [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] - BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] - Project [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] - BroadcastHashJoin [cs_sold_time_sk,t_time_sk] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - Filter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #3 - InputAdapter - ReusedExchange [t_time_sk] #5 - InputAdapter - ReusedExchange [sm_ship_mode_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/explain.txt deleted file mode 100644 index 51db3be961..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,317 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (51) -+- CometTakeOrderedAndProject (50) - +- CometHashAggregate (49) - +- CometExchange (48) - +- CometHashAggregate (47) - +- CometUnion (46) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (14) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode (20) - +- CometHashAggregate (45) - +- CometExchange (44) - +- CometHashAggregate (43) - +- CometProject (42) - +- CometBroadcastHashJoin (41) - :- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometFilter (30) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_ship_mode_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Condition : ((isnotnull(ws_warehouse_sk#3) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_ship_mode_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Condition : isnotnull(w_warehouse_sk#9) - -(5) CometProject -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Arguments: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15], [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(w_state#14, 2)) AS w_state#16, w_country#15] - -(6) CometBroadcastExchange -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Arguments: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] - -(7) CometBroadcastHashJoin -Left output [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Right output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Arguments: [ws_warehouse_sk#3], [w_warehouse_sk#9], Inner, BuildRight - -(8) CometProject -Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Arguments: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15], [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [d_date_sk#17, d_year#18, d_moy#19] - -(12) CometBroadcastHashJoin -Left output [12]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Right output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [ws_sold_date_sk#7], [d_date_sk#17], Inner, BuildRight - -(13) CometProject -Input [15]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_date_sk#17, d_year#18, d_moy#19] -Arguments: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19], [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#20, t_time#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_time), GreaterThanOrEqual(t_time,30838), LessThanOrEqual(t_time,59638), IsNotNull(t_time_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [t_time_sk#20, t_time#21] -Condition : (((isnotnull(t_time#21) AND (t_time#21 >= 30838)) AND (t_time#21 <= 59638)) AND isnotnull(t_time_sk#20)) - -(16) CometProject -Input [2]: [t_time_sk#20, t_time#21] -Arguments: [t_time_sk#20], [t_time_sk#20] - -(17) CometBroadcastExchange -Input [1]: [t_time_sk#20] -Arguments: [t_time_sk#20] - -(18) CometBroadcastHashJoin -Left output [13]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Right output [1]: [t_time_sk#20] -Arguments: [ws_sold_time_sk#1], [t_time_sk#20], Inner, BuildRight - -(19) CometProject -Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, t_time_sk#20] -Arguments: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19], [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(sm_carrier#23, 20)) IN (DHL ,BARIAN ) AND isnotnull(sm_ship_mode_sk#22)) - -(22) CometProject -Input [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Arguments: [sm_ship_mode_sk#22], [sm_ship_mode_sk#22] - -(23) CometBroadcastExchange -Input [1]: [sm_ship_mode_sk#22] -Arguments: [sm_ship_mode_sk#22] - -(24) CometBroadcastHashJoin -Left output [12]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Right output [1]: [sm_ship_mode_sk#22] -Arguments: [ws_ship_mode_sk#2], [sm_ship_mode_sk#22], Inner, BuildRight - -(25) CometProject -Input [13]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, sm_ship_mode_sk#22] -Arguments: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19], [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] - -(26) CometHashAggregate -Input [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] -Functions [24]: [partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] - -(27) CometExchange -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#24, isEmpty#25, sum#26, isEmpty#27, sum#28, isEmpty#29, sum#30, isEmpty#31, sum#32, isEmpty#33, sum#34, isEmpty#35, sum#36, isEmpty#37, sum#38, isEmpty#39, sum#40, isEmpty#41, sum#42, isEmpty#43, sum#44, isEmpty#45, sum#46, isEmpty#47, sum#48, isEmpty#49, sum#50, isEmpty#51, sum#52, isEmpty#53, sum#54, isEmpty#55, sum#56, isEmpty#57, sum#58, isEmpty#59, sum#60, isEmpty#61, sum#62, isEmpty#63, sum#64, isEmpty#65, sum#66, isEmpty#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#24, isEmpty#25, sum#26, isEmpty#27, sum#28, isEmpty#29, sum#30, isEmpty#31, sum#32, isEmpty#33, sum#34, isEmpty#35, sum#36, isEmpty#37, sum#38, isEmpty#39, sum#40, isEmpty#41, sum#42, isEmpty#43, sum#44, isEmpty#45, sum#46, isEmpty#47, sum#48, isEmpty#49, sum#50, isEmpty#51, sum#52, isEmpty#53, sum#54, isEmpty#55, sum#56, isEmpty#57, sum#58, isEmpty#59, sum#60, isEmpty#61, sum#62, isEmpty#63, sum#64, isEmpty#65, sum#66, isEmpty#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] -Functions [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#78), dynamicpruningexpression(cs_sold_date_sk#78 IN dynamicpruning#79)] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] -ReadSchema: struct - -(30) CometFilter -Input [7]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78] -Condition : ((isnotnull(cs_warehouse_sk#74) AND isnotnull(cs_sold_time_sk#72)) AND isnotnull(cs_ship_mode_sk#73)) - -(31) ReusedExchange [Reuses operator id: 6] -Output [7]: [w_warehouse_sk#80, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] - -(32) CometBroadcastHashJoin -Left output [7]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78] -Right output [7]: [w_warehouse_sk#80, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] -Arguments: [cs_warehouse_sk#74], [w_warehouse_sk#80], Inner, BuildRight - -(33) CometProject -Input [14]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_sk#80, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] -Arguments: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86], [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] - -(34) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#87, d_year#88, d_moy#89] - -(35) CometBroadcastHashJoin -Left output [12]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] -Right output [3]: [d_date_sk#87, d_year#88, d_moy#89] -Arguments: [cs_sold_date_sk#78], [d_date_sk#87], Inner, BuildRight - -(36) CometProject -Input [15]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_date_sk#87, d_year#88, d_moy#89] -Arguments: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89], [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] - -(37) ReusedExchange [Reuses operator id: 17] -Output [1]: [t_time_sk#90] - -(38) CometBroadcastHashJoin -Left output [13]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] -Right output [1]: [t_time_sk#90] -Arguments: [cs_sold_time_sk#72], [t_time_sk#90], Inner, BuildRight - -(39) CometProject -Input [14]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89, t_time_sk#90] -Arguments: [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89], [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] - -(40) ReusedExchange [Reuses operator id: 23] -Output [1]: [sm_ship_mode_sk#91] - -(41) CometBroadcastHashJoin -Left output [12]: [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] -Right output [1]: [sm_ship_mode_sk#91] -Arguments: [cs_ship_mode_sk#73], [sm_ship_mode_sk#91], Inner, BuildRight - -(42) CometProject -Input [13]: [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89, sm_ship_mode_sk#91] -Arguments: [cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89], [cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] - -(43) CometHashAggregate -Input [11]: [cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] -Keys [7]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88] -Functions [24]: [partial_sum(CASE WHEN (d_moy#89 = 1) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 2) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 3) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 4) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 5) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 6) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 7) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 8) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 9) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 10) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 11) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 12) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 1) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 2) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 3) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 4) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 5) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 6) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 7) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 8) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 9) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 10) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 11) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 12) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END)] - -(44) CometExchange -Input [55]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Arguments: hashpartitioning(w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(45) CometHashAggregate -Input [55]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Keys [7]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88] -Functions [24]: [sum(CASE WHEN (d_moy#89 = 1) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 2) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 3) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 4) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 5) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 6) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 7) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 8) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 9) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 10) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 11) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 12) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 1) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 2) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 3) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 4) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 5) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 6) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 7) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 8) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 9) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 10) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 11) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 12) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END)] - -(46) CometUnion -Child 0 Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#142, feb_sales#143, mar_sales#144, apr_sales#145, may_sales#146, jun_sales#147, jul_sales#148, aug_sales#149, sep_sales#150, oct_sales#151, nov_sales#152, dec_sales#153, jan_net#154, feb_net#155, mar_net#156, apr_net#157, may_net#158, jun_net#159, jul_net#160, aug_net#161, sep_net#162, oct_net#163, nov_net#164, dec_net#165] -Child 1 Input [32]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, ship_carriers#166, year#167, jan_sales#168, feb_sales#169, mar_sales#170, apr_sales#171, may_sales#172, jun_sales#173, jul_sales#174, aug_sales#175, sep_sales#176, oct_sales#177, nov_sales#178, dec_sales#179, jan_net#180, feb_net#181, mar_net#182, apr_net#183, may_net#184, jun_net#185, jul_net#186, aug_net#187, sep_net#188, oct_net#189, nov_net#190, dec_net#191] - -(47) CometHashAggregate -Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#142, feb_sales#143, mar_sales#144, apr_sales#145, may_sales#146, jun_sales#147, jul_sales#148, aug_sales#149, sep_sales#150, oct_sales#151, nov_sales#152, dec_sales#153, jan_net#154, feb_net#155, mar_net#156, apr_net#157, may_net#158, jun_net#159, jul_net#160, aug_net#161, sep_net#162, oct_net#163, nov_net#164, dec_net#165] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141] -Functions [36]: [partial_sum(jan_sales#142), partial_sum(feb_sales#143), partial_sum(mar_sales#144), partial_sum(apr_sales#145), partial_sum(may_sales#146), partial_sum(jun_sales#147), partial_sum(jul_sales#148), partial_sum(aug_sales#149), partial_sum(sep_sales#150), partial_sum(oct_sales#151), partial_sum(nov_sales#152), partial_sum(dec_sales#153), partial_sum((jan_sales#142 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#143 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#144 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#154), partial_sum(feb_net#155), partial_sum(mar_net#156), partial_sum(apr_net#157), partial_sum(may_net#158), partial_sum(jun_net#159), partial_sum(jul_net#160), partial_sum(aug_net#161), partial_sum(sep_net#162), partial_sum(oct_net#163), partial_sum(nov_net#164), partial_sum(dec_net#165)] - -(48) CometExchange -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211, sum#212, isEmpty#213, sum#214, isEmpty#215, sum#216, isEmpty#217, sum#218, isEmpty#219, sum#220, isEmpty#221, sum#222, isEmpty#223, sum#224, isEmpty#225, sum#226, isEmpty#227, sum#228, isEmpty#229, sum#230, isEmpty#231, sum#232, isEmpty#233, sum#234, isEmpty#235, sum#236, isEmpty#237, sum#238, isEmpty#239, sum#240, isEmpty#241, sum#242, isEmpty#243, sum#244, isEmpty#245, sum#246, isEmpty#247, sum#248, isEmpty#249, sum#250, isEmpty#251, sum#252, isEmpty#253, sum#254, isEmpty#255, sum#256, isEmpty#257, sum#258, isEmpty#259, sum#260, isEmpty#261, sum#262, isEmpty#263] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211, sum#212, isEmpty#213, sum#214, isEmpty#215, sum#216, isEmpty#217, sum#218, isEmpty#219, sum#220, isEmpty#221, sum#222, isEmpty#223, sum#224, isEmpty#225, sum#226, isEmpty#227, sum#228, isEmpty#229, sum#230, isEmpty#231, sum#232, isEmpty#233, sum#234, isEmpty#235, sum#236, isEmpty#237, sum#238, isEmpty#239, sum#240, isEmpty#241, sum#242, isEmpty#243, sum#244, isEmpty#245, sum#246, isEmpty#247, sum#248, isEmpty#249, sum#250, isEmpty#251, sum#252, isEmpty#253, sum#254, isEmpty#255, sum#256, isEmpty#257, sum#258, isEmpty#259, sum#260, isEmpty#261, sum#262, isEmpty#263] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141] -Functions [36]: [sum(jan_sales#142), sum(feb_sales#143), sum(mar_sales#144), sum(apr_sales#145), sum(may_sales#146), sum(jun_sales#147), sum(jul_sales#148), sum(aug_sales#149), sum(sep_sales#150), sum(oct_sales#151), sum(nov_sales#152), sum(dec_sales#153), sum((jan_sales#142 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#143 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#144 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#154), sum(feb_net#155), sum(mar_net#156), sum(apr_net#157), sum(may_net#158), sum(jun_net#159), sum(jul_net#160), sum(aug_net#161), sum(sep_net#162), sum(oct_net#163), sum(nov_net#164), sum(dec_net#165)] - -(50) CometTakeOrderedAndProject -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#10 ASC NULLS FIRST], output=[w_warehouse_name#10,w_warehouse_sq_ft#11,w_city#12,w_county#13,w_state#16,w_country#15,ship_carriers#140,year#141,jan_sales#264,feb_sales#265,mar_sales#266,apr_sales#267,may_sales#268,jun_sales#269,jul_sales#270,aug_sales#271,sep_sales#272,oct_sales#273,nov_sales#274,dec_sales#275,jan_sales_per_sq_foot#276,feb_sales_per_sq_foot#277,mar_sales_per_sq_foot#278,apr_sales_per_sq_foot#279,may_sales_per_sq_foot#280,jun_sales_per_sq_foot#281,jul_sales_per_sq_foot#282,aug_sales_per_sq_foot#283,sep_sales_per_sq_foot#284,oct_sales_per_sq_foot#285,nov_sales_per_sq_foot#286,dec_sales_per_sq_foot#287,jan_net#288,feb_net#289,mar_net#290,apr_net#291,may_net#292,jun_net#293,jul_net#294,aug_net#295,sep_net#296,oct_net#297,nov_net#298,dec_net#299]), [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299], 100, 0, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299] - -(51) CometColumnarToRow [codegen id : 1] -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometFilter (53) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) - - -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(54) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] - -(55) BroadcastExchange -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#78 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/simplified.txt deleted file mode 100644 index bc6bde24b2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - CometHashAggregate [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,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,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(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum((jan_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((feb_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((mar_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((apr_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((may_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jun_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jul_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((aug_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((sep_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((oct_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((nov_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((dec_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net)] - CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - CometHashAggregate [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] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,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,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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] - CometHashAggregate [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] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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,d_year,sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END)] - CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - CometHashAggregate [d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [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] - 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,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,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_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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 - CometProject [w_state] [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [t_time_sk] #6 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_time] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_time] - CometBroadcastExchange [sm_ship_mode_sk] #7 - CometProject [sm_ship_mode_sk] - CometFilter [sm_ship_mode_sk,sm_carrier] - CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] - CometHashAggregate [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] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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,d_year,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)] - CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #8 - CometHashAggregate [d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [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] - 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,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,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_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_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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year,d_moy] #5 - ReusedExchange [t_time_sk] #6 - ReusedExchange [sm_ship_mode_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/explain.txt deleted file mode 100644 index 51db3be961..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/explain.txt +++ /dev/null @@ -1,317 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (51) -+- CometTakeOrderedAndProject (50) - +- CometHashAggregate (49) - +- CometExchange (48) - +- CometHashAggregate (47) - +- CometUnion (46) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (14) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode (20) - +- CometHashAggregate (45) - +- CometExchange (44) - +- CometHashAggregate (43) - +- CometProject (42) - +- CometBroadcastHashJoin (41) - :- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometFilter (30) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_ship_mode_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Condition : ((isnotnull(ws_warehouse_sk#3) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_ship_mode_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Condition : isnotnull(w_warehouse_sk#9) - -(5) CometProject -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Arguments: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15], [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(w_state#14, 2)) AS w_state#16, w_country#15] - -(6) CometBroadcastExchange -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Arguments: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] - -(7) CometBroadcastHashJoin -Left output [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Right output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Arguments: [ws_warehouse_sk#3], [w_warehouse_sk#9], Inner, BuildRight - -(8) CometProject -Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Arguments: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15], [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [d_date_sk#17, d_year#18, d_moy#19] - -(12) CometBroadcastHashJoin -Left output [12]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Right output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [ws_sold_date_sk#7], [d_date_sk#17], Inner, BuildRight - -(13) CometProject -Input [15]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_date_sk#17, d_year#18, d_moy#19] -Arguments: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19], [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#20, t_time#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_time), GreaterThanOrEqual(t_time,30838), LessThanOrEqual(t_time,59638), IsNotNull(t_time_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [t_time_sk#20, t_time#21] -Condition : (((isnotnull(t_time#21) AND (t_time#21 >= 30838)) AND (t_time#21 <= 59638)) AND isnotnull(t_time_sk#20)) - -(16) CometProject -Input [2]: [t_time_sk#20, t_time#21] -Arguments: [t_time_sk#20], [t_time_sk#20] - -(17) CometBroadcastExchange -Input [1]: [t_time_sk#20] -Arguments: [t_time_sk#20] - -(18) CometBroadcastHashJoin -Left output [13]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Right output [1]: [t_time_sk#20] -Arguments: [ws_sold_time_sk#1], [t_time_sk#20], Inner, BuildRight - -(19) CometProject -Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, t_time_sk#20] -Arguments: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19], [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(sm_carrier#23, 20)) IN (DHL ,BARIAN ) AND isnotnull(sm_ship_mode_sk#22)) - -(22) CometProject -Input [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Arguments: [sm_ship_mode_sk#22], [sm_ship_mode_sk#22] - -(23) CometBroadcastExchange -Input [1]: [sm_ship_mode_sk#22] -Arguments: [sm_ship_mode_sk#22] - -(24) CometBroadcastHashJoin -Left output [12]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Right output [1]: [sm_ship_mode_sk#22] -Arguments: [ws_ship_mode_sk#2], [sm_ship_mode_sk#22], Inner, BuildRight - -(25) CometProject -Input [13]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, sm_ship_mode_sk#22] -Arguments: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19], [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] - -(26) CometHashAggregate -Input [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] -Functions [24]: [partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] - -(27) CometExchange -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#24, isEmpty#25, sum#26, isEmpty#27, sum#28, isEmpty#29, sum#30, isEmpty#31, sum#32, isEmpty#33, sum#34, isEmpty#35, sum#36, isEmpty#37, sum#38, isEmpty#39, sum#40, isEmpty#41, sum#42, isEmpty#43, sum#44, isEmpty#45, sum#46, isEmpty#47, sum#48, isEmpty#49, sum#50, isEmpty#51, sum#52, isEmpty#53, sum#54, isEmpty#55, sum#56, isEmpty#57, sum#58, isEmpty#59, sum#60, isEmpty#61, sum#62, isEmpty#63, sum#64, isEmpty#65, sum#66, isEmpty#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#24, isEmpty#25, sum#26, isEmpty#27, sum#28, isEmpty#29, sum#30, isEmpty#31, sum#32, isEmpty#33, sum#34, isEmpty#35, sum#36, isEmpty#37, sum#38, isEmpty#39, sum#40, isEmpty#41, sum#42, isEmpty#43, sum#44, isEmpty#45, sum#46, isEmpty#47, sum#48, isEmpty#49, sum#50, isEmpty#51, sum#52, isEmpty#53, sum#54, isEmpty#55, sum#56, isEmpty#57, sum#58, isEmpty#59, sum#60, isEmpty#61, sum#62, isEmpty#63, sum#64, isEmpty#65, sum#66, isEmpty#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] -Functions [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#78), dynamicpruningexpression(cs_sold_date_sk#78 IN dynamicpruning#79)] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] -ReadSchema: struct - -(30) CometFilter -Input [7]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78] -Condition : ((isnotnull(cs_warehouse_sk#74) AND isnotnull(cs_sold_time_sk#72)) AND isnotnull(cs_ship_mode_sk#73)) - -(31) ReusedExchange [Reuses operator id: 6] -Output [7]: [w_warehouse_sk#80, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] - -(32) CometBroadcastHashJoin -Left output [7]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78] -Right output [7]: [w_warehouse_sk#80, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] -Arguments: [cs_warehouse_sk#74], [w_warehouse_sk#80], Inner, BuildRight - -(33) CometProject -Input [14]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_sk#80, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] -Arguments: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86], [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] - -(34) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#87, d_year#88, d_moy#89] - -(35) CometBroadcastHashJoin -Left output [12]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] -Right output [3]: [d_date_sk#87, d_year#88, d_moy#89] -Arguments: [cs_sold_date_sk#78], [d_date_sk#87], Inner, BuildRight - -(36) CometProject -Input [15]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_date_sk#87, d_year#88, d_moy#89] -Arguments: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89], [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] - -(37) ReusedExchange [Reuses operator id: 17] -Output [1]: [t_time_sk#90] - -(38) CometBroadcastHashJoin -Left output [13]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] -Right output [1]: [t_time_sk#90] -Arguments: [cs_sold_time_sk#72], [t_time_sk#90], Inner, BuildRight - -(39) CometProject -Input [14]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89, t_time_sk#90] -Arguments: [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89], [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] - -(40) ReusedExchange [Reuses operator id: 23] -Output [1]: [sm_ship_mode_sk#91] - -(41) CometBroadcastHashJoin -Left output [12]: [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] -Right output [1]: [sm_ship_mode_sk#91] -Arguments: [cs_ship_mode_sk#73], [sm_ship_mode_sk#91], Inner, BuildRight - -(42) CometProject -Input [13]: [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89, sm_ship_mode_sk#91] -Arguments: [cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89], [cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] - -(43) CometHashAggregate -Input [11]: [cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] -Keys [7]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88] -Functions [24]: [partial_sum(CASE WHEN (d_moy#89 = 1) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 2) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 3) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 4) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 5) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 6) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 7) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 8) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 9) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 10) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 11) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 12) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 1) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 2) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 3) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 4) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 5) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 6) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 7) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 8) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 9) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 10) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 11) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 12) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END)] - -(44) CometExchange -Input [55]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Arguments: hashpartitioning(w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(45) CometHashAggregate -Input [55]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Keys [7]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88] -Functions [24]: [sum(CASE WHEN (d_moy#89 = 1) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 2) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 3) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 4) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 5) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 6) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 7) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 8) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 9) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 10) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 11) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 12) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 1) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 2) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 3) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 4) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 5) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 6) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 7) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 8) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 9) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 10) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 11) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 12) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END)] - -(46) CometUnion -Child 0 Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#142, feb_sales#143, mar_sales#144, apr_sales#145, may_sales#146, jun_sales#147, jul_sales#148, aug_sales#149, sep_sales#150, oct_sales#151, nov_sales#152, dec_sales#153, jan_net#154, feb_net#155, mar_net#156, apr_net#157, may_net#158, jun_net#159, jul_net#160, aug_net#161, sep_net#162, oct_net#163, nov_net#164, dec_net#165] -Child 1 Input [32]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, ship_carriers#166, year#167, jan_sales#168, feb_sales#169, mar_sales#170, apr_sales#171, may_sales#172, jun_sales#173, jul_sales#174, aug_sales#175, sep_sales#176, oct_sales#177, nov_sales#178, dec_sales#179, jan_net#180, feb_net#181, mar_net#182, apr_net#183, may_net#184, jun_net#185, jul_net#186, aug_net#187, sep_net#188, oct_net#189, nov_net#190, dec_net#191] - -(47) CometHashAggregate -Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#142, feb_sales#143, mar_sales#144, apr_sales#145, may_sales#146, jun_sales#147, jul_sales#148, aug_sales#149, sep_sales#150, oct_sales#151, nov_sales#152, dec_sales#153, jan_net#154, feb_net#155, mar_net#156, apr_net#157, may_net#158, jun_net#159, jul_net#160, aug_net#161, sep_net#162, oct_net#163, nov_net#164, dec_net#165] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141] -Functions [36]: [partial_sum(jan_sales#142), partial_sum(feb_sales#143), partial_sum(mar_sales#144), partial_sum(apr_sales#145), partial_sum(may_sales#146), partial_sum(jun_sales#147), partial_sum(jul_sales#148), partial_sum(aug_sales#149), partial_sum(sep_sales#150), partial_sum(oct_sales#151), partial_sum(nov_sales#152), partial_sum(dec_sales#153), partial_sum((jan_sales#142 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#143 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#144 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#154), partial_sum(feb_net#155), partial_sum(mar_net#156), partial_sum(apr_net#157), partial_sum(may_net#158), partial_sum(jun_net#159), partial_sum(jul_net#160), partial_sum(aug_net#161), partial_sum(sep_net#162), partial_sum(oct_net#163), partial_sum(nov_net#164), partial_sum(dec_net#165)] - -(48) CometExchange -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211, sum#212, isEmpty#213, sum#214, isEmpty#215, sum#216, isEmpty#217, sum#218, isEmpty#219, sum#220, isEmpty#221, sum#222, isEmpty#223, sum#224, isEmpty#225, sum#226, isEmpty#227, sum#228, isEmpty#229, sum#230, isEmpty#231, sum#232, isEmpty#233, sum#234, isEmpty#235, sum#236, isEmpty#237, sum#238, isEmpty#239, sum#240, isEmpty#241, sum#242, isEmpty#243, sum#244, isEmpty#245, sum#246, isEmpty#247, sum#248, isEmpty#249, sum#250, isEmpty#251, sum#252, isEmpty#253, sum#254, isEmpty#255, sum#256, isEmpty#257, sum#258, isEmpty#259, sum#260, isEmpty#261, sum#262, isEmpty#263] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211, sum#212, isEmpty#213, sum#214, isEmpty#215, sum#216, isEmpty#217, sum#218, isEmpty#219, sum#220, isEmpty#221, sum#222, isEmpty#223, sum#224, isEmpty#225, sum#226, isEmpty#227, sum#228, isEmpty#229, sum#230, isEmpty#231, sum#232, isEmpty#233, sum#234, isEmpty#235, sum#236, isEmpty#237, sum#238, isEmpty#239, sum#240, isEmpty#241, sum#242, isEmpty#243, sum#244, isEmpty#245, sum#246, isEmpty#247, sum#248, isEmpty#249, sum#250, isEmpty#251, sum#252, isEmpty#253, sum#254, isEmpty#255, sum#256, isEmpty#257, sum#258, isEmpty#259, sum#260, isEmpty#261, sum#262, isEmpty#263] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141] -Functions [36]: [sum(jan_sales#142), sum(feb_sales#143), sum(mar_sales#144), sum(apr_sales#145), sum(may_sales#146), sum(jun_sales#147), sum(jul_sales#148), sum(aug_sales#149), sum(sep_sales#150), sum(oct_sales#151), sum(nov_sales#152), sum(dec_sales#153), sum((jan_sales#142 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#143 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#144 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#154), sum(feb_net#155), sum(mar_net#156), sum(apr_net#157), sum(may_net#158), sum(jun_net#159), sum(jul_net#160), sum(aug_net#161), sum(sep_net#162), sum(oct_net#163), sum(nov_net#164), sum(dec_net#165)] - -(50) CometTakeOrderedAndProject -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#10 ASC NULLS FIRST], output=[w_warehouse_name#10,w_warehouse_sq_ft#11,w_city#12,w_county#13,w_state#16,w_country#15,ship_carriers#140,year#141,jan_sales#264,feb_sales#265,mar_sales#266,apr_sales#267,may_sales#268,jun_sales#269,jul_sales#270,aug_sales#271,sep_sales#272,oct_sales#273,nov_sales#274,dec_sales#275,jan_sales_per_sq_foot#276,feb_sales_per_sq_foot#277,mar_sales_per_sq_foot#278,apr_sales_per_sq_foot#279,may_sales_per_sq_foot#280,jun_sales_per_sq_foot#281,jul_sales_per_sq_foot#282,aug_sales_per_sq_foot#283,sep_sales_per_sq_foot#284,oct_sales_per_sq_foot#285,nov_sales_per_sq_foot#286,dec_sales_per_sq_foot#287,jan_net#288,feb_net#289,mar_net#290,apr_net#291,may_net#292,jun_net#293,jul_net#294,aug_net#295,sep_net#296,oct_net#297,nov_net#298,dec_net#299]), [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299], 100, 0, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299] - -(51) CometColumnarToRow [codegen id : 1] -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometFilter (53) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) - - -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(54) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] - -(55) BroadcastExchange -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#78 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/extended.txt deleted file mode 100644 index ca39e001b0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/extended.txt +++ /dev/null @@ -1,70 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - -Comet accelerated 63 out of 66 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index bc6bde24b2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - CometHashAggregate [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,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,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(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum((jan_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((feb_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((mar_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((apr_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((may_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jun_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jul_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((aug_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((sep_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((oct_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((nov_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((dec_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net)] - CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - CometHashAggregate [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] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,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,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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] - CometHashAggregate [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] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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,d_year,sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END)] - CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - CometHashAggregate [d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [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] - 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,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,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_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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 - CometProject [w_state] [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [t_time_sk] #6 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_time] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_time] - CometBroadcastExchange [sm_ship_mode_sk] #7 - CometProject [sm_ship_mode_sk] - CometFilter [sm_ship_mode_sk,sm_carrier] - CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] - CometHashAggregate [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] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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,d_year,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)] - CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #8 - CometHashAggregate [d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [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] - 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,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,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_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_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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year,d_moy] #5 - ReusedExchange [t_time_sk] #6 - ReusedExchange [sm_ship_mode_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/explain.txt deleted file mode 100644 index 1cc7d86902..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (34) -+- * Filter (33) - +- Window (32) - +- WindowGroupLimit (31) - +- * CometColumnarToRow (30) - +- CometSort (29) - +- CometColumnarExchange (28) - +- WindowGroupLimit (27) - +- * Sort (26) - +- * HashAggregate (25) - +- * CometColumnarToRow (24) - +- CometColumnarExchange (23) - +- * HashAggregate (22) - +- * Expand (21) - +- * Project (20) - +- * BroadcastHashJoin Inner BuildRight (19) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) - : +- CometProject (9) - : +- CometFilter (8) - : +- CometNativeScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (18) - +- * CometColumnarToRow (17) - +- CometProject (16) - +- CometFilter (15) - +- CometNativeScan parquet spark_catalog.default.item (14) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(4) ReusedExchange [Reuses operator id: 39] -Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_store_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [s_store_sk#11, s_store_id#12] -Condition : isnotnull(s_store_sk#11) - -(9) CometProject -Input [2]: [s_store_sk#11, s_store_id#12] -Arguments: [s_store_sk#11, s_store_id#13], [s_store_sk#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#12, 16)) AS s_store_id#13] - -(10) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#11, s_store_id#13] - -(11) BroadcastExchange -Input [2]: [s_store_sk#11, s_store_id#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#13] - -(14) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -Condition : isnotnull(i_item_sk#14) - -(16) CometProject -Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -Arguments: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22], [i_item_sk#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#15, 50)) AS i_brand#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#16, 50)) AS i_class#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#17, 50)) AS i_category#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#18, 50)) AS i_product_name#22] - -(17) CometColumnarToRow [codegen id : 3] -Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] - -(18) BroadcastExchange -Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#14] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [10]: [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] - -(21) Expand [codegen id : 4] -Input [10]: [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] -Arguments: [[ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, 0], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, null, 1], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, null, null, 3], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, null, null, null, 7], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, null, null, null, null, 15], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, null, null, null, null, null, 31], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, null, null, null, null, null, null, 63], [ss_quantity#3, ss_sales_price#4, i_category#21, null, null, null, null, null, null, null, 127], [ss_quantity#3, ss_sales_price#4, null, null, null, null, null, null, null, null, 255]], [ss_quantity#3, ss_sales_price#4, i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] - -(22) HashAggregate [codegen id : 4] -Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] -Keys [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] -Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [2]: [sum#32, isEmpty#33] -Results [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, sum#34, isEmpty#35] - -(23) CometColumnarExchange -Input [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, sum#34, isEmpty#35] -Arguments: hashpartitioning(i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometColumnarToRow [codegen id : 5] -Input [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, sum#34, isEmpty#35] - -(25) HashAggregate [codegen id : 5] -Input [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, sum#34, isEmpty#35] -Keys [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] -Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#36] -Results [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#36 AS sumsales#37] - -(26) Sort [codegen id : 5] -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] -Arguments: [i_category#23 ASC NULLS FIRST, sumsales#37 DESC NULLS LAST], false, 0 - -(27) WindowGroupLimit -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] -Arguments: [i_category#23], [sumsales#37 DESC NULLS LAST], rank(sumsales#37), 100, Partial - -(28) CometColumnarExchange -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] -Arguments: hashpartitioning(i_category#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(29) CometSort -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] -Arguments: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37], [i_category#23 ASC NULLS FIRST, sumsales#37 DESC NULLS LAST] - -(30) CometColumnarToRow [codegen id : 6] -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] - -(31) WindowGroupLimit -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] -Arguments: [i_category#23], [sumsales#37 DESC NULLS LAST], rank(sumsales#37), 100, Final - -(32) Window -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] -Arguments: [rank(sumsales#37) windowspecdefinition(i_category#23, sumsales#37 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#38], [i_category#23], [sumsales#37 DESC NULLS LAST] - -(33) Filter [codegen id : 7] -Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37, rk#38] -Condition : (rk#38 <= 100) - -(34) TakeOrderedAndProject -Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37, rk#38] -Arguments: 100, [i_category#23 ASC NULLS FIRST, i_class#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_product_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_qoy#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, s_store_id#30 ASC NULLS FIRST, sumsales#37 ASC NULLS FIRST, rk#38 ASC NULLS FIRST], [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37, rk#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (39) -+- * CometColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometNativeScan parquet spark_catalog.default.date_dim (35) - - -(35) CometNativeScan parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#39, d_year#8, d_moy#9, d_qoy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(36) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#39, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#39) AND (d_month_seq#39 >= 1200)) AND (d_month_seq#39 <= 1211)) AND isnotnull(d_date_sk#7)) - -(37) CometProject -Input [5]: [d_date_sk#7, d_month_seq#39, d_year#8, d_moy#9, d_qoy#10] -Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(38) CometColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(39) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/simplified.txt deleted file mode 100644 index 7c5b24a903..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/simplified.txt +++ /dev/null @@ -1,57 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (7) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (5) - Sort [i_category,sumsales] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [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] [sum,isEmpty,sum,isEmpty] - Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometNativeScan 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/q67.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/explain.txt deleted file mode 100644 index 6330b854bb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (34) -+- * Filter (33) - +- Window (32) - +- WindowGroupLimit (31) - +- * CometColumnarToRow (30) - +- CometSort (29) - +- CometColumnarExchange (28) - +- WindowGroupLimit (27) - +- * CometColumnarToRow (26) - +- CometSort (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExpand (21) - +- CometProject (20) - +- CometBroadcastHashJoin (19) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - +- CometBroadcastExchange (18) - +- CometProject (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_month_seq#8 <= 1211)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(6) CometBroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Right output [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11], [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_store_id#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#12, s_store_id#13] -Condition : isnotnull(s_store_sk#12) - -(11) CometProject -Input [2]: [s_store_sk#12, s_store_id#13] -Arguments: [s_store_sk#12, s_store_id#14], [s_store_sk#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#13, 16)) AS s_store_id#14] - -(12) CometBroadcastExchange -Input [2]: [s_store_sk#12, s_store_id#14] -Arguments: [s_store_sk#12, s_store_id#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] -Right output [2]: [s_store_sk#12, s_store_id#14] -Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_sk#12, s_store_id#14] -Arguments: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14], [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Condition : isnotnull(i_item_sk#15) - -(17) CometProject -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23], [i_item_sk#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#16, 50)) AS i_brand#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#17, 50)) AS i_class#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#18, 50)) AS i_category#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#19, 50)) AS i_product_name#23] - -(18) CometBroadcastExchange -Input [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] -Right output [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight - -(20) CometProject -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] - -(21) CometExpand -Input [10]: [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -Arguments: [[ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 0], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, null, 1], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, null, null, 3], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, null, null, null, 7], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, null, null, null, null, 15], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, null, null, null, null, null, 31], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, null, null, null, null, null, null, 63], [ss_quantity#3, ss_sales_price#4, i_category#22, null, null, null, null, null, null, null, 127], [ss_quantity#3, ss_sales_price#4, null, null, null, null, null, null, null, null, 255]], [ss_quantity#3, ss_sales_price#4, i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] - -(22) CometHashAggregate -Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] -Keys [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] -Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(23) CometExchange -Input [11]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32, sum#33, isEmpty#34] -Arguments: hashpartitioning(i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [11]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32, sum#33, isEmpty#34] -Keys [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] -Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(25) CometSort -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35], [i_category#24 ASC NULLS FIRST, sumsales#35 DESC NULLS LAST] - -(26) CometColumnarToRow [codegen id : 1] -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] - -(27) WindowGroupLimit -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [i_category#24], [sumsales#35 DESC NULLS LAST], rank(sumsales#35), 100, Partial - -(28) CometColumnarExchange -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: hashpartitioning(i_category#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(29) CometSort -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35], [i_category#24 ASC NULLS FIRST, sumsales#35 DESC NULLS LAST] - -(30) CometColumnarToRow [codegen id : 2] -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] - -(31) WindowGroupLimit -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [i_category#24], [sumsales#35 DESC NULLS LAST], rank(sumsales#35), 100, Final - -(32) Window -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [rank(sumsales#35) windowspecdefinition(i_category#24, sumsales#35 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#36], [i_category#24], [sumsales#35 DESC NULLS LAST] - -(33) Filter [codegen id : 3] -Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] -Condition : (rk#36 <= 100) - -(34) TakeOrderedAndProject -Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] -Arguments: 100, [i_category#24 ASC NULLS FIRST, i_class#25 ASC NULLS FIRST, i_brand#26 ASC NULLS FIRST, i_product_name#27 ASC NULLS FIRST, d_year#28 ASC NULLS FIRST, d_qoy#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, s_store_id#31 ASC NULLS FIRST, sumsales#35 ASC NULLS FIRST, rk#36 ASC NULLS FIRST], [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (39) -+- * CometColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (35) - - -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(36) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_month_seq#8 <= 1211)) AND isnotnull(d_date_sk#7)) - -(37) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(38) CometColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(39) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/simplified.txt deleted file mode 100644 index 9069117a5b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,48 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (3) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometHashAggregate [spark_grouping_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 - CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] - 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,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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] 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/q67/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/explain.txt deleted file mode 100644 index 6330b854bb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (34) -+- * Filter (33) - +- Window (32) - +- WindowGroupLimit (31) - +- * CometColumnarToRow (30) - +- CometSort (29) - +- CometColumnarExchange (28) - +- WindowGroupLimit (27) - +- * CometColumnarToRow (26) - +- CometSort (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExpand (21) - +- CometProject (20) - +- CometBroadcastHashJoin (19) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - +- CometBroadcastExchange (18) - +- CometProject (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_month_seq#8 <= 1211)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(6) CometBroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Right output [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11], [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_store_id#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#12, s_store_id#13] -Condition : isnotnull(s_store_sk#12) - -(11) CometProject -Input [2]: [s_store_sk#12, s_store_id#13] -Arguments: [s_store_sk#12, s_store_id#14], [s_store_sk#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#13, 16)) AS s_store_id#14] - -(12) CometBroadcastExchange -Input [2]: [s_store_sk#12, s_store_id#14] -Arguments: [s_store_sk#12, s_store_id#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] -Right output [2]: [s_store_sk#12, s_store_id#14] -Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_sk#12, s_store_id#14] -Arguments: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14], [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Condition : isnotnull(i_item_sk#15) - -(17) CometProject -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23], [i_item_sk#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#16, 50)) AS i_brand#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#17, 50)) AS i_class#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#18, 50)) AS i_category#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#19, 50)) AS i_product_name#23] - -(18) CometBroadcastExchange -Input [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] -Right output [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight - -(20) CometProject -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] - -(21) CometExpand -Input [10]: [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -Arguments: [[ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 0], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, null, 1], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, null, null, 3], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, null, null, null, 7], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, null, null, null, null, 15], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, null, null, null, null, null, 31], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, null, null, null, null, null, null, 63], [ss_quantity#3, ss_sales_price#4, i_category#22, null, null, null, null, null, null, null, 127], [ss_quantity#3, ss_sales_price#4, null, null, null, null, null, null, null, null, 255]], [ss_quantity#3, ss_sales_price#4, i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] - -(22) CometHashAggregate -Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] -Keys [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] -Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(23) CometExchange -Input [11]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32, sum#33, isEmpty#34] -Arguments: hashpartitioning(i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [11]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32, sum#33, isEmpty#34] -Keys [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] -Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(25) CometSort -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35], [i_category#24 ASC NULLS FIRST, sumsales#35 DESC NULLS LAST] - -(26) CometColumnarToRow [codegen id : 1] -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] - -(27) WindowGroupLimit -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [i_category#24], [sumsales#35 DESC NULLS LAST], rank(sumsales#35), 100, Partial - -(28) CometColumnarExchange -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: hashpartitioning(i_category#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(29) CometSort -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35], [i_category#24 ASC NULLS FIRST, sumsales#35 DESC NULLS LAST] - -(30) CometColumnarToRow [codegen id : 2] -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] - -(31) WindowGroupLimit -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [i_category#24], [sumsales#35 DESC NULLS LAST], rank(sumsales#35), 100, Final - -(32) Window -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [rank(sumsales#35) windowspecdefinition(i_category#24, sumsales#35 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#36], [i_category#24], [sumsales#35 DESC NULLS LAST] - -(33) Filter [codegen id : 3] -Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] -Condition : (rk#36 <= 100) - -(34) TakeOrderedAndProject -Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] -Arguments: 100, [i_category#24 ASC NULLS FIRST, i_class#25 ASC NULLS FIRST, i_brand#26 ASC NULLS FIRST, i_product_name#27 ASC NULLS FIRST, d_year#28 ASC NULLS FIRST, d_qoy#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, s_store_id#31 ASC NULLS FIRST, sumsales#35 ASC NULLS FIRST, rk#36 ASC NULLS FIRST], [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (39) -+- * CometColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (35) - - -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(36) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_month_seq#8 <= 1211)) AND isnotnull(d_date_sk#7)) - -(37) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(38) CometColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(39) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/extended.txt deleted file mode 100644 index 7504243c7a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 30 out of 37 eligible operators (81%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 9069117a5b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt +++ /dev/null @@ -1,48 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (3) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometHashAggregate [spark_grouping_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 - CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] - 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,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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/explain.txt deleted file mode 100644 index 1e2350c06a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/explain.txt +++ /dev/null @@ -1,267 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (41) -+- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * HashAggregate (30) - : : +- * CometColumnarToRow (29) - : : +- CometColumnarExchange (28) - : : +- * HashAggregate (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Project (6) - : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : :- * Filter (3) - : : : : : : +- * ColumnarToRow (2) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : +- ReusedExchange (4) - : : : : +- BroadcastExchange (11) - : : : : +- * CometColumnarToRow (10) - : : : : +- CometProject (9) - : : : : +- CometFilter (8) - : : : : +- CometNativeScan parquet spark_catalog.default.store (7) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) - : : +- BroadcastExchange (24) - : : +- * CometColumnarToRow (23) - : : +- CometFilter (22) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (21) - : +- BroadcastExchange (35) - : +- * CometColumnarToRow (34) - : +- CometProject (33) - : +- CometFilter (32) - : +- CometNativeScan parquet spark_catalog.default.customer (31) - +- ReusedExchange (38) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] - -(3) Filter [codegen id : 5] -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 46] -Output [1]: [d_date_sk#11] - -(5) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 5] -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Input [10]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9, d_date_sk#11] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_city#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [s_store_sk#12, s_city#13] -Condition : (s_city#13 IN (Midway,Fairview) AND isnotnull(s_store_sk#12)) - -(9) CometProject -Input [2]: [s_store_sk#12, s_city#13] -Arguments: [s_store_sk#12], [s_store_sk#12] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#12] - -(11) BroadcastExchange -Input [1]: [s_store_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#12] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, s_store_sk#12] - -(14) CometNativeScan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Condition : (((hd_dep_count#15 = 4) OR (hd_vehicle_count#16 = 3)) AND isnotnull(hd_demo_sk#14)) - -(16) CometProject -Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Arguments: [hd_demo_sk#14], [hd_demo_sk#14] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#14] - -(18) BroadcastExchange -Input [1]: [hd_demo_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#14] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, hd_demo_sk#14] - -(21) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#17, ca_city#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [ca_address_sk#17, ca_city#18] -Condition : (isnotnull(ca_address_sk#17) AND isnotnull(ca_city#18)) - -(23) CometColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#17, ca_city#18] - -(24) BroadcastExchange -Input [2]: [ca_address_sk#17, ca_city#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#17] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 5] -Output [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#18] -Input [8]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_address_sk#17, ca_city#18] - -(27) HashAggregate [codegen id : 5] -Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#18] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#6)), partial_sum(UnscaledValue(ss_ext_list_price#7)), partial_sum(UnscaledValue(ss_ext_tax#8))] -Aggregate Attributes [3]: [sum#19, sum#20, sum#21] -Results [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] - -(28) CometColumnarExchange -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(29) CometColumnarToRow [codegen id : 8] -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] - -(30) HashAggregate [codegen id : 8] -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#6)), sum(UnscaledValue(ss_ext_list_price#7)), sum(UnscaledValue(ss_ext_tax#8))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#6))#25, sum(UnscaledValue(ss_ext_list_price#7))#26, sum(UnscaledValue(ss_ext_tax#8))#27] -Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#18 AS bought_city#28, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#25,17,2) AS extended_price#29, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#7))#26,17,2) AS list_price#30, MakeDecimal(sum(UnscaledValue(ss_ext_tax#8))#27,17,2) AS extended_tax#31] - -(31) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(32) CometFilter -Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#33)) - -(33) CometProject -Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] -Arguments: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37], [c_customer_sk#32, c_current_addr_sk#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#34, 20)) AS c_first_name#36, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#35, 30)) AS c_last_name#37] - -(34) CometColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37] - -(35) BroadcastExchange -Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(36) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#32] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 8] -Output [8]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#36, c_last_name#37] -Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37] - -(38) ReusedExchange [Reuses operator id: 24] -Output [2]: [ca_address_sk#38, ca_city#39] - -(39) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [c_current_addr_sk#33] -Right keys [1]: [ca_address_sk#38] -Join type: Inner -Join condition: NOT (ca_city#39 = bought_city#28) - -(40) Project [codegen id : 8] -Output [8]: [c_last_name#37, c_first_name#36, ca_city#39, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] -Input [10]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#36, c_last_name#37, ca_address_sk#38, ca_city#39] - -(41) TakeOrderedAndProject -Input [8]: [c_last_name#37, c_first_name#36, ca_city#39, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] -Arguments: 100, [c_last_name#37 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#39, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometNativeScan parquet spark_catalog.default.date_dim (42) - - -(42) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#40, d_dom#41] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [3]: [d_date_sk#11, d_year#40, d_dom#41] -Condition : ((((isnotnull(d_dom#41) AND (d_dom#41 >= 1)) AND (d_dom#41 <= 2)) AND d_year#40 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) - -(44) CometProject -Input [3]: [d_date_sk#11, d_year#40, d_dom#41] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(45) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(46) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/simplified.txt deleted file mode 100644 index 05b883b1e6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/simplified.txt +++ /dev/null @@ -1,67 +0,0 @@ -TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_city,extended_price,extended_tax,list_price] - WholeStageCodegen (8) - Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] - Project [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax)),bought_city,extended_price,list_price,extended_tax,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - WholeStageCodegen (5) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [sum,sum,sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_city] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - InputAdapter - ReusedExchange [ca_address_sk,ca_city] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/explain.txt deleted file mode 100644 index 02afc31b71..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,256 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (39) -+- CometTakeOrderedAndProject (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (34) - : +- CometBroadcastHashJoin (33) - : :- CometHashAggregate (28) - : : +- CometExchange (27) - : : +- CometHashAggregate (26) - : : +- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (20) - : : : +- CometBroadcastHashJoin (19) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometProject (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : : : +- CometBroadcastExchange (18) - : : : +- CometProject (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - : : +- CometBroadcastExchange (23) - : : +- CometFilter (22) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (21) - : +- CometBroadcastExchange (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (29) - +- ReusedExchange (35) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_dom#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AND d_year#12 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) - -(5) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(7) CometBroadcastHashJoin -Left output [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(8) CometProject -Input [10]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9, d_date_sk#11] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_city#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#14, s_city#15] -Condition : (s_city#15 IN (Midway,Fairview) AND isnotnull(s_store_sk#14)) - -(11) CometProject -Input [2]: [s_store_sk#14, s_city#15] -Arguments: [s_store_sk#14], [s_store_sk#14] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#14] -Arguments: [s_store_sk#14] - -(13) CometBroadcastHashJoin -Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Right output [1]: [s_store_sk#14] -Arguments: [ss_store_sk#4], [s_store_sk#14], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, s_store_sk#14] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Condition : (((hd_dep_count#17 = 4) OR (hd_vehicle_count#18 = 3)) AND isnotnull(hd_demo_sk#16)) - -(17) CometProject -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Arguments: [hd_demo_sk#16], [hd_demo_sk#16] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#16] -Arguments: [hd_demo_sk#16] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Right output [1]: [hd_demo_sk#16] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, hd_demo_sk#16] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_city#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [ca_address_sk#19, ca_city#20] -Condition : (isnotnull(ca_address_sk#19) AND isnotnull(ca_city#20)) - -(23) CometBroadcastExchange -Input [2]: [ca_address_sk#19, ca_city#20] -Arguments: [ca_address_sk#19, ca_city#20] - -(24) CometBroadcastHashJoin -Left output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Right output [2]: [ca_address_sk#19, ca_city#20] -Arguments: [ss_addr_sk#3], [ca_address_sk#19], Inner, BuildRight - -(25) CometProject -Input [8]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_address_sk#19, ca_city#20] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20] - -(26) CometHashAggregate -Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#6)), partial_sum(UnscaledValue(ss_ext_list_price#7)), partial_sum(UnscaledValue(ss_ext_tax#8))] - -(27) CometExchange -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#21, sum#22, sum#23] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#21, sum#22, sum#23] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#6)), sum(UnscaledValue(ss_ext_list_price#7)), sum(UnscaledValue(ss_ext_tax#8))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] -Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_current_addr_sk#25)) - -(31) CometProject -Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29], [c_customer_sk#24, c_current_addr_sk#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#26, 20)) AS c_first_name#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#27, 30)) AS c_last_name#29] - -(32) CometBroadcastExchange -Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Arguments: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] - -(33) CometBroadcastHashJoin -Left output [6]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#30, extended_price#31, list_price#32, extended_tax#33] -Right output [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Arguments: [ss_customer_sk#1], [c_customer_sk#24], Inner, BuildRight - -(34) CometProject -Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Arguments: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29], [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29] - -(35) ReusedExchange [Reuses operator id: 23] -Output [2]: [ca_address_sk#34, ca_city#35] - -(36) CometBroadcastHashJoin -Left output [8]: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Right output [2]: [ca_address_sk#34, ca_city#35] -Arguments: [c_current_addr_sk#25], [ca_address_sk#34], Inner, NOT (ca_city#35 = bought_city#30), BuildRight - -(37) CometProject -Input [10]: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29, ca_address_sk#34, ca_city#35] -Arguments: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32], [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] - -(38) CometTakeOrderedAndProject -Input [8]: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#29 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#29,c_first_name#28,ca_city#35,bought_city#30,ss_ticket_number#5,extended_price#31,extended_tax#33,list_price#32]), [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32], 100, 0, [c_last_name#29 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] - -(39) CometColumnarToRow [codegen id : 1] -Input [8]: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) - - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_dom#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(41) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AND d_year#12 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) - -(42) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(43) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(44) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/simplified.txt deleted file mode 100644 index fdddd82de0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] - CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] - CometBroadcastHashJoin [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] - CometProject [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometHashAggregate [ss_addr_sk,sum,sum,sum] [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,ca_city,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax))] - CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - CometHashAggregate [ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] - 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_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_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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [ca_address_sk,ca_city] #6 - CometFilter [ca_address_sk,ca_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - ReusedExchange [ca_address_sk,ca_city] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/explain.txt deleted file mode 100644 index 02afc31b71..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/explain.txt +++ /dev/null @@ -1,256 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (39) -+- CometTakeOrderedAndProject (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (34) - : +- CometBroadcastHashJoin (33) - : :- CometHashAggregate (28) - : : +- CometExchange (27) - : : +- CometHashAggregate (26) - : : +- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (20) - : : : +- CometBroadcastHashJoin (19) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometProject (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : : : +- CometBroadcastExchange (18) - : : : +- CometProject (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - : : +- CometBroadcastExchange (23) - : : +- CometFilter (22) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (21) - : +- CometBroadcastExchange (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (29) - +- ReusedExchange (35) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_dom#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AND d_year#12 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) - -(5) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(7) CometBroadcastHashJoin -Left output [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(8) CometProject -Input [10]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9, d_date_sk#11] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_city#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#14, s_city#15] -Condition : (s_city#15 IN (Midway,Fairview) AND isnotnull(s_store_sk#14)) - -(11) CometProject -Input [2]: [s_store_sk#14, s_city#15] -Arguments: [s_store_sk#14], [s_store_sk#14] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#14] -Arguments: [s_store_sk#14] - -(13) CometBroadcastHashJoin -Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Right output [1]: [s_store_sk#14] -Arguments: [ss_store_sk#4], [s_store_sk#14], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, s_store_sk#14] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Condition : (((hd_dep_count#17 = 4) OR (hd_vehicle_count#18 = 3)) AND isnotnull(hd_demo_sk#16)) - -(17) CometProject -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Arguments: [hd_demo_sk#16], [hd_demo_sk#16] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#16] -Arguments: [hd_demo_sk#16] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Right output [1]: [hd_demo_sk#16] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, hd_demo_sk#16] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_city#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [ca_address_sk#19, ca_city#20] -Condition : (isnotnull(ca_address_sk#19) AND isnotnull(ca_city#20)) - -(23) CometBroadcastExchange -Input [2]: [ca_address_sk#19, ca_city#20] -Arguments: [ca_address_sk#19, ca_city#20] - -(24) CometBroadcastHashJoin -Left output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Right output [2]: [ca_address_sk#19, ca_city#20] -Arguments: [ss_addr_sk#3], [ca_address_sk#19], Inner, BuildRight - -(25) CometProject -Input [8]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_address_sk#19, ca_city#20] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20] - -(26) CometHashAggregate -Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#6)), partial_sum(UnscaledValue(ss_ext_list_price#7)), partial_sum(UnscaledValue(ss_ext_tax#8))] - -(27) CometExchange -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#21, sum#22, sum#23] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#21, sum#22, sum#23] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#6)), sum(UnscaledValue(ss_ext_list_price#7)), sum(UnscaledValue(ss_ext_tax#8))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] -Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_current_addr_sk#25)) - -(31) CometProject -Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29], [c_customer_sk#24, c_current_addr_sk#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#26, 20)) AS c_first_name#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#27, 30)) AS c_last_name#29] - -(32) CometBroadcastExchange -Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Arguments: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] - -(33) CometBroadcastHashJoin -Left output [6]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#30, extended_price#31, list_price#32, extended_tax#33] -Right output [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Arguments: [ss_customer_sk#1], [c_customer_sk#24], Inner, BuildRight - -(34) CometProject -Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Arguments: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29], [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29] - -(35) ReusedExchange [Reuses operator id: 23] -Output [2]: [ca_address_sk#34, ca_city#35] - -(36) CometBroadcastHashJoin -Left output [8]: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Right output [2]: [ca_address_sk#34, ca_city#35] -Arguments: [c_current_addr_sk#25], [ca_address_sk#34], Inner, NOT (ca_city#35 = bought_city#30), BuildRight - -(37) CometProject -Input [10]: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29, ca_address_sk#34, ca_city#35] -Arguments: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32], [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] - -(38) CometTakeOrderedAndProject -Input [8]: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#29 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#29,c_first_name#28,ca_city#35,bought_city#30,ss_ticket_number#5,extended_price#31,extended_tax#33,list_price#32]), [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32], 100, 0, [c_last_name#29 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] - -(39) CometColumnarToRow [codegen id : 1] -Input [8]: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) - - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_dom#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(41) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AND d_year#12 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) - -(42) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(43) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(44) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/extended.txt deleted file mode 100644 index c842000e11..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/extended.txt +++ /dev/null @@ -1,49 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index fdddd82de0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] - CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] - CometBroadcastHashJoin [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] - CometProject [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometHashAggregate [ss_addr_sk,sum,sum,sum] [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,ca_city,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax))] - CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - CometHashAggregate [ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] - 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_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_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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [ca_address_sk,ca_city] #6 - CometFilter [ca_address_sk,ca_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - ReusedExchange [ca_address_sk,ca_city] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/explain.txt deleted file mode 100644 index cd1eacdb81..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/explain.txt +++ /dev/null @@ -1,290 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (44) -+- * HashAggregate (43) - +- * CometColumnarToRow (42) - +- CometColumnarExchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (25) - : : +- * BroadcastHashJoin LeftAnti BuildRight (24) - : : :- * BroadcastHashJoin LeftAnti BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (30) - : +- * CometColumnarToRow (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometNativeScan parquet spark_catalog.default.customer_address (26) - +- BroadcastExchange (37) - +- * CometColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometNativeScan parquet spark_catalog.default.customer_demographics (33) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(4) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] - -(6) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#7] - -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#4] -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] - -(9) BroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#4] -Join type: LeftSemi -Join condition: None - -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#6)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] - -(13) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#10] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#8] -Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#10] - -(16) BroadcastExchange -Input [1]: [ws_bill_customer_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(17) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#8] -Join type: LeftAnti -Join condition: None - -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#11, cs_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#12), dynamicpruningexpression(cs_sold_date_sk#12 IN dynamicpruning#6)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#11, cs_sold_date_sk#12] - -(20) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#13] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#12] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#11] -Input [3]: [cs_ship_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13] - -(23) BroadcastExchange -Input [1]: [cs_ship_customer_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_ship_customer_sk#11] -Join type: LeftAnti -Join condition: None - -(25) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(26) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#14, ca_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [ca_address_sk#14, ca_state#15] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#15, 2)) IN (KY,GA,NM) AND isnotnull(ca_address_sk#14)) - -(28) CometProject -Input [2]: [ca_address_sk#14, ca_state#15] -Arguments: [ca_address_sk#14], [ca_address_sk#14] - -(29) CometColumnarToRow [codegen id : 7] -Input [1]: [ca_address_sk#14] - -(30) BroadcastExchange -Input [1]: [ca_address_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#14] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 9] -Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#14] - -(33) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#16, cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(34) CometFilter -Input [6]: [cd_demo_sk#16, cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21] -Condition : isnotnull(cd_demo_sk#16) - -(35) CometProject -Input [6]: [cd_demo_sk#16, cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21] -Arguments: [cd_demo_sk#16, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25], [cd_demo_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#17, 1)) AS cd_gender#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#18, 1)) AS cd_marital_status#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#19, 20)) AS cd_education_status#24, cd_purchase_estimate#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_credit_rating#21, 10)) AS cd_credit_rating#25] - -(36) CometColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#16, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] - -(37) BroadcastExchange -Input [6]: [cd_demo_sk#16, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(38) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#16] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 9] -Output [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#16, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] - -(40) HashAggregate [codegen id : 9] -Input [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] -Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#26] -Results [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25, count#27] - -(41) CometColumnarExchange -Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25, count#27] -Arguments: hashpartitioning(cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(42) CometColumnarToRow [codegen id : 10] -Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25, count#27] - -(43) HashAggregate [codegen id : 10] -Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25, count#27] -Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#20, cd_credit_rating#25] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#28] -Results [8]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, count(1)#28 AS cnt1#29, cd_purchase_estimate#20, count(1)#28 AS cnt2#30, cd_credit_rating#25, count(1)#28 AS cnt3#31] - -(44) TakeOrderedAndProject -Input [8]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#29, cd_purchase_estimate#20, cnt2#30, cd_credit_rating#25, cnt3#31] -Arguments: 100, [cd_gender#22 ASC NULLS FIRST, cd_marital_status#23 ASC NULLS FIRST, cd_education_status#24 ASC NULLS FIRST, cd_purchase_estimate#20 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST], [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#29, cd_purchase_estimate#20, cnt2#30, cd_credit_rating#25, cnt3#31] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometNativeScan parquet spark_catalog.default.date_dim (45) - - -(45) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#32, d_moy#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [3]: [d_date_sk#7, d_year#32, d_moy#33] -Condition : (((((isnotnull(d_year#32) AND isnotnull(d_moy#33)) AND (d_year#32 = 2001)) AND (d_moy#33 >= 4)) AND (d_moy#33 <= 6)) AND isnotnull(d_date_sk#7)) - -(47) CometProject -Input [3]: [d_date_sk#7, d_year#32, d_moy#33] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(49) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#12 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/simplified.txt deleted file mode 100644 index f8868e38be..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/simplified.txt +++ /dev/null @@ -1,76 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (10) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (9) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometNativeScan 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/q69.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/explain.txt deleted file mode 100644 index 96075ed3ec..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,301 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (46) -+- * HashAggregate (45) - +- * CometColumnarToRow (44) - +- CometColumnarExchange (43) - +- * HashAggregate (42) - +- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (34) - : +- * BroadcastHashJoin Inner BuildRight (33) - : :- * Project (27) - : : +- * BroadcastHashJoin LeftAnti BuildRight (26) - : : :- * BroadcastHashJoin LeftAnti BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (32) - : +- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (28) - +- BroadcastExchange (39) - +- * CometColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (35) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2001)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 6)) AND isnotnull(d_date_sk#7)) - -(6) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#4], [ss_customer_sk#4] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: [ss_customer_sk#4] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ss_customer_sk#4] -Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#13] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] -Arguments: [ws_bill_customer_sk#10], [ws_bill_customer_sk#10] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#10] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#10] -Join type: LeftAnti -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#17] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -Right output [1]: [d_date_sk#17] -Arguments: [cs_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] -Arguments: [cs_ship_customer_sk#14], [cs_ship_customer_sk#14] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#14] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_ship_customer_sk#14] -Join type: LeftAnti -Join condition: None - -(27) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_state#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(29) CometFilter -Input [2]: [ca_address_sk#18, ca_state#19] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#19, 2)) IN (KY,GA,NM) AND isnotnull(ca_address_sk#18)) - -(30) CometProject -Input [2]: [ca_address_sk#18, ca_state#19] -Arguments: [ca_address_sk#18], [ca_address_sk#18] - -(31) CometColumnarToRow [codegen id : 3] -Input [1]: [ca_address_sk#18] - -(32) BroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(33) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#18] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 5] -Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] - -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(36) CometFilter -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -Condition : isnotnull(cd_demo_sk#20) - -(37) CometProject -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -Arguments: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29], [cd_demo_sk#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#21, 1)) AS cd_gender#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#22, 1)) AS cd_marital_status#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#23, 20)) AS cd_education_status#28, cd_purchase_estimate#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_credit_rating#25, 10)) AS cd_credit_rating#29] - -(38) CometColumnarToRow [codegen id : 4] -Input [6]: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] - -(39) BroadcastExchange -Input [6]: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(40) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 5] -Output [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] - -(42) HashAggregate [codegen id : 5] -Input [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#30] -Results [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] - -(43) CometColumnarExchange -Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] -Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(44) CometColumnarToRow [codegen id : 6] -Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] - -(45) HashAggregate [codegen id : 6] -Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] -Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#32] -Results [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#32 AS cnt1#33, cd_purchase_estimate#24, count(1)#32 AS cnt2#34, cd_credit_rating#29, count(1)#32 AS cnt3#35] - -(46) TakeOrderedAndProject -Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#33, cd_purchase_estimate#24, cnt2#34, cd_credit_rating#29, cnt3#35] -Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#33, cd_purchase_estimate#24, cnt2#34, cd_credit_rating#29, cnt3#35] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (51) -+- * CometColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) - - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] -ReadSchema: struct - -(48) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2001)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 6)) AND isnotnull(d_date_sk#7)) - -(49) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(50) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(51) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/simplified.txt deleted file mode 100644 index 68bf32c40b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,72 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (6) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (5) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometScan [native_iceberg_compat] 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/q69/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/explain.txt deleted file mode 100644 index 96075ed3ec..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/explain.txt +++ /dev/null @@ -1,301 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (46) -+- * HashAggregate (45) - +- * CometColumnarToRow (44) - +- CometColumnarExchange (43) - +- * HashAggregate (42) - +- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (34) - : +- * BroadcastHashJoin Inner BuildRight (33) - : :- * Project (27) - : : +- * BroadcastHashJoin LeftAnti BuildRight (26) - : : :- * BroadcastHashJoin LeftAnti BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (32) - : +- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (28) - +- BroadcastExchange (39) - +- * CometColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (35) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2001)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 6)) AND isnotnull(d_date_sk#7)) - -(6) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#4], [ss_customer_sk#4] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: [ss_customer_sk#4] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ss_customer_sk#4] -Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#13] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] -Arguments: [ws_bill_customer_sk#10], [ws_bill_customer_sk#10] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#10] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#10] -Join type: LeftAnti -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#17] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -Right output [1]: [d_date_sk#17] -Arguments: [cs_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] -Arguments: [cs_ship_customer_sk#14], [cs_ship_customer_sk#14] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#14] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_ship_customer_sk#14] -Join type: LeftAnti -Join condition: None - -(27) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_state#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(29) CometFilter -Input [2]: [ca_address_sk#18, ca_state#19] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#19, 2)) IN (KY,GA,NM) AND isnotnull(ca_address_sk#18)) - -(30) CometProject -Input [2]: [ca_address_sk#18, ca_state#19] -Arguments: [ca_address_sk#18], [ca_address_sk#18] - -(31) CometColumnarToRow [codegen id : 3] -Input [1]: [ca_address_sk#18] - -(32) BroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(33) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#18] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 5] -Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] - -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(36) CometFilter -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -Condition : isnotnull(cd_demo_sk#20) - -(37) CometProject -Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25] -Arguments: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29], [cd_demo_sk#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#21, 1)) AS cd_gender#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#22, 1)) AS cd_marital_status#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#23, 20)) AS cd_education_status#28, cd_purchase_estimate#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_credit_rating#25, 10)) AS cd_credit_rating#29] - -(38) CometColumnarToRow [codegen id : 4] -Input [6]: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] - -(39) BroadcastExchange -Input [6]: [cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(40) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 5] -Output [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#20, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] - -(42) HashAggregate [codegen id : 5] -Input [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#30] -Results [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] - -(43) CometColumnarExchange -Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] -Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(44) CometColumnarToRow [codegen id : 6] -Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] - -(45) HashAggregate [codegen id : 6] -Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29, count#31] -Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#24, cd_credit_rating#29] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#32] -Results [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#32 AS cnt1#33, cd_purchase_estimate#24, count(1)#32 AS cnt2#34, cd_credit_rating#29, count(1)#32 AS cnt3#35] - -(46) TakeOrderedAndProject -Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#33, cd_purchase_estimate#24, cnt2#34, cd_credit_rating#29, cnt3#35] -Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#33, cd_purchase_estimate#24, cnt2#34, cd_credit_rating#29, cnt3#35] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (51) -+- * CometColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) - - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] -ReadSchema: struct - -(48) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2001)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 6)) AND isnotnull(d_date_sk#7)) - -(49) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(50) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(51) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/extended.txt deleted file mode 100644 index b4fbb455ef..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/extended.txt +++ /dev/null @@ -1,62 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 68bf32c40b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt +++ /dev/null @@ -1,72 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (6) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (5) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/explain.txt deleted file mode 100644 index 627eb9e23d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- * CometColumnarToRow (30) - +- CometColumnarExchange (29) - +- * HashAggregate (28) - +- * Project (27) - +- * BroadcastHashJoin Inner BuildRight (26) - :- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.item (14) - +- BroadcastExchange (25) - +- * CometColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometNativeScan parquet spark_catalog.default.promotion (21) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(3) Filter [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(4) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#11, 1)) = M)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#12, 1)) = S)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#13, 20)) = College )) AND isnotnull(cd_demo_sk#10)) - -(6) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [cd_demo_sk#10] - -(8) BroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 5] -Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] - -(11) ReusedExchange [Reuses operator id: 37] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] - -(14) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#15, i_item_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [i_item_sk#15, i_item_id#16] -Condition : isnotnull(i_item_sk#15) - -(16) CometProject -Input [2]: [i_item_sk#15, i_item_id#16] -Arguments: [i_item_sk#15, i_item_id#17], [i_item_sk#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#16, 16)) AS i_item_id#17] - -(17) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#15, i_item_id#17] - -(18) BroadcastExchange -Input [2]: [i_item_sk#15, i_item_id#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [6]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#15, i_item_id#17] - -(21) CometNativeScan parquet spark_catalog.default.promotion -Output [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] -Condition : (((static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_email#19, 1)) = N) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_event#20, 1)) = N)) AND isnotnull(p_promo_sk#18)) - -(23) CometProject -Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] -Arguments: [p_promo_sk#18], [p_promo_sk#18] - -(24) CometColumnarToRow [codegen id : 4] -Input [1]: [p_promo_sk#18] - -(25) BroadcastExchange -Input [1]: [p_promo_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_promo_sk#3] -Right keys [1]: [p_promo_sk#18] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 5] -Output [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] -Input [7]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17, p_promo_sk#18] - -(28) HashAggregate [codegen id : 5] -Input [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] -Keys [1]: [i_item_id#17] -Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [8]: [sum#21, count#22, sum#23, count#24, sum#25, count#26, sum#27, count#28] -Results [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] - -(29) CometColumnarExchange -Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] -Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 6] -Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] - -(31) HashAggregate [codegen id : 6] -Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] -Keys [1]: [i_item_id#17] -Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [4]: [avg(ss_quantity#4)#37, avg(UnscaledValue(ss_list_price#5))#38, avg(UnscaledValue(ss_coupon_amt#7))#39, avg(UnscaledValue(ss_sales_price#6))#40] -Results [5]: [i_item_id#17, avg(ss_quantity#4)#37 AS agg1#41, cast((avg(UnscaledValue(ss_list_price#5))#38 / 100.0) as decimal(11,6)) AS agg2#42, cast((avg(UnscaledValue(ss_coupon_amt#7))#39 / 100.0) as decimal(11,6)) AS agg3#43, cast((avg(UnscaledValue(ss_sales_price#6))#40 / 100.0) as decimal(11,6)) AS agg4#44] - -(32) TakeOrderedAndProject -Input [5]: [i_item_id#17, agg1#41, agg2#42, agg3#43, agg4#44] -Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#41, agg2#42, agg3#43, agg4#44] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometNativeScan parquet spark_catalog.default.date_dim (33) - - -(33) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [d_date_sk#14, d_year#45] -Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2000)) AND isnotnull(d_date_sk#14)) - -(35) CometProject -Input [2]: [d_date_sk#14, d_year#45] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(37) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/simplified.txt deleted file mode 100644 index bb670b4a73..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) - HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Filter [ss_cdemo_sk,ss_item_sk,ss_promo_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometNativeScan 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/q7.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/explain.txt deleted file mode 100644 index b1e3b154ac..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,212 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (31) -+- CometTakeOrderedAndProject (30) - +- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#11, 1)) = M)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#12, 1)) = S)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#13, 20)) = College )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#16, i_item_id#17] -Condition : isnotnull(i_item_sk#16) - -(17) CometProject -Input [2]: [i_item_sk#16, i_item_id#17] -Arguments: [i_item_sk#16, i_item_id#18], [i_item_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#17, 16)) AS i_item_id#18] - -(18) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_id#18] -Arguments: [i_item_sk#16, i_item_id#18] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Right output [2]: [i_item_sk#16, i_item_id#18] -Arguments: [ss_item_sk#1], [i_item_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#16, i_item_id#18] -Arguments: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18], [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Condition : (((static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_email#20, 1)) = N) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_event#21, 1)) = N)) AND isnotnull(p_promo_sk#19)) - -(23) CometProject -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Arguments: [p_promo_sk#19], [p_promo_sk#19] - -(24) CometBroadcastExchange -Input [1]: [p_promo_sk#19] -Arguments: [p_promo_sk#19] - -(25) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] -Right output [1]: [p_promo_sk#19] -Arguments: [ss_promo_sk#3], [p_promo_sk#19], Inner, BuildRight - -(26) CometProject -Input [7]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18, p_promo_sk#19] -Arguments: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] - -(27) CometHashAggregate -Input [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] - -(28) CometExchange -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Keys [1]: [i_item_id#18] -Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] - -(30) CometTakeOrderedAndProject -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#18 ASC NULLS FIRST], output=[i_item_id#18,agg1#30,agg2#31,agg3#32,agg4#33]), [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33], 100, 0, [i_item_id#18 ASC NULLS FIRST], [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(34) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(36) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/simplified.txt deleted file mode 100644 index a15bcd5c0d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,agg1,agg2,agg3,agg4,avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price))] - CometExchange [i_item_id] #1 - CometHashAggregate [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,sum,count,sum,count,sum,count,sum,count] - CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - 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,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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [p_promo_sk] #6 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometScan [native_iceberg_compat] 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/q7/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/explain.txt deleted file mode 100644 index b1e3b154ac..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/explain.txt +++ /dev/null @@ -1,212 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (31) -+- CometTakeOrderedAndProject (30) - +- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#11, 1)) = M)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#12, 1)) = S)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#13, 20)) = College )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#16, i_item_id#17] -Condition : isnotnull(i_item_sk#16) - -(17) CometProject -Input [2]: [i_item_sk#16, i_item_id#17] -Arguments: [i_item_sk#16, i_item_id#18], [i_item_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#17, 16)) AS i_item_id#18] - -(18) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_id#18] -Arguments: [i_item_sk#16, i_item_id#18] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Right output [2]: [i_item_sk#16, i_item_id#18] -Arguments: [ss_item_sk#1], [i_item_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#16, i_item_id#18] -Arguments: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18], [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Condition : (((static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_email#20, 1)) = N) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_event#21, 1)) = N)) AND isnotnull(p_promo_sk#19)) - -(23) CometProject -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Arguments: [p_promo_sk#19], [p_promo_sk#19] - -(24) CometBroadcastExchange -Input [1]: [p_promo_sk#19] -Arguments: [p_promo_sk#19] - -(25) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] -Right output [1]: [p_promo_sk#19] -Arguments: [ss_promo_sk#3], [p_promo_sk#19], Inner, BuildRight - -(26) CometProject -Input [7]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18, p_promo_sk#19] -Arguments: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] - -(27) CometHashAggregate -Input [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] - -(28) CometExchange -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Keys [1]: [i_item_id#18] -Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] - -(30) CometTakeOrderedAndProject -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#18 ASC NULLS FIRST], output=[i_item_id#18,agg1#30,agg2#31,agg3#32,agg4#33]), [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33], 100, 0, [i_item_id#18 ASC NULLS FIRST], [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(34) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(36) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/extended.txt deleted file mode 100644 index 655f651a41..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/extended.txt +++ /dev/null @@ -1,39 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index a15bcd5c0d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,agg1,agg2,agg3,agg4,avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price))] - CometExchange [i_item_id] #1 - CometHashAggregate [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,sum,count,sum,count,sum,count,sum,count] - CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - 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,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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [p_promo_sk] #6 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/explain.txt deleted file mode 100644 index 071f670c73..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/explain.txt +++ /dev/null @@ -1,305 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- CometColumnarExchange (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Expand (38) - +- * Project (37) - +- * BroadcastHashJoin Inner BuildRight (36) - :- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.store_sales (1) - : +- ReusedExchange (4) - +- BroadcastExchange (35) - +- * Project (34) - +- * BroadcastHashJoin LeftSemi BuildRight (33) - :- * CometColumnarToRow (9) - : +- CometFilter (8) - : +- CometNativeScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (32) - +- * Project (31) - +- * Filter (30) - +- Window (29) - +- WindowGroupLimit (28) - +- * Sort (27) - +- * HashAggregate (26) - +- * CometColumnarToRow (25) - +- CometColumnarExchange (24) - +- * HashAggregate (23) - +- * Project (22) - +- * BroadcastHashJoin Inner BuildRight (21) - :- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet spark_catalog.default.store_sales (10) - : +- BroadcastExchange (17) - : +- * CometColumnarToRow (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometNativeScan parquet spark_catalog.default.store (13) - +- ReusedExchange (20) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 8] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] - -(3) Filter [codegen id : 8] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(4) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#5] - -(5) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 8] -Output [2]: [ss_store_sk#1, ss_net_profit#2] -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#6, s_county#7, s_state#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Condition : isnotnull(s_store_sk#6) - -(9) CometColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#6, s_county#7, s_state#8] - -(10) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] - -(12) Filter [codegen id : 4] -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_store_sk#9) - -(13) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_state#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [s_store_sk#12, s_state#13] -Condition : isnotnull(s_store_sk#12) - -(15) CometProject -Input [2]: [s_store_sk#12, s_state#13] -Arguments: [s_store_sk#12, s_state#14], [s_store_sk#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#13, 2)) AS s_state#14] - -(16) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#12, s_state#14] - -(17) BroadcastExchange -Input [2]: [s_store_sk#12, s_state#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#9] -Right keys [1]: [s_store_sk#12] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14] -Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#12, s_state#14] - -(20) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#15] - -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 4] -Output [2]: [ss_net_profit#10, s_state#14] -Input [4]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14, d_date_sk#15] - -(23) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#10, s_state#14] -Keys [1]: [s_state#14] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum#16] -Results [2]: [s_state#14, sum#17] - -(24) CometColumnarExchange -Input [2]: [s_state#14, sum#17] -Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(25) CometColumnarToRow [codegen id : 5] -Input [2]: [s_state#14, sum#17] - -(26) HashAggregate [codegen id : 5] -Input [2]: [s_state#14, sum#17] -Keys [1]: [s_state#14] -Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#18] -Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#18,17,2) AS _w0#19, s_state#14] - -(27) Sort [codegen id : 5] -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 - -(28) WindowGroupLimit -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [s_state#14], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final - -(29) Window -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#14, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#14], [_w0#19 DESC NULLS LAST] - -(30) Filter [codegen id : 6] -Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] -Condition : (ranking#20 <= 5) - -(31) Project [codegen id : 6] -Output [1]: [s_state#14] -Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] - -(32) BroadcastExchange -Input [1]: [s_state#14] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(33) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#8, 2))] -Right keys [1]: [s_state#14] -Join type: LeftSemi -Join condition: None - -(34) Project [codegen id : 7] -Output [3]: [s_store_sk#6, s_county#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#8, 2)) AS s_state#21] -Input [3]: [s_store_sk#6, s_county#7, s_state#8] - -(35) BroadcastExchange -Input [3]: [s_store_sk#6, s_county#7, s_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(36) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#6] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 8] -Output [3]: [ss_net_profit#2, s_state#21, s_county#7] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#21] - -(38) Expand [codegen id : 8] -Input [3]: [ss_net_profit#2, s_state#21, s_county#7] -Arguments: [[ss_net_profit#2, s_state#21, s_county#7, 0], [ss_net_profit#2, s_state#21, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] - -(39) HashAggregate [codegen id : 8] -Input [4]: [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#25] -Results [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] - -(40) CometColumnarExchange -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Arguments: hashpartitioning(s_state#22, s_county#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(41) CometColumnarToRow [codegen id : 9] -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] - -(42) HashAggregate [codegen id : 9] -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#27] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS total_sum#28, s_state#22, s_county#23, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS lochierarchy#29, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS _w0#30, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS _w1#31, CASE WHEN (cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint) = 0) THEN s_state#22 END AS _w2#32] - -(43) CometColumnarExchange -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] -Arguments: hashpartitioning(_w1#31, _w2#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(44) CometSort -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] -Arguments: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32], [_w1#31 ASC NULLS FIRST, _w2#32 ASC NULLS FIRST, _w0#30 DESC NULLS LAST] - -(45) CometColumnarToRow [codegen id : 10] -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] - -(46) Window -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] -Arguments: [rank(_w0#30) windowspecdefinition(_w1#31, _w2#32, _w0#30 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#33], [_w1#31, _w2#32], [_w0#30 DESC NULLS LAST] - -(47) Project [codegen id : 11] -Output [5]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] -Input [8]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32, rank_within_parent#33] - -(48) TakeOrderedAndProject -Input [5]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] -Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0) THEN s_state#22 END ASC NULLS FIRST, rank_within_parent#33 ASC NULLS FIRST], [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometNativeScan parquet spark_catalog.default.date_dim (49) - - -(49) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#34] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1200)) AND (d_month_seq#34 <= 1211)) AND isnotnull(d_date_sk#5)) - -(51) CometProject -Input [2]: [d_date_sk#5, d_month_seq#34] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(53) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/simplified.txt deleted file mode 100644 index f61238590a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/simplified.txt +++ /dev/null @@ -1,80 +0,0 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (11) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (9) - HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (8) - HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [ss_net_profit,s_state,s_county] - Project [ss_net_profit,s_state,s_county] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (5) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #6 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/explain.txt deleted file mode 100644 index 44f6ce791c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,309 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (49) -+- * Project (48) - +- Window (47) - +- * CometColumnarToRow (46) - +- CometSort (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * CometColumnarToRow (42) - +- CometColumnarExchange (41) - +- * HashAggregate (40) - +- * Expand (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (9) - : +- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- BroadcastExchange (36) - +- * Project (35) - +- * BroadcastHashJoin LeftSemi BuildRight (34) - :- * CometColumnarToRow (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) - +- BroadcastExchange (33) - +- * Project (32) - +- * Filter (31) - +- Window (30) - +- WindowGroupLimit (29) - +- * CometColumnarToRow (28) - +- CometSort (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometBroadcastHashJoin (22) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - +- ReusedExchange (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] - -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ss_store_sk#1, ss_net_profit#2] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#7, s_county#8, s_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Condition : isnotnull(s_store_sk#7) - -(12) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_store_sk#10) - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : isnotnull(s_store_sk#14) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14, s_state#16], [s_store_sk#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#15, 2)) AS s_state#16] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#14, s_state#16] -Arguments: [s_store_sk#14, s_state#16] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Right output [2]: [s_store_sk#14, s_state#16] -Arguments: [ss_store_sk#10], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#16] -Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16], [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#17] - -(22) CometBroadcastHashJoin -Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] -Right output [1]: [d_date_sk#17] -Arguments: [ss_sold_date_sk#12], [d_date_sk#17], Inner, BuildRight - -(23) CometProject -Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16, d_date_sk#17] -Arguments: [ss_net_profit#11, s_state#16], [ss_net_profit#11, s_state#16] - -(24) CometHashAggregate -Input [2]: [ss_net_profit#11, s_state#16] -Keys [1]: [s_state#16] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] - -(25) CometExchange -Input [2]: [s_state#16, sum#18] -Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(26) CometHashAggregate -Input [2]: [s_state#16, sum#18] -Keys [1]: [s_state#16] -Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] - -(27) CometSort -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16, _w0#19, s_state#16], [s_state#16 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] - -(28) CometColumnarToRow [codegen id : 1] -Input [3]: [s_state#16, _w0#19, s_state#16] - -(29) WindowGroupLimit -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final - -(30) Window -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#16, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#16], [_w0#19 DESC NULLS LAST] - -(31) Filter [codegen id : 2] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] -Condition : (ranking#20 <= 5) - -(32) Project [codegen id : 2] -Output [1]: [s_state#16] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] - -(33) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] - -(34) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#9, 2))] -Right keys [1]: [s_state#16] -Join type: LeftSemi -Join condition: None - -(35) Project [codegen id : 3] -Output [3]: [s_store_sk#7, s_county#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#9, 2)) AS s_state#21] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(36) BroadcastExchange -Input [3]: [s_store_sk#7, s_county#8, s_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_state#21, s_county#8] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#21] - -(39) Expand [codegen id : 4] -Input [3]: [ss_net_profit#2, s_state#21, s_county#8] -Arguments: [[ss_net_profit#2, s_state#21, s_county#8, 0], [ss_net_profit#2, s_state#21, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] - -(40) HashAggregate [codegen id : 4] -Input [4]: [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#25] -Results [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] - -(41) CometColumnarExchange -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Arguments: hashpartitioning(s_state#22, s_county#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(42) CometColumnarToRow [codegen id : 5] -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] - -(43) HashAggregate [codegen id : 5] -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#27] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS total_sum#28, s_state#22, s_county#23, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS lochierarchy#29, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS _w0#30, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS _w1#31, CASE WHEN (cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint) = 0) THEN s_state#22 END AS _w2#32] - -(44) CometColumnarExchange -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] -Arguments: hashpartitioning(_w1#31, _w2#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometSort -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] -Arguments: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32], [_w1#31 ASC NULLS FIRST, _w2#32 ASC NULLS FIRST, _w0#30 DESC NULLS LAST] - -(46) CometColumnarToRow [codegen id : 6] -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] - -(47) Window -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] -Arguments: [rank(_w0#30) windowspecdefinition(_w1#31, _w2#32, _w0#30 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#33], [_w1#31, _w2#32], [_w0#30 DESC NULLS LAST] - -(48) Project [codegen id : 7] -Output [5]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] -Input [8]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32, rank_within_parent#33] - -(49) TakeOrderedAndProject -Input [5]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] -Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0) THEN s_state#22 END ASC NULLS FIRST, rank_within_parent#33 ASC NULLS FIRST], [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) - - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(52) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(53) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(54) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/simplified.txt deleted file mode 100644 index 80f0cfc8c2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (7) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (5) - HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [ss_net_profit,s_state,s_county] - Project [ss_net_profit,s_state,s_county] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [s_state,_w0] - CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] - CometExchange [s_state] #7 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - 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,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] 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/q70/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt deleted file mode 100644 index 44f6ce791c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt +++ /dev/null @@ -1,309 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (49) -+- * Project (48) - +- Window (47) - +- * CometColumnarToRow (46) - +- CometSort (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * CometColumnarToRow (42) - +- CometColumnarExchange (41) - +- * HashAggregate (40) - +- * Expand (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (9) - : +- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- BroadcastExchange (36) - +- * Project (35) - +- * BroadcastHashJoin LeftSemi BuildRight (34) - :- * CometColumnarToRow (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) - +- BroadcastExchange (33) - +- * Project (32) - +- * Filter (31) - +- Window (30) - +- WindowGroupLimit (29) - +- * CometColumnarToRow (28) - +- CometSort (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometBroadcastHashJoin (22) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - +- ReusedExchange (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] - -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ss_store_sk#1, ss_net_profit#2] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#7, s_county#8, s_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Condition : isnotnull(s_store_sk#7) - -(12) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_store_sk#10) - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : isnotnull(s_store_sk#14) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14, s_state#16], [s_store_sk#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#15, 2)) AS s_state#16] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#14, s_state#16] -Arguments: [s_store_sk#14, s_state#16] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Right output [2]: [s_store_sk#14, s_state#16] -Arguments: [ss_store_sk#10], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#16] -Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16], [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#17] - -(22) CometBroadcastHashJoin -Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] -Right output [1]: [d_date_sk#17] -Arguments: [ss_sold_date_sk#12], [d_date_sk#17], Inner, BuildRight - -(23) CometProject -Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16, d_date_sk#17] -Arguments: [ss_net_profit#11, s_state#16], [ss_net_profit#11, s_state#16] - -(24) CometHashAggregate -Input [2]: [ss_net_profit#11, s_state#16] -Keys [1]: [s_state#16] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] - -(25) CometExchange -Input [2]: [s_state#16, sum#18] -Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(26) CometHashAggregate -Input [2]: [s_state#16, sum#18] -Keys [1]: [s_state#16] -Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] - -(27) CometSort -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16, _w0#19, s_state#16], [s_state#16 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] - -(28) CometColumnarToRow [codegen id : 1] -Input [3]: [s_state#16, _w0#19, s_state#16] - -(29) WindowGroupLimit -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final - -(30) Window -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#16, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#16], [_w0#19 DESC NULLS LAST] - -(31) Filter [codegen id : 2] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] -Condition : (ranking#20 <= 5) - -(32) Project [codegen id : 2] -Output [1]: [s_state#16] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] - -(33) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] - -(34) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#9, 2))] -Right keys [1]: [s_state#16] -Join type: LeftSemi -Join condition: None - -(35) Project [codegen id : 3] -Output [3]: [s_store_sk#7, s_county#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#9, 2)) AS s_state#21] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(36) BroadcastExchange -Input [3]: [s_store_sk#7, s_county#8, s_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_state#21, s_county#8] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#21] - -(39) Expand [codegen id : 4] -Input [3]: [ss_net_profit#2, s_state#21, s_county#8] -Arguments: [[ss_net_profit#2, s_state#21, s_county#8, 0], [ss_net_profit#2, s_state#21, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] - -(40) HashAggregate [codegen id : 4] -Input [4]: [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#25] -Results [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] - -(41) CometColumnarExchange -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Arguments: hashpartitioning(s_state#22, s_county#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(42) CometColumnarToRow [codegen id : 5] -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] - -(43) HashAggregate [codegen id : 5] -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#27] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS total_sum#28, s_state#22, s_county#23, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS lochierarchy#29, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS _w0#30, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS _w1#31, CASE WHEN (cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint) = 0) THEN s_state#22 END AS _w2#32] - -(44) CometColumnarExchange -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] -Arguments: hashpartitioning(_w1#31, _w2#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometSort -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] -Arguments: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32], [_w1#31 ASC NULLS FIRST, _w2#32 ASC NULLS FIRST, _w0#30 DESC NULLS LAST] - -(46) CometColumnarToRow [codegen id : 6] -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] - -(47) Window -Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] -Arguments: [rank(_w0#30) windowspecdefinition(_w1#31, _w2#32, _w0#30 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#33], [_w1#31, _w2#32], [_w0#30 DESC NULLS LAST] - -(48) Project [codegen id : 7] -Output [5]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] -Input [8]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32, rank_within_parent#33] - -(49) TakeOrderedAndProject -Input [5]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] -Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0) THEN s_state#22 END ASC NULLS FIRST, rank_within_parent#33 ASC NULLS FIRST], [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) - - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(52) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(53) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(54) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/extended.txt deleted file mode 100644 index 45a2c7a669..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/extended.txt +++ /dev/null @@ -1,61 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 34 out of 53 eligible operators (64%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 80f0cfc8c2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (7) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (5) - HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [ss_net_profit,s_state,s_county] - Project [ss_net_profit,s_state,s_county] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [s_state,_w0] - CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] - CometExchange [s_state] #7 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - 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,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/explain.txt deleted file mode 100644 index d0c100dc27..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/explain.txt +++ /dev/null @@ -1,262 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometSort (39) - +- CometColumnarExchange (38) - +- * HashAggregate (37) - +- * CometColumnarToRow (36) - +- CometColumnarExchange (35) - +- * HashAggregate (34) - +- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * Project (26) - : +- * BroadcastHashJoin Inner BuildLeft (25) - : :- BroadcastExchange (5) - : : +- * CometColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometNativeScan parquet spark_catalog.default.item (1) - : +- Union (24) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * Filter (8) - : : : +- * ColumnarToRow (7) - : : : +- Scan parquet spark_catalog.default.web_sales (6) - : : +- ReusedExchange (9) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Filter (14) - : : : +- * ColumnarToRow (13) - : : : +- Scan parquet spark_catalog.default.catalog_sales (12) - : : +- ReusedExchange (15) - : +- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * Filter (20) - : : +- * ColumnarToRow (19) - : : +- Scan parquet spark_catalog.default.store_sales (18) - : +- ReusedExchange (21) - +- BroadcastExchange (31) - +- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometNativeScan parquet spark_catalog.default.time_dim (27) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Condition : ((isnotnull(i_manager_id#4) AND (i_manager_id#4 = 1)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Arguments: [i_item_sk#1, i_brand_id#2, i_brand#5], [i_item_sk#1, i_brand_id#2, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#3, 50)) AS i_brand#5] - -(4) CometColumnarToRow [codegen id : 1] -Input [3]: [i_item_sk#1, i_brand_id#2, i_brand#5] - -(5) BroadcastExchange -Input [3]: [i_item_sk#1, i_brand_id#2, i_brand#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(6) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_time_sk)] -ReadSchema: struct - -(7) ColumnarToRow [codegen id : 3] -Input [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] - -(8) Filter [codegen id : 3] -Input [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Condition : (isnotnull(ws_item_sk#7) AND isnotnull(ws_sold_time_sk#6)) - -(9) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#11] - -(10) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 3] -Output [3]: [ws_ext_sales_price#8 AS ext_price#12, ws_item_sk#7 AS sold_item_sk#13, ws_sold_time_sk#6 AS time_sk#14] -Input [5]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9, d_date_sk#11] - -(12) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_sold_time_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_time_sk)] -ReadSchema: struct - -(13) ColumnarToRow [codegen id : 5] -Input [4]: [cs_sold_time_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] - -(14) Filter [codegen id : 5] -Input [4]: [cs_sold_time_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] -Condition : (isnotnull(cs_item_sk#16) AND isnotnull(cs_sold_time_sk#15)) - -(15) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#19] - -(16) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 5] -Output [3]: [cs_ext_sales_price#17 AS ext_price#20, cs_item_sk#16 AS sold_item_sk#21, cs_sold_time_sk#15 AS time_sk#22] -Input [5]: [cs_sold_time_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18, d_date_sk#19] - -(18) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 7] -Input [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] - -(20) Filter [codegen id : 7] -Input [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : (isnotnull(ss_item_sk#24) AND isnotnull(ss_sold_time_sk#23)) - -(21) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#27] - -(22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: None - -(23) Project [codegen id : 7] -Output [3]: [ss_ext_sales_price#25 AS ext_price#28, ss_item_sk#24 AS sold_item_sk#29, ss_sold_time_sk#23 AS time_sk#30] -Input [5]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26, d_date_sk#27] - -(24) Union - -(25) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [sold_item_sk#13] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 9] -Output [4]: [i_brand_id#2, i_brand#5, ext_price#12, time_sk#14] -Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#5, ext_price#12, sold_item_sk#13, time_sk#14] - -(27) CometNativeScan parquet spark_catalog.default.time_dim -Output [4]: [t_time_sk#31, t_hour#32, t_minute#33, t_meal_time#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_time_sk)] -ReadSchema: struct - -(28) CometFilter -Input [4]: [t_time_sk#31, t_hour#32, t_minute#33, t_meal_time#34] -Condition : (((static_invoke(CharVarcharCodegenUtils.readSidePadding(t_meal_time#34, 20)) = breakfast ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(t_meal_time#34, 20)) = dinner )) AND isnotnull(t_time_sk#31)) - -(29) CometProject -Input [4]: [t_time_sk#31, t_hour#32, t_minute#33, t_meal_time#34] -Arguments: [t_time_sk#31, t_hour#32, t_minute#33], [t_time_sk#31, t_hour#32, t_minute#33] - -(30) CometColumnarToRow [codegen id : 8] -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] - -(31) BroadcastExchange -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [time_sk#14] -Right keys [1]: [t_time_sk#31] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 9] -Output [5]: [i_brand_id#2, i_brand#5, ext_price#12, t_hour#32, t_minute#33] -Input [7]: [i_brand_id#2, i_brand#5, ext_price#12, time_sk#14, t_time_sk#31, t_hour#32, t_minute#33] - -(34) HashAggregate [codegen id : 9] -Input [5]: [i_brand_id#2, i_brand#5, ext_price#12, t_hour#32, t_minute#33] -Keys [4]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33] -Functions [1]: [partial_sum(UnscaledValue(ext_price#12))] -Aggregate Attributes [1]: [sum#35] -Results [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] - -(35) CometColumnarExchange -Input [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] -Arguments: hashpartitioning(i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(36) CometColumnarToRow [codegen id : 10] -Input [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] - -(37) HashAggregate [codegen id : 10] -Input [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] -Keys [4]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33] -Functions [1]: [sum(UnscaledValue(ext_price#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#12))#37] -Results [5]: [i_brand_id#2 AS brand_id#38, i_brand#5 AS brand#39, t_hour#32, t_minute#33, MakeDecimal(sum(UnscaledValue(ext_price#12))#37,17,2) AS ext_price#40] - -(38) CometColumnarExchange -Input [5]: [brand_id#38, brand#39, t_hour#32, t_minute#33, ext_price#40] -Arguments: rangepartitioning(ext_price#40 DESC NULLS LAST, brand_id#38 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(39) CometSort -Input [5]: [brand_id#38, brand#39, t_hour#32, t_minute#33, ext_price#40] -Arguments: [brand_id#38, brand#39, t_hour#32, t_minute#33, ext_price#40], [ext_price#40 DESC NULLS LAST, brand_id#38 ASC NULLS FIRST] - -(40) CometColumnarToRow [codegen id : 11] -Input [5]: [brand_id#38, brand#39, t_hour#32, t_minute#33, ext_price#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 6 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometNativeScan parquet spark_catalog.default.date_dim (41) - - -(41) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#41, d_moy#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#11, d_year#41, d_moy#42] -Condition : ((((isnotnull(d_moy#42) AND isnotnull(d_year#41)) AND (d_moy#42 = 11)) AND (d_year#41 = 1999)) AND isnotnull(d_date_sk#11)) - -(43) CometProject -Input [3]: [d_date_sk#11, d_year#41, d_moy#42] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(45) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 12 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#10 - -Subquery:3 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#10 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/simplified.txt deleted file mode 100644 index 838a3e6604..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometColumnarExchange [ext_price,brand_id] #1 - WholeStageCodegen (10) - HashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] [sum(UnscaledValue(ext_price)),brand_id,brand,ext_price,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand,i_brand_id,t_hour,t_minute] #2 - WholeStageCodegen (9) - HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] - Project [i_brand_id,i_brand,ext_price,t_hour,t_minute] - BroadcastHashJoin [time_sk,t_time_sk] - Project [i_brand_id,i_brand,ext_price,time_sk] - BroadcastHashJoin [i_item_sk,sold_item_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - InputAdapter - Union - WholeStageCodegen (3) - Project [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk,ws_sold_time_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (5) - Project [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_time_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (7) - Project [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_sold_time_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] - CometNativeScan 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/q71.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/explain.txt deleted file mode 100644 index 00ff66d3eb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,251 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (37) -+- CometSort (36) - +- CometExchange (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometBroadcastExchange (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : +- CometUnion (23) - : :- CometProject (12) - : : +- CometBroadcastHashJoin (11) - : : :- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (5) - : : +- CometBroadcastExchange (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (7) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (13) - : : +- ReusedExchange (15) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (19) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (18) - : +- ReusedExchange (20) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Condition : ((isnotnull(i_manager_id#4) AND (i_manager_id#4 = 1)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Arguments: [i_item_sk#1, i_brand_id#2, i_brand#5], [i_item_sk#1, i_brand_id#2, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#3, 50)) AS i_brand#5] - -(4) CometBroadcastExchange -Input [3]: [i_item_sk#1, i_brand_id#2, i_brand#5] -Arguments: [i_item_sk#1, i_brand_id#2, i_brand#5] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_time_sk)] -ReadSchema: struct - -(6) CometFilter -Input [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Condition : (isnotnull(ws_item_sk#7) AND isnotnull(ws_sold_time_sk#6)) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_moy#13) AND isnotnull(d_year#12)) AND (d_moy#13 = 11)) AND (d_year#12 = 1999)) AND isnotnull(d_date_sk#11)) - -(9) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(10) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(11) CometBroadcastHashJoin -Left output [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [ws_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(12) CometProject -Input [5]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9, d_date_sk#11] -Arguments: [ext_price#14, sold_item_sk#15, time_sk#16], [ws_ext_sales_price#8 AS ext_price#14, ws_item_sk#7 AS sold_item_sk#15, ws_sold_time_sk#6 AS time_sk#16] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#21)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_time_sk)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#18) AND isnotnull(cs_sold_time_sk#17)) - -(15) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#22] - -(16) CometBroadcastHashJoin -Left output [4]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Right output [1]: [d_date_sk#22] -Arguments: [cs_sold_date_sk#20], [d_date_sk#22], Inner, BuildRight - -(17) CometProject -Input [5]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] -Arguments: [ext_price#23, sold_item_sk#24, time_sk#25], [cs_ext_sales_price#19 AS ext_price#23, cs_item_sk#18 AS sold_item_sk#24, cs_sold_time_sk#17 AS time_sk#25] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] -ReadSchema: struct - -(19) CometFilter -Input [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Condition : (isnotnull(ss_item_sk#27) AND isnotnull(ss_sold_time_sk#26)) - -(20) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#31] - -(21) CometBroadcastHashJoin -Left output [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Right output [1]: [d_date_sk#31] -Arguments: [ss_sold_date_sk#29], [d_date_sk#31], Inner, BuildRight - -(22) CometProject -Input [5]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#31] -Arguments: [ext_price#32, sold_item_sk#33, time_sk#34], [ss_ext_sales_price#28 AS ext_price#32, ss_item_sk#27 AS sold_item_sk#33, ss_sold_time_sk#26 AS time_sk#34] - -(23) CometUnion -Child 0 Input [3]: [ext_price#14, sold_item_sk#15, time_sk#16] -Child 1 Input [3]: [ext_price#23, sold_item_sk#24, time_sk#25] -Child 2 Input [3]: [ext_price#32, sold_item_sk#33, time_sk#34] - -(24) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand_id#2, i_brand#5] -Right output [3]: [ext_price#14, sold_item_sk#15, time_sk#16] -Arguments: [i_item_sk#1], [sold_item_sk#15], Inner, BuildLeft - -(25) CometProject -Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#5, ext_price#14, sold_item_sk#15, time_sk#16] -Arguments: [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16], [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [4]: [t_time_sk#35, t_hour#36, t_minute#37, t_meal_time#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_time_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [t_time_sk#35, t_hour#36, t_minute#37, t_meal_time#38] -Condition : (((static_invoke(CharVarcharCodegenUtils.readSidePadding(t_meal_time#38, 20)) = breakfast ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(t_meal_time#38, 20)) = dinner )) AND isnotnull(t_time_sk#35)) - -(28) CometProject -Input [4]: [t_time_sk#35, t_hour#36, t_minute#37, t_meal_time#38] -Arguments: [t_time_sk#35, t_hour#36, t_minute#37], [t_time_sk#35, t_hour#36, t_minute#37] - -(29) CometBroadcastExchange -Input [3]: [t_time_sk#35, t_hour#36, t_minute#37] -Arguments: [t_time_sk#35, t_hour#36, t_minute#37] - -(30) CometBroadcastHashJoin -Left output [4]: [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16] -Right output [3]: [t_time_sk#35, t_hour#36, t_minute#37] -Arguments: [time_sk#16], [t_time_sk#35], Inner, BuildRight - -(31) CometProject -Input [7]: [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16, t_time_sk#35, t_hour#36, t_minute#37] -Arguments: [i_brand_id#2, i_brand#5, ext_price#14, t_hour#36, t_minute#37], [i_brand_id#2, i_brand#5, ext_price#14, t_hour#36, t_minute#37] - -(32) CometHashAggregate -Input [5]: [i_brand_id#2, i_brand#5, ext_price#14, t_hour#36, t_minute#37] -Keys [4]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37] -Functions [1]: [partial_sum(UnscaledValue(ext_price#14))] - -(33) CometExchange -Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] -Arguments: hashpartitioning(i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(34) CometHashAggregate -Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] -Keys [4]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37] -Functions [1]: [sum(UnscaledValue(ext_price#14))] - -(35) CometExchange -Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] -Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(36) CometSort -Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] -Arguments: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42], [ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST] - -(37) CometColumnarToRow [codegen id : 1] -Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (42) -+- * CometColumnarToRow (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) - - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(39) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_moy#13) AND isnotnull(d_year#12)) AND (d_moy#13 = 11)) AND (d_year#12 = 1999)) AND isnotnull(d_date_sk#11)) - -(40) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(41) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(42) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#10 - -Subquery:3 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#10 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/simplified.txt deleted file mode 100644 index 63e7c353ac..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometExchange [ext_price,brand_id] #1 - CometHashAggregate [sum] [brand_id,brand,t_hour,t_minute,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ext_price))] - CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 - CometHashAggregate [ext_price] [i_brand,i_brand_id,t_hour,t_minute,sum] - CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] - 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,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - 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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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_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 [native_iceberg_compat] 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_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 [native_iceberg_compat] 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 [t_time_sk,t_hour,t_minute] #6 - CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] - CometScan [native_iceberg_compat] 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/q71/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/explain.txt deleted file mode 100644 index 00ff66d3eb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/explain.txt +++ /dev/null @@ -1,251 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (37) -+- CometSort (36) - +- CometExchange (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometBroadcastExchange (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : +- CometUnion (23) - : :- CometProject (12) - : : +- CometBroadcastHashJoin (11) - : : :- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (5) - : : +- CometBroadcastExchange (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (7) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (13) - : : +- ReusedExchange (15) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (19) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (18) - : +- ReusedExchange (20) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Condition : ((isnotnull(i_manager_id#4) AND (i_manager_id#4 = 1)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Arguments: [i_item_sk#1, i_brand_id#2, i_brand#5], [i_item_sk#1, i_brand_id#2, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#3, 50)) AS i_brand#5] - -(4) CometBroadcastExchange -Input [3]: [i_item_sk#1, i_brand_id#2, i_brand#5] -Arguments: [i_item_sk#1, i_brand_id#2, i_brand#5] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_time_sk)] -ReadSchema: struct - -(6) CometFilter -Input [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Condition : (isnotnull(ws_item_sk#7) AND isnotnull(ws_sold_time_sk#6)) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_moy#13) AND isnotnull(d_year#12)) AND (d_moy#13 = 11)) AND (d_year#12 = 1999)) AND isnotnull(d_date_sk#11)) - -(9) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(10) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(11) CometBroadcastHashJoin -Left output [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [ws_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(12) CometProject -Input [5]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9, d_date_sk#11] -Arguments: [ext_price#14, sold_item_sk#15, time_sk#16], [ws_ext_sales_price#8 AS ext_price#14, ws_item_sk#7 AS sold_item_sk#15, ws_sold_time_sk#6 AS time_sk#16] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#21)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_time_sk)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#18) AND isnotnull(cs_sold_time_sk#17)) - -(15) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#22] - -(16) CometBroadcastHashJoin -Left output [4]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Right output [1]: [d_date_sk#22] -Arguments: [cs_sold_date_sk#20], [d_date_sk#22], Inner, BuildRight - -(17) CometProject -Input [5]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] -Arguments: [ext_price#23, sold_item_sk#24, time_sk#25], [cs_ext_sales_price#19 AS ext_price#23, cs_item_sk#18 AS sold_item_sk#24, cs_sold_time_sk#17 AS time_sk#25] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] -ReadSchema: struct - -(19) CometFilter -Input [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Condition : (isnotnull(ss_item_sk#27) AND isnotnull(ss_sold_time_sk#26)) - -(20) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#31] - -(21) CometBroadcastHashJoin -Left output [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Right output [1]: [d_date_sk#31] -Arguments: [ss_sold_date_sk#29], [d_date_sk#31], Inner, BuildRight - -(22) CometProject -Input [5]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#31] -Arguments: [ext_price#32, sold_item_sk#33, time_sk#34], [ss_ext_sales_price#28 AS ext_price#32, ss_item_sk#27 AS sold_item_sk#33, ss_sold_time_sk#26 AS time_sk#34] - -(23) CometUnion -Child 0 Input [3]: [ext_price#14, sold_item_sk#15, time_sk#16] -Child 1 Input [3]: [ext_price#23, sold_item_sk#24, time_sk#25] -Child 2 Input [3]: [ext_price#32, sold_item_sk#33, time_sk#34] - -(24) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand_id#2, i_brand#5] -Right output [3]: [ext_price#14, sold_item_sk#15, time_sk#16] -Arguments: [i_item_sk#1], [sold_item_sk#15], Inner, BuildLeft - -(25) CometProject -Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#5, ext_price#14, sold_item_sk#15, time_sk#16] -Arguments: [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16], [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [4]: [t_time_sk#35, t_hour#36, t_minute#37, t_meal_time#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_time_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [t_time_sk#35, t_hour#36, t_minute#37, t_meal_time#38] -Condition : (((static_invoke(CharVarcharCodegenUtils.readSidePadding(t_meal_time#38, 20)) = breakfast ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(t_meal_time#38, 20)) = dinner )) AND isnotnull(t_time_sk#35)) - -(28) CometProject -Input [4]: [t_time_sk#35, t_hour#36, t_minute#37, t_meal_time#38] -Arguments: [t_time_sk#35, t_hour#36, t_minute#37], [t_time_sk#35, t_hour#36, t_minute#37] - -(29) CometBroadcastExchange -Input [3]: [t_time_sk#35, t_hour#36, t_minute#37] -Arguments: [t_time_sk#35, t_hour#36, t_minute#37] - -(30) CometBroadcastHashJoin -Left output [4]: [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16] -Right output [3]: [t_time_sk#35, t_hour#36, t_minute#37] -Arguments: [time_sk#16], [t_time_sk#35], Inner, BuildRight - -(31) CometProject -Input [7]: [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16, t_time_sk#35, t_hour#36, t_minute#37] -Arguments: [i_brand_id#2, i_brand#5, ext_price#14, t_hour#36, t_minute#37], [i_brand_id#2, i_brand#5, ext_price#14, t_hour#36, t_minute#37] - -(32) CometHashAggregate -Input [5]: [i_brand_id#2, i_brand#5, ext_price#14, t_hour#36, t_minute#37] -Keys [4]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37] -Functions [1]: [partial_sum(UnscaledValue(ext_price#14))] - -(33) CometExchange -Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] -Arguments: hashpartitioning(i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(34) CometHashAggregate -Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] -Keys [4]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37] -Functions [1]: [sum(UnscaledValue(ext_price#14))] - -(35) CometExchange -Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] -Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(36) CometSort -Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] -Arguments: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42], [ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST] - -(37) CometColumnarToRow [codegen id : 1] -Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (42) -+- * CometColumnarToRow (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) - - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(39) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_moy#13) AND isnotnull(d_year#12)) AND (d_moy#13 = 11)) AND (d_year#12 = 1999)) AND isnotnull(d_date_sk#11)) - -(40) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(41) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(42) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#10 - -Subquery:3 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#10 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/extended.txt deleted file mode 100644 index 25ea15c536..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/extended.txt +++ /dev/null @@ -1,53 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometUnion - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - -Comet accelerated 45 out of 49 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 63e7c353ac..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometExchange [ext_price,brand_id] #1 - CometHashAggregate [sum] [brand_id,brand,t_hour,t_minute,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ext_price))] - CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 - CometHashAggregate [ext_price] [i_brand,i_brand_id,t_hour,t_minute,sum] - CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] - 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,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - 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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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_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 [native_iceberg_compat] 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_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 [native_iceberg_compat] 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 [t_time_sk,t_hour,t_minute] #6 - CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/explain.txt deleted file mode 100644 index 34b69fc8d2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/explain.txt +++ /dev/null @@ -1,428 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (70) -+- CometTakeOrderedAndProject (69) - +- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometProject (65) - +- CometSortMergeJoin (64) - :- CometSort (58) - : +- CometColumnarExchange (57) - : +- * Project (56) - : +- * BroadcastHashJoin LeftOuter BuildRight (55) - : :- * Project (50) - : : +- * BroadcastHashJoin Inner BuildRight (49) - : : :- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (28) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Project (15) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : : :- * Project (9) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : : : : :- * Filter (3) - : : : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * CometColumnarToRow (6) - : : : : : : : : : +- CometFilter (5) - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory (4) - : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * CometColumnarToRow (12) - : : : : : : : : +- CometFilter (11) - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * CometColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) - : : : : : : +- BroadcastExchange (26) - : : : : : : +- * CometColumnarToRow (25) - : : : : : : +- CometProject (24) - : : : : : : +- CometFilter (23) - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (22) - : : : : : +- BroadcastExchange (33) - : : : : : +- * CometColumnarToRow (32) - : : : : : +- CometProject (31) - : : : : : +- CometFilter (30) - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (29) - : : : : +- ReusedExchange (36) - : : : +- BroadcastExchange (42) - : : : +- * CometColumnarToRow (41) - : : : +- CometFilter (40) - : : : +- CometNativeScan parquet spark_catalog.default.date_dim (39) - : : +- BroadcastExchange (48) - : : +- * CometColumnarToRow (47) - : : +- CometFilter (46) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (45) - : +- BroadcastExchange (54) - : +- * CometColumnarToRow (53) - : +- CometFilter (52) - : +- CometNativeScan parquet spark_catalog.default.promotion (51) - +- CometSort (63) - +- CometExchange (62) - +- CometProject (61) - +- CometFilter (60) - +- CometNativeScan parquet spark_catalog.default.catalog_returns (59) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 10] -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] - -(3) Filter [codegen id : 10] -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) - -(4) CometNativeScan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(inv_date_sk#13)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) - -(6) CometColumnarToRow [codegen id : 1] -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(7) BroadcastExchange -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [inv_item_sk#10] -Join type: Inner -Join condition: (inv_quantity_on_hand#12 < cs_quantity#7) - -(9) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(10) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Condition : isnotnull(w_warehouse_sk#14) - -(12) CometColumnarToRow [codegen id : 2] -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] - -(13) BroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_warehouse_sk#11] -Right keys [1]: [w_warehouse_sk#14] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] - -(16) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_desc#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [i_item_sk#16, i_item_desc#17] -Condition : isnotnull(i_item_sk#16) - -(18) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#16, i_item_desc#17] - -(19) BroadcastExchange -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(20) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#16] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 10] -Output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] - -(22) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#18, cd_marital_status#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_marital_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(23) CometFilter -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Condition : ((isnotnull(cd_marital_status#19) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#19, 1)) = D)) AND isnotnull(cd_demo_sk#18)) - -(24) CometProject -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Arguments: [cd_demo_sk#18], [cd_demo_sk#18] - -(25) CometColumnarToRow [codegen id : 4] -Input [1]: [cd_demo_sk#18] - -(26) BroadcastExchange -Input [1]: [cd_demo_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(27) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] - -(29) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_buy_potential#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Condition : ((isnotnull(hd_buy_potential#21) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#21, 15)) = >10000 )) AND isnotnull(hd_demo_sk#20)) - -(31) CometProject -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Arguments: [hd_demo_sk#20], [hd_demo_sk#20] - -(32) CometColumnarToRow [codegen id : 5] -Input [1]: [hd_demo_sk#20] - -(33) BroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] - -(36) ReusedExchange [Reuses operator id: 75] -Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#8] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] - -(39) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_week_seq#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(40) CometFilter -Input [2]: [d_date_sk#25, d_week_seq#26] -Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) - -(41) CometColumnarToRow [codegen id : 7] -Input [2]: [d_date_sk#25, d_week_seq#26] - -(42) BroadcastExchange -Input [2]: [d_date_sk#25, d_week_seq#26] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] - -(43) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [d_week_seq#24, inv_date_sk#13] -Right keys [2]: [d_week_seq#26, d_date_sk#25] -Join type: Inner -Join condition: None - -(44) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#25, d_week_seq#26] - -(45) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#27, d_date#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [d_date_sk#27, d_date#28] -Condition : (isnotnull(d_date#28) AND isnotnull(d_date_sk#27)) - -(47) CometColumnarToRow [codegen id : 8] -Input [2]: [d_date_sk#27, d_date#28] - -(48) BroadcastExchange -Input [2]: [d_date_sk#27, d_date#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: (d_date#28 > date_add(d_date#23, 5)) - -(50) Project [codegen id : 10] -Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#27, d_date#28] - -(51) CometNativeScan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(52) CometFilter -Input [1]: [p_promo_sk#29] -Condition : isnotnull(p_promo_sk#29) - -(53) CometColumnarToRow [codegen id : 9] -Input [1]: [p_promo_sk#29] - -(54) BroadcastExchange -Input [1]: [p_promo_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(55) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_promo_sk#5] -Right keys [1]: [p_promo_sk#29] -Join type: LeftOuter -Join condition: None - -(56) Project [codegen id : 10] -Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] - -(57) CometColumnarExchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(58) CometSort -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] - -(59) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(60) CometFilter -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Condition : (isnotnull(cr_item_sk#30) AND isnotnull(cr_order_number#31)) - -(61) CometProject -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30, cr_order_number#31] - -(62) CometExchange -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(63) CometSort -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST] - -(64) CometSortMergeJoin -Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#30, cr_order_number#31], LeftOuter - -(65) CometProject -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#30, cr_order_number#31] -Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(66) CometHashAggregate -Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [partial_count(1)] - -(67) CometExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(68) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [count(1)] - -(69) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#36 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#34,promo#35,total_cnt#36]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36], 100, 0, [total_cnt#36 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] - -(70) CometColumnarToRow [codegen id : 11] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (75) -+- * CometColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometNativeScan parquet spark_catalog.default.date_dim (71) - - -(71) CometNativeScan parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(72) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] -Condition : ((((isnotnull(d_year#37) AND (d_year#37 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(73) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(74) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(75) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/simplified.txt deleted file mode 100644 index 025217a607..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/simplified.txt +++ /dev/null @@ -1,107 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] - CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] - CometProject [w_warehouse_name,i_item_desc,d_week_seq] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometColumnarExchange [cs_item_sk,cs_order_number] #2 - WholeStageCodegen (10) - Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [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] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometNativeScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometNativeScan 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/q72.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/explain.txt deleted file mode 100644 index 7287cba3be..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,402 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (64) -+- CometTakeOrderedAndProject (63) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometSortMergeJoin (58) - :- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (45) - : : +- CometBroadcastHashJoin (44) - : : :- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (35) - : : : : +- CometBroadcastHashJoin (34) - : : : : :- CometProject (29) - : : : : : +- CometBroadcastHashJoin (28) - : : : : : :- CometProject (23) - : : : : : : +- CometBroadcastHashJoin (22) - : : : : : : :- CometProject (17) - : : : : : : : +- CometBroadcastHashJoin (16) - : : : : : : : :- CometProject (12) - : : : : : : : : +- CometBroadcastHashJoin (11) - : : : : : : : : :- CometProject (7) - : : : : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : : : : :- CometFilter (2) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- CometBroadcastExchange (5) - : : : : : : : : : +- CometFilter (4) - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (3) - : : : : : : : : +- CometBroadcastExchange (10) - : : : : : : : : +- CometFilter (9) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : : : : : : : +- CometBroadcastExchange (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (13) - : : : : : : +- CometBroadcastExchange (21) - : : : : : : +- CometProject (20) - : : : : : : +- CometFilter (19) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (18) - : : : : : +- CometBroadcastExchange (27) - : : : : : +- CometProject (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (24) - : : : : +- CometBroadcastExchange (33) - : : : : +- CometProject (32) - : : : : +- CometFilter (31) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - : : : +- CometBroadcastExchange (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (36) - : : +- CometBroadcastExchange (43) - : : +- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - : +- CometBroadcastExchange (48) - : +- CometFilter (47) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (46) - +- CometSort (57) - +- CometExchange (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (53) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#13)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) - -(5) CometBroadcastExchange -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(6) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Right output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_item_sk#4], [inv_item_sk#10], Inner, (inv_quantity_on_hand#12 < cs_quantity#7), BuildRight - -(7) CometProject -Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Condition : isnotnull(w_warehouse_sk#14) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [w_warehouse_sk#14, w_warehouse_name#15] - -(11) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -Right output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [inv_warehouse_sk#11], [w_warehouse_sk#14], Inner, BuildRight - -(12) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_desc#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [i_item_sk#16, i_item_desc#17] -Condition : isnotnull(i_item_sk#16) - -(15) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [i_item_sk#16, i_item_desc#17] - -(16) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -Right output [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [cs_item_sk#4], [i_item_sk#16], Inner, BuildRight - -(17) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#18, cd_marital_status#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_marital_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Condition : ((isnotnull(cd_marital_status#19) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#19, 1)) = D)) AND isnotnull(cd_demo_sk#18)) - -(20) CometProject -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Arguments: [cd_demo_sk#18], [cd_demo_sk#18] - -(21) CometBroadcastExchange -Input [1]: [cd_demo_sk#18] -Arguments: [cd_demo_sk#18] - -(22) CometBroadcastHashJoin -Left output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [cd_demo_sk#18] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#18], Inner, BuildRight - -(23) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] -Arguments: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_buy_potential#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Condition : ((isnotnull(hd_buy_potential#21) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#21, 15)) = >10000 )) AND isnotnull(hd_demo_sk#20)) - -(26) CometProject -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Arguments: [hd_demo_sk#20], [hd_demo_sk#20] - -(27) CometBroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: [hd_demo_sk#20] - -(28) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [hd_demo_sk#20] -Arguments: [cs_bill_hdemo_sk#3], [hd_demo_sk#20], Inner, BuildRight - -(29) CometProject -Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(31) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(32) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(33) CometBroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24] - -(34) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_sold_date_sk#8], [d_date_sk#22], Inner, BuildRight - -(35) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_week_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [d_date_sk#26, d_week_seq#27] -Condition : (isnotnull(d_week_seq#27) AND isnotnull(d_date_sk#26)) - -(38) CometBroadcastExchange -Input [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_date_sk#26, d_week_seq#27] - -(39) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_week_seq#24, inv_date_sk#13], [d_week_seq#27, d_date_sk#26], Inner, BuildRight - -(40) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#26, d_week_seq#27] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_date#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#28, d_date#29] -Condition : (isnotnull(d_date#29) AND isnotnull(d_date_sk#28)) - -(43) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_date#29] -Arguments: [d_date_sk#28, d_date#29] - -(44) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#28, d_date#29] -Arguments: [cs_ship_date_sk#1], [d_date_sk#28], Inner, (d_date#29 > date_add(d_date#23, 5)), BuildRight - -(45) CometProject -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#28, d_date#29] -Arguments: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(47) CometFilter -Input [1]: [p_promo_sk#30] -Condition : isnotnull(p_promo_sk#30) - -(48) CometBroadcastExchange -Input [1]: [p_promo_sk#30] -Arguments: [p_promo_sk#30] - -(49) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [1]: [p_promo_sk#30] -Arguments: [cs_promo_sk#5], [p_promo_sk#30], LeftOuter, BuildRight - -(50) CometProject -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#30] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(51) CometExchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(52) CometSort -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Condition : (isnotnull(cr_item_sk#31) AND isnotnull(cr_order_number#32)) - -(55) CometProject -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31, cr_order_number#32] - -(56) CometExchange -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(57) CometSort -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#31, cr_order_number#32], LeftOuter - -(59) CometProject -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#31, cr_order_number#32] -Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(60) CometHashAggregate -Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [partial_count(1)] - -(61) CometExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(62) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [count(1)] - -(63) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, 0, [total_cnt#37 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -(64) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (69) -+- * CometColumnarToRow (68) - +- CometProject (67) - +- CometFilter (66) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) - - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(67) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(68) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(69) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/simplified.txt deleted file mode 100644 index 678f8c67df..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] - CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] - CometProject [w_warehouse_name,i_item_desc,d_week_seq] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometExchange [cs_item_sk,cs_order_number] #2 - CometProject [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,p_promo_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_date] - 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 [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_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_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_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_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 [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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_desc] #6 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange [cd_demo_sk] #7 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange [hd_demo_sk] #8 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [d_date_sk,d_week_seq] #10 - CometFilter [d_date_sk,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [d_date_sk,d_date] #11 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [p_promo_sk] #12 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/explain.txt deleted file mode 100644 index 7287cba3be..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/explain.txt +++ /dev/null @@ -1,402 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (64) -+- CometTakeOrderedAndProject (63) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometSortMergeJoin (58) - :- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (45) - : : +- CometBroadcastHashJoin (44) - : : :- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (35) - : : : : +- CometBroadcastHashJoin (34) - : : : : :- CometProject (29) - : : : : : +- CometBroadcastHashJoin (28) - : : : : : :- CometProject (23) - : : : : : : +- CometBroadcastHashJoin (22) - : : : : : : :- CometProject (17) - : : : : : : : +- CometBroadcastHashJoin (16) - : : : : : : : :- CometProject (12) - : : : : : : : : +- CometBroadcastHashJoin (11) - : : : : : : : : :- CometProject (7) - : : : : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : : : : :- CometFilter (2) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- CometBroadcastExchange (5) - : : : : : : : : : +- CometFilter (4) - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (3) - : : : : : : : : +- CometBroadcastExchange (10) - : : : : : : : : +- CometFilter (9) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : : : : : : : +- CometBroadcastExchange (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (13) - : : : : : : +- CometBroadcastExchange (21) - : : : : : : +- CometProject (20) - : : : : : : +- CometFilter (19) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (18) - : : : : : +- CometBroadcastExchange (27) - : : : : : +- CometProject (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (24) - : : : : +- CometBroadcastExchange (33) - : : : : +- CometProject (32) - : : : : +- CometFilter (31) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - : : : +- CometBroadcastExchange (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (36) - : : +- CometBroadcastExchange (43) - : : +- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - : +- CometBroadcastExchange (48) - : +- CometFilter (47) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (46) - +- CometSort (57) - +- CometExchange (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (53) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#13)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) - -(5) CometBroadcastExchange -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(6) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Right output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_item_sk#4], [inv_item_sk#10], Inner, (inv_quantity_on_hand#12 < cs_quantity#7), BuildRight - -(7) CometProject -Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Condition : isnotnull(w_warehouse_sk#14) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [w_warehouse_sk#14, w_warehouse_name#15] - -(11) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -Right output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [inv_warehouse_sk#11], [w_warehouse_sk#14], Inner, BuildRight - -(12) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_desc#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [i_item_sk#16, i_item_desc#17] -Condition : isnotnull(i_item_sk#16) - -(15) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [i_item_sk#16, i_item_desc#17] - -(16) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -Right output [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [cs_item_sk#4], [i_item_sk#16], Inner, BuildRight - -(17) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#18, cd_marital_status#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_marital_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Condition : ((isnotnull(cd_marital_status#19) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#19, 1)) = D)) AND isnotnull(cd_demo_sk#18)) - -(20) CometProject -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Arguments: [cd_demo_sk#18], [cd_demo_sk#18] - -(21) CometBroadcastExchange -Input [1]: [cd_demo_sk#18] -Arguments: [cd_demo_sk#18] - -(22) CometBroadcastHashJoin -Left output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [cd_demo_sk#18] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#18], Inner, BuildRight - -(23) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] -Arguments: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_buy_potential#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Condition : ((isnotnull(hd_buy_potential#21) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#21, 15)) = >10000 )) AND isnotnull(hd_demo_sk#20)) - -(26) CometProject -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Arguments: [hd_demo_sk#20], [hd_demo_sk#20] - -(27) CometBroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: [hd_demo_sk#20] - -(28) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [hd_demo_sk#20] -Arguments: [cs_bill_hdemo_sk#3], [hd_demo_sk#20], Inner, BuildRight - -(29) CometProject -Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(31) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(32) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(33) CometBroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24] - -(34) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_sold_date_sk#8], [d_date_sk#22], Inner, BuildRight - -(35) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_week_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [d_date_sk#26, d_week_seq#27] -Condition : (isnotnull(d_week_seq#27) AND isnotnull(d_date_sk#26)) - -(38) CometBroadcastExchange -Input [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_date_sk#26, d_week_seq#27] - -(39) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_week_seq#24, inv_date_sk#13], [d_week_seq#27, d_date_sk#26], Inner, BuildRight - -(40) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#26, d_week_seq#27] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_date#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#28, d_date#29] -Condition : (isnotnull(d_date#29) AND isnotnull(d_date_sk#28)) - -(43) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_date#29] -Arguments: [d_date_sk#28, d_date#29] - -(44) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#28, d_date#29] -Arguments: [cs_ship_date_sk#1], [d_date_sk#28], Inner, (d_date#29 > date_add(d_date#23, 5)), BuildRight - -(45) CometProject -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#28, d_date#29] -Arguments: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(47) CometFilter -Input [1]: [p_promo_sk#30] -Condition : isnotnull(p_promo_sk#30) - -(48) CometBroadcastExchange -Input [1]: [p_promo_sk#30] -Arguments: [p_promo_sk#30] - -(49) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [1]: [p_promo_sk#30] -Arguments: [cs_promo_sk#5], [p_promo_sk#30], LeftOuter, BuildRight - -(50) CometProject -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#30] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(51) CometExchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(52) CometSort -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Condition : (isnotnull(cr_item_sk#31) AND isnotnull(cr_order_number#32)) - -(55) CometProject -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31, cr_order_number#32] - -(56) CometExchange -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(57) CometSort -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#31, cr_order_number#32], LeftOuter - -(59) CometProject -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#31, cr_order_number#32] -Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(60) CometHashAggregate -Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [partial_count(1)] - -(61) CometExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(62) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [count(1)] - -(63) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, 0, [total_cnt#37 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -(64) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (69) -+- * CometColumnarToRow (68) - +- CometProject (67) - +- CometFilter (66) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) - - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(67) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(68) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(69) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/extended.txt deleted file mode 100644 index 9000d27d1b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/extended.txt +++ /dev/null @@ -1,72 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 678f8c67df..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] - CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] - CometProject [w_warehouse_name,i_item_desc,d_week_seq] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometExchange [cs_item_sk,cs_order_number] #2 - CometProject [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,p_promo_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_date] - 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 [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_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_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_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_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 [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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_desc] #6 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange [cd_demo_sk] #7 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange [hd_demo_sk] #8 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [d_date_sk,d_week_seq] #10 - CometFilter [d_date_sk,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [d_date_sk,d_date] #11 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [p_promo_sk] #12 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/explain.txt deleted file mode 100644 index 9cb3536ab0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/explain.txt +++ /dev/null @@ -1,231 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (35) -+- CometSort (34) - +- CometColumnarExchange (33) - +- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * Filter (25) - : +- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (30) - +- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometNativeScan parquet spark_catalog.default.customer (26) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 40] -Output [1]: [d_date_sk#7] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_county#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_county, [Bronx County,Franklin Parish,Orange County,Williamson County]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [s_store_sk#8, s_county#9] -Condition : (s_county#9 IN (Williamson County,Franklin Parish,Bronx County,Orange County) AND isnotnull(s_store_sk#8)) - -(9) CometProject -Input [2]: [s_store_sk#8, s_county#9] -Arguments: [s_store_sk#8], [s_store_sk#8] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#8] - -(11) BroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] - -(14) CometNativeScan parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : ((((isnotnull(hd_vehicle_count#13) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#11, 15)) = >10000 ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#11, 15)) = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#12 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#13 as double)))))) > 1.0) END) AND isnotnull(hd_demo_sk#10)) - -(16) CometProject -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Arguments: [hd_demo_sk#10], [hd_demo_sk#10] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#10] - -(18) BroadcastExchange -Input [1]: [hd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#10] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [2]: [ss_customer_sk#1, ss_ticket_number#4] -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] - -(21) HashAggregate [codegen id : 4] -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#14] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(22) CometColumnarExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(24) HashAggregate [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#16] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] - -(25) Filter [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 1) AND (cnt#17 <= 5)) - -(26) CometNativeScan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(27) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(28) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#19, 10)) AS c_salutation#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#20, 20)) AS c_first_name#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#21, 30)) AS c_last_name#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#22, 1)) AS c_preferred_cust_flag#26] - -(29) CometColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(30) BroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#18] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 6] -Output [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(33) CometColumnarExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(34) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [cnt#17 DESC NULLS LAST] - -(35) CometColumnarToRow [codegen id : 7] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (40) -+- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometNativeScan parquet spark_catalog.default.date_dim (36) - - -(36) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#27, d_dom#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [d_date_sk#7, d_year#27, d_dom#28] -Condition : ((((isnotnull(d_dom#28) AND (d_dom#28 >= 1)) AND (d_dom#28 <= 2)) AND d_year#27 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(38) CometProject -Input [3]: [d_date_sk#7, d_year#27, d_dom#28] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(39) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(40) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/simplified.txt deleted file mode 100644 index 4d2a6a74ab..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/simplified.txt +++ /dev/null @@ -1,59 +0,0 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [cnt] #1 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometNativeScan 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/q73.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/explain.txt deleted file mode 100644 index 3bb13dcb30..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometSort (32) - +- CometExchange (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (24) - : +- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_county#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_county, [Bronx County,Franklin Parish,Orange County,Williamson County]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#10, s_county#11] -Condition : (s_county#11 IN (Williamson County,Franklin Parish,Bronx County,Orange County) AND isnotnull(s_store_sk#10)) - -(11) CometProject -Input [2]: [s_store_sk#10, s_county#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : ((((isnotnull(hd_vehicle_count#15) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#13, 15)) = >10000 ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#13, 15)) = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#14 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#15 as double)))))) > 1.0) END) AND isnotnull(hd_demo_sk#12)) - -(17) CometProject -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#12], [hd_demo_sk#12] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#12] -Arguments: [hd_demo_sk#12] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Right output [1]: [hd_demo_sk#12] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight - -(20) CometProject -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] -Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] - -(21) CometHashAggregate -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] - -(22) CometExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] - -(24) CometFilter -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 1) AND (cnt#17 <= 5)) - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(27) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#19, 10)) AS c_salutation#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#20, 20)) AS c_first_name#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#21, 30)) AS c_last_name#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#22, 1)) AS c_preferred_cust_flag#26] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(29) CometBroadcastHashJoin -Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Right output [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -(31) CometExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(32) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [cnt#17 DESC NULLS LAST] - -(33) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(36) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(37) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(38) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/simplified.txt deleted file mode 100644 index 227dd00f2b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometExchange [cnt] #1 - CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] - CometExchange [ss_ticket_number,ss_customer_sk] #2 - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - 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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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/q73/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/explain.txt deleted file mode 100644 index 3bb13dcb30..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometSort (32) - +- CometExchange (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (24) - : +- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_county#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_county, [Bronx County,Franklin Parish,Orange County,Williamson County]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#10, s_county#11] -Condition : (s_county#11 IN (Williamson County,Franklin Parish,Bronx County,Orange County) AND isnotnull(s_store_sk#10)) - -(11) CometProject -Input [2]: [s_store_sk#10, s_county#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : ((((isnotnull(hd_vehicle_count#15) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#13, 15)) = >10000 ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#13, 15)) = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#14 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#15 as double)))))) > 1.0) END) AND isnotnull(hd_demo_sk#12)) - -(17) CometProject -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#12], [hd_demo_sk#12] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#12] -Arguments: [hd_demo_sk#12] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Right output [1]: [hd_demo_sk#12] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight - -(20) CometProject -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] -Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] - -(21) CometHashAggregate -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] - -(22) CometExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] - -(24) CometFilter -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 1) AND (cnt#17 <= 5)) - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(27) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#19, 10)) AS c_salutation#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#20, 20)) AS c_first_name#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#21, 30)) AS c_last_name#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#22, 1)) AS c_preferred_cust_flag#26] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(29) CometBroadcastHashJoin -Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Right output [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -(31) CometExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(32) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [cnt#17 DESC NULLS LAST] - -(33) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(36) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(37) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(38) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/extended.txt deleted file mode 100644 index 3d9d23d2cc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/extended.txt +++ /dev/null @@ -1,41 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 227dd00f2b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometExchange [cnt] #1 - CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] - CometExchange [ss_ticket_number,ss_customer_sk] #2 - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - 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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/explain.txt deleted file mode 100644 index 174c3d72a2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/explain.txt +++ /dev/null @@ -1,513 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (79) -+- * Project (78) - +- * BroadcastHashJoin Inner BuildRight (77) - :- * Project (58) - : +- * BroadcastHashJoin Inner BuildRight (57) - : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- * HashAggregate (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * Project (28) - : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (26) - : : : +- * Filter (25) - : : : +- * ColumnarToRow (24) - : : : +- Scan parquet spark_catalog.default.store_sales (23) - : : +- ReusedExchange (29) - : +- BroadcastExchange (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- * CometColumnarToRow (53) - : +- CometColumnarExchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (41) - : : : +- CometProject (40) - : : : +- CometFilter (39) - : : : +- CometNativeScan parquet spark_catalog.default.customer (38) - : : +- BroadcastExchange (45) - : : +- * Filter (44) - : : +- * ColumnarToRow (43) - : : +- Scan parquet spark_catalog.default.web_sales (42) - : +- ReusedExchange (48) - +- BroadcastExchange (76) - +- * HashAggregate (75) - +- * CometColumnarToRow (74) - +- CometColumnarExchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * CometColumnarToRow (62) - : : +- CometProject (61) - : : +- CometFilter (60) - : : +- CometNativeScan parquet spark_catalog.default.customer (59) - : +- BroadcastExchange (66) - : +- * Filter (65) - : +- * ColumnarToRow (64) - : +- Scan parquet spark_catalog.default.web_sales (63) - +- ReusedExchange (69) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)))) - -(3) CometProject -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)) AS c_customer_id#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#3, 20)) AS c_first_name#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#4, 30)) AS c_last_name#7] - -(4) CometColumnarToRow [codegen id : 3] -Input [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] - -(5) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(7) Filter [codegen id : 1] -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Condition : isnotnull(ss_customer_sk#8) - -(8) BroadcastExchange -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#8] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(11) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#12, d_year#13] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#10] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] - -(14) HashAggregate [codegen id : 3] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] -Aggregate Attributes [1]: [sum#14] -Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] - -(15) CometColumnarExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 16] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] - -(17) HashAggregate [codegen id : 16] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#9))#16] -Results [2]: [c_customer_id#5 AS customer_id#17, MakeDecimal(sum(UnscaledValue(ss_net_paid#9))#16,17,2) AS year_total#18] - -(18) Filter [codegen id : 16] -Input [2]: [customer_id#17, year_total#18] -Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) - -(19) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(20) CometFilter -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Condition : (isnotnull(c_customer_sk#19) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#20, 16)))) - -(21) CometProject -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Arguments: [c_customer_sk#19, c_customer_id#23, c_first_name#24, c_last_name#25], [c_customer_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#20, 16)) AS c_customer_id#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#21, 20)) AS c_first_name#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#22, 30)) AS c_last_name#25] - -(22) CometColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#19, c_customer_id#23, c_first_name#24, c_last_name#25] - -(23) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#26, ss_net_paid#27, ss_sold_date_sk#28] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#29)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#26, ss_net_paid#27, ss_sold_date_sk#28] - -(25) Filter [codegen id : 4] -Input [3]: [ss_customer_sk#26, ss_net_paid#27, ss_sold_date_sk#28] -Condition : isnotnull(ss_customer_sk#26) - -(26) BroadcastExchange -Input [3]: [ss_customer_sk#26, ss_net_paid#27, ss_sold_date_sk#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#19] -Right keys [1]: [ss_customer_sk#26] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, ss_net_paid#27, ss_sold_date_sk#28] -Input [7]: [c_customer_sk#19, c_customer_id#23, c_first_name#24, c_last_name#25, ss_customer_sk#26, ss_net_paid#27, ss_sold_date_sk#28] - -(29) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#30, d_year#31] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#28] -Right keys [1]: [d_date_sk#30] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, ss_net_paid#27, d_year#31] -Input [7]: [c_customer_id#23, c_first_name#24, c_last_name#25, ss_net_paid#27, ss_sold_date_sk#28, d_date_sk#30, d_year#31] - -(32) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, ss_net_paid#27, d_year#31] -Keys [4]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#27))] -Aggregate Attributes [1]: [sum#32] -Results [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31, sum#33] - -(33) CometColumnarExchange -Input [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31, sum#33] -Arguments: hashpartitioning(c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 7] -Input [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31, sum#33] - -(35) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31, sum#33] -Keys [4]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31] -Functions [1]: [sum(UnscaledValue(ss_net_paid#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#27))#16] -Results [4]: [c_customer_id#23 AS customer_id#34, c_first_name#24 AS customer_first_name#35, c_last_name#25 AS customer_last_name#36, MakeDecimal(sum(UnscaledValue(ss_net_paid#27))#16,17,2) AS year_total#37] - -(36) BroadcastExchange -Input [4]: [customer_id#34, customer_first_name#35, customer_last_name#36, year_total#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(37) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#34] -Join type: Inner -Join condition: None - -(38) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#38, c_customer_id#39, c_first_name#40, c_last_name#41] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(39) CometFilter -Input [4]: [c_customer_sk#38, c_customer_id#39, c_first_name#40, c_last_name#41] -Condition : (isnotnull(c_customer_sk#38) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#39, 16)))) - -(40) CometProject -Input [4]: [c_customer_sk#38, c_customer_id#39, c_first_name#40, c_last_name#41] -Arguments: [c_customer_sk#38, c_customer_id#42, c_first_name#43, c_last_name#44], [c_customer_sk#38, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#39, 16)) AS c_customer_id#42, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#40, 20)) AS c_first_name#43, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#41, 30)) AS c_last_name#44] - -(41) CometColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#38, c_customer_id#42, c_first_name#43, c_last_name#44] - -(42) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#45, ws_net_paid#46, ws_sold_date_sk#47] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#45, ws_net_paid#46, ws_sold_date_sk#47] - -(44) Filter [codegen id : 8] -Input [3]: [ws_bill_customer_sk#45, ws_net_paid#46, ws_sold_date_sk#47] -Condition : isnotnull(ws_bill_customer_sk#45) - -(45) BroadcastExchange -Input [3]: [ws_bill_customer_sk#45, ws_net_paid#46, ws_sold_date_sk#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -(46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#38] -Right keys [1]: [ws_bill_customer_sk#45] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 10] -Output [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, ws_net_paid#46, ws_sold_date_sk#47] -Input [7]: [c_customer_sk#38, c_customer_id#42, c_first_name#43, c_last_name#44, ws_bill_customer_sk#45, ws_net_paid#46, ws_sold_date_sk#47] - -(48) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#48, d_year#49] - -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#47] -Right keys [1]: [d_date_sk#48] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 10] -Output [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, ws_net_paid#46, d_year#49] -Input [7]: [c_customer_id#42, c_first_name#43, c_last_name#44, ws_net_paid#46, ws_sold_date_sk#47, d_date_sk#48, d_year#49] - -(51) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, ws_net_paid#46, d_year#49] -Keys [4]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#46))] -Aggregate Attributes [1]: [sum#50] -Results [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49, sum#51] - -(52) CometColumnarExchange -Input [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49, sum#51] -Arguments: hashpartitioning(c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(53) CometColumnarToRow [codegen id : 11] -Input [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49, sum#51] - -(54) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49, sum#51] -Keys [4]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49] -Functions [1]: [sum(UnscaledValue(ws_net_paid#46))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#46))#52] -Results [2]: [c_customer_id#42 AS customer_id#53, MakeDecimal(sum(UnscaledValue(ws_net_paid#46))#52,17,2) AS year_total#54] - -(55) Filter [codegen id : 11] -Input [2]: [customer_id#53, year_total#54] -Condition : (isnotnull(year_total#54) AND (year_total#54 > 0.00)) - -(56) BroadcastExchange -Input [2]: [customer_id#53, year_total#54] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(57) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#53] -Join type: Inner -Join condition: None - -(58) Project [codegen id : 16] -Output [7]: [customer_id#17, year_total#18, customer_id#34, customer_first_name#35, customer_last_name#36, year_total#37, year_total#54] -Input [8]: [customer_id#17, year_total#18, customer_id#34, customer_first_name#35, customer_last_name#36, year_total#37, customer_id#53, year_total#54] - -(59) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(60) CometFilter -Input [4]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58] -Condition : (isnotnull(c_customer_sk#55) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#56, 16)))) - -(61) CometProject -Input [4]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58] -Arguments: [c_customer_sk#55, c_customer_id#59, c_first_name#60, c_last_name#61], [c_customer_sk#55, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#56, 16)) AS c_customer_id#59, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#57, 20)) AS c_first_name#60, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#58, 30)) AS c_last_name#61] - -(62) CometColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#55, c_customer_id#59, c_first_name#60, c_last_name#61] - -(63) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#64), dynamicpruningexpression(ws_sold_date_sk#64 IN dynamicpruning#29)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(64) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] - -(65) Filter [codegen id : 12] -Input [3]: [ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] -Condition : isnotnull(ws_bill_customer_sk#62) - -(66) BroadcastExchange -Input [3]: [ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#55] -Right keys [1]: [ws_bill_customer_sk#62] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 14] -Output [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, ws_sold_date_sk#64] -Input [7]: [c_customer_sk#55, c_customer_id#59, c_first_name#60, c_last_name#61, ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] - -(69) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#65, d_year#66] - -(70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#64] -Right keys [1]: [d_date_sk#65] -Join type: Inner -Join condition: None - -(71) Project [codegen id : 14] -Output [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, d_year#66] -Input [7]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, ws_sold_date_sk#64, d_date_sk#65, d_year#66] - -(72) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, d_year#66] -Keys [4]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#63))] -Aggregate Attributes [1]: [sum#67] -Results [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66, sum#68] - -(73) CometColumnarExchange -Input [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66, sum#68] -Arguments: hashpartitioning(c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(74) CometColumnarToRow [codegen id : 15] -Input [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66, sum#68] - -(75) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66, sum#68] -Keys [4]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66] -Functions [1]: [sum(UnscaledValue(ws_net_paid#63))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#63))#52] -Results [2]: [c_customer_id#59 AS customer_id#69, MakeDecimal(sum(UnscaledValue(ws_net_paid#63))#52,17,2) AS year_total#70] - -(76) BroadcastExchange -Input [2]: [customer_id#69, year_total#70] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] - -(77) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#69] -Join type: Inner -Join condition: (CASE WHEN (year_total#54 > 0.00) THEN (year_total#70 / year_total#54) END > CASE WHEN (year_total#18 > 0.00) THEN (year_total#37 / year_total#18) END) - -(78) Project [codegen id : 16] -Output [3]: [customer_id#34, customer_first_name#35, customer_last_name#36] -Input [9]: [customer_id#17, year_total#18, customer_id#34, customer_first_name#35, customer_last_name#36, year_total#37, year_total#54, customer_id#69, year_total#70] - -(79) TakeOrderedAndProject -Input [3]: [customer_id#34, customer_first_name#35, customer_last_name#36] -Arguments: 100, [customer_id#34 ASC NULLS FIRST, customer_id#34 ASC NULLS FIRST, customer_id#34 ASC NULLS FIRST], [customer_id#34, customer_first_name#35, customer_last_name#36] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometFilter (81) - +- CometNativeScan parquet spark_catalog.default.date_dim (80) - - -(80) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(81) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(82) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(83) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 -BroadcastExchange (87) -+- * CometColumnarToRow (86) - +- CometFilter (85) - +- CometNativeScan parquet spark_catalog.default.date_dim (84) - - -(84) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#30, d_year#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(85) CometFilter -Input [2]: [d_date_sk#30, d_year#31] -Condition : (((isnotnull(d_year#31) AND (d_year#31 = 2002)) AND d_year#31 IN (2001,2002)) AND isnotnull(d_date_sk#30)) - -(86) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#30, d_year#31] - -(87) BroadcastExchange -Input [2]: [d_date_sk#30, d_year#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] - -Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#11 - -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#64 IN dynamicpruning#29 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/simplified.txt deleted file mode 100644 index f488e18c7c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/simplified.txt +++ /dev/null @@ -1,130 +0,0 @@ -TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - WholeStageCodegen (16) - Project [customer_id,customer_first_name,customer_last_name] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/explain.txt deleted file mode 100644 index ce451747c0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,470 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometProject (53) - : +- CometBroadcastHashJoin (52) - : :- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (51) - : +- CometFilter (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : +- CometBroadcastExchange (41) - : : +- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) - : +- ReusedExchange (44) - +- CometBroadcastExchange (68) - +- CometHashAggregate (67) - +- CometExchange (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometFilter (55) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (62) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)))) - -(3) CometProject -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)) AS c_customer_id#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#3, 20)) AS c_first_name#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#4, 30)) AS c_last_name#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Condition : isnotnull(ss_customer_sk#8) - -(6) CometBroadcastExchange -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_sk#1], [ss_customer_sk#8], Inner, BuildRight - -(8) CometProject -Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(12) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#10], [d_date_sk#12], Inner, BuildRight - -(13) CometProject -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] - -(14) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] - -(15) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] - -(17) CometFilter -Input [2]: [customer_id#15, year_total#16] -Condition : (isnotnull(year_total#16) AND (year_total#16 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#18, 16)))) - -(20) CometProject -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Arguments: [c_customer_sk#17, c_customer_id#21, c_first_name#22, c_last_name#23], [c_customer_sk#17, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#18, 16)) AS c_customer_id#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#19, 20)) AS c_first_name#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#20, 30)) AS c_last_name#23] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_customer_sk#24) - -(23) CometBroadcastExchange -Input [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -Arguments: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] - -(24) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#17, c_customer_id#21, c_first_name#22, c_last_name#23] -Right output [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -Arguments: [c_customer_sk#17], [ss_customer_sk#24], Inner, BuildRight - -(25) CometProject -Input [7]: [c_customer_sk#17, c_customer_id#21, c_first_name#22, c_last_name#23, ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -Arguments: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26], [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_year#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#28, d_year#29] -Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_year#29] -Arguments: [d_date_sk#28, d_year#29] - -(29) CometBroadcastHashJoin -Left output [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26] -Right output [2]: [d_date_sk#28, d_year#29] -Arguments: [ss_sold_date_sk#26], [d_date_sk#28], Inner, BuildRight - -(30) CometProject -Input [7]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26, d_date_sk#28, d_year#29] -Arguments: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29], [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29] - -(31) CometHashAggregate -Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29] -Keys [4]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#25))] - -(32) CometExchange -Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, sum#30] -Arguments: hashpartitioning(c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, sum#30] -Keys [4]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29] -Functions [1]: [sum(UnscaledValue(ss_net_paid#25))] - -(34) CometBroadcastExchange -Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] -Arguments: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#15, year_total#16] -Right output [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] -Arguments: [customer_id#15], [customer_id#31], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Condition : (isnotnull(c_customer_sk#35) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#36, 16)))) - -(38) CometProject -Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Arguments: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#36, 16)) AS c_customer_id#39, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#37, 20)) AS c_first_name#40, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#38, 30)) AS c_last_name#41] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#45)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_bill_customer_sk#42) - -(41) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Arguments: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] - -(42) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41] -Right output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Arguments: [c_customer_sk#35], [ws_bill_customer_sk#42], Inner, BuildRight - -(43) CometProject -Input [7]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Arguments: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44], [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#46, d_year#47] - -(45) CometBroadcastHashJoin -Left output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [ws_sold_date_sk#44], [d_date_sk#46], Inner, BuildRight - -(46) CometProject -Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#46, d_year#47] -Arguments: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47], [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47] - -(47) CometHashAggregate -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] - -(48) CometExchange -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#48] -Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#48] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47] -Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] - -(50) CometFilter -Input [2]: [customer_id#49, year_total#50] -Condition : (isnotnull(year_total#50) AND (year_total#50 > 0.00)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#49, year_total#50] -Arguments: [customer_id#49, year_total#50] - -(52) CometBroadcastHashJoin -Left output [6]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] -Right output [2]: [customer_id#49, year_total#50] -Arguments: [customer_id#15], [customer_id#49], Inner, BuildRight - -(53) CometProject -Input [8]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, customer_id#49, year_total#50] -Arguments: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50], [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] -Condition : (isnotnull(c_customer_sk#51) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#52, 16)))) - -(56) CometProject -Input [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] -Arguments: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57], [c_customer_sk#51, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#52, 16)) AS c_customer_id#55, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#53, 20)) AS c_first_name#56, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#54, 30)) AS c_last_name#57] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#61)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] -Condition : isnotnull(ws_bill_customer_sk#58) - -(59) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] -Arguments: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] - -(60) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57] -Right output [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] -Arguments: [c_customer_sk#51], [ws_bill_customer_sk#58], Inner, BuildRight - -(61) CometProject -Input [7]: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57, ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] -Arguments: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60], [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#62, d_year#63] - -(63) CometBroadcastHashJoin -Left output [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60] -Right output [2]: [d_date_sk#62, d_year#63] -Arguments: [ws_sold_date_sk#60], [d_date_sk#62], Inner, BuildRight - -(64) CometProject -Input [7]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60, d_date_sk#62, d_year#63] -Arguments: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63], [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63] - -(65) CometHashAggregate -Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63] -Keys [4]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#59))] - -(66) CometExchange -Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, sum#64] -Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, sum#64] -Keys [4]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63] -Functions [1]: [sum(UnscaledValue(ws_net_paid#59))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#65, year_total#66] -Arguments: [customer_id#65, year_total#66] - -(69) CometBroadcastHashJoin -Left output [7]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50] -Right output [2]: [customer_id#65, year_total#66] -Arguments: [customer_id#15], [customer_id#65], Inner, (CASE WHEN (year_total#50 > 0.00) THEN (year_total#66 / year_total#50) END > CASE WHEN (year_total#16 > 0.00) THEN (year_total#34 / year_total#16) END), BuildRight - -(70) CometProject -Input [9]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50, customer_id#65, year_total#66] -Arguments: [customer_id#31, customer_first_name#32, customer_last_name#33], [customer_id#31, customer_first_name#32, customer_last_name#33] - -(71) CometTakeOrderedAndProject -Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#31 ASC NULLS FIRST,customer_id#31 ASC NULLS FIRST,customer_id#31 ASC NULLS FIRST], output=[customer_id#31,customer_first_name#32,customer_last_name#33]), [customer_id#31, customer_first_name#32, customer_last_name#33], 100, 0, [customer_id#31 ASC NULLS FIRST, customer_id#31 ASC NULLS FIRST, customer_id#31 ASC NULLS FIRST], [customer_id#31, customer_first_name#32, customer_last_name#33] - -(72) CometColumnarToRow [codegen id : 1] -Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(75) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(76) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#27 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_year#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [2]: [d_date_sk#28, d_year#29] -Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) - -(79) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#28, d_year#29] - -(80) BroadcastExchange -Input [2]: [d_date_sk#28, d_year#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#11 - -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#27 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/simplified.txt deleted file mode 100644 index 72e39422e0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - CometProject [customer_id,customer_first_name,customer_last_name] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 - CometHashAggregate [d_year,sum] [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/explain.txt deleted file mode 100644 index ce451747c0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/explain.txt +++ /dev/null @@ -1,470 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometProject (53) - : +- CometBroadcastHashJoin (52) - : :- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (51) - : +- CometFilter (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : +- CometBroadcastExchange (41) - : : +- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) - : +- ReusedExchange (44) - +- CometBroadcastExchange (68) - +- CometHashAggregate (67) - +- CometExchange (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometFilter (55) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (62) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)))) - -(3) CometProject -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)) AS c_customer_id#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#3, 20)) AS c_first_name#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#4, 30)) AS c_last_name#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Condition : isnotnull(ss_customer_sk#8) - -(6) CometBroadcastExchange -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_sk#1], [ss_customer_sk#8], Inner, BuildRight - -(8) CometProject -Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(12) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#10], [d_date_sk#12], Inner, BuildRight - -(13) CometProject -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] - -(14) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] - -(15) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] - -(17) CometFilter -Input [2]: [customer_id#15, year_total#16] -Condition : (isnotnull(year_total#16) AND (year_total#16 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#18, 16)))) - -(20) CometProject -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Arguments: [c_customer_sk#17, c_customer_id#21, c_first_name#22, c_last_name#23], [c_customer_sk#17, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#18, 16)) AS c_customer_id#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#19, 20)) AS c_first_name#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#20, 30)) AS c_last_name#23] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_customer_sk#24) - -(23) CometBroadcastExchange -Input [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -Arguments: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] - -(24) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#17, c_customer_id#21, c_first_name#22, c_last_name#23] -Right output [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -Arguments: [c_customer_sk#17], [ss_customer_sk#24], Inner, BuildRight - -(25) CometProject -Input [7]: [c_customer_sk#17, c_customer_id#21, c_first_name#22, c_last_name#23, ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -Arguments: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26], [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_year#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#28, d_year#29] -Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_year#29] -Arguments: [d_date_sk#28, d_year#29] - -(29) CometBroadcastHashJoin -Left output [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26] -Right output [2]: [d_date_sk#28, d_year#29] -Arguments: [ss_sold_date_sk#26], [d_date_sk#28], Inner, BuildRight - -(30) CometProject -Input [7]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26, d_date_sk#28, d_year#29] -Arguments: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29], [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29] - -(31) CometHashAggregate -Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29] -Keys [4]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#25))] - -(32) CometExchange -Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, sum#30] -Arguments: hashpartitioning(c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, sum#30] -Keys [4]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29] -Functions [1]: [sum(UnscaledValue(ss_net_paid#25))] - -(34) CometBroadcastExchange -Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] -Arguments: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#15, year_total#16] -Right output [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] -Arguments: [customer_id#15], [customer_id#31], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Condition : (isnotnull(c_customer_sk#35) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#36, 16)))) - -(38) CometProject -Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Arguments: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#36, 16)) AS c_customer_id#39, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#37, 20)) AS c_first_name#40, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#38, 30)) AS c_last_name#41] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#45)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_bill_customer_sk#42) - -(41) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Arguments: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] - -(42) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41] -Right output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Arguments: [c_customer_sk#35], [ws_bill_customer_sk#42], Inner, BuildRight - -(43) CometProject -Input [7]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Arguments: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44], [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#46, d_year#47] - -(45) CometBroadcastHashJoin -Left output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [ws_sold_date_sk#44], [d_date_sk#46], Inner, BuildRight - -(46) CometProject -Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#46, d_year#47] -Arguments: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47], [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47] - -(47) CometHashAggregate -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] - -(48) CometExchange -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#48] -Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#48] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47] -Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] - -(50) CometFilter -Input [2]: [customer_id#49, year_total#50] -Condition : (isnotnull(year_total#50) AND (year_total#50 > 0.00)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#49, year_total#50] -Arguments: [customer_id#49, year_total#50] - -(52) CometBroadcastHashJoin -Left output [6]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] -Right output [2]: [customer_id#49, year_total#50] -Arguments: [customer_id#15], [customer_id#49], Inner, BuildRight - -(53) CometProject -Input [8]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, customer_id#49, year_total#50] -Arguments: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50], [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] -Condition : (isnotnull(c_customer_sk#51) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#52, 16)))) - -(56) CometProject -Input [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] -Arguments: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57], [c_customer_sk#51, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#52, 16)) AS c_customer_id#55, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#53, 20)) AS c_first_name#56, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#54, 30)) AS c_last_name#57] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#61)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] -Condition : isnotnull(ws_bill_customer_sk#58) - -(59) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] -Arguments: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] - -(60) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57] -Right output [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] -Arguments: [c_customer_sk#51], [ws_bill_customer_sk#58], Inner, BuildRight - -(61) CometProject -Input [7]: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57, ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] -Arguments: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60], [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#62, d_year#63] - -(63) CometBroadcastHashJoin -Left output [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60] -Right output [2]: [d_date_sk#62, d_year#63] -Arguments: [ws_sold_date_sk#60], [d_date_sk#62], Inner, BuildRight - -(64) CometProject -Input [7]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60, d_date_sk#62, d_year#63] -Arguments: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63], [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63] - -(65) CometHashAggregate -Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63] -Keys [4]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#59))] - -(66) CometExchange -Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, sum#64] -Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, sum#64] -Keys [4]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63] -Functions [1]: [sum(UnscaledValue(ws_net_paid#59))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#65, year_total#66] -Arguments: [customer_id#65, year_total#66] - -(69) CometBroadcastHashJoin -Left output [7]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50] -Right output [2]: [customer_id#65, year_total#66] -Arguments: [customer_id#15], [customer_id#65], Inner, (CASE WHEN (year_total#50 > 0.00) THEN (year_total#66 / year_total#50) END > CASE WHEN (year_total#16 > 0.00) THEN (year_total#34 / year_total#16) END), BuildRight - -(70) CometProject -Input [9]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50, customer_id#65, year_total#66] -Arguments: [customer_id#31, customer_first_name#32, customer_last_name#33], [customer_id#31, customer_first_name#32, customer_last_name#33] - -(71) CometTakeOrderedAndProject -Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#31 ASC NULLS FIRST,customer_id#31 ASC NULLS FIRST,customer_id#31 ASC NULLS FIRST], output=[customer_id#31,customer_first_name#32,customer_last_name#33]), [customer_id#31, customer_first_name#32, customer_last_name#33], 100, 0, [customer_id#31 ASC NULLS FIRST, customer_id#31 ASC NULLS FIRST, customer_id#31 ASC NULLS FIRST], [customer_id#31, customer_first_name#32, customer_last_name#33] - -(72) CometColumnarToRow [codegen id : 1] -Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(75) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(76) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#27 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_year#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [2]: [d_date_sk#28, d_year#29] -Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) - -(79) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#28, d_year#29] - -(80) BroadcastExchange -Input [2]: [d_date_sk#28, d_year#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#11 - -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#27 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/extended.txt deleted file mode 100644 index 476c7be954..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/extended.txt +++ /dev/null @@ -1,90 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 72e39422e0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - CometProject [customer_id,customer_first_name,customer_last_name] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 - CometHashAggregate [d_year,sum] [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/explain.txt deleted file mode 100644 index 87315ef48b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/explain.txt +++ /dev/null @@ -1,754 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (127) -+- CometTakeOrderedAndProject (126) - +- CometProject (125) - +- CometSortMergeJoin (124) - :- CometSort (68) - : +- CometExchange (67) - : +- CometFilter (66) - : +- CometHashAggregate (65) - : +- CometExchange (64) - : +- CometHashAggregate (63) - : +- CometHashAggregate (62) - : +- CometExchange (61) - : +- CometHashAggregate (60) - : +- CometUnion (59) - : :- CometProject (22) - : : +- CometSortMergeJoin (21) - : : :- CometSort (15) - : : : +- CometColumnarExchange (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : : +- ReusedExchange (11) - : : +- CometSort (20) - : : +- CometExchange (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (40) - : : +- CometSortMergeJoin (39) - : : :- CometSort (33) - : : : +- CometColumnarExchange (32) - : : : +- * Project (31) - : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : :- * Project (28) - : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : :- * Filter (25) - : : : : : +- * ColumnarToRow (24) - : : : : : +- Scan parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (26) - : : : +- ReusedExchange (29) - : : +- CometSort (38) - : : +- CometExchange (37) - : : +- CometProject (36) - : : +- CometFilter (35) - : : +- CometNativeScan parquet spark_catalog.default.store_returns (34) - : +- CometProject (58) - : +- CometSortMergeJoin (57) - : :- CometSort (51) - : : +- CometColumnarExchange (50) - : : +- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * Project (46) - : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : :- * Filter (43) - : : : : +- * ColumnarToRow (42) - : : : : +- Scan parquet spark_catalog.default.web_sales (41) - : : : +- ReusedExchange (44) - : : +- ReusedExchange (47) - : +- CometSort (56) - : +- CometExchange (55) - : +- CometProject (54) - : +- CometFilter (53) - : +- CometNativeScan parquet spark_catalog.default.web_returns (52) - +- CometSort (123) - +- CometExchange (122) - +- CometFilter (121) - +- CometHashAggregate (120) - +- CometExchange (119) - +- CometHashAggregate (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometUnion (114) - :- CometProject (83) - : +- CometSortMergeJoin (82) - : :- CometSort (79) - : : +- CometColumnarExchange (78) - : : +- * Project (77) - : : +- * BroadcastHashJoin Inner BuildRight (76) - : : :- * Project (74) - : : : +- * BroadcastHashJoin Inner BuildRight (73) - : : : :- * Filter (71) - : : : : +- * ColumnarToRow (70) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (69) - : : : +- ReusedExchange (72) - : : +- ReusedExchange (75) - : +- CometSort (81) - : +- ReusedExchange (80) - :- CometProject (98) - : +- CometSortMergeJoin (97) - : :- CometSort (94) - : : +- CometColumnarExchange (93) - : : +- * Project (92) - : : +- * BroadcastHashJoin Inner BuildRight (91) - : : :- * Project (89) - : : : +- * BroadcastHashJoin Inner BuildRight (88) - : : : :- * Filter (86) - : : : : +- * ColumnarToRow (85) - : : : : +- Scan parquet spark_catalog.default.store_sales (84) - : : : +- ReusedExchange (87) - : : +- ReusedExchange (90) - : +- CometSort (96) - : +- ReusedExchange (95) - +- CometProject (113) - +- CometSortMergeJoin (112) - :- CometSort (109) - : +- CometColumnarExchange (108) - : +- * Project (107) - : +- * BroadcastHashJoin Inner BuildRight (106) - : :- * Project (104) - : : +- * BroadcastHashJoin Inner BuildRight (103) - : : :- * Filter (101) - : : : +- * ColumnarToRow (100) - : : : +- Scan parquet spark_catalog.default.web_sales (99) - : : +- ReusedExchange (102) - : +- ReusedExchange (105) - +- CometSort (111) - +- ReusedExchange (110) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] - -(3) Filter [codegen id : 3] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((isnotnull(i_category#11) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#11, 50)) = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) - -(6) CometProject -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(7) CometColumnarToRow [codegen id : 1] -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(8) BroadcastExchange -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(11) ReusedExchange [Reuses operator id: 131] -Output [2]: [d_date_sk#13, d_year#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] - -(14) CometColumnarExchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(15) CometSort -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] - -(16) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) - -(18) CometProject -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(19) CometExchange -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter - -(22) CometProject -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21], [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] - -(23) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 6] -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] - -(25) Filter [codegen id : 6] -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#22) - -(26) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#22] -Right keys [1]: [i_item_sk#27] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] -Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] - -(29) ReusedExchange [Reuses operator id: 131] -Output [2]: [d_date_sk#32, d_year#33] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#32] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_date_sk#32, d_year#33] - -(32) CometColumnarExchange -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(33) CometSort -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33], [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] - -(34) CometNativeScan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37, sr_returned_date_sk#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [5]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37, sr_returned_date_sk#38] -Condition : (isnotnull(sr_ticket_number#35) AND isnotnull(sr_item_sk#34)) - -(36) CometProject -Input [5]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37, sr_returned_date_sk#38] -Arguments: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37], [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] - -(37) CometExchange -Input [4]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: hashpartitioning(sr_ticket_number#35, sr_item_sk#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(38) CometSort -Input [4]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37], [sr_ticket_number#35 ASC NULLS FIRST, sr_item_sk#34 ASC NULLS FIRST] - -(39) CometSortMergeJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Right output [4]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#35, sr_item_sk#34], LeftOuter - -(40) CometProject -Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33, sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: [d_year#33, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, sales_cnt#39, sales_amt#40], [d_year#33, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, (ss_quantity#24 - coalesce(sr_return_quantity#36, 0)) AS sales_cnt#39, (ss_ext_sales_price#25 - coalesce(sr_return_amt#37, 0.00)) AS sales_amt#40] - -(41) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(42) ColumnarToRow [codegen id : 9] -Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] - -(43) Filter [codegen id : 9] -Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -Condition : isnotnull(ws_item_sk#41) - -(44) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] - -(45) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#41] -Right keys [1]: [i_item_sk#46] -Join type: Inner -Join condition: None - -(46) Project [codegen id : 9] -Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] -Input [10]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] - -(47) ReusedExchange [Reuses operator id: 131] -Output [2]: [d_date_sk#51, d_year#52] - -(48) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#45] -Right keys [1]: [d_date_sk#51] -Join type: Inner -Join condition: None - -(49) Project [codegen id : 9] -Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Input [11]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_date_sk#51, d_year#52] - -(50) CometColumnarExchange -Input [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Arguments: hashpartitioning(ws_order_number#42, ws_item_sk#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(51) CometSort -Input [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Arguments: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52], [ws_order_number#42 ASC NULLS FIRST, ws_item_sk#41 ASC NULLS FIRST] - -(52) CometNativeScan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56, wr_returned_date_sk#57] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(53) CometFilter -Input [5]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56, wr_returned_date_sk#57] -Condition : (isnotnull(wr_order_number#54) AND isnotnull(wr_item_sk#53)) - -(54) CometProject -Input [5]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56, wr_returned_date_sk#57] -Arguments: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56], [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] - -(55) CometExchange -Input [4]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: hashpartitioning(wr_order_number#54, wr_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(56) CometSort -Input [4]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56], [wr_order_number#54 ASC NULLS FIRST, wr_item_sk#53 ASC NULLS FIRST] - -(57) CometSortMergeJoin -Left output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Right output [4]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: [ws_order_number#42, ws_item_sk#41], [wr_order_number#54, wr_item_sk#53], LeftOuter - -(58) CometProject -Input [13]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52, wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: [d_year#52, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, sales_cnt#58, sales_amt#59], [d_year#52, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, (ws_quantity#43 - coalesce(wr_return_quantity#55, 0)) AS sales_cnt#58, (ws_ext_sales_price#44 - coalesce(wr_return_amt#56, 0.00)) AS sales_amt#59] - -(59) CometUnion -Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#33, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, sales_cnt#39, sales_amt#40] -Child 2 Input [7]: [d_year#52, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, sales_cnt#58, sales_amt#59] - -(60) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(61) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(62) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(63) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(64) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#60, sum#61] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(65) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#60, sum#61] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(66) CometFilter -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Condition : isnotnull(sales_cnt#62) - -(67) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(68) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] - -(69) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#68), dynamicpruningexpression(cs_sold_date_sk#68 IN dynamicpruning#69)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(70) ColumnarToRow [codegen id : 12] -Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] - -(71) Filter [codegen id : 12] -Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] -Condition : isnotnull(cs_item_sk#64) - -(72) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] - -(73) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cs_item_sk#64] -Right keys [1]: [i_item_sk#70] -Join type: Inner -Join condition: None - -(74) Project [codegen id : 12] -Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -Input [10]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] - -(75) ReusedExchange [Reuses operator id: 135] -Output [2]: [d_date_sk#75, d_year#76] - -(76) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cs_sold_date_sk#68] -Right keys [1]: [d_date_sk#75] -Join type: Inner -Join condition: None - -(77) Project [codegen id : 12] -Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Input [11]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_date_sk#75, d_year#76] - -(78) CometColumnarExchange -Input [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Arguments: hashpartitioning(cs_order_number#65, cs_item_sk#64, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(79) CometSort -Input [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Arguments: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76], [cs_order_number#65 ASC NULLS FIRST, cs_item_sk#64 ASC NULLS FIRST] - -(80) ReusedExchange [Reuses operator id: 19] -Output [4]: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] - -(81) CometSort -Input [4]: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] -Arguments: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80], [cr_order_number#78 ASC NULLS FIRST, cr_item_sk#77 ASC NULLS FIRST] - -(82) CometSortMergeJoin -Left output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Right output [4]: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] -Arguments: [cs_order_number#65, cs_item_sk#64], [cr_order_number#78, cr_item_sk#77], LeftOuter - -(83) CometProject -Input [13]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76, cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] -Arguments: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#81, sales_amt#82], [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, (cs_quantity#66 - coalesce(cr_return_quantity#79, 0)) AS sales_cnt#81, (cs_ext_sales_price#67 - coalesce(cr_return_amount#80, 0.00)) AS sales_amt#82] - -(84) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#69)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(85) ColumnarToRow [codegen id : 15] -Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] - -(86) Filter [codegen id : 15] -Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Condition : isnotnull(ss_item_sk#83) - -(87) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#88, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92] - -(88) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ss_item_sk#83] -Right keys [1]: [i_item_sk#88] -Join type: Inner -Join condition: None - -(89) Project [codegen id : 15] -Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92] -Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#88, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92] - -(90) ReusedExchange [Reuses operator id: 135] -Output [2]: [d_date_sk#93, d_year#94] - -(91) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ss_sold_date_sk#87] -Right keys [1]: [d_date_sk#93] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 15] -Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, d_year#94] -Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, d_date_sk#93, d_year#94] - -(93) CometColumnarExchange -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, d_year#94] -Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(94) CometSort -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, d_year#94] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, d_year#94], [ss_ticket_number#84 ASC NULLS FIRST, ss_item_sk#83 ASC NULLS FIRST] - -(95) ReusedExchange [Reuses operator id: 37] -Output [4]: [sr_item_sk#95, sr_ticket_number#96, sr_return_quantity#97, sr_return_amt#98] - -(96) CometSort -Input [4]: [sr_item_sk#95, sr_ticket_number#96, sr_return_quantity#97, sr_return_amt#98] -Arguments: [sr_item_sk#95, sr_ticket_number#96, sr_return_quantity#97, sr_return_amt#98], [sr_ticket_number#96 ASC NULLS FIRST, sr_item_sk#95 ASC NULLS FIRST] - -(97) CometSortMergeJoin -Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, d_year#94] -Right output [4]: [sr_item_sk#95, sr_ticket_number#96, sr_return_quantity#97, sr_return_amt#98] -Arguments: [ss_ticket_number#84, ss_item_sk#83], [sr_ticket_number#96, sr_item_sk#95], LeftOuter - -(98) CometProject -Input [13]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, d_year#94, sr_item_sk#95, sr_ticket_number#96, sr_return_quantity#97, sr_return_amt#98] -Arguments: [d_year#94, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, sales_cnt#99, sales_amt#100], [d_year#94, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, (ss_quantity#85 - coalesce(sr_return_quantity#97, 0)) AS sales_cnt#99, (ss_ext_sales_price#86 - coalesce(sr_return_amt#98, 0.00)) AS sales_amt#100] - -(99) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#105), dynamicpruningexpression(ws_sold_date_sk#105 IN dynamicpruning#69)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(100) ColumnarToRow [codegen id : 18] -Input [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105] - -(101) Filter [codegen id : 18] -Input [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105] -Condition : isnotnull(ws_item_sk#101) - -(102) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] - -(103) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_item_sk#101] -Right keys [1]: [i_item_sk#106] -Join type: Inner -Join condition: None - -(104) Project [codegen id : 18] -Output [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Input [10]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] - -(105) ReusedExchange [Reuses operator id: 135] -Output [2]: [d_date_sk#111, d_year#112] - -(106) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#105] -Right keys [1]: [d_date_sk#111] -Join type: Inner -Join condition: None - -(107) Project [codegen id : 18] -Output [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Input [11]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] - -(108) CometColumnarExchange -Input [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: hashpartitioning(ws_order_number#102, ws_item_sk#101, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(109) CometSort -Input [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_order_number#102 ASC NULLS FIRST, ws_item_sk#101 ASC NULLS FIRST] - -(110) ReusedExchange [Reuses operator id: 55] -Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] - -(111) CometSort -Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116], [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST] - -(112) CometSortMergeJoin -Left output [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Right output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [ws_order_number#102, ws_item_sk#101], [wr_order_number#114, wr_item_sk#113], LeftOuter - -(113) CometProject -Input [13]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#117, sales_amt#118], [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#103 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#117, (ws_ext_sales_price#104 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#118] - -(114) CometUnion -Child 0 Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#81, sales_amt#82] -Child 1 Input [7]: [d_year#94, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, sales_cnt#99, sales_amt#100] -Child 2 Input [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#117, sales_amt#118] - -(115) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#81, sales_amt#82] -Keys [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#81, sales_amt#82] -Functions: [] - -(116) CometExchange -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#81, sales_amt#82] -Arguments: hashpartitioning(d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#81, sales_amt#82, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(117) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#81, sales_amt#82] -Keys [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#81, sales_amt#82] -Functions: [] - -(118) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#81, sales_amt#82] -Keys [5]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -Functions [2]: [partial_sum(sales_cnt#81), partial_sum(UnscaledValue(sales_amt#82))] - -(119) CometExchange -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sum#119, sum#120] -Arguments: hashpartitioning(d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(120) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sum#119, sum#120] -Keys [5]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -Functions [2]: [sum(sales_cnt#81), sum(UnscaledValue(sales_amt#82))] - -(121) CometFilter -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#121, sales_amt#122] -Condition : isnotnull(sales_cnt#121) - -(122) CometExchange -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#121, sales_amt#122] -Arguments: hashpartitioning(i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=16] - -(123) CometSort -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#121, sales_amt#122] -Arguments: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#121, sales_amt#122], [i_brand_id#71 ASC NULLS FIRST, i_class_id#72 ASC NULLS FIRST, i_category_id#73 ASC NULLS FIRST, i_manufact_id#74 ASC NULLS FIRST] - -(124) CometSortMergeJoin -Left output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Right output [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#121, sales_amt#122] -Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74], Inner, ((cast(sales_cnt#62 as decimal(17,2)) / cast(sales_cnt#121 as decimal(17,2))) < 0.90000000000000000000) - -(125) CometProject -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#121, sales_amt#122] -Arguments: [prev_year#123, year#124, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#125, curr_yr_cnt#126, sales_cnt_diff#127, sales_amt_diff#128], [d_year#76 AS prev_year#123, d_year#14 AS year#124, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#121 AS prev_yr_cnt#125, sales_cnt#62 AS curr_yr_cnt#126, (sales_cnt#62 - sales_cnt#121) AS sales_cnt_diff#127, (sales_amt#63 - sales_amt#122) AS sales_amt_diff#128] - -(126) CometTakeOrderedAndProject -Input [10]: [prev_year#123, year#124, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#125, curr_yr_cnt#126, sales_cnt_diff#127, sales_amt_diff#128] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#127 ASC NULLS FIRST], output=[prev_year#123,year#124,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#125,curr_yr_cnt#126,sales_cnt_diff#127,sales_amt_diff#128]), [prev_year#123, year#124, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#125, curr_yr_cnt#126, sales_cnt_diff#127, sales_amt_diff#128], 100, 0, [sales_cnt_diff#127 ASC NULLS FIRST], [prev_year#123, year#124, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#125, curr_yr_cnt#126, sales_cnt_diff#127, sales_amt_diff#128] - -(127) CometColumnarToRow [codegen id : 19] -Input [10]: [prev_year#123, year#124, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#125, curr_yr_cnt#126, sales_cnt_diff#127, sales_amt_diff#128] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (131) -+- * CometColumnarToRow (130) - +- CometFilter (129) - +- CometNativeScan parquet spark_catalog.default.date_dim (128) - - -(128) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(129) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(130) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_year#14] - -(131) BroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#6 - -Subquery:4 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#68 IN dynamicpruning#69 -BroadcastExchange (135) -+- * CometColumnarToRow (134) - +- CometFilter (133) - +- CometNativeScan parquet spark_catalog.default.date_dim (132) - - -(132) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#75, d_year#76] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(133) CometFilter -Input [2]: [d_date_sk#75, d_year#76] -Condition : ((isnotnull(d_year#76) AND (d_year#76 = 2001)) AND isnotnull(d_date_sk#75)) - -(134) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#75, d_year#76] - -(135) BroadcastExchange -Input [2]: [d_date_sk#75, d_year#76] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] - -Subquery:5 Hosting operator id = 84 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#69 - -Subquery:6 Hosting operator id = 99 Hosting Expression = ws_sold_date_sk#105 IN dynamicpruning#69 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/simplified.txt deleted file mode 100644 index aaf98f4b09..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/simplified.txt +++ /dev/null @@ -1,173 +0,0 @@ -WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometColumnarExchange [cs_order_number,cs_item_sk] #4 - WholeStageCodegen (3) - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [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] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #7 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 - WholeStageCodegen (6) - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #9 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometColumnarExchange [ws_order_number,ws_item_sk] #10 - WholeStageCodegen (9) - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #11 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometColumnarExchange [cs_order_number,cs_item_sk] #15 - WholeStageCodegen (12) - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 #16 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 - WholeStageCodegen (15) - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometColumnarExchange [ws_order_number,ws_item_sk] #18 - WholeStageCodegen (18) - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/explain.txt deleted file mode 100644 index 02ac3ecdc2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,742 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (124) -+- CometTakeOrderedAndProject (123) - +- CometProject (122) - +- CometSortMergeJoin (121) - :- CometSort (66) - : +- CometExchange (65) - : +- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometHashAggregate (60) - : +- CometExchange (59) - : +- CometHashAggregate (58) - : +- CometUnion (57) - : :- CometProject (22) - : : +- CometSortMergeJoin (21) - : : :- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometSort (20) - : : +- CometExchange (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (39) - : : +- CometSortMergeJoin (38) - : : :- CometSort (32) - : : : +- CometExchange (31) - : : : +- CometProject (30) - : : : +- CometBroadcastHashJoin (29) - : : : :- CometProject (27) - : : : : +- CometBroadcastHashJoin (26) - : : : : :- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (25) - : : : +- ReusedExchange (28) - : : +- CometSort (37) - : : +- CometExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (33) - : +- CometProject (56) - : +- CometSortMergeJoin (55) - : :- CometSort (49) - : : +- CometExchange (48) - : : +- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometProject (44) - : : : +- CometBroadcastHashJoin (43) - : : : :- CometFilter (41) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) - : : : +- ReusedExchange (42) - : : +- ReusedExchange (45) - : +- CometSort (54) - : +- CometExchange (53) - : +- CometProject (52) - : +- CometFilter (51) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (50) - +- CometSort (120) - +- CometExchange (119) - +- CometFilter (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometHashAggregate (114) - +- CometExchange (113) - +- CometHashAggregate (112) - +- CometUnion (111) - :- CometProject (82) - : +- CometSortMergeJoin (81) - : :- CometSort (78) - : : +- CometExchange (77) - : : +- CometProject (76) - : : +- CometBroadcastHashJoin (75) - : : :- CometProject (71) - : : : +- CometBroadcastHashJoin (70) - : : : :- CometFilter (68) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) - : : : +- ReusedExchange (69) - : : +- CometBroadcastExchange (74) - : : +- CometFilter (73) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - : +- CometSort (80) - : +- ReusedExchange (79) - :- CometProject (96) - : +- CometSortMergeJoin (95) - : :- CometSort (92) - : : +- CometExchange (91) - : : +- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (87) - : : : +- CometBroadcastHashJoin (86) - : : : :- CometFilter (84) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (83) - : : : +- ReusedExchange (85) - : : +- ReusedExchange (88) - : +- CometSort (94) - : +- ReusedExchange (93) - +- CometProject (110) - +- CometSortMergeJoin (109) - :- CometSort (106) - : +- CometExchange (105) - : +- CometProject (104) - : +- CometBroadcastHashJoin (103) - : :- CometProject (101) - : : +- CometBroadcastHashJoin (100) - : : :- CometFilter (98) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (97) - : : +- ReusedExchange (99) - : +- ReusedExchange (102) - +- CometSort (108) - +- ReusedExchange (107) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((isnotnull(i_category#11) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#11, 50)) = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) - -(5) CometProject -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(6) CometBroadcastExchange -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(7) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Right output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: [d_date_sk#13, d_year#14] - -(12) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Right output [2]: [d_date_sk#13, d_year#14] -Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight - -(13) CometProject -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] - -(14) CometExchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(15) CometSort -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) - -(18) CometProject -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(19) CometExchange -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter - -(22) CometProject -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21], [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(24) CometFilter -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#22) - -(25) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(26) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Right output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22], [i_item_sk#28], Inner, BuildRight - -(27) CometProject -Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(28) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#33, d_year#34] - -(29) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Right output [2]: [d_date_sk#33, d_year#34] -Arguments: [ss_sold_date_sk#26], [d_date_sk#33], Inner, BuildRight - -(30) CometProject -Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_date_sk#33, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] - -(31) CometExchange -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometSort -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) - -(35) CometProject -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] - -(36) CometExchange -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(37) CometSort -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] - -(38) CometSortMergeJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter - -(39) CometProject -Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#47)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(41) CometFilter -Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Condition : isnotnull(ws_item_sk#42) - -(42) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(43) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Right output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42], [i_item_sk#48], Inner, BuildRight - -(44) CometProject -Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(45) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#53, d_year#54] - -(46) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [ws_sold_date_sk#46], [d_date_sk#53], Inner, BuildRight - -(47) CometProject -Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] - -(48) CometExchange -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(49) CometSort -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) - -(52) CometProject -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] - -(53) CometExchange -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(54) CometSort -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] - -(55) CometSortMergeJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter - -(56) CometProject -Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] - -(57) CometUnion -Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] - -(58) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(59) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(60) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(61) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(62) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(63) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(64) CometFilter -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Condition : isnotnull(sales_cnt#64) - -(65) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(66) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(68) CometFilter -Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Condition : isnotnull(cs_item_sk#66) - -(69) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(70) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Right output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66], [i_item_sk#72], Inner, BuildRight - -(71) CometProject -Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(74) CometBroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: [d_date_sk#77, d_year#78] - -(75) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Right output [2]: [d_date_sk#77, d_year#78] -Arguments: [cs_sold_date_sk#70], [d_date_sk#77], Inner, BuildRight - -(76) CometProject -Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] - -(77) CometExchange -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(78) CometSort -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] - -(79) ReusedExchange [Reuses operator id: 19] -Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] - -(80) CometSort -Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] - -(81) CometSortMergeJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter - -(82) CometProject -Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#83, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#84] - -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#89), dynamicpruningexpression(ss_sold_date_sk#89 IN dynamicpruning#90)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(84) CometFilter -Input [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] -Condition : isnotnull(ss_item_sk#85) - -(85) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] - -(86) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] -Right output [5]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] -Arguments: [ss_item_sk#85], [i_item_sk#91], Inner, BuildRight - -(87) CometProject -Input [10]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] -Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95], [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] - -(88) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#96, d_year#97] - -(89) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] -Right output [2]: [d_date_sk#96, d_year#97] -Arguments: [ss_sold_date_sk#89], [d_date_sk#96], Inner, BuildRight - -(90) CometProject -Input [11]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_date_sk#96, d_year#97] -Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97], [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] - -(91) CometExchange -Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] -Arguments: hashpartitioning(ss_ticket_number#86, ss_item_sk#85, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(92) CometSort -Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] -Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97], [ss_ticket_number#86 ASC NULLS FIRST, ss_item_sk#85 ASC NULLS FIRST] - -(93) ReusedExchange [Reuses operator id: 36] -Output [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] - -(94) CometSort -Input [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] -Arguments: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101], [sr_ticket_number#99 ASC NULLS FIRST, sr_item_sk#98 ASC NULLS FIRST] - -(95) CometSortMergeJoin -Left output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] -Right output [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] -Arguments: [ss_ticket_number#86, ss_item_sk#85], [sr_ticket_number#99, sr_item_sk#98], LeftOuter - -(96) CometProject -Input [13]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97, sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] -Arguments: [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, sales_cnt#102, sales_amt#103], [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, (ss_quantity#87 - coalesce(sr_return_quantity#100, 0)) AS sales_cnt#102, (ss_ext_sales_price#88 - coalesce(sr_return_amt#101, 0.00)) AS sales_amt#103] - -(97) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_sold_date_sk#108 IN dynamicpruning#109)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(98) CometFilter -Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] -Condition : isnotnull(ws_item_sk#104) - -(99) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] - -(100) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] -Right output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -Arguments: [ws_item_sk#104], [i_item_sk#110], Inner, BuildRight - -(101) CometProject -Input [10]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114], [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] - -(102) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#115, d_year#116] - -(103) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -Right output [2]: [d_date_sk#115, d_year#116] -Arguments: [ws_sold_date_sk#108], [d_date_sk#115], Inner, BuildRight - -(104) CometProject -Input [11]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_date_sk#115, d_year#116] -Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116], [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] - -(105) CometExchange -Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -Arguments: hashpartitioning(ws_order_number#105, ws_item_sk#104, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(106) CometSort -Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116], [ws_order_number#105 ASC NULLS FIRST, ws_item_sk#104 ASC NULLS FIRST] - -(107) ReusedExchange [Reuses operator id: 53] -Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] - -(108) CometSort -Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -Arguments: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120], [wr_order_number#118 ASC NULLS FIRST, wr_item_sk#117 ASC NULLS FIRST] - -(109) CometSortMergeJoin -Left output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -Right output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -Arguments: [ws_order_number#105, ws_item_sk#104], [wr_order_number#118, wr_item_sk#117], LeftOuter - -(110) CometProject -Input [13]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116, wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -Arguments: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, sales_cnt#121, sales_amt#122], [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, (ws_quantity#106 - coalesce(wr_return_quantity#119, 0)) AS sales_cnt#121, (ws_ext_sales_price#107 - coalesce(wr_return_amt#120, 0.00)) AS sales_amt#122] - -(111) CometUnion -Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Child 1 Input [7]: [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, sales_cnt#102, sales_amt#103] -Child 2 Input [7]: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, sales_cnt#121, sales_amt#122] - -(112) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Functions: [] - -(113) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] - -(114) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Functions: [] - -(115) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [partial_sum(sales_cnt#83), partial_sum(UnscaledValue(sales_amt#84))] - -(116) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#123, sum#124] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(117) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#123, sum#124] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [sum(sales_cnt#83), sum(UnscaledValue(sales_amt#84))] - -(118) CometFilter -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126] -Condition : isnotnull(sales_cnt#125) - -(119) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126] -Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(120) CometSort -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] - -(121) CometSortMergeJoin -Left output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Right output [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126] -Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], Inner, ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#125 as decimal(17,2))) < 0.90000000000000000000) - -(122) CometProject -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126] -Arguments: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132], [d_year#78 AS prev_year#127, d_year#14 AS year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#125 AS prev_yr_cnt#129, sales_cnt#64 AS curr_yr_cnt#130, (sales_cnt#64 - sales_cnt#125) AS sales_cnt_diff#131, (sales_amt#65 - sales_amt#126) AS sales_amt_diff#132] - -(123) CometTakeOrderedAndProject -Input [10]: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#131 ASC NULLS FIRST], output=[prev_year#127,year#128,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#129,curr_yr_cnt#130,sales_cnt_diff#131,sales_amt_diff#132]), [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132], 100, 0, [sales_cnt_diff#131 ASC NULLS FIRST], [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] - -(124) CometColumnarToRow [codegen id : 1] -Input [10]: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (128) -+- * CometColumnarToRow (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) - - -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(127) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_year#14] - -(128) BroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 - -Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 -BroadcastExchange (132) -+- * CometColumnarToRow (131) - +- CometFilter (130) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) - - -(129) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(130) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(131) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#77, d_year#78] - -(132) BroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] - -Subquery:5 Hosting operator id = 83 Hosting Expression = ss_sold_date_sk#89 IN dynamicpruning#71 - -Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#71 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/simplified.txt deleted file mode 100644 index d0520c8b63..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,144 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #4 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange [d_date_sk,d_year] #7 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #8 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #9 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #10 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #11 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #12 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #16 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_year] #18 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #19 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #20 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/explain.txt deleted file mode 100644 index 02ac3ecdc2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/explain.txt +++ /dev/null @@ -1,742 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (124) -+- CometTakeOrderedAndProject (123) - +- CometProject (122) - +- CometSortMergeJoin (121) - :- CometSort (66) - : +- CometExchange (65) - : +- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometHashAggregate (60) - : +- CometExchange (59) - : +- CometHashAggregate (58) - : +- CometUnion (57) - : :- CometProject (22) - : : +- CometSortMergeJoin (21) - : : :- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometSort (20) - : : +- CometExchange (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (39) - : : +- CometSortMergeJoin (38) - : : :- CometSort (32) - : : : +- CometExchange (31) - : : : +- CometProject (30) - : : : +- CometBroadcastHashJoin (29) - : : : :- CometProject (27) - : : : : +- CometBroadcastHashJoin (26) - : : : : :- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (25) - : : : +- ReusedExchange (28) - : : +- CometSort (37) - : : +- CometExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (33) - : +- CometProject (56) - : +- CometSortMergeJoin (55) - : :- CometSort (49) - : : +- CometExchange (48) - : : +- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometProject (44) - : : : +- CometBroadcastHashJoin (43) - : : : :- CometFilter (41) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) - : : : +- ReusedExchange (42) - : : +- ReusedExchange (45) - : +- CometSort (54) - : +- CometExchange (53) - : +- CometProject (52) - : +- CometFilter (51) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (50) - +- CometSort (120) - +- CometExchange (119) - +- CometFilter (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometHashAggregate (114) - +- CometExchange (113) - +- CometHashAggregate (112) - +- CometUnion (111) - :- CometProject (82) - : +- CometSortMergeJoin (81) - : :- CometSort (78) - : : +- CometExchange (77) - : : +- CometProject (76) - : : +- CometBroadcastHashJoin (75) - : : :- CometProject (71) - : : : +- CometBroadcastHashJoin (70) - : : : :- CometFilter (68) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) - : : : +- ReusedExchange (69) - : : +- CometBroadcastExchange (74) - : : +- CometFilter (73) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - : +- CometSort (80) - : +- ReusedExchange (79) - :- CometProject (96) - : +- CometSortMergeJoin (95) - : :- CometSort (92) - : : +- CometExchange (91) - : : +- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (87) - : : : +- CometBroadcastHashJoin (86) - : : : :- CometFilter (84) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (83) - : : : +- ReusedExchange (85) - : : +- ReusedExchange (88) - : +- CometSort (94) - : +- ReusedExchange (93) - +- CometProject (110) - +- CometSortMergeJoin (109) - :- CometSort (106) - : +- CometExchange (105) - : +- CometProject (104) - : +- CometBroadcastHashJoin (103) - : :- CometProject (101) - : : +- CometBroadcastHashJoin (100) - : : :- CometFilter (98) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (97) - : : +- ReusedExchange (99) - : +- ReusedExchange (102) - +- CometSort (108) - +- ReusedExchange (107) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((isnotnull(i_category#11) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#11, 50)) = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) - -(5) CometProject -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(6) CometBroadcastExchange -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(7) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Right output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: [d_date_sk#13, d_year#14] - -(12) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Right output [2]: [d_date_sk#13, d_year#14] -Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight - -(13) CometProject -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] - -(14) CometExchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(15) CometSort -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) - -(18) CometProject -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(19) CometExchange -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter - -(22) CometProject -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21], [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(24) CometFilter -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#22) - -(25) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(26) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Right output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22], [i_item_sk#28], Inner, BuildRight - -(27) CometProject -Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(28) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#33, d_year#34] - -(29) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Right output [2]: [d_date_sk#33, d_year#34] -Arguments: [ss_sold_date_sk#26], [d_date_sk#33], Inner, BuildRight - -(30) CometProject -Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_date_sk#33, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] - -(31) CometExchange -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometSort -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) - -(35) CometProject -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] - -(36) CometExchange -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(37) CometSort -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] - -(38) CometSortMergeJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter - -(39) CometProject -Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#47)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(41) CometFilter -Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Condition : isnotnull(ws_item_sk#42) - -(42) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(43) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Right output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42], [i_item_sk#48], Inner, BuildRight - -(44) CometProject -Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(45) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#53, d_year#54] - -(46) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [ws_sold_date_sk#46], [d_date_sk#53], Inner, BuildRight - -(47) CometProject -Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] - -(48) CometExchange -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(49) CometSort -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) - -(52) CometProject -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] - -(53) CometExchange -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(54) CometSort -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] - -(55) CometSortMergeJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter - -(56) CometProject -Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] - -(57) CometUnion -Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] - -(58) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(59) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(60) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(61) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(62) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(63) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(64) CometFilter -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Condition : isnotnull(sales_cnt#64) - -(65) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(66) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(68) CometFilter -Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Condition : isnotnull(cs_item_sk#66) - -(69) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(70) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Right output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66], [i_item_sk#72], Inner, BuildRight - -(71) CometProject -Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(74) CometBroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: [d_date_sk#77, d_year#78] - -(75) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Right output [2]: [d_date_sk#77, d_year#78] -Arguments: [cs_sold_date_sk#70], [d_date_sk#77], Inner, BuildRight - -(76) CometProject -Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] - -(77) CometExchange -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(78) CometSort -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] - -(79) ReusedExchange [Reuses operator id: 19] -Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] - -(80) CometSort -Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] - -(81) CometSortMergeJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter - -(82) CometProject -Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#83, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#84] - -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#89), dynamicpruningexpression(ss_sold_date_sk#89 IN dynamicpruning#90)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(84) CometFilter -Input [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] -Condition : isnotnull(ss_item_sk#85) - -(85) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] - -(86) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] -Right output [5]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] -Arguments: [ss_item_sk#85], [i_item_sk#91], Inner, BuildRight - -(87) CometProject -Input [10]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] -Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95], [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] - -(88) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#96, d_year#97] - -(89) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] -Right output [2]: [d_date_sk#96, d_year#97] -Arguments: [ss_sold_date_sk#89], [d_date_sk#96], Inner, BuildRight - -(90) CometProject -Input [11]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_date_sk#96, d_year#97] -Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97], [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] - -(91) CometExchange -Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] -Arguments: hashpartitioning(ss_ticket_number#86, ss_item_sk#85, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(92) CometSort -Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] -Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97], [ss_ticket_number#86 ASC NULLS FIRST, ss_item_sk#85 ASC NULLS FIRST] - -(93) ReusedExchange [Reuses operator id: 36] -Output [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] - -(94) CometSort -Input [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] -Arguments: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101], [sr_ticket_number#99 ASC NULLS FIRST, sr_item_sk#98 ASC NULLS FIRST] - -(95) CometSortMergeJoin -Left output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] -Right output [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] -Arguments: [ss_ticket_number#86, ss_item_sk#85], [sr_ticket_number#99, sr_item_sk#98], LeftOuter - -(96) CometProject -Input [13]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97, sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] -Arguments: [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, sales_cnt#102, sales_amt#103], [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, (ss_quantity#87 - coalesce(sr_return_quantity#100, 0)) AS sales_cnt#102, (ss_ext_sales_price#88 - coalesce(sr_return_amt#101, 0.00)) AS sales_amt#103] - -(97) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_sold_date_sk#108 IN dynamicpruning#109)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(98) CometFilter -Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] -Condition : isnotnull(ws_item_sk#104) - -(99) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] - -(100) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] -Right output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -Arguments: [ws_item_sk#104], [i_item_sk#110], Inner, BuildRight - -(101) CometProject -Input [10]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114], [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] - -(102) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#115, d_year#116] - -(103) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -Right output [2]: [d_date_sk#115, d_year#116] -Arguments: [ws_sold_date_sk#108], [d_date_sk#115], Inner, BuildRight - -(104) CometProject -Input [11]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_date_sk#115, d_year#116] -Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116], [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] - -(105) CometExchange -Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -Arguments: hashpartitioning(ws_order_number#105, ws_item_sk#104, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(106) CometSort -Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116], [ws_order_number#105 ASC NULLS FIRST, ws_item_sk#104 ASC NULLS FIRST] - -(107) ReusedExchange [Reuses operator id: 53] -Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] - -(108) CometSort -Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -Arguments: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120], [wr_order_number#118 ASC NULLS FIRST, wr_item_sk#117 ASC NULLS FIRST] - -(109) CometSortMergeJoin -Left output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -Right output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -Arguments: [ws_order_number#105, ws_item_sk#104], [wr_order_number#118, wr_item_sk#117], LeftOuter - -(110) CometProject -Input [13]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116, wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -Arguments: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, sales_cnt#121, sales_amt#122], [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, (ws_quantity#106 - coalesce(wr_return_quantity#119, 0)) AS sales_cnt#121, (ws_ext_sales_price#107 - coalesce(wr_return_amt#120, 0.00)) AS sales_amt#122] - -(111) CometUnion -Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Child 1 Input [7]: [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, sales_cnt#102, sales_amt#103] -Child 2 Input [7]: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, sales_cnt#121, sales_amt#122] - -(112) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Functions: [] - -(113) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] - -(114) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Functions: [] - -(115) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [partial_sum(sales_cnt#83), partial_sum(UnscaledValue(sales_amt#84))] - -(116) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#123, sum#124] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(117) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#123, sum#124] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [sum(sales_cnt#83), sum(UnscaledValue(sales_amt#84))] - -(118) CometFilter -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126] -Condition : isnotnull(sales_cnt#125) - -(119) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126] -Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(120) CometSort -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] - -(121) CometSortMergeJoin -Left output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Right output [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126] -Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], Inner, ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#125 as decimal(17,2))) < 0.90000000000000000000) - -(122) CometProject -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126] -Arguments: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132], [d_year#78 AS prev_year#127, d_year#14 AS year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#125 AS prev_yr_cnt#129, sales_cnt#64 AS curr_yr_cnt#130, (sales_cnt#64 - sales_cnt#125) AS sales_cnt_diff#131, (sales_amt#65 - sales_amt#126) AS sales_amt_diff#132] - -(123) CometTakeOrderedAndProject -Input [10]: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#131 ASC NULLS FIRST], output=[prev_year#127,year#128,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#129,curr_yr_cnt#130,sales_cnt_diff#131,sales_amt_diff#132]), [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132], 100, 0, [sales_cnt_diff#131 ASC NULLS FIRST], [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] - -(124) CometColumnarToRow [codegen id : 1] -Input [10]: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (128) -+- * CometColumnarToRow (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) - - -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(127) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_year#14] - -(128) BroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 - -Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 -BroadcastExchange (132) -+- * CometColumnarToRow (131) - +- CometFilter (130) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) - - -(129) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(130) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(131) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#77, d_year#78] - -(132) BroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] - -Subquery:5 Hosting operator id = 83 Hosting Expression = ss_sold_date_sk#89 IN dynamicpruning#71 - -Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#71 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/extended.txt deleted file mode 100644 index 22ccc17641..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/extended.txt +++ /dev/null @@ -1,172 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : +- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - +- CometSort - +- CometExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - -Comet accelerated 159 out of 167 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d0520c8b63..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt +++ /dev/null @@ -1,144 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #4 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange [d_date_sk,d_year] #7 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #8 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #9 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #10 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #11 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #12 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #16 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_year] #18 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #19 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #20 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_datafusion/explain.txt deleted file mode 100644 index fab101ea7c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_datafusion/explain.txt +++ /dev/null @@ -1,200 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (35) -+- CometTakeOrderedAndProject (34) - +- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometUnion (30) - :- CometProject (13) - : +- CometBroadcastHashJoin (12) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometNativeScan parquet spark_catalog.default.item (3) - : +- CometBroadcastExchange (11) - : +- CometFilter (10) - : +- CometNativeScan parquet spark_catalog.default.date_dim (9) - :- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometFilter (15) - : : : +- CometNativeScan parquet spark_catalog.default.web_sales (14) - : : +- ReusedExchange (16) - : +- ReusedExchange (19) - +- CometProject (29) - +- CometBroadcastHashJoin (28) - :- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometFilter (23) - : : +- CometNativeScan parquet spark_catalog.default.catalog_sales (22) - : +- ReusedExchange (24) - +- ReusedExchange (27) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#4)] -PushedFilters: [IsNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_category#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_category#6] -Condition : isnotnull(i_item_sk#5) - -(5) CometProject -Input [2]: [i_item_sk#5, i_category#6] -Arguments: [i_item_sk#5, i_category#7], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#6, 50)) AS i_category#7] - -(6) CometBroadcastExchange -Input [2]: [i_item_sk#5, i_category#7] -Arguments: [i_item_sk#5, i_category#7] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [2]: [i_item_sk#5, i_category#7] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_item_sk#5, i_category#7] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7], [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7] - -(9) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Condition : isnotnull(d_date_sk#8) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [d_date_sk#8, d_year#9, d_qoy#10] - -(12) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7] -Right output [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [ss_sold_date_sk#4], [d_date_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7, d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13], [store AS channel#11, ss_store_sk#2 AS col_name#12, d_year#9, d_qoy#10, i_category#7, ss_ext_sales_price#3 AS ext_sales_price#13] - -(14) CometNativeScan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ws_sold_date_sk#17)] -PushedFilters: [IsNull(ws_ship_customer_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Condition : (isnull(ws_ship_customer_sk#15) AND isnotnull(ws_item_sk#14)) - -(16) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#18, i_category#19] - -(17) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Right output [2]: [i_item_sk#18, i_category#19] -Arguments: [ws_item_sk#14], [i_item_sk#18], Inner, BuildRight - -(18) CometProject -Input [6]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_item_sk#18, i_category#19] -Arguments: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19], [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19] - -(19) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#20, d_year#21, d_qoy#22] - -(20) CometBroadcastHashJoin -Left output [4]: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19] -Right output [3]: [d_date_sk#20, d_year#21, d_qoy#22] -Arguments: [ws_sold_date_sk#17], [d_date_sk#20], Inner, BuildRight - -(21) CometProject -Input [7]: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19, d_date_sk#20, d_year#21, d_qoy#22] -Arguments: [channel#23, col_name#24, d_year#21, d_qoy#22, i_category#19, ext_sales_price#25], [web AS channel#23, ws_ship_customer_sk#15 AS col_name#24, d_year#21, d_qoy#22, i_category#19, ws_ext_sales_price#16 AS ext_sales_price#25] - -(22) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(cs_sold_date_sk#29)] -PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(23) CometFilter -Input [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Condition : (isnull(cs_ship_addr_sk#26) AND isnotnull(cs_item_sk#27)) - -(24) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#30, i_category#31] - -(25) CometBroadcastHashJoin -Left output [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Right output [2]: [i_item_sk#30, i_category#31] -Arguments: [cs_item_sk#27], [i_item_sk#30], Inner, BuildRight - -(26) CometProject -Input [6]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29, i_item_sk#30, i_category#31] -Arguments: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31], [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31] - -(27) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#32, d_year#33, d_qoy#34] - -(28) CometBroadcastHashJoin -Left output [4]: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31] -Right output [3]: [d_date_sk#32, d_year#33, d_qoy#34] -Arguments: [cs_sold_date_sk#29], [d_date_sk#32], Inner, BuildRight - -(29) CometProject -Input [7]: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31, d_date_sk#32, d_year#33, d_qoy#34] -Arguments: [channel#35, col_name#36, d_year#33, d_qoy#34, i_category#31, ext_sales_price#37], [catalog AS channel#35, cs_ship_addr_sk#26 AS col_name#36, d_year#33, d_qoy#34, i_category#31, cs_ext_sales_price#28 AS ext_sales_price#37] - -(30) CometUnion -Child 0 Input [6]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13] -Child 1 Input [6]: [channel#23, col_name#24, d_year#21, d_qoy#22, i_category#19, ext_sales_price#25] -Child 2 Input [6]: [channel#35, col_name#36, d_year#33, d_qoy#34, i_category#31, ext_sales_price#37] - -(31) CometHashAggregate -Input [6]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13] -Keys [5]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7] -Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#13))] - -(32) CometExchange -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] -Arguments: hashpartitioning(channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(33) CometHashAggregate -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] -Keys [5]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7] -Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#13))] - -(34) CometTakeOrderedAndProject -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#11 ASC NULLS FIRST,col_name#12 ASC NULLS FIRST,d_year#9 ASC NULLS FIRST,d_qoy#10 ASC NULLS FIRST,i_category#7 ASC NULLS FIRST], output=[channel#11,col_name#12,d_year#9,d_qoy#10,i_category#7,sales_cnt#40,sales_amt#41]), [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41], 100, 0, [channel#11 ASC NULLS FIRST, col_name#12 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#7 ASC NULLS FIRST], [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] - -(35) CometColumnarToRow [codegen id : 1] -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_datafusion/simplified.txt deleted file mode 100644 index abb482c3d5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_datafusion/simplified.txt +++ /dev/null @@ -1,37 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] - CometHashAggregate [count,sum] [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt,count(1),sum(UnscaledValue(ext_sales_price))] - CometExchange [channel,col_name,d_year,d_qoy,i_category] #1 - CometHashAggregate [ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,count,sum] - 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_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,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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category] #2 - CometProject [i_category] [i_item_sk,i_category] - CometFilter [i_item_sk,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan 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_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,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] - CometNativeScan 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_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_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] - CometNativeScan 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/q76.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/explain.txt deleted file mode 100644 index 8b346eb5c9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,200 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (35) -+- CometTakeOrderedAndProject (34) - +- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometUnion (30) - :- CometProject (13) - : +- CometBroadcastHashJoin (12) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : +- CometBroadcastExchange (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - :- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometFilter (15) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (14) - : : +- ReusedExchange (16) - : +- ReusedExchange (19) - +- CometProject (29) - +- CometBroadcastHashJoin (28) - :- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometFilter (23) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (22) - : +- ReusedExchange (24) - +- ReusedExchange (27) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4)] -PushedFilters: [IsNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_category#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_category#6] -Condition : isnotnull(i_item_sk#5) - -(5) CometProject -Input [2]: [i_item_sk#5, i_category#6] -Arguments: [i_item_sk#5, i_category#7], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#6, 50)) AS i_category#7] - -(6) CometBroadcastExchange -Input [2]: [i_item_sk#5, i_category#7] -Arguments: [i_item_sk#5, i_category#7] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [2]: [i_item_sk#5, i_category#7] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_item_sk#5, i_category#7] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7], [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Condition : isnotnull(d_date_sk#8) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [d_date_sk#8, d_year#9, d_qoy#10] - -(12) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7] -Right output [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [ss_sold_date_sk#4], [d_date_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7, d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13], [store AS channel#11, ss_store_sk#2 AS col_name#12, d_year#9, d_qoy#10, i_category#7, ss_ext_sales_price#3 AS ext_sales_price#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#17)] -PushedFilters: [IsNull(ws_ship_customer_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Condition : (isnull(ws_ship_customer_sk#15) AND isnotnull(ws_item_sk#14)) - -(16) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#18, i_category#19] - -(17) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Right output [2]: [i_item_sk#18, i_category#19] -Arguments: [ws_item_sk#14], [i_item_sk#18], Inner, BuildRight - -(18) CometProject -Input [6]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_item_sk#18, i_category#19] -Arguments: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19], [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19] - -(19) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#20, d_year#21, d_qoy#22] - -(20) CometBroadcastHashJoin -Left output [4]: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19] -Right output [3]: [d_date_sk#20, d_year#21, d_qoy#22] -Arguments: [ws_sold_date_sk#17], [d_date_sk#20], Inner, BuildRight - -(21) CometProject -Input [7]: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19, d_date_sk#20, d_year#21, d_qoy#22] -Arguments: [channel#23, col_name#24, d_year#21, d_qoy#22, i_category#19, ext_sales_price#25], [web AS channel#23, ws_ship_customer_sk#15 AS col_name#24, d_year#21, d_qoy#22, i_category#19, ws_ext_sales_price#16 AS ext_sales_price#25] - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#29)] -PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(23) CometFilter -Input [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Condition : (isnull(cs_ship_addr_sk#26) AND isnotnull(cs_item_sk#27)) - -(24) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#30, i_category#31] - -(25) CometBroadcastHashJoin -Left output [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Right output [2]: [i_item_sk#30, i_category#31] -Arguments: [cs_item_sk#27], [i_item_sk#30], Inner, BuildRight - -(26) CometProject -Input [6]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29, i_item_sk#30, i_category#31] -Arguments: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31], [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31] - -(27) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#32, d_year#33, d_qoy#34] - -(28) CometBroadcastHashJoin -Left output [4]: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31] -Right output [3]: [d_date_sk#32, d_year#33, d_qoy#34] -Arguments: [cs_sold_date_sk#29], [d_date_sk#32], Inner, BuildRight - -(29) CometProject -Input [7]: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31, d_date_sk#32, d_year#33, d_qoy#34] -Arguments: [channel#35, col_name#36, d_year#33, d_qoy#34, i_category#31, ext_sales_price#37], [catalog AS channel#35, cs_ship_addr_sk#26 AS col_name#36, d_year#33, d_qoy#34, i_category#31, cs_ext_sales_price#28 AS ext_sales_price#37] - -(30) CometUnion -Child 0 Input [6]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13] -Child 1 Input [6]: [channel#23, col_name#24, d_year#21, d_qoy#22, i_category#19, ext_sales_price#25] -Child 2 Input [6]: [channel#35, col_name#36, d_year#33, d_qoy#34, i_category#31, ext_sales_price#37] - -(31) CometHashAggregate -Input [6]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13] -Keys [5]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7] -Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#13))] - -(32) CometExchange -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] -Arguments: hashpartitioning(channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(33) CometHashAggregate -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] -Keys [5]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7] -Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#13))] - -(34) CometTakeOrderedAndProject -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#11 ASC NULLS FIRST,col_name#12 ASC NULLS FIRST,d_year#9 ASC NULLS FIRST,d_qoy#10 ASC NULLS FIRST,i_category#7 ASC NULLS FIRST], output=[channel#11,col_name#12,d_year#9,d_qoy#10,i_category#7,sales_cnt#40,sales_amt#41]), [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41], 100, 0, [channel#11 ASC NULLS FIRST, col_name#12 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#7 ASC NULLS FIRST], [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] - -(35) CometColumnarToRow [codegen id : 1] -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/simplified.txt deleted file mode 100644 index b5449831cb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,37 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] - CometHashAggregate [count,sum] [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt,count(1),sum(UnscaledValue(ext_sales_price))] - CometExchange [channel,col_name,d_year,d_qoy,i_category] #1 - CometHashAggregate [ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,count,sum] - 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_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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category] #2 - CometProject [i_category] [i_item_sk,i_category] - CometFilter [i_item_sk,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] 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_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,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 [native_iceberg_compat] 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_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_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 [native_iceberg_compat] 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/q76/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/explain.txt deleted file mode 100644 index 8b346eb5c9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/explain.txt +++ /dev/null @@ -1,200 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (35) -+- CometTakeOrderedAndProject (34) - +- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometUnion (30) - :- CometProject (13) - : +- CometBroadcastHashJoin (12) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : +- CometBroadcastExchange (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - :- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometFilter (15) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (14) - : : +- ReusedExchange (16) - : +- ReusedExchange (19) - +- CometProject (29) - +- CometBroadcastHashJoin (28) - :- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometFilter (23) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (22) - : +- ReusedExchange (24) - +- ReusedExchange (27) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4)] -PushedFilters: [IsNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_category#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_category#6] -Condition : isnotnull(i_item_sk#5) - -(5) CometProject -Input [2]: [i_item_sk#5, i_category#6] -Arguments: [i_item_sk#5, i_category#7], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#6, 50)) AS i_category#7] - -(6) CometBroadcastExchange -Input [2]: [i_item_sk#5, i_category#7] -Arguments: [i_item_sk#5, i_category#7] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [2]: [i_item_sk#5, i_category#7] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_item_sk#5, i_category#7] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7], [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Condition : isnotnull(d_date_sk#8) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [d_date_sk#8, d_year#9, d_qoy#10] - -(12) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7] -Right output [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [ss_sold_date_sk#4], [d_date_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7, d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13], [store AS channel#11, ss_store_sk#2 AS col_name#12, d_year#9, d_qoy#10, i_category#7, ss_ext_sales_price#3 AS ext_sales_price#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#17)] -PushedFilters: [IsNull(ws_ship_customer_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Condition : (isnull(ws_ship_customer_sk#15) AND isnotnull(ws_item_sk#14)) - -(16) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#18, i_category#19] - -(17) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Right output [2]: [i_item_sk#18, i_category#19] -Arguments: [ws_item_sk#14], [i_item_sk#18], Inner, BuildRight - -(18) CometProject -Input [6]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_item_sk#18, i_category#19] -Arguments: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19], [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19] - -(19) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#20, d_year#21, d_qoy#22] - -(20) CometBroadcastHashJoin -Left output [4]: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19] -Right output [3]: [d_date_sk#20, d_year#21, d_qoy#22] -Arguments: [ws_sold_date_sk#17], [d_date_sk#20], Inner, BuildRight - -(21) CometProject -Input [7]: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19, d_date_sk#20, d_year#21, d_qoy#22] -Arguments: [channel#23, col_name#24, d_year#21, d_qoy#22, i_category#19, ext_sales_price#25], [web AS channel#23, ws_ship_customer_sk#15 AS col_name#24, d_year#21, d_qoy#22, i_category#19, ws_ext_sales_price#16 AS ext_sales_price#25] - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#29)] -PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(23) CometFilter -Input [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Condition : (isnull(cs_ship_addr_sk#26) AND isnotnull(cs_item_sk#27)) - -(24) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#30, i_category#31] - -(25) CometBroadcastHashJoin -Left output [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Right output [2]: [i_item_sk#30, i_category#31] -Arguments: [cs_item_sk#27], [i_item_sk#30], Inner, BuildRight - -(26) CometProject -Input [6]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29, i_item_sk#30, i_category#31] -Arguments: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31], [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31] - -(27) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#32, d_year#33, d_qoy#34] - -(28) CometBroadcastHashJoin -Left output [4]: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31] -Right output [3]: [d_date_sk#32, d_year#33, d_qoy#34] -Arguments: [cs_sold_date_sk#29], [d_date_sk#32], Inner, BuildRight - -(29) CometProject -Input [7]: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31, d_date_sk#32, d_year#33, d_qoy#34] -Arguments: [channel#35, col_name#36, d_year#33, d_qoy#34, i_category#31, ext_sales_price#37], [catalog AS channel#35, cs_ship_addr_sk#26 AS col_name#36, d_year#33, d_qoy#34, i_category#31, cs_ext_sales_price#28 AS ext_sales_price#37] - -(30) CometUnion -Child 0 Input [6]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13] -Child 1 Input [6]: [channel#23, col_name#24, d_year#21, d_qoy#22, i_category#19, ext_sales_price#25] -Child 2 Input [6]: [channel#35, col_name#36, d_year#33, d_qoy#34, i_category#31, ext_sales_price#37] - -(31) CometHashAggregate -Input [6]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13] -Keys [5]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7] -Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#13))] - -(32) CometExchange -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] -Arguments: hashpartitioning(channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(33) CometHashAggregate -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] -Keys [5]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7] -Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#13))] - -(34) CometTakeOrderedAndProject -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#11 ASC NULLS FIRST,col_name#12 ASC NULLS FIRST,d_year#9 ASC NULLS FIRST,d_qoy#10 ASC NULLS FIRST,i_category#7 ASC NULLS FIRST], output=[channel#11,col_name#12,d_year#9,d_qoy#10,i_category#7,sales_cnt#40,sales_amt#41]), [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41], 100, 0, [channel#11 ASC NULLS FIRST, col_name#12 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#7 ASC NULLS FIRST], [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] - -(35) CometColumnarToRow [codegen id : 1] -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/extended.txt deleted file mode 100644 index b56a6590db..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/extended.txt +++ /dev/null @@ -1,47 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 44 out of 44 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index b5449831cb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt +++ /dev/null @@ -1,37 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] - CometHashAggregate [count,sum] [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt,count(1),sum(UnscaledValue(ext_sales_price))] - CometExchange [channel,col_name,d_year,d_qoy,i_category] #1 - CometHashAggregate [ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,count,sum] - 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_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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category] #2 - CometProject [i_category] [i_item_sk,i_category] - CometFilter [i_item_sk,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] 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_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,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 [native_iceberg_compat] 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_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_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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/explain.txt deleted file mode 100644 index ae530b4900..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/explain.txt +++ /dev/null @@ -1,575 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (92) -+- * HashAggregate (91) - +- * CometColumnarToRow (90) - +- CometColumnarExchange (89) - +- * HashAggregate (88) - +- * Expand (87) - +- Union (86) - :- * Project (32) - : +- * BroadcastHashJoin LeftOuter BuildRight (31) - : :- * HashAggregate (16) - : : +- * CometColumnarToRow (15) - : : +- CometColumnarExchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * CometColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (30) - : +- * HashAggregate (29) - : +- * CometColumnarToRow (28) - : +- CometColumnarExchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet spark_catalog.default.store_returns (17) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - :- * Project (53) - : +- * BroadcastNestedLoopJoin Inner BuildLeft (52) - : :- BroadcastExchange (42) - : : +- * HashAggregate (41) - : : +- * CometColumnarToRow (40) - : : +- CometColumnarExchange (39) - : : +- * HashAggregate (38) - : : +- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * ColumnarToRow (34) - : : : +- Scan parquet spark_catalog.default.catalog_sales (33) - : : +- ReusedExchange (35) - : +- * HashAggregate (51) - : +- * CometColumnarToRow (50) - : +- CometColumnarExchange (49) - : +- * HashAggregate (48) - : +- * Project (47) - : +- * BroadcastHashJoin Inner BuildRight (46) - : :- * ColumnarToRow (44) - : : +- Scan parquet spark_catalog.default.catalog_returns (43) - : +- ReusedExchange (45) - +- * Project (85) - +- * BroadcastHashJoin LeftOuter BuildRight (84) - :- * HashAggregate (69) - : +- * CometColumnarToRow (68) - : +- CometColumnarExchange (67) - : +- * HashAggregate (66) - : +- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * Filter (56) - : : : +- * ColumnarToRow (55) - : : : +- Scan parquet spark_catalog.default.web_sales (54) - : : +- ReusedExchange (57) - : +- BroadcastExchange (63) - : +- * CometColumnarToRow (62) - : +- CometFilter (61) - : +- CometNativeScan parquet spark_catalog.default.web_page (60) - +- BroadcastExchange (83) - +- * HashAggregate (82) - +- * CometColumnarToRow (81) - +- CometColumnarExchange (80) - +- * HashAggregate (79) - +- * Project (78) - +- * BroadcastHashJoin Inner BuildRight (77) - :- * Project (75) - : +- * BroadcastHashJoin Inner BuildRight (74) - : :- * Filter (72) - : : +- * ColumnarToRow (71) - : : +- Scan parquet spark_catalog.default.web_returns (70) - : +- ReusedExchange (73) - +- ReusedExchange (76) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 3] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(4) ReusedExchange [Reuses operator id: 97] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [1]: [s_store_sk#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [1]: [s_store_sk#7] -Condition : isnotnull(s_store_sk#7) - -(9) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#7] - -(10) BroadcastExchange -Input [1]: [s_store_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None - -(12) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] - -(13) HashAggregate [codegen id : 3] -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Keys [1]: [s_store_sk#7] -Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum#8, sum#9] -Results [3]: [s_store_sk#7, sum#10, sum#11] - -(14) CometColumnarExchange -Input [3]: [s_store_sk#7, sum#10, sum#11] -Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(15) CometColumnarToRow [codegen id : 8] -Input [3]: [s_store_sk#7, sum#10, sum#11] - -(16) HashAggregate [codegen id : 8] -Input [3]: [s_store_sk#7, sum#10, sum#11] -Keys [1]: [s_store_sk#7] -Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#12, sum(UnscaledValue(ss_net_profit#3))#13] -Results [3]: [s_store_sk#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS sales#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#13,17,2) AS profit#15] - -(17) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 6] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] - -(19) Filter [codegen id : 6] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_store_sk#16) - -(20) ReusedExchange [Reuses operator id: 97] -Output [1]: [d_date_sk#20] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_returned_date_sk#19] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] -Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#20] - -(23) ReusedExchange [Reuses operator id: 10] -Output [1]: [s_store_sk#21] - -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_store_sk#16] -Right keys [1]: [s_store_sk#21] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#21] - -(26) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] -Keys [1]: [s_store_sk#21] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum#22, sum#23] -Results [3]: [s_store_sk#21, sum#24, sum#25] - -(27) CometColumnarExchange -Input [3]: [s_store_sk#21, sum#24, sum#25] -Arguments: hashpartitioning(s_store_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#21, sum#24, sum#25] - -(29) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#21, sum#24, sum#25] -Keys [1]: [s_store_sk#21] -Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#17))#26, sum(UnscaledValue(sr_net_loss#18))#27] -Results [3]: [s_store_sk#21, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#26,17,2) AS returns#28, MakeDecimal(sum(UnscaledValue(sr_net_loss#18))#27,17,2) AS profit_loss#29] - -(30) BroadcastExchange -Input [3]: [s_store_sk#21, returns#28, profit_loss#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [s_store_sk#7] -Right keys [1]: [s_store_sk#21] -Join type: LeftOuter -Join condition: None - -(32) Project [codegen id : 8] -Output [5]: [sales#14, coalesce(returns#28, 0.00) AS returns#30, (profit#15 - coalesce(profit_loss#29, 0.00)) AS profit#31, store channel AS channel#32, s_store_sk#7 AS id#33] -Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#21, returns#28, profit_loss#29] - -(33) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#5)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37] - -(35) ReusedExchange [Reuses operator id: 97] -Output [1]: [d_date_sk#38] - -(36) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#37] -Right keys [1]: [d_date_sk#38] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] -Input [5]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37, d_date_sk#38] - -(38) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] -Keys [1]: [cs_call_center_sk#34] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#35)), partial_sum(UnscaledValue(cs_net_profit#36))] -Aggregate Attributes [2]: [sum#39, sum#40] -Results [3]: [cs_call_center_sk#34, sum#41, sum#42] - -(39) CometColumnarExchange -Input [3]: [cs_call_center_sk#34, sum#41, sum#42] -Arguments: hashpartitioning(cs_call_center_sk#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(40) CometColumnarToRow [codegen id : 11] -Input [3]: [cs_call_center_sk#34, sum#41, sum#42] - -(41) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#34, sum#41, sum#42] -Keys [1]: [cs_call_center_sk#34] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#35)), sum(UnscaledValue(cs_net_profit#36))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#35))#43, sum(UnscaledValue(cs_net_profit#36))#44] -Results [3]: [cs_call_center_sk#34, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#35))#43,17,2) AS sales#45, MakeDecimal(sum(UnscaledValue(cs_net_profit#36))#44,17,2) AS profit#46] - -(42) BroadcastExchange -Input [3]: [cs_call_center_sk#34, sales#45, profit#46] -Arguments: IdentityBroadcastMode, [plan_id=6] - -(43) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#49), dynamicpruningexpression(cr_returned_date_sk#49 IN dynamicpruning#5)] -ReadSchema: struct - -(44) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49] - -(45) ReusedExchange [Reuses operator id: 97] -Output [1]: [d_date_sk#50] - -(46) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#49] -Right keys [1]: [d_date_sk#50] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 13] -Output [2]: [cr_return_amount#47, cr_net_loss#48] -Input [4]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49, d_date_sk#50] - -(48) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#47, cr_net_loss#48] -Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#47)), partial_sum(UnscaledValue(cr_net_loss#48))] -Aggregate Attributes [2]: [sum#51, sum#52] -Results [2]: [sum#53, sum#54] - -(49) CometColumnarExchange -Input [2]: [sum#53, sum#54] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(50) CometColumnarToRow -Input [2]: [sum#53, sum#54] - -(51) HashAggregate -Input [2]: [sum#53, sum#54] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#47)), sum(UnscaledValue(cr_net_loss#48))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#47))#55, sum(UnscaledValue(cr_net_loss#48))#56] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#47))#55,17,2) AS returns#57, MakeDecimal(sum(UnscaledValue(cr_net_loss#48))#56,17,2) AS profit_loss#58] - -(52) BroadcastNestedLoopJoin [codegen id : 14] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 14] -Output [5]: [sales#45, returns#57, (profit#46 - profit_loss#58) AS profit#59, catalog channel AS channel#60, cs_call_center_sk#34 AS id#61] -Input [5]: [cs_call_center_sk#34, sales#45, profit#46, returns#57, profit_loss#58] - -(54) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] - -(56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] -Condition : isnotnull(ws_web_page_sk#62) - -(57) ReusedExchange [Reuses operator id: 97] -Output [1]: [d_date_sk#66] - -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#65] -Right keys [1]: [d_date_sk#66] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64] -Input [5]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65, d_date_sk#66] - -(60) CometNativeScan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(61) CometFilter -Input [1]: [wp_web_page_sk#67] -Condition : isnotnull(wp_web_page_sk#67) - -(62) CometColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#67] - -(63) BroadcastExchange -Input [1]: [wp_web_page_sk#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#62] -Right keys [1]: [wp_web_page_sk#67] -Join type: Inner -Join condition: None - -(65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] -Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] - -(66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] -Keys [1]: [wp_web_page_sk#67] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#63)), partial_sum(UnscaledValue(ws_net_profit#64))] -Aggregate Attributes [2]: [sum#68, sum#69] -Results [3]: [wp_web_page_sk#67, sum#70, sum#71] - -(67) CometColumnarExchange -Input [3]: [wp_web_page_sk#67, sum#70, sum#71] -Arguments: hashpartitioning(wp_web_page_sk#67, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(68) CometColumnarToRow [codegen id : 22] -Input [3]: [wp_web_page_sk#67, sum#70, sum#71] - -(69) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#67, sum#70, sum#71] -Keys [1]: [wp_web_page_sk#67] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#63)), sum(UnscaledValue(ws_net_profit#64))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#63))#72, sum(UnscaledValue(ws_net_profit#64))#73] -Results [3]: [wp_web_page_sk#67, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#63))#72,17,2) AS sales#74, MakeDecimal(sum(UnscaledValue(ws_net_profit#64))#73,17,2) AS profit#75] - -(70) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#79), dynamicpruningexpression(wr_returned_date_sk#79 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_web_page_sk)] -ReadSchema: struct - -(71) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] - -(72) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] -Condition : isnotnull(wr_web_page_sk#76) - -(73) ReusedExchange [Reuses operator id: 97] -Output [1]: [d_date_sk#80] - -(74) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#79] -Right keys [1]: [d_date_sk#80] -Join type: Inner -Join condition: None - -(75) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78] -Input [5]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79, d_date_sk#80] - -(76) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#81] - -(77) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#76] -Right keys [1]: [wp_web_page_sk#81] -Join type: Inner -Join condition: None - -(78) Project [codegen id : 20] -Output [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] -Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] - -(79) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] -Keys [1]: [wp_web_page_sk#81] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#77)), partial_sum(UnscaledValue(wr_net_loss#78))] -Aggregate Attributes [2]: [sum#82, sum#83] -Results [3]: [wp_web_page_sk#81, sum#84, sum#85] - -(80) CometColumnarExchange -Input [3]: [wp_web_page_sk#81, sum#84, sum#85] -Arguments: hashpartitioning(wp_web_page_sk#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(81) CometColumnarToRow [codegen id : 21] -Input [3]: [wp_web_page_sk#81, sum#84, sum#85] - -(82) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#81, sum#84, sum#85] -Keys [1]: [wp_web_page_sk#81] -Functions [2]: [sum(UnscaledValue(wr_return_amt#77)), sum(UnscaledValue(wr_net_loss#78))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#77))#86, sum(UnscaledValue(wr_net_loss#78))#87] -Results [3]: [wp_web_page_sk#81, MakeDecimal(sum(UnscaledValue(wr_return_amt#77))#86,17,2) AS returns#88, MakeDecimal(sum(UnscaledValue(wr_net_loss#78))#87,17,2) AS profit_loss#89] - -(83) BroadcastExchange -Input [3]: [wp_web_page_sk#81, returns#88, profit_loss#89] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -(84) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#67] -Right keys [1]: [wp_web_page_sk#81] -Join type: LeftOuter -Join condition: None - -(85) Project [codegen id : 22] -Output [5]: [sales#74, coalesce(returns#88, 0.00) AS returns#90, (profit#75 - coalesce(profit_loss#89, 0.00)) AS profit#91, web channel AS channel#92, wp_web_page_sk#67 AS id#93] -Input [6]: [wp_web_page_sk#67, sales#74, profit#75, wp_web_page_sk#81, returns#88, profit_loss#89] - -(86) Union - -(87) Expand [codegen id : 23] -Input [5]: [sales#14, returns#30, profit#31, channel#32, id#33] -Arguments: [[sales#14, returns#30, profit#31, channel#32, id#33, 0], [sales#14, returns#30, profit#31, channel#32, null, 1], [sales#14, returns#30, profit#31, null, null, 3]], [sales#14, returns#30, profit#31, channel#94, id#95, spark_grouping_id#96] - -(88) HashAggregate [codegen id : 23] -Input [6]: [sales#14, returns#30, profit#31, channel#94, id#95, spark_grouping_id#96] -Keys [3]: [channel#94, id#95, spark_grouping_id#96] -Functions [3]: [partial_sum(sales#14), partial_sum(returns#30), partial_sum(profit#31)] -Aggregate Attributes [6]: [sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Results [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] - -(89) CometColumnarExchange -Input [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Arguments: hashpartitioning(channel#94, id#95, spark_grouping_id#96, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(90) CometColumnarToRow [codegen id : 24] -Input [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] - -(91) HashAggregate [codegen id : 24] -Input [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Keys [3]: [channel#94, id#95, spark_grouping_id#96] -Functions [3]: [sum(sales#14), sum(returns#30), sum(profit#31)] -Aggregate Attributes [3]: [sum(sales#14)#109, sum(returns#30)#110, sum(profit#31)#111] -Results [5]: [channel#94, id#95, sum(sales#14)#109 AS sales#112, sum(returns#30)#110 AS returns#113, sum(profit#31)#111 AS profit#114] - -(92) TakeOrderedAndProject -Input [5]: [channel#94, id#95, sales#112, returns#113, profit#114] -Arguments: 100, [channel#94 ASC NULLS FIRST, id#95 ASC NULLS FIRST], [channel#94, id#95, sales#112, returns#113, profit#114] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (97) -+- * CometColumnarToRow (96) - +- CometProject (95) - +- CometFilter (94) - +- CometNativeScan parquet spark_catalog.default.date_dim (93) - - -(93) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#115] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(94) CometFilter -Input [2]: [d_date_sk#6, d_date#115] -Condition : (((isnotnull(d_date#115) AND (d_date#115 >= 2000-08-03)) AND (d_date#115 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) - -(95) CometProject -Input [2]: [d_date_sk#6, d_date#115] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(96) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(97) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#49 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 70 Hosting Expression = wr_returned_date_sk#79 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/simplified.txt deleted file mode 100644 index 4256e90759..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/simplified.txt +++ /dev/null @@ -1,150 +0,0 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (24) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (23) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (8) - Project [sales,returns,profit,profit_loss,s_store_sk] - BroadcastHashJoin [s_store_sk,s_store_sk] - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #2 - WholeStageCodegen (3) - HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,s_store_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #6 - WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - Project [sr_return_amt,sr_net_loss,s_store_sk] - BroadcastHashJoin [sr_store_sk,s_store_sk] - Project [sr_store_sk,sr_return_amt,sr_net_loss] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [s_store_sk] #4 - WholeStageCodegen (14) - Project [sales,returns,profit,profit_loss,cs_call_center_sk] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) - HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_call_center_sk] #8 - WholeStageCodegen (10) - HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (13) - HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - Project [cr_return_amount,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (22) - Project [sales,returns,profit,profit_loss,wp_web_page_sk] - BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wp_web_page_sk] #10 - WholeStageCodegen (17) - HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_web_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (21) - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wp_web_page_sk] #13 - WholeStageCodegen (20) - HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - Project [wr_return_amt,wr_net_loss,wp_web_page_sk] - BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] - Project [wr_web_page_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Filter [wr_web_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [wp_web_page_sk] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/explain.txt deleted file mode 100644 index bb7ed0a50c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,518 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (85) -+- * HashAggregate (84) - +- * CometColumnarToRow (83) - +- CometColumnarExchange (82) - +- * HashAggregate (81) - +- * Expand (80) - +- Union (79) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometHashAggregate (16) - : : +- CometExchange (15) - : : +- CometHashAggregate (14) - : : +- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (28) - : +- CometHashAggregate (27) - : +- CometExchange (26) - : +- CometHashAggregate (25) - : +- CometProject (24) - : +- CometBroadcastHashJoin (23) - : :- CometProject (21) - : : +- CometBroadcastHashJoin (20) - : : :- CometFilter (18) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (17) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - :- * Project (50) - : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) - : :- BroadcastExchange (40) - : : +- * CometColumnarToRow (39) - : : +- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (32) - : : +- ReusedExchange (33) - : +- * CometColumnarToRow (48) - : +- CometHashAggregate (47) - : +- CometExchange (46) - : +- CometHashAggregate (45) - : +- CometProject (44) - : +- CometBroadcastHashJoin (43) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (41) - : +- ReusedExchange (42) - +- * CometColumnarToRow (78) - +- CometProject (77) - +- CometBroadcastHashJoin (76) - :- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (55) - : : +- CometBroadcastHashJoin (54) - : : :- CometFilter (52) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (51) - : : +- ReusedExchange (53) - : +- CometBroadcastExchange (58) - : +- CometFilter (57) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (56) - +- CometBroadcastExchange (75) - +- CometHashAggregate (74) - +- CometExchange (73) - +- CometHashAggregate (72) - +- CometProject (71) - +- CometBroadcastHashJoin (70) - :- CometProject (68) - : +- CometBroadcastHashJoin (67) - : :- CometFilter (65) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (64) - : +- ReusedExchange (66) - +- ReusedExchange (69) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3], [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [1]: [s_store_sk#8] -Condition : isnotnull(s_store_sk#8) - -(11) CometBroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: [s_store_sk#8] - -(12) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -Right output [1]: [s_store_sk#8] -Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight - -(13) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] - -(14) CometHashAggregate -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Keys [1]: [s_store_sk#8] -Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] - -(15) CometExchange -Input [3]: [s_store_sk#8, sum#9, sum#10] -Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [3]: [s_store_sk#8, sum#9, sum#10] -Keys [1]: [s_store_sk#8] -Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(18) CometFilter -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Condition : isnotnull(sr_store_sk#11) - -(19) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#16] - -(20) CometBroadcastHashJoin -Left output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Right output [1]: [d_date_sk#16] -Arguments: [sr_returned_date_sk#14], [d_date_sk#16], Inner, BuildRight - -(21) CometProject -Input [5]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14, d_date_sk#16] -Arguments: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13], [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] - -(22) ReusedExchange [Reuses operator id: 11] -Output [1]: [s_store_sk#17] - -(23) CometBroadcastHashJoin -Left output [3]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] -Right output [1]: [s_store_sk#17] -Arguments: [sr_store_sk#11], [s_store_sk#17], Inner, BuildRight - -(24) CometProject -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Arguments: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17], [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] - -(25) CometHashAggregate -Input [3]: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Keys [1]: [s_store_sk#17] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#12)), partial_sum(UnscaledValue(sr_net_loss#13))] - -(26) CometExchange -Input [3]: [s_store_sk#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(27) CometHashAggregate -Input [3]: [s_store_sk#17, sum#18, sum#19] -Keys [1]: [s_store_sk#17] -Functions [2]: [sum(UnscaledValue(sr_return_amt#12)), sum(UnscaledValue(sr_net_loss#13))] - -(28) CometBroadcastExchange -Input [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#17, returns#20, profit_loss#21] - -(29) CometBroadcastHashJoin -Left output [3]: [s_store_sk#8, sales#22, profit#23] -Right output [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight - -(30) CometProject -Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] -Arguments: [sales#22, returns#24, profit#25, channel#26, id#27], [sales#22, coalesce(returns#20, 0.00) AS returns#24, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#25, store channel AS channel#26, s_store_sk#8 AS id#27] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [sales#22, returns#24, profit#25, channel#26, id#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] -ReadSchema: struct - -(33) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(34) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [cs_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(35) CometProject -Input [5]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31, d_date_sk#33] -Arguments: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30], [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] - -(36) CometHashAggregate -Input [3]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#29)), partial_sum(UnscaledValue(cs_net_profit#30))] - -(37) CometExchange -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Arguments: hashpartitioning(cs_call_center_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] - -(39) CometColumnarToRow [codegen id : 2] -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] - -(40) BroadcastExchange -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] -Arguments: IdentityBroadcastMode, [plan_id=4] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] -ReadSchema: struct - -(42) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#42] - -(43) CometBroadcastHashJoin -Left output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Right output [1]: [d_date_sk#42] -Arguments: [cr_returned_date_sk#40], [d_date_sk#42], Inner, BuildRight - -(44) CometProject -Input [4]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40, d_date_sk#42] -Arguments: [cr_return_amount#38, cr_net_loss#39], [cr_return_amount#38, cr_net_loss#39] - -(45) CometHashAggregate -Input [2]: [cr_return_amount#38, cr_net_loss#39] -Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] - -(46) CometExchange -Input [2]: [sum#43, sum#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometHashAggregate -Input [2]: [sum#43, sum#44] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] - -(48) CometColumnarToRow -Input [2]: [returns#45, profit_loss#46] - -(49) BroadcastNestedLoopJoin [codegen id : 3] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 3] -Output [5]: [sales#36, returns#45, (profit#37 - profit_loss#46) AS profit#47, catalog channel AS channel#48, cs_call_center_sk#28 AS id#49] -Input [5]: [cs_call_center_sk#28, sales#36, profit#37, returns#45, profit_loss#46] - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] -PushedFilters: [IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(52) CometFilter -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Condition : isnotnull(ws_web_page_sk#50) - -(53) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#55] - -(54) CometBroadcastHashJoin -Left output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Right output [1]: [d_date_sk#55] -Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight - -(55) CometProject -Input [5]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53, d_date_sk#55] -Arguments: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52], [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(57) CometFilter -Input [1]: [wp_web_page_sk#56] -Condition : isnotnull(wp_web_page_sk#56) - -(58) CometBroadcastExchange -Input [1]: [wp_web_page_sk#56] -Arguments: [wp_web_page_sk#56] - -(59) CometBroadcastHashJoin -Left output [3]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] -Right output [1]: [wp_web_page_sk#56] -Arguments: [ws_web_page_sk#50], [wp_web_page_sk#56], Inner, BuildRight - -(60) CometProject -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Arguments: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56], [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] - -(61) CometHashAggregate -Input [3]: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#51)), partial_sum(UnscaledValue(ws_net_profit#52))] - -(62) CometExchange -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Arguments: hashpartitioning(wp_web_page_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(63) CometHashAggregate -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#51)), sum(UnscaledValue(ws_net_profit#52))] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] -PushedFilters: [IsNotNull(wr_web_page_sk)] -ReadSchema: struct - -(65) CometFilter -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Condition : isnotnull(wr_web_page_sk#59) - -(66) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#64] - -(67) CometBroadcastHashJoin -Left output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Right output [1]: [d_date_sk#64] -Arguments: [wr_returned_date_sk#62], [d_date_sk#64], Inner, BuildRight - -(68) CometProject -Input [5]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62, d_date_sk#64] -Arguments: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61], [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] - -(69) ReusedExchange [Reuses operator id: 58] -Output [1]: [wp_web_page_sk#65] - -(70) CometBroadcastHashJoin -Left output [3]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] -Right output [1]: [wp_web_page_sk#65] -Arguments: [wr_web_page_sk#59], [wp_web_page_sk#65], Inner, BuildRight - -(71) CometProject -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Arguments: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65], [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] - -(72) CometHashAggregate -Input [3]: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#60)), partial_sum(UnscaledValue(wr_net_loss#61))] - -(73) CometExchange -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Arguments: hashpartitioning(wp_web_page_sk#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(74) CometHashAggregate -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [sum(UnscaledValue(wr_return_amt#60)), sum(UnscaledValue(wr_net_loss#61))] - -(75) CometBroadcastExchange -Input [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#65, returns#68, profit_loss#69] - -(76) CometBroadcastHashJoin -Left output [3]: [wp_web_page_sk#56, sales#70, profit#71] -Right output [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight - -(77) CometProject -Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [sales#70, returns#72, profit#73, channel#74, id#75], [sales#70, coalesce(returns#68, 0.00) AS returns#72, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#73, web channel AS channel#74, wp_web_page_sk#56 AS id#75] - -(78) CometColumnarToRow [codegen id : 4] -Input [5]: [sales#70, returns#72, profit#73, channel#74, id#75] - -(79) Union - -(80) Expand [codegen id : 5] -Input [5]: [sales#22, returns#24, profit#25, channel#26, id#27] -Arguments: [[sales#22, returns#24, profit#25, channel#26, id#27, 0], [sales#22, returns#24, profit#25, channel#26, null, 1], [sales#22, returns#24, profit#25, null, null, 3]], [sales#22, returns#24, profit#25, channel#76, id#77, spark_grouping_id#78] - -(81) HashAggregate [codegen id : 5] -Input [6]: [sales#22, returns#24, profit#25, channel#76, id#77, spark_grouping_id#78] -Keys [3]: [channel#76, id#77, spark_grouping_id#78] -Functions [3]: [partial_sum(sales#22), partial_sum(returns#24), partial_sum(profit#25)] -Aggregate Attributes [6]: [sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84] -Results [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] - -(82) CometColumnarExchange -Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] -Arguments: hashpartitioning(channel#76, id#77, spark_grouping_id#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(83) CometColumnarToRow [codegen id : 6] -Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] - -(84) HashAggregate [codegen id : 6] -Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] -Keys [3]: [channel#76, id#77, spark_grouping_id#78] -Functions [3]: [sum(sales#22), sum(returns#24), sum(profit#25)] -Aggregate Attributes [3]: [sum(sales#22)#91, sum(returns#24)#92, sum(profit#25)#93] -Results [5]: [channel#76, id#77, sum(sales#22)#91 AS sales#94, sum(returns#24)#92 AS returns#95, sum(profit#25)#93 AS profit#96] - -(85) TakeOrderedAndProject -Input [5]: [channel#76, id#77, sales#94, returns#95, profit#96] -Arguments: 100, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#94, returns#95, profit#96] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (90) -+- * CometColumnarToRow (89) - +- CometProject (88) - +- CometFilter (87) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (86) - - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(87) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) - -(88) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(89) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(90) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#14 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 64 Hosting Expression = wr_returned_date_sk#62 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/simplified.txt deleted file mode 100644 index a1243769e5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,111 +0,0 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (6) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (5) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [returns,profit,profit_loss,s_store_sk] [sales,returns,profit,channel,id] - CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [s_store_sk,sales,profit,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] - CometExchange [s_store_sk] #2 - CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk] #5 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [s_store_sk,returns,profit_loss] #6 - CometHashAggregate [sum,sum] [s_store_sk,returns,profit_loss,sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss))] - CometExchange [s_store_sk] #7 - CometHashAggregate [sr_return_amt,sr_net_loss] [s_store_sk,sum,sum] - CometProject [sr_return_amt,sr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [s_store_sk] #5 - WholeStageCodegen (3) - Project [sales,returns,profit,profit_loss,cs_call_center_sk] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [cs_call_center_sk,sales,profit,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] - CometExchange [cs_call_center_sk] #9 - CometHashAggregate [cs_ext_sales_price,cs_net_profit] [cs_call_center_sk,sum,sum] - CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [returns,profit_loss,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] - CometExchange #10 - CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] - CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [returns,profit,profit_loss,wp_web_page_sk] [sales,returns,profit,channel,id] - CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [wp_web_page_sk,sales,profit,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] - CometExchange [wp_web_page_sk] #11 - CometHashAggregate [ws_ext_sales_price,ws_net_profit] [wp_web_page_sk,sum,sum] - CometProject [ws_ext_sales_price,ws_net_profit,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_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 [native_iceberg_compat] 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 [wp_web_page_sk] #12 - CometFilter [wp_web_page_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [wp_web_page_sk,returns,profit_loss] #13 - CometHashAggregate [sum,sum] [wp_web_page_sk,returns,profit_loss,sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss))] - CometExchange [wp_web_page_sk] #14 - CometHashAggregate [wr_return_amt,wr_net_loss] [wp_web_page_sk,sum,sum] - CometProject [wr_return_amt,wr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [wp_web_page_sk] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/explain.txt deleted file mode 100644 index bb7ed0a50c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/explain.txt +++ /dev/null @@ -1,518 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (85) -+- * HashAggregate (84) - +- * CometColumnarToRow (83) - +- CometColumnarExchange (82) - +- * HashAggregate (81) - +- * Expand (80) - +- Union (79) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometHashAggregate (16) - : : +- CometExchange (15) - : : +- CometHashAggregate (14) - : : +- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (28) - : +- CometHashAggregate (27) - : +- CometExchange (26) - : +- CometHashAggregate (25) - : +- CometProject (24) - : +- CometBroadcastHashJoin (23) - : :- CometProject (21) - : : +- CometBroadcastHashJoin (20) - : : :- CometFilter (18) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (17) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - :- * Project (50) - : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) - : :- BroadcastExchange (40) - : : +- * CometColumnarToRow (39) - : : +- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (32) - : : +- ReusedExchange (33) - : +- * CometColumnarToRow (48) - : +- CometHashAggregate (47) - : +- CometExchange (46) - : +- CometHashAggregate (45) - : +- CometProject (44) - : +- CometBroadcastHashJoin (43) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (41) - : +- ReusedExchange (42) - +- * CometColumnarToRow (78) - +- CometProject (77) - +- CometBroadcastHashJoin (76) - :- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (55) - : : +- CometBroadcastHashJoin (54) - : : :- CometFilter (52) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (51) - : : +- ReusedExchange (53) - : +- CometBroadcastExchange (58) - : +- CometFilter (57) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (56) - +- CometBroadcastExchange (75) - +- CometHashAggregate (74) - +- CometExchange (73) - +- CometHashAggregate (72) - +- CometProject (71) - +- CometBroadcastHashJoin (70) - :- CometProject (68) - : +- CometBroadcastHashJoin (67) - : :- CometFilter (65) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (64) - : +- ReusedExchange (66) - +- ReusedExchange (69) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3], [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [1]: [s_store_sk#8] -Condition : isnotnull(s_store_sk#8) - -(11) CometBroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: [s_store_sk#8] - -(12) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -Right output [1]: [s_store_sk#8] -Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight - -(13) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] - -(14) CometHashAggregate -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Keys [1]: [s_store_sk#8] -Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] - -(15) CometExchange -Input [3]: [s_store_sk#8, sum#9, sum#10] -Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [3]: [s_store_sk#8, sum#9, sum#10] -Keys [1]: [s_store_sk#8] -Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(18) CometFilter -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Condition : isnotnull(sr_store_sk#11) - -(19) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#16] - -(20) CometBroadcastHashJoin -Left output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Right output [1]: [d_date_sk#16] -Arguments: [sr_returned_date_sk#14], [d_date_sk#16], Inner, BuildRight - -(21) CometProject -Input [5]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14, d_date_sk#16] -Arguments: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13], [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] - -(22) ReusedExchange [Reuses operator id: 11] -Output [1]: [s_store_sk#17] - -(23) CometBroadcastHashJoin -Left output [3]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] -Right output [1]: [s_store_sk#17] -Arguments: [sr_store_sk#11], [s_store_sk#17], Inner, BuildRight - -(24) CometProject -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Arguments: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17], [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] - -(25) CometHashAggregate -Input [3]: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Keys [1]: [s_store_sk#17] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#12)), partial_sum(UnscaledValue(sr_net_loss#13))] - -(26) CometExchange -Input [3]: [s_store_sk#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(27) CometHashAggregate -Input [3]: [s_store_sk#17, sum#18, sum#19] -Keys [1]: [s_store_sk#17] -Functions [2]: [sum(UnscaledValue(sr_return_amt#12)), sum(UnscaledValue(sr_net_loss#13))] - -(28) CometBroadcastExchange -Input [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#17, returns#20, profit_loss#21] - -(29) CometBroadcastHashJoin -Left output [3]: [s_store_sk#8, sales#22, profit#23] -Right output [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight - -(30) CometProject -Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] -Arguments: [sales#22, returns#24, profit#25, channel#26, id#27], [sales#22, coalesce(returns#20, 0.00) AS returns#24, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#25, store channel AS channel#26, s_store_sk#8 AS id#27] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [sales#22, returns#24, profit#25, channel#26, id#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] -ReadSchema: struct - -(33) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(34) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [cs_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(35) CometProject -Input [5]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31, d_date_sk#33] -Arguments: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30], [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] - -(36) CometHashAggregate -Input [3]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#29)), partial_sum(UnscaledValue(cs_net_profit#30))] - -(37) CometExchange -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Arguments: hashpartitioning(cs_call_center_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] - -(39) CometColumnarToRow [codegen id : 2] -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] - -(40) BroadcastExchange -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] -Arguments: IdentityBroadcastMode, [plan_id=4] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] -ReadSchema: struct - -(42) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#42] - -(43) CometBroadcastHashJoin -Left output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Right output [1]: [d_date_sk#42] -Arguments: [cr_returned_date_sk#40], [d_date_sk#42], Inner, BuildRight - -(44) CometProject -Input [4]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40, d_date_sk#42] -Arguments: [cr_return_amount#38, cr_net_loss#39], [cr_return_amount#38, cr_net_loss#39] - -(45) CometHashAggregate -Input [2]: [cr_return_amount#38, cr_net_loss#39] -Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] - -(46) CometExchange -Input [2]: [sum#43, sum#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometHashAggregate -Input [2]: [sum#43, sum#44] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] - -(48) CometColumnarToRow -Input [2]: [returns#45, profit_loss#46] - -(49) BroadcastNestedLoopJoin [codegen id : 3] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 3] -Output [5]: [sales#36, returns#45, (profit#37 - profit_loss#46) AS profit#47, catalog channel AS channel#48, cs_call_center_sk#28 AS id#49] -Input [5]: [cs_call_center_sk#28, sales#36, profit#37, returns#45, profit_loss#46] - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] -PushedFilters: [IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(52) CometFilter -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Condition : isnotnull(ws_web_page_sk#50) - -(53) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#55] - -(54) CometBroadcastHashJoin -Left output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Right output [1]: [d_date_sk#55] -Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight - -(55) CometProject -Input [5]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53, d_date_sk#55] -Arguments: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52], [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(57) CometFilter -Input [1]: [wp_web_page_sk#56] -Condition : isnotnull(wp_web_page_sk#56) - -(58) CometBroadcastExchange -Input [1]: [wp_web_page_sk#56] -Arguments: [wp_web_page_sk#56] - -(59) CometBroadcastHashJoin -Left output [3]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] -Right output [1]: [wp_web_page_sk#56] -Arguments: [ws_web_page_sk#50], [wp_web_page_sk#56], Inner, BuildRight - -(60) CometProject -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Arguments: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56], [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] - -(61) CometHashAggregate -Input [3]: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#51)), partial_sum(UnscaledValue(ws_net_profit#52))] - -(62) CometExchange -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Arguments: hashpartitioning(wp_web_page_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(63) CometHashAggregate -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#51)), sum(UnscaledValue(ws_net_profit#52))] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] -PushedFilters: [IsNotNull(wr_web_page_sk)] -ReadSchema: struct - -(65) CometFilter -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Condition : isnotnull(wr_web_page_sk#59) - -(66) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#64] - -(67) CometBroadcastHashJoin -Left output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Right output [1]: [d_date_sk#64] -Arguments: [wr_returned_date_sk#62], [d_date_sk#64], Inner, BuildRight - -(68) CometProject -Input [5]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62, d_date_sk#64] -Arguments: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61], [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] - -(69) ReusedExchange [Reuses operator id: 58] -Output [1]: [wp_web_page_sk#65] - -(70) CometBroadcastHashJoin -Left output [3]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] -Right output [1]: [wp_web_page_sk#65] -Arguments: [wr_web_page_sk#59], [wp_web_page_sk#65], Inner, BuildRight - -(71) CometProject -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Arguments: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65], [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] - -(72) CometHashAggregate -Input [3]: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#60)), partial_sum(UnscaledValue(wr_net_loss#61))] - -(73) CometExchange -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Arguments: hashpartitioning(wp_web_page_sk#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(74) CometHashAggregate -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [sum(UnscaledValue(wr_return_amt#60)), sum(UnscaledValue(wr_net_loss#61))] - -(75) CometBroadcastExchange -Input [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#65, returns#68, profit_loss#69] - -(76) CometBroadcastHashJoin -Left output [3]: [wp_web_page_sk#56, sales#70, profit#71] -Right output [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight - -(77) CometProject -Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [sales#70, returns#72, profit#73, channel#74, id#75], [sales#70, coalesce(returns#68, 0.00) AS returns#72, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#73, web channel AS channel#74, wp_web_page_sk#56 AS id#75] - -(78) CometColumnarToRow [codegen id : 4] -Input [5]: [sales#70, returns#72, profit#73, channel#74, id#75] - -(79) Union - -(80) Expand [codegen id : 5] -Input [5]: [sales#22, returns#24, profit#25, channel#26, id#27] -Arguments: [[sales#22, returns#24, profit#25, channel#26, id#27, 0], [sales#22, returns#24, profit#25, channel#26, null, 1], [sales#22, returns#24, profit#25, null, null, 3]], [sales#22, returns#24, profit#25, channel#76, id#77, spark_grouping_id#78] - -(81) HashAggregate [codegen id : 5] -Input [6]: [sales#22, returns#24, profit#25, channel#76, id#77, spark_grouping_id#78] -Keys [3]: [channel#76, id#77, spark_grouping_id#78] -Functions [3]: [partial_sum(sales#22), partial_sum(returns#24), partial_sum(profit#25)] -Aggregate Attributes [6]: [sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84] -Results [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] - -(82) CometColumnarExchange -Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] -Arguments: hashpartitioning(channel#76, id#77, spark_grouping_id#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(83) CometColumnarToRow [codegen id : 6] -Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] - -(84) HashAggregate [codegen id : 6] -Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] -Keys [3]: [channel#76, id#77, spark_grouping_id#78] -Functions [3]: [sum(sales#22), sum(returns#24), sum(profit#25)] -Aggregate Attributes [3]: [sum(sales#22)#91, sum(returns#24)#92, sum(profit#25)#93] -Results [5]: [channel#76, id#77, sum(sales#22)#91 AS sales#94, sum(returns#24)#92 AS returns#95, sum(profit#25)#93 AS profit#96] - -(85) TakeOrderedAndProject -Input [5]: [channel#76, id#77, sales#94, returns#95, profit#96] -Arguments: 100, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#94, returns#95, profit#96] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (90) -+- * CometColumnarToRow (89) - +- CometProject (88) - +- CometFilter (87) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (86) - - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(87) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) - -(88) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(89) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(90) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#14 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 64 Hosting Expression = wr_returned_date_sk#62 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/extended.txt deleted file mode 100644 index ed8a9e38ca..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/extended.txt +++ /dev/null @@ -1,117 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : :- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - -Comet accelerated 94 out of 109 eligible operators (86%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index a1243769e5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt +++ /dev/null @@ -1,111 +0,0 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (6) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (5) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [returns,profit,profit_loss,s_store_sk] [sales,returns,profit,channel,id] - CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [s_store_sk,sales,profit,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] - CometExchange [s_store_sk] #2 - CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk] #5 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [s_store_sk,returns,profit_loss] #6 - CometHashAggregate [sum,sum] [s_store_sk,returns,profit_loss,sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss))] - CometExchange [s_store_sk] #7 - CometHashAggregate [sr_return_amt,sr_net_loss] [s_store_sk,sum,sum] - CometProject [sr_return_amt,sr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [s_store_sk] #5 - WholeStageCodegen (3) - Project [sales,returns,profit,profit_loss,cs_call_center_sk] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [cs_call_center_sk,sales,profit,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] - CometExchange [cs_call_center_sk] #9 - CometHashAggregate [cs_ext_sales_price,cs_net_profit] [cs_call_center_sk,sum,sum] - CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [returns,profit_loss,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] - CometExchange #10 - CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] - CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [returns,profit,profit_loss,wp_web_page_sk] [sales,returns,profit,channel,id] - CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [wp_web_page_sk,sales,profit,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] - CometExchange [wp_web_page_sk] #11 - CometHashAggregate [ws_ext_sales_price,ws_net_profit] [wp_web_page_sk,sum,sum] - CometProject [ws_ext_sales_price,ws_net_profit,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_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 [native_iceberg_compat] 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 [wp_web_page_sk] #12 - CometFilter [wp_web_page_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [wp_web_page_sk,returns,profit_loss] #13 - CometHashAggregate [sum,sum] [wp_web_page_sk,returns,profit_loss,sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss))] - CometExchange [wp_web_page_sk] #14 - CometHashAggregate [wr_return_amt,wr_net_loss] [wp_web_page_sk,sum,sum] - CometProject [wr_return_amt,wr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [wp_web_page_sk] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/explain.txt deleted file mode 100644 index 4dc7ac380a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/explain.txt +++ /dev/null @@ -1,417 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (70) -+- * Project (69) - +- * CometColumnarToRow (68) - +- CometSortMergeJoin (67) - :- CometProject (45) - : +- CometSortMergeJoin (44) - : :- CometSort (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometFilter (12) - : : : +- CometSortMergeJoin (11) - : : : :- CometSort (5) - : : : : +- CometColumnarExchange (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (10) - : : : +- CometExchange (9) - : : : +- CometProject (8) - : : : +- CometFilter (7) - : : : +- CometNativeScan parquet spark_catalog.default.store_returns (6) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (14) - : +- CometSort (43) - : +- CometFilter (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometFilter (34) - : : +- CometSortMergeJoin (33) - : : :- CometSort (27) - : : : +- CometColumnarExchange (26) - : : : +- * Filter (25) - : : : +- * ColumnarToRow (24) - : : : +- Scan parquet spark_catalog.default.web_sales (23) - : : +- CometSort (32) - : : +- CometExchange (31) - : : +- CometProject (30) - : : +- CometFilter (29) - : : +- CometNativeScan parquet spark_catalog.default.web_returns (28) - : +- ReusedExchange (36) - +- CometSort (66) - +- CometFilter (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometFilter (57) - : +- CometSortMergeJoin (56) - : :- CometSort (50) - : : +- CometColumnarExchange (49) - : : +- * Filter (48) - : : +- * ColumnarToRow (47) - : : +- Scan parquet spark_catalog.default.catalog_sales (46) - : +- CometSort (55) - : +- CometExchange (54) - : +- CometProject (53) - : +- CometFilter (52) - : +- CometNativeScan parquet spark_catalog.default.catalog_returns (51) - +- ReusedExchange (59) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(3) Filter [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_customer_sk#2)) - -(4) CometColumnarExchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_ticket_number#3, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_ticket_number#3 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) - -(8) CometProject -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] - -(9) CometExchange -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter - -(12) CometFilter -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Condition : isnull(sr_ticket_number#10) - -(13) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(14) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(17) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] - -(19) CometHashAggregate -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] - -(20) CometExchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(21) CometHashAggregate -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] - -(22) CometSort -Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] - -(23) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 2] -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(25) Filter [codegen id : 2] -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) - -(26) CometColumnarExchange -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometSort -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] - -(28) CometNativeScan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#28, wr_order_number#29, wr_returned_date_sk#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(29) CometFilter -Input [3]: [wr_item_sk#28, wr_order_number#29, wr_returned_date_sk#30] -Condition : (isnotnull(wr_order_number#29) AND isnotnull(wr_item_sk#28)) - -(30) CometProject -Input [3]: [wr_item_sk#28, wr_order_number#29, wr_returned_date_sk#30] -Arguments: [wr_item_sk#28, wr_order_number#29], [wr_item_sk#28, wr_order_number#29] - -(31) CometExchange -Input [2]: [wr_item_sk#28, wr_order_number#29] -Arguments: hashpartitioning(wr_order_number#29, wr_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(32) CometSort -Input [2]: [wr_item_sk#28, wr_order_number#29] -Arguments: [wr_item_sk#28, wr_order_number#29], [wr_order_number#29 ASC NULLS FIRST, wr_item_sk#28 ASC NULLS FIRST] - -(33) CometSortMergeJoin -Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [wr_item_sk#28, wr_order_number#29] -Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#29, wr_item_sk#28], LeftOuter - -(34) CometFilter -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#28, wr_order_number#29] -Condition : isnull(wr_order_number#29) - -(35) CometProject -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#28, wr_order_number#29] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(36) ReusedExchange [Reuses operator id: 16] -Output [2]: [d_date_sk#31, d_year#32] - -(37) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [d_date_sk#31, d_year#32] -Arguments: [ws_sold_date_sk#27], [d_date_sk#31], Inner, BuildRight - -(38) CometProject -Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#31, d_year#32] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#32], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#32] - -(39) CometHashAggregate -Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#32] -Keys [3]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] - -(40) CometExchange -Input [6]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22, sum#33, sum#34, sum#35] -Arguments: hashpartitioning(d_year#32, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(41) CometHashAggregate -Input [6]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22, sum#33, sum#34, sum#35] -Keys [3]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] - -(42) CometFilter -Input [6]: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Condition : (coalesce(ws_qty#38, 0) > 0) - -(43) CometSort -Input [6]: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Arguments: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40], [ws_sold_year#36 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#37 ASC NULLS FIRST] - -(44) CometSortMergeJoin -Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Right output [6]: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37], Inner - -(45) CometProject -Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40] - -(46) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#47), dynamicpruningexpression(cs_sold_date_sk#47 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(47) ColumnarToRow [codegen id : 3] -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] - -(48) Filter [codegen id : 3] -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Condition : (isnotnull(cs_item_sk#42) AND isnotnull(cs_bill_customer_sk#41)) - -(49) CometColumnarExchange -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Arguments: hashpartitioning(cs_order_number#43, cs_item_sk#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(50) CometSort -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Arguments: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47], [cs_order_number#43 ASC NULLS FIRST, cs_item_sk#42 ASC NULLS FIRST] - -(51) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#48, cr_order_number#49, cr_returned_date_sk#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(52) CometFilter -Input [3]: [cr_item_sk#48, cr_order_number#49, cr_returned_date_sk#50] -Condition : (isnotnull(cr_order_number#49) AND isnotnull(cr_item_sk#48)) - -(53) CometProject -Input [3]: [cr_item_sk#48, cr_order_number#49, cr_returned_date_sk#50] -Arguments: [cr_item_sk#48, cr_order_number#49], [cr_item_sk#48, cr_order_number#49] - -(54) CometExchange -Input [2]: [cr_item_sk#48, cr_order_number#49] -Arguments: hashpartitioning(cr_order_number#49, cr_item_sk#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(55) CometSort -Input [2]: [cr_item_sk#48, cr_order_number#49] -Arguments: [cr_item_sk#48, cr_order_number#49], [cr_order_number#49 ASC NULLS FIRST, cr_item_sk#48 ASC NULLS FIRST] - -(56) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Right output [2]: [cr_item_sk#48, cr_order_number#49] -Arguments: [cs_order_number#43, cs_item_sk#42], [cr_order_number#49, cr_item_sk#48], LeftOuter - -(57) CometFilter -Input [9]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47, cr_item_sk#48, cr_order_number#49] -Condition : isnull(cr_order_number#49) - -(58) CometProject -Input [9]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47, cr_item_sk#48, cr_order_number#49] -Arguments: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47], [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] - -(59) ReusedExchange [Reuses operator id: 16] -Output [2]: [d_date_sk#51, d_year#52] - -(60) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Right output [2]: [d_date_sk#51, d_year#52] -Arguments: [cs_sold_date_sk#47], [d_date_sk#51], Inner, BuildRight - -(61) CometProject -Input [8]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47, d_date_sk#51, d_year#52] -Arguments: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, d_year#52], [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, d_year#52] - -(62) CometHashAggregate -Input [6]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, d_year#52] -Keys [3]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41] -Functions [3]: [partial_sum(cs_quantity#44), partial_sum(UnscaledValue(cs_wholesale_cost#45)), partial_sum(UnscaledValue(cs_sales_price#46))] - -(63) CometExchange -Input [6]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41, sum#53, sum#54, sum#55] -Arguments: hashpartitioning(d_year#52, cs_item_sk#42, cs_bill_customer_sk#41, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(64) CometHashAggregate -Input [6]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41, sum#53, sum#54, sum#55] -Keys [3]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41] -Functions [3]: [sum(cs_quantity#44), sum(UnscaledValue(cs_wholesale_cost#45)), sum(UnscaledValue(cs_sales_price#46))] - -(65) CometFilter -Input [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] -Condition : (coalesce(cs_qty#58, 0) > 0) - -(66) CometSort -Input [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] -Arguments: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60], [cs_sold_year#56 ASC NULLS FIRST, cs_item_sk#42 ASC NULLS FIRST, cs_customer_sk#57 ASC NULLS FIRST] - -(67) CometSortMergeJoin -Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40] -Right output [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57], Inner - -(68) CometColumnarToRow [codegen id : 4] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40, cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] - -(69) Project [codegen id : 4] -Output [12]: [round((cast(ss_qty#18 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#38 + cs_qty#58), 1) as double)))), 2) AS ratio#61, ss_qty#18 AS store_qty#62, ss_wc#19 AS store_wholesale_cost#63, ss_sp#20 AS store_sales_price#64, (coalesce(ws_qty#38, 0) + coalesce(cs_qty#58, 0)) AS other_chan_qty#65, (coalesce(ws_wc#39, 0.00) + coalesce(cs_wc#59, 0.00)) AS other_chan_wholesale_cost#66, (coalesce(ws_sp#40, 0.00) + coalesce(cs_sp#60, 0.00)) AS other_chan_sales_price#67, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, cs_qty#58] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40, cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] - -(70) TakeOrderedAndProject -Input [12]: [ratio#61, store_qty#62, store_wholesale_cost#63, store_sales_price#64, other_chan_qty#65, other_chan_wholesale_cost#66, other_chan_sales_price#67, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, cs_qty#58] -Arguments: 100, [ratio#61 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#65 ASC NULLS FIRST, other_chan_wholesale_cost#66 ASC NULLS FIRST, other_chan_sales_price#67 ASC NULLS FIRST, round((cast(ss_qty#18 as double) / cast(coalesce((ws_qty#38 + cs_qty#58), 1) as double)), 2) ASC NULLS FIRST], [ratio#61, store_qty#62, store_wholesale_cost#63, store_sales_price#64, other_chan_qty#65, other_chan_wholesale_cost#66, other_chan_sales_price#67] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (74) -+- * CometColumnarToRow (73) - +- CometFilter (72) - +- CometNativeScan parquet spark_catalog.default.date_dim (71) - - -(71) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(72) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(73) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(74) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#47 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/simplified.txt deleted file mode 100644 index ebd0840eec..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/simplified.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (4) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - CometColumnarToRow - InputAdapter - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] - CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #2 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #5 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] - CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometColumnarExchange [ws_order_number,ws_item_sk] #7 - WholeStageCodegen (2) - Filter [ws_item_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_order_number,wr_item_sk] #8 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] - CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometColumnarExchange [cs_order_number,cs_item_sk] #10 - WholeStageCodegen (3) - Filter [cs_item_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_order_number,cr_item_sk] #11 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/explain.txt deleted file mode 100644 index 3d3ea5ab86..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,405 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (67) -+- * Project (66) - +- * CometColumnarToRow (65) - +- CometSortMergeJoin (64) - :- CometProject (43) - : +- CometSortMergeJoin (42) - : :- CometSort (21) - : : +- CometHashAggregate (20) - : : +- CometExchange (19) - : : +- CometHashAggregate (18) - : : +- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : +- CometBroadcastExchange (15) - : : +- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : +- CometSort (41) - : +- CometFilter (40) - : +- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (33) - : : +- CometFilter (32) - : : +- CometSortMergeJoin (31) - : : :- CometSort (25) - : : : +- CometExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (22) - : : +- CometSort (30) - : : +- CometExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (26) - : +- ReusedExchange (34) - +- CometSort (63) - +- CometFilter (62) - +- CometHashAggregate (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometProject (55) - : +- CometFilter (54) - : +- CometSortMergeJoin (53) - : :- CometSort (47) - : : +- CometExchange (46) - : : +- CometFilter (45) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (44) - : +- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometFilter (49) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (48) - +- ReusedExchange (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_customer_sk#2)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_ticket_number#3, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_ticket_number#3 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) - -(7) CometProject -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] - -(8) CometExchange -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter - -(11) CometFilter -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Condition : isnull(sr_ticket_number#10) - -(12) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(15) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(16) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight - -(17) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] - -(18) CometHashAggregate -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] - -(19) CometExchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometHashAggregate -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] - -(21) CometSort -Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(23) CometFilter -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) - -(24) CometExchange -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometSort -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Condition : (isnotnull(wr_order_number#30) AND isnotnull(wr_item_sk#29)) - -(28) CometProject -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_item_sk#29, wr_order_number#30] - -(29) CometExchange -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: hashpartitioning(wr_order_number#30, wr_item_sk#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(30) CometSort -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_order_number#30 ASC NULLS FIRST, wr_item_sk#29 ASC NULLS FIRST] - -(31) CometSortMergeJoin -Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#30, wr_item_sk#29], LeftOuter - -(32) CometFilter -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Condition : isnull(wr_order_number#30) - -(33) CometProject -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(34) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#32, d_year#33] - -(35) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ws_sold_date_sk#27], [d_date_sk#32], Inner, BuildRight - -(36) CometProject -Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#32, d_year#33] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] - -(37) CometHashAggregate -Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] - -(38) CometExchange -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Arguments: hashpartitioning(d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(39) CometHashAggregate -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] - -(40) CometFilter -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Condition : (coalesce(ws_qty#39, 0) > 0) - -(41) CometSort -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41], [ws_sold_year#37 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#38 ASC NULLS FIRST] - -(42) CometSortMergeJoin -Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Right output [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38], Inner - -(43) CometProject -Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#48), dynamicpruningexpression(cs_sold_date_sk#48 IN dynamicpruning#49)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(45) CometFilter -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Condition : (isnotnull(cs_item_sk#43) AND isnotnull(cs_bill_customer_sk#42)) - -(46) CometExchange -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(47) CometSort -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST] - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Condition : (isnotnull(cr_order_number#51) AND isnotnull(cr_item_sk#50)) - -(50) CometProject -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_item_sk#50, cr_order_number#51] - -(51) CometExchange -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(52) CometSort -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST] - -(53) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cs_order_number#44, cs_item_sk#43], [cr_order_number#51, cr_item_sk#50], LeftOuter - -(54) CometFilter -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Condition : isnull(cr_order_number#51) - -(55) CometProject -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] - -(56) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#53, d_year#54] - -(57) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [cs_sold_date_sk#48], [d_date_sk#53], Inner, BuildRight - -(58) CometProject -Input [8]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, d_date_sk#53, d_year#54] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] - -(59) CometHashAggregate -Input [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [partial_sum(cs_quantity#45), partial_sum(UnscaledValue(cs_wholesale_cost#46)), partial_sum(UnscaledValue(cs_sales_price#47))] - -(60) CometExchange -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Arguments: hashpartitioning(d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(61) CometHashAggregate -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [sum(cs_quantity#45), sum(UnscaledValue(cs_wholesale_cost#46)), sum(UnscaledValue(cs_sales_price#47))] - -(62) CometFilter -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Condition : (coalesce(cs_qty#60, 0) > 0) - -(63) CometSort -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62], [cs_sold_year#58 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST, cs_customer_sk#59 ASC NULLS FIRST] - -(64) CometSortMergeJoin -Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] -Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner - -(65) CometColumnarToRow [codegen id : 1] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(66) Project [codegen id : 1] -Output [12]: [round((cast(ss_qty#18 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)))), 2) AS ratio#63, ss_qty#18 AS store_qty#64, ss_wc#19 AS store_wholesale_cost#65, ss_sp#20 AS store_sales_price#66, (coalesce(ws_qty#39, 0) + coalesce(cs_qty#60, 0)) AS other_chan_qty#67, (coalesce(ws_wc#40, 0.00) + coalesce(cs_wc#61, 0.00)) AS other_chan_wholesale_cost#68, (coalesce(ws_sp#41, 0.00) + coalesce(cs_sp#62, 0.00)) AS other_chan_sales_price#69, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, cs_qty#60] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(67) TakeOrderedAndProject -Input [12]: [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, cs_qty#60] -Arguments: 100, [ratio#63 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#67 ASC NULLS FIRST, other_chan_wholesale_cost#68 ASC NULLS FIRST, other_chan_sales_price#69 ASC NULLS FIRST, round((cast(ss_qty#18 as double) / cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)), 2) ASC NULLS FIRST], [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) - - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(70) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(71) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/simplified.txt deleted file mode 100644 index 77c05217f4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,78 +0,0 @@ -TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (1) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - CometColumnarToRow - InputAdapter - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] - CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #5 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] - CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometExchange [ws_order_number,ws_item_sk] #7 - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_order_number,wr_item_sk] #8 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] - CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometExchange [cs_order_number,cs_item_sk] #10 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_order_number,cr_item_sk] #11 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/explain.txt deleted file mode 100644 index 3d3ea5ab86..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/explain.txt +++ /dev/null @@ -1,405 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (67) -+- * Project (66) - +- * CometColumnarToRow (65) - +- CometSortMergeJoin (64) - :- CometProject (43) - : +- CometSortMergeJoin (42) - : :- CometSort (21) - : : +- CometHashAggregate (20) - : : +- CometExchange (19) - : : +- CometHashAggregate (18) - : : +- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : +- CometBroadcastExchange (15) - : : +- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : +- CometSort (41) - : +- CometFilter (40) - : +- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (33) - : : +- CometFilter (32) - : : +- CometSortMergeJoin (31) - : : :- CometSort (25) - : : : +- CometExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (22) - : : +- CometSort (30) - : : +- CometExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (26) - : +- ReusedExchange (34) - +- CometSort (63) - +- CometFilter (62) - +- CometHashAggregate (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometProject (55) - : +- CometFilter (54) - : +- CometSortMergeJoin (53) - : :- CometSort (47) - : : +- CometExchange (46) - : : +- CometFilter (45) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (44) - : +- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometFilter (49) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (48) - +- ReusedExchange (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_customer_sk#2)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_ticket_number#3, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_ticket_number#3 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) - -(7) CometProject -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] - -(8) CometExchange -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter - -(11) CometFilter -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Condition : isnull(sr_ticket_number#10) - -(12) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(15) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(16) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight - -(17) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] - -(18) CometHashAggregate -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] - -(19) CometExchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometHashAggregate -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] - -(21) CometSort -Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(23) CometFilter -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) - -(24) CometExchange -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometSort -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Condition : (isnotnull(wr_order_number#30) AND isnotnull(wr_item_sk#29)) - -(28) CometProject -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_item_sk#29, wr_order_number#30] - -(29) CometExchange -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: hashpartitioning(wr_order_number#30, wr_item_sk#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(30) CometSort -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_order_number#30 ASC NULLS FIRST, wr_item_sk#29 ASC NULLS FIRST] - -(31) CometSortMergeJoin -Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#30, wr_item_sk#29], LeftOuter - -(32) CometFilter -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Condition : isnull(wr_order_number#30) - -(33) CometProject -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(34) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#32, d_year#33] - -(35) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ws_sold_date_sk#27], [d_date_sk#32], Inner, BuildRight - -(36) CometProject -Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#32, d_year#33] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] - -(37) CometHashAggregate -Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] - -(38) CometExchange -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Arguments: hashpartitioning(d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(39) CometHashAggregate -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] - -(40) CometFilter -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Condition : (coalesce(ws_qty#39, 0) > 0) - -(41) CometSort -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41], [ws_sold_year#37 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#38 ASC NULLS FIRST] - -(42) CometSortMergeJoin -Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Right output [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38], Inner - -(43) CometProject -Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#48), dynamicpruningexpression(cs_sold_date_sk#48 IN dynamicpruning#49)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(45) CometFilter -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Condition : (isnotnull(cs_item_sk#43) AND isnotnull(cs_bill_customer_sk#42)) - -(46) CometExchange -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(47) CometSort -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST] - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Condition : (isnotnull(cr_order_number#51) AND isnotnull(cr_item_sk#50)) - -(50) CometProject -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_item_sk#50, cr_order_number#51] - -(51) CometExchange -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(52) CometSort -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST] - -(53) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cs_order_number#44, cs_item_sk#43], [cr_order_number#51, cr_item_sk#50], LeftOuter - -(54) CometFilter -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Condition : isnull(cr_order_number#51) - -(55) CometProject -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] - -(56) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#53, d_year#54] - -(57) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [cs_sold_date_sk#48], [d_date_sk#53], Inner, BuildRight - -(58) CometProject -Input [8]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, d_date_sk#53, d_year#54] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] - -(59) CometHashAggregate -Input [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [partial_sum(cs_quantity#45), partial_sum(UnscaledValue(cs_wholesale_cost#46)), partial_sum(UnscaledValue(cs_sales_price#47))] - -(60) CometExchange -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Arguments: hashpartitioning(d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(61) CometHashAggregate -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [sum(cs_quantity#45), sum(UnscaledValue(cs_wholesale_cost#46)), sum(UnscaledValue(cs_sales_price#47))] - -(62) CometFilter -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Condition : (coalesce(cs_qty#60, 0) > 0) - -(63) CometSort -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62], [cs_sold_year#58 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST, cs_customer_sk#59 ASC NULLS FIRST] - -(64) CometSortMergeJoin -Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] -Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner - -(65) CometColumnarToRow [codegen id : 1] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(66) Project [codegen id : 1] -Output [12]: [round((cast(ss_qty#18 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)))), 2) AS ratio#63, ss_qty#18 AS store_qty#64, ss_wc#19 AS store_wholesale_cost#65, ss_sp#20 AS store_sales_price#66, (coalesce(ws_qty#39, 0) + coalesce(cs_qty#60, 0)) AS other_chan_qty#67, (coalesce(ws_wc#40, 0.00) + coalesce(cs_wc#61, 0.00)) AS other_chan_wholesale_cost#68, (coalesce(ws_sp#41, 0.00) + coalesce(cs_sp#62, 0.00)) AS other_chan_sales_price#69, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, cs_qty#60] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(67) TakeOrderedAndProject -Input [12]: [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, cs_qty#60] -Arguments: 100, [ratio#63 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#67 ASC NULLS FIRST, other_chan_wholesale_cost#68 ASC NULLS FIRST, other_chan_sales_price#69 ASC NULLS FIRST, round((cast(ss_qty#18 as double) / cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)), 2) ASC NULLS FIRST], [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) - - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(70) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(71) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/extended.txt deleted file mode 100644 index 9d5ba53be1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/extended.txt +++ /dev/null @@ -1,80 +0,0 @@ -TakeOrderedAndProject -+- Project [COMET: Comet does not support Spark's BigDecimal rounding] - +- CometColumnarToRow - +- CometSortMergeJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 77c05217f4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt +++ /dev/null @@ -1,78 +0,0 @@ -TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (1) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - CometColumnarToRow - InputAdapter - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] - CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #5 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] - CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometExchange [ws_order_number,ws_item_sk] #7 - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_order_number,wr_item_sk] #8 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] - CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometExchange [cs_order_number,cs_item_sk] #10 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_order_number,cr_item_sk] #11 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/explain.txt deleted file mode 100644 index e51e7eb2af..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (32) -+- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (29) - +- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometNativeScan parquet spark_catalog.default.customer (25) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] - -(3) Filter [codegen id : 4] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 37] -Output [1]: [d_date_sk#10] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#11, s_number_employees#12, s_city#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_number_employees), GreaterThanOrEqual(s_number_employees,200), LessThanOrEqual(s_number_employees,295), IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [s_store_sk#11, s_number_employees#12, s_city#13] -Condition : (((isnotnull(s_number_employees#12) AND (s_number_employees#12 >= 200)) AND (s_number_employees#12 <= 295)) AND isnotnull(s_store_sk#11)) - -(9) CometProject -Input [3]: [s_store_sk#11, s_number_employees#12, s_city#13] -Arguments: [s_store_sk#11, s_city#13], [s_store_sk#11, s_city#13] - -(10) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#11, s_city#13] - -(11) BroadcastExchange -Input [2]: [s_store_sk#11, s_city#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#11, s_city#13] - -(14) CometNativeScan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,6),GreaterThan(hd_vehicle_count,2)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Condition : (((hd_dep_count#15 = 6) OR (hd_vehicle_count#16 > 2)) AND isnotnull(hd_demo_sk#14)) - -(16) CometProject -Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Arguments: [hd_demo_sk#14], [hd_demo_sk#14] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#14] - -(18) BroadcastExchange -Input [1]: [hd_demo_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#14] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13, hd_demo_sk#14] - -(21) HashAggregate [codegen id : 4] -Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13] -Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum#17, sum#18] -Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] - -(22) CometColumnarExchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 6] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] - -(24) HashAggregate [codegen id : 6] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13] -Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#6))#21, sum(UnscaledValue(ss_net_profit#7))#22] -Results [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#21,17,2) AS amt#23, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#22,17,2) AS profit#24] - -(25) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Condition : isnotnull(c_customer_sk#25) - -(27) CometProject -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_customer_sk#25, c_first_name#28, c_last_name#29], [c_customer_sk#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#26, 20)) AS c_first_name#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#27, 30)) AS c_last_name#29] - -(28) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#25, c_first_name#28, c_last_name#29] - -(29) BroadcastExchange -Input [3]: [c_customer_sk#25, c_first_name#28, c_last_name#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#25] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [7]: [c_last_name#29, c_first_name#28, substr(s_city#13, 1, 30) AS substr(s_city, 1, 30)#30, ss_ticket_number#5, amt#23, profit#24, s_city#13] -Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, amt#23, profit#24, c_customer_sk#25, c_first_name#28, c_last_name#29] - -(32) TakeOrderedAndProject -Input [7]: [c_last_name#29, c_first_name#28, substr(s_city, 1, 30)#30, ss_ticket_number#5, amt#23, profit#24, s_city#13] -Arguments: 100, [c_last_name#29 ASC NULLS FIRST, c_first_name#28 ASC NULLS FIRST, substr(s_city#13, 1, 30) ASC NULLS FIRST, profit#24 ASC NULLS FIRST], [c_last_name#29, c_first_name#28, substr(s_city, 1, 30)#30, ss_ticket_number#5, amt#23, profit#24] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometNativeScan parquet spark_catalog.default.date_dim (33) - - -(33) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#31, d_dow#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [3]: [d_date_sk#10, d_year#31, d_dow#32] -Condition : (((isnotnull(d_dow#32) AND (d_dow#32 = 1)) AND d_year#31 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(35) CometProject -Input [3]: [d_date_sk#10, d_year#31, d_dow#32] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(37) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/simplified.txt deleted file mode 100644 index 1ee7a286a4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, 30),ss_ticket_number,amt] - WholeStageCodegen (6) - Project [c_last_name,c_first_name,s_city,ss_ticket_number,amt,profit] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),amt,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk,s_city] - CometFilter [s_store_sk,s_number_employees,s_city] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan 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/q79.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/explain.txt deleted file mode 100644 index 16bf78be93..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,212 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (31) -+- CometTakeOrderedAndProject (30) - +- CometProject (29) - +- CometBroadcastHashJoin (28) - :- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(5) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#13, s_number_employees#14, s_city#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_number_employees), GreaterThanOrEqual(s_number_employees,200), LessThanOrEqual(s_number_employees,295), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [s_store_sk#13, s_number_employees#14, s_city#15] -Condition : (((isnotnull(s_number_employees#14) AND (s_number_employees#14 >= 200)) AND (s_number_employees#14 <= 295)) AND isnotnull(s_store_sk#13)) - -(11) CometProject -Input [3]: [s_store_sk#13, s_number_employees#14, s_city#15] -Arguments: [s_store_sk#13, s_city#15], [s_store_sk#13, s_city#15] - -(12) CometBroadcastExchange -Input [2]: [s_store_sk#13, s_city#15] -Arguments: [s_store_sk#13, s_city#15] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [2]: [s_store_sk#13, s_city#15] -Arguments: [ss_store_sk#4], [s_store_sk#13], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#13, s_city#15] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,6),GreaterThan(hd_vehicle_count,2)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Condition : (((hd_dep_count#17 = 6) OR (hd_vehicle_count#18 > 2)) AND isnotnull(hd_demo_sk#16)) - -(17) CometProject -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Arguments: [hd_demo_sk#16], [hd_demo_sk#16] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#16] -Arguments: [hd_demo_sk#16] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] -Right output [1]: [hd_demo_sk#16] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15, hd_demo_sk#16] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] - -(21) CometHashAggregate -Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15] -Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] - -(22) CometExchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#19, sum#20] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#19, sum#20] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15] -Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(25) CometFilter -Input [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] -Condition : isnotnull(c_customer_sk#21) - -(26) CometProject -Input [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] -Arguments: [c_customer_sk#21, c_first_name#24, c_last_name#25], [c_customer_sk#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#22, 20)) AS c_first_name#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#23, 30)) AS c_last_name#25] - -(27) CometBroadcastExchange -Input [3]: [c_customer_sk#21, c_first_name#24, c_last_name#25] -Arguments: [c_customer_sk#21, c_first_name#24, c_last_name#25] - -(28) CometBroadcastHashJoin -Left output [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#26, profit#27] -Right output [3]: [c_customer_sk#21, c_first_name#24, c_last_name#25] -Arguments: [ss_customer_sk#1], [c_customer_sk#21], Inner, BuildRight - -(29) CometProject -Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#26, profit#27, c_customer_sk#21, c_first_name#24, c_last_name#25] -Arguments: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15], [c_last_name#25, c_first_name#24, substr(s_city#15, 1, 30) AS substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15] - -(30) CometTakeOrderedAndProject -Input [7]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#25 ASC NULLS FIRST,c_first_name#24 ASC NULLS FIRST,substr(s_city#15, 1, 30) ASC NULLS FIRST,profit#27 ASC NULLS FIRST], output=[c_last_name#25,c_first_name#24,substr(s_city, 1, 30)#28,ss_ticket_number#5,amt#26,profit#27]), [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27], 100, 0, [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, substr(s_city#15, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27] - -(31) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(34) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(36) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/simplified.txt deleted file mode 100644 index 5b252a906b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_city] [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit] - CometProject [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit,s_city] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,s_city,amt,profit,c_customer_sk,c_first_name,c_last_name] - CometHashAggregate [ss_addr_sk,sum,sum] [ss_ticket_number,ss_customer_sk,s_city,amt,profit,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] - CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 - CometHashAggregate [ss_coupon_amt,ss_net_profit] [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] - CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - 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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [s_store_sk,s_city] #4 - CometProject [s_store_sk,s_city] - CometFilter [s_store_sk,s_number_employees,s_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #6 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] 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/q79/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/explain.txt deleted file mode 100644 index 16bf78be93..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/explain.txt +++ /dev/null @@ -1,212 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (31) -+- CometTakeOrderedAndProject (30) - +- CometProject (29) - +- CometBroadcastHashJoin (28) - :- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(5) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#13, s_number_employees#14, s_city#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_number_employees), GreaterThanOrEqual(s_number_employees,200), LessThanOrEqual(s_number_employees,295), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [s_store_sk#13, s_number_employees#14, s_city#15] -Condition : (((isnotnull(s_number_employees#14) AND (s_number_employees#14 >= 200)) AND (s_number_employees#14 <= 295)) AND isnotnull(s_store_sk#13)) - -(11) CometProject -Input [3]: [s_store_sk#13, s_number_employees#14, s_city#15] -Arguments: [s_store_sk#13, s_city#15], [s_store_sk#13, s_city#15] - -(12) CometBroadcastExchange -Input [2]: [s_store_sk#13, s_city#15] -Arguments: [s_store_sk#13, s_city#15] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [2]: [s_store_sk#13, s_city#15] -Arguments: [ss_store_sk#4], [s_store_sk#13], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#13, s_city#15] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,6),GreaterThan(hd_vehicle_count,2)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Condition : (((hd_dep_count#17 = 6) OR (hd_vehicle_count#18 > 2)) AND isnotnull(hd_demo_sk#16)) - -(17) CometProject -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Arguments: [hd_demo_sk#16], [hd_demo_sk#16] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#16] -Arguments: [hd_demo_sk#16] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] -Right output [1]: [hd_demo_sk#16] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15, hd_demo_sk#16] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] - -(21) CometHashAggregate -Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15] -Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] - -(22) CometExchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#19, sum#20] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#19, sum#20] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15] -Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(25) CometFilter -Input [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] -Condition : isnotnull(c_customer_sk#21) - -(26) CometProject -Input [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] -Arguments: [c_customer_sk#21, c_first_name#24, c_last_name#25], [c_customer_sk#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#22, 20)) AS c_first_name#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#23, 30)) AS c_last_name#25] - -(27) CometBroadcastExchange -Input [3]: [c_customer_sk#21, c_first_name#24, c_last_name#25] -Arguments: [c_customer_sk#21, c_first_name#24, c_last_name#25] - -(28) CometBroadcastHashJoin -Left output [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#26, profit#27] -Right output [3]: [c_customer_sk#21, c_first_name#24, c_last_name#25] -Arguments: [ss_customer_sk#1], [c_customer_sk#21], Inner, BuildRight - -(29) CometProject -Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#26, profit#27, c_customer_sk#21, c_first_name#24, c_last_name#25] -Arguments: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15], [c_last_name#25, c_first_name#24, substr(s_city#15, 1, 30) AS substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15] - -(30) CometTakeOrderedAndProject -Input [7]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#25 ASC NULLS FIRST,c_first_name#24 ASC NULLS FIRST,substr(s_city#15, 1, 30) ASC NULLS FIRST,profit#27 ASC NULLS FIRST], output=[c_last_name#25,c_first_name#24,substr(s_city, 1, 30)#28,ss_ticket_number#5,amt#26,profit#27]), [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27], 100, 0, [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, substr(s_city#15, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27] - -(31) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(34) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(36) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/extended.txt deleted file mode 100644 index 012403275a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/extended.txt +++ /dev/null @@ -1,39 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 5b252a906b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_city] [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit] - CometProject [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit,s_city] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,s_city,amt,profit,c_customer_sk,c_first_name,c_last_name] - CometHashAggregate [ss_addr_sk,sum,sum] [ss_ticket_number,ss_customer_sk,s_city,amt,profit,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] - CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 - CometHashAggregate [ss_coupon_amt,ss_net_profit] [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] - CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - 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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [s_store_sk,s_city] #4 - CometProject [s_store_sk,s_city] - CometFilter [s_store_sk,s_number_employees,s_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #6 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/explain.txt deleted file mode 100644 index 225159b9e0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/explain.txt +++ /dev/null @@ -1,284 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (44) -+- * HashAggregate (43) - +- * CometColumnarToRow (42) - +- CometColumnarExchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) - : +- CometProject (9) - : +- CometFilter (8) - : +- CometNativeScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (37) - +- * CometColumnarToRow (36) - +- CometHashAggregate (35) - +- CometExchange (34) - +- CometHashAggregate (33) - +- CometBroadcastHashJoin (32) - :- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.customer_address (14) - +- CometBroadcastExchange (31) - +- CometProject (30) - +- CometFilter (29) - +- CometHashAggregate (28) - +- CometExchange (27) - +- CometHashAggregate (26) - +- CometProject (25) - +- CometBroadcastHashJoin (24) - :- CometProject (19) - : +- CometFilter (18) - : +- CometNativeScan parquet spark_catalog.default.customer_address (17) - +- CometBroadcastExchange (23) - +- CometProject (22) - +- CometFilter (21) - +- CometNativeScan parquet spark_catalog.default.customer (20) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] - -(3) Filter [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(4) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#5] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [2]: [ss_store_sk#1, ss_net_profit#2] -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#6, s_store_name#7, s_zip#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [s_store_sk#6, s_store_name#7, s_zip#8] -Condition : (isnotnull(s_store_sk#6) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#8, 10)))) - -(9) CometProject -Input [3]: [s_store_sk#6, s_store_name#7, s_zip#8] -Arguments: [s_store_sk#6, s_store_name#7, s_zip#9], [s_store_sk#6, s_store_name#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#8, 10)) AS s_zip#9] - -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [s_store_sk#6, s_store_name#7, s_zip#9] - -(11) BroadcastExchange -Input [3]: [s_store_sk#6, s_store_name#7, s_zip#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#6] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_store_name#7, s_zip#9] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_store_name#7, s_zip#9] - -(14) CometNativeScan parquet spark_catalog.default.customer_address -Output [1]: [ca_zip#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -ReadSchema: struct - -(15) CometFilter -Input [1]: [ca_zip#10] -Condition : (substr(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#10, 10)), 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#10, 10)), 1, 5))) - -(16) CometProject -Input [1]: [ca_zip#10] -Arguments: [ca_zip#11], [substr(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#10, 10)), 1, 5) AS ca_zip#11] - -(17) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#12, ca_zip#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [ca_address_sk#12, ca_zip#13] -Condition : isnotnull(ca_address_sk#12) - -(19) CometProject -Input [2]: [ca_address_sk#12, ca_zip#13] -Arguments: [ca_address_sk#12, ca_zip#14], [ca_address_sk#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#13, 10)) AS ca_zip#14] - -(20) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_preferred_cust_flag), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] -Condition : ((isnotnull(c_preferred_cust_flag#16) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#16, 1)) = Y)) AND isnotnull(c_current_addr_sk#15)) - -(22) CometProject -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] -Arguments: [c_current_addr_sk#15], [c_current_addr_sk#15] - -(23) CometBroadcastExchange -Input [1]: [c_current_addr_sk#15] -Arguments: [c_current_addr_sk#15] - -(24) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#12, ca_zip#14] -Right output [1]: [c_current_addr_sk#15] -Arguments: [ca_address_sk#12], [c_current_addr_sk#15], Inner, BuildRight - -(25) CometProject -Input [3]: [ca_address_sk#12, ca_zip#14, c_current_addr_sk#15] -Arguments: [ca_zip#14], [ca_zip#14] - -(26) CometHashAggregate -Input [1]: [ca_zip#14] -Keys [1]: [ca_zip#14] -Functions [1]: [partial_count(1)] - -(27) CometExchange -Input [2]: [ca_zip#14, count#17] -Arguments: hashpartitioning(ca_zip#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [2]: [ca_zip#14, count#17] -Keys [1]: [ca_zip#14] -Functions [1]: [count(1)] - -(29) CometFilter -Input [2]: [ca_zip#18, cnt#19] -Condition : (cnt#19 > 10) - -(30) CometProject -Input [2]: [ca_zip#18, cnt#19] -Arguments: [ca_zip#18], [ca_zip#18] - -(31) CometBroadcastExchange -Input [1]: [ca_zip#18] -Arguments: [ca_zip#18] - -(32) CometBroadcastHashJoin -Left output [1]: [ca_zip#11] -Right output [1]: [ca_zip#18] -Arguments: [coalesce(ca_zip#11, ), isnull(ca_zip#11)], [coalesce(ca_zip#18, ), isnull(ca_zip#18)], LeftSemi, BuildRight - -(33) CometHashAggregate -Input [1]: [ca_zip#11] -Keys [1]: [ca_zip#11] -Functions: [] - -(34) CometExchange -Input [1]: [ca_zip#11] -Arguments: hashpartitioning(ca_zip#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(35) CometHashAggregate -Input [1]: [ca_zip#11] -Keys [1]: [ca_zip#11] -Functions: [] - -(36) CometColumnarToRow [codegen id : 3] -Input [1]: [ca_zip#11] - -(37) BroadcastExchange -Input [1]: [ca_zip#11] -Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [substr(s_zip#9, 1, 2)] -Right keys [1]: [substr(ca_zip#11, 1, 2)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 4] -Output [2]: [ss_net_profit#2, s_store_name#7] -Input [4]: [ss_net_profit#2, s_store_name#7, s_zip#9, ca_zip#11] - -(40) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#2, s_store_name#7] -Keys [1]: [s_store_name#7] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#20] -Results [2]: [s_store_name#7, sum#21] - -(41) CometColumnarExchange -Input [2]: [s_store_name#7, sum#21] -Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(42) CometColumnarToRow [codegen id : 5] -Input [2]: [s_store_name#7, sum#21] - -(43) HashAggregate [codegen id : 5] -Input [2]: [s_store_name#7, sum#21] -Keys [1]: [s_store_name#7] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22] -Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS sum(ss_net_profit)#23] - -(44) TakeOrderedAndProject -Input [2]: [s_store_name#7, sum(ss_net_profit)#23] -Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometNativeScan parquet spark_catalog.default.date_dim (45) - - -(45) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#24, d_qoy#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [3]: [d_date_sk#5, d_year#24, d_qoy#25] -Condition : ((((isnotnull(d_qoy#25) AND isnotnull(d_year#24)) AND (d_qoy#25 = 2)) AND (d_year#24 = 1998)) AND isnotnull(d_date_sk#5)) - -(47) CometProject -Input [3]: [d_date_sk#5, d_year#24, d_qoy#25] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(49) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/simplified.txt deleted file mode 100644 index 4df6d8f659..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/simplified.txt +++ /dev/null @@ -1,63 +0,0 @@ -TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - WholeStageCodegen (5) - HashAggregate [s_store_name,sum] [sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit),sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_name] #1 - WholeStageCodegen (4) - HashAggregate [s_store_name,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_store_name] - BroadcastHashJoin [s_zip,ca_zip] - Project [ss_net_profit,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [ca_zip] - CometExchange [ca_zip] #5 - CometHashAggregate [ca_zip] - CometBroadcastHashJoin [ca_zip,ca_zip] - CometProject [ca_zip] [ca_zip] - CometFilter [ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_zip] - CometBroadcastExchange [ca_zip] #6 - CometProject [ca_zip] - CometFilter [ca_zip,cnt] - CometHashAggregate [count] [ca_zip,cnt,ca_zip,count(1)] - CometExchange [ca_zip] #7 - CometHashAggregate [ca_zip,count] - CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [c_current_addr_sk] #8 - CometProject [c_current_addr_sk] - CometFilter [c_current_addr_sk,c_preferred_cust_flag] - CometNativeScan 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/q8.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/explain.txt deleted file mode 100644 index 1b9324fe93..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,284 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - +- CometBroadcastExchange (37) - +- CometHashAggregate (36) - +- CometExchange (35) - +- CometHashAggregate (34) - +- CometBroadcastHashJoin (33) - :- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (15) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (18) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) AND (d_year#6 = 1998)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#8, s_store_name#9, s_zip#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [s_store_sk#8, s_store_name#9, s_zip#10] -Condition : (isnotnull(s_store_sk#8) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#10, 10)))) - -(11) CometProject -Input [3]: [s_store_sk#8, s_store_name#9, s_zip#10] -Arguments: [s_store_sk#8, s_store_name#9, s_zip#11], [s_store_sk#8, s_store_name#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#10, 10)) AS s_zip#11] - -(12) CometBroadcastExchange -Input [3]: [s_store_sk#8, s_store_name#9, s_zip#11] -Arguments: [s_store_sk#8, s_store_name#9, s_zip#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_store_sk#1, ss_net_profit#2] -Right output [3]: [s_store_sk#8, s_store_name#9, s_zip#11] -Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_store_name#9, s_zip#11] -Arguments: [ss_net_profit#2, s_store_name#9, s_zip#11], [ss_net_profit#2, s_store_name#9, s_zip#11] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [1]: [ca_zip#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -ReadSchema: struct - -(16) CometFilter -Input [1]: [ca_zip#12] -Condition : (substr(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#12, 10)), 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#12, 10)), 1, 5))) - -(17) CometProject -Input [1]: [ca_zip#12] -Arguments: [ca_zip#13], [substr(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#12, 10)), 1, 5) AS ca_zip#13] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#14, ca_zip#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [ca_address_sk#14, ca_zip#15] -Condition : isnotnull(ca_address_sk#14) - -(20) CometProject -Input [2]: [ca_address_sk#14, ca_zip#15] -Arguments: [ca_address_sk#14, ca_zip#16], [ca_address_sk#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#15, 10)) AS ca_zip#16] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_preferred_cust_flag), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -Condition : ((isnotnull(c_preferred_cust_flag#18) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#18, 1)) = Y)) AND isnotnull(c_current_addr_sk#17)) - -(23) CometProject -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -Arguments: [c_current_addr_sk#17], [c_current_addr_sk#17] - -(24) CometBroadcastExchange -Input [1]: [c_current_addr_sk#17] -Arguments: [c_current_addr_sk#17] - -(25) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#14, ca_zip#16] -Right output [1]: [c_current_addr_sk#17] -Arguments: [ca_address_sk#14], [c_current_addr_sk#17], Inner, BuildRight - -(26) CometProject -Input [3]: [ca_address_sk#14, ca_zip#16, c_current_addr_sk#17] -Arguments: [ca_zip#16], [ca_zip#16] - -(27) CometHashAggregate -Input [1]: [ca_zip#16] -Keys [1]: [ca_zip#16] -Functions [1]: [partial_count(1)] - -(28) CometExchange -Input [2]: [ca_zip#16, count#19] -Arguments: hashpartitioning(ca_zip#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [2]: [ca_zip#16, count#19] -Keys [1]: [ca_zip#16] -Functions [1]: [count(1)] - -(30) CometFilter -Input [2]: [ca_zip#20, cnt#21] -Condition : (cnt#21 > 10) - -(31) CometProject -Input [2]: [ca_zip#20, cnt#21] -Arguments: [ca_zip#20], [ca_zip#20] - -(32) CometBroadcastExchange -Input [1]: [ca_zip#20] -Arguments: [ca_zip#20] - -(33) CometBroadcastHashJoin -Left output [1]: [ca_zip#13] -Right output [1]: [ca_zip#20] -Arguments: [coalesce(ca_zip#13, ), isnull(ca_zip#13)], [coalesce(ca_zip#20, ), isnull(ca_zip#20)], LeftSemi, BuildRight - -(34) CometHashAggregate -Input [1]: [ca_zip#13] -Keys [1]: [ca_zip#13] -Functions: [] - -(35) CometExchange -Input [1]: [ca_zip#13] -Arguments: hashpartitioning(ca_zip#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(36) CometHashAggregate -Input [1]: [ca_zip#13] -Keys [1]: [ca_zip#13] -Functions: [] - -(37) CometBroadcastExchange -Input [1]: [ca_zip#13] -Arguments: [ca_zip#13] - -(38) CometBroadcastHashJoin -Left output [3]: [ss_net_profit#2, s_store_name#9, s_zip#11] -Right output [1]: [ca_zip#13] -Arguments: [substr(s_zip#11, 1, 2)], [substr(ca_zip#13, 1, 2)], Inner, BuildRight - -(39) CometProject -Input [4]: [ss_net_profit#2, s_store_name#9, s_zip#11, ca_zip#13] -Arguments: [ss_net_profit#2, s_store_name#9], [ss_net_profit#2, s_store_name#9] - -(40) CometHashAggregate -Input [2]: [ss_net_profit#2, s_store_name#9] -Keys [1]: [s_store_name#9] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] - -(41) CometExchange -Input [2]: [s_store_name#9, sum#22] -Arguments: hashpartitioning(s_store_name#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(42) CometHashAggregate -Input [2]: [s_store_name#9, sum#22] -Keys [1]: [s_store_name#9] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] - -(43) CometTakeOrderedAndProject -Input [2]: [s_store_name#9, sum(ss_net_profit)#23] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#9 ASC NULLS FIRST], output=[s_store_name#9,sum(ss_net_profit)#23]), [s_store_name#9, sum(ss_net_profit)#23], 100, 0, [s_store_name#9 ASC NULLS FIRST], [s_store_name#9, sum(ss_net_profit)#23] - -(44) CometColumnarToRow [codegen id : 1] -Input [2]: [s_store_name#9, sum(ss_net_profit)#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) AND (d_year#6 = 1998)) AND isnotnull(d_date_sk#5)) - -(47) CometProject -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(49) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/simplified.txt deleted file mode 100644 index d1542ab607..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - CometHashAggregate [sum] [s_store_name,sum(ss_net_profit),sum(UnscaledValue(ss_net_profit))] - CometExchange [s_store_name] #1 - CometHashAggregate [ss_net_profit] [s_store_name,sum] - CometProject [ss_net_profit,s_store_name] - CometBroadcastHashJoin [ss_net_profit,s_store_name,s_zip,ca_zip] - CometProject [ss_net_profit,s_store_name,s_zip] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] - CometProject [ss_store_sk,ss_net_profit] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [ca_zip] #5 - CometHashAggregate [ca_zip] - CometExchange [ca_zip] #6 - CometHashAggregate [ca_zip] - CometBroadcastHashJoin [ca_zip,ca_zip] - CometProject [ca_zip] [ca_zip] - CometFilter [ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_zip] - CometBroadcastExchange [ca_zip] #7 - CometProject [ca_zip] - CometFilter [ca_zip,cnt] - CometHashAggregate [count] [ca_zip,cnt,ca_zip,count(1)] - CometExchange [ca_zip] #8 - CometHashAggregate [ca_zip,count] - CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [c_current_addr_sk] #9 - CometProject [c_current_addr_sk] - CometFilter [c_current_addr_sk,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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/q8/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/explain.txt deleted file mode 100644 index 1b9324fe93..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/explain.txt +++ /dev/null @@ -1,284 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - +- CometBroadcastExchange (37) - +- CometHashAggregate (36) - +- CometExchange (35) - +- CometHashAggregate (34) - +- CometBroadcastHashJoin (33) - :- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (15) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (18) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) AND (d_year#6 = 1998)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#8, s_store_name#9, s_zip#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [s_store_sk#8, s_store_name#9, s_zip#10] -Condition : (isnotnull(s_store_sk#8) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#10, 10)))) - -(11) CometProject -Input [3]: [s_store_sk#8, s_store_name#9, s_zip#10] -Arguments: [s_store_sk#8, s_store_name#9, s_zip#11], [s_store_sk#8, s_store_name#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#10, 10)) AS s_zip#11] - -(12) CometBroadcastExchange -Input [3]: [s_store_sk#8, s_store_name#9, s_zip#11] -Arguments: [s_store_sk#8, s_store_name#9, s_zip#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_store_sk#1, ss_net_profit#2] -Right output [3]: [s_store_sk#8, s_store_name#9, s_zip#11] -Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_store_name#9, s_zip#11] -Arguments: [ss_net_profit#2, s_store_name#9, s_zip#11], [ss_net_profit#2, s_store_name#9, s_zip#11] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [1]: [ca_zip#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -ReadSchema: struct - -(16) CometFilter -Input [1]: [ca_zip#12] -Condition : (substr(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#12, 10)), 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#12, 10)), 1, 5))) - -(17) CometProject -Input [1]: [ca_zip#12] -Arguments: [ca_zip#13], [substr(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#12, 10)), 1, 5) AS ca_zip#13] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#14, ca_zip#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [ca_address_sk#14, ca_zip#15] -Condition : isnotnull(ca_address_sk#14) - -(20) CometProject -Input [2]: [ca_address_sk#14, ca_zip#15] -Arguments: [ca_address_sk#14, ca_zip#16], [ca_address_sk#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#15, 10)) AS ca_zip#16] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_preferred_cust_flag), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -Condition : ((isnotnull(c_preferred_cust_flag#18) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#18, 1)) = Y)) AND isnotnull(c_current_addr_sk#17)) - -(23) CometProject -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -Arguments: [c_current_addr_sk#17], [c_current_addr_sk#17] - -(24) CometBroadcastExchange -Input [1]: [c_current_addr_sk#17] -Arguments: [c_current_addr_sk#17] - -(25) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#14, ca_zip#16] -Right output [1]: [c_current_addr_sk#17] -Arguments: [ca_address_sk#14], [c_current_addr_sk#17], Inner, BuildRight - -(26) CometProject -Input [3]: [ca_address_sk#14, ca_zip#16, c_current_addr_sk#17] -Arguments: [ca_zip#16], [ca_zip#16] - -(27) CometHashAggregate -Input [1]: [ca_zip#16] -Keys [1]: [ca_zip#16] -Functions [1]: [partial_count(1)] - -(28) CometExchange -Input [2]: [ca_zip#16, count#19] -Arguments: hashpartitioning(ca_zip#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [2]: [ca_zip#16, count#19] -Keys [1]: [ca_zip#16] -Functions [1]: [count(1)] - -(30) CometFilter -Input [2]: [ca_zip#20, cnt#21] -Condition : (cnt#21 > 10) - -(31) CometProject -Input [2]: [ca_zip#20, cnt#21] -Arguments: [ca_zip#20], [ca_zip#20] - -(32) CometBroadcastExchange -Input [1]: [ca_zip#20] -Arguments: [ca_zip#20] - -(33) CometBroadcastHashJoin -Left output [1]: [ca_zip#13] -Right output [1]: [ca_zip#20] -Arguments: [coalesce(ca_zip#13, ), isnull(ca_zip#13)], [coalesce(ca_zip#20, ), isnull(ca_zip#20)], LeftSemi, BuildRight - -(34) CometHashAggregate -Input [1]: [ca_zip#13] -Keys [1]: [ca_zip#13] -Functions: [] - -(35) CometExchange -Input [1]: [ca_zip#13] -Arguments: hashpartitioning(ca_zip#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(36) CometHashAggregate -Input [1]: [ca_zip#13] -Keys [1]: [ca_zip#13] -Functions: [] - -(37) CometBroadcastExchange -Input [1]: [ca_zip#13] -Arguments: [ca_zip#13] - -(38) CometBroadcastHashJoin -Left output [3]: [ss_net_profit#2, s_store_name#9, s_zip#11] -Right output [1]: [ca_zip#13] -Arguments: [substr(s_zip#11, 1, 2)], [substr(ca_zip#13, 1, 2)], Inner, BuildRight - -(39) CometProject -Input [4]: [ss_net_profit#2, s_store_name#9, s_zip#11, ca_zip#13] -Arguments: [ss_net_profit#2, s_store_name#9], [ss_net_profit#2, s_store_name#9] - -(40) CometHashAggregate -Input [2]: [ss_net_profit#2, s_store_name#9] -Keys [1]: [s_store_name#9] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] - -(41) CometExchange -Input [2]: [s_store_name#9, sum#22] -Arguments: hashpartitioning(s_store_name#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(42) CometHashAggregate -Input [2]: [s_store_name#9, sum#22] -Keys [1]: [s_store_name#9] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] - -(43) CometTakeOrderedAndProject -Input [2]: [s_store_name#9, sum(ss_net_profit)#23] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#9 ASC NULLS FIRST], output=[s_store_name#9,sum(ss_net_profit)#23]), [s_store_name#9, sum(ss_net_profit)#23], 100, 0, [s_store_name#9 ASC NULLS FIRST], [s_store_name#9, sum(ss_net_profit)#23] - -(44) CometColumnarToRow [codegen id : 1] -Input [2]: [s_store_name#9, sum(ss_net_profit)#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) AND (d_year#6 = 1998)) AND isnotnull(d_date_sk#5)) - -(47) CometProject -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(49) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/extended.txt deleted file mode 100644 index 8e04bbeba7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/extended.txt +++ /dev/null @@ -1,52 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d1542ab607..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - CometHashAggregate [sum] [s_store_name,sum(ss_net_profit),sum(UnscaledValue(ss_net_profit))] - CometExchange [s_store_name] #1 - CometHashAggregate [ss_net_profit] [s_store_name,sum] - CometProject [ss_net_profit,s_store_name] - CometBroadcastHashJoin [ss_net_profit,s_store_name,s_zip,ca_zip] - CometProject [ss_net_profit,s_store_name,s_zip] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] - CometProject [ss_store_sk,ss_net_profit] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [ca_zip] #5 - CometHashAggregate [ca_zip] - CometExchange [ca_zip] #6 - CometHashAggregate [ca_zip] - CometBroadcastHashJoin [ca_zip,ca_zip] - CometProject [ca_zip] [ca_zip] - CometFilter [ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_zip] - CometBroadcastExchange [ca_zip] #7 - CometProject [ca_zip] - CometFilter [ca_zip,cnt] - CometHashAggregate [count] [ca_zip,cnt,ca_zip,count(1)] - CometExchange [ca_zip] #8 - CometHashAggregate [ca_zip,count] - CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [c_current_addr_sk] #9 - CometProject [c_current_addr_sk] - CometFilter [c_current_addr_sk,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/explain.txt deleted file mode 100644 index 76d323f165..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/explain.txt +++ /dev/null @@ -1,623 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (106) -+- CometTakeOrderedAndProject (105) - +- CometHashAggregate (104) - +- CometExchange (103) - +- CometHashAggregate (102) - +- CometExpand (101) - +- CometUnion (100) - :- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometProject (18) - : : : : +- CometBroadcastHashJoin (17) - : : : : :- CometProject (12) - : : : : : +- CometSortMergeJoin (11) - : : : : : :- CometSort (5) - : : : : : : +- CometColumnarExchange (4) - : : : : : : +- * Filter (3) - : : : : : : +- * ColumnarToRow (2) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : +- CometSort (10) - : : : : : +- CometExchange (9) - : : : : : +- CometProject (8) - : : : : : +- CometFilter (7) - : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (6) - : : : : +- CometBroadcastExchange (16) - : : : : +- CometProject (15) - : : : : +- CometFilter (14) - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (13) - : : : +- CometBroadcastExchange (22) - : : : +- CometProject (21) - : : : +- CometFilter (20) - : : : +- CometNativeScan parquet spark_catalog.default.store (19) - : : +- CometBroadcastExchange (28) - : : +- CometProject (27) - : : +- CometFilter (26) - : : +- CometNativeScan parquet spark_catalog.default.item (25) - : +- CometBroadcastExchange (34) - : +- CometProject (33) - : +- CometFilter (32) - : +- CometNativeScan parquet spark_catalog.default.promotion (31) - :- CometHashAggregate (69) - : +- CometExchange (68) - : +- CometHashAggregate (67) - : +- CometProject (66) - : +- CometBroadcastHashJoin (65) - : :- CometProject (63) - : : +- CometBroadcastHashJoin (62) - : : :- CometProject (60) - : : : +- CometBroadcastHashJoin (59) - : : : :- CometProject (54) - : : : : +- CometBroadcastHashJoin (53) - : : : : :- CometProject (51) - : : : : : +- CometSortMergeJoin (50) - : : : : : :- CometSort (44) - : : : : : : +- CometColumnarExchange (43) - : : : : : : +- * Filter (42) - : : : : : : +- * ColumnarToRow (41) - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (40) - : : : : : +- CometSort (49) - : : : : : +- CometExchange (48) - : : : : : +- CometProject (47) - : : : : : +- CometFilter (46) - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (45) - : : : : +- ReusedExchange (52) - : : : +- CometBroadcastExchange (58) - : : : +- CometProject (57) - : : : +- CometFilter (56) - : : : +- CometNativeScan parquet spark_catalog.default.catalog_page (55) - : : +- ReusedExchange (61) - : +- ReusedExchange (64) - +- CometHashAggregate (99) - +- CometExchange (98) - +- CometHashAggregate (97) - +- CometProject (96) - +- CometBroadcastHashJoin (95) - :- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (84) - : : : +- CometBroadcastHashJoin (83) - : : : :- CometProject (81) - : : : : +- CometSortMergeJoin (80) - : : : : :- CometSort (74) - : : : : : +- CometColumnarExchange (73) - : : : : : +- * Filter (72) - : : : : : +- * ColumnarToRow (71) - : : : : : +- Scan parquet spark_catalog.default.web_sales (70) - : : : : +- CometSort (79) - : : : : +- CometExchange (78) - : : : : +- CometProject (77) - : : : : +- CometFilter (76) - : : : : +- CometNativeScan parquet spark_catalog.default.web_returns (75) - : : : +- ReusedExchange (82) - : : +- CometBroadcastExchange (88) - : : +- CometProject (87) - : : +- CometFilter (86) - : : +- CometNativeScan parquet spark_catalog.default.web_site (85) - : +- ReusedExchange (91) - +- ReusedExchange (94) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] - -(3) Filter [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(4) CometColumnarExchange -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(7) CometFilter -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) - -(8) CometProject -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(9) CometExchange -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter - -(12) CometProject -Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] - -(13) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) - -(15) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] - -(19) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_store_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [s_store_sk#16, s_store_id#17] -Condition : isnotnull(s_store_sk#16) - -(21) CometProject -Input [2]: [s_store_sk#16, s_store_id#17] -Arguments: [s_store_sk#16, s_store_id#18], [s_store_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#17, 16)) AS s_store_id#18] - -(22) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_store_id#18] -Arguments: [s_store_sk#16, s_store_id#18] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -Right output [2]: [s_store_sk#16, s_store_id#18] -Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight - -(24) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#18] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(25) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_current_price#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [i_item_sk#19, i_current_price#20] -Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) - -(27) CometProject -Input [2]: [i_item_sk#19, i_current_price#20] -Arguments: [i_item_sk#19], [i_item_sk#19] - -(28) CometBroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: [i_item_sk#19] - -(29) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [i_item_sk#19] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, i_item_sk#19] -Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(31) CometNativeScan parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#21, p_channel_tv#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_channel_tv), IsNotNull(p_promo_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Condition : ((isnotnull(p_channel_tv#22) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_tv#22, 1)) = N)) AND isnotnull(p_promo_sk#21)) - -(33) CometProject -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Arguments: [p_promo_sk#21], [p_promo_sk#21] - -(34) CometBroadcastExchange -Input [1]: [p_promo_sk#21] -Arguments: [p_promo_sk#21] - -(35) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [p_promo_sk#21] -Arguments: [ss_promo_sk#3], [p_promo_sk#21], Inner, BuildRight - -(36) CometProject -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, p_promo_sk#21] -Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(37) CometHashAggregate -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Keys [1]: [s_store_id#18] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(38) CometExchange -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(39) CometHashAggregate -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [s_store_id#18] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(40) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(41) ColumnarToRow [codegen id : 2] -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] - -(42) Filter [codegen id : 2] -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) - -(43) CometColumnarExchange -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(44) CometSort -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] - -(45) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38, cr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(46) CometFilter -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38, cr_returned_date_sk#39] -Condition : (isnotnull(cr_item_sk#35) AND isnotnull(cr_order_number#36)) - -(47) CometProject -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38, cr_returned_date_sk#39] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38], [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] - -(48) CometExchange -Input [4]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: hashpartitioning(cr_item_sk#35, cr_order_number#36, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(49) CometSort -Input [4]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38], [cr_item_sk#35 ASC NULLS FIRST, cr_order_number#36 ASC NULLS FIRST] - -(50) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#35, cr_order_number#36], LeftOuter - -(51) CometProject -Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38] - -(52) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#40] - -(53) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38] -Right output [1]: [d_date_sk#40] -Arguments: [cs_sold_date_sk#34], [d_date_sk#40], Inner, BuildRight - -(54) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38, d_date_sk#40] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38] - -(55) CometNativeScan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#42] -Condition : isnotnull(cp_catalog_page_sk#41) - -(57) CometProject -Input [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#42] -Arguments: [cp_catalog_page_sk#41, cp_catalog_page_id#43], [cp_catalog_page_sk#41, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#42, 16)) AS cp_catalog_page_id#43] - -(58) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#43] -Arguments: [cp_catalog_page_sk#41, cp_catalog_page_id#43] - -(59) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38] -Right output [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#43] -Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#41], Inner, BuildRight - -(60) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_sk#41, cp_catalog_page_id#43] -Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] - -(61) ReusedExchange [Reuses operator id: 28] -Output [1]: [i_item_sk#44] - -(62) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] -Right output [1]: [i_item_sk#44] -Arguments: [cs_item_sk#29], [i_item_sk#44], Inner, BuildRight - -(63) CometProject -Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43, i_item_sk#44] -Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] - -(64) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#45] - -(65) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] -Right output [1]: [p_promo_sk#45] -Arguments: [cs_promo_sk#30], [p_promo_sk#45], Inner, BuildRight - -(66) CometProject -Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43, p_promo_sk#45] -Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] - -(67) CometHashAggregate -Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] -Keys [1]: [cp_catalog_page_id#43] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#37 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#38 as decimal(12,2)), 0.00)))] - -(68) CometExchange -Input [6]: [cp_catalog_page_id#43, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] -Arguments: hashpartitioning(cp_catalog_page_id#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(69) CometHashAggregate -Input [6]: [cp_catalog_page_id#43, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] -Keys [1]: [cp_catalog_page_id#43] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#37 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#38 as decimal(12,2)), 0.00)))] - -(70) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#57), dynamicpruningexpression(ws_sold_date_sk#57 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] -ReadSchema: struct - -(71) ColumnarToRow [codegen id : 3] -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] - -(72) Filter [codegen id : 3] -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Condition : ((isnotnull(ws_web_site_sk#52) AND isnotnull(ws_item_sk#51)) AND isnotnull(ws_promo_sk#53)) - -(73) CometColumnarExchange -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Arguments: hashpartitioning(ws_item_sk#51, ws_order_number#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(74) CometSort -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Arguments: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57], [ws_item_sk#51 ASC NULLS FIRST, ws_order_number#54 ASC NULLS FIRST] - -(75) CometNativeScan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] -ReadSchema: struct - -(76) CometFilter -Input [5]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Condition : (isnotnull(wr_item_sk#58) AND isnotnull(wr_order_number#59)) - -(77) CometProject -Input [5]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Arguments: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61], [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] - -(78) CometExchange -Input [4]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: hashpartitioning(wr_item_sk#58, wr_order_number#59, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(79) CometSort -Input [4]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61], [wr_item_sk#58 ASC NULLS FIRST, wr_order_number#59 ASC NULLS FIRST] - -(80) CometSortMergeJoin -Left output [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Right output [4]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: [ws_item_sk#51, ws_order_number#54], [wr_item_sk#58, wr_order_number#59], LeftOuter - -(81) CometProject -Input [11]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61], [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61] - -(82) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#63] - -(83) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61] -Right output [1]: [d_date_sk#63] -Arguments: [ws_sold_date_sk#57], [d_date_sk#63], Inner, BuildRight - -(84) CometProject -Input [9]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61, d_date_sk#63] -Arguments: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61], [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61] - -(85) CometNativeScan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#64, web_site_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(86) CometFilter -Input [2]: [web_site_sk#64, web_site_id#65] -Condition : isnotnull(web_site_sk#64) - -(87) CometProject -Input [2]: [web_site_sk#64, web_site_id#65] -Arguments: [web_site_sk#64, web_site_id#66], [web_site_sk#64, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#65, 16)) AS web_site_id#66] - -(88) CometBroadcastExchange -Input [2]: [web_site_sk#64, web_site_id#66] -Arguments: [web_site_sk#64, web_site_id#66] - -(89) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61] -Right output [2]: [web_site_sk#64, web_site_id#66] -Arguments: [ws_web_site_sk#52], [web_site_sk#64], Inner, BuildRight - -(90) CometProject -Input [9]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_sk#64, web_site_id#66] -Arguments: [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66], [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] - -(91) ReusedExchange [Reuses operator id: 28] -Output [1]: [i_item_sk#67] - -(92) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] -Right output [1]: [i_item_sk#67] -Arguments: [ws_item_sk#51], [i_item_sk#67], Inner, BuildRight - -(93) CometProject -Input [8]: [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66, i_item_sk#67] -Arguments: [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66], [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] - -(94) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#68] - -(95) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] -Right output [1]: [p_promo_sk#68] -Arguments: [ws_promo_sk#53], [p_promo_sk#68], Inner, BuildRight - -(96) CometProject -Input [7]: [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66, p_promo_sk#68] -Arguments: [ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66], [ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] - -(97) CometHashAggregate -Input [5]: [ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] -Keys [1]: [web_site_id#66] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#55)), partial_sum(coalesce(cast(wr_return_amt#60 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#56 - coalesce(cast(wr_net_loss#61 as decimal(12,2)), 0.00)))] - -(98) CometExchange -Input [6]: [web_site_id#66, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Arguments: hashpartitioning(web_site_id#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(99) CometHashAggregate -Input [6]: [web_site_id#66, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Keys [1]: [web_site_id#66] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#55)), sum(coalesce(cast(wr_return_amt#60 as decimal(12,2)), 0.00)), sum((ws_net_profit#56 - coalesce(cast(wr_net_loss#61 as decimal(12,2)), 0.00)))] - -(100) CometUnion -Child 0 Input [5]: [sales#74, returns#75, profit#76, channel#77, id#78] -Child 1 Input [5]: [sales#79, returns#80, profit#81, channel#82, id#83] -Child 2 Input [5]: [sales#84, returns#85, profit#86, channel#87, id#88] - -(101) CometExpand -Input [5]: [sales#74, returns#75, profit#76, channel#77, id#78] -Arguments: [[sales#74, returns#75, profit#76, channel#77, id#78, 0], [sales#74, returns#75, profit#76, channel#77, null, 1], [sales#74, returns#75, profit#76, null, null, 3]], [sales#74, returns#75, profit#76, channel#89, id#90, spark_grouping_id#91] - -(102) CometHashAggregate -Input [6]: [sales#74, returns#75, profit#76, channel#89, id#90, spark_grouping_id#91] -Keys [3]: [channel#89, id#90, spark_grouping_id#91] -Functions [3]: [partial_sum(sales#74), partial_sum(returns#75), partial_sum(profit#76)] - -(103) CometExchange -Input [9]: [channel#89, id#90, spark_grouping_id#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97] -Arguments: hashpartitioning(channel#89, id#90, spark_grouping_id#91, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(104) CometHashAggregate -Input [9]: [channel#89, id#90, spark_grouping_id#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97] -Keys [3]: [channel#89, id#90, spark_grouping_id#91] -Functions [3]: [sum(sales#74), sum(returns#75), sum(profit#76)] - -(105) CometTakeOrderedAndProject -Input [5]: [channel#89, id#90, sales#98, returns#99, profit#100] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#89 ASC NULLS FIRST,id#90 ASC NULLS FIRST], output=[channel#89,id#90,sales#98,returns#99,profit#100]), [channel#89, id#90, sales#98, returns#99, profit#100], 100, 0, [channel#89 ASC NULLS FIRST, id#90 ASC NULLS FIRST], [channel#89, id#90, sales#98, returns#99, profit#100] - -(106) CometColumnarToRow [codegen id : 4] -Input [5]: [channel#89, id#90, sales#98, returns#99, profit#100] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (111) -+- * CometColumnarToRow (110) - +- CometProject (109) - +- CometFilter (108) - +- CometNativeScan parquet spark_catalog.default.date_dim (107) - - -(107) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] -ReadSchema: struct - -(108) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) - -(109) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(110) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(111) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#57 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/simplified.txt deleted file mode 100644 index 09e9224cef..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/simplified.txt +++ /dev/null @@ -1,124 +0,0 @@ -WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id,spark_grouping_id] #1 - CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] - CometUnion [sales,returns,profit,channel,id] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),s_store_id] - CometExchange [s_store_id] #2 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk,ss_ticket_number] #3 - WholeStageCodegen (1) - Filter [ss_store_sk,ss_item_sk,ss_promo_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #5 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #7 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #9 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,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))),cp_catalog_page_id] - CometExchange [cp_catalog_page_id] #10 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometColumnarExchange [cs_item_sk,cs_order_number] #11 - WholeStageCodegen (2) - Filter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),web_site_id] - CometExchange [web_site_id] #14 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometColumnarExchange [ws_item_sk,ws_order_number] #15 - WholeStageCodegen (3) - Filter [ws_web_site_sk,ws_item_sk,ws_promo_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #16 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [web_site_sk,web_site_id] #17 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/explain.txt deleted file mode 100644 index 223374746d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,611 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (103) -+- CometTakeOrderedAndProject (102) - +- CometHashAggregate (101) - +- CometExchange (100) - +- CometHashAggregate (99) - +- CometExpand (98) - +- CometUnion (97) - :- CometHashAggregate (38) - : +- CometExchange (37) - : +- CometHashAggregate (36) - : +- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometProject (29) - : : +- CometBroadcastHashJoin (28) - : : :- CometProject (23) - : : : +- CometBroadcastHashJoin (22) - : : : :- CometProject (17) - : : : : +- CometBroadcastHashJoin (16) - : : : : :- CometProject (11) - : : : : : +- CometSortMergeJoin (10) - : : : : : :- CometSort (4) - : : : : : : +- CometExchange (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : +- CometSort (9) - : : : : : +- CometExchange (8) - : : : : : +- CometProject (7) - : : : : : +- CometFilter (6) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : +- CometBroadcastExchange (15) - : : : : +- CometProject (14) - : : : : +- CometFilter (13) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) - : : : +- CometBroadcastExchange (21) - : : : +- CometProject (20) - : : : +- CometFilter (19) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) - : : +- CometBroadcastExchange (27) - : : +- CometProject (26) - : : +- CometFilter (25) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - : +- CometBroadcastExchange (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - :- CometHashAggregate (67) - : +- CometExchange (66) - : +- CometHashAggregate (65) - : +- CometProject (64) - : +- CometBroadcastHashJoin (63) - : :- CometProject (61) - : : +- CometBroadcastHashJoin (60) - : : :- CometProject (58) - : : : +- CometBroadcastHashJoin (57) - : : : :- CometProject (52) - : : : : +- CometBroadcastHashJoin (51) - : : : : :- CometProject (49) - : : : : : +- CometSortMergeJoin (48) - : : : : : :- CometSort (42) - : : : : : : +- CometExchange (41) - : : : : : : +- CometFilter (40) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) - : : : : : +- CometSort (47) - : : : : : +- CometExchange (46) - : : : : : +- CometProject (45) - : : : : : +- CometFilter (44) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) - : : : : +- ReusedExchange (50) - : : : +- CometBroadcastExchange (56) - : : : +- CometProject (55) - : : : +- CometFilter (54) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) - : : +- ReusedExchange (59) - : +- ReusedExchange (62) - +- CometHashAggregate (96) - +- CometExchange (95) - +- CometHashAggregate (94) - +- CometProject (93) - +- CometBroadcastHashJoin (92) - :- CometProject (90) - : +- CometBroadcastHashJoin (89) - : :- CometProject (87) - : : +- CometBroadcastHashJoin (86) - : : :- CometProject (81) - : : : +- CometBroadcastHashJoin (80) - : : : :- CometProject (78) - : : : : +- CometSortMergeJoin (77) - : : : : :- CometSort (71) - : : : : : +- CometExchange (70) - : : : : : +- CometFilter (69) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) - : : : : +- CometSort (76) - : : : : +- CometExchange (75) - : : : : +- CometProject (74) - : : : : +- CometFilter (73) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) - : : : +- ReusedExchange (79) - : : +- CometBroadcastExchange (85) - : : +- CometProject (84) - : : +- CometFilter (83) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) - : +- ReusedExchange (88) - +- ReusedExchange (91) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) - -(7) CometProject -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(8) CometExchange -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter - -(11) CometProject -Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) - -(14) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(15) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(16) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight - -(17) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_store_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [s_store_sk#16, s_store_id#17] -Condition : isnotnull(s_store_sk#16) - -(20) CometProject -Input [2]: [s_store_sk#16, s_store_id#17] -Arguments: [s_store_sk#16, s_store_id#18], [s_store_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#17, 16)) AS s_store_id#18] - -(21) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_store_id#18] -Arguments: [s_store_sk#16, s_store_id#18] - -(22) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -Right output [2]: [s_store_sk#16, s_store_id#18] -Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight - -(23) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#18] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_current_price#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [i_item_sk#19, i_current_price#20] -Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) - -(26) CometProject -Input [2]: [i_item_sk#19, i_current_price#20] -Arguments: [i_item_sk#19], [i_item_sk#19] - -(27) CometBroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: [i_item_sk#19] - -(28) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [i_item_sk#19] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(29) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, i_item_sk#19] -Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#21, p_channel_tv#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_channel_tv), IsNotNull(p_promo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Condition : ((isnotnull(p_channel_tv#22) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_tv#22, 1)) = N)) AND isnotnull(p_promo_sk#21)) - -(32) CometProject -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Arguments: [p_promo_sk#21], [p_promo_sk#21] - -(33) CometBroadcastExchange -Input [1]: [p_promo_sk#21] -Arguments: [p_promo_sk#21] - -(34) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [p_promo_sk#21] -Arguments: [ss_promo_sk#3], [p_promo_sk#21], Inner, BuildRight - -(35) CometProject -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, p_promo_sk#21] -Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(36) CometHashAggregate -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Keys [1]: [s_store_id#18] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(37) CometExchange -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [s_store_id#18] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(40) CometFilter -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) - -(41) CometExchange -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(42) CometSort -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(44) CometFilter -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) - -(45) CometProject -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] - -(46) CometExchange -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometSort -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] - -(48) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter - -(49) CometProject -Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] - -(50) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#41] - -(51) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -Right output [1]: [d_date_sk#41] -Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight - -(52) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(54) CometFilter -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Condition : isnotnull(cp_catalog_page_sk#42) - -(55) CometProject -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#43, 16)) AS cp_catalog_page_id#44] - -(56) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] - -(57) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight - -(58) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(59) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#45] - -(60) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [i_item_sk#45] -Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight - -(61) CometProject -Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] -Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(62) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#46] - -(63) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [p_promo_sk#46] -Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight - -(64) CometProject -Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] -Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(65) CometHashAggregate -Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(66) CometExchange -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(67) CometHashAggregate -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] -ReadSchema: struct - -(69) CometFilter -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) - -(70) CometExchange -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(71) CometSort -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] -ReadSchema: struct - -(73) CometFilter -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) - -(74) CometProject -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] - -(75) CometExchange -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(76) CometSort -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] - -(77) CometSortMergeJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter - -(78) CometProject -Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] - -(79) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#65] - -(80) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -Right output [1]: [d_date_sk#65] -Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(81) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] - -(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#66, web_site_id#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(83) CometFilter -Input [2]: [web_site_sk#66, web_site_id#67] -Condition : isnotnull(web_site_sk#66) - -(84) CometProject -Input [2]: [web_site_sk#66, web_site_id#67] -Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#67, 16)) AS web_site_id#68] - -(85) CometBroadcastExchange -Input [2]: [web_site_sk#66, web_site_id#68] -Arguments: [web_site_sk#66, web_site_id#68] - -(86) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -Right output [2]: [web_site_sk#66, web_site_id#68] -Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight - -(87) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] -Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(88) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#69] - -(89) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [i_item_sk#69] -Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight - -(90) CometProject -Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] -Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(91) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#70] - -(92) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [p_promo_sk#70] -Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight - -(93) CometProject -Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] -Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(94) CometHashAggregate -Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Keys [1]: [web_site_id#68] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(95) CometExchange -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(96) CometHashAggregate -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Keys [1]: [web_site_id#68] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(97) CometUnion -Child 0 Input [5]: [sales#76, returns#77, profit#78, channel#79, id#80] -Child 1 Input [5]: [sales#81, returns#82, profit#83, channel#84, id#85] -Child 2 Input [5]: [sales#86, returns#87, profit#88, channel#89, id#90] - -(98) CometExpand -Input [5]: [sales#76, returns#77, profit#78, channel#79, id#80] -Arguments: [[sales#76, returns#77, profit#78, channel#79, id#80, 0], [sales#76, returns#77, profit#78, channel#79, null, 1], [sales#76, returns#77, profit#78, null, null, 3]], [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] - -(99) CometHashAggregate -Input [6]: [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] -Keys [3]: [channel#91, id#92, spark_grouping_id#93] -Functions [3]: [partial_sum(sales#76), partial_sum(returns#77), partial_sum(profit#78)] - -(100) CometExchange -Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] -Arguments: hashpartitioning(channel#91, id#92, spark_grouping_id#93, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(101) CometHashAggregate -Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] -Keys [3]: [channel#91, id#92, spark_grouping_id#93] -Functions [3]: [sum(sales#76), sum(returns#77), sum(profit#78)] - -(102) CometTakeOrderedAndProject -Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#91 ASC NULLS FIRST,id#92 ASC NULLS FIRST], output=[channel#91,id#92,sales#100,returns#101,profit#102]), [channel#91, id#92, sales#100, returns#101, profit#102], 100, 0, [channel#91 ASC NULLS FIRST, id#92 ASC NULLS FIRST], [channel#91, id#92, sales#100, returns#101, profit#102] - -(103) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (108) -+- * CometColumnarToRow (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) - - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) - -(106) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(107) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(108) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/simplified.txt deleted file mode 100644 index 198646e0fd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,115 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id,spark_grouping_id] #1 - CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] - CometUnion [sales,returns,profit,channel,id] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),s_store_id] - CometExchange [s_store_id] #2 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #3 - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #5 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #7 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #9 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,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))),cp_catalog_page_id] - CometExchange [cp_catalog_page_id] #10 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #11 - 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 [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),web_site_id] - CometExchange [web_site_id] #14 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #15 - 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 [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #16 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [web_site_sk,web_site_id] #17 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/explain.txt deleted file mode 100644 index 223374746d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/explain.txt +++ /dev/null @@ -1,611 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (103) -+- CometTakeOrderedAndProject (102) - +- CometHashAggregate (101) - +- CometExchange (100) - +- CometHashAggregate (99) - +- CometExpand (98) - +- CometUnion (97) - :- CometHashAggregate (38) - : +- CometExchange (37) - : +- CometHashAggregate (36) - : +- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometProject (29) - : : +- CometBroadcastHashJoin (28) - : : :- CometProject (23) - : : : +- CometBroadcastHashJoin (22) - : : : :- CometProject (17) - : : : : +- CometBroadcastHashJoin (16) - : : : : :- CometProject (11) - : : : : : +- CometSortMergeJoin (10) - : : : : : :- CometSort (4) - : : : : : : +- CometExchange (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : +- CometSort (9) - : : : : : +- CometExchange (8) - : : : : : +- CometProject (7) - : : : : : +- CometFilter (6) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : +- CometBroadcastExchange (15) - : : : : +- CometProject (14) - : : : : +- CometFilter (13) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) - : : : +- CometBroadcastExchange (21) - : : : +- CometProject (20) - : : : +- CometFilter (19) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) - : : +- CometBroadcastExchange (27) - : : +- CometProject (26) - : : +- CometFilter (25) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - : +- CometBroadcastExchange (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - :- CometHashAggregate (67) - : +- CometExchange (66) - : +- CometHashAggregate (65) - : +- CometProject (64) - : +- CometBroadcastHashJoin (63) - : :- CometProject (61) - : : +- CometBroadcastHashJoin (60) - : : :- CometProject (58) - : : : +- CometBroadcastHashJoin (57) - : : : :- CometProject (52) - : : : : +- CometBroadcastHashJoin (51) - : : : : :- CometProject (49) - : : : : : +- CometSortMergeJoin (48) - : : : : : :- CometSort (42) - : : : : : : +- CometExchange (41) - : : : : : : +- CometFilter (40) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) - : : : : : +- CometSort (47) - : : : : : +- CometExchange (46) - : : : : : +- CometProject (45) - : : : : : +- CometFilter (44) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) - : : : : +- ReusedExchange (50) - : : : +- CometBroadcastExchange (56) - : : : +- CometProject (55) - : : : +- CometFilter (54) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) - : : +- ReusedExchange (59) - : +- ReusedExchange (62) - +- CometHashAggregate (96) - +- CometExchange (95) - +- CometHashAggregate (94) - +- CometProject (93) - +- CometBroadcastHashJoin (92) - :- CometProject (90) - : +- CometBroadcastHashJoin (89) - : :- CometProject (87) - : : +- CometBroadcastHashJoin (86) - : : :- CometProject (81) - : : : +- CometBroadcastHashJoin (80) - : : : :- CometProject (78) - : : : : +- CometSortMergeJoin (77) - : : : : :- CometSort (71) - : : : : : +- CometExchange (70) - : : : : : +- CometFilter (69) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) - : : : : +- CometSort (76) - : : : : +- CometExchange (75) - : : : : +- CometProject (74) - : : : : +- CometFilter (73) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) - : : : +- ReusedExchange (79) - : : +- CometBroadcastExchange (85) - : : +- CometProject (84) - : : +- CometFilter (83) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) - : +- ReusedExchange (88) - +- ReusedExchange (91) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) - -(7) CometProject -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(8) CometExchange -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter - -(11) CometProject -Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) - -(14) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(15) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(16) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight - -(17) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_store_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [s_store_sk#16, s_store_id#17] -Condition : isnotnull(s_store_sk#16) - -(20) CometProject -Input [2]: [s_store_sk#16, s_store_id#17] -Arguments: [s_store_sk#16, s_store_id#18], [s_store_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#17, 16)) AS s_store_id#18] - -(21) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_store_id#18] -Arguments: [s_store_sk#16, s_store_id#18] - -(22) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -Right output [2]: [s_store_sk#16, s_store_id#18] -Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight - -(23) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#18] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_current_price#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [i_item_sk#19, i_current_price#20] -Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) - -(26) CometProject -Input [2]: [i_item_sk#19, i_current_price#20] -Arguments: [i_item_sk#19], [i_item_sk#19] - -(27) CometBroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: [i_item_sk#19] - -(28) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [i_item_sk#19] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(29) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, i_item_sk#19] -Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#21, p_channel_tv#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_channel_tv), IsNotNull(p_promo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Condition : ((isnotnull(p_channel_tv#22) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_tv#22, 1)) = N)) AND isnotnull(p_promo_sk#21)) - -(32) CometProject -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Arguments: [p_promo_sk#21], [p_promo_sk#21] - -(33) CometBroadcastExchange -Input [1]: [p_promo_sk#21] -Arguments: [p_promo_sk#21] - -(34) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [p_promo_sk#21] -Arguments: [ss_promo_sk#3], [p_promo_sk#21], Inner, BuildRight - -(35) CometProject -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, p_promo_sk#21] -Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(36) CometHashAggregate -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Keys [1]: [s_store_id#18] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(37) CometExchange -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [s_store_id#18] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(40) CometFilter -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) - -(41) CometExchange -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(42) CometSort -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(44) CometFilter -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) - -(45) CometProject -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] - -(46) CometExchange -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometSort -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] - -(48) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter - -(49) CometProject -Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] - -(50) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#41] - -(51) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -Right output [1]: [d_date_sk#41] -Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight - -(52) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(54) CometFilter -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Condition : isnotnull(cp_catalog_page_sk#42) - -(55) CometProject -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#43, 16)) AS cp_catalog_page_id#44] - -(56) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] - -(57) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight - -(58) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(59) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#45] - -(60) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [i_item_sk#45] -Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight - -(61) CometProject -Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] -Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(62) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#46] - -(63) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [p_promo_sk#46] -Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight - -(64) CometProject -Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] -Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(65) CometHashAggregate -Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(66) CometExchange -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(67) CometHashAggregate -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] -ReadSchema: struct - -(69) CometFilter -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) - -(70) CometExchange -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(71) CometSort -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] -ReadSchema: struct - -(73) CometFilter -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) - -(74) CometProject -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] - -(75) CometExchange -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(76) CometSort -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] - -(77) CometSortMergeJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter - -(78) CometProject -Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] - -(79) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#65] - -(80) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -Right output [1]: [d_date_sk#65] -Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(81) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] - -(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#66, web_site_id#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(83) CometFilter -Input [2]: [web_site_sk#66, web_site_id#67] -Condition : isnotnull(web_site_sk#66) - -(84) CometProject -Input [2]: [web_site_sk#66, web_site_id#67] -Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#67, 16)) AS web_site_id#68] - -(85) CometBroadcastExchange -Input [2]: [web_site_sk#66, web_site_id#68] -Arguments: [web_site_sk#66, web_site_id#68] - -(86) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -Right output [2]: [web_site_sk#66, web_site_id#68] -Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight - -(87) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] -Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(88) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#69] - -(89) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [i_item_sk#69] -Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight - -(90) CometProject -Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] -Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(91) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#70] - -(92) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [p_promo_sk#70] -Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight - -(93) CometProject -Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] -Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(94) CometHashAggregate -Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Keys [1]: [web_site_id#68] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(95) CometExchange -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(96) CometHashAggregate -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Keys [1]: [web_site_id#68] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(97) CometUnion -Child 0 Input [5]: [sales#76, returns#77, profit#78, channel#79, id#80] -Child 1 Input [5]: [sales#81, returns#82, profit#83, channel#84, id#85] -Child 2 Input [5]: [sales#86, returns#87, profit#88, channel#89, id#90] - -(98) CometExpand -Input [5]: [sales#76, returns#77, profit#78, channel#79, id#80] -Arguments: [[sales#76, returns#77, profit#78, channel#79, id#80, 0], [sales#76, returns#77, profit#78, channel#79, null, 1], [sales#76, returns#77, profit#78, null, null, 3]], [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] - -(99) CometHashAggregate -Input [6]: [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] -Keys [3]: [channel#91, id#92, spark_grouping_id#93] -Functions [3]: [partial_sum(sales#76), partial_sum(returns#77), partial_sum(profit#78)] - -(100) CometExchange -Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] -Arguments: hashpartitioning(channel#91, id#92, spark_grouping_id#93, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(101) CometHashAggregate -Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] -Keys [3]: [channel#91, id#92, spark_grouping_id#93] -Functions [3]: [sum(sales#76), sum(returns#77), sum(profit#78)] - -(102) CometTakeOrderedAndProject -Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#91 ASC NULLS FIRST,id#92 ASC NULLS FIRST], output=[channel#91,id#92,sales#100,returns#101,profit#102]), [channel#91, id#92, sales#100, returns#101, profit#102], 100, 0, [channel#91 ASC NULLS FIRST, id#92 ASC NULLS FIRST], [channel#91, id#92, sales#100, returns#101, profit#102] - -(103) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (108) -+- * CometColumnarToRow (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) - - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) - -(106) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(107) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(108) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/extended.txt deleted file mode 100644 index ae6fc19b60..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/extended.txt +++ /dev/null @@ -1,131 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - -Comet accelerated 123 out of 127 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 198646e0fd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt +++ /dev/null @@ -1,115 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id,spark_grouping_id] #1 - CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] - CometUnion [sales,returns,profit,channel,id] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),s_store_id] - CometExchange [s_store_id] #2 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #3 - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #5 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #7 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #9 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,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))),cp_catalog_page_id] - CometExchange [cp_catalog_page_id] #10 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #11 - 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 [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),web_site_id] - CometExchange [web_site_id] #14 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #15 - 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 [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #16 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [web_site_sk,web_site_id] #17 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/explain.txt deleted file mode 100644 index bcc4d0f4a0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/explain.txt +++ /dev/null @@ -1,346 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (54) -+- * Project (53) - +- * BroadcastHashJoin Inner BuildRight (52) - :- * Project (46) - : +- * BroadcastHashJoin Inner BuildRight (45) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- * HashAggregate (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.catalog_returns (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (11) - : : : +- * CometColumnarToRow (10) - : : : +- CometProject (9) - : : : +- CometFilter (8) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) - : : +- BroadcastExchange (37) - : : +- * Filter (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * HashAggregate (31) - : : +- * CometColumnarToRow (30) - : : +- CometColumnarExchange (29) - : : +- * HashAggregate (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (24) - : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : :- * Filter (21) - : : : : +- * ColumnarToRow (20) - : : : : +- Scan parquet spark_catalog.default.catalog_returns (19) - : : : +- ReusedExchange (22) - : : +- ReusedExchange (25) - : +- BroadcastExchange (44) - : +- * CometColumnarToRow (43) - : +- CometProject (42) - : +- CometFilter (41) - : +- CometNativeScan parquet spark_catalog.default.customer (40) - +- BroadcastExchange (51) - +- * CometColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometNativeScan parquet spark_catalog.default.customer_address (47) - - -(1) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] - -(3) Filter [codegen id : 3] -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Condition : (isnotnull(cr_returning_addr_sk#2) AND isnotnull(cr_returning_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_state#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_state#8] -Condition : (isnotnull(ca_address_sk#7) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#8, 2)))) - -(9) CometProject -Input [2]: [ca_address_sk#7, ca_state#8] -Arguments: [ca_address_sk#7, ca_state#9], [ca_address_sk#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#8, 2)) AS ca_state#9] - -(10) CometColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#7, ca_state#9] - -(11) BroadcastExchange -Input [2]: [ca_address_sk#7, ca_state#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cr_returning_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#7, ca_state#9] - -(14) HashAggregate [codegen id : 3] -Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] -Keys [2]: [cr_returning_customer_sk#1, ca_state#9] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum#10] -Results [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] - -(15) CometColumnarExchange -Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 11] -Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] - -(17) HashAggregate [codegen id : 11] -Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] -Keys [2]: [cr_returning_customer_sk#1, ca_state#9] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#12] -Results [3]: [cr_returning_customer_sk#1 AS ctr_customer_sk#13, ca_state#9 AS ctr_state#14, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#12,17,2) AS ctr_total_return#15] - -(18) Filter [codegen id : 11] -Input [3]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15] -Condition : isnotnull(ctr_total_return#15) - -(19) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#19), dynamicpruningexpression(cr_returned_date_sk#19 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cr_returning_addr_sk)] -ReadSchema: struct - -(20) ColumnarToRow [codegen id : 6] -Input [4]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19] - -(21) Filter [codegen id : 6] -Input [4]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19] -Condition : isnotnull(cr_returning_addr_sk#17) - -(22) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#20] - -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returned_date_sk#19] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18] -Input [5]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19, d_date_sk#20] - -(25) ReusedExchange [Reuses operator id: 11] -Output [2]: [ca_address_sk#21, ca_state#22] - -(26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returning_addr_sk#17] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#16, cr_return_amt_inc_tax#18, ca_state#22] -Input [5]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, ca_address_sk#21, ca_state#22] - -(28) HashAggregate [codegen id : 6] -Input [3]: [cr_returning_customer_sk#16, cr_return_amt_inc_tax#18, ca_state#22] -Keys [2]: [cr_returning_customer_sk#16, ca_state#22] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#18))] -Aggregate Attributes [1]: [sum#23] -Results [3]: [cr_returning_customer_sk#16, ca_state#22, sum#24] - -(29) CometColumnarExchange -Input [3]: [cr_returning_customer_sk#16, ca_state#22, sum#24] -Arguments: hashpartitioning(cr_returning_customer_sk#16, ca_state#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(30) CometColumnarToRow [codegen id : 7] -Input [3]: [cr_returning_customer_sk#16, ca_state#22, sum#24] - -(31) HashAggregate [codegen id : 7] -Input [3]: [cr_returning_customer_sk#16, ca_state#22, sum#24] -Keys [2]: [cr_returning_customer_sk#16, ca_state#22] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#18))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#18))#12] -Results [2]: [ca_state#22 AS ctr_state#25, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#18))#12,17,2) AS ctr_total_return#26] - -(32) HashAggregate [codegen id : 7] -Input [2]: [ctr_state#25, ctr_total_return#26] -Keys [1]: [ctr_state#25] -Functions [1]: [partial_avg(ctr_total_return#26)] -Aggregate Attributes [2]: [sum#27, count#28] -Results [3]: [ctr_state#25, sum#29, count#30] - -(33) CometColumnarExchange -Input [3]: [ctr_state#25, sum#29, count#30] -Arguments: hashpartitioning(ctr_state#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 8] -Input [3]: [ctr_state#25, sum#29, count#30] - -(35) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#25, sum#29, count#30] -Keys [1]: [ctr_state#25] -Functions [1]: [avg(ctr_total_return#26)] -Aggregate Attributes [1]: [avg(ctr_total_return#26)#31] -Results [2]: [(avg(ctr_total_return#26)#31 * 1.2) AS (avg(ctr_total_return) * 1.2)#32, ctr_state#25] - -(36) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#32) - -(37) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] - -(38) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_state#14] -Right keys [1]: [ctr_state#25] -Join type: Inner -Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#32) - -(39) Project [codegen id : 11] -Output [2]: [ctr_customer_sk#13, ctr_total_return#15] -Input [5]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#32, ctr_state#25] - -(40) CometNativeScan parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(41) CometFilter -Input [6]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38] -Condition : (isnotnull(c_customer_sk#33) AND isnotnull(c_current_addr_sk#35)) - -(42) CometProject -Input [6]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38] -Arguments: [c_customer_sk#33, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42], [c_customer_sk#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#34, 16)) AS c_customer_id#39, c_current_addr_sk#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#36, 10)) AS c_salutation#40, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#37, 20)) AS c_first_name#41, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#38, 30)) AS c_last_name#42] - -(43) CometColumnarToRow [codegen id : 9] -Input [6]: [c_customer_sk#33, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42] - -(44) BroadcastExchange -Input [6]: [c_customer_sk#33, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(45) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_customer_sk#13] -Right keys [1]: [c_customer_sk#33] -Join type: Inner -Join condition: None - -(46) Project [codegen id : 11] -Output [6]: [ctr_total_return#15, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42] -Input [8]: [ctr_customer_sk#13, ctr_total_return#15, c_customer_sk#33, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42] - -(47) CometNativeScan parquet spark_catalog.default.customer_address -Output [12]: [ca_address_sk#43, ca_street_number#44, ca_street_name#45, ca_street_type#46, ca_suite_number#47, ca_city#48, ca_county#49, ca_state#50, ca_zip#51, ca_country#52, ca_gmt_offset#53, ca_location_type#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(48) CometFilter -Input [12]: [ca_address_sk#43, ca_street_number#44, ca_street_name#45, ca_street_type#46, ca_suite_number#47, ca_city#48, ca_county#49, ca_state#50, ca_zip#51, ca_country#52, ca_gmt_offset#53, ca_location_type#54] -Condition : ((isnotnull(ca_state#50) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#50, 2)) = GA)) AND isnotnull(ca_address_sk#43)) - -(49) CometProject -Input [12]: [ca_address_sk#43, ca_street_number#44, ca_street_name#45, ca_street_type#46, ca_suite_number#47, ca_city#48, ca_county#49, ca_state#50, ca_zip#51, ca_country#52, ca_gmt_offset#53, ca_location_type#54] -Arguments: [ca_address_sk#43, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60], [ca_address_sk#43, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_number#44, 10)) AS ca_street_number#55, ca_street_name#45, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_type#46, 15)) AS ca_street_type#56, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_suite_number#47, 10)) AS ca_suite_number#57, ca_city#48, ca_county#49, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#50, 2)) AS ca_state#58, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#51, 10)) AS ca_zip#59, ca_country#52, ca_gmt_offset#53, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_location_type#54, 20)) AS ca_location_type#60] - -(50) CometColumnarToRow [codegen id : 10] -Input [12]: [ca_address_sk#43, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60] - -(51) BroadcastExchange -Input [12]: [ca_address_sk#43, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(52) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#35] -Right keys [1]: [ca_address_sk#43] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 11] -Output [16]: [c_customer_id#39, c_salutation#40, c_first_name#41, c_last_name#42, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60, ctr_total_return#15] -Input [18]: [ctr_total_return#15, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42, ca_address_sk#43, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60] - -(54) TakeOrderedAndProject -Input [16]: [c_customer_id#39, c_salutation#40, c_first_name#41, c_last_name#42, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60, ctr_total_return#15] -Arguments: 100, [c_customer_id#39 ASC NULLS FIRST, c_salutation#40 ASC NULLS FIRST, c_first_name#41 ASC NULLS FIRST, c_last_name#42 ASC NULLS FIRST, ca_street_number#55 ASC NULLS FIRST, ca_street_name#45 ASC NULLS FIRST, ca_street_type#56 ASC NULLS FIRST, ca_suite_number#57 ASC NULLS FIRST, ca_city#48 ASC NULLS FIRST, ca_county#49 ASC NULLS FIRST, ca_state#58 ASC NULLS FIRST, ca_zip#59 ASC NULLS FIRST, ca_country#52 ASC NULLS FIRST, ca_gmt_offset#53 ASC NULLS FIRST, ca_location_type#60 ASC NULLS FIRST, ctr_total_return#15 ASC NULLS FIRST], [c_customer_id#39, c_salutation#40, c_first_name#41, c_last_name#42, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60, ctr_total_return#15] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (59) -+- * CometColumnarToRow (58) - +- CometProject (57) - +- CometFilter (56) - +- CometNativeScan parquet spark_catalog.default.date_dim (55) - - -(55) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#61] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date_sk#6, d_year#61] -Condition : ((isnotnull(d_year#61) AND (d_year#61 = 2000)) AND isnotnull(d_date_sk#6)) - -(57) CometProject -Input [2]: [d_date_sk#6, d_year#61] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(58) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(59) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 19 Hosting Expression = cr_returned_date_sk#19 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/simplified.txt deleted file mode 100644 index da396ce8a9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/simplified.txt +++ /dev/null @@ -1,86 +0,0 @@ -TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_total_return] - WholeStageCodegen (11) - Project [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_total_return] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (3) - HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] - Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Filter [cr_returning_addr_sk,cr_returning_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (8) - Filter [(avg(ctr_total_return) * 1.2)] - HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ctr_state] #5 - WholeStageCodegen (7) - HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cr_returning_customer_sk,ca_state] #6 - WholeStageCodegen (6) - HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] - Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Filter [cr_returning_addr_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [ca_address_sk,ca_state] #3 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometNativeScan 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 #8 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [ca_street_number,ca_street_type,ca_suite_number,ca_state,ca_zip,ca_location_type] [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] - 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] - CometNativeScan 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/q81.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/explain.txt deleted file mode 100644 index 492a321f97..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,318 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (50) -+- CometTakeOrderedAndProject (49) - +- CometProject (48) - +- CometBroadcastHashJoin (47) - :- CometProject (42) - : +- CometBroadcastHashJoin (41) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometFilter (18) - : : : +- CometHashAggregate (17) - : : : +- CometExchange (16) - : : : +- CometHashAggregate (15) - : : : +- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : : +- CometBroadcastExchange (34) - : : +- CometFilter (33) - : : +- CometHashAggregate (32) - : : +- CometExchange (31) - : : +- CometHashAggregate (30) - : : +- CometHashAggregate (29) - : : +- CometExchange (28) - : : +- CometHashAggregate (27) - : : +- CometProject (26) - : : +- CometBroadcastHashJoin (25) - : : :- CometProject (23) - : : : +- CometBroadcastHashJoin (22) - : : : :- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (19) - : : : +- ReusedExchange (21) - : : +- ReusedExchange (24) - : +- CometBroadcastExchange (40) - : +- CometProject (39) - : +- CometFilter (38) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) - +- CometBroadcastExchange (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (43) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Condition : (isnotnull(cr_returning_addr_sk#2) AND isnotnull(cr_returning_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [cr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] -Arguments: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3], [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#8, ca_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#8, ca_state#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#9, 2)))) - -(11) CometProject -Input [2]: [ca_address_sk#8, ca_state#9] -Arguments: [ca_address_sk#8, ca_state#10], [ca_address_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#9, 2)) AS ca_state#10] - -(12) CometBroadcastExchange -Input [2]: [ca_address_sk#8, ca_state#10] -Arguments: [ca_address_sk#8, ca_state#10] - -(13) CometBroadcastHashJoin -Left output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -Right output [2]: [ca_address_sk#8, ca_state#10] -Arguments: [cr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight - -(14) CometProject -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#8, ca_state#10] -Arguments: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10], [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] - -(15) CometHashAggregate -Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] -Keys [2]: [cr_returning_customer_sk#1, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] - -(16) CometExchange -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#11] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#11] -Keys [2]: [cr_returning_customer_sk#1, ca_state#10] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] - -(18) CometFilter -Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Condition : isnotnull(ctr_total_return#14) - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#18), dynamicpruningexpression(cr_returned_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cr_returning_addr_sk)] -ReadSchema: struct - -(20) CometFilter -Input [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] -Condition : isnotnull(cr_returning_addr_sk#16) - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#20] - -(22) CometBroadcastHashJoin -Left output [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] -Right output [1]: [d_date_sk#20] -Arguments: [cr_returned_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(23) CometProject -Input [5]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18, d_date_sk#20] -Arguments: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17], [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17] - -(24) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#21, ca_state#22] - -(25) CometBroadcastHashJoin -Left output [3]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17] -Right output [2]: [ca_address_sk#21, ca_state#22] -Arguments: [cr_returning_addr_sk#16], [ca_address_sk#21], Inner, BuildRight - -(26) CometProject -Input [5]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, ca_address_sk#21, ca_state#22] -Arguments: [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#22], [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#22] - -(27) CometHashAggregate -Input [3]: [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#22] -Keys [2]: [cr_returning_customer_sk#15, ca_state#22] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#17))] - -(28) CometExchange -Input [3]: [cr_returning_customer_sk#15, ca_state#22, sum#23] -Arguments: hashpartitioning(cr_returning_customer_sk#15, ca_state#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [3]: [cr_returning_customer_sk#15, ca_state#22, sum#23] -Keys [2]: [cr_returning_customer_sk#15, ca_state#22] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#17))] - -(30) CometHashAggregate -Input [2]: [ctr_state#24, ctr_total_return#25] -Keys [1]: [ctr_state#24] -Functions [1]: [partial_avg(ctr_total_return#25)] - -(31) CometExchange -Input [3]: [ctr_state#24, sum#26, count#27] -Arguments: hashpartitioning(ctr_state#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometHashAggregate -Input [3]: [ctr_state#24, sum#26, count#27] -Keys [1]: [ctr_state#24] -Functions [1]: [avg(ctr_total_return#25)] - -(33) CometFilter -Input [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#28) - -(34) CometBroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] -Arguments: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] - -(35) CometBroadcastHashJoin -Left output [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Right output [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] -Arguments: [ctr_state#13], [ctr_state#24], Inner, (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#28), BuildRight - -(36) CometProject -Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#28, ctr_state#24] -Arguments: [ctr_customer_sk#12, ctr_total_return#14], [ctr_customer_sk#12, ctr_total_return#14] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(38) CometFilter -Input [6]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34] -Condition : (isnotnull(c_customer_sk#29) AND isnotnull(c_current_addr_sk#31)) - -(39) CometProject -Input [6]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34] -Arguments: [c_customer_sk#29, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38], [c_customer_sk#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#30, 16)) AS c_customer_id#35, c_current_addr_sk#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#32, 10)) AS c_salutation#36, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#33, 20)) AS c_first_name#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#34, 30)) AS c_last_name#38] - -(40) CometBroadcastExchange -Input [6]: [c_customer_sk#29, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] -Arguments: [c_customer_sk#29, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] - -(41) CometBroadcastHashJoin -Left output [2]: [ctr_customer_sk#12, ctr_total_return#14] -Right output [6]: [c_customer_sk#29, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] -Arguments: [ctr_customer_sk#12], [c_customer_sk#29], Inner, BuildRight - -(42) CometProject -Input [8]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#29, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] -Arguments: [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38], [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [12]: [ca_address_sk#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(44) CometFilter -Input [12]: [ca_address_sk#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Condition : ((isnotnull(ca_state#46) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#46, 2)) = GA)) AND isnotnull(ca_address_sk#39)) - -(45) CometProject -Input [12]: [ca_address_sk#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Arguments: [ca_address_sk#39, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56], [ca_address_sk#39, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_number#40, 10)) AS ca_street_number#51, ca_street_name#41, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_type#42, 15)) AS ca_street_type#52, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_suite_number#43, 10)) AS ca_suite_number#53, ca_city#44, ca_county#45, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#46, 2)) AS ca_state#54, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#47, 10)) AS ca_zip#55, ca_country#48, ca_gmt_offset#49, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_location_type#50, 20)) AS ca_location_type#56] - -(46) CometBroadcastExchange -Input [12]: [ca_address_sk#39, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56] -Arguments: [ca_address_sk#39, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56] - -(47) CometBroadcastHashJoin -Left output [6]: [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] -Right output [12]: [ca_address_sk#39, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56] -Arguments: [c_current_addr_sk#31], [ca_address_sk#39], Inner, BuildRight - -(48) CometProject -Input [18]: [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38, ca_address_sk#39, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56] -Arguments: [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14], [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14] - -(49) CometTakeOrderedAndProject -Input [16]: [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#35 ASC NULLS FIRST,c_salutation#36 ASC NULLS FIRST,c_first_name#37 ASC NULLS FIRST,c_last_name#38 ASC NULLS FIRST,ca_street_number#51 ASC NULLS FIRST,ca_street_name#41 ASC NULLS FIRST,ca_street_type#52 ASC NULLS FIRST,ca_suite_number#53 ASC NULLS FIRST,ca_city#44 ASC NULLS FIRST,ca_county#45 ASC NULLS FIRST,ca_state#54 ASC NULLS FIRST,ca_zip#55 ASC NULLS FIRST,ca_country#48 ASC NULLS FIRST,ca_gmt_offset#49 ASC NULLS FIRST,ca_location_type#56 ASC NULLS FIRST,ctr_total_return#14 ASC NULLS FIRST], output=[c_customer_id#35,c_salutation#36,c_first_name#37,c_last_name#38,ca_street_number#51,ca_street_name#41,ca_street_type#52,ca_suite_number#53,ca_city#44,ca_county#45,ca_state#54,ca_zip#55,ca_country#48,ca_gmt_offset#49,ca_location_type#56,ctr_total_return#14]), [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14], 100, 0, [c_customer_id#35 ASC NULLS FIRST, c_salutation#36 ASC NULLS FIRST, c_first_name#37 ASC NULLS FIRST, c_last_name#38 ASC NULLS FIRST, ca_street_number#51 ASC NULLS FIRST, ca_street_name#41 ASC NULLS FIRST, ca_street_type#52 ASC NULLS FIRST, ca_suite_number#53 ASC NULLS FIRST, ca_city#44 ASC NULLS FIRST, ca_county#45 ASC NULLS FIRST, ca_state#54 ASC NULLS FIRST, ca_zip#55 ASC NULLS FIRST, ca_country#48 ASC NULLS FIRST, ca_gmt_offset#49 ASC NULLS FIRST, ca_location_type#56 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14] - -(50) CometColumnarToRow [codegen id : 1] -Input [16]: [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) - - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(52) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(53) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(54) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(55) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 19 Hosting Expression = cr_returned_date_sk#18 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/simplified.txt deleted file mode 100644 index 1091e272cc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_total_return] - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_total_return] - CometBroadcastHashJoin [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,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] - CometProject [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometBroadcastHashJoin [ctr_customer_sk,ctr_total_return,c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometProject [ctr_customer_sk,ctr_total_return] - CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] - CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] - CometHashAggregate [sum] [ctr_customer_sk,ctr_state,ctr_total_return,cr_returning_customer_sk,ca_state,sum(UnscaledValue(cr_return_amt_inc_tax))] - CometExchange [cr_returning_customer_sk,ca_state] #1 - CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] - CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - 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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 - CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] - CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_state,avg(ctr_total_return)] - CometExchange [ctr_state] #6 - CometHashAggregate [ctr_total_return] [ctr_state,sum,count] - CometHashAggregate [cr_returning_customer_sk,sum] [ctr_state,ctr_total_return,ca_state,sum(UnscaledValue(cr_return_amt_inc_tax))] - CometExchange [cr_returning_customer_sk,ca_state] #7 - CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] - CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - 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_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 [native_iceberg_compat] 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 - ReusedExchange [ca_address_sk,ca_state] #4 - CometBroadcastExchange [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] #8 - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometBroadcastExchange [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] #9 - CometProject [ca_street_number,ca_street_type,ca_suite_number,ca_state,ca_zip,ca_location_type] [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] - 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 [native_iceberg_compat] 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/q81/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/explain.txt deleted file mode 100644 index 492a321f97..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/explain.txt +++ /dev/null @@ -1,318 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (50) -+- CometTakeOrderedAndProject (49) - +- CometProject (48) - +- CometBroadcastHashJoin (47) - :- CometProject (42) - : +- CometBroadcastHashJoin (41) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometFilter (18) - : : : +- CometHashAggregate (17) - : : : +- CometExchange (16) - : : : +- CometHashAggregate (15) - : : : +- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : : +- CometBroadcastExchange (34) - : : +- CometFilter (33) - : : +- CometHashAggregate (32) - : : +- CometExchange (31) - : : +- CometHashAggregate (30) - : : +- CometHashAggregate (29) - : : +- CometExchange (28) - : : +- CometHashAggregate (27) - : : +- CometProject (26) - : : +- CometBroadcastHashJoin (25) - : : :- CometProject (23) - : : : +- CometBroadcastHashJoin (22) - : : : :- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (19) - : : : +- ReusedExchange (21) - : : +- ReusedExchange (24) - : +- CometBroadcastExchange (40) - : +- CometProject (39) - : +- CometFilter (38) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) - +- CometBroadcastExchange (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (43) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Condition : (isnotnull(cr_returning_addr_sk#2) AND isnotnull(cr_returning_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [cr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] -Arguments: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3], [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#8, ca_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#8, ca_state#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#9, 2)))) - -(11) CometProject -Input [2]: [ca_address_sk#8, ca_state#9] -Arguments: [ca_address_sk#8, ca_state#10], [ca_address_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#9, 2)) AS ca_state#10] - -(12) CometBroadcastExchange -Input [2]: [ca_address_sk#8, ca_state#10] -Arguments: [ca_address_sk#8, ca_state#10] - -(13) CometBroadcastHashJoin -Left output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -Right output [2]: [ca_address_sk#8, ca_state#10] -Arguments: [cr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight - -(14) CometProject -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#8, ca_state#10] -Arguments: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10], [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] - -(15) CometHashAggregate -Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] -Keys [2]: [cr_returning_customer_sk#1, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] - -(16) CometExchange -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#11] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#11] -Keys [2]: [cr_returning_customer_sk#1, ca_state#10] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] - -(18) CometFilter -Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Condition : isnotnull(ctr_total_return#14) - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#18), dynamicpruningexpression(cr_returned_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cr_returning_addr_sk)] -ReadSchema: struct - -(20) CometFilter -Input [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] -Condition : isnotnull(cr_returning_addr_sk#16) - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#20] - -(22) CometBroadcastHashJoin -Left output [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] -Right output [1]: [d_date_sk#20] -Arguments: [cr_returned_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(23) CometProject -Input [5]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18, d_date_sk#20] -Arguments: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17], [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17] - -(24) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#21, ca_state#22] - -(25) CometBroadcastHashJoin -Left output [3]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17] -Right output [2]: [ca_address_sk#21, ca_state#22] -Arguments: [cr_returning_addr_sk#16], [ca_address_sk#21], Inner, BuildRight - -(26) CometProject -Input [5]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, ca_address_sk#21, ca_state#22] -Arguments: [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#22], [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#22] - -(27) CometHashAggregate -Input [3]: [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#22] -Keys [2]: [cr_returning_customer_sk#15, ca_state#22] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#17))] - -(28) CometExchange -Input [3]: [cr_returning_customer_sk#15, ca_state#22, sum#23] -Arguments: hashpartitioning(cr_returning_customer_sk#15, ca_state#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [3]: [cr_returning_customer_sk#15, ca_state#22, sum#23] -Keys [2]: [cr_returning_customer_sk#15, ca_state#22] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#17))] - -(30) CometHashAggregate -Input [2]: [ctr_state#24, ctr_total_return#25] -Keys [1]: [ctr_state#24] -Functions [1]: [partial_avg(ctr_total_return#25)] - -(31) CometExchange -Input [3]: [ctr_state#24, sum#26, count#27] -Arguments: hashpartitioning(ctr_state#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometHashAggregate -Input [3]: [ctr_state#24, sum#26, count#27] -Keys [1]: [ctr_state#24] -Functions [1]: [avg(ctr_total_return#25)] - -(33) CometFilter -Input [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#28) - -(34) CometBroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] -Arguments: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] - -(35) CometBroadcastHashJoin -Left output [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Right output [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] -Arguments: [ctr_state#13], [ctr_state#24], Inner, (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#28), BuildRight - -(36) CometProject -Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#28, ctr_state#24] -Arguments: [ctr_customer_sk#12, ctr_total_return#14], [ctr_customer_sk#12, ctr_total_return#14] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(38) CometFilter -Input [6]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34] -Condition : (isnotnull(c_customer_sk#29) AND isnotnull(c_current_addr_sk#31)) - -(39) CometProject -Input [6]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34] -Arguments: [c_customer_sk#29, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38], [c_customer_sk#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#30, 16)) AS c_customer_id#35, c_current_addr_sk#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#32, 10)) AS c_salutation#36, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#33, 20)) AS c_first_name#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#34, 30)) AS c_last_name#38] - -(40) CometBroadcastExchange -Input [6]: [c_customer_sk#29, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] -Arguments: [c_customer_sk#29, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] - -(41) CometBroadcastHashJoin -Left output [2]: [ctr_customer_sk#12, ctr_total_return#14] -Right output [6]: [c_customer_sk#29, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] -Arguments: [ctr_customer_sk#12], [c_customer_sk#29], Inner, BuildRight - -(42) CometProject -Input [8]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#29, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] -Arguments: [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38], [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [12]: [ca_address_sk#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(44) CometFilter -Input [12]: [ca_address_sk#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Condition : ((isnotnull(ca_state#46) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#46, 2)) = GA)) AND isnotnull(ca_address_sk#39)) - -(45) CometProject -Input [12]: [ca_address_sk#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Arguments: [ca_address_sk#39, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56], [ca_address_sk#39, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_number#40, 10)) AS ca_street_number#51, ca_street_name#41, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_type#42, 15)) AS ca_street_type#52, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_suite_number#43, 10)) AS ca_suite_number#53, ca_city#44, ca_county#45, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#46, 2)) AS ca_state#54, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#47, 10)) AS ca_zip#55, ca_country#48, ca_gmt_offset#49, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_location_type#50, 20)) AS ca_location_type#56] - -(46) CometBroadcastExchange -Input [12]: [ca_address_sk#39, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56] -Arguments: [ca_address_sk#39, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56] - -(47) CometBroadcastHashJoin -Left output [6]: [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] -Right output [12]: [ca_address_sk#39, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56] -Arguments: [c_current_addr_sk#31], [ca_address_sk#39], Inner, BuildRight - -(48) CometProject -Input [18]: [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38, ca_address_sk#39, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56] -Arguments: [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14], [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14] - -(49) CometTakeOrderedAndProject -Input [16]: [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#35 ASC NULLS FIRST,c_salutation#36 ASC NULLS FIRST,c_first_name#37 ASC NULLS FIRST,c_last_name#38 ASC NULLS FIRST,ca_street_number#51 ASC NULLS FIRST,ca_street_name#41 ASC NULLS FIRST,ca_street_type#52 ASC NULLS FIRST,ca_suite_number#53 ASC NULLS FIRST,ca_city#44 ASC NULLS FIRST,ca_county#45 ASC NULLS FIRST,ca_state#54 ASC NULLS FIRST,ca_zip#55 ASC NULLS FIRST,ca_country#48 ASC NULLS FIRST,ca_gmt_offset#49 ASC NULLS FIRST,ca_location_type#56 ASC NULLS FIRST,ctr_total_return#14 ASC NULLS FIRST], output=[c_customer_id#35,c_salutation#36,c_first_name#37,c_last_name#38,ca_street_number#51,ca_street_name#41,ca_street_type#52,ca_suite_number#53,ca_city#44,ca_county#45,ca_state#54,ca_zip#55,ca_country#48,ca_gmt_offset#49,ca_location_type#56,ctr_total_return#14]), [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14], 100, 0, [c_customer_id#35 ASC NULLS FIRST, c_salutation#36 ASC NULLS FIRST, c_first_name#37 ASC NULLS FIRST, c_last_name#38 ASC NULLS FIRST, ca_street_number#51 ASC NULLS FIRST, ca_street_name#41 ASC NULLS FIRST, ca_street_type#52 ASC NULLS FIRST, ca_suite_number#53 ASC NULLS FIRST, ca_city#44 ASC NULLS FIRST, ca_county#45 ASC NULLS FIRST, ca_state#54 ASC NULLS FIRST, ca_zip#55 ASC NULLS FIRST, ca_country#48 ASC NULLS FIRST, ca_gmt_offset#49 ASC NULLS FIRST, ca_location_type#56 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14] - -(50) CometColumnarToRow [codegen id : 1] -Input [16]: [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) - - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(52) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(53) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(54) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(55) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 19 Hosting Expression = cr_returned_date_sk#18 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/extended.txt deleted file mode 100644 index 9708802e79..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 1091e272cc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_total_return] - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_total_return] - CometBroadcastHashJoin [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,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] - CometProject [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometBroadcastHashJoin [ctr_customer_sk,ctr_total_return,c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometProject [ctr_customer_sk,ctr_total_return] - CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] - CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] - CometHashAggregate [sum] [ctr_customer_sk,ctr_state,ctr_total_return,cr_returning_customer_sk,ca_state,sum(UnscaledValue(cr_return_amt_inc_tax))] - CometExchange [cr_returning_customer_sk,ca_state] #1 - CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] - CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - 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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 - CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] - CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_state,avg(ctr_total_return)] - CometExchange [ctr_state] #6 - CometHashAggregate [ctr_total_return] [ctr_state,sum,count] - CometHashAggregate [cr_returning_customer_sk,sum] [ctr_state,ctr_total_return,ca_state,sum(UnscaledValue(cr_return_amt_inc_tax))] - CometExchange [cr_returning_customer_sk,ca_state] #7 - CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] - CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - 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_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 [native_iceberg_compat] 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 - ReusedExchange [ca_address_sk,ca_state] #4 - CometBroadcastExchange [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] #8 - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometBroadcastExchange [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] #9 - CometProject [ca_street_number,ca_street_type,ca_suite_number,ca_state,ca_zip,ca_location_type] [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] - 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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/explain.txt deleted file mode 100644 index 6ff1da2cc8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/explain.txt +++ /dev/null @@ -1,181 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometTakeOrderedAndProject (25) - +- CometHashAggregate (24) - +- CometColumnarExchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildLeft (20) - :- BroadcastExchange (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * CometColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (9) - : : +- * Project (8) - : : +- * Filter (7) - : : +- * ColumnarToRow (6) - : : +- Scan parquet spark_catalog.default.inventory (5) - : +- ReusedExchange (12) - +- * CometColumnarToRow (19) - +- CometProject (18) - +- CometFilter (17) - +- CometNativeScan parquet spark_catalog.default.store_sales (16) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,62.00), LessThanOrEqual(i_current_price,92.00), In(i_manufact_id, [129,270,423,821]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 62.00)) AND (i_current_price#4 <= 92.00)) AND i_manufact_id#5 IN (129,270,821,423)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#2, 16)) AS i_item_id#6, i_item_desc#3, i_current_price#4] - -(4) CometColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(5) Scan parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#9), dynamicpruningexpression(inv_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] - -(7) Filter [codegen id : 1] -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Condition : (((isnotnull(inv_quantity_on_hand#8) AND (inv_quantity_on_hand#8 >= 100)) AND (inv_quantity_on_hand#8 <= 500)) AND isnotnull(inv_item_sk#7)) - -(8) Project [codegen id : 1] -Output [2]: [inv_item_sk#7, inv_date_sk#9] -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] - -(9) BroadcastExchange -Input [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [inv_item_sk#7] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 3] -Output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] - -(12) ReusedExchange [Reuses operator id: 31] -Output [1]: [d_date_sk#11] - -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [inv_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 3] -Output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] - -(15) BroadcastExchange -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) CometNativeScan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#12, ss_sold_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [ss_item_sk#12, ss_sold_date_sk#13] -Condition : isnotnull(ss_item_sk#12) - -(18) CometProject -Input [2]: [ss_item_sk#12, ss_sold_date_sk#13] -Arguments: [ss_item_sk#12], [ss_item_sk#12] - -(19) CometColumnarToRow -Input [1]: [ss_item_sk#12] - -(20) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#12] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 4] -Output [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, ss_item_sk#12] - -(22) HashAggregate [codegen id : 4] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -(23) CometColumnarExchange -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(25) CometTakeOrderedAndProject -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(26) CometColumnarToRow [codegen id : 5] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometNativeScan parquet spark_catalog.default.date_dim (27) - - -(27) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#11, d_date#14] -Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 2000-05-25)) AND (d_date#14 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) - -(29) CometProject -Input [2]: [d_date_sk#11, d_date#14] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(31) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/simplified.txt deleted file mode 100644 index b60bf92c5b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_current_price] - Project [i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (3) - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [inv_item_sk,inv_date_sk] - Filter [inv_quantity_on_hand,inv_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #4 - CometColumnarToRow - InputAdapter - CometProject [ss_item_sk] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometNativeScan 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/q82.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/explain.txt deleted file mode 100644 index d61d464b98..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometTakeOrderedAndProject (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (4) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - +- CometProject (19) - +- CometFilter (18) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (17) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,62.00), LessThanOrEqual(i_current_price,92.00), In(i_manufact_id, [129,270,423,821]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 62.00)) AND (i_current_price#4 <= 92.00)) AND i_manufact_id#5 IN (129,270,821,423)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#2, 16)) AS i_item_id#6, i_item_desc#3, i_current_price#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#9), dynamicpruningexpression(inv_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Condition : (((isnotnull(inv_quantity_on_hand#8) AND (inv_quantity_on_hand#8 >= 100)) AND (inv_quantity_on_hand#8 <= 500)) AND isnotnull(inv_item_sk#7)) - -(6) CometProject -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9], [inv_item_sk#7, inv_date_sk#9] - -(7) CometBroadcastExchange -Input [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1], [inv_item_sk#7], Inner, BuildRight - -(9) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#12 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) - -(12) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(14) CometBroadcastHashJoin -Left output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [inv_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(15) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(16) CometBroadcastExchange -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#13, ss_sold_date_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [ss_item_sk#13, ss_sold_date_sk#14] -Condition : isnotnull(ss_item_sk#13) - -(19) CometProject -Input [2]: [ss_item_sk#13, ss_sold_date_sk#14] -Arguments: [ss_item_sk#13], [ss_item_sk#13] - -(20) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [1]: [ss_item_sk#13] -Arguments: [i_item_sk#1], [ss_item_sk#13], Inner, BuildLeft - -(21) CometProject -Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, ss_item_sk#13] -Arguments: [i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(22) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(23) CometExchange -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(25) CometTakeOrderedAndProject -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(26) CometColumnarToRow [codegen id : 1] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#12 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) - -(29) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(31) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/simplified.txt deleted file mode 100644 index 28d160a4a8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometExchange [i_item_id,i_item_desc,i_current_price] #1 - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometProject [i_item_id,i_item_desc,i_current_price] - 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 [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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [ss_item_sk] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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/q82/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/explain.txt deleted file mode 100644 index d61d464b98..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometTakeOrderedAndProject (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (4) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - +- CometProject (19) - +- CometFilter (18) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (17) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,62.00), LessThanOrEqual(i_current_price,92.00), In(i_manufact_id, [129,270,423,821]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 62.00)) AND (i_current_price#4 <= 92.00)) AND i_manufact_id#5 IN (129,270,821,423)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#2, 16)) AS i_item_id#6, i_item_desc#3, i_current_price#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#9), dynamicpruningexpression(inv_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Condition : (((isnotnull(inv_quantity_on_hand#8) AND (inv_quantity_on_hand#8 >= 100)) AND (inv_quantity_on_hand#8 <= 500)) AND isnotnull(inv_item_sk#7)) - -(6) CometProject -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9], [inv_item_sk#7, inv_date_sk#9] - -(7) CometBroadcastExchange -Input [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1], [inv_item_sk#7], Inner, BuildRight - -(9) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#12 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) - -(12) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(14) CometBroadcastHashJoin -Left output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [inv_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(15) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(16) CometBroadcastExchange -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#13, ss_sold_date_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [ss_item_sk#13, ss_sold_date_sk#14] -Condition : isnotnull(ss_item_sk#13) - -(19) CometProject -Input [2]: [ss_item_sk#13, ss_sold_date_sk#14] -Arguments: [ss_item_sk#13], [ss_item_sk#13] - -(20) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [1]: [ss_item_sk#13] -Arguments: [i_item_sk#1], [ss_item_sk#13], Inner, BuildLeft - -(21) CometProject -Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, ss_item_sk#13] -Arguments: [i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(22) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(23) CometExchange -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(25) CometTakeOrderedAndProject -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(26) CometColumnarToRow [codegen id : 1] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#12 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) - -(29) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(31) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/extended.txt deleted file mode 100644 index 9fbb7893b8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/extended.txt +++ /dev/null @@ -1,34 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 28d160a4a8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometExchange [i_item_id,i_item_desc,i_current_price] #1 - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometProject [i_item_id,i_item_desc,i_current_price] - 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 [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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [ss_item_sk] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/explain.txt deleted file mode 100644 index 5b445f714a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/explain.txt +++ /dev/null @@ -1,379 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (50) -+- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * HashAggregate (17) - : : +- * CometColumnarToRow (16) - : : +- CometColumnarExchange (15) - : : +- * HashAggregate (14) - : : +- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_returns (1) - : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (31) - : +- * HashAggregate (30) - : +- * CometColumnarToRow (29) - : +- CometColumnarExchange (28) - : +- * HashAggregate (27) - : +- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * Filter (20) - : : : +- * ColumnarToRow (19) - : : : +- Scan parquet spark_catalog.default.catalog_returns (18) - : : +- ReusedExchange (21) - : +- ReusedExchange (24) - +- BroadcastExchange (47) - +- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Filter (36) - : : +- * ColumnarToRow (35) - : : +- Scan parquet spark_catalog.default.web_returns (34) - : +- ReusedExchange (37) - +- ReusedExchange (40) - - -(1) Scan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(sr_returned_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(sr_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Condition : isnotnull(sr_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_item_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [i_item_sk#5, i_item_id#6] -Condition : (isnotnull(i_item_sk#5) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)))) - -(6) CometProject -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#7] - -(7) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_sk#5, i_item_id#7] - -(8) BroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [sr_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7] -Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#5, i_item_id#7] - -(11) ReusedExchange [Reuses operator id: 64] -Output [1]: [d_date_sk#8] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [sr_returned_date_sk#3] -Right keys [1]: [d_date_sk#8] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [2]: [sr_return_quantity#2, i_item_id#7] -Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7, d_date_sk#8] - -(14) HashAggregate [codegen id : 3] -Input [2]: [sr_return_quantity#2, i_item_id#7] -Keys [1]: [i_item_id#7] -Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#9] -Results [2]: [i_item_id#7, sum#10] - -(15) CometColumnarExchange -Input [2]: [i_item_id#7, sum#10] -Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 12] -Input [2]: [i_item_id#7, sum#10] - -(17) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#7, sum#10] -Keys [1]: [i_item_id#7] -Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#11] -Results [2]: [i_item_id#7 AS item_id#12, sum(sr_return_quantity#2)#11 AS sr_item_qty#13] - -(18) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#16), dynamicpruningexpression(cr_returned_date_sk#16 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cr_item_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 6] -Input [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] - -(20) Filter [codegen id : 6] -Input [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] -Condition : isnotnull(cr_item_sk#14) - -(21) ReusedExchange [Reuses operator id: 8] -Output [2]: [i_item_sk#17, i_item_id#18] - -(22) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_item_sk#14] -Right keys [1]: [i_item_sk#17] -Join type: Inner -Join condition: None - -(23) Project [codegen id : 6] -Output [3]: [cr_return_quantity#15, cr_returned_date_sk#16, i_item_id#18] -Input [5]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16, i_item_sk#17, i_item_id#18] - -(24) ReusedExchange [Reuses operator id: 64] -Output [1]: [d_date_sk#19] - -(25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returned_date_sk#16] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 6] -Output [2]: [cr_return_quantity#15, i_item_id#18] -Input [4]: [cr_return_quantity#15, cr_returned_date_sk#16, i_item_id#18, d_date_sk#19] - -(27) HashAggregate [codegen id : 6] -Input [2]: [cr_return_quantity#15, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [1]: [partial_sum(cr_return_quantity#15)] -Aggregate Attributes [1]: [sum#20] -Results [2]: [i_item_id#18, sum#21] - -(28) CometColumnarExchange -Input [2]: [i_item_id#18, sum#21] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(29) CometColumnarToRow [codegen id : 7] -Input [2]: [i_item_id#18, sum#21] - -(30) HashAggregate [codegen id : 7] -Input [2]: [i_item_id#18, sum#21] -Keys [1]: [i_item_id#18] -Functions [1]: [sum(cr_return_quantity#15)] -Aggregate Attributes [1]: [sum(cr_return_quantity#15)#22] -Results [2]: [i_item_id#18 AS item_id#23, sum(cr_return_quantity#15)#22 AS cr_item_qty#24] - -(31) BroadcastExchange -Input [2]: [item_id#23, cr_item_qty#24] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [item_id#12] -Right keys [1]: [item_id#23] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 12] -Output [3]: [item_id#12, sr_item_qty#13, cr_item_qty#24] -Input [4]: [item_id#12, sr_item_qty#13, item_id#23, cr_item_qty#24] - -(34) Scan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#27), dynamicpruningexpression(wr_returned_date_sk#27 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(wr_item_sk)] -ReadSchema: struct - -(35) ColumnarToRow [codegen id : 10] -Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] - -(36) Filter [codegen id : 10] -Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] -Condition : isnotnull(wr_item_sk#25) - -(37) ReusedExchange [Reuses operator id: 8] -Output [2]: [i_item_sk#28, i_item_id#29] - -(38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [wr_item_sk#25] -Right keys [1]: [i_item_sk#28] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 10] -Output [3]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#29] -Input [5]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27, i_item_sk#28, i_item_id#29] - -(40) ReusedExchange [Reuses operator id: 64] -Output [1]: [d_date_sk#30] - -(41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [wr_returned_date_sk#27] -Right keys [1]: [d_date_sk#30] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 10] -Output [2]: [wr_return_quantity#26, i_item_id#29] -Input [4]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#29, d_date_sk#30] - -(43) HashAggregate [codegen id : 10] -Input [2]: [wr_return_quantity#26, i_item_id#29] -Keys [1]: [i_item_id#29] -Functions [1]: [partial_sum(wr_return_quantity#26)] -Aggregate Attributes [1]: [sum#31] -Results [2]: [i_item_id#29, sum#32] - -(44) CometColumnarExchange -Input [2]: [i_item_id#29, sum#32] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 11] -Input [2]: [i_item_id#29, sum#32] - -(46) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#29, sum#32] -Keys [1]: [i_item_id#29] -Functions [1]: [sum(wr_return_quantity#26)] -Aggregate Attributes [1]: [sum(wr_return_quantity#26)#33] -Results [2]: [i_item_id#29 AS item_id#34, sum(wr_return_quantity#26)#33 AS wr_item_qty#35] - -(47) BroadcastExchange -Input [2]: [item_id#34, wr_item_qty#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] - -(48) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [item_id#12] -Right keys [1]: [item_id#34] -Join type: Inner -Join condition: None - -(49) Project [codegen id : 12] -Output [8]: [item_id#12, sr_item_qty#13, (((cast(sr_item_qty#13 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as double)))) / 3.0) * 100.0) AS sr_dev#36, cr_item_qty#24, (((cast(cr_item_qty#24 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as double)))) / 3.0) * 100.0) AS cr_dev#37, wr_item_qty#35, (((cast(wr_item_qty#35 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as double)))) / 3.0) * 100.0) AS wr_dev#38, (cast(((sr_item_qty#13 + cr_item_qty#24) + wr_item_qty#35) as decimal(20,0)) / 3.0) AS average#39] -Input [5]: [item_id#12, sr_item_qty#13, cr_item_qty#24, item_id#34, wr_item_qty#35] - -(50) TakeOrderedAndProject -Input [8]: [item_id#12, sr_item_qty#13, sr_dev#36, cr_item_qty#24, cr_dev#37, wr_item_qty#35, wr_dev#38, average#39] -Arguments: 100, [item_id#12 ASC NULLS FIRST, sr_item_qty#13 ASC NULLS FIRST], [item_id#12, sr_item_qty#13, sr_dev#36, cr_item_qty#24, cr_dev#37, wr_item_qty#35, wr_dev#38, average#39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (64) -+- * CometColumnarToRow (63) - +- CometProject (62) - +- CometBroadcastHashJoin (61) - :- CometFilter (52) - : +- CometNativeScan parquet spark_catalog.default.date_dim (51) - +- CometBroadcastExchange (60) - +- CometProject (59) - +- CometBroadcastHashJoin (58) - :- CometNativeScan parquet spark_catalog.default.date_dim (53) - +- CometBroadcastExchange (57) - +- CometProject (56) - +- CometFilter (55) - +- CometNativeScan parquet spark_catalog.default.date_dim (54) - - -(51) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(52) CometFilter -Input [2]: [d_date_sk#8, d_date#40] -Condition : isnotnull(d_date_sk#8) - -(53) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date#41, d_week_seq#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(54) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_date, [2000-06-30,2000-09-27,2000-11-17])] -ReadSchema: struct - -(55) CometFilter -Input [2]: [d_date#43, d_week_seq#44] -Condition : d_date#43 IN (2000-06-30,2000-09-27,2000-11-17) - -(56) CometProject -Input [2]: [d_date#43, d_week_seq#44] -Arguments: [d_week_seq#44], [d_week_seq#44] - -(57) CometBroadcastExchange -Input [1]: [d_week_seq#44] -Arguments: [d_week_seq#44] - -(58) CometBroadcastHashJoin -Left output [2]: [d_date#41, d_week_seq#42] -Right output [1]: [d_week_seq#44] -Arguments: [d_week_seq#42], [d_week_seq#44], LeftSemi, BuildRight - -(59) CometProject -Input [2]: [d_date#41, d_week_seq#42] -Arguments: [d_date#41], [d_date#41] - -(60) CometBroadcastExchange -Input [1]: [d_date#41] -Arguments: [d_date#41] - -(61) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#40] -Right output [1]: [d_date#41] -Arguments: [d_date#40], [d_date#41], LeftSemi, BuildRight - -(62) CometProject -Input [2]: [d_date_sk#8, d_date#40] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(63) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(64) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 18 Hosting Expression = cr_returned_date_sk#16 IN dynamicpruning#4 - -Subquery:3 Hosting operator id = 34 Hosting Expression = wr_returned_date_sk#27 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/simplified.txt deleted file mode 100644 index 0673c590f5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/simplified.txt +++ /dev/null @@ -1,91 +0,0 @@ -TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - WholeStageCodegen (12) - Project [item_id,sr_item_qty,cr_item_qty,wr_item_qty] - BroadcastHashJoin [item_id,item_id] - Project [item_id,sr_item_qty,cr_item_qty] - BroadcastHashJoin [item_id,item_id] - HashAggregate [i_item_id,sum] [sum(sr_return_quantity),item_id,sr_item_qty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (3) - HashAggregate [i_item_id,sr_return_quantity] [sum,sum] - Project [sr_return_quantity,i_item_id] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [sr_return_quantity,sr_returned_date_sk,i_item_id] - BroadcastHashJoin [sr_item_sk,i_item_sk] - Filter [sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - HashAggregate [i_item_id,sum] [sum(cr_return_quantity),item_id,cr_item_qty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #7 - WholeStageCodegen (6) - HashAggregate [i_item_id,cr_return_quantity] [sum,sum] - Project [cr_return_quantity,i_item_id] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Project [cr_return_quantity,cr_returned_date_sk,i_item_id] - BroadcastHashJoin [cr_item_sk,i_item_sk] - Filter [cr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #9 - WholeStageCodegen (10) - HashAggregate [i_item_id,wr_return_quantity] [sum,sum] - Project [wr_return_quantity,i_item_id] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Project [wr_return_quantity,wr_returned_date_sk,i_item_id] - BroadcastHashJoin [wr_item_sk,i_item_sk] - Filter [wr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - InputAdapter - ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/explain.txt deleted file mode 100644 index 0078e83622..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,364 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (49) -+- CometTakeOrderedAndProject (48) - +- CometProject (47) - +- CometBroadcastHashJoin (46) - :- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometHashAggregate (19) - : : +- CometExchange (18) - : : +- CometHashAggregate (17) - : : +- CometProject (16) - : : +- CometBroadcastHashJoin (15) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (14) - : : +- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- ReusedExchange (11) - : +- CometBroadcastExchange (31) - : +- CometHashAggregate (30) - : +- CometExchange (29) - : +- CometHashAggregate (28) - : +- CometProject (27) - : +- CometBroadcastHashJoin (26) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometFilter (21) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (20) - : : +- ReusedExchange (22) - : +- ReusedExchange (25) - +- CometBroadcastExchange (45) - +- CometHashAggregate (44) - +- CometExchange (43) - +- CometHashAggregate (42) - +- CometProject (41) - +- CometBroadcastHashJoin (40) - :- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometFilter (35) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (34) - : +- ReusedExchange (36) - +- ReusedExchange (39) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(sr_returned_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(sr_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Condition : isnotnull(sr_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_item_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_item_id#6] -Condition : (isnotnull(i_item_sk#5) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)))) - -(5) CometProject -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#7] - -(6) CometBroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#7] -Arguments: [i_item_sk#5, i_item_id#7] - -(7) CometBroadcastHashJoin -Left output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Right output [2]: [i_item_sk#5, i_item_id#7] -Arguments: [sr_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#5, i_item_id#7] -Arguments: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7], [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(11) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date#10] - -(12) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#10] -Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight - -(13) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: [d_date_sk#8] - -(15) CometBroadcastHashJoin -Left output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7] -Right output [1]: [d_date_sk#8] -Arguments: [sr_returned_date_sk#3], [d_date_sk#8], Inner, BuildRight - -(16) CometProject -Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7, d_date_sk#8] -Arguments: [sr_return_quantity#2, i_item_id#7], [sr_return_quantity#2, i_item_id#7] - -(17) CometHashAggregate -Input [2]: [sr_return_quantity#2, i_item_id#7] -Keys [1]: [i_item_id#7] -Functions [1]: [partial_sum(sr_return_quantity#2)] - -(18) CometExchange -Input [2]: [i_item_id#7, sum#11] -Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [2]: [i_item_id#7, sum#11] -Keys [1]: [i_item_id#7] -Functions [1]: [sum(sr_return_quantity#2)] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#14), dynamicpruningexpression(cr_returned_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(cr_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] -Condition : isnotnull(cr_item_sk#12) - -(22) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#16, i_item_id#17] - -(23) CometBroadcastHashJoin -Left output [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] -Right output [2]: [i_item_sk#16, i_item_id#17] -Arguments: [cr_item_sk#12], [i_item_sk#16], Inner, BuildRight - -(24) CometProject -Input [5]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14, i_item_sk#16, i_item_id#17] -Arguments: [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17], [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17] - -(25) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#18] - -(26) CometBroadcastHashJoin -Left output [3]: [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17] -Right output [1]: [d_date_sk#18] -Arguments: [cr_returned_date_sk#14], [d_date_sk#18], Inner, BuildRight - -(27) CometProject -Input [4]: [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17, d_date_sk#18] -Arguments: [cr_return_quantity#13, i_item_id#17], [cr_return_quantity#13, i_item_id#17] - -(28) CometHashAggregate -Input [2]: [cr_return_quantity#13, i_item_id#17] -Keys [1]: [i_item_id#17] -Functions [1]: [partial_sum(cr_return_quantity#13)] - -(29) CometExchange -Input [2]: [i_item_id#17, sum#19] -Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(30) CometHashAggregate -Input [2]: [i_item_id#17, sum#19] -Keys [1]: [i_item_id#17] -Functions [1]: [sum(cr_return_quantity#13)] - -(31) CometBroadcastExchange -Input [2]: [item_id#20, cr_item_qty#21] -Arguments: [item_id#20, cr_item_qty#21] - -(32) CometBroadcastHashJoin -Left output [2]: [item_id#22, sr_item_qty#23] -Right output [2]: [item_id#20, cr_item_qty#21] -Arguments: [item_id#22], [item_id#20], Inner, BuildRight - -(33) CometProject -Input [4]: [item_id#22, sr_item_qty#23, item_id#20, cr_item_qty#21] -Arguments: [item_id#22, sr_item_qty#23, cr_item_qty#21], [item_id#22, sr_item_qty#23, cr_item_qty#21] - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#24, wr_return_quantity#25, wr_returned_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#26), dynamicpruningexpression(wr_returned_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(wr_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [wr_item_sk#24, wr_return_quantity#25, wr_returned_date_sk#26] -Condition : isnotnull(wr_item_sk#24) - -(36) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#28, i_item_id#29] - -(37) CometBroadcastHashJoin -Left output [3]: [wr_item_sk#24, wr_return_quantity#25, wr_returned_date_sk#26] -Right output [2]: [i_item_sk#28, i_item_id#29] -Arguments: [wr_item_sk#24], [i_item_sk#28], Inner, BuildRight - -(38) CometProject -Input [5]: [wr_item_sk#24, wr_return_quantity#25, wr_returned_date_sk#26, i_item_sk#28, i_item_id#29] -Arguments: [wr_return_quantity#25, wr_returned_date_sk#26, i_item_id#29], [wr_return_quantity#25, wr_returned_date_sk#26, i_item_id#29] - -(39) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#30] - -(40) CometBroadcastHashJoin -Left output [3]: [wr_return_quantity#25, wr_returned_date_sk#26, i_item_id#29] -Right output [1]: [d_date_sk#30] -Arguments: [wr_returned_date_sk#26], [d_date_sk#30], Inner, BuildRight - -(41) CometProject -Input [4]: [wr_return_quantity#25, wr_returned_date_sk#26, i_item_id#29, d_date_sk#30] -Arguments: [wr_return_quantity#25, i_item_id#29], [wr_return_quantity#25, i_item_id#29] - -(42) CometHashAggregate -Input [2]: [wr_return_quantity#25, i_item_id#29] -Keys [1]: [i_item_id#29] -Functions [1]: [partial_sum(wr_return_quantity#25)] - -(43) CometExchange -Input [2]: [i_item_id#29, sum#31] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(44) CometHashAggregate -Input [2]: [i_item_id#29, sum#31] -Keys [1]: [i_item_id#29] -Functions [1]: [sum(wr_return_quantity#25)] - -(45) CometBroadcastExchange -Input [2]: [item_id#32, wr_item_qty#33] -Arguments: [item_id#32, wr_item_qty#33] - -(46) CometBroadcastHashJoin -Left output [3]: [item_id#22, sr_item_qty#23, cr_item_qty#21] -Right output [2]: [item_id#32, wr_item_qty#33] -Arguments: [item_id#22], [item_id#32], Inner, BuildRight - -(47) CometProject -Input [5]: [item_id#22, sr_item_qty#23, cr_item_qty#21, item_id#32, wr_item_qty#33] -Arguments: [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37], [item_id#22, sr_item_qty#23, (((cast(sr_item_qty#23 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#23 + cr_item_qty#21) + wr_item_qty#33) as double)))) / 3.0) * 100.0) AS sr_dev#34, cr_item_qty#21, (((cast(cr_item_qty#21 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#23 + cr_item_qty#21) + wr_item_qty#33) as double)))) / 3.0) * 100.0) AS cr_dev#35, wr_item_qty#33, (((cast(wr_item_qty#33 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#23 + cr_item_qty#21) + wr_item_qty#33) as double)))) / 3.0) * 100.0) AS wr_dev#36, (cast(((sr_item_qty#23 + cr_item_qty#21) + wr_item_qty#33) as decimal(20,0)) / 3.0) AS average#37] - -(48) CometTakeOrderedAndProject -Input [8]: [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_id#22 ASC NULLS FIRST,sr_item_qty#23 ASC NULLS FIRST], output=[item_id#22,sr_item_qty#23,sr_dev#34,cr_item_qty#21,cr_dev#35,wr_item_qty#33,wr_dev#36,average#37]), [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37], 100, 0, [item_id#22 ASC NULLS FIRST, sr_item_qty#23 ASC NULLS FIRST], [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37] - -(49) CometColumnarToRow [codegen id : 1] -Input [8]: [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (63) -+- * CometColumnarToRow (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometFilter (51) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) - +- CometBroadcastExchange (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) - +- CometBroadcastExchange (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#10, d_week_seq#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#39, d_week_seq#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_date, [2000-06-30,2000-09-27,2000-11-17])] -ReadSchema: struct - -(54) CometFilter -Input [2]: [d_date#39, d_week_seq#40] -Condition : d_date#39 IN (2000-06-30,2000-09-27,2000-11-17) - -(55) CometProject -Input [2]: [d_date#39, d_week_seq#40] -Arguments: [d_week_seq#40], [d_week_seq#40] - -(56) CometBroadcastExchange -Input [1]: [d_week_seq#40] -Arguments: [d_week_seq#40] - -(57) CometBroadcastHashJoin -Left output [2]: [d_date#10, d_week_seq#38] -Right output [1]: [d_week_seq#40] -Arguments: [d_week_seq#38], [d_week_seq#40], LeftSemi, BuildRight - -(58) CometProject -Input [2]: [d_date#10, d_week_seq#38] -Arguments: [d_date#10], [d_date#10] - -(59) CometBroadcastExchange -Input [1]: [d_date#10] -Arguments: [d_date#10] - -(60) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#10] -Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight - -(61) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(62) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(63) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 20 Hosting Expression = cr_returned_date_sk#14 IN dynamicpruning#4 - -Subquery:3 Hosting operator id = 34 Hosting Expression = wr_returned_date_sk#26 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/simplified.txt deleted file mode 100644 index d016a5ee2d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,70 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - CometProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - CometBroadcastHashJoin [item_id,sr_item_qty,cr_item_qty,item_id,wr_item_qty] - CometProject [item_id,sr_item_qty,cr_item_qty] - CometBroadcastHashJoin [item_id,sr_item_qty,item_id,cr_item_qty] - CometHashAggregate [sum] [item_id,sr_item_qty,i_item_id,sum(sr_return_quantity)] - CometExchange [i_item_id] #1 - CometHashAggregate [sr_return_quantity] [i_item_id,sum] - CometProject [sr_return_quantity,i_item_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #3 - CometBroadcastExchange [item_id,cr_item_qty] #7 - CometHashAggregate [sum] [item_id,cr_item_qty,i_item_id,sum(cr_return_quantity)] - CometExchange [i_item_id] #8 - CometHashAggregate [cr_return_quantity] [i_item_id,sum] - CometProject [cr_return_quantity,i_item_id] - 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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [item_id,wr_item_qty] #9 - CometHashAggregate [sum] [item_id,wr_item_qty,i_item_id,sum(wr_return_quantity)] - CometExchange [i_item_id] #10 - CometHashAggregate [wr_return_quantity] [i_item_id,sum] - CometProject [wr_return_quantity,i_item_id] - 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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/explain.txt deleted file mode 100644 index 0078e83622..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/explain.txt +++ /dev/null @@ -1,364 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (49) -+- CometTakeOrderedAndProject (48) - +- CometProject (47) - +- CometBroadcastHashJoin (46) - :- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometHashAggregate (19) - : : +- CometExchange (18) - : : +- CometHashAggregate (17) - : : +- CometProject (16) - : : +- CometBroadcastHashJoin (15) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (14) - : : +- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- ReusedExchange (11) - : +- CometBroadcastExchange (31) - : +- CometHashAggregate (30) - : +- CometExchange (29) - : +- CometHashAggregate (28) - : +- CometProject (27) - : +- CometBroadcastHashJoin (26) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometFilter (21) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (20) - : : +- ReusedExchange (22) - : +- ReusedExchange (25) - +- CometBroadcastExchange (45) - +- CometHashAggregate (44) - +- CometExchange (43) - +- CometHashAggregate (42) - +- CometProject (41) - +- CometBroadcastHashJoin (40) - :- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometFilter (35) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (34) - : +- ReusedExchange (36) - +- ReusedExchange (39) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(sr_returned_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(sr_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Condition : isnotnull(sr_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_item_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_item_id#6] -Condition : (isnotnull(i_item_sk#5) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)))) - -(5) CometProject -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#7] - -(6) CometBroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#7] -Arguments: [i_item_sk#5, i_item_id#7] - -(7) CometBroadcastHashJoin -Left output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Right output [2]: [i_item_sk#5, i_item_id#7] -Arguments: [sr_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#5, i_item_id#7] -Arguments: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7], [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(11) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date#10] - -(12) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#10] -Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight - -(13) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: [d_date_sk#8] - -(15) CometBroadcastHashJoin -Left output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7] -Right output [1]: [d_date_sk#8] -Arguments: [sr_returned_date_sk#3], [d_date_sk#8], Inner, BuildRight - -(16) CometProject -Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7, d_date_sk#8] -Arguments: [sr_return_quantity#2, i_item_id#7], [sr_return_quantity#2, i_item_id#7] - -(17) CometHashAggregate -Input [2]: [sr_return_quantity#2, i_item_id#7] -Keys [1]: [i_item_id#7] -Functions [1]: [partial_sum(sr_return_quantity#2)] - -(18) CometExchange -Input [2]: [i_item_id#7, sum#11] -Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [2]: [i_item_id#7, sum#11] -Keys [1]: [i_item_id#7] -Functions [1]: [sum(sr_return_quantity#2)] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#14), dynamicpruningexpression(cr_returned_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(cr_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] -Condition : isnotnull(cr_item_sk#12) - -(22) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#16, i_item_id#17] - -(23) CometBroadcastHashJoin -Left output [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] -Right output [2]: [i_item_sk#16, i_item_id#17] -Arguments: [cr_item_sk#12], [i_item_sk#16], Inner, BuildRight - -(24) CometProject -Input [5]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14, i_item_sk#16, i_item_id#17] -Arguments: [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17], [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17] - -(25) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#18] - -(26) CometBroadcastHashJoin -Left output [3]: [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17] -Right output [1]: [d_date_sk#18] -Arguments: [cr_returned_date_sk#14], [d_date_sk#18], Inner, BuildRight - -(27) CometProject -Input [4]: [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17, d_date_sk#18] -Arguments: [cr_return_quantity#13, i_item_id#17], [cr_return_quantity#13, i_item_id#17] - -(28) CometHashAggregate -Input [2]: [cr_return_quantity#13, i_item_id#17] -Keys [1]: [i_item_id#17] -Functions [1]: [partial_sum(cr_return_quantity#13)] - -(29) CometExchange -Input [2]: [i_item_id#17, sum#19] -Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(30) CometHashAggregate -Input [2]: [i_item_id#17, sum#19] -Keys [1]: [i_item_id#17] -Functions [1]: [sum(cr_return_quantity#13)] - -(31) CometBroadcastExchange -Input [2]: [item_id#20, cr_item_qty#21] -Arguments: [item_id#20, cr_item_qty#21] - -(32) CometBroadcastHashJoin -Left output [2]: [item_id#22, sr_item_qty#23] -Right output [2]: [item_id#20, cr_item_qty#21] -Arguments: [item_id#22], [item_id#20], Inner, BuildRight - -(33) CometProject -Input [4]: [item_id#22, sr_item_qty#23, item_id#20, cr_item_qty#21] -Arguments: [item_id#22, sr_item_qty#23, cr_item_qty#21], [item_id#22, sr_item_qty#23, cr_item_qty#21] - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#24, wr_return_quantity#25, wr_returned_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#26), dynamicpruningexpression(wr_returned_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(wr_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [wr_item_sk#24, wr_return_quantity#25, wr_returned_date_sk#26] -Condition : isnotnull(wr_item_sk#24) - -(36) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#28, i_item_id#29] - -(37) CometBroadcastHashJoin -Left output [3]: [wr_item_sk#24, wr_return_quantity#25, wr_returned_date_sk#26] -Right output [2]: [i_item_sk#28, i_item_id#29] -Arguments: [wr_item_sk#24], [i_item_sk#28], Inner, BuildRight - -(38) CometProject -Input [5]: [wr_item_sk#24, wr_return_quantity#25, wr_returned_date_sk#26, i_item_sk#28, i_item_id#29] -Arguments: [wr_return_quantity#25, wr_returned_date_sk#26, i_item_id#29], [wr_return_quantity#25, wr_returned_date_sk#26, i_item_id#29] - -(39) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#30] - -(40) CometBroadcastHashJoin -Left output [3]: [wr_return_quantity#25, wr_returned_date_sk#26, i_item_id#29] -Right output [1]: [d_date_sk#30] -Arguments: [wr_returned_date_sk#26], [d_date_sk#30], Inner, BuildRight - -(41) CometProject -Input [4]: [wr_return_quantity#25, wr_returned_date_sk#26, i_item_id#29, d_date_sk#30] -Arguments: [wr_return_quantity#25, i_item_id#29], [wr_return_quantity#25, i_item_id#29] - -(42) CometHashAggregate -Input [2]: [wr_return_quantity#25, i_item_id#29] -Keys [1]: [i_item_id#29] -Functions [1]: [partial_sum(wr_return_quantity#25)] - -(43) CometExchange -Input [2]: [i_item_id#29, sum#31] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(44) CometHashAggregate -Input [2]: [i_item_id#29, sum#31] -Keys [1]: [i_item_id#29] -Functions [1]: [sum(wr_return_quantity#25)] - -(45) CometBroadcastExchange -Input [2]: [item_id#32, wr_item_qty#33] -Arguments: [item_id#32, wr_item_qty#33] - -(46) CometBroadcastHashJoin -Left output [3]: [item_id#22, sr_item_qty#23, cr_item_qty#21] -Right output [2]: [item_id#32, wr_item_qty#33] -Arguments: [item_id#22], [item_id#32], Inner, BuildRight - -(47) CometProject -Input [5]: [item_id#22, sr_item_qty#23, cr_item_qty#21, item_id#32, wr_item_qty#33] -Arguments: [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37], [item_id#22, sr_item_qty#23, (((cast(sr_item_qty#23 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#23 + cr_item_qty#21) + wr_item_qty#33) as double)))) / 3.0) * 100.0) AS sr_dev#34, cr_item_qty#21, (((cast(cr_item_qty#21 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#23 + cr_item_qty#21) + wr_item_qty#33) as double)))) / 3.0) * 100.0) AS cr_dev#35, wr_item_qty#33, (((cast(wr_item_qty#33 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#23 + cr_item_qty#21) + wr_item_qty#33) as double)))) / 3.0) * 100.0) AS wr_dev#36, (cast(((sr_item_qty#23 + cr_item_qty#21) + wr_item_qty#33) as decimal(20,0)) / 3.0) AS average#37] - -(48) CometTakeOrderedAndProject -Input [8]: [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_id#22 ASC NULLS FIRST,sr_item_qty#23 ASC NULLS FIRST], output=[item_id#22,sr_item_qty#23,sr_dev#34,cr_item_qty#21,cr_dev#35,wr_item_qty#33,wr_dev#36,average#37]), [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37], 100, 0, [item_id#22 ASC NULLS FIRST, sr_item_qty#23 ASC NULLS FIRST], [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37] - -(49) CometColumnarToRow [codegen id : 1] -Input [8]: [item_id#22, sr_item_qty#23, sr_dev#34, cr_item_qty#21, cr_dev#35, wr_item_qty#33, wr_dev#36, average#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (63) -+- * CometColumnarToRow (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometFilter (51) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) - +- CometBroadcastExchange (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) - +- CometBroadcastExchange (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#10, d_week_seq#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#39, d_week_seq#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_date, [2000-06-30,2000-09-27,2000-11-17])] -ReadSchema: struct - -(54) CometFilter -Input [2]: [d_date#39, d_week_seq#40] -Condition : d_date#39 IN (2000-06-30,2000-09-27,2000-11-17) - -(55) CometProject -Input [2]: [d_date#39, d_week_seq#40] -Arguments: [d_week_seq#40], [d_week_seq#40] - -(56) CometBroadcastExchange -Input [1]: [d_week_seq#40] -Arguments: [d_week_seq#40] - -(57) CometBroadcastHashJoin -Left output [2]: [d_date#10, d_week_seq#38] -Right output [1]: [d_week_seq#40] -Arguments: [d_week_seq#38], [d_week_seq#40], LeftSemi, BuildRight - -(58) CometProject -Input [2]: [d_date#10, d_week_seq#38] -Arguments: [d_date#10], [d_date#10] - -(59) CometBroadcastExchange -Input [1]: [d_date#10] -Arguments: [d_date#10] - -(60) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#10] -Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight - -(61) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(62) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(63) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 20 Hosting Expression = cr_returned_date_sk#14 IN dynamicpruning#4 - -Subquery:3 Hosting operator id = 34 Hosting Expression = wr_returned_date_sk#26 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/extended.txt deleted file mode 100644 index 924b12cf01..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 97 out of 101 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d016a5ee2d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt +++ /dev/null @@ -1,70 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - CometProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - CometBroadcastHashJoin [item_id,sr_item_qty,cr_item_qty,item_id,wr_item_qty] - CometProject [item_id,sr_item_qty,cr_item_qty] - CometBroadcastHashJoin [item_id,sr_item_qty,item_id,cr_item_qty] - CometHashAggregate [sum] [item_id,sr_item_qty,i_item_id,sum(sr_return_quantity)] - CometExchange [i_item_id] #1 - CometHashAggregate [sr_return_quantity] [i_item_id,sum] - CometProject [sr_return_quantity,i_item_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #3 - CometBroadcastExchange [item_id,cr_item_qty] #7 - CometHashAggregate [sum] [item_id,cr_item_qty,i_item_id,sum(cr_return_quantity)] - CometExchange [i_item_id] #8 - CometHashAggregate [cr_return_quantity] [i_item_id,sum] - CometProject [cr_return_quantity,i_item_id] - 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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [item_id,wr_item_qty] #9 - CometHashAggregate [sum] [item_id,wr_item_qty,i_item_id,sum(wr_return_quantity)] - CometExchange [i_item_id] #10 - CometHashAggregate [wr_return_quantity] [i_item_id,sum] - CometProject [wr_return_quantity,i_item_id] - 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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_datafusion/explain.txt deleted file mode 100644 index 31bc59394a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_datafusion/explain.txt +++ /dev/null @@ -1,190 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometTakeOrderedAndProject (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometFilter (11) - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (10) - : : +- CometBroadcastExchange (17) - : : +- CometFilter (16) - : : +- CometNativeScan parquet spark_catalog.default.household_demographics (15) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometNativeScan parquet spark_catalog.default.income_band (20) - +- CometProject (29) - +- CometFilter (28) - +- CometNativeScan parquet spark_catalog.default.store_returns (27) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Condition : ((isnotnull(c_current_addr_sk#4) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_current_hdemo_sk#3)) - -(3) CometProject -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Arguments: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9], [static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#1, 16)) AS c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#5, 20)) AS c_first_name#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#6, 30)) AS c_last_name#9] - -(4) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#10, ca_city#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_city), EqualTo(ca_city,Edgewood), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [ca_address_sk#10, ca_city#11] -Condition : ((isnotnull(ca_city#11) AND (ca_city#11 = Edgewood)) AND isnotnull(ca_address_sk#10)) - -(6) CometProject -Input [2]: [ca_address_sk#10, ca_city#11] -Arguments: [ca_address_sk#10], [ca_address_sk#10] - -(7) CometBroadcastExchange -Input [1]: [ca_address_sk#10] -Arguments: [ca_address_sk#10] - -(8) CometBroadcastHashJoin -Left output [6]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9] -Right output [1]: [ca_address_sk#10] -Arguments: [c_current_addr_sk#4], [ca_address_sk#10], Inner, BuildRight - -(9) CometProject -Input [7]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9, ca_address_sk#10] -Arguments: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9], [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9] - -(10) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(11) CometFilter -Input [1]: [cd_demo_sk#12] -Condition : isnotnull(cd_demo_sk#12) - -(12) CometBroadcastExchange -Input [1]: [cd_demo_sk#12] -Arguments: [cd_demo_sk#12] - -(13) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9] -Right output [1]: [cd_demo_sk#12] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#12], Inner, BuildRight - -(14) CometProject -Input [6]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Arguments: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12], [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(15) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Condition : (isnotnull(hd_demo_sk#13) AND isnotnull(hd_income_band_sk#14)) - -(17) CometBroadcastExchange -Input [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [hd_demo_sk#13, hd_income_band_sk#14] - -(18) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Right output [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [c_current_hdemo_sk#3], [hd_demo_sk#13], Inner, BuildRight - -(19) CometProject -Input [7]: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14], [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14] - -(20) CometNativeScan parquet spark_catalog.default.income_band -Output [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_lower_bound), IsNotNull(ib_upper_bound), GreaterThanOrEqual(ib_lower_bound,38128), LessThanOrEqual(ib_upper_bound,88128), IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Condition : ((((isnotnull(ib_lower_bound#16) AND isnotnull(ib_upper_bound#17)) AND (ib_lower_bound#16 >= 38128)) AND (ib_upper_bound#17 <= 88128)) AND isnotnull(ib_income_band_sk#15)) - -(22) CometProject -Input [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Arguments: [ib_income_band_sk#15], [ib_income_band_sk#15] - -(23) CometBroadcastExchange -Input [1]: [ib_income_band_sk#15] -Arguments: [ib_income_band_sk#15] - -(24) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14] -Right output [1]: [ib_income_band_sk#15] -Arguments: [hd_income_band_sk#14], [ib_income_band_sk#15], Inner, BuildRight - -(25) CometProject -Input [6]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14, ib_income_band_sk#15] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12], [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(26) CometBroadcastExchange -Input [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(27) CometNativeScan parquet spark_catalog.default.store_returns -Output [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_cdemo_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_cdemo_sk#18) - -(29) CometProject -Input [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Arguments: [sr_cdemo_sk#18], [sr_cdemo_sk#18] - -(30) CometBroadcastHashJoin -Left output [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Right output [1]: [sr_cdemo_sk#18] -Arguments: [cd_demo_sk#12], [sr_cdemo_sk#18], Inner, BuildLeft - -(31) CometProject -Input [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, sr_cdemo_sk#18] -Arguments: [customer_id#20, customername#21, c_customer_id#7], [c_customer_id#7 AS customer_id#20, concat(c_last_name#9, , , c_first_name#8) AS customername#21, c_customer_id#7] - -(32) CometTakeOrderedAndProject -Input [3]: [customer_id#20, customername#21, c_customer_id#7] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#7 ASC NULLS FIRST], output=[customer_id#20,customername#21]), [customer_id#20, customername#21], 100, 0, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [customer_id#20, customername#21] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_datafusion/simplified.txt deleted file mode 100644 index 9df82b0e98..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_datafusion/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id] [customer_id,customername] - CometProject [c_last_name,c_first_name] [customer_id,customername,c_customer_id] - 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 [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_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_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_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometNativeScan 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 [ca_address_sk] #2 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_city] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [cd_demo_sk] #3 - CometFilter [cd_demo_sk] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange [ib_income_band_sk] #5 - CometProject [ib_income_band_sk] - CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk,sr_returned_date_sk] - CometNativeScan 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/q84.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_iceberg_compat/explain.txt deleted file mode 100644 index ec24951771..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,190 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometTakeOrderedAndProject (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometFilter (11) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (10) - : : +- CometBroadcastExchange (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (20) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (27) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Condition : ((isnotnull(c_current_addr_sk#4) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_current_hdemo_sk#3)) - -(3) CometProject -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Arguments: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9], [static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#1, 16)) AS c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#5, 20)) AS c_first_name#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#6, 30)) AS c_last_name#9] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#10, ca_city#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_city), EqualTo(ca_city,Edgewood), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [ca_address_sk#10, ca_city#11] -Condition : ((isnotnull(ca_city#11) AND (ca_city#11 = Edgewood)) AND isnotnull(ca_address_sk#10)) - -(6) CometProject -Input [2]: [ca_address_sk#10, ca_city#11] -Arguments: [ca_address_sk#10], [ca_address_sk#10] - -(7) CometBroadcastExchange -Input [1]: [ca_address_sk#10] -Arguments: [ca_address_sk#10] - -(8) CometBroadcastHashJoin -Left output [6]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9] -Right output [1]: [ca_address_sk#10] -Arguments: [c_current_addr_sk#4], [ca_address_sk#10], Inner, BuildRight - -(9) CometProject -Input [7]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9, ca_address_sk#10] -Arguments: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9], [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(11) CometFilter -Input [1]: [cd_demo_sk#12] -Condition : isnotnull(cd_demo_sk#12) - -(12) CometBroadcastExchange -Input [1]: [cd_demo_sk#12] -Arguments: [cd_demo_sk#12] - -(13) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9] -Right output [1]: [cd_demo_sk#12] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#12], Inner, BuildRight - -(14) CometProject -Input [6]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Arguments: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12], [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Condition : (isnotnull(hd_demo_sk#13) AND isnotnull(hd_income_band_sk#14)) - -(17) CometBroadcastExchange -Input [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [hd_demo_sk#13, hd_income_band_sk#14] - -(18) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Right output [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [c_current_hdemo_sk#3], [hd_demo_sk#13], Inner, BuildRight - -(19) CometProject -Input [7]: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14], [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band -Output [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_lower_bound), IsNotNull(ib_upper_bound), GreaterThanOrEqual(ib_lower_bound,38128), LessThanOrEqual(ib_upper_bound,88128), IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Condition : ((((isnotnull(ib_lower_bound#16) AND isnotnull(ib_upper_bound#17)) AND (ib_lower_bound#16 >= 38128)) AND (ib_upper_bound#17 <= 88128)) AND isnotnull(ib_income_band_sk#15)) - -(22) CometProject -Input [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Arguments: [ib_income_band_sk#15], [ib_income_band_sk#15] - -(23) CometBroadcastExchange -Input [1]: [ib_income_band_sk#15] -Arguments: [ib_income_band_sk#15] - -(24) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14] -Right output [1]: [ib_income_band_sk#15] -Arguments: [hd_income_band_sk#14], [ib_income_band_sk#15], Inner, BuildRight - -(25) CometProject -Input [6]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14, ib_income_band_sk#15] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12], [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(26) CometBroadcastExchange -Input [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_cdemo_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_cdemo_sk#18) - -(29) CometProject -Input [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Arguments: [sr_cdemo_sk#18], [sr_cdemo_sk#18] - -(30) CometBroadcastHashJoin -Left output [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Right output [1]: [sr_cdemo_sk#18] -Arguments: [cd_demo_sk#12], [sr_cdemo_sk#18], Inner, BuildLeft - -(31) CometProject -Input [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, sr_cdemo_sk#18] -Arguments: [customer_id#20, customername#21, c_customer_id#7], [c_customer_id#7 AS customer_id#20, concat(c_last_name#9, , , c_first_name#8) AS customername#21, c_customer_id#7] - -(32) CometTakeOrderedAndProject -Input [3]: [customer_id#20, customername#21, c_customer_id#7] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#7 ASC NULLS FIRST], output=[customer_id#20,customername#21]), [customer_id#20, customername#21], 100, 0, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [customer_id#20, customername#21] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_iceberg_compat/simplified.txt deleted file mode 100644 index e43557c27d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id] [customer_id,customername] - CometProject [c_last_name,c_first_name] [customer_id,customername,c_customer_id] - 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 [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_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_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_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] 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 [ca_address_sk] #2 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [cd_demo_sk] #3 - CometFilter [cd_demo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange [ib_income_band_sk] #5 - CometProject [ib_income_band_sk] - CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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/q84/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/explain.txt deleted file mode 100644 index ec24951771..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/explain.txt +++ /dev/null @@ -1,190 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometTakeOrderedAndProject (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometFilter (11) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (10) - : : +- CometBroadcastExchange (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (20) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (27) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Condition : ((isnotnull(c_current_addr_sk#4) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_current_hdemo_sk#3)) - -(3) CometProject -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Arguments: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9], [static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#1, 16)) AS c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#5, 20)) AS c_first_name#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#6, 30)) AS c_last_name#9] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#10, ca_city#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_city), EqualTo(ca_city,Edgewood), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [ca_address_sk#10, ca_city#11] -Condition : ((isnotnull(ca_city#11) AND (ca_city#11 = Edgewood)) AND isnotnull(ca_address_sk#10)) - -(6) CometProject -Input [2]: [ca_address_sk#10, ca_city#11] -Arguments: [ca_address_sk#10], [ca_address_sk#10] - -(7) CometBroadcastExchange -Input [1]: [ca_address_sk#10] -Arguments: [ca_address_sk#10] - -(8) CometBroadcastHashJoin -Left output [6]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9] -Right output [1]: [ca_address_sk#10] -Arguments: [c_current_addr_sk#4], [ca_address_sk#10], Inner, BuildRight - -(9) CometProject -Input [7]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9, ca_address_sk#10] -Arguments: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9], [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(11) CometFilter -Input [1]: [cd_demo_sk#12] -Condition : isnotnull(cd_demo_sk#12) - -(12) CometBroadcastExchange -Input [1]: [cd_demo_sk#12] -Arguments: [cd_demo_sk#12] - -(13) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9] -Right output [1]: [cd_demo_sk#12] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#12], Inner, BuildRight - -(14) CometProject -Input [6]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Arguments: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12], [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Condition : (isnotnull(hd_demo_sk#13) AND isnotnull(hd_income_band_sk#14)) - -(17) CometBroadcastExchange -Input [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [hd_demo_sk#13, hd_income_band_sk#14] - -(18) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Right output [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [c_current_hdemo_sk#3], [hd_demo_sk#13], Inner, BuildRight - -(19) CometProject -Input [7]: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14], [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band -Output [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_lower_bound), IsNotNull(ib_upper_bound), GreaterThanOrEqual(ib_lower_bound,38128), LessThanOrEqual(ib_upper_bound,88128), IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Condition : ((((isnotnull(ib_lower_bound#16) AND isnotnull(ib_upper_bound#17)) AND (ib_lower_bound#16 >= 38128)) AND (ib_upper_bound#17 <= 88128)) AND isnotnull(ib_income_band_sk#15)) - -(22) CometProject -Input [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Arguments: [ib_income_band_sk#15], [ib_income_band_sk#15] - -(23) CometBroadcastExchange -Input [1]: [ib_income_band_sk#15] -Arguments: [ib_income_band_sk#15] - -(24) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14] -Right output [1]: [ib_income_band_sk#15] -Arguments: [hd_income_band_sk#14], [ib_income_band_sk#15], Inner, BuildRight - -(25) CometProject -Input [6]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14, ib_income_band_sk#15] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12], [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(26) CometBroadcastExchange -Input [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_cdemo_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_cdemo_sk#18) - -(29) CometProject -Input [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Arguments: [sr_cdemo_sk#18], [sr_cdemo_sk#18] - -(30) CometBroadcastHashJoin -Left output [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Right output [1]: [sr_cdemo_sk#18] -Arguments: [cd_demo_sk#12], [sr_cdemo_sk#18], Inner, BuildLeft - -(31) CometProject -Input [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, sr_cdemo_sk#18] -Arguments: [customer_id#20, customername#21, c_customer_id#7], [c_customer_id#7 AS customer_id#20, concat(c_last_name#9, , , c_first_name#8) AS customername#21, c_customer_id#7] - -(32) CometTakeOrderedAndProject -Input [3]: [customer_id#20, customername#21, c_customer_id#7] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#7 ASC NULLS FIRST], output=[customer_id#20,customername#21]), [customer_id#20, customername#21], 100, 0, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [customer_id#20, customername#21] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/extended.txt deleted file mode 100644 index 5fce933878..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/extended.txt +++ /dev/null @@ -1,35 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - -Comet accelerated 32 out of 32 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index e43557c27d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id] [customer_id,customername] - CometProject [c_last_name,c_first_name] [customer_id,customername,c_customer_id] - 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 [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_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_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_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] 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 [ca_address_sk] #2 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [cd_demo_sk] #3 - CometFilter [cd_demo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange [ib_income_band_sk] #5 - CometProject [ib_income_band_sk] - CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/explain.txt deleted file mode 100644 index ea154af6c8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/explain.txt +++ /dev/null @@ -1,329 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (52) -+- * HashAggregate (51) - +- * CometColumnarToRow (50) - +- CometColumnarExchange (49) - +- * HashAggregate (48) - +- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (30) - : : : +- * BroadcastHashJoin Inner BuildRight (29) - : : : :- * Project (23) - : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : :- * Project (16) - : : : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : : : :- BroadcastExchange (4) - : : : : : : : +- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : : : : +- * CometColumnarToRow (8) - : : : : : : +- CometProject (7) - : : : : : : +- CometFilter (6) - : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns (5) - : : : : : +- BroadcastExchange (14) - : : : : : +- * CometColumnarToRow (13) - : : : : : +- CometFilter (12) - : : : : : +- CometNativeScan parquet spark_catalog.default.web_page (11) - : : : : +- BroadcastExchange (21) - : : : : +- * CometColumnarToRow (20) - : : : : +- CometProject (19) - : : : : +- CometFilter (18) - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (17) - : : : +- BroadcastExchange (28) - : : : +- * CometColumnarToRow (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (24) - : : +- BroadcastExchange (35) - : : +- * CometColumnarToRow (34) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (31) - : +- ReusedExchange (38) - +- BroadcastExchange (45) - +- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometNativeScan parquet spark_catalog.default.reason (41) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_page_sk), Or(Or(And(GreaterThanOrEqual(ws_sales_price,100.00),LessThanOrEqual(ws_sales_price,150.00)),And(GreaterThanOrEqual(ws_sales_price,50.00),LessThanOrEqual(ws_sales_price,100.00))),And(GreaterThanOrEqual(ws_sales_price,150.00),LessThanOrEqual(ws_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ws_net_profit,100.00),LessThanOrEqual(ws_net_profit,200.00)),And(GreaterThanOrEqual(ws_net_profit,150.00),LessThanOrEqual(ws_net_profit,300.00))),And(GreaterThanOrEqual(ws_net_profit,50.00),LessThanOrEqual(ws_net_profit,250.00)))] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] - -(3) Filter [codegen id : 1] -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((((isnotnull(ws_item_sk#1) AND isnotnull(ws_order_number#3)) AND isnotnull(ws_web_page_sk#2)) AND ((((ws_sales_price#5 >= 100.00) AND (ws_sales_price#5 <= 150.00)) OR ((ws_sales_price#5 >= 50.00) AND (ws_sales_price#5 <= 100.00))) OR ((ws_sales_price#5 >= 150.00) AND (ws_sales_price#5 <= 200.00)))) AND ((((ws_net_profit#6 >= 100.00) AND (ws_net_profit#6 <= 200.00)) OR ((ws_net_profit#6 >= 150.00) AND (ws_net_profit#6 <= 300.00))) OR ((ws_net_profit#6 >= 50.00) AND (ws_net_profit#6 <= 250.00)))) - -(4) BroadcastExchange -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[2, int, false] as bigint) & 4294967295))),false), [plan_id=1] - -(5) CometNativeScan parquet spark_catalog.default.web_returns -Output [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number), IsNotNull(wr_refunded_cdemo_sk), IsNotNull(wr_returning_cdemo_sk), IsNotNull(wr_refunded_addr_sk), IsNotNull(wr_reason_sk)] -ReadSchema: struct - -(6) CometFilter -Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Condition : (((((isnotnull(wr_item_sk#9) AND isnotnull(wr_order_number#14)) AND isnotnull(wr_refunded_cdemo_sk#10)) AND isnotnull(wr_returning_cdemo_sk#12)) AND isnotnull(wr_refunded_addr_sk#11)) AND isnotnull(wr_reason_sk#13)) - -(7) CometProject -Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Arguments: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16], [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] - -(8) CometColumnarToRow -Input [8]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] - -(9) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [ws_item_sk#1, ws_order_number#3] -Right keys [2]: [wr_item_sk#9, wr_order_number#14] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 8] -Output [11]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Input [15]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] - -(11) CometNativeScan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(12) CometFilter -Input [1]: [wp_web_page_sk#18] -Condition : isnotnull(wp_web_page_sk#18) - -(13) CometColumnarToRow [codegen id : 2] -Input [1]: [wp_web_page_sk#18] - -(14) BroadcastExchange -Input [1]: [wp_web_page_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_web_page_sk#2] -Right keys [1]: [wp_web_page_sk#18] -Join type: Inner -Join condition: None - -(16) Project [codegen id : 8] -Output [10]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Input [12]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, wp_web_page_sk#18] - -(17) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(18) CometFilter -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Condition : (((isnotnull(cd_demo_sk#19) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)))) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)))) AND ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) = M) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) = Advanced Degree )) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) = S) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) = College ))) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) = W) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) = 2 yr Degree )))) - -(19) CometProject -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) AS cd_marital_status#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) AS cd_education_status#23] - -(20) CometColumnarToRow [codegen id : 3] -Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] - -(21) BroadcastExchange -Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(22) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [wr_refunded_cdemo_sk#10] -Right keys [1]: [cd_demo_sk#19] -Join type: Inner -Join condition: ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) - -(23) Project [codegen id : 8] -Output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23] -Input [13]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] - -(24) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(25) CometFilter -Input [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Condition : ((isnotnull(cd_demo_sk#24) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#25, 1)))) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#26, 20)))) - -(26) CometProject -Input [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Arguments: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28], [cd_demo_sk#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#25, 1)) AS cd_marital_status#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#26, 20)) AS cd_education_status#28] - -(27) CometColumnarToRow [codegen id : 4] -Input [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] - -(28) BroadcastExchange -Input [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, string, true], input[2, string, true]),false), [plan_id=4] - -(29) BroadcastHashJoin [codegen id : 8] -Left keys [3]: [wr_returning_cdemo_sk#12, cd_marital_status#22, cd_education_status#23] -Right keys [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Join type: Inner -Join condition: None - -(30) Project [codegen id : 8] -Output [7]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Input [13]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23, cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] - -(31) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Condition : (((isnotnull(ca_country#31) AND (ca_country#31 = United States)) AND isnotnull(ca_address_sk#29)) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#30, 2)) IN (IN,OH,NJ) OR static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#30, 2)) IN (WI,CT,KY)) OR static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#30, 2)) IN (LA,IA,AR))) - -(33) CometProject -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Arguments: [ca_address_sk#29, ca_state#32], [ca_address_sk#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#30, 2)) AS ca_state#32] - -(34) CometColumnarToRow [codegen id : 5] -Input [2]: [ca_address_sk#29, ca_state#32] - -(35) BroadcastExchange -Input [2]: [ca_address_sk#29, ca_state#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(36) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [wr_refunded_addr_sk#11] -Right keys [1]: [ca_address_sk#29] -Join type: Inner -Join condition: ((((ca_state#32 IN (IN,OH,NJ) AND (ws_net_profit#6 >= 100.00)) AND (ws_net_profit#6 <= 200.00)) OR ((ca_state#32 IN (WI,CT,KY) AND (ws_net_profit#6 >= 150.00)) AND (ws_net_profit#6 <= 300.00))) OR ((ca_state#32 IN (LA,IA,AR) AND (ws_net_profit#6 >= 50.00)) AND (ws_net_profit#6 <= 250.00))) - -(37) Project [codegen id : 8] -Output [5]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Input [9]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, ca_address_sk#29, ca_state#32] - -(38) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#33] - -(39) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_date_sk#7] -Right keys [1]: [d_date_sk#33] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 8] -Output [4]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Input [6]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, d_date_sk#33] - -(41) CometNativeScan parquet spark_catalog.default.reason -Output [2]: [r_reason_sk#34, r_reason_desc#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [r_reason_sk#34, r_reason_desc#35] -Condition : isnotnull(r_reason_sk#34) - -(43) CometProject -Input [2]: [r_reason_sk#34, r_reason_desc#35] -Arguments: [r_reason_sk#34, r_reason_desc#36], [r_reason_sk#34, static_invoke(CharVarcharCodegenUtils.readSidePadding(r_reason_desc#35, 100)) AS r_reason_desc#36] - -(44) CometColumnarToRow [codegen id : 7] -Input [2]: [r_reason_sk#34, r_reason_desc#36] - -(45) BroadcastExchange -Input [2]: [r_reason_sk#34, r_reason_desc#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(46) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [wr_reason_sk#13] -Right keys [1]: [r_reason_sk#34] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 8] -Output [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#36] -Input [6]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, r_reason_sk#34, r_reason_desc#36] - -(48) HashAggregate [codegen id : 8] -Input [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#36] -Keys [1]: [r_reason_desc#36] -Functions [3]: [partial_avg(ws_quantity#4), partial_avg(UnscaledValue(wr_refunded_cash#16)), partial_avg(UnscaledValue(wr_fee#15))] -Aggregate Attributes [6]: [sum#37, count#38, sum#39, count#40, sum#41, count#42] -Results [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, count#48] - -(49) CometColumnarExchange -Input [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, count#48] -Arguments: hashpartitioning(r_reason_desc#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(50) CometColumnarToRow [codegen id : 9] -Input [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, count#48] - -(51) HashAggregate [codegen id : 9] -Input [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, count#48] -Keys [1]: [r_reason_desc#36] -Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg(UnscaledValue(wr_fee#15))] -Aggregate Attributes [3]: [avg(ws_quantity#4)#49, avg(UnscaledValue(wr_refunded_cash#16))#50, avg(UnscaledValue(wr_fee#15))#51] -Results [4]: [substr(r_reason_desc#36, 1, 20) AS substr(r_reason_desc, 1, 20)#52, avg(ws_quantity#4)#49 AS avg(ws_quantity)#53, cast((avg(UnscaledValue(wr_refunded_cash#16))#50 / 100.0) as decimal(11,6)) AS avg(wr_refunded_cash)#54, cast((avg(UnscaledValue(wr_fee#15))#51 / 100.0) as decimal(11,6)) AS avg(wr_fee)#55] - -(52) TakeOrderedAndProject -Input [4]: [substr(r_reason_desc, 1, 20)#52, avg(ws_quantity)#53, avg(wr_refunded_cash)#54, avg(wr_fee)#55] -Arguments: 100, [substr(r_reason_desc, 1, 20)#52 ASC NULLS FIRST, avg(ws_quantity)#53 ASC NULLS FIRST, avg(wr_refunded_cash)#54 ASC NULLS FIRST, avg(wr_fee)#55 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#52, avg(ws_quantity)#53, avg(wr_refunded_cash)#54, avg(wr_fee)#55] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometNativeScan parquet spark_catalog.default.date_dim (53) - - -(53) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [2]: [d_date_sk#33, d_year#56] -Condition : ((isnotnull(d_year#56) AND (d_year#56 = 2000)) AND isnotnull(d_date_sk#33)) - -(55) CometProject -Input [2]: [d_date_sk#33, d_year#56] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(56) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#33] - -(57) BroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/simplified.txt deleted file mode 100644 index 68d46e608d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/simplified.txt +++ /dev/null @@ -1,83 +0,0 @@ -TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] - WholeStageCodegen (9) - HashAggregate [r_reason_desc,sum,count,sum,count,sum,count] [avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee)),substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [r_reason_desc] #1 - WholeStageCodegen (8) - HashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - BroadcastHashJoin [wr_reason_sk,r_reason_sk] - Project [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] - Project [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] - Project [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] - BroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] - Project [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] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [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] - BroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - 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_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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [r_reason_desc] [r_reason_sk,r_reason_desc] - CometFilter [r_reason_sk,r_reason_desc] - CometNativeScan 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/q85.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/explain.txt deleted file mode 100644 index dff4c2b171..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,309 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (48) -+- CometTakeOrderedAndProject (47) - +- CometHashAggregate (46) - +- CometExchange (45) - +- CometHashAggregate (44) - +- CometProject (43) - +- CometBroadcastHashJoin (42) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (19) - : : : : +- CometBroadcastHashJoin (18) - : : : : :- CometProject (13) - : : : : : +- CometBroadcastHashJoin (12) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometBroadcastExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (4) - : : : : : +- CometBroadcastExchange (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (9) - : : : : +- CometBroadcastExchange (17) - : : : : +- CometProject (16) - : : : : +- CometFilter (15) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (14) - : : : +- CometBroadcastExchange (23) - : : : +- CometProject (22) - : : : +- CometFilter (21) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (20) - : : +- CometBroadcastExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (26) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - +- CometBroadcastExchange (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (38) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_page_sk), Or(Or(And(GreaterThanOrEqual(ws_sales_price,100.00),LessThanOrEqual(ws_sales_price,150.00)),And(GreaterThanOrEqual(ws_sales_price,50.00),LessThanOrEqual(ws_sales_price,100.00))),And(GreaterThanOrEqual(ws_sales_price,150.00),LessThanOrEqual(ws_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ws_net_profit,100.00),LessThanOrEqual(ws_net_profit,200.00)),And(GreaterThanOrEqual(ws_net_profit,150.00),LessThanOrEqual(ws_net_profit,300.00))),And(GreaterThanOrEqual(ws_net_profit,50.00),LessThanOrEqual(ws_net_profit,250.00)))] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((((isnotnull(ws_item_sk#1) AND isnotnull(ws_order_number#3)) AND isnotnull(ws_web_page_sk#2)) AND ((((ws_sales_price#5 >= 100.00) AND (ws_sales_price#5 <= 150.00)) OR ((ws_sales_price#5 >= 50.00) AND (ws_sales_price#5 <= 100.00))) OR ((ws_sales_price#5 >= 150.00) AND (ws_sales_price#5 <= 200.00)))) AND ((((ws_net_profit#6 >= 100.00) AND (ws_net_profit#6 <= 200.00)) OR ((ws_net_profit#6 >= 150.00) AND (ws_net_profit#6 <= 300.00))) OR ((ws_net_profit#6 >= 50.00) AND (ws_net_profit#6 <= 250.00)))) - -(3) CometBroadcastExchange -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Arguments: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number), IsNotNull(wr_refunded_cdemo_sk), IsNotNull(wr_returning_cdemo_sk), IsNotNull(wr_refunded_addr_sk), IsNotNull(wr_reason_sk)] -ReadSchema: struct - -(5) CometFilter -Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Condition : (((((isnotnull(wr_item_sk#9) AND isnotnull(wr_order_number#14)) AND isnotnull(wr_refunded_cdemo_sk#10)) AND isnotnull(wr_returning_cdemo_sk#12)) AND isnotnull(wr_refunded_addr_sk#11)) AND isnotnull(wr_reason_sk#13)) - -(6) CometProject -Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Arguments: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16], [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] - -(7) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Right output [8]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] -Arguments: [ws_item_sk#1, ws_order_number#3], [wr_item_sk#9, wr_order_number#14], Inner, BuildLeft - -(8) CometProject -Input [15]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] -Arguments: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(10) CometFilter -Input [1]: [wp_web_page_sk#18] -Condition : isnotnull(wp_web_page_sk#18) - -(11) CometBroadcastExchange -Input [1]: [wp_web_page_sk#18] -Arguments: [wp_web_page_sk#18] - -(12) CometBroadcastHashJoin -Left output [11]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [1]: [wp_web_page_sk#18] -Arguments: [ws_web_page_sk#2], [wp_web_page_sk#18], Inner, BuildRight - -(13) CometProject -Input [12]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, wp_web_page_sk#18] -Arguments: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Condition : (((isnotnull(cd_demo_sk#19) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)))) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)))) AND ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) = M) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) = Advanced Degree )) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) = S) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) = College ))) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) = W) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) = 2 yr Degree )))) - -(16) CometProject -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) AS cd_marital_status#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) AS cd_education_status#23] - -(17) CometBroadcastExchange -Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] - -(18) CometBroadcastHashJoin -Left output [10]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [wr_refunded_cdemo_sk#10], [cd_demo_sk#19], Inner, ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))), BuildRight - -(19) CometProject -Input [13]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23], [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Condition : ((isnotnull(cd_demo_sk#24) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#25, 1)))) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#26, 20)))) - -(22) CometProject -Input [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Arguments: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28], [cd_demo_sk#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#25, 1)) AS cd_marital_status#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#26, 20)) AS cd_education_status#28] - -(23) CometBroadcastExchange -Input [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Arguments: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] - -(24) CometBroadcastHashJoin -Left output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23] -Right output [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Arguments: [wr_returning_cdemo_sk#12, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28], Inner, BuildRight - -(25) CometProject -Input [13]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23, cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Arguments: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Condition : (((isnotnull(ca_country#31) AND (ca_country#31 = United States)) AND isnotnull(ca_address_sk#29)) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#30, 2)) IN (IN,OH,NJ) OR static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#30, 2)) IN (WI,CT,KY)) OR static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#30, 2)) IN (LA,IA,AR))) - -(28) CometProject -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Arguments: [ca_address_sk#29, ca_state#32], [ca_address_sk#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#30, 2)) AS ca_state#32] - -(29) CometBroadcastExchange -Input [2]: [ca_address_sk#29, ca_state#32] -Arguments: [ca_address_sk#29, ca_state#32] - -(30) CometBroadcastHashJoin -Left output [7]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [2]: [ca_address_sk#29, ca_state#32] -Arguments: [wr_refunded_addr_sk#11], [ca_address_sk#29], Inner, ((((ca_state#32 IN (IN,OH,NJ) AND (ws_net_profit#6 >= 100.00)) AND (ws_net_profit#6 <= 200.00)) OR ((ca_state#32 IN (WI,CT,KY) AND (ws_net_profit#6 >= 150.00)) AND (ws_net_profit#6 <= 300.00))) OR ((ca_state#32 IN (LA,IA,AR) AND (ws_net_profit#6 >= 50.00)) AND (ws_net_profit#6 <= 250.00))), BuildRight - -(31) CometProject -Input [9]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, ca_address_sk#29, ca_state#32] -Arguments: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#33, d_year#34] -Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2000)) AND isnotnull(d_date_sk#33)) - -(34) CometProject -Input [2]: [d_date_sk#33, d_year#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(35) CometBroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: [d_date_sk#33] - -(36) CometBroadcastHashJoin -Left output [5]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [1]: [d_date_sk#33] -Arguments: [ws_sold_date_sk#7], [d_date_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, d_date_sk#33] -Arguments: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Output [2]: [r_reason_sk#35, r_reason_desc#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk)] -ReadSchema: struct - -(39) CometFilter -Input [2]: [r_reason_sk#35, r_reason_desc#36] -Condition : isnotnull(r_reason_sk#35) - -(40) CometProject -Input [2]: [r_reason_sk#35, r_reason_desc#36] -Arguments: [r_reason_sk#35, r_reason_desc#37], [r_reason_sk#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(r_reason_desc#36, 100)) AS r_reason_desc#37] - -(41) CometBroadcastExchange -Input [2]: [r_reason_sk#35, r_reason_desc#37] -Arguments: [r_reason_sk#35, r_reason_desc#37] - -(42) CometBroadcastHashJoin -Left output [4]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [2]: [r_reason_sk#35, r_reason_desc#37] -Arguments: [wr_reason_sk#13], [r_reason_sk#35], Inner, BuildRight - -(43) CometProject -Input [6]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, r_reason_sk#35, r_reason_desc#37] -Arguments: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#37], [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#37] - -(44) CometHashAggregate -Input [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#37] -Keys [1]: [r_reason_desc#37] -Functions [3]: [partial_avg(ws_quantity#4), partial_avg(UnscaledValue(wr_refunded_cash#16)), partial_avg(UnscaledValue(wr_fee#15))] - -(45) CometExchange -Input [7]: [r_reason_desc#37, sum#38, count#39, sum#40, count#41, sum#42, count#43] -Arguments: hashpartitioning(r_reason_desc#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(46) CometHashAggregate -Input [7]: [r_reason_desc#37, sum#38, count#39, sum#40, count#41, sum#42, count#43] -Keys [1]: [r_reason_desc#37] -Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg(UnscaledValue(wr_fee#15))] - -(47) CometTakeOrderedAndProject -Input [4]: [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(r_reason_desc, 1, 20)#44 ASC NULLS FIRST,avg(ws_quantity)#45 ASC NULLS FIRST,avg(wr_refunded_cash)#46 ASC NULLS FIRST,avg(wr_fee)#47 ASC NULLS FIRST], output=[substr(r_reason_desc, 1, 20)#44,avg(ws_quantity)#45,avg(wr_refunded_cash)#46,avg(wr_fee)#47]), [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47], 100, 0, [substr(r_reason_desc, 1, 20)#44 ASC NULLS FIRST, avg(ws_quantity)#45 ASC NULLS FIRST, avg(wr_refunded_cash)#46 ASC NULLS FIRST, avg(wr_fee)#47 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47] - -(48) CometColumnarToRow [codegen id : 1] -Input [4]: [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [2]: [d_date_sk#33, d_year#34] -Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2000)) AND isnotnull(d_date_sk#33)) - -(51) CometProject -Input [2]: [d_date_sk#33, d_year#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#33] - -(53) BroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/simplified.txt deleted file mode 100644 index dfcca32f03..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,58 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] - CometHashAggregate [sum,count,sum,count,sum,count] [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),r_reason_desc,avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee))] - CometExchange [r_reason_desc] #1 - CometHashAggregate [ws_quantity,wr_refunded_cash,wr_fee] [r_reason_desc,sum,count,sum,count,sum,count] - CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - 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_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 [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 [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 [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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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_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 [native_iceberg_compat] 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 [wp_web_page_sk] #4 - CometFilter [wp_web_page_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [ca_address_sk,ca_state] #7 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #8 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 - CometProject [r_reason_desc] [r_reason_sk,r_reason_desc] - CometFilter [r_reason_sk,r_reason_desc] - CometScan [native_iceberg_compat] 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/q85/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/explain.txt deleted file mode 100644 index dff4c2b171..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/explain.txt +++ /dev/null @@ -1,309 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (48) -+- CometTakeOrderedAndProject (47) - +- CometHashAggregate (46) - +- CometExchange (45) - +- CometHashAggregate (44) - +- CometProject (43) - +- CometBroadcastHashJoin (42) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (19) - : : : : +- CometBroadcastHashJoin (18) - : : : : :- CometProject (13) - : : : : : +- CometBroadcastHashJoin (12) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometBroadcastExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (4) - : : : : : +- CometBroadcastExchange (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (9) - : : : : +- CometBroadcastExchange (17) - : : : : +- CometProject (16) - : : : : +- CometFilter (15) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (14) - : : : +- CometBroadcastExchange (23) - : : : +- CometProject (22) - : : : +- CometFilter (21) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (20) - : : +- CometBroadcastExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (26) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - +- CometBroadcastExchange (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (38) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_page_sk), Or(Or(And(GreaterThanOrEqual(ws_sales_price,100.00),LessThanOrEqual(ws_sales_price,150.00)),And(GreaterThanOrEqual(ws_sales_price,50.00),LessThanOrEqual(ws_sales_price,100.00))),And(GreaterThanOrEqual(ws_sales_price,150.00),LessThanOrEqual(ws_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ws_net_profit,100.00),LessThanOrEqual(ws_net_profit,200.00)),And(GreaterThanOrEqual(ws_net_profit,150.00),LessThanOrEqual(ws_net_profit,300.00))),And(GreaterThanOrEqual(ws_net_profit,50.00),LessThanOrEqual(ws_net_profit,250.00)))] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((((isnotnull(ws_item_sk#1) AND isnotnull(ws_order_number#3)) AND isnotnull(ws_web_page_sk#2)) AND ((((ws_sales_price#5 >= 100.00) AND (ws_sales_price#5 <= 150.00)) OR ((ws_sales_price#5 >= 50.00) AND (ws_sales_price#5 <= 100.00))) OR ((ws_sales_price#5 >= 150.00) AND (ws_sales_price#5 <= 200.00)))) AND ((((ws_net_profit#6 >= 100.00) AND (ws_net_profit#6 <= 200.00)) OR ((ws_net_profit#6 >= 150.00) AND (ws_net_profit#6 <= 300.00))) OR ((ws_net_profit#6 >= 50.00) AND (ws_net_profit#6 <= 250.00)))) - -(3) CometBroadcastExchange -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Arguments: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number), IsNotNull(wr_refunded_cdemo_sk), IsNotNull(wr_returning_cdemo_sk), IsNotNull(wr_refunded_addr_sk), IsNotNull(wr_reason_sk)] -ReadSchema: struct - -(5) CometFilter -Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Condition : (((((isnotnull(wr_item_sk#9) AND isnotnull(wr_order_number#14)) AND isnotnull(wr_refunded_cdemo_sk#10)) AND isnotnull(wr_returning_cdemo_sk#12)) AND isnotnull(wr_refunded_addr_sk#11)) AND isnotnull(wr_reason_sk#13)) - -(6) CometProject -Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Arguments: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16], [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] - -(7) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Right output [8]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] -Arguments: [ws_item_sk#1, ws_order_number#3], [wr_item_sk#9, wr_order_number#14], Inner, BuildLeft - -(8) CometProject -Input [15]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] -Arguments: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(10) CometFilter -Input [1]: [wp_web_page_sk#18] -Condition : isnotnull(wp_web_page_sk#18) - -(11) CometBroadcastExchange -Input [1]: [wp_web_page_sk#18] -Arguments: [wp_web_page_sk#18] - -(12) CometBroadcastHashJoin -Left output [11]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [1]: [wp_web_page_sk#18] -Arguments: [ws_web_page_sk#2], [wp_web_page_sk#18], Inner, BuildRight - -(13) CometProject -Input [12]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, wp_web_page_sk#18] -Arguments: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Condition : (((isnotnull(cd_demo_sk#19) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)))) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)))) AND ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) = M) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) = Advanced Degree )) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) = S) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) = College ))) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) = W) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) = 2 yr Degree )))) - -(16) CometProject -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) AS cd_marital_status#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) AS cd_education_status#23] - -(17) CometBroadcastExchange -Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] - -(18) CometBroadcastHashJoin -Left output [10]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [wr_refunded_cdemo_sk#10], [cd_demo_sk#19], Inner, ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))), BuildRight - -(19) CometProject -Input [13]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23], [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Condition : ((isnotnull(cd_demo_sk#24) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#25, 1)))) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#26, 20)))) - -(22) CometProject -Input [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Arguments: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28], [cd_demo_sk#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#25, 1)) AS cd_marital_status#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#26, 20)) AS cd_education_status#28] - -(23) CometBroadcastExchange -Input [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Arguments: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] - -(24) CometBroadcastHashJoin -Left output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23] -Right output [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Arguments: [wr_returning_cdemo_sk#12, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28], Inner, BuildRight - -(25) CometProject -Input [13]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23, cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Arguments: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Condition : (((isnotnull(ca_country#31) AND (ca_country#31 = United States)) AND isnotnull(ca_address_sk#29)) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#30, 2)) IN (IN,OH,NJ) OR static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#30, 2)) IN (WI,CT,KY)) OR static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#30, 2)) IN (LA,IA,AR))) - -(28) CometProject -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Arguments: [ca_address_sk#29, ca_state#32], [ca_address_sk#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#30, 2)) AS ca_state#32] - -(29) CometBroadcastExchange -Input [2]: [ca_address_sk#29, ca_state#32] -Arguments: [ca_address_sk#29, ca_state#32] - -(30) CometBroadcastHashJoin -Left output [7]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [2]: [ca_address_sk#29, ca_state#32] -Arguments: [wr_refunded_addr_sk#11], [ca_address_sk#29], Inner, ((((ca_state#32 IN (IN,OH,NJ) AND (ws_net_profit#6 >= 100.00)) AND (ws_net_profit#6 <= 200.00)) OR ((ca_state#32 IN (WI,CT,KY) AND (ws_net_profit#6 >= 150.00)) AND (ws_net_profit#6 <= 300.00))) OR ((ca_state#32 IN (LA,IA,AR) AND (ws_net_profit#6 >= 50.00)) AND (ws_net_profit#6 <= 250.00))), BuildRight - -(31) CometProject -Input [9]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, ca_address_sk#29, ca_state#32] -Arguments: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#33, d_year#34] -Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2000)) AND isnotnull(d_date_sk#33)) - -(34) CometProject -Input [2]: [d_date_sk#33, d_year#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(35) CometBroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: [d_date_sk#33] - -(36) CometBroadcastHashJoin -Left output [5]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [1]: [d_date_sk#33] -Arguments: [ws_sold_date_sk#7], [d_date_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, d_date_sk#33] -Arguments: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Output [2]: [r_reason_sk#35, r_reason_desc#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk)] -ReadSchema: struct - -(39) CometFilter -Input [2]: [r_reason_sk#35, r_reason_desc#36] -Condition : isnotnull(r_reason_sk#35) - -(40) CometProject -Input [2]: [r_reason_sk#35, r_reason_desc#36] -Arguments: [r_reason_sk#35, r_reason_desc#37], [r_reason_sk#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(r_reason_desc#36, 100)) AS r_reason_desc#37] - -(41) CometBroadcastExchange -Input [2]: [r_reason_sk#35, r_reason_desc#37] -Arguments: [r_reason_sk#35, r_reason_desc#37] - -(42) CometBroadcastHashJoin -Left output [4]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [2]: [r_reason_sk#35, r_reason_desc#37] -Arguments: [wr_reason_sk#13], [r_reason_sk#35], Inner, BuildRight - -(43) CometProject -Input [6]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, r_reason_sk#35, r_reason_desc#37] -Arguments: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#37], [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#37] - -(44) CometHashAggregate -Input [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#37] -Keys [1]: [r_reason_desc#37] -Functions [3]: [partial_avg(ws_quantity#4), partial_avg(UnscaledValue(wr_refunded_cash#16)), partial_avg(UnscaledValue(wr_fee#15))] - -(45) CometExchange -Input [7]: [r_reason_desc#37, sum#38, count#39, sum#40, count#41, sum#42, count#43] -Arguments: hashpartitioning(r_reason_desc#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(46) CometHashAggregate -Input [7]: [r_reason_desc#37, sum#38, count#39, sum#40, count#41, sum#42, count#43] -Keys [1]: [r_reason_desc#37] -Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg(UnscaledValue(wr_fee#15))] - -(47) CometTakeOrderedAndProject -Input [4]: [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(r_reason_desc, 1, 20)#44 ASC NULLS FIRST,avg(ws_quantity)#45 ASC NULLS FIRST,avg(wr_refunded_cash)#46 ASC NULLS FIRST,avg(wr_fee)#47 ASC NULLS FIRST], output=[substr(r_reason_desc, 1, 20)#44,avg(ws_quantity)#45,avg(wr_refunded_cash)#46,avg(wr_fee)#47]), [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47], 100, 0, [substr(r_reason_desc, 1, 20)#44 ASC NULLS FIRST, avg(ws_quantity)#45 ASC NULLS FIRST, avg(wr_refunded_cash)#46 ASC NULLS FIRST, avg(wr_fee)#47 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47] - -(48) CometColumnarToRow [codegen id : 1] -Input [4]: [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [2]: [d_date_sk#33, d_year#34] -Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2000)) AND isnotnull(d_date_sk#33)) - -(51) CometProject -Input [2]: [d_date_sk#33, d_year#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#33] - -(53) BroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/extended.txt deleted file mode 100644 index e2383394d1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/extended.txt +++ /dev/null @@ -1,56 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason - -Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index dfcca32f03..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt +++ /dev/null @@ -1,58 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] - CometHashAggregate [sum,count,sum,count,sum,count] [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),r_reason_desc,avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee))] - CometExchange [r_reason_desc] #1 - CometHashAggregate [ws_quantity,wr_refunded_cash,wr_fee] [r_reason_desc,sum,count,sum,count,sum,count] - CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - 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_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 [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 [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 [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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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_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 [native_iceberg_compat] 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 [wp_web_page_sk] #4 - CometFilter [wp_web_page_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [ca_address_sk,ca_state] #7 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #8 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 - CometProject [r_reason_desc] [r_reason_sk,r_reason_desc] - CometFilter [r_reason_sk,r_reason_desc] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/explain.txt deleted file mode 100644 index 829abbaec4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (17) - +- CometColumnarExchange (16) - +- * HashAggregate (15) - +- * Expand (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.web_sales (1) - : +- ReusedExchange (4) - +- BroadcastExchange (11) - +- * CometColumnarToRow (10) - +- CometProject (9) - +- CometFilter (8) - +- CometNativeScan parquet spark_catalog.default.item (7) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(4) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#5] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [2]: [ws_item_sk#1, ws_net_paid#2] -Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] - -(7) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#6, i_class#7, i_category#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Condition : isnotnull(i_item_sk#6) - -(9) CometProject -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Arguments: [i_item_sk#6, i_class#9, i_category#10], [i_item_sk#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#7, 50)) AS i_class#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#8, 50)) AS i_category#10] - -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#6, i_class#9, i_category#10] - -(11) BroadcastExchange -Input [3]: [i_item_sk#6, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [3]: [ws_net_paid#2, i_category#10, i_class#9] -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#6, i_class#9, i_category#10] - -(14) Expand [codegen id : 3] -Input [3]: [ws_net_paid#2, i_category#10, i_class#9] -Arguments: [[ws_net_paid#2, i_category#10, i_class#9, 0], [ws_net_paid#2, i_category#10, null, 1], [ws_net_paid#2, null, null, 3]], [ws_net_paid#2, i_category#11, i_class#12, spark_grouping_id#13] - -(15) HashAggregate [codegen id : 3] -Input [4]: [ws_net_paid#2, i_category#11, i_class#12, spark_grouping_id#13] -Keys [3]: [i_category#11, i_class#12, spark_grouping_id#13] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum#14] -Results [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] - -(16) CometColumnarExchange -Input [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] -Arguments: hashpartitioning(i_category#11, i_class#12, spark_grouping_id#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(17) CometColumnarToRow [codegen id : 4] -Input [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] - -(18) HashAggregate [codegen id : 4] -Input [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] -Keys [3]: [i_category#11, i_class#12, spark_grouping_id#13] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#16] -Results [7]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#16,17,2) AS total_sum#17, i_category#11, i_class#12, (cast((shiftright(spark_grouping_id#13, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#13, 0) & 1) as tinyint)) AS lochierarchy#18, MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#16,17,2) AS _w0#19, (cast((shiftright(spark_grouping_id#13, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#13, 0) & 1) as tinyint)) AS _w1#20, CASE WHEN (cast((shiftright(spark_grouping_id#13, 0) & 1) as tinyint) = 0) THEN i_category#11 END AS _w2#21] - -(19) CometColumnarExchange -Input [7]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21] -Arguments: hashpartitioning(_w1#20, _w2#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(20) CometSort -Input [7]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21] -Arguments: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21], [_w1#20 ASC NULLS FIRST, _w2#21 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] - -(21) CometColumnarToRow [codegen id : 5] -Input [7]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21] - -(22) Window -Input [7]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21] -Arguments: [rank(_w0#19) windowspecdefinition(_w1#20, _w2#21, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#22], [_w1#20, _w2#21], [_w0#19 DESC NULLS LAST] - -(23) Project [codegen id : 6] -Output [5]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, rank_within_parent#22] -Input [8]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21, rank_within_parent#22] - -(24) TakeOrderedAndProject -Input [5]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, rank_within_parent#22] -Arguments: 100, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0) THEN i_category#11 END ASC NULLS FIRST, rank_within_parent#22 ASC NULLS FIRST], [total_sum#17, i_category#11, i_class#12, lochierarchy#18, rank_within_parent#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometNativeScan parquet spark_catalog.default.date_dim (25) - - -(25) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#23] -Condition : (((isnotnull(d_month_seq#23) AND (d_month_seq#23 >= 1200)) AND (d_month_seq#23 <= 1211)) AND isnotnull(d_date_sk#5)) - -(27) CometProject -Input [2]: [d_date_sk#5, d_month_seq#23] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(29) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/simplified.txt deleted file mode 100644 index 0b24fe234a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/simplified.txt +++ /dev/null @@ -1,44 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (6) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 - WholeStageCodegen (3) - HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] - Expand [ws_net_paid,i_category,i_class] - Project [ws_net_paid,i_category,i_class] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan 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/q86.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/explain.txt deleted file mode 100644 index 54520c592a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,169 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometExchange (19) - +- CometHashAggregate (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometExpand (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] -Arguments: [ws_item_sk#1, ws_net_paid#2], [ws_item_sk#1, ws_net_paid#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#7, i_class#8, i_category#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Condition : isnotnull(i_item_sk#7) - -(11) CometProject -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Arguments: [i_item_sk#7, i_class#10, i_category#11], [i_item_sk#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#8, 50)) AS i_class#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#9, 50)) AS i_category#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [i_item_sk#7, i_class#10, i_category#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#1, ws_net_paid#2] -Right output [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_net_paid#2, i_category#11, i_class#10], [ws_net_paid#2, i_category#11, i_class#10] - -(15) CometExpand -Input [3]: [ws_net_paid#2, i_category#11, i_class#10] -Arguments: [[ws_net_paid#2, i_category#11, i_class#10, 0], [ws_net_paid#2, i_category#11, null, 1], [ws_net_paid#2, null, null, 3]], [ws_net_paid#2, i_category#12, i_class#13, spark_grouping_id#14] - -(16) CometHashAggregate -Input [4]: [ws_net_paid#2, i_category#12, i_class#13, spark_grouping_id#14] -Keys [3]: [i_category#12, i_class#13, spark_grouping_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] - -(17) CometExchange -Input [4]: [i_category#12, i_class#13, spark_grouping_id#14, sum#15] -Arguments: hashpartitioning(i_category#12, i_class#13, spark_grouping_id#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [4]: [i_category#12, i_class#13, spark_grouping_id#14, sum#15] -Keys [3]: [i_category#12, i_class#13, spark_grouping_id#14] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] - -(19) CometExchange -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -Arguments: hashpartitioning(_w1#19, _w2#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -Arguments: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20], [_w1#19 ASC NULLS FIRST, _w2#20 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] - -(21) CometColumnarToRow [codegen id : 1] -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] - -(22) Window -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -Arguments: [rank(_w0#18) windowspecdefinition(_w1#19, _w2#20, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#21], [_w1#19, _w2#20], [_w0#18 DESC NULLS LAST] - -(23) Project [codegen id : 2] -Output [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] -Input [8]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20, rank_within_parent#21] - -(24) TakeOrderedAndProject -Input [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] -Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#21 ASC NULLS FIRST], [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(27) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(29) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/simplified.txt deleted file mode 100644 index 29e2d72920..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (2) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometExchange [_w1,_w2] #1 - CometHashAggregate [sum] [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ws_net_paid)),spark_grouping_id] - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] - 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,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] 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/q86/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt deleted file mode 100644 index 54520c592a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt +++ /dev/null @@ -1,169 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometExchange (19) - +- CometHashAggregate (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometExpand (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] -Arguments: [ws_item_sk#1, ws_net_paid#2], [ws_item_sk#1, ws_net_paid#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#7, i_class#8, i_category#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Condition : isnotnull(i_item_sk#7) - -(11) CometProject -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Arguments: [i_item_sk#7, i_class#10, i_category#11], [i_item_sk#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#8, 50)) AS i_class#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#9, 50)) AS i_category#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [i_item_sk#7, i_class#10, i_category#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#1, ws_net_paid#2] -Right output [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_net_paid#2, i_category#11, i_class#10], [ws_net_paid#2, i_category#11, i_class#10] - -(15) CometExpand -Input [3]: [ws_net_paid#2, i_category#11, i_class#10] -Arguments: [[ws_net_paid#2, i_category#11, i_class#10, 0], [ws_net_paid#2, i_category#11, null, 1], [ws_net_paid#2, null, null, 3]], [ws_net_paid#2, i_category#12, i_class#13, spark_grouping_id#14] - -(16) CometHashAggregate -Input [4]: [ws_net_paid#2, i_category#12, i_class#13, spark_grouping_id#14] -Keys [3]: [i_category#12, i_class#13, spark_grouping_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] - -(17) CometExchange -Input [4]: [i_category#12, i_class#13, spark_grouping_id#14, sum#15] -Arguments: hashpartitioning(i_category#12, i_class#13, spark_grouping_id#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [4]: [i_category#12, i_class#13, spark_grouping_id#14, sum#15] -Keys [3]: [i_category#12, i_class#13, spark_grouping_id#14] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] - -(19) CometExchange -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -Arguments: hashpartitioning(_w1#19, _w2#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -Arguments: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20], [_w1#19 ASC NULLS FIRST, _w2#20 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] - -(21) CometColumnarToRow [codegen id : 1] -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] - -(22) Window -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -Arguments: [rank(_w0#18) windowspecdefinition(_w1#19, _w2#20, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#21], [_w1#19, _w2#20], [_w0#18 DESC NULLS LAST] - -(23) Project [codegen id : 2] -Output [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] -Input [8]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20, rank_within_parent#21] - -(24) TakeOrderedAndProject -Input [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] -Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#21 ASC NULLS FIRST], [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(27) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(29) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/extended.txt deleted file mode 100644 index dfc810b108..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/extended.txt +++ /dev/null @@ -1,32 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 29e2d72920..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (2) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometExchange [_w1,_w2] #1 - CometHashAggregate [sum] [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ws_net_paid)),spark_grouping_id] - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] - 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,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/explain.txt deleted file mode 100644 index 4742064612..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/explain.txt +++ /dev/null @@ -1,336 +0,0 @@ -== Physical Plan == -* HashAggregate (52) -+- * CometColumnarToRow (51) - +- CometColumnarExchange (50) - +- * HashAggregate (49) - +- * Project (48) - +- * BroadcastHashJoin LeftAnti BuildRight (47) - :- * BroadcastHashJoin LeftAnti BuildRight (32) - : :- * CometColumnarToRow (17) - : : +- CometHashAggregate (16) - : : +- CometColumnarExchange (15) - : : +- * HashAggregate (14) - : : +- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.customer (7) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometHashAggregate (29) - : +- CometColumnarExchange (28) - : +- * HashAggregate (27) - : +- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * Filter (20) - : : : +- * ColumnarToRow (19) - : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (21) - : +- ReusedExchange (24) - +- BroadcastExchange (46) - +- * CometColumnarToRow (45) - +- CometHashAggregate (44) - +- CometColumnarExchange (43) - +- * HashAggregate (42) - +- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Filter (35) - : : +- * ColumnarToRow (34) - : : +- Scan parquet spark_catalog.default.web_sales (33) - : +- ReusedExchange (36) - +- ReusedExchange (39) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sold_date_sk#2 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] - -(3) Filter [codegen id : 3] -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Condition : isnotnull(ss_customer_sk#1) - -(4) ReusedExchange [Reuses operator id: 57] -Output [2]: [d_date_sk#4, d_date#5] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#2] -Right keys [1]: [d_date_sk#4] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [2]: [ss_customer_sk#1, d_date#5] -Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] - -(7) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Condition : isnotnull(c_customer_sk#6) - -(9) CometProject -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Arguments: [c_customer_sk#6, c_first_name#9, c_last_name#10], [c_customer_sk#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#7, 20)) AS c_first_name#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#8, 30)) AS c_last_name#10] - -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] - -(11) BroadcastExchange -Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#6] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [3]: [c_last_name#10, c_first_name#9, d_date#5] -Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#6, c_first_name#9, c_last_name#10] - -(14) HashAggregate [codegen id : 3] -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#5] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#5] - -(15) CometColumnarExchange -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] -Arguments: hashpartitioning(c_last_name#10, c_first_name#9, d_date#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometHashAggregate -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#5] -Functions: [] - -(17) CometColumnarToRow [codegen id : 12] -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] - -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#12), dynamicpruningexpression(cs_sold_date_sk#12 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] - -(20) Filter [codegen id : 6] -Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] -Condition : isnotnull(cs_bill_customer_sk#11) - -(21) ReusedExchange [Reuses operator id: 57] -Output [2]: [d_date_sk#13, d_date#14] - -(22) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#12] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(23) Project [codegen id : 6] -Output [2]: [cs_bill_customer_sk#11, d_date#14] -Input [4]: [cs_bill_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13, d_date#14] - -(24) ReusedExchange [Reuses operator id: 11] -Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] - -(25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_bill_customer_sk#11] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 6] -Output [3]: [c_last_name#17, c_first_name#16, d_date#14] -Input [5]: [cs_bill_customer_sk#11, d_date#14, c_customer_sk#15, c_first_name#16, c_last_name#17] - -(27) HashAggregate [codegen id : 6] -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#17, c_first_name#16, d_date#14] - -(28) CometColumnarExchange -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, d_date#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(29) CometHashAggregate -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] -Functions: [] - -(30) CometColumnarToRow [codegen id : 7] -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] - -(31) BroadcastExchange -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#17, ), isnull(c_last_name#17), coalesce(c_first_name#16, ), isnull(c_first_name#16), coalesce(d_date#14, 1970-01-01), isnull(d_date#14)] -Join type: LeftAnti -Join condition: None - -(33) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_sold_date_sk#19 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 10] -Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] - -(35) Filter [codegen id : 10] -Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] -Condition : isnotnull(ws_bill_customer_sk#18) - -(36) ReusedExchange [Reuses operator id: 57] -Output [2]: [d_date_sk#20, d_date#21] - -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#19] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 10] -Output [2]: [ws_bill_customer_sk#18, d_date#21] -Input [4]: [ws_bill_customer_sk#18, ws_sold_date_sk#19, d_date_sk#20, d_date#21] - -(39) ReusedExchange [Reuses operator id: 11] -Output [3]: [c_customer_sk#22, c_first_name#23, c_last_name#24] - -(40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_bill_customer_sk#18] -Right keys [1]: [c_customer_sk#22] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 10] -Output [3]: [c_last_name#24, c_first_name#23, d_date#21] -Input [5]: [ws_bill_customer_sk#18, d_date#21, c_customer_sk#22, c_first_name#23, c_last_name#24] - -(42) HashAggregate [codegen id : 10] -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#24, c_first_name#23, d_date#21] - -(43) CometColumnarExchange -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, d_date#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(44) CometHashAggregate -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] -Functions: [] - -(45) CometColumnarToRow [codegen id : 11] -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] - -(46) BroadcastExchange -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=6] - -(47) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#24, ), isnull(c_last_name#24), coalesce(c_first_name#23, ), isnull(c_first_name#23), coalesce(d_date#21, 1970-01-01), isnull(d_date#21)] -Join type: LeftAnti -Join condition: None - -(48) Project [codegen id : 12] -Output: [] -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] - -(49) HashAggregate [codegen id : 12] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#25] -Results [1]: [count#26] - -(50) CometColumnarExchange -Input [1]: [count#26] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(51) CometColumnarToRow [codegen id : 13] -Input [1]: [count#26] - -(52) HashAggregate [codegen id : 13] -Input [1]: [count#26] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#27] -Results [1]: [count(1)#27 AS count(1)#28] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometNativeScan parquet spark_catalog.default.date_dim (53) - - -(53) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] -Condition : (((isnotnull(d_month_seq#29) AND (d_month_seq#29 >= 1200)) AND (d_month_seq#29 <= 1211)) AND isnotnull(d_date_sk#4)) - -(55) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(56) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] - -(57) BroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#12 IN dynamicpruning#3 - -Subquery:3 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#19 IN dynamicpruning#3 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/simplified.txt deleted file mode 100644 index afc302e779..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/simplified.txt +++ /dev/null @@ -1,86 +0,0 @@ -WholeStageCodegen (13) - HashAggregate [count] [count(1),count(1),count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (12) - HashAggregate [count,count] - Project - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/explain.txt deleted file mode 100644 index 3dd56d08f4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,327 +0,0 @@ -== Physical Plan == -* HashAggregate (51) -+- * CometColumnarToRow (50) - +- CometColumnarExchange (49) - +- * HashAggregate (48) - +- * Project (47) - +- * BroadcastHashJoin LeftAnti BuildRight (46) - :- * BroadcastHashJoin LeftAnti BuildRight (32) - : :- * CometColumnarToRow (18) - : : +- CometHashAggregate (17) - : : +- CometExchange (16) - : : +- CometHashAggregate (15) - : : +- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometHashAggregate (29) - : +- CometExchange (28) - : +- CometHashAggregate (27) - : +- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (19) - : : +- ReusedExchange (21) - : +- ReusedExchange (24) - +- BroadcastExchange (45) - +- * CometColumnarToRow (44) - +- CometHashAggregate (43) - +- CometExchange (42) - +- CometHashAggregate (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : +- ReusedExchange (35) - +- ReusedExchange (38) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sold_date_sk#2 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Condition : isnotnull(ss_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(5) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: [d_date_sk#4, d_date#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Right output [2]: [d_date_sk#4, d_date#5] -Arguments: [ss_sold_date_sk#2], [d_date_sk#4], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] -Arguments: [ss_customer_sk#1, d_date#5], [ss_customer_sk#1, d_date#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Condition : isnotnull(c_customer_sk#7) - -(11) CometProject -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11], [c_customer_sk#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#8, 20)) AS c_first_name#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#9, 30)) AS c_last_name#11] - -(12) CometBroadcastExchange -Input [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, d_date#5] -Right output [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [ss_customer_sk#1], [c_customer_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_last_name#11, c_first_name#10, d_date#5], [c_last_name#11, c_first_name#10, d_date#5] - -(15) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(16) CometExchange -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(c_last_name#11, c_first_name#10, d_date#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(18) CometColumnarToRow [codegen id : 3] -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Condition : isnotnull(cs_bill_customer_sk#12) - -(21) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#15, d_date#16] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_date#16] -Arguments: [cs_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(23) CometProject -Input [4]: [cs_bill_customer_sk#12, cs_sold_date_sk#13, d_date_sk#15, d_date#16] -Arguments: [cs_bill_customer_sk#12, d_date#16], [cs_bill_customer_sk#12, d_date#16] - -(24) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] - -(25) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, d_date#16] -Right output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [cs_bill_customer_sk#12], [c_customer_sk#17], Inner, BuildRight - -(26) CometProject -Input [5]: [cs_bill_customer_sk#12, d_date#16, c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [c_last_name#19, c_first_name#18, d_date#16], [c_last_name#19, c_first_name#18, d_date#16] - -(27) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(28) CometExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: hashpartitioning(c_last_name#19, c_first_name#18, d_date#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(30) CometColumnarToRow [codegen id : 1] -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] - -(31) BroadcastExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=3] - -(32) BroadcastHashJoin [codegen id : 3] -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 1970-01-01), isnull(d_date#16)] -Join type: LeftAnti -Join condition: None - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#21), dynamicpruningexpression(ws_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Condition : isnotnull(ws_bill_customer_sk#20) - -(35) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#23, d_date#24] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Right output [2]: [d_date_sk#23, d_date#24] -Arguments: [ws_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(37) CometProject -Input [4]: [ws_bill_customer_sk#20, ws_sold_date_sk#21, d_date_sk#23, d_date#24] -Arguments: [ws_bill_customer_sk#20, d_date#24], [ws_bill_customer_sk#20, d_date#24] - -(38) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] - -(39) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, d_date#24] -Right output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [ws_bill_customer_sk#20], [c_customer_sk#25], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_bill_customer_sk#20, d_date#24, c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_last_name#27, c_first_name#26, d_date#24], [c_last_name#27, c_first_name#26, d_date#24] - -(41) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(42) CometExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: hashpartitioning(c_last_name#27, c_first_name#26, d_date#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(43) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(44) CometColumnarToRow [codegen id : 2] -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] - -(45) BroadcastExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 3] -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#27, ), isnull(c_last_name#27), coalesce(c_first_name#26, ), isnull(c_first_name#26), coalesce(d_date#24, 1970-01-01), isnull(d_date#24)] -Join type: LeftAnti -Join condition: None - -(47) Project [codegen id : 3] -Output: [] -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] - -(48) HashAggregate [codegen id : 3] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#28] -Results [1]: [count#29] - -(49) CometColumnarExchange -Input [1]: [count#29] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(50) CometColumnarToRow [codegen id : 4] -Input [1]: [count#29] - -(51) HashAggregate [codegen id : 4] -Input [1]: [count#29] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#30] -Results [1]: [count(1)#30 AS count(1)#31] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (56) -+- * CometColumnarToRow (55) - +- CometProject (54) - +- CometFilter (53) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) - - -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(54) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(55) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] - -(56) BroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#3 - -Subquery:3 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#21 IN dynamicpruning#3 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/simplified.txt deleted file mode 100644 index f687139735..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -WholeStageCodegen (4) - HashAggregate [count] [count(1),count(1),count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (3) - HashAggregate [count,count] - Project - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #2 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] - CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_customer_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #4 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #7 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] - CometFilter [cs_bill_customer_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #9 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_bill_customer_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/explain.txt deleted file mode 100644 index 3dd56d08f4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/explain.txt +++ /dev/null @@ -1,327 +0,0 @@ -== Physical Plan == -* HashAggregate (51) -+- * CometColumnarToRow (50) - +- CometColumnarExchange (49) - +- * HashAggregate (48) - +- * Project (47) - +- * BroadcastHashJoin LeftAnti BuildRight (46) - :- * BroadcastHashJoin LeftAnti BuildRight (32) - : :- * CometColumnarToRow (18) - : : +- CometHashAggregate (17) - : : +- CometExchange (16) - : : +- CometHashAggregate (15) - : : +- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometHashAggregate (29) - : +- CometExchange (28) - : +- CometHashAggregate (27) - : +- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (19) - : : +- ReusedExchange (21) - : +- ReusedExchange (24) - +- BroadcastExchange (45) - +- * CometColumnarToRow (44) - +- CometHashAggregate (43) - +- CometExchange (42) - +- CometHashAggregate (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : +- ReusedExchange (35) - +- ReusedExchange (38) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sold_date_sk#2 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Condition : isnotnull(ss_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(5) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: [d_date_sk#4, d_date#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Right output [2]: [d_date_sk#4, d_date#5] -Arguments: [ss_sold_date_sk#2], [d_date_sk#4], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] -Arguments: [ss_customer_sk#1, d_date#5], [ss_customer_sk#1, d_date#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Condition : isnotnull(c_customer_sk#7) - -(11) CometProject -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11], [c_customer_sk#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#8, 20)) AS c_first_name#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#9, 30)) AS c_last_name#11] - -(12) CometBroadcastExchange -Input [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, d_date#5] -Right output [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [ss_customer_sk#1], [c_customer_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_last_name#11, c_first_name#10, d_date#5], [c_last_name#11, c_first_name#10, d_date#5] - -(15) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(16) CometExchange -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(c_last_name#11, c_first_name#10, d_date#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(18) CometColumnarToRow [codegen id : 3] -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Condition : isnotnull(cs_bill_customer_sk#12) - -(21) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#15, d_date#16] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_date#16] -Arguments: [cs_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(23) CometProject -Input [4]: [cs_bill_customer_sk#12, cs_sold_date_sk#13, d_date_sk#15, d_date#16] -Arguments: [cs_bill_customer_sk#12, d_date#16], [cs_bill_customer_sk#12, d_date#16] - -(24) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] - -(25) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, d_date#16] -Right output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [cs_bill_customer_sk#12], [c_customer_sk#17], Inner, BuildRight - -(26) CometProject -Input [5]: [cs_bill_customer_sk#12, d_date#16, c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [c_last_name#19, c_first_name#18, d_date#16], [c_last_name#19, c_first_name#18, d_date#16] - -(27) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(28) CometExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: hashpartitioning(c_last_name#19, c_first_name#18, d_date#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(30) CometColumnarToRow [codegen id : 1] -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] - -(31) BroadcastExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=3] - -(32) BroadcastHashJoin [codegen id : 3] -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 1970-01-01), isnull(d_date#16)] -Join type: LeftAnti -Join condition: None - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#21), dynamicpruningexpression(ws_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Condition : isnotnull(ws_bill_customer_sk#20) - -(35) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#23, d_date#24] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Right output [2]: [d_date_sk#23, d_date#24] -Arguments: [ws_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(37) CometProject -Input [4]: [ws_bill_customer_sk#20, ws_sold_date_sk#21, d_date_sk#23, d_date#24] -Arguments: [ws_bill_customer_sk#20, d_date#24], [ws_bill_customer_sk#20, d_date#24] - -(38) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] - -(39) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, d_date#24] -Right output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [ws_bill_customer_sk#20], [c_customer_sk#25], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_bill_customer_sk#20, d_date#24, c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_last_name#27, c_first_name#26, d_date#24], [c_last_name#27, c_first_name#26, d_date#24] - -(41) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(42) CometExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: hashpartitioning(c_last_name#27, c_first_name#26, d_date#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(43) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(44) CometColumnarToRow [codegen id : 2] -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] - -(45) BroadcastExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 3] -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#27, ), isnull(c_last_name#27), coalesce(c_first_name#26, ), isnull(c_first_name#26), coalesce(d_date#24, 1970-01-01), isnull(d_date#24)] -Join type: LeftAnti -Join condition: None - -(47) Project [codegen id : 3] -Output: [] -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] - -(48) HashAggregate [codegen id : 3] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#28] -Results [1]: [count#29] - -(49) CometColumnarExchange -Input [1]: [count#29] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(50) CometColumnarToRow [codegen id : 4] -Input [1]: [count#29] - -(51) HashAggregate [codegen id : 4] -Input [1]: [count#29] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#30] -Results [1]: [count(1)#30 AS count(1)#31] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (56) -+- * CometColumnarToRow (55) - +- CometProject (54) - +- CometFilter (53) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) - - -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(54) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(55) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] - -(56) BroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#3 - -Subquery:3 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#21 IN dynamicpruning#3 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/extended.txt deleted file mode 100644 index ddee139acf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/extended.txt +++ /dev/null @@ -1,73 +0,0 @@ -HashAggregate -+- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] - : :- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 55 out of 66 eligible operators (83%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index f687139735..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -WholeStageCodegen (4) - HashAggregate [count] [count(1),count(1),count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (3) - HashAggregate [count,count] - Project - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #2 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] - CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_customer_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #4 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #7 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] - CometFilter [cs_bill_customer_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #9 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_bill_customer_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_datafusion/explain.txt deleted file mode 100644 index 25b4c305f2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_datafusion/explain.txt +++ /dev/null @@ -1,927 +0,0 @@ -== Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (172) -:- * BroadcastNestedLoopJoin Inner BuildRight (151) -: :- * BroadcastNestedLoopJoin Inner BuildRight (130) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (109) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) -: : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) -: : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : : : : : :- * CometColumnarToRow (25) -: : : : : : : +- CometHashAggregate (24) -: : : : : : : +- CometExchange (23) -: : : : : : : +- CometHashAggregate (22) -: : : : : : : +- CometProject (21) -: : : : : : : +- CometBroadcastHashJoin (20) -: : : : : : : :- CometProject (15) -: : : : : : : : +- CometBroadcastHashJoin (14) -: : : : : : : : :- CometProject (9) -: : : : : : : : : +- CometBroadcastHashJoin (8) -: : : : : : : : : :- CometProject (3) -: : : : : : : : : : +- CometFilter (2) -: : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) -: : : : : : : : : +- CometBroadcastExchange (7) -: : : : : : : : : +- CometProject (6) -: : : : : : : : : +- CometFilter (5) -: : : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (4) -: : : : : : : : +- CometBroadcastExchange (13) -: : : : : : : : +- CometProject (12) -: : : : : : : : +- CometFilter (11) -: : : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (10) -: : : : : : : +- CometBroadcastExchange (19) -: : : : : : : +- CometProject (18) -: : : : : : : +- CometFilter (17) -: : : : : : : +- CometNativeScan parquet spark_catalog.default.store (16) -: : : : : : +- BroadcastExchange (45) -: : : : : : +- * CometColumnarToRow (44) -: : : : : : +- CometHashAggregate (43) -: : : : : : +- CometExchange (42) -: : : : : : +- CometHashAggregate (41) -: : : : : : +- CometProject (40) -: : : : : : +- CometBroadcastHashJoin (39) -: : : : : : :- CometProject (37) -: : : : : : : +- CometBroadcastHashJoin (36) -: : : : : : : :- CometProject (31) -: : : : : : : : +- CometBroadcastHashJoin (30) -: : : : : : : : :- CometProject (28) -: : : : : : : : : +- CometFilter (27) -: : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (26) -: : : : : : : : +- ReusedExchange (29) -: : : : : : : +- CometBroadcastExchange (35) -: : : : : : : +- CometProject (34) -: : : : : : : +- CometFilter (33) -: : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (32) -: : : : : : +- ReusedExchange (38) -: : : : : +- BroadcastExchange (66) -: : : : : +- * CometColumnarToRow (65) -: : : : : +- CometHashAggregate (64) -: : : : : +- CometExchange (63) -: : : : : +- CometHashAggregate (62) -: : : : : +- CometProject (61) -: : : : : +- CometBroadcastHashJoin (60) -: : : : : :- CometProject (58) -: : : : : : +- CometBroadcastHashJoin (57) -: : : : : : :- CometProject (52) -: : : : : : : +- CometBroadcastHashJoin (51) -: : : : : : : :- CometProject (49) -: : : : : : : : +- CometFilter (48) -: : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (47) -: : : : : : : +- ReusedExchange (50) -: : : : : : +- CometBroadcastExchange (56) -: : : : : : +- CometProject (55) -: : : : : : +- CometFilter (54) -: : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (53) -: : : : : +- ReusedExchange (59) -: : : : +- BroadcastExchange (87) -: : : : +- * CometColumnarToRow (86) -: : : : +- CometHashAggregate (85) -: : : : +- CometExchange (84) -: : : : +- CometHashAggregate (83) -: : : : +- CometProject (82) -: : : : +- CometBroadcastHashJoin (81) -: : : : :- CometProject (79) -: : : : : +- CometBroadcastHashJoin (78) -: : : : : :- CometProject (73) -: : : : : : +- CometBroadcastHashJoin (72) -: : : : : : :- CometProject (70) -: : : : : : : +- CometFilter (69) -: : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (68) -: : : : : : +- ReusedExchange (71) -: : : : : +- CometBroadcastExchange (77) -: : : : : +- CometProject (76) -: : : : : +- CometFilter (75) -: : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (74) -: : : : +- ReusedExchange (80) -: : : +- BroadcastExchange (108) -: : : +- * CometColumnarToRow (107) -: : : +- CometHashAggregate (106) -: : : +- CometExchange (105) -: : : +- CometHashAggregate (104) -: : : +- CometProject (103) -: : : +- CometBroadcastHashJoin (102) -: : : :- CometProject (100) -: : : : +- CometBroadcastHashJoin (99) -: : : : :- CometProject (94) -: : : : : +- CometBroadcastHashJoin (93) -: : : : : :- CometProject (91) -: : : : : : +- CometFilter (90) -: : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (89) -: : : : : +- ReusedExchange (92) -: : : : +- CometBroadcastExchange (98) -: : : : +- CometProject (97) -: : : : +- CometFilter (96) -: : : : +- CometNativeScan parquet spark_catalog.default.time_dim (95) -: : : +- ReusedExchange (101) -: : +- BroadcastExchange (129) -: : +- * CometColumnarToRow (128) -: : +- CometHashAggregate (127) -: : +- CometExchange (126) -: : +- CometHashAggregate (125) -: : +- CometProject (124) -: : +- CometBroadcastHashJoin (123) -: : :- CometProject (121) -: : : +- CometBroadcastHashJoin (120) -: : : :- CometProject (115) -: : : : +- CometBroadcastHashJoin (114) -: : : : :- CometProject (112) -: : : : : +- CometFilter (111) -: : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (110) -: : : : +- ReusedExchange (113) -: : : +- CometBroadcastExchange (119) -: : : +- CometProject (118) -: : : +- CometFilter (117) -: : : +- CometNativeScan parquet spark_catalog.default.time_dim (116) -: : +- ReusedExchange (122) -: +- BroadcastExchange (150) -: +- * CometColumnarToRow (149) -: +- CometHashAggregate (148) -: +- CometExchange (147) -: +- CometHashAggregate (146) -: +- CometProject (145) -: +- CometBroadcastHashJoin (144) -: :- CometProject (142) -: : +- CometBroadcastHashJoin (141) -: : :- CometProject (136) -: : : +- CometBroadcastHashJoin (135) -: : : :- CometProject (133) -: : : : +- CometFilter (132) -: : : : +- CometNativeScan parquet spark_catalog.default.store_sales (131) -: : : +- ReusedExchange (134) -: : +- CometBroadcastExchange (140) -: : +- CometProject (139) -: : +- CometFilter (138) -: : +- CometNativeScan parquet spark_catalog.default.time_dim (137) -: +- ReusedExchange (143) -+- BroadcastExchange (171) - +- * CometColumnarToRow (170) - +- CometHashAggregate (169) - +- CometExchange (168) - +- CometHashAggregate (167) - +- CometProject (166) - +- CometBroadcastHashJoin (165) - :- CometProject (163) - : +- CometBroadcastHashJoin (162) - : :- CometProject (157) - : : +- CometBroadcastHashJoin (156) - : : :- CometProject (154) - : : : +- CometFilter (153) - : : : +- CometNativeScan parquet spark_catalog.default.store_sales (152) - : : +- ReusedExchange (155) - : +- CometBroadcastExchange (161) - : +- CometProject (160) - : +- CometFilter (159) - : +- CometNativeScan parquet spark_catalog.default.time_dim (158) - +- ReusedExchange (164) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(3) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(4) CometNativeScan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(Or(And(EqualTo(hd_dep_count,4),LessThanOrEqual(hd_vehicle_count,6)),And(EqualTo(hd_dep_count,2),LessThanOrEqual(hd_vehicle_count,4))),And(EqualTo(hd_dep_count,0),LessThanOrEqual(hd_vehicle_count,2))), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Condition : (((((hd_dep_count#6 = 4) AND (hd_vehicle_count#7 <= 6)) OR ((hd_dep_count#6 = 2) AND (hd_vehicle_count#7 <= 4))) OR ((hd_dep_count#6 = 0) AND (hd_vehicle_count#7 <= 2))) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] -Arguments: [ss_sold_time_sk#1, ss_store_sk#3], [ss_sold_time_sk#1, ss_store_sk#3] - -(10) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,8), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Condition : ((((isnotnull(t_hour#9) AND isnotnull(t_minute#10)) AND (t_hour#9 = 8)) AND (t_minute#10 >= 30)) AND isnotnull(t_time_sk#8)) - -(12) CometProject -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Arguments: [t_time_sk#8], [t_time_sk#8] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#8] -Arguments: [t_time_sk#8] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Right output [1]: [t_time_sk#8] -Arguments: [ss_sold_time_sk#1], [t_time_sk#8], Inner, BuildRight - -(15) CometProject -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#8] -Arguments: [ss_store_sk#3], [ss_store_sk#3] - -(16) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_store_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [s_store_sk#11, s_store_name#12] -Condition : ((isnotnull(s_store_name#12) AND (s_store_name#12 = ese)) AND isnotnull(s_store_sk#11)) - -(18) CometProject -Input [2]: [s_store_sk#11, s_store_name#12] -Arguments: [s_store_sk#11], [s_store_sk#11] - -(19) CometBroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: [s_store_sk#11] - -(20) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#3] -Right output [1]: [s_store_sk#11] -Arguments: [ss_store_sk#3], [s_store_sk#11], Inner, BuildRight - -(21) CometProject -Input [2]: [ss_store_sk#3, s_store_sk#11] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#13] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#13] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 8] -Input [1]: [h8_30_to_9#14] - -(26) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Condition : ((isnotnull(ss_hdemo_sk#16) AND isnotnull(ss_sold_time_sk#15)) AND isnotnull(ss_store_sk#17)) - -(28) CometProject -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Arguments: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17], [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] - -(29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#19] - -(30) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] -Right output [1]: [hd_demo_sk#19] -Arguments: [ss_hdemo_sk#16], [hd_demo_sk#19], Inner, BuildRight - -(31) CometProject -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, hd_demo_sk#19] -Arguments: [ss_sold_time_sk#15, ss_store_sk#17], [ss_sold_time_sk#15, ss_store_sk#17] - -(32) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Condition : ((((isnotnull(t_hour#21) AND isnotnull(t_minute#22)) AND (t_hour#21 = 9)) AND (t_minute#22 < 30)) AND isnotnull(t_time_sk#20)) - -(34) CometProject -Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Arguments: [t_time_sk#20], [t_time_sk#20] - -(35) CometBroadcastExchange -Input [1]: [t_time_sk#20] -Arguments: [t_time_sk#20] - -(36) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#15, ss_store_sk#17] -Right output [1]: [t_time_sk#20] -Arguments: [ss_sold_time_sk#15], [t_time_sk#20], Inner, BuildRight - -(37) CometProject -Input [3]: [ss_sold_time_sk#15, ss_store_sk#17, t_time_sk#20] -Arguments: [ss_store_sk#17], [ss_store_sk#17] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#23] - -(39) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#17] -Right output [1]: [s_store_sk#23] -Arguments: [ss_store_sk#17], [s_store_sk#23], Inner, BuildRight - -(40) CometProject -Input [2]: [ss_store_sk#17, s_store_sk#23] - -(41) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(42) CometExchange -Input [1]: [count#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(43) CometHashAggregate -Input [1]: [count#24] -Keys: [] -Functions [1]: [count(1)] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [h9_to_9_30#25] - -(45) BroadcastExchange -Input [1]: [h9_to_9_30#25] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(46) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(47) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(48) CometFilter -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_hdemo_sk#27) AND isnotnull(ss_sold_time_sk#26)) AND isnotnull(ss_store_sk#28)) - -(49) CometProject -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Arguments: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28], [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] - -(50) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#30] - -(51) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] -Right output [1]: [hd_demo_sk#30] -Arguments: [ss_hdemo_sk#27], [hd_demo_sk#30], Inner, BuildRight - -(52) CometProject -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, hd_demo_sk#30] -Arguments: [ss_sold_time_sk#26, ss_store_sk#28], [ss_sold_time_sk#26, ss_store_sk#28] - -(53) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Condition : ((((isnotnull(t_hour#32) AND isnotnull(t_minute#33)) AND (t_hour#32 = 9)) AND (t_minute#33 >= 30)) AND isnotnull(t_time_sk#31)) - -(55) CometProject -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Arguments: [t_time_sk#31], [t_time_sk#31] - -(56) CometBroadcastExchange -Input [1]: [t_time_sk#31] -Arguments: [t_time_sk#31] - -(57) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#26, ss_store_sk#28] -Right output [1]: [t_time_sk#31] -Arguments: [ss_sold_time_sk#26], [t_time_sk#31], Inner, BuildRight - -(58) CometProject -Input [3]: [ss_sold_time_sk#26, ss_store_sk#28, t_time_sk#31] -Arguments: [ss_store_sk#28], [ss_store_sk#28] - -(59) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#34] - -(60) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#28] -Right output [1]: [s_store_sk#34] -Arguments: [ss_store_sk#28], [s_store_sk#34], Inner, BuildRight - -(61) CometProject -Input [2]: [ss_store_sk#28, s_store_sk#34] - -(62) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(63) CometExchange -Input [1]: [count#35] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(64) CometHashAggregate -Input [1]: [count#35] -Keys: [] -Functions [1]: [count(1)] - -(65) CometColumnarToRow [codegen id : 2] -Input [1]: [h9_30_to_10#36] - -(66) BroadcastExchange -Input [1]: [h9_30_to_10#36] -Arguments: IdentityBroadcastMode, [plan_id=5] - -(67) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(68) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(69) CometFilter -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Condition : ((isnotnull(ss_hdemo_sk#38) AND isnotnull(ss_sold_time_sk#37)) AND isnotnull(ss_store_sk#39)) - -(70) CometProject -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Arguments: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39], [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] - -(71) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#41] - -(72) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] -Right output [1]: [hd_demo_sk#41] -Arguments: [ss_hdemo_sk#38], [hd_demo_sk#41], Inner, BuildRight - -(73) CometProject -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, hd_demo_sk#41] -Arguments: [ss_sold_time_sk#37, ss_store_sk#39], [ss_sold_time_sk#37, ss_store_sk#39] - -(74) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(75) CometFilter -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Condition : ((((isnotnull(t_hour#43) AND isnotnull(t_minute#44)) AND (t_hour#43 = 10)) AND (t_minute#44 < 30)) AND isnotnull(t_time_sk#42)) - -(76) CometProject -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Arguments: [t_time_sk#42], [t_time_sk#42] - -(77) CometBroadcastExchange -Input [1]: [t_time_sk#42] -Arguments: [t_time_sk#42] - -(78) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#37, ss_store_sk#39] -Right output [1]: [t_time_sk#42] -Arguments: [ss_sold_time_sk#37], [t_time_sk#42], Inner, BuildRight - -(79) CometProject -Input [3]: [ss_sold_time_sk#37, ss_store_sk#39, t_time_sk#42] -Arguments: [ss_store_sk#39], [ss_store_sk#39] - -(80) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#45] - -(81) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#39] -Right output [1]: [s_store_sk#45] -Arguments: [ss_store_sk#39], [s_store_sk#45], Inner, BuildRight - -(82) CometProject -Input [2]: [ss_store_sk#39, s_store_sk#45] - -(83) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(84) CometExchange -Input [1]: [count#46] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(85) CometHashAggregate -Input [1]: [count#46] -Keys: [] -Functions [1]: [count(1)] - -(86) CometColumnarToRow [codegen id : 3] -Input [1]: [h10_to_10_30#47] - -(87) BroadcastExchange -Input [1]: [h10_to_10_30#47] -Arguments: IdentityBroadcastMode, [plan_id=7] - -(88) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(89) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(90) CometFilter -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Condition : ((isnotnull(ss_hdemo_sk#49) AND isnotnull(ss_sold_time_sk#48)) AND isnotnull(ss_store_sk#50)) - -(91) CometProject -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Arguments: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50], [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] - -(92) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#52] - -(93) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] -Right output [1]: [hd_demo_sk#52] -Arguments: [ss_hdemo_sk#49], [hd_demo_sk#52], Inner, BuildRight - -(94) CometProject -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, hd_demo_sk#52] -Arguments: [ss_sold_time_sk#48, ss_store_sk#50], [ss_sold_time_sk#48, ss_store_sk#50] - -(95) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(96) CometFilter -Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Condition : ((((isnotnull(t_hour#54) AND isnotnull(t_minute#55)) AND (t_hour#54 = 10)) AND (t_minute#55 >= 30)) AND isnotnull(t_time_sk#53)) - -(97) CometProject -Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Arguments: [t_time_sk#53], [t_time_sk#53] - -(98) CometBroadcastExchange -Input [1]: [t_time_sk#53] -Arguments: [t_time_sk#53] - -(99) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#48, ss_store_sk#50] -Right output [1]: [t_time_sk#53] -Arguments: [ss_sold_time_sk#48], [t_time_sk#53], Inner, BuildRight - -(100) CometProject -Input [3]: [ss_sold_time_sk#48, ss_store_sk#50, t_time_sk#53] -Arguments: [ss_store_sk#50], [ss_store_sk#50] - -(101) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#56] - -(102) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#50] -Right output [1]: [s_store_sk#56] -Arguments: [ss_store_sk#50], [s_store_sk#56], Inner, BuildRight - -(103) CometProject -Input [2]: [ss_store_sk#50, s_store_sk#56] - -(104) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(105) CometExchange -Input [1]: [count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(106) CometHashAggregate -Input [1]: [count#57] -Keys: [] -Functions [1]: [count(1)] - -(107) CometColumnarToRow [codegen id : 4] -Input [1]: [h10_30_to_11#58] - -(108) BroadcastExchange -Input [1]: [h10_30_to_11#58] -Arguments: IdentityBroadcastMode, [plan_id=9] - -(109) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(110) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(111) CometFilter -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Condition : ((isnotnull(ss_hdemo_sk#60) AND isnotnull(ss_sold_time_sk#59)) AND isnotnull(ss_store_sk#61)) - -(112) CometProject -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Arguments: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61], [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] - -(113) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#63] - -(114) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] -Right output [1]: [hd_demo_sk#63] -Arguments: [ss_hdemo_sk#60], [hd_demo_sk#63], Inner, BuildRight - -(115) CometProject -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, hd_demo_sk#63] -Arguments: [ss_sold_time_sk#59, ss_store_sk#61], [ss_sold_time_sk#59, ss_store_sk#61] - -(116) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(117) CometFilter -Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Condition : ((((isnotnull(t_hour#65) AND isnotnull(t_minute#66)) AND (t_hour#65 = 11)) AND (t_minute#66 < 30)) AND isnotnull(t_time_sk#64)) - -(118) CometProject -Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Arguments: [t_time_sk#64], [t_time_sk#64] - -(119) CometBroadcastExchange -Input [1]: [t_time_sk#64] -Arguments: [t_time_sk#64] - -(120) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#59, ss_store_sk#61] -Right output [1]: [t_time_sk#64] -Arguments: [ss_sold_time_sk#59], [t_time_sk#64], Inner, BuildRight - -(121) CometProject -Input [3]: [ss_sold_time_sk#59, ss_store_sk#61, t_time_sk#64] -Arguments: [ss_store_sk#61], [ss_store_sk#61] - -(122) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#67] - -(123) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#61] -Right output [1]: [s_store_sk#67] -Arguments: [ss_store_sk#61], [s_store_sk#67], Inner, BuildRight - -(124) CometProject -Input [2]: [ss_store_sk#61, s_store_sk#67] - -(125) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(126) CometExchange -Input [1]: [count#68] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(127) CometHashAggregate -Input [1]: [count#68] -Keys: [] -Functions [1]: [count(1)] - -(128) CometColumnarToRow [codegen id : 5] -Input [1]: [h11_to_11_30#69] - -(129) BroadcastExchange -Input [1]: [h11_to_11_30#69] -Arguments: IdentityBroadcastMode, [plan_id=11] - -(130) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(131) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(132) CometFilter -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Condition : ((isnotnull(ss_hdemo_sk#71) AND isnotnull(ss_sold_time_sk#70)) AND isnotnull(ss_store_sk#72)) - -(133) CometProject -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Arguments: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72], [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] - -(134) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#74] - -(135) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] -Right output [1]: [hd_demo_sk#74] -Arguments: [ss_hdemo_sk#71], [hd_demo_sk#74], Inner, BuildRight - -(136) CometProject -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, hd_demo_sk#74] -Arguments: [ss_sold_time_sk#70, ss_store_sk#72], [ss_sold_time_sk#70, ss_store_sk#72] - -(137) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(138) CometFilter -Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Condition : ((((isnotnull(t_hour#76) AND isnotnull(t_minute#77)) AND (t_hour#76 = 11)) AND (t_minute#77 >= 30)) AND isnotnull(t_time_sk#75)) - -(139) CometProject -Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Arguments: [t_time_sk#75], [t_time_sk#75] - -(140) CometBroadcastExchange -Input [1]: [t_time_sk#75] -Arguments: [t_time_sk#75] - -(141) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#70, ss_store_sk#72] -Right output [1]: [t_time_sk#75] -Arguments: [ss_sold_time_sk#70], [t_time_sk#75], Inner, BuildRight - -(142) CometProject -Input [3]: [ss_sold_time_sk#70, ss_store_sk#72, t_time_sk#75] -Arguments: [ss_store_sk#72], [ss_store_sk#72] - -(143) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#78] - -(144) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#72] -Right output [1]: [s_store_sk#78] -Arguments: [ss_store_sk#72], [s_store_sk#78], Inner, BuildRight - -(145) CometProject -Input [2]: [ss_store_sk#72, s_store_sk#78] - -(146) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(147) CometExchange -Input [1]: [count#79] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(148) CometHashAggregate -Input [1]: [count#79] -Keys: [] -Functions [1]: [count(1)] - -(149) CometColumnarToRow [codegen id : 6] -Input [1]: [h11_30_to_12#80] - -(150) BroadcastExchange -Input [1]: [h11_30_to_12#80] -Arguments: IdentityBroadcastMode, [plan_id=13] - -(151) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(152) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(153) CometFilter -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Condition : ((isnotnull(ss_hdemo_sk#82) AND isnotnull(ss_sold_time_sk#81)) AND isnotnull(ss_store_sk#83)) - -(154) CometProject -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Arguments: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83], [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] - -(155) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#85] - -(156) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] -Right output [1]: [hd_demo_sk#85] -Arguments: [ss_hdemo_sk#82], [hd_demo_sk#85], Inner, BuildRight - -(157) CometProject -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, hd_demo_sk#85] -Arguments: [ss_sold_time_sk#81, ss_store_sk#83], [ss_sold_time_sk#81, ss_store_sk#83] - -(158) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(159) CometFilter -Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Condition : ((((isnotnull(t_hour#87) AND isnotnull(t_minute#88)) AND (t_hour#87 = 12)) AND (t_minute#88 < 30)) AND isnotnull(t_time_sk#86)) - -(160) CometProject -Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Arguments: [t_time_sk#86], [t_time_sk#86] - -(161) CometBroadcastExchange -Input [1]: [t_time_sk#86] -Arguments: [t_time_sk#86] - -(162) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#81, ss_store_sk#83] -Right output [1]: [t_time_sk#86] -Arguments: [ss_sold_time_sk#81], [t_time_sk#86], Inner, BuildRight - -(163) CometProject -Input [3]: [ss_sold_time_sk#81, ss_store_sk#83, t_time_sk#86] -Arguments: [ss_store_sk#83], [ss_store_sk#83] - -(164) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#89] - -(165) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#83] -Right output [1]: [s_store_sk#89] -Arguments: [ss_store_sk#83], [s_store_sk#89], Inner, BuildRight - -(166) CometProject -Input [2]: [ss_store_sk#83, s_store_sk#89] - -(167) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(168) CometExchange -Input [1]: [count#90] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(169) CometHashAggregate -Input [1]: [count#90] -Keys: [] -Functions [1]: [count(1)] - -(170) CometColumnarToRow [codegen id : 7] -Input [1]: [h12_to_12_30#91] - -(171) BroadcastExchange -Input [1]: [h12_to_12_30#91] -Arguments: IdentityBroadcastMode, [plan_id=15] - -(172) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_datafusion/simplified.txt deleted file mode 100644 index cef8e0e760..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_datafusion/simplified.txt +++ /dev/null @@ -1,195 +0,0 @@ -WholeStageCodegen (8) - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h8_30_to_9,count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_to_9_30,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan 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 [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_30_to_10,count(1)] - CometExchange #9 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan 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 [t_time_sk] #10 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_to_10_30,count(1)] - CometExchange #12 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan 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 [t_time_sk] #13 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_30_to_11,count(1)] - CometExchange #15 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan 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 [t_time_sk] #16 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_to_11_30,count(1)] - CometExchange #18 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan 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 [t_time_sk] #19 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_30_to_12,count(1)] - CometExchange #21 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan 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 [t_time_sk] #22 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #23 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h12_to_12_30,count(1)] - CometExchange #24 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan 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 [t_time_sk] #25 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan 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/q88.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_iceberg_compat/explain.txt deleted file mode 100644 index 2e49d20af4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,927 +0,0 @@ -== Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (172) -:- * BroadcastNestedLoopJoin Inner BuildRight (151) -: :- * BroadcastNestedLoopJoin Inner BuildRight (130) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (109) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) -: : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) -: : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : : : : : :- * CometColumnarToRow (25) -: : : : : : : +- CometHashAggregate (24) -: : : : : : : +- CometExchange (23) -: : : : : : : +- CometHashAggregate (22) -: : : : : : : +- CometProject (21) -: : : : : : : +- CometBroadcastHashJoin (20) -: : : : : : : :- CometProject (15) -: : : : : : : : +- CometBroadcastHashJoin (14) -: : : : : : : : :- CometProject (9) -: : : : : : : : : +- CometBroadcastHashJoin (8) -: : : : : : : : : :- CometProject (3) -: : : : : : : : : : +- CometFilter (2) -: : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) -: : : : : : : : : +- CometBroadcastExchange (7) -: : : : : : : : : +- CometProject (6) -: : : : : : : : : +- CometFilter (5) -: : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (4) -: : : : : : : : +- CometBroadcastExchange (13) -: : : : : : : : +- CometProject (12) -: : : : : : : : +- CometFilter (11) -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (10) -: : : : : : : +- CometBroadcastExchange (19) -: : : : : : : +- CometProject (18) -: : : : : : : +- CometFilter (17) -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (16) -: : : : : : +- BroadcastExchange (45) -: : : : : : +- * CometColumnarToRow (44) -: : : : : : +- CometHashAggregate (43) -: : : : : : +- CometExchange (42) -: : : : : : +- CometHashAggregate (41) -: : : : : : +- CometProject (40) -: : : : : : +- CometBroadcastHashJoin (39) -: : : : : : :- CometProject (37) -: : : : : : : +- CometBroadcastHashJoin (36) -: : : : : : : :- CometProject (31) -: : : : : : : : +- CometBroadcastHashJoin (30) -: : : : : : : : :- CometProject (28) -: : : : : : : : : +- CometFilter (27) -: : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (26) -: : : : : : : : +- ReusedExchange (29) -: : : : : : : +- CometBroadcastExchange (35) -: : : : : : : +- CometProject (34) -: : : : : : : +- CometFilter (33) -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (32) -: : : : : : +- ReusedExchange (38) -: : : : : +- BroadcastExchange (66) -: : : : : +- * CometColumnarToRow (65) -: : : : : +- CometHashAggregate (64) -: : : : : +- CometExchange (63) -: : : : : +- CometHashAggregate (62) -: : : : : +- CometProject (61) -: : : : : +- CometBroadcastHashJoin (60) -: : : : : :- CometProject (58) -: : : : : : +- CometBroadcastHashJoin (57) -: : : : : : :- CometProject (52) -: : : : : : : +- CometBroadcastHashJoin (51) -: : : : : : : :- CometProject (49) -: : : : : : : : +- CometFilter (48) -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (47) -: : : : : : : +- ReusedExchange (50) -: : : : : : +- CometBroadcastExchange (56) -: : : : : : +- CometProject (55) -: : : : : : +- CometFilter (54) -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (53) -: : : : : +- ReusedExchange (59) -: : : : +- BroadcastExchange (87) -: : : : +- * CometColumnarToRow (86) -: : : : +- CometHashAggregate (85) -: : : : +- CometExchange (84) -: : : : +- CometHashAggregate (83) -: : : : +- CometProject (82) -: : : : +- CometBroadcastHashJoin (81) -: : : : :- CometProject (79) -: : : : : +- CometBroadcastHashJoin (78) -: : : : : :- CometProject (73) -: : : : : : +- CometBroadcastHashJoin (72) -: : : : : : :- CometProject (70) -: : : : : : : +- CometFilter (69) -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (68) -: : : : : : +- ReusedExchange (71) -: : : : : +- CometBroadcastExchange (77) -: : : : : +- CometProject (76) -: : : : : +- CometFilter (75) -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (74) -: : : : +- ReusedExchange (80) -: : : +- BroadcastExchange (108) -: : : +- * CometColumnarToRow (107) -: : : +- CometHashAggregate (106) -: : : +- CometExchange (105) -: : : +- CometHashAggregate (104) -: : : +- CometProject (103) -: : : +- CometBroadcastHashJoin (102) -: : : :- CometProject (100) -: : : : +- CometBroadcastHashJoin (99) -: : : : :- CometProject (94) -: : : : : +- CometBroadcastHashJoin (93) -: : : : : :- CometProject (91) -: : : : : : +- CometFilter (90) -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (89) -: : : : : +- ReusedExchange (92) -: : : : +- CometBroadcastExchange (98) -: : : : +- CometProject (97) -: : : : +- CometFilter (96) -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (95) -: : : +- ReusedExchange (101) -: : +- BroadcastExchange (129) -: : +- * CometColumnarToRow (128) -: : +- CometHashAggregate (127) -: : +- CometExchange (126) -: : +- CometHashAggregate (125) -: : +- CometProject (124) -: : +- CometBroadcastHashJoin (123) -: : :- CometProject (121) -: : : +- CometBroadcastHashJoin (120) -: : : :- CometProject (115) -: : : : +- CometBroadcastHashJoin (114) -: : : : :- CometProject (112) -: : : : : +- CometFilter (111) -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (110) -: : : : +- ReusedExchange (113) -: : : +- CometBroadcastExchange (119) -: : : +- CometProject (118) -: : : +- CometFilter (117) -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (116) -: : +- ReusedExchange (122) -: +- BroadcastExchange (150) -: +- * CometColumnarToRow (149) -: +- CometHashAggregate (148) -: +- CometExchange (147) -: +- CometHashAggregate (146) -: +- CometProject (145) -: +- CometBroadcastHashJoin (144) -: :- CometProject (142) -: : +- CometBroadcastHashJoin (141) -: : :- CometProject (136) -: : : +- CometBroadcastHashJoin (135) -: : : :- CometProject (133) -: : : : +- CometFilter (132) -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (131) -: : : +- ReusedExchange (134) -: : +- CometBroadcastExchange (140) -: : +- CometProject (139) -: : +- CometFilter (138) -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (137) -: +- ReusedExchange (143) -+- BroadcastExchange (171) - +- * CometColumnarToRow (170) - +- CometHashAggregate (169) - +- CometExchange (168) - +- CometHashAggregate (167) - +- CometProject (166) - +- CometBroadcastHashJoin (165) - :- CometProject (163) - : +- CometBroadcastHashJoin (162) - : :- CometProject (157) - : : +- CometBroadcastHashJoin (156) - : : :- CometProject (154) - : : : +- CometFilter (153) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (152) - : : +- ReusedExchange (155) - : +- CometBroadcastExchange (161) - : +- CometProject (160) - : +- CometFilter (159) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (158) - +- ReusedExchange (164) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(3) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(Or(And(EqualTo(hd_dep_count,4),LessThanOrEqual(hd_vehicle_count,6)),And(EqualTo(hd_dep_count,2),LessThanOrEqual(hd_vehicle_count,4))),And(EqualTo(hd_dep_count,0),LessThanOrEqual(hd_vehicle_count,2))), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Condition : (((((hd_dep_count#6 = 4) AND (hd_vehicle_count#7 <= 6)) OR ((hd_dep_count#6 = 2) AND (hd_vehicle_count#7 <= 4))) OR ((hd_dep_count#6 = 0) AND (hd_vehicle_count#7 <= 2))) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] -Arguments: [ss_sold_time_sk#1, ss_store_sk#3], [ss_sold_time_sk#1, ss_store_sk#3] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,8), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Condition : ((((isnotnull(t_hour#9) AND isnotnull(t_minute#10)) AND (t_hour#9 = 8)) AND (t_minute#10 >= 30)) AND isnotnull(t_time_sk#8)) - -(12) CometProject -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Arguments: [t_time_sk#8], [t_time_sk#8] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#8] -Arguments: [t_time_sk#8] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Right output [1]: [t_time_sk#8] -Arguments: [ss_sold_time_sk#1], [t_time_sk#8], Inner, BuildRight - -(15) CometProject -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#8] -Arguments: [ss_store_sk#3], [ss_store_sk#3] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_store_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [s_store_sk#11, s_store_name#12] -Condition : ((isnotnull(s_store_name#12) AND (s_store_name#12 = ese)) AND isnotnull(s_store_sk#11)) - -(18) CometProject -Input [2]: [s_store_sk#11, s_store_name#12] -Arguments: [s_store_sk#11], [s_store_sk#11] - -(19) CometBroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: [s_store_sk#11] - -(20) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#3] -Right output [1]: [s_store_sk#11] -Arguments: [ss_store_sk#3], [s_store_sk#11], Inner, BuildRight - -(21) CometProject -Input [2]: [ss_store_sk#3, s_store_sk#11] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#13] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#13] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 8] -Input [1]: [h8_30_to_9#14] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Condition : ((isnotnull(ss_hdemo_sk#16) AND isnotnull(ss_sold_time_sk#15)) AND isnotnull(ss_store_sk#17)) - -(28) CometProject -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Arguments: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17], [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] - -(29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#19] - -(30) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] -Right output [1]: [hd_demo_sk#19] -Arguments: [ss_hdemo_sk#16], [hd_demo_sk#19], Inner, BuildRight - -(31) CometProject -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, hd_demo_sk#19] -Arguments: [ss_sold_time_sk#15, ss_store_sk#17], [ss_sold_time_sk#15, ss_store_sk#17] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Condition : ((((isnotnull(t_hour#21) AND isnotnull(t_minute#22)) AND (t_hour#21 = 9)) AND (t_minute#22 < 30)) AND isnotnull(t_time_sk#20)) - -(34) CometProject -Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Arguments: [t_time_sk#20], [t_time_sk#20] - -(35) CometBroadcastExchange -Input [1]: [t_time_sk#20] -Arguments: [t_time_sk#20] - -(36) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#15, ss_store_sk#17] -Right output [1]: [t_time_sk#20] -Arguments: [ss_sold_time_sk#15], [t_time_sk#20], Inner, BuildRight - -(37) CometProject -Input [3]: [ss_sold_time_sk#15, ss_store_sk#17, t_time_sk#20] -Arguments: [ss_store_sk#17], [ss_store_sk#17] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#23] - -(39) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#17] -Right output [1]: [s_store_sk#23] -Arguments: [ss_store_sk#17], [s_store_sk#23], Inner, BuildRight - -(40) CometProject -Input [2]: [ss_store_sk#17, s_store_sk#23] - -(41) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(42) CometExchange -Input [1]: [count#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(43) CometHashAggregate -Input [1]: [count#24] -Keys: [] -Functions [1]: [count(1)] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [h9_to_9_30#25] - -(45) BroadcastExchange -Input [1]: [h9_to_9_30#25] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(46) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(48) CometFilter -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_hdemo_sk#27) AND isnotnull(ss_sold_time_sk#26)) AND isnotnull(ss_store_sk#28)) - -(49) CometProject -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Arguments: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28], [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] - -(50) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#30] - -(51) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] -Right output [1]: [hd_demo_sk#30] -Arguments: [ss_hdemo_sk#27], [hd_demo_sk#30], Inner, BuildRight - -(52) CometProject -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, hd_demo_sk#30] -Arguments: [ss_sold_time_sk#26, ss_store_sk#28], [ss_sold_time_sk#26, ss_store_sk#28] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Condition : ((((isnotnull(t_hour#32) AND isnotnull(t_minute#33)) AND (t_hour#32 = 9)) AND (t_minute#33 >= 30)) AND isnotnull(t_time_sk#31)) - -(55) CometProject -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Arguments: [t_time_sk#31], [t_time_sk#31] - -(56) CometBroadcastExchange -Input [1]: [t_time_sk#31] -Arguments: [t_time_sk#31] - -(57) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#26, ss_store_sk#28] -Right output [1]: [t_time_sk#31] -Arguments: [ss_sold_time_sk#26], [t_time_sk#31], Inner, BuildRight - -(58) CometProject -Input [3]: [ss_sold_time_sk#26, ss_store_sk#28, t_time_sk#31] -Arguments: [ss_store_sk#28], [ss_store_sk#28] - -(59) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#34] - -(60) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#28] -Right output [1]: [s_store_sk#34] -Arguments: [ss_store_sk#28], [s_store_sk#34], Inner, BuildRight - -(61) CometProject -Input [2]: [ss_store_sk#28, s_store_sk#34] - -(62) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(63) CometExchange -Input [1]: [count#35] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(64) CometHashAggregate -Input [1]: [count#35] -Keys: [] -Functions [1]: [count(1)] - -(65) CometColumnarToRow [codegen id : 2] -Input [1]: [h9_30_to_10#36] - -(66) BroadcastExchange -Input [1]: [h9_30_to_10#36] -Arguments: IdentityBroadcastMode, [plan_id=5] - -(67) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(69) CometFilter -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Condition : ((isnotnull(ss_hdemo_sk#38) AND isnotnull(ss_sold_time_sk#37)) AND isnotnull(ss_store_sk#39)) - -(70) CometProject -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Arguments: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39], [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] - -(71) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#41] - -(72) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] -Right output [1]: [hd_demo_sk#41] -Arguments: [ss_hdemo_sk#38], [hd_demo_sk#41], Inner, BuildRight - -(73) CometProject -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, hd_demo_sk#41] -Arguments: [ss_sold_time_sk#37, ss_store_sk#39], [ss_sold_time_sk#37, ss_store_sk#39] - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(75) CometFilter -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Condition : ((((isnotnull(t_hour#43) AND isnotnull(t_minute#44)) AND (t_hour#43 = 10)) AND (t_minute#44 < 30)) AND isnotnull(t_time_sk#42)) - -(76) CometProject -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Arguments: [t_time_sk#42], [t_time_sk#42] - -(77) CometBroadcastExchange -Input [1]: [t_time_sk#42] -Arguments: [t_time_sk#42] - -(78) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#37, ss_store_sk#39] -Right output [1]: [t_time_sk#42] -Arguments: [ss_sold_time_sk#37], [t_time_sk#42], Inner, BuildRight - -(79) CometProject -Input [3]: [ss_sold_time_sk#37, ss_store_sk#39, t_time_sk#42] -Arguments: [ss_store_sk#39], [ss_store_sk#39] - -(80) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#45] - -(81) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#39] -Right output [1]: [s_store_sk#45] -Arguments: [ss_store_sk#39], [s_store_sk#45], Inner, BuildRight - -(82) CometProject -Input [2]: [ss_store_sk#39, s_store_sk#45] - -(83) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(84) CometExchange -Input [1]: [count#46] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(85) CometHashAggregate -Input [1]: [count#46] -Keys: [] -Functions [1]: [count(1)] - -(86) CometColumnarToRow [codegen id : 3] -Input [1]: [h10_to_10_30#47] - -(87) BroadcastExchange -Input [1]: [h10_to_10_30#47] -Arguments: IdentityBroadcastMode, [plan_id=7] - -(88) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(89) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(90) CometFilter -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Condition : ((isnotnull(ss_hdemo_sk#49) AND isnotnull(ss_sold_time_sk#48)) AND isnotnull(ss_store_sk#50)) - -(91) CometProject -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Arguments: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50], [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] - -(92) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#52] - -(93) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] -Right output [1]: [hd_demo_sk#52] -Arguments: [ss_hdemo_sk#49], [hd_demo_sk#52], Inner, BuildRight - -(94) CometProject -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, hd_demo_sk#52] -Arguments: [ss_sold_time_sk#48, ss_store_sk#50], [ss_sold_time_sk#48, ss_store_sk#50] - -(95) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(96) CometFilter -Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Condition : ((((isnotnull(t_hour#54) AND isnotnull(t_minute#55)) AND (t_hour#54 = 10)) AND (t_minute#55 >= 30)) AND isnotnull(t_time_sk#53)) - -(97) CometProject -Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Arguments: [t_time_sk#53], [t_time_sk#53] - -(98) CometBroadcastExchange -Input [1]: [t_time_sk#53] -Arguments: [t_time_sk#53] - -(99) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#48, ss_store_sk#50] -Right output [1]: [t_time_sk#53] -Arguments: [ss_sold_time_sk#48], [t_time_sk#53], Inner, BuildRight - -(100) CometProject -Input [3]: [ss_sold_time_sk#48, ss_store_sk#50, t_time_sk#53] -Arguments: [ss_store_sk#50], [ss_store_sk#50] - -(101) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#56] - -(102) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#50] -Right output [1]: [s_store_sk#56] -Arguments: [ss_store_sk#50], [s_store_sk#56], Inner, BuildRight - -(103) CometProject -Input [2]: [ss_store_sk#50, s_store_sk#56] - -(104) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(105) CometExchange -Input [1]: [count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(106) CometHashAggregate -Input [1]: [count#57] -Keys: [] -Functions [1]: [count(1)] - -(107) CometColumnarToRow [codegen id : 4] -Input [1]: [h10_30_to_11#58] - -(108) BroadcastExchange -Input [1]: [h10_30_to_11#58] -Arguments: IdentityBroadcastMode, [plan_id=9] - -(109) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(110) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(111) CometFilter -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Condition : ((isnotnull(ss_hdemo_sk#60) AND isnotnull(ss_sold_time_sk#59)) AND isnotnull(ss_store_sk#61)) - -(112) CometProject -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Arguments: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61], [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] - -(113) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#63] - -(114) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] -Right output [1]: [hd_demo_sk#63] -Arguments: [ss_hdemo_sk#60], [hd_demo_sk#63], Inner, BuildRight - -(115) CometProject -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, hd_demo_sk#63] -Arguments: [ss_sold_time_sk#59, ss_store_sk#61], [ss_sold_time_sk#59, ss_store_sk#61] - -(116) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(117) CometFilter -Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Condition : ((((isnotnull(t_hour#65) AND isnotnull(t_minute#66)) AND (t_hour#65 = 11)) AND (t_minute#66 < 30)) AND isnotnull(t_time_sk#64)) - -(118) CometProject -Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Arguments: [t_time_sk#64], [t_time_sk#64] - -(119) CometBroadcastExchange -Input [1]: [t_time_sk#64] -Arguments: [t_time_sk#64] - -(120) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#59, ss_store_sk#61] -Right output [1]: [t_time_sk#64] -Arguments: [ss_sold_time_sk#59], [t_time_sk#64], Inner, BuildRight - -(121) CometProject -Input [3]: [ss_sold_time_sk#59, ss_store_sk#61, t_time_sk#64] -Arguments: [ss_store_sk#61], [ss_store_sk#61] - -(122) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#67] - -(123) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#61] -Right output [1]: [s_store_sk#67] -Arguments: [ss_store_sk#61], [s_store_sk#67], Inner, BuildRight - -(124) CometProject -Input [2]: [ss_store_sk#61, s_store_sk#67] - -(125) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(126) CometExchange -Input [1]: [count#68] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(127) CometHashAggregate -Input [1]: [count#68] -Keys: [] -Functions [1]: [count(1)] - -(128) CometColumnarToRow [codegen id : 5] -Input [1]: [h11_to_11_30#69] - -(129) BroadcastExchange -Input [1]: [h11_to_11_30#69] -Arguments: IdentityBroadcastMode, [plan_id=11] - -(130) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(131) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(132) CometFilter -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Condition : ((isnotnull(ss_hdemo_sk#71) AND isnotnull(ss_sold_time_sk#70)) AND isnotnull(ss_store_sk#72)) - -(133) CometProject -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Arguments: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72], [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] - -(134) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#74] - -(135) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] -Right output [1]: [hd_demo_sk#74] -Arguments: [ss_hdemo_sk#71], [hd_demo_sk#74], Inner, BuildRight - -(136) CometProject -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, hd_demo_sk#74] -Arguments: [ss_sold_time_sk#70, ss_store_sk#72], [ss_sold_time_sk#70, ss_store_sk#72] - -(137) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(138) CometFilter -Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Condition : ((((isnotnull(t_hour#76) AND isnotnull(t_minute#77)) AND (t_hour#76 = 11)) AND (t_minute#77 >= 30)) AND isnotnull(t_time_sk#75)) - -(139) CometProject -Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Arguments: [t_time_sk#75], [t_time_sk#75] - -(140) CometBroadcastExchange -Input [1]: [t_time_sk#75] -Arguments: [t_time_sk#75] - -(141) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#70, ss_store_sk#72] -Right output [1]: [t_time_sk#75] -Arguments: [ss_sold_time_sk#70], [t_time_sk#75], Inner, BuildRight - -(142) CometProject -Input [3]: [ss_sold_time_sk#70, ss_store_sk#72, t_time_sk#75] -Arguments: [ss_store_sk#72], [ss_store_sk#72] - -(143) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#78] - -(144) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#72] -Right output [1]: [s_store_sk#78] -Arguments: [ss_store_sk#72], [s_store_sk#78], Inner, BuildRight - -(145) CometProject -Input [2]: [ss_store_sk#72, s_store_sk#78] - -(146) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(147) CometExchange -Input [1]: [count#79] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(148) CometHashAggregate -Input [1]: [count#79] -Keys: [] -Functions [1]: [count(1)] - -(149) CometColumnarToRow [codegen id : 6] -Input [1]: [h11_30_to_12#80] - -(150) BroadcastExchange -Input [1]: [h11_30_to_12#80] -Arguments: IdentityBroadcastMode, [plan_id=13] - -(151) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(152) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(153) CometFilter -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Condition : ((isnotnull(ss_hdemo_sk#82) AND isnotnull(ss_sold_time_sk#81)) AND isnotnull(ss_store_sk#83)) - -(154) CometProject -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Arguments: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83], [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] - -(155) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#85] - -(156) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] -Right output [1]: [hd_demo_sk#85] -Arguments: [ss_hdemo_sk#82], [hd_demo_sk#85], Inner, BuildRight - -(157) CometProject -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, hd_demo_sk#85] -Arguments: [ss_sold_time_sk#81, ss_store_sk#83], [ss_sold_time_sk#81, ss_store_sk#83] - -(158) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(159) CometFilter -Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Condition : ((((isnotnull(t_hour#87) AND isnotnull(t_minute#88)) AND (t_hour#87 = 12)) AND (t_minute#88 < 30)) AND isnotnull(t_time_sk#86)) - -(160) CometProject -Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Arguments: [t_time_sk#86], [t_time_sk#86] - -(161) CometBroadcastExchange -Input [1]: [t_time_sk#86] -Arguments: [t_time_sk#86] - -(162) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#81, ss_store_sk#83] -Right output [1]: [t_time_sk#86] -Arguments: [ss_sold_time_sk#81], [t_time_sk#86], Inner, BuildRight - -(163) CometProject -Input [3]: [ss_sold_time_sk#81, ss_store_sk#83, t_time_sk#86] -Arguments: [ss_store_sk#83], [ss_store_sk#83] - -(164) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#89] - -(165) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#83] -Right output [1]: [s_store_sk#89] -Arguments: [ss_store_sk#83], [s_store_sk#89], Inner, BuildRight - -(166) CometProject -Input [2]: [ss_store_sk#83, s_store_sk#89] - -(167) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(168) CometExchange -Input [1]: [count#90] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(169) CometHashAggregate -Input [1]: [count#90] -Keys: [] -Functions [1]: [count(1)] - -(170) CometColumnarToRow [codegen id : 7] -Input [1]: [h12_to_12_30#91] - -(171) BroadcastExchange -Input [1]: [h12_to_12_30#91] -Arguments: IdentityBroadcastMode, [plan_id=15] - -(172) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_iceberg_compat/simplified.txt deleted file mode 100644 index e3923fb3ea..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,195 +0,0 @@ -WholeStageCodegen (8) - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h8_30_to_9,count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_to_9_30,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_30_to_10,count(1)] - CometExchange #9 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #10 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_to_10_30,count(1)] - CometExchange #12 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #13 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_30_to_11,count(1)] - CometExchange #15 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #16 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_to_11_30,count(1)] - CometExchange #18 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #19 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_30_to_12,count(1)] - CometExchange #21 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #22 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #23 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h12_to_12_30,count(1)] - CometExchange #24 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #25 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] 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/q88/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/explain.txt deleted file mode 100644 index 2e49d20af4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/explain.txt +++ /dev/null @@ -1,927 +0,0 @@ -== Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (172) -:- * BroadcastNestedLoopJoin Inner BuildRight (151) -: :- * BroadcastNestedLoopJoin Inner BuildRight (130) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (109) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) -: : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) -: : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : : : : : :- * CometColumnarToRow (25) -: : : : : : : +- CometHashAggregate (24) -: : : : : : : +- CometExchange (23) -: : : : : : : +- CometHashAggregate (22) -: : : : : : : +- CometProject (21) -: : : : : : : +- CometBroadcastHashJoin (20) -: : : : : : : :- CometProject (15) -: : : : : : : : +- CometBroadcastHashJoin (14) -: : : : : : : : :- CometProject (9) -: : : : : : : : : +- CometBroadcastHashJoin (8) -: : : : : : : : : :- CometProject (3) -: : : : : : : : : : +- CometFilter (2) -: : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) -: : : : : : : : : +- CometBroadcastExchange (7) -: : : : : : : : : +- CometProject (6) -: : : : : : : : : +- CometFilter (5) -: : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (4) -: : : : : : : : +- CometBroadcastExchange (13) -: : : : : : : : +- CometProject (12) -: : : : : : : : +- CometFilter (11) -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (10) -: : : : : : : +- CometBroadcastExchange (19) -: : : : : : : +- CometProject (18) -: : : : : : : +- CometFilter (17) -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (16) -: : : : : : +- BroadcastExchange (45) -: : : : : : +- * CometColumnarToRow (44) -: : : : : : +- CometHashAggregate (43) -: : : : : : +- CometExchange (42) -: : : : : : +- CometHashAggregate (41) -: : : : : : +- CometProject (40) -: : : : : : +- CometBroadcastHashJoin (39) -: : : : : : :- CometProject (37) -: : : : : : : +- CometBroadcastHashJoin (36) -: : : : : : : :- CometProject (31) -: : : : : : : : +- CometBroadcastHashJoin (30) -: : : : : : : : :- CometProject (28) -: : : : : : : : : +- CometFilter (27) -: : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (26) -: : : : : : : : +- ReusedExchange (29) -: : : : : : : +- CometBroadcastExchange (35) -: : : : : : : +- CometProject (34) -: : : : : : : +- CometFilter (33) -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (32) -: : : : : : +- ReusedExchange (38) -: : : : : +- BroadcastExchange (66) -: : : : : +- * CometColumnarToRow (65) -: : : : : +- CometHashAggregate (64) -: : : : : +- CometExchange (63) -: : : : : +- CometHashAggregate (62) -: : : : : +- CometProject (61) -: : : : : +- CometBroadcastHashJoin (60) -: : : : : :- CometProject (58) -: : : : : : +- CometBroadcastHashJoin (57) -: : : : : : :- CometProject (52) -: : : : : : : +- CometBroadcastHashJoin (51) -: : : : : : : :- CometProject (49) -: : : : : : : : +- CometFilter (48) -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (47) -: : : : : : : +- ReusedExchange (50) -: : : : : : +- CometBroadcastExchange (56) -: : : : : : +- CometProject (55) -: : : : : : +- CometFilter (54) -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (53) -: : : : : +- ReusedExchange (59) -: : : : +- BroadcastExchange (87) -: : : : +- * CometColumnarToRow (86) -: : : : +- CometHashAggregate (85) -: : : : +- CometExchange (84) -: : : : +- CometHashAggregate (83) -: : : : +- CometProject (82) -: : : : +- CometBroadcastHashJoin (81) -: : : : :- CometProject (79) -: : : : : +- CometBroadcastHashJoin (78) -: : : : : :- CometProject (73) -: : : : : : +- CometBroadcastHashJoin (72) -: : : : : : :- CometProject (70) -: : : : : : : +- CometFilter (69) -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (68) -: : : : : : +- ReusedExchange (71) -: : : : : +- CometBroadcastExchange (77) -: : : : : +- CometProject (76) -: : : : : +- CometFilter (75) -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (74) -: : : : +- ReusedExchange (80) -: : : +- BroadcastExchange (108) -: : : +- * CometColumnarToRow (107) -: : : +- CometHashAggregate (106) -: : : +- CometExchange (105) -: : : +- CometHashAggregate (104) -: : : +- CometProject (103) -: : : +- CometBroadcastHashJoin (102) -: : : :- CometProject (100) -: : : : +- CometBroadcastHashJoin (99) -: : : : :- CometProject (94) -: : : : : +- CometBroadcastHashJoin (93) -: : : : : :- CometProject (91) -: : : : : : +- CometFilter (90) -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (89) -: : : : : +- ReusedExchange (92) -: : : : +- CometBroadcastExchange (98) -: : : : +- CometProject (97) -: : : : +- CometFilter (96) -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (95) -: : : +- ReusedExchange (101) -: : +- BroadcastExchange (129) -: : +- * CometColumnarToRow (128) -: : +- CometHashAggregate (127) -: : +- CometExchange (126) -: : +- CometHashAggregate (125) -: : +- CometProject (124) -: : +- CometBroadcastHashJoin (123) -: : :- CometProject (121) -: : : +- CometBroadcastHashJoin (120) -: : : :- CometProject (115) -: : : : +- CometBroadcastHashJoin (114) -: : : : :- CometProject (112) -: : : : : +- CometFilter (111) -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (110) -: : : : +- ReusedExchange (113) -: : : +- CometBroadcastExchange (119) -: : : +- CometProject (118) -: : : +- CometFilter (117) -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (116) -: : +- ReusedExchange (122) -: +- BroadcastExchange (150) -: +- * CometColumnarToRow (149) -: +- CometHashAggregate (148) -: +- CometExchange (147) -: +- CometHashAggregate (146) -: +- CometProject (145) -: +- CometBroadcastHashJoin (144) -: :- CometProject (142) -: : +- CometBroadcastHashJoin (141) -: : :- CometProject (136) -: : : +- CometBroadcastHashJoin (135) -: : : :- CometProject (133) -: : : : +- CometFilter (132) -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (131) -: : : +- ReusedExchange (134) -: : +- CometBroadcastExchange (140) -: : +- CometProject (139) -: : +- CometFilter (138) -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (137) -: +- ReusedExchange (143) -+- BroadcastExchange (171) - +- * CometColumnarToRow (170) - +- CometHashAggregate (169) - +- CometExchange (168) - +- CometHashAggregate (167) - +- CometProject (166) - +- CometBroadcastHashJoin (165) - :- CometProject (163) - : +- CometBroadcastHashJoin (162) - : :- CometProject (157) - : : +- CometBroadcastHashJoin (156) - : : :- CometProject (154) - : : : +- CometFilter (153) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (152) - : : +- ReusedExchange (155) - : +- CometBroadcastExchange (161) - : +- CometProject (160) - : +- CometFilter (159) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (158) - +- ReusedExchange (164) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(3) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(Or(And(EqualTo(hd_dep_count,4),LessThanOrEqual(hd_vehicle_count,6)),And(EqualTo(hd_dep_count,2),LessThanOrEqual(hd_vehicle_count,4))),And(EqualTo(hd_dep_count,0),LessThanOrEqual(hd_vehicle_count,2))), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Condition : (((((hd_dep_count#6 = 4) AND (hd_vehicle_count#7 <= 6)) OR ((hd_dep_count#6 = 2) AND (hd_vehicle_count#7 <= 4))) OR ((hd_dep_count#6 = 0) AND (hd_vehicle_count#7 <= 2))) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] -Arguments: [ss_sold_time_sk#1, ss_store_sk#3], [ss_sold_time_sk#1, ss_store_sk#3] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,8), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Condition : ((((isnotnull(t_hour#9) AND isnotnull(t_minute#10)) AND (t_hour#9 = 8)) AND (t_minute#10 >= 30)) AND isnotnull(t_time_sk#8)) - -(12) CometProject -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Arguments: [t_time_sk#8], [t_time_sk#8] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#8] -Arguments: [t_time_sk#8] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Right output [1]: [t_time_sk#8] -Arguments: [ss_sold_time_sk#1], [t_time_sk#8], Inner, BuildRight - -(15) CometProject -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#8] -Arguments: [ss_store_sk#3], [ss_store_sk#3] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_store_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [s_store_sk#11, s_store_name#12] -Condition : ((isnotnull(s_store_name#12) AND (s_store_name#12 = ese)) AND isnotnull(s_store_sk#11)) - -(18) CometProject -Input [2]: [s_store_sk#11, s_store_name#12] -Arguments: [s_store_sk#11], [s_store_sk#11] - -(19) CometBroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: [s_store_sk#11] - -(20) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#3] -Right output [1]: [s_store_sk#11] -Arguments: [ss_store_sk#3], [s_store_sk#11], Inner, BuildRight - -(21) CometProject -Input [2]: [ss_store_sk#3, s_store_sk#11] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#13] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#13] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 8] -Input [1]: [h8_30_to_9#14] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Condition : ((isnotnull(ss_hdemo_sk#16) AND isnotnull(ss_sold_time_sk#15)) AND isnotnull(ss_store_sk#17)) - -(28) CometProject -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Arguments: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17], [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] - -(29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#19] - -(30) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] -Right output [1]: [hd_demo_sk#19] -Arguments: [ss_hdemo_sk#16], [hd_demo_sk#19], Inner, BuildRight - -(31) CometProject -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, hd_demo_sk#19] -Arguments: [ss_sold_time_sk#15, ss_store_sk#17], [ss_sold_time_sk#15, ss_store_sk#17] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Condition : ((((isnotnull(t_hour#21) AND isnotnull(t_minute#22)) AND (t_hour#21 = 9)) AND (t_minute#22 < 30)) AND isnotnull(t_time_sk#20)) - -(34) CometProject -Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Arguments: [t_time_sk#20], [t_time_sk#20] - -(35) CometBroadcastExchange -Input [1]: [t_time_sk#20] -Arguments: [t_time_sk#20] - -(36) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#15, ss_store_sk#17] -Right output [1]: [t_time_sk#20] -Arguments: [ss_sold_time_sk#15], [t_time_sk#20], Inner, BuildRight - -(37) CometProject -Input [3]: [ss_sold_time_sk#15, ss_store_sk#17, t_time_sk#20] -Arguments: [ss_store_sk#17], [ss_store_sk#17] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#23] - -(39) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#17] -Right output [1]: [s_store_sk#23] -Arguments: [ss_store_sk#17], [s_store_sk#23], Inner, BuildRight - -(40) CometProject -Input [2]: [ss_store_sk#17, s_store_sk#23] - -(41) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(42) CometExchange -Input [1]: [count#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(43) CometHashAggregate -Input [1]: [count#24] -Keys: [] -Functions [1]: [count(1)] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [h9_to_9_30#25] - -(45) BroadcastExchange -Input [1]: [h9_to_9_30#25] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(46) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(48) CometFilter -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_hdemo_sk#27) AND isnotnull(ss_sold_time_sk#26)) AND isnotnull(ss_store_sk#28)) - -(49) CometProject -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Arguments: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28], [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] - -(50) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#30] - -(51) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] -Right output [1]: [hd_demo_sk#30] -Arguments: [ss_hdemo_sk#27], [hd_demo_sk#30], Inner, BuildRight - -(52) CometProject -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, hd_demo_sk#30] -Arguments: [ss_sold_time_sk#26, ss_store_sk#28], [ss_sold_time_sk#26, ss_store_sk#28] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Condition : ((((isnotnull(t_hour#32) AND isnotnull(t_minute#33)) AND (t_hour#32 = 9)) AND (t_minute#33 >= 30)) AND isnotnull(t_time_sk#31)) - -(55) CometProject -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Arguments: [t_time_sk#31], [t_time_sk#31] - -(56) CometBroadcastExchange -Input [1]: [t_time_sk#31] -Arguments: [t_time_sk#31] - -(57) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#26, ss_store_sk#28] -Right output [1]: [t_time_sk#31] -Arguments: [ss_sold_time_sk#26], [t_time_sk#31], Inner, BuildRight - -(58) CometProject -Input [3]: [ss_sold_time_sk#26, ss_store_sk#28, t_time_sk#31] -Arguments: [ss_store_sk#28], [ss_store_sk#28] - -(59) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#34] - -(60) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#28] -Right output [1]: [s_store_sk#34] -Arguments: [ss_store_sk#28], [s_store_sk#34], Inner, BuildRight - -(61) CometProject -Input [2]: [ss_store_sk#28, s_store_sk#34] - -(62) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(63) CometExchange -Input [1]: [count#35] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(64) CometHashAggregate -Input [1]: [count#35] -Keys: [] -Functions [1]: [count(1)] - -(65) CometColumnarToRow [codegen id : 2] -Input [1]: [h9_30_to_10#36] - -(66) BroadcastExchange -Input [1]: [h9_30_to_10#36] -Arguments: IdentityBroadcastMode, [plan_id=5] - -(67) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(69) CometFilter -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Condition : ((isnotnull(ss_hdemo_sk#38) AND isnotnull(ss_sold_time_sk#37)) AND isnotnull(ss_store_sk#39)) - -(70) CometProject -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Arguments: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39], [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] - -(71) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#41] - -(72) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] -Right output [1]: [hd_demo_sk#41] -Arguments: [ss_hdemo_sk#38], [hd_demo_sk#41], Inner, BuildRight - -(73) CometProject -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, hd_demo_sk#41] -Arguments: [ss_sold_time_sk#37, ss_store_sk#39], [ss_sold_time_sk#37, ss_store_sk#39] - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(75) CometFilter -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Condition : ((((isnotnull(t_hour#43) AND isnotnull(t_minute#44)) AND (t_hour#43 = 10)) AND (t_minute#44 < 30)) AND isnotnull(t_time_sk#42)) - -(76) CometProject -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Arguments: [t_time_sk#42], [t_time_sk#42] - -(77) CometBroadcastExchange -Input [1]: [t_time_sk#42] -Arguments: [t_time_sk#42] - -(78) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#37, ss_store_sk#39] -Right output [1]: [t_time_sk#42] -Arguments: [ss_sold_time_sk#37], [t_time_sk#42], Inner, BuildRight - -(79) CometProject -Input [3]: [ss_sold_time_sk#37, ss_store_sk#39, t_time_sk#42] -Arguments: [ss_store_sk#39], [ss_store_sk#39] - -(80) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#45] - -(81) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#39] -Right output [1]: [s_store_sk#45] -Arguments: [ss_store_sk#39], [s_store_sk#45], Inner, BuildRight - -(82) CometProject -Input [2]: [ss_store_sk#39, s_store_sk#45] - -(83) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(84) CometExchange -Input [1]: [count#46] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(85) CometHashAggregate -Input [1]: [count#46] -Keys: [] -Functions [1]: [count(1)] - -(86) CometColumnarToRow [codegen id : 3] -Input [1]: [h10_to_10_30#47] - -(87) BroadcastExchange -Input [1]: [h10_to_10_30#47] -Arguments: IdentityBroadcastMode, [plan_id=7] - -(88) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(89) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(90) CometFilter -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Condition : ((isnotnull(ss_hdemo_sk#49) AND isnotnull(ss_sold_time_sk#48)) AND isnotnull(ss_store_sk#50)) - -(91) CometProject -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Arguments: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50], [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] - -(92) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#52] - -(93) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] -Right output [1]: [hd_demo_sk#52] -Arguments: [ss_hdemo_sk#49], [hd_demo_sk#52], Inner, BuildRight - -(94) CometProject -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, hd_demo_sk#52] -Arguments: [ss_sold_time_sk#48, ss_store_sk#50], [ss_sold_time_sk#48, ss_store_sk#50] - -(95) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(96) CometFilter -Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Condition : ((((isnotnull(t_hour#54) AND isnotnull(t_minute#55)) AND (t_hour#54 = 10)) AND (t_minute#55 >= 30)) AND isnotnull(t_time_sk#53)) - -(97) CometProject -Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Arguments: [t_time_sk#53], [t_time_sk#53] - -(98) CometBroadcastExchange -Input [1]: [t_time_sk#53] -Arguments: [t_time_sk#53] - -(99) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#48, ss_store_sk#50] -Right output [1]: [t_time_sk#53] -Arguments: [ss_sold_time_sk#48], [t_time_sk#53], Inner, BuildRight - -(100) CometProject -Input [3]: [ss_sold_time_sk#48, ss_store_sk#50, t_time_sk#53] -Arguments: [ss_store_sk#50], [ss_store_sk#50] - -(101) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#56] - -(102) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#50] -Right output [1]: [s_store_sk#56] -Arguments: [ss_store_sk#50], [s_store_sk#56], Inner, BuildRight - -(103) CometProject -Input [2]: [ss_store_sk#50, s_store_sk#56] - -(104) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(105) CometExchange -Input [1]: [count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(106) CometHashAggregate -Input [1]: [count#57] -Keys: [] -Functions [1]: [count(1)] - -(107) CometColumnarToRow [codegen id : 4] -Input [1]: [h10_30_to_11#58] - -(108) BroadcastExchange -Input [1]: [h10_30_to_11#58] -Arguments: IdentityBroadcastMode, [plan_id=9] - -(109) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(110) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(111) CometFilter -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Condition : ((isnotnull(ss_hdemo_sk#60) AND isnotnull(ss_sold_time_sk#59)) AND isnotnull(ss_store_sk#61)) - -(112) CometProject -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Arguments: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61], [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] - -(113) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#63] - -(114) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] -Right output [1]: [hd_demo_sk#63] -Arguments: [ss_hdemo_sk#60], [hd_demo_sk#63], Inner, BuildRight - -(115) CometProject -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, hd_demo_sk#63] -Arguments: [ss_sold_time_sk#59, ss_store_sk#61], [ss_sold_time_sk#59, ss_store_sk#61] - -(116) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(117) CometFilter -Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Condition : ((((isnotnull(t_hour#65) AND isnotnull(t_minute#66)) AND (t_hour#65 = 11)) AND (t_minute#66 < 30)) AND isnotnull(t_time_sk#64)) - -(118) CometProject -Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Arguments: [t_time_sk#64], [t_time_sk#64] - -(119) CometBroadcastExchange -Input [1]: [t_time_sk#64] -Arguments: [t_time_sk#64] - -(120) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#59, ss_store_sk#61] -Right output [1]: [t_time_sk#64] -Arguments: [ss_sold_time_sk#59], [t_time_sk#64], Inner, BuildRight - -(121) CometProject -Input [3]: [ss_sold_time_sk#59, ss_store_sk#61, t_time_sk#64] -Arguments: [ss_store_sk#61], [ss_store_sk#61] - -(122) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#67] - -(123) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#61] -Right output [1]: [s_store_sk#67] -Arguments: [ss_store_sk#61], [s_store_sk#67], Inner, BuildRight - -(124) CometProject -Input [2]: [ss_store_sk#61, s_store_sk#67] - -(125) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(126) CometExchange -Input [1]: [count#68] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(127) CometHashAggregate -Input [1]: [count#68] -Keys: [] -Functions [1]: [count(1)] - -(128) CometColumnarToRow [codegen id : 5] -Input [1]: [h11_to_11_30#69] - -(129) BroadcastExchange -Input [1]: [h11_to_11_30#69] -Arguments: IdentityBroadcastMode, [plan_id=11] - -(130) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(131) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(132) CometFilter -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Condition : ((isnotnull(ss_hdemo_sk#71) AND isnotnull(ss_sold_time_sk#70)) AND isnotnull(ss_store_sk#72)) - -(133) CometProject -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Arguments: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72], [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] - -(134) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#74] - -(135) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] -Right output [1]: [hd_demo_sk#74] -Arguments: [ss_hdemo_sk#71], [hd_demo_sk#74], Inner, BuildRight - -(136) CometProject -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, hd_demo_sk#74] -Arguments: [ss_sold_time_sk#70, ss_store_sk#72], [ss_sold_time_sk#70, ss_store_sk#72] - -(137) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(138) CometFilter -Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Condition : ((((isnotnull(t_hour#76) AND isnotnull(t_minute#77)) AND (t_hour#76 = 11)) AND (t_minute#77 >= 30)) AND isnotnull(t_time_sk#75)) - -(139) CometProject -Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Arguments: [t_time_sk#75], [t_time_sk#75] - -(140) CometBroadcastExchange -Input [1]: [t_time_sk#75] -Arguments: [t_time_sk#75] - -(141) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#70, ss_store_sk#72] -Right output [1]: [t_time_sk#75] -Arguments: [ss_sold_time_sk#70], [t_time_sk#75], Inner, BuildRight - -(142) CometProject -Input [3]: [ss_sold_time_sk#70, ss_store_sk#72, t_time_sk#75] -Arguments: [ss_store_sk#72], [ss_store_sk#72] - -(143) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#78] - -(144) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#72] -Right output [1]: [s_store_sk#78] -Arguments: [ss_store_sk#72], [s_store_sk#78], Inner, BuildRight - -(145) CometProject -Input [2]: [ss_store_sk#72, s_store_sk#78] - -(146) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(147) CometExchange -Input [1]: [count#79] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(148) CometHashAggregate -Input [1]: [count#79] -Keys: [] -Functions [1]: [count(1)] - -(149) CometColumnarToRow [codegen id : 6] -Input [1]: [h11_30_to_12#80] - -(150) BroadcastExchange -Input [1]: [h11_30_to_12#80] -Arguments: IdentityBroadcastMode, [plan_id=13] - -(151) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(152) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(153) CometFilter -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Condition : ((isnotnull(ss_hdemo_sk#82) AND isnotnull(ss_sold_time_sk#81)) AND isnotnull(ss_store_sk#83)) - -(154) CometProject -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Arguments: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83], [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] - -(155) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#85] - -(156) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] -Right output [1]: [hd_demo_sk#85] -Arguments: [ss_hdemo_sk#82], [hd_demo_sk#85], Inner, BuildRight - -(157) CometProject -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, hd_demo_sk#85] -Arguments: [ss_sold_time_sk#81, ss_store_sk#83], [ss_sold_time_sk#81, ss_store_sk#83] - -(158) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(159) CometFilter -Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Condition : ((((isnotnull(t_hour#87) AND isnotnull(t_minute#88)) AND (t_hour#87 = 12)) AND (t_minute#88 < 30)) AND isnotnull(t_time_sk#86)) - -(160) CometProject -Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Arguments: [t_time_sk#86], [t_time_sk#86] - -(161) CometBroadcastExchange -Input [1]: [t_time_sk#86] -Arguments: [t_time_sk#86] - -(162) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#81, ss_store_sk#83] -Right output [1]: [t_time_sk#86] -Arguments: [ss_sold_time_sk#81], [t_time_sk#86], Inner, BuildRight - -(163) CometProject -Input [3]: [ss_sold_time_sk#81, ss_store_sk#83, t_time_sk#86] -Arguments: [ss_store_sk#83], [ss_store_sk#83] - -(164) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#89] - -(165) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#83] -Right output [1]: [s_store_sk#89] -Arguments: [ss_store_sk#83], [s_store_sk#89], Inner, BuildRight - -(166) CometProject -Input [2]: [ss_store_sk#83, s_store_sk#89] - -(167) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(168) CometExchange -Input [1]: [count#90] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(169) CometHashAggregate -Input [1]: [count#90] -Keys: [] -Functions [1]: [count(1)] - -(170) CometColumnarToRow [codegen id : 7] -Input [1]: [h12_to_12_30#91] - -(171) BroadcastExchange -Input [1]: [h12_to_12_30#91] -Arguments: IdentityBroadcastMode, [plan_id=15] - -(172) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/extended.txt deleted file mode 100644 index 1e1247665c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/extended.txt +++ /dev/null @@ -1,216 +0,0 @@ -BroadcastNestedLoopJoin -:- BroadcastNestedLoopJoin -: :- BroadcastNestedLoopJoin -: : :- BroadcastNestedLoopJoin -: : : :- BroadcastNestedLoopJoin -: : : : :- BroadcastNestedLoopJoin -: : : : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] -: : : : : : :- CometColumnarToRow -: : : : : : : +- CometHashAggregate -: : : : : : : +- CometExchange -: : : : : : : +- CometHashAggregate -: : : : : : : +- CometProject -: : : : : : : +- CometBroadcastHashJoin -: : : : : : : :- CometProject -: : : : : : : : +- CometBroadcastHashJoin -: : : : : : : : :- CometProject -: : : : : : : : : +- CometBroadcastHashJoin -: : : : : : : : : :- CometProject -: : : : : : : : : : +- CometFilter -: : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : : : : : +- CometBroadcastExchange -: : : : : : : : : +- CometProject -: : : : : : : : : +- CometFilter -: : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : : : : : +- CometBroadcastExchange -: : : : : : : : +- CometProject -: : : : : : : : +- CometFilter -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : : : : : +- CometBroadcastExchange -: : : : : : : +- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : : : : +- BroadcastExchange -: : : : : : +- CometColumnarToRow -: : : : : : +- CometHashAggregate -: : : : : : +- CometExchange -: : : : : : +- CometHashAggregate -: : : : : : +- CometProject -: : : : : : +- CometBroadcastHashJoin -: : : : : : :- CometProject -: : : : : : : +- CometBroadcastHashJoin -: : : : : : : :- CometProject -: : : : : : : : +- CometBroadcastHashJoin -: : : : : : : : :- CometProject -: : : : : : : : : +- CometFilter -: : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : : : : +- CometBroadcastExchange -: : : : : : : : +- CometProject -: : : : : : : : +- CometFilter -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : : : : +- CometBroadcastExchange -: : : : : : : +- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : : : : +- CometBroadcastExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : : : +- BroadcastExchange -: : : : : +- CometColumnarToRow -: : : : : +- CometHashAggregate -: : : : : +- CometExchange -: : : : : +- CometHashAggregate -: : : : : +- CometProject -: : : : : +- CometBroadcastHashJoin -: : : : : :- CometProject -: : : : : : +- CometBroadcastHashJoin -: : : : : : :- CometProject -: : : : : : : +- CometBroadcastHashJoin -: : : : : : : :- CometProject -: : : : : : : : +- CometFilter -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : : : +- CometBroadcastExchange -: : : : : : : +- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : : : +- CometBroadcastExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : : : +- CometBroadcastExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : : +- BroadcastExchange -: : : : +- CometColumnarToRow -: : : : +- CometHashAggregate -: : : : +- CometExchange -: : : : +- CometHashAggregate -: : : : +- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometBroadcastHashJoin -: : : : : :- CometProject -: : : : : : +- CometBroadcastHashJoin -: : : : : : :- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : : +- CometBroadcastExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : : +- CometBroadcastExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : +- BroadcastExchange -: : : +- CometColumnarToRow -: : : +- CometHashAggregate -: : : +- CometExchange -: : : +- CometHashAggregate -: : : +- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometBroadcastHashJoin -: : : : : :- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : +- CometBroadcastExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : +- BroadcastExchange -: : +- CometColumnarToRow -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: +- BroadcastExchange -: +- CometColumnarToRow -: +- CometHashAggregate -: +- CometExchange -: +- CometHashAggregate -: +- CometProject -: +- CometBroadcastHashJoin -: :- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: +- CometBroadcastExchange -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -+- BroadcastExchange - +- CometColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 192 out of 206 eligible operators (93%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index e3923fb3ea..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt +++ /dev/null @@ -1,195 +0,0 @@ -WholeStageCodegen (8) - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h8_30_to_9,count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_to_9_30,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_30_to_10,count(1)] - CometExchange #9 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #10 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_to_10_30,count(1)] - CometExchange #12 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #13 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_30_to_11,count(1)] - CometExchange #15 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #16 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_to_11_30,count(1)] - CometExchange #18 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #19 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_30_to_12,count(1)] - CometExchange #21 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #22 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #23 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h12_to_12_30,count(1)] - CometExchange #24 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #25 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/explain.txt deleted file mode 100644 index 54bc4a6948..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/explain.txt +++ /dev/null @@ -1,202 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * CometColumnarToRow (26) - +- CometSort (25) - +- CometColumnarExchange (24) - +- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometColumnarExchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * CometColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (8) - : : +- * Filter (7) - : : +- * ColumnarToRow (6) - : : +- Scan parquet spark_catalog.default.store_sales (5) - : +- ReusedExchange (11) - +- BroadcastExchange (17) - +- * CometColumnarToRow (16) - +- CometFilter (15) - +- CometNativeScan parquet spark_catalog.default.store (14) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Condition : (((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) IN (Books ,Electronics ,Sports ) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#3, 50)) IN (computers ,stereo ,football )) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) IN (Men ,Jewelry ,Women ) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#3, 50)) IN (shirts ,birdal ,dresses ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Arguments: [i_item_sk#1, i_brand#5, i_class#6, i_category#7], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) AS i_brand#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#3, 50)) AS i_class#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) AS i_category#7] - -(4) CometColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7] - -(5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] - -(7) Filter [codegen id : 1] -Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Condition : (isnotnull(ss_item_sk#8) AND isnotnull(ss_store_sk#9)) - -(8) BroadcastExchange -Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#8] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Input [8]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7, ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] - -(11) ReusedExchange [Reuses operator id: 35] -Output [2]: [d_date_sk#13, d_moy#14] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#14] -Input [8]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11, d_date_sk#13, d_moy#14] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#15, s_store_name#16, s_company_name#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [s_store_sk#15, s_store_name#16, s_company_name#17] -Condition : isnotnull(s_store_sk#15) - -(16) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#15, s_store_name#16, s_company_name#17] - -(17) BroadcastExchange -Input [3]: [s_store_sk#15, s_store_name#16, s_company_name#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#9] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [7]: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#14, s_store_name#16, s_company_name#17] -Input [9]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#14, s_store_sk#15, s_store_name#16, s_company_name#17] - -(20) HashAggregate [codegen id : 4] -Input [7]: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#14, s_store_name#16, s_company_name#17] -Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#10))] -Aggregate Attributes [1]: [sum#18] -Results [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum#19] - -(21) CometColumnarExchange -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum#19] -Arguments: hashpartitioning(i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum#19] - -(23) HashAggregate [codegen id : 5] -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum#19] -Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14] -Functions [1]: [sum(UnscaledValue(ss_sales_price#10))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#10))#20] -Results [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#20,17,2) AS _w0#22] - -(24) CometColumnarExchange -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#16, s_company_name#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22] -Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#16 ASC NULLS FIRST, s_company_name#17 ASC NULLS FIRST] - -(26) CometColumnarToRow [codegen id : 6] -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22] - -(27) Window -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_category#7, i_brand#5, s_store_name#16, s_company_name#17, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#7, i_brand#5, s_store_name#16, s_company_name#17] - -(28) Filter [codegen id : 7] -Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22, avg_monthly_sales#23] -Condition : CASE WHEN NOT (avg_monthly_sales#23 = 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END - -(29) Project [codegen id : 7] -Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, avg_monthly_sales#23] -Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22, avg_monthly_sales#23] - -(30) TakeOrderedAndProject -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, avg_monthly_sales#23] -Arguments: 100, [(sum_sales#21 - avg_monthly_sales#23) ASC NULLS FIRST, s_store_name#16 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, avg_monthly_sales#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometNativeScan parquet spark_catalog.default.date_dim (31) - - -(31) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#24, d_moy#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [d_date_sk#13, d_year#24, d_moy#14] -Condition : ((isnotnull(d_year#24) AND (d_year#24 = 1999)) AND isnotnull(d_date_sk#13)) - -(33) CometProject -Input [3]: [d_date_sk#13, d_year#24, d_moy#14] -Arguments: [d_date_sk#13, d_moy#14], [d_date_sk#13, d_moy#14] - -(34) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_moy#14] - -(35) BroadcastExchange -Input [2]: [d_date_sk#13, d_moy#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/simplified.txt deleted file mode 100644 index f87ef33db4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/simplified.txt +++ /dev/null @@ -1,53 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (7) - Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometNativeScan 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/q89.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/explain.txt deleted file mode 100644 index 185c9d264c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,198 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) - +- CometSort (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Condition : (((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) IN (Books ,Electronics ,Sports ) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#3, 50)) IN (computers ,stereo ,football )) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) IN (Men ,Jewelry ,Women ) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#3, 50)) IN (shirts ,birdal ,dresses ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Arguments: [i_item_sk#1, i_brand#5, i_class#6, i_category#7], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) AS i_brand#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#3, 50)) AS i_class#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) AS i_category#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Condition : (isnotnull(ss_item_sk#8) AND isnotnull(ss_store_sk#9)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Arguments: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] - -(7) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7] -Right output [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Arguments: [i_item_sk#1], [ss_item_sk#8], Inner, BuildRight - -(8) CometProject -Input [8]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7, ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Arguments: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11], [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 1999)) AND isnotnull(d_date_sk#13)) - -(11) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13, d_moy#15], [d_date_sk#13, d_moy#15] - -(12) CometBroadcastExchange -Input [2]: [d_date_sk#13, d_moy#15] -Arguments: [d_date_sk#13, d_moy#15] - -(13) CometBroadcastHashJoin -Left output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Right output [2]: [d_date_sk#13, d_moy#15] -Arguments: [ss_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(14) CometProject -Input [8]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11, d_date_sk#13, d_moy#15] -Arguments: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15], [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Condition : isnotnull(s_store_sk#16) - -(17) CometBroadcastExchange -Input [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Arguments: [s_store_sk#16, s_store_name#17, s_company_name#18] - -(18) CometBroadcastHashJoin -Left output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15] -Right output [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Arguments: [ss_store_sk#9], [s_store_sk#16], Inner, BuildRight - -(19) CometProject -Input [9]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15, s_store_sk#16, s_store_name#17, s_company_name#18] -Arguments: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#15, s_store_name#17, s_company_name#18], [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#15, s_store_name#17, s_company_name#18] - -(20) CometHashAggregate -Input [7]: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#15, s_store_name#17, s_company_name#18] -Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#10))] - -(21) CometExchange -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum#19] -Arguments: hashpartitioning(i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum#19] -Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15] -Functions [1]: [sum(UnscaledValue(ss_sales_price#10))] - -(23) CometExchange -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST, s_company_name#18 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 1] -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] - -(26) Window -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] - -(27) Filter [codegen id : 2] -Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] -Condition : CASE WHEN NOT (avg_monthly_sales#22 = 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END - -(28) Project [codegen id : 2] -Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] -Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] - -(29) TakeOrderedAndProject -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] -Arguments: 100, [(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 1999)) AND isnotnull(d_date_sk#13)) - -(32) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13, d_moy#15], [d_date_sk#13, d_moy#15] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_moy#15] - -(34) BroadcastExchange -Input [2]: [d_date_sk#13, d_moy#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/simplified.txt deleted file mode 100644 index 61bfd1d960..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (2) - Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] - CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - 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 [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,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] 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/q89/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt deleted file mode 100644 index 185c9d264c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt +++ /dev/null @@ -1,198 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) - +- CometSort (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Condition : (((static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) IN (Books ,Electronics ,Sports ) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#3, 50)) IN (computers ,stereo ,football )) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) IN (Men ,Jewelry ,Women ) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#3, 50)) IN (shirts ,birdal ,dresses ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Arguments: [i_item_sk#1, i_brand#5, i_class#6, i_category#7], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) AS i_brand#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#3, 50)) AS i_class#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#4, 50)) AS i_category#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Condition : (isnotnull(ss_item_sk#8) AND isnotnull(ss_store_sk#9)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Arguments: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] - -(7) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7] -Right output [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Arguments: [i_item_sk#1], [ss_item_sk#8], Inner, BuildRight - -(8) CometProject -Input [8]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7, ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Arguments: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11], [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 1999)) AND isnotnull(d_date_sk#13)) - -(11) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13, d_moy#15], [d_date_sk#13, d_moy#15] - -(12) CometBroadcastExchange -Input [2]: [d_date_sk#13, d_moy#15] -Arguments: [d_date_sk#13, d_moy#15] - -(13) CometBroadcastHashJoin -Left output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Right output [2]: [d_date_sk#13, d_moy#15] -Arguments: [ss_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(14) CometProject -Input [8]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11, d_date_sk#13, d_moy#15] -Arguments: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15], [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Condition : isnotnull(s_store_sk#16) - -(17) CometBroadcastExchange -Input [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Arguments: [s_store_sk#16, s_store_name#17, s_company_name#18] - -(18) CometBroadcastHashJoin -Left output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15] -Right output [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Arguments: [ss_store_sk#9], [s_store_sk#16], Inner, BuildRight - -(19) CometProject -Input [9]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15, s_store_sk#16, s_store_name#17, s_company_name#18] -Arguments: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#15, s_store_name#17, s_company_name#18], [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#15, s_store_name#17, s_company_name#18] - -(20) CometHashAggregate -Input [7]: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#15, s_store_name#17, s_company_name#18] -Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#10))] - -(21) CometExchange -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum#19] -Arguments: hashpartitioning(i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum#19] -Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15] -Functions [1]: [sum(UnscaledValue(ss_sales_price#10))] - -(23) CometExchange -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST, s_company_name#18 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 1] -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] - -(26) Window -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] - -(27) Filter [codegen id : 2] -Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] -Condition : CASE WHEN NOT (avg_monthly_sales#22 = 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END - -(28) Project [codegen id : 2] -Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] -Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] - -(29) TakeOrderedAndProject -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] -Arguments: 100, [(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 1999)) AND isnotnull(d_date_sk#13)) - -(32) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13, d_moy#15], [d_date_sk#13, d_moy#15] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_moy#15] - -(34) BroadcastExchange -Input [2]: [d_date_sk#13, d_moy#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/extended.txt deleted file mode 100644 index 825b1ed81c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 61bfd1d960..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (2) - Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] - CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - 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 [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,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_datafusion/explain.txt deleted file mode 100644 index da194f2825..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_datafusion/explain.txt +++ /dev/null @@ -1,283 +0,0 @@ -== Physical Plan == -* Project (4) -+- * CometColumnarToRow (3) - +- CometFilter (2) - +- CometNativeScan parquet spark_catalog.default.reason (1) - - -(1) CometNativeScan parquet spark_catalog.default.reason -Output [1]: [r_reason_sk#1] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk), EqualTo(r_reason_sk,1)] -ReadSchema: struct - -(2) CometFilter -Input [1]: [r_reason_sk#1] -Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) - -(3) CometColumnarToRow [codegen id : 1] -Input [1]: [r_reason_sk#1] - -(4) Project [codegen id : 1] -Output [5]: [CASE WHEN (Subquery scalar-subquery#2, [id=#3].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket1#4, CASE WHEN (Subquery scalar-subquery#5, [id=#6].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_net_paid) END AS bucket2#7, CASE WHEN (Subquery scalar-subquery#8, [id=#9].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_net_paid) END AS bucket3#10, CASE WHEN (Subquery scalar-subquery#11, [id=#12].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_net_paid) END AS bucket4#13, CASE WHEN (Subquery scalar-subquery#14, [id=#15].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_net_paid) END AS bucket5#16] -Input [1]: [r_reason_sk#1] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] -* CometColumnarToRow (12) -+- CometProject (11) - +- CometHashAggregate (10) - +- CometExchange (9) - +- CometHashAggregate (8) - +- CometProject (7) - +- CometFilter (6) - +- CometNativeScan parquet spark_catalog.default.store_sales (5) - - -(5) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] -ReadSchema: struct - -(6) CometFilter -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Condition : ((isnotnull(ss_quantity#17) AND (ss_quantity#17 >= 1)) AND (ss_quantity#17 <= 20)) - -(7) CometProject -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Arguments: [ss_ext_discount_amt#18, ss_net_paid#19], [ss_ext_discount_amt#18, ss_net_paid#19] - -(8) CometHashAggregate -Input [2]: [ss_ext_discount_amt#18, ss_net_paid#19] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#18)), partial_avg(UnscaledValue(ss_net_paid#19))] - -(9) CometExchange -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometHashAggregate -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(UnscaledValue(ss_net_paid#19))] - -(11) CometProject -Input [3]: [count(1)#26, avg(ss_ext_discount_amt)#27, avg(ss_net_paid)#28] -Arguments: [mergedValue#29], [named_struct(count(1), count(1)#26, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#27, avg(ss_net_paid), avg(ss_net_paid)#28) AS mergedValue#29] - -(12) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#29] - -Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] - -Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] - -Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] -* CometColumnarToRow (20) -+- CometProject (19) - +- CometHashAggregate (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometProject (15) - +- CometFilter (14) - +- CometNativeScan parquet spark_catalog.default.store_sales (13) - - -(13) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Condition : ((isnotnull(ss_quantity#30) AND (ss_quantity#30 >= 21)) AND (ss_quantity#30 <= 40)) - -(15) CometProject -Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Arguments: [ss_ext_discount_amt#31, ss_net_paid#32], [ss_ext_discount_amt#31, ss_net_paid#32] - -(16) CometHashAggregate -Input [2]: [ss_ext_discount_amt#31, ss_net_paid#32] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#31)), partial_avg(UnscaledValue(ss_net_paid#32))] - -(17) CometExchange -Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(18) CometHashAggregate -Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#31)), avg(UnscaledValue(ss_net_paid#32))] - -(19) CometProject -Input [3]: [count(1)#39, avg(ss_ext_discount_amt)#40, avg(ss_net_paid)#41] -Arguments: [mergedValue#42], [named_struct(count(1), count(1)#39, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#40, avg(ss_net_paid), avg(ss_net_paid)#41) AS mergedValue#42] - -(20) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#42] - -Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] - -Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] - -Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* CometColumnarToRow (28) -+- CometProject (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometFilter (22) - +- CometNativeScan parquet spark_catalog.default.store_sales (21) - - -(21) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] -ReadSchema: struct - -(22) CometFilter -Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Condition : ((isnotnull(ss_quantity#43) AND (ss_quantity#43 >= 41)) AND (ss_quantity#43 <= 60)) - -(23) CometProject -Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Arguments: [ss_ext_discount_amt#44, ss_net_paid#45], [ss_ext_discount_amt#44, ss_net_paid#45] - -(24) CometHashAggregate -Input [2]: [ss_ext_discount_amt#44, ss_net_paid#45] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#44)), partial_avg(UnscaledValue(ss_net_paid#45))] - -(25) CometExchange -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(26) CometHashAggregate -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#44)), avg(UnscaledValue(ss_net_paid#45))] - -(27) CometProject -Input [3]: [count(1)#52, avg(ss_ext_discount_amt)#53, avg(ss_net_paid)#54] -Arguments: [mergedValue#55], [named_struct(count(1), count(1)#52, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#53, avg(ss_net_paid), avg(ss_net_paid)#54) AS mergedValue#55] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#55] - -Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] - -Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] - -Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* CometColumnarToRow (36) -+- CometProject (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometFilter (30) - +- CometNativeScan parquet spark_catalog.default.store_sales (29) - - -(29) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_quantity#56) AND (ss_quantity#56 >= 61)) AND (ss_quantity#56 <= 80)) - -(31) CometProject -Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Arguments: [ss_ext_discount_amt#57, ss_net_paid#58], [ss_ext_discount_amt#57, ss_net_paid#58] - -(32) CometHashAggregate -Input [2]: [ss_ext_discount_amt#57, ss_net_paid#58] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#57)), partial_avg(UnscaledValue(ss_net_paid#58))] - -(33) CometExchange -Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(34) CometHashAggregate -Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#57)), avg(UnscaledValue(ss_net_paid#58))] - -(35) CometProject -Input [3]: [count(1)#65, avg(ss_ext_discount_amt)#66, avg(ss_net_paid)#67] -Arguments: [mergedValue#68], [named_struct(count(1), count(1)#65, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#66, avg(ss_net_paid), avg(ss_net_paid)#67) AS mergedValue#68] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#68] - -Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] - -Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] - -Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* CometColumnarToRow (44) -+- CometProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometFilter (38) - +- CometNativeScan parquet spark_catalog.default.store_sales (37) - - -(37) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] -ReadSchema: struct - -(38) CometFilter -Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Condition : ((isnotnull(ss_quantity#69) AND (ss_quantity#69 >= 81)) AND (ss_quantity#69 <= 100)) - -(39) CometProject -Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Arguments: [ss_ext_discount_amt#70, ss_net_paid#71], [ss_ext_discount_amt#70, ss_net_paid#71] - -(40) CometHashAggregate -Input [2]: [ss_ext_discount_amt#70, ss_net_paid#71] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#70)), partial_avg(UnscaledValue(ss_net_paid#71))] - -(41) CometExchange -Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(42) CometHashAggregate -Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#70)), avg(UnscaledValue(ss_net_paid#71))] - -(43) CometProject -Input [3]: [count(1)#78, avg(ss_ext_discount_amt)#79, avg(ss_net_paid)#80] -Arguments: [mergedValue#81], [named_struct(count(1), count(1)#78, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#79, avg(ss_net_paid), avg(ss_net_paid)#80) AS mergedValue#81] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#81] - -Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] - -Subquery:15 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_datafusion/simplified.txt deleted file mode 100644 index 9593d6ebd1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_datafusion/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -WholeStageCodegen (1) - Project - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #1 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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 - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #2 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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 - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #3 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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 - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #4 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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 - Subquery #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #5 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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 - CometColumnarToRow - InputAdapter - CometFilter [r_reason_sk] - CometNativeScan parquet spark_catalog.default.reason [r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/explain.txt deleted file mode 100644 index b3f32555de..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,283 +0,0 @@ -== Physical Plan == -* Project (4) -+- * CometColumnarToRow (3) - +- CometFilter (2) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (1) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Output [1]: [r_reason_sk#1] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk), EqualTo(r_reason_sk,1)] -ReadSchema: struct - -(2) CometFilter -Input [1]: [r_reason_sk#1] -Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) - -(3) CometColumnarToRow [codegen id : 1] -Input [1]: [r_reason_sk#1] - -(4) Project [codegen id : 1] -Output [5]: [CASE WHEN (Subquery scalar-subquery#2, [id=#3].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket1#4, CASE WHEN (Subquery scalar-subquery#5, [id=#6].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_net_paid) END AS bucket2#7, CASE WHEN (Subquery scalar-subquery#8, [id=#9].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_net_paid) END AS bucket3#10, CASE WHEN (Subquery scalar-subquery#11, [id=#12].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_net_paid) END AS bucket4#13, CASE WHEN (Subquery scalar-subquery#14, [id=#15].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_net_paid) END AS bucket5#16] -Input [1]: [r_reason_sk#1] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] -* CometColumnarToRow (12) -+- CometProject (11) - +- CometHashAggregate (10) - +- CometExchange (9) - +- CometHashAggregate (8) - +- CometProject (7) - +- CometFilter (6) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] -ReadSchema: struct - -(6) CometFilter -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Condition : ((isnotnull(ss_quantity#17) AND (ss_quantity#17 >= 1)) AND (ss_quantity#17 <= 20)) - -(7) CometProject -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Arguments: [ss_ext_discount_amt#18, ss_net_paid#19], [ss_ext_discount_amt#18, ss_net_paid#19] - -(8) CometHashAggregate -Input [2]: [ss_ext_discount_amt#18, ss_net_paid#19] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#18)), partial_avg(UnscaledValue(ss_net_paid#19))] - -(9) CometExchange -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometHashAggregate -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(UnscaledValue(ss_net_paid#19))] - -(11) CometProject -Input [3]: [count(1)#26, avg(ss_ext_discount_amt)#27, avg(ss_net_paid)#28] -Arguments: [mergedValue#29], [named_struct(count(1), count(1)#26, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#27, avg(ss_net_paid), avg(ss_net_paid)#28) AS mergedValue#29] - -(12) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#29] - -Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] - -Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] - -Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] -* CometColumnarToRow (20) -+- CometProject (19) - +- CometHashAggregate (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometProject (15) - +- CometFilter (14) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Condition : ((isnotnull(ss_quantity#30) AND (ss_quantity#30 >= 21)) AND (ss_quantity#30 <= 40)) - -(15) CometProject -Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Arguments: [ss_ext_discount_amt#31, ss_net_paid#32], [ss_ext_discount_amt#31, ss_net_paid#32] - -(16) CometHashAggregate -Input [2]: [ss_ext_discount_amt#31, ss_net_paid#32] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#31)), partial_avg(UnscaledValue(ss_net_paid#32))] - -(17) CometExchange -Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(18) CometHashAggregate -Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#31)), avg(UnscaledValue(ss_net_paid#32))] - -(19) CometProject -Input [3]: [count(1)#39, avg(ss_ext_discount_amt)#40, avg(ss_net_paid)#41] -Arguments: [mergedValue#42], [named_struct(count(1), count(1)#39, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#40, avg(ss_net_paid), avg(ss_net_paid)#41) AS mergedValue#42] - -(20) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#42] - -Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] - -Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] - -Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* CometColumnarToRow (28) -+- CometProject (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] -ReadSchema: struct - -(22) CometFilter -Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Condition : ((isnotnull(ss_quantity#43) AND (ss_quantity#43 >= 41)) AND (ss_quantity#43 <= 60)) - -(23) CometProject -Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Arguments: [ss_ext_discount_amt#44, ss_net_paid#45], [ss_ext_discount_amt#44, ss_net_paid#45] - -(24) CometHashAggregate -Input [2]: [ss_ext_discount_amt#44, ss_net_paid#45] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#44)), partial_avg(UnscaledValue(ss_net_paid#45))] - -(25) CometExchange -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(26) CometHashAggregate -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#44)), avg(UnscaledValue(ss_net_paid#45))] - -(27) CometProject -Input [3]: [count(1)#52, avg(ss_ext_discount_amt)#53, avg(ss_net_paid)#54] -Arguments: [mergedValue#55], [named_struct(count(1), count(1)#52, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#53, avg(ss_net_paid), avg(ss_net_paid)#54) AS mergedValue#55] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#55] - -Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] - -Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] - -Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* CometColumnarToRow (36) -+- CometProject (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (29) - - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_quantity#56) AND (ss_quantity#56 >= 61)) AND (ss_quantity#56 <= 80)) - -(31) CometProject -Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Arguments: [ss_ext_discount_amt#57, ss_net_paid#58], [ss_ext_discount_amt#57, ss_net_paid#58] - -(32) CometHashAggregate -Input [2]: [ss_ext_discount_amt#57, ss_net_paid#58] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#57)), partial_avg(UnscaledValue(ss_net_paid#58))] - -(33) CometExchange -Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(34) CometHashAggregate -Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#57)), avg(UnscaledValue(ss_net_paid#58))] - -(35) CometProject -Input [3]: [count(1)#65, avg(ss_ext_discount_amt)#66, avg(ss_net_paid)#67] -Arguments: [mergedValue#68], [named_struct(count(1), count(1)#65, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#66, avg(ss_net_paid), avg(ss_net_paid)#67) AS mergedValue#68] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#68] - -Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] - -Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] - -Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* CometColumnarToRow (44) -+- CometProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (37) - - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] -ReadSchema: struct - -(38) CometFilter -Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Condition : ((isnotnull(ss_quantity#69) AND (ss_quantity#69 >= 81)) AND (ss_quantity#69 <= 100)) - -(39) CometProject -Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Arguments: [ss_ext_discount_amt#70, ss_net_paid#71], [ss_ext_discount_amt#70, ss_net_paid#71] - -(40) CometHashAggregate -Input [2]: [ss_ext_discount_amt#70, ss_net_paid#71] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#70)), partial_avg(UnscaledValue(ss_net_paid#71))] - -(41) CometExchange -Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(42) CometHashAggregate -Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#70)), avg(UnscaledValue(ss_net_paid#71))] - -(43) CometProject -Input [3]: [count(1)#78, avg(ss_ext_discount_amt)#79, avg(ss_net_paid)#80] -Arguments: [mergedValue#81], [named_struct(count(1), count(1)#78, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#79, avg(ss_net_paid), avg(ss_net_paid)#80) AS mergedValue#81] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#81] - -Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] - -Subquery:15 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/simplified.txt deleted file mode 100644 index 558f5f4b36..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -WholeStageCodegen (1) - Project - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #1 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #2 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #3 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #4 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #5 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometColumnarToRow - InputAdapter - CometFilter [r_reason_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.reason [r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/explain.txt deleted file mode 100644 index b3f32555de..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/explain.txt +++ /dev/null @@ -1,283 +0,0 @@ -== Physical Plan == -* Project (4) -+- * CometColumnarToRow (3) - +- CometFilter (2) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (1) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Output [1]: [r_reason_sk#1] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk), EqualTo(r_reason_sk,1)] -ReadSchema: struct - -(2) CometFilter -Input [1]: [r_reason_sk#1] -Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) - -(3) CometColumnarToRow [codegen id : 1] -Input [1]: [r_reason_sk#1] - -(4) Project [codegen id : 1] -Output [5]: [CASE WHEN (Subquery scalar-subquery#2, [id=#3].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket1#4, CASE WHEN (Subquery scalar-subquery#5, [id=#6].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_net_paid) END AS bucket2#7, CASE WHEN (Subquery scalar-subquery#8, [id=#9].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_net_paid) END AS bucket3#10, CASE WHEN (Subquery scalar-subquery#11, [id=#12].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_net_paid) END AS bucket4#13, CASE WHEN (Subquery scalar-subquery#14, [id=#15].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_net_paid) END AS bucket5#16] -Input [1]: [r_reason_sk#1] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] -* CometColumnarToRow (12) -+- CometProject (11) - +- CometHashAggregate (10) - +- CometExchange (9) - +- CometHashAggregate (8) - +- CometProject (7) - +- CometFilter (6) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] -ReadSchema: struct - -(6) CometFilter -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Condition : ((isnotnull(ss_quantity#17) AND (ss_quantity#17 >= 1)) AND (ss_quantity#17 <= 20)) - -(7) CometProject -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Arguments: [ss_ext_discount_amt#18, ss_net_paid#19], [ss_ext_discount_amt#18, ss_net_paid#19] - -(8) CometHashAggregate -Input [2]: [ss_ext_discount_amt#18, ss_net_paid#19] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#18)), partial_avg(UnscaledValue(ss_net_paid#19))] - -(9) CometExchange -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometHashAggregate -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(UnscaledValue(ss_net_paid#19))] - -(11) CometProject -Input [3]: [count(1)#26, avg(ss_ext_discount_amt)#27, avg(ss_net_paid)#28] -Arguments: [mergedValue#29], [named_struct(count(1), count(1)#26, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#27, avg(ss_net_paid), avg(ss_net_paid)#28) AS mergedValue#29] - -(12) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#29] - -Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] - -Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] - -Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] -* CometColumnarToRow (20) -+- CometProject (19) - +- CometHashAggregate (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometProject (15) - +- CometFilter (14) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Condition : ((isnotnull(ss_quantity#30) AND (ss_quantity#30 >= 21)) AND (ss_quantity#30 <= 40)) - -(15) CometProject -Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Arguments: [ss_ext_discount_amt#31, ss_net_paid#32], [ss_ext_discount_amt#31, ss_net_paid#32] - -(16) CometHashAggregate -Input [2]: [ss_ext_discount_amt#31, ss_net_paid#32] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#31)), partial_avg(UnscaledValue(ss_net_paid#32))] - -(17) CometExchange -Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(18) CometHashAggregate -Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#31)), avg(UnscaledValue(ss_net_paid#32))] - -(19) CometProject -Input [3]: [count(1)#39, avg(ss_ext_discount_amt)#40, avg(ss_net_paid)#41] -Arguments: [mergedValue#42], [named_struct(count(1), count(1)#39, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#40, avg(ss_net_paid), avg(ss_net_paid)#41) AS mergedValue#42] - -(20) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#42] - -Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] - -Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] - -Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* CometColumnarToRow (28) -+- CometProject (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] -ReadSchema: struct - -(22) CometFilter -Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Condition : ((isnotnull(ss_quantity#43) AND (ss_quantity#43 >= 41)) AND (ss_quantity#43 <= 60)) - -(23) CometProject -Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Arguments: [ss_ext_discount_amt#44, ss_net_paid#45], [ss_ext_discount_amt#44, ss_net_paid#45] - -(24) CometHashAggregate -Input [2]: [ss_ext_discount_amt#44, ss_net_paid#45] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#44)), partial_avg(UnscaledValue(ss_net_paid#45))] - -(25) CometExchange -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(26) CometHashAggregate -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#44)), avg(UnscaledValue(ss_net_paid#45))] - -(27) CometProject -Input [3]: [count(1)#52, avg(ss_ext_discount_amt)#53, avg(ss_net_paid)#54] -Arguments: [mergedValue#55], [named_struct(count(1), count(1)#52, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#53, avg(ss_net_paid), avg(ss_net_paid)#54) AS mergedValue#55] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#55] - -Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] - -Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] - -Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* CometColumnarToRow (36) -+- CometProject (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (29) - - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_quantity#56) AND (ss_quantity#56 >= 61)) AND (ss_quantity#56 <= 80)) - -(31) CometProject -Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Arguments: [ss_ext_discount_amt#57, ss_net_paid#58], [ss_ext_discount_amt#57, ss_net_paid#58] - -(32) CometHashAggregate -Input [2]: [ss_ext_discount_amt#57, ss_net_paid#58] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#57)), partial_avg(UnscaledValue(ss_net_paid#58))] - -(33) CometExchange -Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(34) CometHashAggregate -Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#57)), avg(UnscaledValue(ss_net_paid#58))] - -(35) CometProject -Input [3]: [count(1)#65, avg(ss_ext_discount_amt)#66, avg(ss_net_paid)#67] -Arguments: [mergedValue#68], [named_struct(count(1), count(1)#65, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#66, avg(ss_net_paid), avg(ss_net_paid)#67) AS mergedValue#68] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#68] - -Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] - -Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] - -Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* CometColumnarToRow (44) -+- CometProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (37) - - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] -ReadSchema: struct - -(38) CometFilter -Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Condition : ((isnotnull(ss_quantity#69) AND (ss_quantity#69 >= 81)) AND (ss_quantity#69 <= 100)) - -(39) CometProject -Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Arguments: [ss_ext_discount_amt#70, ss_net_paid#71], [ss_ext_discount_amt#70, ss_net_paid#71] - -(40) CometHashAggregate -Input [2]: [ss_ext_discount_amt#70, ss_net_paid#71] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#70)), partial_avg(UnscaledValue(ss_net_paid#71))] - -(41) CometExchange -Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(42) CometHashAggregate -Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#70)), avg(UnscaledValue(ss_net_paid#71))] - -(43) CometProject -Input [3]: [count(1)#78, avg(ss_ext_discount_amt)#79, avg(ss_net_paid)#80] -Arguments: [mergedValue#81], [named_struct(count(1), count(1)#78, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#79, avg(ss_net_paid), avg(ss_net_paid)#80) AS mergedValue#81] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#81] - -Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] - -Subquery:15 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/extended.txt deleted file mode 100644 index ed71033b26..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/extended.txt +++ /dev/null @@ -1,61 +0,0 @@ - Project [COMET: ] -: :- Subquery -: : +- CometColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: +- ReusedSubquery -+- CometColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason - -Comet accelerated 37 out of 53 eligible operators (69%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 558f5f4b36..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -WholeStageCodegen (1) - Project - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #1 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #2 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #3 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #4 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #5 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometColumnarToRow - InputAdapter - CometFilter [r_reason_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.reason [r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_datafusion/explain.txt deleted file mode 100644 index 73b54f439b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_datafusion/explain.txt +++ /dev/null @@ -1,260 +0,0 @@ -== Physical Plan == -* Project (47) -+- * BroadcastNestedLoopJoin Inner BuildRight (46) - :- * CometColumnarToRow (25) - : +- CometHashAggregate (24) - : +- CometExchange (23) - : +- CometHashAggregate (22) - : +- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometProject (15) - : : +- CometBroadcastHashJoin (14) - : : :- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (4) - : : +- CometBroadcastExchange (13) - : : +- CometProject (12) - : : +- CometFilter (11) - : : +- CometNativeScan parquet spark_catalog.default.time_dim (10) - : +- CometBroadcastExchange (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometNativeScan parquet spark_catalog.default.web_page (16) - +- BroadcastExchange (45) - +- * CometColumnarToRow (44) - +- CometHashAggregate (43) - +- CometExchange (42) - +- CometHashAggregate (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometNativeScan parquet spark_catalog.default.web_sales (26) - : : +- ReusedExchange (29) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometNativeScan parquet spark_catalog.default.time_dim (32) - +- ReusedExchange (38) - - -(1) CometNativeScan parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3)) - -(3) CometProject -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Arguments: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] - -(4) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#5, hd_dep_count#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,6), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 6)) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ws_ship_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, hd_demo_sk#5] -Arguments: [ws_sold_time_sk#1, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_web_page_sk#3] - -(10) CometNativeScan parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#7, t_hour#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,8), LessThanOrEqual(t_hour,9), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [t_time_sk#7, t_hour#8] -Condition : (((isnotnull(t_hour#8) AND (t_hour#8 >= 8)) AND (t_hour#8 <= 9)) AND isnotnull(t_time_sk#7)) - -(12) CometProject -Input [2]: [t_time_sk#7, t_hour#8] -Arguments: [t_time_sk#7], [t_time_sk#7] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#7] -Arguments: [t_time_sk#7] - -(14) CometBroadcastHashJoin -Left output [2]: [ws_sold_time_sk#1, ws_web_page_sk#3] -Right output [1]: [t_time_sk#7] -Arguments: [ws_sold_time_sk#1], [t_time_sk#7], Inner, BuildRight - -(15) CometProject -Input [3]: [ws_sold_time_sk#1, ws_web_page_sk#3, t_time_sk#7] -Arguments: [ws_web_page_sk#3], [ws_web_page_sk#3] - -(16) CometNativeScan parquet spark_catalog.default.web_page -Output [2]: [wp_web_page_sk#9, wp_char_count#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_char_count), GreaterThanOrEqual(wp_char_count,5000), LessThanOrEqual(wp_char_count,5200), IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [wp_web_page_sk#9, wp_char_count#10] -Condition : (((isnotnull(wp_char_count#10) AND (wp_char_count#10 >= 5000)) AND (wp_char_count#10 <= 5200)) AND isnotnull(wp_web_page_sk#9)) - -(18) CometProject -Input [2]: [wp_web_page_sk#9, wp_char_count#10] -Arguments: [wp_web_page_sk#9], [wp_web_page_sk#9] - -(19) CometBroadcastExchange -Input [1]: [wp_web_page_sk#9] -Arguments: [wp_web_page_sk#9] - -(20) CometBroadcastHashJoin -Left output [1]: [ws_web_page_sk#3] -Right output [1]: [wp_web_page_sk#9] -Arguments: [ws_web_page_sk#3], [wp_web_page_sk#9], Inner, BuildRight - -(21) CometProject -Input [2]: [ws_web_page_sk#3, wp_web_page_sk#9] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#11] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#11] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 2] -Input [1]: [amc#12] - -(26) CometNativeScan parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Condition : ((isnotnull(ws_ship_hdemo_sk#14) AND isnotnull(ws_sold_time_sk#13)) AND isnotnull(ws_web_page_sk#15)) - -(28) CometProject -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Arguments: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] - -(29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#17] - -(30) CometBroadcastHashJoin -Left output [3]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] -Right output [1]: [hd_demo_sk#17] -Arguments: [ws_ship_hdemo_sk#14], [hd_demo_sk#17], Inner, BuildRight - -(31) CometProject -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, hd_demo_sk#17] -Arguments: [ws_sold_time_sk#13, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_web_page_sk#15] - -(32) CometNativeScan parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#18, t_hour#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [t_time_sk#18, t_hour#19] -Condition : (((isnotnull(t_hour#19) AND (t_hour#19 >= 19)) AND (t_hour#19 <= 20)) AND isnotnull(t_time_sk#18)) - -(34) CometProject -Input [2]: [t_time_sk#18, t_hour#19] -Arguments: [t_time_sk#18], [t_time_sk#18] - -(35) CometBroadcastExchange -Input [1]: [t_time_sk#18] -Arguments: [t_time_sk#18] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_sold_time_sk#13, ws_web_page_sk#15] -Right output [1]: [t_time_sk#18] -Arguments: [ws_sold_time_sk#13], [t_time_sk#18], Inner, BuildRight - -(37) CometProject -Input [3]: [ws_sold_time_sk#13, ws_web_page_sk#15, t_time_sk#18] -Arguments: [ws_web_page_sk#15], [ws_web_page_sk#15] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [wp_web_page_sk#20] - -(39) CometBroadcastHashJoin -Left output [1]: [ws_web_page_sk#15] -Right output [1]: [wp_web_page_sk#20] -Arguments: [ws_web_page_sk#15], [wp_web_page_sk#20], Inner, BuildRight - -(40) CometProject -Input [2]: [ws_web_page_sk#15, wp_web_page_sk#20] - -(41) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(42) CometExchange -Input [1]: [count#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(43) CometHashAggregate -Input [1]: [count#21] -Keys: [] -Functions [1]: [count(1)] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [pmc#22] - -(45) BroadcastExchange -Input [1]: [pmc#22] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(46) BroadcastNestedLoopJoin [codegen id : 2] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 2] -Output [1]: [(cast(amc#12 as decimal(15,4)) / cast(pmc#22 as decimal(15,4))) AS am_pm_ratio#23] -Input [2]: [amc#12, pmc#22] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_datafusion/simplified.txt deleted file mode 100644 index a13072cd91..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_datafusion/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (2) - Project [amc,pmc] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [amc,count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_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_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange [wp_web_page_sk] #4 - CometProject [wp_web_page_sk] - CometFilter [wp_web_page_sk,wp_char_count] - CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [pmc,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_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_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometNativeScan 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 [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometNativeScan 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/q90.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_iceberg_compat/explain.txt deleted file mode 100644 index 08ae744b5a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,260 +0,0 @@ -== Physical Plan == -* Project (47) -+- * BroadcastNestedLoopJoin Inner BuildRight (46) - :- * CometColumnarToRow (25) - : +- CometHashAggregate (24) - : +- CometExchange (23) - : +- CometHashAggregate (22) - : +- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometProject (15) - : : +- CometBroadcastHashJoin (14) - : : :- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (4) - : : +- CometBroadcastExchange (13) - : : +- CometProject (12) - : : +- CometFilter (11) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (10) - : +- CometBroadcastExchange (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (16) - +- BroadcastExchange (45) - +- * CometColumnarToRow (44) - +- CometHashAggregate (43) - +- CometExchange (42) - +- CometHashAggregate (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (26) - : : +- ReusedExchange (29) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (32) - +- ReusedExchange (38) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3)) - -(3) CometProject -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Arguments: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#5, hd_dep_count#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,6), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 6)) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ws_ship_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, hd_demo_sk#5] -Arguments: [ws_sold_time_sk#1, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_web_page_sk#3] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#7, t_hour#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,8), LessThanOrEqual(t_hour,9), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [t_time_sk#7, t_hour#8] -Condition : (((isnotnull(t_hour#8) AND (t_hour#8 >= 8)) AND (t_hour#8 <= 9)) AND isnotnull(t_time_sk#7)) - -(12) CometProject -Input [2]: [t_time_sk#7, t_hour#8] -Arguments: [t_time_sk#7], [t_time_sk#7] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#7] -Arguments: [t_time_sk#7] - -(14) CometBroadcastHashJoin -Left output [2]: [ws_sold_time_sk#1, ws_web_page_sk#3] -Right output [1]: [t_time_sk#7] -Arguments: [ws_sold_time_sk#1], [t_time_sk#7], Inner, BuildRight - -(15) CometProject -Input [3]: [ws_sold_time_sk#1, ws_web_page_sk#3, t_time_sk#7] -Arguments: [ws_web_page_sk#3], [ws_web_page_sk#3] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [2]: [wp_web_page_sk#9, wp_char_count#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_char_count), GreaterThanOrEqual(wp_char_count,5000), LessThanOrEqual(wp_char_count,5200), IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [wp_web_page_sk#9, wp_char_count#10] -Condition : (((isnotnull(wp_char_count#10) AND (wp_char_count#10 >= 5000)) AND (wp_char_count#10 <= 5200)) AND isnotnull(wp_web_page_sk#9)) - -(18) CometProject -Input [2]: [wp_web_page_sk#9, wp_char_count#10] -Arguments: [wp_web_page_sk#9], [wp_web_page_sk#9] - -(19) CometBroadcastExchange -Input [1]: [wp_web_page_sk#9] -Arguments: [wp_web_page_sk#9] - -(20) CometBroadcastHashJoin -Left output [1]: [ws_web_page_sk#3] -Right output [1]: [wp_web_page_sk#9] -Arguments: [ws_web_page_sk#3], [wp_web_page_sk#9], Inner, BuildRight - -(21) CometProject -Input [2]: [ws_web_page_sk#3, wp_web_page_sk#9] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#11] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#11] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 2] -Input [1]: [amc#12] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Condition : ((isnotnull(ws_ship_hdemo_sk#14) AND isnotnull(ws_sold_time_sk#13)) AND isnotnull(ws_web_page_sk#15)) - -(28) CometProject -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Arguments: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] - -(29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#17] - -(30) CometBroadcastHashJoin -Left output [3]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] -Right output [1]: [hd_demo_sk#17] -Arguments: [ws_ship_hdemo_sk#14], [hd_demo_sk#17], Inner, BuildRight - -(31) CometProject -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, hd_demo_sk#17] -Arguments: [ws_sold_time_sk#13, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_web_page_sk#15] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#18, t_hour#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [t_time_sk#18, t_hour#19] -Condition : (((isnotnull(t_hour#19) AND (t_hour#19 >= 19)) AND (t_hour#19 <= 20)) AND isnotnull(t_time_sk#18)) - -(34) CometProject -Input [2]: [t_time_sk#18, t_hour#19] -Arguments: [t_time_sk#18], [t_time_sk#18] - -(35) CometBroadcastExchange -Input [1]: [t_time_sk#18] -Arguments: [t_time_sk#18] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_sold_time_sk#13, ws_web_page_sk#15] -Right output [1]: [t_time_sk#18] -Arguments: [ws_sold_time_sk#13], [t_time_sk#18], Inner, BuildRight - -(37) CometProject -Input [3]: [ws_sold_time_sk#13, ws_web_page_sk#15, t_time_sk#18] -Arguments: [ws_web_page_sk#15], [ws_web_page_sk#15] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [wp_web_page_sk#20] - -(39) CometBroadcastHashJoin -Left output [1]: [ws_web_page_sk#15] -Right output [1]: [wp_web_page_sk#20] -Arguments: [ws_web_page_sk#15], [wp_web_page_sk#20], Inner, BuildRight - -(40) CometProject -Input [2]: [ws_web_page_sk#15, wp_web_page_sk#20] - -(41) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(42) CometExchange -Input [1]: [count#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(43) CometHashAggregate -Input [1]: [count#21] -Keys: [] -Functions [1]: [count(1)] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [pmc#22] - -(45) BroadcastExchange -Input [1]: [pmc#22] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(46) BroadcastNestedLoopJoin [codegen id : 2] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 2] -Output [1]: [(cast(amc#12 as decimal(15,4)) / cast(pmc#22 as decimal(15,4))) AS am_pm_ratio#23] -Input [2]: [amc#12, pmc#22] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_iceberg_compat/simplified.txt deleted file mode 100644 index 0991e4e0dc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (2) - Project [amc,pmc] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [amc,count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_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_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange [wp_web_page_sk] #4 - CometProject [wp_web_page_sk] - CometFilter [wp_web_page_sk,wp_char_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [pmc,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_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_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometScan [native_iceberg_compat] 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/q90/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/explain.txt deleted file mode 100644 index 08ae744b5a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/explain.txt +++ /dev/null @@ -1,260 +0,0 @@ -== Physical Plan == -* Project (47) -+- * BroadcastNestedLoopJoin Inner BuildRight (46) - :- * CometColumnarToRow (25) - : +- CometHashAggregate (24) - : +- CometExchange (23) - : +- CometHashAggregate (22) - : +- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometProject (15) - : : +- CometBroadcastHashJoin (14) - : : :- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (4) - : : +- CometBroadcastExchange (13) - : : +- CometProject (12) - : : +- CometFilter (11) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (10) - : +- CometBroadcastExchange (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (16) - +- BroadcastExchange (45) - +- * CometColumnarToRow (44) - +- CometHashAggregate (43) - +- CometExchange (42) - +- CometHashAggregate (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (26) - : : +- ReusedExchange (29) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (32) - +- ReusedExchange (38) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3)) - -(3) CometProject -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Arguments: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#5, hd_dep_count#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,6), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 6)) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ws_ship_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, hd_demo_sk#5] -Arguments: [ws_sold_time_sk#1, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_web_page_sk#3] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#7, t_hour#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,8), LessThanOrEqual(t_hour,9), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [t_time_sk#7, t_hour#8] -Condition : (((isnotnull(t_hour#8) AND (t_hour#8 >= 8)) AND (t_hour#8 <= 9)) AND isnotnull(t_time_sk#7)) - -(12) CometProject -Input [2]: [t_time_sk#7, t_hour#8] -Arguments: [t_time_sk#7], [t_time_sk#7] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#7] -Arguments: [t_time_sk#7] - -(14) CometBroadcastHashJoin -Left output [2]: [ws_sold_time_sk#1, ws_web_page_sk#3] -Right output [1]: [t_time_sk#7] -Arguments: [ws_sold_time_sk#1], [t_time_sk#7], Inner, BuildRight - -(15) CometProject -Input [3]: [ws_sold_time_sk#1, ws_web_page_sk#3, t_time_sk#7] -Arguments: [ws_web_page_sk#3], [ws_web_page_sk#3] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [2]: [wp_web_page_sk#9, wp_char_count#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_char_count), GreaterThanOrEqual(wp_char_count,5000), LessThanOrEqual(wp_char_count,5200), IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [wp_web_page_sk#9, wp_char_count#10] -Condition : (((isnotnull(wp_char_count#10) AND (wp_char_count#10 >= 5000)) AND (wp_char_count#10 <= 5200)) AND isnotnull(wp_web_page_sk#9)) - -(18) CometProject -Input [2]: [wp_web_page_sk#9, wp_char_count#10] -Arguments: [wp_web_page_sk#9], [wp_web_page_sk#9] - -(19) CometBroadcastExchange -Input [1]: [wp_web_page_sk#9] -Arguments: [wp_web_page_sk#9] - -(20) CometBroadcastHashJoin -Left output [1]: [ws_web_page_sk#3] -Right output [1]: [wp_web_page_sk#9] -Arguments: [ws_web_page_sk#3], [wp_web_page_sk#9], Inner, BuildRight - -(21) CometProject -Input [2]: [ws_web_page_sk#3, wp_web_page_sk#9] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#11] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#11] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 2] -Input [1]: [amc#12] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Condition : ((isnotnull(ws_ship_hdemo_sk#14) AND isnotnull(ws_sold_time_sk#13)) AND isnotnull(ws_web_page_sk#15)) - -(28) CometProject -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Arguments: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] - -(29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#17] - -(30) CometBroadcastHashJoin -Left output [3]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] -Right output [1]: [hd_demo_sk#17] -Arguments: [ws_ship_hdemo_sk#14], [hd_demo_sk#17], Inner, BuildRight - -(31) CometProject -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, hd_demo_sk#17] -Arguments: [ws_sold_time_sk#13, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_web_page_sk#15] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#18, t_hour#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [t_time_sk#18, t_hour#19] -Condition : (((isnotnull(t_hour#19) AND (t_hour#19 >= 19)) AND (t_hour#19 <= 20)) AND isnotnull(t_time_sk#18)) - -(34) CometProject -Input [2]: [t_time_sk#18, t_hour#19] -Arguments: [t_time_sk#18], [t_time_sk#18] - -(35) CometBroadcastExchange -Input [1]: [t_time_sk#18] -Arguments: [t_time_sk#18] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_sold_time_sk#13, ws_web_page_sk#15] -Right output [1]: [t_time_sk#18] -Arguments: [ws_sold_time_sk#13], [t_time_sk#18], Inner, BuildRight - -(37) CometProject -Input [3]: [ws_sold_time_sk#13, ws_web_page_sk#15, t_time_sk#18] -Arguments: [ws_web_page_sk#15], [ws_web_page_sk#15] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [wp_web_page_sk#20] - -(39) CometBroadcastHashJoin -Left output [1]: [ws_web_page_sk#15] -Right output [1]: [wp_web_page_sk#20] -Arguments: [ws_web_page_sk#15], [wp_web_page_sk#20], Inner, BuildRight - -(40) CometProject -Input [2]: [ws_web_page_sk#15, wp_web_page_sk#20] - -(41) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(42) CometExchange -Input [1]: [count#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(43) CometHashAggregate -Input [1]: [count#21] -Keys: [] -Functions [1]: [count(1)] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [pmc#22] - -(45) BroadcastExchange -Input [1]: [pmc#22] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(46) BroadcastNestedLoopJoin [codegen id : 2] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 2] -Output [1]: [(cast(amc#12 as decimal(15,4)) / cast(pmc#22 as decimal(15,4))) AS am_pm_ratio#23] -Input [2]: [amc#12, pmc#22] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/extended.txt deleted file mode 100644 index aa6c577ed7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/extended.txt +++ /dev/null @@ -1,55 +0,0 @@ -Project -+- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- BroadcastExchange - +- CometColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - -Comet accelerated 48 out of 51 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 0991e4e0dc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (2) - Project [amc,pmc] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [amc,count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_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_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange [wp_web_page_sk] #4 - CometProject [wp_web_page_sk] - CometFilter [wp_web_page_sk,wp_char_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [pmc,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_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_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/explain.txt deleted file mode 100644 index f6cfa7bdef..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/explain.txt +++ /dev/null @@ -1,299 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometSort (46) - +- CometColumnarExchange (45) - +- * HashAggregate (44) - +- * CometColumnarToRow (43) - +- CometColumnarExchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (19) - : : : +- * BroadcastHashJoin Inner BuildRight (18) - : : : :- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Project (10) - : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : :- * CometColumnarToRow (4) - : : : : : : +- CometProject (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometNativeScan parquet spark_catalog.default.call_center (1) - : : : : : +- BroadcastExchange (8) - : : : : : +- * Filter (7) - : : : : : +- * ColumnarToRow (6) - : : : : : +- Scan parquet spark_catalog.default.catalog_returns (5) - : : : : +- ReusedExchange (11) - : : : +- BroadcastExchange (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometFilter (15) - : : : +- CometNativeScan parquet spark_catalog.default.customer (14) - : : +- BroadcastExchange (24) - : : +- * CometColumnarToRow (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (20) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometNativeScan parquet spark_catalog.default.customer_demographics (27) - +- BroadcastExchange (38) - +- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometNativeScan parquet spark_catalog.default.household_demographics (34) - - -(1) CometNativeScan parquet spark_catalog.default.call_center -Output [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Condition : isnotnull(cc_call_center_sk#1) - -(3) CometProject -Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Arguments: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4], [cc_call_center_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(cc_call_center_id#2, 16)) AS cc_call_center_id#5, cc_name#3, cc_manager#4] - -(4) CometColumnarToRow [codegen id : 7] -Input [4]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4] - -(5) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#9), dynamicpruningexpression(cr_returned_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] - -(7) Filter [codegen id : 1] -Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Condition : (isnotnull(cr_call_center_sk#7) AND isnotnull(cr_returning_customer_sk#6)) - -(8) BroadcastExchange -Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cc_call_center_sk#1] -Right keys [1]: [cr_call_center_sk#7] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 7] -Output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9] -Input [8]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] - -(11) ReusedExchange [Reuses operator id: 52] -Output [1]: [d_date_sk#11] - -(12) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cr_returned_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 7] -Output [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8] -Input [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9, d_date_sk#11] - -(14) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#12, c_current_cdemo_sk#13, c_current_hdemo_sk#14, c_current_addr_sk#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [c_customer_sk#12, c_current_cdemo_sk#13, c_current_hdemo_sk#14, c_current_addr_sk#15] -Condition : (((isnotnull(c_customer_sk#12) AND isnotnull(c_current_addr_sk#15)) AND isnotnull(c_current_cdemo_sk#13)) AND isnotnull(c_current_hdemo_sk#14)) - -(16) CometColumnarToRow [codegen id : 3] -Input [4]: [c_customer_sk#12, c_current_cdemo_sk#13, c_current_hdemo_sk#14, c_current_addr_sk#15] - -(17) BroadcastExchange -Input [4]: [c_customer_sk#12, c_current_cdemo_sk#13, c_current_hdemo_sk#14, c_current_addr_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cr_returning_customer_sk#6] -Right keys [1]: [c_customer_sk#12] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 7] -Output [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#13, c_current_hdemo_sk#14, c_current_addr_sk#15] -Input [9]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, c_customer_sk#12, c_current_cdemo_sk#13, c_current_hdemo_sk#14, c_current_addr_sk#15] - -(20) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_gmt_offset#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-7.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [ca_address_sk#16, ca_gmt_offset#17] -Condition : ((isnotnull(ca_gmt_offset#17) AND (ca_gmt_offset#17 = -7.00)) AND isnotnull(ca_address_sk#16)) - -(22) CometProject -Input [2]: [ca_address_sk#16, ca_gmt_offset#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(23) CometColumnarToRow [codegen id : 4] -Input [1]: [ca_address_sk#16] - -(24) BroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(25) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#15] -Right keys [1]: [ca_address_sk#16] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 7] -Output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#13, c_current_hdemo_sk#14] -Input [8]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#13, c_current_hdemo_sk#14, c_current_addr_sk#15, ca_address_sk#16] - -(27) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(28) CometFilter -Input [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] -Condition : ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#19, 1)) = M) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#20, 20)) = Unknown )) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#19, 1)) = W) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#20, 20)) = Advanced Degree ))) AND isnotnull(cd_demo_sk#18)) - -(29) CometProject -Input [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] -Arguments: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22], [cd_demo_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#19, 1)) AS cd_marital_status#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#20, 20)) AS cd_education_status#22] - -(30) CometColumnarToRow [codegen id : 5] -Input [3]: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] - -(31) BroadcastExchange -Input [3]: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#13] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 7] -Output [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#14, cd_marital_status#21, cd_education_status#22] -Input [9]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#13, c_current_hdemo_sk#14, cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] - -(34) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#23, hd_buy_potential#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [hd_demo_sk#23, hd_buy_potential#24] -Condition : ((isnotnull(hd_buy_potential#24) AND StartsWith(static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#24, 15)), Unknown)) AND isnotnull(hd_demo_sk#23)) - -(36) CometProject -Input [2]: [hd_demo_sk#23, hd_buy_potential#24] -Arguments: [hd_demo_sk#23], [hd_demo_sk#23] - -(37) CometColumnarToRow [codegen id : 6] -Input [1]: [hd_demo_sk#23] - -(38) BroadcastExchange -Input [1]: [hd_demo_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_hdemo_sk#14] -Right keys [1]: [hd_demo_sk#23] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 7] -Output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#21, cd_education_status#22] -Input [8]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#14, cd_marital_status#21, cd_education_status#22, hd_demo_sk#23] - -(41) HashAggregate [codegen id : 7] -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#21, cd_education_status#22] -Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#21, cd_education_status#22] -Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#8))] -Aggregate Attributes [1]: [sum#25] -Results [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#21, cd_education_status#22, sum#26] - -(42) CometColumnarExchange -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#21, cd_education_status#22, sum#26] -Arguments: hashpartitioning(cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#21, cd_education_status#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) CometColumnarToRow [codegen id : 8] -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#21, cd_education_status#22, sum#26] - -(44) HashAggregate [codegen id : 8] -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#21, cd_education_status#22, sum#26] -Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#21, cd_education_status#22] -Functions [1]: [sum(UnscaledValue(cr_net_loss#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#8))#27] -Results [4]: [cc_call_center_id#5 AS Call_Center#28, cc_name#3 AS Call_Center_Name#29, cc_manager#4 AS Manager#30, MakeDecimal(sum(UnscaledValue(cr_net_loss#8))#27,17,2) AS Returns_Loss#31] - -(45) CometColumnarExchange -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(46) CometSort -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31], [Returns_Loss#31 DESC NULLS LAST] - -(47) CometColumnarToRow [codegen id : 9] -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = cr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometNativeScan parquet spark_catalog.default.date_dim (48) - - -(48) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#32, d_moy#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#11, d_year#32, d_moy#33] -Condition : ((((isnotnull(d_year#32) AND isnotnull(d_moy#33)) AND (d_year#32 = 1998)) AND (d_moy#33 = 11)) AND isnotnull(d_date_sk#11)) - -(50) CometProject -Input [3]: [d_date_sk#11, d_year#32, d_moy#33] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(52) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/simplified.txt deleted file mode 100644 index 75fea0ec7f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/simplified.txt +++ /dev/null @@ -1,77 +0,0 @@ -WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] - CometColumnarExchange [Returns_Loss] #1 - WholeStageCodegen (8) - HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] [sum(UnscaledValue(cr_net_loss)),Call_Center,Call_Center_Name,Manager,Returns_Loss,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 - WholeStageCodegen (7) - HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] [sum,sum] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - BroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] - BroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] - CometColumnarToRow - InputAdapter - CometProject [cc_call_center_id] [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [cr_call_center_sk,cr_returning_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometNativeScan 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/q91.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/explain.txt deleted file mode 100644 index f4fc7165aa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,280 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (43) -+- CometSort (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (19) - : : : +- CometBroadcastHashJoin (18) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometProject (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (4) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : : +- CometBroadcastExchange (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (15) - : : +- CometBroadcastExchange (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Condition : isnotnull(cc_call_center_sk#1) - -(3) CometProject -Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Arguments: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4], [cc_call_center_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(cc_call_center_id#2, 16)) AS cc_call_center_id#5, cc_name#3, cc_manager#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#9), dynamicpruningexpression(cr_returned_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Condition : (isnotnull(cr_call_center_sk#7) AND isnotnull(cr_returning_customer_sk#6)) - -(6) CometBroadcastExchange -Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Arguments: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [4]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4] -Right output [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Arguments: [cc_call_center_sk#1], [cr_call_center_sk#7], Inner, BuildRight - -(8) CometProject -Input [8]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#13)) AND (d_year#12 = 1998)) AND (d_moy#13 = 11)) AND isnotnull(d_date_sk#11)) - -(11) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(13) CometBroadcastHashJoin -Left output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [cr_returned_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(14) CometProject -Input [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9, d_date_sk#11] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Condition : (((isnotnull(c_customer_sk#14) AND isnotnull(c_current_addr_sk#17)) AND isnotnull(c_current_cdemo_sk#15)) AND isnotnull(c_current_hdemo_sk#16)) - -(17) CometBroadcastExchange -Input [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Arguments: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] - -(18) CometBroadcastHashJoin -Left output [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8] -Right output [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Arguments: [cr_returning_customer_sk#6], [c_customer_sk#14], Inner, BuildRight - -(19) CometProject -Input [9]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_gmt_offset#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-7.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [ca_address_sk#18, ca_gmt_offset#19] -Condition : ((isnotnull(ca_gmt_offset#19) AND (ca_gmt_offset#19 = -7.00)) AND isnotnull(ca_address_sk#18)) - -(22) CometProject -Input [2]: [ca_address_sk#18, ca_gmt_offset#19] -Arguments: [ca_address_sk#18], [ca_address_sk#18] - -(23) CometBroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: [ca_address_sk#18] - -(24) CometBroadcastHashJoin -Left output [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Right output [1]: [ca_address_sk#18] -Arguments: [c_current_addr_sk#17], [ca_address_sk#18], Inner, BuildRight - -(25) CometProject -Input [8]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17, ca_address_sk#18] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Condition : ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#21, 1)) = M) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#22, 20)) = Unknown )) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#21, 1)) = W) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#22, 20)) = Advanced Degree ))) AND isnotnull(cd_demo_sk#20)) - -(28) CometProject -Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Arguments: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24], [cd_demo_sk#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#21, 1)) AS cd_marital_status#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#22, 20)) AS cd_education_status#24] - -(29) CometBroadcastExchange -Input [3]: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] -Arguments: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] - -(30) CometBroadcastHashJoin -Left output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16] -Right output [3]: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] -Arguments: [c_current_cdemo_sk#15], [cd_demo_sk#20], Inner, BuildRight - -(31) CometProject -Input [9]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#25, hd_buy_potential#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [hd_demo_sk#25, hd_buy_potential#26] -Condition : ((isnotnull(hd_buy_potential#26) AND StartsWith(static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#26, 15)), Unknown)) AND isnotnull(hd_demo_sk#25)) - -(34) CometProject -Input [2]: [hd_demo_sk#25, hd_buy_potential#26] -Arguments: [hd_demo_sk#25], [hd_demo_sk#25] - -(35) CometBroadcastExchange -Input [1]: [hd_demo_sk#25] -Arguments: [hd_demo_sk#25] - -(36) CometBroadcastHashJoin -Left output [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24] -Right output [1]: [hd_demo_sk#25] -Arguments: [c_current_hdemo_sk#16], [hd_demo_sk#25], Inner, BuildRight - -(37) CometProject -Input [8]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24, hd_demo_sk#25] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#23, cd_education_status#24], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#23, cd_education_status#24] - -(38) CometHashAggregate -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#23, cd_education_status#24] -Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24] -Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#8))] - -(39) CometExchange -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, sum#27] -Arguments: hashpartitioning(cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(40) CometHashAggregate -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, sum#27] -Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24] -Functions [1]: [sum(UnscaledValue(cr_net_loss#8))] - -(41) CometExchange -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(42) CometSort -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31], [Returns_Loss#31 DESC NULLS LAST] - -(43) CometColumnarToRow [codegen id : 1] -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = cr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) - - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#13)) AND (d_year#12 = 1998)) AND (d_moy#13 = 11)) AND isnotnull(d_date_sk#11)) - -(46) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(48) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/simplified.txt deleted file mode 100644 index 3e9b8945da..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,53 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] - CometExchange [Returns_Loss] #1 - CometHashAggregate [cd_marital_status,cd_education_status,sum] [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,sum(UnscaledValue(cr_net_loss))] - CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 - CometHashAggregate [cr_net_loss] [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - 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 [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 [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 [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 [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,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] - CometProject [cc_call_center_id] [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk] #7 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [hd_demo_sk] #9 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan [native_iceberg_compat] 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/q91/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/explain.txt deleted file mode 100644 index f4fc7165aa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/explain.txt +++ /dev/null @@ -1,280 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (43) -+- CometSort (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (19) - : : : +- CometBroadcastHashJoin (18) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometProject (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (4) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : : +- CometBroadcastExchange (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (15) - : : +- CometBroadcastExchange (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Condition : isnotnull(cc_call_center_sk#1) - -(3) CometProject -Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Arguments: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4], [cc_call_center_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(cc_call_center_id#2, 16)) AS cc_call_center_id#5, cc_name#3, cc_manager#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#9), dynamicpruningexpression(cr_returned_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Condition : (isnotnull(cr_call_center_sk#7) AND isnotnull(cr_returning_customer_sk#6)) - -(6) CometBroadcastExchange -Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Arguments: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [4]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4] -Right output [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Arguments: [cc_call_center_sk#1], [cr_call_center_sk#7], Inner, BuildRight - -(8) CometProject -Input [8]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#13)) AND (d_year#12 = 1998)) AND (d_moy#13 = 11)) AND isnotnull(d_date_sk#11)) - -(11) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(13) CometBroadcastHashJoin -Left output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [cr_returned_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(14) CometProject -Input [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9, d_date_sk#11] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Condition : (((isnotnull(c_customer_sk#14) AND isnotnull(c_current_addr_sk#17)) AND isnotnull(c_current_cdemo_sk#15)) AND isnotnull(c_current_hdemo_sk#16)) - -(17) CometBroadcastExchange -Input [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Arguments: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] - -(18) CometBroadcastHashJoin -Left output [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8] -Right output [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Arguments: [cr_returning_customer_sk#6], [c_customer_sk#14], Inner, BuildRight - -(19) CometProject -Input [9]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_gmt_offset#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-7.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [ca_address_sk#18, ca_gmt_offset#19] -Condition : ((isnotnull(ca_gmt_offset#19) AND (ca_gmt_offset#19 = -7.00)) AND isnotnull(ca_address_sk#18)) - -(22) CometProject -Input [2]: [ca_address_sk#18, ca_gmt_offset#19] -Arguments: [ca_address_sk#18], [ca_address_sk#18] - -(23) CometBroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: [ca_address_sk#18] - -(24) CometBroadcastHashJoin -Left output [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Right output [1]: [ca_address_sk#18] -Arguments: [c_current_addr_sk#17], [ca_address_sk#18], Inner, BuildRight - -(25) CometProject -Input [8]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17, ca_address_sk#18] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Condition : ((((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#21, 1)) = M) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#22, 20)) = Unknown )) OR ((static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#21, 1)) = W) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#22, 20)) = Advanced Degree ))) AND isnotnull(cd_demo_sk#20)) - -(28) CometProject -Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Arguments: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24], [cd_demo_sk#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#21, 1)) AS cd_marital_status#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#22, 20)) AS cd_education_status#24] - -(29) CometBroadcastExchange -Input [3]: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] -Arguments: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] - -(30) CometBroadcastHashJoin -Left output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16] -Right output [3]: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] -Arguments: [c_current_cdemo_sk#15], [cd_demo_sk#20], Inner, BuildRight - -(31) CometProject -Input [9]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#25, hd_buy_potential#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [hd_demo_sk#25, hd_buy_potential#26] -Condition : ((isnotnull(hd_buy_potential#26) AND StartsWith(static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#26, 15)), Unknown)) AND isnotnull(hd_demo_sk#25)) - -(34) CometProject -Input [2]: [hd_demo_sk#25, hd_buy_potential#26] -Arguments: [hd_demo_sk#25], [hd_demo_sk#25] - -(35) CometBroadcastExchange -Input [1]: [hd_demo_sk#25] -Arguments: [hd_demo_sk#25] - -(36) CometBroadcastHashJoin -Left output [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24] -Right output [1]: [hd_demo_sk#25] -Arguments: [c_current_hdemo_sk#16], [hd_demo_sk#25], Inner, BuildRight - -(37) CometProject -Input [8]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24, hd_demo_sk#25] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#23, cd_education_status#24], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#23, cd_education_status#24] - -(38) CometHashAggregate -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#23, cd_education_status#24] -Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24] -Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#8))] - -(39) CometExchange -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, sum#27] -Arguments: hashpartitioning(cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(40) CometHashAggregate -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, sum#27] -Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24] -Functions [1]: [sum(UnscaledValue(cr_net_loss#8))] - -(41) CometExchange -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(42) CometSort -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31], [Returns_Loss#31 DESC NULLS LAST] - -(43) CometColumnarToRow [codegen id : 1] -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = cr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) - - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#13)) AND (d_year#12 = 1998)) AND (d_moy#13 = 11)) AND isnotnull(d_date_sk#11)) - -(46) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(48) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/extended.txt deleted file mode 100644 index a1c6b73d37..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/extended.txt +++ /dev/null @@ -1,51 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 3e9b8945da..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt +++ /dev/null @@ -1,53 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] - CometExchange [Returns_Loss] #1 - CometHashAggregate [cd_marital_status,cd_education_status,sum] [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,sum(UnscaledValue(cr_net_loss))] - CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 - CometHashAggregate [cr_net_loss] [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - 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 [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 [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 [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 [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,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] - CometProject [cc_call_center_id] [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk] #7 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [hd_demo_sk] #9 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/explain.txt deleted file mode 100644 index d96384bc72..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -* HashAggregate (31) -+- * CometColumnarToRow (30) - +- CometColumnarExchange (29) - +- * HashAggregate (28) - +- * Project (27) - +- * BroadcastHashJoin Inner BuildRight (26) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : +- BroadcastExchange (8) - : : +- * CometColumnarToRow (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometNativeScan parquet spark_catalog.default.item (4) - : +- BroadcastExchange (22) - : +- * Filter (21) - : +- * HashAggregate (20) - : +- * CometColumnarToRow (19) - : +- CometColumnarExchange (18) - : +- * HashAggregate (17) - : +- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet spark_catalog.default.web_sales (11) - : +- ReusedExchange (14) - +- ReusedExchange (25) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_ext_discount_amt)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] - -(3) Filter [codegen id : 6] -Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Condition : (isnotnull(ws_item_sk#1) AND isnotnull(ws_ext_discount_amt#2)) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_manufact_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,350), IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [i_item_sk#5, i_manufact_id#6] -Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 350)) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [2]: [i_item_sk#5, i_manufact_id#6] -Arguments: [i_item_sk#5], [i_item_sk#5] - -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [i_item_sk#5] - -(8) BroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 6] -Output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] -Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] - -(11) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] - -(13) Filter [codegen id : 3] -Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Condition : isnotnull(ws_item_sk#7) - -(14) ReusedExchange [Reuses operator id: 36] -Output [1]: [d_date_sk#10] - -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(16) Project [codegen id : 3] -Output [2]: [ws_item_sk#7, ws_ext_discount_amt#8] -Input [4]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9, d_date_sk#10] - -(17) HashAggregate [codegen id : 3] -Input [2]: [ws_item_sk#7, ws_ext_discount_amt#8] -Keys [1]: [ws_item_sk#7] -Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#8))] -Aggregate Attributes [2]: [sum#11, count#12] -Results [3]: [ws_item_sk#7, sum#13, count#14] - -(18) CometColumnarExchange -Input [3]: [ws_item_sk#7, sum#13, count#14] -Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(19) CometColumnarToRow [codegen id : 4] -Input [3]: [ws_item_sk#7, sum#13, count#14] - -(20) HashAggregate [codegen id : 4] -Input [3]: [ws_item_sk#7, sum#13, count#14] -Keys [1]: [ws_item_sk#7] -Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))] -Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))#15] -Results [2]: [(1.3 * cast((avg(UnscaledValue(ws_ext_discount_amt#8))#15 / 100.0) as decimal(11,6))) AS (1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] - -(21) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] -Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#16) - -(22) BroadcastExchange -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] - -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [i_item_sk#5] -Right keys [1]: [ws_item_sk#7] -Join type: Inner -Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#16) - -(24) Project [codegen id : 6] -Output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] -Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] - -(25) ReusedExchange [Reuses operator id: 36] -Output [1]: [d_date_sk#17] - -(26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 6] -Output [1]: [ws_ext_discount_amt#2] -Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#17] - -(28) HashAggregate [codegen id : 6] -Input [1]: [ws_ext_discount_amt#2] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#18] -Results [1]: [sum#19] - -(29) CometColumnarExchange -Input [1]: [sum#19] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 7] -Input [1]: [sum#19] - -(31) HashAggregate [codegen id : 7] -Input [1]: [sum#19] -Keys: [] -Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#20] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#20,17,2) AS Excess Discount Amount #21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.date_dim (32) - - -(32) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#17, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 2000-01-27)) AND (d_date#22 <= 2000-04-26)) AND isnotnull(d_date_sk#17)) - -(34) CometProject -Input [2]: [d_date_sk#17, d_date#22] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] - -(36) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/simplified.txt deleted file mode 100644 index 17df728966..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -WholeStageCodegen (7) - HashAggregate [sum] [sum(UnscaledValue(ws_ext_discount_amt)),Excess Discount Amount ,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (6) - HashAggregate [ws_ext_discount_amt] [sum,sum] - Project [ws_ext_discount_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_discount_amt,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk,ws_ext_discount_amt,(1.3 * avg(ws_ext_discount_amt))] - Project [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk,ws_ext_discount_amt] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Filter [(1.3 * avg(ws_ext_discount_amt))] - HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk] #5 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,ws_ext_discount_amt] [sum,count,sum,count] - Project [ws_item_sk,ws_ext_discount_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/explain.txt deleted file mode 100644 index cdc8dbc3bc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (30) -+- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : +- CometBroadcastExchange (21) - : +- CometFilter (20) - : +- CometHashAggregate (19) - : +- CometExchange (18) - : +- CometHashAggregate (17) - : +- CometProject (16) - : +- CometBroadcastHashJoin (15) - : :- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (9) - : +- CometBroadcastExchange (14) - : +- CometProject (13) - : +- CometFilter (12) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (11) - +- ReusedExchange (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_ext_discount_amt)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Condition : (isnotnull(ws_item_sk#1) AND isnotnull(ws_ext_discount_amt#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_manufact_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,350), IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_manufact_id#6] -Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 350)) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [2]: [i_item_sk#5, i_manufact_id#6] -Arguments: [i_item_sk#5], [i_item_sk#5] - -(6) CometBroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: [i_item_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Right output [1]: [i_item_sk#5] -Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] -Arguments: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5], [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Condition : isnotnull(ws_item_sk#7) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) - -(13) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(15) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [ws_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(16) CometProject -Input [4]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9, d_date_sk#11] -Arguments: [ws_item_sk#7, ws_ext_discount_amt#8], [ws_item_sk#7, ws_ext_discount_amt#8] - -(17) CometHashAggregate -Input [2]: [ws_item_sk#7, ws_ext_discount_amt#8] -Keys [1]: [ws_item_sk#7] -Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#8))] - -(18) CometExchange -Input [3]: [ws_item_sk#7, sum#13, count#14] -Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [3]: [ws_item_sk#7, sum#13, count#14] -Keys [1]: [ws_item_sk#7] -Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))] - -(20) CometFilter -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#15) - -(21) CometBroadcastExchange -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Arguments: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] - -(22) CometBroadcastHashJoin -Left output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] -Right output [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Arguments: [i_item_sk#5], [ws_item_sk#7], Inner, (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#15), BuildRight - -(23) CometProject -Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Arguments: [ws_ext_discount_amt#2, ws_sold_date_sk#3], [ws_ext_discount_amt#2, ws_sold_date_sk#3] - -(24) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] -Right output [1]: [d_date_sk#16] -Arguments: [ws_sold_date_sk#3], [d_date_sk#16], Inner, BuildRight - -(26) CometProject -Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#16] -Arguments: [ws_ext_discount_amt#2], [ws_ext_discount_amt#2] - -(27) CometHashAggregate -Input [1]: [ws_ext_discount_amt#2] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] - -(28) CometExchange -Input [1]: [sum#17] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [1]: [sum#17] -Keys: [] -Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [Excess Discount Amount #18] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#16, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#19 <= 2000-04-26)) AND isnotnull(d_date_sk#16)) - -(33) CometProject -Input [2]: [d_date_sk#16, d_date#19] -Arguments: [d_date_sk#16], [d_date_sk#16] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#16] - -(35) BroadcastExchange -Input [1]: [d_date_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 9 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/simplified.txt deleted file mode 100644 index 4ec5755df7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [Excess Discount Amount ,sum(UnscaledValue(ws_ext_discount_amt))] - CometExchange #1 - CometHashAggregate [ws_ext_discount_amt] [sum] - CometProject [ws_ext_discount_amt] - CometBroadcastHashJoin [ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] - CometProject [ws_ext_discount_amt,ws_sold_date_sk] - CometBroadcastHashJoin [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk,(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] - CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk] #3 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] #4 - CometFilter [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] - CometHashAggregate [sum,count] [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,avg(UnscaledValue(ws_ext_discount_amt))] - CometExchange [ws_item_sk] #5 - CometHashAggregate [ws_ext_discount_amt] [ws_item_sk,sum,count] - CometProject [ws_item_sk,ws_ext_discount_amt] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/explain.txt deleted file mode 100644 index cdc8dbc3bc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (30) -+- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : +- CometBroadcastExchange (21) - : +- CometFilter (20) - : +- CometHashAggregate (19) - : +- CometExchange (18) - : +- CometHashAggregate (17) - : +- CometProject (16) - : +- CometBroadcastHashJoin (15) - : :- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (9) - : +- CometBroadcastExchange (14) - : +- CometProject (13) - : +- CometFilter (12) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (11) - +- ReusedExchange (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_ext_discount_amt)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Condition : (isnotnull(ws_item_sk#1) AND isnotnull(ws_ext_discount_amt#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_manufact_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,350), IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_manufact_id#6] -Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 350)) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [2]: [i_item_sk#5, i_manufact_id#6] -Arguments: [i_item_sk#5], [i_item_sk#5] - -(6) CometBroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: [i_item_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Right output [1]: [i_item_sk#5] -Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] -Arguments: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5], [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Condition : isnotnull(ws_item_sk#7) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) - -(13) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(15) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [ws_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(16) CometProject -Input [4]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9, d_date_sk#11] -Arguments: [ws_item_sk#7, ws_ext_discount_amt#8], [ws_item_sk#7, ws_ext_discount_amt#8] - -(17) CometHashAggregate -Input [2]: [ws_item_sk#7, ws_ext_discount_amt#8] -Keys [1]: [ws_item_sk#7] -Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#8))] - -(18) CometExchange -Input [3]: [ws_item_sk#7, sum#13, count#14] -Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [3]: [ws_item_sk#7, sum#13, count#14] -Keys [1]: [ws_item_sk#7] -Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))] - -(20) CometFilter -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#15) - -(21) CometBroadcastExchange -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Arguments: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] - -(22) CometBroadcastHashJoin -Left output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] -Right output [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Arguments: [i_item_sk#5], [ws_item_sk#7], Inner, (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#15), BuildRight - -(23) CometProject -Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Arguments: [ws_ext_discount_amt#2, ws_sold_date_sk#3], [ws_ext_discount_amt#2, ws_sold_date_sk#3] - -(24) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] -Right output [1]: [d_date_sk#16] -Arguments: [ws_sold_date_sk#3], [d_date_sk#16], Inner, BuildRight - -(26) CometProject -Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#16] -Arguments: [ws_ext_discount_amt#2], [ws_ext_discount_amt#2] - -(27) CometHashAggregate -Input [1]: [ws_ext_discount_amt#2] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] - -(28) CometExchange -Input [1]: [sum#17] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [1]: [sum#17] -Keys: [] -Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [Excess Discount Amount #18] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#16, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#19 <= 2000-04-26)) AND isnotnull(d_date_sk#16)) - -(33) CometProject -Input [2]: [d_date_sk#16, d_date#19] -Arguments: [d_date_sk#16], [d_date_sk#16] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#16] - -(35) BroadcastExchange -Input [1]: [d_date_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 9 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/extended.txt deleted file mode 100644 index 20df9a8047..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 4ec5755df7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [Excess Discount Amount ,sum(UnscaledValue(ws_ext_discount_amt))] - CometExchange #1 - CometHashAggregate [ws_ext_discount_amt] [sum] - CometProject [ws_ext_discount_amt] - CometBroadcastHashJoin [ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] - CometProject [ws_ext_discount_amt,ws_sold_date_sk] - CometBroadcastHashJoin [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk,(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] - CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk] #3 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] #4 - CometFilter [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] - CometHashAggregate [sum,count] [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,avg(UnscaledValue(ws_ext_discount_amt))] - CometExchange [ws_item_sk] #5 - CometHashAggregate [ws_ext_discount_amt] [ws_item_sk,sum,count] - CometProject [ws_item_sk,ws_ext_discount_amt] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_datafusion/explain.txt deleted file mode 100644 index 4b430e436e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_datafusion/explain.txt +++ /dev/null @@ -1,124 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (22) -+- CometTakeOrderedAndProject (21) - +- CometHashAggregate (20) - +- CometExchange (19) - +- CometHashAggregate (18) - +- CometProject (17) - +- CometBroadcastHashJoin (16) - :- CometProject (11) - : +- CometSortMergeJoin (10) - : :- CometSort (4) - : : +- CometExchange (3) - : : +- CometProject (2) - : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : +- CometSort (9) - : +- CometExchange (8) - : +- CometProject (7) - : +- CometFilter (6) - : +- CometNativeScan parquet spark_catalog.default.store_returns (5) - +- CometBroadcastExchange (15) - +- CometProject (14) - +- CometFilter (13) - +- CometNativeScan parquet spark_catalog.default.reason (12) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -ReadSchema: struct - -(2) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5], [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] - -(3) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#3 ASC NULLS FIRST] - -(5) CometNativeScan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_reason_sk)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Condition : ((isnotnull(sr_item_sk#7) AND isnotnull(sr_ticket_number#9)) AND isnotnull(sr_reason_sk#8)) - -(7) CometProject -Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] - -(8) CometExchange -Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: hashpartitioning(sr_item_sk#7, sr_ticket_number#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7 ASC NULLS FIRST, sr_ticket_number#9 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Right output [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [ss_item_sk#1, ss_ticket_number#3], [sr_item_sk#7, sr_ticket_number#9], Inner - -(11) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10], [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] - -(12) CometNativeScan parquet spark_catalog.default.reason -Output [2]: [r_reason_sk#12, r_reason_desc#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_desc), IsNotNull(r_reason_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [r_reason_sk#12, r_reason_desc#13] -Condition : ((isnotnull(r_reason_desc#13) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(r_reason_desc#13, 100)) = reason 28 )) AND isnotnull(r_reason_sk#12)) - -(14) CometProject -Input [2]: [r_reason_sk#12, r_reason_desc#13] -Arguments: [r_reason_sk#12], [r_reason_sk#12] - -(15) CometBroadcastExchange -Input [1]: [r_reason_sk#12] -Arguments: [r_reason_sk#12] - -(16) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] -Right output [1]: [r_reason_sk#12] -Arguments: [sr_reason_sk#8], [r_reason_sk#12], Inner, BuildRight - -(17) CometProject -Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10, r_reason_sk#12] -Arguments: [ss_customer_sk#2, act_sales#14], [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#10) THEN (cast((ss_quantity#4 - sr_return_quantity#10) as decimal(10,0)) * ss_sales_price#5) ELSE (cast(ss_quantity#4 as decimal(10,0)) * ss_sales_price#5) END AS act_sales#14] - -(18) CometHashAggregate -Input [2]: [ss_customer_sk#2, act_sales#14] -Keys [1]: [ss_customer_sk#2] -Functions [1]: [partial_sum(act_sales#14)] - -(19) CometExchange -Input [3]: [ss_customer_sk#2, sum#15, isEmpty#16] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometHashAggregate -Input [3]: [ss_customer_sk#2, sum#15, isEmpty#16] -Keys [1]: [ss_customer_sk#2] -Functions [1]: [sum(act_sales#14)] - -(21) CometTakeOrderedAndProject -Input [2]: [ss_customer_sk#2, sumsales#17] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sumsales#17 ASC NULLS FIRST,ss_customer_sk#2 ASC NULLS FIRST], output=[ss_customer_sk#2,sumsales#17]), [ss_customer_sk#2, sumsales#17], 100, 0, [sumsales#17 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#17] - -(22) CometColumnarToRow [codegen id : 1] -Input [2]: [ss_customer_sk#2, sumsales#17] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_datafusion/simplified.txt deleted file mode 100644 index c0b74010df..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_datafusion/simplified.txt +++ /dev/null @@ -1,24 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ss_customer_sk,sumsales] - CometHashAggregate [sum,isEmpty] [ss_customer_sk,sumsales,sum(act_sales)] - CometExchange [ss_customer_sk] #1 - CometHashAggregate [act_sales] [ss_customer_sk,sum,isEmpty] - CometProject [sr_return_quantity,ss_quantity,ss_sales_price] [ss_customer_sk,act_sales] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity,r_reason_sk] - CometProject [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometExchange [ss_item_sk,ss_ticket_number] #2 - CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometSort [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometExchange [sr_item_sk,sr_ticket_number] #3 - CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometBroadcastExchange [r_reason_sk] #4 - CometProject [r_reason_sk] - CometFilter [r_reason_sk,r_reason_desc] - CometNativeScan 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/q93.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_iceberg_compat/explain.txt deleted file mode 100644 index cbefa941c7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,124 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (22) -+- CometTakeOrderedAndProject (21) - +- CometHashAggregate (20) - +- CometExchange (19) - +- CometHashAggregate (18) - +- CometProject (17) - +- CometBroadcastHashJoin (16) - :- CometProject (11) - : +- CometSortMergeJoin (10) - : :- CometSort (4) - : : +- CometExchange (3) - : : +- CometProject (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometSort (9) - : +- CometExchange (8) - : +- CometProject (7) - : +- CometFilter (6) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - +- CometBroadcastExchange (15) - +- CometProject (14) - +- CometFilter (13) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (12) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -ReadSchema: struct - -(2) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5], [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] - -(3) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#3 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_reason_sk)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Condition : ((isnotnull(sr_item_sk#7) AND isnotnull(sr_ticket_number#9)) AND isnotnull(sr_reason_sk#8)) - -(7) CometProject -Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] - -(8) CometExchange -Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: hashpartitioning(sr_item_sk#7, sr_ticket_number#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7 ASC NULLS FIRST, sr_ticket_number#9 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Right output [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [ss_item_sk#1, ss_ticket_number#3], [sr_item_sk#7, sr_ticket_number#9], Inner - -(11) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10], [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Output [2]: [r_reason_sk#12, r_reason_desc#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_desc), IsNotNull(r_reason_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [r_reason_sk#12, r_reason_desc#13] -Condition : ((isnotnull(r_reason_desc#13) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(r_reason_desc#13, 100)) = reason 28 )) AND isnotnull(r_reason_sk#12)) - -(14) CometProject -Input [2]: [r_reason_sk#12, r_reason_desc#13] -Arguments: [r_reason_sk#12], [r_reason_sk#12] - -(15) CometBroadcastExchange -Input [1]: [r_reason_sk#12] -Arguments: [r_reason_sk#12] - -(16) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] -Right output [1]: [r_reason_sk#12] -Arguments: [sr_reason_sk#8], [r_reason_sk#12], Inner, BuildRight - -(17) CometProject -Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10, r_reason_sk#12] -Arguments: [ss_customer_sk#2, act_sales#14], [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#10) THEN (cast((ss_quantity#4 - sr_return_quantity#10) as decimal(10,0)) * ss_sales_price#5) ELSE (cast(ss_quantity#4 as decimal(10,0)) * ss_sales_price#5) END AS act_sales#14] - -(18) CometHashAggregate -Input [2]: [ss_customer_sk#2, act_sales#14] -Keys [1]: [ss_customer_sk#2] -Functions [1]: [partial_sum(act_sales#14)] - -(19) CometExchange -Input [3]: [ss_customer_sk#2, sum#15, isEmpty#16] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometHashAggregate -Input [3]: [ss_customer_sk#2, sum#15, isEmpty#16] -Keys [1]: [ss_customer_sk#2] -Functions [1]: [sum(act_sales#14)] - -(21) CometTakeOrderedAndProject -Input [2]: [ss_customer_sk#2, sumsales#17] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sumsales#17 ASC NULLS FIRST,ss_customer_sk#2 ASC NULLS FIRST], output=[ss_customer_sk#2,sumsales#17]), [ss_customer_sk#2, sumsales#17], 100, 0, [sumsales#17 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#17] - -(22) CometColumnarToRow [codegen id : 1] -Input [2]: [ss_customer_sk#2, sumsales#17] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_iceberg_compat/simplified.txt deleted file mode 100644 index d1de4f3475..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,24 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ss_customer_sk,sumsales] - CometHashAggregate [sum,isEmpty] [ss_customer_sk,sumsales,sum(act_sales)] - CometExchange [ss_customer_sk] #1 - CometHashAggregate [act_sales] [ss_customer_sk,sum,isEmpty] - CometProject [sr_return_quantity,ss_quantity,ss_sales_price] [ss_customer_sk,act_sales] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity,r_reason_sk] - CometProject [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometExchange [ss_item_sk,ss_ticket_number] #2 - CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometSort [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometExchange [sr_item_sk,sr_ticket_number] #3 - CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometBroadcastExchange [r_reason_sk] #4 - CometProject [r_reason_sk] - CometFilter [r_reason_sk,r_reason_desc] - CometScan [native_iceberg_compat] 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/q93/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/explain.txt deleted file mode 100644 index cbefa941c7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/explain.txt +++ /dev/null @@ -1,124 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (22) -+- CometTakeOrderedAndProject (21) - +- CometHashAggregate (20) - +- CometExchange (19) - +- CometHashAggregate (18) - +- CometProject (17) - +- CometBroadcastHashJoin (16) - :- CometProject (11) - : +- CometSortMergeJoin (10) - : :- CometSort (4) - : : +- CometExchange (3) - : : +- CometProject (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometSort (9) - : +- CometExchange (8) - : +- CometProject (7) - : +- CometFilter (6) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - +- CometBroadcastExchange (15) - +- CometProject (14) - +- CometFilter (13) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (12) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -ReadSchema: struct - -(2) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5], [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] - -(3) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#3 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_reason_sk)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Condition : ((isnotnull(sr_item_sk#7) AND isnotnull(sr_ticket_number#9)) AND isnotnull(sr_reason_sk#8)) - -(7) CometProject -Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] - -(8) CometExchange -Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: hashpartitioning(sr_item_sk#7, sr_ticket_number#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7 ASC NULLS FIRST, sr_ticket_number#9 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Right output [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [ss_item_sk#1, ss_ticket_number#3], [sr_item_sk#7, sr_ticket_number#9], Inner - -(11) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10], [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Output [2]: [r_reason_sk#12, r_reason_desc#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_desc), IsNotNull(r_reason_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [r_reason_sk#12, r_reason_desc#13] -Condition : ((isnotnull(r_reason_desc#13) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(r_reason_desc#13, 100)) = reason 28 )) AND isnotnull(r_reason_sk#12)) - -(14) CometProject -Input [2]: [r_reason_sk#12, r_reason_desc#13] -Arguments: [r_reason_sk#12], [r_reason_sk#12] - -(15) CometBroadcastExchange -Input [1]: [r_reason_sk#12] -Arguments: [r_reason_sk#12] - -(16) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] -Right output [1]: [r_reason_sk#12] -Arguments: [sr_reason_sk#8], [r_reason_sk#12], Inner, BuildRight - -(17) CometProject -Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10, r_reason_sk#12] -Arguments: [ss_customer_sk#2, act_sales#14], [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#10) THEN (cast((ss_quantity#4 - sr_return_quantity#10) as decimal(10,0)) * ss_sales_price#5) ELSE (cast(ss_quantity#4 as decimal(10,0)) * ss_sales_price#5) END AS act_sales#14] - -(18) CometHashAggregate -Input [2]: [ss_customer_sk#2, act_sales#14] -Keys [1]: [ss_customer_sk#2] -Functions [1]: [partial_sum(act_sales#14)] - -(19) CometExchange -Input [3]: [ss_customer_sk#2, sum#15, isEmpty#16] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometHashAggregate -Input [3]: [ss_customer_sk#2, sum#15, isEmpty#16] -Keys [1]: [ss_customer_sk#2] -Functions [1]: [sum(act_sales#14)] - -(21) CometTakeOrderedAndProject -Input [2]: [ss_customer_sk#2, sumsales#17] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sumsales#17 ASC NULLS FIRST,ss_customer_sk#2 ASC NULLS FIRST], output=[ss_customer_sk#2,sumsales#17]), [ss_customer_sk#2, sumsales#17], 100, 0, [sumsales#17 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#17] - -(22) CometColumnarToRow [codegen id : 1] -Input [2]: [ss_customer_sk#2, sumsales#17] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/extended.txt deleted file mode 100644 index 335f2765d7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/extended.txt +++ /dev/null @@ -1,24 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason - -Comet accelerated 21 out of 21 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d1de4f3475..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt +++ /dev/null @@ -1,24 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ss_customer_sk,sumsales] - CometHashAggregate [sum,isEmpty] [ss_customer_sk,sumsales,sum(act_sales)] - CometExchange [ss_customer_sk] #1 - CometHashAggregate [act_sales] [ss_customer_sk,sum,isEmpty] - CometProject [sr_return_quantity,ss_quantity,ss_sales_price] [ss_customer_sk,act_sales] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity,r_reason_sk] - CometProject [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometExchange [ss_item_sk,ss_ticket_number] #2 - CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometSort [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometExchange [sr_item_sk,sr_ticket_number] #3 - CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometBroadcastExchange [r_reason_sk] #4 - CometProject [r_reason_sk] - CometFilter [r_reason_sk,r_reason_desc] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_datafusion/explain.txt deleted file mode 100644 index 2f78d0dc4b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_datafusion/explain.txt +++ /dev/null @@ -1,235 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometColumnarExchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * CometColumnarToRow (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometNativeScan parquet spark_catalog.default.web_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometNativeScan parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometNativeScan parquet spark_catalog.default.web_site (29) - - -(1) CometNativeScan parquet spark_catalog.default.web_sales -Output [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) - -(3) CometProject -Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(4) CometExchange -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.web_sales -Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -ReadSchema: struct - -(7) CometProject -Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] -Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_warehouse_sk#9, ws_order_number#10] - -(8) CometExchange -Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_order_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_order_number#5], [ws_order_number#10], LeftSemi, NOT (ws_warehouse_sk#4 = ws_warehouse_sk#9) - -(11) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(12) CometNativeScan parquet spark_catalog.default.web_returns -Output [2]: [wr_order_number#12, wr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -ReadSchema: struct - -(13) CometProject -Input [2]: [wr_order_number#12, wr_returned_date_sk#13] -Arguments: [wr_order_number#12], [wr_order_number#12] - -(14) CometExchange -Input [1]: [wr_order_number#12] -Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(15) CometSort -Input [1]: [wr_order_number#12] -Arguments: [wr_order_number#12], [wr_order_number#12 ASC NULLS FIRST] - -(16) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [wr_order_number#12] -Arguments: [ws_order_number#5], [wr_order_number#12], LeftAnti - -(17) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-01)) AND (d_date#15 <= 1999-04-02)) AND isnotnull(d_date_sk#14)) - -(19) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(20) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(21) CometBroadcastHashJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [d_date_sk#14] -Arguments: [ws_ship_date_sk#1], [d_date_sk#14], Inner, BuildRight - -(22) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#14] -Arguments: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(23) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((isnotnull(ca_state#17) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#17, 2)) = IL)) AND isnotnull(ca_address_sk#16)) - -(25) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(26) CometBroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: [ca_address_sk#16] - -(27) CometBroadcastHashJoin -Left output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [ca_address_sk#16] -Arguments: [ws_ship_addr_sk#2], [ca_address_sk#16], Inner, BuildRight - -(28) CometProject -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#16] -Arguments: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(29) CometNativeScan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#18, web_company_name#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_company_name), IsNotNull(web_site_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [web_site_sk#18, web_company_name#19] -Condition : ((isnotnull(web_company_name#19) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(web_company_name#19, 50)) = pri )) AND isnotnull(web_site_sk#18)) - -(31) CometProject -Input [2]: [web_site_sk#18, web_company_name#19] -Arguments: [web_site_sk#18], [web_site_sk#18] - -(32) CometBroadcastExchange -Input [1]: [web_site_sk#18] -Arguments: [web_site_sk#18] - -(33) CometBroadcastHashJoin -Left output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [web_site_sk#18] -Arguments: [ws_web_site_sk#3], [web_site_sk#18], Inner, BuildRight - -(34) CometProject -Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#18] -Arguments: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(35) CometHashAggregate -Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Keys [1]: [ws_order_number#5] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] - -(36) CometColumnarToRow [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] - -(37) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] -Keys [1]: [ws_order_number#5] -Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23] -Results [3]: [ws_order_number#5, sum#20, sum#21] - -(38) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23, count(ws_order_number#5)#24] -Results [3]: [sum#20, sum#21, count#25] - -(39) CometColumnarExchange -Input [3]: [sum#20, sum#21, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(40) CometHashAggregate -Input [3]: [sum#20, sum#21, count#25] -Keys: [] -Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] - -(41) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #26, total shipping cost #27, total net profit #28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_datafusion/simplified.txt deleted file mode 100644 index feab73bcd6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_datafusion/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometSortMergeJoin [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_order_number,ws_warehouse_sk] - CometSort [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] - CometExchange [ws_order_number] #2 - 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,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometNativeScan 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] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometNativeScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometNativeScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometNativeScan 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/q94.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/explain.txt deleted file mode 100644 index 3efafa3b41..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,235 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometColumnarExchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * CometColumnarToRow (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (29) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) - -(3) CometProject -Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(4) CometExchange -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -ReadSchema: struct - -(7) CometProject -Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] -Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_warehouse_sk#9, ws_order_number#10] - -(8) CometExchange -Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_order_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_order_number#5], [ws_order_number#10], LeftSemi, NOT (ws_warehouse_sk#4 = ws_warehouse_sk#9) - -(11) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [2]: [wr_order_number#12, wr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -ReadSchema: struct - -(13) CometProject -Input [2]: [wr_order_number#12, wr_returned_date_sk#13] -Arguments: [wr_order_number#12], [wr_order_number#12] - -(14) CometExchange -Input [1]: [wr_order_number#12] -Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(15) CometSort -Input [1]: [wr_order_number#12] -Arguments: [wr_order_number#12], [wr_order_number#12 ASC NULLS FIRST] - -(16) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [wr_order_number#12] -Arguments: [ws_order_number#5], [wr_order_number#12], LeftAnti - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-01)) AND (d_date#15 <= 1999-04-02)) AND isnotnull(d_date_sk#14)) - -(19) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(20) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(21) CometBroadcastHashJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [d_date_sk#14] -Arguments: [ws_ship_date_sk#1], [d_date_sk#14], Inner, BuildRight - -(22) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#14] -Arguments: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((isnotnull(ca_state#17) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#17, 2)) = IL)) AND isnotnull(ca_address_sk#16)) - -(25) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(26) CometBroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: [ca_address_sk#16] - -(27) CometBroadcastHashJoin -Left output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [ca_address_sk#16] -Arguments: [ws_ship_addr_sk#2], [ca_address_sk#16], Inner, BuildRight - -(28) CometProject -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#16] -Arguments: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#18, web_company_name#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_company_name), IsNotNull(web_site_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [web_site_sk#18, web_company_name#19] -Condition : ((isnotnull(web_company_name#19) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(web_company_name#19, 50)) = pri )) AND isnotnull(web_site_sk#18)) - -(31) CometProject -Input [2]: [web_site_sk#18, web_company_name#19] -Arguments: [web_site_sk#18], [web_site_sk#18] - -(32) CometBroadcastExchange -Input [1]: [web_site_sk#18] -Arguments: [web_site_sk#18] - -(33) CometBroadcastHashJoin -Left output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [web_site_sk#18] -Arguments: [ws_web_site_sk#3], [web_site_sk#18], Inner, BuildRight - -(34) CometProject -Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#18] -Arguments: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(35) CometHashAggregate -Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Keys [1]: [ws_order_number#5] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] - -(36) CometColumnarToRow [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] - -(37) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] -Keys [1]: [ws_order_number#5] -Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23] -Results [3]: [ws_order_number#5, sum#20, sum#21] - -(38) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23, count(ws_order_number#5)#24] -Results [3]: [sum#20, sum#21, count#25] - -(39) CometColumnarExchange -Input [3]: [sum#20, sum#21, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(40) CometHashAggregate -Input [3]: [sum#20, sum#21, count#25] -Keys: [] -Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] - -(41) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #26, total shipping cost #27, total net profit #28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/simplified.txt deleted file mode 100644 index 629178d106..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometSortMergeJoin [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_order_number,ws_warehouse_sk] - CometSort [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] - CometExchange [ws_order_number] #2 - 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,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometScan [native_iceberg_compat] 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/q94/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/explain.txt deleted file mode 100644 index 3efafa3b41..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/explain.txt +++ /dev/null @@ -1,235 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometColumnarExchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * CometColumnarToRow (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (29) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) - -(3) CometProject -Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(4) CometExchange -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -ReadSchema: struct - -(7) CometProject -Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] -Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_warehouse_sk#9, ws_order_number#10] - -(8) CometExchange -Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_order_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_order_number#5], [ws_order_number#10], LeftSemi, NOT (ws_warehouse_sk#4 = ws_warehouse_sk#9) - -(11) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [2]: [wr_order_number#12, wr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -ReadSchema: struct - -(13) CometProject -Input [2]: [wr_order_number#12, wr_returned_date_sk#13] -Arguments: [wr_order_number#12], [wr_order_number#12] - -(14) CometExchange -Input [1]: [wr_order_number#12] -Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(15) CometSort -Input [1]: [wr_order_number#12] -Arguments: [wr_order_number#12], [wr_order_number#12 ASC NULLS FIRST] - -(16) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [wr_order_number#12] -Arguments: [ws_order_number#5], [wr_order_number#12], LeftAnti - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-01)) AND (d_date#15 <= 1999-04-02)) AND isnotnull(d_date_sk#14)) - -(19) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(20) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(21) CometBroadcastHashJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [d_date_sk#14] -Arguments: [ws_ship_date_sk#1], [d_date_sk#14], Inner, BuildRight - -(22) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#14] -Arguments: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((isnotnull(ca_state#17) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#17, 2)) = IL)) AND isnotnull(ca_address_sk#16)) - -(25) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(26) CometBroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: [ca_address_sk#16] - -(27) CometBroadcastHashJoin -Left output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [ca_address_sk#16] -Arguments: [ws_ship_addr_sk#2], [ca_address_sk#16], Inner, BuildRight - -(28) CometProject -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#16] -Arguments: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#18, web_company_name#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_company_name), IsNotNull(web_site_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [web_site_sk#18, web_company_name#19] -Condition : ((isnotnull(web_company_name#19) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(web_company_name#19, 50)) = pri )) AND isnotnull(web_site_sk#18)) - -(31) CometProject -Input [2]: [web_site_sk#18, web_company_name#19] -Arguments: [web_site_sk#18], [web_site_sk#18] - -(32) CometBroadcastExchange -Input [1]: [web_site_sk#18] -Arguments: [web_site_sk#18] - -(33) CometBroadcastHashJoin -Left output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [web_site_sk#18] -Arguments: [ws_web_site_sk#3], [web_site_sk#18], Inner, BuildRight - -(34) CometProject -Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#18] -Arguments: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(35) CometHashAggregate -Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Keys [1]: [ws_order_number#5] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] - -(36) CometColumnarToRow [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] - -(37) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] -Keys [1]: [ws_order_number#5] -Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23] -Results [3]: [ws_order_number#5, sum#20, sum#21] - -(38) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23, count(ws_order_number#5)#24] -Results [3]: [sum#20, sum#21, count#25] - -(39) CometColumnarExchange -Input [3]: [sum#20, sum#21, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(40) CometHashAggregate -Input [3]: [sum#20, sum#21, count#25] -Keys: [] -Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] - -(41) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #26, total shipping cost #27, total net profit #28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/extended.txt deleted file mode 100644 index eac4939621..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/extended.txt +++ /dev/null @@ -1,43 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - -Comet accelerated 37 out of 39 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 629178d106..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometSortMergeJoin [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_order_number,ws_warehouse_sk] - CometSort [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] - CometExchange [ws_order_number] #2 - 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,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_datafusion/explain.txt deleted file mode 100644 index a65b77bf6a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_datafusion/explain.txt +++ /dev/null @@ -1,302 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (54) -+- CometHashAggregate (53) - +- CometColumnarExchange (52) - +- * HashAggregate (51) - +- * HashAggregate (50) - +- * CometColumnarToRow (49) - +- CometHashAggregate (48) - +- CometProject (47) - +- CometBroadcastHashJoin (46) - :- CometProject (41) - : +- CometBroadcastHashJoin (40) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometSortMergeJoin (29) - : : : :- CometSortMergeJoin (15) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales (1) - : : : : +- CometProject (14) - : : : : +- CometSortMergeJoin (13) - : : : : :- CometSort (10) - : : : : : +- CometExchange (9) - : : : : : +- CometProject (8) - : : : : : +- CometFilter (7) - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales (6) - : : : : +- CometSort (12) - : : : : +- ReusedExchange (11) - : : : +- CometProject (28) - : : : +- CometSortMergeJoin (27) - : : : :- CometSort (20) - : : : : +- CometExchange (19) - : : : : +- CometProject (18) - : : : : +- CometFilter (17) - : : : : +- CometNativeScan parquet spark_catalog.default.web_returns (16) - : : : +- CometProject (26) - : : : +- CometSortMergeJoin (25) - : : : :- CometSort (22) - : : : : +- ReusedExchange (21) - : : : +- CometSort (24) - : : : +- ReusedExchange (23) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (30) - : +- CometBroadcastExchange (39) - : +- CometProject (38) - : +- CometFilter (37) - : +- CometNativeScan parquet spark_catalog.default.customer_address (36) - +- CometBroadcastExchange (45) - +- CometProject (44) - +- CometFilter (43) - +- CometNativeScan parquet spark_catalog.default.web_site (42) - - -(1) CometNativeScan parquet spark_catalog.default.web_sales -Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) - -(3) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(4) CometExchange -Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.web_sales -Output [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Condition : (isnotnull(ws_order_number#9) AND isnotnull(ws_warehouse_sk#8)) - -(8) CometProject -Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_warehouse_sk#8, ws_order_number#9] - -(9) CometExchange -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: hashpartitioning(ws_order_number#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_order_number#9 ASC NULLS FIRST] - -(11) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#11, ws_order_number#12] - -(12) CometSort -Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_warehouse_sk#11, ws_order_number#12], [ws_order_number#12 ASC NULLS FIRST] - -(13) CometSortMergeJoin -Left output [2]: [ws_warehouse_sk#8, ws_order_number#9] -Right output [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#12], Inner, NOT (ws_warehouse_sk#8 = ws_warehouse_sk#11) - -(14) CometProject -Input [4]: [ws_warehouse_sk#8, ws_order_number#9, ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#9] - -(15) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [ws_order_number#9] -Arguments: [ws_order_number#4], [ws_order_number#9], LeftSemi - -(16) CometNativeScan parquet spark_catalog.default.web_returns -Output [2]: [wr_order_number#13, wr_returned_date_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [wr_order_number#13, wr_returned_date_sk#14] -Condition : isnotnull(wr_order_number#13) - -(18) CometProject -Input [2]: [wr_order_number#13, wr_returned_date_sk#14] -Arguments: [wr_order_number#13], [wr_order_number#13] - -(19) CometExchange -Input [1]: [wr_order_number#13] -Arguments: hashpartitioning(wr_order_number#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [1]: [wr_order_number#13] -Arguments: [wr_order_number#13], [wr_order_number#13 ASC NULLS FIRST] - -(21) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#15, ws_order_number#16] - -(22) CometSort -Input [2]: [ws_warehouse_sk#15, ws_order_number#16] -Arguments: [ws_warehouse_sk#15, ws_order_number#16], [ws_order_number#16 ASC NULLS FIRST] - -(23) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#17, ws_order_number#18] - -(24) CometSort -Input [2]: [ws_warehouse_sk#17, ws_order_number#18] -Arguments: [ws_warehouse_sk#17, ws_order_number#18], [ws_order_number#18 ASC NULLS FIRST] - -(25) CometSortMergeJoin -Left output [2]: [ws_warehouse_sk#15, ws_order_number#16] -Right output [2]: [ws_warehouse_sk#17, ws_order_number#18] -Arguments: [ws_order_number#16], [ws_order_number#18], Inner, NOT (ws_warehouse_sk#15 = ws_warehouse_sk#17) - -(26) CometProject -Input [4]: [ws_warehouse_sk#15, ws_order_number#16, ws_warehouse_sk#17, ws_order_number#18] -Arguments: [ws_order_number#16], [ws_order_number#16] - -(27) CometSortMergeJoin -Left output [1]: [wr_order_number#13] -Right output [1]: [ws_order_number#16] -Arguments: [wr_order_number#13], [ws_order_number#16], Inner - -(28) CometProject -Input [2]: [wr_order_number#13, ws_order_number#16] -Arguments: [wr_order_number#13], [wr_order_number#13] - -(29) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [wr_order_number#13] -Arguments: [ws_order_number#4], [wr_order_number#13], LeftSemi - -(30) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_date#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [d_date_sk#19, d_date#20] -Condition : (((isnotnull(d_date#20) AND (d_date#20 >= 1999-02-01)) AND (d_date#20 <= 1999-04-02)) AND isnotnull(d_date_sk#19)) - -(32) CometProject -Input [2]: [d_date_sk#19, d_date#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(33) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(34) CometBroadcastHashJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [d_date_sk#19] -Arguments: [ws_ship_date_sk#1], [d_date_sk#19], Inner, BuildRight - -(35) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#19] -Arguments: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(36) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#21, ca_state#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [ca_address_sk#21, ca_state#22] -Condition : ((isnotnull(ca_state#22) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#22, 2)) = IL)) AND isnotnull(ca_address_sk#21)) - -(38) CometProject -Input [2]: [ca_address_sk#21, ca_state#22] -Arguments: [ca_address_sk#21], [ca_address_sk#21] - -(39) CometBroadcastExchange -Input [1]: [ca_address_sk#21] -Arguments: [ca_address_sk#21] - -(40) CometBroadcastHashJoin -Left output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [ca_address_sk#21] -Arguments: [ws_ship_addr_sk#2], [ca_address_sk#21], Inner, BuildRight - -(41) CometProject -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#21] -Arguments: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(42) CometNativeScan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#23, web_company_name#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_company_name), IsNotNull(web_site_sk)] -ReadSchema: struct - -(43) CometFilter -Input [2]: [web_site_sk#23, web_company_name#24] -Condition : ((isnotnull(web_company_name#24) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(web_company_name#24, 50)) = pri )) AND isnotnull(web_site_sk#23)) - -(44) CometProject -Input [2]: [web_site_sk#23, web_company_name#24] -Arguments: [web_site_sk#23], [web_site_sk#23] - -(45) CometBroadcastExchange -Input [1]: [web_site_sk#23] -Arguments: [web_site_sk#23] - -(46) CometBroadcastHashJoin -Left output [4]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [web_site_sk#23] -Arguments: [ws_web_site_sk#3], [web_site_sk#23], Inner, BuildRight - -(47) CometProject -Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#23] -Arguments: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(48) CometHashAggregate -Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Keys [1]: [ws_order_number#4] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] - -(49) CometColumnarToRow [codegen id : 1] -Input [3]: [ws_order_number#4, sum#25, sum#26] - -(50) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#4, sum#25, sum#26] -Keys [1]: [ws_order_number#4] -Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28] -Results [3]: [ws_order_number#4, sum#25, sum#26] - -(51) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#4, sum#25, sum#26] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28, count(ws_order_number#4)#29] -Results [3]: [sum#25, sum#26, count#30] - -(52) CometColumnarExchange -Input [3]: [sum#25, sum#26, count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(53) CometHashAggregate -Input [3]: [sum#25, sum#26, count#30] -Keys: [] -Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] - -(54) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #31, total shipping cost #32, total net profit #33] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_datafusion/simplified.txt deleted file mode 100644 index 15f29c507c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_datafusion/simplified.txt +++ /dev/null @@ -1,58 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number] - CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometExchange [ws_order_number] #2 - 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,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometNativeScan 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] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometProject [wr_order_number] - CometSortMergeJoin [wr_order_number,ws_order_number] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometFilter [wr_order_number,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometNativeScan 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.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/explain.txt deleted file mode 100644 index d86eccd434..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,302 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (54) -+- CometHashAggregate (53) - +- CometColumnarExchange (52) - +- * HashAggregate (51) - +- * HashAggregate (50) - +- * CometColumnarToRow (49) - +- CometHashAggregate (48) - +- CometProject (47) - +- CometBroadcastHashJoin (46) - :- CometProject (41) - : +- CometBroadcastHashJoin (40) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometSortMergeJoin (29) - : : : :- CometSortMergeJoin (15) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometProject (14) - : : : : +- CometSortMergeJoin (13) - : : : : :- CometSort (10) - : : : : : +- CometExchange (9) - : : : : : +- CometProject (8) - : : : : : +- CometFilter (7) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (6) - : : : : +- CometSort (12) - : : : : +- ReusedExchange (11) - : : : +- CometProject (28) - : : : +- CometSortMergeJoin (27) - : : : :- CometSort (20) - : : : : +- CometExchange (19) - : : : : +- CometProject (18) - : : : : +- CometFilter (17) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (16) - : : : +- CometProject (26) - : : : +- CometSortMergeJoin (25) - : : : :- CometSort (22) - : : : : +- ReusedExchange (21) - : : : +- CometSort (24) - : : : +- ReusedExchange (23) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - : +- CometBroadcastExchange (39) - : +- CometProject (38) - : +- CometFilter (37) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (36) - +- CometBroadcastExchange (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (42) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) - -(3) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(4) CometExchange -Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Condition : (isnotnull(ws_order_number#9) AND isnotnull(ws_warehouse_sk#8)) - -(8) CometProject -Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_warehouse_sk#8, ws_order_number#9] - -(9) CometExchange -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: hashpartitioning(ws_order_number#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_order_number#9 ASC NULLS FIRST] - -(11) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#11, ws_order_number#12] - -(12) CometSort -Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_warehouse_sk#11, ws_order_number#12], [ws_order_number#12 ASC NULLS FIRST] - -(13) CometSortMergeJoin -Left output [2]: [ws_warehouse_sk#8, ws_order_number#9] -Right output [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#12], Inner, NOT (ws_warehouse_sk#8 = ws_warehouse_sk#11) - -(14) CometProject -Input [4]: [ws_warehouse_sk#8, ws_order_number#9, ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#9] - -(15) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [ws_order_number#9] -Arguments: [ws_order_number#4], [ws_order_number#9], LeftSemi - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [2]: [wr_order_number#13, wr_returned_date_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [wr_order_number#13, wr_returned_date_sk#14] -Condition : isnotnull(wr_order_number#13) - -(18) CometProject -Input [2]: [wr_order_number#13, wr_returned_date_sk#14] -Arguments: [wr_order_number#13], [wr_order_number#13] - -(19) CometExchange -Input [1]: [wr_order_number#13] -Arguments: hashpartitioning(wr_order_number#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [1]: [wr_order_number#13] -Arguments: [wr_order_number#13], [wr_order_number#13 ASC NULLS FIRST] - -(21) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#15, ws_order_number#16] - -(22) CometSort -Input [2]: [ws_warehouse_sk#15, ws_order_number#16] -Arguments: [ws_warehouse_sk#15, ws_order_number#16], [ws_order_number#16 ASC NULLS FIRST] - -(23) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#17, ws_order_number#18] - -(24) CometSort -Input [2]: [ws_warehouse_sk#17, ws_order_number#18] -Arguments: [ws_warehouse_sk#17, ws_order_number#18], [ws_order_number#18 ASC NULLS FIRST] - -(25) CometSortMergeJoin -Left output [2]: [ws_warehouse_sk#15, ws_order_number#16] -Right output [2]: [ws_warehouse_sk#17, ws_order_number#18] -Arguments: [ws_order_number#16], [ws_order_number#18], Inner, NOT (ws_warehouse_sk#15 = ws_warehouse_sk#17) - -(26) CometProject -Input [4]: [ws_warehouse_sk#15, ws_order_number#16, ws_warehouse_sk#17, ws_order_number#18] -Arguments: [ws_order_number#16], [ws_order_number#16] - -(27) CometSortMergeJoin -Left output [1]: [wr_order_number#13] -Right output [1]: [ws_order_number#16] -Arguments: [wr_order_number#13], [ws_order_number#16], Inner - -(28) CometProject -Input [2]: [wr_order_number#13, ws_order_number#16] -Arguments: [wr_order_number#13], [wr_order_number#13] - -(29) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [wr_order_number#13] -Arguments: [ws_order_number#4], [wr_order_number#13], LeftSemi - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_date#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [d_date_sk#19, d_date#20] -Condition : (((isnotnull(d_date#20) AND (d_date#20 >= 1999-02-01)) AND (d_date#20 <= 1999-04-02)) AND isnotnull(d_date_sk#19)) - -(32) CometProject -Input [2]: [d_date_sk#19, d_date#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(33) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(34) CometBroadcastHashJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [d_date_sk#19] -Arguments: [ws_ship_date_sk#1], [d_date_sk#19], Inner, BuildRight - -(35) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#19] -Arguments: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#21, ca_state#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [ca_address_sk#21, ca_state#22] -Condition : ((isnotnull(ca_state#22) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#22, 2)) = IL)) AND isnotnull(ca_address_sk#21)) - -(38) CometProject -Input [2]: [ca_address_sk#21, ca_state#22] -Arguments: [ca_address_sk#21], [ca_address_sk#21] - -(39) CometBroadcastExchange -Input [1]: [ca_address_sk#21] -Arguments: [ca_address_sk#21] - -(40) CometBroadcastHashJoin -Left output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [ca_address_sk#21] -Arguments: [ws_ship_addr_sk#2], [ca_address_sk#21], Inner, BuildRight - -(41) CometProject -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#21] -Arguments: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#23, web_company_name#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_company_name), IsNotNull(web_site_sk)] -ReadSchema: struct - -(43) CometFilter -Input [2]: [web_site_sk#23, web_company_name#24] -Condition : ((isnotnull(web_company_name#24) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(web_company_name#24, 50)) = pri )) AND isnotnull(web_site_sk#23)) - -(44) CometProject -Input [2]: [web_site_sk#23, web_company_name#24] -Arguments: [web_site_sk#23], [web_site_sk#23] - -(45) CometBroadcastExchange -Input [1]: [web_site_sk#23] -Arguments: [web_site_sk#23] - -(46) CometBroadcastHashJoin -Left output [4]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [web_site_sk#23] -Arguments: [ws_web_site_sk#3], [web_site_sk#23], Inner, BuildRight - -(47) CometProject -Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#23] -Arguments: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(48) CometHashAggregate -Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Keys [1]: [ws_order_number#4] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] - -(49) CometColumnarToRow [codegen id : 1] -Input [3]: [ws_order_number#4, sum#25, sum#26] - -(50) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#4, sum#25, sum#26] -Keys [1]: [ws_order_number#4] -Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28] -Results [3]: [ws_order_number#4, sum#25, sum#26] - -(51) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#4, sum#25, sum#26] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28, count(ws_order_number#4)#29] -Results [3]: [sum#25, sum#26, count#30] - -(52) CometColumnarExchange -Input [3]: [sum#25, sum#26, count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(53) CometHashAggregate -Input [3]: [sum#25, sum#26, count#30] -Keys: [] -Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] - -(54) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #31, total shipping cost #32, total net profit #33] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/simplified.txt deleted file mode 100644 index 6c1b1bccfa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,58 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number] - CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometExchange [ws_order_number] #2 - 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,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometProject [wr_order_number] - CometSortMergeJoin [wr_order_number,ws_order_number] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometFilter [wr_order_number,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometScan [native_iceberg_compat] 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/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/explain.txt deleted file mode 100644 index d86eccd434..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/explain.txt +++ /dev/null @@ -1,302 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (54) -+- CometHashAggregate (53) - +- CometColumnarExchange (52) - +- * HashAggregate (51) - +- * HashAggregate (50) - +- * CometColumnarToRow (49) - +- CometHashAggregate (48) - +- CometProject (47) - +- CometBroadcastHashJoin (46) - :- CometProject (41) - : +- CometBroadcastHashJoin (40) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometSortMergeJoin (29) - : : : :- CometSortMergeJoin (15) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometProject (14) - : : : : +- CometSortMergeJoin (13) - : : : : :- CometSort (10) - : : : : : +- CometExchange (9) - : : : : : +- CometProject (8) - : : : : : +- CometFilter (7) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (6) - : : : : +- CometSort (12) - : : : : +- ReusedExchange (11) - : : : +- CometProject (28) - : : : +- CometSortMergeJoin (27) - : : : :- CometSort (20) - : : : : +- CometExchange (19) - : : : : +- CometProject (18) - : : : : +- CometFilter (17) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (16) - : : : +- CometProject (26) - : : : +- CometSortMergeJoin (25) - : : : :- CometSort (22) - : : : : +- ReusedExchange (21) - : : : +- CometSort (24) - : : : +- ReusedExchange (23) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - : +- CometBroadcastExchange (39) - : +- CometProject (38) - : +- CometFilter (37) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (36) - +- CometBroadcastExchange (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (42) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) - -(3) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(4) CometExchange -Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Condition : (isnotnull(ws_order_number#9) AND isnotnull(ws_warehouse_sk#8)) - -(8) CometProject -Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_warehouse_sk#8, ws_order_number#9] - -(9) CometExchange -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: hashpartitioning(ws_order_number#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_order_number#9 ASC NULLS FIRST] - -(11) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#11, ws_order_number#12] - -(12) CometSort -Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_warehouse_sk#11, ws_order_number#12], [ws_order_number#12 ASC NULLS FIRST] - -(13) CometSortMergeJoin -Left output [2]: [ws_warehouse_sk#8, ws_order_number#9] -Right output [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#12], Inner, NOT (ws_warehouse_sk#8 = ws_warehouse_sk#11) - -(14) CometProject -Input [4]: [ws_warehouse_sk#8, ws_order_number#9, ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#9] - -(15) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [ws_order_number#9] -Arguments: [ws_order_number#4], [ws_order_number#9], LeftSemi - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [2]: [wr_order_number#13, wr_returned_date_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [wr_order_number#13, wr_returned_date_sk#14] -Condition : isnotnull(wr_order_number#13) - -(18) CometProject -Input [2]: [wr_order_number#13, wr_returned_date_sk#14] -Arguments: [wr_order_number#13], [wr_order_number#13] - -(19) CometExchange -Input [1]: [wr_order_number#13] -Arguments: hashpartitioning(wr_order_number#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [1]: [wr_order_number#13] -Arguments: [wr_order_number#13], [wr_order_number#13 ASC NULLS FIRST] - -(21) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#15, ws_order_number#16] - -(22) CometSort -Input [2]: [ws_warehouse_sk#15, ws_order_number#16] -Arguments: [ws_warehouse_sk#15, ws_order_number#16], [ws_order_number#16 ASC NULLS FIRST] - -(23) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#17, ws_order_number#18] - -(24) CometSort -Input [2]: [ws_warehouse_sk#17, ws_order_number#18] -Arguments: [ws_warehouse_sk#17, ws_order_number#18], [ws_order_number#18 ASC NULLS FIRST] - -(25) CometSortMergeJoin -Left output [2]: [ws_warehouse_sk#15, ws_order_number#16] -Right output [2]: [ws_warehouse_sk#17, ws_order_number#18] -Arguments: [ws_order_number#16], [ws_order_number#18], Inner, NOT (ws_warehouse_sk#15 = ws_warehouse_sk#17) - -(26) CometProject -Input [4]: [ws_warehouse_sk#15, ws_order_number#16, ws_warehouse_sk#17, ws_order_number#18] -Arguments: [ws_order_number#16], [ws_order_number#16] - -(27) CometSortMergeJoin -Left output [1]: [wr_order_number#13] -Right output [1]: [ws_order_number#16] -Arguments: [wr_order_number#13], [ws_order_number#16], Inner - -(28) CometProject -Input [2]: [wr_order_number#13, ws_order_number#16] -Arguments: [wr_order_number#13], [wr_order_number#13] - -(29) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [wr_order_number#13] -Arguments: [ws_order_number#4], [wr_order_number#13], LeftSemi - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_date#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [d_date_sk#19, d_date#20] -Condition : (((isnotnull(d_date#20) AND (d_date#20 >= 1999-02-01)) AND (d_date#20 <= 1999-04-02)) AND isnotnull(d_date_sk#19)) - -(32) CometProject -Input [2]: [d_date_sk#19, d_date#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(33) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(34) CometBroadcastHashJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [d_date_sk#19] -Arguments: [ws_ship_date_sk#1], [d_date_sk#19], Inner, BuildRight - -(35) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#19] -Arguments: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#21, ca_state#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [ca_address_sk#21, ca_state#22] -Condition : ((isnotnull(ca_state#22) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#22, 2)) = IL)) AND isnotnull(ca_address_sk#21)) - -(38) CometProject -Input [2]: [ca_address_sk#21, ca_state#22] -Arguments: [ca_address_sk#21], [ca_address_sk#21] - -(39) CometBroadcastExchange -Input [1]: [ca_address_sk#21] -Arguments: [ca_address_sk#21] - -(40) CometBroadcastHashJoin -Left output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [ca_address_sk#21] -Arguments: [ws_ship_addr_sk#2], [ca_address_sk#21], Inner, BuildRight - -(41) CometProject -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#21] -Arguments: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#23, web_company_name#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_company_name), IsNotNull(web_site_sk)] -ReadSchema: struct - -(43) CometFilter -Input [2]: [web_site_sk#23, web_company_name#24] -Condition : ((isnotnull(web_company_name#24) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(web_company_name#24, 50)) = pri )) AND isnotnull(web_site_sk#23)) - -(44) CometProject -Input [2]: [web_site_sk#23, web_company_name#24] -Arguments: [web_site_sk#23], [web_site_sk#23] - -(45) CometBroadcastExchange -Input [1]: [web_site_sk#23] -Arguments: [web_site_sk#23] - -(46) CometBroadcastHashJoin -Left output [4]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [web_site_sk#23] -Arguments: [ws_web_site_sk#3], [web_site_sk#23], Inner, BuildRight - -(47) CometProject -Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#23] -Arguments: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(48) CometHashAggregate -Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Keys [1]: [ws_order_number#4] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] - -(49) CometColumnarToRow [codegen id : 1] -Input [3]: [ws_order_number#4, sum#25, sum#26] - -(50) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#4, sum#25, sum#26] -Keys [1]: [ws_order_number#4] -Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28] -Results [3]: [ws_order_number#4, sum#25, sum#26] - -(51) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#4, sum#25, sum#26] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28, count(ws_order_number#4)#29] -Results [3]: [sum#25, sum#26, count#30] - -(52) CometColumnarExchange -Input [3]: [sum#25, sum#26, count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(53) CometHashAggregate -Input [3]: [sum#25, sum#26, count#30] -Keys: [] -Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] - -(54) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #31, total shipping cost #32, total net profit #33] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/extended.txt deleted file mode 100644 index 6ff8eba58f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometProject - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- CometProject - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - -Comet accelerated 59 out of 61 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 6c1b1bccfa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt +++ /dev/null @@ -1,58 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number] - CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometExchange [ws_order_number] #2 - 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,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometProject [wr_order_number] - CometSortMergeJoin [wr_order_number,ws_order_number] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometFilter [wr_order_number,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_datafusion/explain.txt deleted file mode 100644 index db701aae34..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_datafusion/explain.txt +++ /dev/null @@ -1,143 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometNativeScan parquet spark_catalog.default.household_demographics (4) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometNativeScan parquet spark_catalog.default.time_dim (10) - +- CometBroadcastExchange (19) - +- CometProject (18) - +- CometFilter (17) - +- CometNativeScan parquet spark_catalog.default.store (16) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(3) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(4) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#5, hd_dep_count#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,7), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 7)) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] -Arguments: [ss_sold_time_sk#1, ss_store_sk#3], [ss_sold_time_sk#1, ss_store_sk#3] - -(10) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,20), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Condition : ((((isnotnull(t_hour#8) AND isnotnull(t_minute#9)) AND (t_hour#8 = 20)) AND (t_minute#9 >= 30)) AND isnotnull(t_time_sk#7)) - -(12) CometProject -Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Arguments: [t_time_sk#7], [t_time_sk#7] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#7] -Arguments: [t_time_sk#7] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Right output [1]: [t_time_sk#7] -Arguments: [ss_sold_time_sk#1], [t_time_sk#7], Inner, BuildRight - -(15) CometProject -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#7] -Arguments: [ss_store_sk#3], [ss_store_sk#3] - -(16) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_store_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [s_store_sk#10, s_store_name#11] -Condition : ((isnotnull(s_store_name#11) AND (s_store_name#11 = ese)) AND isnotnull(s_store_sk#10)) - -(18) CometProject -Input [2]: [s_store_sk#10, s_store_name#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(19) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(20) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#3] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(21) CometProject -Input [2]: [ss_store_sk#3, s_store_sk#10] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#12] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 1] -Input [1]: [count(1)#13] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_datafusion/simplified.txt deleted file mode 100644 index b40f41659f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_datafusion/simplified.txt +++ /dev/null @@ -1,27 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometNativeScan 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/q96.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_iceberg_compat/explain.txt deleted file mode 100644 index f90d8b12b9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,143 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (4) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (10) - +- CometBroadcastExchange (19) - +- CometProject (18) - +- CometFilter (17) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (16) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(3) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#5, hd_dep_count#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,7), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 7)) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] -Arguments: [ss_sold_time_sk#1, ss_store_sk#3], [ss_sold_time_sk#1, ss_store_sk#3] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,20), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Condition : ((((isnotnull(t_hour#8) AND isnotnull(t_minute#9)) AND (t_hour#8 = 20)) AND (t_minute#9 >= 30)) AND isnotnull(t_time_sk#7)) - -(12) CometProject -Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Arguments: [t_time_sk#7], [t_time_sk#7] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#7] -Arguments: [t_time_sk#7] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Right output [1]: [t_time_sk#7] -Arguments: [ss_sold_time_sk#1], [t_time_sk#7], Inner, BuildRight - -(15) CometProject -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#7] -Arguments: [ss_store_sk#3], [ss_store_sk#3] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_store_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [s_store_sk#10, s_store_name#11] -Condition : ((isnotnull(s_store_name#11) AND (s_store_name#11 = ese)) AND isnotnull(s_store_sk#10)) - -(18) CometProject -Input [2]: [s_store_sk#10, s_store_name#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(19) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(20) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#3] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(21) CometProject -Input [2]: [ss_store_sk#3, s_store_sk#10] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#12] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 1] -Input [1]: [count(1)#13] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_iceberg_compat/simplified.txt deleted file mode 100644 index bbbd07245e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,27 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometScan [native_iceberg_compat] 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/q96/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/explain.txt deleted file mode 100644 index f90d8b12b9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/explain.txt +++ /dev/null @@ -1,143 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (4) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (10) - +- CometBroadcastExchange (19) - +- CometProject (18) - +- CometFilter (17) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (16) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(3) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#5, hd_dep_count#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,7), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 7)) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] -Arguments: [ss_sold_time_sk#1, ss_store_sk#3], [ss_sold_time_sk#1, ss_store_sk#3] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,20), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Condition : ((((isnotnull(t_hour#8) AND isnotnull(t_minute#9)) AND (t_hour#8 = 20)) AND (t_minute#9 >= 30)) AND isnotnull(t_time_sk#7)) - -(12) CometProject -Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Arguments: [t_time_sk#7], [t_time_sk#7] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#7] -Arguments: [t_time_sk#7] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Right output [1]: [t_time_sk#7] -Arguments: [ss_sold_time_sk#1], [t_time_sk#7], Inner, BuildRight - -(15) CometProject -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#7] -Arguments: [ss_store_sk#3], [ss_store_sk#3] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_store_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [s_store_sk#10, s_store_name#11] -Condition : ((isnotnull(s_store_name#11) AND (s_store_name#11 = ese)) AND isnotnull(s_store_sk#10)) - -(18) CometProject -Input [2]: [s_store_sk#10, s_store_name#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(19) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(20) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#3] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(21) CometProject -Input [2]: [ss_store_sk#3, s_store_sk#10] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#12] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 1] -Input [1]: [count(1)#13] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/extended.txt deleted file mode 100644 index 0f623c9021..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/extended.txt +++ /dev/null @@ -1,27 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 24 out of 24 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index bbbd07245e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt +++ /dev/null @@ -1,27 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/explain.txt deleted file mode 100644 index a7f1cb451d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/explain.txt +++ /dev/null @@ -1,174 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (24) -+- CometHashAggregate (23) - +- CometExchange (22) - +- CometHashAggregate (21) - +- CometProject (20) - +- CometSortMergeJoin (19) - :- CometSort (9) - : +- CometHashAggregate (8) - : +- CometColumnarExchange (7) - : +- * HashAggregate (6) - : +- * Project (5) - : +- * BroadcastHashJoin Inner BuildRight (4) - : :- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.store_sales (1) - : +- ReusedExchange (3) - +- CometSort (18) - +- CometHashAggregate (17) - +- CometColumnarExchange (16) - +- * HashAggregate (15) - +- * Project (14) - +- * BroadcastHashJoin Inner BuildRight (13) - :- * ColumnarToRow (11) - : +- Scan parquet spark_catalog.default.catalog_sales (10) - +- ReusedExchange (12) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 2] -Input [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] - -(3) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#5] - -(4) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(5) Project [codegen id : 2] -Output [2]: [ss_item_sk#1, ss_customer_sk#2] -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#5] - -(6) HashAggregate [codegen id : 2] -Input [2]: [ss_item_sk#1, ss_customer_sk#2] -Keys [2]: [ss_customer_sk#2, ss_item_sk#1] -Functions: [] -Aggregate Attributes: [] -Results [2]: [ss_customer_sk#2, ss_item_sk#1] - -(7) CometColumnarExchange -Input [2]: [ss_customer_sk#2, ss_item_sk#1] -Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(8) CometHashAggregate -Input [2]: [ss_customer_sk#2, ss_item_sk#1] -Keys [2]: [ss_customer_sk#2, ss_item_sk#1] -Functions: [] - -(9) CometSort -Input [2]: [customer_sk#6, item_sk#7] -Arguments: [customer_sk#6, item_sk#7], [customer_sk#6 ASC NULLS FIRST, item_sk#7 ASC NULLS FIRST] - -(10) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#4)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 4] -Input [3]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10] - -(12) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#11] - -(13) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#10] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 4] -Output [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Input [4]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10, d_date_sk#11] - -(15) HashAggregate [codegen id : 4] -Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Keys [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Functions: [] -Aggregate Attributes: [] -Results [2]: [cs_bill_customer_sk#8, cs_item_sk#9] - -(16) CometColumnarExchange -Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Arguments: hashpartitioning(cs_bill_customer_sk#8, cs_item_sk#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(17) CometHashAggregate -Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Keys [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Functions: [] - -(18) CometSort -Input [2]: [customer_sk#12, item_sk#13] -Arguments: [customer_sk#12, item_sk#13], [customer_sk#12 ASC NULLS FIRST, item_sk#13 ASC NULLS FIRST] - -(19) CometSortMergeJoin -Left output [2]: [customer_sk#6, item_sk#7] -Right output [2]: [customer_sk#12, item_sk#13] -Arguments: [customer_sk#6, item_sk#7], [customer_sk#12, item_sk#13], FullOuter - -(20) CometProject -Input [4]: [customer_sk#6, item_sk#7, customer_sk#12, item_sk#13] -Arguments: [customer_sk#6, customer_sk#12], [customer_sk#6, customer_sk#12] - -(21) CometHashAggregate -Input [2]: [customer_sk#6, customer_sk#12] -Keys: [] -Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#12)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#12)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#12)) THEN 1 ELSE 0 END)] - -(22) CometExchange -Input [3]: [sum#14, sum#15, sum#16] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(23) CometHashAggregate -Input [3]: [sum#14, sum#15, sum#16] -Keys: [] -Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#12)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#12)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#12)) THEN 1 ELSE 0 END)] - -(24) CometColumnarToRow [codegen id : 5] -Input [3]: [store_only#17, catalog_only#18, store_and_catalog#19] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometNativeScan parquet spark_catalog.default.date_dim (25) - - -(25) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#20] -Condition : (((isnotnull(d_month_seq#20) AND (d_month_seq#20 >= 1200)) AND (d_month_seq#20 <= 1211)) AND isnotnull(d_date_sk#5)) - -(27) CometProject -Input [2]: [d_date_sk#5, d_month_seq#20] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(29) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/simplified.txt deleted file mode 100644 index dac440b122..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,sum] [store_only,catalog_only,store_and_catalog,sum(CASE WHEN (isnotnull(customer_sk) AND isnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnotnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END)] - CometExchange #1 - CometHashAggregate [customer_sk,customer_sk] [sum,sum,sum] - CometProject [customer_sk,customer_sk] - CometSortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] - CometSort [customer_sk,item_sk] - CometHashAggregate [customer_sk,item_sk,ss_customer_sk,ss_item_sk] - CometColumnarExchange [ss_customer_sk,ss_item_sk] #2 - WholeStageCodegen (2) - HashAggregate [ss_customer_sk,ss_item_sk] - Project [ss_item_sk,ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 - CometSort [customer_sk,item_sk] - CometHashAggregate [customer_sk,item_sk,cs_bill_customer_sk,cs_item_sk] - CometColumnarExchange [cs_bill_customer_sk,cs_item_sk] #4 - WholeStageCodegen (4) - HashAggregate [cs_bill_customer_sk,cs_item_sk] - Project [cs_bill_customer_sk,cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/explain.txt deleted file mode 100644 index 420f37fccc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,179 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometSortMergeJoin (20) - :- CometSort (11) - : +- CometHashAggregate (10) - : +- CometExchange (9) - : +- CometHashAggregate (8) - : +- CometProject (7) - : +- CometBroadcastHashJoin (6) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (5) - : +- CometProject (4) - : +- CometFilter (3) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (2) - +- CometSort (19) - +- CometHashAggregate (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometProject (15) - +- CometBroadcastHashJoin (14) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (12) - +- ReusedExchange (13) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -ReadSchema: struct - -(2) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(3) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(4) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(5) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(6) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(7) CometProject -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2], [ss_item_sk#1, ss_customer_sk#2] - -(8) CometHashAggregate -Input [2]: [ss_item_sk#1, ss_customer_sk#2] -Keys [2]: [ss_customer_sk#2, ss_item_sk#1] -Functions: [] - -(9) CometExchange -Input [2]: [ss_customer_sk#2, ss_item_sk#1] -Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometHashAggregate -Input [2]: [ss_customer_sk#2, ss_item_sk#1] -Keys [2]: [ss_customer_sk#2, ss_item_sk#1] -Functions: [] - -(11) CometSort -Input [2]: [customer_sk#7, item_sk#8] -Arguments: [customer_sk#7, item_sk#8], [customer_sk#7 ASC NULLS FIRST, item_sk#8 ASC NULLS FIRST] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#11), dynamicpruningexpression(cs_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(13) ReusedExchange [Reuses operator id: 5] -Output [1]: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [3]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11] -Right output [1]: [d_date_sk#13] -Arguments: [cs_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [4]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11, d_date_sk#13] -Arguments: [cs_bill_customer_sk#9, cs_item_sk#10], [cs_bill_customer_sk#9, cs_item_sk#10] - -(16) CometHashAggregate -Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Keys [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Functions: [] - -(17) CometExchange -Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Arguments: hashpartitioning(cs_bill_customer_sk#9, cs_item_sk#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(18) CometHashAggregate -Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Keys [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Functions: [] - -(19) CometSort -Input [2]: [customer_sk#14, item_sk#15] -Arguments: [customer_sk#14, item_sk#15], [customer_sk#14 ASC NULLS FIRST, item_sk#15 ASC NULLS FIRST] - -(20) CometSortMergeJoin -Left output [2]: [customer_sk#7, item_sk#8] -Right output [2]: [customer_sk#14, item_sk#15] -Arguments: [customer_sk#7, item_sk#8], [customer_sk#14, item_sk#15], FullOuter - -(21) CometProject -Input [4]: [customer_sk#7, item_sk#8, customer_sk#14, item_sk#15] -Arguments: [customer_sk#7, customer_sk#14], [customer_sk#7, customer_sk#14] - -(22) CometHashAggregate -Input [2]: [customer_sk#7, customer_sk#14] -Keys: [] -Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)] - -(23) CometExchange -Input [3]: [sum#16, sum#17, sum#18] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(24) CometHashAggregate -Input [3]: [sum#16, sum#17, sum#18] -Keys: [] -Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)] - -(25) CometColumnarToRow [codegen id : 1] -Input [3]: [store_only#19, catalog_only#20, store_and_catalog#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(28) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(30) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 12 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/simplified.txt deleted file mode 100644 index a7de047b25..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,sum] [store_only,catalog_only,store_and_catalog,sum(CASE WHEN (isnotnull(customer_sk) AND isnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnotnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END)] - CometExchange #1 - CometHashAggregate [customer_sk,customer_sk] [sum,sum,sum] - CometProject [customer_sk,customer_sk] - CometSortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] - CometSort [customer_sk,item_sk] - CometHashAggregate [customer_sk,item_sk,ss_customer_sk,ss_item_sk] - CometExchange [ss_customer_sk,ss_item_sk] #2 - CometHashAggregate [ss_customer_sk,ss_item_sk] - CometProject [ss_item_sk,ss_customer_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometSort [customer_sk,item_sk] - CometHashAggregate [customer_sk,item_sk,cs_bill_customer_sk,cs_item_sk] - CometExchange [cs_bill_customer_sk,cs_item_sk] #5 - CometHashAggregate [cs_bill_customer_sk,cs_item_sk] - CometProject [cs_bill_customer_sk,cs_item_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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/q97/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/explain.txt deleted file mode 100644 index 420f37fccc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/explain.txt +++ /dev/null @@ -1,179 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometSortMergeJoin (20) - :- CometSort (11) - : +- CometHashAggregate (10) - : +- CometExchange (9) - : +- CometHashAggregate (8) - : +- CometProject (7) - : +- CometBroadcastHashJoin (6) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (5) - : +- CometProject (4) - : +- CometFilter (3) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (2) - +- CometSort (19) - +- CometHashAggregate (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometProject (15) - +- CometBroadcastHashJoin (14) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (12) - +- ReusedExchange (13) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -ReadSchema: struct - -(2) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(3) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(4) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(5) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(6) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(7) CometProject -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2], [ss_item_sk#1, ss_customer_sk#2] - -(8) CometHashAggregate -Input [2]: [ss_item_sk#1, ss_customer_sk#2] -Keys [2]: [ss_customer_sk#2, ss_item_sk#1] -Functions: [] - -(9) CometExchange -Input [2]: [ss_customer_sk#2, ss_item_sk#1] -Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometHashAggregate -Input [2]: [ss_customer_sk#2, ss_item_sk#1] -Keys [2]: [ss_customer_sk#2, ss_item_sk#1] -Functions: [] - -(11) CometSort -Input [2]: [customer_sk#7, item_sk#8] -Arguments: [customer_sk#7, item_sk#8], [customer_sk#7 ASC NULLS FIRST, item_sk#8 ASC NULLS FIRST] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#11), dynamicpruningexpression(cs_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(13) ReusedExchange [Reuses operator id: 5] -Output [1]: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [3]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11] -Right output [1]: [d_date_sk#13] -Arguments: [cs_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [4]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11, d_date_sk#13] -Arguments: [cs_bill_customer_sk#9, cs_item_sk#10], [cs_bill_customer_sk#9, cs_item_sk#10] - -(16) CometHashAggregate -Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Keys [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Functions: [] - -(17) CometExchange -Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Arguments: hashpartitioning(cs_bill_customer_sk#9, cs_item_sk#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(18) CometHashAggregate -Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Keys [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Functions: [] - -(19) CometSort -Input [2]: [customer_sk#14, item_sk#15] -Arguments: [customer_sk#14, item_sk#15], [customer_sk#14 ASC NULLS FIRST, item_sk#15 ASC NULLS FIRST] - -(20) CometSortMergeJoin -Left output [2]: [customer_sk#7, item_sk#8] -Right output [2]: [customer_sk#14, item_sk#15] -Arguments: [customer_sk#7, item_sk#8], [customer_sk#14, item_sk#15], FullOuter - -(21) CometProject -Input [4]: [customer_sk#7, item_sk#8, customer_sk#14, item_sk#15] -Arguments: [customer_sk#7, customer_sk#14], [customer_sk#7, customer_sk#14] - -(22) CometHashAggregate -Input [2]: [customer_sk#7, customer_sk#14] -Keys: [] -Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)] - -(23) CometExchange -Input [3]: [sum#16, sum#17, sum#18] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(24) CometHashAggregate -Input [3]: [sum#16, sum#17, sum#18] -Keys: [] -Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)] - -(25) CometColumnarToRow [codegen id : 1] -Input [3]: [store_only#19, catalog_only#20, store_and_catalog#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(28) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(30) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 12 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/extended.txt deleted file mode 100644 index e1aefb8382..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index a7de047b25..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,sum] [store_only,catalog_only,store_and_catalog,sum(CASE WHEN (isnotnull(customer_sk) AND isnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnotnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END)] - CometExchange #1 - CometHashAggregate [customer_sk,customer_sk] [sum,sum,sum] - CometProject [customer_sk,customer_sk] - CometSortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] - CometSort [customer_sk,item_sk] - CometHashAggregate [customer_sk,item_sk,ss_customer_sk,ss_item_sk] - CometExchange [ss_customer_sk,ss_item_sk] #2 - CometHashAggregate [ss_customer_sk,ss_item_sk] - CometProject [ss_item_sk,ss_customer_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometSort [customer_sk,item_sk] - CometHashAggregate [customer_sk,item_sk,cs_bill_customer_sk,cs_item_sk] - CometExchange [cs_bill_customer_sk,cs_item_sk] #5 - CometHashAggregate [cs_bill_customer_sk,cs_item_sk] - CometProject [cs_bill_customer_sk,cs_item_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/explain.txt deleted file mode 100644 index 32c4235d27..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/explain.txt +++ /dev/null @@ -1,177 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometProject (25) - +- CometSort (24) - +- CometColumnarExchange (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.store_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.item (4) - +- ReusedExchange (11) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] - -(7) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(8) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(11) ReusedExchange [Reuses operator id: 31] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] - -(14) HashAggregate [codegen id : 3] -Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(15) CometColumnarExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(17) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#17] -Results [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#17,17,2) AS _w0#19, i_item_id#11] - -(18) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(22) Project [codegen id : 6] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] - -(23) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(24) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST] - -(25) CometProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -(26) CometColumnarToRow [codegen id : 7] -Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometNativeScan parquet spark_catalog.default.date_dim (27) - - -(27) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(29) CometProject -Input [2]: [d_date_sk#14, d_date#22] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(31) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/simplified.txt deleted file mode 100644 index 3f64b57ffa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/simplified.txt +++ /dev/null @@ -1,48 +0,0 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (6) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/explain.txt deleted file mode 100644 index 158e7520c2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,178 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometProject (25) - +- CometSort (24) - +- CometColumnarExchange (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] - -(23) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] - -(25) CometProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -(26) CometColumnarToRow [codegen id : 3] -Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(29) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(31) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/simplified.txt deleted file mode 100644 index af05a7d8c6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,40 +0,0 @@ -WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #2 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt deleted file mode 100644 index 158e7520c2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt +++ /dev/null @@ -1,178 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometProject (25) - +- CometSort (24) - +- CometColumnarExchange (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] - -(23) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] - -(25) CometProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -(26) CometColumnarToRow [codegen id : 3] -Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(29) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(31) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/extended.txt deleted file mode 100644 index 4c972848e7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/extended.txt +++ /dev/null @@ -1,34 +0,0 @@ -CometColumnarToRow -+- CometProject - +- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index af05a7d8c6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt +++ /dev/null @@ -1,40 +0,0 @@ -WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #2 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/explain.txt deleted file mode 100644 index 12e87af3b5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometNativeScan parquet spark_catalog.default.warehouse (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometNativeScan parquet spark_catalog.default.ship_mode (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.call_center (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometNativeScan parquet spark_catalog.default.date_dim (19) - - -(1) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_ship_mode_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Condition : (((isnotnull(cs_warehouse_sk#4) AND isnotnull(cs_ship_mode_sk#3)) AND isnotnull(cs_call_center_sk#2)) AND isnotnull(cs_ship_date_sk#1)) - -(3) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [cs_warehouse_sk#4], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7], [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] - -(8) CometNativeScan parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#8, sm_type#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Condition : isnotnull(sm_ship_mode_sk#8) - -(10) CometProject -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Arguments: [sm_ship_mode_sk#8, sm_type#10], [sm_ship_mode_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(sm_type#9, 30)) AS sm_type#10] - -(11) CometBroadcastExchange -Input [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [sm_ship_mode_sk#8, sm_type#10] - -(12) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] -Right output [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [cs_ship_mode_sk#3], [sm_ship_mode_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#10] -Arguments: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10], [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10] - -(14) CometNativeScan parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#11, cc_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#11, cc_name#12] -Condition : isnotnull(cc_call_center_sk#11) - -(16) CometBroadcastExchange -Input [2]: [cc_call_center_sk#11, cc_name#12] -Arguments: [cc_call_center_sk#11, cc_name#12] - -(17) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10] -Right output [2]: [cc_call_center_sk#11, cc_name#12] -Arguments: [cs_call_center_sk#2], [cc_call_center_sk#11], Inner, BuildRight - -(18) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_call_center_sk#11, cc_name#12] -Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12], [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12] - -(19) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_month_seq#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#13, d_month_seq#14] -Condition : (((isnotnull(d_month_seq#14) AND (d_month_seq#14 >= 1200)) AND (d_month_seq#14 <= 1211)) AND isnotnull(d_date_sk#13)) - -(21) CometProject -Input [2]: [d_date_sk#13, d_month_seq#14] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(23) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12] -Right output [1]: [d_date_sk#13] -Arguments: [cs_ship_date_sk#1], [d_date_sk#13], Inner, BuildRight - -(24) CometProject -Input [6]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12, d_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, _groupingexpression#15], [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#15] - -(25) CometHashAggregate -Input [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, sm_type#10, cc_name#12] -Functions [5]: [partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [8]: [_groupingexpression#15, sm_type#10, cc_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Arguments: hashpartitioning(_groupingexpression#15, sm_type#10, cc_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [8]: [_groupingexpression#15, sm_type#10, cc_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Keys [3]: [_groupingexpression#15, sm_type#10, cc_name#12] -Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,cc_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,cc_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, cc_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - -(29) CometColumnarToRow [codegen id : 1] -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/simplified.txt deleted file mode 100644 index 98a215dbc6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_datafusion/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 30) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 60) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 90) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [_groupingexpression,sm_type,cc_name] #1 - CometHashAggregate [cs_ship_date_sk,cs_sold_date_sk] [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum] - CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] - 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_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_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_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] - CometNativeScan 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 [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [cc_call_center_sk,cc_name] #4 - CometFilter [cc_call_center_sk,cc_name] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan 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/q99.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_iceberg_compat/explain.txt deleted file mode 100644 index 99bf2167b6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_ship_mode_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Condition : (((isnotnull(cs_warehouse_sk#4) AND isnotnull(cs_ship_mode_sk#3)) AND isnotnull(cs_call_center_sk#2)) AND isnotnull(cs_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [cs_warehouse_sk#4], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7], [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#8, sm_type#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Condition : isnotnull(sm_ship_mode_sk#8) - -(10) CometProject -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Arguments: [sm_ship_mode_sk#8, sm_type#10], [sm_ship_mode_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(sm_type#9, 30)) AS sm_type#10] - -(11) CometBroadcastExchange -Input [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [sm_ship_mode_sk#8, sm_type#10] - -(12) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] -Right output [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [cs_ship_mode_sk#3], [sm_ship_mode_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#10] -Arguments: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10], [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#11, cc_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#11, cc_name#12] -Condition : isnotnull(cc_call_center_sk#11) - -(16) CometBroadcastExchange -Input [2]: [cc_call_center_sk#11, cc_name#12] -Arguments: [cc_call_center_sk#11, cc_name#12] - -(17) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10] -Right output [2]: [cc_call_center_sk#11, cc_name#12] -Arguments: [cs_call_center_sk#2], [cc_call_center_sk#11], Inner, BuildRight - -(18) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_call_center_sk#11, cc_name#12] -Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12], [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_month_seq#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#13, d_month_seq#14] -Condition : (((isnotnull(d_month_seq#14) AND (d_month_seq#14 >= 1200)) AND (d_month_seq#14 <= 1211)) AND isnotnull(d_date_sk#13)) - -(21) CometProject -Input [2]: [d_date_sk#13, d_month_seq#14] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(23) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12] -Right output [1]: [d_date_sk#13] -Arguments: [cs_ship_date_sk#1], [d_date_sk#13], Inner, BuildRight - -(24) CometProject -Input [6]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12, d_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, _groupingexpression#15], [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#15] - -(25) CometHashAggregate -Input [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, sm_type#10, cc_name#12] -Functions [5]: [partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [8]: [_groupingexpression#15, sm_type#10, cc_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Arguments: hashpartitioning(_groupingexpression#15, sm_type#10, cc_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [8]: [_groupingexpression#15, sm_type#10, cc_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Keys [3]: [_groupingexpression#15, sm_type#10, cc_name#12] -Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,cc_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,cc_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, cc_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - -(29) CometColumnarToRow [codegen id : 1] -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_iceberg_compat/simplified.txt deleted file mode 100644 index 22896ee566..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 30) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 60) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 90) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [_groupingexpression,sm_type,cc_name] #1 - CometHashAggregate [cs_ship_date_sk,cs_sold_date_sk] [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum] - CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] - 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_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_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_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 [native_iceberg_compat] 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 [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [cc_call_center_sk,cc_name] #4 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] 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/q99/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/explain.txt deleted file mode 100644 index 99bf2167b6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_ship_mode_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Condition : (((isnotnull(cs_warehouse_sk#4) AND isnotnull(cs_ship_mode_sk#3)) AND isnotnull(cs_call_center_sk#2)) AND isnotnull(cs_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [cs_warehouse_sk#4], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7], [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#8, sm_type#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Condition : isnotnull(sm_ship_mode_sk#8) - -(10) CometProject -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Arguments: [sm_ship_mode_sk#8, sm_type#10], [sm_ship_mode_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(sm_type#9, 30)) AS sm_type#10] - -(11) CometBroadcastExchange -Input [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [sm_ship_mode_sk#8, sm_type#10] - -(12) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] -Right output [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [cs_ship_mode_sk#3], [sm_ship_mode_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#10] -Arguments: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10], [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#11, cc_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#11, cc_name#12] -Condition : isnotnull(cc_call_center_sk#11) - -(16) CometBroadcastExchange -Input [2]: [cc_call_center_sk#11, cc_name#12] -Arguments: [cc_call_center_sk#11, cc_name#12] - -(17) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10] -Right output [2]: [cc_call_center_sk#11, cc_name#12] -Arguments: [cs_call_center_sk#2], [cc_call_center_sk#11], Inner, BuildRight - -(18) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_call_center_sk#11, cc_name#12] -Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12], [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_month_seq#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#13, d_month_seq#14] -Condition : (((isnotnull(d_month_seq#14) AND (d_month_seq#14 >= 1200)) AND (d_month_seq#14 <= 1211)) AND isnotnull(d_date_sk#13)) - -(21) CometProject -Input [2]: [d_date_sk#13, d_month_seq#14] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(23) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12] -Right output [1]: [d_date_sk#13] -Arguments: [cs_ship_date_sk#1], [d_date_sk#13], Inner, BuildRight - -(24) CometProject -Input [6]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12, d_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, _groupingexpression#15], [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#15] - -(25) CometHashAggregate -Input [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, sm_type#10, cc_name#12] -Functions [5]: [partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [8]: [_groupingexpression#15, sm_type#10, cc_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Arguments: hashpartitioning(_groupingexpression#15, sm_type#10, cc_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [8]: [_groupingexpression#15, sm_type#10, cc_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Keys [3]: [_groupingexpression#15, sm_type#10, cc_name#12] -Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,cc_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,cc_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, cc_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - -(29) CometColumnarToRow [codegen id : 1] -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/extended.txt deleted file mode 100644 index 86cf2fc2f3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 28 out of 28 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 22896ee566..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 30) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 60) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 90) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [_groupingexpression,sm_type,cc_name] #1 - CometHashAggregate [cs_ship_date_sk,cs_sold_date_sk] [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum] - CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] - 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_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_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_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 [native_iceberg_compat] 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 [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [cc_call_center_sk,cc_name] #4 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/explain.txt deleted file mode 100644 index 5943563361..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/explain.txt +++ /dev/null @@ -1,291 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (44) -+- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Project (29) - : : +- * BroadcastHashJoin Inner BuildRight (28) - : : :- * Filter (11) - : : : +- * HashAggregate (10) - : : : +- * CometColumnarToRow (9) - : : : +- CometColumnarExchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_returns (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (27) - : : +- * Filter (26) - : : +- * HashAggregate (25) - : : +- * CometColumnarToRow (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * HashAggregate (21) - : : +- * CometColumnarToRow (20) - : : +- CometColumnarExchange (19) - : : +- * HashAggregate (18) - : : +- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Filter (14) - : : : +- * ColumnarToRow (13) - : : : +- Scan parquet spark_catalog.default.store_returns (12) - : : +- ReusedExchange (15) - : +- BroadcastExchange (34) - : +- * CometColumnarToRow (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometNativeScan parquet spark_catalog.default.store (30) - +- BroadcastExchange (41) - +- * CometColumnarToRow (40) - +- CometProject (39) - +- CometFilter (38) - +- CometNativeScan parquet spark_catalog.default.customer (37) - - -(1) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk), IsNotNull(sr_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 2] -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] - -(3) Filter [codegen id : 2] -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [sr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 2] -Output [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] -Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] - -(7) HashAggregate [codegen id : 2] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] -Aggregate Attributes [1]: [sum#7] -Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] - -(8) CometColumnarExchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(9) CometColumnarToRow [codegen id : 9] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] - -(10) HashAggregate [codegen id : 9] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#9] -Results [3]: [sr_customer_sk#1 AS ctr_customer_sk#10, sr_store_sk#2 AS ctr_store_sk#11, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#9,17,2) AS ctr_total_return#12] - -(11) Filter [codegen id : 9] -Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12] -Condition : isnotnull(ctr_total_return#12) - -(12) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(13) ColumnarToRow [codegen id : 4] -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] - -(14) Filter [codegen id : 4] -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Condition : isnotnull(sr_store_sk#2) - -(15) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#6] - -(16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 4] -Output [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] -Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] - -(18) HashAggregate [codegen id : 4] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] -Aggregate Attributes [1]: [sum#13] -Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#14] - -(19) CometColumnarExchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#14] -Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(20) CometColumnarToRow [codegen id : 5] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#14] - -(21) HashAggregate [codegen id : 5] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#14] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#9] -Results [2]: [sr_store_sk#2 AS ctr_store_sk#11, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#9,17,2) AS ctr_total_return#12] - -(22) HashAggregate [codegen id : 5] -Input [2]: [ctr_store_sk#11, ctr_total_return#12] -Keys [1]: [ctr_store_sk#11] -Functions [1]: [partial_avg(ctr_total_return#12)] -Aggregate Attributes [2]: [sum#15, count#16] -Results [3]: [ctr_store_sk#11, sum#17, count#18] - -(23) CometColumnarExchange -Input [3]: [ctr_store_sk#11, sum#17, count#18] -Arguments: hashpartitioning(ctr_store_sk#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometColumnarToRow [codegen id : 6] -Input [3]: [ctr_store_sk#11, sum#17, count#18] - -(25) HashAggregate [codegen id : 6] -Input [3]: [ctr_store_sk#11, sum#17, count#18] -Keys [1]: [ctr_store_sk#11] -Functions [1]: [avg(ctr_total_return#12)] -Aggregate Attributes [1]: [avg(ctr_total_return#12)#19] -Results [2]: [(avg(ctr_total_return#12)#19 * 1.2) AS (avg(ctr_total_return) * 1.2)#20, ctr_store_sk#11 AS ctr_store_sk#11#21] - -(26) Filter [codegen id : 6] -Input [2]: [(avg(ctr_total_return) * 1.2)#20, ctr_store_sk#11#21] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#20) - -(27) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#20, ctr_store_sk#11#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=4] - -(28) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ctr_store_sk#11] -Right keys [1]: [ctr_store_sk#11#21] -Join type: Inner -Join condition: (cast(ctr_total_return#12 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#20) - -(29) Project [codegen id : 9] -Output [2]: [ctr_customer_sk#10, ctr_store_sk#11] -Input [5]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12, (avg(ctr_total_return) * 1.2)#20, ctr_store_sk#11#21] - -(30) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#22, s_state#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [s_store_sk#22, s_state#23] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#23, 2, true, false, true) = TN) AND isnotnull(s_store_sk#22)) - -(32) CometProject -Input [2]: [s_store_sk#22, s_state#23] -Arguments: [s_store_sk#22], [s_store_sk#22] - -(33) CometColumnarToRow [codegen id : 7] -Input [1]: [s_store_sk#22] - -(34) BroadcastExchange -Input [1]: [s_store_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ctr_store_sk#11] -Right keys [1]: [s_store_sk#22] -Join type: Inner -Join condition: None - -(36) Project [codegen id : 9] -Output [1]: [ctr_customer_sk#10] -Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, s_store_sk#22] - -(37) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#24, c_customer_id#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(38) CometFilter -Input [2]: [c_customer_sk#24, c_customer_id#25] -Condition : isnotnull(c_customer_sk#24) - -(39) CometProject -Input [2]: [c_customer_sk#24, c_customer_id#25] -Arguments: [c_customer_sk#24, c_customer_id#26], [c_customer_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#25, 16, true, false, true) AS c_customer_id#26] - -(40) CometColumnarToRow [codegen id : 8] -Input [2]: [c_customer_sk#24, c_customer_id#26] - -(41) BroadcastExchange -Input [2]: [c_customer_sk#24, c_customer_id#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(42) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ctr_customer_sk#10] -Right keys [1]: [c_customer_sk#24] -Join type: Inner -Join condition: None - -(43) Project [codegen id : 9] -Output [1]: [c_customer_id#26] -Input [3]: [ctr_customer_sk#10, c_customer_sk#24, c_customer_id#26] - -(44) TakeOrderedAndProject -Input [1]: [c_customer_id#26] -Arguments: 100, [c_customer_id#26 ASC NULLS FIRST], [c_customer_id#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometNativeScan parquet spark_catalog.default.date_dim (45) - - -(45) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [d_date_sk#6, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2000)) AND isnotnull(d_date_sk#6)) - -(47) CometProject -Input [2]: [d_date_sk#6, d_year#27] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(49) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 12 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/simplified.txt deleted file mode 100644 index b45596c7a2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/simplified.txt +++ /dev/null @@ -1,72 +0,0 @@ -TakeOrderedAndProject [c_customer_id] - WholeStageCodegen (9) - Project [c_customer_id] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk] - BroadcastHashJoin [ctr_store_sk,s_store_sk] - Project [ctr_customer_sk,ctr_store_sk] - BroadcastHashJoin [ctr_store_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [sr_customer_sk,sr_store_sk] #1 - WholeStageCodegen (2) - HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] - Project [sr_customer_sk,sr_store_sk,sr_return_amt] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_store_sk,sr_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (6) - Filter [(avg(ctr_total_return) * 1.2)] - HashAggregate [ctr_store_sk,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),ctr_store_sk,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ctr_store_sk] #4 - WholeStageCodegen (5) - HashAggregate [ctr_store_sk,ctr_total_return] [sum,count,sum,count] - HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_store_sk,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [sr_customer_sk,sr_store_sk] #5 - WholeStageCodegen (4) - HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] - Project [sr_customer_sk,sr_store_sk,sr_return_amt] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id] [c_customer_sk,c_customer_id] - CometFilter [c_customer_sk,c_customer_id] - CometNativeScan 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/q1.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/explain.txt deleted file mode 100644 index b8ba9b884c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,269 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometTakeOrderedAndProject (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometFilter (12) - : : : +- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (25) - : : +- CometFilter (24) - : : +- CometHashAggregate (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometHashAggregate (20) - : : +- CometExchange (19) - : : +- CometHashAggregate (18) - : : +- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (13) - : : +- ReusedExchange (15) - : +- CometBroadcastExchange (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (28) - +- CometBroadcastExchange (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (34) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk), IsNotNull(sr_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [sr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] -Arguments: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3], [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] - -(9) CometHashAggregate -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] - -(10) CometExchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] - -(12) CometFilter -Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] -Condition : isnotnull(ctr_total_return#11) - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Condition : isnotnull(sr_store_sk#2) - -(15) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#6] - -(16) CometBroadcastHashJoin -Left output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [sr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(17) CometProject -Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] -Arguments: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3], [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] - -(18) CometHashAggregate -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] - -(19) CometExchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#13] -Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometHashAggregate -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#13] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] - -(21) CometHashAggregate -Input [2]: [ctr_store_sk#10, ctr_total_return#11] -Keys [1]: [ctr_store_sk#10] -Functions [1]: [partial_avg(ctr_total_return#11)] - -(22) CometExchange -Input [3]: [ctr_store_sk#10, sum#14, count#15] -Arguments: hashpartitioning(ctr_store_sk#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(23) CometHashAggregate -Input [3]: [ctr_store_sk#10, sum#14, count#15] -Keys [1]: [ctr_store_sk#10] -Functions [1]: [avg(ctr_total_return#11)] - -(24) CometFilter -Input [2]: [(avg(ctr_total_return) * 1.2)#16, ctr_store_sk#10#17] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#16) - -(25) CometBroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#16, ctr_store_sk#10#17] -Arguments: [(avg(ctr_total_return) * 1.2)#16, ctr_store_sk#10#17] - -(26) CometBroadcastHashJoin -Left output [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] -Right output [2]: [(avg(ctr_total_return) * 1.2)#16, ctr_store_sk#10#17] -Arguments: [ctr_store_sk#10], [ctr_store_sk#10#17], Inner, (cast(ctr_total_return#11 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#16), BuildRight - -(27) CometProject -Input [5]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11, (avg(ctr_total_return) * 1.2)#16, ctr_store_sk#10#17] -Arguments: [ctr_customer_sk#9, ctr_store_sk#10], [ctr_customer_sk#9, ctr_store_sk#10] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#18, s_state#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(29) CometFilter -Input [2]: [s_store_sk#18, s_state#19] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#19, 2, true, false, true) = TN) AND isnotnull(s_store_sk#18)) - -(30) CometProject -Input [2]: [s_store_sk#18, s_state#19] -Arguments: [s_store_sk#18], [s_store_sk#18] - -(31) CometBroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: [s_store_sk#18] - -(32) CometBroadcastHashJoin -Left output [2]: [ctr_customer_sk#9, ctr_store_sk#10] -Right output [1]: [s_store_sk#18] -Arguments: [ctr_store_sk#10], [s_store_sk#18], Inner, BuildRight - -(33) CometProject -Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, s_store_sk#18] -Arguments: [ctr_customer_sk#9], [ctr_customer_sk#9] - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#20, c_customer_id#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [c_customer_sk#20, c_customer_id#21] -Condition : isnotnull(c_customer_sk#20) - -(36) CometProject -Input [2]: [c_customer_sk#20, c_customer_id#21] -Arguments: [c_customer_sk#20, c_customer_id#22], [c_customer_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#21, 16, true, false, true) AS c_customer_id#22] - -(37) CometBroadcastExchange -Input [2]: [c_customer_sk#20, c_customer_id#22] -Arguments: [c_customer_sk#20, c_customer_id#22] - -(38) CometBroadcastHashJoin -Left output [1]: [ctr_customer_sk#9] -Right output [2]: [c_customer_sk#20, c_customer_id#22] -Arguments: [ctr_customer_sk#9], [c_customer_sk#20], Inner, BuildRight - -(39) CometProject -Input [3]: [ctr_customer_sk#9, c_customer_sk#20, c_customer_id#22] -Arguments: [c_customer_id#22], [c_customer_id#22] - -(40) CometTakeOrderedAndProject -Input [1]: [c_customer_id#22] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#22 ASC NULLS FIRST], output=[c_customer_id#22]), [c_customer_id#22], 100, 0, [c_customer_id#22 ASC NULLS FIRST], [c_customer_id#22] - -(41) CometColumnarToRow [codegen id : 1] -Input [1]: [c_customer_id#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) - - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(44) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(45) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(46) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 13 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/simplified.txt deleted file mode 100644 index c745a5b255..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id] - CometProject [c_customer_id] - CometBroadcastHashJoin [ctr_customer_sk,c_customer_sk,c_customer_id] - CometProject [ctr_customer_sk] - CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,s_store_sk] - CometProject [ctr_customer_sk,ctr_store_sk] - CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_store_sk] - CometFilter [ctr_customer_sk,ctr_store_sk,ctr_total_return] - CometHashAggregate [sum] [ctr_customer_sk,ctr_store_sk,ctr_total_return,sr_customer_sk,sr_store_sk,sum(UnscaledValue(sr_return_amt))] - CometExchange [sr_customer_sk,sr_store_sk] #1 - CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] - CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_store_sk] #4 - CometFilter [(avg(ctr_total_return) * 1.2),ctr_store_sk] - CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_store_sk,avg(ctr_total_return),ctr_store_sk] - CometExchange [ctr_store_sk] #5 - CometHashAggregate [ctr_total_return] [ctr_store_sk,sum,count] - CometHashAggregate [sr_customer_sk,sum] [ctr_store_sk,ctr_total_return,sr_store_sk,sum(UnscaledValue(sr_return_amt))] - CometExchange [sr_customer_sk,sr_store_sk] #6 - CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] - CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - 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 [native_iceberg_compat] 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 - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [c_customer_sk,c_customer_id] #8 - CometProject [c_customer_id] [c_customer_sk,c_customer_id] - CometFilter [c_customer_sk,c_customer_id] - CometScan [native_iceberg_compat] 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/q1/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt deleted file mode 100644 index b8ba9b884c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt +++ /dev/null @@ -1,269 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometTakeOrderedAndProject (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometFilter (12) - : : : +- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (25) - : : +- CometFilter (24) - : : +- CometHashAggregate (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometHashAggregate (20) - : : +- CometExchange (19) - : : +- CometHashAggregate (18) - : : +- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (13) - : : +- ReusedExchange (15) - : +- CometBroadcastExchange (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (28) - +- CometBroadcastExchange (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (34) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk), IsNotNull(sr_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [sr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] -Arguments: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3], [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] - -(9) CometHashAggregate -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] - -(10) CometExchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] - -(12) CometFilter -Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] -Condition : isnotnull(ctr_total_return#11) - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Condition : isnotnull(sr_store_sk#2) - -(15) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#6] - -(16) CometBroadcastHashJoin -Left output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [sr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(17) CometProject -Input [5]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4, d_date_sk#6] -Arguments: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3], [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] - -(18) CometHashAggregate -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] - -(19) CometExchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#13] -Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometHashAggregate -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#13] -Keys [2]: [sr_customer_sk#1, sr_store_sk#2] -Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] - -(21) CometHashAggregate -Input [2]: [ctr_store_sk#10, ctr_total_return#11] -Keys [1]: [ctr_store_sk#10] -Functions [1]: [partial_avg(ctr_total_return#11)] - -(22) CometExchange -Input [3]: [ctr_store_sk#10, sum#14, count#15] -Arguments: hashpartitioning(ctr_store_sk#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(23) CometHashAggregate -Input [3]: [ctr_store_sk#10, sum#14, count#15] -Keys [1]: [ctr_store_sk#10] -Functions [1]: [avg(ctr_total_return#11)] - -(24) CometFilter -Input [2]: [(avg(ctr_total_return) * 1.2)#16, ctr_store_sk#10#17] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#16) - -(25) CometBroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#16, ctr_store_sk#10#17] -Arguments: [(avg(ctr_total_return) * 1.2)#16, ctr_store_sk#10#17] - -(26) CometBroadcastHashJoin -Left output [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] -Right output [2]: [(avg(ctr_total_return) * 1.2)#16, ctr_store_sk#10#17] -Arguments: [ctr_store_sk#10], [ctr_store_sk#10#17], Inner, (cast(ctr_total_return#11 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#16), BuildRight - -(27) CometProject -Input [5]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11, (avg(ctr_total_return) * 1.2)#16, ctr_store_sk#10#17] -Arguments: [ctr_customer_sk#9, ctr_store_sk#10], [ctr_customer_sk#9, ctr_store_sk#10] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#18, s_state#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(29) CometFilter -Input [2]: [s_store_sk#18, s_state#19] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#19, 2, true, false, true) = TN) AND isnotnull(s_store_sk#18)) - -(30) CometProject -Input [2]: [s_store_sk#18, s_state#19] -Arguments: [s_store_sk#18], [s_store_sk#18] - -(31) CometBroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: [s_store_sk#18] - -(32) CometBroadcastHashJoin -Left output [2]: [ctr_customer_sk#9, ctr_store_sk#10] -Right output [1]: [s_store_sk#18] -Arguments: [ctr_store_sk#10], [s_store_sk#18], Inner, BuildRight - -(33) CometProject -Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, s_store_sk#18] -Arguments: [ctr_customer_sk#9], [ctr_customer_sk#9] - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#20, c_customer_id#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [c_customer_sk#20, c_customer_id#21] -Condition : isnotnull(c_customer_sk#20) - -(36) CometProject -Input [2]: [c_customer_sk#20, c_customer_id#21] -Arguments: [c_customer_sk#20, c_customer_id#22], [c_customer_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#21, 16, true, false, true) AS c_customer_id#22] - -(37) CometBroadcastExchange -Input [2]: [c_customer_sk#20, c_customer_id#22] -Arguments: [c_customer_sk#20, c_customer_id#22] - -(38) CometBroadcastHashJoin -Left output [1]: [ctr_customer_sk#9] -Right output [2]: [c_customer_sk#20, c_customer_id#22] -Arguments: [ctr_customer_sk#9], [c_customer_sk#20], Inner, BuildRight - -(39) CometProject -Input [3]: [ctr_customer_sk#9, c_customer_sk#20, c_customer_id#22] -Arguments: [c_customer_id#22], [c_customer_id#22] - -(40) CometTakeOrderedAndProject -Input [1]: [c_customer_id#22] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#22 ASC NULLS FIRST], output=[c_customer_id#22]), [c_customer_id#22], 100, 0, [c_customer_id#22 ASC NULLS FIRST], [c_customer_id#22] - -(41) CometColumnarToRow [codegen id : 1] -Input [1]: [c_customer_id#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) - - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(44) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(45) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(46) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 13 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/extended.txt deleted file mode 100644 index 8aa14c43dd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/extended.txt +++ /dev/null @@ -1,53 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 46 out of 49 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index c745a5b255..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id] - CometProject [c_customer_id] - CometBroadcastHashJoin [ctr_customer_sk,c_customer_sk,c_customer_id] - CometProject [ctr_customer_sk] - CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,s_store_sk] - CometProject [ctr_customer_sk,ctr_store_sk] - CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_store_sk] - CometFilter [ctr_customer_sk,ctr_store_sk,ctr_total_return] - CometHashAggregate [sum] [ctr_customer_sk,ctr_store_sk,ctr_total_return,sr_customer_sk,sr_store_sk,sum(UnscaledValue(sr_return_amt))] - CometExchange [sr_customer_sk,sr_store_sk] #1 - CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] - CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_store_sk] #4 - CometFilter [(avg(ctr_total_return) * 1.2),ctr_store_sk] - CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_store_sk,avg(ctr_total_return),ctr_store_sk] - CometExchange [ctr_store_sk] #5 - CometHashAggregate [ctr_total_return] [ctr_store_sk,sum,count] - CometHashAggregate [sr_customer_sk,sum] [ctr_store_sk,ctr_total_return,sr_store_sk,sum(UnscaledValue(sr_return_amt))] - CometExchange [sr_customer_sk,sr_store_sk] #6 - CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] - CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - 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 [native_iceberg_compat] 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 - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [c_customer_sk,c_customer_id] #8 - CometProject [c_customer_id] [c_customer_sk,c_customer_id] - CometFilter [c_customer_sk,c_customer_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/explain.txt deleted file mode 100644 index a603d43983..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/explain.txt +++ /dev/null @@ -1,295 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (45) -+- * HashAggregate (44) - +- * CometColumnarToRow (43) - +- CometColumnarExchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometNativeScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (38) - +- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometNativeScan parquet spark_catalog.default.customer_demographics (34) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(4) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] - -(6) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#9] - -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None - -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#6] -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] - -(9) BroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#6] -Join type: LeftSemi -Join condition: None - -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] - -(13) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#9] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#10] -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#9] - -(16) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(17) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#10] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#8)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] - -(20) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#9] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#12] -Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#9] - -(23) BroadcastExchange -Input [1]: [cs_ship_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#12] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(25) Filter [codegen id : 9] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(26) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(27) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#14, ca_county#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [ca_address_sk#14, ca_county#15] -Condition : (ca_county#15 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#14)) - -(29) CometProject -Input [2]: [ca_address_sk#14, ca_county#15] -Arguments: [ca_address_sk#14], [ca_address_sk#14] - -(30) CometColumnarToRow [codegen id : 7] -Input [1]: [ca_address_sk#14] - -(31) BroadcastExchange -Input [1]: [ca_address_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#14] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 9] -Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#14] - -(34) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#16, cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(35) CometFilter -Input [9]: [cd_demo_sk#16, cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Condition : isnotnull(cd_demo_sk#16) - -(36) CometProject -Input [9]: [cd_demo_sk#16, cd_gender#17, cd_marital_status#18, cd_education_status#19, cd_purchase_estimate#20, cd_credit_rating#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Arguments: [cd_demo_sk#16, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24], [cd_demo_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#17, 1, true, false, true) AS cd_gender#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#18, 1, true, false, true) AS cd_marital_status#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#19, 20, true, false, true) AS cd_education_status#27, cd_purchase_estimate#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#21, 10, true, false, true) AS cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] - -(37) CometColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#16, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] - -(38) BroadcastExchange -Input [9]: [cd_demo_sk#16, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#16] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 9] -Output [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#16, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] - -(41) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#29] -Results [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#30] - -(42) CometColumnarExchange -Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#30] -Arguments: hashpartitioning(cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) CometColumnarToRow [codegen id : 10] -Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#30] - -(44) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#30] -Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#20, cd_credit_rating#28, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#31] -Results [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, count(1)#31 AS cnt1#32, cd_purchase_estimate#20, count(1)#31 AS cnt2#33, cd_credit_rating#28, count(1)#31 AS cnt3#34, cd_dep_count#22, count(1)#31 AS cnt4#35, cd_dep_employed_count#23, count(1)#31 AS cnt5#36, cd_dep_college_count#24, count(1)#31 AS cnt6#37] - -(45) TakeOrderedAndProject -Input [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#32, cd_purchase_estimate#20, cnt2#33, cd_credit_rating#28, cnt3#34, cd_dep_count#22, cnt4#35, cd_dep_employed_count#23, cnt5#36, cd_dep_college_count#24, cnt6#37] -Arguments: 100, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#20 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST, cd_dep_count#22 ASC NULLS FIRST, cd_dep_employed_count#23 ASC NULLS FIRST, cd_dep_college_count#24 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#32, cd_purchase_estimate#20, cnt2#33, cd_credit_rating#28, cnt3#34, cd_dep_count#22, cnt4#35, cd_dep_employed_count#23, cnt5#36, cd_dep_college_count#24, cnt6#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (50) -+- * CometColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometNativeScan parquet spark_catalog.default.date_dim (46) - - -(46) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#38, d_moy#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_date_sk#9, d_year#38, d_moy#39] -Condition : (((((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2002)) AND (d_moy#39 >= 1)) AND (d_moy#39 <= 4)) AND isnotnull(d_date_sk#9)) - -(48) CometProject -Input [3]: [d_date_sk#9, d_year#38, d_moy#39] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(49) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(50) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/simplified.txt deleted file mode 100644 index 0cc108b0b0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/simplified.txt +++ /dev/null @@ -1,77 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (10) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [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] - 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] - CometNativeScan 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/q10.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/explain.txt deleted file mode 100644 index 28df28d9a7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * Filter (27) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2002)) AND (d_moy#11 >= 1)) AND (d_moy#11 <= 4)) AND isnotnull(d_date_sk#9)) - -(6) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: [d_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#9] -Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -Arguments: [ss_customer_sk#6], [ss_customer_sk#6] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: [ss_customer_sk#6] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Right output [1]: [ss_customer_sk#6] -Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#9] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Right output [1]: [d_date_sk#9] -Arguments: [ws_sold_date_sk#13], [d_date_sk#9], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#9] -Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#12] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#12] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#9] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Right output [1]: [d_date_sk#9] -Arguments: [cs_sold_date_sk#16], [d_date_sk#9], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#9] -Arguments: [cs_ship_customer_sk#15], [cs_ship_customer_sk#15] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#15] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#15] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(27) Filter [codegen id : 5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(28) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_county#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [ca_address_sk#18, ca_county#19] -Condition : (ca_county#19 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#18)) - -(31) CometProject -Input [2]: [ca_address_sk#18, ca_county#19] -Arguments: [ca_address_sk#18], [ca_address_sk#18] - -(32) CometColumnarToRow [codegen id : 3] -Input [1]: [ca_address_sk#18] - -(33) BroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(34) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#18] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 5] -Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(37) CometFilter -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#20) - -(38) CometProject -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#21, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#22, 1, true, false, true) AS cd_marital_status#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#23, 20, true, false, true) AS cd_education_status#31, cd_purchase_estimate#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#25, 10, true, false, true) AS cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(39) CometColumnarToRow [codegen id : 4] -Input [9]: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(40) BroadcastExchange -Input [9]: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(41) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 5] -Output [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(43) HashAggregate [codegen id : 5] -Input [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#33] -Results [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] - -(44) CometColumnarExchange -Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] -Arguments: hashpartitioning(cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 6] -Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] - -(46) HashAggregate [codegen id : 6] -Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] -Keys [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#35] -Results [14]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, count(1)#35 AS cnt1#36, cd_purchase_estimate#24, count(1)#35 AS cnt2#37, cd_credit_rating#32, count(1)#35 AS cnt3#38, cd_dep_count#26, count(1)#35 AS cnt4#39, cd_dep_employed_count#27, count(1)#35 AS cnt5#40, cd_dep_college_count#28, count(1)#35 AS cnt6#41] - -(47) TakeOrderedAndProject -Input [14]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41] -Arguments: 100, [cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_education_status#31 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#32 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2002)) AND (d_moy#11 >= 1)) AND (d_moy#11 <= 4)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(52) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/simplified.txt deleted file mode 100644 index e7193f87e1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (6) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [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] - 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 [native_iceberg_compat] 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/q10/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt deleted file mode 100644 index 28df28d9a7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * Filter (27) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2002)) AND (d_moy#11 >= 1)) AND (d_moy#11 <= 4)) AND isnotnull(d_date_sk#9)) - -(6) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: [d_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#9] -Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -Arguments: [ss_customer_sk#6], [ss_customer_sk#6] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: [ss_customer_sk#6] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Right output [1]: [ss_customer_sk#6] -Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#9] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Right output [1]: [d_date_sk#9] -Arguments: [ws_sold_date_sk#13], [d_date_sk#9], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#9] -Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#12] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#12] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#9] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Right output [1]: [d_date_sk#9] -Arguments: [cs_sold_date_sk#16], [d_date_sk#9], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#9] -Arguments: [cs_ship_customer_sk#15], [cs_ship_customer_sk#15] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#15] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#15] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(27) Filter [codegen id : 5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(28) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_county#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [ca_address_sk#18, ca_county#19] -Condition : (ca_county#19 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#18)) - -(31) CometProject -Input [2]: [ca_address_sk#18, ca_county#19] -Arguments: [ca_address_sk#18], [ca_address_sk#18] - -(32) CometColumnarToRow [codegen id : 3] -Input [1]: [ca_address_sk#18] - -(33) BroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(34) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#18] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 5] -Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(37) CometFilter -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#20) - -(38) CometProject -Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#21, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#22, 1, true, false, true) AS cd_marital_status#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#23, 20, true, false, true) AS cd_education_status#31, cd_purchase_estimate#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#25, 10, true, false, true) AS cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(39) CometColumnarToRow [codegen id : 4] -Input [9]: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(40) BroadcastExchange -Input [9]: [cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(41) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 5] -Output [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#20, cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(43) HashAggregate [codegen id : 5] -Input [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#33] -Results [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] - -(44) CometColumnarExchange -Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] -Arguments: hashpartitioning(cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 6] -Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] - -(46) HashAggregate [codegen id : 6] -Input [9]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#34] -Keys [8]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cd_purchase_estimate#24, cd_credit_rating#32, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#35] -Results [14]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, count(1)#35 AS cnt1#36, cd_purchase_estimate#24, count(1)#35 AS cnt2#37, cd_credit_rating#32, count(1)#35 AS cnt3#38, cd_dep_count#26, count(1)#35 AS cnt4#39, cd_dep_employed_count#27, count(1)#35 AS cnt5#40, cd_dep_college_count#28, count(1)#35 AS cnt6#41] - -(47) TakeOrderedAndProject -Input [14]: [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41] -Arguments: 100, [cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_education_status#31 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#32 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#29, cd_marital_status#30, cd_education_status#31, cnt1#36, cd_purchase_estimate#24, cnt2#37, cd_credit_rating#32, cnt3#38, cd_dep_count#26, cnt4#39, cd_dep_employed_count#27, cnt5#40, cd_dep_college_count#28, cnt6#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : (((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2002)) AND (d_moy#11 >= 1)) AND (d_moy#11 <= 4)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(52) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/extended.txt deleted file mode 100644 index a6f33d6f7e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/extended.txt +++ /dev/null @@ -1,63 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index e7193f87e1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (6) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [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] - 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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/explain.txt deleted file mode 100644 index d697d27f18..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/explain.txt +++ /dev/null @@ -1,518 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (80) -+- * Project (79) - +- * BroadcastHashJoin Inner BuildRight (78) - :- * Project (59) - : +- * BroadcastHashJoin Inner BuildRight (58) - : :- * Project (38) - : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- * HashAggregate (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * Project (28) - : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (26) - : : : +- * Filter (25) - : : : +- * ColumnarToRow (24) - : : : +- Scan parquet spark_catalog.default.store_sales (23) - : : +- ReusedExchange (29) - : +- BroadcastExchange (57) - : +- * Filter (56) - : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) - : +- CometColumnarExchange (53) - : +- * HashAggregate (52) - : +- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Project (48) - : : +- * BroadcastHashJoin Inner BuildRight (47) - : : :- * CometColumnarToRow (42) - : : : +- CometProject (41) - : : : +- CometFilter (40) - : : : +- CometNativeScan parquet spark_catalog.default.customer (39) - : : +- BroadcastExchange (46) - : : +- * Filter (45) - : : +- * ColumnarToRow (44) - : : +- Scan parquet spark_catalog.default.web_sales (43) - : +- ReusedExchange (49) - +- BroadcastExchange (77) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- CometColumnarExchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * Project (69) - : +- * BroadcastHashJoin Inner BuildRight (68) - : :- * CometColumnarToRow (63) - : : +- CometProject (62) - : : +- CometFilter (61) - : : +- CometNativeScan parquet spark_catalog.default.customer (60) - : +- BroadcastExchange (67) - : +- * Filter (66) - : +- * ColumnarToRow (65) - : +- Scan parquet spark_catalog.default.web_sales (64) - +- ReusedExchange (70) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] - -(4) CometColumnarToRow [codegen id : 3] -Input [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] - -(5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(7) Filter [codegen id : 1] -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#15) - -(8) BroadcastExchange -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#15] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(11) ReusedExchange [Reuses operator id: 84] -Output [2]: [d_date_sk#20, d_year#21] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#18] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] - -(14) HashAggregate [codegen id : 3] -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] -Aggregate Attributes [1]: [sum#22] -Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] - -(15) CometColumnarExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 16] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] - -(17) HashAggregate [codegen id : 16] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24] -Results [2]: [c_customer_id#9 AS customer_id#25, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24,18,2) AS year_total#26] - -(18) Filter [codegen id : 16] -Input [2]: [customer_id#25, year_total#26] -Condition : (isnotnull(year_total#26) AND (year_total#26 > 0.00)) - -(19) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(20) CometFilter -Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] -Condition : (isnotnull(c_customer_sk#27) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#28, 16, true, false, true))) - -(21) CometProject -Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] -Arguments: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14], [c_customer_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#28, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#29, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#30, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#31, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#33, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#34, 50, true, false, true) AS c_email_address#14] - -(22) CometColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] - -(23) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_sold_date_sk#38 IN dynamicpruning#39)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] - -(25) Filter [codegen id : 4] -Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Condition : isnotnull(ss_customer_sk#35) - -(26) BroadcastExchange -Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#27] -Right keys [1]: [ss_customer_sk#35] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Input [12]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] - -(29) ReusedExchange [Reuses operator id: 88] -Output [2]: [d_date_sk#40, d_year#41] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38, d_date_sk#40, d_year#41] - -(32) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] -Aggregate Attributes [1]: [sum#42] -Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] - -(33) CometColumnarExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 7] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] - -(35) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))#24] -Results [3]: [c_customer_id#9 AS customer_id#44, c_preferred_cust_flag#12 AS customer_preferred_cust_flag#45, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))#24,18,2) AS year_total#46] - -(36) BroadcastExchange -Input [3]: [customer_id#44, customer_preferred_cust_flag#45, year_total#46] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(37) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#44] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 16] -Output [4]: [customer_id#25, year_total#26, customer_preferred_cust_flag#45, year_total#46] -Input [5]: [customer_id#25, year_total#26, customer_id#44, customer_preferred_cust_flag#45, year_total#46] - -(39) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [8]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54] -Condition : (isnotnull(c_customer_sk#47) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#48, 16, true, false, true))) - -(41) CometProject -Input [8]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54] -Arguments: [c_customer_sk#47, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60], [c_customer_sk#47, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#48, 16, true, false, true) AS c_customer_id#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#49, 20, true, false, true) AS c_first_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#50, 30, true, false, true) AS c_last_name#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#51, 1, true, false, true) AS c_preferred_cust_flag#58, c_birth_country#52, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#53, 13, true, false, true) AS c_login#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#54, 50, true, false, true) AS c_email_address#60] - -(42) CometColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#47, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60] - -(43) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#64), dynamicpruningexpression(ws_sold_date_sk#64 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(44) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64] - -(45) Filter [codegen id : 8] -Input [4]: [ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64] -Condition : isnotnull(ws_bill_customer_sk#61) - -(46) BroadcastExchange -Input [4]: [ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -(47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#47] -Right keys [1]: [ws_bill_customer_sk#61] -Join type: Inner -Join condition: None - -(48) Project [codegen id : 10] -Output [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64] -Input [12]: [c_customer_sk#47, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, ws_bill_customer_sk#61, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64] - -(49) ReusedExchange [Reuses operator id: 84] -Output [2]: [d_date_sk#65, d_year#66] - -(50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#64] -Right keys [1]: [d_date_sk#65] -Join type: Inner -Join condition: None - -(51) Project [codegen id : 10] -Output [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, d_year#66] -Input [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, ws_sold_date_sk#64, d_date_sk#65, d_year#66] - -(52) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, ws_ext_discount_amt#62, ws_ext_list_price#63, d_year#66] -Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#63 - ws_ext_discount_amt#62)))] -Aggregate Attributes [1]: [sum#67] -Results [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66, sum#68] - -(53) CometColumnarExchange -Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66, sum#68] -Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(54) CometColumnarToRow [codegen id : 11] -Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66, sum#68] - -(55) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66, sum#68] -Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#52, c_login#59, c_email_address#60, d_year#66] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#63 - ws_ext_discount_amt#62)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#63 - ws_ext_discount_amt#62)))#69] -Results [2]: [c_customer_id#55 AS customer_id#70, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#63 - ws_ext_discount_amt#62)))#69,18,2) AS year_total#71] - -(56) Filter [codegen id : 11] -Input [2]: [customer_id#70, year_total#71] -Condition : (isnotnull(year_total#71) AND (year_total#71 > 0.00)) - -(57) BroadcastExchange -Input [2]: [customer_id#70, year_total#71] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(58) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#70] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 16] -Output [5]: [customer_id#25, year_total#26, customer_preferred_cust_flag#45, year_total#46, year_total#71] -Input [6]: [customer_id#25, year_total#26, customer_preferred_cust_flag#45, year_total#46, customer_id#70, year_total#71] - -(60) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(61) CometFilter -Input [8]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79] -Condition : (isnotnull(c_customer_sk#72) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#73, 16, true, false, true))) - -(62) CometProject -Input [8]: [c_customer_sk#72, c_customer_id#73, c_first_name#74, c_last_name#75, c_preferred_cust_flag#76, c_birth_country#77, c_login#78, c_email_address#79] -Arguments: [c_customer_sk#72, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60], [c_customer_sk#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#73, 16, true, false, true) AS c_customer_id#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#74, 20, true, false, true) AS c_first_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#75, 30, true, false, true) AS c_last_name#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#76, 1, true, false, true) AS c_preferred_cust_flag#58, c_birth_country#77, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#78, 13, true, false, true) AS c_login#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#79, 50, true, false, true) AS c_email_address#60] - -(63) CometColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#72, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60] - -(64) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#83), dynamicpruningexpression(ws_sold_date_sk#83 IN dynamicpruning#39)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(65) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] - -(66) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] -Condition : isnotnull(ws_bill_customer_sk#80) - -(67) BroadcastExchange -Input [4]: [ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(68) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#72] -Right keys [1]: [ws_bill_customer_sk#80] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 14] -Output [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] -Input [12]: [c_customer_sk#72, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, ws_bill_customer_sk#80, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83] - -(70) ReusedExchange [Reuses operator id: 88] -Output [2]: [d_date_sk#84, d_year#85] - -(71) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#83] -Right keys [1]: [d_date_sk#84] -Join type: Inner -Join condition: None - -(72) Project [codegen id : 14] -Output [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, ws_ext_discount_amt#81, ws_ext_list_price#82, d_year#85] -Input [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, ws_ext_discount_amt#81, ws_ext_list_price#82, ws_sold_date_sk#83, d_date_sk#84, d_year#85] - -(73) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, ws_ext_discount_amt#81, ws_ext_list_price#82, d_year#85] -Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#82 - ws_ext_discount_amt#81)))] -Aggregate Attributes [1]: [sum#86] -Results [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85, sum#87] - -(74) CometColumnarExchange -Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85, sum#87] -Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(75) CometColumnarToRow [codegen id : 15] -Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85, sum#87] - -(76) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85, sum#87] -Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#77, c_login#59, c_email_address#60, d_year#85] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#82 - ws_ext_discount_amt#81)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#82 - ws_ext_discount_amt#81)))#69] -Results [2]: [c_customer_id#55 AS customer_id#88, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#82 - ws_ext_discount_amt#81)))#69,18,2) AS year_total#89] - -(77) BroadcastExchange -Input [2]: [customer_id#88, year_total#89] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] - -(78) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#88] -Join type: Inner -Join condition: (CASE WHEN (year_total#71 > 0.00) THEN (year_total#89 / year_total#71) END > CASE WHEN (year_total#26 > 0.00) THEN (year_total#46 / year_total#26) END) - -(79) Project [codegen id : 16] -Output [1]: [customer_preferred_cust_flag#45] -Input [7]: [customer_id#25, year_total#26, customer_preferred_cust_flag#45, year_total#46, year_total#71, customer_id#88, year_total#89] - -(80) TakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#45] -Arguments: 100, [customer_preferred_cust_flag#45 ASC NULLS FIRST], [customer_preferred_cust_flag#45] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (84) -+- * CometColumnarToRow (83) - +- CometFilter (82) - +- CometNativeScan parquet spark_catalog.default.date_dim (81) - - -(81) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(82) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(83) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#20, d_year#21] - -(84) BroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 -BroadcastExchange (88) -+- * CometColumnarToRow (87) - +- CometFilter (86) - +- CometNativeScan parquet spark_catalog.default.date_dim (85) - - -(85) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#40, d_year#41] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(86) CometFilter -Input [2]: [d_date_sk#40, d_year#41] -Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#40)) - -(87) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#40, d_year#41] - -(88) BroadcastExchange -Input [2]: [d_date_sk#40, d_year#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] - -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#64 IN dynamicpruning#19 - -Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#83 IN dynamicpruning#39 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/simplified.txt deleted file mode 100644 index 6c9e276c01..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/simplified.txt +++ /dev/null @@ -1,131 +0,0 @@ -TakeOrderedAndProject [customer_preferred_cust_flag] - WholeStageCodegen (16) - Project [customer_preferred_cust_flag] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - Project [customer_id,year_total,customer_preferred_cust_flag,year_total] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_preferred_cust_flag,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - WholeStageCodegen (6) - HashAggregate [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] [sum,sum] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) - HashAggregate [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] [sum,sum] - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) - HashAggregate [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] [sum,sum] - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/explain.txt deleted file mode 100644 index 091ad62567..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,475 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (73) -+- CometTakeOrderedAndProject (72) - +- CometProject (71) - +- CometBroadcastHashJoin (70) - :- CometProject (54) - : +- CometBroadcastHashJoin (53) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (52) - : +- CometFilter (51) - : +- CometHashAggregate (50) - : +- CometExchange (49) - : +- CometHashAggregate (48) - : +- CometProject (47) - : +- CometBroadcastHashJoin (46) - : :- CometProject (44) - : : +- CometBroadcastHashJoin (43) - : : :- CometProject (39) - : : : +- CometFilter (38) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) - : : +- CometBroadcastExchange (42) - : : +- CometFilter (41) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) - : +- ReusedExchange (45) - +- CometBroadcastExchange (69) - +- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometProject (65) - +- CometBroadcastHashJoin (64) - :- CometProject (62) - : +- CometBroadcastHashJoin (61) - : :- CometProject (57) - : : +- CometFilter (56) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (55) - : +- CometBroadcastExchange (60) - : +- CometFilter (59) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (58) - +- ReusedExchange (63) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#15) - -(6) CometBroadcastExchange -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(7) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_sk#1], [ss_customer_sk#15], Inner, BuildRight - -(8) CometProject -Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: [d_date_sk#20, d_year#21] - -(12) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Right output [2]: [d_date_sk#20, d_year#21] -Arguments: [ss_sold_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(13) CometProject -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] - -(14) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(15) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(17) CometFilter -Input [2]: [customer_id#23, year_total#24] -Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true))) - -(20) CometProject -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Arguments: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14], [c_customer_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#29, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#31, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#32, 50, true, false, true) AS c_email_address#14] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#36), dynamicpruningexpression(ss_sold_date_sk#36 IN dynamicpruning#37)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Condition : isnotnull(ss_customer_sk#33) - -(23) CometBroadcastExchange -Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] - -(24) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [c_customer_sk#25], [ss_customer_sk#33], Inner, BuildRight - -(25) CometProject -Input [12]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: [d_date_sk#38, d_year#39] - -(29) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Right output [2]: [d_date_sk#38, d_year#39] -Arguments: [ss_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight - -(30) CometProject -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36, d_date_sk#38, d_year#39] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] - -(31) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] - -(32) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] - -(34) CometBroadcastExchange -Input [3]: [customer_id#41, customer_preferred_cust_flag#42, year_total#43] -Arguments: [customer_id#41, customer_preferred_cust_flag#42, year_total#43] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#23, year_total#24] -Right output [3]: [customer_id#41, customer_preferred_cust_flag#42, year_total#43] -Arguments: [customer_id#23], [customer_id#41], Inner, BuildRight - -(36) CometProject -Input [5]: [customer_id#23, year_total#24, customer_id#41, customer_preferred_cust_flag#42, year_total#43] -Arguments: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43], [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47, c_preferred_cust_flag#48, c_birth_country#49, c_login#50, c_email_address#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(38) CometFilter -Input [8]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47, c_preferred_cust_flag#48, c_birth_country#49, c_login#50, c_email_address#51] -Condition : (isnotnull(c_customer_sk#44) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#45, 16, true, false, true))) - -(39) CometProject -Input [8]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47, c_preferred_cust_flag#48, c_birth_country#49, c_login#50, c_email_address#51] -Arguments: [c_customer_sk#44, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57], [c_customer_sk#44, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#45, 16, true, false, true) AS c_customer_id#52, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#46, 20, true, false, true) AS c_first_name#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#47, 30, true, false, true) AS c_last_name#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#48, 1, true, false, true) AS c_preferred_cust_flag#55, c_birth_country#49, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#50, 13, true, false, true) AS c_login#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#51, 50, true, false, true) AS c_email_address#57] - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#61), dynamicpruningexpression(ws_sold_date_sk#61 IN dynamicpruning#62)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(41) CometFilter -Input [4]: [ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Condition : isnotnull(ws_bill_customer_sk#58) - -(42) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Arguments: [ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] - -(43) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#44, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57] -Right output [4]: [ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Arguments: [c_customer_sk#44], [ws_bill_customer_sk#58], Inner, BuildRight - -(44) CometProject -Input [12]: [c_customer_sk#44, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Arguments: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61], [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] - -(45) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#63, d_year#64] - -(46) CometBroadcastHashJoin -Left output [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Right output [2]: [d_date_sk#63, d_year#64] -Arguments: [ws_sold_date_sk#61], [d_date_sk#63], Inner, BuildRight - -(47) CometProject -Input [12]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61, d_date_sk#63, d_year#64] -Arguments: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, d_year#64], [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, d_year#64] - -(48) CometHashAggregate -Input [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, d_year#64] -Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, d_year#64] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#60 - ws_ext_discount_amt#59)))] - -(49) CometExchange -Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, d_year#64, sum#65] -Arguments: hashpartitioning(c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, d_year#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(50) CometHashAggregate -Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, d_year#64, sum#65] -Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, d_year#64] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#60 - ws_ext_discount_amt#59)))] - -(51) CometFilter -Input [2]: [customer_id#66, year_total#67] -Condition : (isnotnull(year_total#67) AND (year_total#67 > 0.00)) - -(52) CometBroadcastExchange -Input [2]: [customer_id#66, year_total#67] -Arguments: [customer_id#66, year_total#67] - -(53) CometBroadcastHashJoin -Left output [4]: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43] -Right output [2]: [customer_id#66, year_total#67] -Arguments: [customer_id#23], [customer_id#66], Inner, BuildRight - -(54) CometProject -Input [6]: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43, customer_id#66, year_total#67] -Arguments: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43, year_total#67], [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43, year_total#67] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#68, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(56) CometFilter -Input [8]: [c_customer_sk#68, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75] -Condition : (isnotnull(c_customer_sk#68) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#69, 16, true, false, true))) - -(57) CometProject -Input [8]: [c_customer_sk#68, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75] -Arguments: [c_customer_sk#68, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57], [c_customer_sk#68, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#69, 16, true, false, true) AS c_customer_id#52, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#70, 20, true, false, true) AS c_first_name#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#71, 30, true, false, true) AS c_last_name#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#72, 1, true, false, true) AS c_preferred_cust_flag#55, c_birth_country#73, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#74, 13, true, false, true) AS c_login#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#75, 50, true, false, true) AS c_email_address#57] - -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#80)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(59) CometFilter -Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Condition : isnotnull(ws_bill_customer_sk#76) - -(60) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Arguments: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] - -(61) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#68, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57] -Right output [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Arguments: [c_customer_sk#68], [ws_bill_customer_sk#76], Inner, BuildRight - -(62) CometProject -Input [12]: [c_customer_sk#68, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Arguments: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79], [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] - -(63) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#81, d_year#82] - -(64) CometBroadcastHashJoin -Left output [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Right output [2]: [d_date_sk#81, d_year#82] -Arguments: [ws_sold_date_sk#79], [d_date_sk#81], Inner, BuildRight - -(65) CometProject -Input [12]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79, d_date_sk#81, d_year#82] -Arguments: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#82], [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#82] - -(66) CometHashAggregate -Input [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#82] -Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, d_year#82] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#78 - ws_ext_discount_amt#77)))] - -(67) CometExchange -Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, d_year#82, sum#83] -Arguments: hashpartitioning(c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, d_year#82, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(68) CometHashAggregate -Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, d_year#82, sum#83] -Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, d_year#82] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#78 - ws_ext_discount_amt#77)))] - -(69) CometBroadcastExchange -Input [2]: [customer_id#84, year_total#85] -Arguments: [customer_id#84, year_total#85] - -(70) CometBroadcastHashJoin -Left output [5]: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43, year_total#67] -Right output [2]: [customer_id#84, year_total#85] -Arguments: [customer_id#23], [customer_id#84], Inner, (CASE WHEN (year_total#67 > 0.00) THEN (year_total#85 / year_total#67) END > CASE WHEN (year_total#24 > 0.00) THEN (year_total#43 / year_total#24) END), BuildRight - -(71) CometProject -Input [7]: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43, year_total#67, customer_id#84, year_total#85] -Arguments: [customer_preferred_cust_flag#42], [customer_preferred_cust_flag#42] - -(72) CometTakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#42] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_preferred_cust_flag#42 ASC NULLS FIRST], output=[customer_preferred_cust_flag#42]), [customer_preferred_cust_flag#42], 100, 0, [customer_preferred_cust_flag#42 ASC NULLS FIRST], [customer_preferred_cust_flag#42] - -(73) CometColumnarToRow [codegen id : 1] -Input [1]: [customer_preferred_cust_flag#42] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (77) -+- * CometColumnarToRow (76) - +- CometFilter (75) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (74) - - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(75) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(76) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#20, d_year#21] - -(77) BroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#36 IN dynamicpruning#37 -BroadcastExchange (81) -+- * CometColumnarToRow (80) - +- CometFilter (79) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (78) - - -(78) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(79) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) - -(80) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#38, d_year#39] - -(81) BroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#61 IN dynamicpruning#19 - -Subquery:4 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#37 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/simplified.txt deleted file mode 100644 index 4599ecf539..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,91 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_preferred_cust_flag] - CometProject [customer_preferred_cust_flag] - CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_preferred_cust_flag,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_preferred_cust_flag,year_total] #5 - CometHashAggregate [c_first_name,c_last_name,d_year,c_birth_country,c_login,c_email_address,sum] [customer_id,customer_preferred_cust_flag,year_total,c_customer_id,c_preferred_cust_flag,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt deleted file mode 100644 index 091ad62567..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt +++ /dev/null @@ -1,475 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (73) -+- CometTakeOrderedAndProject (72) - +- CometProject (71) - +- CometBroadcastHashJoin (70) - :- CometProject (54) - : +- CometBroadcastHashJoin (53) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (52) - : +- CometFilter (51) - : +- CometHashAggregate (50) - : +- CometExchange (49) - : +- CometHashAggregate (48) - : +- CometProject (47) - : +- CometBroadcastHashJoin (46) - : :- CometProject (44) - : : +- CometBroadcastHashJoin (43) - : : :- CometProject (39) - : : : +- CometFilter (38) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) - : : +- CometBroadcastExchange (42) - : : +- CometFilter (41) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) - : +- ReusedExchange (45) - +- CometBroadcastExchange (69) - +- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometProject (65) - +- CometBroadcastHashJoin (64) - :- CometProject (62) - : +- CometBroadcastHashJoin (61) - : :- CometProject (57) - : : +- CometFilter (56) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (55) - : +- CometBroadcastExchange (60) - : +- CometFilter (59) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (58) - +- ReusedExchange (63) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#15) - -(6) CometBroadcastExchange -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(7) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_sk#1], [ss_customer_sk#15], Inner, BuildRight - -(8) CometProject -Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: [d_date_sk#20, d_year#21] - -(12) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Right output [2]: [d_date_sk#20, d_year#21] -Arguments: [ss_sold_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(13) CometProject -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] - -(14) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(15) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(17) CometFilter -Input [2]: [customer_id#23, year_total#24] -Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true))) - -(20) CometProject -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Arguments: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14], [c_customer_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#29, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#31, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#32, 50, true, false, true) AS c_email_address#14] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#36), dynamicpruningexpression(ss_sold_date_sk#36 IN dynamicpruning#37)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Condition : isnotnull(ss_customer_sk#33) - -(23) CometBroadcastExchange -Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] - -(24) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [c_customer_sk#25], [ss_customer_sk#33], Inner, BuildRight - -(25) CometProject -Input [12]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: [d_date_sk#38, d_year#39] - -(29) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Right output [2]: [d_date_sk#38, d_year#39] -Arguments: [ss_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight - -(30) CometProject -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36, d_date_sk#38, d_year#39] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] - -(31) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] - -(32) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] - -(34) CometBroadcastExchange -Input [3]: [customer_id#41, customer_preferred_cust_flag#42, year_total#43] -Arguments: [customer_id#41, customer_preferred_cust_flag#42, year_total#43] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#23, year_total#24] -Right output [3]: [customer_id#41, customer_preferred_cust_flag#42, year_total#43] -Arguments: [customer_id#23], [customer_id#41], Inner, BuildRight - -(36) CometProject -Input [5]: [customer_id#23, year_total#24, customer_id#41, customer_preferred_cust_flag#42, year_total#43] -Arguments: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43], [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47, c_preferred_cust_flag#48, c_birth_country#49, c_login#50, c_email_address#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(38) CometFilter -Input [8]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47, c_preferred_cust_flag#48, c_birth_country#49, c_login#50, c_email_address#51] -Condition : (isnotnull(c_customer_sk#44) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#45, 16, true, false, true))) - -(39) CometProject -Input [8]: [c_customer_sk#44, c_customer_id#45, c_first_name#46, c_last_name#47, c_preferred_cust_flag#48, c_birth_country#49, c_login#50, c_email_address#51] -Arguments: [c_customer_sk#44, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57], [c_customer_sk#44, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#45, 16, true, false, true) AS c_customer_id#52, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#46, 20, true, false, true) AS c_first_name#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#47, 30, true, false, true) AS c_last_name#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#48, 1, true, false, true) AS c_preferred_cust_flag#55, c_birth_country#49, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#50, 13, true, false, true) AS c_login#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#51, 50, true, false, true) AS c_email_address#57] - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#61), dynamicpruningexpression(ws_sold_date_sk#61 IN dynamicpruning#62)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(41) CometFilter -Input [4]: [ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Condition : isnotnull(ws_bill_customer_sk#58) - -(42) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Arguments: [ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] - -(43) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#44, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57] -Right output [4]: [ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Arguments: [c_customer_sk#44], [ws_bill_customer_sk#58], Inner, BuildRight - -(44) CometProject -Input [12]: [c_customer_sk#44, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_bill_customer_sk#58, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Arguments: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61], [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] - -(45) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#63, d_year#64] - -(46) CometBroadcastHashJoin -Left output [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61] -Right output [2]: [d_date_sk#63, d_year#64] -Arguments: [ws_sold_date_sk#61], [d_date_sk#63], Inner, BuildRight - -(47) CometProject -Input [12]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, ws_sold_date_sk#61, d_date_sk#63, d_year#64] -Arguments: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, d_year#64], [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, d_year#64] - -(48) CometHashAggregate -Input [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, ws_ext_discount_amt#59, ws_ext_list_price#60, d_year#64] -Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, d_year#64] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#60 - ws_ext_discount_amt#59)))] - -(49) CometExchange -Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, d_year#64, sum#65] -Arguments: hashpartitioning(c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, d_year#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(50) CometHashAggregate -Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, d_year#64, sum#65] -Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#49, c_login#56, c_email_address#57, d_year#64] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#60 - ws_ext_discount_amt#59)))] - -(51) CometFilter -Input [2]: [customer_id#66, year_total#67] -Condition : (isnotnull(year_total#67) AND (year_total#67 > 0.00)) - -(52) CometBroadcastExchange -Input [2]: [customer_id#66, year_total#67] -Arguments: [customer_id#66, year_total#67] - -(53) CometBroadcastHashJoin -Left output [4]: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43] -Right output [2]: [customer_id#66, year_total#67] -Arguments: [customer_id#23], [customer_id#66], Inner, BuildRight - -(54) CometProject -Input [6]: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43, customer_id#66, year_total#67] -Arguments: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43, year_total#67], [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43, year_total#67] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#68, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(56) CometFilter -Input [8]: [c_customer_sk#68, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75] -Condition : (isnotnull(c_customer_sk#68) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#69, 16, true, false, true))) - -(57) CometProject -Input [8]: [c_customer_sk#68, c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75] -Arguments: [c_customer_sk#68, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57], [c_customer_sk#68, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#69, 16, true, false, true) AS c_customer_id#52, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#70, 20, true, false, true) AS c_first_name#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#71, 30, true, false, true) AS c_last_name#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#72, 1, true, false, true) AS c_preferred_cust_flag#55, c_birth_country#73, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#74, 13, true, false, true) AS c_login#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#75, 50, true, false, true) AS c_email_address#57] - -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#80)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(59) CometFilter -Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Condition : isnotnull(ws_bill_customer_sk#76) - -(60) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Arguments: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] - -(61) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#68, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57] -Right output [4]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Arguments: [c_customer_sk#68], [ws_bill_customer_sk#76], Inner, BuildRight - -(62) CometProject -Input [12]: [c_customer_sk#68, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Arguments: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79], [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] - -(63) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#81, d_year#82] - -(64) CometBroadcastHashJoin -Left output [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Right output [2]: [d_date_sk#81, d_year#82] -Arguments: [ws_sold_date_sk#79], [d_date_sk#81], Inner, BuildRight - -(65) CometProject -Input [12]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, ws_sold_date_sk#79, d_date_sk#81, d_year#82] -Arguments: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#82], [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#82] - -(66) CometHashAggregate -Input [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, ws_ext_discount_amt#77, ws_ext_list_price#78, d_year#82] -Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, d_year#82] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#78 - ws_ext_discount_amt#77)))] - -(67) CometExchange -Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, d_year#82, sum#83] -Arguments: hashpartitioning(c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, d_year#82, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(68) CometHashAggregate -Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, d_year#82, sum#83] -Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#73, c_login#56, c_email_address#57, d_year#82] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#78 - ws_ext_discount_amt#77)))] - -(69) CometBroadcastExchange -Input [2]: [customer_id#84, year_total#85] -Arguments: [customer_id#84, year_total#85] - -(70) CometBroadcastHashJoin -Left output [5]: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43, year_total#67] -Right output [2]: [customer_id#84, year_total#85] -Arguments: [customer_id#23], [customer_id#84], Inner, (CASE WHEN (year_total#67 > 0.00) THEN (year_total#85 / year_total#67) END > CASE WHEN (year_total#24 > 0.00) THEN (year_total#43 / year_total#24) END), BuildRight - -(71) CometProject -Input [7]: [customer_id#23, year_total#24, customer_preferred_cust_flag#42, year_total#43, year_total#67, customer_id#84, year_total#85] -Arguments: [customer_preferred_cust_flag#42], [customer_preferred_cust_flag#42] - -(72) CometTakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#42] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_preferred_cust_flag#42 ASC NULLS FIRST], output=[customer_preferred_cust_flag#42]), [customer_preferred_cust_flag#42], 100, 0, [customer_preferred_cust_flag#42 ASC NULLS FIRST], [customer_preferred_cust_flag#42] - -(73) CometColumnarToRow [codegen id : 1] -Input [1]: [customer_preferred_cust_flag#42] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (77) -+- * CometColumnarToRow (76) - +- CometFilter (75) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (74) - - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(75) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(76) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#20, d_year#21] - -(77) BroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#36 IN dynamicpruning#37 -BroadcastExchange (81) -+- * CometColumnarToRow (80) - +- CometFilter (79) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (78) - - -(78) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(79) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) - -(80) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#38, d_year#39] - -(81) BroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#61 IN dynamicpruning#19 - -Subquery:4 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#37 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/extended.txt deleted file mode 100644 index 781eae9054..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/extended.txt +++ /dev/null @@ -1,91 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 80 out of 86 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 4599ecf539..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt +++ /dev/null @@ -1,91 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_preferred_cust_flag] - CometProject [customer_preferred_cust_flag] - CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_preferred_cust_flag,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_preferred_cust_flag,year_total] #5 - CometHashAggregate [c_first_name,c_last_name,d_year,c_birth_country,c_login,c_email_address,sum] [customer_id,customer_preferred_cust_flag,year_total,c_customer_id,c_preferred_cust_flag,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/explain.txt deleted file mode 100644 index 19d16c378b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/explain.txt +++ /dev/null @@ -1,163 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.web_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.item (4) - +- ReusedExchange (11) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(7) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(8) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(11) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] - -(14) HashAggregate [codegen id : 3] -Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(15) CometColumnarExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(17) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#17] -Results [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#17,17,2) AS _w0#19, i_item_id#11] - -(18) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(22) Project [codegen id : 6] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] - -(23) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometNativeScan parquet spark_catalog.default.date_dim (24) - - -(24) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#22] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/simplified.txt deleted file mode 100644 index b7b0a89774..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/explain.txt deleted file mode 100644 index 93308784a9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ws_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/simplified.txt deleted file mode 100644 index fb83fd2f9a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt deleted file mode 100644 index 93308784a9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ws_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/extended.txt deleted file mode 100644 index 3f41c97ff5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index fb83fd2f9a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/explain.txt deleted file mode 100644 index dbacf525ef..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/explain.txt +++ /dev/null @@ -1,241 +0,0 @@ -== Physical Plan == -* HashAggregate (36) -+- * CometColumnarToRow (35) - +- CometColumnarExchange (34) - +- * HashAggregate (33) - +- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * CometColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.store (4) - : : : +- BroadcastExchange (14) - : : : +- * CometColumnarToRow (13) - : : : +- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (10) - : : +- ReusedExchange (17) - : +- BroadcastExchange (24) - : +- * CometColumnarToRow (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometNativeScan parquet spark_catalog.default.customer_demographics (20) - +- BroadcastExchange (30) - +- * CometColumnarToRow (29) - +- CometFilter (28) - +- CometNativeScan parquet spark_catalog.default.household_demographics (27) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00))), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00)))] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] - -(3) Filter [codegen id : 6] -Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Condition : (((((isnotnull(ss_store_sk#4) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_hdemo_sk#2)) AND ((((ss_net_profit#9 >= 100.00) AND (ss_net_profit#9 <= 200.00)) OR ((ss_net_profit#9 >= 150.00) AND (ss_net_profit#9 <= 300.00))) OR ((ss_net_profit#9 >= 50.00) AND (ss_net_profit#9 <= 250.00)))) AND ((((ss_sales_price#6 >= 100.00) AND (ss_sales_price#6 <= 150.00)) OR ((ss_sales_price#6 >= 50.00) AND (ss_sales_price#6 <= 100.00))) OR ((ss_sales_price#6 >= 150.00) AND (ss_sales_price#6 <= 200.00)))) - -(4) CometNativeScan parquet spark_catalog.default.store -Output [1]: [s_store_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [1]: [s_store_sk#12] -Condition : isnotnull(s_store_sk#12) - -(6) CometColumnarToRow [codegen id : 1] -Input [1]: [s_store_sk#12] - -(7) BroadcastExchange -Input [1]: [s_store_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#12] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 6] -Output [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, s_store_sk#12] - -(10) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Condition : (((isnotnull(ca_country#15) AND (ca_country#15 = United States)) AND isnotnull(ca_address_sk#13)) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) IN (TX,OH) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) IN (OR,NM,KY)) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) IN (VA,TX,MS))) - -(12) CometProject -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Arguments: [ca_address_sk#13, ca_state#16], [ca_address_sk#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) AS ca_state#16] - -(13) CometColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#13, ca_state#16] - -(14) BroadcastExchange -Input [2]: [ca_address_sk#13, ca_state#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#13] -Join type: Inner -Join condition: ((((ca_state#16 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#16 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#16 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))) - -(16) Project [codegen id : 6] -Output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] -Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#13, ca_state#16] - -(17) ReusedExchange [Reuses operator id: 41] -Output [1]: [d_date_sk#17] - -(18) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#10] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 6] -Output [6]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10, d_date_sk#17] - -(20) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] -Condition : (isnotnull(cd_demo_sk#18) AND ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#20, 20, true, false, true) = Advanced Degree )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = S) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#20, 20, true, false, true) = College ))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = W) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#20, 20, true, false, true) = 2 yr Degree )))) - -(22) CometProject -Input [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] -Arguments: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22], [cd_demo_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) AS cd_marital_status#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#20, 20, true, false, true) AS cd_education_status#22] - -(23) CometColumnarToRow [codegen id : 4] -Input [3]: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] - -(24) BroadcastExchange -Input [3]: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_cdemo_sk#1] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: ((((((cd_marital_status#21 = M) AND (cd_education_status#22 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#21 = S) AND (cd_education_status#22 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#21 = W) AND (cd_education_status#22 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) - -(26) Project [codegen id : 6] -Output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#21, cd_education_status#22] -Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] - -(27) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#23, hd_dep_count#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), Or(EqualTo(hd_dep_count,3),EqualTo(hd_dep_count,1))] -ReadSchema: struct - -(28) CometFilter -Input [2]: [hd_demo_sk#23, hd_dep_count#24] -Condition : (isnotnull(hd_demo_sk#23) AND ((hd_dep_count#24 = 3) OR (hd_dep_count#24 = 1))) - -(29) CometColumnarToRow [codegen id : 5] -Input [2]: [hd_demo_sk#23, hd_dep_count#24] - -(30) BroadcastExchange -Input [2]: [hd_demo_sk#23, hd_dep_count#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#23] -Join type: Inner -Join condition: (((((((cd_marital_status#21 = M) AND (cd_education_status#22 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#24 = 3)) OR (((((cd_marital_status#21 = S) AND (cd_education_status#22 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#24 = 1))) OR (((((cd_marital_status#21 = W) AND (cd_education_status#22 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#24 = 1))) - -(32) Project [codegen id : 6] -Output [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#21, cd_education_status#22, hd_demo_sk#23, hd_dep_count#24] - -(33) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Keys: [] -Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sales_price#7)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#8)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#8))] -Aggregate Attributes [7]: [sum#25, count#26, sum#27, count#28, sum#29, count#30, sum#31] -Results [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] - -(34) CometColumnarExchange -Input [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(35) CometColumnarToRow [codegen id : 7] -Input [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] - -(36) HashAggregate [codegen id : 7] -Input [7]: [sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38] -Keys: [] -Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] -Aggregate Attributes [4]: [avg(ss_quantity#5)#39, avg(UnscaledValue(ss_ext_sales_price#7))#40, avg(UnscaledValue(ss_ext_wholesale_cost#8))#41, sum(UnscaledValue(ss_ext_wholesale_cost#8))#42] -Results [4]: [avg(ss_quantity#5)#39 AS avg(ss_quantity)#43, cast((avg(UnscaledValue(ss_ext_sales_price#7))#40 / 100.0) as decimal(11,6)) AS avg(ss_ext_sales_price)#44, cast((avg(UnscaledValue(ss_ext_wholesale_cost#8))#41 / 100.0) as decimal(11,6)) AS avg(ss_ext_wholesale_cost)#45, MakeDecimal(sum(UnscaledValue(ss_ext_wholesale_cost#8))#42,17,2) AS sum(ss_ext_wholesale_cost)#46] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (41) -+- * CometColumnarToRow (40) - +- CometProject (39) - +- CometFilter (38) - +- CometNativeScan parquet spark_catalog.default.date_dim (37) - - -(37) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_year#47] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(38) CometFilter -Input [2]: [d_date_sk#17, d_year#47] -Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2001)) AND isnotnull(d_date_sk#17)) - -(39) CometProject -Input [2]: [d_date_sk#17, d_year#47] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(40) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] - -(41) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/simplified.txt deleted file mode 100644 index a33ae5a161..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (7) - HashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost)),avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (6) - HashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum,sum,count,sum,count,sum,count,sum] - Project [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] - Project [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] - Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] - Project [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] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometFilter [hd_demo_sk,hd_dep_count] - CometNativeScan 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/q13.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/explain.txt deleted file mode 100644 index 74da8ba883..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,231 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (34) -+- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (7) - : : : : +- CometBroadcastHashJoin (6) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (20) - +- CometBroadcastExchange (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00))), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00)))] -ReadSchema: struct - -(2) CometFilter -Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Condition : (((((isnotnull(ss_store_sk#4) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_hdemo_sk#2)) AND ((((ss_net_profit#9 >= 100.00) AND (ss_net_profit#9 <= 200.00)) OR ((ss_net_profit#9 >= 150.00) AND (ss_net_profit#9 <= 300.00))) OR ((ss_net_profit#9 >= 50.00) AND (ss_net_profit#9 <= 250.00)))) AND ((((ss_sales_price#6 >= 100.00) AND (ss_sales_price#6 <= 150.00)) OR ((ss_sales_price#6 >= 50.00) AND (ss_sales_price#6 <= 100.00))) OR ((ss_sales_price#6 >= 150.00) AND (ss_sales_price#6 <= 200.00)))) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [s_store_sk#12] -Condition : isnotnull(s_store_sk#12) - -(5) CometBroadcastExchange -Input [1]: [s_store_sk#12] -Arguments: [s_store_sk#12] - -(6) CometBroadcastHashJoin -Left output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Right output [1]: [s_store_sk#12] -Arguments: [ss_store_sk#4], [s_store_sk#12], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, s_store_sk#12] -Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Condition : (((isnotnull(ca_country#15) AND (ca_country#15 = United States)) AND isnotnull(ca_address_sk#13)) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) IN (TX,OH) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) IN (OR,NM,KY)) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) IN (VA,TX,MS))) - -(10) CometProject -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Arguments: [ca_address_sk#13, ca_state#16], [ca_address_sk#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) AS ca_state#16] - -(11) CometBroadcastExchange -Input [2]: [ca_address_sk#13, ca_state#16] -Arguments: [ca_address_sk#13, ca_state#16] - -(12) CometBroadcastHashJoin -Left output [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Right output [2]: [ca_address_sk#13, ca_state#16] -Arguments: [ss_addr_sk#3], [ca_address_sk#13], Inner, ((((ca_state#16 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#16 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#16 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))), BuildRight - -(13) CometProject -Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#13, ca_state#16] -Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_year#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#17, d_year#18] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(16) CometProject -Input [2]: [d_date_sk#17, d_year#18] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: [d_date_sk#17] - -(18) CometBroadcastHashJoin -Left output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] -Right output [1]: [d_date_sk#17] -Arguments: [ss_sold_date_sk#10], [d_date_sk#17], Inner, BuildRight - -(19) CometProject -Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10, d_date_sk#17] -Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Condition : (isnotnull(cd_demo_sk#19) AND ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = Advanced Degree )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = S) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = College ))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = W) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = 2 yr Degree )))) - -(22) CometProject -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) AS cd_marital_status#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) AS cd_education_status#23] - -(23) CometBroadcastExchange -Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] - -(24) CometBroadcastHashJoin -Left output [6]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Right output [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [ss_cdemo_sk#1], [cd_demo_sk#19], Inner, ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))), BuildRight - -(25) CometProject -Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23], [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#24, hd_dep_count#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), Or(EqualTo(hd_dep_count,3),EqualTo(hd_dep_count,1))] -ReadSchema: struct - -(27) CometFilter -Input [2]: [hd_demo_sk#24, hd_dep_count#25] -Condition : (isnotnull(hd_demo_sk#24) AND ((hd_dep_count#25 = 3) OR (hd_dep_count#25 = 1))) - -(28) CometBroadcastExchange -Input [2]: [hd_demo_sk#24, hd_dep_count#25] -Arguments: [hd_demo_sk#24, hd_dep_count#25] - -(29) CometBroadcastHashJoin -Left output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23] -Right output [2]: [hd_demo_sk#24, hd_dep_count#25] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#24], Inner, (((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#25 = 3)) OR (((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#25 = 1))) OR (((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#25 = 1))), BuildRight - -(30) CometProject -Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23, hd_demo_sk#24, hd_dep_count#25] -Arguments: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] - -(31) CometHashAggregate -Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Keys: [] -Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sales_price#7)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#8)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#8))] - -(32) CometExchange -Input [7]: [sum#26, count#27, sum#28, count#29, sum#30, count#31, sum#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(33) CometHashAggregate -Input [7]: [sum#26, count#27, sum#28, count#29, sum#30, count#31, sum#32] -Keys: [] -Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] - -(34) CometColumnarToRow [codegen id : 1] -Input [4]: [avg(ss_quantity)#33, avg(ss_ext_sales_price)#34, avg(ss_ext_wholesale_cost)#35, sum(ss_ext_wholesale_cost)#36] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (39) -+- * CometColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (35) - - -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_year#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(36) CometFilter -Input [2]: [d_date_sk#17, d_year#18] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(37) CometProject -Input [2]: [d_date_sk#17, d_year#18] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(38) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] - -(39) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/simplified.txt deleted file mode 100644 index 8ef882a435..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,44 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost))] - CometExchange #1 - CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum] - CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - 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,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_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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk] #3 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [hd_demo_sk,hd_dep_count] #7 - CometFilter [hd_demo_sk,hd_dep_count] - CometScan [native_iceberg_compat] 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/q13/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt deleted file mode 100644 index 74da8ba883..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt +++ /dev/null @@ -1,231 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (34) -+- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (7) - : : : : +- CometBroadcastHashJoin (6) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (20) - +- CometBroadcastExchange (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_hdemo_sk), Or(Or(And(GreaterThanOrEqual(ss_net_profit,100.00),LessThanOrEqual(ss_net_profit,200.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,300.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,250.00))), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00)))] -ReadSchema: struct - -(2) CometFilter -Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Condition : (((((isnotnull(ss_store_sk#4) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_hdemo_sk#2)) AND ((((ss_net_profit#9 >= 100.00) AND (ss_net_profit#9 <= 200.00)) OR ((ss_net_profit#9 >= 150.00) AND (ss_net_profit#9 <= 300.00))) OR ((ss_net_profit#9 >= 50.00) AND (ss_net_profit#9 <= 250.00)))) AND ((((ss_sales_price#6 >= 100.00) AND (ss_sales_price#6 <= 150.00)) OR ((ss_sales_price#6 >= 50.00) AND (ss_sales_price#6 <= 100.00))) OR ((ss_sales_price#6 >= 150.00) AND (ss_sales_price#6 <= 200.00)))) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [s_store_sk#12] -Condition : isnotnull(s_store_sk#12) - -(5) CometBroadcastExchange -Input [1]: [s_store_sk#12] -Arguments: [s_store_sk#12] - -(6) CometBroadcastHashJoin -Left output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Right output [1]: [s_store_sk#12] -Arguments: [ss_store_sk#4], [s_store_sk#12], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, s_store_sk#12] -Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Condition : (((isnotnull(ca_country#15) AND (ca_country#15 = United States)) AND isnotnull(ca_address_sk#13)) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) IN (TX,OH) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) IN (OR,NM,KY)) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) IN (VA,TX,MS))) - -(10) CometProject -Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] -Arguments: [ca_address_sk#13, ca_state#16], [ca_address_sk#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#14, 2, true, false, true) AS ca_state#16] - -(11) CometBroadcastExchange -Input [2]: [ca_address_sk#13, ca_state#16] -Arguments: [ca_address_sk#13, ca_state#16] - -(12) CometBroadcastHashJoin -Left output [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] -Right output [2]: [ca_address_sk#13, ca_state#16] -Arguments: [ss_addr_sk#3], [ca_address_sk#13], Inner, ((((ca_state#16 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#16 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#16 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))), BuildRight - -(13) CometProject -Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#13, ca_state#16] -Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_year#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#17, d_year#18] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(16) CometProject -Input [2]: [d_date_sk#17, d_year#18] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: [d_date_sk#17] - -(18) CometBroadcastHashJoin -Left output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] -Right output [1]: [d_date_sk#17] -Arguments: [ss_sold_date_sk#10], [d_date_sk#17], Inner, BuildRight - -(19) CometProject -Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10, d_date_sk#17] -Arguments: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Condition : (isnotnull(cd_demo_sk#19) AND ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = Advanced Degree )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = S) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = College ))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = W) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = 2 yr Degree )))) - -(22) CometProject -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) AS cd_marital_status#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) AS cd_education_status#23] - -(23) CometBroadcastExchange -Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] - -(24) CometBroadcastHashJoin -Left output [6]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Right output [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [ss_cdemo_sk#1], [cd_demo_sk#19], Inner, ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))), BuildRight - -(25) CometProject -Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23], [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#24, hd_dep_count#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), Or(EqualTo(hd_dep_count,3),EqualTo(hd_dep_count,1))] -ReadSchema: struct - -(27) CometFilter -Input [2]: [hd_demo_sk#24, hd_dep_count#25] -Condition : (isnotnull(hd_demo_sk#24) AND ((hd_dep_count#25 = 3) OR (hd_dep_count#25 = 1))) - -(28) CometBroadcastExchange -Input [2]: [hd_demo_sk#24, hd_dep_count#25] -Arguments: [hd_demo_sk#24, hd_dep_count#25] - -(29) CometBroadcastHashJoin -Left output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23] -Right output [2]: [hd_demo_sk#24, hd_dep_count#25] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#24], Inner, (((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#25 = 3)) OR (((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#25 = 1))) OR (((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#25 = 1))), BuildRight - -(30) CometProject -Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23, hd_demo_sk#24, hd_dep_count#25] -Arguments: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8], [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] - -(31) CometHashAggregate -Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] -Keys: [] -Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sales_price#7)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#8)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#8))] - -(32) CometExchange -Input [7]: [sum#26, count#27, sum#28, count#29, sum#30, count#31, sum#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(33) CometHashAggregate -Input [7]: [sum#26, count#27, sum#28, count#29, sum#30, count#31, sum#32] -Keys: [] -Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] - -(34) CometColumnarToRow [codegen id : 1] -Input [4]: [avg(ss_quantity)#33, avg(ss_ext_sales_price)#34, avg(ss_ext_wholesale_cost)#35, sum(ss_ext_wholesale_cost)#36] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (39) -+- * CometColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (35) - - -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#17, d_year#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(36) CometFilter -Input [2]: [d_date_sk#17, d_year#18] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(37) CometProject -Input [2]: [d_date_sk#17, d_year#18] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(38) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] - -(39) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/extended.txt deleted file mode 100644 index 08e9beb692..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - -Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 8ef882a435..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/simplified.txt +++ /dev/null @@ -1,44 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost))] - CometExchange #1 - CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum] - CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - 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,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_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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk] #3 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [hd_demo_sk,hd_dep_count] #7 - CometFilter [hd_demo_sk,hd_dep_count] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/explain.txt deleted file mode 100644 index 8abc7aabfb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/explain.txt +++ /dev/null @@ -1,822 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (110) -+- * HashAggregate (109) - +- * CometColumnarToRow (108) - +- CometColumnarExchange (107) - +- * HashAggregate (106) - +- * Expand (105) - +- Union (104) - :- * Project (69) - : +- * Filter (68) - : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) - : +- CometColumnarExchange (65) - : +- * HashAggregate (64) - : +- * Project (63) - : +- * BroadcastHashJoin Inner BuildRight (62) - : :- * Project (60) - : : +- * BroadcastHashJoin Inner BuildRight (59) - : : :- * BroadcastHashJoin LeftSemi BuildRight (52) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * CometColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (48) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * CometColumnarToRow (36) - : : : : +- CometHashAggregate (35) - : : : : +- CometColumnarExchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * CometColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * CometColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * Filter (39) - : : : : : +- * ColumnarToRow (38) - : : : : : +- Scan parquet spark_catalog.default.web_sales (37) - : : : : +- ReusedExchange (40) - : : : +- ReusedExchange (43) - : : +- BroadcastExchange (58) - : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : :- * CometColumnarToRow (55) - : : : +- CometFilter (54) - : : : +- CometNativeScan parquet spark_catalog.default.item (53) - : : +- ReusedExchange (56) - : +- ReusedExchange (61) - :- * Project (86) - : +- * Filter (85) - : +- * HashAggregate (84) - : +- * CometColumnarToRow (83) - : +- CometColumnarExchange (82) - : +- * HashAggregate (81) - : +- * Project (80) - : +- * BroadcastHashJoin Inner BuildRight (79) - : :- * Project (77) - : : +- * BroadcastHashJoin Inner BuildRight (76) - : : :- * BroadcastHashJoin LeftSemi BuildRight (74) - : : : :- * Filter (72) - : : : : +- * ColumnarToRow (71) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (70) - : : : +- ReusedExchange (73) - : : +- ReusedExchange (75) - : +- ReusedExchange (78) - +- * Project (103) - +- * Filter (102) - +- * HashAggregate (101) - +- * CometColumnarToRow (100) - +- CometColumnarExchange (99) - +- * HashAggregate (98) - +- * Project (97) - +- * BroadcastHashJoin Inner BuildRight (96) - :- * Project (94) - : +- * BroadcastHashJoin Inner BuildRight (93) - : :- * BroadcastHashJoin LeftSemi BuildRight (91) - : : :- * Filter (89) - : : : +- * ColumnarToRow (88) - : : : +- Scan parquet spark_catalog.default.web_sales (87) - : : +- ReusedExchange (90) - : +- ReusedExchange (92) - +- ReusedExchange (95) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(6) CometColumnarToRow [codegen id : 11] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(10) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(11) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(12) CometColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(16) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) - -(18) CometColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(19) BroadcastExchange -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#19] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(22) ReusedExchange [Reuses operator id: 140] -Output [1]: [d_date_sk#23] - -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#23] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#23] - -(25) BroadcastExchange -Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] -Join type: LeftSemi -Join condition: None - -(27) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None - -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(30) ReusedExchange [Reuses operator id: 140] -Output [1]: [d_date_sk#24] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#25, i_class_id#15 AS class_id#26, i_category_id#16 AS category_id#27] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#24] - -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#25, class_id#26, category_id#27] - -(34) CometColumnarExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(35) CometHashAggregate -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] - -(36) CometColumnarToRow [codegen id : 10] -Input [3]: [brand_id#25, class_id#26, category_id#27] - -(37) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] - -(39) Filter [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Condition : isnotnull(ws_item_sk#28) - -(40) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#28] -Right keys [1]: [i_item_sk#30] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] -Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(43) ReusedExchange [Reuses operator id: 140] -Output [1]: [d_date_sk#34] - -(44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#29] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 9] -Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] - -(46) BroadcastExchange -Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] - -(47) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] -Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] -Join type: LeftSemi -Join condition: None - -(48) BroadcastExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] - -(49) BroadcastHashJoin [codegen id : 11] -Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#25, class_id#26, category_id#27] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#35] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] - -(51) BroadcastExchange -Input [1]: [ss_item_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(52) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(53) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(54) CometFilter -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Condition : isnotnull(i_item_sk#36) - -(55) CometColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(56) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#35] - -(57) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#36] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(58) BroadcastExchange -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(59) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#36] -Join type: Inner -Join condition: None - -(60) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(61) ReusedExchange [Reuses operator id: 135] -Output [1]: [d_date_sk#40] - -(62) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] - -(64) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] -Results [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(65) CometColumnarExchange -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Arguments: hashpartitioning(i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(66) CometColumnarToRow [codegen id : 26] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(67) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] -Results [5]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#49, count(1)#48 AS number_sales#50] - -(68) Filter [codegen id : 26] -Input [5]: [i_brand_id#37, i_class_id#38, i_category_id#39, sales#49, number_sales#50] -Condition : (isnotnull(sales#49) AND (cast(sales#49 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(69) Project [codegen id : 26] -Output [6]: [sales#49, number_sales#50, store AS channel#53, i_brand_id#37 AS i_brand_id#54, i_class_id#38 AS i_class_id#55, i_category_id#39 AS i_category_id#56] -Input [5]: [i_brand_id#37, i_class_id#38, i_category_id#39, sales#49, number_sales#50] - -(70) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(71) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] - -(72) Filter [codegen id : 51] -Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Condition : isnotnull(cs_item_sk#57) - -(73) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#35] - -(74) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#57] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(75) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] - -(76) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#57] -Right keys [1]: [i_item_sk#61] -Join type: Inner -Join condition: None - -(77) Project [codegen id : 51] -Output [6]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#62, i_class_id#63, i_category_id#64] -Input [8]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] - -(78) ReusedExchange [Reuses operator id: 135] -Output [1]: [d_date_sk#65] - -(79) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#60] -Right keys [1]: [d_date_sk#65] -Join type: Inner -Join condition: None - -(80) Project [codegen id : 51] -Output [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#62, i_class_id#63, i_category_id#64] -Input [7]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] - -(81) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#62, i_class_id#63, i_category_id#64] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [partial_sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), partial_count(1)] -Aggregate Attributes [3]: [sum#66, isEmpty#67, count#68] -Results [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] - -(82) CometColumnarExchange -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(83) CometColumnarToRow [codegen id : 52] -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] - -(84) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), count(1)] -Aggregate Attributes [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#72, count(1)#73] -Results [5]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59))#72 AS sales#74, count(1)#73 AS number_sales#75] - -(85) Filter [codegen id : 52] -Input [5]: [i_brand_id#62, i_class_id#63, i_category_id#64, sales#74, number_sales#75] -Condition : (isnotnull(sales#74) AND (cast(sales#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(86) Project [codegen id : 52] -Output [6]: [sales#74, number_sales#75, catalog AS channel#76, i_brand_id#62, i_class_id#63, i_category_id#64] -Input [5]: [i_brand_id#62, i_class_id#63, i_category_id#64, sales#74, number_sales#75] - -(87) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_sold_date_sk#80 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(88) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] - -(89) Filter [codegen id : 77] -Input [4]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80] -Condition : isnotnull(ws_item_sk#77) - -(90) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#35] - -(91) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#77] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(92) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#81, i_brand_id#82, i_class_id#83, i_category_id#84] - -(93) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#77] -Right keys [1]: [i_item_sk#81] -Join type: Inner -Join condition: None - -(94) Project [codegen id : 77] -Output [6]: [ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, i_brand_id#82, i_class_id#83, i_category_id#84] -Input [8]: [ws_item_sk#77, ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, i_item_sk#81, i_brand_id#82, i_class_id#83, i_category_id#84] - -(95) ReusedExchange [Reuses operator id: 135] -Output [1]: [d_date_sk#85] - -(96) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#80] -Right keys [1]: [d_date_sk#85] -Join type: Inner -Join condition: None - -(97) Project [codegen id : 77] -Output [5]: [ws_quantity#78, ws_list_price#79, i_brand_id#82, i_class_id#83, i_category_id#84] -Input [7]: [ws_quantity#78, ws_list_price#79, ws_sold_date_sk#80, i_brand_id#82, i_class_id#83, i_category_id#84, d_date_sk#85] - -(98) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#78, ws_list_price#79, i_brand_id#82, i_class_id#83, i_category_id#84] -Keys [3]: [i_brand_id#82, i_class_id#83, i_category_id#84] -Functions [2]: [partial_sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79)), partial_count(1)] -Aggregate Attributes [3]: [sum#86, isEmpty#87, count#88] -Results [6]: [i_brand_id#82, i_class_id#83, i_category_id#84, sum#89, isEmpty#90, count#91] - -(99) CometColumnarExchange -Input [6]: [i_brand_id#82, i_class_id#83, i_category_id#84, sum#89, isEmpty#90, count#91] -Arguments: hashpartitioning(i_brand_id#82, i_class_id#83, i_category_id#84, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(100) CometColumnarToRow [codegen id : 78] -Input [6]: [i_brand_id#82, i_class_id#83, i_category_id#84, sum#89, isEmpty#90, count#91] - -(101) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#82, i_class_id#83, i_category_id#84, sum#89, isEmpty#90, count#91] -Keys [3]: [i_brand_id#82, i_class_id#83, i_category_id#84] -Functions [2]: [sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79)), count(1)] -Aggregate Attributes [2]: [sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79))#92, count(1)#93] -Results [5]: [i_brand_id#82, i_class_id#83, i_category_id#84, sum((cast(ws_quantity#78 as decimal(10,0)) * ws_list_price#79))#92 AS sales#94, count(1)#93 AS number_sales#95] - -(102) Filter [codegen id : 78] -Input [5]: [i_brand_id#82, i_class_id#83, i_category_id#84, sales#94, number_sales#95] -Condition : (isnotnull(sales#94) AND (cast(sales#94 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(103) Project [codegen id : 78] -Output [6]: [sales#94, number_sales#95, web AS channel#96, i_brand_id#82, i_class_id#83, i_category_id#84] -Input [5]: [i_brand_id#82, i_class_id#83, i_category_id#84, sales#94, number_sales#95] - -(104) Union - -(105) Expand [codegen id : 79] -Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: [[sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, null, 1], [sales#49, number_sales#50, channel#53, i_brand_id#54, null, null, 3], [sales#49, number_sales#50, channel#53, null, null, null, 7], [sales#49, number_sales#50, null, null, null, null, 15]], [sales#49, number_sales#50, channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101] - -(106) HashAggregate [codegen id : 79] -Input [7]: [sales#49, number_sales#50, channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101] -Keys [5]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101] -Functions [2]: [partial_sum(sales#49), partial_sum(number_sales#50)] -Aggregate Attributes [3]: [sum#102, isEmpty#103, sum#104] -Results [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, sum#105, isEmpty#106, sum#107] - -(107) CometColumnarExchange -Input [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, sum#105, isEmpty#106, sum#107] -Arguments: hashpartitioning(channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(108) CometColumnarToRow [codegen id : 80] -Input [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, sum#105, isEmpty#106, sum#107] - -(109) HashAggregate [codegen id : 80] -Input [8]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101, sum#105, isEmpty#106, sum#107] -Keys [5]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, spark_grouping_id#101] -Functions [2]: [sum(sales#49), sum(number_sales#50)] -Aggregate Attributes [2]: [sum(sales#49)#108, sum(number_sales#50)#109] -Results [6]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, sum(sales#49)#108 AS sum(sales)#110, sum(number_sales#50)#109 AS sum(number_sales)#111] - -(110) TakeOrderedAndProject -Input [6]: [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, sum(sales)#110, sum(number_sales)#111] -Arguments: 100, [channel#97 ASC NULLS FIRST, i_brand_id#98 ASC NULLS FIRST, i_class_id#99 ASC NULLS FIRST, i_category_id#100 ASC NULLS FIRST], [channel#97, i_brand_id#98, i_class_id#99, i_category_id#100, sum(sales)#110, sum(number_sales)#111] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* HashAggregate (130) -+- * CometColumnarToRow (129) - +- CometColumnarExchange (128) - +- * HashAggregate (127) - +- Union (126) - :- * Project (115) - : +- * BroadcastHashJoin Inner BuildRight (114) - : :- * ColumnarToRow (112) - : : +- Scan parquet spark_catalog.default.store_sales (111) - : +- ReusedExchange (113) - :- * Project (120) - : +- * BroadcastHashJoin Inner BuildRight (119) - : :- * ColumnarToRow (117) - : : +- Scan parquet spark_catalog.default.catalog_sales (116) - : +- ReusedExchange (118) - +- * Project (125) - +- * BroadcastHashJoin Inner BuildRight (124) - :- * ColumnarToRow (122) - : +- Scan parquet spark_catalog.default.web_sales (121) - +- ReusedExchange (123) - - -(111) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#112, ss_list_price#113, ss_sold_date_sk#114] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#114), dynamicpruningexpression(ss_sold_date_sk#114 IN dynamicpruning#12)] -ReadSchema: struct - -(112) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#112, ss_list_price#113, ss_sold_date_sk#114] - -(113) ReusedExchange [Reuses operator id: 140] -Output [1]: [d_date_sk#115] - -(114) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#114] -Right keys [1]: [d_date_sk#115] -Join type: Inner -Join condition: None - -(115) Project [codegen id : 2] -Output [2]: [ss_quantity#112 AS quantity#116, ss_list_price#113 AS list_price#117] -Input [4]: [ss_quantity#112, ss_list_price#113, ss_sold_date_sk#114, d_date_sk#115] - -(116) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#118, cs_list_price#119, cs_sold_date_sk#120] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#120), dynamicpruningexpression(cs_sold_date_sk#120 IN dynamicpruning#12)] -ReadSchema: struct - -(117) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#118, cs_list_price#119, cs_sold_date_sk#120] - -(118) ReusedExchange [Reuses operator id: 140] -Output [1]: [d_date_sk#121] - -(119) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#120] -Right keys [1]: [d_date_sk#121] -Join type: Inner -Join condition: None - -(120) Project [codegen id : 4] -Output [2]: [cs_quantity#118 AS quantity#122, cs_list_price#119 AS list_price#123] -Input [4]: [cs_quantity#118, cs_list_price#119, cs_sold_date_sk#120, d_date_sk#121] - -(121) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#124, ws_list_price#125, ws_sold_date_sk#126] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#126), dynamicpruningexpression(ws_sold_date_sk#126 IN dynamicpruning#12)] -ReadSchema: struct - -(122) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#124, ws_list_price#125, ws_sold_date_sk#126] - -(123) ReusedExchange [Reuses operator id: 140] -Output [1]: [d_date_sk#127] - -(124) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#126] -Right keys [1]: [d_date_sk#127] -Join type: Inner -Join condition: None - -(125) Project [codegen id : 6] -Output [2]: [ws_quantity#124 AS quantity#128, ws_list_price#125 AS list_price#129] -Input [4]: [ws_quantity#124, ws_list_price#125, ws_sold_date_sk#126, d_date_sk#127] - -(126) Union - -(127) HashAggregate [codegen id : 7] -Input [2]: [quantity#116, list_price#117] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#116 as decimal(10,0)) * list_price#117))] -Aggregate Attributes [2]: [sum#130, count#131] -Results [2]: [sum#132, count#133] - -(128) CometColumnarExchange -Input [2]: [sum#132, count#133] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(129) CometColumnarToRow [codegen id : 8] -Input [2]: [sum#132, count#133] - -(130) HashAggregate [codegen id : 8] -Input [2]: [sum#132, count#133] -Keys: [] -Functions [1]: [avg((cast(quantity#116 as decimal(10,0)) * list_price#117))] -Aggregate Attributes [1]: [avg((cast(quantity#116 as decimal(10,0)) * list_price#117))#134] -Results [1]: [avg((cast(quantity#116 as decimal(10,0)) * list_price#117))#134 AS average_sales#135] - -Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#114 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 116 Hosting Expression = cs_sold_date_sk#120 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 121 Hosting Expression = ws_sold_date_sk#126 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (135) -+- * CometColumnarToRow (134) - +- CometProject (133) - +- CometFilter (132) - +- CometNativeScan parquet spark_catalog.default.date_dim (131) - - -(131) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#40, d_year#136, d_moy#137] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(132) CometFilter -Input [3]: [d_date_sk#40, d_year#136, d_moy#137] -Condition : ((((isnotnull(d_year#136) AND isnotnull(d_moy#137)) AND (d_year#136 = 2001)) AND (d_moy#137 = 11)) AND isnotnull(d_date_sk#40)) - -(133) CometProject -Input [3]: [d_date_sk#40, d_year#136, d_moy#137] -Arguments: [d_date_sk#40], [d_date_sk#40] - -(134) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#40] - -(135) BroadcastExchange -Input [1]: [d_date_sk#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (140) -+- * CometColumnarToRow (139) - +- CometProject (138) - +- CometFilter (137) - +- CometNativeScan parquet spark_catalog.default.date_dim (136) - - -(136) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#138] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(137) CometFilter -Input [2]: [d_date_sk#24, d_year#138] -Condition : (((isnotnull(d_year#138) AND (d_year#138 >= 1999)) AND (d_year#138 <= 2001)) AND isnotnull(d_date_sk#24)) - -(138) CometProject -Input [2]: [d_date_sk#24, d_year#138] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(139) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#24] - -(140) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] - -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:8 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 85 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] - -Subquery:10 Hosting operator id = 70 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 - -Subquery:11 Hosting operator id = 102 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] - -Subquery:12 Hosting operator id = 87 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/simplified.txt deleted file mode 100644 index c989fe9a81..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/simplified.txt +++ /dev/null @@ -1,220 +0,0 @@ -TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - WholeStageCodegen (80) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - WholeStageCodegen (79) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - InputAdapter - Union - WholeStageCodegen (26) - Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] - Filter [sales] - Subquery #3 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #13 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #7 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #7 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #7 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (52) - Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (51) - HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (78) - Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (77) - HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/explain.txt deleted file mode 100644 index 0289abc42d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,754 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (102) -+- CometTakeOrderedAndProject (101) - +- CometHashAggregate (100) - +- CometExchange (99) - +- CometHashAggregate (98) - +- CometExpand (97) - +- CometUnion (96) - :- CometProject (65) - : +- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometBroadcastHashJoin (47) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (46) - : : : +- CometProject (45) - : : : +- CometBroadcastHashJoin (44) - : : : :- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (43) - : : : +- CometBroadcastHashJoin (42) - : : : :- CometHashAggregate (32) - : : : : +- CometExchange (31) - : : : : +- CometHashAggregate (30) - : : : : +- CometProject (29) - : : : : +- CometBroadcastHashJoin (28) - : : : : :- CometProject (26) - : : : : : +- CometBroadcastHashJoin (25) - : : : : : :- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : +- CometBroadcastExchange (24) - : : : : : +- CometBroadcastHashJoin (23) - : : : : : :- CometFilter (8) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : +- CometBroadcastExchange (22) - : : : : : +- CometProject (21) - : : : : : +- CometBroadcastHashJoin (20) - : : : : : :- CometProject (15) - : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : :- CometFilter (10) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : +- CometBroadcastExchange (13) - : : : : : : +- CometFilter (12) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : +- CometBroadcastExchange (19) - : : : : : +- CometProject (18) - : : : : : +- CometFilter (17) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : +- ReusedExchange (27) - : : : +- CometBroadcastExchange (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (34) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometFilter (49) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : +- ReusedExchange (50) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - :- CometProject (80) - : +- CometFilter (79) - : +- CometHashAggregate (78) - : +- CometExchange (77) - : +- CometHashAggregate (76) - : +- CometProject (75) - : +- CometBroadcastHashJoin (74) - : :- CometProject (72) - : : +- CometBroadcastHashJoin (71) - : : :- CometBroadcastHashJoin (69) - : : : :- CometFilter (67) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (66) - : : : +- ReusedExchange (68) - : : +- ReusedExchange (70) - : +- ReusedExchange (73) - +- CometProject (95) - +- CometFilter (94) - +- CometHashAggregate (93) - +- CometExchange (92) - +- CometHashAggregate (91) - +- CometProject (90) - +- CometBroadcastHashJoin (89) - :- CometProject (87) - : +- CometBroadcastHashJoin (86) - : :- CometBroadcastHashJoin (84) - : : :- CometFilter (82) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (81) - : : +- ReusedExchange (83) - : +- ReusedExchange (85) - +- ReusedExchange (88) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : isnotnull(i_item_sk#39) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2001)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#49, number_sales#50] -Condition : (isnotnull(sales#49) AND (cast(sales#49 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(65) CometProject -Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#49, number_sales#50] -Arguments: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56], [sales#49, number_sales#50, store AS channel#53, i_brand_id#40 AS i_brand_id#54, i_class_id#41 AS i_class_id#55, i_category_id#42 AS i_category_id#56] - -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(67) CometFilter -Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Condition : isnotnull(cs_item_sk#57) - -(68) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(69) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Right output [1]: [ss_item_sk#38] -Arguments: [cs_item_sk#57], [ss_item_sk#38], LeftSemi, BuildRight - -(70) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] - -(71) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Right output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -Arguments: [cs_item_sk#57], [i_item_sk#62], Inner, BuildRight - -(72) CometProject -Input [8]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -Arguments: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65], [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65] - -(73) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#66] - -(74) CometBroadcastHashJoin -Left output [6]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65] -Right output [1]: [d_date_sk#66] -Arguments: [cs_sold_date_sk#60], [d_date_sk#66], Inner, BuildRight - -(75) CometProject -Input [7]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] -Arguments: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65], [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] - -(76) CometHashAggregate -Input [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [partial_sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), partial_count(1)] - -(77) CometExchange -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#67, isEmpty#68, count#69] -Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(78) CometHashAggregate -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#67, isEmpty#68, count#69] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), count(1)] - -(79) CometFilter -Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#70, number_sales#71] -Condition : (isnotnull(sales#70) AND (cast(sales#70 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(80) CometProject -Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#70, number_sales#71] -Arguments: [sales#70, number_sales#71, channel#72, i_brand_id#63, i_class_id#64, i_category_id#65], [sales#70, number_sales#71, catalog AS channel#72, i_brand_id#63, i_class_id#64, i_category_id#65] - -(81) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#76), dynamicpruningexpression(ws_sold_date_sk#76 IN dynamicpruning#77)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(82) CometFilter -Input [4]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76] -Condition : isnotnull(ws_item_sk#73) - -(83) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(84) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76] -Right output [1]: [ss_item_sk#38] -Arguments: [ws_item_sk#73], [ss_item_sk#38], LeftSemi, BuildRight - -(85) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81] - -(86) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76] -Right output [4]: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81] -Arguments: [ws_item_sk#73], [i_item_sk#78], Inner, BuildRight - -(87) CometProject -Input [8]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76, i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81] -Arguments: [ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81], [ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81] - -(88) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#82] - -(89) CometBroadcastHashJoin -Left output [6]: [ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81] -Right output [1]: [d_date_sk#82] -Arguments: [ws_sold_date_sk#76], [d_date_sk#82], Inner, BuildRight - -(90) CometProject -Input [7]: [ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81, d_date_sk#82] -Arguments: [ws_quantity#74, ws_list_price#75, i_brand_id#79, i_class_id#80, i_category_id#81], [ws_quantity#74, ws_list_price#75, i_brand_id#79, i_class_id#80, i_category_id#81] - -(91) CometHashAggregate -Input [5]: [ws_quantity#74, ws_list_price#75, i_brand_id#79, i_class_id#80, i_category_id#81] -Keys [3]: [i_brand_id#79, i_class_id#80, i_category_id#81] -Functions [2]: [partial_sum((cast(ws_quantity#74 as decimal(10,0)) * ws_list_price#75)), partial_count(1)] - -(92) CometExchange -Input [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#83, isEmpty#84, count#85] -Arguments: hashpartitioning(i_brand_id#79, i_class_id#80, i_category_id#81, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(93) CometHashAggregate -Input [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#83, isEmpty#84, count#85] -Keys [3]: [i_brand_id#79, i_class_id#80, i_category_id#81] -Functions [2]: [sum((cast(ws_quantity#74 as decimal(10,0)) * ws_list_price#75)), count(1)] - -(94) CometFilter -Input [5]: [i_brand_id#79, i_class_id#80, i_category_id#81, sales#86, number_sales#87] -Condition : (isnotnull(sales#86) AND (cast(sales#86 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(95) CometProject -Input [5]: [i_brand_id#79, i_class_id#80, i_category_id#81, sales#86, number_sales#87] -Arguments: [sales#86, number_sales#87, channel#88, i_brand_id#79, i_class_id#80, i_category_id#81], [sales#86, number_sales#87, web AS channel#88, i_brand_id#79, i_class_id#80, i_category_id#81] - -(96) CometUnion -Child 0 Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Child 1 Input [6]: [sales#70, number_sales#71, channel#72, i_brand_id#63, i_class_id#64, i_category_id#65] -Child 2 Input [6]: [sales#86, number_sales#87, channel#88, i_brand_id#79, i_class_id#80, i_category_id#81] - -(97) CometExpand -Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: [[sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, null, 1], [sales#49, number_sales#50, channel#53, i_brand_id#54, null, null, 3], [sales#49, number_sales#50, channel#53, null, null, null, 7], [sales#49, number_sales#50, null, null, null, null, 15]], [sales#49, number_sales#50, channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, spark_grouping_id#93] - -(98) CometHashAggregate -Input [7]: [sales#49, number_sales#50, channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, spark_grouping_id#93] -Keys [5]: [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, spark_grouping_id#93] -Functions [2]: [partial_sum(sales#49), partial_sum(number_sales#50)] - -(99) CometExchange -Input [8]: [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96] -Arguments: hashpartitioning(channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, spark_grouping_id#93, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(100) CometHashAggregate -Input [8]: [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96] -Keys [5]: [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, spark_grouping_id#93] -Functions [2]: [sum(sales#49), sum(number_sales#50)] - -(101) CometTakeOrderedAndProject -Input [6]: [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, sum(sales)#97, sum(number_sales)#98] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#89 ASC NULLS FIRST,i_brand_id#90 ASC NULLS FIRST,i_class_id#91 ASC NULLS FIRST,i_category_id#92 ASC NULLS FIRST], output=[channel#89,i_brand_id#90,i_class_id#91,i_category_id#92,sum(sales)#97,sum(number_sales)#98]), [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, sum(sales)#97, sum(number_sales)#98], 100, 0, [channel#89 ASC NULLS FIRST, i_brand_id#90 ASC NULLS FIRST, i_class_id#91 ASC NULLS FIRST, i_category_id#92 ASC NULLS FIRST], [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, sum(sales)#97, sum(number_sales)#98] - -(102) CometColumnarToRow [codegen id : 1] -Input [6]: [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, sum(sales)#97, sum(number_sales)#98] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* CometColumnarToRow (119) -+- CometHashAggregate (118) - +- CometExchange (117) - +- CometHashAggregate (116) - +- CometUnion (115) - :- CometProject (106) - : +- CometBroadcastHashJoin (105) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (103) - : +- ReusedExchange (104) - :- CometProject (110) - : +- CometBroadcastHashJoin (109) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (107) - : +- ReusedExchange (108) - +- CometProject (114) - +- CometBroadcastHashJoin (113) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (111) - +- ReusedExchange (112) - - -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#99, ss_list_price#100, ss_sold_date_sk#101] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#101), dynamicpruningexpression(ss_sold_date_sk#101 IN dynamicpruning#102)] -ReadSchema: struct - -(104) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#103] - -(105) CometBroadcastHashJoin -Left output [3]: [ss_quantity#99, ss_list_price#100, ss_sold_date_sk#101] -Right output [1]: [d_date_sk#103] -Arguments: [ss_sold_date_sk#101], [d_date_sk#103], Inner, BuildRight - -(106) CometProject -Input [4]: [ss_quantity#99, ss_list_price#100, ss_sold_date_sk#101, d_date_sk#103] -Arguments: [quantity#104, list_price#105], [ss_quantity#99 AS quantity#104, ss_list_price#100 AS list_price#105] - -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#106, cs_list_price#107, cs_sold_date_sk#108] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#108), dynamicpruningexpression(cs_sold_date_sk#108 IN dynamicpruning#109)] -ReadSchema: struct - -(108) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#110] - -(109) CometBroadcastHashJoin -Left output [3]: [cs_quantity#106, cs_list_price#107, cs_sold_date_sk#108] -Right output [1]: [d_date_sk#110] -Arguments: [cs_sold_date_sk#108], [d_date_sk#110], Inner, BuildRight - -(110) CometProject -Input [4]: [cs_quantity#106, cs_list_price#107, cs_sold_date_sk#108, d_date_sk#110] -Arguments: [quantity#111, list_price#112], [cs_quantity#106 AS quantity#111, cs_list_price#107 AS list_price#112] - -(111) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#115), dynamicpruningexpression(ws_sold_date_sk#115 IN dynamicpruning#116)] -ReadSchema: struct - -(112) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#117] - -(113) CometBroadcastHashJoin -Left output [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] -Right output [1]: [d_date_sk#117] -Arguments: [ws_sold_date_sk#115], [d_date_sk#117], Inner, BuildRight - -(114) CometProject -Input [4]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115, d_date_sk#117] -Arguments: [quantity#118, list_price#119], [ws_quantity#113 AS quantity#118, ws_list_price#114 AS list_price#119] - -(115) CometUnion -Child 0 Input [2]: [quantity#104, list_price#105] -Child 1 Input [2]: [quantity#111, list_price#112] -Child 2 Input [2]: [quantity#118, list_price#119] - -(116) CometHashAggregate -Input [2]: [quantity#104, list_price#105] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#104 as decimal(10,0)) * list_price#105))] - -(117) CometExchange -Input [2]: [sum#120, count#121] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(118) CometHashAggregate -Input [2]: [sum#120, count#121] -Keys: [] -Functions [1]: [avg((cast(quantity#104 as decimal(10,0)) * list_price#105))] - -(119) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#122] - -Subquery:2 Hosting operator id = 103 Hosting Expression = ss_sold_date_sk#101 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 107 Hosting Expression = cs_sold_date_sk#108 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 111 Hosting Expression = ws_sold_date_sk#115 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (124) -+- * CometColumnarToRow (123) - +- CometProject (122) - +- CometFilter (121) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (120) - - -(120) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(121) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2001)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(122) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(123) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(124) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (129) -+- * CometColumnarToRow (128) - +- CometProject (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) - - -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#26, d_year#123] -Condition : (((isnotnull(d_year#123) AND (d_year#123 >= 1999)) AND (d_year#123 <= 2001)) AND isnotnull(d_date_sk#26)) - -(127) CometProject -Input [2]: [d_date_sk#26, d_year#123] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(128) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(129) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 79 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] - -Subquery:10 Hosting operator id = 66 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 - -Subquery:11 Hosting operator id = 94 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] - -Subquery:12 Hosting operator id = 81 Hosting Expression = ws_sold_date_sk#76 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 79c782f2ca..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,149 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales),sum(sales),sum(number_salesL)] - CometExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - CometHashAggregate [sales,number_sales] [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] - CometExpand [channel,i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] - CometUnion [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometProject [i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #15 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #2 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk] #4 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #5 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #6 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #11 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #11 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #12 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - ReusedExchange [d_date_sk] #11 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #4 - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [cs_quantity,cs_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk] - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #4 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - ReusedExchange [d_date_sk] #14 - CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #17 - CometHashAggregate [ws_quantity,ws_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk] - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #4 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - ReusedExchange [d_date_sk] #14 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt deleted file mode 100644 index 0289abc42d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ /dev/null @@ -1,754 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (102) -+- CometTakeOrderedAndProject (101) - +- CometHashAggregate (100) - +- CometExchange (99) - +- CometHashAggregate (98) - +- CometExpand (97) - +- CometUnion (96) - :- CometProject (65) - : +- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometBroadcastHashJoin (47) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (46) - : : : +- CometProject (45) - : : : +- CometBroadcastHashJoin (44) - : : : :- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (43) - : : : +- CometBroadcastHashJoin (42) - : : : :- CometHashAggregate (32) - : : : : +- CometExchange (31) - : : : : +- CometHashAggregate (30) - : : : : +- CometProject (29) - : : : : +- CometBroadcastHashJoin (28) - : : : : :- CometProject (26) - : : : : : +- CometBroadcastHashJoin (25) - : : : : : :- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : +- CometBroadcastExchange (24) - : : : : : +- CometBroadcastHashJoin (23) - : : : : : :- CometFilter (8) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : +- CometBroadcastExchange (22) - : : : : : +- CometProject (21) - : : : : : +- CometBroadcastHashJoin (20) - : : : : : :- CometProject (15) - : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : :- CometFilter (10) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : +- CometBroadcastExchange (13) - : : : : : : +- CometFilter (12) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : +- CometBroadcastExchange (19) - : : : : : +- CometProject (18) - : : : : : +- CometFilter (17) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : +- ReusedExchange (27) - : : : +- CometBroadcastExchange (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (34) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometFilter (49) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : +- ReusedExchange (50) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - :- CometProject (80) - : +- CometFilter (79) - : +- CometHashAggregate (78) - : +- CometExchange (77) - : +- CometHashAggregate (76) - : +- CometProject (75) - : +- CometBroadcastHashJoin (74) - : :- CometProject (72) - : : +- CometBroadcastHashJoin (71) - : : :- CometBroadcastHashJoin (69) - : : : :- CometFilter (67) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (66) - : : : +- ReusedExchange (68) - : : +- ReusedExchange (70) - : +- ReusedExchange (73) - +- CometProject (95) - +- CometFilter (94) - +- CometHashAggregate (93) - +- CometExchange (92) - +- CometHashAggregate (91) - +- CometProject (90) - +- CometBroadcastHashJoin (89) - :- CometProject (87) - : +- CometBroadcastHashJoin (86) - : :- CometBroadcastHashJoin (84) - : : :- CometFilter (82) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (81) - : : +- ReusedExchange (83) - : +- ReusedExchange (85) - +- ReusedExchange (88) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : isnotnull(i_item_sk#39) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2001)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#49, number_sales#50] -Condition : (isnotnull(sales#49) AND (cast(sales#49 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(65) CometProject -Input [5]: [i_brand_id#40, i_class_id#41, i_category_id#42, sales#49, number_sales#50] -Arguments: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56], [sales#49, number_sales#50, store AS channel#53, i_brand_id#40 AS i_brand_id#54, i_class_id#41 AS i_class_id#55, i_category_id#42 AS i_category_id#56] - -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(67) CometFilter -Input [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Condition : isnotnull(cs_item_sk#57) - -(68) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(69) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Right output [1]: [ss_item_sk#38] -Arguments: [cs_item_sk#57], [ss_item_sk#38], LeftSemi, BuildRight - -(70) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] - -(71) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60] -Right output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -Arguments: [cs_item_sk#57], [i_item_sk#62], Inner, BuildRight - -(72) CometProject -Input [8]: [cs_item_sk#57, cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] -Arguments: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65], [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65] - -(73) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#66] - -(74) CometBroadcastHashJoin -Left output [6]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65] -Right output [1]: [d_date_sk#66] -Arguments: [cs_sold_date_sk#60], [d_date_sk#66], Inner, BuildRight - -(75) CometProject -Input [7]: [cs_quantity#58, cs_list_price#59, cs_sold_date_sk#60, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] -Arguments: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65], [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] - -(76) CometHashAggregate -Input [5]: [cs_quantity#58, cs_list_price#59, i_brand_id#63, i_class_id#64, i_category_id#65] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [partial_sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), partial_count(1)] - -(77) CometExchange -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#67, isEmpty#68, count#69] -Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(78) CometHashAggregate -Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#67, isEmpty#68, count#69] -Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] -Functions [2]: [sum((cast(cs_quantity#58 as decimal(10,0)) * cs_list_price#59)), count(1)] - -(79) CometFilter -Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#70, number_sales#71] -Condition : (isnotnull(sales#70) AND (cast(sales#70 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(80) CometProject -Input [5]: [i_brand_id#63, i_class_id#64, i_category_id#65, sales#70, number_sales#71] -Arguments: [sales#70, number_sales#71, channel#72, i_brand_id#63, i_class_id#64, i_category_id#65], [sales#70, number_sales#71, catalog AS channel#72, i_brand_id#63, i_class_id#64, i_category_id#65] - -(81) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#76), dynamicpruningexpression(ws_sold_date_sk#76 IN dynamicpruning#77)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(82) CometFilter -Input [4]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76] -Condition : isnotnull(ws_item_sk#73) - -(83) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(84) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76] -Right output [1]: [ss_item_sk#38] -Arguments: [ws_item_sk#73], [ss_item_sk#38], LeftSemi, BuildRight - -(85) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81] - -(86) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76] -Right output [4]: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81] -Arguments: [ws_item_sk#73], [i_item_sk#78], Inner, BuildRight - -(87) CometProject -Input [8]: [ws_item_sk#73, ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76, i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81] -Arguments: [ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81], [ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81] - -(88) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#82] - -(89) CometBroadcastHashJoin -Left output [6]: [ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81] -Right output [1]: [d_date_sk#82] -Arguments: [ws_sold_date_sk#76], [d_date_sk#82], Inner, BuildRight - -(90) CometProject -Input [7]: [ws_quantity#74, ws_list_price#75, ws_sold_date_sk#76, i_brand_id#79, i_class_id#80, i_category_id#81, d_date_sk#82] -Arguments: [ws_quantity#74, ws_list_price#75, i_brand_id#79, i_class_id#80, i_category_id#81], [ws_quantity#74, ws_list_price#75, i_brand_id#79, i_class_id#80, i_category_id#81] - -(91) CometHashAggregate -Input [5]: [ws_quantity#74, ws_list_price#75, i_brand_id#79, i_class_id#80, i_category_id#81] -Keys [3]: [i_brand_id#79, i_class_id#80, i_category_id#81] -Functions [2]: [partial_sum((cast(ws_quantity#74 as decimal(10,0)) * ws_list_price#75)), partial_count(1)] - -(92) CometExchange -Input [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#83, isEmpty#84, count#85] -Arguments: hashpartitioning(i_brand_id#79, i_class_id#80, i_category_id#81, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(93) CometHashAggregate -Input [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#83, isEmpty#84, count#85] -Keys [3]: [i_brand_id#79, i_class_id#80, i_category_id#81] -Functions [2]: [sum((cast(ws_quantity#74 as decimal(10,0)) * ws_list_price#75)), count(1)] - -(94) CometFilter -Input [5]: [i_brand_id#79, i_class_id#80, i_category_id#81, sales#86, number_sales#87] -Condition : (isnotnull(sales#86) AND (cast(sales#86 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) - -(95) CometProject -Input [5]: [i_brand_id#79, i_class_id#80, i_category_id#81, sales#86, number_sales#87] -Arguments: [sales#86, number_sales#87, channel#88, i_brand_id#79, i_class_id#80, i_category_id#81], [sales#86, number_sales#87, web AS channel#88, i_brand_id#79, i_class_id#80, i_category_id#81] - -(96) CometUnion -Child 0 Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Child 1 Input [6]: [sales#70, number_sales#71, channel#72, i_brand_id#63, i_class_id#64, i_category_id#65] -Child 2 Input [6]: [sales#86, number_sales#87, channel#88, i_brand_id#79, i_class_id#80, i_category_id#81] - -(97) CometExpand -Input [6]: [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: [[sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, i_category_id#56, 0], [sales#49, number_sales#50, channel#53, i_brand_id#54, i_class_id#55, null, 1], [sales#49, number_sales#50, channel#53, i_brand_id#54, null, null, 3], [sales#49, number_sales#50, channel#53, null, null, null, 7], [sales#49, number_sales#50, null, null, null, null, 15]], [sales#49, number_sales#50, channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, spark_grouping_id#93] - -(98) CometHashAggregate -Input [7]: [sales#49, number_sales#50, channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, spark_grouping_id#93] -Keys [5]: [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, spark_grouping_id#93] -Functions [2]: [partial_sum(sales#49), partial_sum(number_sales#50)] - -(99) CometExchange -Input [8]: [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96] -Arguments: hashpartitioning(channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, spark_grouping_id#93, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(100) CometHashAggregate -Input [8]: [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96] -Keys [5]: [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, spark_grouping_id#93] -Functions [2]: [sum(sales#49), sum(number_sales#50)] - -(101) CometTakeOrderedAndProject -Input [6]: [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, sum(sales)#97, sum(number_sales)#98] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#89 ASC NULLS FIRST,i_brand_id#90 ASC NULLS FIRST,i_class_id#91 ASC NULLS FIRST,i_category_id#92 ASC NULLS FIRST], output=[channel#89,i_brand_id#90,i_class_id#91,i_category_id#92,sum(sales)#97,sum(number_sales)#98]), [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, sum(sales)#97, sum(number_sales)#98], 100, 0, [channel#89 ASC NULLS FIRST, i_brand_id#90 ASC NULLS FIRST, i_class_id#91 ASC NULLS FIRST, i_category_id#92 ASC NULLS FIRST], [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, sum(sales)#97, sum(number_sales)#98] - -(102) CometColumnarToRow [codegen id : 1] -Input [6]: [channel#89, i_brand_id#90, i_class_id#91, i_category_id#92, sum(sales)#97, sum(number_sales)#98] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#51, [id=#52] -* CometColumnarToRow (119) -+- CometHashAggregate (118) - +- CometExchange (117) - +- CometHashAggregate (116) - +- CometUnion (115) - :- CometProject (106) - : +- CometBroadcastHashJoin (105) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (103) - : +- ReusedExchange (104) - :- CometProject (110) - : +- CometBroadcastHashJoin (109) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (107) - : +- ReusedExchange (108) - +- CometProject (114) - +- CometBroadcastHashJoin (113) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (111) - +- ReusedExchange (112) - - -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#99, ss_list_price#100, ss_sold_date_sk#101] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#101), dynamicpruningexpression(ss_sold_date_sk#101 IN dynamicpruning#102)] -ReadSchema: struct - -(104) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#103] - -(105) CometBroadcastHashJoin -Left output [3]: [ss_quantity#99, ss_list_price#100, ss_sold_date_sk#101] -Right output [1]: [d_date_sk#103] -Arguments: [ss_sold_date_sk#101], [d_date_sk#103], Inner, BuildRight - -(106) CometProject -Input [4]: [ss_quantity#99, ss_list_price#100, ss_sold_date_sk#101, d_date_sk#103] -Arguments: [quantity#104, list_price#105], [ss_quantity#99 AS quantity#104, ss_list_price#100 AS list_price#105] - -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#106, cs_list_price#107, cs_sold_date_sk#108] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#108), dynamicpruningexpression(cs_sold_date_sk#108 IN dynamicpruning#109)] -ReadSchema: struct - -(108) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#110] - -(109) CometBroadcastHashJoin -Left output [3]: [cs_quantity#106, cs_list_price#107, cs_sold_date_sk#108] -Right output [1]: [d_date_sk#110] -Arguments: [cs_sold_date_sk#108], [d_date_sk#110], Inner, BuildRight - -(110) CometProject -Input [4]: [cs_quantity#106, cs_list_price#107, cs_sold_date_sk#108, d_date_sk#110] -Arguments: [quantity#111, list_price#112], [cs_quantity#106 AS quantity#111, cs_list_price#107 AS list_price#112] - -(111) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#115), dynamicpruningexpression(ws_sold_date_sk#115 IN dynamicpruning#116)] -ReadSchema: struct - -(112) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#117] - -(113) CometBroadcastHashJoin -Left output [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] -Right output [1]: [d_date_sk#117] -Arguments: [ws_sold_date_sk#115], [d_date_sk#117], Inner, BuildRight - -(114) CometProject -Input [4]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115, d_date_sk#117] -Arguments: [quantity#118, list_price#119], [ws_quantity#113 AS quantity#118, ws_list_price#114 AS list_price#119] - -(115) CometUnion -Child 0 Input [2]: [quantity#104, list_price#105] -Child 1 Input [2]: [quantity#111, list_price#112] -Child 2 Input [2]: [quantity#118, list_price#119] - -(116) CometHashAggregate -Input [2]: [quantity#104, list_price#105] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#104 as decimal(10,0)) * list_price#105))] - -(117) CometExchange -Input [2]: [sum#120, count#121] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(118) CometHashAggregate -Input [2]: [sum#120, count#121] -Keys: [] -Functions [1]: [avg((cast(quantity#104 as decimal(10,0)) * list_price#105))] - -(119) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#122] - -Subquery:2 Hosting operator id = 103 Hosting Expression = ss_sold_date_sk#101 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 107 Hosting Expression = cs_sold_date_sk#108 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 111 Hosting Expression = ws_sold_date_sk#115 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (124) -+- * CometColumnarToRow (123) - +- CometProject (122) - +- CometFilter (121) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (120) - - -(120) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(121) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2001)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(122) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(123) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(124) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (129) -+- * CometColumnarToRow (128) - +- CometProject (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) - - -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#26, d_year#123] -Condition : (((isnotnull(d_year#123) AND (d_year#123 >= 1999)) AND (d_year#123 <= 2001)) AND isnotnull(d_date_sk#26)) - -(127) CometProject -Input [2]: [d_date_sk#26, d_year#123] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(128) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(129) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 79 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] - -Subquery:10 Hosting operator id = 66 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#5 - -Subquery:11 Hosting operator id = 94 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] - -Subquery:12 Hosting operator id = 81 Hosting Expression = ws_sold_date_sk#76 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/extended.txt deleted file mode 100644 index 4af04a7846..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/extended.txt +++ /dev/null @@ -1,469 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 424 out of 458 eligible operators (92%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 79c782f2ca..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ /dev/null @@ -1,149 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales),sum(sales),sum(number_salesL)] - CometExchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - CometHashAggregate [sales,number_sales] [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] - CometExpand [channel,i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] - CometUnion [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometProject [i_brand_id,i_class_id,i_category_id] [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #15 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #2 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk] #4 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #5 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #6 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #11 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #11 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #12 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 - ReusedExchange [d_date_sk] #11 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #4 - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [cs_quantity,cs_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk] - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #4 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - ReusedExchange [d_date_sk] #14 - CometProject [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] - CometFilter [i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #17 - CometHashAggregate [ws_quantity,ws_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk] - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #4 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - ReusedExchange [d_date_sk] #14 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/explain.txt deleted file mode 100644 index d539836be8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/explain.txt +++ /dev/null @@ -1,769 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (87) -+- * BroadcastHashJoin Inner BuildRight (86) - :- * Filter (68) - : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) - : +- CometColumnarExchange (65) - : +- * HashAggregate (64) - : +- * Project (63) - : +- * BroadcastHashJoin Inner BuildRight (62) - : :- * Project (60) - : : +- * BroadcastHashJoin Inner BuildRight (59) - : : :- * BroadcastHashJoin LeftSemi BuildRight (52) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * CometColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (48) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * CometColumnarToRow (36) - : : : : +- CometHashAggregate (35) - : : : : +- CometColumnarExchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * CometColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * CometColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * Filter (39) - : : : : : +- * ColumnarToRow (38) - : : : : : +- Scan parquet spark_catalog.default.web_sales (37) - : : : : +- ReusedExchange (40) - : : : +- ReusedExchange (43) - : : +- BroadcastExchange (58) - : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : :- * CometColumnarToRow (55) - : : : +- CometFilter (54) - : : : +- CometNativeScan parquet spark_catalog.default.item (53) - : : +- ReusedExchange (56) - : +- ReusedExchange (61) - +- BroadcastExchange (85) - +- * Filter (84) - +- * HashAggregate (83) - +- * CometColumnarToRow (82) - +- CometColumnarExchange (81) - +- * HashAggregate (80) - +- * Project (79) - +- * BroadcastHashJoin Inner BuildRight (78) - :- * Project (76) - : +- * BroadcastHashJoin Inner BuildRight (75) - : :- * BroadcastHashJoin LeftSemi BuildRight (73) - : : :- * Filter (71) - : : : +- * ColumnarToRow (70) - : : : +- Scan parquet spark_catalog.default.store_sales (69) - : : +- ReusedExchange (72) - : +- ReusedExchange (74) - +- ReusedExchange (77) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(6) CometColumnarToRow [codegen id : 11] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(10) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(11) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(12) CometColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(16) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) - -(18) CometColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(19) BroadcastExchange -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#19] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(22) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#23] - -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#23] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#23] - -(25) BroadcastExchange -Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] -Join type: LeftSemi -Join condition: None - -(27) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None - -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(30) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#24] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#25, i_class_id#15 AS class_id#26, i_category_id#16 AS category_id#27] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#24] - -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#25, class_id#26, category_id#27] - -(34) CometColumnarExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(35) CometHashAggregate -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] - -(36) CometColumnarToRow [codegen id : 10] -Input [3]: [brand_id#25, class_id#26, category_id#27] - -(37) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] - -(39) Filter [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Condition : isnotnull(ws_item_sk#28) - -(40) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#28] -Right keys [1]: [i_item_sk#30] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] -Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(43) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#34] - -(44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#29] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 9] -Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] - -(46) BroadcastExchange -Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] - -(47) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] -Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] -Join type: LeftSemi -Join condition: None - -(48) BroadcastExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] - -(49) BroadcastHashJoin [codegen id : 11] -Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#25, class_id#26, category_id#27] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#35] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] - -(51) BroadcastExchange -Input [1]: [ss_item_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(52) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(53) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(54) CometFilter -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Condition : (((isnotnull(i_item_sk#36) AND isnotnull(i_brand_id#37)) AND isnotnull(i_class_id#38)) AND isnotnull(i_category_id#39)) - -(55) CometColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(56) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#35] - -(57) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#36] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(58) BroadcastExchange -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(59) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#36] -Join type: Inner -Join condition: None - -(60) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(61) ReusedExchange [Reuses operator id: 112] -Output [1]: [d_date_sk#40] - -(62) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] - -(64) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] -Results [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(65) CometColumnarExchange -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Arguments: hashpartitioning(i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(66) CometColumnarToRow [codegen id : 52] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(67) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] -Results [6]: [store AS channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51] - -(68) Filter [codegen id : 52] -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51] -Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(69) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_sold_date_sk#57 IN dynamicpruning#58)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(70) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] - -(71) Filter [codegen id : 50] -Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -Condition : isnotnull(ss_item_sk#54) - -(72) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#35] - -(73) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(74) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] - -(75) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [i_item_sk#59] -Join type: Inner -Join condition: None - -(76) Project [codegen id : 50] -Output [6]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62] -Input [8]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] - -(77) ReusedExchange [Reuses operator id: 126] -Output [1]: [d_date_sk#63] - -(78) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#57] -Right keys [1]: [d_date_sk#63] -Join type: Inner -Join condition: None - -(79) Project [codegen id : 50] -Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] -Input [7]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62, d_date_sk#63] - -(80) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)] -Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] -Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] - -(81) CometColumnarExchange -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] -Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(82) CometColumnarToRow [codegen id : 51] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] - -(83) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70, count(1)#71] -Results [6]: [store AS channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70 AS sales#73, count(1)#71 AS number_sales#74] - -(84) Filter [codegen id : 51] -Input [6]: [channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] -Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(85) BroadcastExchange -Input [6]: [channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] - -(86) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Right keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Join type: Inner -Join condition: None - -(87) TakeOrderedAndProject -Input [12]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51, channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] -Arguments: 100, [i_brand_id#37 ASC NULLS FIRST, i_class_id#38 ASC NULLS FIRST, i_category_id#39 ASC NULLS FIRST], [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51, channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* HashAggregate (107) -+- * CometColumnarToRow (106) - +- CometColumnarExchange (105) - +- * HashAggregate (104) - +- Union (103) - :- * Project (92) - : +- * BroadcastHashJoin Inner BuildRight (91) - : :- * ColumnarToRow (89) - : : +- Scan parquet spark_catalog.default.store_sales (88) - : +- ReusedExchange (90) - :- * Project (97) - : +- * BroadcastHashJoin Inner BuildRight (96) - : :- * ColumnarToRow (94) - : : +- Scan parquet spark_catalog.default.catalog_sales (93) - : +- ReusedExchange (95) - +- * Project (102) - +- * BroadcastHashJoin Inner BuildRight (101) - :- * ColumnarToRow (99) - : +- Scan parquet spark_catalog.default.web_sales (98) - +- ReusedExchange (100) - - -(88) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#12)] -ReadSchema: struct - -(89) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] - -(90) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#78] - -(91) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#77] -Right keys [1]: [d_date_sk#78] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 2] -Output [2]: [ss_quantity#75 AS quantity#79, ss_list_price#76 AS list_price#80] -Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#78] - -(93) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#12)] -ReadSchema: struct - -(94) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] - -(95) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#84] - -(96) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#83] -Right keys [1]: [d_date_sk#84] -Join type: Inner -Join condition: None - -(97) Project [codegen id : 4] -Output [2]: [cs_quantity#81 AS quantity#85, cs_list_price#82 AS list_price#86] -Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83, d_date_sk#84] - -(98) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#12)] -ReadSchema: struct - -(99) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] - -(100) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#90] - -(101) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#89] -Right keys [1]: [d_date_sk#90] -Join type: Inner -Join condition: None - -(102) Project [codegen id : 6] -Output [2]: [ws_quantity#87 AS quantity#91, ws_list_price#88 AS list_price#92] -Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89, d_date_sk#90] - -(103) Union - -(104) HashAggregate [codegen id : 7] -Input [2]: [quantity#79, list_price#80] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] -Aggregate Attributes [2]: [sum#93, count#94] -Results [2]: [sum#95, count#96] - -(105) CometColumnarExchange -Input [2]: [sum#95, count#96] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(106) CometColumnarToRow [codegen id : 8] -Input [2]: [sum#95, count#96] - -(107) HashAggregate [codegen id : 8] -Input [2]: [sum#95, count#96] -Keys: [] -Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] -Aggregate Attributes [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97] -Results [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97 AS average_sales#98] - -Subquery:2 Hosting operator id = 88 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 93 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 98 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (112) -+- * CometColumnarToRow (111) - +- CometProject (110) - +- CometFilter (109) - +- CometNativeScan parquet spark_catalog.default.date_dim (108) - - -(108) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#40, d_week_seq#99] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(109) CometFilter -Input [2]: [d_date_sk#40, d_week_seq#99] -Condition : ((isnotnull(d_week_seq#99) AND (d_week_seq#99 = Subquery scalar-subquery#100, [id=#101])) AND isnotnull(d_date_sk#40)) - -(110) CometProject -Input [2]: [d_date_sk#40, d_week_seq#99] -Arguments: [d_date_sk#40], [d_date_sk#40] - -(111) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#40] - -(112) BroadcastExchange -Input [1]: [d_date_sk#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:6 Hosting operator id = 109 Hosting Expression = Subquery scalar-subquery#100, [id=#101] -* CometColumnarToRow (116) -+- CometProject (115) - +- CometFilter (114) - +- CometNativeScan parquet spark_catalog.default.date_dim (113) - - -(113) CometNativeScan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] -ReadSchema: struct - -(114) CometFilter -Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] -Condition : (((((isnotnull(d_year#103) AND isnotnull(d_moy#104)) AND isnotnull(d_dom#105)) AND (d_year#103 = 2000)) AND (d_moy#104 = 12)) AND (d_dom#105 = 11)) - -(115) CometProject -Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] -Arguments: [d_week_seq#102], [d_week_seq#102] - -(116) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#102] - -Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometNativeScan parquet spark_catalog.default.date_dim (117) - - -(117) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#103] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#24, d_year#103] -Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1999)) AND (d_year#103 <= 2001)) AND isnotnull(d_date_sk#24)) - -(119) CometProject -Input [2]: [d_date_sk#24, d_year#103] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#24] - -(121) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:8 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 - -Subquery:10 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:11 Hosting operator id = 69 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 -BroadcastExchange (126) -+- * CometColumnarToRow (125) - +- CometProject (124) - +- CometFilter (123) - +- CometNativeScan parquet spark_catalog.default.date_dim (122) - - -(122) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#63, d_week_seq#106] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(123) CometFilter -Input [2]: [d_date_sk#63, d_week_seq#106] -Condition : ((isnotnull(d_week_seq#106) AND (d_week_seq#106 = Subquery scalar-subquery#107, [id=#108])) AND isnotnull(d_date_sk#63)) - -(124) CometProject -Input [2]: [d_date_sk#63, d_week_seq#106] -Arguments: [d_date_sk#63], [d_date_sk#63] - -(125) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#63] - -(126) BroadcastExchange -Input [1]: [d_date_sk#63] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] - -Subquery:12 Hosting operator id = 123 Hosting Expression = Subquery scalar-subquery#107, [id=#108] -* CometColumnarToRow (130) -+- CometProject (129) - +- CometFilter (128) - +- CometNativeScan parquet spark_catalog.default.date_dim (127) - - -(127) CometNativeScan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] -ReadSchema: struct - -(128) CometFilter -Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] -Condition : (((((isnotnull(d_year#103) AND isnotnull(d_moy#104)) AND isnotnull(d_dom#105)) AND (d_year#103 = 1999)) AND (d_moy#104 = 12)) AND (d_dom#105 = 11)) - -(129) CometProject -Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] -Arguments: [d_week_seq#102], [d_week_seq#102] - -(130) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#102] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/simplified.txt deleted file mode 100644 index b0eae963c3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/simplified.txt +++ /dev/null @@ -1,206 +0,0 @@ -TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (52) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Filter [sales] - Subquery #4 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #12 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (51) - Filter [sales] - ReusedSubquery [average_sales] #4 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (50) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #15 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/explain.txt deleted file mode 100644 index bcef4db073..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,743 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (85) -+- CometTakeOrderedAndProject (84) - +- CometBroadcastHashJoin (83) - :- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometBroadcastHashJoin (47) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (46) - : : : +- CometProject (45) - : : : +- CometBroadcastHashJoin (44) - : : : :- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (43) - : : : +- CometBroadcastHashJoin (42) - : : : :- CometHashAggregate (32) - : : : : +- CometExchange (31) - : : : : +- CometHashAggregate (30) - : : : : +- CometProject (29) - : : : : +- CometBroadcastHashJoin (28) - : : : : :- CometProject (26) - : : : : : +- CometBroadcastHashJoin (25) - : : : : : :- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : +- CometBroadcastExchange (24) - : : : : : +- CometBroadcastHashJoin (23) - : : : : : :- CometFilter (8) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : +- CometBroadcastExchange (22) - : : : : : +- CometProject (21) - : : : : : +- CometBroadcastHashJoin (20) - : : : : : :- CometProject (15) - : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : :- CometFilter (10) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : +- CometBroadcastExchange (13) - : : : : : : +- CometFilter (12) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : +- CometBroadcastExchange (19) - : : : : : +- CometProject (18) - : : : : : +- CometFilter (17) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : +- ReusedExchange (27) - : : : +- CometBroadcastExchange (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (34) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometFilter (49) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : +- ReusedExchange (50) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometBroadcastExchange (82) - +- CometFilter (81) - +- CometHashAggregate (80) - +- CometExchange (79) - +- CometHashAggregate (78) - +- CometProject (77) - +- CometBroadcastHashJoin (76) - :- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometBroadcastHashJoin (68) - : : :- CometFilter (66) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (65) - : : +- ReusedExchange (67) - : +- ReusedExchange (69) - +- CometBroadcastExchange (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42)) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Condition : (isnotnull(sales#51) AND (cast(sales#51 as decimal(32,6)) > cast(Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Condition : isnotnull(ss_item_sk#55) - -(67) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(68) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#55], [ss_item_sk#38], LeftSemi, BuildRight - -(69) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] - -(70) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -Arguments: [ss_item_sk#55], [i_item_sk#60], Inner, BuildRight - -(71) CometProject -Input [8]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -Arguments: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63], [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#64, d_week_seq#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#64, d_week_seq#65] -Condition : ((isnotnull(d_week_seq#65) AND (d_week_seq#65 = ReusedSubquery Subquery scalar-subquery#66, [id=#67])) AND isnotnull(d_date_sk#64)) - -(74) CometProject -Input [2]: [d_date_sk#64, d_week_seq#65] -Arguments: [d_date_sk#64], [d_date_sk#64] - -(75) CometBroadcastExchange -Input [1]: [d_date_sk#64] -Arguments: [d_date_sk#64] - -(76) CometBroadcastHashJoin -Left output [6]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63] -Right output [1]: [d_date_sk#64] -Arguments: [ss_sold_date_sk#58], [d_date_sk#64], Inner, BuildRight - -(77) CometProject -Input [7]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] -Arguments: [ss_quantity#56, ss_list_price#57, i_brand_id#61, i_class_id#62, i_category_id#63], [ss_quantity#56, ss_list_price#57, i_brand_id#61, i_class_id#62, i_category_id#63] - -(78) CometHashAggregate -Input [5]: [ss_quantity#56, ss_list_price#57, i_brand_id#61, i_class_id#62, i_category_id#63] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [partial_sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), partial_count(1)] - -(79) CometExchange -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] -Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(80) CometHashAggregate -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), count(1)] - -(81) CometFilter -Input [6]: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] -Condition : (isnotnull(sales#72) AND (cast(sales#72 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(82) CometBroadcastExchange -Input [6]: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] -Arguments: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] - -(83) CometBroadcastHashJoin -Left output [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Right output [6]: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] -Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#61, i_class_id#62, i_category_id#63], Inner, BuildRight - -(84) CometTakeOrderedAndProject -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#71,i_brand_id#61,i_class_id#62,i_category_id#63,sales#72,number_sales#73]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73], 100, 0, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] - -(85) CometColumnarToRow [codegen id : 1] -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* CometColumnarToRow (102) -+- CometHashAggregate (101) - +- CometExchange (100) - +- CometHashAggregate (99) - +- CometUnion (98) - :- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (86) - : +- ReusedExchange (87) - :- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (90) - : +- ReusedExchange (91) - +- CometProject (97) - +- CometBroadcastHashJoin (96) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (94) - +- ReusedExchange (95) - - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#74, ss_list_price#75, ss_sold_date_sk#76] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#76), dynamicpruningexpression(ss_sold_date_sk#76 IN dynamicpruning#77)] -ReadSchema: struct - -(87) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#78] - -(88) CometBroadcastHashJoin -Left output [3]: [ss_quantity#74, ss_list_price#75, ss_sold_date_sk#76] -Right output [1]: [d_date_sk#78] -Arguments: [ss_sold_date_sk#76], [d_date_sk#78], Inner, BuildRight - -(89) CometProject -Input [4]: [ss_quantity#74, ss_list_price#75, ss_sold_date_sk#76, d_date_sk#78] -Arguments: [quantity#79, list_price#80], [ss_quantity#74 AS quantity#79, ss_list_price#75 AS list_price#80] - -(90) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] -ReadSchema: struct - -(91) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#85] - -(92) CometBroadcastHashJoin -Left output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] -Right output [1]: [d_date_sk#85] -Arguments: [cs_sold_date_sk#83], [d_date_sk#85], Inner, BuildRight - -(93) CometProject -Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83, d_date_sk#85] -Arguments: [quantity#86, list_price#87], [cs_quantity#81 AS quantity#86, cs_list_price#82 AS list_price#87] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#90), dynamicpruningexpression(ws_sold_date_sk#90 IN dynamicpruning#91)] -ReadSchema: struct - -(95) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#92] - -(96) CometBroadcastHashJoin -Left output [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] -Right output [1]: [d_date_sk#92] -Arguments: [ws_sold_date_sk#90], [d_date_sk#92], Inner, BuildRight - -(97) CometProject -Input [4]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90, d_date_sk#92] -Arguments: [quantity#93, list_price#94], [ws_quantity#88 AS quantity#93, ws_list_price#89 AS list_price#94] - -(98) CometUnion -Child 0 Input [2]: [quantity#79, list_price#80] -Child 1 Input [2]: [quantity#86, list_price#87] -Child 2 Input [2]: [quantity#93, list_price#94] - -(99) CometHashAggregate -Input [2]: [quantity#79, list_price#80] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] - -(100) CometExchange -Input [2]: [sum#95, count#96] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(101) CometHashAggregate -Input [2]: [sum#95, count#96] -Keys: [] -Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] - -(102) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#97] - -Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#90 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (107) -+- * CometColumnarToRow (106) - +- CometProject (105) - +- CometFilter (104) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) - - -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(104) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(105) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(106) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(107) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) -+- CometProject (110) - +- CometFilter (109) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) - - -(108) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] -ReadSchema: struct - -(109) CometFilter -Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Condition : (((((isnotnull(d_year#99) AND isnotnull(d_moy#100)) AND isnotnull(d_dom#101)) AND (d_year#99 = 2000)) AND (d_moy#100 = 12)) AND (d_dom#101 = 11)) - -(110) CometProject -Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Arguments: [d_week_seq#98], [d_week_seq#98] - -(111) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#98] - -Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (116) -+- * CometColumnarToRow (115) - +- CometProject (114) - +- CometFilter (113) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) - - -(112) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#99] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(113) CometFilter -Input [2]: [d_date_sk#26, d_year#99] -Condition : (((isnotnull(d_year#99) AND (d_year#99 >= 1999)) AND (d_year#99 <= 2001)) AND isnotnull(d_date_sk#26)) - -(114) CometProject -Input [2]: [d_date_sk#26, d_year#99] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(115) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(116) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:8 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:10 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] - -Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] - -Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#64, d_week_seq#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#64, d_week_seq#65] -Condition : ((isnotnull(d_week_seq#65) AND (d_week_seq#65 = Subquery scalar-subquery#66, [id=#67])) AND isnotnull(d_date_sk#64)) - -(119) CometProject -Input [2]: [d_date_sk#64, d_week_seq#65] -Arguments: [d_date_sk#64], [d_date_sk#64] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#64] - -(121) BroadcastExchange -Input [1]: [d_date_sk#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#66, [id=#67] -* CometColumnarToRow (125) -+- CometProject (124) - +- CometFilter (123) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) - - -(122) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] -ReadSchema: struct - -(123) CometFilter -Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Condition : (((((isnotnull(d_year#99) AND isnotnull(d_moy#100)) AND isnotnull(d_dom#101)) AND (d_year#99 = 1999)) AND (d_moy#100 = 12)) AND (d_dom#101 = 11)) - -(124) CometProject -Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Arguments: [d_week_seq#98], [d_week_seq#98] - -(125) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#98] - -Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/simplified.txt deleted file mode 100644 index fb9abae378..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,153 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #14 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #1 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [ss_item_sk] #3 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #4 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #5 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #10 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #11 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #3 - CometBroadcastExchange [d_date_sk] #13 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #4 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - ReusedExchange [ss_item_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastExchange [d_date_sk] #18 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt deleted file mode 100644 index bcef4db073..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ /dev/null @@ -1,743 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (85) -+- CometTakeOrderedAndProject (84) - +- CometBroadcastHashJoin (83) - :- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometBroadcastHashJoin (47) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (46) - : : : +- CometProject (45) - : : : +- CometBroadcastHashJoin (44) - : : : :- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (43) - : : : +- CometBroadcastHashJoin (42) - : : : :- CometHashAggregate (32) - : : : : +- CometExchange (31) - : : : : +- CometHashAggregate (30) - : : : : +- CometProject (29) - : : : : +- CometBroadcastHashJoin (28) - : : : : :- CometProject (26) - : : : : : +- CometBroadcastHashJoin (25) - : : : : : :- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : +- CometBroadcastExchange (24) - : : : : : +- CometBroadcastHashJoin (23) - : : : : : :- CometFilter (8) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : +- CometBroadcastExchange (22) - : : : : : +- CometProject (21) - : : : : : +- CometBroadcastHashJoin (20) - : : : : : :- CometProject (15) - : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : :- CometFilter (10) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : +- CometBroadcastExchange (13) - : : : : : : +- CometFilter (12) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : +- CometBroadcastExchange (19) - : : : : : +- CometProject (18) - : : : : : +- CometFilter (17) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : +- ReusedExchange (27) - : : : +- CometBroadcastExchange (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (34) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometFilter (49) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : +- ReusedExchange (50) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometBroadcastExchange (82) - +- CometFilter (81) - +- CometHashAggregate (80) - +- CometExchange (79) - +- CometHashAggregate (78) - +- CometProject (77) - +- CometBroadcastHashJoin (76) - :- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometBroadcastHashJoin (68) - : : :- CometFilter (66) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (65) - : : +- ReusedExchange (67) - : +- ReusedExchange (69) - +- CometBroadcastExchange (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42)) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Condition : (isnotnull(sales#51) AND (cast(sales#51 as decimal(32,6)) > cast(Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Condition : isnotnull(ss_item_sk#55) - -(67) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(68) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#55], [ss_item_sk#38], LeftSemi, BuildRight - -(69) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] - -(70) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -Arguments: [ss_item_sk#55], [i_item_sk#60], Inner, BuildRight - -(71) CometProject -Input [8]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -Arguments: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63], [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#64, d_week_seq#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#64, d_week_seq#65] -Condition : ((isnotnull(d_week_seq#65) AND (d_week_seq#65 = ReusedSubquery Subquery scalar-subquery#66, [id=#67])) AND isnotnull(d_date_sk#64)) - -(74) CometProject -Input [2]: [d_date_sk#64, d_week_seq#65] -Arguments: [d_date_sk#64], [d_date_sk#64] - -(75) CometBroadcastExchange -Input [1]: [d_date_sk#64] -Arguments: [d_date_sk#64] - -(76) CometBroadcastHashJoin -Left output [6]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63] -Right output [1]: [d_date_sk#64] -Arguments: [ss_sold_date_sk#58], [d_date_sk#64], Inner, BuildRight - -(77) CometProject -Input [7]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] -Arguments: [ss_quantity#56, ss_list_price#57, i_brand_id#61, i_class_id#62, i_category_id#63], [ss_quantity#56, ss_list_price#57, i_brand_id#61, i_class_id#62, i_category_id#63] - -(78) CometHashAggregate -Input [5]: [ss_quantity#56, ss_list_price#57, i_brand_id#61, i_class_id#62, i_category_id#63] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [partial_sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), partial_count(1)] - -(79) CometExchange -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] -Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(80) CometHashAggregate -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), count(1)] - -(81) CometFilter -Input [6]: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] -Condition : (isnotnull(sales#72) AND (cast(sales#72 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(82) CometBroadcastExchange -Input [6]: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] -Arguments: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] - -(83) CometBroadcastHashJoin -Left output [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Right output [6]: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] -Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#61, i_class_id#62, i_category_id#63], Inner, BuildRight - -(84) CometTakeOrderedAndProject -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#71,i_brand_id#61,i_class_id#62,i_category_id#63,sales#72,number_sales#73]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73], 100, 0, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] - -(85) CometColumnarToRow [codegen id : 1] -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* CometColumnarToRow (102) -+- CometHashAggregate (101) - +- CometExchange (100) - +- CometHashAggregate (99) - +- CometUnion (98) - :- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (86) - : +- ReusedExchange (87) - :- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (90) - : +- ReusedExchange (91) - +- CometProject (97) - +- CometBroadcastHashJoin (96) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (94) - +- ReusedExchange (95) - - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#74, ss_list_price#75, ss_sold_date_sk#76] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#76), dynamicpruningexpression(ss_sold_date_sk#76 IN dynamicpruning#77)] -ReadSchema: struct - -(87) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#78] - -(88) CometBroadcastHashJoin -Left output [3]: [ss_quantity#74, ss_list_price#75, ss_sold_date_sk#76] -Right output [1]: [d_date_sk#78] -Arguments: [ss_sold_date_sk#76], [d_date_sk#78], Inner, BuildRight - -(89) CometProject -Input [4]: [ss_quantity#74, ss_list_price#75, ss_sold_date_sk#76, d_date_sk#78] -Arguments: [quantity#79, list_price#80], [ss_quantity#74 AS quantity#79, ss_list_price#75 AS list_price#80] - -(90) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] -ReadSchema: struct - -(91) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#85] - -(92) CometBroadcastHashJoin -Left output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] -Right output [1]: [d_date_sk#85] -Arguments: [cs_sold_date_sk#83], [d_date_sk#85], Inner, BuildRight - -(93) CometProject -Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83, d_date_sk#85] -Arguments: [quantity#86, list_price#87], [cs_quantity#81 AS quantity#86, cs_list_price#82 AS list_price#87] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#90), dynamicpruningexpression(ws_sold_date_sk#90 IN dynamicpruning#91)] -ReadSchema: struct - -(95) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#92] - -(96) CometBroadcastHashJoin -Left output [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] -Right output [1]: [d_date_sk#92] -Arguments: [ws_sold_date_sk#90], [d_date_sk#92], Inner, BuildRight - -(97) CometProject -Input [4]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90, d_date_sk#92] -Arguments: [quantity#93, list_price#94], [ws_quantity#88 AS quantity#93, ws_list_price#89 AS list_price#94] - -(98) CometUnion -Child 0 Input [2]: [quantity#79, list_price#80] -Child 1 Input [2]: [quantity#86, list_price#87] -Child 2 Input [2]: [quantity#93, list_price#94] - -(99) CometHashAggregate -Input [2]: [quantity#79, list_price#80] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] - -(100) CometExchange -Input [2]: [sum#95, count#96] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(101) CometHashAggregate -Input [2]: [sum#95, count#96] -Keys: [] -Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] - -(102) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#97] - -Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#90 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (107) -+- * CometColumnarToRow (106) - +- CometProject (105) - +- CometFilter (104) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) - - -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(104) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(105) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(106) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(107) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) -+- CometProject (110) - +- CometFilter (109) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) - - -(108) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] -ReadSchema: struct - -(109) CometFilter -Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Condition : (((((isnotnull(d_year#99) AND isnotnull(d_moy#100)) AND isnotnull(d_dom#101)) AND (d_year#99 = 2000)) AND (d_moy#100 = 12)) AND (d_dom#101 = 11)) - -(110) CometProject -Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Arguments: [d_week_seq#98], [d_week_seq#98] - -(111) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#98] - -Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (116) -+- * CometColumnarToRow (115) - +- CometProject (114) - +- CometFilter (113) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) - - -(112) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#99] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(113) CometFilter -Input [2]: [d_date_sk#26, d_year#99] -Condition : (((isnotnull(d_year#99) AND (d_year#99 >= 1999)) AND (d_year#99 <= 2001)) AND isnotnull(d_date_sk#26)) - -(114) CometProject -Input [2]: [d_date_sk#26, d_year#99] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(115) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(116) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:8 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:10 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] - -Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] - -Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#64, d_week_seq#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#64, d_week_seq#65] -Condition : ((isnotnull(d_week_seq#65) AND (d_week_seq#65 = Subquery scalar-subquery#66, [id=#67])) AND isnotnull(d_date_sk#64)) - -(119) CometProject -Input [2]: [d_date_sk#64, d_week_seq#65] -Arguments: [d_date_sk#64], [d_date_sk#64] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#64] - -(121) BroadcastExchange -Input [1]: [d_date_sk#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#66, [id=#67] -* CometColumnarToRow (125) -+- CometProject (124) - +- CometFilter (123) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) - - -(122) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] -ReadSchema: struct - -(123) CometFilter -Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Condition : (((((isnotnull(d_year#99) AND isnotnull(d_moy#100)) AND isnotnull(d_dom#101)) AND (d_year#99 = 1999)) AND (d_moy#100 = 12)) AND (d_dom#101 = 11)) - -(124) CometProject -Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Arguments: [d_week_seq#98], [d_week_seq#98] - -(125) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#98] - -Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/extended.txt deleted file mode 100644 index f56d229b68..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/extended.txt +++ /dev/null @@ -1,339 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometBroadcastHashJoin - :- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 298 out of 327 eligible operators (91%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index fb9abae378..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ /dev/null @@ -1,153 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #14 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #1 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [ss_item_sk] #3 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #4 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #5 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #10 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #11 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #3 - CometBroadcastExchange [d_date_sk] #13 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #4 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - ReusedExchange [ss_item_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastExchange [d_date_sk] #18 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/explain.txt deleted file mode 100644 index 594939616e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/explain.txt +++ /dev/null @@ -1,173 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (24) -+- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometColumnarExchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : +- BroadcastExchange (7) - : : +- * CometColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometNativeScan parquet spark_catalog.default.customer (4) - : +- BroadcastExchange (14) - : +- * CometColumnarToRow (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometNativeScan parquet spark_catalog.default.customer_address (10) - +- ReusedExchange (17) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] - -(3) Filter [codegen id : 4] -Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_bill_customer_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#5, c_current_addr_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] -Condition : (isnotnull(c_customer_sk#5) AND isnotnull(c_current_addr_sk#6)) - -(6) CometColumnarToRow [codegen id : 1] -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] - -(7) BroadcastExchange -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#5] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 4] -Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] -Input [5]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3, c_customer_sk#5, c_current_addr_sk#6] - -(10) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Condition : isnotnull(ca_address_sk#7) - -(12) CometProject -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Arguments: [ca_address_sk#7, ca_state#10, ca_zip#11], [ca_address_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#8, 2, true, false, true) AS ca_state#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#9, 10, true, false, true) AS ca_zip#11] - -(13) CometColumnarToRow [codegen id : 2] -Input [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] - -(14) BroadcastExchange -Input [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [c_current_addr_sk#6] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: ((substr(ca_zip#11, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#10 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)) - -(16) Project [codegen id : 4] -Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11] -Input [6]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, ca_address_sk#7, ca_state#10, ca_zip#11] - -(17) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#12] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [2]: [cs_sales_price#2, ca_zip#11] -Input [4]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11, d_date_sk#12] - -(20) HashAggregate [codegen id : 4] -Input [2]: [cs_sales_price#2, ca_zip#11] -Keys [1]: [ca_zip#11] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum#13] -Results [2]: [ca_zip#11, sum#14] - -(21) CometColumnarExchange -Input [2]: [ca_zip#11, sum#14] -Arguments: hashpartitioning(ca_zip#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [2]: [ca_zip#11, sum#14] - -(23) HashAggregate [codegen id : 5] -Input [2]: [ca_zip#11, sum#14] -Keys [1]: [ca_zip#11] -Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#15] -Results [2]: [ca_zip#11, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#15,17,2) AS sum(cs_sales_price)#16] - -(24) TakeOrderedAndProject -Input [2]: [ca_zip#11, sum(cs_sales_price)#16] -Arguments: 100, [ca_zip#11 ASC NULLS FIRST], [ca_zip#11, sum(cs_sales_price)#16] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometNativeScan parquet spark_catalog.default.date_dim (25) - - -(25) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#12, d_year#17, d_qoy#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [3]: [d_date_sk#12, d_year#17, d_qoy#18] -Condition : ((((isnotnull(d_qoy#18) AND isnotnull(d_year#17)) AND (d_qoy#18 = 2)) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#12)) - -(27) CometProject -Input [3]: [d_date_sk#12, d_year#17, d_qoy#18] -Arguments: [d_date_sk#12], [d_date_sk#12] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#12] - -(29) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/simplified.txt deleted file mode 100644 index f31442dcfe..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - WholeStageCodegen (5) - HashAggregate [ca_zip,sum] [sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price),sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_zip] #1 - WholeStageCodegen (4) - HashAggregate [ca_zip,cs_sales_price] [sum,sum] - Project [cs_sales_price,ca_zip] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sales_price,cs_sold_date_sk,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] - Project [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip] - CometFilter [ca_address_sk,ca_state,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - InputAdapter - ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/explain.txt deleted file mode 100644 index af9b2efbd1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,173 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (13) - : +- CometBroadcastHashJoin (12) - : :- CometProject (7) - : : +- CometBroadcastHashJoin (6) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : +- CometBroadcastExchange (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (3) - : +- CometBroadcastExchange (11) - : +- CometProject (10) - : +- CometFilter (9) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - +- CometBroadcastExchange (17) - +- CometProject (16) - +- CometFilter (15) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_bill_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#5, c_current_addr_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] -Condition : (isnotnull(c_customer_sk#5) AND isnotnull(c_current_addr_sk#6)) - -(5) CometBroadcastExchange -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] -Arguments: [c_customer_sk#5, c_current_addr_sk#6] - -(6) CometBroadcastHashJoin -Left output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Right output [2]: [c_customer_sk#5, c_current_addr_sk#6] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#5], Inner, BuildRight - -(7) CometProject -Input [5]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3, c_customer_sk#5, c_current_addr_sk#6] -Arguments: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6], [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Condition : isnotnull(ca_address_sk#7) - -(10) CometProject -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Arguments: [ca_address_sk#7, ca_state#10, ca_zip#11], [ca_address_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#8, 2, true, false, true) AS ca_state#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#9, 10, true, false, true) AS ca_zip#11] - -(11) CometBroadcastExchange -Input [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] -Arguments: [ca_address_sk#7, ca_state#10, ca_zip#11] - -(12) CometBroadcastHashJoin -Left output [3]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] -Right output [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] -Arguments: [c_current_addr_sk#6], [ca_address_sk#7], Inner, ((substr(ca_zip#11, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#10 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)), BuildRight - -(13) CometProject -Input [6]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, ca_address_sk#7, ca_state#10, ca_zip#11] -Arguments: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11], [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2)) AND (d_year#13 = 2001)) AND isnotnull(d_date_sk#12)) - -(16) CometProject -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Arguments: [d_date_sk#12], [d_date_sk#12] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: [d_date_sk#12] - -(18) CometBroadcastHashJoin -Left output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11] -Right output [1]: [d_date_sk#12] -Arguments: [cs_sold_date_sk#3], [d_date_sk#12], Inner, BuildRight - -(19) CometProject -Input [4]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11, d_date_sk#12] -Arguments: [cs_sales_price#2, ca_zip#11], [cs_sales_price#2, ca_zip#11] - -(20) CometHashAggregate -Input [2]: [cs_sales_price#2, ca_zip#11] -Keys [1]: [ca_zip#11] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] - -(21) CometExchange -Input [2]: [ca_zip#11, sum#15] -Arguments: hashpartitioning(ca_zip#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [2]: [ca_zip#11, sum#15] -Keys [1]: [ca_zip#11] -Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] - -(23) CometTakeOrderedAndProject -Input [2]: [ca_zip#11, sum(cs_sales_price)#16] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_zip#11 ASC NULLS FIRST], output=[ca_zip#11,sum(cs_sales_price)#16]), [ca_zip#11, sum(cs_sales_price)#16], 100, 0, [ca_zip#11 ASC NULLS FIRST], [ca_zip#11, sum(cs_sales_price)#16] - -(24) CometColumnarToRow [codegen id : 1] -Input [2]: [ca_zip#11, sum(cs_sales_price)#16] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2)) AND (d_year#13 = 2001)) AND isnotnull(d_date_sk#12)) - -(27) CometProject -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Arguments: [d_date_sk#12], [d_date_sk#12] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#12] - -(29) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/simplified.txt deleted file mode 100644 index c39b96efe3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,34 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - CometHashAggregate [sum] [ca_zip,sum(cs_sales_price),sum(UnscaledValue(cs_sales_price))] - CometExchange [ca_zip] #1 - CometHashAggregate [cs_sales_price] [ca_zip,sum] - CometProject [cs_sales_price,ca_zip] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] - CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip] - CometFilter [ca_address_sk,ca_state,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] 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/q15/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt deleted file mode 100644 index af9b2efbd1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt +++ /dev/null @@ -1,173 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (13) - : +- CometBroadcastHashJoin (12) - : :- CometProject (7) - : : +- CometBroadcastHashJoin (6) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : +- CometBroadcastExchange (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (3) - : +- CometBroadcastExchange (11) - : +- CometProject (10) - : +- CometFilter (9) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - +- CometBroadcastExchange (17) - +- CometProject (16) - +- CometFilter (15) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_bill_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#5, c_current_addr_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] -Condition : (isnotnull(c_customer_sk#5) AND isnotnull(c_current_addr_sk#6)) - -(5) CometBroadcastExchange -Input [2]: [c_customer_sk#5, c_current_addr_sk#6] -Arguments: [c_customer_sk#5, c_current_addr_sk#6] - -(6) CometBroadcastHashJoin -Left output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] -Right output [2]: [c_customer_sk#5, c_current_addr_sk#6] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#5], Inner, BuildRight - -(7) CometProject -Input [5]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3, c_customer_sk#5, c_current_addr_sk#6] -Arguments: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6], [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Condition : isnotnull(ca_address_sk#7) - -(10) CometProject -Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] -Arguments: [ca_address_sk#7, ca_state#10, ca_zip#11], [ca_address_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#8, 2, true, false, true) AS ca_state#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#9, 10, true, false, true) AS ca_zip#11] - -(11) CometBroadcastExchange -Input [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] -Arguments: [ca_address_sk#7, ca_state#10, ca_zip#11] - -(12) CometBroadcastHashJoin -Left output [3]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] -Right output [3]: [ca_address_sk#7, ca_state#10, ca_zip#11] -Arguments: [c_current_addr_sk#6], [ca_address_sk#7], Inner, ((substr(ca_zip#11, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#10 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)), BuildRight - -(13) CometProject -Input [6]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, ca_address_sk#7, ca_state#10, ca_zip#11] -Arguments: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11], [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2)) AND (d_year#13 = 2001)) AND isnotnull(d_date_sk#12)) - -(16) CometProject -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Arguments: [d_date_sk#12], [d_date_sk#12] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: [d_date_sk#12] - -(18) CometBroadcastHashJoin -Left output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11] -Right output [1]: [d_date_sk#12] -Arguments: [cs_sold_date_sk#3], [d_date_sk#12], Inner, BuildRight - -(19) CometProject -Input [4]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#11, d_date_sk#12] -Arguments: [cs_sales_price#2, ca_zip#11], [cs_sales_price#2, ca_zip#11] - -(20) CometHashAggregate -Input [2]: [cs_sales_price#2, ca_zip#11] -Keys [1]: [ca_zip#11] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] - -(21) CometExchange -Input [2]: [ca_zip#11, sum#15] -Arguments: hashpartitioning(ca_zip#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [2]: [ca_zip#11, sum#15] -Keys [1]: [ca_zip#11] -Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] - -(23) CometTakeOrderedAndProject -Input [2]: [ca_zip#11, sum(cs_sales_price)#16] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_zip#11 ASC NULLS FIRST], output=[ca_zip#11,sum(cs_sales_price)#16]), [ca_zip#11, sum(cs_sales_price)#16], 100, 0, [ca_zip#11 ASC NULLS FIRST], [ca_zip#11, sum(cs_sales_price)#16] - -(24) CometColumnarToRow [codegen id : 1] -Input [2]: [ca_zip#11, sum(cs_sales_price)#16] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Condition : ((((isnotnull(d_qoy#14) AND isnotnull(d_year#13)) AND (d_qoy#14 = 2)) AND (d_year#13 = 2001)) AND isnotnull(d_date_sk#12)) - -(27) CometProject -Input [3]: [d_date_sk#12, d_year#13, d_qoy#14] -Arguments: [d_date_sk#12], [d_date_sk#12] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#12] - -(29) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/extended.txt deleted file mode 100644 index 6de0c64850..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/extended.txt +++ /dev/null @@ -1,32 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index c39b96efe3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/simplified.txt +++ /dev/null @@ -1,34 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - CometHashAggregate [sum] [ca_zip,sum(cs_sales_price),sum(UnscaledValue(cs_sales_price))] - CometExchange [ca_zip] #1 - CometHashAggregate [cs_sales_price] [ca_zip,sum] - CometProject [cs_sales_price,ca_zip] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,ca_zip,d_date_sk] - CometProject [cs_sales_price,cs_sold_date_sk,ca_zip] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_state,ca_zip] #4 - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip] - CometFilter [ca_address_sk,ca_state,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_datafusion/explain.txt deleted file mode 100644 index 2a3c8932ec..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_datafusion/explain.txt +++ /dev/null @@ -1,235 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometColumnarExchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * CometColumnarToRow (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometNativeScan parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometNativeScan parquet spark_catalog.default.call_center (29) - - -(1) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) - -(3) CometProject -Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(4) CometExchange -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -ReadSchema: struct - -(7) CometProject -Input [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] -Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_warehouse_sk#9, cs_order_number#10] - -(8) CometExchange -Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_order_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_order_number#5], [cs_order_number#10], LeftSemi, NOT (cs_warehouse_sk#4 = cs_warehouse_sk#9) - -(11) CometProject -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(12) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [2]: [cr_order_number#12, cr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -ReadSchema: struct - -(13) CometProject -Input [2]: [cr_order_number#12, cr_returned_date_sk#13] -Arguments: [cr_order_number#12], [cr_order_number#12] - -(14) CometExchange -Input [1]: [cr_order_number#12] -Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(15) CometSort -Input [1]: [cr_order_number#12] -Arguments: [cr_order_number#12], [cr_order_number#12 ASC NULLS FIRST] - -(16) CometSortMergeJoin -Left output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [cr_order_number#12] -Arguments: [cs_order_number#5], [cr_order_number#12], LeftAnti - -(17) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2002-02-01)) AND (d_date#15 <= 2002-04-02)) AND isnotnull(d_date_sk#14)) - -(19) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(20) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(21) CometBroadcastHashJoin -Left output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [d_date_sk#14] -Arguments: [cs_ship_date_sk#1], [d_date_sk#14], Inner, BuildRight - -(22) CometProject -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#14] -Arguments: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(23) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#17, 2, true, false, true) = GA) AND isnotnull(ca_address_sk#16)) - -(25) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(26) CometBroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: [ca_address_sk#16] - -(27) CometBroadcastHashJoin -Left output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [ca_address_sk#16] -Arguments: [cs_ship_addr_sk#2], [ca_address_sk#16], Inner, BuildRight - -(28) CometProject -Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] -Arguments: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(29) CometNativeScan parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#18, cc_county#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [cc_call_center_sk#18, cc_county#19] -Condition : ((isnotnull(cc_county#19) AND (cc_county#19 = Williamson County)) AND isnotnull(cc_call_center_sk#18)) - -(31) CometProject -Input [2]: [cc_call_center_sk#18, cc_county#19] -Arguments: [cc_call_center_sk#18], [cc_call_center_sk#18] - -(32) CometBroadcastExchange -Input [1]: [cc_call_center_sk#18] -Arguments: [cc_call_center_sk#18] - -(33) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [cc_call_center_sk#18] -Arguments: [cs_call_center_sk#3], [cc_call_center_sk#18], Inner, BuildRight - -(34) CometProject -Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#18] -Arguments: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(35) CometHashAggregate -Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Keys [1]: [cs_order_number#5] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] - -(36) CometColumnarToRow [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] - -(37) HashAggregate [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] -Keys [1]: [cs_order_number#5] -Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23] -Results [3]: [cs_order_number#5, sum#20, sum#21] - -(38) HashAggregate [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23, count(cs_order_number#5)#24] -Results [3]: [sum#20, sum#21, count#25] - -(39) CometColumnarExchange -Input [3]: [sum#20, sum#21, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(40) CometHashAggregate -Input [3]: [sum#20, sum#21, count#25] -Keys: [] -Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] - -(41) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #26, total shipping cost #27, total net profit #28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_datafusion/simplified.txt deleted file mode 100644 index 429d83d08c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_datafusion/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(cs_order_number),sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [cs_ext_ship_cost,cs_net_profit] [cs_order_number,sum,sum] - CometProject [cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cc_call_center_sk] - CometProject [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,ca_address_sk] - CometProject [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,d_date_sk] - CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cr_order_number] - CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometSortMergeJoin [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_order_number,cs_warehouse_sk] - CometSort [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] - CometExchange [cs_order_number] #2 - 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,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - CometNativeScan 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] - CometSort [cs_warehouse_sk,cs_order_number] - CometExchange [cs_order_number] #3 - CometProject [cs_warehouse_sk,cs_order_number] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] - CometSort [cr_order_number] - CometExchange [cr_order_number] #4 - CometProject [cr_order_number] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cc_call_center_sk] #7 - CometProject [cc_call_center_sk] - CometFilter [cc_call_center_sk,cc_county] - CometNativeScan 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/q16.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_iceberg_compat/explain.txt deleted file mode 100644 index f2473f1f01..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,235 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometColumnarExchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * CometColumnarToRow (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (29) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) - -(3) CometProject -Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(4) CometExchange -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -ReadSchema: struct - -(7) CometProject -Input [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] -Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_warehouse_sk#9, cs_order_number#10] - -(8) CometExchange -Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_order_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_order_number#5], [cs_order_number#10], LeftSemi, NOT (cs_warehouse_sk#4 = cs_warehouse_sk#9) - -(11) CometProject -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [2]: [cr_order_number#12, cr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -ReadSchema: struct - -(13) CometProject -Input [2]: [cr_order_number#12, cr_returned_date_sk#13] -Arguments: [cr_order_number#12], [cr_order_number#12] - -(14) CometExchange -Input [1]: [cr_order_number#12] -Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(15) CometSort -Input [1]: [cr_order_number#12] -Arguments: [cr_order_number#12], [cr_order_number#12 ASC NULLS FIRST] - -(16) CometSortMergeJoin -Left output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [cr_order_number#12] -Arguments: [cs_order_number#5], [cr_order_number#12], LeftAnti - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2002-02-01)) AND (d_date#15 <= 2002-04-02)) AND isnotnull(d_date_sk#14)) - -(19) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(20) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(21) CometBroadcastHashJoin -Left output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [d_date_sk#14] -Arguments: [cs_ship_date_sk#1], [d_date_sk#14], Inner, BuildRight - -(22) CometProject -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#14] -Arguments: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#17, 2, true, false, true) = GA) AND isnotnull(ca_address_sk#16)) - -(25) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(26) CometBroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: [ca_address_sk#16] - -(27) CometBroadcastHashJoin -Left output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [ca_address_sk#16] -Arguments: [cs_ship_addr_sk#2], [ca_address_sk#16], Inner, BuildRight - -(28) CometProject -Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] -Arguments: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#18, cc_county#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [cc_call_center_sk#18, cc_county#19] -Condition : ((isnotnull(cc_county#19) AND (cc_county#19 = Williamson County)) AND isnotnull(cc_call_center_sk#18)) - -(31) CometProject -Input [2]: [cc_call_center_sk#18, cc_county#19] -Arguments: [cc_call_center_sk#18], [cc_call_center_sk#18] - -(32) CometBroadcastExchange -Input [1]: [cc_call_center_sk#18] -Arguments: [cc_call_center_sk#18] - -(33) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [cc_call_center_sk#18] -Arguments: [cs_call_center_sk#3], [cc_call_center_sk#18], Inner, BuildRight - -(34) CometProject -Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#18] -Arguments: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(35) CometHashAggregate -Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Keys [1]: [cs_order_number#5] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] - -(36) CometColumnarToRow [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] - -(37) HashAggregate [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] -Keys [1]: [cs_order_number#5] -Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23] -Results [3]: [cs_order_number#5, sum#20, sum#21] - -(38) HashAggregate [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23, count(cs_order_number#5)#24] -Results [3]: [sum#20, sum#21, count#25] - -(39) CometColumnarExchange -Input [3]: [sum#20, sum#21, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(40) CometHashAggregate -Input [3]: [sum#20, sum#21, count#25] -Keys: [] -Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] - -(41) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #26, total shipping cost #27, total net profit #28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_iceberg_compat/simplified.txt deleted file mode 100644 index 8427aa49a9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(cs_order_number),sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [cs_ext_ship_cost,cs_net_profit] [cs_order_number,sum,sum] - CometProject [cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cc_call_center_sk] - CometProject [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,ca_address_sk] - CometProject [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,d_date_sk] - CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cr_order_number] - CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometSortMergeJoin [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_order_number,cs_warehouse_sk] - CometSort [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] - CometExchange [cs_order_number] #2 - 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,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [cs_warehouse_sk,cs_order_number] - CometExchange [cs_order_number] #3 - CometProject [cs_warehouse_sk,cs_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] - CometSort [cr_order_number] - CometExchange [cr_order_number] #4 - CometProject [cr_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cc_call_center_sk] #7 - CometProject [cc_call_center_sk] - CometFilter [cc_call_center_sk,cc_county] - CometScan [native_iceberg_compat] 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/q16/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt deleted file mode 100644 index f2473f1f01..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ /dev/null @@ -1,235 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometColumnarExchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * CometColumnarToRow (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (29) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) - -(3) CometProject -Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(4) CometExchange -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -ReadSchema: struct - -(7) CometProject -Input [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] -Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_warehouse_sk#9, cs_order_number#10] - -(8) CometExchange -Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_order_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [2]: [cs_warehouse_sk#9, cs_order_number#10] -Arguments: [cs_order_number#5], [cs_order_number#10], LeftSemi, NOT (cs_warehouse_sk#4 = cs_warehouse_sk#9) - -(11) CometProject -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [2]: [cr_order_number#12, cr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -ReadSchema: struct - -(13) CometProject -Input [2]: [cr_order_number#12, cr_returned_date_sk#13] -Arguments: [cr_order_number#12], [cr_order_number#12] - -(14) CometExchange -Input [1]: [cr_order_number#12] -Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(15) CometSort -Input [1]: [cr_order_number#12] -Arguments: [cr_order_number#12], [cr_order_number#12 ASC NULLS FIRST] - -(16) CometSortMergeJoin -Left output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [cr_order_number#12] -Arguments: [cs_order_number#5], [cr_order_number#12], LeftAnti - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2002-02-01)) AND (d_date#15 <= 2002-04-02)) AND isnotnull(d_date_sk#14)) - -(19) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(20) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(21) CometBroadcastHashJoin -Left output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [d_date_sk#14] -Arguments: [cs_ship_date_sk#1], [d_date_sk#14], Inner, BuildRight - -(22) CometProject -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#14] -Arguments: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#17, 2, true, false, true) = GA) AND isnotnull(ca_address_sk#16)) - -(25) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(26) CometBroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: [ca_address_sk#16] - -(27) CometBroadcastHashJoin -Left output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [ca_address_sk#16] -Arguments: [cs_ship_addr_sk#2], [ca_address_sk#16], Inner, BuildRight - -(28) CometProject -Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] -Arguments: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#18, cc_county#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [cc_call_center_sk#18, cc_county#19] -Condition : ((isnotnull(cc_county#19) AND (cc_county#19 = Williamson County)) AND isnotnull(cc_call_center_sk#18)) - -(31) CometProject -Input [2]: [cc_call_center_sk#18, cc_county#19] -Arguments: [cc_call_center_sk#18], [cc_call_center_sk#18] - -(32) CometBroadcastExchange -Input [1]: [cc_call_center_sk#18] -Arguments: [cc_call_center_sk#18] - -(33) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Right output [1]: [cc_call_center_sk#18] -Arguments: [cs_call_center_sk#3], [cc_call_center_sk#18], Inner, BuildRight - -(34) CometProject -Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#18] -Arguments: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] - -(35) CometHashAggregate -Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Keys [1]: [cs_order_number#5] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] - -(36) CometColumnarToRow [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] - -(37) HashAggregate [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] -Keys [1]: [cs_order_number#5] -Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23] -Results [3]: [cs_order_number#5, sum#20, sum#21] - -(38) HashAggregate [codegen id : 1] -Input [3]: [cs_order_number#5, sum#20, sum#21] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23, count(cs_order_number#5)#24] -Results [3]: [sum#20, sum#21, count#25] - -(39) CometColumnarExchange -Input [3]: [sum#20, sum#21, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(40) CometHashAggregate -Input [3]: [sum#20, sum#21, count#25] -Keys: [] -Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] - -(41) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #26, total shipping cost #27, total net profit #28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/extended.txt deleted file mode 100644 index 2ad029e444..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/extended.txt +++ /dev/null @@ -1,43 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - -Comet accelerated 37 out of 39 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 8427aa49a9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(cs_order_number),sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] - HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [cs_ext_ship_cost,cs_net_profit] [cs_order_number,sum,sum] - CometProject [cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cc_call_center_sk] - CometProject [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,ca_address_sk] - CometProject [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometBroadcastHashJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,d_date_sk] - CometSortMergeJoin [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cr_order_number] - CometProject [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] - CometSortMergeJoin [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_order_number,cs_warehouse_sk] - CometSort [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] - CometExchange [cs_order_number] #2 - 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,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [cs_warehouse_sk,cs_order_number] - CometExchange [cs_order_number] #3 - CometProject [cs_warehouse_sk,cs_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] - CometSort [cr_order_number] - CometExchange [cr_order_number] #4 - CometProject [cr_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cc_call_center_sk] #7 - CometProject [cc_call_center_sk] - CometFilter [cc_call_center_sk,cc_county] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/explain.txt deleted file mode 100644 index 5005b676fc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/explain.txt +++ /dev/null @@ -1,312 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (18) - : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet spark_catalog.default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * Filter (12) - : : : : : +- * ColumnarToRow (11) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (10) - : : : : +- ReusedExchange (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.store (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.item (32) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 8] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] - -(3) Filter [codegen id : 8] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(4) Scan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(6) Filter [codegen id : 1] -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(7) BroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 8] -Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] -Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(10) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(12) Filter [codegen id : 2] -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(13) BroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] -Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(16) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#18] - -(17) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#18] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 8] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#18] - -(19) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#19] - -(20) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [sr_returned_date_sk#12] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] - -(22) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#20] - -(23) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 8] -Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#20] - -(25) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#21, s_state#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [s_store_sk#21, s_state#22] -Condition : isnotnull(s_store_sk#21) - -(27) CometProject -Input [2]: [s_store_sk#21, s_state#22] -Arguments: [s_store_sk#21, s_state#23], [s_store_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#22, 2, true, false, true) AS s_state#23] - -(28) CometColumnarToRow [codegen id : 6] -Input [2]: [s_store_sk#21, s_state#23] - -(29) BroadcastExchange -Input [2]: [s_store_sk#21, s_state#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(30) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#21] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 8] -Output [5]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23] -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#21, s_state#23] - -(32) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] -Condition : isnotnull(i_item_sk#24) - -(34) CometProject -Input [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] -Arguments: [i_item_sk#24, i_item_id#27, i_item_desc#26], [i_item_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#25, 16, true, false, true) AS i_item_id#27, i_item_desc#26] - -(35) CometColumnarToRow [codegen id : 7] -Input [3]: [i_item_sk#24, i_item_id#27, i_item_desc#26] - -(36) BroadcastExchange -Input [3]: [i_item_sk#24, i_item_id#27, i_item_desc#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(37) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#24] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 8] -Output [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_id#27, i_item_desc#26] -Input [8]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_sk#24, i_item_id#27, i_item_desc#26] - -(39) HashAggregate [codegen id : 8] -Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_id#27, i_item_desc#26] -Keys [3]: [i_item_id#27, i_item_desc#26, s_state#23] -Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partial_stddev_samp(cast(ss_quantity#5 as double)), partial_count(sr_return_quantity#11), partial_avg(sr_return_quantity#11), partial_stddev_samp(cast(sr_return_quantity#11 as double)), partial_count(cs_quantity#16), partial_avg(cs_quantity#16), partial_stddev_samp(cast(cs_quantity#16 as double))] -Aggregate Attributes [18]: [count#28, sum#29, count#30, n#31, avg#32, m2#33, count#34, sum#35, count#36, n#37, avg#38, m2#39, count#40, sum#41, count#42, n#43, avg#44, m2#45] -Results [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count#48, n#49, avg#50, m2#51, count#52, sum#53, count#54, n#55, avg#56, m2#57, count#58, sum#59, count#60, n#61, avg#62, m2#63] - -(40) CometColumnarExchange -Input [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count#48, n#49, avg#50, m2#51, count#52, sum#53, count#54, n#55, avg#56, m2#57, count#58, sum#59, count#60, n#61, avg#62, m2#63] -Arguments: hashpartitioning(i_item_id#27, i_item_desc#26, s_state#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(41) CometColumnarToRow [codegen id : 9] -Input [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count#48, n#49, avg#50, m2#51, count#52, sum#53, count#54, n#55, avg#56, m2#57, count#58, sum#59, count#60, n#61, avg#62, m2#63] - -(42) HashAggregate [codegen id : 9] -Input [21]: [i_item_id#27, i_item_desc#26, s_state#23, count#46, sum#47, count#48, n#49, avg#50, m2#51, count#52, sum#53, count#54, n#55, avg#56, m2#57, count#58, sum#59, count#60, n#61, avg#62, m2#63] -Keys [3]: [i_item_id#27, i_item_desc#26, s_state#23] -Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#11), avg(sr_return_quantity#11), stddev_samp(cast(sr_return_quantity#11 as double)), count(cs_quantity#16), avg(cs_quantity#16), stddev_samp(cast(cs_quantity#16 as double))] -Aggregate Attributes [9]: [count(ss_quantity#5)#64, avg(ss_quantity#5)#65, stddev_samp(cast(ss_quantity#5 as double))#66, count(sr_return_quantity#11)#67, avg(sr_return_quantity#11)#68, stddev_samp(cast(sr_return_quantity#11 as double))#69, count(cs_quantity#16)#70, avg(cs_quantity#16)#71, stddev_samp(cast(cs_quantity#16 as double))#72] -Results [15]: [i_item_id#27, i_item_desc#26, s_state#23, count(ss_quantity#5)#64 AS store_sales_quantitycount#73, avg(ss_quantity#5)#65 AS store_sales_quantityave#74, stddev_samp(cast(ss_quantity#5 as double))#66 AS store_sales_quantitystdev#75, (stddev_samp(cast(ss_quantity#5 as double))#66 / avg(ss_quantity#5)#65) AS store_sales_quantitycov#76, count(sr_return_quantity#11)#67 AS as_store_returns_quantitycount#77, avg(sr_return_quantity#11)#68 AS as_store_returns_quantityave#78, stddev_samp(cast(sr_return_quantity#11 as double))#69 AS as_store_returns_quantitystdev#79, (stddev_samp(cast(sr_return_quantity#11 as double))#69 / avg(sr_return_quantity#11)#68) AS store_returns_quantitycov#80, count(cs_quantity#16)#70 AS catalog_sales_quantitycount#81, avg(cs_quantity#16)#71 AS catalog_sales_quantityave#82, (stddev_samp(cast(cs_quantity#16 as double))#72 / avg(cs_quantity#16)#71) AS catalog_sales_quantitystdev#83, (stddev_samp(cast(cs_quantity#16 as double))#72 / avg(cs_quantity#16)#71) AS catalog_sales_quantitycov#84] - -(43) TakeOrderedAndProject -Input [15]: [i_item_id#27, i_item_desc#26, s_state#23, store_sales_quantitycount#73, store_sales_quantityave#74, store_sales_quantitystdev#75, store_sales_quantitycov#76, as_store_returns_quantitycount#77, as_store_returns_quantityave#78, as_store_returns_quantitystdev#79, store_returns_quantitycov#80, catalog_sales_quantitycount#81, catalog_sales_quantityave#82, catalog_sales_quantitystdev#83, catalog_sales_quantitycov#84] -Arguments: 100, [i_item_id#27 ASC NULLS FIRST, i_item_desc#26 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#27, i_item_desc#26, s_state#23, store_sales_quantitycount#73, store_sales_quantityave#74, store_sales_quantitystdev#75, store_sales_quantitycov#76, as_store_returns_quantitycount#77, as_store_returns_quantityave#78, as_store_returns_quantitystdev#79, store_returns_quantitycov#80, catalog_sales_quantitycount#81, catalog_sales_quantityave#82, catalog_sales_quantitystdev#83, catalog_sales_quantitycov#84] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometNativeScan parquet spark_catalog.default.date_dim (44) - - -(44) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_quarter_name#85] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [2]: [d_date_sk#18, d_quarter_name#85] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_quarter_name#85, 6, true, false, true) = 2001Q1) AND isnotnull(d_date_sk#18)) - -(46) CometProject -Input [2]: [d_date_sk#18, d_quarter_name#85] -Arguments: [d_date_sk#18], [d_date_sk#18] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#18] - -(48) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometNativeScan parquet spark_catalog.default.date_dim (49) - - -(49) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_quarter_name#86] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [2]: [d_date_sk#19, d_quarter_name#86] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_quarter_name#86, 6, true, false, true) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#19)) - -(51) CometProject -Input [2]: [d_date_sk#19, d_quarter_name#86] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(53) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/simplified.txt deleted file mode 100644 index c63dd716a1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/simplified.txt +++ /dev/null @@ -1,79 +0,0 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] - WholeStageCodegen (9) - HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(ss_quantity),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(sr_return_quantity),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cs_quantity),stddev_samp(cast(cs_quantity as double)),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,s_state] #1 - WholeStageCodegen (8) - HashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] - Project [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [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] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Filter [cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometNativeScan 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/q17.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/explain.txt deleted file mode 100644 index f502a5cd8d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,319 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometProject (18) - : : : : +- CometBroadcastHashJoin (17) - : : : : :- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (8) - : : : : +- CometBroadcastExchange (16) - : : : : +- CometProject (15) - : : : : +- CometFilter (14) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : : : +- CometBroadcastExchange (22) - : : : +- CometProject (21) - : : : +- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - : : +- ReusedExchange (25) - : +- CometBroadcastExchange (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (28) - +- CometBroadcastExchange (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (34) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(5) CometBroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(6) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(10) CometBroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(11) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] -Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight - -(12) CometProject -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_quarter_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#19, d_quarter_name#20] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_quarter_name#20, 6, true, false, true) = 2001Q1) AND isnotnull(d_date_sk#19)) - -(15) CometProject -Input [2]: [d_date_sk#19, d_quarter_name#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#21, d_quarter_name#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#21, d_quarter_name#22] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_quarter_name#22, 6, true, false, true) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#21)) - -(21) CometProject -Input [2]: [d_date_sk#21, d_quarter_name#22] -Arguments: [d_date_sk#21], [d_date_sk#21] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: [d_date_sk#21] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#21] -Arguments: [sr_returned_date_sk#12], [d_date_sk#21], Inner, BuildRight - -(24) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#21] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] - -(25) ReusedExchange [Reuses operator id: 22] -Output [1]: [d_date_sk#23] - -(26) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#23] -Arguments: [cs_sold_date_sk#17], [d_date_sk#23], Inner, BuildRight - -(27) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#23] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_state#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(29) CometFilter -Input [2]: [s_store_sk#24, s_state#25] -Condition : isnotnull(s_store_sk#24) - -(30) CometProject -Input [2]: [s_store_sk#24, s_state#25] -Arguments: [s_store_sk#24, s_state#26], [s_store_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#25, 2, true, false, true) AS s_state#26] - -(31) CometBroadcastExchange -Input [2]: [s_store_sk#24, s_state#26] -Arguments: [s_store_sk#24, s_state#26] - -(32) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Right output [2]: [s_store_sk#24, s_state#26] -Arguments: [ss_store_sk#3], [s_store_sk#24], Inner, BuildRight - -(33) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#24, s_state#26] -Arguments: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26], [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26] - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#27, i_item_id#28, i_item_desc#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [i_item_sk#27, i_item_id#28, i_item_desc#29] -Condition : isnotnull(i_item_sk#27) - -(36) CometProject -Input [3]: [i_item_sk#27, i_item_id#28, i_item_desc#29] -Arguments: [i_item_sk#27, i_item_id#30, i_item_desc#29], [i_item_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#28, 16, true, false, true) AS i_item_id#30, i_item_desc#29] - -(37) CometBroadcastExchange -Input [3]: [i_item_sk#27, i_item_id#30, i_item_desc#29] -Arguments: [i_item_sk#27, i_item_id#30, i_item_desc#29] - -(38) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26] -Right output [3]: [i_item_sk#27, i_item_id#30, i_item_desc#29] -Arguments: [ss_item_sk#1], [i_item_sk#27], Inner, BuildRight - -(39) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_sk#27, i_item_id#30, i_item_desc#29] -Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#30, i_item_desc#29], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#30, i_item_desc#29] - -(40) CometHashAggregate -Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#30, i_item_desc#29] -Keys [3]: [i_item_id#30, i_item_desc#29, s_state#26] -Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partial_stddev_samp(cast(ss_quantity#5 as double)), partial_count(sr_return_quantity#11), partial_avg(sr_return_quantity#11), partial_stddev_samp(cast(sr_return_quantity#11 as double)), partial_count(cs_quantity#16), partial_avg(cs_quantity#16), partial_stddev_samp(cast(cs_quantity#16 as double))] - -(41) CometExchange -Input [21]: [i_item_id#30, i_item_desc#29, s_state#26, count#31, sum#32, count#33, n#34, avg#35, m2#36, count#37, sum#38, count#39, n#40, avg#41, m2#42, count#43, sum#44, count#45, n#46, avg#47, m2#48] -Arguments: hashpartitioning(i_item_id#30, i_item_desc#29, s_state#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(42) CometHashAggregate -Input [21]: [i_item_id#30, i_item_desc#29, s_state#26, count#31, sum#32, count#33, n#34, avg#35, m2#36, count#37, sum#38, count#39, n#40, avg#41, m2#42, count#43, sum#44, count#45, n#46, avg#47, m2#48] -Keys [3]: [i_item_id#30, i_item_desc#29, s_state#26] -Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#11), avg(sr_return_quantity#11), stddev_samp(cast(sr_return_quantity#11 as double)), count(cs_quantity#16), avg(cs_quantity#16), stddev_samp(cast(cs_quantity#16 as double))] - -(43) CometTakeOrderedAndProject -Input [15]: [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#30 ASC NULLS FIRST,i_item_desc#29 ASC NULLS FIRST,s_state#26 ASC NULLS FIRST], output=[i_item_id#30,i_item_desc#29,s_state#26,store_sales_quantitycount#49,store_sales_quantityave#50,store_sales_quantitystdev#51,store_sales_quantitycov#52,as_store_returns_quantitycount#53,as_store_returns_quantityave#54,as_store_returns_quantitystdev#55,store_returns_quantitycov#56,catalog_sales_quantitycount#57,catalog_sales_quantityave#58,catalog_sales_quantitystdev#59,catalog_sales_quantitycov#60]), [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60], 100, 0, [i_item_id#30 ASC NULLS FIRST, i_item_desc#29 ASC NULLS FIRST, s_state#26 ASC NULLS FIRST], [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60] - -(44) CometColumnarToRow [codegen id : 1] -Input [15]: [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_quarter_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [d_date_sk#19, d_quarter_name#20] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_quarter_name#20, 6, true, false, true) = 2001Q1) AND isnotnull(d_date_sk#19)) - -(47) CometProject -Input [2]: [d_date_sk#19, d_quarter_name#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(49) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) - - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#21, d_quarter_name#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [2]: [d_date_sk#21, d_quarter_name#22] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_quarter_name#22, 6, true, false, true) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#21)) - -(52) CometProject -Input [2]: [d_date_sk#21, d_quarter_name#22] -Arguments: [d_date_sk#21], [d_date_sk#21] - -(53) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#21] - -(54) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/simplified.txt deleted file mode 100644 index 1d48f96b09..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,63 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] - CometHashAggregate [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count(ss_quantity),avg(ss_quantity),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(sr_return_quantity),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cs_quantity),stddev_samp(cast(cs_quantity as double))] - CometExchange [i_item_id,i_item_desc,s_state] #1 - CometHashAggregate [ss_quantity,sr_return_quantity,cs_quantity] [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] - CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - 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_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 [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 [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_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 [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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - 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 [native_iceberg_compat] 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 [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan [native_iceberg_compat] 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/q17/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt deleted file mode 100644 index f502a5cd8d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt +++ /dev/null @@ -1,319 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometProject (18) - : : : : +- CometBroadcastHashJoin (17) - : : : : :- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (8) - : : : : +- CometBroadcastExchange (16) - : : : : +- CometProject (15) - : : : : +- CometFilter (14) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : : : +- CometBroadcastExchange (22) - : : : +- CometProject (21) - : : : +- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - : : +- ReusedExchange (25) - : +- CometBroadcastExchange (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (28) - +- CometBroadcastExchange (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (34) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(5) CometBroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(6) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(10) CometBroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(11) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] -Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight - -(12) CometProject -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_quarter_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#19, d_quarter_name#20] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_quarter_name#20, 6, true, false, true) = 2001Q1) AND isnotnull(d_date_sk#19)) - -(15) CometProject -Input [2]: [d_date_sk#19, d_quarter_name#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#21, d_quarter_name#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#21, d_quarter_name#22] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_quarter_name#22, 6, true, false, true) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#21)) - -(21) CometProject -Input [2]: [d_date_sk#21, d_quarter_name#22] -Arguments: [d_date_sk#21], [d_date_sk#21] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: [d_date_sk#21] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#21] -Arguments: [sr_returned_date_sk#12], [d_date_sk#21], Inner, BuildRight - -(24) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#21] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] - -(25) ReusedExchange [Reuses operator id: 22] -Output [1]: [d_date_sk#23] - -(26) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#23] -Arguments: [cs_sold_date_sk#17], [d_date_sk#23], Inner, BuildRight - -(27) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#23] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_state#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(29) CometFilter -Input [2]: [s_store_sk#24, s_state#25] -Condition : isnotnull(s_store_sk#24) - -(30) CometProject -Input [2]: [s_store_sk#24, s_state#25] -Arguments: [s_store_sk#24, s_state#26], [s_store_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#25, 2, true, false, true) AS s_state#26] - -(31) CometBroadcastExchange -Input [2]: [s_store_sk#24, s_state#26] -Arguments: [s_store_sk#24, s_state#26] - -(32) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Right output [2]: [s_store_sk#24, s_state#26] -Arguments: [ss_store_sk#3], [s_store_sk#24], Inner, BuildRight - -(33) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#24, s_state#26] -Arguments: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26], [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26] - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#27, i_item_id#28, i_item_desc#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [i_item_sk#27, i_item_id#28, i_item_desc#29] -Condition : isnotnull(i_item_sk#27) - -(36) CometProject -Input [3]: [i_item_sk#27, i_item_id#28, i_item_desc#29] -Arguments: [i_item_sk#27, i_item_id#30, i_item_desc#29], [i_item_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#28, 16, true, false, true) AS i_item_id#30, i_item_desc#29] - -(37) CometBroadcastExchange -Input [3]: [i_item_sk#27, i_item_id#30, i_item_desc#29] -Arguments: [i_item_sk#27, i_item_id#30, i_item_desc#29] - -(38) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26] -Right output [3]: [i_item_sk#27, i_item_id#30, i_item_desc#29] -Arguments: [ss_item_sk#1], [i_item_sk#27], Inner, BuildRight - -(39) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_sk#27, i_item_id#30, i_item_desc#29] -Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#30, i_item_desc#29], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#30, i_item_desc#29] - -(40) CometHashAggregate -Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#30, i_item_desc#29] -Keys [3]: [i_item_id#30, i_item_desc#29, s_state#26] -Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partial_stddev_samp(cast(ss_quantity#5 as double)), partial_count(sr_return_quantity#11), partial_avg(sr_return_quantity#11), partial_stddev_samp(cast(sr_return_quantity#11 as double)), partial_count(cs_quantity#16), partial_avg(cs_quantity#16), partial_stddev_samp(cast(cs_quantity#16 as double))] - -(41) CometExchange -Input [21]: [i_item_id#30, i_item_desc#29, s_state#26, count#31, sum#32, count#33, n#34, avg#35, m2#36, count#37, sum#38, count#39, n#40, avg#41, m2#42, count#43, sum#44, count#45, n#46, avg#47, m2#48] -Arguments: hashpartitioning(i_item_id#30, i_item_desc#29, s_state#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(42) CometHashAggregate -Input [21]: [i_item_id#30, i_item_desc#29, s_state#26, count#31, sum#32, count#33, n#34, avg#35, m2#36, count#37, sum#38, count#39, n#40, avg#41, m2#42, count#43, sum#44, count#45, n#46, avg#47, m2#48] -Keys [3]: [i_item_id#30, i_item_desc#29, s_state#26] -Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#11), avg(sr_return_quantity#11), stddev_samp(cast(sr_return_quantity#11 as double)), count(cs_quantity#16), avg(cs_quantity#16), stddev_samp(cast(cs_quantity#16 as double))] - -(43) CometTakeOrderedAndProject -Input [15]: [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#30 ASC NULLS FIRST,i_item_desc#29 ASC NULLS FIRST,s_state#26 ASC NULLS FIRST], output=[i_item_id#30,i_item_desc#29,s_state#26,store_sales_quantitycount#49,store_sales_quantityave#50,store_sales_quantitystdev#51,store_sales_quantitycov#52,as_store_returns_quantitycount#53,as_store_returns_quantityave#54,as_store_returns_quantitystdev#55,store_returns_quantitycov#56,catalog_sales_quantitycount#57,catalog_sales_quantityave#58,catalog_sales_quantitystdev#59,catalog_sales_quantitycov#60]), [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60], 100, 0, [i_item_id#30 ASC NULLS FIRST, i_item_desc#29 ASC NULLS FIRST, s_state#26 ASC NULLS FIRST], [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60] - -(44) CometColumnarToRow [codegen id : 1] -Input [15]: [i_item_id#30, i_item_desc#29, s_state#26, store_sales_quantitycount#49, store_sales_quantityave#50, store_sales_quantitystdev#51, store_sales_quantitycov#52, as_store_returns_quantitycount#53, as_store_returns_quantityave#54, as_store_returns_quantitystdev#55, store_returns_quantitycov#56, catalog_sales_quantitycount#57, catalog_sales_quantityave#58, catalog_sales_quantitystdev#59, catalog_sales_quantitycov#60] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_quarter_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [d_date_sk#19, d_quarter_name#20] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_quarter_name#20, 6, true, false, true) = 2001Q1) AND isnotnull(d_date_sk#19)) - -(47) CometProject -Input [2]: [d_date_sk#19, d_quarter_name#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(49) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) - - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#21, d_quarter_name#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [2]: [d_date_sk#21, d_quarter_name#22] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_quarter_name#22, 6, true, false, true) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#21)) - -(52) CometProject -Input [2]: [d_date_sk#21, d_quarter_name#22] -Arguments: [d_date_sk#21], [d_date_sk#21] - -(53) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#21] - -(54) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/extended.txt deleted file mode 100644 index 4e1a4f3ec0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/extended.txt +++ /dev/null @@ -1,62 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 1d48f96b09..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt +++ /dev/null @@ -1,63 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] - CometHashAggregate [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count(ss_quantity),avg(ss_quantity),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(sr_return_quantity),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cs_quantity),stddev_samp(cast(cs_quantity as double))] - CometExchange [i_item_id,i_item_desc,s_state] #1 - CometHashAggregate [ss_quantity,sr_return_quantity,cs_quantity] [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] - CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - 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_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 [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 [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_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 [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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - 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 [native_iceberg_compat] 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 [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_quarter_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/explain.txt deleted file mode 100644 index 0503116eb3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/explain.txt +++ /dev/null @@ -1,295 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (46) -+- * HashAggregate (45) - +- * CometColumnarToRow (44) - +- CometColumnarExchange (43) - +- * HashAggregate (42) - +- * Expand (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * Project (23) - : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : :- * Project (17) - : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : :- * Project (10) - : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : :- * Filter (3) - : : : : : : +- * ColumnarToRow (2) - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : : +- BroadcastExchange (8) - : : : : : +- * CometColumnarToRow (7) - : : : : : +- CometProject (6) - : : : : : +- CometFilter (5) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) - : : : : +- BroadcastExchange (15) - : : : : +- * CometColumnarToRow (14) - : : : : +- CometProject (13) - : : : : +- CometFilter (12) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (11) - : : : +- BroadcastExchange (21) - : : : +- * CometColumnarToRow (20) - : : : +- CometFilter (19) - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (18) - : : +- BroadcastExchange (28) - : : +- * CometColumnarToRow (27) - : : +- CometProject (26) - : : +- CometFilter (25) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (24) - : +- ReusedExchange (31) - +- BroadcastExchange (38) - +- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometNativeScan parquet spark_catalog.default.item (34) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 7] -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] - -(3) Filter [codegen id : 7] -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) - -(4) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#12, 1, true, false, true) = F) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = Unknown )) AND isnotnull(cd_demo_sk#11)) - -(6) CometProject -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] - -(7) CometColumnarToRow [codegen id : 1] -Input [2]: [cd_demo_sk#11, cd_dep_count#14] - -(8) BroadcastExchange -Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 7] -Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] - -(11) CometNativeScan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [In(c_birth_month, [1,12,2,6,8,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(12) CometFilter -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Condition : (((c_birth_month#18 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) - -(13) CometProject -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(14) CometColumnarToRow [codegen id : 2] -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(15) BroadcastExchange -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(18) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(19) CometFilter -Input [1]: [cd_demo_sk#20] -Condition : isnotnull(cd_demo_sk#20) - -(20) CometColumnarToRow [codegen id : 3] -Input [1]: [cd_demo_sk#20] - -(21) BroadcastExchange -Input [1]: [cd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#16] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None - -(23) Project [codegen id : 7] -Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] - -(24) CometNativeScan parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#21)) - -(26) CometProject -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) AS ca_state#25, ca_country#24] - -(27) CometColumnarToRow [codegen id : 4] -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(28) BroadcastExchange -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(29) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#17] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None - -(30) Project [codegen id : 7] -Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(31) ReusedExchange [Reuses operator id: 51] -Output [1]: [d_date_sk#26] - -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#26] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] - -(34) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#27, i_item_id#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [i_item_sk#27, i_item_id#28] -Condition : isnotnull(i_item_sk#27) - -(36) CometProject -Input [2]: [i_item_sk#27, i_item_id#28] -Arguments: [i_item_sk#27, i_item_id#29], [i_item_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#28, 16, true, false, true) AS i_item_id#29] - -(37) CometColumnarToRow [codegen id : 6] -Input [2]: [i_item_sk#27, i_item_id#29] - -(38) BroadcastExchange -Input [2]: [i_item_sk#27, i_item_id#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#27] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 7] -Output [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, ca_county#22] -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#27, i_item_id#29] - -(41) Expand [codegen id : 7] -Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, ca_county#22] -Arguments: [[cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, ca_county#22, 0], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, ca_state#25, null, 1], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, ca_country#24, null, null, 3], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#29, null, null, null, 7], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, null, null, null, null, 15]], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] - -(42) HashAggregate [codegen id : 7] -Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] -Keys [5]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] -Functions [7]: [partial_avg(cast(cs_quantity#4 as decimal(12,2))), partial_avg(cast(cs_list_price#5 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#7 as decimal(12,2))), partial_avg(cast(cs_sales_price#6 as decimal(12,2))), partial_avg(cast(cs_net_profit#8 as decimal(12,2))), partial_avg(cast(c_birth_year#19 as decimal(12,2))), partial_avg(cast(cd_dep_count#14 as decimal(12,2)))] -Aggregate Attributes [14]: [sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42, sum#43, count#44, sum#45, count#46, sum#47, count#48] -Results [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] - -(43) CometColumnarExchange -Input [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Arguments: hashpartitioning(i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(44) CometColumnarToRow [codegen id : 8] -Input [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] - -(45) HashAggregate [codegen id : 8] -Input [19]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Keys [5]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, spark_grouping_id#34] -Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_price#5 as decimal(12,2))), avg(cast(cs_coupon_amt#7 as decimal(12,2))), avg(cast(cs_sales_price#6 as decimal(12,2))), avg(cast(cs_net_profit#8 as decimal(12,2))), avg(cast(c_birth_year#19 as decimal(12,2))), avg(cast(cd_dep_count#14 as decimal(12,2)))] -Aggregate Attributes [7]: [avg(cast(cs_quantity#4 as decimal(12,2)))#63, avg(cast(cs_list_price#5 as decimal(12,2)))#64, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#65, avg(cast(cs_sales_price#6 as decimal(12,2)))#66, avg(cast(cs_net_profit#8 as decimal(12,2)))#67, avg(cast(c_birth_year#19 as decimal(12,2)))#68, avg(cast(cd_dep_count#14 as decimal(12,2)))#69] -Results [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, avg(cast(cs_quantity#4 as decimal(12,2)))#63 AS agg1#70, avg(cast(cs_list_price#5 as decimal(12,2)))#64 AS agg2#71, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#65 AS agg3#72, avg(cast(cs_sales_price#6 as decimal(12,2)))#66 AS agg4#73, avg(cast(cs_net_profit#8 as decimal(12,2)))#67 AS agg5#74, avg(cast(c_birth_year#19 as decimal(12,2)))#68 AS agg6#75, avg(cast(cd_dep_count#14 as decimal(12,2)))#69 AS agg7#76] - -(46) TakeOrderedAndProject -Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#70, agg2#71, agg3#72, agg4#73, agg5#74, agg6#75, agg7#76] -Arguments: 100, [ca_country#31 ASC NULLS FIRST, ca_state#32 ASC NULLS FIRST, ca_county#33 ASC NULLS FIRST, i_item_id#30 ASC NULLS FIRST], [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#70, agg2#71, agg3#72, agg4#73, agg5#74, agg6#75, agg7#76] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (51) -+- * CometColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometNativeScan parquet spark_catalog.default.date_dim (47) - - -(47) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#77] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(48) CometFilter -Input [2]: [d_date_sk#26, d_year#77] -Condition : ((isnotnull(d_year#77) AND (d_year#77 = 1998)) AND isnotnull(d_date_sk#26)) - -(49) CometProject -Input [2]: [d_date_sk#26, d_year#77] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(50) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(51) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/simplified.txt deleted file mode 100644 index 77a45c46cd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - WholeStageCodegen (8) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2))),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 - WholeStageCodegen (7) - HashAggregate [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] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Expand [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] - Project [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] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [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] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan 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/q18.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/explain.txt deleted file mode 100644 index 5fefd21bfa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,280 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (43) -+- CometTakeOrderedAndProject (42) - +- CometHashAggregate (41) - +- CometExchange (40) - +- CometHashAggregate (39) - +- CometExpand (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (19) - : : : +- CometBroadcastHashJoin (18) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometProject (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : : : +- CometBroadcastExchange (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (15) - : : +- CometBroadcastExchange (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#12, 1, true, false, true) = F) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = Unknown )) AND isnotnull(cd_demo_sk#11)) - -(5) CometProject -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] - -(6) CometBroadcastExchange -Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14] - -(7) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Right output [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight - -(8) CometProject -Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [In(c_birth_month, [1,12,2,6,8,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Condition : (((c_birth_month#18 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) - -(11) CometProject -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(12) CometBroadcastExchange -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(13) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight - -(14) CometProject -Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [cd_demo_sk#20] -Condition : isnotnull(cd_demo_sk#20) - -(17) CometBroadcastExchange -Input [1]: [cd_demo_sk#20] -Arguments: [cd_demo_sk#20] - -(18) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Right output [1]: [cd_demo_sk#20] -Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight - -(19) CometProject -Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#21)) - -(22) CometProject -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) AS ca_state#25, ca_country#24] - -(23) CometBroadcastExchange -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(24) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -Right output [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight - -(25) CometProject -Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 1998)) AND isnotnull(d_date_sk#26)) - -(28) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(29) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] - -(30) CometBroadcastHashJoin -Left output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [1]: [d_date_sk#26] -Arguments: [cs_sold_date_sk#9], [d_date_sk#26], Inner, BuildRight - -(31) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#28, i_item_id#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [i_item_sk#28, i_item_id#29] -Condition : isnotnull(i_item_sk#28) - -(34) CometProject -Input [2]: [i_item_sk#28, i_item_id#29] -Arguments: [i_item_sk#28, i_item_id#30], [i_item_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#29, 16, true, false, true) AS i_item_id#30] - -(35) CometBroadcastExchange -Input [2]: [i_item_sk#28, i_item_id#30] -Arguments: [i_item_sk#28, i_item_id#30] - -(36) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [2]: [i_item_sk#28, i_item_id#30] -Arguments: [cs_item_sk#3], [i_item_sk#28], Inner, BuildRight - -(37) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#28, i_item_id#30] -Arguments: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22] - -(38) CometExpand -Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22] -Arguments: [[cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22, 0], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, null, 1], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, null, null, 3], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, null, null, null, 7], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, null, null, null, null, 15]], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] - -(39) CometHashAggregate -Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] -Keys [5]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] -Functions [7]: [partial_avg(cast(cs_quantity#4 as decimal(12,2))), partial_avg(cast(cs_list_price#5 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#7 as decimal(12,2))), partial_avg(cast(cs_sales_price#6 as decimal(12,2))), partial_avg(cast(cs_net_profit#8 as decimal(12,2))), partial_avg(cast(c_birth_year#19 as decimal(12,2))), partial_avg(cast(cd_dep_count#14 as decimal(12,2)))] - -(40) CometExchange -Input [19]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35, sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49] -Arguments: hashpartitioning(i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(41) CometHashAggregate -Input [19]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35, sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49] -Keys [5]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] -Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_price#5 as decimal(12,2))), avg(cast(cs_coupon_amt#7 as decimal(12,2))), avg(cast(cs_sales_price#6 as decimal(12,2))), avg(cast(cs_net_profit#8 as decimal(12,2))), avg(cast(c_birth_year#19 as decimal(12,2))), avg(cast(cd_dep_count#14 as decimal(12,2)))] - -(42) CometTakeOrderedAndProject -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#32 ASC NULLS FIRST,ca_state#33 ASC NULLS FIRST,ca_county#34 ASC NULLS FIRST,i_item_id#31 ASC NULLS FIRST], output=[i_item_id#31,ca_country#32,ca_state#33,ca_county#34,agg1#50,agg2#51,agg3#52,agg4#53,agg5#54,agg6#55,agg7#56]), [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56], 100, 0, [ca_country#32 ASC NULLS FIRST, ca_state#33 ASC NULLS FIRST, ca_county#34 ASC NULLS FIRST, i_item_id#31 ASC NULLS FIRST], [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56] - -(43) CometColumnarToRow [codegen id : 1] -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) - - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 1998)) AND isnotnull(d_date_sk#26)) - -(46) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(48) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/simplified.txt deleted file mode 100644 index 6c2b8b2e4f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,53 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometHashAggregate [spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2)))] - CometExchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 - CometHashAggregate [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,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,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,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_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 [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 [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,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_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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - 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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange [cd_demo_sk] #5 - CometFilter [cd_demo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #8 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] 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/q18/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt deleted file mode 100644 index 5fefd21bfa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt +++ /dev/null @@ -1,280 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (43) -+- CometTakeOrderedAndProject (42) - +- CometHashAggregate (41) - +- CometExchange (40) - +- CometHashAggregate (39) - +- CometExpand (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (19) - : : : +- CometBroadcastHashJoin (18) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometProject (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : : : +- CometBroadcastExchange (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (15) - : : +- CometBroadcastExchange (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#12, 1, true, false, true) = F) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = Unknown )) AND isnotnull(cd_demo_sk#11)) - -(5) CometProject -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] - -(6) CometBroadcastExchange -Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14] - -(7) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Right output [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight - -(8) CometProject -Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [In(c_birth_month, [1,12,2,6,8,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Condition : (((c_birth_month#18 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) - -(11) CometProject -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(12) CometBroadcastExchange -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(13) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight - -(14) CometProject -Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [cd_demo_sk#20] -Condition : isnotnull(cd_demo_sk#20) - -(17) CometBroadcastExchange -Input [1]: [cd_demo_sk#20] -Arguments: [cd_demo_sk#20] - -(18) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Right output [1]: [cd_demo_sk#20] -Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight - -(19) CometProject -Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#21)) - -(22) CometProject -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) AS ca_state#25, ca_country#24] - -(23) CometBroadcastExchange -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(24) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -Right output [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight - -(25) CometProject -Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 1998)) AND isnotnull(d_date_sk#26)) - -(28) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(29) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] - -(30) CometBroadcastHashJoin -Left output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [1]: [d_date_sk#26] -Arguments: [cs_sold_date_sk#9], [d_date_sk#26], Inner, BuildRight - -(31) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#28, i_item_id#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [i_item_sk#28, i_item_id#29] -Condition : isnotnull(i_item_sk#28) - -(34) CometProject -Input [2]: [i_item_sk#28, i_item_id#29] -Arguments: [i_item_sk#28, i_item_id#30], [i_item_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#29, 16, true, false, true) AS i_item_id#30] - -(35) CometBroadcastExchange -Input [2]: [i_item_sk#28, i_item_id#30] -Arguments: [i_item_sk#28, i_item_id#30] - -(36) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [2]: [i_item_sk#28, i_item_id#30] -Arguments: [cs_item_sk#3], [i_item_sk#28], Inner, BuildRight - -(37) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#28, i_item_id#30] -Arguments: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22] - -(38) CometExpand -Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22] -Arguments: [[cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, ca_county#22, 0], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, ca_state#25, null, 1], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, ca_country#24, null, null, 3], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#30, null, null, null, 7], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, null, null, null, null, 15]], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] - -(39) CometHashAggregate -Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] -Keys [5]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] -Functions [7]: [partial_avg(cast(cs_quantity#4 as decimal(12,2))), partial_avg(cast(cs_list_price#5 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#7 as decimal(12,2))), partial_avg(cast(cs_sales_price#6 as decimal(12,2))), partial_avg(cast(cs_net_profit#8 as decimal(12,2))), partial_avg(cast(c_birth_year#19 as decimal(12,2))), partial_avg(cast(cd_dep_count#14 as decimal(12,2)))] - -(40) CometExchange -Input [19]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35, sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49] -Arguments: hashpartitioning(i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(41) CometHashAggregate -Input [19]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35, sum#36, count#37, sum#38, count#39, sum#40, count#41, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49] -Keys [5]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, spark_grouping_id#35] -Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_price#5 as decimal(12,2))), avg(cast(cs_coupon_amt#7 as decimal(12,2))), avg(cast(cs_sales_price#6 as decimal(12,2))), avg(cast(cs_net_profit#8 as decimal(12,2))), avg(cast(c_birth_year#19 as decimal(12,2))), avg(cast(cd_dep_count#14 as decimal(12,2)))] - -(42) CometTakeOrderedAndProject -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#32 ASC NULLS FIRST,ca_state#33 ASC NULLS FIRST,ca_county#34 ASC NULLS FIRST,i_item_id#31 ASC NULLS FIRST], output=[i_item_id#31,ca_country#32,ca_state#33,ca_county#34,agg1#50,agg2#51,agg3#52,agg4#53,agg5#54,agg6#55,agg7#56]), [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56], 100, 0, [ca_country#32 ASC NULLS FIRST, ca_state#33 ASC NULLS FIRST, ca_county#34 ASC NULLS FIRST, i_item_id#31 ASC NULLS FIRST], [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56] - -(43) CometColumnarToRow [codegen id : 1] -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#50, agg2#51, agg3#52, agg4#53, agg5#54, agg6#55, agg7#56] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) - - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 1998)) AND isnotnull(d_date_sk#26)) - -(46) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(48) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/extended.txt deleted file mode 100644 index 0b554c7e7c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/extended.txt +++ /dev/null @@ -1,51 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 6c2b8b2e4f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/simplified.txt +++ /dev/null @@ -1,53 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometHashAggregate [spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2)))] - CometExchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 - CometHashAggregate [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,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,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,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_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 [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 [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,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_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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - 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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange [cd_demo_sk] #5 - CometFilter [cd_demo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #8 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/explain.txt deleted file mode 100644 index 539af26bc5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (36) -+- CometTakeOrderedAndProject (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometNativeScan parquet spark_catalog.default.item (9) - : : +- CometBroadcastExchange (17) - : : +- CometFilter (16) - : : +- CometNativeScan parquet spark_catalog.default.customer (15) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometNativeScan parquet spark_catalog.default.customer_address (20) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometFilter (27) - +- CometNativeScan parquet spark_catalog.default.store (26) - - -(1) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1998)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1], [d_date_sk#1] - -(4) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_item_sk#4) AND isnotnull(ss_customer_sk#5)) AND isnotnull(ss_store_sk#6)) - -(6) CometBroadcastExchange -Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] - -(7) CometBroadcastHashJoin -Left output [1]: [d_date_sk#1] -Right output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [d_date_sk#1], [ss_sold_date_sk#8], Inner, BuildRight - -(8) CometProject -Input [6]: [d_date_sk#1, ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7], [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] - -(9) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Condition : ((isnotnull(i_manager_id#14) AND (i_manager_id#14 = 8)) AND isnotnull(i_item_sk#9)) - -(11) CometProject -Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Arguments: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [i_item_sk#9, i_brand_id#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#11, 50, true, false, true) AS i_brand#15, i_manufact_id#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#13, 50, true, false, true) AS i_manufact#16] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] -Right output [5]: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [ss_item_sk#4], [i_item_sk#9], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(15) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(c_current_addr_sk#18)) - -(17) CometBroadcastExchange -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [c_customer_sk#17, c_current_addr_sk#18] - -(18) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Right output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_customer_sk#5], [c_customer_sk#17], Inner, BuildRight - -(19) CometProject -Input [9]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18] - -(20) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_zip#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [ca_address_sk#19, ca_zip#20] -Condition : (isnotnull(ca_address_sk#19) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#20, 10, true, false, true))) - -(22) CometProject -Input [2]: [ca_address_sk#19, ca_zip#20] -Arguments: [ca_address_sk#19, ca_zip#21], [ca_address_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#20, 10, true, false, true) AS ca_zip#21] - -(23) CometBroadcastExchange -Input [2]: [ca_address_sk#19, ca_zip#21] -Arguments: [ca_address_sk#19, ca_zip#21] - -(24) CometBroadcastHashJoin -Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18] -Right output [2]: [ca_address_sk#19, ca_zip#21] -Arguments: [c_current_addr_sk#18], [ca_address_sk#19], Inner, BuildRight - -(25) CometProject -Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18, ca_address_sk#19, ca_zip#21] -Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21] - -(26) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#22, s_zip#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [s_store_sk#22, s_zip#23] -Condition : (isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#23, 10, true, false, true)) AND isnotnull(s_store_sk#22)) - -(28) CometProject -Input [2]: [s_store_sk#22, s_zip#23] -Arguments: [s_store_sk#22, s_zip#24], [s_store_sk#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#23, 10, true, false, true) AS s_zip#24] - -(29) CometBroadcastExchange -Input [2]: [s_store_sk#22, s_zip#24] -Arguments: [s_store_sk#22, s_zip#24] - -(30) CometBroadcastHashJoin -Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21] -Right output [2]: [s_store_sk#22, s_zip#24] -Arguments: [ss_store_sk#6], [s_store_sk#22], Inner, NOT (substr(ca_zip#21, 1, 5) = substr(s_zip#24, 1, 5)), BuildRight - -(31) CometProject -Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21, s_store_sk#22, s_zip#24] -Arguments: [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(32) CometHashAggregate -Input [5]: [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Keys [4]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] - -(33) CometExchange -Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] -Arguments: hashpartitioning(i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(34) CometHashAggregate -Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] -Keys [4]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] - -(35) CometTakeOrderedAndProject -Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#28 DESC NULLS LAST,brand#27 ASC NULLS FIRST,brand_id#26 ASC NULLS FIRST,i_manufact_id#12 ASC NULLS FIRST,i_manufact#16 ASC NULLS FIRST], output=[brand_id#26,brand#27,i_manufact_id#12,i_manufact#16,ext_price#28]), [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28], 100, 0, [ext_price#28 DESC NULLS LAST, brand#27 ASC NULLS FIRST, brand_id#26 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#16 ASC NULLS FIRST], [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] - -(36) CometColumnarToRow [codegen id : 1] -Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/simplified.txt deleted file mode 100644 index 675500cd0b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/simplified.txt +++ /dev/null @@ -1,38 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,i_manufact_id,i_manufact,ext_price] - CometHashAggregate [sum] [brand_id,brand,i_manufact_id,i_manufact,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 - CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] - CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - 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 [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,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,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_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 - CometProject [i_brand,i_manufact] [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_zip] #5 - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [s_store_sk,s_zip] #6 - CometProject [s_zip] [s_store_sk,s_zip] - CometFilter [s_store_sk,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_iceberg_compat/explain.txt deleted file mode 100644 index 0724450064..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (36) -+- CometTakeOrderedAndProject (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - : : +- CometBroadcastExchange (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (15) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1998)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1], [d_date_sk#1] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_item_sk#4) AND isnotnull(ss_customer_sk#5)) AND isnotnull(ss_store_sk#6)) - -(6) CometBroadcastExchange -Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] - -(7) CometBroadcastHashJoin -Left output [1]: [d_date_sk#1] -Right output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [d_date_sk#1], [ss_sold_date_sk#8], Inner, BuildRight - -(8) CometProject -Input [6]: [d_date_sk#1, ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7], [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Condition : ((isnotnull(i_manager_id#14) AND (i_manager_id#14 = 8)) AND isnotnull(i_item_sk#9)) - -(11) CometProject -Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Arguments: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [i_item_sk#9, i_brand_id#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#11, 50, true, false, true) AS i_brand#15, i_manufact_id#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#13, 50, true, false, true) AS i_manufact#16] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] -Right output [5]: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [ss_item_sk#4], [i_item_sk#9], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(c_current_addr_sk#18)) - -(17) CometBroadcastExchange -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [c_customer_sk#17, c_current_addr_sk#18] - -(18) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Right output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_customer_sk#5], [c_customer_sk#17], Inner, BuildRight - -(19) CometProject -Input [9]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_zip#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [ca_address_sk#19, ca_zip#20] -Condition : (isnotnull(ca_address_sk#19) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#20, 10, true, false, true))) - -(22) CometProject -Input [2]: [ca_address_sk#19, ca_zip#20] -Arguments: [ca_address_sk#19, ca_zip#21], [ca_address_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#20, 10, true, false, true) AS ca_zip#21] - -(23) CometBroadcastExchange -Input [2]: [ca_address_sk#19, ca_zip#21] -Arguments: [ca_address_sk#19, ca_zip#21] - -(24) CometBroadcastHashJoin -Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18] -Right output [2]: [ca_address_sk#19, ca_zip#21] -Arguments: [c_current_addr_sk#18], [ca_address_sk#19], Inner, BuildRight - -(25) CometProject -Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18, ca_address_sk#19, ca_zip#21] -Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#22, s_zip#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [s_store_sk#22, s_zip#23] -Condition : (isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#23, 10, true, false, true)) AND isnotnull(s_store_sk#22)) - -(28) CometProject -Input [2]: [s_store_sk#22, s_zip#23] -Arguments: [s_store_sk#22, s_zip#24], [s_store_sk#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#23, 10, true, false, true) AS s_zip#24] - -(29) CometBroadcastExchange -Input [2]: [s_store_sk#22, s_zip#24] -Arguments: [s_store_sk#22, s_zip#24] - -(30) CometBroadcastHashJoin -Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21] -Right output [2]: [s_store_sk#22, s_zip#24] -Arguments: [ss_store_sk#6], [s_store_sk#22], Inner, NOT (substr(ca_zip#21, 1, 5) = substr(s_zip#24, 1, 5)), BuildRight - -(31) CometProject -Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21, s_store_sk#22, s_zip#24] -Arguments: [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(32) CometHashAggregate -Input [5]: [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Keys [4]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] - -(33) CometExchange -Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] -Arguments: hashpartitioning(i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(34) CometHashAggregate -Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] -Keys [4]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] - -(35) CometTakeOrderedAndProject -Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#28 DESC NULLS LAST,brand#27 ASC NULLS FIRST,brand_id#26 ASC NULLS FIRST,i_manufact_id#12 ASC NULLS FIRST,i_manufact#16 ASC NULLS FIRST], output=[brand_id#26,brand#27,i_manufact_id#12,i_manufact#16,ext_price#28]), [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28], 100, 0, [ext_price#28 DESC NULLS LAST, brand#27 ASC NULLS FIRST, brand_id#26 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#16 ASC NULLS FIRST], [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] - -(36) CometColumnarToRow [codegen id : 1] -Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_iceberg_compat/simplified.txt deleted file mode 100644 index 93ab89c142..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,38 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,i_manufact_id,i_manufact,ext_price] - CometHashAggregate [sum] [brand_id,brand,i_manufact_id,i_manufact,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 - CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] - CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - 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 [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,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,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_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 - CometProject [i_brand,i_manufact] [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_zip] #5 - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [s_store_sk,s_zip] #6 - CometProject [s_zip] [s_store_sk,s_zip] - CometFilter [s_store_sk,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt deleted file mode 100644 index 0724450064..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (36) -+- CometTakeOrderedAndProject (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - : : +- CometBroadcastExchange (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (15) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1998)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1], [d_date_sk#1] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_item_sk#4) AND isnotnull(ss_customer_sk#5)) AND isnotnull(ss_store_sk#6)) - -(6) CometBroadcastExchange -Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] - -(7) CometBroadcastHashJoin -Left output [1]: [d_date_sk#1] -Right output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [d_date_sk#1], [ss_sold_date_sk#8], Inner, BuildRight - -(8) CometProject -Input [6]: [d_date_sk#1, ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7], [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Condition : ((isnotnull(i_manager_id#14) AND (i_manager_id#14 = 8)) AND isnotnull(i_item_sk#9)) - -(11) CometProject -Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] -Arguments: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [i_item_sk#9, i_brand_id#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#11, 50, true, false, true) AS i_brand#15, i_manufact_id#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#13, 50, true, false, true) AS i_manufact#16] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] -Right output [5]: [i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [ss_item_sk#4], [i_item_sk#9], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_item_sk#9, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Arguments: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(c_current_addr_sk#18)) - -(17) CometBroadcastExchange -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [c_customer_sk#17, c_current_addr_sk#18] - -(18) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Right output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_customer_sk#5], [c_customer_sk#17], Inner, BuildRight - -(19) CometProject -Input [9]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_zip#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [ca_address_sk#19, ca_zip#20] -Condition : (isnotnull(ca_address_sk#19) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#20, 10, true, false, true))) - -(22) CometProject -Input [2]: [ca_address_sk#19, ca_zip#20] -Arguments: [ca_address_sk#19, ca_zip#21], [ca_address_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#20, 10, true, false, true) AS ca_zip#21] - -(23) CometBroadcastExchange -Input [2]: [ca_address_sk#19, ca_zip#21] -Arguments: [ca_address_sk#19, ca_zip#21] - -(24) CometBroadcastHashJoin -Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18] -Right output [2]: [ca_address_sk#19, ca_zip#21] -Arguments: [c_current_addr_sk#18], [ca_address_sk#19], Inner, BuildRight - -(25) CometProject -Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, c_current_addr_sk#18, ca_address_sk#19, ca_zip#21] -Arguments: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21], [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#22, s_zip#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [s_store_sk#22, s_zip#23] -Condition : (isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#23, 10, true, false, true)) AND isnotnull(s_store_sk#22)) - -(28) CometProject -Input [2]: [s_store_sk#22, s_zip#23] -Arguments: [s_store_sk#22, s_zip#24], [s_store_sk#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#23, 10, true, false, true) AS s_zip#24] - -(29) CometBroadcastExchange -Input [2]: [s_store_sk#22, s_zip#24] -Arguments: [s_store_sk#22, s_zip#24] - -(30) CometBroadcastHashJoin -Left output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21] -Right output [2]: [s_store_sk#22, s_zip#24] -Arguments: [ss_store_sk#6], [s_store_sk#22], Inner, NOT (substr(ca_zip#21, 1, 5) = substr(s_zip#24, 1, 5)), BuildRight - -(31) CometProject -Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16, ca_zip#21, s_store_sk#22, s_zip#24] -Arguments: [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16], [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] - -(32) CometHashAggregate -Input [5]: [ss_ext_sales_price#7, i_brand_id#10, i_brand#15, i_manufact_id#12, i_manufact#16] -Keys [4]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] - -(33) CometExchange -Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] -Arguments: hashpartitioning(i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(34) CometHashAggregate -Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] -Keys [4]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] - -(35) CometTakeOrderedAndProject -Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#28 DESC NULLS LAST,brand#27 ASC NULLS FIRST,brand_id#26 ASC NULLS FIRST,i_manufact_id#12 ASC NULLS FIRST,i_manufact#16 ASC NULLS FIRST], output=[brand_id#26,brand#27,i_manufact_id#12,i_manufact#16,ext_price#28]), [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28], 100, 0, [ext_price#28 DESC NULLS LAST, brand#27 ASC NULLS FIRST, brand_id#26 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#16 ASC NULLS FIRST], [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] - -(36) CometColumnarToRow [codegen id : 1] -Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/extended.txt deleted file mode 100644 index a8cfde2d31..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/extended.txt +++ /dev/null @@ -1,38 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 35 out of 35 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 93ab89c142..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/simplified.txt +++ /dev/null @@ -1,38 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,i_manufact_id,i_manufact,ext_price] - CometHashAggregate [sum] [brand_id,brand,i_manufact_id,i_manufact,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 - CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] - CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - 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 [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,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,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_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] #3 - CometProject [i_brand,i_manufact] [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #4 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_zip] #5 - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [s_store_sk,s_zip] #6 - CometProject [s_zip] [s_store_sk,s_zip] - CometFilter [s_store_sk,s_zip] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_datafusion/explain.txt deleted file mode 100644 index 22fbab2817..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_datafusion/explain.txt +++ /dev/null @@ -1,193 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (34) -+- CometSort (33) - +- CometExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometHashAggregate (14) - : : +- CometExchange (13) - : : +- CometHashAggregate (12) - : : +- CometProject (11) - : : +- CometBroadcastHashJoin (10) - : : :- CometUnion (5) - : : : :- CometProject (2) - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales (1) - : : : +- CometProject (4) - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales (3) - : : +- CometBroadcastExchange (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (6) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometNativeScan parquet spark_catalog.default.date_dim (15) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometBroadcastHashJoin (27) - :- CometHashAggregate (22) - : +- ReusedExchange (21) - +- CometBroadcastExchange (26) - +- CometProject (25) - +- CometFilter (24) - +- CometNativeScan parquet spark_catalog.default.date_dim (23) - - -(1) CometNativeScan parquet spark_catalog.default.web_sales -Output [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ws_sold_date_sk#2)] -ReadSchema: struct - -(2) CometProject -Input [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] -Arguments: [sold_date_sk#3, sales_price#4], [ws_sold_date_sk#2 AS sold_date_sk#3, ws_ext_sales_price#1 AS sales_price#4] - -(3) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(cs_sold_date_sk#6)] -ReadSchema: struct - -(4) CometProject -Input [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] -Arguments: [sold_date_sk#7, sales_price#8], [cs_sold_date_sk#6 AS sold_date_sk#7, cs_ext_sales_price#5 AS sales_price#8] - -(5) CometUnion -Child 0 Input [2]: [sold_date_sk#3, sales_price#4] -Child 1 Input [2]: [sold_date_sk#7, sales_price#8] - -(6) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Condition : (isnotnull(d_date_sk#9) AND isnotnull(d_week_seq#10)) - -(8) CometProject -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Arguments: [d_date_sk#9, d_week_seq#10, d_day_name#12], [d_date_sk#9, d_week_seq#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_day_name#11, 9, true, false, true) AS d_day_name#12] - -(9) CometBroadcastExchange -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [d_date_sk#9, d_week_seq#10, d_day_name#12] - -(10) CometBroadcastHashJoin -Left output [2]: [sold_date_sk#3, sales_price#4] -Right output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [sold_date_sk#3], [d_date_sk#9], Inner, BuildRight - -(11) CometProject -Input [5]: [sold_date_sk#3, sales_price#4, d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [sales_price#4, d_week_seq#10, d_day_name#12], [sales_price#4, d_week_seq#10, d_day_name#12] - -(12) CometHashAggregate -Input [3]: [sales_price#4, d_week_seq#10, d_day_name#12] -Keys [1]: [d_week_seq#10] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(13) CometExchange -Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(14) CometHashAggregate -Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Keys [1]: [d_week_seq#10] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(15) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_week_seq)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [d_week_seq#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_week_seq#20)) - -(17) CometProject -Input [2]: [d_week_seq#20, d_year#21] -Arguments: [d_week_seq#20], [d_week_seq#20] - -(18) CometBroadcastExchange -Input [1]: [d_week_seq#20] -Arguments: [d_week_seq#20] - -(19) CometBroadcastHashJoin -Left output [8]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28] -Right output [1]: [d_week_seq#20] -Arguments: [d_week_seq#10], [d_week_seq#20], Inner, BuildRight - -(20) CometProject -Input [9]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28, d_week_seq#20] -Arguments: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36], [d_week_seq#10 AS d_week_seq1#29, sun_sales#22 AS sun_sales1#30, mon_sales#23 AS mon_sales1#31, tue_sales#24 AS tue_sales1#32, wed_sales#25 AS wed_sales1#33, thu_sales#26 AS thu_sales1#34, fri_sales#27 AS fri_sales1#35, sat_sales#28 AS sat_sales1#36] - -(21) ReusedExchange [Reuses operator id: 13] -Output [8]: [d_week_seq#10, sum#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43] - -(22) CometHashAggregate -Input [8]: [d_week_seq#10, sum#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43] -Keys [1]: [d_week_seq#10] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(23) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [d_week_seq#44, d_year#45] -Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_week_seq#44)) - -(25) CometProject -Input [2]: [d_week_seq#44, d_year#45] -Arguments: [d_week_seq#44], [d_week_seq#44] - -(26) CometBroadcastExchange -Input [1]: [d_week_seq#44] -Arguments: [d_week_seq#44] - -(27) CometBroadcastHashJoin -Left output [8]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28] -Right output [1]: [d_week_seq#44] -Arguments: [d_week_seq#10], [d_week_seq#44], Inner, BuildRight - -(28) CometProject -Input [9]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28, d_week_seq#44] -Arguments: [d_week_seq2#46, sun_sales2#47, mon_sales2#48, tue_sales2#49, wed_sales2#50, thu_sales2#51, fri_sales2#52, sat_sales2#53], [d_week_seq#10 AS d_week_seq2#46, sun_sales#22 AS sun_sales2#47, mon_sales#23 AS mon_sales2#48, tue_sales#24 AS tue_sales2#49, wed_sales#25 AS wed_sales2#50, thu_sales#26 AS thu_sales2#51, fri_sales#27 AS fri_sales2#52, sat_sales#28 AS sat_sales2#53] - -(29) CometBroadcastExchange -Input [8]: [d_week_seq2#46, sun_sales2#47, mon_sales2#48, tue_sales2#49, wed_sales2#50, thu_sales2#51, fri_sales2#52, sat_sales2#53] -Arguments: [d_week_seq2#46, sun_sales2#47, mon_sales2#48, tue_sales2#49, wed_sales2#50, thu_sales2#51, fri_sales2#52, sat_sales2#53] - -(30) CometBroadcastHashJoin -Left output [8]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36] -Right output [8]: [d_week_seq2#46, sun_sales2#47, mon_sales2#48, tue_sales2#49, wed_sales2#50, thu_sales2#51, fri_sales2#52, sat_sales2#53] -Arguments: [d_week_seq1#29], [(d_week_seq2#46 - 53)], Inner, BuildRight - -(31) CometProject -Input [16]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36, d_week_seq2#46, sun_sales2#47, mon_sales2#48, tue_sales2#49, wed_sales2#50, thu_sales2#51, fri_sales2#52, sat_sales2#53] -Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60], [d_week_seq1#29, round((sun_sales1#30 / sun_sales2#47), 2) AS round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1#31 / mon_sales2#48), 2) AS round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1#32 / tue_sales2#49), 2) AS round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1#33 / wed_sales2#50), 2) AS round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1#34 / thu_sales2#51), 2) AS round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1#35 / fri_sales2#52), 2) AS round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1#36 / sat_sales2#53), 2) AS round((sat_sales1 / sat_sales2), 2)#60] - -(32) CometExchange -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60] -Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometSort -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60] -Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60], [d_week_seq1#29 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 1] -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_datafusion/simplified.txt deleted file mode 100644 index e4b6e81639..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_datafusion/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] - CometExchange [d_week_seq1] #1 - CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] - CometBroadcastHashJoin [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] - CometExchange [d_week_seq] #2 - CometHashAggregate [d_day_name,sales_price] [d_week_seq,sum,sum,sum,sum,sum,sum,sum] - CometProject [sales_price,d_week_seq,d_day_name] - 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] - CometNativeScan 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] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] - CometBroadcastExchange [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] - ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 - CometBroadcastExchange [d_week_seq] #6 - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometNativeScan 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/q2.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_iceberg_compat/explain.txt deleted file mode 100644 index e13ae7371e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,193 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (34) -+- CometSort (33) - +- CometExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometHashAggregate (14) - : : +- CometExchange (13) - : : +- CometHashAggregate (12) - : : +- CometProject (11) - : : +- CometBroadcastHashJoin (10) - : : :- CometUnion (5) - : : : :- CometProject (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometProject (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (3) - : : +- CometBroadcastExchange (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (6) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (15) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometBroadcastHashJoin (27) - :- CometHashAggregate (22) - : +- ReusedExchange (21) - +- CometBroadcastExchange (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (23) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#2)] -ReadSchema: struct - -(2) CometProject -Input [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] -Arguments: [sold_date_sk#3, sales_price#4], [ws_sold_date_sk#2 AS sold_date_sk#3, ws_ext_sales_price#1 AS sales_price#4] - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#6)] -ReadSchema: struct - -(4) CometProject -Input [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] -Arguments: [sold_date_sk#7, sales_price#8], [cs_sold_date_sk#6 AS sold_date_sk#7, cs_ext_sales_price#5 AS sales_price#8] - -(5) CometUnion -Child 0 Input [2]: [sold_date_sk#3, sales_price#4] -Child 1 Input [2]: [sold_date_sk#7, sales_price#8] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Condition : (isnotnull(d_date_sk#9) AND isnotnull(d_week_seq#10)) - -(8) CometProject -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Arguments: [d_date_sk#9, d_week_seq#10, d_day_name#12], [d_date_sk#9, d_week_seq#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_day_name#11, 9, true, false, true) AS d_day_name#12] - -(9) CometBroadcastExchange -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [d_date_sk#9, d_week_seq#10, d_day_name#12] - -(10) CometBroadcastHashJoin -Left output [2]: [sold_date_sk#3, sales_price#4] -Right output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [sold_date_sk#3], [d_date_sk#9], Inner, BuildRight - -(11) CometProject -Input [5]: [sold_date_sk#3, sales_price#4, d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [sales_price#4, d_week_seq#10, d_day_name#12], [sales_price#4, d_week_seq#10, d_day_name#12] - -(12) CometHashAggregate -Input [3]: [sales_price#4, d_week_seq#10, d_day_name#12] -Keys [1]: [d_week_seq#10] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(13) CometExchange -Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(14) CometHashAggregate -Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Keys [1]: [d_week_seq#10] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_week_seq)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [d_week_seq#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_week_seq#20)) - -(17) CometProject -Input [2]: [d_week_seq#20, d_year#21] -Arguments: [d_week_seq#20], [d_week_seq#20] - -(18) CometBroadcastExchange -Input [1]: [d_week_seq#20] -Arguments: [d_week_seq#20] - -(19) CometBroadcastHashJoin -Left output [8]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28] -Right output [1]: [d_week_seq#20] -Arguments: [d_week_seq#10], [d_week_seq#20], Inner, BuildRight - -(20) CometProject -Input [9]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28, d_week_seq#20] -Arguments: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36], [d_week_seq#10 AS d_week_seq1#29, sun_sales#22 AS sun_sales1#30, mon_sales#23 AS mon_sales1#31, tue_sales#24 AS tue_sales1#32, wed_sales#25 AS wed_sales1#33, thu_sales#26 AS thu_sales1#34, fri_sales#27 AS fri_sales1#35, sat_sales#28 AS sat_sales1#36] - -(21) ReusedExchange [Reuses operator id: 13] -Output [8]: [d_week_seq#10, sum#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43] - -(22) CometHashAggregate -Input [8]: [d_week_seq#10, sum#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43] -Keys [1]: [d_week_seq#10] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [d_week_seq#44, d_year#45] -Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_week_seq#44)) - -(25) CometProject -Input [2]: [d_week_seq#44, d_year#45] -Arguments: [d_week_seq#44], [d_week_seq#44] - -(26) CometBroadcastExchange -Input [1]: [d_week_seq#44] -Arguments: [d_week_seq#44] - -(27) CometBroadcastHashJoin -Left output [8]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28] -Right output [1]: [d_week_seq#44] -Arguments: [d_week_seq#10], [d_week_seq#44], Inner, BuildRight - -(28) CometProject -Input [9]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28, d_week_seq#44] -Arguments: [d_week_seq2#46, sun_sales2#47, mon_sales2#48, tue_sales2#49, wed_sales2#50, thu_sales2#51, fri_sales2#52, sat_sales2#53], [d_week_seq#10 AS d_week_seq2#46, sun_sales#22 AS sun_sales2#47, mon_sales#23 AS mon_sales2#48, tue_sales#24 AS tue_sales2#49, wed_sales#25 AS wed_sales2#50, thu_sales#26 AS thu_sales2#51, fri_sales#27 AS fri_sales2#52, sat_sales#28 AS sat_sales2#53] - -(29) CometBroadcastExchange -Input [8]: [d_week_seq2#46, sun_sales2#47, mon_sales2#48, tue_sales2#49, wed_sales2#50, thu_sales2#51, fri_sales2#52, sat_sales2#53] -Arguments: [d_week_seq2#46, sun_sales2#47, mon_sales2#48, tue_sales2#49, wed_sales2#50, thu_sales2#51, fri_sales2#52, sat_sales2#53] - -(30) CometBroadcastHashJoin -Left output [8]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36] -Right output [8]: [d_week_seq2#46, sun_sales2#47, mon_sales2#48, tue_sales2#49, wed_sales2#50, thu_sales2#51, fri_sales2#52, sat_sales2#53] -Arguments: [d_week_seq1#29], [(d_week_seq2#46 - 53)], Inner, BuildRight - -(31) CometProject -Input [16]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36, d_week_seq2#46, sun_sales2#47, mon_sales2#48, tue_sales2#49, wed_sales2#50, thu_sales2#51, fri_sales2#52, sat_sales2#53] -Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60], [d_week_seq1#29, round((sun_sales1#30 / sun_sales2#47), 2) AS round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1#31 / mon_sales2#48), 2) AS round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1#32 / tue_sales2#49), 2) AS round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1#33 / wed_sales2#50), 2) AS round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1#34 / thu_sales2#51), 2) AS round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1#35 / fri_sales2#52), 2) AS round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1#36 / sat_sales2#53), 2) AS round((sat_sales1 / sat_sales2), 2)#60] - -(32) CometExchange -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60] -Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometSort -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60] -Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60], [d_week_seq1#29 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 1] -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_iceberg_compat/simplified.txt deleted file mode 100644 index 852c5fca0f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] - CometExchange [d_week_seq1] #1 - CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] - CometBroadcastHashJoin [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] - CometExchange [d_week_seq] #2 - CometHashAggregate [d_day_name,sales_price] [d_week_seq,sum,sum,sum,sum,sum,sum,sum] - CometProject [sales_price,d_week_seq,d_day_name] - 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 [native_iceberg_compat] 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year] - CometBroadcastExchange [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] - ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 - CometBroadcastExchange [d_week_seq] #6 - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometScan [native_iceberg_compat] 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/q2/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt deleted file mode 100644 index e13ae7371e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt +++ /dev/null @@ -1,193 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (34) -+- CometSort (33) - +- CometExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometHashAggregate (14) - : : +- CometExchange (13) - : : +- CometHashAggregate (12) - : : +- CometProject (11) - : : +- CometBroadcastHashJoin (10) - : : :- CometUnion (5) - : : : :- CometProject (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometProject (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (3) - : : +- CometBroadcastExchange (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (6) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (15) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometBroadcastHashJoin (27) - :- CometHashAggregate (22) - : +- ReusedExchange (21) - +- CometBroadcastExchange (26) - +- CometProject (25) - +- CometFilter (24) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (23) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#2)] -ReadSchema: struct - -(2) CometProject -Input [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] -Arguments: [sold_date_sk#3, sales_price#4], [ws_sold_date_sk#2 AS sold_date_sk#3, ws_ext_sales_price#1 AS sales_price#4] - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#6)] -ReadSchema: struct - -(4) CometProject -Input [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] -Arguments: [sold_date_sk#7, sales_price#8], [cs_sold_date_sk#6 AS sold_date_sk#7, cs_ext_sales_price#5 AS sales_price#8] - -(5) CometUnion -Child 0 Input [2]: [sold_date_sk#3, sales_price#4] -Child 1 Input [2]: [sold_date_sk#7, sales_price#8] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Condition : (isnotnull(d_date_sk#9) AND isnotnull(d_week_seq#10)) - -(8) CometProject -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] -Arguments: [d_date_sk#9, d_week_seq#10, d_day_name#12], [d_date_sk#9, d_week_seq#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_day_name#11, 9, true, false, true) AS d_day_name#12] - -(9) CometBroadcastExchange -Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [d_date_sk#9, d_week_seq#10, d_day_name#12] - -(10) CometBroadcastHashJoin -Left output [2]: [sold_date_sk#3, sales_price#4] -Right output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [sold_date_sk#3], [d_date_sk#9], Inner, BuildRight - -(11) CometProject -Input [5]: [sold_date_sk#3, sales_price#4, d_date_sk#9, d_week_seq#10, d_day_name#12] -Arguments: [sales_price#4, d_week_seq#10, d_day_name#12], [sales_price#4, d_week_seq#10, d_day_name#12] - -(12) CometHashAggregate -Input [3]: [sales_price#4, d_week_seq#10, d_day_name#12] -Keys [1]: [d_week_seq#10] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(13) CometExchange -Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(14) CometHashAggregate -Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Keys [1]: [d_week_seq#10] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_week_seq)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [d_week_seq#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_week_seq#20)) - -(17) CometProject -Input [2]: [d_week_seq#20, d_year#21] -Arguments: [d_week_seq#20], [d_week_seq#20] - -(18) CometBroadcastExchange -Input [1]: [d_week_seq#20] -Arguments: [d_week_seq#20] - -(19) CometBroadcastHashJoin -Left output [8]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28] -Right output [1]: [d_week_seq#20] -Arguments: [d_week_seq#10], [d_week_seq#20], Inner, BuildRight - -(20) CometProject -Input [9]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28, d_week_seq#20] -Arguments: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36], [d_week_seq#10 AS d_week_seq1#29, sun_sales#22 AS sun_sales1#30, mon_sales#23 AS mon_sales1#31, tue_sales#24 AS tue_sales1#32, wed_sales#25 AS wed_sales1#33, thu_sales#26 AS thu_sales1#34, fri_sales#27 AS fri_sales1#35, sat_sales#28 AS sat_sales1#36] - -(21) ReusedExchange [Reuses operator id: 13] -Output [8]: [d_week_seq#10, sum#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43] - -(22) CometHashAggregate -Input [8]: [d_week_seq#10, sum#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43] -Keys [1]: [d_week_seq#10] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [d_week_seq#44, d_year#45] -Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_week_seq#44)) - -(25) CometProject -Input [2]: [d_week_seq#44, d_year#45] -Arguments: [d_week_seq#44], [d_week_seq#44] - -(26) CometBroadcastExchange -Input [1]: [d_week_seq#44] -Arguments: [d_week_seq#44] - -(27) CometBroadcastHashJoin -Left output [8]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28] -Right output [1]: [d_week_seq#44] -Arguments: [d_week_seq#10], [d_week_seq#44], Inner, BuildRight - -(28) CometProject -Input [9]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28, d_week_seq#44] -Arguments: [d_week_seq2#46, sun_sales2#47, mon_sales2#48, tue_sales2#49, wed_sales2#50, thu_sales2#51, fri_sales2#52, sat_sales2#53], [d_week_seq#10 AS d_week_seq2#46, sun_sales#22 AS sun_sales2#47, mon_sales#23 AS mon_sales2#48, tue_sales#24 AS tue_sales2#49, wed_sales#25 AS wed_sales2#50, thu_sales#26 AS thu_sales2#51, fri_sales#27 AS fri_sales2#52, sat_sales#28 AS sat_sales2#53] - -(29) CometBroadcastExchange -Input [8]: [d_week_seq2#46, sun_sales2#47, mon_sales2#48, tue_sales2#49, wed_sales2#50, thu_sales2#51, fri_sales2#52, sat_sales2#53] -Arguments: [d_week_seq2#46, sun_sales2#47, mon_sales2#48, tue_sales2#49, wed_sales2#50, thu_sales2#51, fri_sales2#52, sat_sales2#53] - -(30) CometBroadcastHashJoin -Left output [8]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36] -Right output [8]: [d_week_seq2#46, sun_sales2#47, mon_sales2#48, tue_sales2#49, wed_sales2#50, thu_sales2#51, fri_sales2#52, sat_sales2#53] -Arguments: [d_week_seq1#29], [(d_week_seq2#46 - 53)], Inner, BuildRight - -(31) CometProject -Input [16]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36, d_week_seq2#46, sun_sales2#47, mon_sales2#48, tue_sales2#49, wed_sales2#50, thu_sales2#51, fri_sales2#52, sat_sales2#53] -Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60], [d_week_seq1#29, round((sun_sales1#30 / sun_sales2#47), 2) AS round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1#31 / mon_sales2#48), 2) AS round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1#32 / tue_sales2#49), 2) AS round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1#33 / wed_sales2#50), 2) AS round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1#34 / thu_sales2#51), 2) AS round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1#35 / fri_sales2#52), 2) AS round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1#36 / sat_sales2#53), 2) AS round((sat_sales1 / sat_sales2), 2)#60] - -(32) CometExchange -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60] -Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometSort -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60] -Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60], [d_week_seq1#29 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 1] -Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/extended.txt deleted file mode 100644 index 15f5db847d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/extended.txt +++ /dev/null @@ -1,48 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometProject - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 45 out of 45 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 852c5fca0f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] - CometExchange [d_week_seq1] #1 - CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] - CometBroadcastHashJoin [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] - CometExchange [d_week_seq] #2 - CometHashAggregate [d_day_name,sales_price] [d_week_seq,sum,sum,sum,sum,sum,sum,sum] - CometProject [sales_price,d_week_seq,d_day_name] - 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 [native_iceberg_compat] 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year] - CometBroadcastExchange [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] - ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 - CometBroadcastExchange [d_week_seq] #6 - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/explain.txt deleted file mode 100644 index c699bf18de..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/explain.txt +++ /dev/null @@ -1,163 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.item (4) - +- ReusedExchange (11) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(7) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(8) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(11) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] - -(14) HashAggregate [codegen id : 3] -Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(15) CometColumnarExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(17) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#17] -Results [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#17,17,2) AS _w0#19, i_item_id#11] - -(18) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(22) Project [codegen id : 6] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] - -(23) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometNativeScan parquet spark_catalog.default.date_dim (24) - - -(24) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#22] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/simplified.txt deleted file mode 100644 index fb0ed62abe..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/explain.txt deleted file mode 100644 index c23383e09a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [cs_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/simplified.txt deleted file mode 100644 index 2958d060fe..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt deleted file mode 100644 index c23383e09a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [cs_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/extended.txt deleted file mode 100644 index cd52b2cd12..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 2958d060fe..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/explain.txt deleted file mode 100644 index 65cbe8c435..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/explain.txt +++ /dev/null @@ -1,173 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (25) -+- * Filter (24) - +- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometColumnarExchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.inventory (1) - : : +- BroadcastExchange (7) - : : +- * CometColumnarToRow (6) - : : +- CometFilter (5) - : : +- CometNativeScan parquet spark_catalog.default.warehouse (4) - : +- BroadcastExchange (14) - : +- * CometColumnarToRow (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometNativeScan parquet spark_catalog.default.item (10) - +- ReusedExchange (17) - - -(1) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_warehouse_sk), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_warehouse_sk#2) AND isnotnull(inv_item_sk#1)) - -(4) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(6) CometColumnarToRow [codegen id : 1] -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] - -(7) BroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#6] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 4] -Output [4]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] -Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_sk#6, w_warehouse_name#7] - -(10) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Condition : (((isnotnull(i_current_price#10) AND (i_current_price#10 >= 0.99)) AND (i_current_price#10 <= 1.49)) AND isnotnull(i_item_sk#8)) - -(12) CometProject -Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Arguments: [i_item_sk#8, i_item_id#11], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#9, 16, true, false, true) AS i_item_id#11] - -(13) CometColumnarToRow [codegen id : 2] -Input [2]: [i_item_sk#8, i_item_id#11] - -(14) BroadcastExchange -Input [2]: [i_item_sk#8, i_item_id#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#8] -Join type: Inner -Join condition: None - -(16) Project [codegen id : 4] -Output [4]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11] -Input [6]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_sk#8, i_item_id#11] - -(17) ReusedExchange [Reuses operator id: 29] -Output [2]: [d_date_sk#12, d_date#13] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] -Input [6]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11, d_date_sk#12, d_date#13] - -(20) HashAggregate [codegen id : 4] -Input [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] -Keys [2]: [w_warehouse_name#7, i_item_id#11] -Functions [2]: [partial_sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), partial_sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] -Aggregate Attributes [2]: [sum#14, sum#15] -Results [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] - -(21) CometColumnarExchange -Input [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] -Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] - -(23) HashAggregate [codegen id : 5] -Input [4]: [w_warehouse_name#7, i_item_id#11, sum#16, sum#17] -Keys [2]: [w_warehouse_name#7, i_item_id#11] -Functions [2]: [sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] -Aggregate Attributes [2]: [sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18, sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#19] -Results [4]: [w_warehouse_name#7, i_item_id#11, sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18 AS inv_before#20, sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#19 AS inv_after#21] - -(24) Filter [codegen id : 5] -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#20, inv_after#21] -Condition : (CASE WHEN (inv_before#20 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#21 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#20 as double)))))) >= 0.666667) END AND CASE WHEN (inv_before#20 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#21 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#20 as double)))))) <= 1.5) END) - -(25) TakeOrderedAndProject -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#20, inv_after#21] -Arguments: 100, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#11, inv_before#20, inv_after#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometFilter (27) - +- CometNativeScan parquet spark_catalog.default.date_dim (26) - - -(26) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_date#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) - -(28) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_date#13] - -(29) BroadcastExchange -Input [2]: [d_date_sk#12, d_date#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/simplified.txt deleted file mode 100644 index 94925f8911..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] - WholeStageCodegen (5) - Filter [inv_before,inv_after] - HashAggregate [w_warehouse_name,i_item_id,sum,sum] [sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),inv_before,inv_after,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,i_item_id] #1 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] - Project [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Filter [inv_warehouse_sk,inv_item_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - InputAdapter - ReusedExchange [d_date_sk,d_date] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/explain.txt deleted file mode 100644 index 5656916223..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometFilter (22) - +- CometHashAggregate (21) - +- CometExchange (20) - +- CometHashAggregate (19) - +- CometProject (18) - +- CometBroadcastHashJoin (17) - :- CometProject (13) - : +- CometBroadcastHashJoin (12) - : :- CometProject (7) - : : +- CometBroadcastHashJoin (6) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : +- CometBroadcastExchange (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : +- CometBroadcastExchange (11) - : +- CometProject (10) - : +- CometFilter (9) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (8) - +- CometBroadcastExchange (16) - +- CometFilter (15) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_warehouse_sk), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_warehouse_sk#2) AND isnotnull(inv_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7], [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Condition : (((isnotnull(i_current_price#10) AND (i_current_price#10 >= 0.99)) AND (i_current_price#10 <= 1.49)) AND isnotnull(i_item_sk#8)) - -(10) CometProject -Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Arguments: [i_item_sk#8, i_item_id#11], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#9, 16, true, false, true) AS i_item_id#11] - -(11) CometBroadcastExchange -Input [2]: [i_item_sk#8, i_item_id#11] -Arguments: [i_item_sk#8, i_item_id#11] - -(12) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] -Right output [2]: [i_item_sk#8, i_item_id#11] -Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight - -(13) CometProject -Input [6]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_sk#8, i_item_id#11] -Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11], [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_date#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_date#13] -Arguments: [d_date_sk#12, d_date#13] - -(17) CometBroadcastHashJoin -Left output [4]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11] -Right output [2]: [d_date_sk#12, d_date#13] -Arguments: [inv_date_sk#4], [d_date_sk#12], Inner, BuildRight - -(18) CometProject -Input [6]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11, d_date_sk#12, d_date#13] -Arguments: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13], [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] - -(19) CometHashAggregate -Input [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] -Keys [2]: [w_warehouse_name#7, i_item_id#11] -Functions [2]: [partial_sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), partial_sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] - -(20) CometExchange -Input [4]: [w_warehouse_name#7, i_item_id#11, sum#14, sum#15] -Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [4]: [w_warehouse_name#7, i_item_id#11, sum#14, sum#15] -Keys [2]: [w_warehouse_name#7, i_item_id#11] -Functions [2]: [sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] - -(22) CometFilter -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] -Condition : (CASE WHEN (inv_before#16 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#17 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#16 as double)))))) >= 0.666667) END AND CASE WHEN (inv_before#16 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#17 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#16 as double)))))) <= 1.5) END) - -(23) CometTakeOrderedAndProject -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#7 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST], output=[w_warehouse_name#7,i_item_id#11,inv_before#16,inv_after#17]), [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17], 100, 0, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] - -(24) CometColumnarToRow [codegen id : 1] -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_date#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) - -(27) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_date#13] - -(28) BroadcastExchange -Input [2]: [d_date_sk#12, d_date#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/simplified.txt deleted file mode 100644 index 1c2e80c991..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,33 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] - CometFilter [w_warehouse_name,i_item_id,inv_before,inv_after] - CometHashAggregate [sum,sum] [w_warehouse_name,i_item_id,inv_before,inv_after,sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END)] - CometExchange [w_warehouse_name,i_item_id] #1 - CometHashAggregate [d_date,inv_quantity_on_hand] [w_warehouse_name,i_item_id,sum,sum] - CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - 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,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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_id] #4 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange [d_date_sk,d_date] #5 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt deleted file mode 100644 index 5656916223..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometFilter (22) - +- CometHashAggregate (21) - +- CometExchange (20) - +- CometHashAggregate (19) - +- CometProject (18) - +- CometBroadcastHashJoin (17) - :- CometProject (13) - : +- CometBroadcastHashJoin (12) - : :- CometProject (7) - : : +- CometBroadcastHashJoin (6) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : +- CometBroadcastExchange (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : +- CometBroadcastExchange (11) - : +- CometProject (10) - : +- CometFilter (9) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (8) - +- CometBroadcastExchange (16) - +- CometFilter (15) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_warehouse_sk), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_warehouse_sk#2) AND isnotnull(inv_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7], [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Condition : (((isnotnull(i_current_price#10) AND (i_current_price#10 >= 0.99)) AND (i_current_price#10 <= 1.49)) AND isnotnull(i_item_sk#8)) - -(10) CometProject -Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] -Arguments: [i_item_sk#8, i_item_id#11], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#9, 16, true, false, true) AS i_item_id#11] - -(11) CometBroadcastExchange -Input [2]: [i_item_sk#8, i_item_id#11] -Arguments: [i_item_sk#8, i_item_id#11] - -(12) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] -Right output [2]: [i_item_sk#8, i_item_id#11] -Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight - -(13) CometProject -Input [6]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_sk#8, i_item_id#11] -Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11], [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_date#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_date#13] -Arguments: [d_date_sk#12, d_date#13] - -(17) CometBroadcastHashJoin -Left output [4]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11] -Right output [2]: [d_date_sk#12, d_date#13] -Arguments: [inv_date_sk#4], [d_date_sk#12], Inner, BuildRight - -(18) CometProject -Input [6]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#11, d_date_sk#12, d_date#13] -Arguments: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13], [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] - -(19) CometHashAggregate -Input [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#11, d_date#13] -Keys [2]: [w_warehouse_name#7, i_item_id#11] -Functions [2]: [partial_sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), partial_sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] - -(20) CometExchange -Input [4]: [w_warehouse_name#7, i_item_id#11, sum#14, sum#15] -Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [4]: [w_warehouse_name#7, i_item_id#11, sum#14, sum#15] -Keys [2]: [w_warehouse_name#7, i_item_id#11] -Functions [2]: [sum(CASE WHEN (d_date#13 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), sum(CASE WHEN (d_date#13 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] - -(22) CometFilter -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] -Condition : (CASE WHEN (inv_before#16 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#17 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#16 as double)))))) >= 0.666667) END AND CASE WHEN (inv_before#16 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(inv_after#17 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(inv_before#16 as double)))))) <= 1.5) END) - -(23) CometTakeOrderedAndProject -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#7 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST], output=[w_warehouse_name#7,i_item_id#11,inv_before#16,inv_after#17]), [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17], 100, 0, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] - -(24) CometColumnarToRow [codegen id : 1] -Input [4]: [w_warehouse_name#7, i_item_id#11, inv_before#16, inv_after#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_date#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-10)) AND (d_date#13 <= 2000-04-10)) AND isnotnull(d_date_sk#12)) - -(27) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_date#13] - -(28) BroadcastExchange -Input [2]: [d_date_sk#12, d_date#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/extended.txt deleted file mode 100644 index ef02334507..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 1c2e80c991..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/simplified.txt +++ /dev/null @@ -1,33 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] - CometFilter [w_warehouse_name,i_item_id,inv_before,inv_after] - CometHashAggregate [sum,sum] [w_warehouse_name,i_item_id,inv_before,inv_after,sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END)] - CometExchange [w_warehouse_name,i_item_id] #1 - CometHashAggregate [d_date,inv_quantity_on_hand] [w_warehouse_name,i_item_id,sum,sum] - CometProject [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] - 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,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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #3 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_id] #4 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange [d_date_sk,d_date] #5 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/explain.txt deleted file mode 100644 index f25f305679..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/explain.txt +++ /dev/null @@ -1,178 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (25) -+- * HashAggregate (24) - +- * CometColumnarToRow (23) - +- CometColumnarExchange (22) - +- * HashAggregate (21) - +- * Expand (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.inventory (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) - : +- CometProject (9) - : +- CometFilter (8) - : +- CometNativeScan parquet spark_catalog.default.item (7) - +- BroadcastExchange (17) - +- * CometColumnarToRow (16) - +- CometFilter (15) - +- CometNativeScan parquet spark_catalog.default.warehouse (14) - - -(1) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(4) ReusedExchange [Reuses operator id: 30] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(8) CometFilter -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Condition : isnotnull(i_item_sk#7) - -(9) CometProject -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#8, 50, true, false, true) AS i_brand#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#11, 50, true, false, true) AS i_product_name#15] - -(10) CometColumnarToRow [codegen id : 2] -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(11) BroadcastExchange -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(14) CometNativeScan parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(15) CometFilter -Input [1]: [w_warehouse_sk#16] -Condition : isnotnull(w_warehouse_sk#16) - -(16) CometColumnarToRow [codegen id : 3] -Input [1]: [w_warehouse_sk#16] - -(17) BroadcastExchange -Input [1]: [w_warehouse_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#16] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, i_category#14] -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15, w_warehouse_sk#16] - -(20) Expand [codegen id : 4] -Input [5]: [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, i_category#14] -Arguments: [[inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, i_category#14, 0], [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, i_class#13, null, 1], [inv_quantity_on_hand#3, i_product_name#15, i_brand#12, null, null, 3], [inv_quantity_on_hand#3, i_product_name#15, null, null, null, 7], [inv_quantity_on_hand#3, null, null, null, null, 15]], [inv_quantity_on_hand#3, i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] - -(21) HashAggregate [codegen id : 4] -Input [6]: [inv_quantity_on_hand#3, i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] -Keys [5]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] -Functions [1]: [partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [sum#22, count#23] -Results [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, sum#24, count#25] - -(22) CometColumnarExchange -Input [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, sum#24, count#25] -Arguments: hashpartitioning(i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 5] -Input [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, sum#24, count#25] - -(24) HashAggregate [codegen id : 5] -Input [7]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21, sum#24, count#25] -Keys [5]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, spark_grouping_id#21] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#26] -Results [5]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, avg(inv_quantity_on_hand#3)#26 AS qoh#27] - -(25) TakeOrderedAndProject -Input [5]: [i_product_name#17, i_brand#18, i_class#19, i_category#20, qoh#27] -Arguments: 100, [qoh#27 ASC NULLS FIRST, i_product_name#17 ASC NULLS FIRST, i_brand#18 ASC NULLS FIRST, i_class#19 ASC NULLS FIRST, i_category#20 ASC NULLS FIRST], [i_product_name#17, i_brand#18, i_class#19, i_category#20, qoh#27] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometNativeScan parquet spark_catalog.default.date_dim (26) - - -(26) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#28] -Condition : (((isnotnull(d_month_seq#28) AND (d_month_seq#28 >= 1200)) AND (d_month_seq#28 <= 1211)) AND isnotnull(d_date_sk#6)) - -(28) CometProject -Input [2]: [d_date_sk#6, d_month_seq#28] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(30) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/simplified.txt deleted file mode 100644 index 092e187177..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/simplified.txt +++ /dev/null @@ -1,44 +0,0 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (5) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/explain.txt deleted file mode 100644 index 02cfd90b87..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,178 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometTakeOrderedAndProject (24) - +- CometHashAggregate (23) - +- CometExchange (22) - +- CometHashAggregate (21) - +- CometExpand (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [inv_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] -Arguments: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3], [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Condition : isnotnull(i_item_sk#8) - -(11) CometProject -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#12, 50, true, false, true) AS i_product_name#16] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(13) CometBroadcastHashJoin -Left output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -Right output [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight - -(14) CometProject -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [w_warehouse_sk#17] -Condition : isnotnull(w_warehouse_sk#17) - -(17) CometBroadcastExchange -Input [1]: [w_warehouse_sk#17] -Arguments: [w_warehouse_sk#17] - -(18) CometBroadcastHashJoin -Left output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Right output [1]: [w_warehouse_sk#17] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#17], Inner, BuildRight - -(19) CometProject -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16, w_warehouse_sk#17] -Arguments: [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15], [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15] - -(20) CometExpand -Input [5]: [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15] -Arguments: [[inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15, 0], [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, null, 1], [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, null, null, 3], [inv_quantity_on_hand#3, i_product_name#16, null, null, null, 7], [inv_quantity_on_hand#3, null, null, null, null, 15]], [inv_quantity_on_hand#3, i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] - -(21) CometHashAggregate -Input [6]: [inv_quantity_on_hand#3, i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] -Keys [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] -Functions [1]: [partial_avg(inv_quantity_on_hand#3)] - -(22) CometExchange -Input [7]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22, sum#23, count#24] -Arguments: hashpartitioning(i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [7]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22, sum#23, count#24] -Keys [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] -Functions [1]: [avg(inv_quantity_on_hand#3)] - -(24) CometTakeOrderedAndProject -Input [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#25 ASC NULLS FIRST,i_product_name#18 ASC NULLS FIRST,i_brand#19 ASC NULLS FIRST,i_class#20 ASC NULLS FIRST,i_category#21 ASC NULLS FIRST], output=[i_product_name#18,i_brand#19,i_class#20,i_category#21,qoh#25]), [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25], 100, 0, [qoh#25 ASC NULLS FIRST, i_product_name#18 ASC NULLS FIRST, i_brand#19 ASC NULLS FIRST, i_class#20 ASC NULLS FIRST, i_category#21 ASC NULLS FIRST], [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] - -(25) CometColumnarToRow [codegen id : 1] -Input [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6)) - -(28) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(30) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/simplified.txt deleted file mode 100644 index 9119ee749d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] - CometHashAggregate [spark_grouping_id,sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(inv_quantity_on_hand)] - CometExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - CometHashAggregate [inv_quantity_on_hand] [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] - 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,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,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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange [w_warehouse_sk] #5 - CometFilter [w_warehouse_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt deleted file mode 100644 index 02cfd90b87..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt +++ /dev/null @@ -1,178 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometTakeOrderedAndProject (24) - +- CometHashAggregate (23) - +- CometExchange (22) - +- CometHashAggregate (21) - +- CometExpand (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [inv_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] -Arguments: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3], [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Condition : isnotnull(i_item_sk#8) - -(11) CometProject -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#12, 50, true, false, true) AS i_product_name#16] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(13) CometBroadcastHashJoin -Left output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -Right output [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight - -(14) CometProject -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [w_warehouse_sk#17] -Condition : isnotnull(w_warehouse_sk#17) - -(17) CometBroadcastExchange -Input [1]: [w_warehouse_sk#17] -Arguments: [w_warehouse_sk#17] - -(18) CometBroadcastHashJoin -Left output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Right output [1]: [w_warehouse_sk#17] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#17], Inner, BuildRight - -(19) CometProject -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16, w_warehouse_sk#17] -Arguments: [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15], [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15] - -(20) CometExpand -Input [5]: [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15] -Arguments: [[inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, i_category#15, 0], [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, i_class#14, null, 1], [inv_quantity_on_hand#3, i_product_name#16, i_brand#13, null, null, 3], [inv_quantity_on_hand#3, i_product_name#16, null, null, null, 7], [inv_quantity_on_hand#3, null, null, null, null, 15]], [inv_quantity_on_hand#3, i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] - -(21) CometHashAggregate -Input [6]: [inv_quantity_on_hand#3, i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] -Keys [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] -Functions [1]: [partial_avg(inv_quantity_on_hand#3)] - -(22) CometExchange -Input [7]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22, sum#23, count#24] -Arguments: hashpartitioning(i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [7]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22, sum#23, count#24] -Keys [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, spark_grouping_id#22] -Functions [1]: [avg(inv_quantity_on_hand#3)] - -(24) CometTakeOrderedAndProject -Input [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#25 ASC NULLS FIRST,i_product_name#18 ASC NULLS FIRST,i_brand#19 ASC NULLS FIRST,i_class#20 ASC NULLS FIRST,i_category#21 ASC NULLS FIRST], output=[i_product_name#18,i_brand#19,i_class#20,i_category#21,qoh#25]), [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25], 100, 0, [qoh#25 ASC NULLS FIRST, i_product_name#18 ASC NULLS FIRST, i_brand#19 ASC NULLS FIRST, i_class#20 ASC NULLS FIRST, i_category#21 ASC NULLS FIRST], [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] - -(25) CometColumnarToRow [codegen id : 1] -Input [5]: [i_product_name#18, i_brand#19, i_class#20, i_category#21, qoh#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#6)) - -(28) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(30) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/extended.txt deleted file mode 100644 index 7369619d2f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/extended.txt +++ /dev/null @@ -1,33 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - -Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 9119ee749d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] - CometHashAggregate [spark_grouping_id,sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(inv_quantity_on_hand)] - CometExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - CometHashAggregate [inv_quantity_on_hand] [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] - 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,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,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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange [w_warehouse_sk] #5 - CometFilter [w_warehouse_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/explain.txt deleted file mode 100644 index 0170395540..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/explain.txt +++ /dev/null @@ -1,602 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (69) -+- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometUnion (65) - :- CometProject (47) - : +- CometBroadcastHashJoin (46) - : :- CometProject (41) - : : +- CometSortMergeJoin (40) - : : :- CometSort (25) - : : : +- CometColumnarExchange (24) - : : : +- * Project (23) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (22) - : : : :- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : +- BroadcastExchange (21) - : : : +- * Project (20) - : : : +- * Filter (19) - : : : +- * HashAggregate (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometColumnarExchange (16) - : : : +- * HashAggregate (15) - : : : +- * Project (14) - : : : +- * BroadcastHashJoin Inner BuildRight (13) - : : : :- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * Filter (5) - : : : : : +- * ColumnarToRow (4) - : : : : : +- Scan parquet spark_catalog.default.store_sales (3) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (12) - : : : +- * CometColumnarToRow (11) - : : : +- CometFilter (10) - : : : +- CometNativeScan parquet spark_catalog.default.item (9) - : : +- CometSort (39) - : : +- CometProject (38) - : : +- CometFilter (37) - : : +- CometHashAggregate (36) - : : +- CometExchange (35) - : : +- CometHashAggregate (34) - : : +- CometProject (33) - : : +- CometBroadcastHashJoin (32) - : : :- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometNativeScan parquet spark_catalog.default.store_sales (26) - : : +- CometBroadcastExchange (31) - : : +- CometFilter (30) - : : +- CometNativeScan parquet spark_catalog.default.customer (29) - : +- CometBroadcastExchange (45) - : +- CometProject (44) - : +- CometFilter (43) - : +- CometNativeScan parquet spark_catalog.default.date_dim (42) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometSortMergeJoin (60) - : :- CometSort (54) - : : +- CometColumnarExchange (53) - : : +- * Project (52) - : : +- * BroadcastHashJoin LeftSemi BuildRight (51) - : : :- * ColumnarToRow (49) - : : : +- Scan parquet spark_catalog.default.web_sales (48) - : : +- ReusedExchange (50) - : +- CometSort (59) - : +- CometProject (58) - : +- CometFilter (57) - : +- CometHashAggregate (56) - : +- ReusedExchange (55) - +- ReusedExchange (62) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(3) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(4) ColumnarToRow [codegen id : 3] -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] - -(5) Filter [codegen id : 3] -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_item_sk#7) - -(6) ReusedExchange [Reuses operator id: 79] -Output [2]: [d_date_sk#10, d_date#11] - -(7) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(8) Project [codegen id : 3] -Output [2]: [ss_item_sk#7, d_date#11] -Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] - -(9) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#12, i_item_desc#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [i_item_sk#12, i_item_desc#13] -Condition : isnotnull(i_item_sk#12) - -(11) CometColumnarToRow [codegen id : 2] -Input [2]: [i_item_sk#12, i_item_desc#13] - -(12) BroadcastExchange -Input [2]: [i_item_sk#12, i_item_desc#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#7] -Right keys [1]: [i_item_sk#12] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 3] -Output [3]: [d_date#11, i_item_sk#12, substr(i_item_desc#13, 1, 30) AS _groupingexpression#14] -Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#12, i_item_desc#13] - -(15) HashAggregate [codegen id : 3] -Input [3]: [d_date#11, i_item_sk#12, _groupingexpression#14] -Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#15] -Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] - -(16) CometColumnarExchange -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(17) CometColumnarToRow [codegen id : 4] -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] - -(18) HashAggregate [codegen id : 4] -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#17] -Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] - -(19) Filter [codegen id : 4] -Input [2]: [item_sk#18, cnt#19] -Condition : (cnt#19 > 4) - -(20) Project [codegen id : 4] -Output [1]: [item_sk#18] -Input [2]: [item_sk#18, cnt#19] - -(21) BroadcastExchange -Input [1]: [item_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(22) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [item_sk#18] -Join type: LeftSemi -Join condition: None - -(23) Project [codegen id : 5] -Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(24) CometColumnarExchange -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] - -(26) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_customer_sk#20) - -(28) CometProject -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -Arguments: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22], [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] - -(29) CometNativeScan parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(30) CometFilter -Input [1]: [c_customer_sk#24] -Condition : isnotnull(c_customer_sk#24) - -(31) CometBroadcastExchange -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24] - -(32) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] -Right output [1]: [c_customer_sk#24] -Arguments: [ss_customer_sk#20], [c_customer_sk#24], Inner, BuildRight - -(33) CometProject -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -Arguments: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24], [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] - -(34) CometHashAggregate -Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -Keys [1]: [c_customer_sk#24] -Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] - -(35) CometExchange -Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] -Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(36) CometHashAggregate -Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] -Keys [1]: [c_customer_sk#24] -Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] - -(37) CometFilter -Input [2]: [c_customer_sk#24, ssales#27] -Condition : (isnotnull(ssales#27) AND (cast(ssales#27 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#28, [id=#29]))) - -(38) CometProject -Input [2]: [c_customer_sk#24, ssales#27] -Arguments: [c_customer_sk#24], [c_customer_sk#24] - -(39) CometSort -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24], [c_customer_sk#24 ASC NULLS FIRST] - -(40) CometSortMergeJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [c_customer_sk#24] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#24], LeftSemi - -(41) CometProject -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(42) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#30, d_year#31, d_moy#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [3]: [d_date_sk#30, d_year#31, d_moy#32] -Condition : ((((isnotnull(d_year#31) AND isnotnull(d_moy#32)) AND (d_year#31 = 2000)) AND (d_moy#32 = 2)) AND isnotnull(d_date_sk#30)) - -(44) CometProject -Input [3]: [d_date_sk#30, d_year#31, d_moy#32] -Arguments: [d_date_sk#30], [d_date_sk#30] - -(45) CometBroadcastExchange -Input [1]: [d_date_sk#30] -Arguments: [d_date_sk#30] - -(46) CometBroadcastHashJoin -Left output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [d_date_sk#30] -Arguments: [cs_sold_date_sk#5], [d_date_sk#30], Inner, BuildRight - -(47) CometProject -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#30] -Arguments: [sales#33], [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#33] - -(48) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#34, ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#6)] -ReadSchema: struct - -(49) ColumnarToRow [codegen id : 10] -Input [5]: [ws_item_sk#34, ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] - -(50) ReusedExchange [Reuses operator id: 21] -Output [1]: [item_sk#18] - -(51) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [item_sk#18] -Join type: LeftSemi -Join condition: None - -(52) Project [codegen id : 10] -Output [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -Input [5]: [ws_item_sk#34, ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] - -(53) CometColumnarExchange -Input [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -Arguments: hashpartitioning(ws_bill_customer_sk#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(54) CometSort -Input [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -Arguments: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38], [ws_bill_customer_sk#35 ASC NULLS FIRST] - -(55) ReusedExchange [Reuses operator id: 35] -Output [3]: [c_customer_sk#24, sum#25, isEmpty#26] - -(56) CometHashAggregate -Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] -Keys [1]: [c_customer_sk#24] -Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] - -(57) CometFilter -Input [2]: [c_customer_sk#24, ssales#27] -Condition : (isnotnull(ssales#27) AND (cast(ssales#27 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#28, [id=#29]))) - -(58) CometProject -Input [2]: [c_customer_sk#24, ssales#27] -Arguments: [c_customer_sk#24], [c_customer_sk#24] - -(59) CometSort -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24], [c_customer_sk#24 ASC NULLS FIRST] - -(60) CometSortMergeJoin -Left output [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -Right output [1]: [c_customer_sk#24] -Arguments: [ws_bill_customer_sk#35], [c_customer_sk#24], LeftSemi - -(61) CometProject -Input [4]: [ws_bill_customer_sk#35, ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -Arguments: [ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38], [ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] - -(62) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#39] - -(63) CometBroadcastHashJoin -Left output [3]: [ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38] -Right output [1]: [d_date_sk#39] -Arguments: [ws_sold_date_sk#38], [d_date_sk#39], Inner, BuildRight - -(64) CometProject -Input [4]: [ws_quantity#36, ws_list_price#37, ws_sold_date_sk#38, d_date_sk#39] -Arguments: [sales#40], [(cast(ws_quantity#36 as decimal(10,0)) * ws_list_price#37) AS sales#40] - -(65) CometUnion -Child 0 Input [1]: [sales#33] -Child 1 Input [1]: [sales#40] - -(66) CometHashAggregate -Input [1]: [sales#33] -Keys: [] -Functions [1]: [partial_sum(sales#33)] - -(67) CometExchange -Input [2]: [sum#41, isEmpty#42] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(68) CometHashAggregate -Input [2]: [sum#41, isEmpty#42] -Keys: [] -Functions [1]: [sum(sales#33)] - -(69) CometColumnarToRow [codegen id : 11] -Input [1]: [sum(sales)#43] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (74) -+- * CometColumnarToRow (73) - +- CometProject (72) - +- CometFilter (71) - +- CometNativeScan parquet spark_catalog.default.date_dim (70) - - -(70) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#30, d_year#31, d_moy#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(71) CometFilter -Input [3]: [d_date_sk#30, d_year#31, d_moy#32] -Condition : ((((isnotnull(d_year#31) AND isnotnull(d_moy#32)) AND (d_year#31 = 2000)) AND (d_moy#32 = 2)) AND isnotnull(d_date_sk#30)) - -(72) CometProject -Input [3]: [d_date_sk#30, d_year#31, d_moy#32] -Arguments: [d_date_sk#30], [d_date_sk#30] - -(73) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#30] - -(74) BroadcastExchange -Input [1]: [d_date_sk#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (79) -+- * CometColumnarToRow (78) - +- CometProject (77) - +- CometFilter (76) - +- CometNativeScan parquet spark_catalog.default.date_dim (75) - - -(75) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(76) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#44] -Condition : (d_year#44 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(77) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#44] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(78) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_date#11] - -(79) BroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:3 Hosting operator id = 37 Hosting Expression = Subquery scalar-subquery#28, [id=#29] -* HashAggregate (99) -+- * CometColumnarToRow (98) - +- CometColumnarExchange (97) - +- * HashAggregate (96) - +- * HashAggregate (95) - +- * CometColumnarToRow (94) - +- CometColumnarExchange (93) - +- * HashAggregate (92) - +- * Project (91) - +- * BroadcastHashJoin Inner BuildRight (90) - :- * Project (88) - : +- * BroadcastHashJoin Inner BuildRight (87) - : :- * Filter (82) - : : +- * ColumnarToRow (81) - : : +- Scan parquet spark_catalog.default.store_sales (80) - : +- BroadcastExchange (86) - : +- * CometColumnarToRow (85) - : +- CometFilter (84) - : +- CometNativeScan parquet spark_catalog.default.customer (83) - +- ReusedExchange (89) - - -(80) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#45, ss_quantity#46, ss_sales_price#47, ss_sold_date_sk#48] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#48), dynamicpruningexpression(ss_sold_date_sk#48 IN dynamicpruning#49)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(81) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#45, ss_quantity#46, ss_sales_price#47, ss_sold_date_sk#48] - -(82) Filter [codegen id : 3] -Input [4]: [ss_customer_sk#45, ss_quantity#46, ss_sales_price#47, ss_sold_date_sk#48] -Condition : isnotnull(ss_customer_sk#45) - -(83) CometNativeScan parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(84) CometFilter -Input [1]: [c_customer_sk#50] -Condition : isnotnull(c_customer_sk#50) - -(85) CometColumnarToRow [codegen id : 1] -Input [1]: [c_customer_sk#50] - -(86) BroadcastExchange -Input [1]: [c_customer_sk#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -(87) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#45] -Right keys [1]: [c_customer_sk#50] -Join type: Inner -Join condition: None - -(88) Project [codegen id : 3] -Output [4]: [ss_quantity#46, ss_sales_price#47, ss_sold_date_sk#48, c_customer_sk#50] -Input [5]: [ss_customer_sk#45, ss_quantity#46, ss_sales_price#47, ss_sold_date_sk#48, c_customer_sk#50] - -(89) ReusedExchange [Reuses operator id: 104] -Output [1]: [d_date_sk#51] - -(90) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#48] -Right keys [1]: [d_date_sk#51] -Join type: Inner -Join condition: None - -(91) Project [codegen id : 3] -Output [3]: [ss_quantity#46, ss_sales_price#47, c_customer_sk#50] -Input [5]: [ss_quantity#46, ss_sales_price#47, ss_sold_date_sk#48, c_customer_sk#50, d_date_sk#51] - -(92) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#46, ss_sales_price#47, c_customer_sk#50] -Keys [1]: [c_customer_sk#50] -Functions [1]: [partial_sum((cast(ss_quantity#46 as decimal(10,0)) * ss_sales_price#47))] -Aggregate Attributes [2]: [sum#52, isEmpty#53] -Results [3]: [c_customer_sk#50, sum#54, isEmpty#55] - -(93) CometColumnarExchange -Input [3]: [c_customer_sk#50, sum#54, isEmpty#55] -Arguments: hashpartitioning(c_customer_sk#50, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(94) CometColumnarToRow [codegen id : 4] -Input [3]: [c_customer_sk#50, sum#54, isEmpty#55] - -(95) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#50, sum#54, isEmpty#55] -Keys [1]: [c_customer_sk#50] -Functions [1]: [sum((cast(ss_quantity#46 as decimal(10,0)) * ss_sales_price#47))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#46 as decimal(10,0)) * ss_sales_price#47))#56] -Results [1]: [sum((cast(ss_quantity#46 as decimal(10,0)) * ss_sales_price#47))#56 AS csales#57] - -(96) HashAggregate [codegen id : 4] -Input [1]: [csales#57] -Keys: [] -Functions [1]: [partial_max(csales#57)] -Aggregate Attributes [1]: [max#58] -Results [1]: [max#59] - -(97) CometColumnarExchange -Input [1]: [max#59] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(98) CometColumnarToRow [codegen id : 5] -Input [1]: [max#59] - -(99) HashAggregate [codegen id : 5] -Input [1]: [max#59] -Keys: [] -Functions [1]: [max(csales#57)] -Aggregate Attributes [1]: [max(csales#57)#60] -Results [1]: [max(csales#57)#60 AS tpcds_cmax#61] - -Subquery:4 Hosting operator id = 80 Hosting Expression = ss_sold_date_sk#48 IN dynamicpruning#49 -BroadcastExchange (104) -+- * CometColumnarToRow (103) - +- CometProject (102) - +- CometFilter (101) - +- CometNativeScan parquet spark_catalog.default.date_dim (100) - - -(100) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#51, d_year#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(101) CometFilter -Input [2]: [d_date_sk#51, d_year#62] -Condition : (d_year#62 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#51)) - -(102) CometProject -Input [2]: [d_date_sk#51, d_year#62] -Arguments: [d_date_sk#51], [d_date_sk#51] - -(103) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#51] - -(104) BroadcastExchange -Input [1]: [d_date_sk#51] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:5 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#6 - -Subquery:6 Hosting operator id = 57 Hosting Expression = ReusedSubquery Subquery scalar-subquery#28, [id=#29] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/simplified.txt deleted file mode 100644 index d42a6ba29e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/simplified.txt +++ /dev/null @@ -1,142 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,isEmpty] [sum(sales),sum(sales)] - CometExchange #1 - CometHashAggregate [sales] [sum,isEmpty] - CometUnion [sales] - CometProject [cs_quantity,cs_list_price] [sales] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometColumnarExchange [cs_bill_customer_sk] #2 - WholeStageCodegen (5) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [item_sk] - Filter [cnt] - HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 - WholeStageCodegen (3) - HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (5) - HashAggregate [max] [max(csales),tpcds_cmax,max] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (4) - HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_sk] #11 - WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #12 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] - InputAdapter - ReusedExchange [d_date_sk] #12 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #8 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,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,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #9 - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [ws_quantity,ws_list_price] [sales] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometColumnarExchange [ws_bill_customer_sk] #15 - WholeStageCodegen (10) - Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - BroadcastHashJoin [ws_item_sk,item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #8 - ReusedExchange [d_date_sk] #14 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/explain.txt deleted file mode 100644 index bce5d2805d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,576 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (67) -+- CometHashAggregate (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometUnion (63) - :- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (40) - : : +- CometSortMergeJoin (39) - : : :- CometSort (24) - : : : +- CometExchange (23) - : : : +- CometProject (22) - : : : +- CometBroadcastHashJoin (21) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometBroadcastExchange (20) - : : : +- CometProject (19) - : : : +- CometFilter (18) - : : : +- CometHashAggregate (17) - : : : +- CometExchange (16) - : : : +- CometHashAggregate (15) - : : : +- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometFilter (3) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (2) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometFilter (11) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (10) - : : +- CometSort (38) - : : +- CometProject (37) - : : +- CometFilter (36) - : : +- CometHashAggregate (35) - : : +- CometExchange (34) - : : +- CometHashAggregate (33) - : : +- CometProject (32) - : : +- CometBroadcastHashJoin (31) - : : :- CometProject (27) - : : : +- CometFilter (26) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (25) - : : +- CometBroadcastExchange (30) - : : +- CometFilter (29) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (28) - : +- CometBroadcastExchange (44) - : +- CometProject (43) - : +- CometFilter (42) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - +- CometProject (62) - +- CometBroadcastHashJoin (61) - :- CometProject (59) - : +- CometSortMergeJoin (58) - : :- CometSort (52) - : : +- CometExchange (51) - : : +- CometProject (50) - : : +- CometBroadcastHashJoin (49) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : : +- ReusedExchange (48) - : +- CometSort (57) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometHashAggregate (54) - : +- ReusedExchange (53) - +- ReusedExchange (60) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(2) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(3) CometFilter -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_item_sk#7) - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(6) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(7) CometBroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: [d_date_sk#10, d_date#11] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Right output [2]: [d_date_sk#10, d_date#11] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] -Arguments: [ss_item_sk#7, d_date#11], [ss_item_sk#7, d_date#11] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#13, i_item_desc#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [i_item_sk#13, i_item_desc#14] -Condition : isnotnull(i_item_sk#13) - -(12) CometBroadcastExchange -Input [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [i_item_sk#13, i_item_desc#14] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, d_date#11] -Right output [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [ss_item_sk#7], [i_item_sk#13], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#13, i_item_desc#14] -Arguments: [d_date#11, i_item_sk#13, _groupingexpression#15], [d_date#11, i_item_sk#13, substr(i_item_desc#14, 1, 30) AS _groupingexpression#15] - -(15) CometHashAggregate -Input [3]: [d_date#11, i_item_sk#13, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [partial_count(1)] - -(16) CometExchange -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [count(1)] - -(18) CometFilter -Input [2]: [item_sk#17, cnt#18] -Condition : (cnt#18 > 4) - -(19) CometProject -Input [2]: [item_sk#17, cnt#18] -Arguments: [item_sk#17], [item_sk#17] - -(20) CometBroadcastExchange -Input [1]: [item_sk#17] -Arguments: [item_sk#17] - -(21) CometBroadcastHashJoin -Left output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [item_sk#17] -Arguments: [cs_item_sk#2], [item_sk#17], LeftSemi, BuildRight - -(22) CometProject -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(23) CometExchange -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#19) - -(27) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Arguments: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21], [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(29) CometFilter -Input [1]: [c_customer_sk#23] -Condition : isnotnull(c_customer_sk#23) - -(30) CometBroadcastExchange -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23] - -(31) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] -Right output [1]: [c_customer_sk#23] -Arguments: [ss_customer_sk#19], [c_customer_sk#23], Inner, BuildRight - -(32) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Arguments: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23], [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] - -(33) CometHashAggregate -Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Keys [1]: [c_customer_sk#23] -Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(34) CometExchange -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(35) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(36) CometFilter -Input [2]: [c_customer_sk#23, ssales#26] -Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#27, [id=#28]))) - -(37) CometProject -Input [2]: [c_customer_sk#23, ssales#26] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(38) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(39) CometSortMergeJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [c_customer_sk#23] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#23], LeftSemi - -(40) CometProject -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#29, d_year#30, d_moy#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Condition : ((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2000)) AND (d_moy#31 = 2)) AND isnotnull(d_date_sk#29)) - -(43) CometProject -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Arguments: [d_date_sk#29], [d_date_sk#29] - -(44) CometBroadcastExchange -Input [1]: [d_date_sk#29] -Arguments: [d_date_sk#29] - -(45) CometBroadcastHashJoin -Left output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [d_date_sk#29] -Arguments: [cs_sold_date_sk#5], [d_date_sk#29], Inner, BuildRight - -(46) CometProject -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#29] -Arguments: [sales#32], [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#32] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_sold_date_sk#37 IN dynamicpruning#38)] -ReadSchema: struct - -(48) ReusedExchange [Reuses operator id: 20] -Output [1]: [item_sk#17] - -(49) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Right output [1]: [item_sk#17] -Arguments: [ws_item_sk#33], [item_sk#17], LeftSemi, BuildRight - -(50) CometProject -Input [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] - -(51) CometExchange -Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: hashpartitioning(ws_bill_customer_sk#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(52) CometSort -Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_bill_customer_sk#34 ASC NULLS FIRST] - -(53) ReusedExchange [Reuses operator id: 34] -Output [3]: [c_customer_sk#23, sum#24, isEmpty#25] - -(54) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(55) CometFilter -Input [2]: [c_customer_sk#23, ssales#26] -Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) - -(56) CometProject -Input [2]: [c_customer_sk#23, ssales#26] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(57) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Right output [1]: [c_customer_sk#23] -Arguments: [ws_bill_customer_sk#34], [c_customer_sk#23], LeftSemi - -(59) CometProject -Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] - -(60) ReusedExchange [Reuses operator id: 44] -Output [1]: [d_date_sk#39] - -(61) CometBroadcastHashJoin -Left output [3]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Right output [1]: [d_date_sk#39] -Arguments: [ws_sold_date_sk#37], [d_date_sk#39], Inner, BuildRight - -(62) CometProject -Input [4]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37, d_date_sk#39] -Arguments: [sales#40], [(cast(ws_quantity#35 as decimal(10,0)) * ws_list_price#36) AS sales#40] - -(63) CometUnion -Child 0 Input [1]: [sales#32] -Child 1 Input [1]: [sales#40] - -(64) CometHashAggregate -Input [1]: [sales#32] -Keys: [] -Functions [1]: [partial_sum(sales#32)] - -(65) CometExchange -Input [2]: [sum#41, isEmpty#42] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(66) CometHashAggregate -Input [2]: [sum#41, isEmpty#42] -Keys: [] -Functions [1]: [sum(sales#32)] - -(67) CometColumnarToRow [codegen id : 1] -Input [1]: [sum(sales)#43] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (72) -+- * CometColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) - - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#29, d_year#30, d_moy#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Condition : ((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2000)) AND (d_moy#31 = 2)) AND isnotnull(d_date_sk#29)) - -(70) CometProject -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Arguments: [d_date_sk#29], [d_date_sk#29] - -(71) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#29] - -(72) BroadcastExchange -Input [1]: [d_date_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 2 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (77) -+- * CometColumnarToRow (76) - +- CometProject (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(75) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(76) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_date#11] - -(77) BroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:3 Hosting operator id = 36 Hosting Expression = Subquery scalar-subquery#27, [id=#28] -* CometColumnarToRow (95) -+- CometHashAggregate (94) - +- CometExchange (93) - +- CometHashAggregate (92) - +- CometHashAggregate (91) - +- CometExchange (90) - +- CometHashAggregate (89) - +- CometProject (88) - +- CometBroadcastHashJoin (87) - :- CometProject (82) - : +- CometBroadcastHashJoin (81) - : :- CometFilter (79) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (78) - : +- ReusedExchange (80) - +- CometBroadcastExchange (86) - +- CometProject (85) - +- CometFilter (84) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (83) - - -(78) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#44, ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#47), dynamicpruningexpression(ss_sold_date_sk#47 IN dynamicpruning#48)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(79) CometFilter -Input [4]: [ss_customer_sk#44, ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47] -Condition : isnotnull(ss_customer_sk#44) - -(80) ReusedExchange [Reuses operator id: 30] -Output [1]: [c_customer_sk#49] - -(81) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#44, ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47] -Right output [1]: [c_customer_sk#49] -Arguments: [ss_customer_sk#44], [c_customer_sk#49], Inner, BuildRight - -(82) CometProject -Input [5]: [ss_customer_sk#44, ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47, c_customer_sk#49] -Arguments: [ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47, c_customer_sk#49], [ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47, c_customer_sk#49] - -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#50, d_year#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(84) CometFilter -Input [2]: [d_date_sk#50, d_year#51] -Condition : (d_year#51 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#50)) - -(85) CometProject -Input [2]: [d_date_sk#50, d_year#51] -Arguments: [d_date_sk#50], [d_date_sk#50] - -(86) CometBroadcastExchange -Input [1]: [d_date_sk#50] -Arguments: [d_date_sk#50] - -(87) CometBroadcastHashJoin -Left output [4]: [ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47, c_customer_sk#49] -Right output [1]: [d_date_sk#50] -Arguments: [ss_sold_date_sk#47], [d_date_sk#50], Inner, BuildRight - -(88) CometProject -Input [5]: [ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47, c_customer_sk#49, d_date_sk#50] -Arguments: [ss_quantity#45, ss_sales_price#46, c_customer_sk#49], [ss_quantity#45, ss_sales_price#46, c_customer_sk#49] - -(89) CometHashAggregate -Input [3]: [ss_quantity#45, ss_sales_price#46, c_customer_sk#49] -Keys [1]: [c_customer_sk#49] -Functions [1]: [partial_sum((cast(ss_quantity#45 as decimal(10,0)) * ss_sales_price#46))] - -(90) CometExchange -Input [3]: [c_customer_sk#49, sum#52, isEmpty#53] -Arguments: hashpartitioning(c_customer_sk#49, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(91) CometHashAggregate -Input [3]: [c_customer_sk#49, sum#52, isEmpty#53] -Keys [1]: [c_customer_sk#49] -Functions [1]: [sum((cast(ss_quantity#45 as decimal(10,0)) * ss_sales_price#46))] - -(92) CometHashAggregate -Input [1]: [csales#54] -Keys: [] -Functions [1]: [partial_max(csales#54)] - -(93) CometExchange -Input [1]: [max#55] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(94) CometHashAggregate -Input [1]: [max#55] -Keys: [] -Functions [1]: [max(csales#54)] - -(95) CometColumnarToRow [codegen id : 1] -Input [1]: [tpcds_cmax#56] - -Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#47 IN dynamicpruning#48 -BroadcastExchange (100) -+- * CometColumnarToRow (99) - +- CometProject (98) - +- CometFilter (97) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (96) - - -(96) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#50, d_year#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(97) CometFilter -Input [2]: [d_date_sk#50, d_year#51] -Condition : (d_year#51 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#50)) - -(98) CometProject -Input [2]: [d_date_sk#50, d_year#51] -Arguments: [d_date_sk#50], [d_date_sk#50] - -(99) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#50] - -(100) BroadcastExchange -Input [1]: [d_date_sk#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] - -Subquery:5 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#6 - -Subquery:6 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 8f1bddf6c0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,116 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,isEmpty] [sum(sales),sum(sales)] - CometExchange #1 - CometHashAggregate [sales] [sum,isEmpty] - CometUnion [sales] - CometProject [cs_quantity,cs_list_price] [sales] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometExchange [cs_bill_customer_sk] #2 - CometProject [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [item_sk] #4 - CometProject [item_sk] - CometFilter [item_sk,cnt] - CometHashAggregate [_groupingexpression,d_date,count] [item_sk,cnt,i_item_sk,count(1)] - CometExchange [_groupingexpression,i_item_sk,d_date] #5 - CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] - CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] - CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [i_item_sk,i_item_desc] #8 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [max] [tpcds_cmax,max(csales)] - CometExchange #11 - CometHashAggregate [csales] [max] - CometHashAggregate [c_customer_sk,sum,isEmpty] [csales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #12 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,c_customer_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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #9 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,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,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #10 - CometFilter [c_customer_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [d_date_sk] #15 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [ws_quantity,ws_list_price] [sales] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometExchange [ws_bill_customer_sk] #16 - CometProject [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,item_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #9 - ReusedExchange [d_date_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt deleted file mode 100644 index bce5d2805d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt +++ /dev/null @@ -1,576 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (67) -+- CometHashAggregate (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometUnion (63) - :- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (40) - : : +- CometSortMergeJoin (39) - : : :- CometSort (24) - : : : +- CometExchange (23) - : : : +- CometProject (22) - : : : +- CometBroadcastHashJoin (21) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometBroadcastExchange (20) - : : : +- CometProject (19) - : : : +- CometFilter (18) - : : : +- CometHashAggregate (17) - : : : +- CometExchange (16) - : : : +- CometHashAggregate (15) - : : : +- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometFilter (3) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (2) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometFilter (11) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (10) - : : +- CometSort (38) - : : +- CometProject (37) - : : +- CometFilter (36) - : : +- CometHashAggregate (35) - : : +- CometExchange (34) - : : +- CometHashAggregate (33) - : : +- CometProject (32) - : : +- CometBroadcastHashJoin (31) - : : :- CometProject (27) - : : : +- CometFilter (26) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (25) - : : +- CometBroadcastExchange (30) - : : +- CometFilter (29) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (28) - : +- CometBroadcastExchange (44) - : +- CometProject (43) - : +- CometFilter (42) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - +- CometProject (62) - +- CometBroadcastHashJoin (61) - :- CometProject (59) - : +- CometSortMergeJoin (58) - : :- CometSort (52) - : : +- CometExchange (51) - : : +- CometProject (50) - : : +- CometBroadcastHashJoin (49) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : : +- ReusedExchange (48) - : +- CometSort (57) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometHashAggregate (54) - : +- ReusedExchange (53) - +- ReusedExchange (60) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(2) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(3) CometFilter -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_item_sk#7) - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(6) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(7) CometBroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: [d_date_sk#10, d_date#11] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Right output [2]: [d_date_sk#10, d_date#11] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] -Arguments: [ss_item_sk#7, d_date#11], [ss_item_sk#7, d_date#11] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#13, i_item_desc#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [i_item_sk#13, i_item_desc#14] -Condition : isnotnull(i_item_sk#13) - -(12) CometBroadcastExchange -Input [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [i_item_sk#13, i_item_desc#14] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, d_date#11] -Right output [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [ss_item_sk#7], [i_item_sk#13], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#13, i_item_desc#14] -Arguments: [d_date#11, i_item_sk#13, _groupingexpression#15], [d_date#11, i_item_sk#13, substr(i_item_desc#14, 1, 30) AS _groupingexpression#15] - -(15) CometHashAggregate -Input [3]: [d_date#11, i_item_sk#13, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [partial_count(1)] - -(16) CometExchange -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [count(1)] - -(18) CometFilter -Input [2]: [item_sk#17, cnt#18] -Condition : (cnt#18 > 4) - -(19) CometProject -Input [2]: [item_sk#17, cnt#18] -Arguments: [item_sk#17], [item_sk#17] - -(20) CometBroadcastExchange -Input [1]: [item_sk#17] -Arguments: [item_sk#17] - -(21) CometBroadcastHashJoin -Left output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [item_sk#17] -Arguments: [cs_item_sk#2], [item_sk#17], LeftSemi, BuildRight - -(22) CometProject -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(23) CometExchange -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#19) - -(27) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Arguments: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21], [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(29) CometFilter -Input [1]: [c_customer_sk#23] -Condition : isnotnull(c_customer_sk#23) - -(30) CometBroadcastExchange -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23] - -(31) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] -Right output [1]: [c_customer_sk#23] -Arguments: [ss_customer_sk#19], [c_customer_sk#23], Inner, BuildRight - -(32) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Arguments: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23], [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] - -(33) CometHashAggregate -Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Keys [1]: [c_customer_sk#23] -Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(34) CometExchange -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(35) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(36) CometFilter -Input [2]: [c_customer_sk#23, ssales#26] -Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#27, [id=#28]))) - -(37) CometProject -Input [2]: [c_customer_sk#23, ssales#26] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(38) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(39) CometSortMergeJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [c_customer_sk#23] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#23], LeftSemi - -(40) CometProject -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#29, d_year#30, d_moy#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Condition : ((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2000)) AND (d_moy#31 = 2)) AND isnotnull(d_date_sk#29)) - -(43) CometProject -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Arguments: [d_date_sk#29], [d_date_sk#29] - -(44) CometBroadcastExchange -Input [1]: [d_date_sk#29] -Arguments: [d_date_sk#29] - -(45) CometBroadcastHashJoin -Left output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [d_date_sk#29] -Arguments: [cs_sold_date_sk#5], [d_date_sk#29], Inner, BuildRight - -(46) CometProject -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#29] -Arguments: [sales#32], [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#32] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_sold_date_sk#37 IN dynamicpruning#38)] -ReadSchema: struct - -(48) ReusedExchange [Reuses operator id: 20] -Output [1]: [item_sk#17] - -(49) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Right output [1]: [item_sk#17] -Arguments: [ws_item_sk#33], [item_sk#17], LeftSemi, BuildRight - -(50) CometProject -Input [5]: [ws_item_sk#33, ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] - -(51) CometExchange -Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: hashpartitioning(ws_bill_customer_sk#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(52) CometSort -Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_bill_customer_sk#34 ASC NULLS FIRST] - -(53) ReusedExchange [Reuses operator id: 34] -Output [3]: [c_customer_sk#23, sum#24, isEmpty#25] - -(54) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(55) CometFilter -Input [2]: [c_customer_sk#23, ssales#26] -Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) - -(56) CometProject -Input [2]: [c_customer_sk#23, ssales#26] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(57) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Right output [1]: [c_customer_sk#23] -Arguments: [ws_bill_customer_sk#34], [c_customer_sk#23], LeftSemi - -(59) CometProject -Input [4]: [ws_bill_customer_sk#34, ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Arguments: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37], [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] - -(60) ReusedExchange [Reuses operator id: 44] -Output [1]: [d_date_sk#39] - -(61) CometBroadcastHashJoin -Left output [3]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37] -Right output [1]: [d_date_sk#39] -Arguments: [ws_sold_date_sk#37], [d_date_sk#39], Inner, BuildRight - -(62) CometProject -Input [4]: [ws_quantity#35, ws_list_price#36, ws_sold_date_sk#37, d_date_sk#39] -Arguments: [sales#40], [(cast(ws_quantity#35 as decimal(10,0)) * ws_list_price#36) AS sales#40] - -(63) CometUnion -Child 0 Input [1]: [sales#32] -Child 1 Input [1]: [sales#40] - -(64) CometHashAggregate -Input [1]: [sales#32] -Keys: [] -Functions [1]: [partial_sum(sales#32)] - -(65) CometExchange -Input [2]: [sum#41, isEmpty#42] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(66) CometHashAggregate -Input [2]: [sum#41, isEmpty#42] -Keys: [] -Functions [1]: [sum(sales#32)] - -(67) CometColumnarToRow [codegen id : 1] -Input [1]: [sum(sales)#43] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (72) -+- * CometColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) - - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#29, d_year#30, d_moy#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Condition : ((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2000)) AND (d_moy#31 = 2)) AND isnotnull(d_date_sk#29)) - -(70) CometProject -Input [3]: [d_date_sk#29, d_year#30, d_moy#31] -Arguments: [d_date_sk#29], [d_date_sk#29] - -(71) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#29] - -(72) BroadcastExchange -Input [1]: [d_date_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 2 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (77) -+- * CometColumnarToRow (76) - +- CometProject (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(75) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(76) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_date#11] - -(77) BroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:3 Hosting operator id = 36 Hosting Expression = Subquery scalar-subquery#27, [id=#28] -* CometColumnarToRow (95) -+- CometHashAggregate (94) - +- CometExchange (93) - +- CometHashAggregate (92) - +- CometHashAggregate (91) - +- CometExchange (90) - +- CometHashAggregate (89) - +- CometProject (88) - +- CometBroadcastHashJoin (87) - :- CometProject (82) - : +- CometBroadcastHashJoin (81) - : :- CometFilter (79) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (78) - : +- ReusedExchange (80) - +- CometBroadcastExchange (86) - +- CometProject (85) - +- CometFilter (84) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (83) - - -(78) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#44, ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#47), dynamicpruningexpression(ss_sold_date_sk#47 IN dynamicpruning#48)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(79) CometFilter -Input [4]: [ss_customer_sk#44, ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47] -Condition : isnotnull(ss_customer_sk#44) - -(80) ReusedExchange [Reuses operator id: 30] -Output [1]: [c_customer_sk#49] - -(81) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#44, ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47] -Right output [1]: [c_customer_sk#49] -Arguments: [ss_customer_sk#44], [c_customer_sk#49], Inner, BuildRight - -(82) CometProject -Input [5]: [ss_customer_sk#44, ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47, c_customer_sk#49] -Arguments: [ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47, c_customer_sk#49], [ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47, c_customer_sk#49] - -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#50, d_year#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(84) CometFilter -Input [2]: [d_date_sk#50, d_year#51] -Condition : (d_year#51 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#50)) - -(85) CometProject -Input [2]: [d_date_sk#50, d_year#51] -Arguments: [d_date_sk#50], [d_date_sk#50] - -(86) CometBroadcastExchange -Input [1]: [d_date_sk#50] -Arguments: [d_date_sk#50] - -(87) CometBroadcastHashJoin -Left output [4]: [ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47, c_customer_sk#49] -Right output [1]: [d_date_sk#50] -Arguments: [ss_sold_date_sk#47], [d_date_sk#50], Inner, BuildRight - -(88) CometProject -Input [5]: [ss_quantity#45, ss_sales_price#46, ss_sold_date_sk#47, c_customer_sk#49, d_date_sk#50] -Arguments: [ss_quantity#45, ss_sales_price#46, c_customer_sk#49], [ss_quantity#45, ss_sales_price#46, c_customer_sk#49] - -(89) CometHashAggregate -Input [3]: [ss_quantity#45, ss_sales_price#46, c_customer_sk#49] -Keys [1]: [c_customer_sk#49] -Functions [1]: [partial_sum((cast(ss_quantity#45 as decimal(10,0)) * ss_sales_price#46))] - -(90) CometExchange -Input [3]: [c_customer_sk#49, sum#52, isEmpty#53] -Arguments: hashpartitioning(c_customer_sk#49, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(91) CometHashAggregate -Input [3]: [c_customer_sk#49, sum#52, isEmpty#53] -Keys [1]: [c_customer_sk#49] -Functions [1]: [sum((cast(ss_quantity#45 as decimal(10,0)) * ss_sales_price#46))] - -(92) CometHashAggregate -Input [1]: [csales#54] -Keys: [] -Functions [1]: [partial_max(csales#54)] - -(93) CometExchange -Input [1]: [max#55] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(94) CometHashAggregate -Input [1]: [max#55] -Keys: [] -Functions [1]: [max(csales#54)] - -(95) CometColumnarToRow [codegen id : 1] -Input [1]: [tpcds_cmax#56] - -Subquery:4 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#47 IN dynamicpruning#48 -BroadcastExchange (100) -+- * CometColumnarToRow (99) - +- CometProject (98) - +- CometFilter (97) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (96) - - -(96) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#50, d_year#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(97) CometFilter -Input [2]: [d_date_sk#50, d_year#51] -Condition : (d_year#51 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#50)) - -(98) CometProject -Input [2]: [d_date_sk#50, d_year#51] -Arguments: [d_date_sk#50], [d_date_sk#50] - -(99) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#50] - -(100) BroadcastExchange -Input [1]: [d_date_sk#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] - -Subquery:5 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#6 - -Subquery:6 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/extended.txt deleted file mode 100644 index a84226b077..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/extended.txt +++ /dev/null @@ -1,146 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometSort - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometSort - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 127 out of 138 eligible operators (92%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 8f1bddf6c0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt +++ /dev/null @@ -1,116 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,isEmpty] [sum(sales),sum(sales)] - CometExchange #1 - CometHashAggregate [sales] [sum,isEmpty] - CometUnion [sales] - CometProject [cs_quantity,cs_list_price] [sales] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometExchange [cs_bill_customer_sk] #2 - CometProject [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [item_sk] #4 - CometProject [item_sk] - CometFilter [item_sk,cnt] - CometHashAggregate [_groupingexpression,d_date,count] [item_sk,cnt,i_item_sk,count(1)] - CometExchange [_groupingexpression,i_item_sk,d_date] #5 - CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] - CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] - CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [i_item_sk,i_item_desc] #8 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [max] [tpcds_cmax,max(csales)] - CometExchange #11 - CometHashAggregate [csales] [max] - CometHashAggregate [c_customer_sk,sum,isEmpty] [csales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #12 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,c_customer_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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #9 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,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,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #10 - CometFilter [c_customer_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [d_date_sk] #15 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [ws_quantity,ws_list_price] [sales] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometExchange [ws_bill_customer_sk] #16 - CometProject [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,item_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #9 - ReusedExchange [d_date_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/explain.txt deleted file mode 100644 index c678959b08..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/explain.txt +++ /dev/null @@ -1,718 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (90) -+- CometTakeOrderedAndProject (89) - +- CometUnion (88) - :- CometHashAggregate (64) - : +- CometExchange (63) - : +- CometHashAggregate (62) - : +- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (55) - : : +- CometBroadcastHashJoin (54) - : : :- CometSortMergeJoin (41) - : : : :- CometSort (26) - : : : : +- CometColumnarExchange (25) - : : : : +- * Project (24) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (22) - : : : : +- * Project (21) - : : : : +- * Filter (20) - : : : : +- * HashAggregate (19) - : : : : +- * CometColumnarToRow (18) - : : : : +- CometColumnarExchange (17) - : : : : +- * HashAggregate (16) - : : : : +- * Project (15) - : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : :- * Project (9) - : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : :- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : : : +- ReusedExchange (7) - : : : : +- BroadcastExchange (13) - : : : : +- * CometColumnarToRow (12) - : : : : +- CometFilter (11) - : : : : +- CometNativeScan parquet spark_catalog.default.item (10) - : : : +- CometSort (40) - : : : +- CometProject (39) - : : : +- CometFilter (38) - : : : +- CometHashAggregate (37) - : : : +- CometExchange (36) - : : : +- CometHashAggregate (35) - : : : +- CometProject (34) - : : : +- CometBroadcastHashJoin (33) - : : : :- CometProject (29) - : : : : +- CometFilter (28) - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (27) - : : : +- CometBroadcastExchange (32) - : : : +- CometFilter (31) - : : : +- CometNativeScan parquet spark_catalog.default.customer (30) - : : +- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometSortMergeJoin (51) - : : :- CometSort (45) - : : : +- CometExchange (44) - : : : +- CometFilter (43) - : : : +- CometNativeScan parquet spark_catalog.default.customer (42) - : : +- CometSort (50) - : : +- CometProject (49) - : : +- CometFilter (48) - : : +- CometHashAggregate (47) - : : +- ReusedExchange (46) - : +- CometBroadcastExchange (59) - : +- CometProject (58) - : +- CometFilter (57) - : +- CometNativeScan parquet spark_catalog.default.date_dim (56) - +- CometHashAggregate (87) - +- CometExchange (86) - +- CometHashAggregate (85) - +- CometProject (84) - +- CometBroadcastHashJoin (83) - :- CometProject (81) - : +- CometBroadcastHashJoin (80) - : :- CometSortMergeJoin (78) - : : :- CometSort (72) - : : : +- CometColumnarExchange (71) - : : : +- * Project (70) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (69) - : : : :- * Filter (67) - : : : : +- * ColumnarToRow (66) - : : : : +- Scan parquet spark_catalog.default.web_sales (65) - : : : +- ReusedExchange (68) - : : +- CometSort (77) - : : +- CometProject (76) - : : +- CometFilter (75) - : : +- CometHashAggregate (74) - : : +- ReusedExchange (73) - : +- ReusedExchange (79) - +- ReusedExchange (82) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(3) Filter [codegen id : 5] -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_bill_customer_sk#1) - -(4) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 3] -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] - -(6) Filter [codegen id : 3] -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_item_sk#7) - -(7) ReusedExchange [Reuses operator id: 100] -Output [2]: [d_date_sk#10, d_date#11] - -(8) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 3] -Output [2]: [ss_item_sk#7, d_date#11] -Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] - -(10) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#12, i_item_desc#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [i_item_sk#12, i_item_desc#13] -Condition : isnotnull(i_item_sk#12) - -(12) CometColumnarToRow [codegen id : 2] -Input [2]: [i_item_sk#12, i_item_desc#13] - -(13) BroadcastExchange -Input [2]: [i_item_sk#12, i_item_desc#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(14) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#7] -Right keys [1]: [i_item_sk#12] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 3] -Output [3]: [d_date#11, i_item_sk#12, substr(i_item_desc#13, 1, 30) AS _groupingexpression#14] -Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#12, i_item_desc#13] - -(16) HashAggregate [codegen id : 3] -Input [3]: [d_date#11, i_item_sk#12, _groupingexpression#14] -Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#15] -Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] - -(17) CometColumnarExchange -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(18) CometColumnarToRow [codegen id : 4] -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] - -(19) HashAggregate [codegen id : 4] -Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] -Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#17] -Results [2]: [i_item_sk#12 AS item_sk#18, count(1)#17 AS cnt#19] - -(20) Filter [codegen id : 4] -Input [2]: [item_sk#18, cnt#19] -Condition : (cnt#19 > 4) - -(21) Project [codegen id : 4] -Output [1]: [item_sk#18] -Input [2]: [item_sk#18, cnt#19] - -(22) BroadcastExchange -Input [1]: [item_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(23) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [item_sk#18] -Join type: LeftSemi -Join condition: None - -(24) Project [codegen id : 5] -Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(25) CometColumnarExchange -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(26) CometSort -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] - -(27) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(28) CometFilter -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_customer_sk#20) - -(29) CometProject -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] -Arguments: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22], [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] - -(30) CometNativeScan parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(31) CometFilter -Input [1]: [c_customer_sk#24] -Condition : isnotnull(c_customer_sk#24) - -(32) CometBroadcastExchange -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24] - -(33) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] -Right output [1]: [c_customer_sk#24] -Arguments: [ss_customer_sk#20], [c_customer_sk#24], Inner, BuildRight - -(34) CometProject -Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -Arguments: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24], [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] - -(35) CometHashAggregate -Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] -Keys [1]: [c_customer_sk#24] -Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] - -(36) CometExchange -Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] -Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(37) CometHashAggregate -Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] -Keys [1]: [c_customer_sk#24] -Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] - -(38) CometFilter -Input [2]: [c_customer_sk#24, ssales#27] -Condition : (isnotnull(ssales#27) AND (cast(ssales#27 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#28, [id=#29]))) - -(39) CometProject -Input [2]: [c_customer_sk#24, ssales#27] -Arguments: [c_customer_sk#24], [c_customer_sk#24] - -(40) CometSort -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24], [c_customer_sk#24 ASC NULLS FIRST] - -(41) CometSortMergeJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [c_customer_sk#24] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#24], LeftSemi - -(42) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(43) CometFilter -Input [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Condition : isnotnull(c_customer_sk#30) - -(44) CometExchange -Input [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(45) CometSort -Input [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Arguments: [c_customer_sk#30, c_first_name#31, c_last_name#32], [c_customer_sk#30 ASC NULLS FIRST] - -(46) ReusedExchange [Reuses operator id: 36] -Output [3]: [c_customer_sk#24, sum#25, isEmpty#26] - -(47) CometHashAggregate -Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] -Keys [1]: [c_customer_sk#24] -Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] - -(48) CometFilter -Input [2]: [c_customer_sk#24, ssales#27] -Condition : (isnotnull(ssales#27) AND (cast(ssales#27 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#28, [id=#29]))) - -(49) CometProject -Input [2]: [c_customer_sk#24, ssales#27] -Arguments: [c_customer_sk#24], [c_customer_sk#24] - -(50) CometSort -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24], [c_customer_sk#24 ASC NULLS FIRST] - -(51) CometSortMergeJoin -Left output [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Right output [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#30], [c_customer_sk#24], LeftSemi - -(52) CometProject -Input [3]: [c_customer_sk#30, c_first_name#31, c_last_name#32] -Arguments: [c_customer_sk#30, c_first_name#33, c_last_name#34], [c_customer_sk#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#31, 20, true, false, true) AS c_first_name#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#32, 30, true, false, true) AS c_last_name#34] - -(53) CometBroadcastExchange -Input [3]: [c_customer_sk#30, c_first_name#33, c_last_name#34] -Arguments: [c_customer_sk#30, c_first_name#33, c_last_name#34] - -(54) CometBroadcastHashJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [3]: [c_customer_sk#30, c_first_name#33, c_last_name#34] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#30], Inner, BuildRight - -(55) CometProject -Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#30, c_first_name#33, c_last_name#34] -Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#33, c_last_name#34], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#33, c_last_name#34] - -(56) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#35, d_year#36, d_moy#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(57) CometFilter -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] -Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2000)) AND (d_moy#37 = 2)) AND isnotnull(d_date_sk#35)) - -(58) CometProject -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] -Arguments: [d_date_sk#35], [d_date_sk#35] - -(59) CometBroadcastExchange -Input [1]: [d_date_sk#35] -Arguments: [d_date_sk#35] - -(60) CometBroadcastHashJoin -Left output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#33, c_last_name#34] -Right output [1]: [d_date_sk#35] -Arguments: [cs_sold_date_sk#5], [d_date_sk#35], Inner, BuildRight - -(61) CometProject -Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#33, c_last_name#34, d_date_sk#35] -Arguments: [cs_quantity#3, cs_list_price#4, c_first_name#33, c_last_name#34], [cs_quantity#3, cs_list_price#4, c_first_name#33, c_last_name#34] - -(62) CometHashAggregate -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#33, c_last_name#34] -Keys [2]: [c_last_name#34, c_first_name#33] -Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] - -(63) CometExchange -Input [4]: [c_last_name#34, c_first_name#33, sum#38, isEmpty#39] -Arguments: hashpartitioning(c_last_name#34, c_first_name#33, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(64) CometHashAggregate -Input [4]: [c_last_name#34, c_first_name#33, sum#38, isEmpty#39] -Keys [2]: [c_last_name#34, c_first_name#33] -Functions [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] - -(65) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#40, ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(66) ColumnarToRow [codegen id : 10] -Input [5]: [ws_item_sk#40, ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] - -(67) Filter [codegen id : 10] -Input [5]: [ws_item_sk#40, ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_bill_customer_sk#41) - -(68) ReusedExchange [Reuses operator id: 22] -Output [1]: [item_sk#18] - -(69) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [item_sk#18] -Join type: LeftSemi -Join condition: None - -(70) Project [codegen id : 10] -Output [4]: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -Input [5]: [ws_item_sk#40, ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] - -(71) CometColumnarExchange -Input [4]: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -Arguments: hashpartitioning(ws_bill_customer_sk#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(72) CometSort -Input [4]: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -Arguments: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44], [ws_bill_customer_sk#41 ASC NULLS FIRST] - -(73) ReusedExchange [Reuses operator id: 36] -Output [3]: [c_customer_sk#24, sum#25, isEmpty#26] - -(74) CometHashAggregate -Input [3]: [c_customer_sk#24, sum#25, isEmpty#26] -Keys [1]: [c_customer_sk#24] -Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] - -(75) CometFilter -Input [2]: [c_customer_sk#24, ssales#27] -Condition : (isnotnull(ssales#27) AND (cast(ssales#27 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#28, [id=#29]))) - -(76) CometProject -Input [2]: [c_customer_sk#24, ssales#27] -Arguments: [c_customer_sk#24], [c_customer_sk#24] - -(77) CometSort -Input [1]: [c_customer_sk#24] -Arguments: [c_customer_sk#24], [c_customer_sk#24 ASC NULLS FIRST] - -(78) CometSortMergeJoin -Left output [4]: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -Right output [1]: [c_customer_sk#24] -Arguments: [ws_bill_customer_sk#41], [c_customer_sk#24], LeftSemi - -(79) ReusedExchange [Reuses operator id: 53] -Output [3]: [c_customer_sk#45, c_first_name#46, c_last_name#47] - -(80) CometBroadcastHashJoin -Left output [4]: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44] -Right output [3]: [c_customer_sk#45, c_first_name#46, c_last_name#47] -Arguments: [ws_bill_customer_sk#41], [c_customer_sk#45], Inner, BuildRight - -(81) CometProject -Input [7]: [ws_bill_customer_sk#41, ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44, c_customer_sk#45, c_first_name#46, c_last_name#47] -Arguments: [ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44, c_first_name#46, c_last_name#47], [ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44, c_first_name#46, c_last_name#47] - -(82) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#48] - -(83) CometBroadcastHashJoin -Left output [5]: [ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44, c_first_name#46, c_last_name#47] -Right output [1]: [d_date_sk#48] -Arguments: [ws_sold_date_sk#44], [d_date_sk#48], Inner, BuildRight - -(84) CometProject -Input [6]: [ws_quantity#42, ws_list_price#43, ws_sold_date_sk#44, c_first_name#46, c_last_name#47, d_date_sk#48] -Arguments: [ws_quantity#42, ws_list_price#43, c_first_name#46, c_last_name#47], [ws_quantity#42, ws_list_price#43, c_first_name#46, c_last_name#47] - -(85) CometHashAggregate -Input [4]: [ws_quantity#42, ws_list_price#43, c_first_name#46, c_last_name#47] -Keys [2]: [c_last_name#47, c_first_name#46] -Functions [1]: [partial_sum((cast(ws_quantity#42 as decimal(10,0)) * ws_list_price#43))] - -(86) CometExchange -Input [4]: [c_last_name#47, c_first_name#46, sum#49, isEmpty#50] -Arguments: hashpartitioning(c_last_name#47, c_first_name#46, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(87) CometHashAggregate -Input [4]: [c_last_name#47, c_first_name#46, sum#49, isEmpty#50] -Keys [2]: [c_last_name#47, c_first_name#46] -Functions [1]: [sum((cast(ws_quantity#42 as decimal(10,0)) * ws_list_price#43))] - -(88) CometUnion -Child 0 Input [3]: [c_last_name#34, c_first_name#33, sales#51] -Child 1 Input [3]: [c_last_name#47, c_first_name#46, sales#52] - -(89) CometTakeOrderedAndProject -Input [3]: [c_last_name#34, c_first_name#33, sales#51] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#34 ASC NULLS FIRST,c_first_name#33 ASC NULLS FIRST,sales#51 ASC NULLS FIRST], output=[c_last_name#34,c_first_name#33,sales#51]), [c_last_name#34, c_first_name#33, sales#51], 100, 0, [c_last_name#34 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, sales#51 ASC NULLS FIRST], [c_last_name#34, c_first_name#33, sales#51] - -(90) CometColumnarToRow [codegen id : 11] -Input [3]: [c_last_name#34, c_first_name#33, sales#51] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (95) -+- * CometColumnarToRow (94) - +- CometProject (93) - +- CometFilter (92) - +- CometNativeScan parquet spark_catalog.default.date_dim (91) - - -(91) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#35, d_year#36, d_moy#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(92) CometFilter -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] -Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2000)) AND (d_moy#37 = 2)) AND isnotnull(d_date_sk#35)) - -(93) CometProject -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] -Arguments: [d_date_sk#35], [d_date_sk#35] - -(94) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#35] - -(95) BroadcastExchange -Input [1]: [d_date_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (100) -+- * CometColumnarToRow (99) - +- CometProject (98) - +- CometFilter (97) - +- CometNativeScan parquet spark_catalog.default.date_dim (96) - - -(96) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(97) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#53] -Condition : (d_year#53 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(98) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#53] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(99) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_date#11] - -(100) BroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -Subquery:3 Hosting operator id = 38 Hosting Expression = Subquery scalar-subquery#28, [id=#29] -* HashAggregate (120) -+- * CometColumnarToRow (119) - +- CometColumnarExchange (118) - +- * HashAggregate (117) - +- * HashAggregate (116) - +- * CometColumnarToRow (115) - +- CometColumnarExchange (114) - +- * HashAggregate (113) - +- * Project (112) - +- * BroadcastHashJoin Inner BuildRight (111) - :- * Project (109) - : +- * BroadcastHashJoin Inner BuildRight (108) - : :- * Filter (103) - : : +- * ColumnarToRow (102) - : : +- Scan parquet spark_catalog.default.store_sales (101) - : +- BroadcastExchange (107) - : +- * CometColumnarToRow (106) - : +- CometFilter (105) - : +- CometNativeScan parquet spark_catalog.default.customer (104) - +- ReusedExchange (110) - - -(101) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#54, ss_quantity#55, ss_sales_price#56, ss_sold_date_sk#57] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_sold_date_sk#57 IN dynamicpruning#58)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(102) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#54, ss_quantity#55, ss_sales_price#56, ss_sold_date_sk#57] - -(103) Filter [codegen id : 3] -Input [4]: [ss_customer_sk#54, ss_quantity#55, ss_sales_price#56, ss_sold_date_sk#57] -Condition : isnotnull(ss_customer_sk#54) - -(104) CometNativeScan parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(105) CometFilter -Input [1]: [c_customer_sk#59] -Condition : isnotnull(c_customer_sk#59) - -(106) CometColumnarToRow [codegen id : 1] -Input [1]: [c_customer_sk#59] - -(107) BroadcastExchange -Input [1]: [c_customer_sk#59] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -(108) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#54] -Right keys [1]: [c_customer_sk#59] -Join type: Inner -Join condition: None - -(109) Project [codegen id : 3] -Output [4]: [ss_quantity#55, ss_sales_price#56, ss_sold_date_sk#57, c_customer_sk#59] -Input [5]: [ss_customer_sk#54, ss_quantity#55, ss_sales_price#56, ss_sold_date_sk#57, c_customer_sk#59] - -(110) ReusedExchange [Reuses operator id: 125] -Output [1]: [d_date_sk#60] - -(111) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#57] -Right keys [1]: [d_date_sk#60] -Join type: Inner -Join condition: None - -(112) Project [codegen id : 3] -Output [3]: [ss_quantity#55, ss_sales_price#56, c_customer_sk#59] -Input [5]: [ss_quantity#55, ss_sales_price#56, ss_sold_date_sk#57, c_customer_sk#59, d_date_sk#60] - -(113) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#55, ss_sales_price#56, c_customer_sk#59] -Keys [1]: [c_customer_sk#59] -Functions [1]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_sales_price#56))] -Aggregate Attributes [2]: [sum#61, isEmpty#62] -Results [3]: [c_customer_sk#59, sum#63, isEmpty#64] - -(114) CometColumnarExchange -Input [3]: [c_customer_sk#59, sum#63, isEmpty#64] -Arguments: hashpartitioning(c_customer_sk#59, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(115) CometColumnarToRow [codegen id : 4] -Input [3]: [c_customer_sk#59, sum#63, isEmpty#64] - -(116) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#59, sum#63, isEmpty#64] -Keys [1]: [c_customer_sk#59] -Functions [1]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_sales_price#56))] -Aggregate Attributes [1]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_sales_price#56))#65] -Results [1]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_sales_price#56))#65 AS csales#66] - -(117) HashAggregate [codegen id : 4] -Input [1]: [csales#66] -Keys: [] -Functions [1]: [partial_max(csales#66)] -Aggregate Attributes [1]: [max#67] -Results [1]: [max#68] - -(118) CometColumnarExchange -Input [1]: [max#68] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(119) CometColumnarToRow [codegen id : 5] -Input [1]: [max#68] - -(120) HashAggregate [codegen id : 5] -Input [1]: [max#68] -Keys: [] -Functions [1]: [max(csales#66)] -Aggregate Attributes [1]: [max(csales#66)#69] -Results [1]: [max(csales#66)#69 AS tpcds_cmax#70] - -Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 -BroadcastExchange (125) -+- * CometColumnarToRow (124) - +- CometProject (123) - +- CometFilter (122) - +- CometNativeScan parquet spark_catalog.default.date_dim (121) - - -(121) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#60, d_year#71] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(122) CometFilter -Input [2]: [d_date_sk#60, d_year#71] -Condition : (d_year#71 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#60)) - -(123) CometProject -Input [2]: [d_date_sk#60, d_year#71] -Arguments: [d_date_sk#60], [d_date_sk#60] - -(124) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#60] - -(125) BroadcastExchange -Input [1]: [d_date_sk#60] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] - -Subquery:5 Hosting operator id = 48 Hosting Expression = ReusedSubquery Subquery scalar-subquery#28, [id=#29] - -Subquery:6 Hosting operator id = 65 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 - -Subquery:7 Hosting operator id = 75 Hosting Expression = ReusedSubquery Subquery scalar-subquery#28, [id=#29] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/simplified.txt deleted file mode 100644 index 41f01311f8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/simplified.txt +++ /dev/null @@ -1,164 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,sales] - CometUnion [c_last_name,c_first_name,sales] - CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price))] - CometExchange [c_last_name,c_first_name] #1 - CometHashAggregate [cs_quantity,cs_list_price] [c_last_name,c_first_name,sum,isEmpty] - CometProject [cs_quantity,cs_list_price,c_first_name,c_last_name] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometColumnarExchange [cs_bill_customer_sk] #2 - WholeStageCodegen (5) - Project [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - BroadcastHashJoin [cs_item_sk,item_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [item_sk] - Filter [cnt] - HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [_groupingexpression,i_item_sk,d_date] #5 - WholeStageCodegen (3) - HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] - Project [d_date,i_item_sk,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (5) - HashAggregate [max] [max(csales),tpcds_cmax,max] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (4) - HashAggregate [csales] [max,max] - HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_sk] #11 - WholeStageCodegen (3) - HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,c_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #12 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] - InputAdapter - ReusedExchange [d_date_sk] #12 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #8 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,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,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #9 - CometFilter [c_customer_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #14 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [c_customer_sk,c_first_name,c_last_name,c_customer_sk] - CometSort [c_customer_sk,c_first_name,c_last_name] - CometExchange [c_customer_sk] #15 - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #8 - CometBroadcastExchange [d_date_sk] #16 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price))] - CometExchange [c_last_name,c_first_name] #17 - CometHashAggregate [ws_quantity,ws_list_price] [c_last_name,c_first_name,sum,isEmpty] - CometProject [ws_quantity,ws_list_price,c_first_name,c_last_name] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometColumnarExchange [ws_bill_customer_sk] #18 - WholeStageCodegen (10) - Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - BroadcastHashJoin [ws_item_sk,item_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #8 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #14 - ReusedExchange [d_date_sk] #16 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/explain.txt deleted file mode 100644 index 67e4e39057..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,692 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (88) -+- CometTakeOrderedAndProject (87) - +- CometUnion (86) - :- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometSortMergeJoin (40) - : : : :- CometSort (25) - : : : : +- CometExchange (24) - : : : : +- CometProject (23) - : : : : +- CometBroadcastHashJoin (22) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometBroadcastExchange (21) - : : : : +- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometHashAggregate (18) - : : : : +- CometExchange (17) - : : : : +- CometHashAggregate (16) - : : : : +- CometProject (15) - : : : : +- CometBroadcastHashJoin (14) - : : : : :- CometProject (10) - : : : : : +- CometBroadcastHashJoin (9) - : : : : : :- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : : +- CometBroadcastExchange (8) - : : : : : +- CometProject (7) - : : : : : +- CometFilter (6) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (5) - : : : : +- CometBroadcastExchange (13) - : : : : +- CometFilter (12) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : +- CometSort (39) - : : : +- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometHashAggregate (36) - : : : +- CometExchange (35) - : : : +- CometHashAggregate (34) - : : : +- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometProject (28) - : : : : +- CometFilter (27) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (26) - : : : +- CometBroadcastExchange (31) - : : : +- CometFilter (30) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (29) - : : +- CometBroadcastExchange (52) - : : +- CometProject (51) - : : +- CometSortMergeJoin (50) - : : :- CometSort (44) - : : : +- CometExchange (43) - : : : +- CometFilter (42) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (41) - : : +- CometSort (49) - : : +- CometProject (48) - : : +- CometFilter (47) - : : +- CometHashAggregate (46) - : : +- ReusedExchange (45) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometHashAggregate (85) - +- CometExchange (84) - +- CometHashAggregate (83) - +- CometProject (82) - +- CometBroadcastHashJoin (81) - :- CometProject (79) - : +- CometBroadcastHashJoin (78) - : :- CometSortMergeJoin (76) - : : :- CometSort (70) - : : : +- CometExchange (69) - : : : +- CometProject (68) - : : : +- CometBroadcastHashJoin (67) - : : : :- CometFilter (65) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (64) - : : : +- ReusedExchange (66) - : : +- CometSort (75) - : : +- CometProject (74) - : : +- CometFilter (73) - : : +- CometHashAggregate (72) - : : +- ReusedExchange (71) - : +- ReusedExchange (77) - +- ReusedExchange (80) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_bill_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_item_sk#7) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(7) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(8) CometBroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: [d_date_sk#10, d_date#11] - -(9) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Right output [2]: [d_date_sk#10, d_date#11] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(10) CometProject -Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] -Arguments: [ss_item_sk#7, d_date#11], [ss_item_sk#7, d_date#11] - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#13, i_item_desc#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [2]: [i_item_sk#13, i_item_desc#14] -Condition : isnotnull(i_item_sk#13) - -(13) CometBroadcastExchange -Input [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [i_item_sk#13, i_item_desc#14] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, d_date#11] -Right output [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [ss_item_sk#7], [i_item_sk#13], Inner, BuildRight - -(15) CometProject -Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#13, i_item_desc#14] -Arguments: [d_date#11, i_item_sk#13, _groupingexpression#15], [d_date#11, i_item_sk#13, substr(i_item_desc#14, 1, 30) AS _groupingexpression#15] - -(16) CometHashAggregate -Input [3]: [d_date#11, i_item_sk#13, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [partial_count(1)] - -(17) CometExchange -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [count(1)] - -(19) CometFilter -Input [2]: [item_sk#17, cnt#18] -Condition : (cnt#18 > 4) - -(20) CometProject -Input [2]: [item_sk#17, cnt#18] -Arguments: [item_sk#17], [item_sk#17] - -(21) CometBroadcastExchange -Input [1]: [item_sk#17] -Arguments: [item_sk#17] - -(22) CometBroadcastHashJoin -Left output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [item_sk#17] -Arguments: [cs_item_sk#2], [item_sk#17], LeftSemi, BuildRight - -(23) CometProject -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(24) CometExchange -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(25) CometSort -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#19) - -(28) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Arguments: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21], [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(30) CometFilter -Input [1]: [c_customer_sk#23] -Condition : isnotnull(c_customer_sk#23) - -(31) CometBroadcastExchange -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23] - -(32) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] -Right output [1]: [c_customer_sk#23] -Arguments: [ss_customer_sk#19], [c_customer_sk#23], Inner, BuildRight - -(33) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Arguments: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23], [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] - -(34) CometHashAggregate -Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Keys [1]: [c_customer_sk#23] -Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(35) CometExchange -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(36) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(37) CometFilter -Input [2]: [c_customer_sk#23, ssales#26] -Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#27, [id=#28]))) - -(38) CometProject -Input [2]: [c_customer_sk#23, ssales#26] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(39) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(40) CometSortMergeJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [c_customer_sk#23] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#23], LeftSemi - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Condition : isnotnull(c_customer_sk#29) - -(43) CometExchange -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(44) CometSort -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Arguments: [c_customer_sk#29, c_first_name#30, c_last_name#31], [c_customer_sk#29 ASC NULLS FIRST] - -(45) ReusedExchange [Reuses operator id: 35] -Output [3]: [c_customer_sk#23, sum#24, isEmpty#25] - -(46) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(47) CometFilter -Input [2]: [c_customer_sk#23, ssales#26] -Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) - -(48) CometProject -Input [2]: [c_customer_sk#23, ssales#26] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(49) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(50) CometSortMergeJoin -Left output [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Right output [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#29], [c_customer_sk#23], LeftSemi - -(51) CometProject -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Arguments: [c_customer_sk#29, c_first_name#32, c_last_name#33], [c_customer_sk#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#30, 20, true, false, true) AS c_first_name#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#31, 30, true, false, true) AS c_last_name#33] - -(52) CometBroadcastExchange -Input [3]: [c_customer_sk#29, c_first_name#32, c_last_name#33] -Arguments: [c_customer_sk#29, c_first_name#32, c_last_name#33] - -(53) CometBroadcastHashJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [3]: [c_customer_sk#29, c_first_name#32, c_last_name#33] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#29], Inner, BuildRight - -(54) CometProject -Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#29, c_first_name#32, c_last_name#33] -Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#35, d_moy#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Condition : ((((isnotnull(d_year#35) AND isnotnull(d_moy#36)) AND (d_year#35 = 2000)) AND (d_moy#36 = 2)) AND isnotnull(d_date_sk#34)) - -(57) CometProject -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Arguments: [d_date_sk#34], [d_date_sk#34] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#34] -Arguments: [d_date_sk#34] - -(59) CometBroadcastHashJoin -Left output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33] -Right output [1]: [d_date_sk#34] -Arguments: [cs_sold_date_sk#5], [d_date_sk#34], Inner, BuildRight - -(60) CometProject -Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33, d_date_sk#34] -Arguments: [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33], [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33] - -(61) CometHashAggregate -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33] -Keys [2]: [c_last_name#33, c_first_name#32] -Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] - -(62) CometExchange -Input [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] -Arguments: hashpartitioning(c_last_name#33, c_first_name#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(63) CometHashAggregate -Input [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] -Keys [2]: [c_last_name#33, c_first_name#32] -Functions [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#43), dynamicpruningexpression(ws_sold_date_sk#43 IN dynamicpruning#44)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(65) CometFilter -Input [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Condition : isnotnull(ws_bill_customer_sk#40) - -(66) ReusedExchange [Reuses operator id: 21] -Output [1]: [item_sk#17] - -(67) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Right output [1]: [item_sk#17] -Arguments: [ws_item_sk#39], [item_sk#17], LeftSemi, BuildRight - -(68) CometProject -Input [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Arguments: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43], [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] - -(69) CometExchange -Input [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Arguments: hashpartitioning(ws_bill_customer_sk#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(70) CometSort -Input [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Arguments: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43], [ws_bill_customer_sk#40 ASC NULLS FIRST] - -(71) ReusedExchange [Reuses operator id: 35] -Output [3]: [c_customer_sk#23, sum#24, isEmpty#25] - -(72) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(73) CometFilter -Input [2]: [c_customer_sk#23, ssales#26] -Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) - -(74) CometProject -Input [2]: [c_customer_sk#23, ssales#26] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(75) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(76) CometSortMergeJoin -Left output [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Right output [1]: [c_customer_sk#23] -Arguments: [ws_bill_customer_sk#40], [c_customer_sk#23], LeftSemi - -(77) ReusedExchange [Reuses operator id: 52] -Output [3]: [c_customer_sk#45, c_first_name#46, c_last_name#47] - -(78) CometBroadcastHashJoin -Left output [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Right output [3]: [c_customer_sk#45, c_first_name#46, c_last_name#47] -Arguments: [ws_bill_customer_sk#40], [c_customer_sk#45], Inner, BuildRight - -(79) CometProject -Input [7]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_customer_sk#45, c_first_name#46, c_last_name#47] -Arguments: [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#46, c_last_name#47], [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#46, c_last_name#47] - -(80) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#48] - -(81) CometBroadcastHashJoin -Left output [5]: [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#46, c_last_name#47] -Right output [1]: [d_date_sk#48] -Arguments: [ws_sold_date_sk#43], [d_date_sk#48], Inner, BuildRight - -(82) CometProject -Input [6]: [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#46, c_last_name#47, d_date_sk#48] -Arguments: [ws_quantity#41, ws_list_price#42, c_first_name#46, c_last_name#47], [ws_quantity#41, ws_list_price#42, c_first_name#46, c_last_name#47] - -(83) CometHashAggregate -Input [4]: [ws_quantity#41, ws_list_price#42, c_first_name#46, c_last_name#47] -Keys [2]: [c_last_name#47, c_first_name#46] -Functions [1]: [partial_sum((cast(ws_quantity#41 as decimal(10,0)) * ws_list_price#42))] - -(84) CometExchange -Input [4]: [c_last_name#47, c_first_name#46, sum#49, isEmpty#50] -Arguments: hashpartitioning(c_last_name#47, c_first_name#46, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(85) CometHashAggregate -Input [4]: [c_last_name#47, c_first_name#46, sum#49, isEmpty#50] -Keys [2]: [c_last_name#47, c_first_name#46] -Functions [1]: [sum((cast(ws_quantity#41 as decimal(10,0)) * ws_list_price#42))] - -(86) CometUnion -Child 0 Input [3]: [c_last_name#33, c_first_name#32, sales#51] -Child 1 Input [3]: [c_last_name#47, c_first_name#46, sales#52] - -(87) CometTakeOrderedAndProject -Input [3]: [c_last_name#33, c_first_name#32, sales#51] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#33 ASC NULLS FIRST,c_first_name#32 ASC NULLS FIRST,sales#51 ASC NULLS FIRST], output=[c_last_name#33,c_first_name#32,sales#51]), [c_last_name#33, c_first_name#32, sales#51], 100, 0, [c_last_name#33 ASC NULLS FIRST, c_first_name#32 ASC NULLS FIRST, sales#51 ASC NULLS FIRST], [c_last_name#33, c_first_name#32, sales#51] - -(88) CometColumnarToRow [codegen id : 1] -Input [3]: [c_last_name#33, c_first_name#32, sales#51] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (93) -+- * CometColumnarToRow (92) - +- CometProject (91) - +- CometFilter (90) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (89) - - -(89) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#35, d_moy#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(90) CometFilter -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Condition : ((((isnotnull(d_year#35) AND isnotnull(d_moy#36)) AND (d_year#35 = 2000)) AND (d_moy#36 = 2)) AND isnotnull(d_date_sk#34)) - -(91) CometProject -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Arguments: [d_date_sk#34], [d_date_sk#34] - -(92) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#34] - -(93) BroadcastExchange -Input [1]: [d_date_sk#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (98) -+- * CometColumnarToRow (97) - +- CometProject (96) - +- CometFilter (95) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (94) - - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(95) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(96) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(97) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_date#11] - -(98) BroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:3 Hosting operator id = 37 Hosting Expression = Subquery scalar-subquery#27, [id=#28] -* CometColumnarToRow (116) -+- CometHashAggregate (115) - +- CometExchange (114) - +- CometHashAggregate (113) - +- CometHashAggregate (112) - +- CometExchange (111) - +- CometHashAggregate (110) - +- CometProject (109) - +- CometBroadcastHashJoin (108) - :- CometProject (103) - : +- CometBroadcastHashJoin (102) - : :- CometFilter (100) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (99) - : +- ReusedExchange (101) - +- CometBroadcastExchange (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) - - -(99) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#53, ss_quantity#54, ss_sales_price#55, ss_sold_date_sk#56] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#56), dynamicpruningexpression(ss_sold_date_sk#56 IN dynamicpruning#57)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(100) CometFilter -Input [4]: [ss_customer_sk#53, ss_quantity#54, ss_sales_price#55, ss_sold_date_sk#56] -Condition : isnotnull(ss_customer_sk#53) - -(101) ReusedExchange [Reuses operator id: 31] -Output [1]: [c_customer_sk#58] - -(102) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#53, ss_quantity#54, ss_sales_price#55, ss_sold_date_sk#56] -Right output [1]: [c_customer_sk#58] -Arguments: [ss_customer_sk#53], [c_customer_sk#58], Inner, BuildRight - -(103) CometProject -Input [5]: [ss_customer_sk#53, ss_quantity#54, ss_sales_price#55, ss_sold_date_sk#56, c_customer_sk#58] -Arguments: [ss_quantity#54, ss_sales_price#55, ss_sold_date_sk#56, c_customer_sk#58], [ss_quantity#54, ss_sales_price#55, ss_sold_date_sk#56, c_customer_sk#58] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#59, d_year#60] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [d_date_sk#59, d_year#60] -Condition : (d_year#60 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#59)) - -(106) CometProject -Input [2]: [d_date_sk#59, d_year#60] -Arguments: [d_date_sk#59], [d_date_sk#59] - -(107) CometBroadcastExchange -Input [1]: [d_date_sk#59] -Arguments: [d_date_sk#59] - -(108) CometBroadcastHashJoin -Left output [4]: [ss_quantity#54, ss_sales_price#55, ss_sold_date_sk#56, c_customer_sk#58] -Right output [1]: [d_date_sk#59] -Arguments: [ss_sold_date_sk#56], [d_date_sk#59], Inner, BuildRight - -(109) CometProject -Input [5]: [ss_quantity#54, ss_sales_price#55, ss_sold_date_sk#56, c_customer_sk#58, d_date_sk#59] -Arguments: [ss_quantity#54, ss_sales_price#55, c_customer_sk#58], [ss_quantity#54, ss_sales_price#55, c_customer_sk#58] - -(110) CometHashAggregate -Input [3]: [ss_quantity#54, ss_sales_price#55, c_customer_sk#58] -Keys [1]: [c_customer_sk#58] -Functions [1]: [partial_sum((cast(ss_quantity#54 as decimal(10,0)) * ss_sales_price#55))] - -(111) CometExchange -Input [3]: [c_customer_sk#58, sum#61, isEmpty#62] -Arguments: hashpartitioning(c_customer_sk#58, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(112) CometHashAggregate -Input [3]: [c_customer_sk#58, sum#61, isEmpty#62] -Keys [1]: [c_customer_sk#58] -Functions [1]: [sum((cast(ss_quantity#54 as decimal(10,0)) * ss_sales_price#55))] - -(113) CometHashAggregate -Input [1]: [csales#63] -Keys: [] -Functions [1]: [partial_max(csales#63)] - -(114) CometExchange -Input [1]: [max#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(115) CometHashAggregate -Input [1]: [max#64] -Keys: [] -Functions [1]: [max(csales#63)] - -(116) CometColumnarToRow [codegen id : 1] -Input [1]: [tpcds_cmax#65] - -Subquery:4 Hosting operator id = 99 Hosting Expression = ss_sold_date_sk#56 IN dynamicpruning#57 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#59, d_year#60] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#59, d_year#60] -Condition : (d_year#60 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#59)) - -(119) CometProject -Input [2]: [d_date_sk#59, d_year#60] -Arguments: [d_date_sk#59], [d_date_sk#59] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#59] - -(121) BroadcastExchange -Input [1]: [d_date_sk#59] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -Subquery:5 Hosting operator id = 47 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] - -Subquery:6 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#43 IN dynamicpruning#6 - -Subquery:7 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/simplified.txt deleted file mode 100644 index 92563114ae..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,138 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,sales] - CometUnion [c_last_name,c_first_name,sales] - CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price))] - CometExchange [c_last_name,c_first_name] #1 - CometHashAggregate [cs_quantity,cs_list_price] [c_last_name,c_first_name,sum,isEmpty] - CometProject [cs_quantity,cs_list_price,c_first_name,c_last_name] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometExchange [cs_bill_customer_sk] #2 - CometProject [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,item_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [item_sk] #4 - CometProject [item_sk] - CometFilter [item_sk,cnt] - CometHashAggregate [_groupingexpression,d_date,count] [item_sk,cnt,i_item_sk,count(1)] - CometExchange [_groupingexpression,i_item_sk,d_date] #5 - CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] - CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] - CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [i_item_sk,i_item_desc] #8 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [max] [tpcds_cmax,max(csales)] - CometExchange #11 - CometHashAggregate [csales] [max] - CometHashAggregate [c_customer_sk,sum,isEmpty] [csales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #12 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,c_customer_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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #9 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,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,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #10 - CometFilter [c_customer_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #15 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [c_customer_sk,c_first_name,c_last_name,c_customer_sk] - CometSort [c_customer_sk,c_first_name,c_last_name] - CometExchange [c_customer_sk] #16 - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #9 - CometBroadcastExchange [d_date_sk] #17 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price))] - CometExchange [c_last_name,c_first_name] #18 - CometHashAggregate [ws_quantity,ws_list_price] [c_last_name,c_first_name,sum,isEmpty] - CometProject [ws_quantity,ws_list_price,c_first_name,c_last_name] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometExchange [ws_bill_customer_sk] #19 - CometProject [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #9 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 - ReusedExchange [d_date_sk] #17 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt deleted file mode 100644 index 67e4e39057..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt +++ /dev/null @@ -1,692 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (88) -+- CometTakeOrderedAndProject (87) - +- CometUnion (86) - :- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometSortMergeJoin (40) - : : : :- CometSort (25) - : : : : +- CometExchange (24) - : : : : +- CometProject (23) - : : : : +- CometBroadcastHashJoin (22) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometBroadcastExchange (21) - : : : : +- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometHashAggregate (18) - : : : : +- CometExchange (17) - : : : : +- CometHashAggregate (16) - : : : : +- CometProject (15) - : : : : +- CometBroadcastHashJoin (14) - : : : : :- CometProject (10) - : : : : : +- CometBroadcastHashJoin (9) - : : : : : :- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : : +- CometBroadcastExchange (8) - : : : : : +- CometProject (7) - : : : : : +- CometFilter (6) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (5) - : : : : +- CometBroadcastExchange (13) - : : : : +- CometFilter (12) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : +- CometSort (39) - : : : +- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometHashAggregate (36) - : : : +- CometExchange (35) - : : : +- CometHashAggregate (34) - : : : +- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometProject (28) - : : : : +- CometFilter (27) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (26) - : : : +- CometBroadcastExchange (31) - : : : +- CometFilter (30) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (29) - : : +- CometBroadcastExchange (52) - : : +- CometProject (51) - : : +- CometSortMergeJoin (50) - : : :- CometSort (44) - : : : +- CometExchange (43) - : : : +- CometFilter (42) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (41) - : : +- CometSort (49) - : : +- CometProject (48) - : : +- CometFilter (47) - : : +- CometHashAggregate (46) - : : +- ReusedExchange (45) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometHashAggregate (85) - +- CometExchange (84) - +- CometHashAggregate (83) - +- CometProject (82) - +- CometBroadcastHashJoin (81) - :- CometProject (79) - : +- CometBroadcastHashJoin (78) - : :- CometSortMergeJoin (76) - : : :- CometSort (70) - : : : +- CometExchange (69) - : : : +- CometProject (68) - : : : +- CometBroadcastHashJoin (67) - : : : :- CometFilter (65) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (64) - : : : +- ReusedExchange (66) - : : +- CometSort (75) - : : +- CometProject (74) - : : +- CometFilter (73) - : : +- CometHashAggregate (72) - : : +- ReusedExchange (71) - : +- ReusedExchange (77) - +- ReusedExchange (80) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_bill_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Condition : isnotnull(ss_item_sk#7) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(7) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(8) CometBroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: [d_date_sk#10, d_date#11] - -(9) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, ss_sold_date_sk#8] -Right output [2]: [d_date_sk#10, d_date#11] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(10) CometProject -Input [4]: [ss_item_sk#7, ss_sold_date_sk#8, d_date_sk#10, d_date#11] -Arguments: [ss_item_sk#7, d_date#11], [ss_item_sk#7, d_date#11] - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#13, i_item_desc#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [2]: [i_item_sk#13, i_item_desc#14] -Condition : isnotnull(i_item_sk#13) - -(13) CometBroadcastExchange -Input [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [i_item_sk#13, i_item_desc#14] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#7, d_date#11] -Right output [2]: [i_item_sk#13, i_item_desc#14] -Arguments: [ss_item_sk#7], [i_item_sk#13], Inner, BuildRight - -(15) CometProject -Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#13, i_item_desc#14] -Arguments: [d_date#11, i_item_sk#13, _groupingexpression#15], [d_date#11, i_item_sk#13, substr(i_item_desc#14, 1, 30) AS _groupingexpression#15] - -(16) CometHashAggregate -Input [3]: [d_date#11, i_item_sk#13, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [partial_count(1)] - -(17) CometExchange -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Arguments: hashpartitioning(_groupingexpression#15, i_item_sk#13, d_date#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [4]: [_groupingexpression#15, i_item_sk#13, d_date#11, count#16] -Keys [3]: [_groupingexpression#15, i_item_sk#13, d_date#11] -Functions [1]: [count(1)] - -(19) CometFilter -Input [2]: [item_sk#17, cnt#18] -Condition : (cnt#18 > 4) - -(20) CometProject -Input [2]: [item_sk#17, cnt#18] -Arguments: [item_sk#17], [item_sk#17] - -(21) CometBroadcastExchange -Input [1]: [item_sk#17] -Arguments: [item_sk#17] - -(22) CometBroadcastHashJoin -Left output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [item_sk#17] -Arguments: [cs_item_sk#2], [item_sk#17], LeftSemi, BuildRight - -(23) CometProject -Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] - -(24) CometExchange -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(25) CometSort -Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Arguments: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5], [cs_bill_customer_sk#1 ASC NULLS FIRST] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#19) - -(28) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, ss_sold_date_sk#22] -Arguments: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21], [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [1]: [c_customer_sk#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(30) CometFilter -Input [1]: [c_customer_sk#23] -Condition : isnotnull(c_customer_sk#23) - -(31) CometBroadcastExchange -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23] - -(32) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21] -Right output [1]: [c_customer_sk#23] -Arguments: [ss_customer_sk#19], [c_customer_sk#23], Inner, BuildRight - -(33) CometProject -Input [4]: [ss_customer_sk#19, ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Arguments: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23], [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] - -(34) CometHashAggregate -Input [3]: [ss_quantity#20, ss_sales_price#21, c_customer_sk#23] -Keys [1]: [c_customer_sk#23] -Functions [1]: [partial_sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(35) CometExchange -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(36) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(37) CometFilter -Input [2]: [c_customer_sk#23, ssales#26] -Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#27, [id=#28]))) - -(38) CometProject -Input [2]: [c_customer_sk#23, ssales#26] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(39) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(40) CometSortMergeJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [1]: [c_customer_sk#23] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#23], LeftSemi - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Condition : isnotnull(c_customer_sk#29) - -(43) CometExchange -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(44) CometSort -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Arguments: [c_customer_sk#29, c_first_name#30, c_last_name#31], [c_customer_sk#29 ASC NULLS FIRST] - -(45) ReusedExchange [Reuses operator id: 35] -Output [3]: [c_customer_sk#23, sum#24, isEmpty#25] - -(46) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(47) CometFilter -Input [2]: [c_customer_sk#23, ssales#26] -Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) - -(48) CometProject -Input [2]: [c_customer_sk#23, ssales#26] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(49) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(50) CometSortMergeJoin -Left output [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Right output [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#29], [c_customer_sk#23], LeftSemi - -(51) CometProject -Input [3]: [c_customer_sk#29, c_first_name#30, c_last_name#31] -Arguments: [c_customer_sk#29, c_first_name#32, c_last_name#33], [c_customer_sk#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#30, 20, true, false, true) AS c_first_name#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#31, 30, true, false, true) AS c_last_name#33] - -(52) CometBroadcastExchange -Input [3]: [c_customer_sk#29, c_first_name#32, c_last_name#33] -Arguments: [c_customer_sk#29, c_first_name#32, c_last_name#33] - -(53) CometBroadcastHashJoin -Left output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] -Right output [3]: [c_customer_sk#29, c_first_name#32, c_last_name#33] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#29], Inner, BuildRight - -(54) CometProject -Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#29, c_first_name#32, c_last_name#33] -Arguments: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33], [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#35, d_moy#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Condition : ((((isnotnull(d_year#35) AND isnotnull(d_moy#36)) AND (d_year#35 = 2000)) AND (d_moy#36 = 2)) AND isnotnull(d_date_sk#34)) - -(57) CometProject -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Arguments: [d_date_sk#34], [d_date_sk#34] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#34] -Arguments: [d_date_sk#34] - -(59) CometBroadcastHashJoin -Left output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33] -Right output [1]: [d_date_sk#34] -Arguments: [cs_sold_date_sk#5], [d_date_sk#34], Inner, BuildRight - -(60) CometProject -Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#32, c_last_name#33, d_date_sk#34] -Arguments: [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33], [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33] - -(61) CometHashAggregate -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#32, c_last_name#33] -Keys [2]: [c_last_name#33, c_first_name#32] -Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] - -(62) CometExchange -Input [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] -Arguments: hashpartitioning(c_last_name#33, c_first_name#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(63) CometHashAggregate -Input [4]: [c_last_name#33, c_first_name#32, sum#37, isEmpty#38] -Keys [2]: [c_last_name#33, c_first_name#32] -Functions [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#43), dynamicpruningexpression(ws_sold_date_sk#43 IN dynamicpruning#44)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(65) CometFilter -Input [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Condition : isnotnull(ws_bill_customer_sk#40) - -(66) ReusedExchange [Reuses operator id: 21] -Output [1]: [item_sk#17] - -(67) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Right output [1]: [item_sk#17] -Arguments: [ws_item_sk#39], [item_sk#17], LeftSemi, BuildRight - -(68) CometProject -Input [5]: [ws_item_sk#39, ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Arguments: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43], [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] - -(69) CometExchange -Input [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Arguments: hashpartitioning(ws_bill_customer_sk#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(70) CometSort -Input [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Arguments: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43], [ws_bill_customer_sk#40 ASC NULLS FIRST] - -(71) ReusedExchange [Reuses operator id: 35] -Output [3]: [c_customer_sk#23, sum#24, isEmpty#25] - -(72) CometHashAggregate -Input [3]: [c_customer_sk#23, sum#24, isEmpty#25] -Keys [1]: [c_customer_sk#23] -Functions [1]: [sum((cast(ss_quantity#20 as decimal(10,0)) * ss_sales_price#21))] - -(73) CometFilter -Input [2]: [c_customer_sk#23, ssales#26] -Condition : (isnotnull(ssales#26) AND (cast(ssales#26 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#27, [id=#28]))) - -(74) CometProject -Input [2]: [c_customer_sk#23, ssales#26] -Arguments: [c_customer_sk#23], [c_customer_sk#23] - -(75) CometSort -Input [1]: [c_customer_sk#23] -Arguments: [c_customer_sk#23], [c_customer_sk#23 ASC NULLS FIRST] - -(76) CometSortMergeJoin -Left output [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Right output [1]: [c_customer_sk#23] -Arguments: [ws_bill_customer_sk#40], [c_customer_sk#23], LeftSemi - -(77) ReusedExchange [Reuses operator id: 52] -Output [3]: [c_customer_sk#45, c_first_name#46, c_last_name#47] - -(78) CometBroadcastHashJoin -Left output [4]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43] -Right output [3]: [c_customer_sk#45, c_first_name#46, c_last_name#47] -Arguments: [ws_bill_customer_sk#40], [c_customer_sk#45], Inner, BuildRight - -(79) CometProject -Input [7]: [ws_bill_customer_sk#40, ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_customer_sk#45, c_first_name#46, c_last_name#47] -Arguments: [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#46, c_last_name#47], [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#46, c_last_name#47] - -(80) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#48] - -(81) CometBroadcastHashJoin -Left output [5]: [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#46, c_last_name#47] -Right output [1]: [d_date_sk#48] -Arguments: [ws_sold_date_sk#43], [d_date_sk#48], Inner, BuildRight - -(82) CometProject -Input [6]: [ws_quantity#41, ws_list_price#42, ws_sold_date_sk#43, c_first_name#46, c_last_name#47, d_date_sk#48] -Arguments: [ws_quantity#41, ws_list_price#42, c_first_name#46, c_last_name#47], [ws_quantity#41, ws_list_price#42, c_first_name#46, c_last_name#47] - -(83) CometHashAggregate -Input [4]: [ws_quantity#41, ws_list_price#42, c_first_name#46, c_last_name#47] -Keys [2]: [c_last_name#47, c_first_name#46] -Functions [1]: [partial_sum((cast(ws_quantity#41 as decimal(10,0)) * ws_list_price#42))] - -(84) CometExchange -Input [4]: [c_last_name#47, c_first_name#46, sum#49, isEmpty#50] -Arguments: hashpartitioning(c_last_name#47, c_first_name#46, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(85) CometHashAggregate -Input [4]: [c_last_name#47, c_first_name#46, sum#49, isEmpty#50] -Keys [2]: [c_last_name#47, c_first_name#46] -Functions [1]: [sum((cast(ws_quantity#41 as decimal(10,0)) * ws_list_price#42))] - -(86) CometUnion -Child 0 Input [3]: [c_last_name#33, c_first_name#32, sales#51] -Child 1 Input [3]: [c_last_name#47, c_first_name#46, sales#52] - -(87) CometTakeOrderedAndProject -Input [3]: [c_last_name#33, c_first_name#32, sales#51] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#33 ASC NULLS FIRST,c_first_name#32 ASC NULLS FIRST,sales#51 ASC NULLS FIRST], output=[c_last_name#33,c_first_name#32,sales#51]), [c_last_name#33, c_first_name#32, sales#51], 100, 0, [c_last_name#33 ASC NULLS FIRST, c_first_name#32 ASC NULLS FIRST, sales#51 ASC NULLS FIRST], [c_last_name#33, c_first_name#32, sales#51] - -(88) CometColumnarToRow [codegen id : 1] -Input [3]: [c_last_name#33, c_first_name#32, sales#51] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (93) -+- * CometColumnarToRow (92) - +- CometProject (91) - +- CometFilter (90) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (89) - - -(89) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#35, d_moy#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(90) CometFilter -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Condition : ((((isnotnull(d_year#35) AND isnotnull(d_moy#36)) AND (d_year#35 = 2000)) AND (d_moy#36 = 2)) AND isnotnull(d_date_sk#34)) - -(91) CometProject -Input [3]: [d_date_sk#34, d_year#35, d_moy#36] -Arguments: [d_date_sk#34], [d_date_sk#34] - -(92) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#34] - -(93) BroadcastExchange -Input [1]: [d_date_sk#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (98) -+- * CometColumnarToRow (97) - +- CometProject (96) - +- CometFilter (95) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (94) - - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_date#11, d_year#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(95) CometFilter -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) - -(96) CometProject -Input [3]: [d_date_sk#10, d_date#11, d_year#12] -Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] - -(97) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#10, d_date#11] - -(98) BroadcastExchange -Input [2]: [d_date_sk#10, d_date#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:3 Hosting operator id = 37 Hosting Expression = Subquery scalar-subquery#27, [id=#28] -* CometColumnarToRow (116) -+- CometHashAggregate (115) - +- CometExchange (114) - +- CometHashAggregate (113) - +- CometHashAggregate (112) - +- CometExchange (111) - +- CometHashAggregate (110) - +- CometProject (109) - +- CometBroadcastHashJoin (108) - :- CometProject (103) - : +- CometBroadcastHashJoin (102) - : :- CometFilter (100) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (99) - : +- ReusedExchange (101) - +- CometBroadcastExchange (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) - - -(99) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#53, ss_quantity#54, ss_sales_price#55, ss_sold_date_sk#56] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#56), dynamicpruningexpression(ss_sold_date_sk#56 IN dynamicpruning#57)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(100) CometFilter -Input [4]: [ss_customer_sk#53, ss_quantity#54, ss_sales_price#55, ss_sold_date_sk#56] -Condition : isnotnull(ss_customer_sk#53) - -(101) ReusedExchange [Reuses operator id: 31] -Output [1]: [c_customer_sk#58] - -(102) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#53, ss_quantity#54, ss_sales_price#55, ss_sold_date_sk#56] -Right output [1]: [c_customer_sk#58] -Arguments: [ss_customer_sk#53], [c_customer_sk#58], Inner, BuildRight - -(103) CometProject -Input [5]: [ss_customer_sk#53, ss_quantity#54, ss_sales_price#55, ss_sold_date_sk#56, c_customer_sk#58] -Arguments: [ss_quantity#54, ss_sales_price#55, ss_sold_date_sk#56, c_customer_sk#58], [ss_quantity#54, ss_sales_price#55, ss_sold_date_sk#56, c_customer_sk#58] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#59, d_year#60] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [d_date_sk#59, d_year#60] -Condition : (d_year#60 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#59)) - -(106) CometProject -Input [2]: [d_date_sk#59, d_year#60] -Arguments: [d_date_sk#59], [d_date_sk#59] - -(107) CometBroadcastExchange -Input [1]: [d_date_sk#59] -Arguments: [d_date_sk#59] - -(108) CometBroadcastHashJoin -Left output [4]: [ss_quantity#54, ss_sales_price#55, ss_sold_date_sk#56, c_customer_sk#58] -Right output [1]: [d_date_sk#59] -Arguments: [ss_sold_date_sk#56], [d_date_sk#59], Inner, BuildRight - -(109) CometProject -Input [5]: [ss_quantity#54, ss_sales_price#55, ss_sold_date_sk#56, c_customer_sk#58, d_date_sk#59] -Arguments: [ss_quantity#54, ss_sales_price#55, c_customer_sk#58], [ss_quantity#54, ss_sales_price#55, c_customer_sk#58] - -(110) CometHashAggregate -Input [3]: [ss_quantity#54, ss_sales_price#55, c_customer_sk#58] -Keys [1]: [c_customer_sk#58] -Functions [1]: [partial_sum((cast(ss_quantity#54 as decimal(10,0)) * ss_sales_price#55))] - -(111) CometExchange -Input [3]: [c_customer_sk#58, sum#61, isEmpty#62] -Arguments: hashpartitioning(c_customer_sk#58, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(112) CometHashAggregate -Input [3]: [c_customer_sk#58, sum#61, isEmpty#62] -Keys [1]: [c_customer_sk#58] -Functions [1]: [sum((cast(ss_quantity#54 as decimal(10,0)) * ss_sales_price#55))] - -(113) CometHashAggregate -Input [1]: [csales#63] -Keys: [] -Functions [1]: [partial_max(csales#63)] - -(114) CometExchange -Input [1]: [max#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(115) CometHashAggregate -Input [1]: [max#64] -Keys: [] -Functions [1]: [max(csales#63)] - -(116) CometColumnarToRow [codegen id : 1] -Input [1]: [tpcds_cmax#65] - -Subquery:4 Hosting operator id = 99 Hosting Expression = ss_sold_date_sk#56 IN dynamicpruning#57 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#59, d_year#60] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#59, d_year#60] -Condition : (d_year#60 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#59)) - -(119) CometProject -Input [2]: [d_date_sk#59, d_year#60] -Arguments: [d_date_sk#59], [d_date_sk#59] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#59] - -(121) BroadcastExchange -Input [1]: [d_date_sk#59] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -Subquery:5 Hosting operator id = 47 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] - -Subquery:6 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#43 IN dynamicpruning#6 - -Subquery:7 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/extended.txt deleted file mode 100644 index cd91f5de49..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/extended.txt +++ /dev/null @@ -1,198 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometSort - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometSort - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometSort - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometSort - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 177 out of 190 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 92563114ae..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt +++ /dev/null @@ -1,138 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,sales] - CometUnion [c_last_name,c_first_name,sales] - CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price))] - CometExchange [c_last_name,c_first_name] #1 - CometHashAggregate [cs_quantity,cs_list_price] [c_last_name,c_first_name,sum,isEmpty] - CometProject [cs_quantity,cs_list_price,c_first_name,c_last_name] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk,c_customer_sk] - CometSort [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometExchange [cs_bill_customer_sk] #2 - CometProject [cs_bill_customer_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,item_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [item_sk] #4 - CometProject [item_sk] - CometFilter [item_sk,cnt] - CometHashAggregate [_groupingexpression,d_date,count] [item_sk,cnt,i_item_sk,count(1)] - CometExchange [_groupingexpression,i_item_sk,d_date] #5 - CometHashAggregate [_groupingexpression,i_item_sk,d_date,count] - CometProject [i_item_desc] [d_date,i_item_sk,_groupingexpression] - CometBroadcastHashJoin [ss_item_sk,d_date,i_item_sk,i_item_desc] - CometProject [ss_item_sk,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] - CometBroadcastExchange [i_item_sk,i_item_desc] #8 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [max] [tpcds_cmax,max(csales)] - CometExchange #11 - CometHashAggregate [csales] [max] - CometHashAggregate [c_customer_sk,sum,isEmpty] [csales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #12 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,c_customer_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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #13 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [c_customer_sk] #10 - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - CometExchange [c_customer_sk] #9 - CometHashAggregate [ss_quantity,ss_sales_price] [c_customer_sk,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,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,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [c_customer_sk] #10 - CometFilter [c_customer_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #15 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [c_customer_sk,c_first_name,c_last_name,c_customer_sk] - CometSort [c_customer_sk,c_first_name,c_last_name] - CometExchange [c_customer_sk] #16 - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #9 - CometBroadcastExchange [d_date_sk] #17 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometHashAggregate [sum,isEmpty] [c_last_name,c_first_name,sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price))] - CometExchange [c_last_name,c_first_name] #18 - CometHashAggregate [ws_quantity,ws_list_price] [c_last_name,c_first_name,sum,isEmpty] - CometProject [ws_quantity,ws_list_price,c_first_name,c_last_name] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk,c_first_name,c_last_name] - CometSortMergeJoin [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,c_customer_sk] - CometSort [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometExchange [ws_bill_customer_sk] #19 - CometProject [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [item_sk] #4 - CometSort [c_customer_sk] - CometProject [c_customer_sk] - CometFilter [c_customer_sk,ssales] - ReusedSubquery [tpcds_cmax] #3 - CometHashAggregate [sum,isEmpty] [c_customer_sk,ssales,sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price))] - ReusedExchange [c_customer_sk,sum,isEmpty] #9 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #15 - ReusedExchange [d_date_sk] #17 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/explain.txt deleted file mode 100644 index acc9219af4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/explain.txt +++ /dev/null @@ -1,439 +0,0 @@ -== Physical Plan == -* Filter (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometNativeScan parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometNativeScan parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.customer_address (32) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometNativeScan parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) = pale ) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#19, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#21, 10, true, false, true) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Condition : (isnotnull(c_customer_sk#26) AND isnotnull(c_birth_country#29)) - -(27) CometProject -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29], [c_customer_sk#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#31, c_birth_country#29] - -(28) CometBroadcastExchange -Input [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(31) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(32) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_state#32, ca_zip#33, ca_country#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true))) - -(34) CometProject -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] - -(35) CometColumnarToRow [codegen id : 1] -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] - -(36) BroadcastExchange -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#29, s_zip#16] -Right keys [2]: [upper(ca_country#34), ca_zip#36] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#37] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(40) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [partial_sum(netpaid#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(44) CometColumnarExchange -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [sum(netpaid#40)] -Aggregate Attributes [1]: [sum(netpaid#40)#45] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] -Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (77) -+- * CometColumnarToRow (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (56) - : : : +- CometBroadcastHashJoin (55) - : : : :- CometProject (53) - : : : : +- CometSortMergeJoin (52) - : : : : :- CometSort (49) - : : : : : +- ReusedExchange (48) - : : : : +- CometSort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (54) - : : +- CometBroadcastExchange (60) - : : +- CometProject (59) - : : +- CometFilter (58) - : : +- CometNativeScan parquet spark_catalog.default.item (57) - : +- ReusedExchange (63) - +- ReusedExchange (67) - - -(48) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] - -(49) CometSort -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53], [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST] - -(50) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#54, sr_ticket_number#55] - -(51) CometSort -Input [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [sr_item_sk#54, sr_ticket_number#55], [sr_ticket_number#55 ASC NULLS FIRST, sr_item_sk#54 ASC NULLS FIRST] - -(52) CometSortMergeJoin -Left output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Right output [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_ticket_number#52, ss_item_sk#49], [sr_ticket_number#55, sr_item_sk#54], Inner - -(53) CometProject -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53], [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] - -(54) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] - -(55) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Right output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_store_sk#51], [s_store_sk#56], Inner, BuildRight - -(56) CometProject -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16], [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] - -(57) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Condition : isnotnull(i_item_sk#58) - -(59) CometProject -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [i_item_sk#58, i_current_price#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#60, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#61, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#62, 10, true, false, true) AS i_units#25, i_manager_id#63] - -(60) CometBroadcastExchange -Input [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(61) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_item_sk#49], [i_item_sk#58], Inner, BuildRight - -(62) CometProject -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(63) ReusedExchange [Reuses operator id: 28] -Output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] - -(64) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Right output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight - -(65) CometProject -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(66) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(67) ReusedExchange [Reuses operator id: 36] -Output [3]: [ca_state#35, ca_zip#36, ca_country#66] - -(68) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#65, s_zip#16] -Right keys [2]: [upper(ca_country#66), ca_zip#36] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 2] -Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] - -(70) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#67] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(71) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(72) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(73) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#40] - -(74) HashAggregate [codegen id : 3] -Input [1]: [netpaid#40] -Keys: [] -Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#69, count#70] -Results [2]: [sum#71, count#72] - -(75) CometColumnarExchange -Input [2]: [sum#71, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#71, count#72] - -(77) HashAggregate [codegen id : 4] -Input [2]: [sum#71, count#72] -Keys: [] -Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#73] -Results [1]: [(0.05 * avg(netpaid#40)#73) AS (0.05 * avg(netpaid))#74] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/simplified.txt deleted file mode 100644 index af8d5ee7aa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/simplified.txt +++ /dev/null @@ -1,94 +0,0 @@ -WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #6 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometNativeScan 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/q24a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/explain.txt deleted file mode 100644 index e478cdfc01..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,439 +0,0 @@ -== Physical Plan == -* Filter (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) = pale ) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#19, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#21, 10, true, false, true) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Condition : (isnotnull(c_customer_sk#26) AND isnotnull(c_birth_country#29)) - -(27) CometProject -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29], [c_customer_sk#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#31, c_birth_country#29] - -(28) CometBroadcastExchange -Input [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(31) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_state#32, ca_zip#33, ca_country#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true))) - -(34) CometProject -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] - -(35) CometColumnarToRow [codegen id : 1] -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] - -(36) BroadcastExchange -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#29, s_zip#16] -Right keys [2]: [upper(ca_country#34), ca_zip#36] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#37] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(40) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [partial_sum(netpaid#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(44) CometColumnarExchange -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [sum(netpaid#40)] -Aggregate Attributes [1]: [sum(netpaid#40)#45] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] -Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (77) -+- * CometColumnarToRow (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (56) - : : : +- CometBroadcastHashJoin (55) - : : : :- CometProject (53) - : : : : +- CometSortMergeJoin (52) - : : : : :- CometSort (49) - : : : : : +- ReusedExchange (48) - : : : : +- CometSort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (54) - : : +- CometBroadcastExchange (60) - : : +- CometProject (59) - : : +- CometFilter (58) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (57) - : +- ReusedExchange (63) - +- ReusedExchange (67) - - -(48) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] - -(49) CometSort -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53], [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST] - -(50) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#54, sr_ticket_number#55] - -(51) CometSort -Input [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [sr_item_sk#54, sr_ticket_number#55], [sr_ticket_number#55 ASC NULLS FIRST, sr_item_sk#54 ASC NULLS FIRST] - -(52) CometSortMergeJoin -Left output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Right output [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_ticket_number#52, ss_item_sk#49], [sr_ticket_number#55, sr_item_sk#54], Inner - -(53) CometProject -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53], [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] - -(54) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] - -(55) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Right output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_store_sk#51], [s_store_sk#56], Inner, BuildRight - -(56) CometProject -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16], [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Condition : isnotnull(i_item_sk#58) - -(59) CometProject -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [i_item_sk#58, i_current_price#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#60, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#61, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#62, 10, true, false, true) AS i_units#25, i_manager_id#63] - -(60) CometBroadcastExchange -Input [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(61) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_item_sk#49], [i_item_sk#58], Inner, BuildRight - -(62) CometProject -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(63) ReusedExchange [Reuses operator id: 28] -Output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] - -(64) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Right output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight - -(65) CometProject -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(66) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(67) ReusedExchange [Reuses operator id: 36] -Output [3]: [ca_state#35, ca_zip#36, ca_country#66] - -(68) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#65, s_zip#16] -Right keys [2]: [upper(ca_country#66), ca_zip#36] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 2] -Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] - -(70) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#67] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(71) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(72) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(73) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#40] - -(74) HashAggregate [codegen id : 3] -Input [1]: [netpaid#40] -Keys: [] -Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#69, count#70] -Results [2]: [sum#71, count#72] - -(75) CometColumnarExchange -Input [2]: [sum#71, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#71, count#72] - -(77) HashAggregate [codegen id : 4] -Input [2]: [sum#71, count#72] -Keys: [] -Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#73] -Results [1]: [(0.05 * avg(netpaid#40)#73) AS (0.05 * avg(netpaid))#74] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 440f477d0f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,94 +0,0 @@ -WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #6 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] 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/q24a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt deleted file mode 100644 index e478cdfc01..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt +++ /dev/null @@ -1,439 +0,0 @@ -== Physical Plan == -* Filter (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) = pale ) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#19, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#21, 10, true, false, true) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Condition : (isnotnull(c_customer_sk#26) AND isnotnull(c_birth_country#29)) - -(27) CometProject -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29], [c_customer_sk#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#31, c_birth_country#29] - -(28) CometBroadcastExchange -Input [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(31) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_state#32, ca_zip#33, ca_country#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true))) - -(34) CometProject -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] - -(35) CometColumnarToRow [codegen id : 1] -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] - -(36) BroadcastExchange -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#29, s_zip#16] -Right keys [2]: [upper(ca_country#34), ca_zip#36] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#37] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(40) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [partial_sum(netpaid#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(44) CometColumnarExchange -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [sum(netpaid#40)] -Aggregate Attributes [1]: [sum(netpaid#40)#45] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] -Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (77) -+- * CometColumnarToRow (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (56) - : : : +- CometBroadcastHashJoin (55) - : : : :- CometProject (53) - : : : : +- CometSortMergeJoin (52) - : : : : :- CometSort (49) - : : : : : +- ReusedExchange (48) - : : : : +- CometSort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (54) - : : +- CometBroadcastExchange (60) - : : +- CometProject (59) - : : +- CometFilter (58) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (57) - : +- ReusedExchange (63) - +- ReusedExchange (67) - - -(48) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] - -(49) CometSort -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53], [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST] - -(50) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#54, sr_ticket_number#55] - -(51) CometSort -Input [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [sr_item_sk#54, sr_ticket_number#55], [sr_ticket_number#55 ASC NULLS FIRST, sr_item_sk#54 ASC NULLS FIRST] - -(52) CometSortMergeJoin -Left output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Right output [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_ticket_number#52, ss_item_sk#49], [sr_ticket_number#55, sr_item_sk#54], Inner - -(53) CometProject -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53], [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] - -(54) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] - -(55) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Right output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_store_sk#51], [s_store_sk#56], Inner, BuildRight - -(56) CometProject -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16], [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Condition : isnotnull(i_item_sk#58) - -(59) CometProject -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [i_item_sk#58, i_current_price#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#60, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#61, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#62, 10, true, false, true) AS i_units#25, i_manager_id#63] - -(60) CometBroadcastExchange -Input [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(61) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_item_sk#49], [i_item_sk#58], Inner, BuildRight - -(62) CometProject -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(63) ReusedExchange [Reuses operator id: 28] -Output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] - -(64) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Right output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight - -(65) CometProject -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(66) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(67) ReusedExchange [Reuses operator id: 36] -Output [3]: [ca_state#35, ca_zip#36, ca_country#66] - -(68) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#65, s_zip#16] -Right keys [2]: [upper(ca_country#66), ca_zip#36] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 2] -Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] - -(70) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#67] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(71) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(72) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(73) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#40] - -(74) HashAggregate [codegen id : 3] -Input [1]: [netpaid#40] -Keys: [] -Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#69, count#70] -Results [2]: [sum#71, count#72] - -(75) CometColumnarExchange -Input [2]: [sum#71, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#71, count#72] - -(77) HashAggregate [codegen id : 4] -Input [2]: [sum#71, count#72] -Keys: [] -Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#73] -Results [1]: [(0.05 * avg(netpaid#40)#73) AS (0.05 * avg(netpaid))#74] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/extended.txt deleted file mode 100644 index 8777e81a37..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/extended.txt +++ /dev/null @@ -1,96 +0,0 @@ -Filter -: +- Subquery -: +- HashAggregate -: +- CometColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- HashAggregate -: +- CometColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- Project -: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometColumnarToRow -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometSortMergeJoin -: : : : : :- CometSort -: : : : : : +- CometExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : +- CometSort -: : : : : +- CometExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -: +- BroadcastExchange -: +- CometColumnarToRow -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 70 out of 86 eligible operators (81%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 440f477d0f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/simplified.txt +++ /dev/null @@ -1,94 +0,0 @@ -WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #6 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/explain.txt deleted file mode 100644 index dbebe8f96d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/explain.txt +++ /dev/null @@ -1,439 +0,0 @@ -== Physical Plan == -* Filter (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometNativeScan parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometNativeScan parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.customer_address (32) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometNativeScan parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) = chiffon ) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#19, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#21, 10, true, false, true) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Condition : (isnotnull(c_customer_sk#26) AND isnotnull(c_birth_country#29)) - -(27) CometProject -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29], [c_customer_sk#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#31, c_birth_country#29] - -(28) CometBroadcastExchange -Input [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(31) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(32) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_state#32, ca_zip#33, ca_country#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true))) - -(34) CometProject -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] - -(35) CometColumnarToRow [codegen id : 1] -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] - -(36) BroadcastExchange -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#29, s_zip#16] -Right keys [2]: [upper(ca_country#34), ca_zip#36] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#37] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(40) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [partial_sum(netpaid#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(44) CometColumnarExchange -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [sum(netpaid#40)] -Aggregate Attributes [1]: [sum(netpaid#40)#45] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] -Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (77) -+- * CometColumnarToRow (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (56) - : : : +- CometBroadcastHashJoin (55) - : : : :- CometProject (53) - : : : : +- CometSortMergeJoin (52) - : : : : :- CometSort (49) - : : : : : +- ReusedExchange (48) - : : : : +- CometSort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (54) - : : +- CometBroadcastExchange (60) - : : +- CometProject (59) - : : +- CometFilter (58) - : : +- CometNativeScan parquet spark_catalog.default.item (57) - : +- ReusedExchange (63) - +- ReusedExchange (67) - - -(48) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] - -(49) CometSort -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53], [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST] - -(50) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#54, sr_ticket_number#55] - -(51) CometSort -Input [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [sr_item_sk#54, sr_ticket_number#55], [sr_ticket_number#55 ASC NULLS FIRST, sr_item_sk#54 ASC NULLS FIRST] - -(52) CometSortMergeJoin -Left output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Right output [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_ticket_number#52, ss_item_sk#49], [sr_ticket_number#55, sr_item_sk#54], Inner - -(53) CometProject -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53], [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] - -(54) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] - -(55) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Right output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_store_sk#51], [s_store_sk#56], Inner, BuildRight - -(56) CometProject -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16], [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] - -(57) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Condition : isnotnull(i_item_sk#58) - -(59) CometProject -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [i_item_sk#58, i_current_price#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#60, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#61, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#62, 10, true, false, true) AS i_units#25, i_manager_id#63] - -(60) CometBroadcastExchange -Input [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(61) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_item_sk#49], [i_item_sk#58], Inner, BuildRight - -(62) CometProject -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(63) ReusedExchange [Reuses operator id: 28] -Output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] - -(64) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Right output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight - -(65) CometProject -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(66) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(67) ReusedExchange [Reuses operator id: 36] -Output [3]: [ca_state#35, ca_zip#36, ca_country#66] - -(68) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#65, s_zip#16] -Right keys [2]: [upper(ca_country#66), ca_zip#36] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 2] -Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] - -(70) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#67] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(71) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(72) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(73) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#40] - -(74) HashAggregate [codegen id : 3] -Input [1]: [netpaid#40] -Keys: [] -Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#69, count#70] -Results [2]: [sum#71, count#72] - -(75) CometColumnarExchange -Input [2]: [sum#71, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#71, count#72] - -(77) HashAggregate [codegen id : 4] -Input [2]: [sum#71, count#72] -Keys: [] -Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#73] -Results [1]: [(0.05 * avg(netpaid#40)#73) AS (0.05 * avg(netpaid))#74] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/simplified.txt deleted file mode 100644 index af8d5ee7aa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/simplified.txt +++ /dev/null @@ -1,94 +0,0 @@ -WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #6 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometNativeScan 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.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/explain.txt deleted file mode 100644 index 548959be1b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,439 +0,0 @@ -== Physical Plan == -* Filter (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) = chiffon ) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#19, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#21, 10, true, false, true) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Condition : (isnotnull(c_customer_sk#26) AND isnotnull(c_birth_country#29)) - -(27) CometProject -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29], [c_customer_sk#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#31, c_birth_country#29] - -(28) CometBroadcastExchange -Input [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(31) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_state#32, ca_zip#33, ca_country#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true))) - -(34) CometProject -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] - -(35) CometColumnarToRow [codegen id : 1] -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] - -(36) BroadcastExchange -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#29, s_zip#16] -Right keys [2]: [upper(ca_country#34), ca_zip#36] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#37] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(40) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [partial_sum(netpaid#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(44) CometColumnarExchange -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [sum(netpaid#40)] -Aggregate Attributes [1]: [sum(netpaid#40)#45] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] -Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (77) -+- * CometColumnarToRow (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (56) - : : : +- CometBroadcastHashJoin (55) - : : : :- CometProject (53) - : : : : +- CometSortMergeJoin (52) - : : : : :- CometSort (49) - : : : : : +- ReusedExchange (48) - : : : : +- CometSort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (54) - : : +- CometBroadcastExchange (60) - : : +- CometProject (59) - : : +- CometFilter (58) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (57) - : +- ReusedExchange (63) - +- ReusedExchange (67) - - -(48) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] - -(49) CometSort -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53], [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST] - -(50) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#54, sr_ticket_number#55] - -(51) CometSort -Input [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [sr_item_sk#54, sr_ticket_number#55], [sr_ticket_number#55 ASC NULLS FIRST, sr_item_sk#54 ASC NULLS FIRST] - -(52) CometSortMergeJoin -Left output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Right output [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_ticket_number#52, ss_item_sk#49], [sr_ticket_number#55, sr_item_sk#54], Inner - -(53) CometProject -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53], [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] - -(54) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] - -(55) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Right output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_store_sk#51], [s_store_sk#56], Inner, BuildRight - -(56) CometProject -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16], [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Condition : isnotnull(i_item_sk#58) - -(59) CometProject -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [i_item_sk#58, i_current_price#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#60, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#61, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#62, 10, true, false, true) AS i_units#25, i_manager_id#63] - -(60) CometBroadcastExchange -Input [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(61) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_item_sk#49], [i_item_sk#58], Inner, BuildRight - -(62) CometProject -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(63) ReusedExchange [Reuses operator id: 28] -Output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] - -(64) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Right output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight - -(65) CometProject -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(66) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(67) ReusedExchange [Reuses operator id: 36] -Output [3]: [ca_state#35, ca_zip#36, ca_country#66] - -(68) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#65, s_zip#16] -Right keys [2]: [upper(ca_country#66), ca_zip#36] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 2] -Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] - -(70) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#67] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(71) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(72) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(73) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#40] - -(74) HashAggregate [codegen id : 3] -Input [1]: [netpaid#40] -Keys: [] -Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#69, count#70] -Results [2]: [sum#71, count#72] - -(75) CometColumnarExchange -Input [2]: [sum#71, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#71, count#72] - -(77) HashAggregate [codegen id : 4] -Input [2]: [sum#71, count#72] -Keys: [] -Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#73] -Results [1]: [(0.05 * avg(netpaid#40)#73) AS (0.05 * avg(netpaid))#74] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/simplified.txt deleted file mode 100644 index 440f477d0f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,94 +0,0 @@ -WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #6 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] 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/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt deleted file mode 100644 index 548959be1b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt +++ /dev/null @@ -1,439 +0,0 @@ -== Physical Plan == -* Filter (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) = chiffon ) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#19, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#21, 10, true, false, true) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Condition : (isnotnull(c_customer_sk#26) AND isnotnull(c_birth_country#29)) - -(27) CometProject -Input [4]: [c_customer_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29], [c_customer_sk#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#31, c_birth_country#29] - -(28) CometBroadcastExchange -Input [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [4]: [c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_first_name#30, c_last_name#31, c_birth_country#29] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(31) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_state#32, ca_zip#33, ca_country#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Condition : (isnotnull(ca_country#34) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true))) - -(34) CometProject -Input [3]: [ca_state#32, ca_zip#33, ca_country#34] -Arguments: [ca_state#35, ca_zip#36, ca_country#34], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#32, 2, true, false, true) AS ca_state#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#33, 10, true, false, true) AS ca_zip#36, ca_country#34] - -(35) CometColumnarToRow [codegen id : 1] -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] - -(36) BroadcastExchange -Input [3]: [ca_state#35, ca_zip#36, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, true]), input[1, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#29, s_zip#16] -Right keys [2]: [upper(ca_country#34), ca_zip#36] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, c_birth_country#29, ca_state#35, ca_zip#36, ca_country#34] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#37] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(40) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#38] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#11, ca_state#35, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#39] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#39,17,2) AS netpaid#40] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, netpaid#40] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [partial_sum(netpaid#40)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(44) CometColumnarExchange -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#31, c_first_name#30, s_store_name#11, sum#43, isEmpty#44] -Keys [3]: [c_last_name#31, c_first_name#30, s_store_name#11] -Functions [1]: [sum(netpaid#40)] -Aggregate Attributes [1]: [sum(netpaid#40)#45] -Results [4]: [c_last_name#31, c_first_name#30, s_store_name#11, sum(netpaid#40)#45 AS paid#46] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#31, c_first_name#30, s_store_name#11, paid#46] -Condition : (isnotnull(paid#46) AND (cast(paid#46 as decimal(33,8)) > cast(Subquery scalar-subquery#47, [id=#48] as decimal(33,8)))) - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#47, [id=#48] -* HashAggregate (77) -+- * CometColumnarToRow (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Project (69) - +- * BroadcastHashJoin Inner BuildRight (68) - :- * CometColumnarToRow (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (56) - : : : +- CometBroadcastHashJoin (55) - : : : :- CometProject (53) - : : : : +- CometSortMergeJoin (52) - : : : : :- CometSort (49) - : : : : : +- ReusedExchange (48) - : : : : +- CometSort (51) - : : : : +- ReusedExchange (50) - : : : +- ReusedExchange (54) - : : +- CometBroadcastExchange (60) - : : +- CometProject (59) - : : +- CometFilter (58) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (57) - : +- ReusedExchange (63) - +- ReusedExchange (67) - - -(48) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] - -(49) CometSort -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53], [ss_ticket_number#52 ASC NULLS FIRST, ss_item_sk#49 ASC NULLS FIRST] - -(50) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#54, sr_ticket_number#55] - -(51) CometSort -Input [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [sr_item_sk#54, sr_ticket_number#55], [sr_ticket_number#55 ASC NULLS FIRST, sr_item_sk#54 ASC NULLS FIRST] - -(52) CometSortMergeJoin -Left output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Right output [2]: [sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_ticket_number#52, ss_item_sk#49], [sr_ticket_number#55, sr_item_sk#54], Inner - -(53) CometProject -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#54, sr_ticket_number#55] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53], [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] - -(54) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] - -(55) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Right output [4]: [s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_store_sk#51], [s_store_sk#56], Inner, BuildRight - -(56) CometProject -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#56, s_store_name#57, s_state#15, s_zip#16] -Arguments: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16], [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Condition : isnotnull(i_item_sk#58) - -(59) CometProject -Input [6]: [i_item_sk#58, i_current_price#59, i_size#60, i_color#61, i_units#62, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [i_item_sk#58, i_current_price#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#60, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#61, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#62, 10, true, false, true) AS i_units#25, i_manager_id#63] - -(60) CometBroadcastExchange -Input [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(61) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_item_sk#49], [i_item_sk#58], Inner, BuildRight - -(62) CometProject -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_item_sk#58, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Arguments: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63], [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] - -(63) ReusedExchange [Reuses operator id: 28] -Output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] - -(64) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63] -Right output [4]: [c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_customer_sk#50], [c_customer_sk#64], Inner, BuildRight - -(65) CometProject -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_customer_sk#64, c_first_name#30, c_last_name#31, c_birth_country#65] -Arguments: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65], [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(66) CometColumnarToRow [codegen id : 2] -Input [12]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65] - -(67) ReusedExchange [Reuses operator id: 36] -Output [3]: [ca_state#35, ca_zip#36, ca_country#66] - -(68) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [c_birth_country#65, s_zip#16] -Right keys [2]: [upper(ca_country#66), ca_zip#36] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 2] -Output [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Input [15]: [ss_net_paid#53, s_store_name#57, s_state#15, s_zip#16, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, c_birth_country#65, ca_state#35, ca_zip#36, ca_country#66] - -(70) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#53, s_store_name#57, s_state#15, i_current_price#59, i_size#23, i_color#24, i_units#25, i_manager_id#63, c_first_name#30, c_last_name#31, ca_state#35] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#67] -Results [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(71) CometColumnarExchange -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Arguments: hashpartitioning(c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(72) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] - -(73) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23, sum#68] -Keys [10]: [c_last_name#31, c_first_name#30, s_store_name#57, ca_state#35, s_state#15, i_color#24, i_current_price#59, i_manager_id#63, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#39,17,2) AS netpaid#40] - -(74) HashAggregate [codegen id : 3] -Input [1]: [netpaid#40] -Keys: [] -Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#69, count#70] -Results [2]: [sum#71, count#72] - -(75) CometColumnarExchange -Input [2]: [sum#71, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#71, count#72] - -(77) HashAggregate [codegen id : 4] -Input [2]: [sum#71, count#72] -Keys: [] -Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#73] -Results [1]: [(0.05 * avg(netpaid#40)#73) AS (0.05 * avg(netpaid))#74] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/extended.txt deleted file mode 100644 index 8777e81a37..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/extended.txt +++ /dev/null @@ -1,96 +0,0 @@ -Filter -: +- Subquery -: +- HashAggregate -: +- CometColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- HashAggregate -: +- CometColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- Project -: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometColumnarToRow -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometSortMergeJoin -: : : : : :- CometSort -: : : : : : +- CometExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : +- CometSort -: : : : : +- CometExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -: +- BroadcastExchange -: +- CometColumnarToRow -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 70 out of 86 eligible operators (81%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 440f477d0f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/simplified.txt +++ /dev/null @@ -1,94 +0,0 @@ -WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #4 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #11 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - InputAdapter - ReusedExchange [ca_state,ca_zip,ca_country] #8 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #3 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #5 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #6 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_first_name,c_last_name,c_birth_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_state,ca_zip,ca_country] - CometFilter [ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/explain.txt deleted file mode 100644 index 15bdd29f61..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/explain.txt +++ /dev/null @@ -1,312 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (18) - : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet spark_catalog.default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * Filter (12) - : : : : : +- * ColumnarToRow (11) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (10) - : : : : +- ReusedExchange (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.store (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.item (32) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 8] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] - -(3) Filter [codegen id : 8] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(4) Scan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] - -(6) Filter [codegen id : 1] -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(7) BroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 8] -Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] -Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] - -(10) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] - -(12) Filter [codegen id : 2] -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(13) BroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] -Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] - -(16) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#18] - -(17) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#18] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 8] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#18] - -(19) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#19] - -(20) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [sr_returned_date_sk#12] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#19] - -(22) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#20] - -(23) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 8] -Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#20] - -(25) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#21, s_store_id#22, s_store_name#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(26) CometFilter -Input [3]: [s_store_sk#21, s_store_id#22, s_store_name#23] -Condition : isnotnull(s_store_sk#21) - -(27) CometProject -Input [3]: [s_store_sk#21, s_store_id#22, s_store_name#23] -Arguments: [s_store_sk#21, s_store_id#24, s_store_name#23], [s_store_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#22, 16, true, false, true) AS s_store_id#24, s_store_name#23] - -(28) CometColumnarToRow [codegen id : 6] -Input [3]: [s_store_sk#21, s_store_id#24, s_store_name#23] - -(29) BroadcastExchange -Input [3]: [s_store_sk#21, s_store_id#24, s_store_name#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(30) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#21] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_sk#21, s_store_id#24, s_store_name#23] - -(32) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] -Condition : isnotnull(i_item_sk#25) - -(34) CometProject -Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] -Arguments: [i_item_sk#25, i_item_id#28, i_item_desc#27], [i_item_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#26, 16, true, false, true) AS i_item_id#28, i_item_desc#27] - -(35) CometColumnarToRow [codegen id : 7] -Input [3]: [i_item_sk#25, i_item_id#28, i_item_desc#27] - -(36) BroadcastExchange -Input [3]: [i_item_sk#25, i_item_id#28, i_item_desc#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(37) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#25] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 8] -Output [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23, i_item_id#28, i_item_desc#27] -Input [9]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23, i_item_sk#25, i_item_id#28, i_item_desc#27] - -(39) HashAggregate [codegen id : 8] -Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#24, s_store_name#23, i_item_id#28, i_item_desc#27] -Keys [4]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23] -Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#11)), partial_sum(UnscaledValue(cs_net_profit#16))] -Aggregate Attributes [3]: [sum#29, sum#30, sum#31] -Results [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum#32, sum#33, sum#34] - -(40) CometColumnarExchange -Input [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum#32, sum#33, sum#34] -Arguments: hashpartitioning(i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(41) CometColumnarToRow [codegen id : 9] -Input [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum#32, sum#33, sum#34] - -(42) HashAggregate [codegen id : 9] -Input [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, sum#32, sum#33, sum#34] -Keys [4]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23] -Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#35, sum(UnscaledValue(sr_net_loss#11))#36, sum(UnscaledValue(cs_net_profit#16))#37] -Results [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#35,17,2) AS store_sales_profit#38, MakeDecimal(sum(UnscaledValue(sr_net_loss#11))#36,17,2) AS store_returns_loss#39, MakeDecimal(sum(UnscaledValue(cs_net_profit#16))#37,17,2) AS catalog_sales_profit#40] - -(43) TakeOrderedAndProject -Input [7]: [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, store_sales_profit#38, store_returns_loss#39, catalog_sales_profit#40] -Arguments: 100, [i_item_id#28 ASC NULLS FIRST, i_item_desc#27 ASC NULLS FIRST, s_store_id#24 ASC NULLS FIRST, s_store_name#23 ASC NULLS FIRST], [i_item_id#28, i_item_desc#27, s_store_id#24, s_store_name#23, store_sales_profit#38, store_returns_loss#39, catalog_sales_profit#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometNativeScan parquet spark_catalog.default.date_dim (44) - - -(44) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#18, d_year#41, d_moy#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [3]: [d_date_sk#18, d_year#41, d_moy#42] -Condition : ((((isnotnull(d_moy#42) AND isnotnull(d_year#41)) AND (d_moy#42 = 4)) AND (d_year#41 = 2001)) AND isnotnull(d_date_sk#18)) - -(46) CometProject -Input [3]: [d_date_sk#18, d_year#41, d_moy#42] -Arguments: [d_date_sk#18], [d_date_sk#18] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#18] - -(48) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometNativeScan parquet spark_catalog.default.date_dim (49) - - -(49) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#43, d_moy#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#19, d_year#43, d_moy#44] -Condition : (((((isnotnull(d_moy#44) AND isnotnull(d_year#43)) AND (d_moy#44 >= 4)) AND (d_moy#44 <= 10)) AND (d_year#43 = 2001)) AND isnotnull(d_date_sk#19)) - -(51) CometProject -Input [3]: [d_date_sk#19, d_year#43, d_moy#44] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(53) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/simplified.txt deleted file mode 100644 index 288561a740..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/simplified.txt +++ /dev/null @@ -1,79 +0,0 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] - WholeStageCodegen (9) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit)),store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - WholeStageCodegen (8) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] [sum,sum,sum,sum,sum,sum] - Project [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [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] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Filter [cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometNativeScan 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/q25.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/explain.txt deleted file mode 100644 index 6a9f9094d3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,319 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometProject (18) - : : : : +- CometBroadcastHashJoin (17) - : : : : :- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (8) - : : : : +- CometBroadcastExchange (16) - : : : : +- CometProject (15) - : : : : +- CometFilter (14) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : : : +- CometBroadcastExchange (22) - : : : +- CometProject (21) - : : : +- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - : : +- ReusedExchange (25) - : +- CometBroadcastExchange (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (28) - +- CometBroadcastExchange (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (34) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(5) CometBroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] - -(6) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(10) CometBroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] - -(11) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] -Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight - -(12) CometProject -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(15) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 4)) AND (d_moy#24 <= 10)) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(21) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#22] -Arguments: [sr_returned_date_sk#12], [d_date_sk#22], Inner, BuildRight - -(24) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#22] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] - -(25) ReusedExchange [Reuses operator id: 22] -Output [1]: [d_date_sk#25] - -(26) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#25] -Arguments: [cs_sold_date_sk#17], [d_date_sk#25], Inner, BuildRight - -(27) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#25] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(29) CometFilter -Input [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] -Condition : isnotnull(s_store_sk#26) - -(30) CometProject -Input [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] -Arguments: [s_store_sk#26, s_store_id#29, s_store_name#28], [s_store_sk#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#27, 16, true, false, true) AS s_store_id#29, s_store_name#28] - -(31) CometBroadcastExchange -Input [3]: [s_store_sk#26, s_store_id#29, s_store_name#28] -Arguments: [s_store_sk#26, s_store_id#29, s_store_name#28] - -(32) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] -Right output [3]: [s_store_sk#26, s_store_id#29, s_store_name#28] -Arguments: [ss_store_sk#3], [s_store_sk#26], Inner, BuildRight - -(33) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_sk#26, s_store_id#29, s_store_name#28] -Arguments: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28], [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28] - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] -Condition : isnotnull(i_item_sk#30) - -(36) CometProject -Input [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] -Arguments: [i_item_sk#30, i_item_id#33, i_item_desc#32], [i_item_sk#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#31, 16, true, false, true) AS i_item_id#33, i_item_desc#32] - -(37) CometBroadcastExchange -Input [3]: [i_item_sk#30, i_item_id#33, i_item_desc#32] -Arguments: [i_item_sk#30, i_item_id#33, i_item_desc#32] - -(38) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28] -Right output [3]: [i_item_sk#30, i_item_id#33, i_item_desc#32] -Arguments: [ss_item_sk#1], [i_item_sk#30], Inner, BuildRight - -(39) CometProject -Input [9]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_sk#30, i_item_id#33, i_item_desc#32] -Arguments: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_id#33, i_item_desc#32], [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_id#33, i_item_desc#32] - -(40) CometHashAggregate -Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_id#33, i_item_desc#32] -Keys [4]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28] -Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#11)), partial_sum(UnscaledValue(cs_net_profit#16))] - -(41) CometExchange -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, sum#34, sum#35, sum#36] -Arguments: hashpartitioning(i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(42) CometHashAggregate -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, sum#34, sum#35, sum#36] -Keys [4]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28] -Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] - -(43) CometTakeOrderedAndProject -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#33 ASC NULLS FIRST,i_item_desc#32 ASC NULLS FIRST,s_store_id#29 ASC NULLS FIRST,s_store_name#28 ASC NULLS FIRST], output=[i_item_id#33,i_item_desc#32,s_store_id#29,s_store_name#28,store_sales_profit#37,store_returns_loss#38,catalog_sales_profit#39]), [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39], 100, 0, [i_item_id#33 ASC NULLS FIRST, i_item_desc#32 ASC NULLS FIRST, s_store_id#29 ASC NULLS FIRST, s_store_name#28 ASC NULLS FIRST], [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] - -(44) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(47) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(49) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) - - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 4)) AND (d_moy#24 <= 10)) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(52) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(53) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(54) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/simplified.txt deleted file mode 100644 index dcad304452..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,63 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] - CometHashAggregate [sum,sum,sum] [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit))] - CometExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - CometHashAggregate [ss_net_profit,sr_net_loss,cs_net_profit] [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] - 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,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_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 [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 [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_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 [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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] 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 [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #8 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan [native_iceberg_compat] 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/q25/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt deleted file mode 100644 index 6a9f9094d3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt +++ /dev/null @@ -1,319 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometProject (18) - : : : : +- CometBroadcastHashJoin (17) - : : : : :- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (8) - : : : : +- CometBroadcastExchange (16) - : : : : +- CometProject (15) - : : : : +- CometFilter (14) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : : : +- CometBroadcastExchange (22) - : : : +- CometProject (21) - : : : +- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - : : +- ReusedExchange (25) - : +- CometBroadcastExchange (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (28) - +- CometBroadcastExchange (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (34) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(5) CometBroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] - -(6) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] -Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(10) CometBroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] - -(11) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12] -Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight - -(12) CometProject -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_net_loss#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(15) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 4)) AND (d_moy#24 <= 10)) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(21) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#22] -Arguments: [sr_returned_date_sk#12], [d_date_sk#22], Inner, BuildRight - -(24) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#22] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] - -(25) ReusedExchange [Reuses operator id: 22] -Output [1]: [d_date_sk#25] - -(26) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#25] -Arguments: [cs_sold_date_sk#17], [d_date_sk#25], Inner, BuildRight - -(27) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#25] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16], [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(29) CometFilter -Input [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] -Condition : isnotnull(s_store_sk#26) - -(30) CometProject -Input [3]: [s_store_sk#26, s_store_id#27, s_store_name#28] -Arguments: [s_store_sk#26, s_store_id#29, s_store_name#28], [s_store_sk#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#27, 16, true, false, true) AS s_store_id#29, s_store_name#28] - -(31) CometBroadcastExchange -Input [3]: [s_store_sk#26, s_store_id#29, s_store_name#28] -Arguments: [s_store_sk#26, s_store_id#29, s_store_name#28] - -(32) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] -Right output [3]: [s_store_sk#26, s_store_id#29, s_store_name#28] -Arguments: [ss_store_sk#3], [s_store_sk#26], Inner, BuildRight - -(33) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_sk#26, s_store_id#29, s_store_name#28] -Arguments: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28], [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28] - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] -Condition : isnotnull(i_item_sk#30) - -(36) CometProject -Input [3]: [i_item_sk#30, i_item_id#31, i_item_desc#32] -Arguments: [i_item_sk#30, i_item_id#33, i_item_desc#32], [i_item_sk#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#31, 16, true, false, true) AS i_item_id#33, i_item_desc#32] - -(37) CometBroadcastExchange -Input [3]: [i_item_sk#30, i_item_id#33, i_item_desc#32] -Arguments: [i_item_sk#30, i_item_id#33, i_item_desc#32] - -(38) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28] -Right output [3]: [i_item_sk#30, i_item_id#33, i_item_desc#32] -Arguments: [ss_item_sk#1], [i_item_sk#30], Inner, BuildRight - -(39) CometProject -Input [9]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_sk#30, i_item_id#33, i_item_desc#32] -Arguments: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_id#33, i_item_desc#32], [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_id#33, i_item_desc#32] - -(40) CometHashAggregate -Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#29, s_store_name#28, i_item_id#33, i_item_desc#32] -Keys [4]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28] -Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#11)), partial_sum(UnscaledValue(cs_net_profit#16))] - -(41) CometExchange -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, sum#34, sum#35, sum#36] -Arguments: hashpartitioning(i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(42) CometHashAggregate -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, sum#34, sum#35, sum#36] -Keys [4]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28] -Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] - -(43) CometTakeOrderedAndProject -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#33 ASC NULLS FIRST,i_item_desc#32 ASC NULLS FIRST,s_store_id#29 ASC NULLS FIRST,s_store_name#28 ASC NULLS FIRST], output=[i_item_id#33,i_item_desc#32,s_store_id#29,s_store_name#28,store_sales_profit#37,store_returns_loss#38,catalog_sales_profit#39]), [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39], 100, 0, [i_item_id#33 ASC NULLS FIRST, i_item_desc#32 ASC NULLS FIRST, s_store_id#29 ASC NULLS FIRST, s_store_name#28 ASC NULLS FIRST], [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] - -(44) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 4)) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(47) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(49) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (50) - - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 4)) AND (d_moy#24 <= 10)) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(52) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(53) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(54) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/extended.txt deleted file mode 100644 index 4e1a4f3ec0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/extended.txt +++ /dev/null @@ -1,62 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index dcad304452..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/simplified.txt +++ /dev/null @@ -1,63 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] - CometHashAggregate [sum,sum,sum] [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit))] - CometExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - CometHashAggregate [ss_net_profit,sr_net_loss,cs_net_profit] [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] - 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,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_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 [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 [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_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 [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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] 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 [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #8 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #9 - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/explain.txt deleted file mode 100644 index fb4c487ea9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- * CometColumnarToRow (30) - +- CometColumnarExchange (29) - +- * HashAggregate (28) - +- * Project (27) - +- * BroadcastHashJoin Inner BuildRight (26) - :- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.item (14) - +- BroadcastExchange (25) - +- * CometColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometNativeScan parquet spark_catalog.default.promotion (21) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] - -(3) Filter [codegen id : 5] -Input [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Condition : ((isnotnull(cs_bill_cdemo_sk#1) AND isnotnull(cs_item_sk#2)) AND isnotnull(cs_promo_sk#3)) - -(4) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = S)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#10)) - -(6) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [cd_demo_sk#10] - -(8) BroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_bill_cdemo_sk#1] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 5] -Output [7]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Input [9]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, cd_demo_sk#10] - -(11) ReusedExchange [Reuses operator id: 37] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [6]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] -Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, d_date_sk#14] - -(14) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#15, i_item_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [i_item_sk#15, i_item_id#16] -Condition : isnotnull(i_item_sk#15) - -(16) CometProject -Input [2]: [i_item_sk#15, i_item_id#16] -Arguments: [i_item_sk#15, i_item_id#17], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#16, 16, true, false, true) AS i_item_id#17] - -(17) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#15, i_item_id#17] - -(18) BroadcastExchange -Input [2]: [i_item_sk#15, i_item_id#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [6]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] -Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_sk#15, i_item_id#17] - -(21) CometNativeScan parquet spark_catalog.default.promotion -Output [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_email#19, 1, true, false, true) = N) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_event#20, 1, true, false, true) = N)) AND isnotnull(p_promo_sk#18)) - -(23) CometProject -Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] -Arguments: [p_promo_sk#18], [p_promo_sk#18] - -(24) CometColumnarToRow [codegen id : 4] -Input [1]: [p_promo_sk#18] - -(25) BroadcastExchange -Input [1]: [p_promo_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_promo_sk#3] -Right keys [1]: [p_promo_sk#18] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 5] -Output [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] -Input [7]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17, p_promo_sk#18] - -(28) HashAggregate [codegen id : 5] -Input [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#17] -Keys [1]: [i_item_id#17] -Functions [4]: [partial_avg(cs_quantity#4), partial_avg(UnscaledValue(cs_list_price#5)), partial_avg(UnscaledValue(cs_coupon_amt#7)), partial_avg(UnscaledValue(cs_sales_price#6))] -Aggregate Attributes [8]: [sum#21, count#22, sum#23, count#24, sum#25, count#26, sum#27, count#28] -Results [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] - -(29) CometColumnarExchange -Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] -Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 6] -Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] - -(31) HashAggregate [codegen id : 6] -Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] -Keys [1]: [i_item_id#17] -Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(UnscaledValue(cs_coupon_amt#7)), avg(UnscaledValue(cs_sales_price#6))] -Aggregate Attributes [4]: [avg(cs_quantity#4)#37, avg(UnscaledValue(cs_list_price#5))#38, avg(UnscaledValue(cs_coupon_amt#7))#39, avg(UnscaledValue(cs_sales_price#6))#40] -Results [5]: [i_item_id#17, avg(cs_quantity#4)#37 AS agg1#41, cast((avg(UnscaledValue(cs_list_price#5))#38 / 100.0) as decimal(11,6)) AS agg2#42, cast((avg(UnscaledValue(cs_coupon_amt#7))#39 / 100.0) as decimal(11,6)) AS agg3#43, cast((avg(UnscaledValue(cs_sales_price#6))#40 / 100.0) as decimal(11,6)) AS agg4#44] - -(32) TakeOrderedAndProject -Input [5]: [i_item_id#17, agg1#41, agg2#42, agg3#43, agg4#44] -Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#41, agg2#42, agg3#43, agg4#44] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometNativeScan parquet spark_catalog.default.date_dim (33) - - -(33) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [d_date_sk#14, d_year#45] -Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2000)) AND isnotnull(d_date_sk#14)) - -(35) CometProject -Input [2]: [d_date_sk#14, d_year#45] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(37) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/simplified.txt deleted file mode 100644 index 7eb9c67611..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) - HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometNativeScan 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/q26.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/explain.txt deleted file mode 100644 index 3d70460689..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,212 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (31) -+- CometTakeOrderedAndProject (30) - +- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Condition : ((isnotnull(cs_bill_cdemo_sk#1) AND isnotnull(cs_item_sk#2)) AND isnotnull(cs_promo_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = S)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [cs_bill_cdemo_sk#1], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, cd_demo_sk#10] -Arguments: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8], [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [cs_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, d_date_sk#14] -Arguments: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7], [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#16, i_item_id#17] -Condition : isnotnull(i_item_sk#16) - -(17) CometProject -Input [2]: [i_item_sk#16, i_item_id#17] -Arguments: [i_item_sk#16, i_item_id#18], [i_item_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#17, 16, true, false, true) AS i_item_id#18] - -(18) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_id#18] -Arguments: [i_item_sk#16, i_item_id#18] - -(19) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] -Right output [2]: [i_item_sk#16, i_item_id#18] -Arguments: [cs_item_sk#2], [i_item_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_sk#16, i_item_id#18] -Arguments: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18], [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_email#20, 1, true, false, true) = N) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_event#21, 1, true, false, true) = N)) AND isnotnull(p_promo_sk#19)) - -(23) CometProject -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Arguments: [p_promo_sk#19], [p_promo_sk#19] - -(24) CometBroadcastExchange -Input [1]: [p_promo_sk#19] -Arguments: [p_promo_sk#19] - -(25) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] -Right output [1]: [p_promo_sk#19] -Arguments: [cs_promo_sk#3], [p_promo_sk#19], Inner, BuildRight - -(26) CometProject -Input [7]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18, p_promo_sk#19] -Arguments: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] - -(27) CometHashAggregate -Input [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [4]: [partial_avg(cs_quantity#4), partial_avg(UnscaledValue(cs_list_price#5)), partial_avg(UnscaledValue(cs_coupon_amt#7)), partial_avg(UnscaledValue(cs_sales_price#6))] - -(28) CometExchange -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Keys [1]: [i_item_id#18] -Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(UnscaledValue(cs_coupon_amt#7)), avg(UnscaledValue(cs_sales_price#6))] - -(30) CometTakeOrderedAndProject -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#18 ASC NULLS FIRST], output=[i_item_id#18,agg1#30,agg2#31,agg3#32,agg4#33]), [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33], 100, 0, [i_item_id#18 ASC NULLS FIRST], [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(34) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(36) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/simplified.txt deleted file mode 100644 index 0b56a47547..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,agg1,agg2,agg3,agg4,avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price))] - CometExchange [i_item_id] #1 - CometHashAggregate [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [i_item_id,sum,count,sum,count,sum,count,sum,count] - CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - 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,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_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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [p_promo_sk] #6 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometScan [native_iceberg_compat] 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/q26/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt deleted file mode 100644 index 3d70460689..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt +++ /dev/null @@ -1,212 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (31) -+- CometTakeOrderedAndProject (30) - +- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Condition : ((isnotnull(cs_bill_cdemo_sk#1) AND isnotnull(cs_item_sk#2)) AND isnotnull(cs_promo_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = S)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [cs_bill_cdemo_sk#1], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, cd_demo_sk#10] -Arguments: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8], [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [cs_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8, d_date_sk#14] -Arguments: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7], [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#16, i_item_id#17] -Condition : isnotnull(i_item_sk#16) - -(17) CometProject -Input [2]: [i_item_sk#16, i_item_id#17] -Arguments: [i_item_sk#16, i_item_id#18], [i_item_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#17, 16, true, false, true) AS i_item_id#18] - -(18) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_id#18] -Arguments: [i_item_sk#16, i_item_id#18] - -(19) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] -Right output [2]: [i_item_sk#16, i_item_id#18] -Arguments: [cs_item_sk#2], [i_item_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_sk#16, i_item_id#18] -Arguments: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18], [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_email#20, 1, true, false, true) = N) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_event#21, 1, true, false, true) = N)) AND isnotnull(p_promo_sk#19)) - -(23) CometProject -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Arguments: [p_promo_sk#19], [p_promo_sk#19] - -(24) CometBroadcastExchange -Input [1]: [p_promo_sk#19] -Arguments: [p_promo_sk#19] - -(25) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] -Right output [1]: [p_promo_sk#19] -Arguments: [cs_promo_sk#3], [p_promo_sk#19], Inner, BuildRight - -(26) CometProject -Input [7]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18, p_promo_sk#19] -Arguments: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] - -(27) CometHashAggregate -Input [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [4]: [partial_avg(cs_quantity#4), partial_avg(UnscaledValue(cs_list_price#5)), partial_avg(UnscaledValue(cs_coupon_amt#7)), partial_avg(UnscaledValue(cs_sales_price#6))] - -(28) CometExchange -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Keys [1]: [i_item_id#18] -Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(UnscaledValue(cs_coupon_amt#7)), avg(UnscaledValue(cs_sales_price#6))] - -(30) CometTakeOrderedAndProject -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#18 ASC NULLS FIRST], output=[i_item_id#18,agg1#30,agg2#31,agg3#32,agg4#33]), [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33], 100, 0, [i_item_id#18 ASC NULLS FIRST], [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(34) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(36) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/extended.txt deleted file mode 100644 index dbea7afe34..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/extended.txt +++ /dev/null @@ -1,39 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 0b56a47547..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,agg1,agg2,agg3,agg4,avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price))] - CometExchange [i_item_id] #1 - CometHashAggregate [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [i_item_id,sum,count,sum,count,sum,count,sum,count] - CometProject [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] - 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,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_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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [p_promo_sk] #6 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/explain.txt deleted file mode 100644 index b9e3e82684..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (33) -+- * HashAggregate (32) - +- * CometColumnarToRow (31) - +- CometColumnarExchange (30) - +- * HashAggregate (29) - +- * Expand (28) - +- * Project (27) - +- * BroadcastHashJoin Inner BuildRight (26) - :- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.store (14) - +- BroadcastExchange (25) - +- * CometColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometNativeScan parquet spark_catalog.default.item (21) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(3) Filter [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) - -(4) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = S)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#10)) - -(6) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [cd_demo_sk#10] - -(8) BroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 5] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] - -(11) ReusedExchange [Reuses operator id: 38] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#15, s_state#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#15, s_state#16] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#16, 2, true, false, true) = TN) AND isnotnull(s_store_sk#15)) - -(16) CometProject -Input [2]: [s_store_sk#15, s_state#16] -Arguments: [s_store_sk#15, s_state#17], [s_store_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#16, 2, true, false, true) AS s_state#17] - -(17) CometColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#15, s_state#17] - -(18) BroadcastExchange -Input [2]: [s_store_sk#15, s_state#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15, s_state#17] - -(21) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#18, i_item_id#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_item_sk#18, i_item_id#19] -Condition : isnotnull(i_item_sk#18) - -(23) CometProject -Input [2]: [i_item_sk#18, i_item_id#19] -Arguments: [i_item_sk#18, i_item_id#20], [i_item_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#19, 16, true, false, true) AS i_item_id#20] - -(24) CometColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#18, i_item_id#20] - -(25) BroadcastExchange -Input [2]: [i_item_sk#18, i_item_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#18] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 5] -Output [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#17] -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17, i_item_sk#18, i_item_id#20] - -(28) Expand [codegen id : 5] -Input [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#17] -Arguments: [[ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, s_state#17, 0], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#20, null, 1], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, null, null, 3]], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#22, spark_grouping_id#23] - -(29) HashAggregate [codegen id : 5] -Input [7]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#22, spark_grouping_id#23] -Keys [3]: [i_item_id#21, s_state#22, spark_grouping_id#23] -Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [8]: [sum#24, count#25, sum#26, count#27, sum#28, count#29, sum#30, count#31] -Results [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] - -(30) CometColumnarExchange -Input [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] -Arguments: hashpartitioning(i_item_id#21, s_state#22, spark_grouping_id#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(31) CometColumnarToRow [codegen id : 6] -Input [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] - -(32) HashAggregate [codegen id : 6] -Input [11]: [i_item_id#21, s_state#22, spark_grouping_id#23, sum#32, count#33, sum#34, count#35, sum#36, count#37, sum#38, count#39] -Keys [3]: [i_item_id#21, s_state#22, spark_grouping_id#23] -Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [4]: [avg(ss_quantity#4)#40, avg(UnscaledValue(ss_list_price#5))#41, avg(UnscaledValue(ss_coupon_amt#7))#42, avg(UnscaledValue(ss_sales_price#6))#43] -Results [7]: [i_item_id#21, s_state#22, cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint) AS g_state#44, avg(ss_quantity#4)#40 AS agg1#45, cast((avg(UnscaledValue(ss_list_price#5))#41 / 100.0) as decimal(11,6)) AS agg2#46, cast((avg(UnscaledValue(ss_coupon_amt#7))#42 / 100.0) as decimal(11,6)) AS agg3#47, cast((avg(UnscaledValue(ss_sales_price#6))#43 / 100.0) as decimal(11,6)) AS agg4#48] - -(33) TakeOrderedAndProject -Input [7]: [i_item_id#21, s_state#22, g_state#44, agg1#45, agg2#46, agg3#47, agg4#48] -Arguments: 100, [i_item_id#21 ASC NULLS FIRST, s_state#22 ASC NULLS FIRST], [i_item_id#21, s_state#22, g_state#44, agg1#45, agg2#46, agg3#47, agg4#48] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometNativeScan parquet spark_catalog.default.date_dim (34) - - -(34) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [d_date_sk#14, d_year#49] -Condition : ((isnotnull(d_year#49) AND (d_year#49 = 2002)) AND isnotnull(d_date_sk#14)) - -(36) CometProject -Input [2]: [d_date_sk#14, d_year#49] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(37) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(38) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/simplified.txt deleted file mode 100644 index dcb4953707..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/simplified.txt +++ /dev/null @@ -1,55 +0,0 @@ -TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) - HashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,s_state,spark_grouping_id] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Expand [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Filter [ss_cdemo_sk,ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan 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/q27.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/explain.txt deleted file mode 100644 index eb158b2889..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (32) -+- CometTakeOrderedAndProject (31) - +- CometHashAggregate (30) - +- CometExchange (29) - +- CometHashAggregate (28) - +- CometExpand (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = S)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#16, s_state#17] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) = TN) AND isnotnull(s_store_sk#16)) - -(17) CometProject -Input [2]: [s_store_sk#16, s_state#17] -Arguments: [s_store_sk#16, s_state#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) AS s_state#18] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_state#18] -Arguments: [s_store_sk#16, s_state#18] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Right output [2]: [s_store_sk#16, s_state#18] -Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16, s_state#18] -Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_item_id#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_item_sk#19, i_item_id#20] -Condition : isnotnull(i_item_sk#19) - -(23) CometProject -Input [2]: [i_item_sk#19, i_item_id#20] -Arguments: [i_item_sk#19, i_item_id#21], [i_item_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#20, 16, true, false, true) AS i_item_id#21] - -(24) CometBroadcastExchange -Input [2]: [i_item_sk#19, i_item_id#21] -Arguments: [i_item_sk#19, i_item_id#21] - -(25) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] -Right output [2]: [i_item_sk#19, i_item_id#21] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(26) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18, i_item_sk#19, i_item_id#21] -Arguments: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18] - -(27) CometExpand -Input [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18] -Arguments: [[ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18, 0], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, null, 1], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, null, null, 3]], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#22, s_state#23, spark_grouping_id#24] - -(28) CometHashAggregate -Input [7]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#22, s_state#23, spark_grouping_id#24] -Keys [3]: [i_item_id#22, s_state#23, spark_grouping_id#24] -Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] - -(29) CometExchange -Input [11]: [i_item_id#22, s_state#23, spark_grouping_id#24, sum#25, count#26, sum#27, count#28, sum#29, count#30, sum#31, count#32] -Arguments: hashpartitioning(i_item_id#22, s_state#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(30) CometHashAggregate -Input [11]: [i_item_id#22, s_state#23, spark_grouping_id#24, sum#25, count#26, sum#27, count#28, sum#29, count#30, sum#31, count#32] -Keys [3]: [i_item_id#22, s_state#23, spark_grouping_id#24] -Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] - -(31) CometTakeOrderedAndProject -Input [7]: [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#22 ASC NULLS FIRST,s_state#23 ASC NULLS FIRST], output=[i_item_id#22,s_state#23,g_state#33,agg1#34,agg2#35,agg3#36,agg4#37]), [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37], 100, 0, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37] - -(32) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (33) - - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_sk#14)) - -(35) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(37) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/simplified.txt deleted file mode 100644 index bdae0cc477..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,42 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,spark_grouping_id,avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price))] - CometExchange [i_item_id,s_state,spark_grouping_id] #1 - CometHashAggregate [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] - 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,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_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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_state] #5 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] 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/q27/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt deleted file mode 100644 index eb158b2889..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (32) -+- CometTakeOrderedAndProject (31) - +- CometHashAggregate (30) - +- CometExchange (29) - +- CometHashAggregate (28) - +- CometExpand (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = S)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#16, s_state#17] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) = TN) AND isnotnull(s_store_sk#16)) - -(17) CometProject -Input [2]: [s_store_sk#16, s_state#17] -Arguments: [s_store_sk#16, s_state#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) AS s_state#18] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_state#18] -Arguments: [s_store_sk#16, s_state#18] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Right output [2]: [s_store_sk#16, s_state#18] -Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16, s_state#18] -Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_item_id#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_item_sk#19, i_item_id#20] -Condition : isnotnull(i_item_sk#19) - -(23) CometProject -Input [2]: [i_item_sk#19, i_item_id#20] -Arguments: [i_item_sk#19, i_item_id#21], [i_item_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#20, 16, true, false, true) AS i_item_id#21] - -(24) CometBroadcastExchange -Input [2]: [i_item_sk#19, i_item_id#21] -Arguments: [i_item_sk#19, i_item_id#21] - -(25) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] -Right output [2]: [i_item_sk#19, i_item_id#21] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(26) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18, i_item_sk#19, i_item_id#21] -Arguments: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18] - -(27) CometExpand -Input [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18] -Arguments: [[ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, s_state#18, 0], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#21, null, 1], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, null, null, 3]], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#22, s_state#23, spark_grouping_id#24] - -(28) CometHashAggregate -Input [7]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#22, s_state#23, spark_grouping_id#24] -Keys [3]: [i_item_id#22, s_state#23, spark_grouping_id#24] -Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] - -(29) CometExchange -Input [11]: [i_item_id#22, s_state#23, spark_grouping_id#24, sum#25, count#26, sum#27, count#28, sum#29, count#30, sum#31, count#32] -Arguments: hashpartitioning(i_item_id#22, s_state#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(30) CometHashAggregate -Input [11]: [i_item_id#22, s_state#23, spark_grouping_id#24, sum#25, count#26, sum#27, count#28, sum#29, count#30, sum#31, count#32] -Keys [3]: [i_item_id#22, s_state#23, spark_grouping_id#24] -Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] - -(31) CometTakeOrderedAndProject -Input [7]: [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#22 ASC NULLS FIRST,s_state#23 ASC NULLS FIRST], output=[i_item_id#22,s_state#23,g_state#33,agg1#34,agg2#35,agg3#36,agg4#37]), [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37], 100, 0, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37] - -(32) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#22, s_state#23, g_state#33, agg1#34, agg2#35, agg3#36, agg4#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (33) - - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2002)) AND isnotnull(d_date_sk#14)) - -(35) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(37) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/extended.txt deleted file mode 100644 index f550f3855c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/extended.txt +++ /dev/null @@ -1,40 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index bdae0cc477..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/simplified.txt +++ /dev/null @@ -1,42 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,spark_grouping_id,avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price))] - CometExchange [i_item_id,s_state,spark_grouping_id] #1 - CometHashAggregate [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] - 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,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_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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_state] #5 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_datafusion/explain.txt deleted file mode 100644 index 985274408b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_datafusion/explain.txt +++ /dev/null @@ -1,437 +0,0 @@ -== Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (76) -:- * BroadcastNestedLoopJoin Inner BuildRight (63) -: :- * BroadcastNestedLoopJoin Inner BuildRight (50) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (37) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (24) -: : : : :- * CometColumnarToRow (11) -: : : : : +- CometHashAggregate (10) -: : : : : +- CometColumnarExchange (9) -: : : : : +- * HashAggregate (8) -: : : : : +- * HashAggregate (7) -: : : : : +- * CometColumnarToRow (6) -: : : : : +- CometExchange (5) -: : : : : +- CometHashAggregate (4) -: : : : : +- CometProject (3) -: : : : : +- CometFilter (2) -: : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) -: : : : +- BroadcastExchange (23) -: : : : +- * CometColumnarToRow (22) -: : : : +- CometHashAggregate (21) -: : : : +- CometColumnarExchange (20) -: : : : +- * HashAggregate (19) -: : : : +- * HashAggregate (18) -: : : : +- * CometColumnarToRow (17) -: : : : +- CometExchange (16) -: : : : +- CometHashAggregate (15) -: : : : +- CometProject (14) -: : : : +- CometFilter (13) -: : : : +- CometNativeScan parquet spark_catalog.default.store_sales (12) -: : : +- BroadcastExchange (36) -: : : +- * CometColumnarToRow (35) -: : : +- CometHashAggregate (34) -: : : +- CometColumnarExchange (33) -: : : +- * HashAggregate (32) -: : : +- * HashAggregate (31) -: : : +- * CometColumnarToRow (30) -: : : +- CometExchange (29) -: : : +- CometHashAggregate (28) -: : : +- CometProject (27) -: : : +- CometFilter (26) -: : : +- CometNativeScan parquet spark_catalog.default.store_sales (25) -: : +- BroadcastExchange (49) -: : +- * CometColumnarToRow (48) -: : +- CometHashAggregate (47) -: : +- CometColumnarExchange (46) -: : +- * HashAggregate (45) -: : +- * HashAggregate (44) -: : +- * CometColumnarToRow (43) -: : +- CometExchange (42) -: : +- CometHashAggregate (41) -: : +- CometProject (40) -: : +- CometFilter (39) -: : +- CometNativeScan parquet spark_catalog.default.store_sales (38) -: +- BroadcastExchange (62) -: +- * CometColumnarToRow (61) -: +- CometHashAggregate (60) -: +- CometColumnarExchange (59) -: +- * HashAggregate (58) -: +- * HashAggregate (57) -: +- * CometColumnarToRow (56) -: +- CometExchange (55) -: +- CometHashAggregate (54) -: +- CometProject (53) -: +- CometFilter (52) -: +- CometNativeScan parquet spark_catalog.default.store_sales (51) -+- BroadcastExchange (75) - +- * CometColumnarToRow (74) - +- CometHashAggregate (73) - +- CometColumnarExchange (72) - +- * HashAggregate (71) - +- * HashAggregate (70) - +- * CometColumnarToRow (69) - +- CometExchange (68) - +- CometHashAggregate (67) - +- CometProject (66) - +- CometFilter (65) - +- CometNativeScan parquet spark_catalog.default.store_sales (64) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,0), LessThanOrEqual(ss_quantity,5), Or(Or(And(GreaterThanOrEqual(ss_list_price,8.00),LessThanOrEqual(ss_list_price,18.00)),And(GreaterThanOrEqual(ss_coupon_amt,459.00),LessThanOrEqual(ss_coupon_amt,1459.00))),And(GreaterThanOrEqual(ss_wholesale_cost,57.00),LessThanOrEqual(ss_wholesale_cost,77.00)))] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 0)) AND (ss_quantity#1 <= 5)) AND ((((ss_list_price#3 >= 8.00) AND (ss_list_price#3 <= 18.00)) OR ((ss_coupon_amt#4 >= 459.00) AND (ss_coupon_amt#4 <= 1459.00))) OR ((ss_wholesale_cost#2 >= 57.00) AND (ss_wholesale_cost#2 <= 77.00)))) - -(3) CometProject -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Arguments: [ss_list_price#3], [ss_list_price#3] - -(4) CometHashAggregate -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] - -(5) CometExchange -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(6) CometColumnarToRow [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] - -(7) HashAggregate [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10] -Results [4]: [ss_list_price#3, sum#6, count#7, count#8] - -(8) HashAggregate [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10, count(ss_list_price#3)#11] -Results [4]: [sum#6, count#7, count#8, count#12] - -(9) CometColumnarExchange -Input [4]: [sum#6, count#7, count#8, count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(10) CometHashAggregate -Input [4]: [sum#6, count#7, count#8, count#12] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] - -(11) CometColumnarToRow [codegen id : 12] -Input [3]: [B1_LP#13, B1_CNT#14, B1_CNTD#15] - -(12) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] -ReadSchema: struct - -(13) CometFilter -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Condition : (((isnotnull(ss_quantity#16) AND (ss_quantity#16 >= 6)) AND (ss_quantity#16 <= 10)) AND ((((ss_list_price#18 >= 90.00) AND (ss_list_price#18 <= 100.00)) OR ((ss_coupon_amt#19 >= 2323.00) AND (ss_coupon_amt#19 <= 3323.00))) OR ((ss_wholesale_cost#17 >= 31.00) AND (ss_wholesale_cost#17 <= 51.00)))) - -(14) CometProject -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Arguments: [ss_list_price#18], [ss_list_price#18] - -(15) CometHashAggregate -Input [1]: [ss_list_price#18] -Keys [1]: [ss_list_price#18] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_list_price#18)] - -(16) CometExchange -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(17) CometColumnarToRow [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] - -(18) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Keys [1]: [ss_list_price#18] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25] -Results [4]: [ss_list_price#18, sum#21, count#22, count#23] - -(19) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18), partial_count(distinct ss_list_price#18)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25, count(ss_list_price#18)#26] -Results [4]: [sum#21, count#22, count#23, count#27] - -(20) CometColumnarExchange -Input [4]: [sum#21, count#22, count#23, count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(21) CometHashAggregate -Input [4]: [sum#21, count#22, count#23, count#27] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#18)), count(ss_list_price#18), count(distinct ss_list_price#18)] - -(22) CometColumnarToRow [codegen id : 3] -Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] - -(23) BroadcastExchange -Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] -Arguments: IdentityBroadcastMode, [plan_id=5] - -(24) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(25) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] -ReadSchema: struct - -(26) CometFilter -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Condition : (((isnotnull(ss_quantity#31) AND (ss_quantity#31 >= 11)) AND (ss_quantity#31 <= 15)) AND ((((ss_list_price#33 >= 142.00) AND (ss_list_price#33 <= 152.00)) OR ((ss_coupon_amt#34 >= 12214.00) AND (ss_coupon_amt#34 <= 13214.00))) OR ((ss_wholesale_cost#32 >= 79.00) AND (ss_wholesale_cost#32 <= 99.00)))) - -(27) CometProject -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Arguments: [ss_list_price#33], [ss_list_price#33] - -(28) CometHashAggregate -Input [1]: [ss_list_price#33] -Keys [1]: [ss_list_price#33] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_list_price#33)] - -(29) CometExchange -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(30) CometColumnarToRow [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] - -(31) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Keys [1]: [ss_list_price#33] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40] -Results [4]: [ss_list_price#33, sum#36, count#37, count#38] - -(32) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33), partial_count(distinct ss_list_price#33)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40, count(ss_list_price#33)#41] -Results [4]: [sum#36, count#37, count#38, count#42] - -(33) CometColumnarExchange -Input [4]: [sum#36, count#37, count#38, count#42] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(34) CometHashAggregate -Input [4]: [sum#36, count#37, count#38, count#42] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#33)), count(ss_list_price#33), count(distinct ss_list_price#33)] - -(35) CometColumnarToRow [codegen id : 5] -Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] - -(36) BroadcastExchange -Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] -Arguments: IdentityBroadcastMode, [plan_id=8] - -(37) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(38) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] -ReadSchema: struct - -(39) CometFilter -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Condition : (((isnotnull(ss_quantity#46) AND (ss_quantity#46 >= 16)) AND (ss_quantity#46 <= 20)) AND ((((ss_list_price#48 >= 135.00) AND (ss_list_price#48 <= 145.00)) OR ((ss_coupon_amt#49 >= 6071.00) AND (ss_coupon_amt#49 <= 7071.00))) OR ((ss_wholesale_cost#47 >= 38.00) AND (ss_wholesale_cost#47 <= 58.00)))) - -(40) CometProject -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Arguments: [ss_list_price#48], [ss_list_price#48] - -(41) CometHashAggregate -Input [1]: [ss_list_price#48] -Keys [1]: [ss_list_price#48] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_list_price#48)] - -(42) CometExchange -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(43) CometColumnarToRow [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] - -(44) HashAggregate [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Keys [1]: [ss_list_price#48] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55] -Results [4]: [ss_list_price#48, sum#51, count#52, count#53] - -(45) HashAggregate [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48), partial_count(distinct ss_list_price#48)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55, count(ss_list_price#48)#56] -Results [4]: [sum#51, count#52, count#53, count#57] - -(46) CometColumnarExchange -Input [4]: [sum#51, count#52, count#53, count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(47) CometHashAggregate -Input [4]: [sum#51, count#52, count#53, count#57] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#48)), count(ss_list_price#48), count(distinct ss_list_price#48)] - -(48) CometColumnarToRow [codegen id : 7] -Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] - -(49) BroadcastExchange -Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] -Arguments: IdentityBroadcastMode, [plan_id=11] - -(50) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(51) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] -ReadSchema: struct - -(52) CometFilter -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Condition : (((isnotnull(ss_quantity#61) AND (ss_quantity#61 >= 21)) AND (ss_quantity#61 <= 25)) AND ((((ss_list_price#63 >= 122.00) AND (ss_list_price#63 <= 132.00)) OR ((ss_coupon_amt#64 >= 836.00) AND (ss_coupon_amt#64 <= 1836.00))) OR ((ss_wholesale_cost#62 >= 17.00) AND (ss_wholesale_cost#62 <= 37.00)))) - -(53) CometProject -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Arguments: [ss_list_price#63], [ss_list_price#63] - -(54) CometHashAggregate -Input [1]: [ss_list_price#63] -Keys [1]: [ss_list_price#63] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_list_price#63)] - -(55) CometExchange -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(56) CometColumnarToRow [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] - -(57) HashAggregate [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Keys [1]: [ss_list_price#63] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70] -Results [4]: [ss_list_price#63, sum#66, count#67, count#68] - -(58) HashAggregate [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63), partial_count(distinct ss_list_price#63)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70, count(ss_list_price#63)#71] -Results [4]: [sum#66, count#67, count#68, count#72] - -(59) CometColumnarExchange -Input [4]: [sum#66, count#67, count#68, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(60) CometHashAggregate -Input [4]: [sum#66, count#67, count#68, count#72] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#63)), count(ss_list_price#63), count(distinct ss_list_price#63)] - -(61) CometColumnarToRow [codegen id : 9] -Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] - -(62) BroadcastExchange -Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] -Arguments: IdentityBroadcastMode, [plan_id=14] - -(63) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(64) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] -ReadSchema: struct - -(65) CometFilter -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Condition : (((isnotnull(ss_quantity#76) AND (ss_quantity#76 >= 26)) AND (ss_quantity#76 <= 30)) AND ((((ss_list_price#78 >= 154.00) AND (ss_list_price#78 <= 164.00)) OR ((ss_coupon_amt#79 >= 7326.00) AND (ss_coupon_amt#79 <= 8326.00))) OR ((ss_wholesale_cost#77 >= 7.00) AND (ss_wholesale_cost#77 <= 27.00)))) - -(66) CometProject -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Arguments: [ss_list_price#78], [ss_list_price#78] - -(67) CometHashAggregate -Input [1]: [ss_list_price#78] -Keys [1]: [ss_list_price#78] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_list_price#78)] - -(68) CometExchange -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(69) CometColumnarToRow [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] - -(70) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Keys [1]: [ss_list_price#78] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85] -Results [4]: [ss_list_price#78, sum#81, count#82, count#83] - -(71) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78), partial_count(distinct ss_list_price#78)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85, count(ss_list_price#78)#86] -Results [4]: [sum#81, count#82, count#83, count#87] - -(72) CometColumnarExchange -Input [4]: [sum#81, count#82, count#83, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(73) CometHashAggregate -Input [4]: [sum#81, count#82, count#83, count#87] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#78)), count(ss_list_price#78), count(distinct ss_list_price#78)] - -(74) CometColumnarToRow [codegen id : 11] -Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] - -(75) BroadcastExchange -Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] -Arguments: IdentityBroadcastMode, [plan_id=17] - -(76) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_datafusion/simplified.txt deleted file mode 100644 index a8540a4abb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_datafusion/simplified.txt +++ /dev/null @@ -1,105 +0,0 @@ -WholeStageCodegen (12) - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B1_LP,B1_CNT,B1_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #2 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B2_LP,B2_CNT,B2_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #4 - WholeStageCodegen (2) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #5 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B3_LP,B3_CNT,B3_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #7 - WholeStageCodegen (4) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #8 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B4_LP,B4_CNT,B4_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #10 - WholeStageCodegen (6) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #11 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B5_LP,B5_CNT,B5_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #13 - WholeStageCodegen (8) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #14 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B6_LP,B6_CNT,B6_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #16 - WholeStageCodegen (10) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #17 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometNativeScan 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/q28.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_iceberg_compat/explain.txt deleted file mode 100644 index 98ac957fc7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,437 +0,0 @@ -== Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (76) -:- * BroadcastNestedLoopJoin Inner BuildRight (63) -: :- * BroadcastNestedLoopJoin Inner BuildRight (50) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (37) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (24) -: : : : :- * CometColumnarToRow (11) -: : : : : +- CometHashAggregate (10) -: : : : : +- CometColumnarExchange (9) -: : : : : +- * HashAggregate (8) -: : : : : +- * HashAggregate (7) -: : : : : +- * CometColumnarToRow (6) -: : : : : +- CometExchange (5) -: : : : : +- CometHashAggregate (4) -: : : : : +- CometProject (3) -: : : : : +- CometFilter (2) -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) -: : : : +- BroadcastExchange (23) -: : : : +- * CometColumnarToRow (22) -: : : : +- CometHashAggregate (21) -: : : : +- CometColumnarExchange (20) -: : : : +- * HashAggregate (19) -: : : : +- * HashAggregate (18) -: : : : +- * CometColumnarToRow (17) -: : : : +- CometExchange (16) -: : : : +- CometHashAggregate (15) -: : : : +- CometProject (14) -: : : : +- CometFilter (13) -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (12) -: : : +- BroadcastExchange (36) -: : : +- * CometColumnarToRow (35) -: : : +- CometHashAggregate (34) -: : : +- CometColumnarExchange (33) -: : : +- * HashAggregate (32) -: : : +- * HashAggregate (31) -: : : +- * CometColumnarToRow (30) -: : : +- CometExchange (29) -: : : +- CometHashAggregate (28) -: : : +- CometProject (27) -: : : +- CometFilter (26) -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (25) -: : +- BroadcastExchange (49) -: : +- * CometColumnarToRow (48) -: : +- CometHashAggregate (47) -: : +- CometColumnarExchange (46) -: : +- * HashAggregate (45) -: : +- * HashAggregate (44) -: : +- * CometColumnarToRow (43) -: : +- CometExchange (42) -: : +- CometHashAggregate (41) -: : +- CometProject (40) -: : +- CometFilter (39) -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (38) -: +- BroadcastExchange (62) -: +- * CometColumnarToRow (61) -: +- CometHashAggregate (60) -: +- CometColumnarExchange (59) -: +- * HashAggregate (58) -: +- * HashAggregate (57) -: +- * CometColumnarToRow (56) -: +- CometExchange (55) -: +- CometHashAggregate (54) -: +- CometProject (53) -: +- CometFilter (52) -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (51) -+- BroadcastExchange (75) - +- * CometColumnarToRow (74) - +- CometHashAggregate (73) - +- CometColumnarExchange (72) - +- * HashAggregate (71) - +- * HashAggregate (70) - +- * CometColumnarToRow (69) - +- CometExchange (68) - +- CometHashAggregate (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (64) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,0), LessThanOrEqual(ss_quantity,5), Or(Or(And(GreaterThanOrEqual(ss_list_price,8.00),LessThanOrEqual(ss_list_price,18.00)),And(GreaterThanOrEqual(ss_coupon_amt,459.00),LessThanOrEqual(ss_coupon_amt,1459.00))),And(GreaterThanOrEqual(ss_wholesale_cost,57.00),LessThanOrEqual(ss_wholesale_cost,77.00)))] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 0)) AND (ss_quantity#1 <= 5)) AND ((((ss_list_price#3 >= 8.00) AND (ss_list_price#3 <= 18.00)) OR ((ss_coupon_amt#4 >= 459.00) AND (ss_coupon_amt#4 <= 1459.00))) OR ((ss_wholesale_cost#2 >= 57.00) AND (ss_wholesale_cost#2 <= 77.00)))) - -(3) CometProject -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Arguments: [ss_list_price#3], [ss_list_price#3] - -(4) CometHashAggregate -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] - -(5) CometExchange -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(6) CometColumnarToRow [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] - -(7) HashAggregate [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10] -Results [4]: [ss_list_price#3, sum#6, count#7, count#8] - -(8) HashAggregate [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10, count(ss_list_price#3)#11] -Results [4]: [sum#6, count#7, count#8, count#12] - -(9) CometColumnarExchange -Input [4]: [sum#6, count#7, count#8, count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(10) CometHashAggregate -Input [4]: [sum#6, count#7, count#8, count#12] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] - -(11) CometColumnarToRow [codegen id : 12] -Input [3]: [B1_LP#13, B1_CNT#14, B1_CNTD#15] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] -ReadSchema: struct - -(13) CometFilter -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Condition : (((isnotnull(ss_quantity#16) AND (ss_quantity#16 >= 6)) AND (ss_quantity#16 <= 10)) AND ((((ss_list_price#18 >= 90.00) AND (ss_list_price#18 <= 100.00)) OR ((ss_coupon_amt#19 >= 2323.00) AND (ss_coupon_amt#19 <= 3323.00))) OR ((ss_wholesale_cost#17 >= 31.00) AND (ss_wholesale_cost#17 <= 51.00)))) - -(14) CometProject -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Arguments: [ss_list_price#18], [ss_list_price#18] - -(15) CometHashAggregate -Input [1]: [ss_list_price#18] -Keys [1]: [ss_list_price#18] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_list_price#18)] - -(16) CometExchange -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(17) CometColumnarToRow [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] - -(18) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Keys [1]: [ss_list_price#18] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25] -Results [4]: [ss_list_price#18, sum#21, count#22, count#23] - -(19) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18), partial_count(distinct ss_list_price#18)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25, count(ss_list_price#18)#26] -Results [4]: [sum#21, count#22, count#23, count#27] - -(20) CometColumnarExchange -Input [4]: [sum#21, count#22, count#23, count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(21) CometHashAggregate -Input [4]: [sum#21, count#22, count#23, count#27] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#18)), count(ss_list_price#18), count(distinct ss_list_price#18)] - -(22) CometColumnarToRow [codegen id : 3] -Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] - -(23) BroadcastExchange -Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] -Arguments: IdentityBroadcastMode, [plan_id=5] - -(24) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] -ReadSchema: struct - -(26) CometFilter -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Condition : (((isnotnull(ss_quantity#31) AND (ss_quantity#31 >= 11)) AND (ss_quantity#31 <= 15)) AND ((((ss_list_price#33 >= 142.00) AND (ss_list_price#33 <= 152.00)) OR ((ss_coupon_amt#34 >= 12214.00) AND (ss_coupon_amt#34 <= 13214.00))) OR ((ss_wholesale_cost#32 >= 79.00) AND (ss_wholesale_cost#32 <= 99.00)))) - -(27) CometProject -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Arguments: [ss_list_price#33], [ss_list_price#33] - -(28) CometHashAggregate -Input [1]: [ss_list_price#33] -Keys [1]: [ss_list_price#33] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_list_price#33)] - -(29) CometExchange -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(30) CometColumnarToRow [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] - -(31) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Keys [1]: [ss_list_price#33] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40] -Results [4]: [ss_list_price#33, sum#36, count#37, count#38] - -(32) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33), partial_count(distinct ss_list_price#33)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40, count(ss_list_price#33)#41] -Results [4]: [sum#36, count#37, count#38, count#42] - -(33) CometColumnarExchange -Input [4]: [sum#36, count#37, count#38, count#42] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(34) CometHashAggregate -Input [4]: [sum#36, count#37, count#38, count#42] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#33)), count(ss_list_price#33), count(distinct ss_list_price#33)] - -(35) CometColumnarToRow [codegen id : 5] -Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] - -(36) BroadcastExchange -Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] -Arguments: IdentityBroadcastMode, [plan_id=8] - -(37) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] -ReadSchema: struct - -(39) CometFilter -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Condition : (((isnotnull(ss_quantity#46) AND (ss_quantity#46 >= 16)) AND (ss_quantity#46 <= 20)) AND ((((ss_list_price#48 >= 135.00) AND (ss_list_price#48 <= 145.00)) OR ((ss_coupon_amt#49 >= 6071.00) AND (ss_coupon_amt#49 <= 7071.00))) OR ((ss_wholesale_cost#47 >= 38.00) AND (ss_wholesale_cost#47 <= 58.00)))) - -(40) CometProject -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Arguments: [ss_list_price#48], [ss_list_price#48] - -(41) CometHashAggregate -Input [1]: [ss_list_price#48] -Keys [1]: [ss_list_price#48] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_list_price#48)] - -(42) CometExchange -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(43) CometColumnarToRow [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] - -(44) HashAggregate [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Keys [1]: [ss_list_price#48] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55] -Results [4]: [ss_list_price#48, sum#51, count#52, count#53] - -(45) HashAggregate [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48), partial_count(distinct ss_list_price#48)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55, count(ss_list_price#48)#56] -Results [4]: [sum#51, count#52, count#53, count#57] - -(46) CometColumnarExchange -Input [4]: [sum#51, count#52, count#53, count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(47) CometHashAggregate -Input [4]: [sum#51, count#52, count#53, count#57] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#48)), count(ss_list_price#48), count(distinct ss_list_price#48)] - -(48) CometColumnarToRow [codegen id : 7] -Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] - -(49) BroadcastExchange -Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] -Arguments: IdentityBroadcastMode, [plan_id=11] - -(50) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] -ReadSchema: struct - -(52) CometFilter -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Condition : (((isnotnull(ss_quantity#61) AND (ss_quantity#61 >= 21)) AND (ss_quantity#61 <= 25)) AND ((((ss_list_price#63 >= 122.00) AND (ss_list_price#63 <= 132.00)) OR ((ss_coupon_amt#64 >= 836.00) AND (ss_coupon_amt#64 <= 1836.00))) OR ((ss_wholesale_cost#62 >= 17.00) AND (ss_wholesale_cost#62 <= 37.00)))) - -(53) CometProject -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Arguments: [ss_list_price#63], [ss_list_price#63] - -(54) CometHashAggregate -Input [1]: [ss_list_price#63] -Keys [1]: [ss_list_price#63] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_list_price#63)] - -(55) CometExchange -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(56) CometColumnarToRow [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] - -(57) HashAggregate [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Keys [1]: [ss_list_price#63] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70] -Results [4]: [ss_list_price#63, sum#66, count#67, count#68] - -(58) HashAggregate [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63), partial_count(distinct ss_list_price#63)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70, count(ss_list_price#63)#71] -Results [4]: [sum#66, count#67, count#68, count#72] - -(59) CometColumnarExchange -Input [4]: [sum#66, count#67, count#68, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(60) CometHashAggregate -Input [4]: [sum#66, count#67, count#68, count#72] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#63)), count(ss_list_price#63), count(distinct ss_list_price#63)] - -(61) CometColumnarToRow [codegen id : 9] -Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] - -(62) BroadcastExchange -Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] -Arguments: IdentityBroadcastMode, [plan_id=14] - -(63) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] -ReadSchema: struct - -(65) CometFilter -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Condition : (((isnotnull(ss_quantity#76) AND (ss_quantity#76 >= 26)) AND (ss_quantity#76 <= 30)) AND ((((ss_list_price#78 >= 154.00) AND (ss_list_price#78 <= 164.00)) OR ((ss_coupon_amt#79 >= 7326.00) AND (ss_coupon_amt#79 <= 8326.00))) OR ((ss_wholesale_cost#77 >= 7.00) AND (ss_wholesale_cost#77 <= 27.00)))) - -(66) CometProject -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Arguments: [ss_list_price#78], [ss_list_price#78] - -(67) CometHashAggregate -Input [1]: [ss_list_price#78] -Keys [1]: [ss_list_price#78] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_list_price#78)] - -(68) CometExchange -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(69) CometColumnarToRow [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] - -(70) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Keys [1]: [ss_list_price#78] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85] -Results [4]: [ss_list_price#78, sum#81, count#82, count#83] - -(71) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78), partial_count(distinct ss_list_price#78)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85, count(ss_list_price#78)#86] -Results [4]: [sum#81, count#82, count#83, count#87] - -(72) CometColumnarExchange -Input [4]: [sum#81, count#82, count#83, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(73) CometHashAggregate -Input [4]: [sum#81, count#82, count#83, count#87] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#78)), count(ss_list_price#78), count(distinct ss_list_price#78)] - -(74) CometColumnarToRow [codegen id : 11] -Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] - -(75) BroadcastExchange -Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] -Arguments: IdentityBroadcastMode, [plan_id=17] - -(76) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_iceberg_compat/simplified.txt deleted file mode 100644 index ce476affda..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,105 +0,0 @@ -WholeStageCodegen (12) - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B1_LP,B1_CNT,B1_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #2 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B2_LP,B2_CNT,B2_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #4 - WholeStageCodegen (2) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #5 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B3_LP,B3_CNT,B3_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #7 - WholeStageCodegen (4) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #8 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B4_LP,B4_CNT,B4_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #10 - WholeStageCodegen (6) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #11 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B5_LP,B5_CNT,B5_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #13 - WholeStageCodegen (8) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #14 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B6_LP,B6_CNT,B6_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #16 - WholeStageCodegen (10) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #17 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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/q28/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt deleted file mode 100644 index 98ac957fc7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt +++ /dev/null @@ -1,437 +0,0 @@ -== Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (76) -:- * BroadcastNestedLoopJoin Inner BuildRight (63) -: :- * BroadcastNestedLoopJoin Inner BuildRight (50) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (37) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (24) -: : : : :- * CometColumnarToRow (11) -: : : : : +- CometHashAggregate (10) -: : : : : +- CometColumnarExchange (9) -: : : : : +- * HashAggregate (8) -: : : : : +- * HashAggregate (7) -: : : : : +- * CometColumnarToRow (6) -: : : : : +- CometExchange (5) -: : : : : +- CometHashAggregate (4) -: : : : : +- CometProject (3) -: : : : : +- CometFilter (2) -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) -: : : : +- BroadcastExchange (23) -: : : : +- * CometColumnarToRow (22) -: : : : +- CometHashAggregate (21) -: : : : +- CometColumnarExchange (20) -: : : : +- * HashAggregate (19) -: : : : +- * HashAggregate (18) -: : : : +- * CometColumnarToRow (17) -: : : : +- CometExchange (16) -: : : : +- CometHashAggregate (15) -: : : : +- CometProject (14) -: : : : +- CometFilter (13) -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (12) -: : : +- BroadcastExchange (36) -: : : +- * CometColumnarToRow (35) -: : : +- CometHashAggregate (34) -: : : +- CometColumnarExchange (33) -: : : +- * HashAggregate (32) -: : : +- * HashAggregate (31) -: : : +- * CometColumnarToRow (30) -: : : +- CometExchange (29) -: : : +- CometHashAggregate (28) -: : : +- CometProject (27) -: : : +- CometFilter (26) -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (25) -: : +- BroadcastExchange (49) -: : +- * CometColumnarToRow (48) -: : +- CometHashAggregate (47) -: : +- CometColumnarExchange (46) -: : +- * HashAggregate (45) -: : +- * HashAggregate (44) -: : +- * CometColumnarToRow (43) -: : +- CometExchange (42) -: : +- CometHashAggregate (41) -: : +- CometProject (40) -: : +- CometFilter (39) -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (38) -: +- BroadcastExchange (62) -: +- * CometColumnarToRow (61) -: +- CometHashAggregate (60) -: +- CometColumnarExchange (59) -: +- * HashAggregate (58) -: +- * HashAggregate (57) -: +- * CometColumnarToRow (56) -: +- CometExchange (55) -: +- CometHashAggregate (54) -: +- CometProject (53) -: +- CometFilter (52) -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (51) -+- BroadcastExchange (75) - +- * CometColumnarToRow (74) - +- CometHashAggregate (73) - +- CometColumnarExchange (72) - +- * HashAggregate (71) - +- * HashAggregate (70) - +- * CometColumnarToRow (69) - +- CometExchange (68) - +- CometHashAggregate (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (64) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,0), LessThanOrEqual(ss_quantity,5), Or(Or(And(GreaterThanOrEqual(ss_list_price,8.00),LessThanOrEqual(ss_list_price,18.00)),And(GreaterThanOrEqual(ss_coupon_amt,459.00),LessThanOrEqual(ss_coupon_amt,1459.00))),And(GreaterThanOrEqual(ss_wholesale_cost,57.00),LessThanOrEqual(ss_wholesale_cost,77.00)))] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 0)) AND (ss_quantity#1 <= 5)) AND ((((ss_list_price#3 >= 8.00) AND (ss_list_price#3 <= 18.00)) OR ((ss_coupon_amt#4 >= 459.00) AND (ss_coupon_amt#4 <= 1459.00))) OR ((ss_wholesale_cost#2 >= 57.00) AND (ss_wholesale_cost#2 <= 77.00)))) - -(3) CometProject -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Arguments: [ss_list_price#3], [ss_list_price#3] - -(4) CometHashAggregate -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] - -(5) CometExchange -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(6) CometColumnarToRow [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] - -(7) HashAggregate [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10] -Results [4]: [ss_list_price#3, sum#6, count#7, count#8] - -(8) HashAggregate [codegen id : 1] -Input [4]: [ss_list_price#3, sum#6, count#7, count#8] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#9, count(ss_list_price#3)#10, count(ss_list_price#3)#11] -Results [4]: [sum#6, count#7, count#8, count#12] - -(9) CometColumnarExchange -Input [4]: [sum#6, count#7, count#8, count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(10) CometHashAggregate -Input [4]: [sum#6, count#7, count#8, count#12] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] - -(11) CometColumnarToRow [codegen id : 12] -Input [3]: [B1_LP#13, B1_CNT#14, B1_CNTD#15] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] -ReadSchema: struct - -(13) CometFilter -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Condition : (((isnotnull(ss_quantity#16) AND (ss_quantity#16 >= 6)) AND (ss_quantity#16 <= 10)) AND ((((ss_list_price#18 >= 90.00) AND (ss_list_price#18 <= 100.00)) OR ((ss_coupon_amt#19 >= 2323.00) AND (ss_coupon_amt#19 <= 3323.00))) OR ((ss_wholesale_cost#17 >= 31.00) AND (ss_wholesale_cost#17 <= 51.00)))) - -(14) CometProject -Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] -Arguments: [ss_list_price#18], [ss_list_price#18] - -(15) CometHashAggregate -Input [1]: [ss_list_price#18] -Keys [1]: [ss_list_price#18] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_list_price#18)] - -(16) CometExchange -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(17) CometColumnarToRow [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] - -(18) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Keys [1]: [ss_list_price#18] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25] -Results [4]: [ss_list_price#18, sum#21, count#22, count#23] - -(19) HashAggregate [codegen id : 2] -Input [4]: [ss_list_price#18, sum#21, count#22, count#23] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18), partial_count(distinct ss_list_price#18)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#24, count(ss_list_price#18)#25, count(ss_list_price#18)#26] -Results [4]: [sum#21, count#22, count#23, count#27] - -(20) CometColumnarExchange -Input [4]: [sum#21, count#22, count#23, count#27] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(21) CometHashAggregate -Input [4]: [sum#21, count#22, count#23, count#27] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#18)), count(ss_list_price#18), count(distinct ss_list_price#18)] - -(22) CometColumnarToRow [codegen id : 3] -Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] - -(23) BroadcastExchange -Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] -Arguments: IdentityBroadcastMode, [plan_id=5] - -(24) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] -ReadSchema: struct - -(26) CometFilter -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Condition : (((isnotnull(ss_quantity#31) AND (ss_quantity#31 >= 11)) AND (ss_quantity#31 <= 15)) AND ((((ss_list_price#33 >= 142.00) AND (ss_list_price#33 <= 152.00)) OR ((ss_coupon_amt#34 >= 12214.00) AND (ss_coupon_amt#34 <= 13214.00))) OR ((ss_wholesale_cost#32 >= 79.00) AND (ss_wholesale_cost#32 <= 99.00)))) - -(27) CometProject -Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] -Arguments: [ss_list_price#33], [ss_list_price#33] - -(28) CometHashAggregate -Input [1]: [ss_list_price#33] -Keys [1]: [ss_list_price#33] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_list_price#33)] - -(29) CometExchange -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(30) CometColumnarToRow [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] - -(31) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Keys [1]: [ss_list_price#33] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40] -Results [4]: [ss_list_price#33, sum#36, count#37, count#38] - -(32) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#33, sum#36, count#37, count#38] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33), partial_count(distinct ss_list_price#33)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#39, count(ss_list_price#33)#40, count(ss_list_price#33)#41] -Results [4]: [sum#36, count#37, count#38, count#42] - -(33) CometColumnarExchange -Input [4]: [sum#36, count#37, count#38, count#42] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(34) CometHashAggregate -Input [4]: [sum#36, count#37, count#38, count#42] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#33)), count(ss_list_price#33), count(distinct ss_list_price#33)] - -(35) CometColumnarToRow [codegen id : 5] -Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] - -(36) BroadcastExchange -Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] -Arguments: IdentityBroadcastMode, [plan_id=8] - -(37) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] -ReadSchema: struct - -(39) CometFilter -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Condition : (((isnotnull(ss_quantity#46) AND (ss_quantity#46 >= 16)) AND (ss_quantity#46 <= 20)) AND ((((ss_list_price#48 >= 135.00) AND (ss_list_price#48 <= 145.00)) OR ((ss_coupon_amt#49 >= 6071.00) AND (ss_coupon_amt#49 <= 7071.00))) OR ((ss_wholesale_cost#47 >= 38.00) AND (ss_wholesale_cost#47 <= 58.00)))) - -(40) CometProject -Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] -Arguments: [ss_list_price#48], [ss_list_price#48] - -(41) CometHashAggregate -Input [1]: [ss_list_price#48] -Keys [1]: [ss_list_price#48] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_list_price#48)] - -(42) CometExchange -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(43) CometColumnarToRow [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] - -(44) HashAggregate [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Keys [1]: [ss_list_price#48] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55] -Results [4]: [ss_list_price#48, sum#51, count#52, count#53] - -(45) HashAggregate [codegen id : 6] -Input [4]: [ss_list_price#48, sum#51, count#52, count#53] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48), partial_count(distinct ss_list_price#48)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#54, count(ss_list_price#48)#55, count(ss_list_price#48)#56] -Results [4]: [sum#51, count#52, count#53, count#57] - -(46) CometColumnarExchange -Input [4]: [sum#51, count#52, count#53, count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(47) CometHashAggregate -Input [4]: [sum#51, count#52, count#53, count#57] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#48)), count(ss_list_price#48), count(distinct ss_list_price#48)] - -(48) CometColumnarToRow [codegen id : 7] -Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] - -(49) BroadcastExchange -Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] -Arguments: IdentityBroadcastMode, [plan_id=11] - -(50) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] -ReadSchema: struct - -(52) CometFilter -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Condition : (((isnotnull(ss_quantity#61) AND (ss_quantity#61 >= 21)) AND (ss_quantity#61 <= 25)) AND ((((ss_list_price#63 >= 122.00) AND (ss_list_price#63 <= 132.00)) OR ((ss_coupon_amt#64 >= 836.00) AND (ss_coupon_amt#64 <= 1836.00))) OR ((ss_wholesale_cost#62 >= 17.00) AND (ss_wholesale_cost#62 <= 37.00)))) - -(53) CometProject -Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Arguments: [ss_list_price#63], [ss_list_price#63] - -(54) CometHashAggregate -Input [1]: [ss_list_price#63] -Keys [1]: [ss_list_price#63] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_list_price#63)] - -(55) CometExchange -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(56) CometColumnarToRow [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] - -(57) HashAggregate [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Keys [1]: [ss_list_price#63] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70] -Results [4]: [ss_list_price#63, sum#66, count#67, count#68] - -(58) HashAggregate [codegen id : 8] -Input [4]: [ss_list_price#63, sum#66, count#67, count#68] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63), partial_count(distinct ss_list_price#63)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#69, count(ss_list_price#63)#70, count(ss_list_price#63)#71] -Results [4]: [sum#66, count#67, count#68, count#72] - -(59) CometColumnarExchange -Input [4]: [sum#66, count#67, count#68, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(60) CometHashAggregate -Input [4]: [sum#66, count#67, count#68, count#72] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#63)), count(ss_list_price#63), count(distinct ss_list_price#63)] - -(61) CometColumnarToRow [codegen id : 9] -Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] - -(62) BroadcastExchange -Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] -Arguments: IdentityBroadcastMode, [plan_id=14] - -(63) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] -ReadSchema: struct - -(65) CometFilter -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Condition : (((isnotnull(ss_quantity#76) AND (ss_quantity#76 >= 26)) AND (ss_quantity#76 <= 30)) AND ((((ss_list_price#78 >= 154.00) AND (ss_list_price#78 <= 164.00)) OR ((ss_coupon_amt#79 >= 7326.00) AND (ss_coupon_amt#79 <= 8326.00))) OR ((ss_wholesale_cost#77 >= 7.00) AND (ss_wholesale_cost#77 <= 27.00)))) - -(66) CometProject -Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] -Arguments: [ss_list_price#78], [ss_list_price#78] - -(67) CometHashAggregate -Input [1]: [ss_list_price#78] -Keys [1]: [ss_list_price#78] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_list_price#78)] - -(68) CometExchange -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(69) CometColumnarToRow [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] - -(70) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Keys [1]: [ss_list_price#78] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85] -Results [4]: [ss_list_price#78, sum#81, count#82, count#83] - -(71) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#78, sum#81, count#82, count#83] -Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78), partial_count(distinct ss_list_price#78)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#84, count(ss_list_price#78)#85, count(ss_list_price#78)#86] -Results [4]: [sum#81, count#82, count#83, count#87] - -(72) CometColumnarExchange -Input [4]: [sum#81, count#82, count#83, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(73) CometHashAggregate -Input [4]: [sum#81, count#82, count#83, count#87] -Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#78)), count(ss_list_price#78), count(distinct ss_list_price#78)] - -(74) CometColumnarToRow [codegen id : 11] -Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] - -(75) BroadcastExchange -Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] -Arguments: IdentityBroadcastMode, [plan_id=17] - -(76) BroadcastNestedLoopJoin [codegen id : 12] -Join type: Inner -Join condition: None - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/extended.txt deleted file mode 100644 index 809265b4d5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/extended.txt +++ /dev/null @@ -1,78 +0,0 @@ -BroadcastNestedLoopJoin -:- BroadcastNestedLoopJoin -: :- BroadcastNestedLoopJoin -: : :- BroadcastNestedLoopJoin -: : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] -: : : : :- CometColumnarToRow -: : : : : +- CometHashAggregate -: : : : : +- CometColumnarExchange -: : : : : +- HashAggregate -: : : : : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : : : : +- CometColumnarToRow -: : : : : +- CometExchange -: : : : : +- CometHashAggregate -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : +- BroadcastExchange -: : : : +- CometColumnarToRow -: : : : +- CometHashAggregate -: : : : +- CometColumnarExchange -: : : : +- HashAggregate -: : : : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : : : +- CometColumnarToRow -: : : : +- CometExchange -: : : : +- CometHashAggregate -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : +- BroadcastExchange -: : : +- CometColumnarToRow -: : : +- CometHashAggregate -: : : +- CometColumnarExchange -: : : +- HashAggregate -: : : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : : +- CometColumnarToRow -: : : +- CometExchange -: : : +- CometHashAggregate -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : +- BroadcastExchange -: : +- CometColumnarToRow -: : +- CometHashAggregate -: : +- CometColumnarExchange -: : +- HashAggregate -: : +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: : +- CometColumnarToRow -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: +- BroadcastExchange -: +- CometColumnarToRow -: +- CometHashAggregate -: +- CometColumnarExchange -: +- HashAggregate -: +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] -: +- CometColumnarToRow -: +- CometExchange -: +- CometHashAggregate -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -+- BroadcastExchange - +- CometColumnarToRow - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - -Comet accelerated 42 out of 64 eligible operators (65%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index ce476affda..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/simplified.txt +++ /dev/null @@ -1,105 +0,0 @@ -WholeStageCodegen (12) - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B1_LP,B1_CNT,B1_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #2 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B2_LP,B2_CNT,B2_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #4 - WholeStageCodegen (2) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #5 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B3_LP,B3_CNT,B3_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #7 - WholeStageCodegen (4) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #8 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B4_LP,B4_CNT,B4_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #10 - WholeStageCodegen (6) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #11 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B5_LP,B5_CNT,B5_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #13 - WholeStageCodegen (8) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #14 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count,count,count] [B6_LP,B6_CNT,B6_CNTD,avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price)] - CometColumnarExchange #16 - WholeStageCodegen (10) - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] - HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] - CometColumnarToRow - InputAdapter - CometExchange [ss_list_price] #17 - CometHashAggregate [ss_list_price,sum,count,count] - CometProject [ss_list_price] - CometFilter [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/explain.txt deleted file mode 100644 index 20f1508413..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/explain.txt +++ /dev/null @@ -1,340 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (21) - : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : :- * Project (18) - : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : :- * Project (15) - : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : :- * Project (9) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (7) - : : : : : : +- * Filter (6) - : : : : : : +- * ColumnarToRow (5) - : : : : : : +- Scan parquet spark_catalog.default.store_returns (4) - : : : : : +- BroadcastExchange (13) - : : : : : +- * Filter (12) - : : : : : +- * ColumnarToRow (11) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (10) - : : : : +- ReusedExchange (16) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.store (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.item (32) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 8] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] - -(3) Filter [codegen id : 8] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(4) Scan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(6) Filter [codegen id : 1] -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(7) BroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(input[1, int, false], input[0, int, false], input[2, int, false]),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 8] -Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] -Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(10) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 2] -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(12) Filter [codegen id : 2] -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(13) BroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] -Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 8] -Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(16) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#19] - -(17) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 8] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] - -(19) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#20] - -(20) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [sr_returned_date_sk#12] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#20] - -(22) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#21] - -(23) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#21] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 8] -Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#21] - -(25) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(26) CometFilter -Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] -Condition : isnotnull(s_store_sk#22) - -(27) CometProject -Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] -Arguments: [s_store_sk#22, s_store_id#25, s_store_name#24], [s_store_sk#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#23, 16, true, false, true) AS s_store_id#25, s_store_name#24] - -(28) CometColumnarToRow [codegen id : 6] -Input [3]: [s_store_sk#22, s_store_id#25, s_store_name#24] - -(29) BroadcastExchange -Input [3]: [s_store_sk#22, s_store_id#25, s_store_name#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(30) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#22] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 8] -Output [6]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#22, s_store_id#25, s_store_name#24] - -(32) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] -Condition : isnotnull(i_item_sk#26) - -(34) CometProject -Input [3]: [i_item_sk#26, i_item_id#27, i_item_desc#28] -Arguments: [i_item_sk#26, i_item_id#29, i_item_desc#28], [i_item_sk#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#27, 16, true, false, true) AS i_item_id#29, i_item_desc#28] - -(35) CometColumnarToRow [codegen id : 7] -Input [3]: [i_item_sk#26, i_item_id#29, i_item_desc#28] - -(36) BroadcastExchange -Input [3]: [i_item_sk#26, i_item_id#29, i_item_desc#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(37) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#26] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 8] -Output [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24, i_item_id#29, i_item_desc#28] -Input [9]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24, i_item_sk#26, i_item_id#29, i_item_desc#28] - -(39) HashAggregate [codegen id : 8] -Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#25, s_store_name#24, i_item_id#29, i_item_desc#28] -Keys [4]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24] -Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#11), partial_sum(cs_quantity#16)] -Aggregate Attributes [3]: [sum#30, sum#31, sum#32] -Results [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum#33, sum#34, sum#35] - -(40) CometColumnarExchange -Input [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum#33, sum#34, sum#35] -Arguments: hashpartitioning(i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(41) CometColumnarToRow [codegen id : 9] -Input [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum#33, sum#34, sum#35] - -(42) HashAggregate [codegen id : 9] -Input [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum#33, sum#34, sum#35] -Keys [4]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24] -Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity#16)] -Aggregate Attributes [3]: [sum(ss_quantity#5)#36, sum(sr_return_quantity#11)#37, sum(cs_quantity#16)#38] -Results [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, sum(ss_quantity#5)#36 AS store_sales_quantity#39, sum(sr_return_quantity#11)#37 AS store_returns_quantity#40, sum(cs_quantity#16)#38 AS catalog_sales_quantity#41] - -(43) TakeOrderedAndProject -Input [7]: [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, store_sales_quantity#39, store_returns_quantity#40, catalog_sales_quantity#41] -Arguments: 100, [i_item_id#29 ASC NULLS FIRST, i_item_desc#28 ASC NULLS FIRST, s_store_id#25 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST], [i_item_id#29, i_item_desc#28, s_store_id#25, s_store_name#24, store_sales_quantity#39, store_returns_quantity#40, catalog_sales_quantity#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometNativeScan parquet spark_catalog.default.date_dim (44) - - -(44) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#42, d_moy#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [3]: [d_date_sk#19, d_year#42, d_moy#43] -Condition : ((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 = 9)) AND (d_year#42 = 1999)) AND isnotnull(d_date_sk#19)) - -(46) CometProject -Input [3]: [d_date_sk#19, d_year#42, d_moy#43] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(48) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometNativeScan parquet spark_catalog.default.date_dim (49) - - -(49) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#20, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#20, d_year#44, d_moy#45] -Condition : (((((isnotnull(d_moy#45) AND isnotnull(d_year#44)) AND (d_moy#45 >= 9)) AND (d_moy#45 <= 12)) AND (d_year#44 = 1999)) AND isnotnull(d_date_sk#20)) - -(51) CometProject -Input [3]: [d_date_sk#20, d_year#44, d_moy#45] -Arguments: [d_date_sk#20], [d_date_sk#20] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#20] - -(53) BroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 -BroadcastExchange (58) -+- * CometColumnarToRow (57) - +- CometProject (56) - +- CometFilter (55) - +- CometNativeScan parquet spark_catalog.default.date_dim (54) - - -(54) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#21, d_year#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(55) CometFilter -Input [2]: [d_date_sk#21, d_year#46] -Condition : (d_year#46 IN (1999,2000,2001) AND isnotnull(d_date_sk#21)) - -(56) CometProject -Input [2]: [d_date_sk#21, d_year#46] -Arguments: [d_date_sk#21], [d_date_sk#21] - -(57) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#21] - -(58) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/simplified.txt deleted file mode 100644 index 54bdbeeb5a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/simplified.txt +++ /dev/null @@ -1,86 +0,0 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] - WholeStageCodegen (9) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity),store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - WholeStageCodegen (8) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] [sum,sum,sum,sum,sum,sum] - Project [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] - Project [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] - BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] - Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [sr_customer_sk,sr_item_sk,sr_ticket_number] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Filter [cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometNativeScan 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/q29.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/explain.txt deleted file mode 100644 index 88c51b72ee..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,366 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometTakeOrderedAndProject (46) - +- CometHashAggregate (45) - +- CometExchange (44) - +- CometHashAggregate (43) - +- CometProject (42) - +- CometBroadcastHashJoin (41) - :- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometProject (18) - : : : : +- CometBroadcastHashJoin (17) - : : : : :- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (8) - : : : : +- CometBroadcastExchange (16) - : : : : +- CometProject (15) - : : : : +- CometFilter (14) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : : : +- CometBroadcastExchange (22) - : : : +- CometProject (21) - : : : +- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - : : +- CometBroadcastExchange (28) - : : +- CometProject (27) - : : +- CometFilter (26) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - : +- CometBroadcastExchange (34) - : +- CometProject (33) - : +- CometFilter (32) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (31) - +- CometBroadcastExchange (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (37) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(5) CometBroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(6) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(10) CometBroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(11) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] -Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight - -(12) CometProject -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9)) AND (d_year#20 = 1999)) AND isnotnull(d_date_sk#19)) - -(15) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 9)) AND (d_moy#24 <= 12)) AND (d_year#23 = 1999)) AND isnotnull(d_date_sk#22)) - -(21) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#22] -Arguments: [sr_returned_date_sk#12], [d_date_sk#22], Inner, BuildRight - -(24) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#22] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) - -(27) CometProject -Input [2]: [d_date_sk#25, d_year#26] -Arguments: [d_date_sk#25], [d_date_sk#25] - -(28) CometBroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: [d_date_sk#25] - -(29) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#25] -Arguments: [cs_sold_date_sk#17], [d_date_sk#25], Inner, BuildRight - -(30) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#25] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] -Condition : isnotnull(s_store_sk#27) - -(33) CometProject -Input [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] -Arguments: [s_store_sk#27, s_store_id#30, s_store_name#29], [s_store_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#28, 16, true, false, true) AS s_store_id#30, s_store_name#29] - -(34) CometBroadcastExchange -Input [3]: [s_store_sk#27, s_store_id#30, s_store_name#29] -Arguments: [s_store_sk#27, s_store_id#30, s_store_name#29] - -(35) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Right output [3]: [s_store_sk#27, s_store_id#30, s_store_name#29] -Arguments: [ss_store_sk#3], [s_store_sk#27], Inner, BuildRight - -(36) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#27, s_store_id#30, s_store_name#29] -Arguments: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29], [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(38) CometFilter -Input [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Condition : isnotnull(i_item_sk#31) - -(39) CometProject -Input [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Arguments: [i_item_sk#31, i_item_id#34, i_item_desc#33], [i_item_sk#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#32, 16, true, false, true) AS i_item_id#34, i_item_desc#33] - -(40) CometBroadcastExchange -Input [3]: [i_item_sk#31, i_item_id#34, i_item_desc#33] -Arguments: [i_item_sk#31, i_item_id#34, i_item_desc#33] - -(41) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29] -Right output [3]: [i_item_sk#31, i_item_id#34, i_item_desc#33] -Arguments: [ss_item_sk#1], [i_item_sk#31], Inner, BuildRight - -(42) CometProject -Input [9]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_sk#31, i_item_id#34, i_item_desc#33] -Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_id#34, i_item_desc#33], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_id#34, i_item_desc#33] - -(43) CometHashAggregate -Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_id#34, i_item_desc#33] -Keys [4]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29] -Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#11), partial_sum(cs_quantity#16)] - -(44) CometExchange -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, sum#35, sum#36, sum#37] -Arguments: hashpartitioning(i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(45) CometHashAggregate -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, sum#35, sum#36, sum#37] -Keys [4]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29] -Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity#16)] - -(46) CometTakeOrderedAndProject -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#34 ASC NULLS FIRST,i_item_desc#33 ASC NULLS FIRST,s_store_id#30 ASC NULLS FIRST,s_store_name#29 ASC NULLS FIRST], output=[i_item_id#34,i_item_desc#33,s_store_id#30,s_store_name#29,store_sales_quantity#38,store_returns_quantity#39,catalog_sales_quantity#40]), [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40], 100, 0, [i_item_id#34 ASC NULLS FIRST, i_item_desc#33 ASC NULLS FIRST, s_store_id#30 ASC NULLS FIRST, s_store_name#29 ASC NULLS FIRST], [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40] - -(47) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9)) AND (d_year#20 = 1999)) AND isnotnull(d_date_sk#19)) - -(50) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(52) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 9)) AND (d_moy#24 <= 12)) AND (d_year#23 = 1999)) AND isnotnull(d_date_sk#22)) - -(55) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(56) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(57) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 -BroadcastExchange (62) -+- * CometColumnarToRow (61) - +- CometProject (60) - +- CometFilter (59) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) - - -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(59) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) - -(60) CometProject -Input [2]: [d_date_sk#25, d_year#26] -Arguments: [d_date_sk#25], [d_date_sk#25] - -(61) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#25] - -(62) BroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/simplified.txt deleted file mode 100644 index 15b992d3a4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] - CometHashAggregate [sum,sum,sum] [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity)] - CometExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - CometHashAggregate [ss_quantity,sr_return_quantity,cs_quantity] [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] - CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - 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_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 [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 [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_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 [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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #8 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #9 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan [native_iceberg_compat] 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/q29/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt deleted file mode 100644 index 88c51b72ee..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt +++ /dev/null @@ -1,366 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometTakeOrderedAndProject (46) - +- CometHashAggregate (45) - +- CometExchange (44) - +- CometHashAggregate (43) - +- CometProject (42) - +- CometBroadcastHashJoin (41) - :- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometProject (18) - : : : : +- CometBroadcastHashJoin (17) - : : : : :- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (8) - : : : : +- CometBroadcastExchange (16) - : : : : +- CometProject (15) - : : : : +- CometFilter (14) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : : : +- CometBroadcastExchange (22) - : : : +- CometProject (21) - : : : +- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - : : +- CometBroadcastExchange (28) - : : +- CometProject (27) - : : +- CometFilter (26) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - : +- CometBroadcastExchange (34) - : +- CometProject (33) - : +- CometFilter (32) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (31) - +- CometBroadcastExchange (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (37) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_ticket_number#4)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) - -(5) CometBroadcastExchange -Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] - -(6) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] -Right output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4], [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10], Inner, BuildRight - -(7) CometProject -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) - -(10) CometBroadcastExchange -Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] - -(11) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] -Right output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [sr_customer_sk#9, sr_item_sk#8], [cs_bill_customer_sk#14, cs_item_sk#15], Inner, BuildRight - -(12) CometProject -Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12, cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9)) AND (d_year#20 = 1999)) AND isnotnull(d_date_sk#19)) - -(15) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#6], [d_date_sk#19], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 9)) AND (d_moy#24 <= 12)) AND (d_year#23 = 1999)) AND isnotnull(d_date_sk#22)) - -(21) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#22] -Arguments: [sr_returned_date_sk#12], [d_date_sk#22], Inner, BuildRight - -(24) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#22] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) - -(27) CometProject -Input [2]: [d_date_sk#25, d_year#26] -Arguments: [d_date_sk#25], [d_date_sk#25] - -(28) CometBroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: [d_date_sk#25] - -(29) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#25] -Arguments: [cs_sold_date_sk#17], [d_date_sk#25], Inner, BuildRight - -(30) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#25] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16], [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] -Condition : isnotnull(s_store_sk#27) - -(33) CometProject -Input [3]: [s_store_sk#27, s_store_id#28, s_store_name#29] -Arguments: [s_store_sk#27, s_store_id#30, s_store_name#29], [s_store_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#28, 16, true, false, true) AS s_store_id#30, s_store_name#29] - -(34) CometBroadcastExchange -Input [3]: [s_store_sk#27, s_store_id#30, s_store_name#29] -Arguments: [s_store_sk#27, s_store_id#30, s_store_name#29] - -(35) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] -Right output [3]: [s_store_sk#27, s_store_id#30, s_store_name#29] -Arguments: [ss_store_sk#3], [s_store_sk#27], Inner, BuildRight - -(36) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#27, s_store_id#30, s_store_name#29] -Arguments: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29], [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(38) CometFilter -Input [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Condition : isnotnull(i_item_sk#31) - -(39) CometProject -Input [3]: [i_item_sk#31, i_item_id#32, i_item_desc#33] -Arguments: [i_item_sk#31, i_item_id#34, i_item_desc#33], [i_item_sk#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#32, 16, true, false, true) AS i_item_id#34, i_item_desc#33] - -(40) CometBroadcastExchange -Input [3]: [i_item_sk#31, i_item_id#34, i_item_desc#33] -Arguments: [i_item_sk#31, i_item_id#34, i_item_desc#33] - -(41) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29] -Right output [3]: [i_item_sk#31, i_item_id#34, i_item_desc#33] -Arguments: [ss_item_sk#1], [i_item_sk#31], Inner, BuildRight - -(42) CometProject -Input [9]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_sk#31, i_item_id#34, i_item_desc#33] -Arguments: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_id#34, i_item_desc#33], [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_id#34, i_item_desc#33] - -(43) CometHashAggregate -Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#30, s_store_name#29, i_item_id#34, i_item_desc#33] -Keys [4]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29] -Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#11), partial_sum(cs_quantity#16)] - -(44) CometExchange -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, sum#35, sum#36, sum#37] -Arguments: hashpartitioning(i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(45) CometHashAggregate -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, sum#35, sum#36, sum#37] -Keys [4]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29] -Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity#16)] - -(46) CometTakeOrderedAndProject -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#34 ASC NULLS FIRST,i_item_desc#33 ASC NULLS FIRST,s_store_id#30 ASC NULLS FIRST,s_store_name#29 ASC NULLS FIRST], output=[i_item_id#34,i_item_desc#33,s_store_id#30,s_store_name#29,store_sales_quantity#38,store_returns_quantity#39,catalog_sales_quantity#40]), [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40], 100, 0, [i_item_id#34 ASC NULLS FIRST, i_item_desc#33 ASC NULLS FIRST, s_store_id#30 ASC NULLS FIRST, s_store_name#29 ASC NULLS FIRST], [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40] - -(47) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#34, i_item_desc#33, s_store_id#30, s_store_name#29, store_sales_quantity#38, store_returns_quantity#39, catalog_sales_quantity#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_moy#21) AND isnotnull(d_year#20)) AND (d_moy#21 = 9)) AND (d_year#20 = 1999)) AND isnotnull(d_date_sk#19)) - -(50) CometProject -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(52) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_moy#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Condition : (((((isnotnull(d_moy#24) AND isnotnull(d_year#23)) AND (d_moy#24 >= 9)) AND (d_moy#24 <= 12)) AND (d_year#23 = 1999)) AND isnotnull(d_date_sk#22)) - -(55) CometProject -Input [3]: [d_date_sk#22, d_year#23, d_moy#24] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(56) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(57) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:3 Hosting operator id = 8 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 -BroadcastExchange (62) -+- * CometColumnarToRow (61) - +- CometProject (60) - +- CometFilter (59) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) - - -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(59) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (d_year#26 IN (1999,2000,2001) AND isnotnull(d_date_sk#25)) - -(60) CometProject -Input [2]: [d_date_sk#25, d_year#26] -Arguments: [d_date_sk#25], [d_date_sk#25] - -(61) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#25] - -(62) BroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/extended.txt deleted file mode 100644 index fe03fef8a7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/extended.txt +++ /dev/null @@ -1,67 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 15b992d3a4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] - CometHashAggregate [sum,sum,sum] [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity)] - CometExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 - CometHashAggregate [ss_quantity,sr_return_quantity,cs_quantity] [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] - CometProject [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] - 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_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 [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 [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_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 [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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #8 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #9 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #10 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc] #11 - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc] - CometFilter [i_item_sk,i_item_id,i_item_desc] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_datafusion/explain.txt deleted file mode 100644 index 9a5ab78af8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_datafusion/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometNativeScan parquet spark_catalog.default.item (9) - - -(1) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((isnotnull(d_moy#3) AND (d_moy#3 = 11)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometNativeScan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,128), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Condition : ((isnotnull(i_manufact_id#10) AND (i_manufact_id#10 = 128)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,sum_agg#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,sum_agg#15]), [d_year#2, brand_id#13, brand#14, sum_agg#15], 100, 0, [d_year#2 ASC NULLS FIRST, sum_agg#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, sum_agg#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_datafusion/simplified.txt deleted file mode 100644 index c19e6caf52..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_datafusion/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,sum_agg] - CometHashAggregate [sum] [d_year,brand_id,brand,sum_agg,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] - CometNativeScan 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/q3.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_iceberg_compat/explain.txt deleted file mode 100644 index 6828b7a3aa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((isnotnull(d_moy#3) AND (d_moy#3 = 11)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,128), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Condition : ((isnotnull(i_manufact_id#10) AND (i_manufact_id#10 = 128)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,sum_agg#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,sum_agg#15]), [d_year#2, brand_id#13, brand#14, sum_agg#15], 100, 0, [d_year#2 ASC NULLS FIRST, sum_agg#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, sum_agg#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_iceberg_compat/simplified.txt deleted file mode 100644 index 7bdad4b94f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,sum_agg] - CometHashAggregate [sum] [d_year,brand_id,brand,sum_agg,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] - CometScan [native_iceberg_compat] 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/q3/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt deleted file mode 100644 index 6828b7a3aa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((isnotnull(d_moy#3) AND (d_moy#3 = 11)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,128), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Condition : ((isnotnull(i_manufact_id#10) AND (i_manufact_id#10 = 128)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,sum_agg#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,sum_agg#15]), [d_year#2, brand_id#13, brand#14, sum_agg#15], 100, 0, [d_year#2 ASC NULLS FIRST, sum_agg#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, sum_agg#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/extended.txt deleted file mode 100644 index a292badf5d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 7bdad4b94f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,sum_agg] - CometHashAggregate [sum] [d_year,brand_id,brand,sum_agg,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manufact_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/explain.txt deleted file mode 100644 index 7992727538..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/explain.txt +++ /dev/null @@ -1,346 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (54) -+- * Project (53) - +- * BroadcastHashJoin Inner BuildRight (52) - :- * Project (46) - : +- * BroadcastHashJoin Inner BuildRight (45) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- * HashAggregate (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.web_returns (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (11) - : : : +- * CometColumnarToRow (10) - : : : +- CometProject (9) - : : : +- CometFilter (8) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) - : : +- BroadcastExchange (37) - : : +- * Filter (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * HashAggregate (31) - : : +- * CometColumnarToRow (30) - : : +- CometColumnarExchange (29) - : : +- * HashAggregate (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (24) - : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : :- * Filter (21) - : : : : +- * ColumnarToRow (20) - : : : : +- Scan parquet spark_catalog.default.web_returns (19) - : : : +- ReusedExchange (22) - : : +- ReusedExchange (25) - : +- BroadcastExchange (44) - : +- * CometColumnarToRow (43) - : +- CometProject (42) - : +- CometFilter (41) - : +- CometNativeScan parquet spark_catalog.default.customer (40) - +- BroadcastExchange (51) - +- * CometColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometNativeScan parquet spark_catalog.default.customer_address (47) - - -(1) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_returning_addr_sk), IsNotNull(wr_returning_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] - -(3) Filter [codegen id : 3] -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Condition : (isnotnull(wr_returning_addr_sk#2) AND isnotnull(wr_returning_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [wr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_state#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_state#8] -Condition : (isnotnull(ca_address_sk#7) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#8, 2, true, false, true))) - -(9) CometProject -Input [2]: [ca_address_sk#7, ca_state#8] -Arguments: [ca_address_sk#7, ca_state#9], [ca_address_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#8, 2, true, false, true) AS ca_state#9] - -(10) CometColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#7, ca_state#9] - -(11) BroadcastExchange -Input [2]: [ca_address_sk#7, ca_state#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [wr_returning_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#7, ca_state#9] - -(14) HashAggregate [codegen id : 3] -Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] -Keys [2]: [wr_returning_customer_sk#1, ca_state#9] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum#10] -Results [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] - -(15) CometColumnarExchange -Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 11] -Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] - -(17) HashAggregate [codegen id : 11] -Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#11] -Keys [2]: [wr_returning_customer_sk#1, ca_state#9] -Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#12] -Results [3]: [wr_returning_customer_sk#1 AS ctr_customer_sk#13, ca_state#9 AS ctr_state#14, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#12,17,2) AS ctr_total_return#15] - -(18) Filter [codegen id : 11] -Input [3]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15] -Condition : isnotnull(ctr_total_return#15) - -(19) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_returning_addr_sk)] -ReadSchema: struct - -(20) ColumnarToRow [codegen id : 6] -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] - -(21) Filter [codegen id : 6] -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Condition : isnotnull(wr_returning_addr_sk#2) - -(22) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#6] - -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6] - -(25) ReusedExchange [Reuses operator id: 11] -Output [2]: [ca_address_sk#7, ca_state#9] - -(26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returning_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#7, ca_state#9] - -(28) HashAggregate [codegen id : 6] -Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#9] -Keys [2]: [wr_returning_customer_sk#1, ca_state#9] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum#16] -Results [3]: [wr_returning_customer_sk#1, ca_state#9, sum#17] - -(29) CometColumnarExchange -Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#17] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(30) CometColumnarToRow [codegen id : 7] -Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#17] - -(31) HashAggregate [codegen id : 7] -Input [3]: [wr_returning_customer_sk#1, ca_state#9, sum#17] -Keys [2]: [wr_returning_customer_sk#1, ca_state#9] -Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#12] -Results [2]: [ca_state#9 AS ctr_state#14, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#12,17,2) AS ctr_total_return#15] - -(32) HashAggregate [codegen id : 7] -Input [2]: [ctr_state#14, ctr_total_return#15] -Keys [1]: [ctr_state#14] -Functions [1]: [partial_avg(ctr_total_return#15)] -Aggregate Attributes [2]: [sum#18, count#19] -Results [3]: [ctr_state#14, sum#20, count#21] - -(33) CometColumnarExchange -Input [3]: [ctr_state#14, sum#20, count#21] -Arguments: hashpartitioning(ctr_state#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 8] -Input [3]: [ctr_state#14, sum#20, count#21] - -(35) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#14, sum#20, count#21] -Keys [1]: [ctr_state#14] -Functions [1]: [avg(ctr_total_return#15)] -Aggregate Attributes [1]: [avg(ctr_total_return#15)#22] -Results [2]: [(avg(ctr_total_return#15)#22 * 1.2) AS (avg(ctr_total_return) * 1.2)#23, ctr_state#14 AS ctr_state#14#24] - -(36) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#14#24] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) - -(37) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#14#24] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] - -(38) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_state#14] -Right keys [1]: [ctr_state#14#24] -Join type: Inner -Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#23) - -(39) Project [codegen id : 11] -Output [2]: [ctr_customer_sk#13, ctr_total_return#15] -Input [5]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#23, ctr_state#14#24] - -(40) CometNativeScan parquet spark_catalog.default.customer -Output [14]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(41) CometFilter -Input [14]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_current_addr_sk#27)) - -(42) CometProject -Input [14]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#36, c_email_address#37, c_last_review_date#38] -Arguments: [c_customer_sk#25, c_customer_id#39, c_current_addr_sk#27, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#44, c_email_address#45, c_last_review_date#38], [c_customer_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true) AS c_customer_id#39, c_current_addr_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#28, 10, true, false, true) AS c_salutation#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#29, 20, true, false, true) AS c_first_name#41, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#30, 30, true, false, true) AS c_last_name#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#31, 1, true, false, true) AS c_preferred_cust_flag#43, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#36, 13, true, false, true) AS c_login#44, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#37, 50, true, false, true) AS c_email_address#45, c_last_review_date#38] - -(43) CometColumnarToRow [codegen id : 9] -Input [14]: [c_customer_sk#25, c_customer_id#39, c_current_addr_sk#27, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#44, c_email_address#45, c_last_review_date#38] - -(44) BroadcastExchange -Input [14]: [c_customer_sk#25, c_customer_id#39, c_current_addr_sk#27, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#44, c_email_address#45, c_last_review_date#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(45) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_customer_sk#13] -Right keys [1]: [c_customer_sk#25] -Join type: Inner -Join condition: None - -(46) Project [codegen id : 11] -Output [14]: [ctr_total_return#15, c_customer_id#39, c_current_addr_sk#27, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#44, c_email_address#45, c_last_review_date#38] -Input [16]: [ctr_customer_sk#13, ctr_total_return#15, c_customer_sk#25, c_customer_id#39, c_current_addr_sk#27, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#44, c_email_address#45, c_last_review_date#38] - -(47) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#46, ca_state#47] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(48) CometFilter -Input [2]: [ca_address_sk#46, ca_state#47] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#47, 2, true, false, true) = GA) AND isnotnull(ca_address_sk#46)) - -(49) CometProject -Input [2]: [ca_address_sk#46, ca_state#47] -Arguments: [ca_address_sk#46], [ca_address_sk#46] - -(50) CometColumnarToRow [codegen id : 10] -Input [1]: [ca_address_sk#46] - -(51) BroadcastExchange -Input [1]: [ca_address_sk#46] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(52) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#27] -Right keys [1]: [ca_address_sk#46] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 11] -Output [13]: [c_customer_id#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#44, c_email_address#45, c_last_review_date#38, ctr_total_return#15] -Input [15]: [ctr_total_return#15, c_customer_id#39, c_current_addr_sk#27, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#44, c_email_address#45, c_last_review_date#38, ca_address_sk#46] - -(54) TakeOrderedAndProject -Input [13]: [c_customer_id#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#44, c_email_address#45, c_last_review_date#38, ctr_total_return#15] -Arguments: 100, [c_customer_id#39 ASC NULLS FIRST, c_salutation#40 ASC NULLS FIRST, c_first_name#41 ASC NULLS FIRST, c_last_name#42 ASC NULLS FIRST, c_preferred_cust_flag#43 ASC NULLS FIRST, c_birth_day#32 ASC NULLS FIRST, c_birth_month#33 ASC NULLS FIRST, c_birth_year#34 ASC NULLS FIRST, c_birth_country#35 ASC NULLS FIRST, c_login#44 ASC NULLS FIRST, c_email_address#45 ASC NULLS FIRST, c_last_review_date#38 ASC NULLS FIRST, ctr_total_return#15 ASC NULLS FIRST], [c_customer_id#39, c_salutation#40, c_first_name#41, c_last_name#42, c_preferred_cust_flag#43, c_birth_day#32, c_birth_month#33, c_birth_year#34, c_birth_country#35, c_login#44, c_email_address#45, c_last_review_date#38, ctr_total_return#15] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (59) -+- * CometColumnarToRow (58) - +- CometProject (57) - +- CometFilter (56) - +- CometNativeScan parquet spark_catalog.default.date_dim (55) - - -(55) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date_sk#6, d_year#48] -Condition : ((isnotnull(d_year#48) AND (d_year#48 = 2002)) AND isnotnull(d_date_sk#6)) - -(57) CometProject -Input [2]: [d_date_sk#6, d_year#48] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(58) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(59) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 19 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/simplified.txt deleted file mode 100644 index b7a7ece27e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/simplified.txt +++ /dev/null @@ -1,86 +0,0 @@ -TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - WholeStageCodegen (11) - Project [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (3) - HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] - Project [wr_returning_customer_sk,wr_return_amt,ca_state] - BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] - Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Filter [wr_returning_addr_sk,wr_returning_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (8) - Filter [(avg(ctr_total_return) * 1.2)] - HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),ctr_state,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ctr_state] #5 - WholeStageCodegen (7) - HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] - HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wr_returning_customer_sk,ca_state] #6 - WholeStageCodegen (6) - HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] - Project [wr_returning_customer_sk,wr_return_amt,ca_state] - BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] - Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Filter [wr_returning_addr_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [ca_address_sk,ca_state] #3 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [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] - 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] - CometNativeScan 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 #8 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan 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/q30.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/explain.txt deleted file mode 100644 index 44c6fdad28..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,318 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (50) -+- CometTakeOrderedAndProject (49) - +- CometProject (48) - +- CometBroadcastHashJoin (47) - :- CometProject (42) - : +- CometBroadcastHashJoin (41) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometFilter (18) - : : : +- CometHashAggregate (17) - : : : +- CometExchange (16) - : : : +- CometHashAggregate (15) - : : : +- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : : +- CometBroadcastExchange (34) - : : +- CometFilter (33) - : : +- CometHashAggregate (32) - : : +- CometExchange (31) - : : +- CometHashAggregate (30) - : : +- CometHashAggregate (29) - : : +- CometExchange (28) - : : +- CometHashAggregate (27) - : : +- CometProject (26) - : : +- CometBroadcastHashJoin (25) - : : :- CometProject (23) - : : : +- CometBroadcastHashJoin (22) - : : : :- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (19) - : : : +- ReusedExchange (21) - : : +- ReusedExchange (24) - : +- CometBroadcastExchange (40) - : +- CometProject (39) - : +- CometFilter (38) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) - +- CometBroadcastExchange (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (43) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_returning_addr_sk), IsNotNull(wr_returning_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Condition : (isnotnull(wr_returning_addr_sk#2) AND isnotnull(wr_returning_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [wr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6] -Arguments: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3], [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#8, ca_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#8, ca_state#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#9, 2, true, false, true))) - -(11) CometProject -Input [2]: [ca_address_sk#8, ca_state#9] -Arguments: [ca_address_sk#8, ca_state#10], [ca_address_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#9, 2, true, false, true) AS ca_state#10] - -(12) CometBroadcastExchange -Input [2]: [ca_address_sk#8, ca_state#10] -Arguments: [ca_address_sk#8, ca_state#10] - -(13) CometBroadcastHashJoin -Left output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] -Right output [2]: [ca_address_sk#8, ca_state#10] -Arguments: [wr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight - -(14) CometProject -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#8, ca_state#10] -Arguments: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10], [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] - -(15) CometHashAggregate -Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] -Keys [2]: [wr_returning_customer_sk#1, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] - -(16) CometExchange -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#11] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#11] -Keys [2]: [wr_returning_customer_sk#1, ca_state#10] -Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] - -(18) CometFilter -Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Condition : isnotnull(ctr_total_return#14) - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(wr_returning_addr_sk)] -ReadSchema: struct - -(20) CometFilter -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Condition : isnotnull(wr_returning_addr_sk#2) - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#6] - -(22) CometBroadcastHashJoin -Left output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [wr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(23) CometProject -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6] -Arguments: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3], [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] - -(24) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#8, ca_state#10] - -(25) CometBroadcastHashJoin -Left output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] -Right output [2]: [ca_address_sk#8, ca_state#10] -Arguments: [wr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight - -(26) CometProject -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#8, ca_state#10] -Arguments: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10], [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] - -(27) CometHashAggregate -Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] -Keys [2]: [wr_returning_customer_sk#1, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] - -(28) CometExchange -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#16] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#16] -Keys [2]: [wr_returning_customer_sk#1, ca_state#10] -Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] - -(30) CometHashAggregate -Input [2]: [ctr_state#13, ctr_total_return#14] -Keys [1]: [ctr_state#13] -Functions [1]: [partial_avg(ctr_total_return#14)] - -(31) CometExchange -Input [3]: [ctr_state#13, sum#17, count#18] -Arguments: hashpartitioning(ctr_state#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometHashAggregate -Input [3]: [ctr_state#13, sum#17, count#18] -Keys [1]: [ctr_state#13] -Functions [1]: [avg(ctr_total_return#14)] - -(33) CometFilter -Input [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#13#20] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#19) - -(34) CometBroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#13#20] -Arguments: [(avg(ctr_total_return) * 1.2)#19, ctr_state#13#20] - -(35) CometBroadcastHashJoin -Left output [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Right output [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#13#20] -Arguments: [ctr_state#13], [ctr_state#13#20], Inner, (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#19), BuildRight - -(36) CometProject -Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#19, ctr_state#13#20] -Arguments: [ctr_customer_sk#12, ctr_total_return#14], [ctr_customer_sk#12, ctr_total_return#14] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [14]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(38) CometFilter -Input [14]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34] -Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_current_addr_sk#23)) - -(39) CometProject -Input [14]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34] -Arguments: [c_customer_sk#21, c_customer_id#35, c_current_addr_sk#23, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34], [c_customer_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#22, 16, true, false, true) AS c_customer_id#35, c_current_addr_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#24, 10, true, false, true) AS c_salutation#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#25, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#26, 30, true, false, true) AS c_last_name#38, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#27, 1, true, false, true) AS c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#32, 13, true, false, true) AS c_login#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#33, 50, true, false, true) AS c_email_address#41, c_last_review_date#34] - -(40) CometBroadcastExchange -Input [14]: [c_customer_sk#21, c_customer_id#35, c_current_addr_sk#23, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34] -Arguments: [c_customer_sk#21, c_customer_id#35, c_current_addr_sk#23, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34] - -(41) CometBroadcastHashJoin -Left output [2]: [ctr_customer_sk#12, ctr_total_return#14] -Right output [14]: [c_customer_sk#21, c_customer_id#35, c_current_addr_sk#23, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34] -Arguments: [ctr_customer_sk#12], [c_customer_sk#21], Inner, BuildRight - -(42) CometProject -Input [16]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#21, c_customer_id#35, c_current_addr_sk#23, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34] -Arguments: [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#23, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34], [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#23, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#42, ca_state#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(44) CometFilter -Input [2]: [ca_address_sk#42, ca_state#43] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#43, 2, true, false, true) = GA) AND isnotnull(ca_address_sk#42)) - -(45) CometProject -Input [2]: [ca_address_sk#42, ca_state#43] -Arguments: [ca_address_sk#42], [ca_address_sk#42] - -(46) CometBroadcastExchange -Input [1]: [ca_address_sk#42] -Arguments: [ca_address_sk#42] - -(47) CometBroadcastHashJoin -Left output [14]: [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#23, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34] -Right output [1]: [ca_address_sk#42] -Arguments: [c_current_addr_sk#23], [ca_address_sk#42], Inner, BuildRight - -(48) CometProject -Input [15]: [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#23, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34, ca_address_sk#42] -Arguments: [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34, ctr_total_return#14], [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34, ctr_total_return#14] - -(49) CometTakeOrderedAndProject -Input [13]: [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34, ctr_total_return#14] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#35 ASC NULLS FIRST,c_salutation#36 ASC NULLS FIRST,c_first_name#37 ASC NULLS FIRST,c_last_name#38 ASC NULLS FIRST,c_preferred_cust_flag#39 ASC NULLS FIRST,c_birth_day#28 ASC NULLS FIRST,c_birth_month#29 ASC NULLS FIRST,c_birth_year#30 ASC NULLS FIRST,c_birth_country#31 ASC NULLS FIRST,c_login#40 ASC NULLS FIRST,c_email_address#41 ASC NULLS FIRST,c_last_review_date#34 ASC NULLS FIRST,ctr_total_return#14 ASC NULLS FIRST], output=[c_customer_id#35,c_salutation#36,c_first_name#37,c_last_name#38,c_preferred_cust_flag#39,c_birth_day#28,c_birth_month#29,c_birth_year#30,c_birth_country#31,c_login#40,c_email_address#41,c_last_review_date#34,ctr_total_return#14]), [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34, ctr_total_return#14], 100, 0, [c_customer_id#35 ASC NULLS FIRST, c_salutation#36 ASC NULLS FIRST, c_first_name#37 ASC NULLS FIRST, c_last_name#38 ASC NULLS FIRST, c_preferred_cust_flag#39 ASC NULLS FIRST, c_birth_day#28 ASC NULLS FIRST, c_birth_month#29 ASC NULLS FIRST, c_birth_year#30 ASC NULLS FIRST, c_birth_country#31 ASC NULLS FIRST, c_login#40 ASC NULLS FIRST, c_email_address#41 ASC NULLS FIRST, c_last_review_date#34 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34, ctr_total_return#14] - -(50) CometColumnarToRow [codegen id : 1] -Input [13]: [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34, ctr_total_return#14] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) - - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(52) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) - -(53) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(54) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(55) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 19 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/simplified.txt deleted file mode 100644 index da5ca97c85..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - CometBroadcastHashJoin [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ca_address_sk] - CometProject [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - CometBroadcastHashJoin [ctr_customer_sk,ctr_total_return,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] - CometProject [ctr_customer_sk,ctr_total_return] - CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] - CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] - CometHashAggregate [sum] [ctr_customer_sk,ctr_state,ctr_total_return,wr_returning_customer_sk,ca_state,sum(UnscaledValue(wr_return_amt))] - CometExchange [wr_returning_customer_sk,ca_state] #1 - CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] - CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 - CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] - CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_state,avg(ctr_total_return),ctr_state] - CometExchange [ctr_state] #6 - CometHashAggregate [ctr_total_return] [ctr_state,sum,count] - CometHashAggregate [wr_returning_customer_sk,sum] [ctr_state,ctr_total_return,ca_state,sum(UnscaledValue(wr_return_amt))] - CometExchange [wr_returning_customer_sk,ca_state] #7 - CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] - CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 - ReusedExchange [ca_address_sk,ca_state] #4 - CometBroadcastExchange [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] #8 - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [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] - 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 [native_iceberg_compat] 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] - CometBroadcastExchange [ca_address_sk] #9 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] 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/q30/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt deleted file mode 100644 index 44c6fdad28..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt +++ /dev/null @@ -1,318 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (50) -+- CometTakeOrderedAndProject (49) - +- CometProject (48) - +- CometBroadcastHashJoin (47) - :- CometProject (42) - : +- CometBroadcastHashJoin (41) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometFilter (18) - : : : +- CometHashAggregate (17) - : : : +- CometExchange (16) - : : : +- CometHashAggregate (15) - : : : +- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : : +- CometBroadcastExchange (34) - : : +- CometFilter (33) - : : +- CometHashAggregate (32) - : : +- CometExchange (31) - : : +- CometHashAggregate (30) - : : +- CometHashAggregate (29) - : : +- CometExchange (28) - : : +- CometHashAggregate (27) - : : +- CometProject (26) - : : +- CometBroadcastHashJoin (25) - : : :- CometProject (23) - : : : +- CometBroadcastHashJoin (22) - : : : :- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (19) - : : : +- ReusedExchange (21) - : : +- ReusedExchange (24) - : +- CometBroadcastExchange (40) - : +- CometProject (39) - : +- CometFilter (38) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) - +- CometBroadcastExchange (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (43) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_returning_addr_sk), IsNotNull(wr_returning_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Condition : (isnotnull(wr_returning_addr_sk#2) AND isnotnull(wr_returning_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [wr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6] -Arguments: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3], [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#8, ca_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#8, ca_state#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#9, 2, true, false, true))) - -(11) CometProject -Input [2]: [ca_address_sk#8, ca_state#9] -Arguments: [ca_address_sk#8, ca_state#10], [ca_address_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#9, 2, true, false, true) AS ca_state#10] - -(12) CometBroadcastExchange -Input [2]: [ca_address_sk#8, ca_state#10] -Arguments: [ca_address_sk#8, ca_state#10] - -(13) CometBroadcastHashJoin -Left output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] -Right output [2]: [ca_address_sk#8, ca_state#10] -Arguments: [wr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight - -(14) CometProject -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#8, ca_state#10] -Arguments: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10], [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] - -(15) CometHashAggregate -Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] -Keys [2]: [wr_returning_customer_sk#1, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] - -(16) CometExchange -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#11] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#11] -Keys [2]: [wr_returning_customer_sk#1, ca_state#10] -Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] - -(18) CometFilter -Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Condition : isnotnull(ctr_total_return#14) - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(wr_returning_addr_sk)] -ReadSchema: struct - -(20) CometFilter -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Condition : isnotnull(wr_returning_addr_sk#2) - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#6] - -(22) CometBroadcastHashJoin -Left output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [wr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(23) CometProject -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4, d_date_sk#6] -Arguments: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3], [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] - -(24) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#8, ca_state#10] - -(25) CometBroadcastHashJoin -Left output [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] -Right output [2]: [ca_address_sk#8, ca_state#10] -Arguments: [wr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight - -(26) CometProject -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#8, ca_state#10] -Arguments: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10], [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] - -(27) CometHashAggregate -Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] -Keys [2]: [wr_returning_customer_sk#1, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] - -(28) CometExchange -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#16] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#16] -Keys [2]: [wr_returning_customer_sk#1, ca_state#10] -Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] - -(30) CometHashAggregate -Input [2]: [ctr_state#13, ctr_total_return#14] -Keys [1]: [ctr_state#13] -Functions [1]: [partial_avg(ctr_total_return#14)] - -(31) CometExchange -Input [3]: [ctr_state#13, sum#17, count#18] -Arguments: hashpartitioning(ctr_state#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometHashAggregate -Input [3]: [ctr_state#13, sum#17, count#18] -Keys [1]: [ctr_state#13] -Functions [1]: [avg(ctr_total_return#14)] - -(33) CometFilter -Input [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#13#20] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#19) - -(34) CometBroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#13#20] -Arguments: [(avg(ctr_total_return) * 1.2)#19, ctr_state#13#20] - -(35) CometBroadcastHashJoin -Left output [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Right output [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#13#20] -Arguments: [ctr_state#13], [ctr_state#13#20], Inner, (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#19), BuildRight - -(36) CometProject -Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#19, ctr_state#13#20] -Arguments: [ctr_customer_sk#12, ctr_total_return#14], [ctr_customer_sk#12, ctr_total_return#14] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [14]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(38) CometFilter -Input [14]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34] -Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_current_addr_sk#23)) - -(39) CometProject -Input [14]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26, c_preferred_cust_flag#27, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#32, c_email_address#33, c_last_review_date#34] -Arguments: [c_customer_sk#21, c_customer_id#35, c_current_addr_sk#23, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34], [c_customer_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#22, 16, true, false, true) AS c_customer_id#35, c_current_addr_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#24, 10, true, false, true) AS c_salutation#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#25, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#26, 30, true, false, true) AS c_last_name#38, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#27, 1, true, false, true) AS c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#32, 13, true, false, true) AS c_login#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#33, 50, true, false, true) AS c_email_address#41, c_last_review_date#34] - -(40) CometBroadcastExchange -Input [14]: [c_customer_sk#21, c_customer_id#35, c_current_addr_sk#23, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34] -Arguments: [c_customer_sk#21, c_customer_id#35, c_current_addr_sk#23, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34] - -(41) CometBroadcastHashJoin -Left output [2]: [ctr_customer_sk#12, ctr_total_return#14] -Right output [14]: [c_customer_sk#21, c_customer_id#35, c_current_addr_sk#23, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34] -Arguments: [ctr_customer_sk#12], [c_customer_sk#21], Inner, BuildRight - -(42) CometProject -Input [16]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#21, c_customer_id#35, c_current_addr_sk#23, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34] -Arguments: [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#23, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34], [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#23, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#42, ca_state#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(44) CometFilter -Input [2]: [ca_address_sk#42, ca_state#43] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#43, 2, true, false, true) = GA) AND isnotnull(ca_address_sk#42)) - -(45) CometProject -Input [2]: [ca_address_sk#42, ca_state#43] -Arguments: [ca_address_sk#42], [ca_address_sk#42] - -(46) CometBroadcastExchange -Input [1]: [ca_address_sk#42] -Arguments: [ca_address_sk#42] - -(47) CometBroadcastHashJoin -Left output [14]: [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#23, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34] -Right output [1]: [ca_address_sk#42] -Arguments: [c_current_addr_sk#23], [ca_address_sk#42], Inner, BuildRight - -(48) CometProject -Input [15]: [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#23, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34, ca_address_sk#42] -Arguments: [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34, ctr_total_return#14], [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34, ctr_total_return#14] - -(49) CometTakeOrderedAndProject -Input [13]: [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34, ctr_total_return#14] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#35 ASC NULLS FIRST,c_salutation#36 ASC NULLS FIRST,c_first_name#37 ASC NULLS FIRST,c_last_name#38 ASC NULLS FIRST,c_preferred_cust_flag#39 ASC NULLS FIRST,c_birth_day#28 ASC NULLS FIRST,c_birth_month#29 ASC NULLS FIRST,c_birth_year#30 ASC NULLS FIRST,c_birth_country#31 ASC NULLS FIRST,c_login#40 ASC NULLS FIRST,c_email_address#41 ASC NULLS FIRST,c_last_review_date#34 ASC NULLS FIRST,ctr_total_return#14 ASC NULLS FIRST], output=[c_customer_id#35,c_salutation#36,c_first_name#37,c_last_name#38,c_preferred_cust_flag#39,c_birth_day#28,c_birth_month#29,c_birth_year#30,c_birth_country#31,c_login#40,c_email_address#41,c_last_review_date#34,ctr_total_return#14]), [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34, ctr_total_return#14], 100, 0, [c_customer_id#35 ASC NULLS FIRST, c_salutation#36 ASC NULLS FIRST, c_first_name#37 ASC NULLS FIRST, c_last_name#38 ASC NULLS FIRST, c_preferred_cust_flag#39 ASC NULLS FIRST, c_birth_day#28 ASC NULLS FIRST, c_birth_month#29 ASC NULLS FIRST, c_birth_year#30 ASC NULLS FIRST, c_birth_country#31 ASC NULLS FIRST, c_login#40 ASC NULLS FIRST, c_email_address#41 ASC NULLS FIRST, c_last_review_date#34 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34, ctr_total_return#14] - -(50) CometColumnarToRow [codegen id : 1] -Input [13]: [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#28, c_birth_month#29, c_birth_year#30, c_birth_country#31, c_login#40, c_email_address#41, c_last_review_date#34, ctr_total_return#14] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) - - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(52) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) - -(53) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(54) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(55) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 19 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/extended.txt deleted file mode 100644 index 7ebd69d340..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index da5ca97c85..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - CometBroadcastHashJoin [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ca_address_sk] - CometProject [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - CometBroadcastHashJoin [ctr_customer_sk,ctr_total_return,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] - CometProject [ctr_customer_sk,ctr_total_return] - CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] - CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] - CometHashAggregate [sum] [ctr_customer_sk,ctr_state,ctr_total_return,wr_returning_customer_sk,ca_state,sum(UnscaledValue(wr_return_amt))] - CometExchange [wr_returning_customer_sk,ca_state] #1 - CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] - CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 - CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] - CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_state,avg(ctr_total_return),ctr_state] - CometExchange [ctr_state] #6 - CometHashAggregate [ctr_total_return] [ctr_state,sum,count] - CometHashAggregate [wr_returning_customer_sk,sum] [ctr_state,ctr_total_return,ca_state,sum(UnscaledValue(wr_return_amt))] - CometExchange [wr_returning_customer_sk,ca_state] #7 - CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] - CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 - ReusedExchange [ca_address_sk,ca_state] #4 - CometBroadcastExchange [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] #8 - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [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] - 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 [native_iceberg_compat] 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] - CometBroadcastExchange [ca_address_sk] #9 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/explain.txt deleted file mode 100644 index 9ce8c738e8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/explain.txt +++ /dev/null @@ -1,644 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (97) -+- CometSort (96) - +- CometColumnarExchange (95) - +- * Project (94) - +- * BroadcastHashJoin Inner BuildRight (93) - :- * Project (78) - : +- * BroadcastHashJoin Inner BuildRight (77) - : :- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (47) - : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : :- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * HashAggregate (16) - : : : : : +- * CometColumnarToRow (15) - : : : : : +- CometColumnarExchange (14) - : : : : : +- * HashAggregate (13) - : : : : : +- * Project (12) - : : : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : : : :- * Project (6) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : +- ReusedExchange (4) - : : : : : +- BroadcastExchange (10) - : : : : : +- * CometColumnarToRow (9) - : : : : : +- CometFilter (8) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) - : : : : +- BroadcastExchange (30) - : : : : +- * HashAggregate (29) - : : : : +- * CometColumnarToRow (28) - : : : : +- CometColumnarExchange (27) - : : : : +- * HashAggregate (26) - : : : : +- * Project (25) - : : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : : :- * Project (22) - : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : :- * Filter (19) - : : : : : : +- * ColumnarToRow (18) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (17) - : : : : : +- ReusedExchange (20) - : : : : +- ReusedExchange (23) - : : : +- BroadcastExchange (45) - : : : +- * HashAggregate (44) - : : : +- * CometColumnarToRow (43) - : : : +- CometColumnarExchange (42) - : : : +- * HashAggregate (41) - : : : +- * Project (40) - : : : +- * BroadcastHashJoin Inner BuildRight (39) - : : : :- * Project (37) - : : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : : :- * Filter (34) - : : : : : +- * ColumnarToRow (33) - : : : : : +- Scan parquet spark_catalog.default.store_sales (32) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- BroadcastExchange (61) - : : +- * HashAggregate (60) - : : +- * CometColumnarToRow (59) - : : +- CometColumnarExchange (58) - : : +- * HashAggregate (57) - : : +- * Project (56) - : : +- * BroadcastHashJoin Inner BuildRight (55) - : : :- * Project (53) - : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : :- * Filter (50) - : : : : +- * ColumnarToRow (49) - : : : : +- Scan parquet spark_catalog.default.web_sales (48) - : : : +- ReusedExchange (51) - : : +- ReusedExchange (54) - : +- BroadcastExchange (76) - : +- * HashAggregate (75) - : +- * CometColumnarToRow (74) - : +- CometColumnarExchange (73) - : +- * HashAggregate (72) - : +- * Project (71) - : +- * BroadcastHashJoin Inner BuildRight (70) - : :- * Project (68) - : : +- * BroadcastHashJoin Inner BuildRight (67) - : : :- * Filter (65) - : : : +- * ColumnarToRow (64) - : : : +- Scan parquet spark_catalog.default.web_sales (63) - : : +- ReusedExchange (66) - : +- ReusedExchange (69) - +- BroadcastExchange (92) - +- * HashAggregate (91) - +- * CometColumnarToRow (90) - +- CometColumnarExchange (89) - +- * HashAggregate (88) - +- * Project (87) - +- * BroadcastHashJoin Inner BuildRight (86) - :- * Project (84) - : +- * BroadcastHashJoin Inner BuildRight (83) - : :- * Filter (81) - : : +- * ColumnarToRow (80) - : : +- Scan parquet spark_catalog.default.web_sales (79) - : +- ReusedExchange (82) - +- ReusedExchange (85) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_addr_sk#1) - -(4) ReusedExchange [Reuses operator id: 101] -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6, d_qoy#7] - -(7) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#8, ca_county#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [ca_address_sk#8, ca_county#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_county#9)) - -(9) CometColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#8, ca_county#9] - -(10) BroadcastExchange -Input [2]: [ca_address_sk#8, ca_county#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_addr_sk#1] -Right keys [1]: [ca_address_sk#8] -Join type: Inner -Join condition: None - -(12) Project [codegen id : 3] -Output [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_address_sk#8, ca_county#9] - -(13) HashAggregate [codegen id : 3] -Input [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] -Keys [3]: [ca_county#9, d_qoy#7, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#10] -Results [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] - -(14) CometColumnarExchange -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] -Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(15) CometColumnarToRow [codegen id : 24] -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] - -(16) HashAggregate [codegen id : 24] -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] -Keys [3]: [ca_county#9, d_qoy#7, d_year#6] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#12] -Results [3]: [ca_county#9, d_year#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS store_sales#13] - -(17) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#16), dynamicpruningexpression(ss_sold_date_sk#16 IN dynamicpruning#17)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 6] -Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] - -(19) Filter [codegen id : 6] -Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] -Condition : isnotnull(ss_addr_sk#14) - -(20) ReusedExchange [Reuses operator id: 105] -Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#16] -Right keys [1]: [d_date_sk#18] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [4]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20] -Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16, d_date_sk#18, d_year#19, d_qoy#20] - -(23) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#21, ca_county#22] - -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#14] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] -Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_address_sk#21, ca_county#22] - -(26) HashAggregate [codegen id : 6] -Input [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] -Keys [3]: [ca_county#22, d_qoy#20, d_year#19] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#15))] -Aggregate Attributes [1]: [sum#23] -Results [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] - -(27) CometColumnarExchange -Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] -Arguments: hashpartitioning(ca_county#22, d_qoy#20, d_year#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometColumnarToRow [codegen id : 7] -Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] - -(29) HashAggregate [codegen id : 7] -Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] -Keys [3]: [ca_county#22, d_qoy#20, d_year#19] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#15))#12] -Results [2]: [ca_county#22, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#15))#12,17,2) AS store_sales#25] - -(30) BroadcastExchange -Input [2]: [ca_county#22, store_sales#25] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#9] -Right keys [1]: [ca_county#22] -Join type: Inner -Join condition: None - -(32) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#29)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(33) ColumnarToRow [codegen id : 10] -Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] - -(34) Filter [codegen id : 10] -Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -Condition : isnotnull(ss_addr_sk#26) - -(35) ReusedExchange [Reuses operator id: 109] -Output [3]: [d_date_sk#30, d_year#31, d_qoy#32] - -(36) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#28] -Right keys [1]: [d_date_sk#30] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 10] -Output [4]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32] -Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#30, d_year#31, d_qoy#32] - -(38) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#33, ca_county#34] - -(39) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_addr_sk#26] -Right keys [1]: [ca_address_sk#33] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 10] -Output [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] -Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_address_sk#33, ca_county#34] - -(41) HashAggregate [codegen id : 10] -Input [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] -Keys [3]: [ca_county#34, d_qoy#32, d_year#31] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum#35] -Results [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] - -(42) CometColumnarExchange -Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] -Arguments: hashpartitioning(ca_county#34, d_qoy#32, d_year#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(43) CometColumnarToRow [codegen id : 11] -Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] - -(44) HashAggregate [codegen id : 11] -Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] -Keys [3]: [ca_county#34, d_qoy#32, d_year#31] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#12] -Results [2]: [ca_county#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#12,17,2) AS store_sales#37] - -(45) BroadcastExchange -Input [2]: [ca_county#34, store_sales#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] - -(46) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#22] -Right keys [1]: [ca_county#34] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 24] -Output [5]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37] -Input [7]: [ca_county#9, d_year#6, store_sales#13, ca_county#22, store_sales#25, ca_county#34, store_sales#37] - -(48) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(49) ColumnarToRow [codegen id : 14] -Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] - -(50) Filter [codegen id : 14] -Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -Condition : isnotnull(ws_bill_addr_sk#38) - -(51) ReusedExchange [Reuses operator id: 101] -Output [3]: [d_date_sk#41, d_year#42, d_qoy#43] - -(52) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#41] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 14] -Output [4]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#42, d_qoy#43] -Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41, d_year#42, d_qoy#43] - -(54) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#44, ca_county#45] - -(55) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#38] -Right keys [1]: [ca_address_sk#44] -Join type: Inner -Join condition: None - -(56) Project [codegen id : 14] -Output [4]: [ws_ext_sales_price#39, d_year#42, d_qoy#43, ca_county#45] -Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#42, d_qoy#43, ca_address_sk#44, ca_county#45] - -(57) HashAggregate [codegen id : 14] -Input [4]: [ws_ext_sales_price#39, d_year#42, d_qoy#43, ca_county#45] -Keys [3]: [ca_county#45, d_qoy#43, d_year#42] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum#46] -Results [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] - -(58) CometColumnarExchange -Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] -Arguments: hashpartitioning(ca_county#45, d_qoy#43, d_year#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(59) CometColumnarToRow [codegen id : 15] -Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] - -(60) HashAggregate [codegen id : 15] -Input [4]: [ca_county#45, d_qoy#43, d_year#42, sum#47] -Keys [3]: [ca_county#45, d_qoy#43, d_year#42] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] -Results [2]: [ca_county#45, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS web_sales#49] - -(61) BroadcastExchange -Input [2]: [ca_county#45, web_sales#49] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(62) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#9] -Right keys [1]: [ca_county#45] -Join type: Inner -Join condition: None - -(63) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#17)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(64) ColumnarToRow [codegen id : 18] -Input [3]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] - -(65) Filter [codegen id : 18] -Input [3]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52] -Condition : isnotnull(ws_bill_addr_sk#50) - -(66) ReusedExchange [Reuses operator id: 105] -Output [3]: [d_date_sk#53, d_year#54, d_qoy#55] - -(67) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#52] -Right keys [1]: [d_date_sk#53] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 18] -Output [4]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, d_year#54, d_qoy#55] -Input [6]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, ws_sold_date_sk#52, d_date_sk#53, d_year#54, d_qoy#55] - -(69) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#56, ca_county#57] - -(70) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_bill_addr_sk#50] -Right keys [1]: [ca_address_sk#56] -Join type: Inner -Join condition: None - -(71) Project [codegen id : 18] -Output [4]: [ws_ext_sales_price#51, d_year#54, d_qoy#55, ca_county#57] -Input [6]: [ws_bill_addr_sk#50, ws_ext_sales_price#51, d_year#54, d_qoy#55, ca_address_sk#56, ca_county#57] - -(72) HashAggregate [codegen id : 18] -Input [4]: [ws_ext_sales_price#51, d_year#54, d_qoy#55, ca_county#57] -Keys [3]: [ca_county#57, d_qoy#55, d_year#54] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#51))] -Aggregate Attributes [1]: [sum#58] -Results [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] - -(73) CometColumnarExchange -Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] -Arguments: hashpartitioning(ca_county#57, d_qoy#55, d_year#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(74) CometColumnarToRow [codegen id : 19] -Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] - -(75) HashAggregate [codegen id : 19] -Input [4]: [ca_county#57, d_qoy#55, d_year#54, sum#59] -Keys [3]: [ca_county#57, d_qoy#55, d_year#54] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#51))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#51))#48] -Results [2]: [ca_county#57, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#51))#48,17,2) AS web_sales#60] - -(76) BroadcastExchange -Input [2]: [ca_county#57, web_sales#60] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=10] - -(77) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#45] -Right keys [1]: [ca_county#57] -Join type: Inner -Join condition: (CASE WHEN (web_sales#49 > 0.00) THEN (web_sales#60 / web_sales#49) END > CASE WHEN (store_sales#13 > 0.00) THEN (store_sales#25 / store_sales#13) END) - -(78) Project [codegen id : 24] -Output [8]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#45, web_sales#49, web_sales#60] -Input [9]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#45, web_sales#49, ca_county#57, web_sales#60] - -(79) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#29)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(80) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] - -(81) Filter [codegen id : 22] -Input [3]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_bill_addr_sk#61) - -(82) ReusedExchange [Reuses operator id: 109] -Output [3]: [d_date_sk#64, d_year#65, d_qoy#66] - -(83) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#63] -Right keys [1]: [d_date_sk#64] -Join type: Inner -Join condition: None - -(84) Project [codegen id : 22] -Output [4]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#65, d_qoy#66] -Input [6]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, ws_sold_date_sk#63, d_date_sk#64, d_year#65, d_qoy#66] - -(85) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#67, ca_county#68] - -(86) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_bill_addr_sk#61] -Right keys [1]: [ca_address_sk#67] -Join type: Inner -Join condition: None - -(87) Project [codegen id : 22] -Output [4]: [ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_county#68] -Input [6]: [ws_bill_addr_sk#61, ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_address_sk#67, ca_county#68] - -(88) HashAggregate [codegen id : 22] -Input [4]: [ws_ext_sales_price#62, d_year#65, d_qoy#66, ca_county#68] -Keys [3]: [ca_county#68, d_qoy#66, d_year#65] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#62))] -Aggregate Attributes [1]: [sum#69] -Results [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] - -(89) CometColumnarExchange -Input [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] -Arguments: hashpartitioning(ca_county#68, d_qoy#66, d_year#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(90) CometColumnarToRow [codegen id : 23] -Input [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] - -(91) HashAggregate [codegen id : 23] -Input [4]: [ca_county#68, d_qoy#66, d_year#65, sum#70] -Keys [3]: [ca_county#68, d_qoy#66, d_year#65] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#62))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#62))#48] -Results [2]: [ca_county#68, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#62))#48,17,2) AS web_sales#71] - -(92) BroadcastExchange -Input [2]: [ca_county#68, web_sales#71] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=12] - -(93) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#45] -Right keys [1]: [ca_county#68] -Join type: Inner -Join condition: (CASE WHEN (web_sales#60 > 0.00) THEN (web_sales#71 / web_sales#60) END > CASE WHEN (store_sales#25 > 0.00) THEN (store_sales#37 / store_sales#25) END) - -(94) Project [codegen id : 24] -Output [6]: [ca_county#9, d_year#6, (web_sales#60 / web_sales#49) AS web_q1_q2_increase#72, (store_sales#25 / store_sales#13) AS store_q1_q2_increase#73, (web_sales#71 / web_sales#60) AS web_q2_q3_increase#74, (store_sales#37 / store_sales#25) AS store_q2_q3_increase#75] -Input [10]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#45, web_sales#49, web_sales#60, ca_county#68, web_sales#71] - -(95) CometColumnarExchange -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(96) CometSort -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75] -Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75], [ca_county#9 ASC NULLS FIRST] - -(97) CometColumnarToRow [codegen id : 25] -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#72, store_q1_q2_increase#73, web_q2_q3_increase#74, store_q2_q3_increase#75] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (101) -+- * CometColumnarToRow (100) - +- CometFilter (99) - +- CometNativeScan parquet spark_catalog.default.date_dim (98) - - -(98) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(99) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) - -(100) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] - -(101) BroadcastExchange -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=14] - -Subquery:2 Hosting operator id = 17 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#17 -BroadcastExchange (105) -+- * CometColumnarToRow (104) - +- CometFilter (103) - +- CometNativeScan parquet spark_catalog.default.date_dim (102) - - -(102) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(103) CometFilter -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] -Condition : ((((isnotnull(d_qoy#20) AND isnotnull(d_year#19)) AND (d_qoy#20 = 2)) AND (d_year#19 = 2000)) AND isnotnull(d_date_sk#18)) - -(104) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] - -(105) BroadcastExchange -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] - -Subquery:3 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 -BroadcastExchange (109) -+- * CometColumnarToRow (108) - +- CometFilter (107) - +- CometNativeScan parquet spark_catalog.default.date_dim (106) - - -(106) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#30, d_year#31, d_qoy#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(107) CometFilter -Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] -Condition : ((((isnotnull(d_qoy#32) AND isnotnull(d_year#31)) AND (d_qoy#32 = 3)) AND (d_year#31 = 2000)) AND isnotnull(d_date_sk#30)) - -(108) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] - -(109) BroadcastExchange -Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] - -Subquery:4 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#4 - -Subquery:5 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#17 - -Subquery:6 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#29 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/simplified.txt deleted file mode 100644 index f3a0572081..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/simplified.txt +++ /dev/null @@ -1,166 +0,0 @@ -WholeStageCodegen (25) - CometColumnarToRow - InputAdapter - CometSort [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] - CometColumnarExchange [ca_county] #1 - WholeStageCodegen (24) - Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] - Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - Project [ca_county,d_year,store_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - BroadcastHashJoin [ca_county,ca_county] - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #2 - WholeStageCodegen (3) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_county] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #6 - WholeStageCodegen (6) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #7 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #9 - WholeStageCodegen (10) - HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #10 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #12 - WholeStageCodegen (14) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #3 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (19) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #14 - WholeStageCodegen (18) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #7 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (23) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_county,d_qoy,d_year] #16 - WholeStageCodegen (22) - HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,d_year,d_qoy,ca_county] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_qoy] #10 - InputAdapter - ReusedExchange [ca_address_sk,ca_county] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/explain.txt deleted file mode 100644 index 83d717e295..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,593 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (90) -+- CometSort (89) - +- CometExchange (88) - +- CometProject (87) - +- CometBroadcastHashJoin (86) - :- CometProject (73) - : +- CometBroadcastHashJoin (72) - : :- CometBroadcastHashJoin (59) - : : :- CometProject (46) - : : : +- CometBroadcastHashJoin (45) - : : : :- CometBroadcastHashJoin (30) - : : : : :- CometHashAggregate (15) - : : : : : +- CometExchange (14) - : : : : : +- CometHashAggregate (13) - : : : : : +- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - : : : : +- CometBroadcastExchange (29) - : : : : +- CometHashAggregate (28) - : : : : +- CometExchange (27) - : : : : +- CometHashAggregate (26) - : : : : +- CometProject (25) - : : : : +- CometBroadcastHashJoin (24) - : : : : :- CometProject (22) - : : : : : +- CometBroadcastHashJoin (21) - : : : : : :- CometFilter (17) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (16) - : : : : : +- CometBroadcastExchange (20) - : : : : : +- CometFilter (19) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (18) - : : : : +- ReusedExchange (23) - : : : +- CometBroadcastExchange (44) - : : : +- CometHashAggregate (43) - : : : +- CometExchange (42) - : : : +- CometHashAggregate (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (32) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (31) - : : : : +- CometBroadcastExchange (35) - : : : : +- CometFilter (34) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (33) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (58) - : : +- CometHashAggregate (57) - : : +- CometExchange (56) - : : +- CometHashAggregate (55) - : : +- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometFilter (48) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : : : +- ReusedExchange (49) - : : +- ReusedExchange (52) - : +- CometBroadcastExchange (71) - : +- CometHashAggregate (70) - : +- CometExchange (69) - : +- CometHashAggregate (68) - : +- CometProject (67) - : +- CometBroadcastHashJoin (66) - : :- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometFilter (61) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (60) - : : +- ReusedExchange (62) - : +- ReusedExchange (65) - +- CometBroadcastExchange (85) - +- CometHashAggregate (84) - +- CometExchange (83) - +- CometHashAggregate (82) - +- CometProject (81) - +- CometBroadcastHashJoin (80) - :- CometProject (78) - : +- CometBroadcastHashJoin (77) - : :- CometFilter (75) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (74) - : +- ReusedExchange (76) - +- ReusedExchange (79) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_addr_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) - -(5) CometBroadcastExchange -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [d_date_sk#5, d_year#6, d_qoy#7] - -(6) CometBroadcastHashJoin -Left output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(7) CometProject -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7], [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#8, ca_county#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [ca_address_sk#8, ca_county#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_county#9)) - -(10) CometBroadcastExchange -Input [2]: [ca_address_sk#8, ca_county#9] -Arguments: [ca_address_sk#8, ca_county#9] - -(11) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] -Right output [2]: [ca_address_sk#8, ca_county#9] -Arguments: [ss_addr_sk#1], [ca_address_sk#8], Inner, BuildRight - -(12) CometProject -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_address_sk#8, ca_county#9] -Arguments: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9], [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] - -(13) CometHashAggregate -Input [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] -Keys [3]: [ca_county#9, d_qoy#7, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(14) CometExchange -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10] -Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(15) CometHashAggregate -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10] -Keys [3]: [ca_county#9, d_qoy#7, d_year#6] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(17) CometFilter -Input [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] -Condition : isnotnull(ss_addr_sk#11) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(19) CometFilter -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) - -(20) CometBroadcastExchange -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Arguments: [d_date_sk#15, d_year#16, d_qoy#17] - -(21) CometBroadcastHashJoin -Left output [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] -Right output [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(22) CometProject -Input [6]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_year#16, d_qoy#17] -Arguments: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17], [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17] - -(23) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#18, ca_county#19] - -(24) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17] -Right output [2]: [ca_address_sk#18, ca_county#19] -Arguments: [ss_addr_sk#11], [ca_address_sk#18], Inner, BuildRight - -(25) CometProject -Input [6]: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_address_sk#18, ca_county#19] -Arguments: [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19], [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19] - -(26) CometHashAggregate -Input [4]: [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19] -Keys [3]: [ca_county#19, d_qoy#17, d_year#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#12))] - -(27) CometExchange -Input [4]: [ca_county#19, d_qoy#17, d_year#16, sum#20] -Arguments: hashpartitioning(ca_county#19, d_qoy#17, d_year#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [4]: [ca_county#19, d_qoy#17, d_year#16, sum#20] -Keys [3]: [ca_county#19, d_qoy#17, d_year#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#12))] - -(29) CometBroadcastExchange -Input [2]: [ca_county#19, store_sales#21] -Arguments: [ca_county#19, store_sales#21] - -(30) CometBroadcastHashJoin -Left output [3]: [ca_county#9, d_year#6, store_sales#22] -Right output [2]: [ca_county#19, store_sales#21] -Arguments: [ca_county#9], [ca_county#19], Inner, BuildRight - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_addr_sk#23) - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) - -(35) CometBroadcastExchange -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Arguments: [d_date_sk#27, d_year#28, d_qoy#29] - -(36) CometBroadcastHashJoin -Left output [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] -Right output [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight - -(37) CometProject -Input [6]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_year#28, d_qoy#29] -Arguments: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29], [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29] - -(38) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#30, ca_county#31] - -(39) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29] -Right output [2]: [ca_address_sk#30, ca_county#31] -Arguments: [ss_addr_sk#23], [ca_address_sk#30], Inner, BuildRight - -(40) CometProject -Input [6]: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_address_sk#30, ca_county#31] -Arguments: [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31], [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31] - -(41) CometHashAggregate -Input [4]: [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31] -Keys [3]: [ca_county#31, d_qoy#29, d_year#28] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#24))] - -(42) CometExchange -Input [4]: [ca_county#31, d_qoy#29, d_year#28, sum#32] -Arguments: hashpartitioning(ca_county#31, d_qoy#29, d_year#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(43) CometHashAggregate -Input [4]: [ca_county#31, d_qoy#29, d_year#28, sum#32] -Keys [3]: [ca_county#31, d_qoy#29, d_year#28] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#24))] - -(44) CometBroadcastExchange -Input [2]: [ca_county#31, store_sales#33] -Arguments: [ca_county#31, store_sales#33] - -(45) CometBroadcastHashJoin -Left output [5]: [ca_county#9, d_year#6, store_sales#22, ca_county#19, store_sales#21] -Right output [2]: [ca_county#31, store_sales#33] -Arguments: [ca_county#19], [ca_county#31], Inner, BuildRight - -(46) CometProject -Input [7]: [ca_county#9, d_year#6, store_sales#22, ca_county#19, store_sales#21, ca_county#31, store_sales#33] -Arguments: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33], [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#37)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(48) CometFilter -Input [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_bill_addr_sk#34) - -(49) ReusedExchange [Reuses operator id: 5] -Output [3]: [d_date_sk#38, d_year#39, d_qoy#40] - -(50) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Right output [3]: [d_date_sk#38, d_year#39, d_qoy#40] -Arguments: [ws_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight - -(51) CometProject -Input [6]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#38, d_year#39, d_qoy#40] -Arguments: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40], [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40] - -(52) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#41, ca_county#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40] -Right output [2]: [ca_address_sk#41, ca_county#42] -Arguments: [ws_bill_addr_sk#34], [ca_address_sk#41], Inner, BuildRight - -(54) CometProject -Input [6]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_address_sk#41, ca_county#42] -Arguments: [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42], [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42] - -(55) CometHashAggregate -Input [4]: [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42] -Keys [3]: [ca_county#42, d_qoy#40, d_year#39] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] - -(56) CometExchange -Input [4]: [ca_county#42, d_qoy#40, d_year#39, sum#43] -Arguments: hashpartitioning(ca_county#42, d_qoy#40, d_year#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(57) CometHashAggregate -Input [4]: [ca_county#42, d_qoy#40, d_year#39, sum#43] -Keys [3]: [ca_county#42, d_qoy#40, d_year#39] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] - -(58) CometBroadcastExchange -Input [2]: [ca_county#42, web_sales#44] -Arguments: [ca_county#42, web_sales#44] - -(59) CometBroadcastHashJoin -Left output [5]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33] -Right output [2]: [ca_county#42, web_sales#44] -Arguments: [ca_county#9], [ca_county#42], Inner, BuildRight - -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#48)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(61) CometFilter -Input [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] -Condition : isnotnull(ws_bill_addr_sk#45) - -(62) ReusedExchange [Reuses operator id: 20] -Output [3]: [d_date_sk#49, d_year#50, d_qoy#51] - -(63) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] -Right output [3]: [d_date_sk#49, d_year#50, d_qoy#51] -Arguments: [ws_sold_date_sk#47], [d_date_sk#49], Inner, BuildRight - -(64) CometProject -Input [6]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47, d_date_sk#49, d_year#50, d_qoy#51] -Arguments: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51], [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51] - -(65) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#52, ca_county#53] - -(66) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51] -Right output [2]: [ca_address_sk#52, ca_county#53] -Arguments: [ws_bill_addr_sk#45], [ca_address_sk#52], Inner, BuildRight - -(67) CometProject -Input [6]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_address_sk#52, ca_county#53] -Arguments: [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53], [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53] - -(68) CometHashAggregate -Input [4]: [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53] -Keys [3]: [ca_county#53, d_qoy#51, d_year#50] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#46))] - -(69) CometExchange -Input [4]: [ca_county#53, d_qoy#51, d_year#50, sum#54] -Arguments: hashpartitioning(ca_county#53, d_qoy#51, d_year#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(70) CometHashAggregate -Input [4]: [ca_county#53, d_qoy#51, d_year#50, sum#54] -Keys [3]: [ca_county#53, d_qoy#51, d_year#50] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#46))] - -(71) CometBroadcastExchange -Input [2]: [ca_county#53, web_sales#55] -Arguments: [ca_county#53, web_sales#55] - -(72) CometBroadcastHashJoin -Left output [7]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44] -Right output [2]: [ca_county#53, web_sales#55] -Arguments: [ca_county#42], [ca_county#53], Inner, (CASE WHEN (web_sales#44 > 0.00) THEN (web_sales#55 / web_sales#44) END > CASE WHEN (store_sales#22 > 0.00) THEN (store_sales#21 / store_sales#22) END), BuildRight - -(73) CometProject -Input [9]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, ca_county#53, web_sales#55] -Arguments: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55], [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55] - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(75) CometFilter -Input [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] -Condition : isnotnull(ws_bill_addr_sk#56) - -(76) ReusedExchange [Reuses operator id: 35] -Output [3]: [d_date_sk#60, d_year#61, d_qoy#62] - -(77) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] -Right output [3]: [d_date_sk#60, d_year#61, d_qoy#62] -Arguments: [ws_sold_date_sk#58], [d_date_sk#60], Inner, BuildRight - -(78) CometProject -Input [6]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58, d_date_sk#60, d_year#61, d_qoy#62] -Arguments: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62], [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62] - -(79) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#63, ca_county#64] - -(80) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62] -Right output [2]: [ca_address_sk#63, ca_county#64] -Arguments: [ws_bill_addr_sk#56], [ca_address_sk#63], Inner, BuildRight - -(81) CometProject -Input [6]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_address_sk#63, ca_county#64] -Arguments: [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64], [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64] - -(82) CometHashAggregate -Input [4]: [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64] -Keys [3]: [ca_county#64, d_qoy#62, d_year#61] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#57))] - -(83) CometExchange -Input [4]: [ca_county#64, d_qoy#62, d_year#61, sum#65] -Arguments: hashpartitioning(ca_county#64, d_qoy#62, d_year#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(84) CometHashAggregate -Input [4]: [ca_county#64, d_qoy#62, d_year#61, sum#65] -Keys [3]: [ca_county#64, d_qoy#62, d_year#61] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#57))] - -(85) CometBroadcastExchange -Input [2]: [ca_county#64, web_sales#66] -Arguments: [ca_county#64, web_sales#66] - -(86) CometBroadcastHashJoin -Left output [8]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55] -Right output [2]: [ca_county#64, web_sales#66] -Arguments: [ca_county#42], [ca_county#64], Inner, (CASE WHEN (web_sales#55 > 0.00) THEN (web_sales#66 / web_sales#55) END > CASE WHEN (store_sales#21 > 0.00) THEN (store_sales#33 / store_sales#21) END), BuildRight - -(87) CometProject -Input [10]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55, ca_county#64, web_sales#66] -Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9, d_year#6, (web_sales#55 / web_sales#44) AS web_q1_q2_increase#67, (store_sales#21 / store_sales#22) AS store_q1_q2_increase#68, (web_sales#66 / web_sales#55) AS web_q2_q3_increase#69, (store_sales#33 / store_sales#21) AS store_q2_q3_increase#70] - -(88) CometExchange -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(89) CometSort -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] -Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9 ASC NULLS FIRST] - -(90) CometColumnarToRow [codegen id : 1] -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (94) -+- * CometColumnarToRow (93) - +- CometFilter (92) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (91) - - -(91) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(92) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) - -(93) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] - -(94) BroadcastExchange -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (98) -+- * CometColumnarToRow (97) - +- CometFilter (96) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (95) - - -(95) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(96) CometFilter -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) - -(97) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] - -(98) BroadcastExchange -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -Subquery:3 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 -BroadcastExchange (102) -+- * CometColumnarToRow (101) - +- CometFilter (100) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (99) - - -(99) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(100) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) - -(101) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] - -(102) BroadcastExchange -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:4 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#4 - -Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#14 - -Subquery:6 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#26 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/simplified.txt deleted file mode 100644 index 2f97384320..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,116 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] - CometExchange [ca_county] #1 - CometProject [web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] - CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales,ca_county,web_sales] - CometProject [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,ca_county,web_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales] - CometProject [ca_county,d_year,store_sales,store_sales,store_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales,ca_county,store_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales] - CometHashAggregate [d_qoy,sum] [ca_county,d_year,store_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #2 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [ca_address_sk,ca_county] #5 - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange [ca_county,store_sales] #6 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,store_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #7 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,store_sales] #10 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,store_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #11 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,web_sales] #14 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #15 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_year,d_qoy] #4 - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,web_sales] #16 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #17 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [d_date_sk,d_year,d_qoy] #9 - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,web_sales] #18 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #19 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [d_date_sk,d_year,d_qoy] #13 - ReusedExchange [ca_address_sk,ca_county] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt deleted file mode 100644 index 83d717e295..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt +++ /dev/null @@ -1,593 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (90) -+- CometSort (89) - +- CometExchange (88) - +- CometProject (87) - +- CometBroadcastHashJoin (86) - :- CometProject (73) - : +- CometBroadcastHashJoin (72) - : :- CometBroadcastHashJoin (59) - : : :- CometProject (46) - : : : +- CometBroadcastHashJoin (45) - : : : :- CometBroadcastHashJoin (30) - : : : : :- CometHashAggregate (15) - : : : : : +- CometExchange (14) - : : : : : +- CometHashAggregate (13) - : : : : : +- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - : : : : +- CometBroadcastExchange (29) - : : : : +- CometHashAggregate (28) - : : : : +- CometExchange (27) - : : : : +- CometHashAggregate (26) - : : : : +- CometProject (25) - : : : : +- CometBroadcastHashJoin (24) - : : : : :- CometProject (22) - : : : : : +- CometBroadcastHashJoin (21) - : : : : : :- CometFilter (17) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (16) - : : : : : +- CometBroadcastExchange (20) - : : : : : +- CometFilter (19) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (18) - : : : : +- ReusedExchange (23) - : : : +- CometBroadcastExchange (44) - : : : +- CometHashAggregate (43) - : : : +- CometExchange (42) - : : : +- CometHashAggregate (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (32) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (31) - : : : : +- CometBroadcastExchange (35) - : : : : +- CometFilter (34) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (33) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (58) - : : +- CometHashAggregate (57) - : : +- CometExchange (56) - : : +- CometHashAggregate (55) - : : +- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometFilter (48) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : : : +- ReusedExchange (49) - : : +- ReusedExchange (52) - : +- CometBroadcastExchange (71) - : +- CometHashAggregate (70) - : +- CometExchange (69) - : +- CometHashAggregate (68) - : +- CometProject (67) - : +- CometBroadcastHashJoin (66) - : :- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometFilter (61) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (60) - : : +- ReusedExchange (62) - : +- ReusedExchange (65) - +- CometBroadcastExchange (85) - +- CometHashAggregate (84) - +- CometExchange (83) - +- CometHashAggregate (82) - +- CometProject (81) - +- CometBroadcastHashJoin (80) - :- CometProject (78) - : +- CometBroadcastHashJoin (77) - : :- CometFilter (75) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (74) - : +- ReusedExchange (76) - +- ReusedExchange (79) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_addr_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) - -(5) CometBroadcastExchange -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [d_date_sk#5, d_year#6, d_qoy#7] - -(6) CometBroadcastHashJoin -Left output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(7) CometProject -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7], [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#8, ca_county#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [ca_address_sk#8, ca_county#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_county#9)) - -(10) CometBroadcastExchange -Input [2]: [ca_address_sk#8, ca_county#9] -Arguments: [ca_address_sk#8, ca_county#9] - -(11) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7] -Right output [2]: [ca_address_sk#8, ca_county#9] -Arguments: [ss_addr_sk#1], [ca_address_sk#8], Inner, BuildRight - -(12) CometProject -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_address_sk#8, ca_county#9] -Arguments: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9], [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] - -(13) CometHashAggregate -Input [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] -Keys [3]: [ca_county#9, d_qoy#7, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(14) CometExchange -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10] -Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(15) CometHashAggregate -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10] -Keys [3]: [ca_county#9, d_qoy#7, d_year#6] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(17) CometFilter -Input [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] -Condition : isnotnull(ss_addr_sk#11) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(19) CometFilter -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) - -(20) CometBroadcastExchange -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Arguments: [d_date_sk#15, d_year#16, d_qoy#17] - -(21) CometBroadcastHashJoin -Left output [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] -Right output [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(22) CometProject -Input [6]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_year#16, d_qoy#17] -Arguments: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17], [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17] - -(23) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#18, ca_county#19] - -(24) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17] -Right output [2]: [ca_address_sk#18, ca_county#19] -Arguments: [ss_addr_sk#11], [ca_address_sk#18], Inner, BuildRight - -(25) CometProject -Input [6]: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_address_sk#18, ca_county#19] -Arguments: [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19], [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19] - -(26) CometHashAggregate -Input [4]: [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19] -Keys [3]: [ca_county#19, d_qoy#17, d_year#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#12))] - -(27) CometExchange -Input [4]: [ca_county#19, d_qoy#17, d_year#16, sum#20] -Arguments: hashpartitioning(ca_county#19, d_qoy#17, d_year#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [4]: [ca_county#19, d_qoy#17, d_year#16, sum#20] -Keys [3]: [ca_county#19, d_qoy#17, d_year#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#12))] - -(29) CometBroadcastExchange -Input [2]: [ca_county#19, store_sales#21] -Arguments: [ca_county#19, store_sales#21] - -(30) CometBroadcastHashJoin -Left output [3]: [ca_county#9, d_year#6, store_sales#22] -Right output [2]: [ca_county#19, store_sales#21] -Arguments: [ca_county#9], [ca_county#19], Inner, BuildRight - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] -PushedFilters: [IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_addr_sk#23) - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) - -(35) CometBroadcastExchange -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Arguments: [d_date_sk#27, d_year#28, d_qoy#29] - -(36) CometBroadcastHashJoin -Left output [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] -Right output [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight - -(37) CometProject -Input [6]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_year#28, d_qoy#29] -Arguments: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29], [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29] - -(38) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#30, ca_county#31] - -(39) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29] -Right output [2]: [ca_address_sk#30, ca_county#31] -Arguments: [ss_addr_sk#23], [ca_address_sk#30], Inner, BuildRight - -(40) CometProject -Input [6]: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_address_sk#30, ca_county#31] -Arguments: [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31], [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31] - -(41) CometHashAggregate -Input [4]: [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31] -Keys [3]: [ca_county#31, d_qoy#29, d_year#28] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#24))] - -(42) CometExchange -Input [4]: [ca_county#31, d_qoy#29, d_year#28, sum#32] -Arguments: hashpartitioning(ca_county#31, d_qoy#29, d_year#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(43) CometHashAggregate -Input [4]: [ca_county#31, d_qoy#29, d_year#28, sum#32] -Keys [3]: [ca_county#31, d_qoy#29, d_year#28] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#24))] - -(44) CometBroadcastExchange -Input [2]: [ca_county#31, store_sales#33] -Arguments: [ca_county#31, store_sales#33] - -(45) CometBroadcastHashJoin -Left output [5]: [ca_county#9, d_year#6, store_sales#22, ca_county#19, store_sales#21] -Right output [2]: [ca_county#31, store_sales#33] -Arguments: [ca_county#19], [ca_county#31], Inner, BuildRight - -(46) CometProject -Input [7]: [ca_county#9, d_year#6, store_sales#22, ca_county#19, store_sales#21, ca_county#31, store_sales#33] -Arguments: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33], [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#37)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(48) CometFilter -Input [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_bill_addr_sk#34) - -(49) ReusedExchange [Reuses operator id: 5] -Output [3]: [d_date_sk#38, d_year#39, d_qoy#40] - -(50) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Right output [3]: [d_date_sk#38, d_year#39, d_qoy#40] -Arguments: [ws_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight - -(51) CometProject -Input [6]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#38, d_year#39, d_qoy#40] -Arguments: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40], [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40] - -(52) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#41, ca_county#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40] -Right output [2]: [ca_address_sk#41, ca_county#42] -Arguments: [ws_bill_addr_sk#34], [ca_address_sk#41], Inner, BuildRight - -(54) CometProject -Input [6]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_address_sk#41, ca_county#42] -Arguments: [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42], [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42] - -(55) CometHashAggregate -Input [4]: [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42] -Keys [3]: [ca_county#42, d_qoy#40, d_year#39] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] - -(56) CometExchange -Input [4]: [ca_county#42, d_qoy#40, d_year#39, sum#43] -Arguments: hashpartitioning(ca_county#42, d_qoy#40, d_year#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(57) CometHashAggregate -Input [4]: [ca_county#42, d_qoy#40, d_year#39, sum#43] -Keys [3]: [ca_county#42, d_qoy#40, d_year#39] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] - -(58) CometBroadcastExchange -Input [2]: [ca_county#42, web_sales#44] -Arguments: [ca_county#42, web_sales#44] - -(59) CometBroadcastHashJoin -Left output [5]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33] -Right output [2]: [ca_county#42, web_sales#44] -Arguments: [ca_county#9], [ca_county#42], Inner, BuildRight - -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#48)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(61) CometFilter -Input [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] -Condition : isnotnull(ws_bill_addr_sk#45) - -(62) ReusedExchange [Reuses operator id: 20] -Output [3]: [d_date_sk#49, d_year#50, d_qoy#51] - -(63) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] -Right output [3]: [d_date_sk#49, d_year#50, d_qoy#51] -Arguments: [ws_sold_date_sk#47], [d_date_sk#49], Inner, BuildRight - -(64) CometProject -Input [6]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47, d_date_sk#49, d_year#50, d_qoy#51] -Arguments: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51], [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51] - -(65) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#52, ca_county#53] - -(66) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51] -Right output [2]: [ca_address_sk#52, ca_county#53] -Arguments: [ws_bill_addr_sk#45], [ca_address_sk#52], Inner, BuildRight - -(67) CometProject -Input [6]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_address_sk#52, ca_county#53] -Arguments: [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53], [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53] - -(68) CometHashAggregate -Input [4]: [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53] -Keys [3]: [ca_county#53, d_qoy#51, d_year#50] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#46))] - -(69) CometExchange -Input [4]: [ca_county#53, d_qoy#51, d_year#50, sum#54] -Arguments: hashpartitioning(ca_county#53, d_qoy#51, d_year#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(70) CometHashAggregate -Input [4]: [ca_county#53, d_qoy#51, d_year#50, sum#54] -Keys [3]: [ca_county#53, d_qoy#51, d_year#50] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#46))] - -(71) CometBroadcastExchange -Input [2]: [ca_county#53, web_sales#55] -Arguments: [ca_county#53, web_sales#55] - -(72) CometBroadcastHashJoin -Left output [7]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44] -Right output [2]: [ca_county#53, web_sales#55] -Arguments: [ca_county#42], [ca_county#53], Inner, (CASE WHEN (web_sales#44 > 0.00) THEN (web_sales#55 / web_sales#44) END > CASE WHEN (store_sales#22 > 0.00) THEN (store_sales#21 / store_sales#22) END), BuildRight - -(73) CometProject -Input [9]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, ca_county#53, web_sales#55] -Arguments: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55], [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55] - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ws_bill_addr_sk)] -ReadSchema: struct - -(75) CometFilter -Input [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] -Condition : isnotnull(ws_bill_addr_sk#56) - -(76) ReusedExchange [Reuses operator id: 35] -Output [3]: [d_date_sk#60, d_year#61, d_qoy#62] - -(77) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] -Right output [3]: [d_date_sk#60, d_year#61, d_qoy#62] -Arguments: [ws_sold_date_sk#58], [d_date_sk#60], Inner, BuildRight - -(78) CometProject -Input [6]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58, d_date_sk#60, d_year#61, d_qoy#62] -Arguments: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62], [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62] - -(79) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#63, ca_county#64] - -(80) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62] -Right output [2]: [ca_address_sk#63, ca_county#64] -Arguments: [ws_bill_addr_sk#56], [ca_address_sk#63], Inner, BuildRight - -(81) CometProject -Input [6]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_address_sk#63, ca_county#64] -Arguments: [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64], [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64] - -(82) CometHashAggregate -Input [4]: [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64] -Keys [3]: [ca_county#64, d_qoy#62, d_year#61] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#57))] - -(83) CometExchange -Input [4]: [ca_county#64, d_qoy#62, d_year#61, sum#65] -Arguments: hashpartitioning(ca_county#64, d_qoy#62, d_year#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(84) CometHashAggregate -Input [4]: [ca_county#64, d_qoy#62, d_year#61, sum#65] -Keys [3]: [ca_county#64, d_qoy#62, d_year#61] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#57))] - -(85) CometBroadcastExchange -Input [2]: [ca_county#64, web_sales#66] -Arguments: [ca_county#64, web_sales#66] - -(86) CometBroadcastHashJoin -Left output [8]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55] -Right output [2]: [ca_county#64, web_sales#66] -Arguments: [ca_county#42], [ca_county#64], Inner, (CASE WHEN (web_sales#55 > 0.00) THEN (web_sales#66 / web_sales#55) END > CASE WHEN (store_sales#21 > 0.00) THEN (store_sales#33 / store_sales#21) END), BuildRight - -(87) CometProject -Input [10]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55, ca_county#64, web_sales#66] -Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9, d_year#6, (web_sales#55 / web_sales#44) AS web_q1_q2_increase#67, (store_sales#21 / store_sales#22) AS store_q1_q2_increase#68, (web_sales#66 / web_sales#55) AS web_q2_q3_increase#69, (store_sales#33 / store_sales#21) AS store_q2_q3_increase#70] - -(88) CometExchange -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(89) CometSort -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] -Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9 ASC NULLS FIRST] - -(90) CometColumnarToRow [codegen id : 1] -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (94) -+- * CometColumnarToRow (93) - +- CometFilter (92) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (91) - - -(91) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(92) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) - -(93) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] - -(94) BroadcastExchange -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (98) -+- * CometColumnarToRow (97) - +- CometFilter (96) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (95) - - -(95) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(96) CometFilter -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) - -(97) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] - -(98) BroadcastExchange -Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -Subquery:3 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 -BroadcastExchange (102) -+- * CometColumnarToRow (101) - +- CometFilter (100) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (99) - - -(99) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(100) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) - -(101) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] - -(102) BroadcastExchange -Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:4 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#4 - -Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#14 - -Subquery:6 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#26 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/extended.txt deleted file mode 100644 index b4e7aa4db6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/extended.txt +++ /dev/null @@ -1,126 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : : +- CometBroadcastExchange - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 111 out of 120 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 2f97384320..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt +++ /dev/null @@ -1,116 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] - CometExchange [ca_county] #1 - CometProject [web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] - CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales,ca_county,web_sales] - CometProject [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,ca_county,web_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales] - CometProject [ca_county,d_year,store_sales,store_sales,store_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales,ca_county,store_sales] - CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales] - CometHashAggregate [d_qoy,sum] [ca_county,d_year,store_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #2 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [ca_address_sk,ca_county] #5 - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - CometBroadcastExchange [ca_county,store_sales] #6 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,store_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #7 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #9 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,store_sales] #10 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,store_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #11 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #13 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,web_sales] #14 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #15 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_year,d_qoy] #4 - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,web_sales] #16 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #17 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [d_date_sk,d_year,d_qoy] #9 - ReusedExchange [ca_address_sk,ca_county] #5 - CometBroadcastExchange [ca_county,web_sales] #18 - CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [ca_county,d_qoy,d_year] #19 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [d_date_sk,d_year,d_qoy] #13 - ReusedExchange [ca_address_sk,ca_county] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/explain.txt deleted file mode 100644 index ea4c14a47b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -* HashAggregate (31) -+- * CometColumnarToRow (30) - +- CometColumnarExchange (29) - +- * HashAggregate (28) - +- * Project (27) - +- * BroadcastHashJoin Inner BuildRight (26) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : +- BroadcastExchange (8) - : : +- * CometColumnarToRow (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometNativeScan parquet spark_catalog.default.item (4) - : +- BroadcastExchange (22) - : +- * Filter (21) - : +- * HashAggregate (20) - : +- * CometColumnarToRow (19) - : +- CometColumnarExchange (18) - : +- * HashAggregate (17) - : +- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet spark_catalog.default.catalog_sales (11) - : +- ReusedExchange (14) - +- ReusedExchange (25) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_ext_discount_amt)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] - -(3) Filter [codegen id : 6] -Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Condition : (isnotnull(cs_item_sk#1) AND isnotnull(cs_ext_discount_amt#2)) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_manufact_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,977), IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [i_item_sk#5, i_manufact_id#6] -Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 977)) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [2]: [i_item_sk#5, i_manufact_id#6] -Arguments: [i_item_sk#5], [i_item_sk#5] - -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [i_item_sk#5] - -(8) BroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 6] -Output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] -Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] - -(11) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] - -(13) Filter [codegen id : 3] -Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Condition : isnotnull(cs_item_sk#7) - -(14) ReusedExchange [Reuses operator id: 36] -Output [1]: [d_date_sk#10] - -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(16) Project [codegen id : 3] -Output [2]: [cs_item_sk#7, cs_ext_discount_amt#8] -Input [4]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9, d_date_sk#10] - -(17) HashAggregate [codegen id : 3] -Input [2]: [cs_item_sk#7, cs_ext_discount_amt#8] -Keys [1]: [cs_item_sk#7] -Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#8))] -Aggregate Attributes [2]: [sum#11, count#12] -Results [3]: [cs_item_sk#7, sum#13, count#14] - -(18) CometColumnarExchange -Input [3]: [cs_item_sk#7, sum#13, count#14] -Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(19) CometColumnarToRow [codegen id : 4] -Input [3]: [cs_item_sk#7, sum#13, count#14] - -(20) HashAggregate [codegen id : 4] -Input [3]: [cs_item_sk#7, sum#13, count#14] -Keys [1]: [cs_item_sk#7] -Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))] -Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))#15] -Results [2]: [(1.3 * cast((avg(UnscaledValue(cs_ext_discount_amt#8))#15 / 100.0) as decimal(11,6))) AS (1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] - -(21) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] -Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#16) - -(22) BroadcastExchange -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] - -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [i_item_sk#5] -Right keys [1]: [cs_item_sk#7] -Join type: Inner -Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#16) - -(24) Project [codegen id : 6] -Output [2]: [cs_ext_discount_amt#2, cs_sold_date_sk#3] -Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7] - -(25) ReusedExchange [Reuses operator id: 36] -Output [1]: [d_date_sk#10] - -(26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 6] -Output [1]: [cs_ext_discount_amt#2] -Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#10] - -(28) HashAggregate [codegen id : 6] -Input [1]: [cs_ext_discount_amt#2] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#17] -Results [1]: [sum#18] - -(29) CometColumnarExchange -Input [1]: [sum#18] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 7] -Input [1]: [sum#18] - -(31) HashAggregate [codegen id : 7] -Input [1]: [sum#18] -Keys: [] -Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))#19] -Results [1]: [MakeDecimal(sum(UnscaledValue(cs_ext_discount_amt#2))#19,17,2) AS excess discount amount#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.date_dim (32) - - -(32) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_date#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#10, d_date#21] -Condition : (((isnotnull(d_date#21) AND (d_date#21 >= 2000-01-27)) AND (d_date#21 <= 2000-04-26)) AND isnotnull(d_date_sk#10)) - -(34) CometProject -Input [2]: [d_date_sk#10, d_date#21] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(36) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/simplified.txt deleted file mode 100644 index 14c21b32a5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -WholeStageCodegen (7) - HashAggregate [sum] [sum(UnscaledValue(cs_ext_discount_amt)),excess discount amount,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (6) - HashAggregate [cs_ext_discount_amt] [sum,sum] - Project [cs_ext_discount_amt] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_discount_amt,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk,cs_ext_discount_amt,(1.3 * avg(cs_ext_discount_amt))] - Project [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk,cs_ext_discount_amt] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Filter [(1.3 * avg(cs_ext_discount_amt))] - HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_item_sk] #5 - WholeStageCodegen (3) - HashAggregate [cs_item_sk,cs_ext_discount_amt] [sum,count,sum,count] - Project [cs_item_sk,cs_ext_discount_amt] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/explain.txt deleted file mode 100644 index b27d40f91b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (30) -+- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : +- CometBroadcastExchange (21) - : +- CometFilter (20) - : +- CometHashAggregate (19) - : +- CometExchange (18) - : +- CometHashAggregate (17) - : +- CometProject (16) - : +- CometBroadcastHashJoin (15) - : :- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : +- CometBroadcastExchange (14) - : +- CometProject (13) - : +- CometFilter (12) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (11) - +- ReusedExchange (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_ext_discount_amt)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Condition : (isnotnull(cs_item_sk#1) AND isnotnull(cs_ext_discount_amt#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_manufact_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,977), IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_manufact_id#6] -Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 977)) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [2]: [i_item_sk#5, i_manufact_id#6] -Arguments: [i_item_sk#5], [i_item_sk#5] - -(6) CometBroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: [i_item_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Right output [1]: [i_item_sk#5] -Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] -Arguments: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5], [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Condition : isnotnull(cs_item_sk#7) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) - -(13) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(15) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(16) CometProject -Input [4]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9, d_date_sk#11] -Arguments: [cs_item_sk#7, cs_ext_discount_amt#8], [cs_item_sk#7, cs_ext_discount_amt#8] - -(17) CometHashAggregate -Input [2]: [cs_item_sk#7, cs_ext_discount_amt#8] -Keys [1]: [cs_item_sk#7] -Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#8))] - -(18) CometExchange -Input [3]: [cs_item_sk#7, sum#13, count#14] -Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [3]: [cs_item_sk#7, sum#13, count#14] -Keys [1]: [cs_item_sk#7] -Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))] - -(20) CometFilter -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#15) - -(21) CometBroadcastExchange -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Arguments: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] - -(22) CometBroadcastHashJoin -Left output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] -Right output [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Arguments: [i_item_sk#5], [cs_item_sk#7], Inner, (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#15), BuildRight - -(23) CometProject -Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Arguments: [cs_ext_discount_amt#2, cs_sold_date_sk#3], [cs_ext_discount_amt#2, cs_sold_date_sk#3] - -(24) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#11] - -(25) CometBroadcastHashJoin -Left output [2]: [cs_ext_discount_amt#2, cs_sold_date_sk#3] -Right output [1]: [d_date_sk#11] -Arguments: [cs_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight - -(26) CometProject -Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#11] -Arguments: [cs_ext_discount_amt#2], [cs_ext_discount_amt#2] - -(27) CometHashAggregate -Input [1]: [cs_ext_discount_amt#2] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))] - -(28) CometExchange -Input [1]: [sum#16] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [1]: [sum#16] -Keys: [] -Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [excess discount amount#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) - -(33) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(35) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/simplified.txt deleted file mode 100644 index 07619fc999..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [excess discount amount,sum(UnscaledValue(cs_ext_discount_amt))] - CometExchange #1 - CometHashAggregate [cs_ext_discount_amt] [sum] - CometProject [cs_ext_discount_amt] - CometBroadcastHashJoin [cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] - CometProject [cs_ext_discount_amt,cs_sold_date_sk] - CometBroadcastHashJoin [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk,(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] - CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk] #3 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] #4 - CometFilter [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] - CometHashAggregate [sum,count] [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,avg(UnscaledValue(cs_ext_discount_amt))] - CometExchange [cs_item_sk] #5 - CometHashAggregate [cs_ext_discount_amt] [cs_item_sk,sum,count] - CometProject [cs_item_sk,cs_ext_discount_amt] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt deleted file mode 100644 index b27d40f91b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (30) -+- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : +- CometBroadcastExchange (21) - : +- CometFilter (20) - : +- CometHashAggregate (19) - : +- CometExchange (18) - : +- CometHashAggregate (17) - : +- CometProject (16) - : +- CometBroadcastHashJoin (15) - : :- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : +- CometBroadcastExchange (14) - : +- CometProject (13) - : +- CometFilter (12) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (11) - +- ReusedExchange (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_ext_discount_amt)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Condition : (isnotnull(cs_item_sk#1) AND isnotnull(cs_ext_discount_amt#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_manufact_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,977), IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_manufact_id#6] -Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 977)) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [2]: [i_item_sk#5, i_manufact_id#6] -Arguments: [i_item_sk#5], [i_item_sk#5] - -(6) CometBroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: [i_item_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Right output [1]: [i_item_sk#5] -Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] -Arguments: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5], [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Condition : isnotnull(cs_item_sk#7) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) - -(13) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(15) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(16) CometProject -Input [4]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9, d_date_sk#11] -Arguments: [cs_item_sk#7, cs_ext_discount_amt#8], [cs_item_sk#7, cs_ext_discount_amt#8] - -(17) CometHashAggregate -Input [2]: [cs_item_sk#7, cs_ext_discount_amt#8] -Keys [1]: [cs_item_sk#7] -Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#8))] - -(18) CometExchange -Input [3]: [cs_item_sk#7, sum#13, count#14] -Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [3]: [cs_item_sk#7, sum#13, count#14] -Keys [1]: [cs_item_sk#7] -Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))] - -(20) CometFilter -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#15) - -(21) CometBroadcastExchange -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Arguments: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] - -(22) CometBroadcastHashJoin -Left output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] -Right output [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Arguments: [i_item_sk#5], [cs_item_sk#7], Inner, (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#15), BuildRight - -(23) CometProject -Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#7] -Arguments: [cs_ext_discount_amt#2, cs_sold_date_sk#3], [cs_ext_discount_amt#2, cs_sold_date_sk#3] - -(24) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#11] - -(25) CometBroadcastHashJoin -Left output [2]: [cs_ext_discount_amt#2, cs_sold_date_sk#3] -Right output [1]: [d_date_sk#11] -Arguments: [cs_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight - -(26) CometProject -Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#11] -Arguments: [cs_ext_discount_amt#2], [cs_ext_discount_amt#2] - -(27) CometHashAggregate -Input [1]: [cs_ext_discount_amt#2] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))] - -(28) CometExchange -Input [1]: [sum#16] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [1]: [sum#16] -Keys: [] -Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [excess discount amount#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) - -(33) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(35) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/extended.txt deleted file mode 100644 index 943bb147ff..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 07619fc999..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [excess discount amount,sum(UnscaledValue(cs_ext_discount_amt))] - CometExchange #1 - CometHashAggregate [cs_ext_discount_amt] [sum] - CometProject [cs_ext_discount_amt] - CometBroadcastHashJoin [cs_ext_discount_amt,cs_sold_date_sk,d_date_sk] - CometProject [cs_ext_discount_amt,cs_sold_date_sk] - CometBroadcastHashJoin [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk,(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] - CometProject [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk] #3 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] #4 - CometFilter [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk] - CometHashAggregate [sum,count] [(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,avg(UnscaledValue(cs_ext_discount_amt))] - CometExchange [cs_item_sk] #5 - CometHashAggregate [cs_ext_discount_amt] [cs_item_sk,sum,count] - CometProject [cs_item_sk,cs_ext_discount_amt] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/explain.txt deleted file mode 100644 index f441da7c27..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/explain.txt +++ /dev/null @@ -1,461 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (74) -+- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- Union (69) - :- * HashAggregate (28) - : +- * CometColumnarToRow (27) - : +- CometColumnarExchange (26) - : +- * HashAggregate (25) - : +- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (22) - : +- * CometColumnarToRow (21) - : +- CometBroadcastHashJoin (20) - : :- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.item (14) - : +- CometBroadcastExchange (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometNativeScan parquet spark_catalog.default.item (16) - :- * HashAggregate (52) - : +- * CometColumnarToRow (51) - : +- CometColumnarExchange (50) - : +- * HashAggregate (49) - : +- * Project (48) - : +- * BroadcastHashJoin Inner BuildRight (47) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (34) - : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : :- * Filter (31) - : : : : +- * ColumnarToRow (30) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (32) - : : +- ReusedExchange (35) - : +- BroadcastExchange (46) - : +- * CometColumnarToRow (45) - : +- CometBroadcastHashJoin (44) - : :- CometFilter (39) - : : +- CometNativeScan parquet spark_catalog.default.item (38) - : +- CometBroadcastExchange (43) - : +- CometProject (42) - : +- CometFilter (41) - : +- CometNativeScan parquet spark_catalog.default.item (40) - +- * HashAggregate (68) - +- * CometColumnarToRow (67) - +- CometColumnarExchange (66) - +- * HashAggregate (65) - +- * Project (64) - +- * BroadcastHashJoin Inner BuildRight (63) - :- * Project (61) - : +- * BroadcastHashJoin Inner BuildRight (60) - : :- * Project (58) - : : +- * BroadcastHashJoin Inner BuildRight (57) - : : :- * Filter (55) - : : : +- * ColumnarToRow (54) - : : : +- Scan parquet spark_catalog.default.web_sales (53) - : : +- ReusedExchange (56) - : +- ReusedExchange (59) - +- ReusedExchange (62) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(4) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_gmt_offset#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isnotnull(ca_address_sk#7)) - -(9) CometProject -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Arguments: [ca_address_sk#7], [ca_address_sk#7] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [ca_address_sk#7] - -(11) BroadcastExchange -Input [1]: [ca_address_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] - -(14) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#9, i_manufact_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [i_item_sk#9, i_manufact_id#10] -Condition : isnotnull(i_item_sk#9) - -(16) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_category#11, i_manufact_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(17) CometFilter -Input [2]: [i_category#11, i_manufact_id#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) = Electronics ) - -(18) CometProject -Input [2]: [i_category#11, i_manufact_id#10] -Arguments: [i_manufact_id#10#12], [i_manufact_id#10 AS i_manufact_id#10#12] - -(19) CometBroadcastExchange -Input [1]: [i_manufact_id#10#12] -Arguments: [i_manufact_id#10#12] - -(20) CometBroadcastHashJoin -Left output [2]: [i_item_sk#9, i_manufact_id#10] -Right output [1]: [i_manufact_id#10#12] -Arguments: [i_manufact_id#10], [i_manufact_id#10#12], LeftSemi, BuildRight - -(21) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#9, i_manufact_id#10] - -(22) BroadcastExchange -Input [2]: [i_item_sk#9, i_manufact_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#9] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 4] -Output [2]: [ss_ext_sales_price#3, i_manufact_id#10] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_manufact_id#10] - -(25) HashAggregate [codegen id : 4] -Input [2]: [ss_ext_sales_price#3, i_manufact_id#10] -Keys [1]: [i_manufact_id#10] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum#13] -Results [2]: [i_manufact_id#10, sum#14] - -(26) CometColumnarExchange -Input [2]: [i_manufact_id#10, sum#14] -Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(27) CometColumnarToRow [codegen id : 5] -Input [2]: [i_manufact_id#10, sum#14] - -(28) HashAggregate [codegen id : 5] -Input [2]: [i_manufact_id#10, sum#14] -Keys [1]: [i_manufact_id#10] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#15] -Results [2]: [i_manufact_id#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#15,17,2) AS total_sales#16] - -(29) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(30) ColumnarToRow [codegen id : 9] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] - -(31) Filter [codegen id : 9] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) - -(32) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#21] - -(33) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#20] -Right keys [1]: [d_date_sk#21] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 9] -Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] -Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#21] - -(35) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#22] - -(36) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_bill_addr_sk#17] -Right keys [1]: [ca_address_sk#22] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 9] -Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] -Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#22] - -(38) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#23, i_manufact_id#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(39) CometFilter -Input [2]: [i_item_sk#23, i_manufact_id#24] -Condition : isnotnull(i_item_sk#23) - -(40) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_category#11, i_manufact_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(41) CometFilter -Input [2]: [i_category#11, i_manufact_id#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) = Electronics ) - -(42) CometProject -Input [2]: [i_category#11, i_manufact_id#10] -Arguments: [i_manufact_id#10], [i_manufact_id#10] - -(43) CometBroadcastExchange -Input [1]: [i_manufact_id#10] -Arguments: [i_manufact_id#10] - -(44) CometBroadcastHashJoin -Left output [2]: [i_item_sk#23, i_manufact_id#24] -Right output [1]: [i_manufact_id#10] -Arguments: [i_manufact_id#24], [i_manufact_id#10], LeftSemi, BuildRight - -(45) CometColumnarToRow [codegen id : 8] -Input [2]: [i_item_sk#23, i_manufact_id#24] - -(46) BroadcastExchange -Input [2]: [i_item_sk#23, i_manufact_id#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - -(47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#23] -Join type: Inner -Join condition: None - -(48) Project [codegen id : 9] -Output [2]: [cs_ext_sales_price#19, i_manufact_id#24] -Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#23, i_manufact_id#24] - -(49) HashAggregate [codegen id : 9] -Input [2]: [cs_ext_sales_price#19, i_manufact_id#24] -Keys [1]: [i_manufact_id#24] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum#25] -Results [2]: [i_manufact_id#24, sum#26] - -(50) CometColumnarExchange -Input [2]: [i_manufact_id#24, sum#26] -Arguments: hashpartitioning(i_manufact_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(51) CometColumnarToRow [codegen id : 10] -Input [2]: [i_manufact_id#24, sum#26] - -(52) HashAggregate [codegen id : 10] -Input [2]: [i_manufact_id#24, sum#26] -Keys [1]: [i_manufact_id#24] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#19))#27] -Results [2]: [i_manufact_id#24, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#19))#27,17,2) AS total_sales#28] - -(53) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#32), dynamicpruningexpression(ws_sold_date_sk#32 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(54) ColumnarToRow [codegen id : 14] -Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32] - -(55) Filter [codegen id : 14] -Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32] -Condition : (isnotnull(ws_bill_addr_sk#30) AND isnotnull(ws_item_sk#29)) - -(56) ReusedExchange [Reuses operator id: 79] -Output [1]: [d_date_sk#33] - -(57) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#32] -Right keys [1]: [d_date_sk#33] -Join type: Inner -Join condition: None - -(58) Project [codegen id : 14] -Output [3]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31] -Input [5]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ws_sold_date_sk#32, d_date_sk#33] - -(59) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#34] - -(60) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#30] -Right keys [1]: [ca_address_sk#34] -Join type: Inner -Join condition: None - -(61) Project [codegen id : 14] -Output [2]: [ws_item_sk#29, ws_ext_sales_price#31] -Input [4]: [ws_item_sk#29, ws_bill_addr_sk#30, ws_ext_sales_price#31, ca_address_sk#34] - -(62) ReusedExchange [Reuses operator id: 46] -Output [2]: [i_item_sk#35, i_manufact_id#36] - -(63) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_item_sk#29] -Right keys [1]: [i_item_sk#35] -Join type: Inner -Join condition: None - -(64) Project [codegen id : 14] -Output [2]: [ws_ext_sales_price#31, i_manufact_id#36] -Input [4]: [ws_item_sk#29, ws_ext_sales_price#31, i_item_sk#35, i_manufact_id#36] - -(65) HashAggregate [codegen id : 14] -Input [2]: [ws_ext_sales_price#31, i_manufact_id#36] -Keys [1]: [i_manufact_id#36] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#31))] -Aggregate Attributes [1]: [sum#37] -Results [2]: [i_manufact_id#36, sum#38] - -(66) CometColumnarExchange -Input [2]: [i_manufact_id#36, sum#38] -Arguments: hashpartitioning(i_manufact_id#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(67) CometColumnarToRow [codegen id : 15] -Input [2]: [i_manufact_id#36, sum#38] - -(68) HashAggregate [codegen id : 15] -Input [2]: [i_manufact_id#36, sum#38] -Keys [1]: [i_manufact_id#36] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#31))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#31))#39] -Results [2]: [i_manufact_id#36, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#31))#39,17,2) AS total_sales#40] - -(69) Union - -(70) HashAggregate [codegen id : 16] -Input [2]: [i_manufact_id#10, total_sales#16] -Keys [1]: [i_manufact_id#10] -Functions [1]: [partial_sum(total_sales#16)] -Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [3]: [i_manufact_id#10, sum#43, isEmpty#44] - -(71) CometColumnarExchange -Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] -Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(72) CometColumnarToRow [codegen id : 17] -Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] - -(73) HashAggregate [codegen id : 17] -Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] -Keys [1]: [i_manufact_id#10] -Functions [1]: [sum(total_sales#16)] -Aggregate Attributes [1]: [sum(total_sales#16)#45] -Results [2]: [i_manufact_id#10, sum(total_sales#16)#45 AS total_sales#46] - -(74) TakeOrderedAndProject -Input [2]: [i_manufact_id#10, total_sales#46] -Arguments: 100, [total_sales#46 ASC NULLS FIRST], [i_manufact_id#10, total_sales#46] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (79) -+- * CometColumnarToRow (78) - +- CometProject (77) - +- CometFilter (76) - +- CometNativeScan parquet spark_catalog.default.date_dim (75) - - -(75) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#47, d_moy#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] -ReadSchema: struct - -(76) CometFilter -Input [3]: [d_date_sk#6, d_year#47, d_moy#48] -Condition : ((((isnotnull(d_year#47) AND isnotnull(d_moy#48)) AND (d_year#47 = 1998)) AND (d_moy#48 = 5)) AND isnotnull(d_date_sk#6)) - -(77) CometProject -Input [3]: [d_date_sk#6, d_year#47, d_moy#48] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(78) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(79) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 53 Hosting Expression = ws_sold_date_sk#32 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/simplified.txt deleted file mode 100644 index 719ccdbb9f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/simplified.txt +++ /dev/null @@ -1,115 +0,0 @@ -TakeOrderedAndProject [total_sales,i_manufact_id] - WholeStageCodegen (17) - HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id] #1 - WholeStageCodegen (16) - HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id] #2 - WholeStageCodegen (4) - HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] - CometFilter [i_item_sk,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [i_manufact_id] #6 - CometProject [i_manufact_id] [i_manufact_id] - CometFilter [i_category,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_category,i_manufact_id] - WholeStageCodegen (10) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id] #7 - WholeStageCodegen (9) - HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_addr_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] - CometFilter [i_item_sk,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [i_manufact_id] #9 - CometProject [i_manufact_id] - CometFilter [i_category,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_category,i_manufact_id] - WholeStageCodegen (15) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id] #10 - WholeStageCodegen (14) - HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/explain.txt deleted file mode 100644 index b78621c4bb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,422 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (68) -+- CometTakeOrderedAndProject (67) - +- CometHashAggregate (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometUnion (63) - :- CometHashAggregate (27) - : +- CometExchange (26) - : +- CometHashAggregate (25) - : +- CometProject (24) - : +- CometBroadcastHashJoin (23) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : +- CometBroadcastExchange (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- CometHashAggregate (48) - : +- CometExchange (47) - : +- CometHashAggregate (46) - : +- CometProject (45) - : +- CometBroadcastHashJoin (44) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometFilter (29) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (28) - : : : +- ReusedExchange (30) - : : +- ReusedExchange (33) - : +- CometBroadcastExchange (43) - : +- CometBroadcastHashJoin (42) - : :- CometFilter (37) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (36) - : +- CometBroadcastExchange (41) - : +- CometProject (40) - : +- CometFilter (39) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (38) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometBroadcastHashJoin (58) - :- CometProject (56) - : +- CometBroadcastHashJoin (55) - : :- CometProject (53) - : : +- CometBroadcastHashJoin (52) - : : :- CometFilter (50) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (49) - : : +- ReusedExchange (51) - : +- ReusedExchange (54) - +- ReusedExchange (57) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(11) CometProject -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Arguments: [ca_address_sk#9], [ca_address_sk#9] - -(12) CometBroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: [ca_address_sk#9] - -(13) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Right output [1]: [ca_address_sk#9] -Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] -Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#11, i_manufact_id#12] -Condition : isnotnull(i_item_sk#11) - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_category#13, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(18) CometFilter -Input [2]: [i_category#13, i_manufact_id#12] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#13, 50, true, false, true) = Electronics ) - -(19) CometProject -Input [2]: [i_category#13, i_manufact_id#12] -Arguments: [i_manufact_id#12#14], [i_manufact_id#12 AS i_manufact_id#12#14] - -(20) CometBroadcastExchange -Input [1]: [i_manufact_id#12#14] -Arguments: [i_manufact_id#12#14] - -(21) CometBroadcastHashJoin -Left output [2]: [i_item_sk#11, i_manufact_id#12] -Right output [1]: [i_manufact_id#12#14] -Arguments: [i_manufact_id#12], [i_manufact_id#12#14], LeftSemi, BuildRight - -(22) CometBroadcastExchange -Input [2]: [i_item_sk#11, i_manufact_id#12] -Arguments: [i_item_sk#11, i_manufact_id#12] - -(23) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Right output [2]: [i_item_sk#11, i_manufact_id#12] -Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight - -(24) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_manufact_id#12] -Arguments: [ss_ext_sales_price#3, i_manufact_id#12], [ss_ext_sales_price#3, i_manufact_id#12] - -(25) CometHashAggregate -Input [2]: [ss_ext_sales_price#3, i_manufact_id#12] -Keys [1]: [i_manufact_id#12] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(26) CometExchange -Input [2]: [i_manufact_id#12, sum#15] -Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [2]: [i_manufact_id#12, sum#15] -Keys [1]: [i_manufact_id#12] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#20)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(29) CometFilter -Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] -Condition : (isnotnull(cs_bill_addr_sk#16) AND isnotnull(cs_item_sk#17)) - -(30) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#21] - -(31) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] -Right output [1]: [d_date_sk#21] -Arguments: [cs_sold_date_sk#19], [d_date_sk#21], Inner, BuildRight - -(32) CometProject -Input [5]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19, d_date_sk#21] -Arguments: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18], [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] - -(33) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#22] - -(34) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] -Right output [1]: [ca_address_sk#22] -Arguments: [cs_bill_addr_sk#16], [ca_address_sk#22], Inner, BuildRight - -(35) CometProject -Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, ca_address_sk#22] -Arguments: [cs_item_sk#17, cs_ext_sales_price#18], [cs_item_sk#17, cs_ext_sales_price#18] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#23, i_manufact_id#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [i_item_sk#23, i_manufact_id#24] -Condition : isnotnull(i_item_sk#23) - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_category#13, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(39) CometFilter -Input [2]: [i_category#13, i_manufact_id#12] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#13, 50, true, false, true) = Electronics ) - -(40) CometProject -Input [2]: [i_category#13, i_manufact_id#12] -Arguments: [i_manufact_id#12], [i_manufact_id#12] - -(41) CometBroadcastExchange -Input [1]: [i_manufact_id#12] -Arguments: [i_manufact_id#12] - -(42) CometBroadcastHashJoin -Left output [2]: [i_item_sk#23, i_manufact_id#24] -Right output [1]: [i_manufact_id#12] -Arguments: [i_manufact_id#24], [i_manufact_id#12], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [2]: [i_item_sk#23, i_manufact_id#24] -Arguments: [i_item_sk#23, i_manufact_id#24] - -(44) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_ext_sales_price#18] -Right output [2]: [i_item_sk#23, i_manufact_id#24] -Arguments: [cs_item_sk#17], [i_item_sk#23], Inner, BuildRight - -(45) CometProject -Input [4]: [cs_item_sk#17, cs_ext_sales_price#18, i_item_sk#23, i_manufact_id#24] -Arguments: [cs_ext_sales_price#18, i_manufact_id#24], [cs_ext_sales_price#18, i_manufact_id#24] - -(46) CometHashAggregate -Input [2]: [cs_ext_sales_price#18, i_manufact_id#24] -Keys [1]: [i_manufact_id#24] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#18))] - -(47) CometExchange -Input [2]: [i_manufact_id#24, sum#25] -Arguments: hashpartitioning(i_manufact_id#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(48) CometHashAggregate -Input [2]: [i_manufact_id#24, sum#25] -Keys [1]: [i_manufact_id#24] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#18))] - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(50) CometFilter -Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Condition : (isnotnull(ws_bill_addr_sk#27) AND isnotnull(ws_item_sk#26)) - -(51) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#31] - -(52) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Right output [1]: [d_date_sk#31] -Arguments: [ws_sold_date_sk#29], [d_date_sk#31], Inner, BuildRight - -(53) CometProject -Input [5]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29, d_date_sk#31] -Arguments: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28], [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] - -(54) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#32] - -(55) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] -Right output [1]: [ca_address_sk#32] -Arguments: [ws_bill_addr_sk#27], [ca_address_sk#32], Inner, BuildRight - -(56) CometProject -Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ca_address_sk#32] -Arguments: [ws_item_sk#26, ws_ext_sales_price#28], [ws_item_sk#26, ws_ext_sales_price#28] - -(57) ReusedExchange [Reuses operator id: 43] -Output [2]: [i_item_sk#33, i_manufact_id#34] - -(58) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#26, ws_ext_sales_price#28] -Right output [2]: [i_item_sk#33, i_manufact_id#34] -Arguments: [ws_item_sk#26], [i_item_sk#33], Inner, BuildRight - -(59) CometProject -Input [4]: [ws_item_sk#26, ws_ext_sales_price#28, i_item_sk#33, i_manufact_id#34] -Arguments: [ws_ext_sales_price#28, i_manufact_id#34], [ws_ext_sales_price#28, i_manufact_id#34] - -(60) CometHashAggregate -Input [2]: [ws_ext_sales_price#28, i_manufact_id#34] -Keys [1]: [i_manufact_id#34] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#28))] - -(61) CometExchange -Input [2]: [i_manufact_id#34, sum#35] -Arguments: hashpartitioning(i_manufact_id#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(62) CometHashAggregate -Input [2]: [i_manufact_id#34, sum#35] -Keys [1]: [i_manufact_id#34] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#28))] - -(63) CometUnion -Child 0 Input [2]: [i_manufact_id#12, total_sales#36] -Child 1 Input [2]: [i_manufact_id#24, total_sales#37] -Child 2 Input [2]: [i_manufact_id#34, total_sales#38] - -(64) CometHashAggregate -Input [2]: [i_manufact_id#12, total_sales#36] -Keys [1]: [i_manufact_id#12] -Functions [1]: [partial_sum(total_sales#36)] - -(65) CometExchange -Input [3]: [i_manufact_id#12, sum#39, isEmpty#40] -Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(66) CometHashAggregate -Input [3]: [i_manufact_id#12, sum#39, isEmpty#40] -Keys [1]: [i_manufact_id#12] -Functions [1]: [sum(total_sales#36)] - -(67) CometTakeOrderedAndProject -Input [2]: [i_manufact_id#12, total_sales#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#41 ASC NULLS FIRST], output=[i_manufact_id#12,total_sales#41]), [i_manufact_id#12, total_sales#41], 100, 0, [total_sales#41 ASC NULLS FIRST], [i_manufact_id#12, total_sales#41] - -(68) CometColumnarToRow [codegen id : 1] -Input [2]: [i_manufact_id#12, total_sales#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (73) -+- * CometColumnarToRow (72) - +- CometProject (71) - +- CometFilter (70) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (69) - - -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] -ReadSchema: struct - -(70) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6)) - -(71) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(72) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(73) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/simplified.txt deleted file mode 100644 index ff7856615f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,80 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_manufact_id,total_sales] - CometHashAggregate [sum,isEmpty] [i_manufact_id,total_sales,sum(total_sales)] - CometExchange [i_manufact_id] #1 - CometHashAggregate [total_sales] [i_manufact_id,sum,isEmpty] - CometUnion [i_manufact_id,total_sales] - CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_manufact_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_manufact_id,sum] - CometProject [ss_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [i_manufact_id] #7 - CometProject [i_manufact_id] [i_manufact_id] - CometFilter [i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact_id] - CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_manufact_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_manufact_id,sum] - CometProject [cs_ext_sales_price,i_manufact_id] - 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,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - CometBroadcastExchange [i_item_sk,i_manufact_id] #9 - CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] - CometFilter [i_item_sk,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [i_manufact_id] #10 - CometProject [i_manufact_id] - CometFilter [i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact_id] - CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_manufact_id] #11 - CometHashAggregate [ws_ext_sales_price] [i_manufact_id,sum] - CometProject [ws_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [ws_item_sk,ws_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_manufact_id] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt deleted file mode 100644 index b78621c4bb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ /dev/null @@ -1,422 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (68) -+- CometTakeOrderedAndProject (67) - +- CometHashAggregate (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometUnion (63) - :- CometHashAggregate (27) - : +- CometExchange (26) - : +- CometHashAggregate (25) - : +- CometProject (24) - : +- CometBroadcastHashJoin (23) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : +- CometBroadcastExchange (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- CometHashAggregate (48) - : +- CometExchange (47) - : +- CometHashAggregate (46) - : +- CometProject (45) - : +- CometBroadcastHashJoin (44) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometFilter (29) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (28) - : : : +- ReusedExchange (30) - : : +- ReusedExchange (33) - : +- CometBroadcastExchange (43) - : +- CometBroadcastHashJoin (42) - : :- CometFilter (37) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (36) - : +- CometBroadcastExchange (41) - : +- CometProject (40) - : +- CometFilter (39) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (38) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometBroadcastHashJoin (58) - :- CometProject (56) - : +- CometBroadcastHashJoin (55) - : :- CometProject (53) - : : +- CometBroadcastHashJoin (52) - : : :- CometFilter (50) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (49) - : : +- ReusedExchange (51) - : +- ReusedExchange (54) - +- ReusedExchange (57) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(11) CometProject -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Arguments: [ca_address_sk#9], [ca_address_sk#9] - -(12) CometBroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: [ca_address_sk#9] - -(13) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Right output [1]: [ca_address_sk#9] -Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] -Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#11, i_manufact_id#12] -Condition : isnotnull(i_item_sk#11) - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_category#13, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(18) CometFilter -Input [2]: [i_category#13, i_manufact_id#12] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#13, 50, true, false, true) = Electronics ) - -(19) CometProject -Input [2]: [i_category#13, i_manufact_id#12] -Arguments: [i_manufact_id#12#14], [i_manufact_id#12 AS i_manufact_id#12#14] - -(20) CometBroadcastExchange -Input [1]: [i_manufact_id#12#14] -Arguments: [i_manufact_id#12#14] - -(21) CometBroadcastHashJoin -Left output [2]: [i_item_sk#11, i_manufact_id#12] -Right output [1]: [i_manufact_id#12#14] -Arguments: [i_manufact_id#12], [i_manufact_id#12#14], LeftSemi, BuildRight - -(22) CometBroadcastExchange -Input [2]: [i_item_sk#11, i_manufact_id#12] -Arguments: [i_item_sk#11, i_manufact_id#12] - -(23) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Right output [2]: [i_item_sk#11, i_manufact_id#12] -Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight - -(24) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_manufact_id#12] -Arguments: [ss_ext_sales_price#3, i_manufact_id#12], [ss_ext_sales_price#3, i_manufact_id#12] - -(25) CometHashAggregate -Input [2]: [ss_ext_sales_price#3, i_manufact_id#12] -Keys [1]: [i_manufact_id#12] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(26) CometExchange -Input [2]: [i_manufact_id#12, sum#15] -Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [2]: [i_manufact_id#12, sum#15] -Keys [1]: [i_manufact_id#12] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#20)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(29) CometFilter -Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] -Condition : (isnotnull(cs_bill_addr_sk#16) AND isnotnull(cs_item_sk#17)) - -(30) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#21] - -(31) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] -Right output [1]: [d_date_sk#21] -Arguments: [cs_sold_date_sk#19], [d_date_sk#21], Inner, BuildRight - -(32) CometProject -Input [5]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19, d_date_sk#21] -Arguments: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18], [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] - -(33) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#22] - -(34) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] -Right output [1]: [ca_address_sk#22] -Arguments: [cs_bill_addr_sk#16], [ca_address_sk#22], Inner, BuildRight - -(35) CometProject -Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, ca_address_sk#22] -Arguments: [cs_item_sk#17, cs_ext_sales_price#18], [cs_item_sk#17, cs_ext_sales_price#18] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#23, i_manufact_id#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [i_item_sk#23, i_manufact_id#24] -Condition : isnotnull(i_item_sk#23) - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_category#13, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(39) CometFilter -Input [2]: [i_category#13, i_manufact_id#12] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#13, 50, true, false, true) = Electronics ) - -(40) CometProject -Input [2]: [i_category#13, i_manufact_id#12] -Arguments: [i_manufact_id#12], [i_manufact_id#12] - -(41) CometBroadcastExchange -Input [1]: [i_manufact_id#12] -Arguments: [i_manufact_id#12] - -(42) CometBroadcastHashJoin -Left output [2]: [i_item_sk#23, i_manufact_id#24] -Right output [1]: [i_manufact_id#12] -Arguments: [i_manufact_id#24], [i_manufact_id#12], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [2]: [i_item_sk#23, i_manufact_id#24] -Arguments: [i_item_sk#23, i_manufact_id#24] - -(44) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_ext_sales_price#18] -Right output [2]: [i_item_sk#23, i_manufact_id#24] -Arguments: [cs_item_sk#17], [i_item_sk#23], Inner, BuildRight - -(45) CometProject -Input [4]: [cs_item_sk#17, cs_ext_sales_price#18, i_item_sk#23, i_manufact_id#24] -Arguments: [cs_ext_sales_price#18, i_manufact_id#24], [cs_ext_sales_price#18, i_manufact_id#24] - -(46) CometHashAggregate -Input [2]: [cs_ext_sales_price#18, i_manufact_id#24] -Keys [1]: [i_manufact_id#24] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#18))] - -(47) CometExchange -Input [2]: [i_manufact_id#24, sum#25] -Arguments: hashpartitioning(i_manufact_id#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(48) CometHashAggregate -Input [2]: [i_manufact_id#24, sum#25] -Keys [1]: [i_manufact_id#24] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#18))] - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(50) CometFilter -Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Condition : (isnotnull(ws_bill_addr_sk#27) AND isnotnull(ws_item_sk#26)) - -(51) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#31] - -(52) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] -Right output [1]: [d_date_sk#31] -Arguments: [ws_sold_date_sk#29], [d_date_sk#31], Inner, BuildRight - -(53) CometProject -Input [5]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29, d_date_sk#31] -Arguments: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28], [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] - -(54) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#32] - -(55) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] -Right output [1]: [ca_address_sk#32] -Arguments: [ws_bill_addr_sk#27], [ca_address_sk#32], Inner, BuildRight - -(56) CometProject -Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ca_address_sk#32] -Arguments: [ws_item_sk#26, ws_ext_sales_price#28], [ws_item_sk#26, ws_ext_sales_price#28] - -(57) ReusedExchange [Reuses operator id: 43] -Output [2]: [i_item_sk#33, i_manufact_id#34] - -(58) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#26, ws_ext_sales_price#28] -Right output [2]: [i_item_sk#33, i_manufact_id#34] -Arguments: [ws_item_sk#26], [i_item_sk#33], Inner, BuildRight - -(59) CometProject -Input [4]: [ws_item_sk#26, ws_ext_sales_price#28, i_item_sk#33, i_manufact_id#34] -Arguments: [ws_ext_sales_price#28, i_manufact_id#34], [ws_ext_sales_price#28, i_manufact_id#34] - -(60) CometHashAggregate -Input [2]: [ws_ext_sales_price#28, i_manufact_id#34] -Keys [1]: [i_manufact_id#34] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#28))] - -(61) CometExchange -Input [2]: [i_manufact_id#34, sum#35] -Arguments: hashpartitioning(i_manufact_id#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(62) CometHashAggregate -Input [2]: [i_manufact_id#34, sum#35] -Keys [1]: [i_manufact_id#34] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#28))] - -(63) CometUnion -Child 0 Input [2]: [i_manufact_id#12, total_sales#36] -Child 1 Input [2]: [i_manufact_id#24, total_sales#37] -Child 2 Input [2]: [i_manufact_id#34, total_sales#38] - -(64) CometHashAggregate -Input [2]: [i_manufact_id#12, total_sales#36] -Keys [1]: [i_manufact_id#12] -Functions [1]: [partial_sum(total_sales#36)] - -(65) CometExchange -Input [3]: [i_manufact_id#12, sum#39, isEmpty#40] -Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(66) CometHashAggregate -Input [3]: [i_manufact_id#12, sum#39, isEmpty#40] -Keys [1]: [i_manufact_id#12] -Functions [1]: [sum(total_sales#36)] - -(67) CometTakeOrderedAndProject -Input [2]: [i_manufact_id#12, total_sales#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#41 ASC NULLS FIRST], output=[i_manufact_id#12,total_sales#41]), [i_manufact_id#12, total_sales#41], 100, 0, [total_sales#41 ASC NULLS FIRST], [i_manufact_id#12, total_sales#41] - -(68) CometColumnarToRow [codegen id : 1] -Input [2]: [i_manufact_id#12, total_sales#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (73) -+- * CometColumnarToRow (72) - +- CometProject (71) - +- CometFilter (70) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (69) - - -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] -ReadSchema: struct - -(70) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6)) - -(71) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(72) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(73) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/extended.txt deleted file mode 100644 index 063fdb6faf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/extended.txt +++ /dev/null @@ -1,97 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 89 out of 93 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index ff7856615f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ /dev/null @@ -1,80 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_manufact_id,total_sales] - CometHashAggregate [sum,isEmpty] [i_manufact_id,total_sales,sum(total_sales)] - CometExchange [i_manufact_id] #1 - CometHashAggregate [total_sales] [i_manufact_id,sum,isEmpty] - CometUnion [i_manufact_id,total_sales] - CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_manufact_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_manufact_id,sum] - CometProject [ss_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [i_manufact_id] #7 - CometProject [i_manufact_id] [i_manufact_id] - CometFilter [i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact_id] - CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_manufact_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_manufact_id,sum] - CometProject [cs_ext_sales_price,i_manufact_id] - 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,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - CometBroadcastExchange [i_item_sk,i_manufact_id] #9 - CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] - CometFilter [i_item_sk,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [i_manufact_id] #10 - CometProject [i_manufact_id] - CometFilter [i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact_id] - CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_manufact_id] #11 - CometHashAggregate [ws_ext_sales_price] [i_manufact_id,sum] - CometProject [ws_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [ws_item_sk,ws_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_manufact_id] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/explain.txt deleted file mode 100644 index 18c904ceed..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/explain.txt +++ /dev/null @@ -1,231 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (35) -+- CometSort (34) - +- CometColumnarExchange (33) - +- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * Filter (25) - : +- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (30) - +- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometNativeScan parquet spark_catalog.default.customer (26) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 40] -Output [1]: [d_date_sk#7] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_county#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [s_store_sk#8, s_county#9] -Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND isnotnull(s_store_sk#8)) - -(9) CometProject -Input [2]: [s_store_sk#8, s_county#9] -Arguments: [s_store_sk#8], [s_store_sk#8] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#8] - -(11) BroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] - -(14) CometNativeScan parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : ((((isnotnull(hd_vehicle_count#13) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#11, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#11, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#12 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#13 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#10)) - -(16) CometProject -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Arguments: [hd_demo_sk#10], [hd_demo_sk#10] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#10] - -(18) BroadcastExchange -Input [1]: [hd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#10] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [2]: [ss_customer_sk#1, ss_ticket_number#4] -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] - -(21) HashAggregate [codegen id : 4] -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#14] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(22) CometColumnarExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(24) HashAggregate [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#16] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] - -(25) Filter [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) - -(26) CometNativeScan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(27) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(28) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] - -(29) CometColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(30) BroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#18] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 6] -Output [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(33) CometColumnarExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(34) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST] - -(35) CometColumnarToRow [codegen id : 7] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (40) -+- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometNativeScan parquet spark_catalog.default.date_dim (36) - - -(36) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#27, d_dom#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [d_date_sk#7, d_year#27, d_dom#28] -Condition : (((((d_dom#28 >= 1) AND (d_dom#28 <= 3)) OR ((d_dom#28 >= 25) AND (d_dom#28 <= 28))) AND d_year#27 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(38) CometProject -Input [3]: [d_date_sk#7, d_year#27, d_dom#28] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(39) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(40) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/simplified.txt deleted file mode 100644 index 10af5ceadc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/simplified.txt +++ /dev/null @@ -1,59 +0,0 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometNativeScan 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/q34.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/explain.txt deleted file mode 100644 index 287e323952..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometSort (32) - +- CometExchange (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (24) - : +- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_county#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#10, s_county#11] -Condition : ((isnotnull(s_county#11) AND (s_county#11 = Williamson County)) AND isnotnull(s_store_sk#10)) - -(11) CometProject -Input [2]: [s_store_sk#10, s_county#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : ((((isnotnull(hd_vehicle_count#15) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#14 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#15 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#12)) - -(17) CometProject -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#12], [hd_demo_sk#12] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#12] -Arguments: [hd_demo_sk#12] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Right output [1]: [hd_demo_sk#12] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight - -(20) CometProject -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] -Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] - -(21) CometHashAggregate -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] - -(22) CometExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] - -(24) CometFilter -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(27) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(29) CometBroadcastHashJoin -Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Right output [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -(31) CometExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(32) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST] - -(33) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(36) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(37) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(38) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/simplified.txt deleted file mode 100644 index 47d0e35dd2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 - CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] - CometExchange [ss_ticket_number,ss_customer_sk] #2 - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - 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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt deleted file mode 100644 index 287e323952..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometSort (32) - +- CometExchange (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (24) - : +- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_county#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#10, s_county#11] -Condition : ((isnotnull(s_county#11) AND (s_county#11 = Williamson County)) AND isnotnull(s_store_sk#10)) - -(11) CometProject -Input [2]: [s_store_sk#10, s_county#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : ((((isnotnull(hd_vehicle_count#15) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#14 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#15 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#12)) - -(17) CometProject -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#12], [hd_demo_sk#12] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#12] -Arguments: [hd_demo_sk#12] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Right output [1]: [hd_demo_sk#12] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight - -(20) CometProject -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] -Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] - -(21) CometHashAggregate -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] - -(22) CometExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] - -(24) CometFilter -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(27) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(29) CometBroadcastHashJoin -Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Right output [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -(31) CometExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(32) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST] - -(33) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(36) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(37) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(38) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/extended.txt deleted file mode 100644 index 3d9d23d2cc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/extended.txt +++ /dev/null @@ -1,41 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 47d0e35dd2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 - CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] - CometExchange [ss_ticket_number,ss_customer_sk] #2 - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - 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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/explain.txt deleted file mode 100644 index c65bb26de3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/explain.txt +++ /dev/null @@ -1,295 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (45) -+- * HashAggregate (44) - +- * CometColumnarToRow (43) - +- CometColumnarExchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometNativeScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (38) - +- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometNativeScan parquet spark_catalog.default.customer_demographics (34) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(4) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] - -(6) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#9] - -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None - -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#6] -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] - -(9) BroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#6] -Join type: LeftSemi -Join condition: None - -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] - -(13) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#9] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#10] -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#9] - -(16) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(17) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#10] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#8)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] - -(20) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#9] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#12] -Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#9] - -(23) BroadcastExchange -Input [1]: [cs_ship_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#12] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(25) Filter [codegen id : 9] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(26) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(27) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#14, ca_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [ca_address_sk#14, ca_state#15] -Condition : isnotnull(ca_address_sk#14) - -(29) CometProject -Input [2]: [ca_address_sk#14, ca_state#15] -Arguments: [ca_address_sk#14, ca_state#16], [ca_address_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#15, 2, true, false, true) AS ca_state#16] - -(30) CometColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#14, ca_state#16] - -(31) BroadcastExchange -Input [2]: [ca_address_sk#14, ca_state#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#14] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#16] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#14, ca_state#16] - -(34) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(35) CometFilter -Input [6]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] -Condition : isnotnull(cd_demo_sk#17) - -(36) CometProject -Input [6]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] -Arguments: [cd_demo_sk#17, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22], [cd_demo_sk#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#18, 1, true, false, true) AS cd_gender#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) AS cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] - -(37) CometColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#17, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] - -(38) BroadcastExchange -Input [6]: [cd_demo_sk#17, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#17] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 9] -Output [6]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] -Input [8]: [c_current_cdemo_sk#4, ca_state#16, cd_demo_sk#17, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] - -(41) HashAggregate [codegen id : 9] -Input [6]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] -Keys [6]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#20), partial_max(cd_dep_count#20), partial_avg(cd_dep_count#20), partial_min(cd_dep_employed_count#21), partial_max(cd_dep_employed_count#21), partial_avg(cd_dep_employed_count#21), partial_min(cd_dep_college_count#22), partial_max(cd_dep_college_count#22), partial_avg(cd_dep_college_count#22)] -Aggregate Attributes [13]: [count#25, min#26, max#27, sum#28, count#29, min#30, max#31, sum#32, count#33, min#34, max#35, sum#36, count#37] -Results [19]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, count#38, min#39, max#40, sum#41, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50] - -(42) CometColumnarExchange -Input [19]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, count#38, min#39, max#40, sum#41, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50] -Arguments: hashpartitioning(ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) CometColumnarToRow [codegen id : 10] -Input [19]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, count#38, min#39, max#40, sum#41, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50] - -(44) HashAggregate [codegen id : 10] -Input [19]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, count#38, min#39, max#40, sum#41, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50] -Keys [6]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] -Functions [10]: [count(1), min(cd_dep_count#20), max(cd_dep_count#20), avg(cd_dep_count#20), min(cd_dep_employed_count#21), max(cd_dep_employed_count#21), avg(cd_dep_employed_count#21), min(cd_dep_college_count#22), max(cd_dep_college_count#22), avg(cd_dep_college_count#22)] -Aggregate Attributes [10]: [count(1)#51, min(cd_dep_count#20)#52, max(cd_dep_count#20)#53, avg(cd_dep_count#20)#54, min(cd_dep_employed_count#21)#55, max(cd_dep_employed_count#21)#56, avg(cd_dep_employed_count#21)#57, min(cd_dep_college_count#22)#58, max(cd_dep_college_count#22)#59, avg(cd_dep_college_count#22)#60] -Results [18]: [ca_state#16, cd_gender#23, cd_marital_status#24, count(1)#51 AS cnt1#61, min(cd_dep_count#20)#52 AS min(cd_dep_count)#62, max(cd_dep_count#20)#53 AS max(cd_dep_count)#63, avg(cd_dep_count#20)#54 AS avg(cd_dep_count)#64, cd_dep_employed_count#21, count(1)#51 AS cnt2#65, min(cd_dep_employed_count#21)#55 AS min(cd_dep_employed_count)#66, max(cd_dep_employed_count#21)#56 AS max(cd_dep_employed_count)#67, avg(cd_dep_employed_count#21)#57 AS avg(cd_dep_employed_count)#68, cd_dep_college_count#22, count(1)#51 AS cnt3#69, min(cd_dep_college_count#22)#58 AS min(cd_dep_college_count)#70, max(cd_dep_college_count#22)#59 AS max(cd_dep_college_count)#71, avg(cd_dep_college_count#22)#60 AS avg(cd_dep_college_count)#72, cd_dep_count#20] - -(45) TakeOrderedAndProject -Input [18]: [ca_state#16, cd_gender#23, cd_marital_status#24, cnt1#61, min(cd_dep_count)#62, max(cd_dep_count)#63, avg(cd_dep_count)#64, cd_dep_employed_count#21, cnt2#65, min(cd_dep_employed_count)#66, max(cd_dep_employed_count)#67, avg(cd_dep_employed_count)#68, cd_dep_college_count#22, cnt3#69, min(cd_dep_college_count)#70, max(cd_dep_college_count)#71, avg(cd_dep_college_count)#72, cd_dep_count#20] -Arguments: 100, [ca_state#16 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#20 ASC NULLS FIRST, cd_dep_employed_count#21 ASC NULLS FIRST, cd_dep_college_count#22 ASC NULLS FIRST], [ca_state#16, cd_gender#23, cd_marital_status#24, cnt1#61, min(cd_dep_count)#62, max(cd_dep_count)#63, avg(cd_dep_count)#64, cd_dep_employed_count#21, cnt2#65, min(cd_dep_employed_count)#66, max(cd_dep_employed_count)#67, avg(cd_dep_employed_count)#68, cd_dep_college_count#22, cnt3#69, min(cd_dep_college_count)#70, max(cd_dep_college_count)#71, avg(cd_dep_college_count)#72] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (50) -+- * CometColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometNativeScan parquet spark_catalog.default.date_dim (46) - - -(46) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#73, d_qoy#74] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_date_sk#9, d_year#73, d_qoy#74] -Condition : ((((isnotnull(d_year#73) AND isnotnull(d_qoy#74)) AND (d_year#73 = 2002)) AND (d_qoy#74 < 4)) AND isnotnull(d_date_sk#9)) - -(48) CometProject -Input [3]: [d_date_sk#9, d_year#73, d_qoy#74] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(49) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(50) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/simplified.txt deleted file mode 100644 index 48bc5637d0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/simplified.txt +++ /dev/null @@ -1,77 +0,0 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (10) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan 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/q35.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/explain.txt deleted file mode 100644 index f01bb79a29..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * Filter (27) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(6) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: [d_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#9] -Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -Arguments: [ss_customer_sk#6], [ss_customer_sk#6] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: [ss_customer_sk#6] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Right output [1]: [ss_customer_sk#6] -Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#9] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Right output [1]: [d_date_sk#9] -Arguments: [ws_sold_date_sk#13], [d_date_sk#9], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#9] -Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#12] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#12] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#9] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Right output [1]: [d_date_sk#9] -Arguments: [cs_sold_date_sk#16], [d_date_sk#9], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#9] -Arguments: [cs_ship_customer_sk#15], [cs_ship_customer_sk#15] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#15] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#15] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(27) Filter [codegen id : 5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(28) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_state#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [ca_address_sk#18, ca_state#19] -Condition : isnotnull(ca_address_sk#18) - -(31) CometProject -Input [2]: [ca_address_sk#18, ca_state#19] -Arguments: [ca_address_sk#18, ca_state#20], [ca_address_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#19, 2, true, false, true) AS ca_state#20] - -(32) CometColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#18, ca_state#20] - -(33) BroadcastExchange -Input [2]: [ca_address_sk#18, ca_state#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(34) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#18] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, ca_state#20] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18, ca_state#20] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(37) CometFilter -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Condition : isnotnull(cd_demo_sk#21) - -(38) CometProject -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#22, 1, true, false, true) AS cd_gender#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#23, 1, true, false, true) AS cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(39) CometColumnarToRow [codegen id : 4] -Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(40) BroadcastExchange -Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(41) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#21] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 5] -Output [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Input [8]: [c_current_cdemo_sk#4, ca_state#20, cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(43) HashAggregate [codegen id : 5] -Input [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#24), partial_max(cd_dep_count#24), partial_avg(cd_dep_count#24), partial_min(cd_dep_employed_count#25), partial_max(cd_dep_employed_count#25), partial_avg(cd_dep_employed_count#25), partial_min(cd_dep_college_count#26), partial_max(cd_dep_college_count#26), partial_avg(cd_dep_college_count#26)] -Aggregate Attributes [13]: [count#29, min#30, max#31, sum#32, count#33, min#34, max#35, sum#36, count#37, min#38, max#39, sum#40, count#41] -Results [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54] - -(44) CometColumnarExchange -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54] -Arguments: hashpartitioning(ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 6] -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54] - -(46) HashAggregate [codegen id : 6] -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54] -Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [10]: [count(1), min(cd_dep_count#24), max(cd_dep_count#24), avg(cd_dep_count#24), min(cd_dep_employed_count#25), max(cd_dep_employed_count#25), avg(cd_dep_employed_count#25), min(cd_dep_college_count#26), max(cd_dep_college_count#26), avg(cd_dep_college_count#26)] -Aggregate Attributes [10]: [count(1)#55, min(cd_dep_count#24)#56, max(cd_dep_count#24)#57, avg(cd_dep_count#24)#58, min(cd_dep_employed_count#25)#59, max(cd_dep_employed_count#25)#60, avg(cd_dep_employed_count#25)#61, min(cd_dep_college_count#26)#62, max(cd_dep_college_count#26)#63, avg(cd_dep_college_count#26)#64] -Results [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, count(1)#55 AS cnt1#65, min(cd_dep_count#24)#56 AS min(cd_dep_count)#66, max(cd_dep_count#24)#57 AS max(cd_dep_count)#67, avg(cd_dep_count#24)#58 AS avg(cd_dep_count)#68, cd_dep_employed_count#25, count(1)#55 AS cnt2#69, min(cd_dep_employed_count#25)#59 AS min(cd_dep_employed_count)#70, max(cd_dep_employed_count#25)#60 AS max(cd_dep_employed_count)#71, avg(cd_dep_employed_count#25)#61 AS avg(cd_dep_employed_count)#72, cd_dep_college_count#26, count(1)#55 AS cnt3#73, min(cd_dep_college_count#26)#62 AS min(cd_dep_college_count)#74, max(cd_dep_college_count#26)#63 AS max(cd_dep_college_count)#75, avg(cd_dep_college_count#26)#64 AS avg(cd_dep_college_count)#76, cd_dep_count#24] - -(47) TakeOrderedAndProject -Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cnt1#65, min(cd_dep_count)#66, max(cd_dep_count)#67, avg(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, min(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, avg(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, min(cd_dep_college_count)#74, max(cd_dep_college_count)#75, avg(cd_dep_college_count)#76, cd_dep_count#24] -Arguments: 100, [ca_state#20 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [ca_state#20, cd_gender#27, cd_marital_status#28, cnt1#65, min(cd_dep_count)#66, max(cd_dep_count)#67, avg(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, min(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, avg(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, min(cd_dep_college_count)#74, max(cd_dep_college_count)#75, avg(cd_dep_college_count)#76] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(52) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/simplified.txt deleted file mode 100644 index f1fe09fb46..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (6) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] 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/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt deleted file mode 100644 index f01bb79a29..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * Filter (27) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(6) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: [d_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#9] -Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -Arguments: [ss_customer_sk#6], [ss_customer_sk#6] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: [ss_customer_sk#6] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Right output [1]: [ss_customer_sk#6] -Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#9] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Right output [1]: [d_date_sk#9] -Arguments: [ws_sold_date_sk#13], [d_date_sk#9], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#9] -Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#12] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#12] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#9] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Right output [1]: [d_date_sk#9] -Arguments: [cs_sold_date_sk#16], [d_date_sk#9], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#9] -Arguments: [cs_ship_customer_sk#15], [cs_ship_customer_sk#15] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#15] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#15] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(27) Filter [codegen id : 5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(28) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_state#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [ca_address_sk#18, ca_state#19] -Condition : isnotnull(ca_address_sk#18) - -(31) CometProject -Input [2]: [ca_address_sk#18, ca_state#19] -Arguments: [ca_address_sk#18, ca_state#20], [ca_address_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#19, 2, true, false, true) AS ca_state#20] - -(32) CometColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#18, ca_state#20] - -(33) BroadcastExchange -Input [2]: [ca_address_sk#18, ca_state#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(34) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#18] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, ca_state#20] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18, ca_state#20] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(37) CometFilter -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Condition : isnotnull(cd_demo_sk#21) - -(38) CometProject -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#22, 1, true, false, true) AS cd_gender#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#23, 1, true, false, true) AS cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(39) CometColumnarToRow [codegen id : 4] -Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(40) BroadcastExchange -Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(41) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#21] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 5] -Output [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Input [8]: [c_current_cdemo_sk#4, ca_state#20, cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(43) HashAggregate [codegen id : 5] -Input [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#24), partial_max(cd_dep_count#24), partial_avg(cd_dep_count#24), partial_min(cd_dep_employed_count#25), partial_max(cd_dep_employed_count#25), partial_avg(cd_dep_employed_count#25), partial_min(cd_dep_college_count#26), partial_max(cd_dep_college_count#26), partial_avg(cd_dep_college_count#26)] -Aggregate Attributes [13]: [count#29, min#30, max#31, sum#32, count#33, min#34, max#35, sum#36, count#37, min#38, max#39, sum#40, count#41] -Results [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54] - -(44) CometColumnarExchange -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54] -Arguments: hashpartitioning(ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 6] -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54] - -(46) HashAggregate [codegen id : 6] -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, min#43, max#44, sum#45, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54] -Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [10]: [count(1), min(cd_dep_count#24), max(cd_dep_count#24), avg(cd_dep_count#24), min(cd_dep_employed_count#25), max(cd_dep_employed_count#25), avg(cd_dep_employed_count#25), min(cd_dep_college_count#26), max(cd_dep_college_count#26), avg(cd_dep_college_count#26)] -Aggregate Attributes [10]: [count(1)#55, min(cd_dep_count#24)#56, max(cd_dep_count#24)#57, avg(cd_dep_count#24)#58, min(cd_dep_employed_count#25)#59, max(cd_dep_employed_count#25)#60, avg(cd_dep_employed_count#25)#61, min(cd_dep_college_count#26)#62, max(cd_dep_college_count#26)#63, avg(cd_dep_college_count#26)#64] -Results [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, count(1)#55 AS cnt1#65, min(cd_dep_count#24)#56 AS min(cd_dep_count)#66, max(cd_dep_count#24)#57 AS max(cd_dep_count)#67, avg(cd_dep_count#24)#58 AS avg(cd_dep_count)#68, cd_dep_employed_count#25, count(1)#55 AS cnt2#69, min(cd_dep_employed_count#25)#59 AS min(cd_dep_employed_count)#70, max(cd_dep_employed_count#25)#60 AS max(cd_dep_employed_count)#71, avg(cd_dep_employed_count#25)#61 AS avg(cd_dep_employed_count)#72, cd_dep_college_count#26, count(1)#55 AS cnt3#73, min(cd_dep_college_count#26)#62 AS min(cd_dep_college_count)#74, max(cd_dep_college_count#26)#63 AS max(cd_dep_college_count)#75, avg(cd_dep_college_count#26)#64 AS avg(cd_dep_college_count)#76, cd_dep_count#24] - -(47) TakeOrderedAndProject -Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cnt1#65, min(cd_dep_count)#66, max(cd_dep_count)#67, avg(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, min(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, avg(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, min(cd_dep_college_count)#74, max(cd_dep_college_count)#75, avg(cd_dep_college_count)#76, cd_dep_count#24] -Arguments: 100, [ca_state#20 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [ca_state#20, cd_gender#27, cd_marital_status#28, cnt1#65, min(cd_dep_count)#66, max(cd_dep_count)#67, avg(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, min(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, avg(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, min(cd_dep_college_count)#74, max(cd_dep_college_count)#75, avg(cd_dep_college_count)#76] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(52) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/extended.txt deleted file mode 100644 index a6f33d6f7e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/extended.txt +++ /dev/null @@ -1,63 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index f1fe09fb46..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count)] - WholeStageCodegen (6) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/explain.txt deleted file mode 100644 index 58c6701f56..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/explain.txt +++ /dev/null @@ -1,207 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (31) -+- * Project (30) - +- Window (29) - +- * CometColumnarToRow (28) - +- CometSort (27) - +- CometColumnarExchange (26) - +- * HashAggregate (25) - +- * CometColumnarToRow (24) - +- CometColumnarExchange (23) - +- * HashAggregate (22) - +- * Expand (21) - +- * Project (20) - +- * BroadcastHashJoin Inner BuildRight (19) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) - : +- CometProject (9) - : +- CometFilter (8) - : +- CometNativeScan parquet spark_catalog.default.item (7) - +- BroadcastExchange (18) - +- * CometColumnarToRow (17) - +- CometProject (16) - +- CometFilter (15) - +- CometNativeScan parquet spark_catalog.default.store (14) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) - -(4) ReusedExchange [Reuses operator id: 36] -Output [1]: [d_date_sk#7] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] - -(7) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Condition : isnotnull(i_item_sk#8) - -(9) CometProject -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Arguments: [i_item_sk#8, i_class#11, i_category#12], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#12] - -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#8, i_class#11, i_category#12] - -(11) BroadcastExchange -Input [3]: [i_item_sk#8, i_class#11, i_category#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#8] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#8, i_class#11, i_category#12] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#13, s_state#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#13, s_state#14] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#14, 2, true, false, true) = TN) AND isnotnull(s_store_sk#13)) - -(16) CometProject -Input [2]: [s_store_sk#13, s_state#14] -Arguments: [s_store_sk#13], [s_store_sk#13] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#13] - -(18) BroadcastExchange -Input [1]: [s_store_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#13] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#12, i_class#11] -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12, s_store_sk#13] - -(21) Expand [codegen id : 4] -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#12, i_class#11] -Arguments: [[ss_ext_sales_price#3, ss_net_profit#4, i_category#12, i_class#11, 0], [ss_ext_sales_price#3, ss_net_profit#4, i_category#12, null, 1], [ss_ext_sales_price#3, ss_net_profit#4, null, null, 3]], [ss_ext_sales_price#3, ss_net_profit#4, i_category#15, i_class#16, spark_grouping_id#17] - -(22) HashAggregate [codegen id : 4] -Input [5]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#15, i_class#16, spark_grouping_id#17] -Keys [3]: [i_category#15, i_class#16, spark_grouping_id#17] -Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum#18, sum#19] -Results [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] - -(23) CometColumnarExchange -Input [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] -Arguments: hashpartitioning(i_category#15, i_class#16, spark_grouping_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometColumnarToRow [codegen id : 5] -Input [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] - -(25) HashAggregate [codegen id : 5] -Input [5]: [i_category#15, i_class#16, spark_grouping_id#17, sum#20, sum#21] -Keys [3]: [i_category#15, i_class#16, spark_grouping_id#17] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#22, sum(UnscaledValue(ss_ext_sales_price#3))#23] -Results [7]: [(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#22,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#23,17,2)) AS gross_margin#24, i_category#15, i_class#16, (cast((shiftright(spark_grouping_id#17, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#17, 0) & 1) as tinyint)) AS lochierarchy#25, (MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#22,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#23,17,2)) AS _w0#26, (cast((shiftright(spark_grouping_id#17, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#17, 0) & 1) as tinyint)) AS _w1#27, CASE WHEN (cast((shiftright(spark_grouping_id#17, 0) & 1) as tinyint) = 0) THEN i_category#15 END AS _w2#28] - -(26) CometColumnarExchange -Input [7]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28] -Arguments: hashpartitioning(_w1#27, _w2#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometSort -Input [7]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28] -Arguments: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28], [_w1#27 ASC NULLS FIRST, _w2#28 ASC NULLS FIRST, _w0#26 ASC NULLS FIRST] - -(28) CometColumnarToRow [codegen id : 6] -Input [7]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28] - -(29) Window -Input [7]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28] -Arguments: [rank(_w0#26) windowspecdefinition(_w1#27, _w2#28, _w0#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#29], [_w1#27, _w2#28], [_w0#26 ASC NULLS FIRST] - -(30) Project [codegen id : 7] -Output [5]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, rank_within_parent#29] -Input [8]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, _w0#26, _w1#27, _w2#28, rank_within_parent#29] - -(31) TakeOrderedAndProject -Input [5]: [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, rank_within_parent#29] -Arguments: 100, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#29 ASC NULLS FIRST], [gross_margin#24, i_category#15, i_class#16, lochierarchy#25, rank_within_parent#29] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.date_dim (32) - - -(32) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#7, d_year#30] -Condition : ((isnotnull(d_year#30) AND (d_year#30 = 2001)) AND isnotnull(d_date_sk#7)) - -(34) CometProject -Input [2]: [d_date_sk#7, d_year#30] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(36) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/simplified.txt deleted file mode 100644 index d6f8548c89..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (7) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Expand [ss_ext_sales_price,ss_net_profit,i_category,i_class] - Project [ss_ext_sales_price,ss_net_profit,i_category,i_class] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/explain.txt deleted file mode 100644 index ae1d4c242b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,203 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- Window (28) - +- * CometColumnarToRow (27) - +- CometSort (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExpand (21) - +- CometProject (20) - +- CometBroadcastHashJoin (19) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - +- CometBroadcastExchange (18) - +- CometProject (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4], [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#9, i_class#10, i_category#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Condition : isnotnull(i_item_sk#9) - -(11) CometProject -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Arguments: [i_item_sk#9, i_class#12, i_category#13], [i_item_sk#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#13] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [i_item_sk#9, i_class#12, i_category#13] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Right output [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#9], Inner, BuildRight - -(14) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) = TN) AND isnotnull(s_store_sk#14)) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14], [s_store_sk#14] - -(18) CometBroadcastExchange -Input [1]: [s_store_sk#14] -Arguments: [s_store_sk#14] - -(19) CometBroadcastHashJoin -Left output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -Right output [1]: [s_store_sk#14] -Arguments: [ss_store_sk#2], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13, s_store_sk#14] -Arguments: [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12], [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12] - -(21) CometExpand -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12] -Arguments: [[ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12, 0], [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, null, 1], [ss_ext_sales_price#3, ss_net_profit#4, null, null, 3]], [ss_ext_sales_price#3, ss_net_profit#4, i_category#16, i_class#17, spark_grouping_id#18] - -(22) CometHashAggregate -Input [5]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#16, i_class#17, spark_grouping_id#18] -Keys [3]: [i_category#16, i_class#17, spark_grouping_id#18] -Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(23) CometExchange -Input [5]: [i_category#16, i_class#17, spark_grouping_id#18, sum#19, sum#20] -Arguments: hashpartitioning(i_category#16, i_class#17, spark_grouping_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [5]: [i_category#16, i_class#17, spark_grouping_id#18, sum#19, sum#20] -Keys [3]: [i_category#16, i_class#17, spark_grouping_id#18] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] - -(25) CometExchange -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -Arguments: hashpartitioning(_w1#24, _w2#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(26) CometSort -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -Arguments: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25], [_w1#24 ASC NULLS FIRST, _w2#25 ASC NULLS FIRST, _w0#23 ASC NULLS FIRST] - -(27) CometColumnarToRow [codegen id : 1] -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] - -(28) Window -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -Arguments: [rank(_w0#23) windowspecdefinition(_w1#24, _w2#25, _w0#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#26], [_w1#24, _w2#25], [_w0#23 ASC NULLS FIRST] - -(29) Project [codegen id : 2] -Output [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] -Input [8]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25, rank_within_parent#26] - -(30) TakeOrderedAndProject -Input [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] -Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#26 ASC NULLS FIRST], [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(33) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(35) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/simplified.txt deleted file mode 100644 index 61782e3011..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,42 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (2) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometExchange [_w1,_w2] #1 - CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),spark_grouping_id] - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] - 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,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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #6 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt deleted file mode 100644 index ae1d4c242b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt +++ /dev/null @@ -1,203 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- Window (28) - +- * CometColumnarToRow (27) - +- CometSort (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExpand (21) - +- CometProject (20) - +- CometBroadcastHashJoin (19) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - +- CometBroadcastExchange (18) - +- CometProject (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4], [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#9, i_class#10, i_category#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Condition : isnotnull(i_item_sk#9) - -(11) CometProject -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Arguments: [i_item_sk#9, i_class#12, i_category#13], [i_item_sk#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#13] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [i_item_sk#9, i_class#12, i_category#13] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Right output [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#9], Inner, BuildRight - -(14) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) = TN) AND isnotnull(s_store_sk#14)) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14], [s_store_sk#14] - -(18) CometBroadcastExchange -Input [1]: [s_store_sk#14] -Arguments: [s_store_sk#14] - -(19) CometBroadcastHashJoin -Left output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -Right output [1]: [s_store_sk#14] -Arguments: [ss_store_sk#2], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13, s_store_sk#14] -Arguments: [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12], [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12] - -(21) CometExpand -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12] -Arguments: [[ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#12, 0], [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, null, 1], [ss_ext_sales_price#3, ss_net_profit#4, null, null, 3]], [ss_ext_sales_price#3, ss_net_profit#4, i_category#16, i_class#17, spark_grouping_id#18] - -(22) CometHashAggregate -Input [5]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#16, i_class#17, spark_grouping_id#18] -Keys [3]: [i_category#16, i_class#17, spark_grouping_id#18] -Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(23) CometExchange -Input [5]: [i_category#16, i_class#17, spark_grouping_id#18, sum#19, sum#20] -Arguments: hashpartitioning(i_category#16, i_class#17, spark_grouping_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [5]: [i_category#16, i_class#17, spark_grouping_id#18, sum#19, sum#20] -Keys [3]: [i_category#16, i_class#17, spark_grouping_id#18] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] - -(25) CometExchange -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -Arguments: hashpartitioning(_w1#24, _w2#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(26) CometSort -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -Arguments: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25], [_w1#24 ASC NULLS FIRST, _w2#25 ASC NULLS FIRST, _w0#23 ASC NULLS FIRST] - -(27) CometColumnarToRow [codegen id : 1] -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] - -(28) Window -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -Arguments: [rank(_w0#23) windowspecdefinition(_w1#24, _w2#25, _w0#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#26], [_w1#24, _w2#25], [_w0#23 ASC NULLS FIRST] - -(29) Project [codegen id : 2] -Output [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] -Input [8]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25, rank_within_parent#26] - -(30) TakeOrderedAndProject -Input [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] -Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#26 ASC NULLS FIRST], [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(33) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(35) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/extended.txt deleted file mode 100644 index 89c523a388..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/extended.txt +++ /dev/null @@ -1,38 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 61782e3011..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt +++ /dev/null @@ -1,42 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (2) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometExchange [_w1,_w2] #1 - CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),spark_grouping_id] - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] - 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,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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #6 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/explain.txt deleted file mode 100644 index e36c995169..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/explain.txt +++ /dev/null @@ -1,181 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometTakeOrderedAndProject (25) - +- CometHashAggregate (24) - +- CometColumnarExchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildLeft (20) - :- BroadcastExchange (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * CometColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (9) - : : +- * Project (8) - : : +- * Filter (7) - : : +- * ColumnarToRow (6) - : : +- Scan parquet spark_catalog.default.inventory (5) - : +- ReusedExchange (12) - +- * CometColumnarToRow (19) - +- CometProject (18) - +- CometFilter (17) - +- CometNativeScan parquet spark_catalog.default.catalog_sales (16) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,68.00), LessThanOrEqual(i_current_price,98.00), In(i_manufact_id, [677,694,808,940]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 68.00)) AND (i_current_price#4 <= 98.00)) AND i_manufact_id#5 IN (677,940,694,808)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#2, 16, true, false, true) AS i_item_id#6, i_item_desc#3, i_current_price#4] - -(4) CometColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(5) Scan parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#9), dynamicpruningexpression(inv_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] - -(7) Filter [codegen id : 1] -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Condition : (((isnotnull(inv_quantity_on_hand#8) AND (inv_quantity_on_hand#8 >= 100)) AND (inv_quantity_on_hand#8 <= 500)) AND isnotnull(inv_item_sk#7)) - -(8) Project [codegen id : 1] -Output [2]: [inv_item_sk#7, inv_date_sk#9] -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] - -(9) BroadcastExchange -Input [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [inv_item_sk#7] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 3] -Output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] - -(12) ReusedExchange [Reuses operator id: 31] -Output [1]: [d_date_sk#11] - -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [inv_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 3] -Output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] - -(15) BroadcastExchange -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#12, cs_sold_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [cs_item_sk#12, cs_sold_date_sk#13] -Condition : isnotnull(cs_item_sk#12) - -(18) CometProject -Input [2]: [cs_item_sk#12, cs_sold_date_sk#13] -Arguments: [cs_item_sk#12], [cs_item_sk#12] - -(19) CometColumnarToRow -Input [1]: [cs_item_sk#12] - -(20) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#12] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 4] -Output [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, cs_item_sk#12] - -(22) HashAggregate [codegen id : 4] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -(23) CometColumnarExchange -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(25) CometTakeOrderedAndProject -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(26) CometColumnarToRow [codegen id : 5] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometNativeScan parquet spark_catalog.default.date_dim (27) - - -(27) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#11, d_date#14] -Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 2000-02-01)) AND (d_date#14 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) - -(29) CometProject -Input [2]: [d_date_sk#11, d_date#14] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(31) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/simplified.txt deleted file mode 100644 index 88a3895990..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_current_price] - Project [i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [i_item_sk,cs_item_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (3) - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [inv_item_sk,inv_date_sk] - Filter [inv_quantity_on_hand,inv_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #4 - CometColumnarToRow - InputAdapter - CometProject [cs_item_sk] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometNativeScan 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/q37.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/explain.txt deleted file mode 100644 index e4cfdbf6d1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometTakeOrderedAndProject (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (4) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - +- CometProject (19) - +- CometFilter (18) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (17) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,68.00), LessThanOrEqual(i_current_price,98.00), In(i_manufact_id, [677,694,808,940]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 68.00)) AND (i_current_price#4 <= 98.00)) AND i_manufact_id#5 IN (677,940,694,808)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#2, 16, true, false, true) AS i_item_id#6, i_item_desc#3, i_current_price#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#9), dynamicpruningexpression(inv_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Condition : (((isnotnull(inv_quantity_on_hand#8) AND (inv_quantity_on_hand#8 >= 100)) AND (inv_quantity_on_hand#8 <= 500)) AND isnotnull(inv_item_sk#7)) - -(6) CometProject -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9], [inv_item_sk#7, inv_date_sk#9] - -(7) CometBroadcastExchange -Input [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1], [inv_item_sk#7], Inner, BuildRight - -(9) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#12 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) - -(12) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(14) CometBroadcastHashJoin -Left output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [inv_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(15) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(16) CometBroadcastExchange -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#13, cs_sold_date_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [cs_item_sk#13, cs_sold_date_sk#14] -Condition : isnotnull(cs_item_sk#13) - -(19) CometProject -Input [2]: [cs_item_sk#13, cs_sold_date_sk#14] -Arguments: [cs_item_sk#13], [cs_item_sk#13] - -(20) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [1]: [cs_item_sk#13] -Arguments: [i_item_sk#1], [cs_item_sk#13], Inner, BuildLeft - -(21) CometProject -Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, cs_item_sk#13] -Arguments: [i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(22) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(23) CometExchange -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(25) CometTakeOrderedAndProject -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(26) CometColumnarToRow [codegen id : 1] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#12 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) - -(29) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(31) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/simplified.txt deleted file mode 100644 index 7ab646a628..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometExchange [i_item_id,i_item_desc,i_current_price] #1 - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometProject [i_item_id,i_item_desc,i_current_price] - 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 [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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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/q37/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt deleted file mode 100644 index e4cfdbf6d1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometTakeOrderedAndProject (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (4) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - +- CometProject (19) - +- CometFilter (18) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (17) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,68.00), LessThanOrEqual(i_current_price,98.00), In(i_manufact_id, [677,694,808,940]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 68.00)) AND (i_current_price#4 <= 98.00)) AND i_manufact_id#5 IN (677,940,694,808)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#2, 16, true, false, true) AS i_item_id#6, i_item_desc#3, i_current_price#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#9), dynamicpruningexpression(inv_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Condition : (((isnotnull(inv_quantity_on_hand#8) AND (inv_quantity_on_hand#8 >= 100)) AND (inv_quantity_on_hand#8 <= 500)) AND isnotnull(inv_item_sk#7)) - -(6) CometProject -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9], [inv_item_sk#7, inv_date_sk#9] - -(7) CometBroadcastExchange -Input [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1], [inv_item_sk#7], Inner, BuildRight - -(9) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#12 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) - -(12) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(14) CometBroadcastHashJoin -Left output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [inv_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(15) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(16) CometBroadcastExchange -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#13, cs_sold_date_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [cs_item_sk#13, cs_sold_date_sk#14] -Condition : isnotnull(cs_item_sk#13) - -(19) CometProject -Input [2]: [cs_item_sk#13, cs_sold_date_sk#14] -Arguments: [cs_item_sk#13], [cs_item_sk#13] - -(20) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [1]: [cs_item_sk#13] -Arguments: [i_item_sk#1], [cs_item_sk#13], Inner, BuildLeft - -(21) CometProject -Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, cs_item_sk#13] -Arguments: [i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(22) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(23) CometExchange -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(25) CometTakeOrderedAndProject -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(26) CometColumnarToRow [codegen id : 1] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#12 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) - -(29) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(31) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/extended.txt deleted file mode 100644 index abe0937d1d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/extended.txt +++ /dev/null @@ -1,34 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 7ab646a628..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometExchange [i_item_id,i_item_desc,i_current_price] #1 - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometProject [i_item_id,i_item_desc,i_current_price] - 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 [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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/explain.txt deleted file mode 100644 index f1a8620998..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/explain.txt +++ /dev/null @@ -1,317 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (49) -+- CometHashAggregate (48) - +- CometExchange (47) - +- CometHashAggregate (46) - +- CometProject (45) - +- CometBroadcastHashJoin (44) - :- CometBroadcastHashJoin (30) - : :- CometHashAggregate (16) - : : +- CometColumnarExchange (15) - : : +- * HashAggregate (14) - : : +- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.customer (7) - : +- CometBroadcastExchange (29) - : +- CometHashAggregate (28) - : +- CometColumnarExchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet spark_catalog.default.catalog_sales (17) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - +- CometBroadcastExchange (43) - +- CometHashAggregate (42) - +- CometColumnarExchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Filter (33) - : : +- * ColumnarToRow (32) - : : +- Scan parquet spark_catalog.default.web_sales (31) - : +- ReusedExchange (34) - +- ReusedExchange (37) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sold_date_sk#2 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] - -(3) Filter [codegen id : 3] -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Condition : isnotnull(ss_customer_sk#1) - -(4) ReusedExchange [Reuses operator id: 54] -Output [2]: [d_date_sk#4, d_date#5] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#2] -Right keys [1]: [d_date_sk#4] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [2]: [ss_customer_sk#1, d_date#5] -Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] - -(7) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Condition : isnotnull(c_customer_sk#6) - -(9) CometProject -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Arguments: [c_customer_sk#6, c_first_name#9, c_last_name#10], [c_customer_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#7, 20, true, false, true) AS c_first_name#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#8, 30, true, false, true) AS c_last_name#10] - -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] - -(11) BroadcastExchange -Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#6] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [3]: [c_last_name#10, c_first_name#9, d_date#5] -Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#6, c_first_name#9, c_last_name#10] - -(14) HashAggregate [codegen id : 3] -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#5] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#5] - -(15) CometColumnarExchange -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] -Arguments: hashpartitioning(c_last_name#10, c_first_name#9, d_date#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometHashAggregate -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#5] -Functions: [] - -(17) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#12), dynamicpruningexpression(cs_sold_date_sk#12 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 6] -Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] - -(19) Filter [codegen id : 6] -Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] -Condition : isnotnull(cs_bill_customer_sk#11) - -(20) ReusedExchange [Reuses operator id: 54] -Output [2]: [d_date_sk#13, d_date#14] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#12] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [2]: [cs_bill_customer_sk#11, d_date#14] -Input [4]: [cs_bill_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13, d_date#14] - -(23) ReusedExchange [Reuses operator id: 11] -Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] - -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_bill_customer_sk#11] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [3]: [c_last_name#17, c_first_name#16, d_date#14] -Input [5]: [cs_bill_customer_sk#11, d_date#14, c_customer_sk#15, c_first_name#16, c_last_name#17] - -(26) HashAggregate [codegen id : 6] -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#17, c_first_name#16, d_date#14] - -(27) CometColumnarExchange -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, d_date#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometHashAggregate -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] -Functions: [] - -(29) CometBroadcastExchange -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Arguments: [c_last_name#17, c_first_name#16, d_date#14] - -(30) CometBroadcastHashJoin -Left output [3]: [c_last_name#10, c_first_name#9, d_date#5] -Right output [3]: [c_last_name#17, c_first_name#16, d_date#14] -Arguments: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)], [coalesce(c_last_name#17, ), isnull(c_last_name#17), coalesce(c_first_name#16, ), isnull(c_first_name#16), coalesce(d_date#14, 1970-01-01), isnull(d_date#14)], LeftSemi, BuildRight - -(31) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_sold_date_sk#19 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(32) ColumnarToRow [codegen id : 9] -Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] - -(33) Filter [codegen id : 9] -Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] -Condition : isnotnull(ws_bill_customer_sk#18) - -(34) ReusedExchange [Reuses operator id: 54] -Output [2]: [d_date_sk#20, d_date#21] - -(35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#19] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(36) Project [codegen id : 9] -Output [2]: [ws_bill_customer_sk#18, d_date#21] -Input [4]: [ws_bill_customer_sk#18, ws_sold_date_sk#19, d_date_sk#20, d_date#21] - -(37) ReusedExchange [Reuses operator id: 11] -Output [3]: [c_customer_sk#22, c_first_name#23, c_last_name#24] - -(38) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_bill_customer_sk#18] -Right keys [1]: [c_customer_sk#22] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 9] -Output [3]: [c_last_name#24, c_first_name#23, d_date#21] -Input [5]: [ws_bill_customer_sk#18, d_date#21, c_customer_sk#22, c_first_name#23, c_last_name#24] - -(40) HashAggregate [codegen id : 9] -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#24, c_first_name#23, d_date#21] - -(41) CometColumnarExchange -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, d_date#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(42) CometHashAggregate -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] -Functions: [] - -(43) CometBroadcastExchange -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: [c_last_name#24, c_first_name#23, d_date#21] - -(44) CometBroadcastHashJoin -Left output [3]: [c_last_name#10, c_first_name#9, d_date#5] -Right output [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)], [coalesce(c_last_name#24, ), isnull(c_last_name#24), coalesce(c_first_name#23, ), isnull(c_first_name#23), coalesce(d_date#21, 1970-01-01), isnull(d_date#21)], LeftSemi, BuildRight - -(45) CometProject -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] - -(46) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(47) CometExchange -Input [1]: [count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(48) CometHashAggregate -Input [1]: [count#25] -Keys: [] -Functions [1]: [count(1)] - -(49) CometColumnarToRow [codegen id : 10] -Input [1]: [count(1)#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometProject (52) - +- CometFilter (51) - +- CometNativeScan parquet spark_catalog.default.date_dim (50) - - -(50) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(51) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#27] -Condition : (((isnotnull(d_month_seq#27) AND (d_month_seq#27 >= 1200)) AND (d_month_seq#27 <= 1211)) AND isnotnull(d_date_sk#4)) - -(52) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#27] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(53) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] - -(54) BroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 17 Hosting Expression = cs_sold_date_sk#12 IN dynamicpruning#3 - -Subquery:3 Hosting operator id = 31 Hosting Expression = ws_sold_date_sk#19 IN dynamicpruning#3 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/simplified.txt deleted file mode 100644 index 547965d09c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/simplified.txt +++ /dev/null @@ -1,75 +0,0 @@ -WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometBroadcastExchange [c_last_name,c_first_name,d_date] #5 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 - CometBroadcastExchange [c_last_name,c_first_name,d_date] #7 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (9) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/explain.txt deleted file mode 100644 index 591fa114fb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,308 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (48) -+- CometHashAggregate (47) - +- CometExchange (46) - +- CometHashAggregate (45) - +- CometProject (44) - +- CometBroadcastHashJoin (43) - :- CometBroadcastHashJoin (30) - : :- CometHashAggregate (17) - : : +- CometExchange (16) - : : +- CometHashAggregate (15) - : : +- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : +- CometBroadcastExchange (29) - : +- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometFilter (19) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - +- CometBroadcastExchange (42) - +- CometHashAggregate (41) - +- CometExchange (40) - +- CometHashAggregate (39) - +- CometProject (38) - +- CometBroadcastHashJoin (37) - :- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (31) - : +- ReusedExchange (33) - +- ReusedExchange (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sold_date_sk#2 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Condition : isnotnull(ss_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(5) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: [d_date_sk#4, d_date#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Right output [2]: [d_date_sk#4, d_date#5] -Arguments: [ss_sold_date_sk#2], [d_date_sk#4], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] -Arguments: [ss_customer_sk#1, d_date#5], [ss_customer_sk#1, d_date#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Condition : isnotnull(c_customer_sk#7) - -(11) CometProject -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11], [c_customer_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#8, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#9, 30, true, false, true) AS c_last_name#11] - -(12) CometBroadcastExchange -Input [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, d_date#5] -Right output [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [ss_customer_sk#1], [c_customer_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_last_name#11, c_first_name#10, d_date#5], [c_last_name#11, c_first_name#10, d_date#5] - -(15) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(16) CometExchange -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(c_last_name#11, c_first_name#10, d_date#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Condition : isnotnull(cs_bill_customer_sk#12) - -(20) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#15, d_date#16] - -(21) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_date#16] -Arguments: [cs_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(22) CometProject -Input [4]: [cs_bill_customer_sk#12, cs_sold_date_sk#13, d_date_sk#15, d_date#16] -Arguments: [cs_bill_customer_sk#12, d_date#16], [cs_bill_customer_sk#12, d_date#16] - -(23) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] - -(24) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, d_date#16] -Right output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [cs_bill_customer_sk#12], [c_customer_sk#17], Inner, BuildRight - -(25) CometProject -Input [5]: [cs_bill_customer_sk#12, d_date#16, c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [c_last_name#19, c_first_name#18, d_date#16], [c_last_name#19, c_first_name#18, d_date#16] - -(26) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(27) CometExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: hashpartitioning(c_last_name#19, c_first_name#18, d_date#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(29) CometBroadcastExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: [c_last_name#19, c_first_name#18, d_date#16] - -(30) CometBroadcastHashJoin -Left output [3]: [c_last_name#11, c_first_name#10, d_date#5] -Right output [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)], [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 1970-01-01), isnull(d_date#16)], LeftSemi, BuildRight - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#21), dynamicpruningexpression(ws_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Condition : isnotnull(ws_bill_customer_sk#20) - -(33) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#23, d_date#24] - -(34) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Right output [2]: [d_date_sk#23, d_date#24] -Arguments: [ws_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(35) CometProject -Input [4]: [ws_bill_customer_sk#20, ws_sold_date_sk#21, d_date_sk#23, d_date#24] -Arguments: [ws_bill_customer_sk#20, d_date#24], [ws_bill_customer_sk#20, d_date#24] - -(36) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] - -(37) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, d_date#24] -Right output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [ws_bill_customer_sk#20], [c_customer_sk#25], Inner, BuildRight - -(38) CometProject -Input [5]: [ws_bill_customer_sk#20, d_date#24, c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_last_name#27, c_first_name#26, d_date#24], [c_last_name#27, c_first_name#26, d_date#24] - -(39) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(40) CometExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: hashpartitioning(c_last_name#27, c_first_name#26, d_date#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(41) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(42) CometBroadcastExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: [c_last_name#27, c_first_name#26, d_date#24] - -(43) CometBroadcastHashJoin -Left output [3]: [c_last_name#11, c_first_name#10, d_date#5] -Right output [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)], [coalesce(c_last_name#27, ), isnull(c_last_name#27), coalesce(c_first_name#26, ), isnull(c_first_name#26), coalesce(d_date#24, 1970-01-01), isnull(d_date#24)], LeftSemi, BuildRight - -(44) CometProject -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] - -(45) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(46) CometExchange -Input [1]: [count#28] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(47) CometHashAggregate -Input [1]: [count#28] -Keys: [] -Functions [1]: [count(1)] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [count(1)#29] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(51) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(52) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] - -(53) BroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#3 - -Subquery:3 Hosting operator id = 31 Hosting Expression = ws_sold_date_sk#21 IN dynamicpruning#3 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/simplified.txt deleted file mode 100644 index 7ceccf5d2c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,60 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #2 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] - CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_customer_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #4 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometBroadcastExchange [c_last_name,c_first_name,d_date] #6 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #7 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] - CometFilter [cs_bill_customer_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometBroadcastExchange [c_last_name,c_first_name,d_date] #8 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #9 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_bill_customer_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt deleted file mode 100644 index 591fa114fb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt +++ /dev/null @@ -1,308 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (48) -+- CometHashAggregate (47) - +- CometExchange (46) - +- CometHashAggregate (45) - +- CometProject (44) - +- CometBroadcastHashJoin (43) - :- CometBroadcastHashJoin (30) - : :- CometHashAggregate (17) - : : +- CometExchange (16) - : : +- CometHashAggregate (15) - : : +- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : +- CometBroadcastExchange (29) - : +- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometFilter (19) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - +- CometBroadcastExchange (42) - +- CometHashAggregate (41) - +- CometExchange (40) - +- CometHashAggregate (39) - +- CometProject (38) - +- CometBroadcastHashJoin (37) - :- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (31) - : +- ReusedExchange (33) - +- ReusedExchange (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sold_date_sk#2 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Condition : isnotnull(ss_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(5) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: [d_date_sk#4, d_date#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Right output [2]: [d_date_sk#4, d_date#5] -Arguments: [ss_sold_date_sk#2], [d_date_sk#4], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] -Arguments: [ss_customer_sk#1, d_date#5], [ss_customer_sk#1, d_date#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Condition : isnotnull(c_customer_sk#7) - -(11) CometProject -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11], [c_customer_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#8, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#9, 30, true, false, true) AS c_last_name#11] - -(12) CometBroadcastExchange -Input [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, d_date#5] -Right output [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [ss_customer_sk#1], [c_customer_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_last_name#11, c_first_name#10, d_date#5], [c_last_name#11, c_first_name#10, d_date#5] - -(15) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(16) CometExchange -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(c_last_name#11, c_first_name#10, d_date#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Condition : isnotnull(cs_bill_customer_sk#12) - -(20) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#15, d_date#16] - -(21) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_date#16] -Arguments: [cs_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(22) CometProject -Input [4]: [cs_bill_customer_sk#12, cs_sold_date_sk#13, d_date_sk#15, d_date#16] -Arguments: [cs_bill_customer_sk#12, d_date#16], [cs_bill_customer_sk#12, d_date#16] - -(23) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] - -(24) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, d_date#16] -Right output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [cs_bill_customer_sk#12], [c_customer_sk#17], Inner, BuildRight - -(25) CometProject -Input [5]: [cs_bill_customer_sk#12, d_date#16, c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [c_last_name#19, c_first_name#18, d_date#16], [c_last_name#19, c_first_name#18, d_date#16] - -(26) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(27) CometExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: hashpartitioning(c_last_name#19, c_first_name#18, d_date#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(29) CometBroadcastExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: [c_last_name#19, c_first_name#18, d_date#16] - -(30) CometBroadcastHashJoin -Left output [3]: [c_last_name#11, c_first_name#10, d_date#5] -Right output [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)], [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 1970-01-01), isnull(d_date#16)], LeftSemi, BuildRight - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#21), dynamicpruningexpression(ws_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Condition : isnotnull(ws_bill_customer_sk#20) - -(33) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#23, d_date#24] - -(34) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Right output [2]: [d_date_sk#23, d_date#24] -Arguments: [ws_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(35) CometProject -Input [4]: [ws_bill_customer_sk#20, ws_sold_date_sk#21, d_date_sk#23, d_date#24] -Arguments: [ws_bill_customer_sk#20, d_date#24], [ws_bill_customer_sk#20, d_date#24] - -(36) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] - -(37) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, d_date#24] -Right output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [ws_bill_customer_sk#20], [c_customer_sk#25], Inner, BuildRight - -(38) CometProject -Input [5]: [ws_bill_customer_sk#20, d_date#24, c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_last_name#27, c_first_name#26, d_date#24], [c_last_name#27, c_first_name#26, d_date#24] - -(39) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(40) CometExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: hashpartitioning(c_last_name#27, c_first_name#26, d_date#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(41) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(42) CometBroadcastExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: [c_last_name#27, c_first_name#26, d_date#24] - -(43) CometBroadcastHashJoin -Left output [3]: [c_last_name#11, c_first_name#10, d_date#5] -Right output [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)], [coalesce(c_last_name#27, ), isnull(c_last_name#27), coalesce(c_first_name#26, ), isnull(c_first_name#26), coalesce(d_date#24, 1970-01-01), isnull(d_date#24)], LeftSemi, BuildRight - -(44) CometProject -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] - -(45) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(46) CometExchange -Input [1]: [count#28] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(47) CometHashAggregate -Input [1]: [count#28] -Keys: [] -Functions [1]: [count(1)] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [count(1)#29] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(51) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(52) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] - -(53) BroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#3 - -Subquery:3 Hosting operator id = 31 Hosting Expression = ws_sold_date_sk#21 IN dynamicpruning#3 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/extended.txt deleted file mode 100644 index 55af64efc6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/extended.txt +++ /dev/null @@ -1,70 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 62 out of 66 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 7ceccf5d2c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/simplified.txt +++ /dev/null @@ -1,60 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #2 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] - CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_customer_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #4 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - CometBroadcastExchange [c_last_name,c_first_name,d_date] #6 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #7 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] - CometFilter [cs_bill_customer_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometBroadcastExchange [c_last_name,c_first_name,d_date] #8 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #9 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_bill_customer_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/explain.txt deleted file mode 100644 index b8ed3a361b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/explain.txt +++ /dev/null @@ -1,330 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometSort (46) - +- CometColumnarExchange (45) - +- * BroadcastHashJoin Inner BuildRight (44) - :- * Project (24) - : +- * Filter (23) - : +- * HashAggregate (22) - : +- * CometColumnarToRow (21) - : +- CometColumnarExchange (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.inventory (1) - : : : +- BroadcastExchange (7) - : : : +- * CometColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : +- BroadcastExchange (13) - : : +- * CometColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) - : +- ReusedExchange (16) - +- BroadcastExchange (43) - +- * Project (42) - +- * Filter (41) - +- * HashAggregate (40) - +- * CometColumnarToRow (39) - +- CometColumnarExchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * Filter (27) - : : : +- * ColumnarToRow (26) - : : : +- Scan parquet spark_catalog.default.inventory (25) - : : +- ReusedExchange (28) - : +- ReusedExchange (31) - +- ReusedExchange (34) - - -(1) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [1]: [i_item_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [1]: [i_item_sk#6] -Condition : isnotnull(i_item_sk#6) - -(6) CometColumnarToRow [codegen id : 1] -Input [1]: [i_item_sk#6] - -(7) BroadcastExchange -Input [1]: [i_item_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 4] -Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] - -(10) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Condition : isnotnull(w_warehouse_sk#7) - -(12) CometColumnarToRow [codegen id : 2] -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] - -(13) BroadcastExchange -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#7] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] - -(16) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#9, d_moy#10] - -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#10] - -(19) HashAggregate [codegen id : 4] -Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [5]: [n#11, avg#12, m2#13, sum#14, count#15] -Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] - -(20) CometColumnarExchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(21) CometColumnarToRow [codegen id : 10] -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] - -(22) HashAggregate [codegen id : 10] -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#21, avg(inv_quantity_on_hand#3)#22] -Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stddev_samp(cast(inv_quantity_on_hand#3 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#3)#22 AS mean#24] - -(23) Filter [codegen id : 10] -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.0) END - -(24) Project [codegen id : 10] -Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN null ELSE (stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))) END AS cov#25] -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] - -(25) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#29), dynamicpruningexpression(inv_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(26) ColumnarToRow [codegen id : 8] -Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] - -(27) Filter [codegen id : 8] -Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -Condition : (isnotnull(inv_item_sk#26) AND isnotnull(inv_warehouse_sk#27)) - -(28) ReusedExchange [Reuses operator id: 7] -Output [1]: [i_item_sk#31] - -(29) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_item_sk#26] -Right keys [1]: [i_item_sk#31] -Join type: Inner -Join condition: None - -(30) Project [codegen id : 8] -Output [4]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] -Input [5]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] - -(31) ReusedExchange [Reuses operator id: 13] -Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] - -(32) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_warehouse_sk#27] -Right keys [1]: [w_warehouse_sk#32] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] -Input [6]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] - -(34) ReusedExchange [Reuses operator id: 57] -Output [2]: [d_date_sk#34, d_moy#35] - -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_date_sk#29] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None - -(36) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] -Input [7]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_date_sk#34, d_moy#35] - -(37) HashAggregate [codegen id : 8] -Input [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] -Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#28 as double)), partial_avg(inv_quantity_on_hand#28)] -Aggregate Attributes [5]: [n#36, avg#37, m2#38, sum#39, count#40] -Results [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] - -(38) CometColumnarExchange -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(39) CometColumnarToRow [codegen id : 9] -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] - -(40) HashAggregate [codegen id : 9] -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] -Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#28)#22 AS mean#24] - -(41) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.0) END - -(42) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#24 AS mean#46, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN null ELSE (stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))) END AS cov#47] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] - -(43) BroadcastExchange -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] - -(44) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] -Right keys [2]: [i_item_sk#31, w_warehouse_sk#32] -Join type: Inner -Join condition: None - -(45) CometColumnarExchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(46) CometSort -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 11] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometNativeScan parquet spark_catalog.default.date_dim (48) - - -(48) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#48, d_moy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#48, d_moy#10] -Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#10)) AND (d_year#48 = 2001)) AND (d_moy#10 = 1)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#48, d_moy#10] -Arguments: [d_date_sk#9, d_moy#10], [d_date_sk#9, d_moy#10] - -(51) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#9, d_moy#10] - -(52) BroadcastExchange -Input [2]: [d_date_sk#9, d_moy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 25 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometNativeScan parquet spark_catalog.default.date_dim (53) - - -(53) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#49, d_moy#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#34, d_year#49, d_moy#35] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#35)) AND (d_year#49 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#34)) - -(55) CometProject -Input [3]: [d_date_sk#34, d_year#49, d_moy#35] -Arguments: [d_date_sk#34, d_moy#35], [d_date_sk#34, d_moy#35] - -(56) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#34, d_moy#35] - -(57) BroadcastExchange -Input [2]: [d_date_sk#34, d_moy#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/simplified.txt deleted file mode 100644 index 11dd59a97f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/simplified.txt +++ /dev/null @@ -1,84 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 - WholeStageCodegen (8) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [i_item_sk] #4 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - InputAdapter - ReusedExchange [d_date_sk,d_moy] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/explain.txt deleted file mode 100644 index ed74780c2c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,329 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometSort (46) - +- CometExchange (45) - +- CometBroadcastHashJoin (44) - :- CometProject (23) - : +- CometFilter (22) - : +- CometHashAggregate (21) - : +- CometExchange (20) - : +- CometHashAggregate (19) - : +- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (12) - : : +- CometBroadcastHashJoin (11) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : +- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - +- CometBroadcastExchange (43) - +- CometProject (42) - +- CometFilter (41) - +- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (28) - : : +- CometBroadcastHashJoin (27) - : : :- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (24) - : : +- ReusedExchange (26) - : +- ReusedExchange (29) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [i_item_sk#6] -Condition : isnotnull(i_item_sk#6) - -(5) CometBroadcastExchange -Input [1]: [i_item_sk#6] -Arguments: [i_item_sk#6] - -(6) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [i_item_sk#6] -Arguments: [inv_item_sk#1], [i_item_sk#6], Inner, BuildRight - -(7) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6], [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Condition : isnotnull(w_warehouse_sk#7) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [w_warehouse_sk#7, w_warehouse_name#8] - -(11) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Right output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#7], Inner, BuildRight - -(12) CometProject -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8], [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(15) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11] - -(17) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Right output [2]: [d_date_sk#9, d_moy#11] -Arguments: [inv_date_sk#4], [d_date_sk#9], Inner, BuildRight - -(18) CometProject -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#11] -Arguments: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11], [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] - -(19) CometHashAggregate -Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] - -(20) CometExchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] - -(22) CometFilter -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.0) END - -(23) CometProject -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19], [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN null ELSE (stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))) END AS cov#19] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#23), dynamicpruningexpression(inv_date_sk#23 IN dynamicpruning#24)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Condition : (isnotnull(inv_item_sk#20) AND isnotnull(inv_warehouse_sk#21)) - -(26) ReusedExchange [Reuses operator id: 5] -Output [1]: [i_item_sk#25] - -(27) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Right output [1]: [i_item_sk#25] -Arguments: [inv_item_sk#20], [i_item_sk#25], Inner, BuildRight - -(28) CometProject -Input [5]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Arguments: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25], [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] - -(29) ReusedExchange [Reuses operator id: 10] -Output [2]: [w_warehouse_sk#26, w_warehouse_name#27] - -(30) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Right output [2]: [w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_warehouse_sk#21], [w_warehouse_sk#26], Inner, BuildRight - -(31) CometProject -Input [6]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27], [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(34) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(35) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30] - -(36) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Right output [2]: [d_date_sk#28, d_moy#30] -Arguments: [inv_date_sk#23], [d_date_sk#28], Inner, BuildRight - -(37) CometProject -Input [7]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_date_sk#28, d_moy#30] -Arguments: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30], [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] - -(38) CometHashAggregate -Input [5]: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#22 as double)), partial_avg(inv_quantity_on_hand#22)] - -(39) CometExchange -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Arguments: hashpartitioning(w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(40) CometHashAggregate -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#22 as double)), avg(inv_quantity_on_hand#22)] - -(41) CometFilter -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.0) END - -(42) CometProject -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#18 AS mean#36, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN null ELSE (stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))) END AS cov#37] - -(43) CometBroadcastExchange -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] - -(44) CometBroadcastHashJoin -Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] -Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight - -(45) CometExchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(46) CometSort -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 1] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(51) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#9, d_moy#11] - -(52) BroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(55) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(56) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#28, d_moy#30] - -(57) BroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 2cd1f70c8a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,65 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [i_item_sk] #4 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [d_date_sk,d_moy] #6 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov] #7 - CometProject [mean,stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_moy] #10 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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/q39a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt deleted file mode 100644 index ed74780c2c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt +++ /dev/null @@ -1,329 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometSort (46) - +- CometExchange (45) - +- CometBroadcastHashJoin (44) - :- CometProject (23) - : +- CometFilter (22) - : +- CometHashAggregate (21) - : +- CometExchange (20) - : +- CometHashAggregate (19) - : +- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (12) - : : +- CometBroadcastHashJoin (11) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : +- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - +- CometBroadcastExchange (43) - +- CometProject (42) - +- CometFilter (41) - +- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (28) - : : +- CometBroadcastHashJoin (27) - : : :- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (24) - : : +- ReusedExchange (26) - : +- ReusedExchange (29) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [i_item_sk#6] -Condition : isnotnull(i_item_sk#6) - -(5) CometBroadcastExchange -Input [1]: [i_item_sk#6] -Arguments: [i_item_sk#6] - -(6) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [i_item_sk#6] -Arguments: [inv_item_sk#1], [i_item_sk#6], Inner, BuildRight - -(7) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6], [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Condition : isnotnull(w_warehouse_sk#7) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [w_warehouse_sk#7, w_warehouse_name#8] - -(11) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Right output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#7], Inner, BuildRight - -(12) CometProject -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8], [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(15) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11] - -(17) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Right output [2]: [d_date_sk#9, d_moy#11] -Arguments: [inv_date_sk#4], [d_date_sk#9], Inner, BuildRight - -(18) CometProject -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#11] -Arguments: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11], [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] - -(19) CometHashAggregate -Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] - -(20) CometExchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] - -(22) CometFilter -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.0) END - -(23) CometProject -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19], [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN null ELSE (stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))) END AS cov#19] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#23), dynamicpruningexpression(inv_date_sk#23 IN dynamicpruning#24)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Condition : (isnotnull(inv_item_sk#20) AND isnotnull(inv_warehouse_sk#21)) - -(26) ReusedExchange [Reuses operator id: 5] -Output [1]: [i_item_sk#25] - -(27) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Right output [1]: [i_item_sk#25] -Arguments: [inv_item_sk#20], [i_item_sk#25], Inner, BuildRight - -(28) CometProject -Input [5]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Arguments: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25], [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] - -(29) ReusedExchange [Reuses operator id: 10] -Output [2]: [w_warehouse_sk#26, w_warehouse_name#27] - -(30) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Right output [2]: [w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_warehouse_sk#21], [w_warehouse_sk#26], Inner, BuildRight - -(31) CometProject -Input [6]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27], [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(34) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(35) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30] - -(36) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Right output [2]: [d_date_sk#28, d_moy#30] -Arguments: [inv_date_sk#23], [d_date_sk#28], Inner, BuildRight - -(37) CometProject -Input [7]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_date_sk#28, d_moy#30] -Arguments: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30], [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] - -(38) CometHashAggregate -Input [5]: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#22 as double)), partial_avg(inv_quantity_on_hand#22)] - -(39) CometExchange -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Arguments: hashpartitioning(w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(40) CometHashAggregate -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#22 as double)), avg(inv_quantity_on_hand#22)] - -(41) CometFilter -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.0) END - -(42) CometProject -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#18 AS mean#36, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN null ELSE (stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))) END AS cov#37] - -(43) CometBroadcastExchange -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] - -(44) CometBroadcastHashJoin -Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] -Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight - -(45) CometExchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(46) CometSort -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 1] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(51) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#9, d_moy#11] - -(52) BroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(55) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(56) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#28, d_moy#30] - -(57) BroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/extended.txt deleted file mode 100644 index 8ff1442cf8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 2cd1f70c8a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt +++ /dev/null @@ -1,65 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [i_item_sk] #4 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [d_date_sk,d_moy] #6 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov] #7 - CometProject [mean,stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_moy] #10 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/explain.txt deleted file mode 100644 index 69fbdb52f0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/explain.txt +++ /dev/null @@ -1,330 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometSort (46) - +- CometColumnarExchange (45) - +- * BroadcastHashJoin Inner BuildRight (44) - :- * Project (24) - : +- * Filter (23) - : +- * HashAggregate (22) - : +- * CometColumnarToRow (21) - : +- CometColumnarExchange (20) - : +- * HashAggregate (19) - : +- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.inventory (1) - : : : +- BroadcastExchange (7) - : : : +- * CometColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : +- BroadcastExchange (13) - : : +- * CometColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) - : +- ReusedExchange (16) - +- BroadcastExchange (43) - +- * Project (42) - +- * Filter (41) - +- * HashAggregate (40) - +- * CometColumnarToRow (39) - +- CometColumnarExchange (38) - +- * HashAggregate (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * Filter (27) - : : : +- * ColumnarToRow (26) - : : : +- Scan parquet spark_catalog.default.inventory (25) - : : +- ReusedExchange (28) - : +- ReusedExchange (31) - +- ReusedExchange (34) - - -(1) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [1]: [i_item_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [1]: [i_item_sk#6] -Condition : isnotnull(i_item_sk#6) - -(6) CometColumnarToRow [codegen id : 1] -Input [1]: [i_item_sk#6] - -(7) BroadcastExchange -Input [1]: [i_item_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 4] -Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] - -(10) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Condition : isnotnull(w_warehouse_sk#7) - -(12) CometColumnarToRow [codegen id : 2] -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] - -(13) BroadcastExchange -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#7] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] - -(16) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#9, d_moy#10] - -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#10] - -(19) HashAggregate [codegen id : 4] -Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [5]: [n#11, avg#12, m2#13, sum#14, count#15] -Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] - -(20) CometColumnarExchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(21) CometColumnarToRow [codegen id : 10] -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] - -(22) HashAggregate [codegen id : 10] -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#21, avg(inv_quantity_on_hand#3)#22] -Results [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stddev_samp(cast(inv_quantity_on_hand#3 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#3)#22 AS mean#24] - -(23) Filter [codegen id : 10] -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] -Condition : (CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.0) END AND CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.5) END) - -(24) Project [codegen id : 10] -Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN null ELSE (stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))) END AS cov#25] -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] - -(25) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#29), dynamicpruningexpression(inv_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(26) ColumnarToRow [codegen id : 8] -Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] - -(27) Filter [codegen id : 8] -Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] -Condition : (isnotnull(inv_item_sk#26) AND isnotnull(inv_warehouse_sk#27)) - -(28) ReusedExchange [Reuses operator id: 7] -Output [1]: [i_item_sk#31] - -(29) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_item_sk#26] -Right keys [1]: [i_item_sk#31] -Join type: Inner -Join condition: None - -(30) Project [codegen id : 8] -Output [4]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] -Input [5]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] - -(31) ReusedExchange [Reuses operator id: 13] -Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] - -(32) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_warehouse_sk#27] -Right keys [1]: [w_warehouse_sk#32] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] -Input [6]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] - -(34) ReusedExchange [Reuses operator id: 57] -Output [2]: [d_date_sk#34, d_moy#35] - -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_date_sk#29] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None - -(36) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] -Input [7]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_date_sk#34, d_moy#35] - -(37) HashAggregate [codegen id : 8] -Input [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] -Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#28 as double)), partial_avg(inv_quantity_on_hand#28)] -Aggregate Attributes [5]: [n#36, avg#37, m2#38, sum#39, count#40] -Results [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] - -(38) CometColumnarExchange -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(39) CometColumnarToRow [codegen id : 9] -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] - -(40) HashAggregate [codegen id : 9] -Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] -Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] -Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#23, avg(inv_quantity_on_hand#28)#22 AS mean#24] - -(41) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))))) > 1.0) END - -(42) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#24 AS mean#46, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#24)) = 0.0) THEN null ELSE (stdev#23 / knownfloatingpointnormalized(normalizenanandzero(mean#24))) END AS cov#47] -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#23, mean#24] - -(43) BroadcastExchange -Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] - -(44) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] -Right keys [2]: [i_item_sk#31, w_warehouse_sk#32] -Join type: Inner -Join condition: None - -(45) CometColumnarExchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(46) CometSort -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#46 ASC NULLS FIRST, cov#47 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 11] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#46, cov#47] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometNativeScan parquet spark_catalog.default.date_dim (48) - - -(48) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#48, d_moy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#48, d_moy#10] -Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#10)) AND (d_year#48 = 2001)) AND (d_moy#10 = 1)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#48, d_moy#10] -Arguments: [d_date_sk#9, d_moy#10], [d_date_sk#9, d_moy#10] - -(51) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#9, d_moy#10] - -(52) BroadcastExchange -Input [2]: [d_date_sk#9, d_moy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 25 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometNativeScan parquet spark_catalog.default.date_dim (53) - - -(53) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#34, d_year#49, d_moy#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#34, d_year#49, d_moy#35] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#35)) AND (d_year#49 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#34)) - -(55) CometProject -Input [3]: [d_date_sk#34, d_year#49, d_moy#35] -Arguments: [d_date_sk#34, d_moy#35], [d_date_sk#34, d_moy#35] - -(56) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#34, d_moy#35] - -(57) BroadcastExchange -Input [2]: [d_date_sk#34, d_moy#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/simplified.txt deleted file mode 100644 index 11dd59a97f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/simplified.txt +++ /dev/null @@ -1,84 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (9) - Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] - Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 - WholeStageCodegen (8) - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] - Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [i_item_sk] #4 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 - InputAdapter - ReusedExchange [d_date_sk,d_moy] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/explain.txt deleted file mode 100644 index 2a31b0d8b2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,329 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometSort (46) - +- CometExchange (45) - +- CometBroadcastHashJoin (44) - :- CometProject (23) - : +- CometFilter (22) - : +- CometHashAggregate (21) - : +- CometExchange (20) - : +- CometHashAggregate (19) - : +- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (12) - : : +- CometBroadcastHashJoin (11) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : +- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - +- CometBroadcastExchange (43) - +- CometProject (42) - +- CometFilter (41) - +- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (28) - : : +- CometBroadcastHashJoin (27) - : : :- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (24) - : : +- ReusedExchange (26) - : +- ReusedExchange (29) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [i_item_sk#6] -Condition : isnotnull(i_item_sk#6) - -(5) CometBroadcastExchange -Input [1]: [i_item_sk#6] -Arguments: [i_item_sk#6] - -(6) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [i_item_sk#6] -Arguments: [inv_item_sk#1], [i_item_sk#6], Inner, BuildRight - -(7) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6], [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Condition : isnotnull(w_warehouse_sk#7) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [w_warehouse_sk#7, w_warehouse_name#8] - -(11) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Right output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#7], Inner, BuildRight - -(12) CometProject -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8], [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(15) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11] - -(17) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Right output [2]: [d_date_sk#9, d_moy#11] -Arguments: [inv_date_sk#4], [d_date_sk#9], Inner, BuildRight - -(18) CometProject -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#11] -Arguments: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11], [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] - -(19) CometHashAggregate -Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] - -(20) CometExchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] - -(22) CometFilter -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Condition : (CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.0) END AND CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.5) END) - -(23) CometProject -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19], [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN null ELSE (stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))) END AS cov#19] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#23), dynamicpruningexpression(inv_date_sk#23 IN dynamicpruning#24)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Condition : (isnotnull(inv_item_sk#20) AND isnotnull(inv_warehouse_sk#21)) - -(26) ReusedExchange [Reuses operator id: 5] -Output [1]: [i_item_sk#25] - -(27) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Right output [1]: [i_item_sk#25] -Arguments: [inv_item_sk#20], [i_item_sk#25], Inner, BuildRight - -(28) CometProject -Input [5]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Arguments: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25], [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] - -(29) ReusedExchange [Reuses operator id: 10] -Output [2]: [w_warehouse_sk#26, w_warehouse_name#27] - -(30) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Right output [2]: [w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_warehouse_sk#21], [w_warehouse_sk#26], Inner, BuildRight - -(31) CometProject -Input [6]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27], [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(34) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(35) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30] - -(36) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Right output [2]: [d_date_sk#28, d_moy#30] -Arguments: [inv_date_sk#23], [d_date_sk#28], Inner, BuildRight - -(37) CometProject -Input [7]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_date_sk#28, d_moy#30] -Arguments: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30], [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] - -(38) CometHashAggregate -Input [5]: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#22 as double)), partial_avg(inv_quantity_on_hand#22)] - -(39) CometExchange -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Arguments: hashpartitioning(w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(40) CometHashAggregate -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#22 as double)), avg(inv_quantity_on_hand#22)] - -(41) CometFilter -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.0) END - -(42) CometProject -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#18 AS mean#36, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN null ELSE (stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))) END AS cov#37] - -(43) CometBroadcastExchange -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] - -(44) CometBroadcastHashJoin -Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] -Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight - -(45) CometExchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(46) CometSort -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 1] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(51) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#9, d_moy#11] - -(52) BroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(55) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(56) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#28, d_moy#30] - -(57) BroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/simplified.txt deleted file mode 100644 index 2cd1f70c8a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,65 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [i_item_sk] #4 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [d_date_sk,d_moy] #6 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov] #7 - CometProject [mean,stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_moy] #10 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt deleted file mode 100644 index 2a31b0d8b2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt +++ /dev/null @@ -1,329 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometSort (46) - +- CometExchange (45) - +- CometBroadcastHashJoin (44) - :- CometProject (23) - : +- CometFilter (22) - : +- CometHashAggregate (21) - : +- CometExchange (20) - : +- CometHashAggregate (19) - : +- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (12) - : : +- CometBroadcastHashJoin (11) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : +- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - +- CometBroadcastExchange (43) - +- CometProject (42) - +- CometFilter (41) - +- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (28) - : : +- CometBroadcastHashJoin (27) - : : :- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (24) - : : +- ReusedExchange (26) - : +- ReusedExchange (29) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [i_item_sk#6] -Condition : isnotnull(i_item_sk#6) - -(5) CometBroadcastExchange -Input [1]: [i_item_sk#6] -Arguments: [i_item_sk#6] - -(6) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [i_item_sk#6] -Arguments: [inv_item_sk#1], [i_item_sk#6], Inner, BuildRight - -(7) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6], [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Condition : isnotnull(w_warehouse_sk#7) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [w_warehouse_sk#7, w_warehouse_name#8] - -(11) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6] -Right output [2]: [w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#7], Inner, BuildRight - -(12) CometProject -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Arguments: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8], [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(15) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11] - -(17) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8] -Right output [2]: [d_date_sk#9, d_moy#11] -Arguments: [inv_date_sk#4], [d_date_sk#9], Inner, BuildRight - -(18) CometProject -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#11] -Arguments: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11], [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] - -(19) CometHashAggregate -Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#11] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] - -(20) CometExchange -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11, n#12, avg#13, m2#14, sum#15, count#16] -Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#11] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] - -(22) CometFilter -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Condition : (CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.0) END AND CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.5) END) - -(23) CometProject -Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, stdev#17, mean#18] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19], [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN null ELSE (stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))) END AS cov#19] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#23), dynamicpruningexpression(inv_date_sk#23 IN dynamicpruning#24)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Condition : (isnotnull(inv_item_sk#20) AND isnotnull(inv_warehouse_sk#21)) - -(26) ReusedExchange [Reuses operator id: 5] -Output [1]: [i_item_sk#25] - -(27) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23] -Right output [1]: [i_item_sk#25] -Arguments: [inv_item_sk#20], [i_item_sk#25], Inner, BuildRight - -(28) CometProject -Input [5]: [inv_item_sk#20, inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Arguments: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25], [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] - -(29) ReusedExchange [Reuses operator id: 10] -Output [2]: [w_warehouse_sk#26, w_warehouse_name#27] - -(30) CometBroadcastHashJoin -Left output [4]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25] -Right output [2]: [w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_warehouse_sk#21], [w_warehouse_sk#26], Inner, BuildRight - -(31) CometProject -Input [6]: [inv_warehouse_sk#21, inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Arguments: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27], [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(34) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(35) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30] - -(36) CometBroadcastHashJoin -Left output [5]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27] -Right output [2]: [d_date_sk#28, d_moy#30] -Arguments: [inv_date_sk#23], [d_date_sk#28], Inner, BuildRight - -(37) CometProject -Input [7]: [inv_quantity_on_hand#22, inv_date_sk#23, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_date_sk#28, d_moy#30] -Arguments: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30], [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] - -(38) CometHashAggregate -Input [5]: [inv_quantity_on_hand#22, i_item_sk#25, w_warehouse_sk#26, w_warehouse_name#27, d_moy#30] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#22 as double)), partial_avg(inv_quantity_on_hand#22)] - -(39) CometExchange -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Arguments: hashpartitioning(w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(40) CometHashAggregate -Input [9]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30, n#31, avg#32, m2#33, sum#34, count#35] -Keys [4]: [w_warehouse_name#27, w_warehouse_sk#26, i_item_sk#25, d_moy#30] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#22 as double)), avg(inv_quantity_on_hand#22)] - -(41) CometFilter -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] -Condition : CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN false ELSE (knownfloatingpointnormalized(normalizenanandzero((stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))))) > 1.0) END - -(42) CometProject -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, stdev#17, mean#18] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#18 AS mean#36, CASE WHEN (knownfloatingpointnormalized(normalizenanandzero(mean#18)) = 0.0) THEN null ELSE (stdev#17 / knownfloatingpointnormalized(normalizenanandzero(mean#18))) END AS cov#37] - -(43) CometBroadcastExchange -Input [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] - -(44) CometBroadcastHashJoin -Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] -Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight - -(45) CometExchange -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(46) CometSort -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37], [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 1] -Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_moy#11)) AND (d_year#10 = 2001)) AND (d_moy#11 = 1)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Arguments: [d_date_sk#9, d_moy#11], [d_date_sk#9, d_moy#11] - -(51) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#9, d_moy#11] - -(52) BroadcastExchange -Input [2]: [d_date_sk#9, d_moy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) - - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2001)) AND (d_moy#30 = 2)) AND isnotnull(d_date_sk#28)) - -(55) CometProject -Input [3]: [d_date_sk#28, d_year#29, d_moy#30] -Arguments: [d_date_sk#28, d_moy#30], [d_date_sk#28, d_moy#30] - -(56) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#28, d_moy#30] - -(57) BroadcastExchange -Input [2]: [d_date_sk#28, d_moy#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/extended.txt deleted file mode 100644 index 8ff1442cf8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 2cd1f70c8a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt +++ /dev/null @@ -1,65 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 - CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [i_item_sk] #4 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [d_date_sk,d_moy] #6 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov] #7 - CometProject [mean,stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] - CometHashAggregate [w_warehouse_name,n,avg,m2,sum,count] [w_warehouse_sk,i_item_sk,d_moy,stdev,mean,stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand)] - CometExchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 - CometHashAggregate [inv_quantity_on_hand] [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] - CometProject [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] - 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,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,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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_moy] #10 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/explain.txt deleted file mode 100644 index 9fffd1dde5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/explain.txt +++ /dev/null @@ -1,752 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (120) -+- * Project (119) - +- * BroadcastHashJoin Inner BuildRight (118) - :- * Project (99) - : +- * BroadcastHashJoin Inner BuildRight (98) - : :- * Project (78) - : : +- * BroadcastHashJoin Inner BuildRight (77) - : : :- * Project (58) - : : : +- * BroadcastHashJoin Inner BuildRight (57) - : : : :- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Filter (18) - : : : : : +- * HashAggregate (17) - : : : : : +- * CometColumnarToRow (16) - : : : : : +- CometColumnarExchange (15) - : : : : : +- * HashAggregate (14) - : : : : : +- * Project (13) - : : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * CometColumnarToRow (4) - : : : : : : : +- CometProject (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * Filter (7) - : : : : : : +- * ColumnarToRow (6) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (5) - : : : : : +- ReusedExchange (11) - : : : : +- BroadcastExchange (36) - : : : : +- * HashAggregate (35) - : : : : +- * CometColumnarToRow (34) - : : : : +- CometColumnarExchange (33) - : : : : +- * HashAggregate (32) - : : : : +- * Project (31) - : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : :- * Project (28) - : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : :- * CometColumnarToRow (22) - : : : : : : +- CometProject (21) - : : : : : : +- CometFilter (20) - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) - : : : : : +- BroadcastExchange (26) - : : : : : +- * Filter (25) - : : : : : +- * ColumnarToRow (24) - : : : : : +- Scan parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (29) - : : : +- BroadcastExchange (56) - : : : +- * Filter (55) - : : : +- * HashAggregate (54) - : : : +- * CometColumnarToRow (53) - : : : +- CometColumnarExchange (52) - : : : +- * HashAggregate (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * Project (47) - : : : : +- * BroadcastHashJoin Inner BuildRight (46) - : : : : :- * CometColumnarToRow (41) - : : : : : +- CometProject (40) - : : : : : +- CometFilter (39) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (38) - : : : : +- BroadcastExchange (45) - : : : : +- * Filter (44) - : : : : +- * ColumnarToRow (43) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (42) - : : : +- ReusedExchange (48) - : : +- BroadcastExchange (76) - : : +- * HashAggregate (75) - : : +- * CometColumnarToRow (74) - : : +- CometColumnarExchange (73) - : : +- * HashAggregate (72) - : : +- * Project (71) - : : +- * BroadcastHashJoin Inner BuildRight (70) - : : :- * Project (68) - : : : +- * BroadcastHashJoin Inner BuildRight (67) - : : : :- * CometColumnarToRow (62) - : : : : +- CometProject (61) - : : : : +- CometFilter (60) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (59) - : : : +- BroadcastExchange (66) - : : : +- * Filter (65) - : : : +- * ColumnarToRow (64) - : : : +- Scan parquet spark_catalog.default.catalog_sales (63) - : : +- ReusedExchange (69) - : +- BroadcastExchange (97) - : +- * Filter (96) - : +- * HashAggregate (95) - : +- * CometColumnarToRow (94) - : +- CometColumnarExchange (93) - : +- * HashAggregate (92) - : +- * Project (91) - : +- * BroadcastHashJoin Inner BuildRight (90) - : :- * Project (88) - : : +- * BroadcastHashJoin Inner BuildRight (87) - : : :- * CometColumnarToRow (82) - : : : +- CometProject (81) - : : : +- CometFilter (80) - : : : +- CometNativeScan parquet spark_catalog.default.customer (79) - : : +- BroadcastExchange (86) - : : +- * Filter (85) - : : +- * ColumnarToRow (84) - : : +- Scan parquet spark_catalog.default.web_sales (83) - : +- ReusedExchange (89) - +- BroadcastExchange (117) - +- * HashAggregate (116) - +- * CometColumnarToRow (115) - +- CometColumnarExchange (114) - +- * HashAggregate (113) - +- * Project (112) - +- * BroadcastHashJoin Inner BuildRight (111) - :- * Project (109) - : +- * BroadcastHashJoin Inner BuildRight (108) - : :- * CometColumnarToRow (103) - : : +- CometProject (102) - : : +- CometFilter (101) - : : +- CometNativeScan parquet spark_catalog.default.customer (100) - : +- BroadcastExchange (107) - : +- * Filter (106) - : +- * ColumnarToRow (105) - : +- Scan parquet spark_catalog.default.web_sales (104) - +- ReusedExchange (110) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] - -(4) CometColumnarToRow [codegen id : 3] -Input [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] - -(5) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#20), dynamicpruningexpression(ss_sold_date_sk#20 IN dynamicpruning#21)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] - -(7) Filter [codegen id : 1] -Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Condition : isnotnull(ss_customer_sk#15) - -(8) BroadcastExchange -Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#15] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Input [14]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] - -(11) ReusedExchange [Reuses operator id: 124] -Output [2]: [d_date_sk#22, d_year#23] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#20] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] -Input [14]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20, d_date_sk#22, d_year#23] - -(14) HashAggregate [codegen id : 3] -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] -Functions [1]: [partial_sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] -Aggregate Attributes [2]: [sum#24, isEmpty#25] -Results [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#26, isEmpty#27] - -(15) CometColumnarExchange -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#26, isEmpty#27] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 24] -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#26, isEmpty#27] - -(17) HashAggregate [codegen id : 24] -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#26, isEmpty#27] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] -Functions [1]: [sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] -Aggregate Attributes [1]: [sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))#28] -Results [2]: [c_customer_id#9 AS customer_id#29, sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))#28 AS year_total#30] - -(18) Filter [codegen id : 24] -Input [2]: [customer_id#29, year_total#30] -Condition : (isnotnull(year_total#30) AND (year_total#30 > 0.000000)) - -(19) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_country#36, c_login#37, c_email_address#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(20) CometFilter -Input [8]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_country#36, c_login#37, c_email_address#38] -Condition : (isnotnull(c_customer_sk#31) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#32, 16, true, false, true))) - -(21) CometProject -Input [8]: [c_customer_sk#31, c_customer_id#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_country#36, c_login#37, c_email_address#38] -Arguments: [c_customer_sk#31, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14], [c_customer_sk#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#32, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#33, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#34, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#35, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#37, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#38, 50, true, false, true) AS c_email_address#14] - -(22) CometColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#31, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14] - -(23) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#44), dynamicpruningexpression(ss_sold_date_sk#44 IN dynamicpruning#45)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [6]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44] - -(25) Filter [codegen id : 4] -Input [6]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44] -Condition : isnotnull(ss_customer_sk#39) - -(26) BroadcastExchange -Input [6]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#31] -Right keys [1]: [ss_customer_sk#39] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44] -Input [14]: [c_customer_sk#31, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44] - -(29) ReusedExchange [Reuses operator id: 128] -Output [2]: [d_date_sk#46, d_year#47] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#44] -Right keys [1]: [d_date_sk#46] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, d_year#47] -Input [14]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, ss_sold_date_sk#44, d_date_sk#46, d_year#47] - -(32) HashAggregate [codegen id : 6] -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, ss_ext_discount_amt#40, ss_ext_sales_price#41, ss_ext_wholesale_cost#42, ss_ext_list_price#43, d_year#47] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47] -Functions [1]: [partial_sum(((((ss_ext_list_price#43 - ss_ext_wholesale_cost#42) - ss_ext_discount_amt#40) + ss_ext_sales_price#41) / 2))] -Aggregate Attributes [2]: [sum#48, isEmpty#49] -Results [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47, sum#50, isEmpty#51] - -(33) CometColumnarExchange -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47, sum#50, isEmpty#51] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 7] -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47, sum#50, isEmpty#51] - -(35) HashAggregate [codegen id : 7] -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47, sum#50, isEmpty#51] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#36, c_login#13, c_email_address#14, d_year#47] -Functions [1]: [sum(((((ss_ext_list_price#43 - ss_ext_wholesale_cost#42) - ss_ext_discount_amt#40) + ss_ext_sales_price#41) / 2))] -Aggregate Attributes [1]: [sum(((((ss_ext_list_price#43 - ss_ext_wholesale_cost#42) - ss_ext_discount_amt#40) + ss_ext_sales_price#41) / 2))#28] -Results [8]: [c_customer_id#9 AS customer_id#52, c_first_name#10 AS customer_first_name#53, c_last_name#11 AS customer_last_name#54, c_preferred_cust_flag#12 AS customer_preferred_cust_flag#55, c_birth_country#36 AS customer_birth_country#56, c_login#13 AS customer_login#57, c_email_address#14 AS customer_email_address#58, sum(((((ss_ext_list_price#43 - ss_ext_wholesale_cost#42) - ss_ext_discount_amt#40) + ss_ext_sales_price#41) / 2))#28 AS year_total#59] - -(36) BroadcastExchange -Input [8]: [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#59] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(37) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#29] -Right keys [1]: [customer_id#52] -Join type: Inner -Join condition: None - -(38) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(39) CometFilter -Input [8]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67] -Condition : (isnotnull(c_customer_sk#60) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#61, 16, true, false, true))) - -(40) CometProject -Input [8]: [c_customer_sk#60, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67] -Arguments: [c_customer_sk#60, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73], [c_customer_sk#60, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#61, 16, true, false, true) AS c_customer_id#68, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#62, 20, true, false, true) AS c_first_name#69, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#63, 30, true, false, true) AS c_last_name#70, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#64, 1, true, false, true) AS c_preferred_cust_flag#71, c_birth_country#65, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#66, 13, true, false, true) AS c_login#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#67, 50, true, false, true) AS c_email_address#73] - -(41) CometColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#60, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73] - -(42) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#74, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#79), dynamicpruningexpression(cs_sold_date_sk#79 IN dynamicpruning#21)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 8] -Input [6]: [cs_bill_customer_sk#74, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79] - -(44) Filter [codegen id : 8] -Input [6]: [cs_bill_customer_sk#74, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79] -Condition : isnotnull(cs_bill_customer_sk#74) - -(45) BroadcastExchange -Input [6]: [cs_bill_customer_sk#74, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -(46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#60] -Right keys [1]: [cs_bill_customer_sk#74] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 10] -Output [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79] -Input [14]: [c_customer_sk#60, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, cs_bill_customer_sk#74, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79] - -(48) ReusedExchange [Reuses operator id: 124] -Output [2]: [d_date_sk#80, d_year#81] - -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#79] -Right keys [1]: [d_date_sk#80] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 10] -Output [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, d_year#81] -Input [14]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, cs_sold_date_sk#79, d_date_sk#80, d_year#81] - -(51) HashAggregate [codegen id : 10] -Input [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, cs_ext_discount_amt#75, cs_ext_sales_price#76, cs_ext_wholesale_cost#77, cs_ext_list_price#78, d_year#81] -Keys [8]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81] -Functions [1]: [partial_sum(((((cs_ext_list_price#78 - cs_ext_wholesale_cost#77) - cs_ext_discount_amt#75) + cs_ext_sales_price#76) / 2))] -Aggregate Attributes [2]: [sum#82, isEmpty#83] -Results [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81, sum#84, isEmpty#85] - -(52) CometColumnarExchange -Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81, sum#84, isEmpty#85] -Arguments: hashpartitioning(c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(53) CometColumnarToRow [codegen id : 11] -Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81, sum#84, isEmpty#85] - -(54) HashAggregate [codegen id : 11] -Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81, sum#84, isEmpty#85] -Keys [8]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#65, c_login#72, c_email_address#73, d_year#81] -Functions [1]: [sum(((((cs_ext_list_price#78 - cs_ext_wholesale_cost#77) - cs_ext_discount_amt#75) + cs_ext_sales_price#76) / 2))] -Aggregate Attributes [1]: [sum(((((cs_ext_list_price#78 - cs_ext_wholesale_cost#77) - cs_ext_discount_amt#75) + cs_ext_sales_price#76) / 2))#86] -Results [2]: [c_customer_id#68 AS customer_id#87, sum(((((cs_ext_list_price#78 - cs_ext_wholesale_cost#77) - cs_ext_discount_amt#75) + cs_ext_sales_price#76) / 2))#86 AS year_total#88] - -(55) Filter [codegen id : 11] -Input [2]: [customer_id#87, year_total#88] -Condition : (isnotnull(year_total#88) AND (year_total#88 > 0.000000)) - -(56) BroadcastExchange -Input [2]: [customer_id#87, year_total#88] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(57) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#29] -Right keys [1]: [customer_id#87] -Join type: Inner -Join condition: None - -(58) Project [codegen id : 24] -Output [11]: [customer_id#29, year_total#30, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#59, year_total#88] -Input [12]: [customer_id#29, year_total#30, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#59, customer_id#87, year_total#88] - -(59) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#89, c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(60) CometFilter -Input [8]: [c_customer_sk#89, c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96] -Condition : (isnotnull(c_customer_sk#89) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#90, 16, true, false, true))) - -(61) CometProject -Input [8]: [c_customer_sk#89, c_customer_id#90, c_first_name#91, c_last_name#92, c_preferred_cust_flag#93, c_birth_country#94, c_login#95, c_email_address#96] -Arguments: [c_customer_sk#89, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73], [c_customer_sk#89, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#90, 16, true, false, true) AS c_customer_id#68, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#91, 20, true, false, true) AS c_first_name#69, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#92, 30, true, false, true) AS c_last_name#70, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#93, 1, true, false, true) AS c_preferred_cust_flag#71, c_birth_country#94, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#95, 13, true, false, true) AS c_login#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#96, 50, true, false, true) AS c_email_address#73] - -(62) CometColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#89, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73] - -(63) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#102), dynamicpruningexpression(cs_sold_date_sk#102 IN dynamicpruning#45)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(64) ColumnarToRow [codegen id : 12] -Input [6]: [cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] - -(65) Filter [codegen id : 12] -Input [6]: [cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] -Condition : isnotnull(cs_bill_customer_sk#97) - -(66) BroadcastExchange -Input [6]: [cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#89] -Right keys [1]: [cs_bill_customer_sk#97] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 14] -Output [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] -Input [14]: [c_customer_sk#89, c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, cs_bill_customer_sk#97, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102] - -(69) ReusedExchange [Reuses operator id: 128] -Output [2]: [d_date_sk#103, d_year#104] - -(70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [cs_sold_date_sk#102] -Right keys [1]: [d_date_sk#103] -Join type: Inner -Join condition: None - -(71) Project [codegen id : 14] -Output [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, d_year#104] -Input [14]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, cs_sold_date_sk#102, d_date_sk#103, d_year#104] - -(72) HashAggregate [codegen id : 14] -Input [12]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, cs_ext_discount_amt#98, cs_ext_sales_price#99, cs_ext_wholesale_cost#100, cs_ext_list_price#101, d_year#104] -Keys [8]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104] -Functions [1]: [partial_sum(((((cs_ext_list_price#101 - cs_ext_wholesale_cost#100) - cs_ext_discount_amt#98) + cs_ext_sales_price#99) / 2))] -Aggregate Attributes [2]: [sum#105, isEmpty#106] -Results [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104, sum#107, isEmpty#108] - -(73) CometColumnarExchange -Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104, sum#107, isEmpty#108] -Arguments: hashpartitioning(c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(74) CometColumnarToRow [codegen id : 15] -Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104, sum#107, isEmpty#108] - -(75) HashAggregate [codegen id : 15] -Input [10]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104, sum#107, isEmpty#108] -Keys [8]: [c_customer_id#68, c_first_name#69, c_last_name#70, c_preferred_cust_flag#71, c_birth_country#94, c_login#72, c_email_address#73, d_year#104] -Functions [1]: [sum(((((cs_ext_list_price#101 - cs_ext_wholesale_cost#100) - cs_ext_discount_amt#98) + cs_ext_sales_price#99) / 2))] -Aggregate Attributes [1]: [sum(((((cs_ext_list_price#101 - cs_ext_wholesale_cost#100) - cs_ext_discount_amt#98) + cs_ext_sales_price#99) / 2))#86] -Results [2]: [c_customer_id#68 AS customer_id#109, sum(((((cs_ext_list_price#101 - cs_ext_wholesale_cost#100) - cs_ext_discount_amt#98) + cs_ext_sales_price#99) / 2))#86 AS year_total#110] - -(76) BroadcastExchange -Input [2]: [customer_id#109, year_total#110] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] - -(77) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#29] -Right keys [1]: [customer_id#109] -Join type: Inner -Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN (year_total#110 / year_total#88) END > CASE WHEN (year_total#30 > 0.000000) THEN (year_total#59 / year_total#30) END) - -(78) Project [codegen id : 24] -Output [10]: [customer_id#29, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#88, year_total#110] -Input [13]: [customer_id#29, year_total#30, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#59, year_total#88, customer_id#109, year_total#110] - -(79) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#111, c_customer_id#112, c_first_name#113, c_last_name#114, c_preferred_cust_flag#115, c_birth_country#116, c_login#117, c_email_address#118] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(80) CometFilter -Input [8]: [c_customer_sk#111, c_customer_id#112, c_first_name#113, c_last_name#114, c_preferred_cust_flag#115, c_birth_country#116, c_login#117, c_email_address#118] -Condition : (isnotnull(c_customer_sk#111) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#112, 16, true, false, true))) - -(81) CometProject -Input [8]: [c_customer_sk#111, c_customer_id#112, c_first_name#113, c_last_name#114, c_preferred_cust_flag#115, c_birth_country#116, c_login#117, c_email_address#118] -Arguments: [c_customer_sk#111, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124], [c_customer_sk#111, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#112, 16, true, false, true) AS c_customer_id#119, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#113, 20, true, false, true) AS c_first_name#120, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#114, 30, true, false, true) AS c_last_name#121, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#115, 1, true, false, true) AS c_preferred_cust_flag#122, c_birth_country#116, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#117, 13, true, false, true) AS c_login#123, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#118, 50, true, false, true) AS c_email_address#124] - -(82) CometColumnarToRow [codegen id : 18] -Input [8]: [c_customer_sk#111, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124] - -(83) Scan parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#125, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#130), dynamicpruningexpression(ws_sold_date_sk#130 IN dynamicpruning#21)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(84) ColumnarToRow [codegen id : 16] -Input [6]: [ws_bill_customer_sk#125, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130] - -(85) Filter [codegen id : 16] -Input [6]: [ws_bill_customer_sk#125, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130] -Condition : isnotnull(ws_bill_customer_sk#125) - -(86) BroadcastExchange -Input [6]: [ws_bill_customer_sk#125, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -(87) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_customer_sk#111] -Right keys [1]: [ws_bill_customer_sk#125] -Join type: Inner -Join condition: None - -(88) Project [codegen id : 18] -Output [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130] -Input [14]: [c_customer_sk#111, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, ws_bill_customer_sk#125, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130] - -(89) ReusedExchange [Reuses operator id: 124] -Output [2]: [d_date_sk#131, d_year#132] - -(90) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#130] -Right keys [1]: [d_date_sk#131] -Join type: Inner -Join condition: None - -(91) Project [codegen id : 18] -Output [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, d_year#132] -Input [14]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, ws_sold_date_sk#130, d_date_sk#131, d_year#132] - -(92) HashAggregate [codegen id : 18] -Input [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, ws_ext_discount_amt#126, ws_ext_sales_price#127, ws_ext_wholesale_cost#128, ws_ext_list_price#129, d_year#132] -Keys [8]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132] -Functions [1]: [partial_sum(((((ws_ext_list_price#129 - ws_ext_wholesale_cost#128) - ws_ext_discount_amt#126) + ws_ext_sales_price#127) / 2))] -Aggregate Attributes [2]: [sum#133, isEmpty#134] -Results [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132, sum#135, isEmpty#136] - -(93) CometColumnarExchange -Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132, sum#135, isEmpty#136] -Arguments: hashpartitioning(c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(94) CometColumnarToRow [codegen id : 19] -Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132, sum#135, isEmpty#136] - -(95) HashAggregate [codegen id : 19] -Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132, sum#135, isEmpty#136] -Keys [8]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#116, c_login#123, c_email_address#124, d_year#132] -Functions [1]: [sum(((((ws_ext_list_price#129 - ws_ext_wholesale_cost#128) - ws_ext_discount_amt#126) + ws_ext_sales_price#127) / 2))] -Aggregate Attributes [1]: [sum(((((ws_ext_list_price#129 - ws_ext_wholesale_cost#128) - ws_ext_discount_amt#126) + ws_ext_sales_price#127) / 2))#137] -Results [2]: [c_customer_id#119 AS customer_id#138, sum(((((ws_ext_list_price#129 - ws_ext_wholesale_cost#128) - ws_ext_discount_amt#126) + ws_ext_sales_price#127) / 2))#137 AS year_total#139] - -(96) Filter [codegen id : 19] -Input [2]: [customer_id#138, year_total#139] -Condition : (isnotnull(year_total#139) AND (year_total#139 > 0.000000)) - -(97) BroadcastExchange -Input [2]: [customer_id#138, year_total#139] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=14] - -(98) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#29] -Right keys [1]: [customer_id#138] -Join type: Inner -Join condition: None - -(99) Project [codegen id : 24] -Output [11]: [customer_id#29, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#88, year_total#110, year_total#139] -Input [12]: [customer_id#29, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#88, year_total#110, customer_id#138, year_total#139] - -(100) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#140, c_customer_id#141, c_first_name#142, c_last_name#143, c_preferred_cust_flag#144, c_birth_country#145, c_login#146, c_email_address#147] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(101) CometFilter -Input [8]: [c_customer_sk#140, c_customer_id#141, c_first_name#142, c_last_name#143, c_preferred_cust_flag#144, c_birth_country#145, c_login#146, c_email_address#147] -Condition : (isnotnull(c_customer_sk#140) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#141, 16, true, false, true))) - -(102) CometProject -Input [8]: [c_customer_sk#140, c_customer_id#141, c_first_name#142, c_last_name#143, c_preferred_cust_flag#144, c_birth_country#145, c_login#146, c_email_address#147] -Arguments: [c_customer_sk#140, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124], [c_customer_sk#140, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#141, 16, true, false, true) AS c_customer_id#119, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#142, 20, true, false, true) AS c_first_name#120, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#143, 30, true, false, true) AS c_last_name#121, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#144, 1, true, false, true) AS c_preferred_cust_flag#122, c_birth_country#145, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#146, 13, true, false, true) AS c_login#123, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#147, 50, true, false, true) AS c_email_address#124] - -(103) CometColumnarToRow [codegen id : 22] -Input [8]: [c_customer_sk#140, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124] - -(104) Scan parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#153), dynamicpruningexpression(ws_sold_date_sk#153 IN dynamicpruning#45)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(105) ColumnarToRow [codegen id : 20] -Input [6]: [ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] - -(106) Filter [codegen id : 20] -Input [6]: [ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] -Condition : isnotnull(ws_bill_customer_sk#148) - -(107) BroadcastExchange -Input [6]: [ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] - -(108) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [c_customer_sk#140] -Right keys [1]: [ws_bill_customer_sk#148] -Join type: Inner -Join condition: None - -(109) Project [codegen id : 22] -Output [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] -Input [14]: [c_customer_sk#140, c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, ws_bill_customer_sk#148, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153] - -(110) ReusedExchange [Reuses operator id: 128] -Output [2]: [d_date_sk#154, d_year#155] - -(111) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#153] -Right keys [1]: [d_date_sk#154] -Join type: Inner -Join condition: None - -(112) Project [codegen id : 22] -Output [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, d_year#155] -Input [14]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, ws_sold_date_sk#153, d_date_sk#154, d_year#155] - -(113) HashAggregate [codegen id : 22] -Input [12]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, ws_ext_discount_amt#149, ws_ext_sales_price#150, ws_ext_wholesale_cost#151, ws_ext_list_price#152, d_year#155] -Keys [8]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155] -Functions [1]: [partial_sum(((((ws_ext_list_price#152 - ws_ext_wholesale_cost#151) - ws_ext_discount_amt#149) + ws_ext_sales_price#150) / 2))] -Aggregate Attributes [2]: [sum#156, isEmpty#157] -Results [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155, sum#158, isEmpty#159] - -(114) CometColumnarExchange -Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155, sum#158, isEmpty#159] -Arguments: hashpartitioning(c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(115) CometColumnarToRow [codegen id : 23] -Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155, sum#158, isEmpty#159] - -(116) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155, sum#158, isEmpty#159] -Keys [8]: [c_customer_id#119, c_first_name#120, c_last_name#121, c_preferred_cust_flag#122, c_birth_country#145, c_login#123, c_email_address#124, d_year#155] -Functions [1]: [sum(((((ws_ext_list_price#152 - ws_ext_wholesale_cost#151) - ws_ext_discount_amt#149) + ws_ext_sales_price#150) / 2))] -Aggregate Attributes [1]: [sum(((((ws_ext_list_price#152 - ws_ext_wholesale_cost#151) - ws_ext_discount_amt#149) + ws_ext_sales_price#150) / 2))#137] -Results [2]: [c_customer_id#119 AS customer_id#160, sum(((((ws_ext_list_price#152 - ws_ext_wholesale_cost#151) - ws_ext_discount_amt#149) + ws_ext_sales_price#150) / 2))#137 AS year_total#161] - -(117) BroadcastExchange -Input [2]: [customer_id#160, year_total#161] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=17] - -(118) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [customer_id#29] -Right keys [1]: [customer_id#160] -Join type: Inner -Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN (year_total#110 / year_total#88) END > CASE WHEN (year_total#139 > 0.000000) THEN (year_total#161 / year_total#139) END) - -(119) Project [codegen id : 24] -Output [7]: [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58] -Input [13]: [customer_id#29, customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58, year_total#88, year_total#110, year_total#139, customer_id#160, year_total#161] - -(120) TakeOrderedAndProject -Input [7]: [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58] -Arguments: 100, [customer_id#52 ASC NULLS FIRST, customer_first_name#53 ASC NULLS FIRST, customer_last_name#54 ASC NULLS FIRST, customer_preferred_cust_flag#55 ASC NULLS FIRST, customer_birth_country#56 ASC NULLS FIRST, customer_login#57 ASC NULLS FIRST, customer_email_address#58 ASC NULLS FIRST], [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 -BroadcastExchange (124) -+- * CometColumnarToRow (123) - +- CometFilter (122) - +- CometNativeScan parquet spark_catalog.default.date_dim (121) - - -(121) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_year#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(122) CometFilter -Input [2]: [d_date_sk#22, d_year#23] -Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(123) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#22, d_year#23] - -(124) BroadcastExchange -Input [2]: [d_date_sk#22, d_year#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#44 IN dynamicpruning#45 -BroadcastExchange (128) -+- * CometColumnarToRow (127) - +- CometFilter (126) - +- CometNativeScan parquet spark_catalog.default.date_dim (125) - - -(125) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#46, d_year#47] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#46, d_year#47] -Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2002)) AND isnotnull(d_date_sk#46)) - -(127) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#46, d_year#47] - -(128) BroadcastExchange -Input [2]: [d_date_sk#46, d_year#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] - -Subquery:3 Hosting operator id = 42 Hosting Expression = cs_sold_date_sk#79 IN dynamicpruning#21 - -Subquery:4 Hosting operator id = 63 Hosting Expression = cs_sold_date_sk#102 IN dynamicpruning#45 - -Subquery:5 Hosting operator id = 83 Hosting Expression = ws_sold_date_sk#130 IN dynamicpruning#21 - -Subquery:6 Hosting operator id = 104 Hosting Expression = ws_sold_date_sk#153 IN dynamicpruning#45 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/simplified.txt deleted file mode 100644 index 56bc32e144..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/simplified.txt +++ /dev/null @@ -1,191 +0,0 @@ -TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - WholeStageCodegen (24) - Project [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 - WholeStageCodegen (3) - HashAggregate [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] [sum,isEmpty,sum,isEmpty] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #5 - WholeStageCodegen (6) - HashAggregate [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] [sum,isEmpty,sum,isEmpty] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) - HashAggregate [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] [sum,isEmpty,sum,isEmpty] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) - HashAggregate [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] [sum,isEmpty,sum,isEmpty] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (19) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #15 - WholeStageCodegen (18) - HashAggregate [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] [sum,isEmpty,sum,isEmpty] - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (16) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (23) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #18 - WholeStageCodegen (22) - HashAggregate [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] [sum,isEmpty,sum,isEmpty] - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #19 - WholeStageCodegen (20) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/explain.txt deleted file mode 100644 index cea1c1d430..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,671 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (107) -+- CometTakeOrderedAndProject (106) - +- CometProject (105) - +- CometBroadcastHashJoin (104) - :- CometProject (88) - : +- CometBroadcastHashJoin (87) - : :- CometProject (70) - : : +- CometBroadcastHashJoin (69) - : : :- CometProject (53) - : : : +- CometBroadcastHashJoin (52) - : : : :- CometBroadcastHashJoin (35) - : : : : :- CometFilter (17) - : : : : : +- CometHashAggregate (16) - : : : : : +- CometExchange (15) - : : : : : +- CometHashAggregate (14) - : : : : : +- CometProject (13) - : : : : : +- CometBroadcastHashJoin (12) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometProject (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : : : +- CometBroadcastExchange (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : : : +- CometBroadcastExchange (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : : : +- CometBroadcastExchange (34) - : : : : +- CometHashAggregate (33) - : : : : +- CometExchange (32) - : : : : +- CometHashAggregate (31) - : : : : +- CometProject (30) - : : : : +- CometBroadcastHashJoin (29) - : : : : :- CometProject (25) - : : : : : +- CometBroadcastHashJoin (24) - : : : : : :- CometProject (20) - : : : : : : +- CometFilter (19) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : : : +- CometBroadcastExchange (23) - : : : : : +- CometFilter (22) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : : : +- CometBroadcastExchange (28) - : : : : +- CometFilter (27) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : : : +- CometBroadcastExchange (51) - : : : +- CometFilter (50) - : : : +- CometHashAggregate (49) - : : : +- CometExchange (48) - : : : +- CometHashAggregate (47) - : : : +- CometProject (46) - : : : +- CometBroadcastHashJoin (45) - : : : :- CometProject (43) - : : : : +- CometBroadcastHashJoin (42) - : : : : :- CometProject (38) - : : : : : +- CometFilter (37) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : : : +- CometBroadcastExchange (41) - : : : : +- CometFilter (40) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) - : : : +- ReusedExchange (44) - : : +- CometBroadcastExchange (68) - : : +- CometHashAggregate (67) - : : +- CometExchange (66) - : : +- CometHashAggregate (65) - : : +- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometProject (61) - : : : +- CometBroadcastHashJoin (60) - : : : :- CometProject (56) - : : : : +- CometFilter (55) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : : : +- CometBroadcastExchange (59) - : : : +- CometFilter (58) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (57) - : : +- ReusedExchange (62) - : +- CometBroadcastExchange (86) - : +- CometFilter (85) - : +- CometHashAggregate (84) - : +- CometExchange (83) - : +- CometHashAggregate (82) - : +- CometProject (81) - : +- CometBroadcastHashJoin (80) - : :- CometProject (78) - : : +- CometBroadcastHashJoin (77) - : : :- CometProject (73) - : : : +- CometFilter (72) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (71) - : : +- CometBroadcastExchange (76) - : : +- CometFilter (75) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (74) - : +- ReusedExchange (79) - +- CometBroadcastExchange (103) - +- CometHashAggregate (102) - +- CometExchange (101) - +- CometHashAggregate (100) - +- CometProject (99) - +- CometBroadcastHashJoin (98) - :- CometProject (96) - : +- CometBroadcastHashJoin (95) - : :- CometProject (91) - : : +- CometFilter (90) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (89) - : +- CometBroadcastExchange (94) - : +- CometFilter (93) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (92) - +- ReusedExchange (97) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#20), dynamicpruningexpression(ss_sold_date_sk#20 IN dynamicpruning#21)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Condition : isnotnull(ss_customer_sk#15) - -(6) CometBroadcastExchange -Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Arguments: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] - -(7) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Right output [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Arguments: [c_customer_sk#1], [ss_customer_sk#15], Inner, BuildRight - -(8) CometProject -Input [14]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_year#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#22, d_year#23] -Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#22, d_year#23] -Arguments: [d_date_sk#22, d_year#23] - -(12) CometBroadcastHashJoin -Left output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Right output [2]: [d_date_sk#22, d_year#23] -Arguments: [ss_sold_date_sk#20], [d_date_sk#22], Inner, BuildRight - -(13) CometProject -Input [14]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20, d_date_sk#22, d_year#23] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] - -(14) CometHashAggregate -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] -Functions [1]: [partial_sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] - -(15) CometExchange -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#24, isEmpty#25] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] -Functions [1]: [sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] - -(17) CometFilter -Input [2]: [customer_id#26, year_total#27] -Condition : (isnotnull(year_total#27) AND (year_total#27 > 0.000000)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] -Condition : (isnotnull(c_customer_sk#28) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#29, 16, true, false, true))) - -(20) CometProject -Input [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] -Arguments: [c_customer_sk#28, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14], [c_customer_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#29, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#30, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#31, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#32, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#34, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#35, 50, true, false, true) AS c_email_address#14] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#41), dynamicpruningexpression(ss_sold_date_sk#41 IN dynamicpruning#42)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Condition : isnotnull(ss_customer_sk#36) - -(23) CometBroadcastExchange -Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Arguments: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] - -(24) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#28, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14] -Right output [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Arguments: [c_customer_sk#28], [ss_customer_sk#36], Inner, BuildRight - -(25) CometProject -Input [14]: [c_customer_sk#28, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_year#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#43, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2002)) AND isnotnull(d_date_sk#43)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#43, d_year#44] -Arguments: [d_date_sk#43, d_year#44] - -(29) CometBroadcastHashJoin -Left output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Right output [2]: [d_date_sk#43, d_year#44] -Arguments: [ss_sold_date_sk#41], [d_date_sk#43], Inner, BuildRight - -(30) CometProject -Input [14]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41, d_date_sk#43, d_year#44] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, d_year#44], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, d_year#44] - -(31) CometHashAggregate -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, d_year#44] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, d_year#44] -Functions [1]: [partial_sum(((((ss_ext_list_price#40 - ss_ext_wholesale_cost#39) - ss_ext_discount_amt#37) + ss_ext_sales_price#38) / 2))] - -(32) CometExchange -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, d_year#44, sum#45, isEmpty#46] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, d_year#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, d_year#44, sum#45, isEmpty#46] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, d_year#44] -Functions [1]: [sum(((((ss_ext_list_price#40 - ss_ext_wholesale_cost#39) - ss_ext_discount_amt#37) + ss_ext_sales_price#38) / 2))] - -(34) CometBroadcastExchange -Input [8]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54] -Arguments: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#26, year_total#27] -Right output [8]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54] -Arguments: [customer_id#26], [customer_id#47], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [8]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62] -Condition : (isnotnull(c_customer_sk#55) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#56, 16, true, false, true))) - -(38) CometProject -Input [8]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62] -Arguments: [c_customer_sk#55, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68], [c_customer_sk#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#56, 16, true, false, true) AS c_customer_id#63, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#57, 20, true, false, true) AS c_first_name#64, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#58, 30, true, false, true) AS c_last_name#65, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#59, 1, true, false, true) AS c_preferred_cust_flag#66, c_birth_country#60, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#61, 13, true, false, true) AS c_login#67, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#62, 50, true, false, true) AS c_email_address#68] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_sold_date_sk#74 IN dynamicpruning#75)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [6]: [cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Condition : isnotnull(cs_bill_customer_sk#69) - -(41) CometBroadcastExchange -Input [6]: [cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Arguments: [cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] - -(42) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#55, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68] -Right output [6]: [cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Arguments: [c_customer_sk#55], [cs_bill_customer_sk#69], Inner, BuildRight - -(43) CometProject -Input [14]: [c_customer_sk#55, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Arguments: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74], [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#76, d_year#77] - -(45) CometBroadcastHashJoin -Left output [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Right output [2]: [d_date_sk#76, d_year#77] -Arguments: [cs_sold_date_sk#74], [d_date_sk#76], Inner, BuildRight - -(46) CometProject -Input [14]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74, d_date_sk#76, d_year#77] -Arguments: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, d_year#77], [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, d_year#77] - -(47) CometHashAggregate -Input [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, d_year#77] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#77] -Functions [1]: [partial_sum(((((cs_ext_list_price#73 - cs_ext_wholesale_cost#72) - cs_ext_discount_amt#70) + cs_ext_sales_price#71) / 2))] - -(48) CometExchange -Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#77, sum#78, isEmpty#79] -Arguments: hashpartitioning(c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#77, sum#78, isEmpty#79] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#77] -Functions [1]: [sum(((((cs_ext_list_price#73 - cs_ext_wholesale_cost#72) - cs_ext_discount_amt#70) + cs_ext_sales_price#71) / 2))] - -(50) CometFilter -Input [2]: [customer_id#80, year_total#81] -Condition : (isnotnull(year_total#81) AND (year_total#81 > 0.000000)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#80, year_total#81] -Arguments: [customer_id#80, year_total#81] - -(52) CometBroadcastHashJoin -Left output [10]: [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54] -Right output [2]: [customer_id#80, year_total#81] -Arguments: [customer_id#26], [customer_id#80], Inner, BuildRight - -(53) CometProject -Input [12]: [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54, customer_id#80, year_total#81] -Arguments: [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54, year_total#81], [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54, year_total#81] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] -Condition : (isnotnull(c_customer_sk#82) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#83, 16, true, false, true))) - -(56) CometProject -Input [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] -Arguments: [c_customer_sk#82, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68], [c_customer_sk#82, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#83, 16, true, false, true) AS c_customer_id#63, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#84, 20, true, false, true) AS c_first_name#64, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#85, 30, true, false, true) AS c_last_name#65, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#86, 1, true, false, true) AS c_preferred_cust_flag#66, c_birth_country#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#88, 13, true, false, true) AS c_login#67, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#89, 50, true, false, true) AS c_email_address#68] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#95), dynamicpruningexpression(cs_sold_date_sk#95 IN dynamicpruning#96)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Condition : isnotnull(cs_bill_customer_sk#90) - -(59) CometBroadcastExchange -Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Arguments: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] - -(60) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#82, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68] -Right output [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Arguments: [c_customer_sk#82], [cs_bill_customer_sk#90], Inner, BuildRight - -(61) CometProject -Input [14]: [c_customer_sk#82, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Arguments: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95], [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#97, d_year#98] - -(63) CometBroadcastHashJoin -Left output [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Right output [2]: [d_date_sk#97, d_year#98] -Arguments: [cs_sold_date_sk#95], [d_date_sk#97], Inner, BuildRight - -(64) CometProject -Input [14]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95, d_date_sk#97, d_year#98] -Arguments: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#98], [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#98] - -(65) CometHashAggregate -Input [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#98] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, d_year#98] -Functions [1]: [partial_sum(((((cs_ext_list_price#94 - cs_ext_wholesale_cost#93) - cs_ext_discount_amt#91) + cs_ext_sales_price#92) / 2))] - -(66) CometExchange -Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, d_year#98, sum#99, isEmpty#100] -Arguments: hashpartitioning(c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, d_year#98, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, d_year#98, sum#99, isEmpty#100] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, d_year#98] -Functions [1]: [sum(((((cs_ext_list_price#94 - cs_ext_wholesale_cost#93) - cs_ext_discount_amt#91) + cs_ext_sales_price#92) / 2))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#101, year_total#102] -Arguments: [customer_id#101, year_total#102] - -(69) CometBroadcastHashJoin -Left output [11]: [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54, year_total#81] -Right output [2]: [customer_id#101, year_total#102] -Arguments: [customer_id#26], [customer_id#101], Inner, (CASE WHEN (year_total#81 > 0.000000) THEN (year_total#102 / year_total#81) END > CASE WHEN (year_total#27 > 0.000000) THEN (year_total#54 / year_total#27) END), BuildRight - -(70) CometProject -Input [13]: [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54, year_total#81, customer_id#101, year_total#102] -Arguments: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102], [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102] - -(71) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#103, c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(72) CometFilter -Input [8]: [c_customer_sk#103, c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110] -Condition : (isnotnull(c_customer_sk#103) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#104, 16, true, false, true))) - -(73) CometProject -Input [8]: [c_customer_sk#103, c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110] -Arguments: [c_customer_sk#103, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116], [c_customer_sk#103, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#104, 16, true, false, true) AS c_customer_id#111, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#105, 20, true, false, true) AS c_first_name#112, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#106, 30, true, false, true) AS c_last_name#113, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#107, 1, true, false, true) AS c_preferred_cust_flag#114, c_birth_country#108, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#109, 13, true, false, true) AS c_login#115, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#110, 50, true, false, true) AS c_email_address#116] - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#122), dynamicpruningexpression(ws_sold_date_sk#122 IN dynamicpruning#123)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(75) CometFilter -Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Condition : isnotnull(ws_bill_customer_sk#117) - -(76) CometBroadcastExchange -Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Arguments: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] - -(77) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#103, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116] -Right output [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Arguments: [c_customer_sk#103], [ws_bill_customer_sk#117], Inner, BuildRight - -(78) CometProject -Input [14]: [c_customer_sk#103, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Arguments: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122], [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] - -(79) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#124, d_year#125] - -(80) CometBroadcastHashJoin -Left output [12]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Right output [2]: [d_date_sk#124, d_year#125] -Arguments: [ws_sold_date_sk#122], [d_date_sk#124], Inner, BuildRight - -(81) CometProject -Input [14]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122, d_date_sk#124, d_year#125] -Arguments: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#125], [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#125] - -(82) CometHashAggregate -Input [12]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#125] -Keys [8]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, d_year#125] -Functions [1]: [partial_sum(((((ws_ext_list_price#121 - ws_ext_wholesale_cost#120) - ws_ext_discount_amt#118) + ws_ext_sales_price#119) / 2))] - -(83) CometExchange -Input [10]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, d_year#125, sum#126, isEmpty#127] -Arguments: hashpartitioning(c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, d_year#125, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(84) CometHashAggregate -Input [10]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, d_year#125, sum#126, isEmpty#127] -Keys [8]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, d_year#125] -Functions [1]: [sum(((((ws_ext_list_price#121 - ws_ext_wholesale_cost#120) - ws_ext_discount_amt#118) + ws_ext_sales_price#119) / 2))] - -(85) CometFilter -Input [2]: [customer_id#128, year_total#129] -Condition : (isnotnull(year_total#129) AND (year_total#129 > 0.000000)) - -(86) CometBroadcastExchange -Input [2]: [customer_id#128, year_total#129] -Arguments: [customer_id#128, year_total#129] - -(87) CometBroadcastHashJoin -Left output [10]: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102] -Right output [2]: [customer_id#128, year_total#129] -Arguments: [customer_id#26], [customer_id#128], Inner, BuildRight - -(88) CometProject -Input [12]: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102, customer_id#128, year_total#129] -Arguments: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102, year_total#129], [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102, year_total#129] - -(89) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#130, c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(90) CometFilter -Input [8]: [c_customer_sk#130, c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137] -Condition : (isnotnull(c_customer_sk#130) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#131, 16, true, false, true))) - -(91) CometProject -Input [8]: [c_customer_sk#130, c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137] -Arguments: [c_customer_sk#130, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116], [c_customer_sk#130, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#131, 16, true, false, true) AS c_customer_id#111, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#132, 20, true, false, true) AS c_first_name#112, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#133, 30, true, false, true) AS c_last_name#113, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#134, 1, true, false, true) AS c_preferred_cust_flag#114, c_birth_country#135, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#136, 13, true, false, true) AS c_login#115, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#137, 50, true, false, true) AS c_email_address#116] - -(92) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#143), dynamicpruningexpression(ws_sold_date_sk#143 IN dynamicpruning#144)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(93) CometFilter -Input [6]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Condition : isnotnull(ws_bill_customer_sk#138) - -(94) CometBroadcastExchange -Input [6]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Arguments: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] - -(95) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#130, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116] -Right output [6]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Arguments: [c_customer_sk#130], [ws_bill_customer_sk#138], Inner, BuildRight - -(96) CometProject -Input [14]: [c_customer_sk#130, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Arguments: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143], [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] - -(97) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#145, d_year#146] - -(98) CometBroadcastHashJoin -Left output [12]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Right output [2]: [d_date_sk#145, d_year#146] -Arguments: [ws_sold_date_sk#143], [d_date_sk#145], Inner, BuildRight - -(99) CometProject -Input [14]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143, d_date_sk#145, d_year#146] -Arguments: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, d_year#146], [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, d_year#146] - -(100) CometHashAggregate -Input [12]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, d_year#146] -Keys [8]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, d_year#146] -Functions [1]: [partial_sum(((((ws_ext_list_price#142 - ws_ext_wholesale_cost#141) - ws_ext_discount_amt#139) + ws_ext_sales_price#140) / 2))] - -(101) CometExchange -Input [10]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, d_year#146, sum#147, isEmpty#148] -Arguments: hashpartitioning(c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, d_year#146, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(102) CometHashAggregate -Input [10]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, d_year#146, sum#147, isEmpty#148] -Keys [8]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, d_year#146] -Functions [1]: [sum(((((ws_ext_list_price#142 - ws_ext_wholesale_cost#141) - ws_ext_discount_amt#139) + ws_ext_sales_price#140) / 2))] - -(103) CometBroadcastExchange -Input [2]: [customer_id#149, year_total#150] -Arguments: [customer_id#149, year_total#150] - -(104) CometBroadcastHashJoin -Left output [11]: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102, year_total#129] -Right output [2]: [customer_id#149, year_total#150] -Arguments: [customer_id#26], [customer_id#149], Inner, (CASE WHEN (year_total#81 > 0.000000) THEN (year_total#102 / year_total#81) END > CASE WHEN (year_total#129 > 0.000000) THEN (year_total#150 / year_total#129) END), BuildRight - -(105) CometProject -Input [13]: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102, year_total#129, customer_id#149, year_total#150] -Arguments: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53], [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53] - -(106) CometTakeOrderedAndProject -Input [7]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#47 ASC NULLS FIRST,customer_first_name#48 ASC NULLS FIRST,customer_last_name#49 ASC NULLS FIRST,customer_preferred_cust_flag#50 ASC NULLS FIRST,customer_birth_country#51 ASC NULLS FIRST,customer_login#52 ASC NULLS FIRST,customer_email_address#53 ASC NULLS FIRST], output=[customer_id#47,customer_first_name#48,customer_last_name#49,customer_preferred_cust_flag#50,customer_birth_country#51,customer_login#52,customer_email_address#53]), [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53], 100, 0, [customer_id#47 ASC NULLS FIRST, customer_first_name#48 ASC NULLS FIRST, customer_last_name#49 ASC NULLS FIRST, customer_preferred_cust_flag#50 ASC NULLS FIRST, customer_birth_country#51 ASC NULLS FIRST, customer_login#52 ASC NULLS FIRST, customer_email_address#53 ASC NULLS FIRST], [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53] - -(107) CometColumnarToRow [codegen id : 1] -Input [7]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 -BroadcastExchange (111) -+- * CometColumnarToRow (110) - +- CometFilter (109) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) - - -(108) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_year#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(109) CometFilter -Input [2]: [d_date_sk#22, d_year#23] -Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(110) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#22, d_year#23] - -(111) BroadcastExchange -Input [2]: [d_date_sk#22, d_year#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#41 IN dynamicpruning#42 -BroadcastExchange (115) -+- * CometColumnarToRow (114) - +- CometFilter (113) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) - - -(112) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_year#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(113) CometFilter -Input [2]: [d_date_sk#43, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2002)) AND isnotnull(d_date_sk#43)) - -(114) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#43, d_year#44] - -(115) BroadcastExchange -Input [2]: [d_date_sk#43, d_year#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -Subquery:3 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#21 - -Subquery:4 Hosting operator id = 57 Hosting Expression = cs_sold_date_sk#95 IN dynamicpruning#42 - -Subquery:5 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#122 IN dynamicpruning#21 - -Subquery:6 Hosting operator id = 92 Hosting Expression = ws_sold_date_sk#143 IN dynamicpruning#42 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/simplified.txt deleted file mode 100644 index 36be965b48..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,127 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - CometProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - CometBroadcastHashJoin [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total,customer_id,year_total] - CometProject [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] - CometBroadcastHashJoin [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,customer_id,year_total] - CometProject [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 - CometHashAggregate [ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total] #5 - CometHashAggregate [d_year,sum,isEmpty] [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #6 - CometHashAggregate [ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #16 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #17 - CometHashAggregate [ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #19 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #20 - CometHashAggregate [ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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/q4/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt deleted file mode 100644 index cea1c1d430..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt +++ /dev/null @@ -1,671 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (107) -+- CometTakeOrderedAndProject (106) - +- CometProject (105) - +- CometBroadcastHashJoin (104) - :- CometProject (88) - : +- CometBroadcastHashJoin (87) - : :- CometProject (70) - : : +- CometBroadcastHashJoin (69) - : : :- CometProject (53) - : : : +- CometBroadcastHashJoin (52) - : : : :- CometBroadcastHashJoin (35) - : : : : :- CometFilter (17) - : : : : : +- CometHashAggregate (16) - : : : : : +- CometExchange (15) - : : : : : +- CometHashAggregate (14) - : : : : : +- CometProject (13) - : : : : : +- CometBroadcastHashJoin (12) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometProject (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : : : +- CometBroadcastExchange (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : : : +- CometBroadcastExchange (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : : : +- CometBroadcastExchange (34) - : : : : +- CometHashAggregate (33) - : : : : +- CometExchange (32) - : : : : +- CometHashAggregate (31) - : : : : +- CometProject (30) - : : : : +- CometBroadcastHashJoin (29) - : : : : :- CometProject (25) - : : : : : +- CometBroadcastHashJoin (24) - : : : : : :- CometProject (20) - : : : : : : +- CometFilter (19) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : : : +- CometBroadcastExchange (23) - : : : : : +- CometFilter (22) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : : : +- CometBroadcastExchange (28) - : : : : +- CometFilter (27) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : : : +- CometBroadcastExchange (51) - : : : +- CometFilter (50) - : : : +- CometHashAggregate (49) - : : : +- CometExchange (48) - : : : +- CometHashAggregate (47) - : : : +- CometProject (46) - : : : +- CometBroadcastHashJoin (45) - : : : :- CometProject (43) - : : : : +- CometBroadcastHashJoin (42) - : : : : :- CometProject (38) - : : : : : +- CometFilter (37) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : : : +- CometBroadcastExchange (41) - : : : : +- CometFilter (40) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) - : : : +- ReusedExchange (44) - : : +- CometBroadcastExchange (68) - : : +- CometHashAggregate (67) - : : +- CometExchange (66) - : : +- CometHashAggregate (65) - : : +- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometProject (61) - : : : +- CometBroadcastHashJoin (60) - : : : :- CometProject (56) - : : : : +- CometFilter (55) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : : : +- CometBroadcastExchange (59) - : : : +- CometFilter (58) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (57) - : : +- ReusedExchange (62) - : +- CometBroadcastExchange (86) - : +- CometFilter (85) - : +- CometHashAggregate (84) - : +- CometExchange (83) - : +- CometHashAggregate (82) - : +- CometProject (81) - : +- CometBroadcastHashJoin (80) - : :- CometProject (78) - : : +- CometBroadcastHashJoin (77) - : : :- CometProject (73) - : : : +- CometFilter (72) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (71) - : : +- CometBroadcastExchange (76) - : : +- CometFilter (75) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (74) - : +- ReusedExchange (79) - +- CometBroadcastExchange (103) - +- CometHashAggregate (102) - +- CometExchange (101) - +- CometHashAggregate (100) - +- CometProject (99) - +- CometBroadcastHashJoin (98) - :- CometProject (96) - : +- CometBroadcastHashJoin (95) - : :- CometProject (91) - : : +- CometFilter (90) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (89) - : +- CometBroadcastExchange (94) - : +- CometFilter (93) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (92) - +- ReusedExchange (97) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#20), dynamicpruningexpression(ss_sold_date_sk#20 IN dynamicpruning#21)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Condition : isnotnull(ss_customer_sk#15) - -(6) CometBroadcastExchange -Input [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Arguments: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] - -(7) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Right output [6]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Arguments: [c_customer_sk#1], [ss_customer_sk#15], Inner, BuildRight - -(8) CometProject -Input [14]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_year#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#22, d_year#23] -Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#22, d_year#23] -Arguments: [d_date_sk#22, d_year#23] - -(12) CometBroadcastHashJoin -Left output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20] -Right output [2]: [d_date_sk#22, d_year#23] -Arguments: [ss_sold_date_sk#20], [d_date_sk#22], Inner, BuildRight - -(13) CometProject -Input [14]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, ss_sold_date_sk#20, d_date_sk#22, d_year#23] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] - -(14) CometHashAggregate -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_sales_price#17, ss_ext_wholesale_cost#18, ss_ext_list_price#19, d_year#23] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] -Functions [1]: [partial_sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] - -(15) CometExchange -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23, sum#24, isEmpty#25] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, d_year#23] -Functions [1]: [sum(((((ss_ext_list_price#19 - ss_ext_wholesale_cost#18) - ss_ext_discount_amt#16) + ss_ext_sales_price#17) / 2))] - -(17) CometFilter -Input [2]: [customer_id#26, year_total#27] -Condition : (isnotnull(year_total#27) AND (year_total#27 > 0.000000)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] -Condition : (isnotnull(c_customer_sk#28) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#29, 16, true, false, true))) - -(20) CometProject -Input [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] -Arguments: [c_customer_sk#28, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14], [c_customer_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#29, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#30, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#31, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#32, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#34, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#35, 50, true, false, true) AS c_email_address#14] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#41), dynamicpruningexpression(ss_sold_date_sk#41 IN dynamicpruning#42)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Condition : isnotnull(ss_customer_sk#36) - -(23) CometBroadcastExchange -Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Arguments: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] - -(24) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#28, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14] -Right output [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Arguments: [c_customer_sk#28], [ss_customer_sk#36], Inner, BuildRight - -(25) CometProject -Input [14]: [c_customer_sk#28, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_year#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#43, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2002)) AND isnotnull(d_date_sk#43)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#43, d_year#44] -Arguments: [d_date_sk#43, d_year#44] - -(29) CometBroadcastHashJoin -Left output [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Right output [2]: [d_date_sk#43, d_year#44] -Arguments: [ss_sold_date_sk#41], [d_date_sk#43], Inner, BuildRight - -(30) CometProject -Input [14]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41, d_date_sk#43, d_year#44] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, d_year#44], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, d_year#44] - -(31) CometHashAggregate -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, d_year#44] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, d_year#44] -Functions [1]: [partial_sum(((((ss_ext_list_price#40 - ss_ext_wholesale_cost#39) - ss_ext_discount_amt#37) + ss_ext_sales_price#38) / 2))] - -(32) CometExchange -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, d_year#44, sum#45, isEmpty#46] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, d_year#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, d_year#44, sum#45, isEmpty#46] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#33, c_login#13, c_email_address#14, d_year#44] -Functions [1]: [sum(((((ss_ext_list_price#40 - ss_ext_wholesale_cost#39) - ss_ext_discount_amt#37) + ss_ext_sales_price#38) / 2))] - -(34) CometBroadcastExchange -Input [8]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54] -Arguments: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#26, year_total#27] -Right output [8]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54] -Arguments: [customer_id#26], [customer_id#47], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [8]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62] -Condition : (isnotnull(c_customer_sk#55) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#56, 16, true, false, true))) - -(38) CometProject -Input [8]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62] -Arguments: [c_customer_sk#55, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68], [c_customer_sk#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#56, 16, true, false, true) AS c_customer_id#63, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#57, 20, true, false, true) AS c_first_name#64, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#58, 30, true, false, true) AS c_last_name#65, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#59, 1, true, false, true) AS c_preferred_cust_flag#66, c_birth_country#60, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#61, 13, true, false, true) AS c_login#67, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#62, 50, true, false, true) AS c_email_address#68] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_sold_date_sk#74 IN dynamicpruning#75)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [6]: [cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Condition : isnotnull(cs_bill_customer_sk#69) - -(41) CometBroadcastExchange -Input [6]: [cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Arguments: [cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] - -(42) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#55, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68] -Right output [6]: [cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Arguments: [c_customer_sk#55], [cs_bill_customer_sk#69], Inner, BuildRight - -(43) CometProject -Input [14]: [c_customer_sk#55, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_bill_customer_sk#69, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Arguments: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74], [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#76, d_year#77] - -(45) CometBroadcastHashJoin -Left output [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74] -Right output [2]: [d_date_sk#76, d_year#77] -Arguments: [cs_sold_date_sk#74], [d_date_sk#76], Inner, BuildRight - -(46) CometProject -Input [14]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, cs_sold_date_sk#74, d_date_sk#76, d_year#77] -Arguments: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, d_year#77], [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, d_year#77] - -(47) CometHashAggregate -Input [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, cs_ext_discount_amt#70, cs_ext_sales_price#71, cs_ext_wholesale_cost#72, cs_ext_list_price#73, d_year#77] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#77] -Functions [1]: [partial_sum(((((cs_ext_list_price#73 - cs_ext_wholesale_cost#72) - cs_ext_discount_amt#70) + cs_ext_sales_price#71) / 2))] - -(48) CometExchange -Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#77, sum#78, isEmpty#79] -Arguments: hashpartitioning(c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#77, sum#78, isEmpty#79] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#77] -Functions [1]: [sum(((((cs_ext_list_price#73 - cs_ext_wholesale_cost#72) - cs_ext_discount_amt#70) + cs_ext_sales_price#71) / 2))] - -(50) CometFilter -Input [2]: [customer_id#80, year_total#81] -Condition : (isnotnull(year_total#81) AND (year_total#81 > 0.000000)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#80, year_total#81] -Arguments: [customer_id#80, year_total#81] - -(52) CometBroadcastHashJoin -Left output [10]: [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54] -Right output [2]: [customer_id#80, year_total#81] -Arguments: [customer_id#26], [customer_id#80], Inner, BuildRight - -(53) CometProject -Input [12]: [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54, customer_id#80, year_total#81] -Arguments: [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54, year_total#81], [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54, year_total#81] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] -Condition : (isnotnull(c_customer_sk#82) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#83, 16, true, false, true))) - -(56) CometProject -Input [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] -Arguments: [c_customer_sk#82, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68], [c_customer_sk#82, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#83, 16, true, false, true) AS c_customer_id#63, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#84, 20, true, false, true) AS c_first_name#64, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#85, 30, true, false, true) AS c_last_name#65, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#86, 1, true, false, true) AS c_preferred_cust_flag#66, c_birth_country#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#88, 13, true, false, true) AS c_login#67, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#89, 50, true, false, true) AS c_email_address#68] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#95), dynamicpruningexpression(cs_sold_date_sk#95 IN dynamicpruning#96)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Condition : isnotnull(cs_bill_customer_sk#90) - -(59) CometBroadcastExchange -Input [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Arguments: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] - -(60) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#82, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68] -Right output [6]: [cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Arguments: [c_customer_sk#82], [cs_bill_customer_sk#90], Inner, BuildRight - -(61) CometProject -Input [14]: [c_customer_sk#82, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_bill_customer_sk#90, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Arguments: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95], [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#97, d_year#98] - -(63) CometBroadcastHashJoin -Left output [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95] -Right output [2]: [d_date_sk#97, d_year#98] -Arguments: [cs_sold_date_sk#95], [d_date_sk#97], Inner, BuildRight - -(64) CometProject -Input [14]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, cs_sold_date_sk#95, d_date_sk#97, d_year#98] -Arguments: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#98], [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#98] - -(65) CometHashAggregate -Input [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, cs_ext_discount_amt#91, cs_ext_sales_price#92, cs_ext_wholesale_cost#93, cs_ext_list_price#94, d_year#98] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, d_year#98] -Functions [1]: [partial_sum(((((cs_ext_list_price#94 - cs_ext_wholesale_cost#93) - cs_ext_discount_amt#91) + cs_ext_sales_price#92) / 2))] - -(66) CometExchange -Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, d_year#98, sum#99, isEmpty#100] -Arguments: hashpartitioning(c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, d_year#98, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, d_year#98, sum#99, isEmpty#100] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#87, c_login#67, c_email_address#68, d_year#98] -Functions [1]: [sum(((((cs_ext_list_price#94 - cs_ext_wholesale_cost#93) - cs_ext_discount_amt#91) + cs_ext_sales_price#92) / 2))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#101, year_total#102] -Arguments: [customer_id#101, year_total#102] - -(69) CometBroadcastHashJoin -Left output [11]: [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54, year_total#81] -Right output [2]: [customer_id#101, year_total#102] -Arguments: [customer_id#26], [customer_id#101], Inner, (CASE WHEN (year_total#81 > 0.000000) THEN (year_total#102 / year_total#81) END > CASE WHEN (year_total#27 > 0.000000) THEN (year_total#54 / year_total#27) END), BuildRight - -(70) CometProject -Input [13]: [customer_id#26, year_total#27, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#54, year_total#81, customer_id#101, year_total#102] -Arguments: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102], [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102] - -(71) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#103, c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(72) CometFilter -Input [8]: [c_customer_sk#103, c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110] -Condition : (isnotnull(c_customer_sk#103) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#104, 16, true, false, true))) - -(73) CometProject -Input [8]: [c_customer_sk#103, c_customer_id#104, c_first_name#105, c_last_name#106, c_preferred_cust_flag#107, c_birth_country#108, c_login#109, c_email_address#110] -Arguments: [c_customer_sk#103, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116], [c_customer_sk#103, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#104, 16, true, false, true) AS c_customer_id#111, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#105, 20, true, false, true) AS c_first_name#112, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#106, 30, true, false, true) AS c_last_name#113, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#107, 1, true, false, true) AS c_preferred_cust_flag#114, c_birth_country#108, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#109, 13, true, false, true) AS c_login#115, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#110, 50, true, false, true) AS c_email_address#116] - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#122), dynamicpruningexpression(ws_sold_date_sk#122 IN dynamicpruning#123)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(75) CometFilter -Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Condition : isnotnull(ws_bill_customer_sk#117) - -(76) CometBroadcastExchange -Input [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Arguments: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] - -(77) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#103, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116] -Right output [6]: [ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Arguments: [c_customer_sk#103], [ws_bill_customer_sk#117], Inner, BuildRight - -(78) CometProject -Input [14]: [c_customer_sk#103, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_bill_customer_sk#117, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Arguments: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122], [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] - -(79) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#124, d_year#125] - -(80) CometBroadcastHashJoin -Left output [12]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122] -Right output [2]: [d_date_sk#124, d_year#125] -Arguments: [ws_sold_date_sk#122], [d_date_sk#124], Inner, BuildRight - -(81) CometProject -Input [14]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, ws_sold_date_sk#122, d_date_sk#124, d_year#125] -Arguments: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#125], [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#125] - -(82) CometHashAggregate -Input [12]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, ws_ext_discount_amt#118, ws_ext_sales_price#119, ws_ext_wholesale_cost#120, ws_ext_list_price#121, d_year#125] -Keys [8]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, d_year#125] -Functions [1]: [partial_sum(((((ws_ext_list_price#121 - ws_ext_wholesale_cost#120) - ws_ext_discount_amt#118) + ws_ext_sales_price#119) / 2))] - -(83) CometExchange -Input [10]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, d_year#125, sum#126, isEmpty#127] -Arguments: hashpartitioning(c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, d_year#125, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(84) CometHashAggregate -Input [10]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, d_year#125, sum#126, isEmpty#127] -Keys [8]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#108, c_login#115, c_email_address#116, d_year#125] -Functions [1]: [sum(((((ws_ext_list_price#121 - ws_ext_wholesale_cost#120) - ws_ext_discount_amt#118) + ws_ext_sales_price#119) / 2))] - -(85) CometFilter -Input [2]: [customer_id#128, year_total#129] -Condition : (isnotnull(year_total#129) AND (year_total#129 > 0.000000)) - -(86) CometBroadcastExchange -Input [2]: [customer_id#128, year_total#129] -Arguments: [customer_id#128, year_total#129] - -(87) CometBroadcastHashJoin -Left output [10]: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102] -Right output [2]: [customer_id#128, year_total#129] -Arguments: [customer_id#26], [customer_id#128], Inner, BuildRight - -(88) CometProject -Input [12]: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102, customer_id#128, year_total#129] -Arguments: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102, year_total#129], [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102, year_total#129] - -(89) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#130, c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(90) CometFilter -Input [8]: [c_customer_sk#130, c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137] -Condition : (isnotnull(c_customer_sk#130) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#131, 16, true, false, true))) - -(91) CometProject -Input [8]: [c_customer_sk#130, c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137] -Arguments: [c_customer_sk#130, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116], [c_customer_sk#130, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#131, 16, true, false, true) AS c_customer_id#111, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#132, 20, true, false, true) AS c_first_name#112, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#133, 30, true, false, true) AS c_last_name#113, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#134, 1, true, false, true) AS c_preferred_cust_flag#114, c_birth_country#135, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#136, 13, true, false, true) AS c_login#115, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#137, 50, true, false, true) AS c_email_address#116] - -(92) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#143), dynamicpruningexpression(ws_sold_date_sk#143 IN dynamicpruning#144)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(93) CometFilter -Input [6]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Condition : isnotnull(ws_bill_customer_sk#138) - -(94) CometBroadcastExchange -Input [6]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Arguments: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] - -(95) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#130, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116] -Right output [6]: [ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Arguments: [c_customer_sk#130], [ws_bill_customer_sk#138], Inner, BuildRight - -(96) CometProject -Input [14]: [c_customer_sk#130, c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_bill_customer_sk#138, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Arguments: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143], [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] - -(97) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#145, d_year#146] - -(98) CometBroadcastHashJoin -Left output [12]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143] -Right output [2]: [d_date_sk#145, d_year#146] -Arguments: [ws_sold_date_sk#143], [d_date_sk#145], Inner, BuildRight - -(99) CometProject -Input [14]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, ws_sold_date_sk#143, d_date_sk#145, d_year#146] -Arguments: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, d_year#146], [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, d_year#146] - -(100) CometHashAggregate -Input [12]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, ws_ext_discount_amt#139, ws_ext_sales_price#140, ws_ext_wholesale_cost#141, ws_ext_list_price#142, d_year#146] -Keys [8]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, d_year#146] -Functions [1]: [partial_sum(((((ws_ext_list_price#142 - ws_ext_wholesale_cost#141) - ws_ext_discount_amt#139) + ws_ext_sales_price#140) / 2))] - -(101) CometExchange -Input [10]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, d_year#146, sum#147, isEmpty#148] -Arguments: hashpartitioning(c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, d_year#146, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(102) CometHashAggregate -Input [10]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, d_year#146, sum#147, isEmpty#148] -Keys [8]: [c_customer_id#111, c_first_name#112, c_last_name#113, c_preferred_cust_flag#114, c_birth_country#135, c_login#115, c_email_address#116, d_year#146] -Functions [1]: [sum(((((ws_ext_list_price#142 - ws_ext_wholesale_cost#141) - ws_ext_discount_amt#139) + ws_ext_sales_price#140) / 2))] - -(103) CometBroadcastExchange -Input [2]: [customer_id#149, year_total#150] -Arguments: [customer_id#149, year_total#150] - -(104) CometBroadcastHashJoin -Left output [11]: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102, year_total#129] -Right output [2]: [customer_id#149, year_total#150] -Arguments: [customer_id#26], [customer_id#149], Inner, (CASE WHEN (year_total#81 > 0.000000) THEN (year_total#102 / year_total#81) END > CASE WHEN (year_total#129 > 0.000000) THEN (year_total#150 / year_total#129) END), BuildRight - -(105) CometProject -Input [13]: [customer_id#26, customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53, year_total#81, year_total#102, year_total#129, customer_id#149, year_total#150] -Arguments: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53], [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53] - -(106) CometTakeOrderedAndProject -Input [7]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#47 ASC NULLS FIRST,customer_first_name#48 ASC NULLS FIRST,customer_last_name#49 ASC NULLS FIRST,customer_preferred_cust_flag#50 ASC NULLS FIRST,customer_birth_country#51 ASC NULLS FIRST,customer_login#52 ASC NULLS FIRST,customer_email_address#53 ASC NULLS FIRST], output=[customer_id#47,customer_first_name#48,customer_last_name#49,customer_preferred_cust_flag#50,customer_birth_country#51,customer_login#52,customer_email_address#53]), [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53], 100, 0, [customer_id#47 ASC NULLS FIRST, customer_first_name#48 ASC NULLS FIRST, customer_last_name#49 ASC NULLS FIRST, customer_preferred_cust_flag#50 ASC NULLS FIRST, customer_birth_country#51 ASC NULLS FIRST, customer_login#52 ASC NULLS FIRST, customer_email_address#53 ASC NULLS FIRST], [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53] - -(107) CometColumnarToRow [codegen id : 1] -Input [7]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_preferred_cust_flag#50, customer_birth_country#51, customer_login#52, customer_email_address#53] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 -BroadcastExchange (111) -+- * CometColumnarToRow (110) - +- CometFilter (109) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) - - -(108) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_year#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(109) CometFilter -Input [2]: [d_date_sk#22, d_year#23] -Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) - -(110) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#22, d_year#23] - -(111) BroadcastExchange -Input [2]: [d_date_sk#22, d_year#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#41 IN dynamicpruning#42 -BroadcastExchange (115) -+- * CometColumnarToRow (114) - +- CometFilter (113) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) - - -(112) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_year#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(113) CometFilter -Input [2]: [d_date_sk#43, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2002)) AND isnotnull(d_date_sk#43)) - -(114) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#43, d_year#44] - -(115) BroadcastExchange -Input [2]: [d_date_sk#43, d_year#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -Subquery:3 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#21 - -Subquery:4 Hosting operator id = 57 Hosting Expression = cs_sold_date_sk#95 IN dynamicpruning#42 - -Subquery:5 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#122 IN dynamicpruning#21 - -Subquery:6 Hosting operator id = 92 Hosting Expression = ws_sold_date_sk#143 IN dynamicpruning#42 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/extended.txt deleted file mode 100644 index 1be0816ee1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/extended.txt +++ /dev/null @@ -1,131 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 118 out of 126 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 36be965b48..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt +++ /dev/null @@ -1,127 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - CometProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] - CometBroadcastHashJoin [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total,customer_id,year_total] - CometProject [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] - CometBroadcastHashJoin [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,customer_id,year_total] - CometProject [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 - CometHashAggregate [ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total] #5 - CometHashAggregate [d_year,sum,isEmpty] [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #6 - CometHashAggregate [ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #16 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #17 - CometHashAggregate [ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #19 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [customer_id,year_total,c_customer_id,sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #20 - CometHashAggregate [ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [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] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/explain.txt deleted file mode 100644 index fe074b9df2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/explain.txt +++ /dev/null @@ -1,221 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (34) -+- CometTakeOrderedAndProject (33) - +- CometHashAggregate (32) - +- CometExchange (31) - +- CometHashAggregate (30) - +- CometProject (29) - +- CometBroadcastHashJoin (28) - :- CometProject (24) - : +- CometBroadcastHashJoin (23) - : :- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (12) - : : : +- CometSortMergeJoin (11) - : : : :- CometSort (5) - : : : : +- CometColumnarExchange (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : +- CometSort (10) - : : : +- CometExchange (9) - : : : +- CometProject (8) - : : : +- CometFilter (7) - : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (6) - : : +- CometBroadcastExchange (16) - : : +- CometProject (15) - : : +- CometFilter (14) - : : +- CometNativeScan parquet spark_catalog.default.warehouse (13) - : +- CometBroadcastExchange (22) - : +- CometProject (21) - : +- CometFilter (20) - : +- CometNativeScan parquet spark_catalog.default.item (19) - +- CometBroadcastExchange (27) - +- CometFilter (26) - +- CometNativeScan parquet spark_catalog.default.date_dim (25) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] - -(3) Filter [codegen id : 1] -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Condition : (isnotnull(cs_warehouse_sk#1) AND isnotnull(cs_item_sk#2)) - -(4) CometColumnarExchange -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_order_number#3, cs_item_sk#2, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5], [cs_order_number#3 ASC NULLS FIRST, cs_item_sk#2 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Condition : (isnotnull(cr_order_number#8) AND isnotnull(cr_item_sk#7)) - -(8) CometProject -Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] - -(9) CometExchange -Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: hashpartitioning(cr_order_number#8, cr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_order_number#8 ASC NULLS FIRST, cr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Right output [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cs_order_number#3, cs_item_sk#2], [cr_order_number#8, cr_item_sk#7], LeftOuter - -(12) CometProject -Input [8]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5, cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9], [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] - -(13) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#11, w_state#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [w_warehouse_sk#11, w_state#12] -Condition : isnotnull(w_warehouse_sk#11) - -(15) CometProject -Input [2]: [w_warehouse_sk#11, w_state#12] -Arguments: [w_warehouse_sk#11, w_state#13], [w_warehouse_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, w_state#12, 2, true, false, true) AS w_state#13] - -(16) CometBroadcastExchange -Input [2]: [w_warehouse_sk#11, w_state#13] -Arguments: [w_warehouse_sk#11, w_state#13] - -(17) CometBroadcastHashJoin -Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] -Right output [2]: [w_warehouse_sk#11, w_state#13] -Arguments: [cs_warehouse_sk#1], [w_warehouse_sk#11], Inner, BuildRight - -(18) CometProject -Input [7]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_warehouse_sk#11, w_state#13] -Arguments: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13], [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13] - -(19) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Condition : (((isnotnull(i_current_price#16) AND (i_current_price#16 >= 0.99)) AND (i_current_price#16 <= 1.49)) AND isnotnull(i_item_sk#14)) - -(21) CometProject -Input [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Arguments: [i_item_sk#14, i_item_id#17], [i_item_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#15, 16, true, false, true) AS i_item_id#17] - -(22) CometBroadcastExchange -Input [2]: [i_item_sk#14, i_item_id#17] -Arguments: [i_item_sk#14, i_item_id#17] - -(23) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13] -Right output [2]: [i_item_sk#14, i_item_id#17] -Arguments: [cs_item_sk#2], [i_item_sk#14], Inner, BuildRight - -(24) CometProject -Input [7]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_sk#14, i_item_id#17] -Arguments: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17], [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17] - -(25) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#18, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) - -(27) CometBroadcastExchange -Input [2]: [d_date_sk#18, d_date#19] -Arguments: [d_date_sk#18, d_date#19] - -(28) CometBroadcastHashJoin -Left output [5]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17] -Right output [2]: [d_date_sk#18, d_date#19] -Arguments: [cs_sold_date_sk#5], [d_date_sk#18], Inner, BuildRight - -(29) CometProject -Input [7]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date_sk#18, d_date#19] -Arguments: [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19], [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19] - -(30) CometHashAggregate -Input [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19] -Keys [2]: [w_state#13, i_item_id#17] -Functions [2]: [partial_sum(CASE WHEN (d_date#19 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), partial_sum(CASE WHEN (d_date#19 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] - -(31) CometExchange -Input [6]: [w_state#13, i_item_id#17, sum#20, isEmpty#21, sum#22, isEmpty#23] -Arguments: hashpartitioning(w_state#13, i_item_id#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometHashAggregate -Input [6]: [w_state#13, i_item_id#17, sum#20, isEmpty#21, sum#22, isEmpty#23] -Keys [2]: [w_state#13, i_item_id#17] -Functions [2]: [sum(CASE WHEN (d_date#19 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), sum(CASE WHEN (d_date#19 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] - -(33) CometTakeOrderedAndProject -Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_state#13 ASC NULLS FIRST,i_item_id#17 ASC NULLS FIRST], output=[w_state#13,i_item_id#17,sales_before#24,sales_after#25]), [w_state#13, i_item_id#17, sales_before#24, sales_after#25], 100, 0, [w_state#13 ASC NULLS FIRST, i_item_id#17 ASC NULLS FIRST], [w_state#13, i_item_id#17, sales_before#24, sales_after#25] - -(34) CometColumnarToRow [codegen id : 2] -Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometFilter (36) - +- CometNativeScan parquet spark_catalog.default.date_dim (35) - - -(35) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(36) CometFilter -Input [2]: [d_date_sk#18, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) - -(37) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#18, d_date#19] - -(38) BroadcastExchange -Input [2]: [d_date_sk#18, d_date#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/simplified.txt deleted file mode 100644 index 13760fcb0b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] - CometHashAggregate [sum,isEmpty,sum,isEmpty] [w_state,i_item_id,sales_before,sales_after,sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END)] - CometExchange [w_state,i_item_id] #1 - CometHashAggregate [d_date,cs_sales_price,cr_refunded_cash] [w_state,i_item_id,sum,isEmpty,sum,isEmpty] - CometProject [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id,d_date_sk,d_date] - CometProject [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] - CometBroadcastHashJoin [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_warehouse_sk,w_state] - CometProject [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] - CometSortMergeJoin [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_refunded_cash] - CometSort [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - CometColumnarExchange [cs_order_number,cs_item_sk] #2 - WholeStageCodegen (1) - Filter [cs_warehouse_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] - CometExchange [cr_order_number,cr_item_sk] #4 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_state] #5 - CometProject [w_state] [w_warehouse_sk,w_state] - CometFilter [w_warehouse_sk,w_state] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometFilter [d_date_sk,d_date] - CometNativeScan 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/q40.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/explain.txt deleted file mode 100644 index 00e2b8fbc1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometTakeOrderedAndProject (32) - +- CometHashAggregate (31) - +- CometExchange (30) - +- CometHashAggregate (29) - +- CometProject (28) - +- CometBroadcastHashJoin (27) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometProject (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (5) - : : +- CometBroadcastExchange (15) - : : +- CometProject (14) - : : +- CometFilter (13) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (12) - : +- CometBroadcastExchange (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (18) - +- CometBroadcastExchange (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Condition : (isnotnull(cs_warehouse_sk#1) AND isnotnull(cs_item_sk#2)) - -(3) CometExchange -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_order_number#3, cs_item_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5], [cs_order_number#3 ASC NULLS FIRST, cs_item_sk#2 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Condition : (isnotnull(cr_order_number#8) AND isnotnull(cr_item_sk#7)) - -(7) CometProject -Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] - -(8) CometExchange -Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: hashpartitioning(cr_order_number#8, cr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_order_number#8 ASC NULLS FIRST, cr_item_sk#7 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Right output [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cs_order_number#3, cs_item_sk#2], [cr_order_number#8, cr_item_sk#7], LeftOuter - -(11) CometProject -Input [8]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5, cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9], [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#11, w_state#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [w_warehouse_sk#11, w_state#12] -Condition : isnotnull(w_warehouse_sk#11) - -(14) CometProject -Input [2]: [w_warehouse_sk#11, w_state#12] -Arguments: [w_warehouse_sk#11, w_state#13], [w_warehouse_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, w_state#12, 2, true, false, true) AS w_state#13] - -(15) CometBroadcastExchange -Input [2]: [w_warehouse_sk#11, w_state#13] -Arguments: [w_warehouse_sk#11, w_state#13] - -(16) CometBroadcastHashJoin -Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] -Right output [2]: [w_warehouse_sk#11, w_state#13] -Arguments: [cs_warehouse_sk#1], [w_warehouse_sk#11], Inner, BuildRight - -(17) CometProject -Input [7]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_warehouse_sk#11, w_state#13] -Arguments: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13], [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] -ReadSchema: struct - -(19) CometFilter -Input [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Condition : (((isnotnull(i_current_price#16) AND (i_current_price#16 >= 0.99)) AND (i_current_price#16 <= 1.49)) AND isnotnull(i_item_sk#14)) - -(20) CometProject -Input [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Arguments: [i_item_sk#14, i_item_id#17], [i_item_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#15, 16, true, false, true) AS i_item_id#17] - -(21) CometBroadcastExchange -Input [2]: [i_item_sk#14, i_item_id#17] -Arguments: [i_item_sk#14, i_item_id#17] - -(22) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13] -Right output [2]: [i_item_sk#14, i_item_id#17] -Arguments: [cs_item_sk#2], [i_item_sk#14], Inner, BuildRight - -(23) CometProject -Input [7]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_sk#14, i_item_id#17] -Arguments: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17], [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#18, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) - -(26) CometBroadcastExchange -Input [2]: [d_date_sk#18, d_date#19] -Arguments: [d_date_sk#18, d_date#19] - -(27) CometBroadcastHashJoin -Left output [5]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17] -Right output [2]: [d_date_sk#18, d_date#19] -Arguments: [cs_sold_date_sk#5], [d_date_sk#18], Inner, BuildRight - -(28) CometProject -Input [7]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date_sk#18, d_date#19] -Arguments: [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19], [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19] - -(29) CometHashAggregate -Input [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19] -Keys [2]: [w_state#13, i_item_id#17] -Functions [2]: [partial_sum(CASE WHEN (d_date#19 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), partial_sum(CASE WHEN (d_date#19 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] - -(30) CometExchange -Input [6]: [w_state#13, i_item_id#17, sum#20, isEmpty#21, sum#22, isEmpty#23] -Arguments: hashpartitioning(w_state#13, i_item_id#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(31) CometHashAggregate -Input [6]: [w_state#13, i_item_id#17, sum#20, isEmpty#21, sum#22, isEmpty#23] -Keys [2]: [w_state#13, i_item_id#17] -Functions [2]: [sum(CASE WHEN (d_date#19 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), sum(CASE WHEN (d_date#19 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] - -(32) CometTakeOrderedAndProject -Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_state#13 ASC NULLS FIRST,i_item_id#17 ASC NULLS FIRST], output=[w_state#13,i_item_id#17,sales_before#24,sales_after#25]), [w_state#13, i_item_id#17, sales_before#24, sales_after#25], 100, 0, [w_state#13 ASC NULLS FIRST, i_item_id#17 ASC NULLS FIRST], [w_state#13, i_item_id#17, sales_before#24, sales_after#25] - -(33) CometColumnarToRow [codegen id : 1] -Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [d_date_sk#18, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) - -(36) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#18, d_date#19] - -(37) BroadcastExchange -Input [2]: [d_date_sk#18, d_date#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/simplified.txt deleted file mode 100644 index 79e3556750..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,42 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] - CometHashAggregate [sum,isEmpty,sum,isEmpty] [w_state,i_item_id,sales_before,sales_after,sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END)] - CometExchange [w_state,i_item_id] #1 - CometHashAggregate [d_date,cs_sales_price,cr_refunded_cash] [w_state,i_item_id,sum,isEmpty,sum,isEmpty] - CometProject [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id,d_date_sk,d_date] - CometProject [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] - CometBroadcastHashJoin [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_warehouse_sk,w_state] - CometProject [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] - CometSortMergeJoin [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_refunded_cash] - CometSort [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - CometExchange [cs_order_number,cs_item_sk] #2 - CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] - CometExchange [cr_order_number,cr_item_sk] #4 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_state] #5 - CometProject [w_state] [w_warehouse_sk,w_state] - CometFilter [w_warehouse_sk,w_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q40/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt deleted file mode 100644 index 00e2b8fbc1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometTakeOrderedAndProject (32) - +- CometHashAggregate (31) - +- CometExchange (30) - +- CometHashAggregate (29) - +- CometProject (28) - +- CometBroadcastHashJoin (27) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometProject (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (5) - : : +- CometBroadcastExchange (15) - : : +- CometProject (14) - : : +- CometFilter (13) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (12) - : +- CometBroadcastExchange (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (18) - +- CometBroadcastExchange (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Condition : (isnotnull(cs_warehouse_sk#1) AND isnotnull(cs_item_sk#2)) - -(3) CometExchange -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Arguments: hashpartitioning(cs_order_number#3, cs_item_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5], [cs_order_number#3 ASC NULLS FIRST, cs_item_sk#2 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Condition : (isnotnull(cr_order_number#8) AND isnotnull(cr_item_sk#7)) - -(7) CometProject -Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] -Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] - -(8) CometExchange -Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: hashpartitioning(cr_order_number#8, cr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_order_number#8 ASC NULLS FIRST, cr_item_sk#7 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] -Right output [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cs_order_number#3, cs_item_sk#2], [cr_order_number#8, cr_item_sk#7], LeftOuter - -(11) CometProject -Input [8]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5, cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] -Arguments: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9], [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#11, w_state#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [w_warehouse_sk#11, w_state#12] -Condition : isnotnull(w_warehouse_sk#11) - -(14) CometProject -Input [2]: [w_warehouse_sk#11, w_state#12] -Arguments: [w_warehouse_sk#11, w_state#13], [w_warehouse_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, w_state#12, 2, true, false, true) AS w_state#13] - -(15) CometBroadcastExchange -Input [2]: [w_warehouse_sk#11, w_state#13] -Arguments: [w_warehouse_sk#11, w_state#13] - -(16) CometBroadcastHashJoin -Left output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] -Right output [2]: [w_warehouse_sk#11, w_state#13] -Arguments: [cs_warehouse_sk#1], [w_warehouse_sk#11], Inner, BuildRight - -(17) CometProject -Input [7]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_warehouse_sk#11, w_state#13] -Arguments: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13], [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] -ReadSchema: struct - -(19) CometFilter -Input [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Condition : (((isnotnull(i_current_price#16) AND (i_current_price#16 >= 0.99)) AND (i_current_price#16 <= 1.49)) AND isnotnull(i_item_sk#14)) - -(20) CometProject -Input [3]: [i_item_sk#14, i_item_id#15, i_current_price#16] -Arguments: [i_item_sk#14, i_item_id#17], [i_item_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#15, 16, true, false, true) AS i_item_id#17] - -(21) CometBroadcastExchange -Input [2]: [i_item_sk#14, i_item_id#17] -Arguments: [i_item_sk#14, i_item_id#17] - -(22) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13] -Right output [2]: [i_item_sk#14, i_item_id#17] -Arguments: [cs_item_sk#2], [i_item_sk#14], Inner, BuildRight - -(23) CometProject -Input [7]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_sk#14, i_item_id#17] -Arguments: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17], [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#18, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) - -(26) CometBroadcastExchange -Input [2]: [d_date_sk#18, d_date#19] -Arguments: [d_date_sk#18, d_date#19] - -(27) CometBroadcastHashJoin -Left output [5]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17] -Right output [2]: [d_date_sk#18, d_date#19] -Arguments: [cs_sold_date_sk#5], [d_date_sk#18], Inner, BuildRight - -(28) CometProject -Input [7]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date_sk#18, d_date#19] -Arguments: [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19], [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19] - -(29) CometHashAggregate -Input [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#13, i_item_id#17, d_date#19] -Keys [2]: [w_state#13, i_item_id#17] -Functions [2]: [partial_sum(CASE WHEN (d_date#19 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), partial_sum(CASE WHEN (d_date#19 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] - -(30) CometExchange -Input [6]: [w_state#13, i_item_id#17, sum#20, isEmpty#21, sum#22, isEmpty#23] -Arguments: hashpartitioning(w_state#13, i_item_id#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(31) CometHashAggregate -Input [6]: [w_state#13, i_item_id#17, sum#20, isEmpty#21, sum#22, isEmpty#23] -Keys [2]: [w_state#13, i_item_id#17] -Functions [2]: [sum(CASE WHEN (d_date#19 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), sum(CASE WHEN (d_date#19 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] - -(32) CometTakeOrderedAndProject -Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_state#13 ASC NULLS FIRST,i_item_id#17 ASC NULLS FIRST], output=[w_state#13,i_item_id#17,sales_before#24,sales_after#25]), [w_state#13, i_item_id#17, sales_before#24, sales_after#25], 100, 0, [w_state#13 ASC NULLS FIRST, i_item_id#17 ASC NULLS FIRST], [w_state#13, i_item_id#17, sales_before#24, sales_after#25] - -(33) CometColumnarToRow [codegen id : 1] -Input [4]: [w_state#13, i_item_id#17, sales_before#24, sales_after#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [d_date_sk#18, d_date#19] -Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-02-10)) AND (d_date#19 <= 2000-04-10)) AND isnotnull(d_date_sk#18)) - -(36) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#18, d_date#19] - -(37) BroadcastExchange -Input [2]: [d_date_sk#18, d_date#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/extended.txt deleted file mode 100644 index 54a7b21486..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/extended.txt +++ /dev/null @@ -1,40 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 79e3556750..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt +++ /dev/null @@ -1,42 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] - CometHashAggregate [sum,isEmpty,sum,isEmpty] [w_state,i_item_id,sales_before,sales_after,sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END)] - CometExchange [w_state,i_item_id] #1 - CometHashAggregate [d_date,cs_sales_price,cr_refunded_cash] [w_state,i_item_id,sum,isEmpty,sum,isEmpty] - CometProject [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] - CometBroadcastHashJoin [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id,d_date_sk,d_date] - CometProject [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] - CometBroadcastHashJoin [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_warehouse_sk,w_state] - CometProject [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] - CometSortMergeJoin [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_refunded_cash] - CometSort [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - CometExchange [cs_order_number,cs_item_sk] #2 - CometFilter [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash] - CometExchange [cr_order_number,cr_item_sk] #4 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_state] #5 - CometProject [w_state] [w_warehouse_sk,w_state] - CometFilter [w_warehouse_sk,w_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q41.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_datafusion/explain.txt deleted file mode 100644 index cbce1b2b45..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_datafusion/explain.txt +++ /dev/null @@ -1,107 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (3) - : +- CometFilter (2) - : +- CometNativeScan parquet spark_catalog.default.item (1) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometHashAggregate (9) - +- CometExchange (8) - +- CometHashAggregate (7) - +- CometProject (6) - +- CometFilter (5) - +- CometNativeScan parquet spark_catalog.default.item (4) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), GreaterThanOrEqual(i_manufact_id,738), LessThanOrEqual(i_manufact_id,778)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Condition : ((isnotnull(i_manufact_id#1) AND (i_manufact_id#1 >= 738)) AND (i_manufact_id#1 <= 778)) - -(3) CometProject -Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Arguments: [i_manufact#2, i_product_name#3], [i_manufact#2, i_product_name#3] - -(4) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(5) CometFilter -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] -Condition : (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Women ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = powder ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = khaki )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Ounce ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Oz ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = extra large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = brown ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = honeydew )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Bunch ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Ton ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = N/A ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = small ))))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Men ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = floral ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = deep )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = N/A ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Dozen ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = petite ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = light ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = cornflower )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Box ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Pound ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = extra large )))))) OR (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Women ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = midnight ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = snow )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Pallet ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Gross ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = extra large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = cyan ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = papaya )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Cup ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Dram ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = N/A ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = small ))))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Men ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = orange ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = frosted )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Each ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Tbl ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = petite ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = forest ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = ghost )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Lb ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Bundle ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = extra large ))))))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#2, 50, true, false, true))) - -(6) CometProject -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] -Arguments: [i_manufact#8], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#2, 50, true, false, true) AS i_manufact#8] - -(7) CometHashAggregate -Input [1]: [i_manufact#8] -Keys [1]: [i_manufact#8] -Functions [1]: [partial_count(1)] - -(8) CometExchange -Input [2]: [i_manufact#8, count#9] -Arguments: hashpartitioning(i_manufact#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(9) CometHashAggregate -Input [2]: [i_manufact#8, count#9] -Keys [1]: [i_manufact#8] -Functions [1]: [count(1)] - -(10) CometFilter -Input [2]: [item_cnt#10, i_manufact#8] -Condition : (item_cnt#10 > 0) - -(11) CometProject -Input [2]: [item_cnt#10, i_manufact#8] -Arguments: [i_manufact#8], [i_manufact#8] - -(12) CometBroadcastExchange -Input [1]: [i_manufact#8] -Arguments: [i_manufact#8] - -(13) CometBroadcastHashJoin -Left output [2]: [i_manufact#2, i_product_name#3] -Right output [1]: [i_manufact#8] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#2, 50, true, false, true)], [i_manufact#8], Inner, BuildRight - -(14) CometProject -Input [3]: [i_manufact#2, i_product_name#3, i_manufact#8] -Arguments: [i_product_name#11], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#3, 50, true, false, true) AS i_product_name#11] - -(15) CometHashAggregate -Input [1]: [i_product_name#11] -Keys [1]: [i_product_name#11] -Functions: [] - -(16) CometExchange -Input [1]: [i_product_name#11] -Arguments: hashpartitioning(i_product_name#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(17) CometHashAggregate -Input [1]: [i_product_name#11] -Keys [1]: [i_product_name#11] -Functions: [] - -(18) CometTakeOrderedAndProject -Input [1]: [i_product_name#11] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#11 ASC NULLS FIRST], output=[i_product_name#11]), [i_product_name#11], 100, 0, [i_product_name#11 ASC NULLS FIRST], [i_product_name#11] - -(19) CometColumnarToRow [codegen id : 1] -Input [1]: [i_product_name#11] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_datafusion/simplified.txt deleted file mode 100644 index 2557f51684..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_datafusion/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name] - CometHashAggregate [i_product_name] - CometExchange [i_product_name] #1 - CometHashAggregate [i_product_name] - CometProject [i_product_name] [i_product_name] - CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - CometBroadcastExchange [i_manufact] #2 - CometProject [i_manufact] - CometFilter [item_cnt,i_manufact] - CometHashAggregate [count] [item_cnt,i_manufact,count(1)] - CometExchange [i_manufact] #3 - CometHashAggregate [i_manufact,count] - CometProject [i_manufact] [i_manufact] - CometFilter [i_category,i_manufact,i_size,i_color,i_units] - CometNativeScan 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/q41.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_iceberg_compat/explain.txt deleted file mode 100644 index 3b1ad984a3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,107 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (3) - : +- CometFilter (2) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometHashAggregate (9) - +- CometExchange (8) - +- CometHashAggregate (7) - +- CometProject (6) - +- CometFilter (5) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), GreaterThanOrEqual(i_manufact_id,738), LessThanOrEqual(i_manufact_id,778)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Condition : ((isnotnull(i_manufact_id#1) AND (i_manufact_id#1 >= 738)) AND (i_manufact_id#1 <= 778)) - -(3) CometProject -Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Arguments: [i_manufact#2, i_product_name#3], [i_manufact#2, i_product_name#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(5) CometFilter -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] -Condition : (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Women ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = powder ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = khaki )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Ounce ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Oz ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = extra large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = brown ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = honeydew )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Bunch ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Ton ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = N/A ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = small ))))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Men ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = floral ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = deep )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = N/A ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Dozen ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = petite ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = light ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = cornflower )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Box ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Pound ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = extra large )))))) OR (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Women ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = midnight ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = snow )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Pallet ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Gross ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = extra large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = cyan ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = papaya )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Cup ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Dram ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = N/A ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = small ))))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Men ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = orange ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = frosted )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Each ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Tbl ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = petite ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = forest ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = ghost )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Lb ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Bundle ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = extra large ))))))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#2, 50, true, false, true))) - -(6) CometProject -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] -Arguments: [i_manufact#8], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#2, 50, true, false, true) AS i_manufact#8] - -(7) CometHashAggregate -Input [1]: [i_manufact#8] -Keys [1]: [i_manufact#8] -Functions [1]: [partial_count(1)] - -(8) CometExchange -Input [2]: [i_manufact#8, count#9] -Arguments: hashpartitioning(i_manufact#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(9) CometHashAggregate -Input [2]: [i_manufact#8, count#9] -Keys [1]: [i_manufact#8] -Functions [1]: [count(1)] - -(10) CometFilter -Input [2]: [item_cnt#10, i_manufact#8] -Condition : (item_cnt#10 > 0) - -(11) CometProject -Input [2]: [item_cnt#10, i_manufact#8] -Arguments: [i_manufact#8], [i_manufact#8] - -(12) CometBroadcastExchange -Input [1]: [i_manufact#8] -Arguments: [i_manufact#8] - -(13) CometBroadcastHashJoin -Left output [2]: [i_manufact#2, i_product_name#3] -Right output [1]: [i_manufact#8] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#2, 50, true, false, true)], [i_manufact#8], Inner, BuildRight - -(14) CometProject -Input [3]: [i_manufact#2, i_product_name#3, i_manufact#8] -Arguments: [i_product_name#11], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#3, 50, true, false, true) AS i_product_name#11] - -(15) CometHashAggregate -Input [1]: [i_product_name#11] -Keys [1]: [i_product_name#11] -Functions: [] - -(16) CometExchange -Input [1]: [i_product_name#11] -Arguments: hashpartitioning(i_product_name#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(17) CometHashAggregate -Input [1]: [i_product_name#11] -Keys [1]: [i_product_name#11] -Functions: [] - -(18) CometTakeOrderedAndProject -Input [1]: [i_product_name#11] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#11 ASC NULLS FIRST], output=[i_product_name#11]), [i_product_name#11], 100, 0, [i_product_name#11 ASC NULLS FIRST], [i_product_name#11] - -(19) CometColumnarToRow [codegen id : 1] -Input [1]: [i_product_name#11] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_iceberg_compat/simplified.txt deleted file mode 100644 index 99c935aaa4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name] - CometHashAggregate [i_product_name] - CometExchange [i_product_name] #1 - CometHashAggregate [i_product_name] - CometProject [i_product_name] [i_product_name] - CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - CometBroadcastExchange [i_manufact] #2 - CometProject [i_manufact] - CometFilter [item_cnt,i_manufact] - CometHashAggregate [count] [item_cnt,i_manufact,count(1)] - CometExchange [i_manufact] #3 - CometHashAggregate [i_manufact,count] - CometProject [i_manufact] [i_manufact] - CometFilter [i_category,i_manufact,i_size,i_color,i_units] - CometScan [native_iceberg_compat] 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/q41/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt deleted file mode 100644 index 3b1ad984a3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt +++ /dev/null @@ -1,107 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (3) - : +- CometFilter (2) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometHashAggregate (9) - +- CometExchange (8) - +- CometHashAggregate (7) - +- CometProject (6) - +- CometFilter (5) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (4) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), GreaterThanOrEqual(i_manufact_id,738), LessThanOrEqual(i_manufact_id,778)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Condition : ((isnotnull(i_manufact_id#1) AND (i_manufact_id#1 >= 738)) AND (i_manufact_id#1 <= 778)) - -(3) CometProject -Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] -Arguments: [i_manufact#2, i_product_name#3], [i_manufact#2, i_product_name#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(5) CometFilter -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] -Condition : (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Women ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = powder ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = khaki )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Ounce ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Oz ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = extra large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = brown ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = honeydew )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Bunch ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Ton ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = N/A ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = small ))))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Men ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = floral ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = deep )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = N/A ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Dozen ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = petite ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = light ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = cornflower )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Box ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Pound ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = extra large )))))) OR (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Women ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = midnight ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = snow )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Pallet ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Gross ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = extra large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = cyan ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = papaya )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Cup ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Dram ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = N/A ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = small ))))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) = Men ) AND (((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = orange ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = frosted )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Each ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Tbl ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = petite ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = large ))) OR ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = forest ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#6, 20, true, false, true) = ghost )) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Lb ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#7, 10, true, false, true) = Bundle ))) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = medium ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#5, 20, true, false, true) = extra large ))))))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#2, 50, true, false, true))) - -(6) CometProject -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] -Arguments: [i_manufact#8], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#2, 50, true, false, true) AS i_manufact#8] - -(7) CometHashAggregate -Input [1]: [i_manufact#8] -Keys [1]: [i_manufact#8] -Functions [1]: [partial_count(1)] - -(8) CometExchange -Input [2]: [i_manufact#8, count#9] -Arguments: hashpartitioning(i_manufact#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(9) CometHashAggregate -Input [2]: [i_manufact#8, count#9] -Keys [1]: [i_manufact#8] -Functions [1]: [count(1)] - -(10) CometFilter -Input [2]: [item_cnt#10, i_manufact#8] -Condition : (item_cnt#10 > 0) - -(11) CometProject -Input [2]: [item_cnt#10, i_manufact#8] -Arguments: [i_manufact#8], [i_manufact#8] - -(12) CometBroadcastExchange -Input [1]: [i_manufact#8] -Arguments: [i_manufact#8] - -(13) CometBroadcastHashJoin -Left output [2]: [i_manufact#2, i_product_name#3] -Right output [1]: [i_manufact#8] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_manufact#2, 50, true, false, true)], [i_manufact#8], Inner, BuildRight - -(14) CometProject -Input [3]: [i_manufact#2, i_product_name#3, i_manufact#8] -Arguments: [i_product_name#11], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#3, 50, true, false, true) AS i_product_name#11] - -(15) CometHashAggregate -Input [1]: [i_product_name#11] -Keys [1]: [i_product_name#11] -Functions: [] - -(16) CometExchange -Input [1]: [i_product_name#11] -Arguments: hashpartitioning(i_product_name#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(17) CometHashAggregate -Input [1]: [i_product_name#11] -Keys [1]: [i_product_name#11] -Functions: [] - -(18) CometTakeOrderedAndProject -Input [1]: [i_product_name#11] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_product_name#11 ASC NULLS FIRST], output=[i_product_name#11]), [i_product_name#11], 100, 0, [i_product_name#11 ASC NULLS FIRST], [i_product_name#11] - -(19) CometColumnarToRow [codegen id : 1] -Input [1]: [i_product_name#11] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/extended.txt deleted file mode 100644 index 9d802b5033..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 99c935aaa4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name] - CometHashAggregate [i_product_name] - CometExchange [i_product_name] #1 - CometHashAggregate [i_product_name] - CometProject [i_product_name] [i_product_name] - CometBroadcastHashJoin [i_manufact,i_product_name,i_manufact] - CometProject [i_manufact,i_product_name] - CometFilter [i_manufact_id,i_manufact,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] - CometBroadcastExchange [i_manufact] #2 - CometProject [i_manufact] - CometFilter [item_cnt,i_manufact] - CometHashAggregate [count] [item_cnt,i_manufact,count(1)] - CometExchange [i_manufact] #3 - CometHashAggregate [i_manufact,count] - CometProject [i_manufact] [i_manufact] - CometFilter [i_category,i_manufact,i_size,i_color,i_units] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_datafusion/explain.txt deleted file mode 100644 index af01446009..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_datafusion/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometNativeScan parquet spark_catalog.default.item (9) - - -(1) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometNativeScan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_category_id#8, i_category#11], [i_item_sk#7, i_category_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [i_item_sk#7, i_category_id#8, i_category#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11], [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11] -Keys [3]: [d_year#2, i_category_id#8, i_category#11] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] -Arguments: hashpartitioning(d_year#2, i_category_id#8, i_category#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] -Keys [3]: [d_year#2, i_category_id#8, i_category#11] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sum(ss_ext_sales_price)#13 DESC NULLS LAST,d_year#2 ASC NULLS FIRST,i_category_id#8 ASC NULLS FIRST,i_category#11 ASC NULLS FIRST], output=[d_year#2,i_category_id#8,i_category#11,sum(ss_ext_sales_price)#13]), [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13], 100, 0, [sum(ss_ext_sales_price)#13 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_datafusion/simplified.txt deleted file mode 100644 index 96ab34454d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_datafusion/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,i_category_id,i_category,sum(ss_ext_sales_price)] - CometHashAggregate [sum] [d_year,i_category_id,i_category,sum(ss_ext_sales_price),sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_category_id,i_category] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_category_id,i_category,sum] - CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 - CometProject [i_category] [i_item_sk,i_category_id,i_category] - CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] - CometNativeScan 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/q42.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_iceberg_compat/explain.txt deleted file mode 100644 index 183fffdf00..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_category_id#8, i_category#11], [i_item_sk#7, i_category_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [i_item_sk#7, i_category_id#8, i_category#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11], [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11] -Keys [3]: [d_year#2, i_category_id#8, i_category#11] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] -Arguments: hashpartitioning(d_year#2, i_category_id#8, i_category#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] -Keys [3]: [d_year#2, i_category_id#8, i_category#11] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sum(ss_ext_sales_price)#13 DESC NULLS LAST,d_year#2 ASC NULLS FIRST,i_category_id#8 ASC NULLS FIRST,i_category#11 ASC NULLS FIRST], output=[d_year#2,i_category_id#8,i_category#11,sum(ss_ext_sales_price)#13]), [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13], 100, 0, [sum(ss_ext_sales_price)#13 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_iceberg_compat/simplified.txt deleted file mode 100644 index 839937d40c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,i_category_id,i_category,sum(ss_ext_sales_price)] - CometHashAggregate [sum] [d_year,i_category_id,i_category,sum(ss_ext_sales_price),sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_category_id,i_category] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_category_id,i_category,sum] - CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 - CometProject [i_category] [i_item_sk,i_category_id,i_category] - CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] - CometScan [native_iceberg_compat] 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/q42/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt deleted file mode 100644 index 183fffdf00..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_category_id#8, i_category#11], [i_item_sk#7, i_category_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [i_item_sk#7, i_category_id#8, i_category#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_category_id#8, i_category#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11], [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#11] -Keys [3]: [d_year#2, i_category_id#8, i_category#11] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] -Arguments: hashpartitioning(d_year#2, i_category_id#8, i_category#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] -Keys [3]: [d_year#2, i_category_id#8, i_category#11] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sum(ss_ext_sales_price)#13 DESC NULLS LAST,d_year#2 ASC NULLS FIRST,i_category_id#8 ASC NULLS FIRST,i_category#11 ASC NULLS FIRST], output=[d_year#2,i_category_id#8,i_category#11,sum(ss_ext_sales_price)#13]), [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13], 100, 0, [sum(ss_ext_sales_price)#13 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/extended.txt deleted file mode 100644 index a292badf5d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 839937d40c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,i_category_id,i_category,sum(ss_ext_sales_price)] - CometHashAggregate [sum] [d_year,i_category_id,i_category,sum(ss_ext_sales_price),sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_category_id,i_category] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_category_id,i_category,sum] - CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category_id,i_category] #3 - CometProject [i_category] [i_item_sk,i_category_id,i_category] - CometFilter [i_item_sk,i_category_id,i_category,i_manager_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_datafusion/explain.txt deleted file mode 100644 index 261e6ddbaa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_datafusion/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometNativeScan parquet spark_catalog.default.store (9) - - -(1) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Condition : ((isnotnull(d_year#2) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Arguments: [d_date_sk#1, d_day_name#4], [d_date_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_day_name#3, 9, true, false, true) AS d_day_name#4] - -(4) CometNativeScan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_store_sk#5) - -(6) CometBroadcastExchange -Input [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_day_name#4] -Right output [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [d_date_sk#1], [ss_sold_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_day_name#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [d_day_name#4, ss_store_sk#5, ss_sales_price#6], [d_day_name#4, ss_store_sk#5, ss_sales_price#6] - -(9) CometNativeScan parquet spark_catalog.default.store -Output [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Condition : ((isnotnull(s_gmt_offset#11) AND (s_gmt_offset#11 = -5.00)) AND isnotnull(s_store_sk#8)) - -(11) CometProject -Input [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Arguments: [s_store_sk#8, s_store_id#12, s_store_name#10], [s_store_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#9, 16, true, false, true) AS s_store_id#12, s_store_name#10] - -(12) CometBroadcastExchange -Input [3]: [s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [s_store_sk#8, s_store_id#12, s_store_name#10] - -(13) CometBroadcastHashJoin -Left output [3]: [d_day_name#4, ss_store_sk#5, ss_sales_price#6] -Right output [3]: [s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [ss_store_sk#5], [s_store_sk#8], Inner, BuildRight - -(14) CometProject -Input [6]: [d_day_name#4, ss_store_sk#5, ss_sales_price#6, s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10], [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10] - -(15) CometHashAggregate -Input [4]: [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10] -Keys [2]: [s_store_name#10, s_store_id#12] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))] - -(16) CometExchange -Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_name#10, s_store_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Keys [2]: [s_store_name#10, s_store_id#12] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))] - -(18) CometTakeOrderedAndProject -Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#10 ASC NULLS FIRST,s_store_id#12 ASC NULLS FIRST,sun_sales#20 ASC NULLS FIRST,mon_sales#21 ASC NULLS FIRST,tue_sales#22 ASC NULLS FIRST,wed_sales#23 ASC NULLS FIRST,thu_sales#24 ASC NULLS FIRST,fri_sales#25 ASC NULLS FIRST,sat_sales#26 ASC NULLS FIRST], output=[s_store_name#10,s_store_id#12,sun_sales#20,mon_sales#21,tue_sales#22,wed_sales#23,thu_sales#24,fri_sales#25,sat_sales#26]), [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26], 100, 0, [s_store_name#10 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sun_sales#20 ASC NULLS FIRST, mon_sales#21 ASC NULLS FIRST, tue_sales#22 ASC NULLS FIRST, wed_sales#23 ASC NULLS FIRST, thu_sales#24 ASC NULLS FIRST, fri_sales#25 ASC NULLS FIRST, sat_sales#26 ASC NULLS FIRST], [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] - -(19) CometColumnarToRow [codegen id : 1] -Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_datafusion/simplified.txt deleted file mode 100644 index c7064b3ad8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_datafusion/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - CometExchange [s_store_name,s_store_id] #1 - CometHashAggregate [d_day_name,ss_sales_price] [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] - CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - 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,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [d_day_name] [d_date_sk,d_day_name] - CometFilter [d_date_sk,d_year,d_day_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] - CometNativeScan 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/q43.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_iceberg_compat/explain.txt deleted file mode 100644 index 62ddd93781..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Condition : ((isnotnull(d_year#2) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Arguments: [d_date_sk#1, d_day_name#4], [d_date_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_day_name#3, 9, true, false, true) AS d_day_name#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_store_sk#5) - -(6) CometBroadcastExchange -Input [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_day_name#4] -Right output [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [d_date_sk#1], [ss_sold_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_day_name#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [d_day_name#4, ss_store_sk#5, ss_sales_price#6], [d_day_name#4, ss_store_sk#5, ss_sales_price#6] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Condition : ((isnotnull(s_gmt_offset#11) AND (s_gmt_offset#11 = -5.00)) AND isnotnull(s_store_sk#8)) - -(11) CometProject -Input [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Arguments: [s_store_sk#8, s_store_id#12, s_store_name#10], [s_store_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#9, 16, true, false, true) AS s_store_id#12, s_store_name#10] - -(12) CometBroadcastExchange -Input [3]: [s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [s_store_sk#8, s_store_id#12, s_store_name#10] - -(13) CometBroadcastHashJoin -Left output [3]: [d_day_name#4, ss_store_sk#5, ss_sales_price#6] -Right output [3]: [s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [ss_store_sk#5], [s_store_sk#8], Inner, BuildRight - -(14) CometProject -Input [6]: [d_day_name#4, ss_store_sk#5, ss_sales_price#6, s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10], [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10] - -(15) CometHashAggregate -Input [4]: [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10] -Keys [2]: [s_store_name#10, s_store_id#12] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))] - -(16) CometExchange -Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_name#10, s_store_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Keys [2]: [s_store_name#10, s_store_id#12] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))] - -(18) CometTakeOrderedAndProject -Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#10 ASC NULLS FIRST,s_store_id#12 ASC NULLS FIRST,sun_sales#20 ASC NULLS FIRST,mon_sales#21 ASC NULLS FIRST,tue_sales#22 ASC NULLS FIRST,wed_sales#23 ASC NULLS FIRST,thu_sales#24 ASC NULLS FIRST,fri_sales#25 ASC NULLS FIRST,sat_sales#26 ASC NULLS FIRST], output=[s_store_name#10,s_store_id#12,sun_sales#20,mon_sales#21,tue_sales#22,wed_sales#23,thu_sales#24,fri_sales#25,sat_sales#26]), [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26], 100, 0, [s_store_name#10 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sun_sales#20 ASC NULLS FIRST, mon_sales#21 ASC NULLS FIRST, tue_sales#22 ASC NULLS FIRST, wed_sales#23 ASC NULLS FIRST, thu_sales#24 ASC NULLS FIRST, fri_sales#25 ASC NULLS FIRST, sat_sales#26 ASC NULLS FIRST], [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] - -(19) CometColumnarToRow [codegen id : 1] -Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_iceberg_compat/simplified.txt deleted file mode 100644 index 59ad7611a5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - CometExchange [s_store_name,s_store_id] #1 - CometHashAggregate [d_day_name,ss_sales_price] [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] - CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - 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,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [d_day_name] [d_date_sk,d_day_name] - CometFilter [d_date_sk,d_year,d_day_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] - CometScan [native_iceberg_compat] 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/q43/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt deleted file mode 100644 index 62ddd93781..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Condition : ((isnotnull(d_year#2) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] -Arguments: [d_date_sk#1, d_day_name#4], [d_date_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_day_name#3, 9, true, false, true) AS d_day_name#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_store_sk#5) - -(6) CometBroadcastExchange -Input [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_day_name#4] -Right output [3]: [ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [d_date_sk#1], [ss_sold_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_day_name#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [d_day_name#4, ss_store_sk#5, ss_sales_price#6], [d_day_name#4, ss_store_sk#5, ss_sales_price#6] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Condition : ((isnotnull(s_gmt_offset#11) AND (s_gmt_offset#11 = -5.00)) AND isnotnull(s_store_sk#8)) - -(11) CometProject -Input [4]: [s_store_sk#8, s_store_id#9, s_store_name#10, s_gmt_offset#11] -Arguments: [s_store_sk#8, s_store_id#12, s_store_name#10], [s_store_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#9, 16, true, false, true) AS s_store_id#12, s_store_name#10] - -(12) CometBroadcastExchange -Input [3]: [s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [s_store_sk#8, s_store_id#12, s_store_name#10] - -(13) CometBroadcastHashJoin -Left output [3]: [d_day_name#4, ss_store_sk#5, ss_sales_price#6] -Right output [3]: [s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [ss_store_sk#5], [s_store_sk#8], Inner, BuildRight - -(14) CometProject -Input [6]: [d_day_name#4, ss_store_sk#5, ss_sales_price#6, s_store_sk#8, s_store_id#12, s_store_name#10] -Arguments: [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10], [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10] - -(15) CometHashAggregate -Input [4]: [d_day_name#4, ss_sales_price#6, s_store_id#12, s_store_name#10] -Keys [2]: [s_store_name#10, s_store_id#12] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))] - -(16) CometExchange -Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_name#10, s_store_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] -Keys [2]: [s_store_name#10, s_store_id#12] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))] - -(18) CometTakeOrderedAndProject -Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#10 ASC NULLS FIRST,s_store_id#12 ASC NULLS FIRST,sun_sales#20 ASC NULLS FIRST,mon_sales#21 ASC NULLS FIRST,tue_sales#22 ASC NULLS FIRST,wed_sales#23 ASC NULLS FIRST,thu_sales#24 ASC NULLS FIRST,fri_sales#25 ASC NULLS FIRST,sat_sales#26 ASC NULLS FIRST], output=[s_store_name#10,s_store_id#12,sun_sales#20,mon_sales#21,tue_sales#22,wed_sales#23,thu_sales#24,fri_sales#25,sat_sales#26]), [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26], 100, 0, [s_store_name#10 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sun_sales#20 ASC NULLS FIRST, mon_sales#21 ASC NULLS FIRST, tue_sales#22 ASC NULLS FIRST, wed_sales#23 ASC NULLS FIRST, thu_sales#24 ASC NULLS FIRST, fri_sales#25 ASC NULLS FIRST, sat_sales#26 ASC NULLS FIRST], [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] - -(19) CometColumnarToRow [codegen id : 1] -Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/extended.txt deleted file mode 100644 index cb125e2456..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 59ad7611a5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - CometExchange [s_store_name,s_store_id] #1 - CometHashAggregate [d_day_name,ss_sales_price] [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] - CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] - 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,d_day_name,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [d_day_name] [d_date_sk,d_day_name] - CometFilter [d_date_sk,d_year,d_day_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] - CometBroadcastExchange [ss_store_sk,ss_sales_price,ss_sold_date_sk] #2 - CometFilter [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name,s_gmt_offset] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/explain.txt deleted file mode 100644 index e66d71b41e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/explain.txt +++ /dev/null @@ -1,227 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (34) -+- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Project (23) - : : +- * SortMergeJoin Inner (22) - : : :- * Sort (14) - : : : +- * Project (13) - : : : +- * Filter (12) - : : : +- Window (11) - : : : +- * CometColumnarToRow (10) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometFilter (7) - : : : +- CometHashAggregate (6) - : : : +- CometExchange (5) - : : : +- CometHashAggregate (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : +- * Sort (21) - : : +- * Project (20) - : : +- * Filter (19) - : : +- Window (18) - : : +- * CometColumnarToRow (17) - : : +- CometSort (16) - : : +- ReusedExchange (15) - : +- BroadcastExchange (28) - : +- * CometColumnarToRow (27) - : +- CometProject (26) - : +- CometFilter (25) - : +- CometNativeScan parquet spark_catalog.default.item (24) - +- ReusedExchange (31) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4)) - -(3) CometProject -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Arguments: [ss_item_sk#1, ss_net_profit#3], [ss_item_sk#1, ss_net_profit#3] - -(4) CometHashAggregate -Input [2]: [ss_item_sk#1, ss_net_profit#3] -Keys [1]: [ss_item_sk#1] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] - -(5) CometExchange -Input [3]: [ss_item_sk#1, sum#5, count#6] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(6) CometHashAggregate -Input [3]: [ss_item_sk#1, sum#5, count#6] -Keys [1]: [ss_item_sk#1] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] - -(7) CometFilter -Input [2]: [item_sk#7, rank_col#8] -Condition : (isnotnull(rank_col#8) AND (cast(rank_col#8 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#9, [id=#10]))) - -(8) CometExchange -Input [2]: [item_sk#7, rank_col#8] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [item_sk#7, rank_col#8] -Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] - -(10) CometColumnarToRow [codegen id : 1] -Input [2]: [item_sk#7, rank_col#8] - -(11) Window -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] - -(12) Filter [codegen id : 2] -Input [3]: [item_sk#7, rank_col#8, rnk#11] -Condition : ((rnk#11 < 11) AND isnotnull(item_sk#7)) - -(13) Project [codegen id : 2] -Output [2]: [item_sk#7, rnk#11] -Input [3]: [item_sk#7, rank_col#8, rnk#11] - -(14) Sort [codegen id : 2] -Input [2]: [item_sk#7, rnk#11] -Arguments: [rnk#11 ASC NULLS FIRST], false, 0 - -(15) ReusedExchange [Reuses operator id: 8] -Output [2]: [item_sk#12, rank_col#13] - -(16) CometSort -Input [2]: [item_sk#12, rank_col#13] -Arguments: [item_sk#12, rank_col#13], [rank_col#13 DESC NULLS LAST] - -(17) CometColumnarToRow [codegen id : 3] -Input [2]: [item_sk#12, rank_col#13] - -(18) Window -Input [2]: [item_sk#12, rank_col#13] -Arguments: [rank(rank_col#13) windowspecdefinition(rank_col#13 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#14], [rank_col#13 DESC NULLS LAST] - -(19) Filter [codegen id : 4] -Input [3]: [item_sk#12, rank_col#13, rnk#14] -Condition : ((rnk#14 < 11) AND isnotnull(item_sk#12)) - -(20) Project [codegen id : 4] -Output [2]: [item_sk#12, rnk#14] -Input [3]: [item_sk#12, rank_col#13, rnk#14] - -(21) Sort [codegen id : 4] -Input [2]: [item_sk#12, rnk#14] -Arguments: [rnk#14 ASC NULLS FIRST], false, 0 - -(22) SortMergeJoin [codegen id : 7] -Left keys [1]: [rnk#11] -Right keys [1]: [rnk#14] -Join type: Inner -Join condition: None - -(23) Project [codegen id : 7] -Output [3]: [item_sk#7, rnk#11, item_sk#12] -Input [4]: [item_sk#7, rnk#11, item_sk#12, rnk#14] - -(24) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#15, i_product_name#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [i_item_sk#15, i_product_name#16] -Condition : isnotnull(i_item_sk#15) - -(26) CometProject -Input [2]: [i_item_sk#15, i_product_name#16] -Arguments: [i_item_sk#15, i_product_name#17], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#16, 50, true, false, true) AS i_product_name#17] - -(27) CometColumnarToRow [codegen id : 5] -Input [2]: [i_item_sk#15, i_product_name#17] - -(28) BroadcastExchange -Input [2]: [i_item_sk#15, i_product_name#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(29) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [item_sk#7] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None - -(30) Project [codegen id : 7] -Output [3]: [rnk#11, item_sk#12, i_product_name#17] -Input [5]: [item_sk#7, rnk#11, item_sk#12, i_item_sk#15, i_product_name#17] - -(31) ReusedExchange [Reuses operator id: 28] -Output [2]: [i_item_sk#18, i_product_name#19] - -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [item_sk#12] -Right keys [1]: [i_item_sk#18] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 7] -Output [3]: [rnk#11, i_product_name#17 AS best_performing#20, i_product_name#19 AS worst_performing#21] -Input [5]: [rnk#11, item_sk#12, i_product_name#17, i_item_sk#18, i_product_name#19] - -(34) TakeOrderedAndProject -Input [3]: [rnk#11, best_performing#20, worst_performing#21] -Arguments: 100, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#20, worst_performing#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometFilter (36) - +- CometNativeScan parquet spark_catalog.default.store_sales (35) - - -(35) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] -ReadSchema: struct - -(36) CometFilter -Input [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] -Condition : ((isnotnull(ss_store_sk#23) AND (ss_store_sk#23 = 4)) AND isnull(ss_addr_sk#22)) - -(37) CometProject -Input [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] -Arguments: [ss_store_sk#23, ss_net_profit#24], [ss_store_sk#23, ss_net_profit#24] - -(38) CometHashAggregate -Input [2]: [ss_store_sk#23, ss_net_profit#24] -Keys [1]: [ss_store_sk#23] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#24))] - -(39) CometExchange -Input [3]: [ss_store_sk#23, sum#26, count#27] -Arguments: hashpartitioning(ss_store_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(40) CometHashAggregate -Input [3]: [ss_store_sk#23, sum#26, count#27] -Keys [1]: [ss_store_sk#23] -Functions [1]: [avg(UnscaledValue(ss_net_profit#24))] - -(41) CometColumnarToRow [codegen id : 1] -Input [1]: [rank_col#28] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/simplified.txt deleted file mode 100644 index 6ff27fc6c4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/simplified.txt +++ /dev/null @@ -1,59 +0,0 @@ -TakeOrderedAndProject [rnk,best_performing,worst_performing] - WholeStageCodegen (7) - Project [rnk,i_product_name,i_product_name] - BroadcastHashJoin [item_sk,i_item_sk] - Project [rnk,item_sk,i_product_name] - BroadcastHashJoin [item_sk,i_item_sk] - Project [item_sk,rnk,item_sk] - SortMergeJoin [rnk,rnk] - InputAdapter - WholeStageCodegen (2) - Sort [rnk] - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometExchange #1 - CometFilter [item_sk,rank_col] - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_store_sk] #3 - CometHashAggregate [ss_net_profit] [ss_store_sk,sum,count] - CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_item_sk] #2 - CometHashAggregate [ss_net_profit] [ss_item_sk,sum,count] - CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Sort [rnk] - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - ReusedExchange [item_sk,rank_col] #1 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_product_name] - CometNativeScan 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/q44.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/explain.txt deleted file mode 100644 index 134ab7939e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,227 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (34) -+- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Project (23) - : : +- * SortMergeJoin Inner (22) - : : :- * Sort (14) - : : : +- * Project (13) - : : : +- * Filter (12) - : : : +- Window (11) - : : : +- * CometColumnarToRow (10) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometFilter (7) - : : : +- CometHashAggregate (6) - : : : +- CometExchange (5) - : : : +- CometHashAggregate (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- * Sort (21) - : : +- * Project (20) - : : +- * Filter (19) - : : +- Window (18) - : : +- * CometColumnarToRow (17) - : : +- CometSort (16) - : : +- ReusedExchange (15) - : +- BroadcastExchange (28) - : +- * CometColumnarToRow (27) - : +- CometProject (26) - : +- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - +- ReusedExchange (31) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4)) - -(3) CometProject -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Arguments: [ss_item_sk#1, ss_net_profit#3], [ss_item_sk#1, ss_net_profit#3] - -(4) CometHashAggregate -Input [2]: [ss_item_sk#1, ss_net_profit#3] -Keys [1]: [ss_item_sk#1] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] - -(5) CometExchange -Input [3]: [ss_item_sk#1, sum#5, count#6] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(6) CometHashAggregate -Input [3]: [ss_item_sk#1, sum#5, count#6] -Keys [1]: [ss_item_sk#1] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] - -(7) CometFilter -Input [2]: [item_sk#7, rank_col#8] -Condition : (isnotnull(rank_col#8) AND (cast(rank_col#8 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#9, [id=#10]))) - -(8) CometExchange -Input [2]: [item_sk#7, rank_col#8] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [item_sk#7, rank_col#8] -Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] - -(10) CometColumnarToRow [codegen id : 1] -Input [2]: [item_sk#7, rank_col#8] - -(11) Window -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] - -(12) Filter [codegen id : 2] -Input [3]: [item_sk#7, rank_col#8, rnk#11] -Condition : ((rnk#11 < 11) AND isnotnull(item_sk#7)) - -(13) Project [codegen id : 2] -Output [2]: [item_sk#7, rnk#11] -Input [3]: [item_sk#7, rank_col#8, rnk#11] - -(14) Sort [codegen id : 2] -Input [2]: [item_sk#7, rnk#11] -Arguments: [rnk#11 ASC NULLS FIRST], false, 0 - -(15) ReusedExchange [Reuses operator id: 8] -Output [2]: [item_sk#12, rank_col#13] - -(16) CometSort -Input [2]: [item_sk#12, rank_col#13] -Arguments: [item_sk#12, rank_col#13], [rank_col#13 DESC NULLS LAST] - -(17) CometColumnarToRow [codegen id : 3] -Input [2]: [item_sk#12, rank_col#13] - -(18) Window -Input [2]: [item_sk#12, rank_col#13] -Arguments: [rank(rank_col#13) windowspecdefinition(rank_col#13 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#14], [rank_col#13 DESC NULLS LAST] - -(19) Filter [codegen id : 4] -Input [3]: [item_sk#12, rank_col#13, rnk#14] -Condition : ((rnk#14 < 11) AND isnotnull(item_sk#12)) - -(20) Project [codegen id : 4] -Output [2]: [item_sk#12, rnk#14] -Input [3]: [item_sk#12, rank_col#13, rnk#14] - -(21) Sort [codegen id : 4] -Input [2]: [item_sk#12, rnk#14] -Arguments: [rnk#14 ASC NULLS FIRST], false, 0 - -(22) SortMergeJoin [codegen id : 7] -Left keys [1]: [rnk#11] -Right keys [1]: [rnk#14] -Join type: Inner -Join condition: None - -(23) Project [codegen id : 7] -Output [3]: [item_sk#7, rnk#11, item_sk#12] -Input [4]: [item_sk#7, rnk#11, item_sk#12, rnk#14] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#15, i_product_name#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [i_item_sk#15, i_product_name#16] -Condition : isnotnull(i_item_sk#15) - -(26) CometProject -Input [2]: [i_item_sk#15, i_product_name#16] -Arguments: [i_item_sk#15, i_product_name#17], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#16, 50, true, false, true) AS i_product_name#17] - -(27) CometColumnarToRow [codegen id : 5] -Input [2]: [i_item_sk#15, i_product_name#17] - -(28) BroadcastExchange -Input [2]: [i_item_sk#15, i_product_name#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(29) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [item_sk#7] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None - -(30) Project [codegen id : 7] -Output [3]: [rnk#11, item_sk#12, i_product_name#17] -Input [5]: [item_sk#7, rnk#11, item_sk#12, i_item_sk#15, i_product_name#17] - -(31) ReusedExchange [Reuses operator id: 28] -Output [2]: [i_item_sk#18, i_product_name#19] - -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [item_sk#12] -Right keys [1]: [i_item_sk#18] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 7] -Output [3]: [rnk#11, i_product_name#17 AS best_performing#20, i_product_name#19 AS worst_performing#21] -Input [5]: [rnk#11, item_sk#12, i_product_name#17, i_item_sk#18, i_product_name#19] - -(34) TakeOrderedAndProject -Input [3]: [rnk#11, best_performing#20, worst_performing#21] -Arguments: 100, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#20, worst_performing#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (35) - - -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] -ReadSchema: struct - -(36) CometFilter -Input [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] -Condition : ((isnotnull(ss_store_sk#23) AND (ss_store_sk#23 = 4)) AND isnull(ss_addr_sk#22)) - -(37) CometProject -Input [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] -Arguments: [ss_store_sk#23, ss_net_profit#24], [ss_store_sk#23, ss_net_profit#24] - -(38) CometHashAggregate -Input [2]: [ss_store_sk#23, ss_net_profit#24] -Keys [1]: [ss_store_sk#23] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#24))] - -(39) CometExchange -Input [3]: [ss_store_sk#23, sum#26, count#27] -Arguments: hashpartitioning(ss_store_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(40) CometHashAggregate -Input [3]: [ss_store_sk#23, sum#26, count#27] -Keys [1]: [ss_store_sk#23] -Functions [1]: [avg(UnscaledValue(ss_net_profit#24))] - -(41) CometColumnarToRow [codegen id : 1] -Input [1]: [rank_col#28] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/simplified.txt deleted file mode 100644 index e3ec416397..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,59 +0,0 @@ -TakeOrderedAndProject [rnk,best_performing,worst_performing] - WholeStageCodegen (7) - Project [rnk,i_product_name,i_product_name] - BroadcastHashJoin [item_sk,i_item_sk] - Project [rnk,item_sk,i_product_name] - BroadcastHashJoin [item_sk,i_item_sk] - Project [item_sk,rnk,item_sk] - SortMergeJoin [rnk,rnk] - InputAdapter - WholeStageCodegen (2) - Sort [rnk] - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometExchange #1 - CometFilter [item_sk,rank_col] - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_store_sk] #3 - CometHashAggregate [ss_net_profit] [ss_store_sk,sum,count] - CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_item_sk] #2 - CometHashAggregate [ss_net_profit] [ss_item_sk,sum,count] - CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Sort [rnk] - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - ReusedExchange [item_sk,rank_col] #1 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_product_name] - CometScan [native_iceberg_compat] 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/q44/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt deleted file mode 100644 index 134ab7939e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt +++ /dev/null @@ -1,227 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (34) -+- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Project (23) - : : +- * SortMergeJoin Inner (22) - : : :- * Sort (14) - : : : +- * Project (13) - : : : +- * Filter (12) - : : : +- Window (11) - : : : +- * CometColumnarToRow (10) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometFilter (7) - : : : +- CometHashAggregate (6) - : : : +- CometExchange (5) - : : : +- CometHashAggregate (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- * Sort (21) - : : +- * Project (20) - : : +- * Filter (19) - : : +- Window (18) - : : +- * CometColumnarToRow (17) - : : +- CometSort (16) - : : +- ReusedExchange (15) - : +- BroadcastExchange (28) - : +- * CometColumnarToRow (27) - : +- CometProject (26) - : +- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - +- ReusedExchange (31) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4)) - -(3) CometProject -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Arguments: [ss_item_sk#1, ss_net_profit#3], [ss_item_sk#1, ss_net_profit#3] - -(4) CometHashAggregate -Input [2]: [ss_item_sk#1, ss_net_profit#3] -Keys [1]: [ss_item_sk#1] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] - -(5) CometExchange -Input [3]: [ss_item_sk#1, sum#5, count#6] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(6) CometHashAggregate -Input [3]: [ss_item_sk#1, sum#5, count#6] -Keys [1]: [ss_item_sk#1] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] - -(7) CometFilter -Input [2]: [item_sk#7, rank_col#8] -Condition : (isnotnull(rank_col#8) AND (cast(rank_col#8 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#9, [id=#10]))) - -(8) CometExchange -Input [2]: [item_sk#7, rank_col#8] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [item_sk#7, rank_col#8] -Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] - -(10) CometColumnarToRow [codegen id : 1] -Input [2]: [item_sk#7, rank_col#8] - -(11) Window -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] - -(12) Filter [codegen id : 2] -Input [3]: [item_sk#7, rank_col#8, rnk#11] -Condition : ((rnk#11 < 11) AND isnotnull(item_sk#7)) - -(13) Project [codegen id : 2] -Output [2]: [item_sk#7, rnk#11] -Input [3]: [item_sk#7, rank_col#8, rnk#11] - -(14) Sort [codegen id : 2] -Input [2]: [item_sk#7, rnk#11] -Arguments: [rnk#11 ASC NULLS FIRST], false, 0 - -(15) ReusedExchange [Reuses operator id: 8] -Output [2]: [item_sk#12, rank_col#13] - -(16) CometSort -Input [2]: [item_sk#12, rank_col#13] -Arguments: [item_sk#12, rank_col#13], [rank_col#13 DESC NULLS LAST] - -(17) CometColumnarToRow [codegen id : 3] -Input [2]: [item_sk#12, rank_col#13] - -(18) Window -Input [2]: [item_sk#12, rank_col#13] -Arguments: [rank(rank_col#13) windowspecdefinition(rank_col#13 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#14], [rank_col#13 DESC NULLS LAST] - -(19) Filter [codegen id : 4] -Input [3]: [item_sk#12, rank_col#13, rnk#14] -Condition : ((rnk#14 < 11) AND isnotnull(item_sk#12)) - -(20) Project [codegen id : 4] -Output [2]: [item_sk#12, rnk#14] -Input [3]: [item_sk#12, rank_col#13, rnk#14] - -(21) Sort [codegen id : 4] -Input [2]: [item_sk#12, rnk#14] -Arguments: [rnk#14 ASC NULLS FIRST], false, 0 - -(22) SortMergeJoin [codegen id : 7] -Left keys [1]: [rnk#11] -Right keys [1]: [rnk#14] -Join type: Inner -Join condition: None - -(23) Project [codegen id : 7] -Output [3]: [item_sk#7, rnk#11, item_sk#12] -Input [4]: [item_sk#7, rnk#11, item_sk#12, rnk#14] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#15, i_product_name#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [i_item_sk#15, i_product_name#16] -Condition : isnotnull(i_item_sk#15) - -(26) CometProject -Input [2]: [i_item_sk#15, i_product_name#16] -Arguments: [i_item_sk#15, i_product_name#17], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#16, 50, true, false, true) AS i_product_name#17] - -(27) CometColumnarToRow [codegen id : 5] -Input [2]: [i_item_sk#15, i_product_name#17] - -(28) BroadcastExchange -Input [2]: [i_item_sk#15, i_product_name#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(29) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [item_sk#7] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None - -(30) Project [codegen id : 7] -Output [3]: [rnk#11, item_sk#12, i_product_name#17] -Input [5]: [item_sk#7, rnk#11, item_sk#12, i_item_sk#15, i_product_name#17] - -(31) ReusedExchange [Reuses operator id: 28] -Output [2]: [i_item_sk#18, i_product_name#19] - -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [item_sk#12] -Right keys [1]: [i_item_sk#18] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 7] -Output [3]: [rnk#11, i_product_name#17 AS best_performing#20, i_product_name#19 AS worst_performing#21] -Input [5]: [rnk#11, item_sk#12, i_product_name#17, i_item_sk#18, i_product_name#19] - -(34) TakeOrderedAndProject -Input [3]: [rnk#11, best_performing#20, worst_performing#21] -Arguments: 100, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#20, worst_performing#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (35) - - -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] -ReadSchema: struct - -(36) CometFilter -Input [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] -Condition : ((isnotnull(ss_store_sk#23) AND (ss_store_sk#23 = 4)) AND isnull(ss_addr_sk#22)) - -(37) CometProject -Input [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] -Arguments: [ss_store_sk#23, ss_net_profit#24], [ss_store_sk#23, ss_net_profit#24] - -(38) CometHashAggregate -Input [2]: [ss_store_sk#23, ss_net_profit#24] -Keys [1]: [ss_store_sk#23] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#24))] - -(39) CometExchange -Input [3]: [ss_store_sk#23, sum#26, count#27] -Arguments: hashpartitioning(ss_store_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(40) CometHashAggregate -Input [3]: [ss_store_sk#23, sum#26, count#27] -Keys [1]: [ss_store_sk#23] -Functions [1]: [avg(UnscaledValue(ss_net_profit#24))] - -(41) CometColumnarToRow [codegen id : 1] -Input [1]: [rank_col#28] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/extended.txt deleted file mode 100644 index 5ef0fb3b87..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/extended.txt +++ /dev/null @@ -1,63 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- SortMergeJoin - : : :- Sort - : : : +- Project - : : : +- Filter - : : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : : +- CometColumnarToRow - : : : +- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- Sort - : : +- Project - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 36 out of 55 eligible operators (65%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index e3ec416397..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt +++ /dev/null @@ -1,59 +0,0 @@ -TakeOrderedAndProject [rnk,best_performing,worst_performing] - WholeStageCodegen (7) - Project [rnk,i_product_name,i_product_name] - BroadcastHashJoin [item_sk,i_item_sk] - Project [rnk,item_sk,i_product_name] - BroadcastHashJoin [item_sk,i_item_sk] - Project [item_sk,rnk,item_sk] - SortMergeJoin [rnk,rnk] - InputAdapter - WholeStageCodegen (2) - Sort [rnk] - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometExchange #1 - CometFilter [item_sk,rank_col] - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_store_sk] #3 - CometHashAggregate [ss_net_profit] [ss_store_sk,sum,count] - CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_item_sk] #2 - CometHashAggregate [ss_net_profit] [ss_item_sk,sum,count] - CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Sort [rnk] - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - ReusedExchange [item_sk,rank_col] #1 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_product_name] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/explain.txt deleted file mode 100644 index 1d20f0a3f0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/explain.txt +++ /dev/null @@ -1,256 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (39) -+- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * Filter (33) - +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (32) - :- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : : +- BroadcastExchange (7) - : : : : +- * CometColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (4) - : : : +- BroadcastExchange (14) - : : : +- * CometColumnarToRow (13) - : : : +- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (10) - : : +- ReusedExchange (17) - : +- BroadcastExchange (24) - : +- * CometColumnarToRow (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometNativeScan parquet spark_catalog.default.item (20) - +- BroadcastExchange (31) - +- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometNativeScan parquet spark_catalog.default.item (27) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#5), dynamicpruningexpression(ws_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] - -(3) Filter [codegen id : 6] -Input [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Condition : (isnotnull(ws_bill_customer_sk#3) AND isnotnull(ws_item_sk#2)) - -(4) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#7, c_current_addr_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Condition : (isnotnull(c_customer_sk#7) AND isnotnull(c_current_addr_sk#8)) - -(6) CometColumnarToRow [codegen id : 1] -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] - -(7) BroadcastExchange -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_bill_customer_sk#3] -Right keys [1]: [c_customer_sk#7] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 6] -Output [4]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] -Input [6]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5, c_customer_sk#7, c_current_addr_sk#8] - -(10) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Condition : isnotnull(ca_address_sk#9) - -(12) CometProject -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Arguments: [ca_address_sk#9, ca_city#10, ca_zip#12], [ca_address_sk#9, ca_city#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#11, 10, true, false, true) AS ca_zip#12] - -(13) CometColumnarToRow [codegen id : 2] -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] - -(14) BroadcastExchange -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_current_addr_sk#8] -Right keys [1]: [ca_address_sk#9] -Join type: Inner -Join condition: None - -(16) Project [codegen id : 6] -Output [5]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12] -Input [7]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8, ca_address_sk#9, ca_city#10, ca_zip#12] - -(17) ReusedExchange [Reuses operator id: 44] -Output [1]: [d_date_sk#13] - -(18) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#5] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 6] -Output [4]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12] -Input [6]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12, d_date_sk#13] - -(20) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#14, i_item_id#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [i_item_sk#14, i_item_id#15] -Condition : isnotnull(i_item_sk#14) - -(22) CometProject -Input [2]: [i_item_sk#14, i_item_id#15] -Arguments: [i_item_sk#14, i_item_id#16], [i_item_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#15, 16, true, false, true) AS i_item_id#16] - -(23) CometColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#14, i_item_id#16] - -(24) BroadcastExchange -Input [2]: [i_item_sk#14, i_item_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#2] -Right keys [1]: [i_item_sk#14] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 6] -Output [4]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#16] -Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12, i_item_sk#14, i_item_id#16] - -(27) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#14, i_item_id#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_item_sk, [11,13,17,19,2,23,29,3,5,7])] -ReadSchema: struct - -(28) CometFilter -Input [2]: [i_item_sk#14, i_item_id#15] -Condition : i_item_sk#14 IN (2,3,5,7,11,13,17,19,23,29) - -(29) CometProject -Input [2]: [i_item_sk#14, i_item_id#15] -Arguments: [i_item_id#17], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#15, 16, true, false, true) AS i_item_id#17] - -(30) CometColumnarToRow [codegen id : 5] -Input [1]: [i_item_id#17] - -(31) BroadcastExchange -Input [1]: [i_item_id#17] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [i_item_id#16] -Right keys [1]: [i_item_id#17] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(33) Filter [codegen id : 6] -Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#16, exists#1] -Condition : (substr(ca_zip#12, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) - -(34) Project [codegen id : 6] -Output [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] -Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#16, exists#1] - -(35) HashAggregate [codegen id : 6] -Input [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] -Keys [2]: [ca_zip#12, ca_city#10] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [ca_zip#12, ca_city#10, sum#19] - -(36) CometColumnarExchange -Input [3]: [ca_zip#12, ca_city#10, sum#19] -Arguments: hashpartitioning(ca_zip#12, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(37) CometColumnarToRow [codegen id : 7] -Input [3]: [ca_zip#12, ca_city#10, sum#19] - -(38) HashAggregate [codegen id : 7] -Input [3]: [ca_zip#12, ca_city#10, sum#19] -Keys [2]: [ca_zip#12, ca_city#10] -Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#20] -Results [3]: [ca_zip#12, ca_city#10, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#20,17,2) AS sum(ws_sales_price)#21] - -(39) TakeOrderedAndProject -Input [3]: [ca_zip#12, ca_city#10, sum(ws_sales_price)#21] -Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip#12, ca_city#10, sum(ws_sales_price)#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometNativeScan parquet spark_catalog.default.date_dim (40) - - -(40) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#22, d_qoy#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(41) CometFilter -Input [3]: [d_date_sk#13, d_year#22, d_qoy#23] -Condition : ((((isnotnull(d_qoy#23) AND isnotnull(d_year#22)) AND (d_qoy#23 = 2)) AND (d_year#22 = 2001)) AND isnotnull(d_date_sk#13)) - -(42) CometProject -Input [3]: [d_date_sk#13, d_year#22, d_qoy#23] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(43) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(44) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/simplified.txt deleted file mode 100644 index 81ecde84b2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/simplified.txt +++ /dev/null @@ -1,64 +0,0 @@ -TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - WholeStageCodegen (7) - HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_zip,ca_city] #1 - WholeStageCodegen (6) - HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] - Project [ws_sales_price,ca_city,ca_zip] - Filter [ca_zip,exists] - BroadcastHashJoin [i_item_id,i_item_id] - Project [ws_sales_price,ca_city,ca_zip,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_sales_price,ca_city,ca_zip] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Filter [ws_bill_customer_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_zip] [ca_address_sk,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_city,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan 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/q45.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/explain.txt deleted file mode 100644 index 9d3cf3d382..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,259 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (39) -+- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * Filter (33) - +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (32) - :- * CometColumnarToRow (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (7) - : : : : +- CometBroadcastHashJoin (6) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometBroadcastExchange (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- BroadcastExchange (31) - +- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (27) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#5), dynamicpruningexpression(ws_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Condition : (isnotnull(ws_bill_customer_sk#3) AND isnotnull(ws_item_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#7, c_current_addr_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Condition : (isnotnull(c_customer_sk#7) AND isnotnull(c_current_addr_sk#8)) - -(5) CometBroadcastExchange -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Arguments: [c_customer_sk#7, c_current_addr_sk#8] - -(6) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Right output [2]: [c_customer_sk#7, c_current_addr_sk#8] -Arguments: [ws_bill_customer_sk#3], [c_customer_sk#7], Inner, BuildRight - -(7) CometProject -Input [6]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5, c_customer_sk#7, c_current_addr_sk#8] -Arguments: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8], [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Condition : isnotnull(ca_address_sk#9) - -(10) CometProject -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Arguments: [ca_address_sk#9, ca_city#10, ca_zip#12], [ca_address_sk#9, ca_city#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#11, 10, true, false, true) AS ca_zip#12] - -(11) CometBroadcastExchange -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] -Arguments: [ca_address_sk#9, ca_city#10, ca_zip#12] - -(12) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] -Right output [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] -Arguments: [c_current_addr_sk#8], [ca_address_sk#9], Inner, BuildRight - -(13) CometProject -Input [7]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8, ca_address_sk#9, ca_city#10, ca_zip#12] -Arguments: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12], [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Condition : ((((isnotnull(d_qoy#15) AND isnotnull(d_year#14)) AND (d_qoy#15 = 2)) AND (d_year#14 = 2001)) AND isnotnull(d_date_sk#13)) - -(16) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(18) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight - -(19) CometProject -Input [6]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12, d_date_sk#13] -Arguments: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12], [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [i_item_sk#16, i_item_id#17] -Condition : isnotnull(i_item_sk#16) - -(22) CometProject -Input [2]: [i_item_sk#16, i_item_id#17] -Arguments: [i_item_sk#16, i_item_id#18], [i_item_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#17, 16, true, false, true) AS i_item_id#18] - -(23) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_id#18] -Arguments: [i_item_sk#16, i_item_id#18] - -(24) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12] -Right output [2]: [i_item_sk#16, i_item_id#18] -Arguments: [ws_item_sk#2], [i_item_sk#16], Inner, BuildRight - -(25) CometProject -Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12, i_item_sk#16, i_item_id#18] -Arguments: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18], [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] - -(26) CometColumnarToRow [codegen id : 2] -Input [4]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_item_sk, [11,13,17,19,2,23,29,3,5,7])] -ReadSchema: struct - -(28) CometFilter -Input [2]: [i_item_sk#16, i_item_id#17] -Condition : i_item_sk#16 IN (2,3,5,7,11,13,17,19,23,29) - -(29) CometProject -Input [2]: [i_item_sk#16, i_item_id#17] -Arguments: [i_item_id#19], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#17, 16, true, false, true) AS i_item_id#19] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [i_item_id#19] - -(31) BroadcastExchange -Input [1]: [i_item_id#19] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=1] - -(32) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [i_item_id#18] -Right keys [1]: [i_item_id#19] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(33) Filter [codegen id : 2] -Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] -Condition : (substr(ca_zip#12, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) - -(34) Project [codegen id : 2] -Output [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] -Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] - -(35) HashAggregate [codegen id : 2] -Input [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] -Keys [2]: [ca_zip#12, ca_city#10] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum#20] -Results [3]: [ca_zip#12, ca_city#10, sum#21] - -(36) CometColumnarExchange -Input [3]: [ca_zip#12, ca_city#10, sum#21] -Arguments: hashpartitioning(ca_zip#12, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(37) CometColumnarToRow [codegen id : 3] -Input [3]: [ca_zip#12, ca_city#10, sum#21] - -(38) HashAggregate [codegen id : 3] -Input [3]: [ca_zip#12, ca_city#10, sum#21] -Keys [2]: [ca_zip#12, ca_city#10] -Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#22] -Results [3]: [ca_zip#12, ca_city#10, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#22,17,2) AS sum(ws_sales_price)#23] - -(39) TakeOrderedAndProject -Input [3]: [ca_zip#12, ca_city#10, sum(ws_sales_price)#23] -Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip#12, ca_city#10, sum(ws_sales_price)#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) - - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(41) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Condition : ((((isnotnull(d_qoy#15) AND isnotnull(d_year#14)) AND (d_qoy#15 = 2)) AND (d_year#14 = 2001)) AND isnotnull(d_date_sk#13)) - -(42) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(43) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(44) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/simplified.txt deleted file mode 100644 index d78fc716ab..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - WholeStageCodegen (3) - HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_zip,ca_city] #1 - WholeStageCodegen (2) - HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] - Project [ws_sales_price,ca_city,ca_zip] - Filter [ca_zip,exists] - BroadcastHashJoin [i_item_id,i_item_id] - CometColumnarToRow - InputAdapter - CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] - 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_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 [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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 - CometProject [ca_zip] [ca_address_sk,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_city,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] 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/q45/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt deleted file mode 100644 index 9d3cf3d382..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt +++ /dev/null @@ -1,259 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (39) -+- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * Filter (33) - +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (32) - :- * CometColumnarToRow (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (7) - : : : : +- CometBroadcastHashJoin (6) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometBroadcastExchange (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- BroadcastExchange (31) - +- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (27) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#5), dynamicpruningexpression(ws_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Condition : (isnotnull(ws_bill_customer_sk#3) AND isnotnull(ws_item_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#7, c_current_addr_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Condition : (isnotnull(c_customer_sk#7) AND isnotnull(c_current_addr_sk#8)) - -(5) CometBroadcastExchange -Input [2]: [c_customer_sk#7, c_current_addr_sk#8] -Arguments: [c_customer_sk#7, c_current_addr_sk#8] - -(6) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] -Right output [2]: [c_customer_sk#7, c_current_addr_sk#8] -Arguments: [ws_bill_customer_sk#3], [c_customer_sk#7], Inner, BuildRight - -(7) CometProject -Input [6]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5, c_customer_sk#7, c_current_addr_sk#8] -Arguments: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8], [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Condition : isnotnull(ca_address_sk#9) - -(10) CometProject -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] -Arguments: [ca_address_sk#9, ca_city#10, ca_zip#12], [ca_address_sk#9, ca_city#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#11, 10, true, false, true) AS ca_zip#12] - -(11) CometBroadcastExchange -Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] -Arguments: [ca_address_sk#9, ca_city#10, ca_zip#12] - -(12) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] -Right output [3]: [ca_address_sk#9, ca_city#10, ca_zip#12] -Arguments: [c_current_addr_sk#8], [ca_address_sk#9], Inner, BuildRight - -(13) CometProject -Input [7]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8, ca_address_sk#9, ca_city#10, ca_zip#12] -Arguments: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12], [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Condition : ((((isnotnull(d_qoy#15) AND isnotnull(d_year#14)) AND (d_qoy#15 = 2)) AND (d_year#14 = 2001)) AND isnotnull(d_date_sk#13)) - -(16) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(18) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight - -(19) CometProject -Input [6]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#12, d_date_sk#13] -Arguments: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12], [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [i_item_sk#16, i_item_id#17] -Condition : isnotnull(i_item_sk#16) - -(22) CometProject -Input [2]: [i_item_sk#16, i_item_id#17] -Arguments: [i_item_sk#16, i_item_id#18], [i_item_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#17, 16, true, false, true) AS i_item_id#18] - -(23) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_id#18] -Arguments: [i_item_sk#16, i_item_id#18] - -(24) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12] -Right output [2]: [i_item_sk#16, i_item_id#18] -Arguments: [ws_item_sk#2], [i_item_sk#16], Inner, BuildRight - -(25) CometProject -Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#12, i_item_sk#16, i_item_id#18] -Arguments: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18], [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] - -(26) CometColumnarToRow [codegen id : 2] -Input [4]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_item_sk, [11,13,17,19,2,23,29,3,5,7])] -ReadSchema: struct - -(28) CometFilter -Input [2]: [i_item_sk#16, i_item_id#17] -Condition : i_item_sk#16 IN (2,3,5,7,11,13,17,19,23,29) - -(29) CometProject -Input [2]: [i_item_sk#16, i_item_id#17] -Arguments: [i_item_id#19], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#17, 16, true, false, true) AS i_item_id#19] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [i_item_id#19] - -(31) BroadcastExchange -Input [1]: [i_item_id#19] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=1] - -(32) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [i_item_id#18] -Right keys [1]: [i_item_id#19] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(33) Filter [codegen id : 2] -Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] -Condition : (substr(ca_zip#12, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) - -(34) Project [codegen id : 2] -Output [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] -Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#12, i_item_id#18, exists#1] - -(35) HashAggregate [codegen id : 2] -Input [3]: [ws_sales_price#4, ca_city#10, ca_zip#12] -Keys [2]: [ca_zip#12, ca_city#10] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum#20] -Results [3]: [ca_zip#12, ca_city#10, sum#21] - -(36) CometColumnarExchange -Input [3]: [ca_zip#12, ca_city#10, sum#21] -Arguments: hashpartitioning(ca_zip#12, ca_city#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(37) CometColumnarToRow [codegen id : 3] -Input [3]: [ca_zip#12, ca_city#10, sum#21] - -(38) HashAggregate [codegen id : 3] -Input [3]: [ca_zip#12, ca_city#10, sum#21] -Keys [2]: [ca_zip#12, ca_city#10] -Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#22] -Results [3]: [ca_zip#12, ca_city#10, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#22,17,2) AS sum(ws_sales_price)#23] - -(39) TakeOrderedAndProject -Input [3]: [ca_zip#12, ca_city#10, sum(ws_sales_price)#23] -Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip#12, ca_city#10, sum(ws_sales_price)#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) - - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(41) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Condition : ((((isnotnull(d_qoy#15) AND isnotnull(d_year#14)) AND (d_qoy#15 = 2)) AND (d_year#14 = 2001)) AND isnotnull(d_date_sk#13)) - -(42) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_qoy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(43) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(44) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/extended.txt deleted file mode 100644 index 4a57091056..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/extended.txt +++ /dev/null @@ -1,47 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- Filter - +- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 32 out of 41 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d78fc716ab..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] - WholeStageCodegen (3) - HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_zip,ca_city] #1 - WholeStageCodegen (2) - HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] - Project [ws_sales_price,ca_city,ca_zip] - Filter [ca_zip,exists] - BroadcastHashJoin [i_item_id,i_item_id] - CometColumnarToRow - InputAdapter - CometProject [ws_sales_price,ca_city,ca_zip,i_item_id] - 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_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 [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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #3 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk,ca_city,ca_zip] #4 - CometProject [ca_zip] [ca_address_sk,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_city,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/explain.txt deleted file mode 100644 index 994cd12469..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/explain.txt +++ /dev/null @@ -1,267 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (41) -+- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * HashAggregate (30) - : : +- * CometColumnarToRow (29) - : : +- CometColumnarExchange (28) - : : +- * HashAggregate (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Project (6) - : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : :- * Filter (3) - : : : : : : +- * ColumnarToRow (2) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : +- ReusedExchange (4) - : : : : +- BroadcastExchange (11) - : : : : +- * CometColumnarToRow (10) - : : : : +- CometProject (9) - : : : : +- CometFilter (8) - : : : : +- CometNativeScan parquet spark_catalog.default.store (7) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) - : : +- BroadcastExchange (24) - : : +- * CometColumnarToRow (23) - : : +- CometFilter (22) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (21) - : +- BroadcastExchange (35) - : +- * CometColumnarToRow (34) - : +- CometProject (33) - : +- CometFilter (32) - : +- CometNativeScan parquet spark_catalog.default.customer (31) - +- ReusedExchange (38) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] - -(3) Filter [codegen id : 5] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 46] -Output [1]: [d_date_sk#10] - -(5) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 5] -Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_city#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [s_store_sk#11, s_city#12] -Condition : (s_city#12 IN (Fairview,Midway) AND isnotnull(s_store_sk#11)) - -(9) CometProject -Input [2]: [s_store_sk#11, s_city#12] -Arguments: [s_store_sk#11], [s_store_sk#11] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#11] - -(11) BroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#11] - -(14) CometNativeScan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#13, hd_dep_count#14, hd_vehicle_count#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [hd_demo_sk#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : (((hd_dep_count#14 = 4) OR (hd_vehicle_count#15 = 3)) AND isnotnull(hd_demo_sk#13)) - -(16) CometProject -Input [3]: [hd_demo_sk#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#13], [hd_demo_sk#13] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#13] - -(18) BroadcastExchange -Input [1]: [hd_demo_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#13] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [5]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Input [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, hd_demo_sk#13] - -(21) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_city#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [ca_address_sk#16, ca_city#17] -Condition : (isnotnull(ca_address_sk#16) AND isnotnull(ca_city#17)) - -(23) CometColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#16, ca_city#17] - -(24) BroadcastExchange -Input [2]: [ca_address_sk#16, ca_city#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#16] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 5] -Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#17] -Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_address_sk#16, ca_city#17] - -(27) HashAggregate [codegen id : 5] -Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#17] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17] -Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum#18, sum#19] -Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] - -(28) CometColumnarExchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(29) CometColumnarToRow [codegen id : 8] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] - -(30) HashAggregate [codegen id : 8] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17] -Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#6))#22, sum(UnscaledValue(ss_net_profit#7))#23] -Results [5]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#17 AS bought_city#24, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#22,17,2) AS amt#25, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#23,17,2) AS profit#26] - -(31) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(32) CometFilter -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] -Condition : (isnotnull(c_customer_sk#27) AND isnotnull(c_current_addr_sk#28)) - -(33) CometProject -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] -Arguments: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32], [c_customer_sk#27, c_current_addr_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#29, 20, true, false, true) AS c_first_name#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#30, 30, true, false, true) AS c_last_name#32] - -(34) CometColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] - -(35) BroadcastExchange -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(36) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#27] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 8] -Output [7]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#31, c_last_name#32] -Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#24, amt#25, profit#26, c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] - -(38) ReusedExchange [Reuses operator id: 24] -Output [2]: [ca_address_sk#33, ca_city#34] - -(39) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [c_current_addr_sk#28] -Right keys [1]: [ca_address_sk#33] -Join type: Inner -Join condition: NOT (ca_city#34 = bought_city#24) - -(40) Project [codegen id : 8] -Output [7]: [c_last_name#32, c_first_name#31, ca_city#34, bought_city#24, ss_ticket_number#5, amt#25, profit#26] -Input [9]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#31, c_last_name#32, ca_address_sk#33, ca_city#34] - -(41) TakeOrderedAndProject -Input [7]: [c_last_name#32, c_first_name#31, ca_city#34, bought_city#24, ss_ticket_number#5, amt#25, profit#26] -Arguments: 100, [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, ca_city#34 ASC NULLS FIRST, bought_city#24 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#32, c_first_name#31, ca_city#34, bought_city#24, ss_ticket_number#5, amt#25, profit#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometNativeScan parquet spark_catalog.default.date_dim (42) - - -(42) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#35, d_dow#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [3]: [d_date_sk#10, d_year#35, d_dow#36] -Condition : ((d_dow#36 IN (6,0) AND d_year#35 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(44) CometProject -Input [3]: [d_date_sk#10, d_year#35, d_dow#36] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(45) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(46) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/simplified.txt deleted file mode 100644 index b4b78689b1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/simplified.txt +++ /dev/null @@ -1,67 +0,0 @@ -TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - WholeStageCodegen (8) - Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] - Project [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - WholeStageCodegen (5) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_city] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - InputAdapter - ReusedExchange [ca_address_sk,ca_city] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/explain.txt deleted file mode 100644 index ca7ae9df0c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,256 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (39) -+- CometTakeOrderedAndProject (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (34) - : +- CometBroadcastHashJoin (33) - : :- CometHashAggregate (28) - : : +- CometExchange (27) - : : +- CometHashAggregate (26) - : : +- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (20) - : : : +- CometBroadcastHashJoin (19) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometProject (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : : : +- CometBroadcastExchange (18) - : : : +- CometProject (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - : : +- CometBroadcastExchange (23) - : : +- CometFilter (22) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (21) - : +- CometBroadcastExchange (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (29) - +- ReusedExchange (35) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(5) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#13, s_city#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#13, s_city#14] -Condition : (s_city#14 IN (Fairview,Midway) AND isnotnull(s_store_sk#13)) - -(11) CometProject -Input [2]: [s_store_sk#13, s_city#14] -Arguments: [s_store_sk#13], [s_store_sk#13] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#13] -Arguments: [s_store_sk#13] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [1]: [s_store_sk#13] -Arguments: [ss_store_sk#4], [s_store_sk#13], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#13] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((hd_dep_count#16 = 4) OR (hd_vehicle_count#17 = 3)) AND isnotnull(hd_demo_sk#15)) - -(17) CometProject -Input [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] -Arguments: [hd_demo_sk#15], [hd_demo_sk#15] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#15] -Arguments: [hd_demo_sk#15] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [1]: [hd_demo_sk#15] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#15], Inner, BuildRight - -(20) CometProject -Input [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, hd_demo_sk#15] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_city#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [ca_address_sk#18, ca_city#19] -Condition : (isnotnull(ca_address_sk#18) AND isnotnull(ca_city#19)) - -(23) CometBroadcastExchange -Input [2]: [ca_address_sk#18, ca_city#19] -Arguments: [ca_address_sk#18, ca_city#19] - -(24) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [2]: [ca_address_sk#18, ca_city#19] -Arguments: [ss_addr_sk#3], [ca_address_sk#18], Inner, BuildRight - -(25) CometProject -Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_address_sk#18, ca_city#19] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19] - -(26) CometHashAggregate -Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19] -Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] - -(27) CometExchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#20, sum#21] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#20, sum#21] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19] -Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] -Condition : (isnotnull(c_customer_sk#22) AND isnotnull(c_current_addr_sk#23)) - -(31) CometProject -Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] -Arguments: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27], [c_customer_sk#22, c_current_addr_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#24, 20, true, false, true) AS c_first_name#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#25, 30, true, false, true) AS c_last_name#27] - -(32) CometBroadcastExchange -Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Arguments: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] - -(33) CometBroadcastHashJoin -Left output [5]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, amt#29, profit#30] -Right output [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Arguments: [ss_customer_sk#1], [c_customer_sk#22], Inner, BuildRight - -(34) CometProject -Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, amt#29, profit#30, c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Arguments: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27], [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27] - -(35) ReusedExchange [Reuses operator id: 23] -Output [2]: [ca_address_sk#31, ca_city#32] - -(36) CometBroadcastHashJoin -Left output [7]: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Right output [2]: [ca_address_sk#31, ca_city#32] -Arguments: [c_current_addr_sk#23], [ca_address_sk#31], Inner, NOT (ca_city#32 = bought_city#28), BuildRight - -(37) CometProject -Input [9]: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27, ca_address_sk#31, ca_city#32] -Arguments: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30], [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] - -(38) CometTakeOrderedAndProject -Input [7]: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#27 ASC NULLS FIRST,c_first_name#26 ASC NULLS FIRST,ca_city#32 ASC NULLS FIRST,bought_city#28 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#27,c_first_name#26,ca_city#32,bought_city#28,ss_ticket_number#5,amt#29,profit#30]), [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30], 100, 0, [c_last_name#27 ASC NULLS FIRST, c_first_name#26 ASC NULLS FIRST, ca_city#32 ASC NULLS FIRST, bought_city#28 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] - -(39) CometColumnarToRow [codegen id : 1] -Input [7]: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) - - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(41) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(42) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(43) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(44) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/simplified.txt deleted file mode 100644 index e646cb620a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - CometBroadcastHashJoin [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] - CometProject [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometHashAggregate [ss_addr_sk,sum,sum] [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,ca_city,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] - CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - CometHashAggregate [ss_coupon_amt,ss_net_profit] [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] - CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - 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_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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [ca_address_sk,ca_city] #6 - CometFilter [ca_address_sk,ca_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - ReusedExchange [ca_address_sk,ca_city] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt deleted file mode 100644 index ca7ae9df0c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt +++ /dev/null @@ -1,256 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (39) -+- CometTakeOrderedAndProject (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (34) - : +- CometBroadcastHashJoin (33) - : :- CometHashAggregate (28) - : : +- CometExchange (27) - : : +- CometHashAggregate (26) - : : +- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (20) - : : : +- CometBroadcastHashJoin (19) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometProject (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : : : +- CometBroadcastExchange (18) - : : : +- CometProject (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - : : +- CometBroadcastExchange (23) - : : +- CometFilter (22) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (21) - : +- CometBroadcastExchange (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (29) - +- ReusedExchange (35) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(5) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#13, s_city#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#13, s_city#14] -Condition : (s_city#14 IN (Fairview,Midway) AND isnotnull(s_store_sk#13)) - -(11) CometProject -Input [2]: [s_store_sk#13, s_city#14] -Arguments: [s_store_sk#13], [s_store_sk#13] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#13] -Arguments: [s_store_sk#13] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [1]: [s_store_sk#13] -Arguments: [ss_store_sk#4], [s_store_sk#13], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#13] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((hd_dep_count#16 = 4) OR (hd_vehicle_count#17 = 3)) AND isnotnull(hd_demo_sk#15)) - -(17) CometProject -Input [3]: [hd_demo_sk#15, hd_dep_count#16, hd_vehicle_count#17] -Arguments: [hd_demo_sk#15], [hd_demo_sk#15] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#15] -Arguments: [hd_demo_sk#15] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [1]: [hd_demo_sk#15] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#15], Inner, BuildRight - -(20) CometProject -Input [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, hd_demo_sk#15] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_city#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [ca_address_sk#18, ca_city#19] -Condition : (isnotnull(ca_address_sk#18) AND isnotnull(ca_city#19)) - -(23) CometBroadcastExchange -Input [2]: [ca_address_sk#18, ca_city#19] -Arguments: [ca_address_sk#18, ca_city#19] - -(24) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [2]: [ca_address_sk#18, ca_city#19] -Arguments: [ss_addr_sk#3], [ca_address_sk#18], Inner, BuildRight - -(25) CometProject -Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_address_sk#18, ca_city#19] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19] - -(26) CometHashAggregate -Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#19] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19] -Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] - -(27) CometExchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#20, sum#21] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#20, sum#21] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19] -Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] -Condition : (isnotnull(c_customer_sk#22) AND isnotnull(c_current_addr_sk#23)) - -(31) CometProject -Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] -Arguments: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27], [c_customer_sk#22, c_current_addr_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#24, 20, true, false, true) AS c_first_name#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#25, 30, true, false, true) AS c_last_name#27] - -(32) CometBroadcastExchange -Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Arguments: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] - -(33) CometBroadcastHashJoin -Left output [5]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, amt#29, profit#30] -Right output [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Arguments: [ss_customer_sk#1], [c_customer_sk#22], Inner, BuildRight - -(34) CometProject -Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, amt#29, profit#30, c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Arguments: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27], [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27] - -(35) ReusedExchange [Reuses operator id: 23] -Output [2]: [ca_address_sk#31, ca_city#32] - -(36) CometBroadcastHashJoin -Left output [7]: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -Right output [2]: [ca_address_sk#31, ca_city#32] -Arguments: [c_current_addr_sk#23], [ca_address_sk#31], Inner, NOT (ca_city#32 = bought_city#28), BuildRight - -(37) CometProject -Input [9]: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27, ca_address_sk#31, ca_city#32] -Arguments: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30], [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] - -(38) CometTakeOrderedAndProject -Input [7]: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#27 ASC NULLS FIRST,c_first_name#26 ASC NULLS FIRST,ca_city#32 ASC NULLS FIRST,bought_city#28 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#27,c_first_name#26,ca_city#32,bought_city#28,ss_ticket_number#5,amt#29,profit#30]), [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30], 100, 0, [c_last_name#27 ASC NULLS FIRST, c_first_name#26 ASC NULLS FIRST, ca_city#32 ASC NULLS FIRST, bought_city#28 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] - -(39) CometColumnarToRow [codegen id : 1] -Input [7]: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) - - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(41) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(42) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(43) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(44) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/extended.txt deleted file mode 100644 index c842000e11..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/extended.txt +++ /dev/null @@ -1,49 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index e646cb620a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - CometBroadcastHashJoin [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] - CometProject [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometHashAggregate [ss_addr_sk,sum,sum] [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,ca_city,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] - CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - CometHashAggregate [ss_coupon_amt,ss_net_profit] [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] - CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] - 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_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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [ca_address_sk,ca_city] #6 - CometFilter [ca_address_sk,ca_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - ReusedExchange [ca_address_sk,ca_city] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/explain.txt deleted file mode 100644 index 3f98e1bd21..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/explain.txt +++ /dev/null @@ -1,304 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (51) -+- * Project (50) - +- * BroadcastHashJoin Inner BuildRight (49) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * CometColumnarToRow (26) - : : +- CometSort (25) - : : +- CometColumnarExchange (24) - : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- * HashAggregate (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.store (14) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- Window (38) - : +- * CometColumnarToRow (37) - : +- CometSort (36) - : +- CometColumnarExchange (35) - : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) - : +- ReusedExchange (32) - +- BroadcastExchange (48) - +- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- ReusedExchange (43) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#4, i_category#5] - -(5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(7) Filter [codegen id : 1] -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) - -(8) BroadcastExchange -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#6] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(11) ReusedExchange [Reuses operator id: 55] -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] -Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) - -(16) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] - -(17) BroadcastExchange -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#7] -Right keys [1]: [s_store_sk#14] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] - -(20) HashAggregate [codegen id : 4] -Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] -Aggregate Attributes [1]: [sum#17] -Results [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] - -(21) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] - -(23) HashAggregate [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#8))#19] -Results [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#19,17,2) AS sum_sales#20, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#19,17,2) AS _w0#21] - -(24) CometColumnarExchange -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) CometColumnarToRow [codegen id : 6] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] - -(27) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(28) Filter [codegen id : 7] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(29) Window -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] -Arguments: [avg(_w0#21) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(30) Filter [codegen id : 22] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] -Condition : ((isnotnull(avg_monthly_sales#23) AND (avg_monthly_sales#23 > 0.000000)) AND CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END) - -(31) Project [codegen id : 22] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] - -(32) ReusedExchange [Reuses operator id: 21] -Output [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] - -(33) CometColumnarToRow [codegen id : 12] -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] - -(34) HashAggregate [codegen id : 12] -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -Keys [6]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] -Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#19] -Results [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#19,17,2) AS sum_sales#20] - -(35) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(36) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] -Arguments: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] - -(37) CometColumnarToRow [codegen id : 13] -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] - -(38) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] -Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#5, i_brand#4, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] - -(39) Project [codegen id : 14] -Output [6]: [i_category#5 AS i_category#31, i_brand#4 AS i_brand#32, s_store_name#24, s_company_name#25, sum_sales#20 AS sum_sales#33, rn#30] -Input [8]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20, rn#30] - -(40) BroadcastExchange -Input [6]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] - -(41) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] -Right keys [5]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, (rn#30 + 1)] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 22] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33] -Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] - -(43) ReusedExchange [Reuses operator id: 35] -Output [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] - -(44) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] -Arguments: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] - -(45) CometColumnarToRow [codegen id : 20] -Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] - -(46) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] -Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#5, i_brand#4, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] - -(47) Project [codegen id : 21] -Output [6]: [i_category#5 AS i_category#39, i_brand#4 AS i_brand#40, s_store_name#34, s_company_name#35, sum_sales#20 AS sum_sales#41, rn#38] -Input [8]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20, rn#38] - -(48) BroadcastExchange -Input [6]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] - -(49) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] -Right keys [5]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, (rn#38 - 1)] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 22] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, sum_sales#33 AS psum#42, sum_sales#41 AS nsum#43] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33, i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] - -(51) TakeOrderedAndProject -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] -Arguments: 100, [(sum_sales#20 - avg_monthly_sales#23) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometFilter (53) - +- CometNativeScan parquet spark_catalog.default.date_dim (52) - - -(52) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(54) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(55) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/simplified.txt deleted file mode 100644 index 017805e5b2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/simplified.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (22) - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (7) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/explain.txt deleted file mode 100644 index 1c039203ce..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_item_sk#1], [ss_item_sk#6], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) - -(16) CometBroadcastExchange -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [s_store_sk#14, s_store_name#15, s_company_name#16] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] -Right output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [ss_store_sk#7], [s_store_sk#14], Inner, BuildRight - -(18) CometProject -Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16], [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] - -(19) CometHashAggregate -Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] - -(20) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] - -(22) CometExchange -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] - -(25) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(28) Filter [codegen id : 7] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] - -(30) ReusedExchange [Reuses operator id: 20] -Output [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] - -(31) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] -Keys [6]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(ss_sales_price#27))] - -(32) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] - -(35) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [6]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] -Input [8]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28] - -(37) BroadcastExchange -Input [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] - -(40) ReusedExchange [Reuses operator id: 32] -Output [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(41) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(43) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#5, i_brand#4, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [6]: [i_category#5 AS i_category#37, i_brand#4 AS i_brand#38, s_store_name#32, s_company_name#33, sum_sales#18 AS sum_sales#39, rn#36] -Input [8]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18, rn#36] - -(45) BroadcastExchange -Input [6]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, (rn#36 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] - -(48) TakeOrderedAndProject -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/simplified.txt deleted file mode 100644 index 19124a9e20..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - 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 [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,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt deleted file mode 100644 index 1c039203ce..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_item_sk#1], [ss_item_sk#6], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) - -(16) CometBroadcastExchange -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [s_store_sk#14, s_store_name#15, s_company_name#16] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] -Right output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [ss_store_sk#7], [s_store_sk#14], Inner, BuildRight - -(18) CometProject -Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16], [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] - -(19) CometHashAggregate -Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] - -(20) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] - -(22) CometExchange -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] - -(25) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(28) Filter [codegen id : 7] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] - -(30) ReusedExchange [Reuses operator id: 20] -Output [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] - -(31) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] -Keys [6]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(ss_sales_price#27))] - -(32) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] - -(35) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [6]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] -Input [8]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28] - -(37) BroadcastExchange -Input [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] - -(40) ReusedExchange [Reuses operator id: 32] -Output [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(41) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(43) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#5, i_brand#4, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [6]: [i_category#5 AS i_category#37, i_brand#4 AS i_brand#38, s_store_name#32, s_company_name#33, sum_sales#18 AS sum_sales#39, rn#36] -Input [8]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18, rn#36] - -(45) BroadcastExchange -Input [6]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, (rn#36 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] - -(48) TakeOrderedAndProject -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/extended.txt deleted file mode 100644 index b50b570b4b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 19124a9e20..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - 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 [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,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/explain.txt deleted file mode 100644 index 494c1d4338..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/explain.txt +++ /dev/null @@ -1,207 +0,0 @@ -== Physical Plan == -* HashAggregate (30) -+- * CometColumnarToRow (29) - +- CometColumnarExchange (28) - +- * HashAggregate (27) - +- * Project (26) - +- * BroadcastHashJoin Inner BuildRight (25) - :- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * CometColumnarToRow (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.store (4) - : : +- BroadcastExchange (14) - : : +- * CometColumnarToRow (13) - : : +- CometProject (12) - : : +- CometFilter (11) - : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (10) - : +- BroadcastExchange (21) - : +- * CometColumnarToRow (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometNativeScan parquet spark_catalog.default.customer_address (17) - +- ReusedExchange (24) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_addr_sk), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ss_net_profit,0.00),LessThanOrEqual(ss_net_profit,2000.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,3000.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,25000.00)))] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] - -(3) Filter [codegen id : 5] -Input [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((((isnotnull(ss_store_sk#3) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_addr_sk#2)) AND ((((ss_sales_price#5 >= 100.00) AND (ss_sales_price#5 <= 150.00)) OR ((ss_sales_price#5 >= 50.00) AND (ss_sales_price#5 <= 100.00))) OR ((ss_sales_price#5 >= 150.00) AND (ss_sales_price#5 <= 200.00)))) AND ((((ss_net_profit#6 >= 0.00) AND (ss_net_profit#6 <= 2000.00)) OR ((ss_net_profit#6 >= 150.00) AND (ss_net_profit#6 <= 3000.00))) OR ((ss_net_profit#6 >= 50.00) AND (ss_net_profit#6 <= 25000.00)))) - -(4) CometNativeScan parquet spark_catalog.default.store -Output [1]: [s_store_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [1]: [s_store_sk#9] -Condition : isnotnull(s_store_sk#9) - -(6) CometColumnarToRow [codegen id : 1] -Input [1]: [s_store_sk#9] - -(7) BroadcastExchange -Input [1]: [s_store_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 5] -Output [6]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Input [8]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, s_store_sk#9] - -(10) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Condition : (isnotnull(cd_demo_sk#10) AND ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) = 4 yr Degree )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) = D) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) = 2 yr Degree ))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) = S) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) = College )))) - -(12) CometProject -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Arguments: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14], [cd_demo_sk#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) AS cd_marital_status#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) AS cd_education_status#14] - -(13) CometColumnarToRow [codegen id : 2] -Input [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] - -(14) BroadcastExchange -Input [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#1] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: ((((((cd_marital_status#13 = M) AND (cd_education_status#14 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#13 = D) AND (cd_education_status#14 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#13 = S) AND (cd_education_status#14 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) - -(16) Project [codegen id : 5] -Output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] -Input [9]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] - -(17) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(18) CometFilter -Input [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Condition : (((isnotnull(ca_country#17) AND (ca_country#17 = United States)) AND isnotnull(ca_address_sk#15)) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) IN (CO,OH,TX) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) IN (OR,MN,KY)) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) IN (VA,CA,MS))) - -(19) CometProject -Input [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Arguments: [ca_address_sk#15, ca_state#18], [ca_address_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) AS ca_state#18] - -(20) CometColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#15, ca_state#18] - -(21) BroadcastExchange -Input [2]: [ca_address_sk#15, ca_state#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(22) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#15] -Join type: Inner -Join condition: ((((ca_state#18 IN (CO,OH,TX) AND (ss_net_profit#6 >= 0.00)) AND (ss_net_profit#6 <= 2000.00)) OR ((ca_state#18 IN (OR,MN,KY) AND (ss_net_profit#6 >= 150.00)) AND (ss_net_profit#6 <= 3000.00))) OR ((ca_state#18 IN (VA,CA,MS) AND (ss_net_profit#6 >= 50.00)) AND (ss_net_profit#6 <= 25000.00))) - -(23) Project [codegen id : 5] -Output [2]: [ss_quantity#4, ss_sold_date_sk#7] -Input [6]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7, ca_address_sk#15, ca_state#18] - -(24) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#19] - -(25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 5] -Output [1]: [ss_quantity#4] -Input [3]: [ss_quantity#4, ss_sold_date_sk#7, d_date_sk#19] - -(27) HashAggregate [codegen id : 5] -Input [1]: [ss_quantity#4] -Keys: [] -Functions [1]: [partial_sum(ss_quantity#4)] -Aggregate Attributes [1]: [sum#20] -Results [1]: [sum#21] - -(28) CometColumnarExchange -Input [1]: [sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(29) CometColumnarToRow [codegen id : 6] -Input [1]: [sum#21] - -(30) HashAggregate [codegen id : 6] -Input [1]: [sum#21] -Keys: [] -Functions [1]: [sum(ss_quantity#4)] -Aggregate Attributes [1]: [sum(ss_quantity#4)#22] -Results [1]: [sum(ss_quantity#4)#22 AS sum(ss_quantity)#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometNativeScan parquet spark_catalog.default.date_dim (31) - - -(31) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_year#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#19, d_year#24] -Condition : ((isnotnull(d_year#24) AND (d_year#24 = 2001)) AND isnotnull(d_date_sk#19)) - -(33) CometProject -Input [2]: [d_date_sk#19, d_year#24] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(35) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/simplified.txt deleted file mode 100644 index 0cd7e900b0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (6) - HashAggregate [sum] [sum(ss_quantity),sum(ss_quantity),sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (5) - HashAggregate [ss_quantity] [sum,sum] - Project [ss_quantity] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_sold_date_sk] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] - Project [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] - Project [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/explain.txt deleted file mode 100644 index 71924d1e6d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,202 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometHashAggregate (28) - +- CometExchange (27) - +- CometHashAggregate (26) - +- CometProject (25) - +- CometBroadcastHashJoin (24) - :- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (8) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (14) - +- CometBroadcastExchange (23) - +- CometProject (22) - +- CometFilter (21) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (20) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_addr_sk), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ss_net_profit,0.00),LessThanOrEqual(ss_net_profit,2000.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,3000.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,25000.00)))] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((((isnotnull(ss_store_sk#3) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_addr_sk#2)) AND ((((ss_sales_price#5 >= 100.00) AND (ss_sales_price#5 <= 150.00)) OR ((ss_sales_price#5 >= 50.00) AND (ss_sales_price#5 <= 100.00))) OR ((ss_sales_price#5 >= 150.00) AND (ss_sales_price#5 <= 200.00)))) AND ((((ss_net_profit#6 >= 0.00) AND (ss_net_profit#6 <= 2000.00)) OR ((ss_net_profit#6 >= 150.00) AND (ss_net_profit#6 <= 3000.00))) OR ((ss_net_profit#6 >= 50.00) AND (ss_net_profit#6 <= 25000.00)))) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [s_store_sk#9] -Condition : isnotnull(s_store_sk#9) - -(5) CometBroadcastExchange -Input [1]: [s_store_sk#9] -Arguments: [s_store_sk#9] - -(6) CometBroadcastHashJoin -Left output [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [1]: [s_store_sk#9] -Arguments: [ss_store_sk#3], [s_store_sk#9], Inner, BuildRight - -(7) CometProject -Input [8]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, s_store_sk#9] -Arguments: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Condition : (isnotnull(cd_demo_sk#10) AND ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) = 4 yr Degree )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) = D) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) = 2 yr Degree ))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) = S) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) = College )))) - -(10) CometProject -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Arguments: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14], [cd_demo_sk#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) AS cd_marital_status#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) AS cd_education_status#14] - -(11) CometBroadcastExchange -Input [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] -Arguments: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] - -(12) CometBroadcastHashJoin -Left output [6]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] -Arguments: [ss_cdemo_sk#1], [cd_demo_sk#10], Inner, ((((((cd_marital_status#13 = M) AND (cd_education_status#14 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#13 = D) AND (cd_education_status#14 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#13 = S) AND (cd_education_status#14 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))), BuildRight - -(13) CometProject -Input [9]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] -Arguments: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7], [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Condition : (((isnotnull(ca_country#17) AND (ca_country#17 = United States)) AND isnotnull(ca_address_sk#15)) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) IN (CO,OH,TX) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) IN (OR,MN,KY)) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) IN (VA,CA,MS))) - -(16) CometProject -Input [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Arguments: [ca_address_sk#15, ca_state#18], [ca_address_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) AS ca_state#18] - -(17) CometBroadcastExchange -Input [2]: [ca_address_sk#15, ca_state#18] -Arguments: [ca_address_sk#15, ca_state#18] - -(18) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] -Right output [2]: [ca_address_sk#15, ca_state#18] -Arguments: [ss_addr_sk#2], [ca_address_sk#15], Inner, ((((ca_state#18 IN (CO,OH,TX) AND (ss_net_profit#6 >= 0.00)) AND (ss_net_profit#6 <= 2000.00)) OR ((ca_state#18 IN (OR,MN,KY) AND (ss_net_profit#6 >= 150.00)) AND (ss_net_profit#6 <= 3000.00))) OR ((ca_state#18 IN (VA,CA,MS) AND (ss_net_profit#6 >= 50.00)) AND (ss_net_profit#6 <= 25000.00))), BuildRight - -(19) CometProject -Input [6]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7, ca_address_sk#15, ca_state#18] -Arguments: [ss_quantity#4, ss_sold_date_sk#7], [ss_quantity#4, ss_sold_date_sk#7] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_year#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [d_date_sk#19, d_year#20] -Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(22) CometProject -Input [2]: [d_date_sk#19, d_year#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(23) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(24) CometBroadcastHashJoin -Left output [2]: [ss_quantity#4, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#7], [d_date_sk#19], Inner, BuildRight - -(25) CometProject -Input [3]: [ss_quantity#4, ss_sold_date_sk#7, d_date_sk#19] -Arguments: [ss_quantity#4], [ss_quantity#4] - -(26) CometHashAggregate -Input [1]: [ss_quantity#4] -Keys: [] -Functions [1]: [partial_sum(ss_quantity#4)] - -(27) CometExchange -Input [1]: [sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [1]: [sum#21] -Keys: [] -Functions [1]: [sum(ss_quantity#4)] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [sum(ss_quantity)#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_year#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [d_date_sk#19, d_year#20] -Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(32) CometProject -Input [2]: [d_date_sk#19, d_year#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(33) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(34) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/simplified.txt deleted file mode 100644 index e29e101c7e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,39 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [sum(ss_quantity),sum(ss_quantity)] - CometExchange #1 - CometHashAggregate [ss_quantity] [sum] - CometProject [ss_quantity] - CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] - CometProject [ss_quantity,ss_sold_date_sk] - 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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk] #3 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #4 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [ca_address_sk,ca_state] #5 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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/q48/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt deleted file mode 100644 index 71924d1e6d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt +++ /dev/null @@ -1,202 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometHashAggregate (28) - +- CometExchange (27) - +- CometHashAggregate (26) - +- CometProject (25) - +- CometBroadcastHashJoin (24) - :- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (8) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (14) - +- CometBroadcastExchange (23) - +- CometProject (22) - +- CometFilter (21) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (20) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_addr_sk), Or(Or(And(GreaterThanOrEqual(ss_sales_price,100.00),LessThanOrEqual(ss_sales_price,150.00)),And(GreaterThanOrEqual(ss_sales_price,50.00),LessThanOrEqual(ss_sales_price,100.00))),And(GreaterThanOrEqual(ss_sales_price,150.00),LessThanOrEqual(ss_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ss_net_profit,0.00),LessThanOrEqual(ss_net_profit,2000.00)),And(GreaterThanOrEqual(ss_net_profit,150.00),LessThanOrEqual(ss_net_profit,3000.00))),And(GreaterThanOrEqual(ss_net_profit,50.00),LessThanOrEqual(ss_net_profit,25000.00)))] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((((isnotnull(ss_store_sk#3) AND isnotnull(ss_cdemo_sk#1)) AND isnotnull(ss_addr_sk#2)) AND ((((ss_sales_price#5 >= 100.00) AND (ss_sales_price#5 <= 150.00)) OR ((ss_sales_price#5 >= 50.00) AND (ss_sales_price#5 <= 100.00))) OR ((ss_sales_price#5 >= 150.00) AND (ss_sales_price#5 <= 200.00)))) AND ((((ss_net_profit#6 >= 0.00) AND (ss_net_profit#6 <= 2000.00)) OR ((ss_net_profit#6 >= 150.00) AND (ss_net_profit#6 <= 3000.00))) OR ((ss_net_profit#6 >= 50.00) AND (ss_net_profit#6 <= 25000.00)))) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(4) CometFilter -Input [1]: [s_store_sk#9] -Condition : isnotnull(s_store_sk#9) - -(5) CometBroadcastExchange -Input [1]: [s_store_sk#9] -Arguments: [s_store_sk#9] - -(6) CometBroadcastHashJoin -Left output [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [1]: [s_store_sk#9] -Arguments: [ss_store_sk#3], [s_store_sk#9], Inner, BuildRight - -(7) CometProject -Input [8]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, s_store_sk#9] -Arguments: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Condition : (isnotnull(cd_demo_sk#10) AND ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) = 4 yr Degree )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) = D) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) = 2 yr Degree ))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) = S) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) = College )))) - -(10) CometProject -Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] -Arguments: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14], [cd_demo_sk#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#11, 1, true, false, true) AS cd_marital_status#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#12, 20, true, false, true) AS cd_education_status#14] - -(11) CometBroadcastExchange -Input [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] -Arguments: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] - -(12) CometBroadcastHashJoin -Left output [6]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [3]: [cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] -Arguments: [ss_cdemo_sk#1], [cd_demo_sk#10], Inner, ((((((cd_marital_status#13 = M) AND (cd_education_status#14 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#13 = D) AND (cd_education_status#14 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#13 = S) AND (cd_education_status#14 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))), BuildRight - -(13) CometProject -Input [9]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, cd_demo_sk#10, cd_marital_status#13, cd_education_status#14] -Arguments: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7], [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Condition : (((isnotnull(ca_country#17) AND (ca_country#17 = United States)) AND isnotnull(ca_address_sk#15)) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) IN (CO,OH,TX) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) IN (OR,MN,KY)) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) IN (VA,CA,MS))) - -(16) CometProject -Input [3]: [ca_address_sk#15, ca_state#16, ca_country#17] -Arguments: [ca_address_sk#15, ca_state#18], [ca_address_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) AS ca_state#18] - -(17) CometBroadcastExchange -Input [2]: [ca_address_sk#15, ca_state#18] -Arguments: [ca_address_sk#15, ca_state#18] - -(18) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] -Right output [2]: [ca_address_sk#15, ca_state#18] -Arguments: [ss_addr_sk#2], [ca_address_sk#15], Inner, ((((ca_state#18 IN (CO,OH,TX) AND (ss_net_profit#6 >= 0.00)) AND (ss_net_profit#6 <= 2000.00)) OR ((ca_state#18 IN (OR,MN,KY) AND (ss_net_profit#6 >= 150.00)) AND (ss_net_profit#6 <= 3000.00))) OR ((ca_state#18 IN (VA,CA,MS) AND (ss_net_profit#6 >= 50.00)) AND (ss_net_profit#6 <= 25000.00))), BuildRight - -(19) CometProject -Input [6]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7, ca_address_sk#15, ca_state#18] -Arguments: [ss_quantity#4, ss_sold_date_sk#7], [ss_quantity#4, ss_sold_date_sk#7] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_year#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [d_date_sk#19, d_year#20] -Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(22) CometProject -Input [2]: [d_date_sk#19, d_year#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(23) CometBroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: [d_date_sk#19] - -(24) CometBroadcastHashJoin -Left output [2]: [ss_quantity#4, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#19] -Arguments: [ss_sold_date_sk#7], [d_date_sk#19], Inner, BuildRight - -(25) CometProject -Input [3]: [ss_quantity#4, ss_sold_date_sk#7, d_date_sk#19] -Arguments: [ss_quantity#4], [ss_quantity#4] - -(26) CometHashAggregate -Input [1]: [ss_quantity#4] -Keys: [] -Functions [1]: [partial_sum(ss_quantity#4)] - -(27) CometExchange -Input [1]: [sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [1]: [sum#21] -Keys: [] -Functions [1]: [sum(ss_quantity#4)] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [sum(ss_quantity)#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#19, d_year#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [d_date_sk#19, d_year#20] -Condition : ((isnotnull(d_year#20) AND (d_year#20 = 2001)) AND isnotnull(d_date_sk#19)) - -(32) CometProject -Input [2]: [d_date_sk#19, d_year#20] -Arguments: [d_date_sk#19], [d_date_sk#19] - -(33) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#19] - -(34) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/extended.txt deleted file mode 100644 index 57fbc2d035..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index e29e101c7e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/simplified.txt +++ /dev/null @@ -1,39 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [sum(ss_quantity),sum(ss_quantity)] - CometExchange #1 - CometHashAggregate [ss_quantity] [sum] - CometProject [ss_quantity] - CometBroadcastHashJoin [ss_quantity,ss_sold_date_sk,d_date_sk] - CometProject [ss_quantity,ss_sold_date_sk] - 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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk] #3 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #4 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [ca_address_sk,ca_state] #5 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/explain.txt deleted file mode 100644 index 0a43ebc390..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/explain.txt +++ /dev/null @@ -1,497 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (84) -+- CometTakeOrderedAndProject (83) - +- CometHashAggregate (82) - +- CometColumnarExchange (81) - +- * HashAggregate (80) - +- Union (79) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometColumnarExchange (19) - : +- * HashAggregate (18) - : +- * CometColumnarToRow (17) - : +- CometColumnarExchange (16) - : +- * HashAggregate (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildLeft (10) - : : :- BroadcastExchange (5) - : : : +- * Project (4) - : : : +- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : +- * CometColumnarToRow (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometNativeScan parquet spark_catalog.default.web_returns (6) - : +- ReusedExchange (12) - :- * Project (52) - : +- * Filter (51) - : +- Window (50) - : +- * Sort (49) - : +- Window (48) - : +- * CometColumnarToRow (47) - : +- CometSort (46) - : +- CometColumnarExchange (45) - : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- CometColumnarExchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildLeft (36) - : : :- BroadcastExchange (31) - : : : +- * Project (30) - : : : +- * Filter (29) - : : : +- * ColumnarToRow (28) - : : : +- Scan parquet spark_catalog.default.catalog_sales (27) - : : +- * CometColumnarToRow (35) - : : +- CometProject (34) - : : +- CometFilter (33) - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (32) - : +- ReusedExchange (38) - +- * Project (78) - +- * Filter (77) - +- Window (76) - +- * Sort (75) - +- Window (74) - +- * CometColumnarToRow (73) - +- CometSort (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * CometColumnarToRow (69) - +- CometColumnarExchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * BroadcastHashJoin Inner BuildRight (65) - :- * Project (63) - : +- * BroadcastHashJoin Inner BuildLeft (62) - : :- BroadcastExchange (57) - : : +- * Project (56) - : : +- * Filter (55) - : : +- * ColumnarToRow (54) - : : +- Scan parquet spark_catalog.default.store_sales (53) - : +- * CometColumnarToRow (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometNativeScan parquet spark_catalog.default.store_returns (58) - +- ReusedExchange (64) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] - -(3) Filter [codegen id : 1] -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND isnotnull(ws_quantity#3)) AND (ws_net_profit#5 > 1.00)) AND (ws_net_paid#4 > 0.00)) AND (ws_quantity#3 > 0)) AND isnotnull(ws_order_number#2)) AND isnotnull(ws_item_sk#1)) - -(4) Project [codegen id : 1] -Output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] - -(5) BroadcastExchange -Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=1] - -(6) CometNativeScan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) - -(8) CometProject -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(9) CometColumnarToRow -Input [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(10) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [ws_order_number#2, ws_item_sk#1] -Right keys [2]: [wr_order_number#9, wr_item_sk#8] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 3] -Output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(12) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#13] - -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#6] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 3] -Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] - -(15) HashAggregate [codegen id : 3] -Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Keys [1]: [ws_item_sk#1] -Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#14, sum#15, sum#16, isEmpty#17, sum#18, isEmpty#19] -Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] - -(16) CometColumnarExchange -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(17) CometColumnarToRow [codegen id : 4] -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] - -(18) HashAggregate [codegen id : 4] -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Keys [1]: [ws_item_sk#1] -Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] -Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] - -(19) CometColumnarExchange -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(20) CometSort -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [item#30, return_ratio#31, currency_ratio#32], [return_ratio#31 ASC NULLS FIRST] - -(21) CometColumnarToRow [codegen id : 5] -Input [3]: [item#30, return_ratio#31, currency_ratio#32] - -(22) Window -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] - -(23) Sort [codegen id : 6] -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 - -(24) Window -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] - -(25) Filter [codegen id : 7] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] -Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) - -(26) Project [codegen id : 7] -Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] - -(27) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(28) ColumnarToRow [codegen id : 8] -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] - -(29) Filter [codegen id : 8] -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) - -(30) Project [codegen id : 8] -Output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] - -(31) BroadcastExchange -Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=4] - -(32) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] -Condition : (((isnotnull(cr_return_amount#45) AND (cr_return_amount#45 > 10000.00)) AND isnotnull(cr_order_number#43)) AND isnotnull(cr_item_sk#42)) - -(34) CometProject -Input [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] -Arguments: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45], [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] - -(35) CometColumnarToRow -Input [4]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] - -(36) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [cs_order_number#37, cs_item_sk#36] -Right keys [2]: [cr_order_number#43, cr_item_sk#42] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 10] -Output [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45] -Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] - -(38) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#47] - -(39) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#41] -Right keys [1]: [d_date_sk#47] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 10] -Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] -Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45, d_date_sk#47] - -(41) HashAggregate [codegen id : 10] -Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] -Keys [1]: [cs_item_sk#36] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#44, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#48, sum#49, sum#50, isEmpty#51, sum#52, isEmpty#53] -Results [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] - -(42) CometColumnarExchange -Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(43) CometColumnarToRow [codegen id : 11] -Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] - -(44) HashAggregate [codegen id : 11] -Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Keys [1]: [cs_item_sk#36] -Functions [4]: [sum(coalesce(cr_return_quantity#44, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#44, 0))#60, sum(coalesce(cs_quantity#38, 0))#61, sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#62, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63] -Results [3]: [cs_item_sk#36 AS item#64, (cast(sum(coalesce(cr_return_quantity#44, 0))#60 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#61 as decimal(15,4))) AS return_ratio#65, (cast(sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#62 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63 as decimal(15,4))) AS currency_ratio#66] - -(45) CometColumnarExchange -Input [3]: [item#64, return_ratio#65, currency_ratio#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(46) CometSort -Input [3]: [item#64, return_ratio#65, currency_ratio#66] -Arguments: [item#64, return_ratio#65, currency_ratio#66], [return_ratio#65 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 12] -Input [3]: [item#64, return_ratio#65, currency_ratio#66] - -(48) Window -Input [3]: [item#64, return_ratio#65, currency_ratio#66] -Arguments: [rank(return_ratio#65) windowspecdefinition(return_ratio#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#67], [return_ratio#65 ASC NULLS FIRST] - -(49) Sort [codegen id : 13] -Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] -Arguments: [currency_ratio#66 ASC NULLS FIRST], false, 0 - -(50) Window -Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] -Arguments: [rank(currency_ratio#66) windowspecdefinition(currency_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#68], [currency_ratio#66 ASC NULLS FIRST] - -(51) Filter [codegen id : 14] -Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] -Condition : ((return_rank#67 <= 10) OR (currency_rank#68 <= 10)) - -(52) Project [codegen id : 14] -Output [5]: [catalog AS channel#69, item#64, return_ratio#65, return_rank#67, currency_rank#68] -Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] - -(53) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#75), dynamicpruningexpression(ss_sold_date_sk#75 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(54) ColumnarToRow [codegen id : 15] -Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] - -(55) Filter [codegen id : 15] -Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] -Condition : (((((((isnotnull(ss_net_profit#74) AND isnotnull(ss_net_paid#73)) AND isnotnull(ss_quantity#72)) AND (ss_net_profit#74 > 1.00)) AND (ss_net_paid#73 > 0.00)) AND (ss_quantity#72 > 0)) AND isnotnull(ss_ticket_number#71)) AND isnotnull(ss_item_sk#70)) - -(56) Project [codegen id : 15] -Output [5]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75] -Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] - -(57) BroadcastExchange -Input [5]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=7] - -(58) CometNativeScan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(59) CometFilter -Input [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] -Condition : (((isnotnull(sr_return_amt#79) AND (sr_return_amt#79 > 10000.00)) AND isnotnull(sr_ticket_number#77)) AND isnotnull(sr_item_sk#76)) - -(60) CometProject -Input [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] -Arguments: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79], [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] - -(61) CometColumnarToRow -Input [4]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] - -(62) BroadcastHashJoin [codegen id : 17] -Left keys [2]: [ss_ticket_number#71, ss_item_sk#70] -Right keys [2]: [sr_ticket_number#77, sr_item_sk#76] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 17] -Output [6]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79] -Input [9]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] - -(64) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#81] - -(65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#75] -Right keys [1]: [d_date_sk#81] -Join type: Inner -Join condition: None - -(66) Project [codegen id : 17] -Output [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] -Input [7]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79, d_date_sk#81] - -(67) HashAggregate [codegen id : 17] -Input [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] -Keys [1]: [ss_item_sk#70] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#78, 0)), partial_sum(coalesce(ss_quantity#72, 0)), partial_sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#82, sum#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Results [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] - -(68) CometColumnarExchange -Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] -Arguments: hashpartitioning(ss_item_sk#70, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(69) CometColumnarToRow [codegen id : 18] -Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] - -(70) HashAggregate [codegen id : 18] -Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] -Keys [1]: [ss_item_sk#70] -Functions [4]: [sum(coalesce(sr_return_quantity#78, 0)), sum(coalesce(ss_quantity#72, 0)), sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#78, 0))#94, sum(coalesce(ss_quantity#72, 0))#95, sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00))#96, sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97] -Results [3]: [ss_item_sk#70 AS item#98, (cast(sum(coalesce(sr_return_quantity#78, 0))#94 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#72, 0))#95 as decimal(15,4))) AS return_ratio#99, (cast(sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00))#96 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97 as decimal(15,4))) AS currency_ratio#100] - -(71) CometColumnarExchange -Input [3]: [item#98, return_ratio#99, currency_ratio#100] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(72) CometSort -Input [3]: [item#98, return_ratio#99, currency_ratio#100] -Arguments: [item#98, return_ratio#99, currency_ratio#100], [return_ratio#99 ASC NULLS FIRST] - -(73) CometColumnarToRow [codegen id : 19] -Input [3]: [item#98, return_ratio#99, currency_ratio#100] - -(74) Window -Input [3]: [item#98, return_ratio#99, currency_ratio#100] -Arguments: [rank(return_ratio#99) windowspecdefinition(return_ratio#99 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#101], [return_ratio#99 ASC NULLS FIRST] - -(75) Sort [codegen id : 20] -Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] -Arguments: [currency_ratio#100 ASC NULLS FIRST], false, 0 - -(76) Window -Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] -Arguments: [rank(currency_ratio#100) windowspecdefinition(currency_ratio#100 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#102], [currency_ratio#100 ASC NULLS FIRST] - -(77) Filter [codegen id : 21] -Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] -Condition : ((return_rank#101 <= 10) OR (currency_rank#102 <= 10)) - -(78) Project [codegen id : 21] -Output [5]: [store AS channel#103, item#98, return_ratio#99, return_rank#101, currency_rank#102] -Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] - -(79) Union - -(80) HashAggregate [codegen id : 22] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] - -(81) CometColumnarExchange -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(82) CometHashAggregate -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Functions: [] - -(83) CometTakeOrderedAndProject -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,return_rank#33 ASC NULLS FIRST,currency_rank#34 ASC NULLS FIRST], output=[channel#35,item#30,return_ratio#31,return_rank#33,currency_rank#34]), [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34], 100, 0, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] - -(84) CometColumnarToRow [codegen id : 23] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (89) -+- * CometColumnarToRow (88) - +- CometProject (87) - +- CometFilter (86) - +- CometNativeScan parquet spark_catalog.default.date_dim (85) - - -(85) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#104, d_moy#105] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(86) CometFilter -Input [3]: [d_date_sk#13, d_year#104, d_moy#105] -Condition : ((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND (d_year#104 = 2001)) AND (d_moy#105 = 12)) AND isnotnull(d_date_sk#13)) - -(87) CometProject -Input [3]: [d_date_sk#13, d_year#104, d_moy#105] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(88) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(89) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 - -Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#75 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/simplified.txt deleted file mode 100644 index 047269434d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/simplified.txt +++ /dev/null @@ -1,140 +0,0 @@ -WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (22) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (7) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (6) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #2 - WholeStageCodegen (4) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (14) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (13) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #6 - WholeStageCodegen (11) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_item_sk] #7 - WholeStageCodegen (10) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (21) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (20) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #9 - WholeStageCodegen (18) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk] #10 - WholeStageCodegen (17) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/explain.txt deleted file mode 100644 index 338f5f4201..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,462 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- * Project (49) - : +- * Filter (48) - : +- Window (47) - : +- * Sort (46) - : +- Window (45) - : +- * CometColumnarToRow (44) - : +- CometSort (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometBroadcastExchange (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometBroadcastHashJoin (57) - : :- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (59) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND isnotnull(ws_quantity#3)) AND (ws_net_profit#5 > 1.00)) AND (ws_net_paid#4 > 0.00)) AND (ws_quantity#3 > 0)) AND isnotnull(ws_order_number#2)) AND isnotnull(ws_item_sk#1)) - -(3) CometProject -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6], [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(4) CometBroadcastExchange -Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) - -(7) CometProject -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(8) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner, BuildLeft - -(9) CometProject -Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(12) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] - -(16) CometHashAggregate -Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Keys [1]: [ws_item_sk#1] -Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(17) CometExchange -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Keys [1]: [ws_item_sk#1] -Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(19) CometExchange -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] - -(21) CometColumnarToRow [codegen id : 1] -Input [3]: [item#22, return_ratio#23, currency_ratio#24] - -(22) Window -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] - -(23) Sort [codegen id : 2] -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 - -(24) Window -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] - -(25) Filter [codegen id : 3] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] -Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) - -(26) Project [codegen id : 3] -Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_sold_date_sk#33 IN dynamicpruning#34)] -PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(28) CometFilter -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) - -(29) CometProject -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(30) CometBroadcastExchange -Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(32) CometFilter -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) - -(33) CometProject -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] - -(34) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft - -(35) CometProject -Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] - -(36) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#40] - -(37) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -Right output [1]: [d_date_sk#40] -Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight - -(38) CometProject -Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] - -(39) CometHashAggregate -Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -Keys [1]: [cs_item_sk#28] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(40) CometExchange -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(41) CometHashAggregate -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Keys [1]: [cs_item_sk#28] -Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(42) CometExchange -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(43) CometSort -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] - -(44) CometColumnarToRow [codegen id : 4] -Input [3]: [item#47, return_ratio#48, currency_ratio#49] - -(45) Window -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] - -(46) Sort [codegen id : 5] -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 - -(47) Window -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] - -(48) Filter [codegen id : 6] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] -Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) - -(49) Project [codegen id : 6] -Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) - -(52) CometProject -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(53) CometBroadcastExchange -Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(55) CometFilter -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) - -(56) CometProject -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] - -(57) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft - -(58) CometProject -Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] - -(59) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#65] - -(60) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(61) CometProject -Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] - -(62) CometHashAggregate -Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -Keys [1]: [ss_item_sk#53] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(63) CometExchange -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(64) CometHashAggregate -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Keys [1]: [ss_item_sk#53] -Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(65) CometExchange -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(66) CometSort -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] - -(67) CometColumnarToRow [codegen id : 7] -Input [3]: [item#72, return_ratio#73, currency_ratio#74] - -(68) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] - -(69) Sort [codegen id : 8] -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 - -(70) Window -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] - -(71) Filter [codegen id : 9] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] -Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) - -(72) Project [codegen id : 9] -Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] - -(73) Union - -(74) HashAggregate [codegen id : 10] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(75) CometColumnarExchange -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometHashAggregate -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] - -(77) CometTakeOrderedAndProject -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(78) CometColumnarToRow [codegen id : 11] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) - - -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(80) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(81) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(82) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(83) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 - -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/simplified.txt deleted file mode 100644 index ca80833ee5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,110 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (10) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (3) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (2) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - 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_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_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - 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_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_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 (9) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (8) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - 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_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_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt deleted file mode 100644 index 338f5f4201..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt +++ /dev/null @@ -1,462 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- * Project (49) - : +- * Filter (48) - : +- Window (47) - : +- * Sort (46) - : +- Window (45) - : +- * CometColumnarToRow (44) - : +- CometSort (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometBroadcastExchange (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometBroadcastHashJoin (57) - : :- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (59) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND isnotnull(ws_quantity#3)) AND (ws_net_profit#5 > 1.00)) AND (ws_net_paid#4 > 0.00)) AND (ws_quantity#3 > 0)) AND isnotnull(ws_order_number#2)) AND isnotnull(ws_item_sk#1)) - -(3) CometProject -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6], [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(4) CometBroadcastExchange -Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) - -(7) CometProject -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(8) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner, BuildLeft - -(9) CometProject -Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(12) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] - -(16) CometHashAggregate -Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Keys [1]: [ws_item_sk#1] -Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(17) CometExchange -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Keys [1]: [ws_item_sk#1] -Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(19) CometExchange -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] - -(21) CometColumnarToRow [codegen id : 1] -Input [3]: [item#22, return_ratio#23, currency_ratio#24] - -(22) Window -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] - -(23) Sort [codegen id : 2] -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 - -(24) Window -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] - -(25) Filter [codegen id : 3] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] -Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) - -(26) Project [codegen id : 3] -Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_sold_date_sk#33 IN dynamicpruning#34)] -PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(28) CometFilter -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) - -(29) CometProject -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(30) CometBroadcastExchange -Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(32) CometFilter -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) - -(33) CometProject -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] - -(34) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft - -(35) CometProject -Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] - -(36) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#40] - -(37) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -Right output [1]: [d_date_sk#40] -Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight - -(38) CometProject -Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] - -(39) CometHashAggregate -Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -Keys [1]: [cs_item_sk#28] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(40) CometExchange -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(41) CometHashAggregate -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Keys [1]: [cs_item_sk#28] -Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(42) CometExchange -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(43) CometSort -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] - -(44) CometColumnarToRow [codegen id : 4] -Input [3]: [item#47, return_ratio#48, currency_ratio#49] - -(45) Window -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] - -(46) Sort [codegen id : 5] -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 - -(47) Window -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] - -(48) Filter [codegen id : 6] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] -Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) - -(49) Project [codegen id : 6] -Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) - -(52) CometProject -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(53) CometBroadcastExchange -Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(55) CometFilter -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) - -(56) CometProject -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] - -(57) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft - -(58) CometProject -Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] - -(59) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#65] - -(60) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(61) CometProject -Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] - -(62) CometHashAggregate -Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -Keys [1]: [ss_item_sk#53] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(63) CometExchange -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(64) CometHashAggregate -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Keys [1]: [ss_item_sk#53] -Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(65) CometExchange -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(66) CometSort -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] - -(67) CometColumnarToRow [codegen id : 7] -Input [3]: [item#72, return_ratio#73, currency_ratio#74] - -(68) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] - -(69) Sort [codegen id : 8] -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 - -(70) Window -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] - -(71) Filter [codegen id : 9] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] -Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) - -(72) Project [codegen id : 9] -Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] - -(73) Union - -(74) HashAggregate [codegen id : 10] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(75) CometColumnarExchange -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometHashAggregate -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] - -(77) CometTakeOrderedAndProject -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(78) CometColumnarToRow [codegen id : 11] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) - - -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(80) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(81) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(82) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(83) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 - -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/extended.txt deleted file mode 100644 index 75684a9669..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/extended.txt +++ /dev/null @@ -1,94 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index ca80833ee5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt +++ /dev/null @@ -1,110 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (10) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (3) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (2) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - 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_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_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - 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_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_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 (9) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (8) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - 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_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_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/explain.txt deleted file mode 100644 index a0a95bd28b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/explain.txt +++ /dev/null @@ -1,499 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (81) -+- * HashAggregate (80) - +- * CometColumnarToRow (79) - +- CometColumnarExchange (78) - +- * HashAggregate (77) - +- * Expand (76) - +- Union (75) - :- * HashAggregate (23) - : +- * CometColumnarToRow (22) - : +- CometColumnarExchange (21) - : +- * HashAggregate (20) - : +- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- Union (9) - : : : :- * Project (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- * Project (8) - : : : +- * Filter (7) - : : : +- * ColumnarToRow (6) - : : : +- Scan parquet spark_catalog.default.store_returns (5) - : : +- ReusedExchange (10) - : +- BroadcastExchange (17) - : +- * CometColumnarToRow (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometNativeScan parquet spark_catalog.default.store (13) - :- * HashAggregate (46) - : +- * CometColumnarToRow (45) - : +- CometColumnarExchange (44) - : +- * HashAggregate (43) - : +- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- Union (32) - : : : :- * Project (27) - : : : : +- * Filter (26) - : : : : +- * ColumnarToRow (25) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (24) - : : : +- * Project (31) - : : : +- * Filter (30) - : : : +- * ColumnarToRow (29) - : : : +- Scan parquet spark_catalog.default.catalog_returns (28) - : : +- ReusedExchange (33) - : +- BroadcastExchange (40) - : +- * CometColumnarToRow (39) - : +- CometProject (38) - : +- CometFilter (37) - : +- CometNativeScan parquet spark_catalog.default.catalog_page (36) - +- * HashAggregate (74) - +- * CometColumnarToRow (73) - +- CometColumnarExchange (72) - +- * HashAggregate (71) - +- * Project (70) - +- * BroadcastHashJoin Inner BuildRight (69) - :- * Project (63) - : +- * BroadcastHashJoin Inner BuildRight (62) - : :- Union (60) - : : :- * Project (50) - : : : +- * Filter (49) - : : : +- * ColumnarToRow (48) - : : : +- Scan parquet spark_catalog.default.web_sales (47) - : : +- * Project (59) - : : +- * BroadcastHashJoin Inner BuildLeft (58) - : : :- BroadcastExchange (53) - : : : +- * ColumnarToRow (52) - : : : +- Scan parquet spark_catalog.default.web_returns (51) - : : +- * CometColumnarToRow (57) - : : +- CometProject (56) - : : +- CometFilter (55) - : : +- CometNativeScan parquet spark_catalog.default.web_sales (54) - : +- ReusedExchange (61) - +- BroadcastExchange (68) - +- * CometColumnarToRow (67) - +- CometProject (66) - +- CometFilter (65) - +- CometNativeScan parquet spark_catalog.default.web_site (64) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 1] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(4) Project [codegen id : 1] -Output [6]: [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] - -(5) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 2] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] - -(7) Filter [codegen id : 2] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#12) - -(8) Project [codegen id : 2] -Output [6]: [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] - -(9) Union - -(10) ReusedExchange [Reuses operator id: 86] -Output [1]: [d_date_sk#22] - -(11) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [date_sk#7] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None - -(12) Project [codegen id : 5] -Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] - -(13) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#23, s_store_id#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [s_store_sk#23, s_store_id#24] -Condition : isnotnull(s_store_sk#23) - -(15) CometProject -Input [2]: [s_store_sk#23, s_store_id#24] -Arguments: [s_store_sk#23, s_store_id#25], [s_store_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#24, 16, true, false, true) AS s_store_id#25] - -(16) CometColumnarToRow [codegen id : 4] -Input [2]: [s_store_sk#23, s_store_id#25] - -(17) BroadcastExchange -Input [2]: [s_store_sk#23, s_store_id#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(18) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [store_sk#6] -Right keys [1]: [s_store_sk#23] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 5] -Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#25] - -(20) HashAggregate [codegen id : 5] -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -Keys [1]: [s_store_id#25] -Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum#26, sum#27, sum#28, sum#29] -Results [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] - -(21) CometColumnarExchange -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(22) CometColumnarToRow [codegen id : 6] -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] - -(23) HashAggregate [codegen id : 6] -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -Keys [1]: [s_store_id#25] -Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#34, sum(UnscaledValue(return_amt#10))#35, sum(UnscaledValue(profit#9))#36, sum(UnscaledValue(net_loss#11))#37] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#8))#34,17,2) AS sales#38, MakeDecimal(sum(UnscaledValue(return_amt#10))#35,17,2) AS returns#39, (MakeDecimal(sum(UnscaledValue(profit#9))#36,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#37,17,2)) AS profit#40, store channel AS channel#41, concat(store, s_store_id#25) AS id#42] - -(24) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_sold_date_sk#46 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cs_catalog_page_sk)] -ReadSchema: struct - -(25) ColumnarToRow [codegen id : 7] -Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] - -(26) Filter [codegen id : 7] -Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Condition : isnotnull(cs_catalog_page_sk#43) - -(27) Project [codegen id : 7] -Output [6]: [cs_catalog_page_sk#43 AS page_sk#47, cs_sold_date_sk#46 AS date_sk#48, cs_ext_sales_price#44 AS sales_price#49, cs_net_profit#45 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] -Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] - -(28) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cr_catalog_page_sk)] -ReadSchema: struct - -(29) ColumnarToRow [codegen id : 8] -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] - -(30) Filter [codegen id : 8] -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -Condition : isnotnull(cr_catalog_page_sk#53) - -(31) Project [codegen id : 8] -Output [6]: [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] - -(32) Union - -(33) ReusedExchange [Reuses operator id: 86] -Output [1]: [d_date_sk#63] - -(34) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [date_sk#48] -Right keys [1]: [d_date_sk#63] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 11] -Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] -Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] - -(36) CometNativeScan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Condition : isnotnull(cp_catalog_page_sk#64) - -(38) CometProject -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Arguments: [cp_catalog_page_sk#64, cp_catalog_page_id#66], [cp_catalog_page_sk#64, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#65, 16, true, false, true) AS cp_catalog_page_id#66] - -(39) CometColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] - -(40) BroadcastExchange -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(41) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [page_sk#47] -Right keys [1]: [cp_catalog_page_sk#64] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 11] -Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] -Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#66] - -(43) HashAggregate [codegen id : 11] -Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] -Keys [1]: [cp_catalog_page_id#66] -Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] -Aggregate Attributes [4]: [sum#67, sum#68, sum#69, sum#70] -Results [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] - -(44) CometColumnarExchange -Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] -Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(45) CometColumnarToRow [codegen id : 12] -Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] - -(46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] -Keys [1]: [cp_catalog_page_id#66] -Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#75, sum(UnscaledValue(return_amt#51))#76, sum(UnscaledValue(profit#50))#77, sum(UnscaledValue(net_loss#52))#78] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#49))#75,17,2) AS sales#79, MakeDecimal(sum(UnscaledValue(return_amt#51))#76,17,2) AS returns#80, (MakeDecimal(sum(UnscaledValue(profit#50))#77,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#52))#78,17,2)) AS profit#81, catalog channel AS channel#82, concat(catalog_page, cp_catalog_page_id#66) AS id#83] - -(47) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_sold_date_sk#87 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] - -(49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] -Condition : isnotnull(ws_web_site_sk#84) - -(50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#84 AS wsr_web_site_sk#88, ws_sold_date_sk#87 AS date_sk#89, ws_ext_sales_price#85 AS sales_price#90, ws_net_profit#86 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] -Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] - -(51) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#5)] -ReadSchema: struct - -(52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] - -(53) BroadcastExchange -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [plan_id=5] - -(54) CometNativeScan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(55) CometFilter -Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) - -(56) CometProject -Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] - -(57) CometColumnarToRow -Input [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] - -(58) BroadcastHashJoin [codegen id : 15] -Left keys [2]: [wr_item_sk#94, wr_order_number#95] -Right keys [2]: [ws_item_sk#99, ws_order_number#101] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 15] -Output [6]: [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] -Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] - -(60) Union - -(61) ReusedExchange [Reuses operator id: 86] -Output [1]: [d_date_sk#109] - -(62) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [date_sk#89] -Right keys [1]: [d_date_sk#109] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 18] -Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] -Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] - -(64) CometNativeScan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#110, web_site_id#111] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(65) CometFilter -Input [2]: [web_site_sk#110, web_site_id#111] -Condition : isnotnull(web_site_sk#110) - -(66) CometProject -Input [2]: [web_site_sk#110, web_site_id#111] -Arguments: [web_site_sk#110, web_site_id#112], [web_site_sk#110, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#111, 16, true, false, true) AS web_site_id#112] - -(67) CometColumnarToRow [codegen id : 17] -Input [2]: [web_site_sk#110, web_site_id#112] - -(68) BroadcastExchange -Input [2]: [web_site_sk#110, web_site_id#112] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(69) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [wsr_web_site_sk#88] -Right keys [1]: [web_site_sk#110] -Join type: Inner -Join condition: None - -(70) Project [codegen id : 18] -Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] -Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#112] - -(71) HashAggregate [codegen id : 18] -Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] -Keys [1]: [web_site_id#112] -Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum#113, sum#114, sum#115, sum#116] -Results [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] - -(72) CometColumnarExchange -Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] -Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(73) CometColumnarToRow [codegen id : 19] -Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] - -(74) HashAggregate [codegen id : 19] -Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] -Keys [1]: [web_site_id#112] -Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#121, sum(UnscaledValue(return_amt#92))#122, sum(UnscaledValue(profit#91))#123, sum(UnscaledValue(net_loss#93))#124] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#90))#121,17,2) AS sales#125, MakeDecimal(sum(UnscaledValue(return_amt#92))#122,17,2) AS returns#126, (MakeDecimal(sum(UnscaledValue(profit#91))#123,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#124,17,2)) AS profit#127, web channel AS channel#128, concat(web_site, web_site_id#112) AS id#129] - -(75) Union - -(76) Expand [codegen id : 20] -Input [5]: [sales#38, returns#39, profit#40, channel#41, id#42] -Arguments: [[sales#38, returns#39, profit#40, channel#41, id#42, 0], [sales#38, returns#39, profit#40, channel#41, null, 1], [sales#38, returns#39, profit#40, null, null, 3]], [sales#38, returns#39, profit#40, channel#130, id#131, spark_grouping_id#132] - -(77) HashAggregate [codegen id : 20] -Input [6]: [sales#38, returns#39, profit#40, channel#130, id#131, spark_grouping_id#132] -Keys [3]: [channel#130, id#131, spark_grouping_id#132] -Functions [3]: [partial_sum(sales#38), partial_sum(returns#39), partial_sum(profit#40)] -Aggregate Attributes [6]: [sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -Results [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] - -(78) CometColumnarExchange -Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] -Arguments: hashpartitioning(channel#130, id#131, spark_grouping_id#132, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(79) CometColumnarToRow [codegen id : 21] -Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] - -(80) HashAggregate [codegen id : 21] -Input [9]: [channel#130, id#131, spark_grouping_id#132, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] -Keys [3]: [channel#130, id#131, spark_grouping_id#132] -Functions [3]: [sum(sales#38), sum(returns#39), sum(profit#40)] -Aggregate Attributes [3]: [sum(sales#38)#145, sum(returns#39)#146, sum(profit#40)#147] -Results [5]: [channel#130, id#131, sum(sales#38)#145 AS sales#148, sum(returns#39)#146 AS returns#149, sum(profit#40)#147 AS profit#150] - -(81) TakeOrderedAndProject -Input [5]: [channel#130, id#131, sales#148, returns#149, profit#150] -Arguments: 100, [channel#130 ASC NULLS FIRST, id#131 ASC NULLS FIRST], [channel#130, id#131, sales#148, returns#149, profit#150] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (86) -+- * CometColumnarToRow (85) - +- CometProject (84) - +- CometFilter (83) - +- CometNativeScan parquet spark_catalog.default.date_dim (82) - - -(82) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#151] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] -ReadSchema: struct - -(83) CometFilter -Input [2]: [d_date_sk#22, d_date#151] -Condition : (((isnotnull(d_date#151) AND (d_date#151 >= 2000-08-23)) AND (d_date#151 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) - -(84) CometProject -Input [2]: [d_date_sk#22, d_date#151] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(85) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(86) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 28 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#87 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/simplified.txt deleted file mode 100644 index 9dc4fc19c7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/simplified.txt +++ /dev/null @@ -1,137 +0,0 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (21) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (20) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [s_store_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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_id] #2 - WholeStageCodegen (5) - HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,s_store_id] - BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (1) - Project [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - WholeStageCodegen (2) - Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] - Filter [sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (12) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cp_catalog_page_id] #5 - WholeStageCodegen (11) - HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (7) - Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] - Filter [cs_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (8) - Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] - Filter [cr_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (19) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [web_site_id] #7 - WholeStageCodegen (18) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (13) - Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] - Filter [ws_web_site_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (15) - Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (14) - ColumnarToRow - InputAdapter - Scan 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 - CometColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (17) - CometColumnarToRow - InputAdapter - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan 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/q5.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/explain.txt deleted file mode 100644 index b4a1dedd61..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,452 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (71) -+- CometTakeOrderedAndProject (70) - +- CometHashAggregate (69) - +- CometExchange (68) - +- CometHashAggregate (67) - +- CometExpand (66) - +- CometUnion (65) - :- CometHashAggregate (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometUnion (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - :- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (32) - : : +- CometBroadcastHashJoin (31) - : : :- CometUnion (29) - : : : :- CometProject (25) - : : : : +- CometFilter (24) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) - : : : +- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) - : : +- ReusedExchange (30) - : +- CometBroadcastExchange (36) - : +- CometProject (35) - : +- CometFilter (34) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (55) - : +- CometBroadcastHashJoin (54) - : :- CometUnion (52) - : : :- CometProject (44) - : : : +- CometFilter (43) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : : +- CometProject (51) - : : +- CometBroadcastHashJoin (50) - : : :- CometBroadcastExchange (46) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) - : : +- CometProject (49) - : : +- CometFilter (48) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : +- ReusedExchange (53) - +- CometBroadcastExchange (59) - +- CometProject (58) - +- CometFilter (57) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Arguments: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11], [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#12) - -(6) CometProject -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Arguments: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21], [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] - -(7) CometUnion -Child 0 Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) - -(10) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(11) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(12) CometBroadcastHashJoin -Left output [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [1]: [d_date_sk#22] -Arguments: [date_sk#7], [d_date_sk#22], Inner, BuildRight - -(13) CometProject -Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] -Arguments: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11], [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_store_id#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#24, s_store_id#25] -Condition : isnotnull(s_store_sk#24) - -(16) CometProject -Input [2]: [s_store_sk#24, s_store_id#25] -Arguments: [s_store_sk#24, s_store_id#26], [s_store_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#25, 16, true, false, true) AS s_store_id#26] - -(17) CometBroadcastExchange -Input [2]: [s_store_sk#24, s_store_id#26] -Arguments: [s_store_sk#24, s_store_id#26] - -(18) CometBroadcastHashJoin -Left output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [2]: [s_store_sk#24, s_store_id#26] -Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight - -(19) CometProject -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#26] -Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] - -(20) CometHashAggregate -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] -Keys [1]: [s_store_id#26] -Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] - -(21) CometExchange -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Keys [1]: [s_store_id#26] -Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk)] -ReadSchema: struct - -(24) CometFilter -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : isnotnull(cs_catalog_page_sk#31) - -(25) CometProject -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cr_catalog_page_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Condition : isnotnull(cr_catalog_page_sk#42) - -(28) CometProject -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] - -(29) CometUnion -Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] - -(30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#52] - -(31) CometBroadcastHashJoin -Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [1]: [d_date_sk#52] -Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight - -(32) CometProject -Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] -Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Condition : isnotnull(cp_catalog_page_sk#53) - -(35) CometProject -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#54, 16, true, false, true) AS cp_catalog_page_id#55] - -(36) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] - -(37) CometBroadcastHashJoin -Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight - -(38) CometProject -Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] - -(39) CometHashAggregate -Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] - -(40) CometExchange -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(41) CometHashAggregate -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] -PushedFilters: [IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(43) CometFilter -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_web_site_sk#60) - -(44) CometProject -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] -ReadSchema: struct - -(46) CometBroadcastExchange -Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(48) CometFilter -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) - -(49) CometProject -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] - -(50) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft - -(51) CometProject -Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] - -(52) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] - -(53) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#86] - -(54) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [1]: [d_date_sk#86] -Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight - -(55) CometProject -Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] -Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#87, web_site_id#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(57) CometFilter -Input [2]: [web_site_sk#87, web_site_id#88] -Condition : isnotnull(web_site_sk#87) - -(58) CometProject -Input [2]: [web_site_sk#87, web_site_id#88] -Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#88, 16, true, false, true) AS web_site_id#89] - -(59) CometBroadcastExchange -Input [2]: [web_site_sk#87, web_site_id#89] -Arguments: [web_site_sk#87, web_site_id#89] - -(60) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [2]: [web_site_sk#87, web_site_id#89] -Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight - -(61) CometProject -Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] -Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] - -(62) CometHashAggregate -Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] -Keys [1]: [web_site_id#89] -Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] - -(63) CometExchange -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(64) CometHashAggregate -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Keys [1]: [web_site_id#89] -Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] - -(65) CometUnion -Child 0 Input [5]: [sales#94, returns#95, profit#96, channel#97, id#98] -Child 1 Input [5]: [sales#99, returns#100, profit#101, channel#102, id#103] -Child 2 Input [5]: [sales#104, returns#105, profit#106, channel#107, id#108] - -(66) CometExpand -Input [5]: [sales#94, returns#95, profit#96, channel#97, id#98] -Arguments: [[sales#94, returns#95, profit#96, channel#97, id#98, 0], [sales#94, returns#95, profit#96, channel#97, null, 1], [sales#94, returns#95, profit#96, null, null, 3]], [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] - -(67) CometHashAggregate -Input [6]: [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] -Keys [3]: [channel#109, id#110, spark_grouping_id#111] -Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] - -(68) CometExchange -Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Arguments: hashpartitioning(channel#109, id#110, spark_grouping_id#111, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(69) CometHashAggregate -Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Keys [3]: [channel#109, id#110, spark_grouping_id#111] -Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] - -(70) CometTakeOrderedAndProject -Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#109 ASC NULLS FIRST,id#110 ASC NULLS FIRST], output=[channel#109,id#110,sales#118,returns#119,profit#120]), [channel#109, id#110, sales#118, returns#119, profit#120], 100, 0, [channel#109 ASC NULLS FIRST, id#110 ASC NULLS FIRST], [channel#109, id#110, sales#118, returns#119, profit#120] - -(71) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) - -(74) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(75) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(76) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/simplified.txt deleted file mode 100644 index 6fff89a25c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,86 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id,spark_grouping_id] #1 - CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] - CometUnion [sales,returns,profit,channel,id] - CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),s_store_id] - CometExchange [s_store_id] #2 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - 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 [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,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),cp_catalog_page_id] - CometExchange [cp_catalog_page_id] #6 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - 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 [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,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [cp_catalog_page_sk,cp_catalog_page_id] #7 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),web_site_id] - CometExchange [web_site_id] #8 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - 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 [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,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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,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 [native_iceberg_compat] 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_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 [web_site_sk,web_site_id] #10 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] 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/q5/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt deleted file mode 100644 index b4a1dedd61..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ /dev/null @@ -1,452 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (71) -+- CometTakeOrderedAndProject (70) - +- CometHashAggregate (69) - +- CometExchange (68) - +- CometHashAggregate (67) - +- CometExpand (66) - +- CometUnion (65) - :- CometHashAggregate (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometUnion (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - :- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (32) - : : +- CometBroadcastHashJoin (31) - : : :- CometUnion (29) - : : : :- CometProject (25) - : : : : +- CometFilter (24) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) - : : : +- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) - : : +- ReusedExchange (30) - : +- CometBroadcastExchange (36) - : +- CometProject (35) - : +- CometFilter (34) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (55) - : +- CometBroadcastHashJoin (54) - : :- CometUnion (52) - : : :- CometProject (44) - : : : +- CometFilter (43) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : : +- CometProject (51) - : : +- CometBroadcastHashJoin (50) - : : :- CometBroadcastExchange (46) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) - : : +- CometProject (49) - : : +- CometFilter (48) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : +- ReusedExchange (53) - +- CometBroadcastExchange (59) - +- CometProject (58) - +- CometFilter (57) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Arguments: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11], [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#12) - -(6) CometProject -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Arguments: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21], [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] - -(7) CometUnion -Child 0 Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) - -(10) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(11) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(12) CometBroadcastHashJoin -Left output [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [1]: [d_date_sk#22] -Arguments: [date_sk#7], [d_date_sk#22], Inner, BuildRight - -(13) CometProject -Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] -Arguments: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11], [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_store_id#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#24, s_store_id#25] -Condition : isnotnull(s_store_sk#24) - -(16) CometProject -Input [2]: [s_store_sk#24, s_store_id#25] -Arguments: [s_store_sk#24, s_store_id#26], [s_store_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#25, 16, true, false, true) AS s_store_id#26] - -(17) CometBroadcastExchange -Input [2]: [s_store_sk#24, s_store_id#26] -Arguments: [s_store_sk#24, s_store_id#26] - -(18) CometBroadcastHashJoin -Left output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [2]: [s_store_sk#24, s_store_id#26] -Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight - -(19) CometProject -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#26] -Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] - -(20) CometHashAggregate -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] -Keys [1]: [s_store_id#26] -Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] - -(21) CometExchange -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Keys [1]: [s_store_id#26] -Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk)] -ReadSchema: struct - -(24) CometFilter -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : isnotnull(cs_catalog_page_sk#31) - -(25) CometProject -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cr_catalog_page_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Condition : isnotnull(cr_catalog_page_sk#42) - -(28) CometProject -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] - -(29) CometUnion -Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] - -(30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#52] - -(31) CometBroadcastHashJoin -Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [1]: [d_date_sk#52] -Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight - -(32) CometProject -Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] -Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Condition : isnotnull(cp_catalog_page_sk#53) - -(35) CometProject -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#54, 16, true, false, true) AS cp_catalog_page_id#55] - -(36) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] - -(37) CometBroadcastHashJoin -Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight - -(38) CometProject -Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] - -(39) CometHashAggregate -Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] - -(40) CometExchange -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(41) CometHashAggregate -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] -PushedFilters: [IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(43) CometFilter -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_web_site_sk#60) - -(44) CometProject -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] -ReadSchema: struct - -(46) CometBroadcastExchange -Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(48) CometFilter -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) - -(49) CometProject -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] - -(50) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft - -(51) CometProject -Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] - -(52) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] - -(53) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#86] - -(54) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [1]: [d_date_sk#86] -Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight - -(55) CometProject -Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] -Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#87, web_site_id#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(57) CometFilter -Input [2]: [web_site_sk#87, web_site_id#88] -Condition : isnotnull(web_site_sk#87) - -(58) CometProject -Input [2]: [web_site_sk#87, web_site_id#88] -Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#88, 16, true, false, true) AS web_site_id#89] - -(59) CometBroadcastExchange -Input [2]: [web_site_sk#87, web_site_id#89] -Arguments: [web_site_sk#87, web_site_id#89] - -(60) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [2]: [web_site_sk#87, web_site_id#89] -Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight - -(61) CometProject -Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] -Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] - -(62) CometHashAggregate -Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] -Keys [1]: [web_site_id#89] -Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] - -(63) CometExchange -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(64) CometHashAggregate -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Keys [1]: [web_site_id#89] -Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] - -(65) CometUnion -Child 0 Input [5]: [sales#94, returns#95, profit#96, channel#97, id#98] -Child 1 Input [5]: [sales#99, returns#100, profit#101, channel#102, id#103] -Child 2 Input [5]: [sales#104, returns#105, profit#106, channel#107, id#108] - -(66) CometExpand -Input [5]: [sales#94, returns#95, profit#96, channel#97, id#98] -Arguments: [[sales#94, returns#95, profit#96, channel#97, id#98, 0], [sales#94, returns#95, profit#96, channel#97, null, 1], [sales#94, returns#95, profit#96, null, null, 3]], [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] - -(67) CometHashAggregate -Input [6]: [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] -Keys [3]: [channel#109, id#110, spark_grouping_id#111] -Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] - -(68) CometExchange -Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Arguments: hashpartitioning(channel#109, id#110, spark_grouping_id#111, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(69) CometHashAggregate -Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Keys [3]: [channel#109, id#110, spark_grouping_id#111] -Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] - -(70) CometTakeOrderedAndProject -Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#109 ASC NULLS FIRST,id#110 ASC NULLS FIRST], output=[channel#109,id#110,sales#118,returns#119,profit#120]), [channel#109, id#110, sales#118, returns#119, profit#120], 100, 0, [channel#109 ASC NULLS FIRST, id#110 ASC NULLS FIRST], [channel#109, id#110, sales#118, returns#119, profit#120] - -(71) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) - -(74) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(75) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(76) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/extended.txt deleted file mode 100644 index d128acaf06..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/extended.txt +++ /dev/null @@ -1,90 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - -Comet accelerated 79 out of 86 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 6fff89a25c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt +++ /dev/null @@ -1,86 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id,spark_grouping_id] #1 - CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] - CometUnion [sales,returns,profit,channel,id] - CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),s_store_id] - CometExchange [s_store_id] #2 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - 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 [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,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),cp_catalog_page_id] - CometExchange [cp_catalog_page_id] #6 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - 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 [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,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [cp_catalog_page_sk,cp_catalog_page_id] #7 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),web_site_id] - CometExchange [web_site_id] #8 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - 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 [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,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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,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 [native_iceberg_compat] 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_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 [web_site_sk,web_site_id] #10 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/explain.txt deleted file mode 100644 index de9a1cd3ec..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (30) -+- * HashAggregate (29) - +- * CometColumnarToRow (28) - +- CometColumnarExchange (27) - +- * HashAggregate (26) - +- * Project (25) - +- * BroadcastHashJoin Inner BuildRight (24) - :- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) - : : :- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet spark_catalog.default.store_returns (4) - : : +- BroadcastExchange (14) - : : +- * CometColumnarToRow (13) - : : +- CometProject (12) - : : +- CometFilter (11) - : : +- CometNativeScan parquet spark_catalog.default.store (10) - : +- BroadcastExchange (20) - : +- * CometColumnarToRow (19) - : +- CometFilter (18) - : +- CometNativeScan parquet spark_catalog.default.date_dim (17) - +- ReusedExchange (23) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#5)] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_store_sk#3)) - -(3) CometColumnarToRow [codegen id : 5] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] - -(4) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#9), dynamicpruningexpression(sr_returned_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk), IsNotNull(sr_customer_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] - -(6) Filter [codegen id : 1] -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : ((isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#6)) AND isnotnull(sr_customer_sk#7)) - -(7) BroadcastExchange -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(input[2, int, false], input[0, int, false], input[1, int, false]),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 5] -Left keys [3]: [ss_ticket_number#4, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [sr_ticket_number#8, sr_item_sk#6, sr_customer_sk#7] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 5] -Output [3]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9] -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] - -(10) CometNativeScan parquet spark_catalog.default.store -Output [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(11) CometFilter -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Condition : isnotnull(s_store_sk#11) - -(12) CometProject -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Arguments: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_street_type#16, 15, true, false, true) AS s_street_type#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_suite_number#17, 10, true, false, true) AS s_suite_number#23, s_city#18, s_county#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#20, 2, true, false, true) AS s_state#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#21, 10, true, false, true) AS s_zip#25] - -(13) CometColumnarToRow [codegen id : 2] -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(14) BroadcastExchange -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None - -(16) Project [codegen id : 5] -Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Input [14]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(17) CometNativeScan parquet spark_catalog.default.date_dim -Output [1]: [d_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) CometFilter -Input [1]: [d_date_sk#26] -Condition : isnotnull(d_date_sk#26) - -(19) CometColumnarToRow [codegen id : 3] -Input [1]: [d_date_sk#26] - -(20) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#26] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 5] -Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#26] - -(23) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#27] - -(24) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [sr_returned_date_sk#9] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 5] -Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#27] - -(26) HashAggregate [codegen id : 5] -Input [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Functions [5]: [partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] -Aggregate Attributes [5]: [sum#28, sum#29, sum#30, sum#31, sum#32] -Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#33, sum#34, sum#35, sum#36, sum#37] - -(27) CometColumnarExchange -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#33, sum#34, sum#35, sum#36, sum#37] -Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(28) CometColumnarToRow [codegen id : 6] -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#33, sum#34, sum#35, sum#36, sum#37] - -(29) HashAggregate [codegen id : 6] -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#33, sum#34, sum#35, sum#36, sum#37] -Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Functions [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] -Aggregate Attributes [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#38, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#39, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#40, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#41, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#42] -Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#38 AS 30 days #43, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#39 AS 31 - 60 days #44, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#40 AS 61 - 90 days #45, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#41 AS 91 - 120 days #46, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#42 AS >120 days #47] - -(30) TakeOrderedAndProject -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #43, 31 - 60 days #44, 61 - 90 days #45, 91 - 120 days #46, >120 days #47] -Arguments: 100, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRST, s_street_number#14 ASC NULLS FIRST, s_street_name#15 ASC NULLS FIRST, s_street_type#22 ASC NULLS FIRST, s_suite_number#23 ASC NULLS FIRST, s_city#18 ASC NULLS FIRST, s_county#19 ASC NULLS FIRST, s_state#24 ASC NULLS FIRST, s_zip#25 ASC NULLS FIRST], [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #43, 31 - 60 days #44, 61 - 90 days #45, 91 - 120 days #46, >120 days #47] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometNativeScan parquet spark_catalog.default.date_dim (31) - - -(31) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#48, d_moy#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [d_date_sk#27, d_year#48, d_moy#49] -Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#49)) AND (d_year#48 = 2001)) AND (d_moy#49 = 8)) AND isnotnull(d_date_sk#27)) - -(33) CometProject -Input [3]: [d_date_sk#27, d_year#48, d_moy#49] -Arguments: [d_date_sk#27], [d_date_sk#27] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#27] - -(35) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/simplified.txt deleted file mode 100644 index 186c041b09..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -TakeOrderedAndProject [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,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - WholeStageCodegen (6) - HashAggregate [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] [sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 30) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 60) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 90) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) > 120) THEN 1 ELSE 0 END),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [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] #1 - WholeStageCodegen (5) - HashAggregate [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] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [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] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [sr_ticket_number,sr_item_sk,sr_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_street_type,s_suite_number,s_state,s_zip] [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] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/explain.txt deleted file mode 100644 index ff37c8f5fb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,203 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5)] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#9), dynamicpruningexpression(sr_returned_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk), IsNotNull(sr_customer_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : ((isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#6)) AND isnotnull(sr_customer_sk#7)) - -(5) CometBroadcastExchange -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] - -(6) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [ss_ticket_number#4, ss_item_sk#1, ss_customer_sk#2], [sr_ticket_number#8, sr_item_sk#6, sr_customer_sk#7], Inner, BuildRight - -(7) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9], [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(9) CometFilter -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Condition : isnotnull(s_store_sk#11) - -(10) CometProject -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Arguments: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_street_type#16, 15, true, false, true) AS s_street_type#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_suite_number#17, 10, true, false, true) AS s_suite_number#23, s_city#18, s_county#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#20, 2, true, false, true) AS s_state#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#21, 10, true, false, true) AS s_zip#25] - -(11) CometBroadcastExchange -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Arguments: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(12) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9] -Right output [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Arguments: [ss_store_sk#3], [s_store_sk#11], Inner, BuildRight - -(13) CometProject -Input [14]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [1]: [d_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [1]: [d_date_sk#26] -Condition : isnotnull(d_date_sk#26) - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] - -(17) CometBroadcastHashJoin -Left output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#5], [d_date_sk#26], Inner, BuildRight - -(18) CometProject -Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#26] -Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_moy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Condition : ((((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2001)) AND (d_moy#29 = 8)) AND isnotnull(d_date_sk#27)) - -(21) CometProject -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Arguments: [d_date_sk#27], [d_date_sk#27] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: [d_date_sk#27] - -(23) CometBroadcastHashJoin -Left output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Right output [1]: [d_date_sk#27] -Arguments: [sr_returned_date_sk#9], [d_date_sk#27], Inner, BuildRight - -(24) CometProject -Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#27] -Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(25) CometHashAggregate -Input [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Functions [5]: [partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#30, sum#31, sum#32, sum#33, sum#34] -Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#30, sum#31, sum#32, sum#33, sum#34] -Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Functions [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#12 ASC NULLS FIRST,s_company_id#13 ASC NULLS FIRST,s_street_number#14 ASC NULLS FIRST,s_street_name#15 ASC NULLS FIRST,s_street_type#22 ASC NULLS FIRST,s_suite_number#23 ASC NULLS FIRST,s_city#18 ASC NULLS FIRST,s_county#19 ASC NULLS FIRST,s_state#24 ASC NULLS FIRST,s_zip#25 ASC NULLS FIRST], output=[s_store_name#12,s_company_id#13,s_street_number#14,s_street_name#15,s_street_type#22,s_suite_number#23,s_city#18,s_county#19,s_state#24,s_zip#25,30 days #35,31 - 60 days #36,61 - 90 days #37,91 - 120 days #38,>120 days #39]), [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39], 100, 0, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRST, s_street_number#14 ASC NULLS FIRST, s_street_name#15 ASC NULLS FIRST, s_street_type#22 ASC NULLS FIRST, s_suite_number#23 ASC NULLS FIRST, s_city#18 ASC NULLS FIRST, s_county#19 ASC NULLS FIRST, s_state#24 ASC NULLS FIRST, s_zip#25 ASC NULLS FIRST], [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39] - -(29) CometColumnarToRow [codegen id : 1] -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_moy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Condition : ((((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2001)) AND (d_moy#29 = 8)) AND isnotnull(d_date_sk#27)) - -(32) CometProject -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Arguments: [d_date_sk#27], [d_date_sk#27] - -(33) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#27] - -(34) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/simplified.txt deleted file mode 100644 index 98a44eaf40..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,39 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [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,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [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,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 30) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 60) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 90) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [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] #1 - CometHashAggregate [sr_returned_date_sk,ss_sold_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,sum,sum,sum,sum,sum] - 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,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,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,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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 - CometProject [s_street_type,s_suite_number,s_state,s_zip] [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] - 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 [native_iceberg_compat] 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 [d_date_sk] #5 - CometFilter [d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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/q50/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt deleted file mode 100644 index ff37c8f5fb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt +++ /dev/null @@ -1,203 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5)] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_store_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#9), dynamicpruningexpression(sr_returned_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk), IsNotNull(sr_customer_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : ((isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#6)) AND isnotnull(sr_customer_sk#7)) - -(5) CometBroadcastExchange -Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] - -(6) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [ss_ticket_number#4, ss_item_sk#1, ss_customer_sk#2], [sr_ticket_number#8, sr_item_sk#6, sr_customer_sk#7], Inner, BuildRight - -(7) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9], [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(9) CometFilter -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Condition : isnotnull(s_store_sk#11) - -(10) CometProject -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] -Arguments: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_street_type#16, 15, true, false, true) AS s_street_type#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_suite_number#17, 10, true, false, true) AS s_suite_number#23, s_city#18, s_county#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#20, 2, true, false, true) AS s_state#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#21, 10, true, false, true) AS s_zip#25] - -(11) CometBroadcastExchange -Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Arguments: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(12) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9] -Right output [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Arguments: [ss_store_sk#3], [s_store_sk#11], Inner, BuildRight - -(13) CometProject -Input [14]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [1]: [d_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [1]: [d_date_sk#26] -Condition : isnotnull(d_date_sk#26) - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] - -(17) CometBroadcastHashJoin -Left output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#5], [d_date_sk#26], Inner, BuildRight - -(18) CometProject -Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#26] -Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_moy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Condition : ((((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2001)) AND (d_moy#29 = 8)) AND isnotnull(d_date_sk#27)) - -(21) CometProject -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Arguments: [d_date_sk#27], [d_date_sk#27] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: [d_date_sk#27] - -(23) CometBroadcastHashJoin -Left output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Right output [1]: [d_date_sk#27] -Arguments: [sr_returned_date_sk#9], [d_date_sk#27], Inner, BuildRight - -(24) CometProject -Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, d_date_sk#27] -Arguments: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25], [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] - -(25) CometHashAggregate -Input [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Functions [5]: [partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#30, sum#31, sum#32, sum#33, sum#34] -Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, sum#30, sum#31, sum#32, sum#33, sum#34] -Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25] -Functions [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#12 ASC NULLS FIRST,s_company_id#13 ASC NULLS FIRST,s_street_number#14 ASC NULLS FIRST,s_street_name#15 ASC NULLS FIRST,s_street_type#22 ASC NULLS FIRST,s_suite_number#23 ASC NULLS FIRST,s_city#18 ASC NULLS FIRST,s_county#19 ASC NULLS FIRST,s_state#24 ASC NULLS FIRST,s_zip#25 ASC NULLS FIRST], output=[s_store_name#12,s_company_id#13,s_street_number#14,s_street_name#15,s_street_type#22,s_suite_number#23,s_city#18,s_county#19,s_state#24,s_zip#25,30 days #35,31 - 60 days #36,61 - 90 days #37,91 - 120 days #38,>120 days #39]), [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39], 100, 0, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRST, s_street_number#14 ASC NULLS FIRST, s_street_name#15 ASC NULLS FIRST, s_street_type#22 ASC NULLS FIRST, s_suite_number#23 ASC NULLS FIRST, s_city#18 ASC NULLS FIRST, s_county#19 ASC NULLS FIRST, s_state#24 ASC NULLS FIRST, s_zip#25 ASC NULLS FIRST], [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39] - -(29) CometColumnarToRow [codegen id : 1] -Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#22, s_suite_number#23, s_city#18, s_county#19, s_state#24, s_zip#25, 30 days #35, 31 - 60 days #36, 61 - 90 days #37, 91 - 120 days #38, >120 days #39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#27, d_year#28, d_moy#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Condition : ((((isnotnull(d_year#28) AND isnotnull(d_moy#29)) AND (d_year#28 = 2001)) AND (d_moy#29 = 8)) AND isnotnull(d_date_sk#27)) - -(32) CometProject -Input [3]: [d_date_sk#27, d_year#28, d_moy#29] -Arguments: [d_date_sk#27], [d_date_sk#27] - -(33) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#27] - -(34) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/extended.txt deleted file mode 100644 index 0fbba7d1e8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 98a44eaf40..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/simplified.txt +++ /dev/null @@ -1,39 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [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,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [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,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 30) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 60) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 90) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [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] #1 - CometHashAggregate [sr_returned_date_sk,ss_sold_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,sum,sum,sum,sum,sum] - 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,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,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,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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 - CometProject [s_street_type,s_suite_number,s_state,s_zip] [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] - 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 [native_iceberg_compat] 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 [d_date_sk] #5 - CometFilter [d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/explain.txt deleted file mode 100644 index 9af40029c8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/explain.txt +++ /dev/null @@ -1,264 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (42) -+- * Filter (41) - +- Window (40) - +- * CometColumnarToRow (39) - +- CometSort (38) - +- CometExchange (37) - +- CometProject (36) - +- CometSortMergeJoin (35) - :- CometSort (17) - : +- CometColumnarExchange (16) - : +- * Project (15) - : +- Window (14) - : +- * CometColumnarToRow (13) - : +- CometSort (12) - : +- CometColumnarExchange (11) - : +- * HashAggregate (10) - : +- * CometColumnarToRow (9) - : +- CometColumnarExchange (8) - : +- * HashAggregate (7) - : +- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.web_sales (1) - : +- ReusedExchange (4) - +- CometSort (34) - +- CometColumnarExchange (33) - +- * Project (32) - +- Window (31) - +- * CometColumnarToRow (30) - +- CometSort (29) - +- CometColumnarExchange (28) - +- * HashAggregate (27) - +- * CometColumnarToRow (26) - +- CometColumnarExchange (25) - +- * HashAggregate (24) - +- * Project (23) - +- * BroadcastHashJoin Inner BuildRight (22) - :- * Filter (20) - : +- * ColumnarToRow (19) - : +- Scan parquet spark_catalog.default.store_sales (18) - +- ReusedExchange (21) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] - -(3) Filter [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(4) ReusedExchange [Reuses operator id: 47] -Output [2]: [d_date_sk#5, d_date#6] - -(5) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 2] -Output [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] - -(7) HashAggregate [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum#7] -Results [3]: [ws_item_sk#1, d_date#6, sum#8] - -(8) CometColumnarExchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(9) CometColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, d_date#6, sum#8] - -(10) HashAggregate [codegen id : 3] -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#9] -Results [4]: [ws_item_sk#1 AS item_sk#10, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#9,17,2) AS _w0#11, ws_item_sk#1] - -(11) CometColumnarExchange -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(12) CometSort -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(13) CometColumnarToRow [codegen id : 4] -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] - -(14) Window -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: [sum(_w0#11) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(15) Project [codegen id : 5] -Output [3]: [item_sk#10, d_date#6, cume_sales#12] -Input [5]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1, cume_sales#12] - -(16) CometColumnarExchange -Input [3]: [item_sk#10, d_date#6, cume_sales#12] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(17) CometSort -Input [3]: [item_sk#10, d_date#6, cume_sales#12] -Arguments: [item_sk#10, d_date#6, cume_sales#12], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(18) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_sold_date_sk#15 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 7] -Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] - -(20) Filter [codegen id : 7] -Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -Condition : isnotnull(ss_item_sk#13) - -(21) ReusedExchange [Reuses operator id: 47] -Output [2]: [d_date_sk#16, d_date#17] - -(22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#15] -Right keys [1]: [d_date_sk#16] -Join type: Inner -Join condition: None - -(23) Project [codegen id : 7] -Output [3]: [ss_item_sk#13, ss_sales_price#14, d_date#17] -Input [5]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#16, d_date#17] - -(24) HashAggregate [codegen id : 7] -Input [3]: [ss_item_sk#13, ss_sales_price#14, d_date#17] -Keys [2]: [ss_item_sk#13, d_date#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [ss_item_sk#13, d_date#17, sum#19] - -(25) CometColumnarExchange -Input [3]: [ss_item_sk#13, d_date#17, sum#19] -Arguments: hashpartitioning(ss_item_sk#13, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(26) CometColumnarToRow [codegen id : 8] -Input [3]: [ss_item_sk#13, d_date#17, sum#19] - -(27) HashAggregate [codegen id : 8] -Input [3]: [ss_item_sk#13, d_date#17, sum#19] -Keys [2]: [ss_item_sk#13, d_date#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#14))#20] -Results [4]: [ss_item_sk#13 AS item_sk#21, d_date#17, MakeDecimal(sum(UnscaledValue(ss_sales_price#14))#20,17,2) AS _w0#22, ss_item_sk#13] - -(28) CometColumnarExchange -Input [4]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13] -Arguments: hashpartitioning(ss_item_sk#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(29) CometSort -Input [4]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13] -Arguments: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13], [ss_item_sk#13 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] - -(30) CometColumnarToRow [codegen id : 9] -Input [4]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13] - -(31) Window -Input [4]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13] -Arguments: [sum(_w0#22) windowspecdefinition(ss_item_sk#13, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#23], [ss_item_sk#13], [d_date#17 ASC NULLS FIRST] - -(32) Project [codegen id : 10] -Output [3]: [item_sk#21, d_date#17, cume_sales#23] -Input [5]: [item_sk#21, d_date#17, _w0#22, ss_item_sk#13, cume_sales#23] - -(33) CometColumnarExchange -Input [3]: [item_sk#21, d_date#17, cume_sales#23] -Arguments: hashpartitioning(item_sk#21, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(34) CometSort -Input [3]: [item_sk#21, d_date#17, cume_sales#23] -Arguments: [item_sk#21, d_date#17, cume_sales#23], [item_sk#21 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] - -(35) CometSortMergeJoin -Left output [3]: [item_sk#10, d_date#6, cume_sales#12] -Right output [3]: [item_sk#21, d_date#17, cume_sales#23] -Arguments: [item_sk#10, d_date#6], [item_sk#21, d_date#17], FullOuter - -(36) CometProject -Input [6]: [item_sk#10, d_date#6, cume_sales#12, item_sk#21, d_date#17, cume_sales#23] -Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#21 END AS item_sk#24, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#17 END AS d_date#25, cume_sales#12 AS web_sales#26, cume_sales#23 AS store_sales#27] - -(37) CometExchange -Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] -Arguments: hashpartitioning(item_sk#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(38) CometSort -Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] -Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27], [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST] - -(39) CometColumnarToRow [codegen id : 11] -Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] - -(40) Window -Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] -Arguments: [max(web_sales#26) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#28, max(store_sales#27) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#29], [item_sk#24], [d_date#25 ASC NULLS FIRST] - -(41) Filter [codegen id : 12] -Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] -Condition : ((isnotnull(web_cumulative#28) AND isnotnull(store_cumulative#29)) AND (web_cumulative#28 > store_cumulative#29)) - -(42) TakeOrderedAndProject -Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] -Arguments: 100, [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST], [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (47) -+- * CometColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometNativeScan parquet spark_catalog.default.date_dim (43) - - -(43) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(44) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#30] -Condition : (((isnotnull(d_month_seq#30) AND (d_month_seq#30 >= 1200)) AND (d_month_seq#30 <= 1211)) AND isnotnull(d_date_sk#5)) - -(45) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#30] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(46) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] - -(47) BroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/simplified.txt deleted file mode 100644 index 51d3f11ae6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (12) - Filter [web_cumulative,store_cumulative] - InputAdapter - Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (5) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ws_item_sk] - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk,d_date] #4 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #6 - WholeStageCodegen (10) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ss_item_sk] - CometColumnarExchange [ss_item_sk] #7 - WholeStageCodegen (8) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk,d_date] #8 - WholeStageCodegen (7) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/explain.txt deleted file mode 100644 index 683cc25d24..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,257 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (41) -+- * Filter (40) - +- Window (39) - +- * CometColumnarToRow (38) - +- CometSort (37) - +- CometExchange (36) - +- CometProject (35) - +- CometSortMergeJoin (34) - :- CometSort (18) - : +- CometColumnarExchange (17) - : +- * Project (16) - : +- Window (15) - : +- * CometColumnarToRow (14) - : +- CometSort (13) - : +- CometExchange (12) - : +- CometHashAggregate (11) - : +- CometExchange (10) - : +- CometHashAggregate (9) - : +- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- CometSort (33) - +- CometColumnarExchange (32) - +- * Project (31) - +- Window (30) - +- * CometColumnarToRow (29) - +- CometSort (28) - +- CometExchange (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometBroadcastHashJoin (22) - :- CometFilter (20) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (19) - +- ReusedExchange (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: [d_date_sk#5, d_date#6] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Right output [2]: [d_date_sk#5, d_date#6] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] -Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] - -(9) CometHashAggregate -Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] - -(10) CometExchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] - -(12) CometExchange -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(13) CometSort -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(14) CometColumnarToRow [codegen id : 1] -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] - -(15) Window -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: [sum(_w0#10) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(16) Project [codegen id : 2] -Output [3]: [item_sk#9, d_date#6, cume_sales#11] -Input [5]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1, cume_sales#11] - -(17) CometColumnarExchange -Input [3]: [item_sk#9, d_date#6, cume_sales#11] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(18) CometSort -Input [3]: [item_sk#9, d_date#6, cume_sales#11] -Arguments: [item_sk#9, d_date#6, cume_sales#11], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#14), dynamicpruningexpression(ss_sold_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] -Condition : isnotnull(ss_item_sk#12) - -(21) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#16, d_date#17] - -(22) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] -Right output [2]: [d_date_sk#16, d_date#17] -Arguments: [ss_sold_date_sk#14], [d_date_sk#16], Inner, BuildRight - -(23) CometProject -Input [5]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14, d_date_sk#16, d_date#17] -Arguments: [ss_item_sk#12, ss_sales_price#13, d_date#17], [ss_item_sk#12, ss_sales_price#13, d_date#17] - -(24) CometHashAggregate -Input [3]: [ss_item_sk#12, ss_sales_price#13, d_date#17] -Keys [2]: [ss_item_sk#12, d_date#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#13))] - -(25) CometExchange -Input [3]: [ss_item_sk#12, d_date#17, sum#18] -Arguments: hashpartitioning(ss_item_sk#12, d_date#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(26) CometHashAggregate -Input [3]: [ss_item_sk#12, d_date#17, sum#18] -Keys [2]: [ss_item_sk#12, d_date#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#13))] - -(27) CometExchange -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -Arguments: hashpartitioning(ss_item_sk#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(28) CometSort -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -Arguments: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12], [ss_item_sk#12 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] - -(29) CometColumnarToRow [codegen id : 3] -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] - -(30) Window -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -Arguments: [sum(_w0#20) windowspecdefinition(ss_item_sk#12, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#21], [ss_item_sk#12], [d_date#17 ASC NULLS FIRST] - -(31) Project [codegen id : 4] -Output [3]: [item_sk#19, d_date#17, cume_sales#21] -Input [5]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12, cume_sales#21] - -(32) CometColumnarExchange -Input [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: hashpartitioning(item_sk#19, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(33) CometSort -Input [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: [item_sk#19, d_date#17, cume_sales#21], [item_sk#19 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] - -(34) CometSortMergeJoin -Left output [3]: [item_sk#9, d_date#6, cume_sales#11] -Right output [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: [item_sk#9, d_date#6], [item_sk#19, d_date#17], FullOuter - -(35) CometProject -Input [6]: [item_sk#9, d_date#6, cume_sales#11, item_sk#19, d_date#17, cume_sales#21] -Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#19 END AS item_sk#22, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#17 END AS d_date#23, cume_sales#11 AS web_sales#24, cume_sales#21 AS store_sales#25] - -(36) CometExchange -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: hashpartitioning(item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(37) CometSort -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST] - -(38) CometColumnarToRow [codegen id : 5] -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] - -(39) Window -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: [max(web_sales#24) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#26, max(store_sales#25) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#27], [item_sk#22], [d_date#23 ASC NULLS FIRST] - -(40) Filter [codegen id : 6] -Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] -Condition : ((isnotnull(web_cumulative#26) AND isnotnull(store_cumulative#27)) AND (web_cumulative#26 > store_cumulative#27)) - -(41) TakeOrderedAndProject -Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] -Arguments: 100, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) - - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) - -(44) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(45) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] - -(46) BroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/simplified.txt deleted file mode 100644 index a5af2e114b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,62 +0,0 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (6) - Filter [web_cumulative,store_cumulative] - InputAdapter - Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (2) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ws_item_sk] - CometExchange [ws_item_sk] #3 - CometHashAggregate [sum] [item_sk,d_date,_w0,ws_item_sk,sum(UnscaledValue(ws_sales_price))] - CometExchange [ws_item_sk,d_date] #4 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #6 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #7 - WholeStageCodegen (4) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ss_item_sk] - CometExchange [ss_item_sk] #8 - CometHashAggregate [sum] [item_sk,d_date,_w0,ss_item_sk,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_item_sk,d_date] #9 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - 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 [native_iceberg_compat] 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/q51/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt deleted file mode 100644 index 683cc25d24..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt +++ /dev/null @@ -1,257 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (41) -+- * Filter (40) - +- Window (39) - +- * CometColumnarToRow (38) - +- CometSort (37) - +- CometExchange (36) - +- CometProject (35) - +- CometSortMergeJoin (34) - :- CometSort (18) - : +- CometColumnarExchange (17) - : +- * Project (16) - : +- Window (15) - : +- * CometColumnarToRow (14) - : +- CometSort (13) - : +- CometExchange (12) - : +- CometHashAggregate (11) - : +- CometExchange (10) - : +- CometHashAggregate (9) - : +- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- CometSort (33) - +- CometColumnarExchange (32) - +- * Project (31) - +- Window (30) - +- * CometColumnarToRow (29) - +- CometSort (28) - +- CometExchange (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometBroadcastHashJoin (22) - :- CometFilter (20) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (19) - +- ReusedExchange (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: [d_date_sk#5, d_date#6] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Right output [2]: [d_date_sk#5, d_date#6] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] -Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] - -(9) CometHashAggregate -Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] - -(10) CometExchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] - -(12) CometExchange -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(13) CometSort -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(14) CometColumnarToRow [codegen id : 1] -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] - -(15) Window -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: [sum(_w0#10) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(16) Project [codegen id : 2] -Output [3]: [item_sk#9, d_date#6, cume_sales#11] -Input [5]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1, cume_sales#11] - -(17) CometColumnarExchange -Input [3]: [item_sk#9, d_date#6, cume_sales#11] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(18) CometSort -Input [3]: [item_sk#9, d_date#6, cume_sales#11] -Arguments: [item_sk#9, d_date#6, cume_sales#11], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#14), dynamicpruningexpression(ss_sold_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] -Condition : isnotnull(ss_item_sk#12) - -(21) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#16, d_date#17] - -(22) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] -Right output [2]: [d_date_sk#16, d_date#17] -Arguments: [ss_sold_date_sk#14], [d_date_sk#16], Inner, BuildRight - -(23) CometProject -Input [5]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14, d_date_sk#16, d_date#17] -Arguments: [ss_item_sk#12, ss_sales_price#13, d_date#17], [ss_item_sk#12, ss_sales_price#13, d_date#17] - -(24) CometHashAggregate -Input [3]: [ss_item_sk#12, ss_sales_price#13, d_date#17] -Keys [2]: [ss_item_sk#12, d_date#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#13))] - -(25) CometExchange -Input [3]: [ss_item_sk#12, d_date#17, sum#18] -Arguments: hashpartitioning(ss_item_sk#12, d_date#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(26) CometHashAggregate -Input [3]: [ss_item_sk#12, d_date#17, sum#18] -Keys [2]: [ss_item_sk#12, d_date#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#13))] - -(27) CometExchange -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -Arguments: hashpartitioning(ss_item_sk#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(28) CometSort -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -Arguments: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12], [ss_item_sk#12 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] - -(29) CometColumnarToRow [codegen id : 3] -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] - -(30) Window -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -Arguments: [sum(_w0#20) windowspecdefinition(ss_item_sk#12, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#21], [ss_item_sk#12], [d_date#17 ASC NULLS FIRST] - -(31) Project [codegen id : 4] -Output [3]: [item_sk#19, d_date#17, cume_sales#21] -Input [5]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12, cume_sales#21] - -(32) CometColumnarExchange -Input [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: hashpartitioning(item_sk#19, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(33) CometSort -Input [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: [item_sk#19, d_date#17, cume_sales#21], [item_sk#19 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] - -(34) CometSortMergeJoin -Left output [3]: [item_sk#9, d_date#6, cume_sales#11] -Right output [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: [item_sk#9, d_date#6], [item_sk#19, d_date#17], FullOuter - -(35) CometProject -Input [6]: [item_sk#9, d_date#6, cume_sales#11, item_sk#19, d_date#17, cume_sales#21] -Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#19 END AS item_sk#22, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#17 END AS d_date#23, cume_sales#11 AS web_sales#24, cume_sales#21 AS store_sales#25] - -(36) CometExchange -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: hashpartitioning(item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(37) CometSort -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST] - -(38) CometColumnarToRow [codegen id : 5] -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] - -(39) Window -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: [max(web_sales#24) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#26, max(store_sales#25) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#27], [item_sk#22], [d_date#23 ASC NULLS FIRST] - -(40) Filter [codegen id : 6] -Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] -Condition : ((isnotnull(web_cumulative#26) AND isnotnull(store_cumulative#27)) AND (web_cumulative#26 > store_cumulative#27)) - -(41) TakeOrderedAndProject -Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] -Arguments: 100, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) - - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) - -(44) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(45) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] - -(46) BroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/extended.txt deleted file mode 100644 index 66c5717cc7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/extended.txt +++ /dev/null @@ -1,53 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometColumnarExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 37 out of 47 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index a5af2e114b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt +++ /dev/null @@ -1,62 +0,0 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (6) - Filter [web_cumulative,store_cumulative] - InputAdapter - Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (2) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ws_item_sk] - CometExchange [ws_item_sk] #3 - CometHashAggregate [sum] [item_sk,d_date,_w0,ws_item_sk,sum(UnscaledValue(ws_sales_price))] - CometExchange [ws_item_sk,d_date] #4 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #6 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #7 - WholeStageCodegen (4) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ss_item_sk] - CometExchange [ss_item_sk] #8 - CometHashAggregate [sum] [item_sk,d_date,_w0,ss_item_sk,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_item_sk,d_date] #9 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - 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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_datafusion/explain.txt deleted file mode 100644 index 4af1b4f562..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_datafusion/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometNativeScan parquet spark_catalog.default.item (9) - - -(1) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometNativeScan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,ext_price#15]), [d_year#2, brand_id#13, brand#14, ext_price#15], 100, 0, [d_year#2 ASC NULLS FIRST, ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, ext_price#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_datafusion/simplified.txt deleted file mode 100644 index cee223d027..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_datafusion/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,ext_price] - CometHashAggregate [sum] [d_year,brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometNativeScan 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/q52.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_iceberg_compat/explain.txt deleted file mode 100644 index a05651a422..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,ext_price#15]), [d_year#2, brand_id#13, brand#14, ext_price#15], 100, 0, [d_year#2 ASC NULLS FIRST, ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, ext_price#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_iceberg_compat/simplified.txt deleted file mode 100644 index 218062c3ef..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,ext_price] - CometHashAggregate [sum] [d_year,brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan [native_iceberg_compat] 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/q52/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt deleted file mode 100644 index a05651a422..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [2]: [d_date_sk#1, d_year#2] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5], [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] -Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,ext_price#15]), [d_year#2, brand_id#13, brand#14, ext_price#15], 100, 0, [d_year#2 ASC NULLS FIRST, ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, ext_price#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/extended.txt deleted file mode 100644 index a292badf5d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 218062c3ef..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [d_year,brand_id,brand,ext_price] - CometHashAggregate [sum] [d_year,brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [d_year,i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] - CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] - 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,d_year,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk,d_year] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/explain.txt deleted file mode 100644 index f8c8dd3bf2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/explain.txt +++ /dev/null @@ -1,202 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * CometColumnarToRow (26) - +- CometSort (25) - +- CometColumnarExchange (24) - +- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometColumnarExchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * CometColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (8) - : : +- * Filter (7) - : : +- * ColumnarToRow (6) - : : +- Scan parquet spark_catalog.default.store_sales (5) - : +- ReusedExchange (11) - +- BroadcastExchange (17) - +- * CometColumnarToRow (16) - +- CometFilter (15) - +- CometNativeScan parquet spark_catalog.default.store (14) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Books ,Children ,Electronics ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (personal ,portable ,reference ,self-help )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Women ,Music ,Men ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (accessories ,classical ,fragrances ,pants )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_manufact_id#5], [i_item_sk#1, i_manufact_id#5] - -(4) CometColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#1, i_manufact_id#5] - -(5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(7) Filter [codegen id : 1] -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) - -(8) BroadcastExchange -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#10] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Input [6]: [i_item_sk#1, i_manufact_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(11) ReusedExchange [Reuses operator id: 35] -Output [2]: [d_date_sk#15, d_qoy#16] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#13] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#16] -Input [6]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_qoy#16] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [1]: [s_store_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [1]: [s_store_sk#17] -Condition : isnotnull(s_store_sk#17) - -(16) CometColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#17] - -(17) BroadcastExchange -Input [1]: [s_store_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#11] -Right keys [1]: [s_store_sk#17] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#16] -Input [5]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#16, s_store_sk#17] - -(20) HashAggregate [codegen id : 4] -Input [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#16] -Keys [2]: [i_manufact_id#5, d_qoy#16] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [i_manufact_id#5, d_qoy#16, sum#19] - -(21) CometColumnarExchange -Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] -Arguments: hashpartitioning(i_manufact_id#5, d_qoy#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] - -(23) HashAggregate [codegen id : 5] -Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] -Keys [2]: [i_manufact_id#5, d_qoy#16] -Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#20] -Results [3]: [i_manufact_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS _w0#22] - -(24) CometColumnarExchange -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: [i_manufact_id#5, sum_sales#21, _w0#22], [i_manufact_id#5 ASC NULLS FIRST] - -(26) CometColumnarToRow [codegen id : 6] -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] - -(27) Window -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#23], [i_manufact_id#5] - -(28) Filter [codegen id : 7] -Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] -Condition : CASE WHEN (avg_quarterly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_quarterly_sales#23)) / avg_quarterly_sales#23) > 0.1000000000000000) ELSE false END - -(29) Project [codegen id : 7] -Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] -Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] - -(30) TakeOrderedAndProject -Input [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] -Arguments: 100, [avg_quarterly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometNativeScan parquet spark_catalog.default.date_dim (31) - - -(31) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] -Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(33) CometProject -Input [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] -Arguments: [d_date_sk#15, d_qoy#16], [d_date_sk#15, d_qoy#16] - -(34) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_qoy#16] - -(35) BroadcastExchange -Input [2]: [d_date_sk#15, d_qoy#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/simplified.txt deleted file mode 100644 index 63c3e1a17a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/simplified.txt +++ /dev/null @@ -1,53 +0,0 @@ -TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (7) - Project [i_manufact_id,sum_sales,avg_quarterly_sales] - Filter [avg_quarterly_sales,sum_sales] - InputAdapter - Window [_w0,i_manufact_id] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_manufact_id,sum_sales,_w0] - CometColumnarExchange [i_manufact_id] #1 - WholeStageCodegen (5) - HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id,d_qoy] #2 - WholeStageCodegen (4) - HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] - Project [i_manufact_id,ss_sales_price,d_qoy] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_sk,i_manufact_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_qoy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/explain.txt deleted file mode 100644 index 2fc38bf838..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,198 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) - +- CometSort (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Books ,Children ,Electronics ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (personal ,portable ,reference ,self-help )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Women ,Music ,Men ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (accessories ,classical ,fragrances ,pants )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_manufact_id#5], [i_item_sk#1, i_manufact_id#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(7) CometBroadcastHashJoin -Left output [2]: [i_item_sk#1, i_manufact_id#5] -Right output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_item_sk#1], [ss_item_sk#10], Inner, BuildRight - -(8) CometProject -Input [6]: [i_item_sk#1, i_manufact_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13], [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(11) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] - -(12) CometBroadcastExchange -Input [2]: [d_date_sk#15, d_qoy#17] -Arguments: [d_date_sk#15, d_qoy#17] - -(13) CometBroadcastHashJoin -Left output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_qoy#17] -Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(14) CometProject -Input [6]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_qoy#17] -Arguments: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17], [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [s_store_sk#18] -Condition : isnotnull(s_store_sk#18) - -(17) CometBroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: [s_store_sk#18] - -(18) CometBroadcastHashJoin -Left output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17] -Right output [1]: [s_store_sk#18] -Arguments: [ss_store_sk#11], [s_store_sk#18], Inner, BuildRight - -(19) CometProject -Input [5]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17, s_store_sk#18] -Arguments: [i_manufact_id#5, ss_sales_price#12, d_qoy#17], [i_manufact_id#5, ss_sales_price#12, d_qoy#17] - -(20) CometHashAggregate -Input [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#17] -Keys [2]: [i_manufact_id#5, d_qoy#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] - -(21) CometExchange -Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] -Arguments: hashpartitioning(i_manufact_id#5, d_qoy#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] -Keys [2]: [i_manufact_id#5, d_qoy#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] - -(23) CometExchange -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: [i_manufact_id#5, sum_sales#20, _w0#21], [i_manufact_id#5 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 1] -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] - -(26) Window -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#22], [i_manufact_id#5] - -(27) Filter [codegen id : 2] -Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] -Condition : CASE WHEN (avg_quarterly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_quarterly_sales#22)) / avg_quarterly_sales#22) > 0.1000000000000000) ELSE false END - -(28) Project [codegen id : 2] -Output [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] -Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] - -(29) TakeOrderedAndProject -Input [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] -Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(32) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_qoy#17] - -(34) BroadcastExchange -Input [2]: [d_date_sk#15, d_qoy#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/simplified.txt deleted file mode 100644 index 0c123eb728..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (2) - Project [i_manufact_id,sum_sales,avg_quarterly_sales] - Filter [avg_quarterly_sales,sum_sales] - InputAdapter - Window [_w0,i_manufact_id] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_manufact_id,sum_sales,_w0] - CometExchange [i_manufact_id] #1 - CometHashAggregate [d_qoy,sum] [i_manufact_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_manufact_id,d_qoy] #2 - CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] - CometProject [i_manufact_id,ss_sales_price,d_qoy] - 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 [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,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_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [d_date_sk,d_qoy] #5 - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt deleted file mode 100644 index 2fc38bf838..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt +++ /dev/null @@ -1,198 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) - +- CometSort (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Books ,Children ,Electronics ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (personal ,portable ,reference ,self-help )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Women ,Music ,Men ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (accessories ,classical ,fragrances ,pants )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_manufact_id#5], [i_item_sk#1, i_manufact_id#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(7) CometBroadcastHashJoin -Left output [2]: [i_item_sk#1, i_manufact_id#5] -Right output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_item_sk#1], [ss_item_sk#10], Inner, BuildRight - -(8) CometProject -Input [6]: [i_item_sk#1, i_manufact_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13], [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(11) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] - -(12) CometBroadcastExchange -Input [2]: [d_date_sk#15, d_qoy#17] -Arguments: [d_date_sk#15, d_qoy#17] - -(13) CometBroadcastHashJoin -Left output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_qoy#17] -Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(14) CometProject -Input [6]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_qoy#17] -Arguments: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17], [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [s_store_sk#18] -Condition : isnotnull(s_store_sk#18) - -(17) CometBroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: [s_store_sk#18] - -(18) CometBroadcastHashJoin -Left output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17] -Right output [1]: [s_store_sk#18] -Arguments: [ss_store_sk#11], [s_store_sk#18], Inner, BuildRight - -(19) CometProject -Input [5]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#17, s_store_sk#18] -Arguments: [i_manufact_id#5, ss_sales_price#12, d_qoy#17], [i_manufact_id#5, ss_sales_price#12, d_qoy#17] - -(20) CometHashAggregate -Input [3]: [i_manufact_id#5, ss_sales_price#12, d_qoy#17] -Keys [2]: [i_manufact_id#5, d_qoy#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] - -(21) CometExchange -Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] -Arguments: hashpartitioning(i_manufact_id#5, d_qoy#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] -Keys [2]: [i_manufact_id#5, d_qoy#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] - -(23) CometExchange -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: [i_manufact_id#5, sum_sales#20, _w0#21], [i_manufact_id#5 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 1] -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] - -(26) Window -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#22], [i_manufact_id#5] - -(27) Filter [codegen id : 2] -Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] -Condition : CASE WHEN (avg_quarterly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_quarterly_sales#22)) / avg_quarterly_sales#22) > 0.1000000000000000) ELSE false END - -(28) Project [codegen id : 2] -Output [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] -Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] - -(29) TakeOrderedAndProject -Input [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] -Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(32) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] -Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_qoy#17] - -(34) BroadcastExchange -Input [2]: [d_date_sk#15, d_qoy#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/extended.txt deleted file mode 100644 index 825b1ed81c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 0c123eb728..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (2) - Project [i_manufact_id,sum_sales,avg_quarterly_sales] - Filter [avg_quarterly_sales,sum_sales] - InputAdapter - Window [_w0,i_manufact_id] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_manufact_id,sum_sales,_w0] - CometExchange [i_manufact_id] #1 - CometHashAggregate [d_qoy,sum] [i_manufact_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_manufact_id,d_qoy] #2 - CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] - CometProject [i_manufact_id,ss_sales_price,d_qoy] - 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 [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,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_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [d_date_sk,d_qoy] #5 - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/explain.txt deleted file mode 100644 index 43120fc051..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/explain.txt +++ /dev/null @@ -1,497 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (61) -+- * HashAggregate (60) - +- * CometColumnarToRow (59) - +- CometColumnarExchange (58) - +- * HashAggregate (57) - +- * HashAggregate (56) - +- * CometColumnarToRow (55) - +- CometColumnarExchange (54) - +- * HashAggregate (53) - +- * Project (52) - +- * BroadcastHashJoin Inner BuildRight (51) - :- * Project (49) - : +- * BroadcastHashJoin Inner BuildRight (48) - : :- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * CometColumnarToRow (29) - : : : : +- CometHashAggregate (28) - : : : : +- CometColumnarExchange (27) - : : : : +- * HashAggregate (26) - : : : : +- * Project (25) - : : : : +- * BroadcastHashJoin Inner BuildRight (24) - : : : : :- * Project (19) - : : : : : +- * BroadcastHashJoin Inner BuildRight (18) - : : : : : :- * Project (16) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : : : : :- Union (9) - : : : : : : : :- * Project (4) - : : : : : : : : +- * Filter (3) - : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : +- * Project (8) - : : : : : : : +- * Filter (7) - : : : : : : : +- * ColumnarToRow (6) - : : : : : : : +- Scan parquet spark_catalog.default.web_sales (5) - : : : : : : +- BroadcastExchange (14) - : : : : : : +- * CometColumnarToRow (13) - : : : : : : +- CometProject (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) - : : : : : +- ReusedExchange (17) - : : : : +- BroadcastExchange (23) - : : : : +- * CometColumnarToRow (22) - : : : : +- CometFilter (21) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (20) - : : : +- BroadcastExchange (33) - : : : +- * Filter (32) - : : : +- * ColumnarToRow (31) - : : : +- Scan parquet spark_catalog.default.store_sales (30) - : : +- BroadcastExchange (40) - : : +- * CometColumnarToRow (39) - : : +- CometProject (38) - : : +- CometFilter (37) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (36) - : +- BroadcastExchange (47) - : +- * CometColumnarToRow (46) - : +- CometProject (45) - : +- CometFilter (44) - : +- CometNativeScan parquet spark_catalog.default.store (43) - +- ReusedExchange (50) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] - -(3) Filter [codegen id : 1] -Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_bill_customer_sk#1)) - -(4) Project [codegen id : 1] -Output [3]: [cs_sold_date_sk#3 AS sold_date_sk#5, cs_bill_customer_sk#1 AS customer_sk#6, cs_item_sk#2 AS item_sk#7] -Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] - -(5) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#10), dynamicpruningexpression(ws_sold_date_sk#10 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 2] -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] - -(7) Filter [codegen id : 2] -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Condition : (isnotnull(ws_item_sk#8) AND isnotnull(ws_bill_customer_sk#9)) - -(8) Project [codegen id : 2] -Output [3]: [ws_sold_date_sk#10 AS sold_date_sk#11, ws_bill_customer_sk#9 AS customer_sk#12, ws_item_sk#8 AS item_sk#13] -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] - -(9) Union - -(10) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_class#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true) = Women ) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#15, 50, true, false, true) = maternity )) AND isnotnull(i_item_sk#14)) - -(12) CometProject -Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Arguments: [i_item_sk#14], [i_item_sk#14] - -(13) CometColumnarToRow [codegen id : 3] -Input [1]: [i_item_sk#14] - -(14) BroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(15) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [item_sk#7] -Right keys [1]: [i_item_sk#14] -Join type: Inner -Join condition: None - -(16) Project [codegen id : 6] -Output [2]: [sold_date_sk#5, customer_sk#6] -Input [4]: [sold_date_sk#5, customer_sk#6, item_sk#7, i_item_sk#14] - -(17) ReusedExchange [Reuses operator id: 66] -Output [1]: [d_date_sk#17] - -(18) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sold_date_sk#5] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 6] -Output [1]: [customer_sk#6] -Input [3]: [sold_date_sk#5, customer_sk#6, d_date_sk#17] - -(20) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#18, c_current_addr_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Condition : (isnotnull(c_customer_sk#18) AND isnotnull(c_current_addr_sk#19)) - -(22) CometColumnarToRow [codegen id : 5] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] - -(23) BroadcastExchange -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [customer_sk#6] -Right keys [1]: [c_customer_sk#18] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [2]: [c_customer_sk#18, c_current_addr_sk#19] -Input [3]: [customer_sk#6, c_customer_sk#18, c_current_addr_sk#19] - -(26) HashAggregate [codegen id : 6] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Keys [2]: [c_customer_sk#18, c_current_addr_sk#19] -Functions: [] -Aggregate Attributes: [] -Results [2]: [c_customer_sk#18, c_current_addr_sk#19] - -(27) CometColumnarExchange -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Arguments: hashpartitioning(c_customer_sk#18, c_current_addr_sk#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometHashAggregate -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] -Keys [2]: [c_customer_sk#18, c_current_addr_sk#19] -Functions: [] - -(29) CometColumnarToRow [codegen id : 11] -Input [2]: [c_customer_sk#18, c_current_addr_sk#19] - -(30) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 7] -Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] - -(32) Filter [codegen id : 7] -Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_customer_sk#20) - -(33) BroadcastExchange -Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - -(34) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_customer_sk#18] -Right keys [1]: [ss_customer_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 11] -Output [4]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22] -Input [5]: [c_customer_sk#18, c_current_addr_sk#19, ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] - -(36) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#24, ca_county#25, ca_state#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] -Condition : ((isnotnull(ca_address_sk#24) AND isnotnull(ca_county#25)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#26, 2, true, false, true))) - -(38) CometProject -Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] -Arguments: [ca_address_sk#24, ca_county#25, ca_state#27], [ca_address_sk#24, ca_county#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#26, 2, true, false, true) AS ca_state#27] - -(39) CometColumnarToRow [codegen id : 8] -Input [3]: [ca_address_sk#24, ca_county#25, ca_state#27] - -(40) BroadcastExchange -Input [3]: [ca_address_sk#24, ca_county#25, ca_state#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(41) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#19] -Right keys [1]: [ca_address_sk#24] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 11] -Output [5]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#27] -Input [7]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_address_sk#24, ca_county#25, ca_state#27] - -(43) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_county#28, s_state#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county)] -ReadSchema: struct - -(44) CometFilter -Input [2]: [s_county#28, s_state#29] -Condition : (isnotnull(s_county#28) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#29, 2, true, false, true))) - -(45) CometProject -Input [2]: [s_county#28, s_state#29] -Arguments: [s_county#28, s_state#30], [s_county#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#29, 2, true, false, true) AS s_state#30] - -(46) CometColumnarToRow [codegen id : 9] -Input [2]: [s_county#28, s_state#30] - -(47) BroadcastExchange -Input [2]: [s_county#28, s_state#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true]),false), [plan_id=6] - -(48) BroadcastHashJoin [codegen id : 11] -Left keys [2]: [ca_county#25, ca_state#27] -Right keys [2]: [s_county#28, s_state#30] -Join type: Inner -Join condition: None - -(49) Project [codegen id : 11] -Output [3]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22] -Input [7]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#27, s_county#28, s_state#30] - -(50) ReusedExchange [Reuses operator id: 71] -Output [1]: [d_date_sk#31] - -(51) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#22] -Right keys [1]: [d_date_sk#31] -Join type: Inner -Join condition: None - -(52) Project [codegen id : 11] -Output [2]: [c_customer_sk#18, ss_ext_sales_price#21] -Input [4]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, d_date_sk#31] - -(53) HashAggregate [codegen id : 11] -Input [2]: [c_customer_sk#18, ss_ext_sales_price#21] -Keys [1]: [c_customer_sk#18] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#21))] -Aggregate Attributes [1]: [sum#32] -Results [2]: [c_customer_sk#18, sum#33] - -(54) CometColumnarExchange -Input [2]: [c_customer_sk#18, sum#33] -Arguments: hashpartitioning(c_customer_sk#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(55) CometColumnarToRow [codegen id : 12] -Input [2]: [c_customer_sk#18, sum#33] - -(56) HashAggregate [codegen id : 12] -Input [2]: [c_customer_sk#18, sum#33] -Keys [1]: [c_customer_sk#18] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#21))#34] -Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#21))#34,17,2) / 50) as int) AS segment#35] - -(57) HashAggregate [codegen id : 12] -Input [1]: [segment#35] -Keys [1]: [segment#35] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#36] -Results [2]: [segment#35, count#37] - -(58) CometColumnarExchange -Input [2]: [segment#35, count#37] -Arguments: hashpartitioning(segment#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(59) CometColumnarToRow [codegen id : 13] -Input [2]: [segment#35, count#37] - -(60) HashAggregate [codegen id : 13] -Input [2]: [segment#35, count#37] -Keys [1]: [segment#35] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#38] -Results [3]: [segment#35, count(1)#38 AS num_customers#39, (segment#35 * 50) AS segment_base#40] - -(61) TakeOrderedAndProject -Input [3]: [segment#35, num_customers#39, segment_base#40] -Arguments: 100, [segment#35 ASC NULLS FIRST, num_customers#39 ASC NULLS FIRST], [segment#35, num_customers#39, segment_base#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (66) -+- * CometColumnarToRow (65) - +- CometProject (64) - +- CometFilter (63) - +- CometNativeScan parquet spark_catalog.default.date_dim (62) - - -(62) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#41, d_moy#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(63) CometFilter -Input [3]: [d_date_sk#17, d_year#41, d_moy#42] -Condition : ((((isnotnull(d_moy#42) AND isnotnull(d_year#41)) AND (d_moy#42 = 12)) AND (d_year#41 = 1998)) AND isnotnull(d_date_sk#17)) - -(64) CometProject -Input [3]: [d_date_sk#17, d_year#41, d_moy#42] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(65) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] - -(66) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 - -Subquery:3 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometProject (69) - +- CometFilter (68) - +- CometNativeScan parquet spark_catalog.default.date_dim (67) - - -(67) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#31, d_month_seq#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(68) CometFilter -Input [2]: [d_date_sk#31, d_month_seq#43] -Condition : (((isnotnull(d_month_seq#43) AND (d_month_seq#43 >= Subquery scalar-subquery#44, [id=#45])) AND (d_month_seq#43 <= Subquery scalar-subquery#46, [id=#47])) AND isnotnull(d_date_sk#31)) - -(69) CometProject -Input [2]: [d_date_sk#31, d_month_seq#43] -Arguments: [d_date_sk#31], [d_date_sk#31] - -(70) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#31] - -(71) BroadcastExchange -Input [1]: [d_date_sk#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] - -Subquery:4 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#44, [id=#45] -* CometColumnarToRow (78) -+- CometHashAggregate (77) - +- CometExchange (76) - +- CometHashAggregate (75) - +- CometProject (74) - +- CometFilter (73) - +- CometNativeScan parquet spark_catalog.default.date_dim (72) - - -(72) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#48, d_year#41, d_moy#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] -ReadSchema: struct - -(73) CometFilter -Input [3]: [d_month_seq#48, d_year#41, d_moy#42] -Condition : (((isnotnull(d_year#41) AND isnotnull(d_moy#42)) AND (d_year#41 = 1998)) AND (d_moy#42 = 12)) - -(74) CometProject -Input [3]: [d_month_seq#48, d_year#41, d_moy#42] -Arguments: [(d_month_seq + 1)#49], [(d_month_seq#48 + 1) AS (d_month_seq + 1)#49] - -(75) CometHashAggregate -Input [1]: [(d_month_seq + 1)#49] -Keys [1]: [(d_month_seq + 1)#49] -Functions: [] - -(76) CometExchange -Input [1]: [(d_month_seq + 1)#49] -Arguments: hashpartitioning((d_month_seq + 1)#49, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(77) CometHashAggregate -Input [1]: [(d_month_seq + 1)#49] -Keys [1]: [(d_month_seq + 1)#49] -Functions: [] - -(78) CometColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 1)#49] - -Subquery:5 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#46, [id=#47] -* CometColumnarToRow (85) -+- CometHashAggregate (84) - +- CometExchange (83) - +- CometHashAggregate (82) - +- CometProject (81) - +- CometFilter (80) - +- CometNativeScan parquet spark_catalog.default.date_dim (79) - - -(79) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#48, d_year#41, d_moy#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] -ReadSchema: struct - -(80) CometFilter -Input [3]: [d_month_seq#48, d_year#41, d_moy#42] -Condition : (((isnotnull(d_year#41) AND isnotnull(d_moy#42)) AND (d_year#41 = 1998)) AND (d_moy#42 = 12)) - -(81) CometProject -Input [3]: [d_month_seq#48, d_year#41, d_moy#42] -Arguments: [(d_month_seq + 3)#50], [(d_month_seq#48 + 3) AS (d_month_seq + 3)#50] - -(82) CometHashAggregate -Input [1]: [(d_month_seq + 3)#50] -Keys [1]: [(d_month_seq + 3)#50] -Functions: [] - -(83) CometExchange -Input [1]: [(d_month_seq + 3)#50] -Arguments: hashpartitioning((d_month_seq + 3)#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(84) CometHashAggregate -Input [1]: [(d_month_seq + 3)#50] -Keys [1]: [(d_month_seq + 3)#50] -Functions: [] - -(85) CometColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 3)#50] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/simplified.txt deleted file mode 100644 index ba04de874d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/simplified.txt +++ /dev/null @@ -1,127 +0,0 @@ -TakeOrderedAndProject [segment,num_customers,segment_base] - WholeStageCodegen (13) - HashAggregate [segment,count] [count(1),num_customers,segment_base,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [segment] #1 - WholeStageCodegen (12) - HashAggregate [segment] [count,count] - HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_sk] #2 - WholeStageCodegen (11) - HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] - Project [c_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ca_county,ca_state,s_county,s_state] - Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometHashAggregate [c_customer_sk,c_current_addr_sk] - CometColumnarExchange [c_customer_sk,c_current_addr_sk] #3 - WholeStageCodegen (6) - HashAggregate [c_customer_sk,c_current_addr_sk] - Project [c_customer_sk,c_current_addr_sk] - BroadcastHashJoin [customer_sk,c_customer_sk] - Project [customer_sk] - BroadcastHashJoin [sold_date_sk,d_date_sk] - Project [sold_date_sk,customer_sk] - BroadcastHashJoin [item_sk,i_item_sk] - InputAdapter - Union - WholeStageCodegen (1) - Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] - Filter [cs_item_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (2) - Project [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] - Filter [ws_item_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (7) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometExchange [(d_month_seq + 1)] #9 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #10 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_county,ca_state] - CometFilter [ca_address_sk,ca_county,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_county,s_state] - CometFilter [s_county,s_state] - CometNativeScan parquet spark_catalog.default.store [s_county,s_state] - InputAdapter - ReusedExchange [d_date_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/explain.txt deleted file mode 100644 index 247d1d2d5a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,488 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (58) -+- CometTakeOrderedAndProject (57) - +- CometHashAggregate (56) - +- CometExchange (55) - +- CometHashAggregate (54) - +- CometHashAggregate (53) - +- CometExchange (52) - +- CometHashAggregate (51) - +- CometProject (50) - +- CometBroadcastHashJoin (49) - :- CometProject (44) - : +- CometBroadcastHashJoin (43) - : :- CometProject (38) - : : +- CometBroadcastHashJoin (37) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometHashAggregate (27) - : : : : +- CometExchange (26) - : : : : +- CometHashAggregate (25) - : : : : +- CometProject (24) - : : : : +- CometBroadcastHashJoin (23) - : : : : :- CometProject (19) - : : : : : +- CometBroadcastHashJoin (18) - : : : : : :- CometProject (13) - : : : : : : +- CometBroadcastHashJoin (12) - : : : : : : :- CometUnion (7) - : : : : : : : :- CometProject (3) - : : : : : : : : +- CometFilter (2) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : +- CometProject (6) - : : : : : : : +- CometFilter (5) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (4) - : : : : : : +- CometBroadcastExchange (11) - : : : : : : +- CometProject (10) - : : : : : : +- CometFilter (9) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (8) - : : : : : +- CometBroadcastExchange (17) - : : : : : +- CometProject (16) - : : : : : +- CometFilter (15) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - : : : : +- CometBroadcastExchange (22) - : : : : +- CometFilter (21) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (20) - : : : +- CometBroadcastExchange (30) - : : : +- CometFilter (29) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (28) - : : +- CometBroadcastExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (33) - : +- CometBroadcastExchange (42) - : +- CometProject (41) - : +- CometFilter (40) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (39) - +- CometBroadcastExchange (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_bill_customer_sk#1)) - -(3) CometProject -Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Arguments: [sold_date_sk#5, customer_sk#6, item_sk#7], [cs_sold_date_sk#3 AS sold_date_sk#5, cs_bill_customer_sk#1 AS customer_sk#6, cs_item_sk#2 AS item_sk#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#10), dynamicpruningexpression(ws_sold_date_sk#10 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Condition : (isnotnull(ws_item_sk#8) AND isnotnull(ws_bill_customer_sk#9)) - -(6) CometProject -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Arguments: [sold_date_sk#11, customer_sk#12, item_sk#13], [ws_sold_date_sk#10 AS sold_date_sk#11, ws_bill_customer_sk#9 AS customer_sk#12, ws_item_sk#8 AS item_sk#13] - -(7) CometUnion -Child 0 Input [3]: [sold_date_sk#5, customer_sk#6, item_sk#7] -Child 1 Input [3]: [sold_date_sk#11, customer_sk#12, item_sk#13] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_class#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true) = Women ) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#15, 50, true, false, true) = maternity )) AND isnotnull(i_item_sk#14)) - -(10) CometProject -Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Arguments: [i_item_sk#14], [i_item_sk#14] - -(11) CometBroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: [i_item_sk#14] - -(12) CometBroadcastHashJoin -Left output [3]: [sold_date_sk#5, customer_sk#6, item_sk#7] -Right output [1]: [i_item_sk#14] -Arguments: [item_sk#7], [i_item_sk#14], Inner, BuildRight - -(13) CometProject -Input [4]: [sold_date_sk#5, customer_sk#6, item_sk#7, i_item_sk#14] -Arguments: [sold_date_sk#5, customer_sk#6], [sold_date_sk#5, customer_sk#6] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12)) AND (d_year#18 = 1998)) AND isnotnull(d_date_sk#17)) - -(16) CometProject -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: [d_date_sk#17] - -(18) CometBroadcastHashJoin -Left output [2]: [sold_date_sk#5, customer_sk#6] -Right output [1]: [d_date_sk#17] -Arguments: [sold_date_sk#5], [d_date_sk#17], Inner, BuildRight - -(19) CometProject -Input [3]: [sold_date_sk#5, customer_sk#6, d_date_sk#17] -Arguments: [customer_sk#6], [customer_sk#6] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#20, c_current_addr_sk#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Condition : (isnotnull(c_customer_sk#20) AND isnotnull(c_current_addr_sk#21)) - -(22) CometBroadcastExchange -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Arguments: [c_customer_sk#20, c_current_addr_sk#21] - -(23) CometBroadcastHashJoin -Left output [1]: [customer_sk#6] -Right output [2]: [c_customer_sk#20, c_current_addr_sk#21] -Arguments: [customer_sk#6], [c_customer_sk#20], Inner, BuildRight - -(24) CometProject -Input [3]: [customer_sk#6, c_customer_sk#20, c_current_addr_sk#21] -Arguments: [c_customer_sk#20, c_current_addr_sk#21], [c_customer_sk#20, c_current_addr_sk#21] - -(25) CometHashAggregate -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Keys [2]: [c_customer_sk#20, c_current_addr_sk#21] -Functions: [] - -(26) CometExchange -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Arguments: hashpartitioning(c_customer_sk#20, c_current_addr_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Keys [2]: [c_customer_sk#20, c_current_addr_sk#21] -Functions: [] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#24), dynamicpruningexpression(ss_sold_date_sk#24 IN dynamicpruning#25)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(29) CometFilter -Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Condition : isnotnull(ss_customer_sk#22) - -(30) CometBroadcastExchange -Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Arguments: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] - -(31) CometBroadcastHashJoin -Left output [2]: [c_customer_sk#20, c_current_addr_sk#21] -Right output [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Arguments: [c_customer_sk#20], [ss_customer_sk#22], Inner, BuildRight - -(32) CometProject -Input [5]: [c_customer_sk#20, c_current_addr_sk#21, ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Arguments: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24], [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#26, ca_county#27, ca_state#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct - -(34) CometFilter -Input [3]: [ca_address_sk#26, ca_county#27, ca_state#28] -Condition : ((isnotnull(ca_address_sk#26) AND isnotnull(ca_county#27)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#28, 2, true, false, true))) - -(35) CometProject -Input [3]: [ca_address_sk#26, ca_county#27, ca_state#28] -Arguments: [ca_address_sk#26, ca_county#27, ca_state#29], [ca_address_sk#26, ca_county#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#28, 2, true, false, true) AS ca_state#29] - -(36) CometBroadcastExchange -Input [3]: [ca_address_sk#26, ca_county#27, ca_state#29] -Arguments: [ca_address_sk#26, ca_county#27, ca_state#29] - -(37) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24] -Right output [3]: [ca_address_sk#26, ca_county#27, ca_state#29] -Arguments: [c_current_addr_sk#21], [ca_address_sk#26], Inner, BuildRight - -(38) CometProject -Input [7]: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_address_sk#26, ca_county#27, ca_state#29] -Arguments: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29], [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_county#30, s_state#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county)] -ReadSchema: struct - -(40) CometFilter -Input [2]: [s_county#30, s_state#31] -Condition : (isnotnull(s_county#30) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#31, 2, true, false, true))) - -(41) CometProject -Input [2]: [s_county#30, s_state#31] -Arguments: [s_county#30, s_state#32], [s_county#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#31, 2, true, false, true) AS s_state#32] - -(42) CometBroadcastExchange -Input [2]: [s_county#30, s_state#32] -Arguments: [s_county#30, s_state#32] - -(43) CometBroadcastHashJoin -Left output [5]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29] -Right output [2]: [s_county#30, s_state#32] -Arguments: [ca_county#27, ca_state#29], [s_county#30, s_state#32], Inner, BuildRight - -(44) CometProject -Input [7]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29, s_county#30, s_state#32] -Arguments: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24], [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_month_seq#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [d_date_sk#33, d_month_seq#34] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= ReusedSubquery Subquery scalar-subquery#35, [id=#36])) AND (d_month_seq#34 <= ReusedSubquery Subquery scalar-subquery#37, [id=#38])) AND isnotnull(d_date_sk#33)) - -(47) CometProject -Input [2]: [d_date_sk#33, d_month_seq#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(48) CometBroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: [d_date_sk#33] - -(49) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] -Right output [1]: [d_date_sk#33] -Arguments: [ss_sold_date_sk#24], [d_date_sk#33], Inner, BuildRight - -(50) CometProject -Input [4]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, d_date_sk#33] -Arguments: [c_customer_sk#20, ss_ext_sales_price#23], [c_customer_sk#20, ss_ext_sales_price#23] - -(51) CometHashAggregate -Input [2]: [c_customer_sk#20, ss_ext_sales_price#23] -Keys [1]: [c_customer_sk#20] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#23))] - -(52) CometExchange -Input [2]: [c_customer_sk#20, sum#39] -Arguments: hashpartitioning(c_customer_sk#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(53) CometHashAggregate -Input [2]: [c_customer_sk#20, sum#39] -Keys [1]: [c_customer_sk#20] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#23))] - -(54) CometHashAggregate -Input [1]: [segment#40] -Keys [1]: [segment#40] -Functions [1]: [partial_count(1)] - -(55) CometExchange -Input [2]: [segment#40, count#41] -Arguments: hashpartitioning(segment#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(56) CometHashAggregate -Input [2]: [segment#40, count#41] -Keys [1]: [segment#40] -Functions [1]: [count(1)] - -(57) CometTakeOrderedAndProject -Input [3]: [segment#40, num_customers#42, segment_base#43] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[segment#40 ASC NULLS FIRST,num_customers#42 ASC NULLS FIRST], output=[segment#40,num_customers#42,segment_base#43]), [segment#40, num_customers#42, segment_base#43], 100, 0, [segment#40 ASC NULLS FIRST, num_customers#42 ASC NULLS FIRST], [segment#40, num_customers#42, segment_base#43] - -(58) CometColumnarToRow [codegen id : 1] -Input [3]: [segment#40, num_customers#42, segment_base#43] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (63) -+- * CometColumnarToRow (62) - +- CometProject (61) - +- CometFilter (60) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (59) - - -(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(60) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12)) AND (d_year#18 = 1998)) AND isnotnull(d_date_sk#17)) - -(61) CometProject -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(62) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] - -(63) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 - -Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 -BroadcastExchange (68) -+- * CometColumnarToRow (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (64) - - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_month_seq#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(65) CometFilter -Input [2]: [d_date_sk#33, d_month_seq#34] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= Subquery scalar-subquery#35, [id=#36])) AND (d_month_seq#34 <= Subquery scalar-subquery#37, [id=#38])) AND isnotnull(d_date_sk#33)) - -(66) CometProject -Input [2]: [d_date_sk#33, d_month_seq#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(67) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#33] - -(68) BroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:4 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#35, [id=#36] -* CometColumnarToRow (75) -+- CometHashAggregate (74) - +- CometExchange (73) - +- CometHashAggregate (72) - +- CometProject (71) - +- CometFilter (70) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (69) - - -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#44, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] -ReadSchema: struct - -(70) CometFilter -Input [3]: [d_month_seq#44, d_year#18, d_moy#19] -Condition : (((isnotnull(d_year#18) AND isnotnull(d_moy#19)) AND (d_year#18 = 1998)) AND (d_moy#19 = 12)) - -(71) CometProject -Input [3]: [d_month_seq#44, d_year#18, d_moy#19] -Arguments: [(d_month_seq + 1)#45], [(d_month_seq#44 + 1) AS (d_month_seq + 1)#45] - -(72) CometHashAggregate -Input [1]: [(d_month_seq + 1)#45] -Keys [1]: [(d_month_seq + 1)#45] -Functions: [] - -(73) CometExchange -Input [1]: [(d_month_seq + 1)#45] -Arguments: hashpartitioning((d_month_seq + 1)#45, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(74) CometHashAggregate -Input [1]: [(d_month_seq + 1)#45] -Keys [1]: [(d_month_seq + 1)#45] -Functions: [] - -(75) CometColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 1)#45] - -Subquery:5 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#37, [id=#38] -* CometColumnarToRow (82) -+- CometHashAggregate (81) - +- CometExchange (80) - +- CometHashAggregate (79) - +- CometProject (78) - +- CometFilter (77) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) - - -(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#44, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] -ReadSchema: struct - -(77) CometFilter -Input [3]: [d_month_seq#44, d_year#18, d_moy#19] -Condition : (((isnotnull(d_year#18) AND isnotnull(d_moy#19)) AND (d_year#18 = 1998)) AND (d_moy#19 = 12)) - -(78) CometProject -Input [3]: [d_month_seq#44, d_year#18, d_moy#19] -Arguments: [(d_month_seq + 3)#46], [(d_month_seq#44 + 3) AS (d_month_seq + 3)#46] - -(79) CometHashAggregate -Input [1]: [(d_month_seq + 3)#46] -Keys [1]: [(d_month_seq + 3)#46] -Functions: [] - -(80) CometExchange -Input [1]: [(d_month_seq + 3)#46] -Arguments: hashpartitioning((d_month_seq + 3)#46, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(81) CometHashAggregate -Input [1]: [(d_month_seq + 3)#46] -Keys [1]: [(d_month_seq + 3)#46] -Functions: [] - -(82) CometColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 3)#46] - -Subquery:6 Hosting operator id = 46 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#36] - -Subquery:7 Hosting operator id = 46 Hosting Expression = ReusedSubquery Subquery scalar-subquery#37, [id=#38] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/simplified.txt deleted file mode 100644 index ae0cecb1ba..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,99 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [segment,num_customers,segment_base] - CometHashAggregate [count] [segment,num_customers,segment_base,count(1)] - CometExchange [segment] #1 - CometHashAggregate [segment,count] - CometHashAggregate [c_customer_sk,sum] [segment,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [c_customer_sk] #2 - CometHashAggregate [ss_ext_sales_price] [c_customer_sk,sum] - CometProject [c_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state,s_county,s_state] - CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] - CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk,ca_address_sk,ca_county,ca_state] - CometProject [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometHashAggregate [c_customer_sk,c_current_addr_sk] - CometExchange [c_customer_sk,c_current_addr_sk] #3 - CometHashAggregate [c_customer_sk,c_current_addr_sk] - CometProject [c_customer_sk,c_current_addr_sk] - CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] - CometProject [customer_sk] - CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] - CometProject [sold_date_sk,customer_sk] - 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_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [i_item_sk] #5 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] #8 - CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometExchange [(d_month_seq + 1)] #10 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #11 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state] #12 - CometProject [ca_state] [ca_address_sk,ca_county,ca_state] - CometFilter [ca_address_sk,ca_county,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] - CometBroadcastExchange [s_county,s_state] #13 - CometProject [s_state] [s_county,s_state] - CometFilter [s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_county,s_state] - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [(d_month_seq + 1)] #3 - ReusedSubquery [(d_month_seq + 3)] #4 - CometScan [native_iceberg_compat] 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/q54/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt deleted file mode 100644 index 247d1d2d5a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt +++ /dev/null @@ -1,488 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (58) -+- CometTakeOrderedAndProject (57) - +- CometHashAggregate (56) - +- CometExchange (55) - +- CometHashAggregate (54) - +- CometHashAggregate (53) - +- CometExchange (52) - +- CometHashAggregate (51) - +- CometProject (50) - +- CometBroadcastHashJoin (49) - :- CometProject (44) - : +- CometBroadcastHashJoin (43) - : :- CometProject (38) - : : +- CometBroadcastHashJoin (37) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometHashAggregate (27) - : : : : +- CometExchange (26) - : : : : +- CometHashAggregate (25) - : : : : +- CometProject (24) - : : : : +- CometBroadcastHashJoin (23) - : : : : :- CometProject (19) - : : : : : +- CometBroadcastHashJoin (18) - : : : : : :- CometProject (13) - : : : : : : +- CometBroadcastHashJoin (12) - : : : : : : :- CometUnion (7) - : : : : : : : :- CometProject (3) - : : : : : : : : +- CometFilter (2) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : +- CometProject (6) - : : : : : : : +- CometFilter (5) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (4) - : : : : : : +- CometBroadcastExchange (11) - : : : : : : +- CometProject (10) - : : : : : : +- CometFilter (9) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (8) - : : : : : +- CometBroadcastExchange (17) - : : : : : +- CometProject (16) - : : : : : +- CometFilter (15) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - : : : : +- CometBroadcastExchange (22) - : : : : +- CometFilter (21) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (20) - : : : +- CometBroadcastExchange (30) - : : : +- CometFilter (29) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (28) - : : +- CometBroadcastExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (33) - : +- CometBroadcastExchange (42) - : +- CometProject (41) - : +- CometFilter (40) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (39) - +- CometBroadcastExchange (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_bill_customer_sk#1)) - -(3) CometProject -Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] -Arguments: [sold_date_sk#5, customer_sk#6, item_sk#7], [cs_sold_date_sk#3 AS sold_date_sk#5, cs_bill_customer_sk#1 AS customer_sk#6, cs_item_sk#2 AS item_sk#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#10), dynamicpruningexpression(ws_sold_date_sk#10 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Condition : (isnotnull(ws_item_sk#8) AND isnotnull(ws_bill_customer_sk#9)) - -(6) CometProject -Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] -Arguments: [sold_date_sk#11, customer_sk#12, item_sk#13], [ws_sold_date_sk#10 AS sold_date_sk#11, ws_bill_customer_sk#9 AS customer_sk#12, ws_item_sk#8 AS item_sk#13] - -(7) CometUnion -Child 0 Input [3]: [sold_date_sk#5, customer_sk#6, item_sk#7] -Child 1 Input [3]: [sold_date_sk#11, customer_sk#12, item_sk#13] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_class#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(9) CometFilter -Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true) = Women ) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#15, 50, true, false, true) = maternity )) AND isnotnull(i_item_sk#14)) - -(10) CometProject -Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Arguments: [i_item_sk#14], [i_item_sk#14] - -(11) CometBroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: [i_item_sk#14] - -(12) CometBroadcastHashJoin -Left output [3]: [sold_date_sk#5, customer_sk#6, item_sk#7] -Right output [1]: [i_item_sk#14] -Arguments: [item_sk#7], [i_item_sk#14], Inner, BuildRight - -(13) CometProject -Input [4]: [sold_date_sk#5, customer_sk#6, item_sk#7, i_item_sk#14] -Arguments: [sold_date_sk#5, customer_sk#6], [sold_date_sk#5, customer_sk#6] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12)) AND (d_year#18 = 1998)) AND isnotnull(d_date_sk#17)) - -(16) CometProject -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: [d_date_sk#17] - -(18) CometBroadcastHashJoin -Left output [2]: [sold_date_sk#5, customer_sk#6] -Right output [1]: [d_date_sk#17] -Arguments: [sold_date_sk#5], [d_date_sk#17], Inner, BuildRight - -(19) CometProject -Input [3]: [sold_date_sk#5, customer_sk#6, d_date_sk#17] -Arguments: [customer_sk#6], [customer_sk#6] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#20, c_current_addr_sk#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Condition : (isnotnull(c_customer_sk#20) AND isnotnull(c_current_addr_sk#21)) - -(22) CometBroadcastExchange -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Arguments: [c_customer_sk#20, c_current_addr_sk#21] - -(23) CometBroadcastHashJoin -Left output [1]: [customer_sk#6] -Right output [2]: [c_customer_sk#20, c_current_addr_sk#21] -Arguments: [customer_sk#6], [c_customer_sk#20], Inner, BuildRight - -(24) CometProject -Input [3]: [customer_sk#6, c_customer_sk#20, c_current_addr_sk#21] -Arguments: [c_customer_sk#20, c_current_addr_sk#21], [c_customer_sk#20, c_current_addr_sk#21] - -(25) CometHashAggregate -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Keys [2]: [c_customer_sk#20, c_current_addr_sk#21] -Functions: [] - -(26) CometExchange -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Arguments: hashpartitioning(c_customer_sk#20, c_current_addr_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [2]: [c_customer_sk#20, c_current_addr_sk#21] -Keys [2]: [c_customer_sk#20, c_current_addr_sk#21] -Functions: [] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#24), dynamicpruningexpression(ss_sold_date_sk#24 IN dynamicpruning#25)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(29) CometFilter -Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Condition : isnotnull(ss_customer_sk#22) - -(30) CometBroadcastExchange -Input [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Arguments: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] - -(31) CometBroadcastHashJoin -Left output [2]: [c_customer_sk#20, c_current_addr_sk#21] -Right output [3]: [ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Arguments: [c_customer_sk#20], [ss_customer_sk#22], Inner, BuildRight - -(32) CometProject -Input [5]: [c_customer_sk#20, c_current_addr_sk#21, ss_customer_sk#22, ss_ext_sales_price#23, ss_sold_date_sk#24] -Arguments: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24], [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#26, ca_county#27, ca_state#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] -ReadSchema: struct - -(34) CometFilter -Input [3]: [ca_address_sk#26, ca_county#27, ca_state#28] -Condition : ((isnotnull(ca_address_sk#26) AND isnotnull(ca_county#27)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#28, 2, true, false, true))) - -(35) CometProject -Input [3]: [ca_address_sk#26, ca_county#27, ca_state#28] -Arguments: [ca_address_sk#26, ca_county#27, ca_state#29], [ca_address_sk#26, ca_county#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#28, 2, true, false, true) AS ca_state#29] - -(36) CometBroadcastExchange -Input [3]: [ca_address_sk#26, ca_county#27, ca_state#29] -Arguments: [ca_address_sk#26, ca_county#27, ca_state#29] - -(37) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24] -Right output [3]: [ca_address_sk#26, ca_county#27, ca_state#29] -Arguments: [c_current_addr_sk#21], [ca_address_sk#26], Inner, BuildRight - -(38) CometProject -Input [7]: [c_customer_sk#20, c_current_addr_sk#21, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_address_sk#26, ca_county#27, ca_state#29] -Arguments: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29], [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_county#30, s_state#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county)] -ReadSchema: struct - -(40) CometFilter -Input [2]: [s_county#30, s_state#31] -Condition : (isnotnull(s_county#30) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#31, 2, true, false, true))) - -(41) CometProject -Input [2]: [s_county#30, s_state#31] -Arguments: [s_county#30, s_state#32], [s_county#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#31, 2, true, false, true) AS s_state#32] - -(42) CometBroadcastExchange -Input [2]: [s_county#30, s_state#32] -Arguments: [s_county#30, s_state#32] - -(43) CometBroadcastHashJoin -Left output [5]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29] -Right output [2]: [s_county#30, s_state#32] -Arguments: [ca_county#27, ca_state#29], [s_county#30, s_state#32], Inner, BuildRight - -(44) CometProject -Input [7]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, ca_county#27, ca_state#29, s_county#30, s_state#32] -Arguments: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24], [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_month_seq#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [d_date_sk#33, d_month_seq#34] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= ReusedSubquery Subquery scalar-subquery#35, [id=#36])) AND (d_month_seq#34 <= ReusedSubquery Subquery scalar-subquery#37, [id=#38])) AND isnotnull(d_date_sk#33)) - -(47) CometProject -Input [2]: [d_date_sk#33, d_month_seq#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(48) CometBroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: [d_date_sk#33] - -(49) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24] -Right output [1]: [d_date_sk#33] -Arguments: [ss_sold_date_sk#24], [d_date_sk#33], Inner, BuildRight - -(50) CometProject -Input [4]: [c_customer_sk#20, ss_ext_sales_price#23, ss_sold_date_sk#24, d_date_sk#33] -Arguments: [c_customer_sk#20, ss_ext_sales_price#23], [c_customer_sk#20, ss_ext_sales_price#23] - -(51) CometHashAggregate -Input [2]: [c_customer_sk#20, ss_ext_sales_price#23] -Keys [1]: [c_customer_sk#20] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#23))] - -(52) CometExchange -Input [2]: [c_customer_sk#20, sum#39] -Arguments: hashpartitioning(c_customer_sk#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(53) CometHashAggregate -Input [2]: [c_customer_sk#20, sum#39] -Keys [1]: [c_customer_sk#20] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#23))] - -(54) CometHashAggregate -Input [1]: [segment#40] -Keys [1]: [segment#40] -Functions [1]: [partial_count(1)] - -(55) CometExchange -Input [2]: [segment#40, count#41] -Arguments: hashpartitioning(segment#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(56) CometHashAggregate -Input [2]: [segment#40, count#41] -Keys [1]: [segment#40] -Functions [1]: [count(1)] - -(57) CometTakeOrderedAndProject -Input [3]: [segment#40, num_customers#42, segment_base#43] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[segment#40 ASC NULLS FIRST,num_customers#42 ASC NULLS FIRST], output=[segment#40,num_customers#42,segment_base#43]), [segment#40, num_customers#42, segment_base#43], 100, 0, [segment#40 ASC NULLS FIRST, num_customers#42 ASC NULLS FIRST], [segment#40, num_customers#42, segment_base#43] - -(58) CometColumnarToRow [codegen id : 1] -Input [3]: [segment#40, num_customers#42, segment_base#43] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (63) -+- * CometColumnarToRow (62) - +- CometProject (61) - +- CometFilter (60) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (59) - - -(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(60) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12)) AND (d_year#18 = 1998)) AND isnotnull(d_date_sk#17)) - -(61) CometProject -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [d_date_sk#17], [d_date_sk#17] - -(62) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#17] - -(63) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 - -Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 -BroadcastExchange (68) -+- * CometColumnarToRow (67) - +- CometProject (66) - +- CometFilter (65) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (64) - - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_month_seq#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(65) CometFilter -Input [2]: [d_date_sk#33, d_month_seq#34] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= Subquery scalar-subquery#35, [id=#36])) AND (d_month_seq#34 <= Subquery scalar-subquery#37, [id=#38])) AND isnotnull(d_date_sk#33)) - -(66) CometProject -Input [2]: [d_date_sk#33, d_month_seq#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(67) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#33] - -(68) BroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:4 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#35, [id=#36] -* CometColumnarToRow (75) -+- CometHashAggregate (74) - +- CometExchange (73) - +- CometHashAggregate (72) - +- CometProject (71) - +- CometFilter (70) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (69) - - -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#44, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] -ReadSchema: struct - -(70) CometFilter -Input [3]: [d_month_seq#44, d_year#18, d_moy#19] -Condition : (((isnotnull(d_year#18) AND isnotnull(d_moy#19)) AND (d_year#18 = 1998)) AND (d_moy#19 = 12)) - -(71) CometProject -Input [3]: [d_month_seq#44, d_year#18, d_moy#19] -Arguments: [(d_month_seq + 1)#45], [(d_month_seq#44 + 1) AS (d_month_seq + 1)#45] - -(72) CometHashAggregate -Input [1]: [(d_month_seq + 1)#45] -Keys [1]: [(d_month_seq + 1)#45] -Functions: [] - -(73) CometExchange -Input [1]: [(d_month_seq + 1)#45] -Arguments: hashpartitioning((d_month_seq + 1)#45, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(74) CometHashAggregate -Input [1]: [(d_month_seq + 1)#45] -Keys [1]: [(d_month_seq + 1)#45] -Functions: [] - -(75) CometColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 1)#45] - -Subquery:5 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#37, [id=#38] -* CometColumnarToRow (82) -+- CometHashAggregate (81) - +- CometExchange (80) - +- CometHashAggregate (79) - +- CometProject (78) - +- CometFilter (77) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) - - -(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#44, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] -ReadSchema: struct - -(77) CometFilter -Input [3]: [d_month_seq#44, d_year#18, d_moy#19] -Condition : (((isnotnull(d_year#18) AND isnotnull(d_moy#19)) AND (d_year#18 = 1998)) AND (d_moy#19 = 12)) - -(78) CometProject -Input [3]: [d_month_seq#44, d_year#18, d_moy#19] -Arguments: [(d_month_seq + 3)#46], [(d_month_seq#44 + 3) AS (d_month_seq + 3)#46] - -(79) CometHashAggregate -Input [1]: [(d_month_seq + 3)#46] -Keys [1]: [(d_month_seq + 3)#46] -Functions: [] - -(80) CometExchange -Input [1]: [(d_month_seq + 3)#46] -Arguments: hashpartitioning((d_month_seq + 3)#46, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(81) CometHashAggregate -Input [1]: [(d_month_seq + 3)#46] -Keys [1]: [(d_month_seq + 3)#46] -Functions: [] - -(82) CometColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 3)#46] - -Subquery:6 Hosting operator id = 46 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#36] - -Subquery:7 Hosting operator id = 46 Hosting Expression = ReusedSubquery Subquery scalar-subquery#37, [id=#38] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/extended.txt deleted file mode 100644 index 98fce92fb1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/extended.txt +++ /dev/null @@ -1,91 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometUnion - : : : : : : : :- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : :- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : :- ReusedSubquery - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 75 out of 84 eligible operators (89%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index ae0cecb1ba..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/simplified.txt +++ /dev/null @@ -1,99 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [segment,num_customers,segment_base] - CometHashAggregate [count] [segment,num_customers,segment_base,count(1)] - CometExchange [segment] #1 - CometHashAggregate [segment,count] - CometHashAggregate [c_customer_sk,sum] [segment,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [c_customer_sk] #2 - CometHashAggregate [ss_ext_sales_price] [c_customer_sk,sum] - CometProject [c_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state,s_county,s_state] - CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] - CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk,ca_address_sk,ca_county,ca_state] - CometProject [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometHashAggregate [c_customer_sk,c_current_addr_sk] - CometExchange [c_customer_sk,c_current_addr_sk] #3 - CometHashAggregate [c_customer_sk,c_current_addr_sk] - CometProject [c_customer_sk,c_current_addr_sk] - CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] - CometProject [customer_sk] - CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] - CometProject [sold_date_sk,customer_sk] - 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_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [i_item_sk] #5 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] #8 - CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometExchange [(d_month_seq + 1)] #10 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #11 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state] #12 - CometProject [ca_state] [ca_address_sk,ca_county,ca_state] - CometFilter [ca_address_sk,ca_county,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] - CometBroadcastExchange [s_county,s_state] #13 - CometProject [s_state] [s_county,s_state] - CometFilter [s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_county,s_state] - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [(d_month_seq + 1)] #3 - ReusedSubquery [(d_month_seq + 3)] #4 - CometScan [native_iceberg_compat] 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/q55.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_datafusion/explain.txt deleted file mode 100644 index 358eb94c69..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_datafusion/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometNativeScan parquet spark_catalog.default.item (9) - - -(1) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1999)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1], [d_date_sk#1] - -(4) CometNativeScan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [1]: [d_date_sk#1] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [4]: [d_date_sk#1, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5], [ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,28), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 28)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [3]: [ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [2]: [i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [3]: [i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [i_brand#11, i_brand_id#8, sum#12] -Keys [2]: [i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [3]: [brand_id#13, brand#14, ext_price#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[brand_id#13,brand#14,ext_price#15]), [brand_id#13, brand#14, ext_price#15], 100, 0, [ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [brand_id#13, brand#14, ext_price#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [3]: [brand_id#13, brand#14, ext_price#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_datafusion/simplified.txt deleted file mode 100644 index a2adacc531..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_datafusion/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,ext_price] - CometHashAggregate [sum] [brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,sum] - CometProject [ss_ext_sales_price,i_brand_id,i_brand] - 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_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometNativeScan 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/q55.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_iceberg_compat/explain.txt deleted file mode 100644 index 47e0117481..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1999)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1], [d_date_sk#1] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [1]: [d_date_sk#1] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [4]: [d_date_sk#1, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5], [ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,28), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 28)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [3]: [ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [2]: [i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [3]: [i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [i_brand#11, i_brand_id#8, sum#12] -Keys [2]: [i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [3]: [brand_id#13, brand#14, ext_price#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[brand_id#13,brand#14,ext_price#15]), [brand_id#13, brand#14, ext_price#15], 100, 0, [ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [brand_id#13, brand#14, ext_price#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [3]: [brand_id#13, brand#14, ext_price#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_iceberg_compat/simplified.txt deleted file mode 100644 index dd7658cbac..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,ext_price] - CometHashAggregate [sum] [brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,sum] - CometProject [ss_ext_sales_price,i_brand_id,i_brand] - 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_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan [native_iceberg_compat] 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/q55/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt deleted file mode 100644 index 47e0117481..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt +++ /dev/null @@ -1,111 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (19) -+- CometTakeOrderedAndProject (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (1) - : +- CometBroadcastExchange (6) - : +- CometFilter (5) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1999)) AND isnotnull(d_date_sk#1)) - -(3) CometProject -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -Arguments: [d_date_sk#1], [d_date_sk#1] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_item_sk#4) - -(6) CometBroadcastExchange -Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [1]: [d_date_sk#1] -Right output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [d_date_sk#1], [ss_sold_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [4]: [d_date_sk#1, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#4, ss_ext_sales_price#5], [ss_item_sk#4, ss_ext_sales_price#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,28), IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 28)) AND isnotnull(i_item_sk#7)) - -(11) CometProject -Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11], [i_item_sk#7, i_brand_id#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [i_item_sk#7, i_brand_id#8, i_brand#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#4, ss_ext_sales_price#5] -Right output [3]: [i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_item_sk#4], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#11] -Arguments: [ss_ext_sales_price#5, i_brand_id#8, i_brand#11], [ss_ext_sales_price#5, i_brand_id#8, i_brand#11] - -(15) CometHashAggregate -Input [3]: [ss_ext_sales_price#5, i_brand_id#8, i_brand#11] -Keys [2]: [i_brand#11, i_brand_id#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(16) CometExchange -Input [3]: [i_brand#11, i_brand_id#8, sum#12] -Arguments: hashpartitioning(i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [i_brand#11, i_brand_id#8, sum#12] -Keys [2]: [i_brand#11, i_brand_id#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(18) CometTakeOrderedAndProject -Input [3]: [brand_id#13, brand#14, ext_price#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[brand_id#13,brand#14,ext_price#15]), [brand_id#13, brand#14, ext_price#15], 100, 0, [ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [brand_id#13, brand#14, ext_price#15] - -(19) CometColumnarToRow [codegen id : 1] -Input [3]: [brand_id#13, brand#14, ext_price#15] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/extended.txt deleted file mode 100644 index a292badf5d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index dd7658cbac..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/simplified.txt +++ /dev/null @@ -1,21 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [brand_id,brand,ext_price] - CometHashAggregate [sum] [brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_brand,i_brand_id] #1 - CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,sum] - CometProject [ss_ext_sales_price,i_brand_id,i_brand] - 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_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/explain.txt deleted file mode 100644 index 62396854a7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/explain.txt +++ /dev/null @@ -1,420 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (67) -+- * HashAggregate (66) - +- * CometColumnarToRow (65) - +- CometColumnarExchange (64) - +- * HashAggregate (63) - +- Union (62) - :- * HashAggregate (29) - : +- * CometColumnarToRow (28) - : +- CometColumnarExchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * CometColumnarToRow (22) - : +- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.item (14) - : +- CometBroadcastExchange (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometNativeScan parquet spark_catalog.default.item (16) - :- * HashAggregate (45) - : +- * CometColumnarToRow (44) - : +- CometColumnarExchange (43) - : +- * HashAggregate (42) - : +- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (38) - : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * Filter (32) - : : : : +- * ColumnarToRow (31) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (30) - : : : +- ReusedExchange (33) - : : +- ReusedExchange (36) - : +- ReusedExchange (39) - +- * HashAggregate (61) - +- * CometColumnarToRow (60) - +- CometColumnarExchange (59) - +- * HashAggregate (58) - +- * Project (57) - +- * BroadcastHashJoin Inner BuildRight (56) - :- * Project (54) - : +- * BroadcastHashJoin Inner BuildRight (53) - : :- * Project (51) - : : +- * BroadcastHashJoin Inner BuildRight (50) - : : :- * Filter (48) - : : : +- * ColumnarToRow (47) - : : : +- Scan parquet spark_catalog.default.web_sales (46) - : : +- ReusedExchange (49) - : +- ReusedExchange (52) - +- ReusedExchange (55) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(4) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_gmt_offset#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isnotnull(ca_address_sk#7)) - -(9) CometProject -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Arguments: [ca_address_sk#7], [ca_address_sk#7] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [ca_address_sk#7] - -(11) BroadcastExchange -Input [1]: [ca_address_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] - -(14) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#9, i_item_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [i_item_sk#9, i_item_id#10] -Condition : isnotnull(i_item_sk#9) - -(16) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_id#11, i_color#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(17) CometFilter -Input [2]: [i_item_id#11, i_color#12] -Condition : staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#12, 20, true, false, true) IN (slate ,blanched ,burnished ) - -(18) CometProject -Input [2]: [i_item_id#11, i_color#12] -Arguments: [i_item_id#13], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#11, 16, true, false, true) AS i_item_id#13] - -(19) CometBroadcastExchange -Input [1]: [i_item_id#13] -Arguments: [i_item_id#13] - -(20) CometBroadcastHashJoin -Left output [2]: [i_item_sk#9, i_item_id#10] -Right output [1]: [i_item_id#13] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#10, 16, true, false, true)], [i_item_id#13], LeftSemi, BuildRight - -(21) CometProject -Input [2]: [i_item_sk#9, i_item_id#10] -Arguments: [i_item_sk#9, i_item_id#14], [i_item_sk#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#10, 16, true, false, true) AS i_item_id#14] - -(22) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#9, i_item_id#14] - -(23) BroadcastExchange -Input [2]: [i_item_sk#9, i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#9] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 4] -Output [2]: [ss_ext_sales_price#3, i_item_id#14] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_item_id#14] - -(26) HashAggregate [codegen id : 4] -Input [2]: [ss_ext_sales_price#3, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum#15] -Results [2]: [i_item_id#14, sum#16] - -(27) CometColumnarExchange -Input [2]: [i_item_id#14, sum#16] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometColumnarToRow [codegen id : 5] -Input [2]: [i_item_id#14, sum#16] - -(29) HashAggregate [codegen id : 5] -Input [2]: [i_item_id#14, sum#16] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#17] -Results [2]: [i_item_id#14 AS i_item_id#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#17,17,2) AS total_sales#19] - -(30) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 9] -Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] - -(32) Filter [codegen id : 9] -Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -Condition : (isnotnull(cs_bill_addr_sk#20) AND isnotnull(cs_item_sk#21)) - -(33) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#24] - -(34) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 9] -Output [3]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22] -Input [5]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#24] - -(36) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#25] - -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_bill_addr_sk#20] -Right keys [1]: [ca_address_sk#25] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 9] -Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] -Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, ca_address_sk#25] - -(39) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#26, i_item_id#27] - -(40) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#21] -Right keys [1]: [i_item_sk#26] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 9] -Output [2]: [cs_ext_sales_price#22, i_item_id#27] -Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#26, i_item_id#27] - -(42) HashAggregate [codegen id : 9] -Input [2]: [cs_ext_sales_price#22, i_item_id#27] -Keys [1]: [i_item_id#27] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#27, sum#29] - -(43) CometColumnarExchange -Input [2]: [i_item_id#27, sum#29] -Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(44) CometColumnarToRow [codegen id : 10] -Input [2]: [i_item_id#27, sum#29] - -(45) HashAggregate [codegen id : 10] -Input [2]: [i_item_id#27, sum#29] -Keys [1]: [i_item_id#27] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] -Results [2]: [i_item_id#27, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS total_sales#31] - -(46) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(47) ColumnarToRow [codegen id : 14] -Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] - -(48) Filter [codegen id : 14] -Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] -Condition : (isnotnull(ws_bill_addr_sk#33) AND isnotnull(ws_item_sk#32)) - -(49) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#36] - -(50) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#36] -Join type: Inner -Join condition: None - -(51) Project [codegen id : 14] -Output [3]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34] -Input [5]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35, d_date_sk#36] - -(52) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#37] - -(53) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#33] -Right keys [1]: [ca_address_sk#37] -Join type: Inner -Join condition: None - -(54) Project [codegen id : 14] -Output [2]: [ws_item_sk#32, ws_ext_sales_price#34] -Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ca_address_sk#37] - -(55) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#38, i_item_id#39] - -(56) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_item_sk#32] -Right keys [1]: [i_item_sk#38] -Join type: Inner -Join condition: None - -(57) Project [codegen id : 14] -Output [2]: [ws_ext_sales_price#34, i_item_id#39] -Input [4]: [ws_item_sk#32, ws_ext_sales_price#34, i_item_sk#38, i_item_id#39] - -(58) HashAggregate [codegen id : 14] -Input [2]: [ws_ext_sales_price#34, i_item_id#39] -Keys [1]: [i_item_id#39] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#34))] -Aggregate Attributes [1]: [sum#40] -Results [2]: [i_item_id#39, sum#41] - -(59) CometColumnarExchange -Input [2]: [i_item_id#39, sum#41] -Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(60) CometColumnarToRow [codegen id : 15] -Input [2]: [i_item_id#39, sum#41] - -(61) HashAggregate [codegen id : 15] -Input [2]: [i_item_id#39, sum#41] -Keys [1]: [i_item_id#39] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#34))#42] -Results [2]: [i_item_id#39, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#34))#42,17,2) AS total_sales#43] - -(62) Union - -(63) HashAggregate [codegen id : 16] -Input [2]: [i_item_id#18, total_sales#19] -Keys [1]: [i_item_id#18] -Functions [1]: [partial_sum(total_sales#19)] -Aggregate Attributes [2]: [sum#44, isEmpty#45] -Results [3]: [i_item_id#18, sum#46, isEmpty#47] - -(64) CometColumnarExchange -Input [3]: [i_item_id#18, sum#46, isEmpty#47] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(65) CometColumnarToRow [codegen id : 17] -Input [3]: [i_item_id#18, sum#46, isEmpty#47] - -(66) HashAggregate [codegen id : 17] -Input [3]: [i_item_id#18, sum#46, isEmpty#47] -Keys [1]: [i_item_id#18] -Functions [1]: [sum(total_sales#19)] -Aggregate Attributes [1]: [sum(total_sales#19)#48] -Results [2]: [i_item_id#18, sum(total_sales#19)#48 AS total_sales#49] - -(67) TakeOrderedAndProject -Input [2]: [i_item_id#18, total_sales#49] -Arguments: 100, [total_sales#49 ASC NULLS FIRST], [i_item_id#18, total_sales#49] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (72) -+- * CometColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometNativeScan parquet spark_catalog.default.date_dim (68) - - -(68) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#50, d_moy#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [3]: [d_date_sk#6, d_year#50, d_moy#51] -Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 2001)) AND (d_moy#51 = 2)) AND isnotnull(d_date_sk#6)) - -(70) CometProject -Input [3]: [d_date_sk#6, d_year#50, d_moy#51] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(71) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(72) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/simplified.txt deleted file mode 100644 index 81d90bdfb2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/simplified.txt +++ /dev/null @@ -1,106 +0,0 @@ -TakeOrderedAndProject [total_sales,i_item_id] - WholeStageCodegen (17) - HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (16) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),i_item_id,total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #6 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_color] - CometNativeScan parquet spark_catalog.default.item [i_item_id,i_color] - WholeStageCodegen (10) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #7 - WholeStageCodegen (9) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_addr_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (15) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #8 - WholeStageCodegen (14) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/explain.txt deleted file mode 100644 index 5562e86b91..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,385 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (62) -+- CometTakeOrderedAndProject (61) - +- CometHashAggregate (60) - +- CometExchange (59) - +- CometHashAggregate (58) - +- CometUnion (57) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- CometHashAggregate (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometFilter (30) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- CometHashAggregate (56) - +- CometExchange (55) - +- CometHashAggregate (54) - +- CometProject (53) - +- CometBroadcastHashJoin (52) - :- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometFilter (44) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (43) - : : +- ReusedExchange (45) - : +- ReusedExchange (48) - +- ReusedExchange (51) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001)) AND (d_moy#8 = 2)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(11) CometProject -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Arguments: [ca_address_sk#9], [ca_address_sk#9] - -(12) CometBroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: [ca_address_sk#9] - -(13) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Right output [1]: [ca_address_sk#9] -Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] -Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#11, i_item_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#11, i_item_id#12] -Condition : isnotnull(i_item_sk#11) - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_id#13, i_color#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(18) CometFilter -Input [2]: [i_item_id#13, i_color#14] -Condition : staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#14, 20, true, false, true) IN (slate ,blanched ,burnished ) - -(19) CometProject -Input [2]: [i_item_id#13, i_color#14] -Arguments: [i_item_id#15], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#13, 16, true, false, true) AS i_item_id#15] - -(20) CometBroadcastExchange -Input [1]: [i_item_id#15] -Arguments: [i_item_id#15] - -(21) CometBroadcastHashJoin -Left output [2]: [i_item_sk#11, i_item_id#12] -Right output [1]: [i_item_id#15] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#12, 16, true, false, true)], [i_item_id#15], LeftSemi, BuildRight - -(22) CometProject -Input [2]: [i_item_sk#11, i_item_id#12] -Arguments: [i_item_sk#11, i_item_id#16], [i_item_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#12, 16, true, false, true) AS i_item_id#16] - -(23) CometBroadcastExchange -Input [2]: [i_item_sk#11, i_item_id#16] -Arguments: [i_item_sk#11, i_item_id#16] - -(24) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Right output [2]: [i_item_sk#11, i_item_id#16] -Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight - -(25) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_item_id#16] -Arguments: [ss_ext_sales_price#3, i_item_id#16], [ss_ext_sales_price#3, i_item_id#16] - -(26) CometHashAggregate -Input [2]: [ss_ext_sales_price#3, i_item_id#16] -Keys [1]: [i_item_id#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(27) CometExchange -Input [2]: [i_item_id#16, sum#17] -Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [2]: [i_item_id#16, sum#17] -Keys [1]: [i_item_id#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Condition : (isnotnull(cs_bill_addr_sk#18) AND isnotnull(cs_item_sk#19)) - -(31) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#23] - -(32) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Right output [1]: [d_date_sk#23] -Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(33) CometProject -Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] -Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] - -(34) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#24] - -(35) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -Right output [1]: [ca_address_sk#24] -Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner, BuildRight - -(36) CometProject -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] -Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] - -(37) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#25, i_item_id#26] - -(38) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] -Right output [2]: [i_item_sk#25, i_item_id#26] -Arguments: [cs_item_sk#19], [i_item_sk#25], Inner, BuildRight - -(39) CometProject -Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_item_id#26] -Arguments: [cs_ext_sales_price#20, i_item_id#26], [cs_ext_sales_price#20, i_item_id#26] - -(40) CometHashAggregate -Input [2]: [cs_ext_sales_price#20, i_item_id#26] -Keys [1]: [i_item_id#26] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] - -(41) CometExchange -Input [2]: [i_item_id#26, sum#27] -Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(42) CometHashAggregate -Input [2]: [i_item_id#26, sum#27] -Keys [1]: [i_item_id#26] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(44) CometFilter -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Condition : (isnotnull(ws_bill_addr_sk#29) AND isnotnull(ws_item_sk#28)) - -(45) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(46) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [ws_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(47) CometProject -Input [5]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31, d_date_sk#33] -Arguments: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30], [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] - -(48) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#34] - -(49) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] -Right output [1]: [ca_address_sk#34] -Arguments: [ws_bill_addr_sk#29], [ca_address_sk#34], Inner, BuildRight - -(50) CometProject -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ca_address_sk#34] -Arguments: [ws_item_sk#28, ws_ext_sales_price#30], [ws_item_sk#28, ws_ext_sales_price#30] - -(51) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#35, i_item_id#36] - -(52) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#28, ws_ext_sales_price#30] -Right output [2]: [i_item_sk#35, i_item_id#36] -Arguments: [ws_item_sk#28], [i_item_sk#35], Inner, BuildRight - -(53) CometProject -Input [4]: [ws_item_sk#28, ws_ext_sales_price#30, i_item_sk#35, i_item_id#36] -Arguments: [ws_ext_sales_price#30, i_item_id#36], [ws_ext_sales_price#30, i_item_id#36] - -(54) CometHashAggregate -Input [2]: [ws_ext_sales_price#30, i_item_id#36] -Keys [1]: [i_item_id#36] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#30))] - -(55) CometExchange -Input [2]: [i_item_id#36, sum#37] -Arguments: hashpartitioning(i_item_id#36, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(56) CometHashAggregate -Input [2]: [i_item_id#36, sum#37] -Keys [1]: [i_item_id#36] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#30))] - -(57) CometUnion -Child 0 Input [2]: [i_item_id#38, total_sales#39] -Child 1 Input [2]: [i_item_id#26, total_sales#40] -Child 2 Input [2]: [i_item_id#36, total_sales#41] - -(58) CometHashAggregate -Input [2]: [i_item_id#38, total_sales#39] -Keys [1]: [i_item_id#38] -Functions [1]: [partial_sum(total_sales#39)] - -(59) CometExchange -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(60) CometHashAggregate -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Keys [1]: [i_item_id#38] -Functions [1]: [sum(total_sales#39)] - -(61) CometTakeOrderedAndProject -Input [2]: [i_item_id#38, total_sales#44] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#44 ASC NULLS FIRST], output=[i_item_id#38,total_sales#44]), [i_item_id#38, total_sales#44], 100, 0, [total_sales#44 ASC NULLS FIRST], [i_item_id#38, total_sales#44] - -(62) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_id#38, total_sales#44] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (67) -+- * CometColumnarToRow (66) - +- CometProject (65) - +- CometFilter (64) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) - - -(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(64) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001)) AND (d_moy#8 = 2)) AND isnotnull(d_date_sk#6)) - -(65) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(66) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(67) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/simplified.txt deleted file mode 100644 index 19eca782cd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,total_sales] - CometHashAggregate [sum,isEmpty] [i_item_id,total_sales,sum(total_sales)] - CometExchange [i_item_id] #1 - CometHashAggregate [total_sales] [i_item_id,sum,isEmpty] - CometUnion [i_item_id,total_sales] - CometHashAggregate [sum] [i_item_id,total_sales,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #7 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_color] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_color] - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - 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,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt deleted file mode 100644 index 5562e86b91..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ /dev/null @@ -1,385 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (62) -+- CometTakeOrderedAndProject (61) - +- CometHashAggregate (60) - +- CometExchange (59) - +- CometHashAggregate (58) - +- CometUnion (57) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- CometHashAggregate (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometFilter (30) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- CometHashAggregate (56) - +- CometExchange (55) - +- CometHashAggregate (54) - +- CometProject (53) - +- CometBroadcastHashJoin (52) - :- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometFilter (44) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (43) - : : +- ReusedExchange (45) - : +- ReusedExchange (48) - +- ReusedExchange (51) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001)) AND (d_moy#8 = 2)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(11) CometProject -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Arguments: [ca_address_sk#9], [ca_address_sk#9] - -(12) CometBroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: [ca_address_sk#9] - -(13) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Right output [1]: [ca_address_sk#9] -Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] -Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#11, i_item_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#11, i_item_id#12] -Condition : isnotnull(i_item_sk#11) - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_id#13, i_color#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(18) CometFilter -Input [2]: [i_item_id#13, i_color#14] -Condition : staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#14, 20, true, false, true) IN (slate ,blanched ,burnished ) - -(19) CometProject -Input [2]: [i_item_id#13, i_color#14] -Arguments: [i_item_id#15], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#13, 16, true, false, true) AS i_item_id#15] - -(20) CometBroadcastExchange -Input [1]: [i_item_id#15] -Arguments: [i_item_id#15] - -(21) CometBroadcastHashJoin -Left output [2]: [i_item_sk#11, i_item_id#12] -Right output [1]: [i_item_id#15] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#12, 16, true, false, true)], [i_item_id#15], LeftSemi, BuildRight - -(22) CometProject -Input [2]: [i_item_sk#11, i_item_id#12] -Arguments: [i_item_sk#11, i_item_id#16], [i_item_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#12, 16, true, false, true) AS i_item_id#16] - -(23) CometBroadcastExchange -Input [2]: [i_item_sk#11, i_item_id#16] -Arguments: [i_item_sk#11, i_item_id#16] - -(24) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Right output [2]: [i_item_sk#11, i_item_id#16] -Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight - -(25) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_item_id#16] -Arguments: [ss_ext_sales_price#3, i_item_id#16], [ss_ext_sales_price#3, i_item_id#16] - -(26) CometHashAggregate -Input [2]: [ss_ext_sales_price#3, i_item_id#16] -Keys [1]: [i_item_id#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(27) CometExchange -Input [2]: [i_item_id#16, sum#17] -Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [2]: [i_item_id#16, sum#17] -Keys [1]: [i_item_id#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Condition : (isnotnull(cs_bill_addr_sk#18) AND isnotnull(cs_item_sk#19)) - -(31) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#23] - -(32) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Right output [1]: [d_date_sk#23] -Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(33) CometProject -Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] -Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] - -(34) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#24] - -(35) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -Right output [1]: [ca_address_sk#24] -Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner, BuildRight - -(36) CometProject -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] -Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] - -(37) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#25, i_item_id#26] - -(38) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] -Right output [2]: [i_item_sk#25, i_item_id#26] -Arguments: [cs_item_sk#19], [i_item_sk#25], Inner, BuildRight - -(39) CometProject -Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_item_id#26] -Arguments: [cs_ext_sales_price#20, i_item_id#26], [cs_ext_sales_price#20, i_item_id#26] - -(40) CometHashAggregate -Input [2]: [cs_ext_sales_price#20, i_item_id#26] -Keys [1]: [i_item_id#26] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] - -(41) CometExchange -Input [2]: [i_item_id#26, sum#27] -Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(42) CometHashAggregate -Input [2]: [i_item_id#26, sum#27] -Keys [1]: [i_item_id#26] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(44) CometFilter -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Condition : (isnotnull(ws_bill_addr_sk#29) AND isnotnull(ws_item_sk#28)) - -(45) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(46) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [ws_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(47) CometProject -Input [5]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31, d_date_sk#33] -Arguments: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30], [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] - -(48) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#34] - -(49) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] -Right output [1]: [ca_address_sk#34] -Arguments: [ws_bill_addr_sk#29], [ca_address_sk#34], Inner, BuildRight - -(50) CometProject -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ca_address_sk#34] -Arguments: [ws_item_sk#28, ws_ext_sales_price#30], [ws_item_sk#28, ws_ext_sales_price#30] - -(51) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#35, i_item_id#36] - -(52) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#28, ws_ext_sales_price#30] -Right output [2]: [i_item_sk#35, i_item_id#36] -Arguments: [ws_item_sk#28], [i_item_sk#35], Inner, BuildRight - -(53) CometProject -Input [4]: [ws_item_sk#28, ws_ext_sales_price#30, i_item_sk#35, i_item_id#36] -Arguments: [ws_ext_sales_price#30, i_item_id#36], [ws_ext_sales_price#30, i_item_id#36] - -(54) CometHashAggregate -Input [2]: [ws_ext_sales_price#30, i_item_id#36] -Keys [1]: [i_item_id#36] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#30))] - -(55) CometExchange -Input [2]: [i_item_id#36, sum#37] -Arguments: hashpartitioning(i_item_id#36, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(56) CometHashAggregate -Input [2]: [i_item_id#36, sum#37] -Keys [1]: [i_item_id#36] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#30))] - -(57) CometUnion -Child 0 Input [2]: [i_item_id#38, total_sales#39] -Child 1 Input [2]: [i_item_id#26, total_sales#40] -Child 2 Input [2]: [i_item_id#36, total_sales#41] - -(58) CometHashAggregate -Input [2]: [i_item_id#38, total_sales#39] -Keys [1]: [i_item_id#38] -Functions [1]: [partial_sum(total_sales#39)] - -(59) CometExchange -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(60) CometHashAggregate -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Keys [1]: [i_item_id#38] -Functions [1]: [sum(total_sales#39)] - -(61) CometTakeOrderedAndProject -Input [2]: [i_item_id#38, total_sales#44] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#44 ASC NULLS FIRST], output=[i_item_id#38,total_sales#44]), [i_item_id#38, total_sales#44], 100, 0, [total_sales#44 ASC NULLS FIRST], [i_item_id#38, total_sales#44] - -(62) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_id#38, total_sales#44] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (67) -+- * CometColumnarToRow (66) - +- CometProject (65) - +- CometFilter (64) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) - - -(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] -ReadSchema: struct - -(64) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001)) AND (d_moy#8 = 2)) AND isnotnull(d_date_sk#6)) - -(65) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(66) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(67) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/extended.txt deleted file mode 100644 index 36b9a8306d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/extended.txt +++ /dev/null @@ -1,100 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 19eca782cd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,total_sales] - CometHashAggregate [sum,isEmpty] [i_item_id,total_sales,sum(total_sales)] - CometExchange [i_item_id] #1 - CometHashAggregate [total_sales] [i_item_id,sum,isEmpty] - CometUnion [i_item_id,total_sales] - CometHashAggregate [sum] [i_item_id,total_sales,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #7 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_color] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_color] - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - 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,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/explain.txt deleted file mode 100644 index c3ba92766c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/explain.txt +++ /dev/null @@ -1,304 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (51) -+- * Project (50) - +- * BroadcastHashJoin Inner BuildRight (49) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * CometColumnarToRow (26) - : : +- CometSort (25) - : : +- CometColumnarExchange (24) - : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- * HashAggregate (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- Window (38) - : +- * CometColumnarToRow (37) - : +- CometSort (36) - : +- CometColumnarExchange (35) - : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) - : +- ReusedExchange (32) - +- BroadcastExchange (48) - +- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- ReusedExchange (43) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#4, i_category#5] - -(5) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(7) Filter [codegen id : 1] -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) - -(8) BroadcastExchange -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#7] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(11) ReusedExchange [Reuses operator id: 55] -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] - -(14) CometNativeScan parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#14, cc_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#14, cc_name#15] -Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) - -(16) CometColumnarToRow [codegen id : 3] -Input [2]: [cc_call_center_sk#14, cc_name#15] - -(17) BroadcastExchange -Input [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_call_center_sk#6] -Right keys [1]: [cc_call_center_sk#14] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] - -(20) HashAggregate [codegen id : 4] -Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] -Aggregate Attributes [1]: [sum#16] -Results [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] - -(21) CometColumnarExchange -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] - -(23) HashAggregate [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#8))#18] -Results [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#18,17,2) AS sum_sales#19, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#18,17,2) AS _w0#20] - -(24) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) CometColumnarToRow [codegen id : 6] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] - -(27) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(28) Filter [codegen id : 7] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(29) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] -Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(30) Filter [codegen id : 22] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] -Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) - -(31) Project [codegen id : 22] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] - -(32) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] - -(33) CometColumnarToRow [codegen id : 12] -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] - -(34) HashAggregate [codegen id : 12] -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] -Keys [5]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#27))#18] -Results [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#18,17,2) AS sum_sales#19] - -(35) CometColumnarExchange -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(36) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] -Arguments: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(37) CometColumnarToRow [codegen id : 13] -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] - -(38) Window -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(39) Project [codegen id : 14] -Output [5]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, cc_name#23, sum_sales#19 AS sum_sales#31, rn#28] -Input [7]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19, rn#28] - -(40) BroadcastExchange -Input [5]: [i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] - -(41) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] -Right keys [4]: [i_category#29, i_brand#30, cc_name#23, (rn#28 + 1)] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 22] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] - -(43) ReusedExchange [Reuses operator id: 35] -Output [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] - -(44) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] -Arguments: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] - -(45) CometColumnarToRow [codegen id : 20] -Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] - -(46) Window -Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] -Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#5, i_brand#4, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#5, i_brand#4, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] - -(47) Project [codegen id : 21] -Output [5]: [i_category#5 AS i_category#36, i_brand#4 AS i_brand#37, cc_name#32, sum_sales#19 AS sum_sales#38, rn#35] -Input [7]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19, rn#35] - -(48) BroadcastExchange -Input [5]: [i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] - -(49) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] -Right keys [4]: [i_category#36, i_brand#37, cc_name#32, (rn#35 - 1)] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 22] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#39, sum_sales#38 AS nsum#40] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] - -(51) TakeOrderedAndProject -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] -Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometFilter (53) - +- CometNativeScan parquet spark_catalog.default.date_dim (52) - - -(52) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(54) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(55) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/simplified.txt deleted file mode 100644 index 4f9ac35f4d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/simplified.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (22) - Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (7) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [cs_item_sk,cs_call_center_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk,cc_name] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/explain.txt deleted file mode 100644 index eb21a3abd7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) - -(6) CometBroadcastExchange -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_item_sk#1], [cs_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#14, cc_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#14, cc_name#15] -Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) - -(16) CometBroadcastExchange -Input [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cc_call_center_sk#14, cc_name#15] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] -Right output [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cs_call_center_sk#6], [cc_call_center_sk#14], Inner, BuildRight - -(18) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] -Arguments: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15], [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] - -(19) CometHashAggregate -Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] - -(20) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] - -(22) CometExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] - -(25) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(28) Filter [codegen id : 7] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] - -(30) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] - -(31) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] -Keys [5]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23] -Functions [1]: [sum(UnscaledValue(cs_sales_price#25))] - -(32) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] - -(35) Window -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [5]: [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] -Input [7]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26] - -(37) BroadcastExchange -Input [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] - -(40) ReusedExchange [Reuses operator id: 32] -Output [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(41) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(43) Window -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#5, i_brand#4, cc_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#5, i_brand#4, cc_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [5]: [i_category#5 AS i_category#34, i_brand#4 AS i_brand#35, cc_name#30, sum_sales#17 AS sum_sales#36, rn#33] -Input [7]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17, rn#33] - -(45) BroadcastExchange -Input [5]: [i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#34, i_brand#35, cc_name#30, (rn#33 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] - -(48) TakeOrderedAndProject -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/simplified.txt deleted file mode 100644 index b854e818db..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - 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 [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,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt deleted file mode 100644 index eb21a3abd7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) - -(6) CometBroadcastExchange -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_item_sk#1], [cs_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#14, cc_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#14, cc_name#15] -Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) - -(16) CometBroadcastExchange -Input [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cc_call_center_sk#14, cc_name#15] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] -Right output [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cs_call_center_sk#6], [cc_call_center_sk#14], Inner, BuildRight - -(18) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] -Arguments: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15], [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] - -(19) CometHashAggregate -Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] - -(20) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] - -(22) CometExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] - -(25) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(28) Filter [codegen id : 7] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] - -(30) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] - -(31) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] -Keys [5]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23] -Functions [1]: [sum(UnscaledValue(cs_sales_price#25))] - -(32) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] - -(35) Window -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [5]: [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] -Input [7]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26] - -(37) BroadcastExchange -Input [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] - -(40) ReusedExchange [Reuses operator id: 32] -Output [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(41) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(43) Window -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#5, i_brand#4, cc_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#5, i_brand#4, cc_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [5]: [i_category#5 AS i_category#34, i_brand#4 AS i_brand#35, cc_name#30, sum_sales#17 AS sum_sales#36, rn#33] -Input [7]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17, rn#33] - -(45) BroadcastExchange -Input [5]: [i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#34, i_brand#35, cc_name#30, (rn#33 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] - -(48) TakeOrderedAndProject -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/extended.txt deleted file mode 100644 index de7d26cd06..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index b854e818db..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - 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 [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,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/explain.txt deleted file mode 100644 index 7d820f4024..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/explain.txt +++ /dev/null @@ -1,455 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (53) -+- * Project (52) - +- * BroadcastHashJoin Inner BuildRight (51) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Filter (18) - : : +- * HashAggregate (17) - : : +- * CometColumnarToRow (16) - : : +- CometColumnarExchange (15) - : : +- * HashAggregate (14) - : : +- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (33) - : +- * Filter (32) - : +- * HashAggregate (31) - : +- * CometColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- * HashAggregate (28) - : +- * Project (27) - : +- * BroadcastHashJoin Inner BuildRight (26) - : :- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Filter (21) - : : : +- * ColumnarToRow (20) - : : : +- Scan parquet spark_catalog.default.catalog_sales (19) - : : +- ReusedExchange (22) - : +- ReusedExchange (25) - +- BroadcastExchange (50) - +- * Filter (49) - +- * HashAggregate (48) - +- * CometColumnarToRow (47) - +- CometColumnarExchange (46) - +- * HashAggregate (45) - +- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Filter (38) - : : +- * ColumnarToRow (37) - : : +- Scan parquet spark_catalog.default.web_sales (36) - : +- ReusedExchange (39) - +- ReusedExchange (42) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_item_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [i_item_sk#5, i_item_id#6] -Condition : (isnotnull(i_item_sk#5) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true))) - -(6) CometProject -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#7] - -(7) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_sk#5, i_item_id#7] - -(8) BroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7] -Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#7] - -(11) ReusedExchange [Reuses operator id: 63] -Output [1]: [d_date_sk#8] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#8] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [2]: [ss_ext_sales_price#2, i_item_id#7] -Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7, d_date_sk#8] - -(14) HashAggregate [codegen id : 3] -Input [2]: [ss_ext_sales_price#2, i_item_id#7] -Keys [1]: [i_item_id#7] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#9] -Results [2]: [i_item_id#7, sum#10] - -(15) CometColumnarExchange -Input [2]: [i_item_id#7, sum#10] -Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 12] -Input [2]: [i_item_id#7, sum#10] - -(17) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#7, sum#10] -Keys [1]: [i_item_id#7] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#11] -Results [2]: [i_item_id#7 AS item_id#12, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#11,17,2) AS ss_item_rev#13] - -(18) Filter [codegen id : 12] -Input [2]: [item_id#12, ss_item_rev#13] -Condition : isnotnull(ss_item_rev#13) - -(19) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(20) ColumnarToRow [codegen id : 6] -Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] - -(21) Filter [codegen id : 6] -Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#14) - -(22) ReusedExchange [Reuses operator id: 8] -Output [2]: [i_item_sk#18, i_item_id#19] - -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_item_sk#14] -Right keys [1]: [i_item_sk#18] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 6] -Output [3]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19] -Input [5]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_sk#18, i_item_id#19] - -(25) ReusedExchange [Reuses operator id: 77] -Output [1]: [d_date_sk#20] - -(26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 6] -Output [2]: [cs_ext_sales_price#15, i_item_id#19] -Input [4]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19, d_date_sk#20] - -(28) HashAggregate [codegen id : 6] -Input [2]: [cs_ext_sales_price#15, i_item_id#19] -Keys [1]: [i_item_id#19] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#15))] -Aggregate Attributes [1]: [sum#21] -Results [2]: [i_item_id#19, sum#22] - -(29) CometColumnarExchange -Input [2]: [i_item_id#19, sum#22] -Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(30) CometColumnarToRow [codegen id : 7] -Input [2]: [i_item_id#19, sum#22] - -(31) HashAggregate [codegen id : 7] -Input [2]: [i_item_id#19, sum#22] -Keys [1]: [i_item_id#19] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#15))#23] -Results [2]: [i_item_id#19 AS item_id#24, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#15))#23,17,2) AS cs_item_rev#25] - -(32) Filter [codegen id : 7] -Input [2]: [item_id#24, cs_item_rev#25] -Condition : isnotnull(cs_item_rev#25) - -(33) BroadcastExchange -Input [2]: [item_id#24, cs_item_rev#25] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] - -(34) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [item_id#12] -Right keys [1]: [item_id#24] -Join type: Inner -Join condition: ((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * cs_item_rev#25)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * cs_item_rev#25))) AND (cast(cs_item_rev#25 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(cs_item_rev#25 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) - -(35) Project [codegen id : 12] -Output [3]: [item_id#12, ss_item_rev#13, cs_item_rev#25] -Input [4]: [item_id#12, ss_item_rev#13, item_id#24, cs_item_rev#25] - -(36) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#26, ws_ext_sales_price#27, ws_sold_date_sk#28] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#28), dynamicpruningexpression(ws_sold_date_sk#28 IN dynamicpruning#17)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(37) ColumnarToRow [codegen id : 10] -Input [3]: [ws_item_sk#26, ws_ext_sales_price#27, ws_sold_date_sk#28] - -(38) Filter [codegen id : 10] -Input [3]: [ws_item_sk#26, ws_ext_sales_price#27, ws_sold_date_sk#28] -Condition : isnotnull(ws_item_sk#26) - -(39) ReusedExchange [Reuses operator id: 8] -Output [2]: [i_item_sk#29, i_item_id#30] - -(40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [i_item_sk#29] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 10] -Output [3]: [ws_ext_sales_price#27, ws_sold_date_sk#28, i_item_id#30] -Input [5]: [ws_item_sk#26, ws_ext_sales_price#27, ws_sold_date_sk#28, i_item_sk#29, i_item_id#30] - -(42) ReusedExchange [Reuses operator id: 77] -Output [1]: [d_date_sk#31] - -(43) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#28] -Right keys [1]: [d_date_sk#31] -Join type: Inner -Join condition: None - -(44) Project [codegen id : 10] -Output [2]: [ws_ext_sales_price#27, i_item_id#30] -Input [4]: [ws_ext_sales_price#27, ws_sold_date_sk#28, i_item_id#30, d_date_sk#31] - -(45) HashAggregate [codegen id : 10] -Input [2]: [ws_ext_sales_price#27, i_item_id#30] -Keys [1]: [i_item_id#30] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#27))] -Aggregate Attributes [1]: [sum#32] -Results [2]: [i_item_id#30, sum#33] - -(46) CometColumnarExchange -Input [2]: [i_item_id#30, sum#33] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(47) CometColumnarToRow [codegen id : 11] -Input [2]: [i_item_id#30, sum#33] - -(48) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#30, sum#33] -Keys [1]: [i_item_id#30] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#27))#34] -Results [2]: [i_item_id#30 AS item_id#35, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#27))#34,17,2) AS ws_item_rev#36] - -(49) Filter [codegen id : 11] -Input [2]: [item_id#35, ws_item_rev#36] -Condition : isnotnull(ws_item_rev#36) - -(50) BroadcastExchange -Input [2]: [item_id#35, ws_item_rev#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] - -(51) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [item_id#12] -Right keys [1]: [item_id#35] -Join type: Inner -Join condition: ((((((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * ws_item_rev#36)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * ws_item_rev#36))) AND (cast(cs_item_rev#25 as decimal(19,3)) >= (0.9 * ws_item_rev#36))) AND (cast(cs_item_rev#25 as decimal(20,3)) <= (1.1 * ws_item_rev#36))) AND (cast(ws_item_rev#36 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(ws_item_rev#36 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) AND (cast(ws_item_rev#36 as decimal(19,3)) >= (0.9 * cs_item_rev#25))) AND (cast(ws_item_rev#36 as decimal(20,3)) <= (1.1 * cs_item_rev#25))) - -(52) Project [codegen id : 12] -Output [8]: [item_id#12, ss_item_rev#13, (((ss_item_rev#13 / ((ss_item_rev#13 + cs_item_rev#25) + ws_item_rev#36)) / 3) * 100) AS ss_dev#37, cs_item_rev#25, (((cs_item_rev#25 / ((ss_item_rev#13 + cs_item_rev#25) + ws_item_rev#36)) / 3) * 100) AS cs_dev#38, ws_item_rev#36, (((ws_item_rev#36 / ((ss_item_rev#13 + cs_item_rev#25) + ws_item_rev#36)) / 3) * 100) AS ws_dev#39, (((ss_item_rev#13 + cs_item_rev#25) + ws_item_rev#36) / 3) AS average#40] -Input [5]: [item_id#12, ss_item_rev#13, cs_item_rev#25, item_id#35, ws_item_rev#36] - -(53) TakeOrderedAndProject -Input [8]: [item_id#12, ss_item_rev#13, ss_dev#37, cs_item_rev#25, cs_dev#38, ws_item_rev#36, ws_dev#39, average#40] -Arguments: 100, [item_id#12 ASC NULLS FIRST, ss_item_rev#13 ASC NULLS FIRST], [item_id#12, ss_item_rev#13, ss_dev#37, cs_item_rev#25, cs_dev#38, ws_item_rev#36, ws_dev#39, average#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (63) -+- * CometColumnarToRow (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometFilter (55) - : +- CometNativeScan parquet spark_catalog.default.date_dim (54) - +- CometBroadcastExchange (59) - +- CometProject (58) - +- CometFilter (57) - +- CometNativeScan parquet spark_catalog.default.date_dim (56) - - -(54) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#41] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(55) CometFilter -Input [2]: [d_date_sk#8, d_date#41] -Condition : isnotnull(d_date_sk#8) - -(56) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date#41, d_week_seq#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] -ReadSchema: struct - -(57) CometFilter -Input [2]: [d_date#41, d_week_seq#42] -Condition : (isnotnull(d_week_seq#42) AND (d_week_seq#42 = Subquery scalar-subquery#43, [id=#44])) - -(58) CometProject -Input [2]: [d_date#41, d_week_seq#42] -Arguments: [d_date#41#45], [d_date#41 AS d_date#41#45] - -(59) CometBroadcastExchange -Input [1]: [d_date#41#45] -Arguments: [d_date#41#45] - -(60) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#41] -Right output [1]: [d_date#41#45] -Arguments: [d_date#41], [d_date#41#45], LeftSemi, BuildRight - -(61) CometProject -Input [2]: [d_date_sk#8, d_date#41] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(62) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(63) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 57 Hosting Expression = Subquery scalar-subquery#43, [id=#44] -* CometColumnarToRow (67) -+- CometProject (66) - +- CometFilter (65) - +- CometNativeScan parquet spark_catalog.default.date_dim (64) - - -(64) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date#41, d_week_seq#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] -ReadSchema: struct - -(65) CometFilter -Input [2]: [d_date#41, d_week_seq#42] -Condition : (isnotnull(d_date#41) AND (d_date#41 = 2000-01-03)) - -(66) CometProject -Input [2]: [d_date#41, d_week_seq#42] -Arguments: [d_week_seq#42], [d_week_seq#42] - -(67) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#42] - -Subquery:3 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#17 -BroadcastExchange (77) -+- * CometColumnarToRow (76) - +- CometProject (75) - +- CometBroadcastHashJoin (74) - :- CometFilter (69) - : +- CometNativeScan parquet spark_catalog.default.date_dim (68) - +- CometBroadcastExchange (73) - +- CometProject (72) - +- CometFilter (71) - +- CometNativeScan parquet spark_catalog.default.date_dim (70) - - -(68) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_date#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [2]: [d_date_sk#20, d_date#46] -Condition : isnotnull(d_date_sk#20) - -(70) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date#41, d_week_seq#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] -ReadSchema: struct - -(71) CometFilter -Input [2]: [d_date#41, d_week_seq#42] -Condition : (isnotnull(d_week_seq#42) AND (d_week_seq#42 = ReusedSubquery Subquery scalar-subquery#43, [id=#44])) - -(72) CometProject -Input [2]: [d_date#41, d_week_seq#42] -Arguments: [d_date#41], [d_date#41] - -(73) CometBroadcastExchange -Input [1]: [d_date#41] -Arguments: [d_date#41] - -(74) CometBroadcastHashJoin -Left output [2]: [d_date_sk#20, d_date#46] -Right output [1]: [d_date#41] -Arguments: [d_date#46], [d_date#41], LeftSemi, BuildRight - -(75) CometProject -Input [2]: [d_date_sk#20, d_date#46] -Arguments: [d_date_sk#20], [d_date_sk#20] - -(76) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#20] - -(77) BroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:4 Hosting operator id = 71 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#44] - -Subquery:5 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#28 IN dynamicpruning#17 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/simplified.txt deleted file mode 100644 index ab910f96e0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/simplified.txt +++ /dev/null @@ -1,110 +0,0 @@ -TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - WholeStageCodegen (12) - Project [item_id,ss_item_rev,cs_item_rev,ws_item_rev] - BroadcastHashJoin [item_id,item_id,ss_item_rev,ws_item_rev,cs_item_rev] - Project [item_id,ss_item_rev,cs_item_rev] - BroadcastHashJoin [item_id,item_id,ss_item_rev,cs_item_rev] - Filter [ss_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (3) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] [d_date] - CometFilter [d_date,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - Filter [cs_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #6 - WholeStageCodegen (6) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #8 - CometProject [d_date] - CometFilter [d_date,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #4 - InputAdapter - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (11) - Filter [ws_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #10 - WholeStageCodegen (10) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #4 - InputAdapter - ReusedExchange [d_date_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/explain.txt deleted file mode 100644 index 85c8622ee4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,469 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (57) -+- CometTakeOrderedAndProject (56) - +- CometProject (55) - +- CometBroadcastHashJoin (54) - :- CometProject (40) - : +- CometBroadcastHashJoin (39) - : :- CometFilter (20) - : : +- CometHashAggregate (19) - : : +- CometExchange (18) - : : +- CometHashAggregate (17) - : : +- CometProject (16) - : : +- CometBroadcastHashJoin (15) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (14) - : : +- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- ReusedExchange (11) - : +- CometBroadcastExchange (38) - : +- CometFilter (37) - : +- CometHashAggregate (36) - : +- CometExchange (35) - : +- CometHashAggregate (34) - : +- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (21) - : : +- ReusedExchange (23) - : +- CometBroadcastExchange (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- ReusedExchange (28) - +- CometBroadcastExchange (53) - +- CometFilter (52) - +- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometBroadcastHashJoin (47) - :- CometProject (45) - : +- CometBroadcastHashJoin (44) - : :- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (41) - : +- ReusedExchange (43) - +- ReusedExchange (46) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_item_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_item_id#6] -Condition : (isnotnull(i_item_sk#5) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true))) - -(5) CometProject -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#7] - -(6) CometBroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#7] -Arguments: [i_item_sk#5, i_item_id#7] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [2]: [i_item_sk#5, i_item_id#7] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#7] -Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(11) ReusedExchange [Reuses operator id: 63] -Output [1]: [d_date#9#10] - -(12) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#9#10] -Arguments: [d_date#9], [d_date#9#10], LeftSemi, BuildRight - -(13) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: [d_date_sk#8] - -(15) CometBroadcastHashJoin -Left output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7] -Right output [1]: [d_date_sk#8] -Arguments: [ss_sold_date_sk#3], [d_date_sk#8], Inner, BuildRight - -(16) CometProject -Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7, d_date_sk#8] -Arguments: [ss_ext_sales_price#2, i_item_id#7], [ss_ext_sales_price#2, i_item_id#7] - -(17) CometHashAggregate -Input [2]: [ss_ext_sales_price#2, i_item_id#7] -Keys [1]: [i_item_id#7] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(18) CometExchange -Input [2]: [i_item_id#7, sum#11] -Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [2]: [i_item_id#7, sum#11] -Keys [1]: [i_item_id#7] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(20) CometFilter -Input [2]: [item_id#12, ss_item_rev#13] -Condition : isnotnull(ss_item_rev#13) - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#14) - -(23) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#18, i_item_id#19] - -(24) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Right output [2]: [i_item_sk#18, i_item_id#19] -Arguments: [cs_item_sk#14], [i_item_sk#18], Inner, BuildRight - -(25) CometProject -Input [5]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_sk#18, i_item_id#19] -Arguments: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19], [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_date#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#20, d_date#21] -Condition : isnotnull(d_date_sk#20) - -(28) ReusedExchange [Reuses operator id: 77] -Output [1]: [d_date#9] - -(29) CometBroadcastHashJoin -Left output [2]: [d_date_sk#20, d_date#21] -Right output [1]: [d_date#9] -Arguments: [d_date#21], [d_date#9], LeftSemi, BuildRight - -(30) CometProject -Input [2]: [d_date_sk#20, d_date#21] -Arguments: [d_date_sk#20], [d_date_sk#20] - -(31) CometBroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: [d_date_sk#20] - -(32) CometBroadcastHashJoin -Left output [3]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19] -Right output [1]: [d_date_sk#20] -Arguments: [cs_sold_date_sk#16], [d_date_sk#20], Inner, BuildRight - -(33) CometProject -Input [4]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19, d_date_sk#20] -Arguments: [cs_ext_sales_price#15, i_item_id#19], [cs_ext_sales_price#15, i_item_id#19] - -(34) CometHashAggregate -Input [2]: [cs_ext_sales_price#15, i_item_id#19] -Keys [1]: [i_item_id#19] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#15))] - -(35) CometExchange -Input [2]: [i_item_id#19, sum#22] -Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(36) CometHashAggregate -Input [2]: [i_item_id#19, sum#22] -Keys [1]: [i_item_id#19] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#15))] - -(37) CometFilter -Input [2]: [item_id#23, cs_item_rev#24] -Condition : isnotnull(cs_item_rev#24) - -(38) CometBroadcastExchange -Input [2]: [item_id#23, cs_item_rev#24] -Arguments: [item_id#23, cs_item_rev#24] - -(39) CometBroadcastHashJoin -Left output [2]: [item_id#12, ss_item_rev#13] -Right output [2]: [item_id#23, cs_item_rev#24] -Arguments: [item_id#12], [item_id#23], Inner, ((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * cs_item_rev#24)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ss_item_rev#13))), BuildRight - -(40) CometProject -Input [4]: [item_id#12, ss_item_rev#13, item_id#23, cs_item_rev#24] -Arguments: [item_id#12, ss_item_rev#13, cs_item_rev#24], [item_id#12, ss_item_rev#13, cs_item_rev#24] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#25) - -(43) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#29, i_item_id#30] - -(44) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [i_item_sk#29, i_item_id#30] -Arguments: [ws_item_sk#25], [i_item_sk#29], Inner, BuildRight - -(45) CometProject -Input [5]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_sk#29, i_item_id#30] -Arguments: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30], [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30] - -(46) ReusedExchange [Reuses operator id: 31] -Output [1]: [d_date_sk#31] - -(47) CometBroadcastHashJoin -Left output [3]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30] -Right output [1]: [d_date_sk#31] -Arguments: [ws_sold_date_sk#27], [d_date_sk#31], Inner, BuildRight - -(48) CometProject -Input [4]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30, d_date_sk#31] -Arguments: [ws_ext_sales_price#26, i_item_id#30], [ws_ext_sales_price#26, i_item_id#30] - -(49) CometHashAggregate -Input [2]: [ws_ext_sales_price#26, i_item_id#30] -Keys [1]: [i_item_id#30] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#26))] - -(50) CometExchange -Input [2]: [i_item_id#30, sum#32] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(51) CometHashAggregate -Input [2]: [i_item_id#30, sum#32] -Keys [1]: [i_item_id#30] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#26))] - -(52) CometFilter -Input [2]: [item_id#33, ws_item_rev#34] -Condition : isnotnull(ws_item_rev#34) - -(53) CometBroadcastExchange -Input [2]: [item_id#33, ws_item_rev#34] -Arguments: [item_id#33, ws_item_rev#34] - -(54) CometBroadcastHashJoin -Left output [3]: [item_id#12, ss_item_rev#13, cs_item_rev#24] -Right output [2]: [item_id#33, ws_item_rev#34] -Arguments: [item_id#12], [item_id#33], Inner, ((((((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * ws_item_rev#34)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * ws_item_rev#34))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ws_item_rev#34))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ws_item_rev#34))) AND (cast(ws_item_rev#34 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(ws_item_rev#34 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) AND (cast(ws_item_rev#34 as decimal(19,3)) >= (0.9 * cs_item_rev#24))) AND (cast(ws_item_rev#34 as decimal(20,3)) <= (1.1 * cs_item_rev#24))), BuildRight - -(55) CometProject -Input [5]: [item_id#12, ss_item_rev#13, cs_item_rev#24, item_id#33, ws_item_rev#34] -Arguments: [item_id#12, ss_item_rev#13, ss_dev#35, cs_item_rev#24, cs_dev#36, ws_item_rev#34, ws_dev#37, average#38], [item_id#12, ss_item_rev#13, (((ss_item_rev#13 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#34)) / 3) * 100) AS ss_dev#35, cs_item_rev#24, (((cs_item_rev#24 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#34)) / 3) * 100) AS cs_dev#36, ws_item_rev#34, (((ws_item_rev#34 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#34)) / 3) * 100) AS ws_dev#37, (((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#34) / 3) AS average#38] - -(56) CometTakeOrderedAndProject -Input [8]: [item_id#12, ss_item_rev#13, ss_dev#35, cs_item_rev#24, cs_dev#36, ws_item_rev#34, ws_dev#37, average#38] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_id#12 ASC NULLS FIRST,ss_item_rev#13 ASC NULLS FIRST], output=[item_id#12,ss_item_rev#13,ss_dev#35,cs_item_rev#24,cs_dev#36,ws_item_rev#34,ws_dev#37,average#38]), [item_id#12, ss_item_rev#13, ss_dev#35, cs_item_rev#24, cs_dev#36, ws_item_rev#34, ws_dev#37, average#38], 100, 0, [item_id#12 ASC NULLS FIRST, ss_item_rev#13 ASC NULLS FIRST], [item_id#12, ss_item_rev#13, ss_dev#35, cs_item_rev#24, cs_dev#36, ws_item_rev#34, ws_dev#37, average#38] - -(57) CometColumnarToRow [codegen id : 1] -Input [8]: [item_id#12, ss_item_rev#13, ss_dev#35, cs_item_rev#24, cs_dev#36, ws_item_rev#34, ws_dev#37, average#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (67) -+- * CometColumnarToRow (66) - +- CometProject (65) - +- CometBroadcastHashJoin (64) - :- CometFilter (59) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) - +- CometBroadcastExchange (63) - +- CometProject (62) - +- CometFilter (61) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (60) - - -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(59) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#9, d_week_seq#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] -ReadSchema: struct - -(61) CometFilter -Input [2]: [d_date#9, d_week_seq#39] -Condition : (isnotnull(d_week_seq#39) AND (d_week_seq#39 = Subquery scalar-subquery#40, [id=#41])) - -(62) CometProject -Input [2]: [d_date#9, d_week_seq#39] -Arguments: [d_date#9#10], [d_date#9 AS d_date#9#10] - -(63) CometBroadcastExchange -Input [1]: [d_date#9#10] -Arguments: [d_date#9#10] - -(64) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#9#10] -Arguments: [d_date#9], [d_date#9#10], LeftSemi, BuildRight - -(65) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(66) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(67) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 61 Hosting Expression = Subquery scalar-subquery#40, [id=#41] -* CometColumnarToRow (71) -+- CometProject (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) - - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#9, d_week_seq#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] -ReadSchema: struct - -(69) CometFilter -Input [2]: [d_date#9, d_week_seq#39] -Condition : (isnotnull(d_date#9) AND (d_date#9 = 2000-01-03)) - -(70) CometProject -Input [2]: [d_date#9, d_week_seq#39] -Arguments: [d_week_seq#39], [d_week_seq#39] - -(71) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#39] - -Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#17 -BroadcastExchange (81) -+- * CometColumnarToRow (80) - +- CometProject (79) - +- CometBroadcastHashJoin (78) - :- CometFilter (73) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - +- CometBroadcastExchange (77) - +- CometProject (76) - +- CometFilter (75) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (74) - - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_date#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#20, d_date#21] -Condition : isnotnull(d_date_sk#20) - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#9, d_week_seq#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] -ReadSchema: struct - -(75) CometFilter -Input [2]: [d_date#9, d_week_seq#39] -Condition : (isnotnull(d_week_seq#39) AND (d_week_seq#39 = ReusedSubquery Subquery scalar-subquery#40, [id=#41])) - -(76) CometProject -Input [2]: [d_date#9, d_week_seq#39] -Arguments: [d_date#9], [d_date#9] - -(77) CometBroadcastExchange -Input [1]: [d_date#9] -Arguments: [d_date#9] - -(78) CometBroadcastHashJoin -Left output [2]: [d_date_sk#20, d_date#21] -Right output [1]: [d_date#9] -Arguments: [d_date#21], [d_date#9], LeftSemi, BuildRight - -(79) CometProject -Input [2]: [d_date_sk#20, d_date#21] -Arguments: [d_date_sk#20], [d_date_sk#20] - -(80) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#20] - -(81) BroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:4 Hosting operator id = 75 Hosting Expression = ReusedSubquery Subquery scalar-subquery#40, [id=#41] - -Subquery:5 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#17 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/simplified.txt deleted file mode 100644 index 2e709c1b54..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,94 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - CometProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - CometBroadcastHashJoin [item_id,ss_item_rev,cs_item_rev,item_id,ws_item_rev] - CometProject [item_id,ss_item_rev,cs_item_rev] - CometBroadcastHashJoin [item_id,ss_item_rev,item_id,cs_item_rev] - CometFilter [item_id,ss_item_rev] - CometHashAggregate [sum] [item_id,ss_item_rev,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id] #1 - CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] - CometProject [ss_ext_sales_price,i_item_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] [d_date] - CometFilter [d_date,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [i_item_sk,i_item_id] #4 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #3 - CometBroadcastExchange [item_id,cs_item_rev] #6 - CometFilter [item_id,cs_item_rev] - CometHashAggregate [sum] [item_id,cs_item_rev,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id] #7 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #9 - CometProject [d_date] - CometFilter [d_date,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - ReusedExchange [i_item_sk,i_item_id] #4 - CometBroadcastExchange [d_date_sk] #10 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #9 - CometBroadcastExchange [item_id,ws_item_rev] #11 - CometFilter [item_id,ws_item_rev] - CometHashAggregate [sum] [item_id,ws_item_rev,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id] #12 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [i_item_sk,i_item_id] #4 - ReusedExchange [d_date_sk] #10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt deleted file mode 100644 index 85c8622ee4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ /dev/null @@ -1,469 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (57) -+- CometTakeOrderedAndProject (56) - +- CometProject (55) - +- CometBroadcastHashJoin (54) - :- CometProject (40) - : +- CometBroadcastHashJoin (39) - : :- CometFilter (20) - : : +- CometHashAggregate (19) - : : +- CometExchange (18) - : : +- CometHashAggregate (17) - : : +- CometProject (16) - : : +- CometBroadcastHashJoin (15) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (14) - : : +- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- ReusedExchange (11) - : +- CometBroadcastExchange (38) - : +- CometFilter (37) - : +- CometHashAggregate (36) - : +- CometExchange (35) - : +- CometHashAggregate (34) - : +- CometProject (33) - : +- CometBroadcastHashJoin (32) - : :- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (21) - : : +- ReusedExchange (23) - : +- CometBroadcastExchange (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- ReusedExchange (28) - +- CometBroadcastExchange (53) - +- CometFilter (52) - +- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometBroadcastHashJoin (47) - :- CometProject (45) - : +- CometBroadcastHashJoin (44) - : :- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (41) - : +- ReusedExchange (43) - +- ReusedExchange (46) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_item_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_item_id#6] -Condition : (isnotnull(i_item_sk#5) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true))) - -(5) CometProject -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#7] - -(6) CometBroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#7] -Arguments: [i_item_sk#5, i_item_id#7] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [2]: [i_item_sk#5, i_item_id#7] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#7] -Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(11) ReusedExchange [Reuses operator id: 63] -Output [1]: [d_date#9#10] - -(12) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#9#10] -Arguments: [d_date#9], [d_date#9#10], LeftSemi, BuildRight - -(13) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: [d_date_sk#8] - -(15) CometBroadcastHashJoin -Left output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7] -Right output [1]: [d_date_sk#8] -Arguments: [ss_sold_date_sk#3], [d_date_sk#8], Inner, BuildRight - -(16) CometProject -Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#7, d_date_sk#8] -Arguments: [ss_ext_sales_price#2, i_item_id#7], [ss_ext_sales_price#2, i_item_id#7] - -(17) CometHashAggregate -Input [2]: [ss_ext_sales_price#2, i_item_id#7] -Keys [1]: [i_item_id#7] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(18) CometExchange -Input [2]: [i_item_id#7, sum#11] -Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [2]: [i_item_id#7, sum#11] -Keys [1]: [i_item_id#7] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(20) CometFilter -Input [2]: [item_id#12, ss_item_rev#13] -Condition : isnotnull(ss_item_rev#13) - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#14) - -(23) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#18, i_item_id#19] - -(24) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] -Right output [2]: [i_item_sk#18, i_item_id#19] -Arguments: [cs_item_sk#14], [i_item_sk#18], Inner, BuildRight - -(25) CometProject -Input [5]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_sk#18, i_item_id#19] -Arguments: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19], [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_date#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#20, d_date#21] -Condition : isnotnull(d_date_sk#20) - -(28) ReusedExchange [Reuses operator id: 77] -Output [1]: [d_date#9] - -(29) CometBroadcastHashJoin -Left output [2]: [d_date_sk#20, d_date#21] -Right output [1]: [d_date#9] -Arguments: [d_date#21], [d_date#9], LeftSemi, BuildRight - -(30) CometProject -Input [2]: [d_date_sk#20, d_date#21] -Arguments: [d_date_sk#20], [d_date_sk#20] - -(31) CometBroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: [d_date_sk#20] - -(32) CometBroadcastHashJoin -Left output [3]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19] -Right output [1]: [d_date_sk#20] -Arguments: [cs_sold_date_sk#16], [d_date_sk#20], Inner, BuildRight - -(33) CometProject -Input [4]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19, d_date_sk#20] -Arguments: [cs_ext_sales_price#15, i_item_id#19], [cs_ext_sales_price#15, i_item_id#19] - -(34) CometHashAggregate -Input [2]: [cs_ext_sales_price#15, i_item_id#19] -Keys [1]: [i_item_id#19] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#15))] - -(35) CometExchange -Input [2]: [i_item_id#19, sum#22] -Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(36) CometHashAggregate -Input [2]: [i_item_id#19, sum#22] -Keys [1]: [i_item_id#19] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#15))] - -(37) CometFilter -Input [2]: [item_id#23, cs_item_rev#24] -Condition : isnotnull(cs_item_rev#24) - -(38) CometBroadcastExchange -Input [2]: [item_id#23, cs_item_rev#24] -Arguments: [item_id#23, cs_item_rev#24] - -(39) CometBroadcastHashJoin -Left output [2]: [item_id#12, ss_item_rev#13] -Right output [2]: [item_id#23, cs_item_rev#24] -Arguments: [item_id#12], [item_id#23], Inner, ((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * cs_item_rev#24)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ss_item_rev#13))), BuildRight - -(40) CometProject -Input [4]: [item_id#12, ss_item_rev#13, item_id#23, cs_item_rev#24] -Arguments: [item_id#12, ss_item_rev#13, cs_item_rev#24], [item_id#12, ss_item_rev#13, cs_item_rev#24] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#25) - -(43) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#29, i_item_id#30] - -(44) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [i_item_sk#29, i_item_id#30] -Arguments: [ws_item_sk#25], [i_item_sk#29], Inner, BuildRight - -(45) CometProject -Input [5]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_sk#29, i_item_id#30] -Arguments: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30], [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30] - -(46) ReusedExchange [Reuses operator id: 31] -Output [1]: [d_date_sk#31] - -(47) CometBroadcastHashJoin -Left output [3]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30] -Right output [1]: [d_date_sk#31] -Arguments: [ws_sold_date_sk#27], [d_date_sk#31], Inner, BuildRight - -(48) CometProject -Input [4]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30, d_date_sk#31] -Arguments: [ws_ext_sales_price#26, i_item_id#30], [ws_ext_sales_price#26, i_item_id#30] - -(49) CometHashAggregate -Input [2]: [ws_ext_sales_price#26, i_item_id#30] -Keys [1]: [i_item_id#30] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#26))] - -(50) CometExchange -Input [2]: [i_item_id#30, sum#32] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(51) CometHashAggregate -Input [2]: [i_item_id#30, sum#32] -Keys [1]: [i_item_id#30] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#26))] - -(52) CometFilter -Input [2]: [item_id#33, ws_item_rev#34] -Condition : isnotnull(ws_item_rev#34) - -(53) CometBroadcastExchange -Input [2]: [item_id#33, ws_item_rev#34] -Arguments: [item_id#33, ws_item_rev#34] - -(54) CometBroadcastHashJoin -Left output [3]: [item_id#12, ss_item_rev#13, cs_item_rev#24] -Right output [2]: [item_id#33, ws_item_rev#34] -Arguments: [item_id#12], [item_id#33], Inner, ((((((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * ws_item_rev#34)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * ws_item_rev#34))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ws_item_rev#34))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ws_item_rev#34))) AND (cast(ws_item_rev#34 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(ws_item_rev#34 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) AND (cast(ws_item_rev#34 as decimal(19,3)) >= (0.9 * cs_item_rev#24))) AND (cast(ws_item_rev#34 as decimal(20,3)) <= (1.1 * cs_item_rev#24))), BuildRight - -(55) CometProject -Input [5]: [item_id#12, ss_item_rev#13, cs_item_rev#24, item_id#33, ws_item_rev#34] -Arguments: [item_id#12, ss_item_rev#13, ss_dev#35, cs_item_rev#24, cs_dev#36, ws_item_rev#34, ws_dev#37, average#38], [item_id#12, ss_item_rev#13, (((ss_item_rev#13 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#34)) / 3) * 100) AS ss_dev#35, cs_item_rev#24, (((cs_item_rev#24 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#34)) / 3) * 100) AS cs_dev#36, ws_item_rev#34, (((ws_item_rev#34 / ((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#34)) / 3) * 100) AS ws_dev#37, (((ss_item_rev#13 + cs_item_rev#24) + ws_item_rev#34) / 3) AS average#38] - -(56) CometTakeOrderedAndProject -Input [8]: [item_id#12, ss_item_rev#13, ss_dev#35, cs_item_rev#24, cs_dev#36, ws_item_rev#34, ws_dev#37, average#38] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_id#12 ASC NULLS FIRST,ss_item_rev#13 ASC NULLS FIRST], output=[item_id#12,ss_item_rev#13,ss_dev#35,cs_item_rev#24,cs_dev#36,ws_item_rev#34,ws_dev#37,average#38]), [item_id#12, ss_item_rev#13, ss_dev#35, cs_item_rev#24, cs_dev#36, ws_item_rev#34, ws_dev#37, average#38], 100, 0, [item_id#12 ASC NULLS FIRST, ss_item_rev#13 ASC NULLS FIRST], [item_id#12, ss_item_rev#13, ss_dev#35, cs_item_rev#24, cs_dev#36, ws_item_rev#34, ws_dev#37, average#38] - -(57) CometColumnarToRow [codegen id : 1] -Input [8]: [item_id#12, ss_item_rev#13, ss_dev#35, cs_item_rev#24, cs_dev#36, ws_item_rev#34, ws_dev#37, average#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (67) -+- * CometColumnarToRow (66) - +- CometProject (65) - +- CometBroadcastHashJoin (64) - :- CometFilter (59) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) - +- CometBroadcastExchange (63) - +- CometProject (62) - +- CometFilter (61) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (60) - - -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(59) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#9, d_week_seq#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] -ReadSchema: struct - -(61) CometFilter -Input [2]: [d_date#9, d_week_seq#39] -Condition : (isnotnull(d_week_seq#39) AND (d_week_seq#39 = Subquery scalar-subquery#40, [id=#41])) - -(62) CometProject -Input [2]: [d_date#9, d_week_seq#39] -Arguments: [d_date#9#10], [d_date#9 AS d_date#9#10] - -(63) CometBroadcastExchange -Input [1]: [d_date#9#10] -Arguments: [d_date#9#10] - -(64) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#9#10] -Arguments: [d_date#9], [d_date#9#10], LeftSemi, BuildRight - -(65) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(66) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(67) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 61 Hosting Expression = Subquery scalar-subquery#40, [id=#41] -* CometColumnarToRow (71) -+- CometProject (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) - - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#9, d_week_seq#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] -ReadSchema: struct - -(69) CometFilter -Input [2]: [d_date#9, d_week_seq#39] -Condition : (isnotnull(d_date#9) AND (d_date#9 = 2000-01-03)) - -(70) CometProject -Input [2]: [d_date#9, d_week_seq#39] -Arguments: [d_week_seq#39], [d_week_seq#39] - -(71) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#39] - -Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#17 -BroadcastExchange (81) -+- * CometColumnarToRow (80) - +- CometProject (79) - +- CometBroadcastHashJoin (78) - :- CometFilter (73) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - +- CometBroadcastExchange (77) - +- CometProject (76) - +- CometFilter (75) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (74) - - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_date#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#20, d_date#21] -Condition : isnotnull(d_date_sk#20) - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#9, d_week_seq#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq)] -ReadSchema: struct - -(75) CometFilter -Input [2]: [d_date#9, d_week_seq#39] -Condition : (isnotnull(d_week_seq#39) AND (d_week_seq#39 = ReusedSubquery Subquery scalar-subquery#40, [id=#41])) - -(76) CometProject -Input [2]: [d_date#9, d_week_seq#39] -Arguments: [d_date#9], [d_date#9] - -(77) CometBroadcastExchange -Input [1]: [d_date#9] -Arguments: [d_date#9] - -(78) CometBroadcastHashJoin -Left output [2]: [d_date_sk#20, d_date#21] -Right output [1]: [d_date#9] -Arguments: [d_date#21], [d_date#9], LeftSemi, BuildRight - -(79) CometProject -Input [2]: [d_date_sk#20, d_date#21] -Arguments: [d_date_sk#20], [d_date_sk#20] - -(80) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#20] - -(81) BroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:4 Hosting operator id = 75 Hosting Expression = ReusedSubquery Subquery scalar-subquery#40, [id=#41] - -Subquery:5 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#17 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/extended.txt deleted file mode 100644 index c2dd549ac6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/extended.txt +++ /dev/null @@ -1,115 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : : +- ReusedSubquery - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 98 out of 108 eligible operators (90%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 2e709c1b54..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/simplified.txt +++ /dev/null @@ -1,94 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - CometProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - CometBroadcastHashJoin [item_id,ss_item_rev,cs_item_rev,item_id,ws_item_rev] - CometProject [item_id,ss_item_rev,cs_item_rev] - CometBroadcastHashJoin [item_id,ss_item_rev,item_id,cs_item_rev] - CometFilter [item_id,ss_item_rev] - CometHashAggregate [sum] [item_id,ss_item_rev,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id] #1 - CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] - CometProject [ss_ext_sales_price,i_item_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] [d_date] - CometFilter [d_date,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [i_item_sk,i_item_id] #4 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #3 - CometBroadcastExchange [item_id,cs_item_rev] #6 - CometFilter [item_id,cs_item_rev] - CometHashAggregate [sum] [item_id,cs_item_rev,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id] #7 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #9 - CometProject [d_date] - CometFilter [d_date,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - ReusedExchange [i_item_sk,i_item_id] #4 - CometBroadcastExchange [d_date_sk] #10 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #9 - CometBroadcastExchange [item_id,ws_item_rev] #11 - CometFilter [item_id,ws_item_rev] - CometHashAggregate [sum] [item_id,ws_item_rev,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id] #12 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [i_item_sk,i_item_id] #4 - ReusedExchange [d_date_sk] #10 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_datafusion/explain.txt deleted file mode 100644 index d25b1ea057..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_datafusion/explain.txt +++ /dev/null @@ -1,239 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (42) -+- CometTakeOrderedAndProject (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometNativeScan parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (15) - : : +- CometProject (14) - : : +- CometFilter (13) - : : +- CometNativeScan parquet spark_catalog.default.store (12) - : +- CometBroadcastExchange (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometNativeScan parquet spark_catalog.default.date_dim (18) - +- CometBroadcastExchange (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometHashAggregate (25) - : : +- ReusedExchange (24) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometNativeScan parquet spark_catalog.default.store (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.date_dim (32) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#3)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Condition : (isnotnull(d_date_sk#4) AND isnotnull(d_week_seq#5)) - -(5) CometProject -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Arguments: [d_date_sk#4, d_week_seq#5, d_day_name#7], [d_date_sk#4, d_week_seq#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_day_name#6, 9, true, false, true) AS d_day_name#7] - -(6) CometBroadcastExchange -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [d_date_sk#4, d_week_seq#5, d_day_name#7] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Right output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [ss_sold_date_sk#3], [d_date_sk#4], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3, d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7], [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7] - -(9) CometHashAggregate -Input [4]: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7] -Keys [2]: [d_week_seq#5, ss_store_sk#1] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] - -(10) CometExchange -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] -Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] -Keys [2]: [d_week_seq#5, ss_store_sk#1] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] - -(12) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(13) CometFilter -Input [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Condition : (isnotnull(s_store_sk#15) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#16, 16, true, false, true))) - -(14) CometProject -Input [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Arguments: [s_store_sk#15, s_store_id#18, s_store_name#17], [s_store_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#16, 16, true, false, true) AS s_store_id#18, s_store_name#17] - -(15) CometBroadcastExchange -Input [3]: [s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [s_store_sk#15, s_store_id#18, s_store_name#17] - -(16) CometBroadcastHashJoin -Left output [9]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25] -Right output [3]: [s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [ss_store_sk#1], [s_store_sk#15], Inner, BuildRight - -(17) CometProject -Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17], [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17] - -(18) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#26, d_week_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_week_seq)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [d_month_seq#26, d_week_seq#27] -Condition : (((isnotnull(d_month_seq#26) AND (d_month_seq#26 >= 1212)) AND (d_month_seq#26 <= 1223)) AND isnotnull(d_week_seq#27)) - -(20) CometProject -Input [2]: [d_month_seq#26, d_week_seq#27] -Arguments: [d_week_seq#27], [d_week_seq#27] - -(21) CometBroadcastExchange -Input [1]: [d_week_seq#27] -Arguments: [d_week_seq#27] - -(22) CometBroadcastHashJoin -Left output [10]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17] -Right output [1]: [d_week_seq#27] -Arguments: [d_week_seq#5], [d_week_seq#27], Inner, BuildRight - -(23) CometProject -Input [11]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17, d_week_seq#27] -Arguments: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37], [s_store_name#17 AS s_store_name1#28, d_week_seq#5 AS d_week_seq1#29, s_store_id#18 AS s_store_id1#30, sun_sales#19 AS sun_sales1#31, mon_sales#20 AS mon_sales1#32, tue_sales#21 AS tue_sales1#33, wed_sales#22 AS wed_sales1#34, thu_sales#23 AS thu_sales1#35, fri_sales#24 AS fri_sales1#36, sat_sales#25 AS sat_sales1#37] - -(24) ReusedExchange [Reuses operator id: 10] -Output [9]: [d_week_seq#5, ss_store_sk#1, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] - -(25) CometHashAggregate -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] -Keys [2]: [d_week_seq#5, ss_store_sk#1] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] - -(26) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#45, s_store_id#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [s_store_sk#45, s_store_id#46] -Condition : (isnotnull(s_store_sk#45) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#46, 16, true, false, true))) - -(28) CometProject -Input [2]: [s_store_sk#45, s_store_id#46] -Arguments: [s_store_sk#45, s_store_id#47], [s_store_sk#45, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#46, 16, true, false, true) AS s_store_id#47] - -(29) CometBroadcastExchange -Input [2]: [s_store_sk#45, s_store_id#47] -Arguments: [s_store_sk#45, s_store_id#47] - -(30) CometBroadcastHashJoin -Left output [9]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25] -Right output [2]: [s_store_sk#45, s_store_id#47] -Arguments: [ss_store_sk#1], [s_store_sk#45], Inner, BuildRight - -(31) CometProject -Input [11]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_sk#45, s_store_id#47] -Arguments: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#47], [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#47] - -(32) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#48, d_week_seq#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1224), LessThanOrEqual(d_month_seq,1235), IsNotNull(d_week_seq)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_month_seq#48, d_week_seq#49] -Condition : (((isnotnull(d_month_seq#48) AND (d_month_seq#48 >= 1224)) AND (d_month_seq#48 <= 1235)) AND isnotnull(d_week_seq#49)) - -(34) CometProject -Input [2]: [d_month_seq#48, d_week_seq#49] -Arguments: [d_week_seq#49], [d_week_seq#49] - -(35) CometBroadcastExchange -Input [1]: [d_week_seq#49] -Arguments: [d_week_seq#49] - -(36) CometBroadcastHashJoin -Left output [9]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#47] -Right output [1]: [d_week_seq#49] -Arguments: [d_week_seq#5], [d_week_seq#49], Inner, BuildRight - -(37) CometProject -Input [10]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#47, d_week_seq#49] -Arguments: [d_week_seq2#50, s_store_id2#51, sun_sales2#52, mon_sales2#53, tue_sales2#54, wed_sales2#55, thu_sales2#56, fri_sales2#57, sat_sales2#58], [d_week_seq#5 AS d_week_seq2#50, s_store_id#47 AS s_store_id2#51, sun_sales#19 AS sun_sales2#52, mon_sales#20 AS mon_sales2#53, tue_sales#21 AS tue_sales2#54, wed_sales#22 AS wed_sales2#55, thu_sales#23 AS thu_sales2#56, fri_sales#24 AS fri_sales2#57, sat_sales#25 AS sat_sales2#58] - -(38) CometBroadcastExchange -Input [9]: [d_week_seq2#50, s_store_id2#51, sun_sales2#52, mon_sales2#53, tue_sales2#54, wed_sales2#55, thu_sales2#56, fri_sales2#57, sat_sales2#58] -Arguments: [d_week_seq2#50, s_store_id2#51, sun_sales2#52, mon_sales2#53, tue_sales2#54, wed_sales2#55, thu_sales2#56, fri_sales2#57, sat_sales2#58] - -(39) CometBroadcastHashJoin -Left output [10]: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37] -Right output [9]: [d_week_seq2#50, s_store_id2#51, sun_sales2#52, mon_sales2#53, tue_sales2#54, wed_sales2#55, thu_sales2#56, fri_sales2#57, sat_sales2#58] -Arguments: [s_store_id1#30, d_week_seq1#29], [s_store_id2#51, (d_week_seq2#50 - 52)], Inner, BuildRight - -(40) CometProject -Input [19]: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37, d_week_seq2#50, s_store_id2#51, sun_sales2#52, mon_sales2#53, tue_sales2#54, wed_sales2#55, thu_sales2#56, fri_sales2#57, sat_sales2#58] -Arguments: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#59, (mon_sales1 / mon_sales2)#60, (tue_sales1 / tue_sales2)#61, (wed_sales1 / wed_sales2)#62, (thu_sales1 / thu_sales2)#63, (fri_sales1 / fri_sales2)#64, (sat_sales1 / sat_sales2)#65], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1#31 / sun_sales2#52) AS (sun_sales1 / sun_sales2)#59, (mon_sales1#32 / mon_sales2#53) AS (mon_sales1 / mon_sales2)#60, (tue_sales1#33 / tue_sales2#54) AS (tue_sales1 / tue_sales2)#61, (wed_sales1#34 / wed_sales2#55) AS (wed_sales1 / wed_sales2)#62, (thu_sales1#35 / thu_sales2#56) AS (thu_sales1 / thu_sales2)#63, (fri_sales1#36 / fri_sales2#57) AS (fri_sales1 / fri_sales2)#64, (sat_sales1#37 / sat_sales2#58) AS (sat_sales1 / sat_sales2)#65] - -(41) CometTakeOrderedAndProject -Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#59, (mon_sales1 / mon_sales2)#60, (tue_sales1 / tue_sales2)#61, (wed_sales1 / wed_sales2)#62, (thu_sales1 / thu_sales2)#63, (fri_sales1 / fri_sales2)#64, (sat_sales1 / sat_sales2)#65] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name1#28 ASC NULLS FIRST,s_store_id1#30 ASC NULLS FIRST,d_week_seq1#29 ASC NULLS FIRST], output=[s_store_name1#28,s_store_id1#30,d_week_seq1#29,(sun_sales1 / sun_sales2)#59,(mon_sales1 / mon_sales2)#60,(tue_sales1 / tue_sales2)#61,(wed_sales1 / wed_sales2)#62,(thu_sales1 / thu_sales2)#63,(fri_sales1 / fri_sales2)#64,(sat_sales1 / sat_sales2)#65]), [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#59, (mon_sales1 / mon_sales2)#60, (tue_sales1 / tue_sales2)#61, (wed_sales1 / wed_sales2)#62, (thu_sales1 / thu_sales2)#63, (fri_sales1 / fri_sales2)#64, (sat_sales1 / sat_sales2)#65], 100, 0, [s_store_name1#28 ASC NULLS FIRST, s_store_id1#30 ASC NULLS FIRST, d_week_seq1#29 ASC NULLS FIRST], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#59, (mon_sales1 / mon_sales2)#60, (tue_sales1 / tue_sales2)#61, (wed_sales1 / wed_sales2)#62, (thu_sales1 / thu_sales2)#63, (fri_sales1 / fri_sales2)#64, (sat_sales1 / sat_sales2)#65] - -(42) CometColumnarToRow [codegen id : 1] -Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#59, (mon_sales1 / mon_sales2)#60, (tue_sales1 / tue_sales2)#61, (wed_sales1 / wed_sales2)#62, (thu_sales1 / thu_sales2)#63, (fri_sales1 / fri_sales2)#64, (sat_sales1 / sat_sales2)#65] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_datafusion/simplified.txt deleted file mode 100644 index 72823b14c1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_datafusion/simplified.txt +++ /dev/null @@ -1,44 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometBroadcastHashJoin [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometProject [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id,s_store_name] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - CometExchange [d_week_seq,ss_store_sk] #1 - CometHashAggregate [d_day_name,ss_sales_price] [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] - CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - 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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] - CometBroadcastExchange [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 - CometBroadcastExchange [s_store_sk,s_store_id] #6 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [d_week_seq] #7 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_iceberg_compat/explain.txt deleted file mode 100644 index 6d7e26d609..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,239 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (42) -+- CometTakeOrderedAndProject (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (15) - : : +- CometProject (14) - : : +- CometFilter (13) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (12) - : +- CometBroadcastExchange (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (18) - +- CometBroadcastExchange (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometHashAggregate (25) - : : +- ReusedExchange (24) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Condition : (isnotnull(d_date_sk#4) AND isnotnull(d_week_seq#5)) - -(5) CometProject -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Arguments: [d_date_sk#4, d_week_seq#5, d_day_name#7], [d_date_sk#4, d_week_seq#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_day_name#6, 9, true, false, true) AS d_day_name#7] - -(6) CometBroadcastExchange -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [d_date_sk#4, d_week_seq#5, d_day_name#7] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Right output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [ss_sold_date_sk#3], [d_date_sk#4], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3, d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7], [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7] - -(9) CometHashAggregate -Input [4]: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7] -Keys [2]: [d_week_seq#5, ss_store_sk#1] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] - -(10) CometExchange -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] -Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] -Keys [2]: [d_week_seq#5, ss_store_sk#1] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(13) CometFilter -Input [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Condition : (isnotnull(s_store_sk#15) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#16, 16, true, false, true))) - -(14) CometProject -Input [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Arguments: [s_store_sk#15, s_store_id#18, s_store_name#17], [s_store_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#16, 16, true, false, true) AS s_store_id#18, s_store_name#17] - -(15) CometBroadcastExchange -Input [3]: [s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [s_store_sk#15, s_store_id#18, s_store_name#17] - -(16) CometBroadcastHashJoin -Left output [9]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25] -Right output [3]: [s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [ss_store_sk#1], [s_store_sk#15], Inner, BuildRight - -(17) CometProject -Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17], [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#26, d_week_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_week_seq)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [d_month_seq#26, d_week_seq#27] -Condition : (((isnotnull(d_month_seq#26) AND (d_month_seq#26 >= 1212)) AND (d_month_seq#26 <= 1223)) AND isnotnull(d_week_seq#27)) - -(20) CometProject -Input [2]: [d_month_seq#26, d_week_seq#27] -Arguments: [d_week_seq#27], [d_week_seq#27] - -(21) CometBroadcastExchange -Input [1]: [d_week_seq#27] -Arguments: [d_week_seq#27] - -(22) CometBroadcastHashJoin -Left output [10]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17] -Right output [1]: [d_week_seq#27] -Arguments: [d_week_seq#5], [d_week_seq#27], Inner, BuildRight - -(23) CometProject -Input [11]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17, d_week_seq#27] -Arguments: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37], [s_store_name#17 AS s_store_name1#28, d_week_seq#5 AS d_week_seq1#29, s_store_id#18 AS s_store_id1#30, sun_sales#19 AS sun_sales1#31, mon_sales#20 AS mon_sales1#32, tue_sales#21 AS tue_sales1#33, wed_sales#22 AS wed_sales1#34, thu_sales#23 AS thu_sales1#35, fri_sales#24 AS fri_sales1#36, sat_sales#25 AS sat_sales1#37] - -(24) ReusedExchange [Reuses operator id: 10] -Output [9]: [d_week_seq#5, ss_store_sk#1, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] - -(25) CometHashAggregate -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] -Keys [2]: [d_week_seq#5, ss_store_sk#1] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#45, s_store_id#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [s_store_sk#45, s_store_id#46] -Condition : (isnotnull(s_store_sk#45) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#46, 16, true, false, true))) - -(28) CometProject -Input [2]: [s_store_sk#45, s_store_id#46] -Arguments: [s_store_sk#45, s_store_id#47], [s_store_sk#45, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#46, 16, true, false, true) AS s_store_id#47] - -(29) CometBroadcastExchange -Input [2]: [s_store_sk#45, s_store_id#47] -Arguments: [s_store_sk#45, s_store_id#47] - -(30) CometBroadcastHashJoin -Left output [9]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25] -Right output [2]: [s_store_sk#45, s_store_id#47] -Arguments: [ss_store_sk#1], [s_store_sk#45], Inner, BuildRight - -(31) CometProject -Input [11]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_sk#45, s_store_id#47] -Arguments: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#47], [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#47] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#48, d_week_seq#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1224), LessThanOrEqual(d_month_seq,1235), IsNotNull(d_week_seq)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_month_seq#48, d_week_seq#49] -Condition : (((isnotnull(d_month_seq#48) AND (d_month_seq#48 >= 1224)) AND (d_month_seq#48 <= 1235)) AND isnotnull(d_week_seq#49)) - -(34) CometProject -Input [2]: [d_month_seq#48, d_week_seq#49] -Arguments: [d_week_seq#49], [d_week_seq#49] - -(35) CometBroadcastExchange -Input [1]: [d_week_seq#49] -Arguments: [d_week_seq#49] - -(36) CometBroadcastHashJoin -Left output [9]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#47] -Right output [1]: [d_week_seq#49] -Arguments: [d_week_seq#5], [d_week_seq#49], Inner, BuildRight - -(37) CometProject -Input [10]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#47, d_week_seq#49] -Arguments: [d_week_seq2#50, s_store_id2#51, sun_sales2#52, mon_sales2#53, tue_sales2#54, wed_sales2#55, thu_sales2#56, fri_sales2#57, sat_sales2#58], [d_week_seq#5 AS d_week_seq2#50, s_store_id#47 AS s_store_id2#51, sun_sales#19 AS sun_sales2#52, mon_sales#20 AS mon_sales2#53, tue_sales#21 AS tue_sales2#54, wed_sales#22 AS wed_sales2#55, thu_sales#23 AS thu_sales2#56, fri_sales#24 AS fri_sales2#57, sat_sales#25 AS sat_sales2#58] - -(38) CometBroadcastExchange -Input [9]: [d_week_seq2#50, s_store_id2#51, sun_sales2#52, mon_sales2#53, tue_sales2#54, wed_sales2#55, thu_sales2#56, fri_sales2#57, sat_sales2#58] -Arguments: [d_week_seq2#50, s_store_id2#51, sun_sales2#52, mon_sales2#53, tue_sales2#54, wed_sales2#55, thu_sales2#56, fri_sales2#57, sat_sales2#58] - -(39) CometBroadcastHashJoin -Left output [10]: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37] -Right output [9]: [d_week_seq2#50, s_store_id2#51, sun_sales2#52, mon_sales2#53, tue_sales2#54, wed_sales2#55, thu_sales2#56, fri_sales2#57, sat_sales2#58] -Arguments: [s_store_id1#30, d_week_seq1#29], [s_store_id2#51, (d_week_seq2#50 - 52)], Inner, BuildRight - -(40) CometProject -Input [19]: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37, d_week_seq2#50, s_store_id2#51, sun_sales2#52, mon_sales2#53, tue_sales2#54, wed_sales2#55, thu_sales2#56, fri_sales2#57, sat_sales2#58] -Arguments: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#59, (mon_sales1 / mon_sales2)#60, (tue_sales1 / tue_sales2)#61, (wed_sales1 / wed_sales2)#62, (thu_sales1 / thu_sales2)#63, (fri_sales1 / fri_sales2)#64, (sat_sales1 / sat_sales2)#65], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1#31 / sun_sales2#52) AS (sun_sales1 / sun_sales2)#59, (mon_sales1#32 / mon_sales2#53) AS (mon_sales1 / mon_sales2)#60, (tue_sales1#33 / tue_sales2#54) AS (tue_sales1 / tue_sales2)#61, (wed_sales1#34 / wed_sales2#55) AS (wed_sales1 / wed_sales2)#62, (thu_sales1#35 / thu_sales2#56) AS (thu_sales1 / thu_sales2)#63, (fri_sales1#36 / fri_sales2#57) AS (fri_sales1 / fri_sales2)#64, (sat_sales1#37 / sat_sales2#58) AS (sat_sales1 / sat_sales2)#65] - -(41) CometTakeOrderedAndProject -Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#59, (mon_sales1 / mon_sales2)#60, (tue_sales1 / tue_sales2)#61, (wed_sales1 / wed_sales2)#62, (thu_sales1 / thu_sales2)#63, (fri_sales1 / fri_sales2)#64, (sat_sales1 / sat_sales2)#65] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name1#28 ASC NULLS FIRST,s_store_id1#30 ASC NULLS FIRST,d_week_seq1#29 ASC NULLS FIRST], output=[s_store_name1#28,s_store_id1#30,d_week_seq1#29,(sun_sales1 / sun_sales2)#59,(mon_sales1 / mon_sales2)#60,(tue_sales1 / tue_sales2)#61,(wed_sales1 / wed_sales2)#62,(thu_sales1 / thu_sales2)#63,(fri_sales1 / fri_sales2)#64,(sat_sales1 / sat_sales2)#65]), [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#59, (mon_sales1 / mon_sales2)#60, (tue_sales1 / tue_sales2)#61, (wed_sales1 / wed_sales2)#62, (thu_sales1 / thu_sales2)#63, (fri_sales1 / fri_sales2)#64, (sat_sales1 / sat_sales2)#65], 100, 0, [s_store_name1#28 ASC NULLS FIRST, s_store_id1#30 ASC NULLS FIRST, d_week_seq1#29 ASC NULLS FIRST], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#59, (mon_sales1 / mon_sales2)#60, (tue_sales1 / tue_sales2)#61, (wed_sales1 / wed_sales2)#62, (thu_sales1 / thu_sales2)#63, (fri_sales1 / fri_sales2)#64, (sat_sales1 / sat_sales2)#65] - -(42) CometColumnarToRow [codegen id : 1] -Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#59, (mon_sales1 / mon_sales2)#60, (tue_sales1 / tue_sales2)#61, (wed_sales1 / wed_sales2)#62, (thu_sales1 / thu_sales2)#63, (fri_sales1 / fri_sales2)#64, (sat_sales1 / sat_sales2)#65] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_iceberg_compat/simplified.txt deleted file mode 100644 index bca9c6463d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,44 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometBroadcastHashJoin [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometProject [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id,s_store_name] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - CometExchange [d_week_seq,ss_store_sk] #1 - CometHashAggregate [d_day_name,ss_sales_price] [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] - CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] - CometBroadcastExchange [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 - CometBroadcastExchange [s_store_sk,s_store_id] #6 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [d_week_seq] #7 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt deleted file mode 100644 index 6d7e26d609..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt +++ /dev/null @@ -1,239 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (42) -+- CometTakeOrderedAndProject (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (15) - : : +- CometProject (14) - : : +- CometFilter (13) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (12) - : +- CometBroadcastExchange (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (18) - +- CometBroadcastExchange (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometHashAggregate (25) - : : +- ReusedExchange (24) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Condition : (isnotnull(d_date_sk#4) AND isnotnull(d_week_seq#5)) - -(5) CometProject -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] -Arguments: [d_date_sk#4, d_week_seq#5, d_day_name#7], [d_date_sk#4, d_week_seq#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, d_day_name#6, 9, true, false, true) AS d_day_name#7] - -(6) CometBroadcastExchange -Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [d_date_sk#4, d_week_seq#5, d_day_name#7] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Right output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [ss_sold_date_sk#3], [d_date_sk#4], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3, d_date_sk#4, d_week_seq#5, d_day_name#7] -Arguments: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7], [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7] - -(9) CometHashAggregate -Input [4]: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#7] -Keys [2]: [d_week_seq#5, ss_store_sk#1] -Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] - -(10) CometExchange -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] -Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] -Keys [2]: [d_week_seq#5, ss_store_sk#1] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(13) CometFilter -Input [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Condition : (isnotnull(s_store_sk#15) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#16, 16, true, false, true))) - -(14) CometProject -Input [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] -Arguments: [s_store_sk#15, s_store_id#18, s_store_name#17], [s_store_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#16, 16, true, false, true) AS s_store_id#18, s_store_name#17] - -(15) CometBroadcastExchange -Input [3]: [s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [s_store_sk#15, s_store_id#18, s_store_name#17] - -(16) CometBroadcastHashJoin -Left output [9]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25] -Right output [3]: [s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [ss_store_sk#1], [s_store_sk#15], Inner, BuildRight - -(17) CometProject -Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_sk#15, s_store_id#18, s_store_name#17] -Arguments: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17], [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#26, d_week_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_week_seq)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [d_month_seq#26, d_week_seq#27] -Condition : (((isnotnull(d_month_seq#26) AND (d_month_seq#26 >= 1212)) AND (d_month_seq#26 <= 1223)) AND isnotnull(d_week_seq#27)) - -(20) CometProject -Input [2]: [d_month_seq#26, d_week_seq#27] -Arguments: [d_week_seq#27], [d_week_seq#27] - -(21) CometBroadcastExchange -Input [1]: [d_week_seq#27] -Arguments: [d_week_seq#27] - -(22) CometBroadcastHashJoin -Left output [10]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17] -Right output [1]: [d_week_seq#27] -Arguments: [d_week_seq#5], [d_week_seq#27], Inner, BuildRight - -(23) CometProject -Input [11]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17, d_week_seq#27] -Arguments: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37], [s_store_name#17 AS s_store_name1#28, d_week_seq#5 AS d_week_seq1#29, s_store_id#18 AS s_store_id1#30, sun_sales#19 AS sun_sales1#31, mon_sales#20 AS mon_sales1#32, tue_sales#21 AS tue_sales1#33, wed_sales#22 AS wed_sales1#34, thu_sales#23 AS thu_sales1#35, fri_sales#24 AS fri_sales1#36, sat_sales#25 AS sat_sales1#37] - -(24) ReusedExchange [Reuses operator id: 10] -Output [9]: [d_week_seq#5, ss_store_sk#1, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] - -(25) CometHashAggregate -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] -Keys [2]: [d_week_seq#5, ss_store_sk#1] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#45, s_store_id#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [s_store_sk#45, s_store_id#46] -Condition : (isnotnull(s_store_sk#45) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#46, 16, true, false, true))) - -(28) CometProject -Input [2]: [s_store_sk#45, s_store_id#46] -Arguments: [s_store_sk#45, s_store_id#47], [s_store_sk#45, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#46, 16, true, false, true) AS s_store_id#47] - -(29) CometBroadcastExchange -Input [2]: [s_store_sk#45, s_store_id#47] -Arguments: [s_store_sk#45, s_store_id#47] - -(30) CometBroadcastHashJoin -Left output [9]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25] -Right output [2]: [s_store_sk#45, s_store_id#47] -Arguments: [ss_store_sk#1], [s_store_sk#45], Inner, BuildRight - -(31) CometProject -Input [11]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_sk#45, s_store_id#47] -Arguments: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#47], [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#47] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#48, d_week_seq#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1224), LessThanOrEqual(d_month_seq,1235), IsNotNull(d_week_seq)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_month_seq#48, d_week_seq#49] -Condition : (((isnotnull(d_month_seq#48) AND (d_month_seq#48 >= 1224)) AND (d_month_seq#48 <= 1235)) AND isnotnull(d_week_seq#49)) - -(34) CometProject -Input [2]: [d_month_seq#48, d_week_seq#49] -Arguments: [d_week_seq#49], [d_week_seq#49] - -(35) CometBroadcastExchange -Input [1]: [d_week_seq#49] -Arguments: [d_week_seq#49] - -(36) CometBroadcastHashJoin -Left output [9]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#47] -Right output [1]: [d_week_seq#49] -Arguments: [d_week_seq#5], [d_week_seq#49], Inner, BuildRight - -(37) CometProject -Input [10]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#47, d_week_seq#49] -Arguments: [d_week_seq2#50, s_store_id2#51, sun_sales2#52, mon_sales2#53, tue_sales2#54, wed_sales2#55, thu_sales2#56, fri_sales2#57, sat_sales2#58], [d_week_seq#5 AS d_week_seq2#50, s_store_id#47 AS s_store_id2#51, sun_sales#19 AS sun_sales2#52, mon_sales#20 AS mon_sales2#53, tue_sales#21 AS tue_sales2#54, wed_sales#22 AS wed_sales2#55, thu_sales#23 AS thu_sales2#56, fri_sales#24 AS fri_sales2#57, sat_sales#25 AS sat_sales2#58] - -(38) CometBroadcastExchange -Input [9]: [d_week_seq2#50, s_store_id2#51, sun_sales2#52, mon_sales2#53, tue_sales2#54, wed_sales2#55, thu_sales2#56, fri_sales2#57, sat_sales2#58] -Arguments: [d_week_seq2#50, s_store_id2#51, sun_sales2#52, mon_sales2#53, tue_sales2#54, wed_sales2#55, thu_sales2#56, fri_sales2#57, sat_sales2#58] - -(39) CometBroadcastHashJoin -Left output [10]: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37] -Right output [9]: [d_week_seq2#50, s_store_id2#51, sun_sales2#52, mon_sales2#53, tue_sales2#54, wed_sales2#55, thu_sales2#56, fri_sales2#57, sat_sales2#58] -Arguments: [s_store_id1#30, d_week_seq1#29], [s_store_id2#51, (d_week_seq2#50 - 52)], Inner, BuildRight - -(40) CometProject -Input [19]: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37, d_week_seq2#50, s_store_id2#51, sun_sales2#52, mon_sales2#53, tue_sales2#54, wed_sales2#55, thu_sales2#56, fri_sales2#57, sat_sales2#58] -Arguments: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#59, (mon_sales1 / mon_sales2)#60, (tue_sales1 / tue_sales2)#61, (wed_sales1 / wed_sales2)#62, (thu_sales1 / thu_sales2)#63, (fri_sales1 / fri_sales2)#64, (sat_sales1 / sat_sales2)#65], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1#31 / sun_sales2#52) AS (sun_sales1 / sun_sales2)#59, (mon_sales1#32 / mon_sales2#53) AS (mon_sales1 / mon_sales2)#60, (tue_sales1#33 / tue_sales2#54) AS (tue_sales1 / tue_sales2)#61, (wed_sales1#34 / wed_sales2#55) AS (wed_sales1 / wed_sales2)#62, (thu_sales1#35 / thu_sales2#56) AS (thu_sales1 / thu_sales2)#63, (fri_sales1#36 / fri_sales2#57) AS (fri_sales1 / fri_sales2)#64, (sat_sales1#37 / sat_sales2#58) AS (sat_sales1 / sat_sales2)#65] - -(41) CometTakeOrderedAndProject -Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#59, (mon_sales1 / mon_sales2)#60, (tue_sales1 / tue_sales2)#61, (wed_sales1 / wed_sales2)#62, (thu_sales1 / thu_sales2)#63, (fri_sales1 / fri_sales2)#64, (sat_sales1 / sat_sales2)#65] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name1#28 ASC NULLS FIRST,s_store_id1#30 ASC NULLS FIRST,d_week_seq1#29 ASC NULLS FIRST], output=[s_store_name1#28,s_store_id1#30,d_week_seq1#29,(sun_sales1 / sun_sales2)#59,(mon_sales1 / mon_sales2)#60,(tue_sales1 / tue_sales2)#61,(wed_sales1 / wed_sales2)#62,(thu_sales1 / thu_sales2)#63,(fri_sales1 / fri_sales2)#64,(sat_sales1 / sat_sales2)#65]), [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#59, (mon_sales1 / mon_sales2)#60, (tue_sales1 / tue_sales2)#61, (wed_sales1 / wed_sales2)#62, (thu_sales1 / thu_sales2)#63, (fri_sales1 / fri_sales2)#64, (sat_sales1 / sat_sales2)#65], 100, 0, [s_store_name1#28 ASC NULLS FIRST, s_store_id1#30 ASC NULLS FIRST, d_week_seq1#29 ASC NULLS FIRST], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#59, (mon_sales1 / mon_sales2)#60, (tue_sales1 / tue_sales2)#61, (wed_sales1 / wed_sales2)#62, (thu_sales1 / thu_sales2)#63, (fri_sales1 / fri_sales2)#64, (sat_sales1 / sat_sales2)#65] - -(42) CometColumnarToRow [codegen id : 1] -Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#59, (mon_sales1 / mon_sales2)#60, (tue_sales1 / tue_sales2)#61, (wed_sales1 / wed_sales2)#62, (thu_sales1 / thu_sales2)#63, (fri_sales1 / fri_sales2)#64, (sat_sales1 / sat_sales2)#65] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/extended.txt deleted file mode 100644 index 3d50a2131e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/extended.txt +++ /dev/null @@ -1,53 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 50 out of 50 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index bca9c6463d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/simplified.txt +++ /dev/null @@ -1,44 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - CometBroadcastHashJoin [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometProject [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id,s_store_name] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - CometExchange [d_week_seq,ss_store_sk] #1 - CometHashAggregate [d_day_name,ss_sales_price] [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] - CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #2 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] - CometBroadcastExchange [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 - CometProject [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] - CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,d_week_seq] - CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] - CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] - ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 - CometBroadcastExchange [s_store_sk,s_store_id] #6 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [d_week_seq] #7 - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/explain.txt deleted file mode 100644 index fb396f2dc2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/explain.txt +++ /dev/null @@ -1,304 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (40) -+- * Filter (39) - +- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * CometColumnarToRow (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.customer (4) - : : +- BroadcastExchange (13) - : : +- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet spark_catalog.default.store_sales (10) - : +- ReusedExchange (16) - +- BroadcastExchange (32) - +- * CometColumnarToRow (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (20) - : +- CometNativeScan parquet spark_catalog.default.item (19) - +- CometBroadcastExchange (28) - +- CometFilter (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometFilter (22) - +- CometNativeScan parquet spark_catalog.default.item (21) - - -(1) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#1, ca_state#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ca_address_sk#1, ca_state#2] -Condition : isnotnull(ca_address_sk#1) - -(3) CometProject -Input [2]: [ca_address_sk#1, ca_state#2] -Arguments: [ca_address_sk#1, ca_state#3], [ca_address_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#2, 2, true, false, true) AS ca_state#3] - -(4) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_customer_sk#4)) - -(6) CometBroadcastExchange -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [c_customer_sk#4, c_current_addr_sk#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#1, ca_state#3] -Right output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] - -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ca_state#3, c_customer_sk#4] - -(10) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 1] -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(12) Filter [codegen id : 1] -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) - -(13) BroadcastExchange -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [c_customer_sk#4] -Right keys [1]: [ss_customer_sk#7] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(16) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#10] - -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [2]: [ca_state#3, ss_item_sk#6] -Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] - -(19) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#11, i_current_price#12, i_category#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [i_item_sk#11, i_current_price#12, i_category#13] -Condition : (isnotnull(i_current_price#12) AND isnotnull(i_item_sk#11)) - -(21) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_current_price#12, i_category#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_current_price#12, i_category#13] -Condition : isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#13, 50, true, false, true)) - -(23) CometProject -Input [2]: [i_current_price#12, i_category#13] -Arguments: [i_current_price#12, i_category#14], [i_current_price#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#13, 50, true, false, true) AS i_category#14] - -(24) CometHashAggregate -Input [2]: [i_current_price#12, i_category#14] -Keys [1]: [i_category#14] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#12))] - -(25) CometExchange -Input [3]: [i_category#14, sum#15, count#16] -Arguments: hashpartitioning(i_category#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(26) CometHashAggregate -Input [3]: [i_category#14, sum#15, count#16] -Keys [1]: [i_category#14] -Functions [1]: [avg(UnscaledValue(i_current_price#12))] - -(27) CometFilter -Input [2]: [avg(i_current_price)#17, i_category#14] -Condition : isnotnull(avg(i_current_price)#17) - -(28) CometBroadcastExchange -Input [2]: [avg(i_current_price)#17, i_category#14] -Arguments: [avg(i_current_price)#17, i_category#14] - -(29) CometBroadcastHashJoin -Left output [3]: [i_item_sk#11, i_current_price#12, i_category#13] -Right output [2]: [avg(i_current_price)#17, i_category#14] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#13, 50, true, false, true)], [i_category#14], Inner, (cast(i_current_price#12 as decimal(14,7)) > (1.2 * avg(i_current_price)#17)), BuildRight - -(30) CometProject -Input [5]: [i_item_sk#11, i_current_price#12, i_category#13, avg(i_current_price)#17, i_category#14] -Arguments: [i_item_sk#11], [i_item_sk#11] - -(31) CometColumnarToRow [codegen id : 3] -Input [1]: [i_item_sk#11] - -(32) BroadcastExchange -Input [1]: [i_item_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(33) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#6] -Right keys [1]: [i_item_sk#11] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 4] -Output [1]: [ca_state#3] -Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#11] - -(35) HashAggregate [codegen id : 4] -Input [1]: [ca_state#3] -Keys [1]: [ca_state#3] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#18] -Results [2]: [ca_state#3, count#19] - -(36) CometColumnarExchange -Input [2]: [ca_state#3, count#19] -Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(37) CometColumnarToRow [codegen id : 5] -Input [2]: [ca_state#3, count#19] - -(38) HashAggregate [codegen id : 5] -Input [2]: [ca_state#3, count#19] -Keys [1]: [ca_state#3] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#20] -Results [2]: [ca_state#3 AS state#21, count(1)#20 AS cnt#22] - -(39) Filter [codegen id : 5] -Input [2]: [state#21, cnt#22] -Condition : (cnt#22 >= 10) - -(40) TakeOrderedAndProject -Input [2]: [state#21, cnt#22] -Arguments: 100, [cnt#22 ASC NULLS FIRST], [state#21, cnt#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometNativeScan parquet spark_catalog.default.date_dim (41) - - -(41) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#23] -Condition : ((isnotnull(d_month_seq#23) AND (d_month_seq#23 = Subquery scalar-subquery#24, [id=#25])) AND isnotnull(d_date_sk#10)) - -(43) CometProject -Input [2]: [d_date_sk#10, d_month_seq#23] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(45) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#24, [id=#25] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometNativeScan parquet spark_catalog.default.date_dim (46) - - -(46) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#23, d_year#26, d_moy#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_month_seq#23, d_year#26, d_moy#27] -Condition : (((isnotnull(d_year#26) AND isnotnull(d_moy#27)) AND (d_year#26 = 2000)) AND (d_moy#27 = 1)) - -(48) CometProject -Input [3]: [d_month_seq#23, d_year#26, d_moy#27] -Arguments: [d_month_seq#23], [d_month_seq#23] - -(49) CometHashAggregate -Input [1]: [d_month_seq#23] -Keys [1]: [d_month_seq#23] -Functions: [] - -(50) CometExchange -Input [1]: [d_month_seq#23] -Arguments: hashpartitioning(d_month_seq#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(51) CometHashAggregate -Input [1]: [d_month_seq#23] -Keys [1]: [d_month_seq#23] -Functions: [] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#23] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/simplified.txt deleted file mode 100644 index cec4eecbc6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/simplified.txt +++ /dev/null @@ -1,69 +0,0 @@ -TakeOrderedAndProject [cnt,state] - WholeStageCodegen (5) - Filter [cnt] - HashAggregate [ca_state,count] [count(1),state,cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state] #1 - WholeStageCodegen (4) - HashAggregate [ca_state] [count,count] - Project [ca_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ca_state,ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ca_state,ss_item_sk,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_customer_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #7 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #8 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_current_price,i_category] - CometFilter [i_current_price,i_category] - CometNativeScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/explain.txt deleted file mode 100644 index b0905506ee..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometFilter (38) - +- CometHashAggregate (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (4) - : : +- CometBroadcastExchange (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (9) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- CometBroadcastExchange (29) - +- CometFilter (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (22) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#1, ca_state#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ca_address_sk#1, ca_state#2] -Condition : isnotnull(ca_address_sk#1) - -(3) CometProject -Input [2]: [ca_address_sk#1, ca_state#2] -Arguments: [ca_address_sk#1, ca_state#3], [ca_address_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#2, 2, true, false, true) AS ca_state#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_customer_sk#4)) - -(6) CometBroadcastExchange -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [c_customer_sk#4, c_current_addr_sk#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#1, ca_state#3] -Right output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) - -(11) CometBroadcastExchange -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(12) CometBroadcastHashJoin -Left output [2]: [ca_state#3, c_customer_sk#4] -Right output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [c_customer_sk#4], [ss_customer_sk#7], Inner, BuildRight - -(13) CometProject -Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8], [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(16) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(18) CometBroadcastHashJoin -Left output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(19) CometProject -Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ca_state#3, ss_item_sk#6], [ca_state#3, ss_item_sk#6] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Condition : (isnotnull(i_current_price#15) AND isnotnull(i_item_sk#14)) - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_current_price#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(23) CometFilter -Input [2]: [i_current_price#15, i_category#16] -Condition : isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true)) - -(24) CometProject -Input [2]: [i_current_price#15, i_category#16] -Arguments: [i_current_price#15, i_category#17], [i_current_price#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true) AS i_category#17] - -(25) CometHashAggregate -Input [2]: [i_current_price#15, i_category#17] -Keys [1]: [i_category#17] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#15))] - -(26) CometExchange -Input [3]: [i_category#17, sum#18, count#19] -Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [3]: [i_category#17, sum#18, count#19] -Keys [1]: [i_category#17] -Functions [1]: [avg(UnscaledValue(i_current_price#15))] - -(28) CometFilter -Input [2]: [avg(i_current_price)#20, i_category#17] -Condition : isnotnull(avg(i_current_price)#20) - -(29) CometBroadcastExchange -Input [2]: [avg(i_current_price)#20, i_category#17] -Arguments: [avg(i_current_price)#20, i_category#17] - -(30) CometBroadcastHashJoin -Left output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Right output [2]: [avg(i_current_price)#20, i_category#17] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true)], [i_category#17], Inner, (cast(i_current_price#15 as decimal(14,7)) > (1.2 * avg(i_current_price)#20)), BuildRight - -(31) CometProject -Input [5]: [i_item_sk#14, i_current_price#15, i_category#16, avg(i_current_price)#20, i_category#17] -Arguments: [i_item_sk#14], [i_item_sk#14] - -(32) CometBroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: [i_item_sk#14] - -(33) CometBroadcastHashJoin -Left output [2]: [ca_state#3, ss_item_sk#6] -Right output [1]: [i_item_sk#14] -Arguments: [ss_item_sk#6], [i_item_sk#14], Inner, BuildRight - -(34) CometProject -Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#14] -Arguments: [ca_state#3], [ca_state#3] - -(35) CometHashAggregate -Input [1]: [ca_state#3] -Keys [1]: [ca_state#3] -Functions [1]: [partial_count(1)] - -(36) CometExchange -Input [2]: [ca_state#3, count#21] -Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(37) CometHashAggregate -Input [2]: [ca_state#3, count#21] -Keys [1]: [ca_state#3] -Functions [1]: [count(1)] - -(38) CometFilter -Input [2]: [state#22, cnt#23] -Condition : (cnt#23 >= 10) - -(39) CometTakeOrderedAndProject -Input [2]: [state#22, cnt#23] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#23 ASC NULLS FIRST], output=[state#22,cnt#23]), [state#22, cnt#23], 100, 0, [cnt#23 ASC NULLS FIRST], [state#22, cnt#23] - -(40) CometColumnarToRow [codegen id : 1] -Input [2]: [state#22, cnt#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(43) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(45) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) - - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#11, d_year#24, d_moy#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_month_seq#11, d_year#24, d_moy#25] -Condition : (((isnotnull(d_year#24) AND isnotnull(d_moy#25)) AND (d_year#24 = 2000)) AND (d_moy#25 = 1)) - -(48) CometProject -Input [3]: [d_month_seq#11, d_year#24, d_moy#25] -Arguments: [d_month_seq#11], [d_month_seq#11] - -(49) CometHashAggregate -Input [1]: [d_month_seq#11] -Keys [1]: [d_month_seq#11] -Functions: [] - -(50) CometExchange -Input [1]: [d_month_seq#11] -Arguments: hashpartitioning(d_month_seq#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(51) CometHashAggregate -Input [1]: [d_month_seq#11] -Keys [1]: [d_month_seq#11] -Functions: [] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#11] - -Subquery:3 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/simplified.txt deleted file mode 100644 index 2c772b5998..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [state,cnt] - CometFilter [state,cnt] - CometHashAggregate [count] [state,cnt,ca_state,count(1)] - CometExchange [ca_state] #1 - CometHashAggregate [ca_state,count] - CometProject [ca_state] - CometBroadcastHashJoin [ca_state,ss_item_sk,i_item_sk] - CometProject [ca_state,ss_item_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 [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,ca_state,c_customer_sk,c_current_addr_sk] - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk] #7 - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #8 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #9 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_current_price,i_category] - CometFilter [i_current_price,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt deleted file mode 100644 index b0905506ee..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometFilter (38) - +- CometHashAggregate (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (4) - : : +- CometBroadcastExchange (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (9) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- CometBroadcastExchange (29) - +- CometFilter (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (22) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#1, ca_state#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ca_address_sk#1, ca_state#2] -Condition : isnotnull(ca_address_sk#1) - -(3) CometProject -Input [2]: [ca_address_sk#1, ca_state#2] -Arguments: [ca_address_sk#1, ca_state#3], [ca_address_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#2, 2, true, false, true) AS ca_state#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_customer_sk#4)) - -(6) CometBroadcastExchange -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [c_customer_sk#4, c_current_addr_sk#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#1, ca_state#3] -Right output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) - -(11) CometBroadcastExchange -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(12) CometBroadcastHashJoin -Left output [2]: [ca_state#3, c_customer_sk#4] -Right output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [c_customer_sk#4], [ss_customer_sk#7], Inner, BuildRight - -(13) CometProject -Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8], [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(16) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(18) CometBroadcastHashJoin -Left output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(19) CometProject -Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ca_state#3, ss_item_sk#6], [ca_state#3, ss_item_sk#6] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Condition : (isnotnull(i_current_price#15) AND isnotnull(i_item_sk#14)) - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_current_price#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(23) CometFilter -Input [2]: [i_current_price#15, i_category#16] -Condition : isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true)) - -(24) CometProject -Input [2]: [i_current_price#15, i_category#16] -Arguments: [i_current_price#15, i_category#17], [i_current_price#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true) AS i_category#17] - -(25) CometHashAggregate -Input [2]: [i_current_price#15, i_category#17] -Keys [1]: [i_category#17] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#15))] - -(26) CometExchange -Input [3]: [i_category#17, sum#18, count#19] -Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [3]: [i_category#17, sum#18, count#19] -Keys [1]: [i_category#17] -Functions [1]: [avg(UnscaledValue(i_current_price#15))] - -(28) CometFilter -Input [2]: [avg(i_current_price)#20, i_category#17] -Condition : isnotnull(avg(i_current_price)#20) - -(29) CometBroadcastExchange -Input [2]: [avg(i_current_price)#20, i_category#17] -Arguments: [avg(i_current_price)#20, i_category#17] - -(30) CometBroadcastHashJoin -Left output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Right output [2]: [avg(i_current_price)#20, i_category#17] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true)], [i_category#17], Inner, (cast(i_current_price#15 as decimal(14,7)) > (1.2 * avg(i_current_price)#20)), BuildRight - -(31) CometProject -Input [5]: [i_item_sk#14, i_current_price#15, i_category#16, avg(i_current_price)#20, i_category#17] -Arguments: [i_item_sk#14], [i_item_sk#14] - -(32) CometBroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: [i_item_sk#14] - -(33) CometBroadcastHashJoin -Left output [2]: [ca_state#3, ss_item_sk#6] -Right output [1]: [i_item_sk#14] -Arguments: [ss_item_sk#6], [i_item_sk#14], Inner, BuildRight - -(34) CometProject -Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#14] -Arguments: [ca_state#3], [ca_state#3] - -(35) CometHashAggregate -Input [1]: [ca_state#3] -Keys [1]: [ca_state#3] -Functions [1]: [partial_count(1)] - -(36) CometExchange -Input [2]: [ca_state#3, count#21] -Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(37) CometHashAggregate -Input [2]: [ca_state#3, count#21] -Keys [1]: [ca_state#3] -Functions [1]: [count(1)] - -(38) CometFilter -Input [2]: [state#22, cnt#23] -Condition : (cnt#23 >= 10) - -(39) CometTakeOrderedAndProject -Input [2]: [state#22, cnt#23] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#23 ASC NULLS FIRST], output=[state#22,cnt#23]), [state#22, cnt#23], 100, 0, [cnt#23 ASC NULLS FIRST], [state#22, cnt#23] - -(40) CometColumnarToRow [codegen id : 1] -Input [2]: [state#22, cnt#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(43) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(45) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) - - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#11, d_year#24, d_moy#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_month_seq#11, d_year#24, d_moy#25] -Condition : (((isnotnull(d_year#24) AND isnotnull(d_moy#25)) AND (d_year#24 = 2000)) AND (d_moy#25 = 1)) - -(48) CometProject -Input [3]: [d_month_seq#11, d_year#24, d_moy#25] -Arguments: [d_month_seq#11], [d_month_seq#11] - -(49) CometHashAggregate -Input [1]: [d_month_seq#11] -Keys [1]: [d_month_seq#11] -Functions: [] - -(50) CometExchange -Input [1]: [d_month_seq#11] -Arguments: hashpartitioning(d_month_seq#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(51) CometHashAggregate -Input [1]: [d_month_seq#11] -Keys [1]: [d_month_seq#11] -Functions: [] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#11] - -Subquery:3 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/extended.txt deleted file mode 100644 index f5b69fc6cf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/extended.txt +++ /dev/null @@ -1,57 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 2c772b5998..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [state,cnt] - CometFilter [state,cnt] - CometHashAggregate [count] [state,cnt,ca_state,count(1)] - CometExchange [ca_state] #1 - CometHashAggregate [ca_state,count] - CometProject [ca_state] - CometBroadcastHashJoin [ca_state,ss_item_sk,i_item_sk] - CometProject [ca_state,ss_item_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 [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,ca_state,c_customer_sk,c_current_addr_sk] - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk] #7 - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #8 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #9 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_current_price,i_category] - CometFilter [i_current_price,i_category] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/explain.txt deleted file mode 100644 index 5584119109..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/explain.txt +++ /dev/null @@ -1,420 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (67) -+- * HashAggregate (66) - +- * CometColumnarToRow (65) - +- CometColumnarExchange (64) - +- * HashAggregate (63) - +- Union (62) - :- * HashAggregate (29) - : +- * CometColumnarToRow (28) - : +- CometColumnarExchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) - : +- BroadcastExchange (23) - : +- * CometColumnarToRow (22) - : +- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.item (14) - : +- CometBroadcastExchange (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometNativeScan parquet spark_catalog.default.item (16) - :- * HashAggregate (45) - : +- * CometColumnarToRow (44) - : +- CometColumnarExchange (43) - : +- * HashAggregate (42) - : +- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (38) - : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- * Filter (32) - : : : : +- * ColumnarToRow (31) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (30) - : : : +- ReusedExchange (33) - : : +- ReusedExchange (36) - : +- ReusedExchange (39) - +- * HashAggregate (61) - +- * CometColumnarToRow (60) - +- CometColumnarExchange (59) - +- * HashAggregate (58) - +- * Project (57) - +- * BroadcastHashJoin Inner BuildRight (56) - :- * Project (54) - : +- * BroadcastHashJoin Inner BuildRight (53) - : :- * Project (51) - : : +- * BroadcastHashJoin Inner BuildRight (50) - : : :- * Filter (48) - : : : +- * ColumnarToRow (47) - : : : +- Scan parquet spark_catalog.default.web_sales (46) - : : +- ReusedExchange (49) - : +- ReusedExchange (52) - +- ReusedExchange (55) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(4) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_gmt_offset#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isnotnull(ca_address_sk#7)) - -(9) CometProject -Input [2]: [ca_address_sk#7, ca_gmt_offset#8] -Arguments: [ca_address_sk#7], [ca_address_sk#7] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [ca_address_sk#7] - -(11) BroadcastExchange -Input [1]: [ca_address_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] - -(14) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#9, i_item_id#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [i_item_sk#9, i_item_id#10] -Condition : isnotnull(i_item_sk#9) - -(16) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_id#11, i_category#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(17) CometFilter -Input [2]: [i_item_id#11, i_category#12] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#12, 50, true, false, true) = Music ) - -(18) CometProject -Input [2]: [i_item_id#11, i_category#12] -Arguments: [i_item_id#13], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#11, 16, true, false, true) AS i_item_id#13] - -(19) CometBroadcastExchange -Input [1]: [i_item_id#13] -Arguments: [i_item_id#13] - -(20) CometBroadcastHashJoin -Left output [2]: [i_item_sk#9, i_item_id#10] -Right output [1]: [i_item_id#13] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#10, 16, true, false, true)], [i_item_id#13], LeftSemi, BuildRight - -(21) CometProject -Input [2]: [i_item_sk#9, i_item_id#10] -Arguments: [i_item_sk#9, i_item_id#14], [i_item_sk#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#10, 16, true, false, true) AS i_item_id#14] - -(22) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#9, i_item_id#14] - -(23) BroadcastExchange -Input [2]: [i_item_sk#9, i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#9] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 4] -Output [2]: [ss_ext_sales_price#3, i_item_id#14] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_item_id#14] - -(26) HashAggregate [codegen id : 4] -Input [2]: [ss_ext_sales_price#3, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum#15] -Results [2]: [i_item_id#14, sum#16] - -(27) CometColumnarExchange -Input [2]: [i_item_id#14, sum#16] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometColumnarToRow [codegen id : 5] -Input [2]: [i_item_id#14, sum#16] - -(29) HashAggregate [codegen id : 5] -Input [2]: [i_item_id#14, sum#16] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#17] -Results [2]: [i_item_id#14 AS i_item_id#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#17,17,2) AS total_sales#19] - -(30) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 9] -Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] - -(32) Filter [codegen id : 9] -Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] -Condition : (isnotnull(cs_bill_addr_sk#20) AND isnotnull(cs_item_sk#21)) - -(33) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#24] - -(34) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 9] -Output [3]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22] -Input [5]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#24] - -(36) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#25] - -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_bill_addr_sk#20] -Right keys [1]: [ca_address_sk#25] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 9] -Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] -Input [4]: [cs_bill_addr_sk#20, cs_item_sk#21, cs_ext_sales_price#22, ca_address_sk#25] - -(39) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#26, i_item_id#27] - -(40) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#21] -Right keys [1]: [i_item_sk#26] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 9] -Output [2]: [cs_ext_sales_price#22, i_item_id#27] -Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#26, i_item_id#27] - -(42) HashAggregate [codegen id : 9] -Input [2]: [cs_ext_sales_price#22, i_item_id#27] -Keys [1]: [i_item_id#27] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#27, sum#29] - -(43) CometColumnarExchange -Input [2]: [i_item_id#27, sum#29] -Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(44) CometColumnarToRow [codegen id : 10] -Input [2]: [i_item_id#27, sum#29] - -(45) HashAggregate [codegen id : 10] -Input [2]: [i_item_id#27, sum#29] -Keys [1]: [i_item_id#27] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] -Results [2]: [i_item_id#27, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS total_sales#31] - -(46) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(47) ColumnarToRow [codegen id : 14] -Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] - -(48) Filter [codegen id : 14] -Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35] -Condition : (isnotnull(ws_bill_addr_sk#33) AND isnotnull(ws_item_sk#32)) - -(49) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#36] - -(50) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#36] -Join type: Inner -Join condition: None - -(51) Project [codegen id : 14] -Output [3]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34] -Input [5]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ws_sold_date_sk#35, d_date_sk#36] - -(52) ReusedExchange [Reuses operator id: 11] -Output [1]: [ca_address_sk#37] - -(53) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#33] -Right keys [1]: [ca_address_sk#37] -Join type: Inner -Join condition: None - -(54) Project [codegen id : 14] -Output [2]: [ws_item_sk#32, ws_ext_sales_price#34] -Input [4]: [ws_item_sk#32, ws_bill_addr_sk#33, ws_ext_sales_price#34, ca_address_sk#37] - -(55) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#38, i_item_id#39] - -(56) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_item_sk#32] -Right keys [1]: [i_item_sk#38] -Join type: Inner -Join condition: None - -(57) Project [codegen id : 14] -Output [2]: [ws_ext_sales_price#34, i_item_id#39] -Input [4]: [ws_item_sk#32, ws_ext_sales_price#34, i_item_sk#38, i_item_id#39] - -(58) HashAggregate [codegen id : 14] -Input [2]: [ws_ext_sales_price#34, i_item_id#39] -Keys [1]: [i_item_id#39] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#34))] -Aggregate Attributes [1]: [sum#40] -Results [2]: [i_item_id#39, sum#41] - -(59) CometColumnarExchange -Input [2]: [i_item_id#39, sum#41] -Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(60) CometColumnarToRow [codegen id : 15] -Input [2]: [i_item_id#39, sum#41] - -(61) HashAggregate [codegen id : 15] -Input [2]: [i_item_id#39, sum#41] -Keys [1]: [i_item_id#39] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#34))#42] -Results [2]: [i_item_id#39, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#34))#42,17,2) AS total_sales#43] - -(62) Union - -(63) HashAggregate [codegen id : 16] -Input [2]: [i_item_id#18, total_sales#19] -Keys [1]: [i_item_id#18] -Functions [1]: [partial_sum(total_sales#19)] -Aggregate Attributes [2]: [sum#44, isEmpty#45] -Results [3]: [i_item_id#18, sum#46, isEmpty#47] - -(64) CometColumnarExchange -Input [3]: [i_item_id#18, sum#46, isEmpty#47] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(65) CometColumnarToRow [codegen id : 17] -Input [3]: [i_item_id#18, sum#46, isEmpty#47] - -(66) HashAggregate [codegen id : 17] -Input [3]: [i_item_id#18, sum#46, isEmpty#47] -Keys [1]: [i_item_id#18] -Functions [1]: [sum(total_sales#19)] -Aggregate Attributes [1]: [sum(total_sales#19)#48] -Results [2]: [i_item_id#18, sum(total_sales#19)#48 AS total_sales#49] - -(67) TakeOrderedAndProject -Input [2]: [i_item_id#18, total_sales#49] -Arguments: 100, [i_item_id#18 ASC NULLS FIRST, total_sales#49 ASC NULLS FIRST], [i_item_id#18, total_sales#49] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (72) -+- * CometColumnarToRow (71) - +- CometProject (70) - +- CometFilter (69) - +- CometNativeScan parquet spark_catalog.default.date_dim (68) - - -(68) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#50, d_moy#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [3]: [d_date_sk#6, d_year#50, d_moy#51] -Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 1998)) AND (d_moy#51 = 9)) AND isnotnull(d_date_sk#6)) - -(70) CometProject -Input [3]: [d_date_sk#6, d_year#50, d_moy#51] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(71) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(72) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/simplified.txt deleted file mode 100644 index 4c05038b34..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/simplified.txt +++ /dev/null @@ -1,106 +0,0 @@ -TakeOrderedAndProject [i_item_id,total_sales] - WholeStageCodegen (17) - HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (16) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),i_item_id,total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_addr_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #6 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_id,i_category] - WholeStageCodegen (10) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #7 - WholeStageCodegen (9) - HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_addr_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (15) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #8 - WholeStageCodegen (14) - HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_addr_sk,ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [ca_address_sk] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/explain.txt deleted file mode 100644 index e5e165b5ec..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,385 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (62) -+- CometTakeOrderedAndProject (61) - +- CometHashAggregate (60) - +- CometExchange (59) - +- CometHashAggregate (58) - +- CometUnion (57) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- CometHashAggregate (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometFilter (30) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- CometHashAggregate (56) - +- CometExchange (55) - +- CometHashAggregate (54) - +- CometProject (53) - +- CometBroadcastHashJoin (52) - :- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometFilter (44) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (43) - : : +- ReusedExchange (45) - : +- ReusedExchange (48) - +- ReusedExchange (51) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 9)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(11) CometProject -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Arguments: [ca_address_sk#9], [ca_address_sk#9] - -(12) CometBroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: [ca_address_sk#9] - -(13) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Right output [1]: [ca_address_sk#9] -Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] -Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#11, i_item_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#11, i_item_id#12] -Condition : isnotnull(i_item_sk#11) - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_id#13, i_category#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(18) CometFilter -Input [2]: [i_item_id#13, i_category#14] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#14, 50, true, false, true) = Music ) - -(19) CometProject -Input [2]: [i_item_id#13, i_category#14] -Arguments: [i_item_id#15], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#13, 16, true, false, true) AS i_item_id#15] - -(20) CometBroadcastExchange -Input [1]: [i_item_id#15] -Arguments: [i_item_id#15] - -(21) CometBroadcastHashJoin -Left output [2]: [i_item_sk#11, i_item_id#12] -Right output [1]: [i_item_id#15] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#12, 16, true, false, true)], [i_item_id#15], LeftSemi, BuildRight - -(22) CometProject -Input [2]: [i_item_sk#11, i_item_id#12] -Arguments: [i_item_sk#11, i_item_id#16], [i_item_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#12, 16, true, false, true) AS i_item_id#16] - -(23) CometBroadcastExchange -Input [2]: [i_item_sk#11, i_item_id#16] -Arguments: [i_item_sk#11, i_item_id#16] - -(24) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Right output [2]: [i_item_sk#11, i_item_id#16] -Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight - -(25) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_item_id#16] -Arguments: [ss_ext_sales_price#3, i_item_id#16], [ss_ext_sales_price#3, i_item_id#16] - -(26) CometHashAggregate -Input [2]: [ss_ext_sales_price#3, i_item_id#16] -Keys [1]: [i_item_id#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(27) CometExchange -Input [2]: [i_item_id#16, sum#17] -Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [2]: [i_item_id#16, sum#17] -Keys [1]: [i_item_id#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Condition : (isnotnull(cs_bill_addr_sk#18) AND isnotnull(cs_item_sk#19)) - -(31) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#23] - -(32) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Right output [1]: [d_date_sk#23] -Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(33) CometProject -Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] -Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] - -(34) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#24] - -(35) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -Right output [1]: [ca_address_sk#24] -Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner, BuildRight - -(36) CometProject -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] -Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] - -(37) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#25, i_item_id#26] - -(38) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] -Right output [2]: [i_item_sk#25, i_item_id#26] -Arguments: [cs_item_sk#19], [i_item_sk#25], Inner, BuildRight - -(39) CometProject -Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_item_id#26] -Arguments: [cs_ext_sales_price#20, i_item_id#26], [cs_ext_sales_price#20, i_item_id#26] - -(40) CometHashAggregate -Input [2]: [cs_ext_sales_price#20, i_item_id#26] -Keys [1]: [i_item_id#26] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] - -(41) CometExchange -Input [2]: [i_item_id#26, sum#27] -Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(42) CometHashAggregate -Input [2]: [i_item_id#26, sum#27] -Keys [1]: [i_item_id#26] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(44) CometFilter -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Condition : (isnotnull(ws_bill_addr_sk#29) AND isnotnull(ws_item_sk#28)) - -(45) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(46) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [ws_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(47) CometProject -Input [5]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31, d_date_sk#33] -Arguments: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30], [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] - -(48) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#34] - -(49) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] -Right output [1]: [ca_address_sk#34] -Arguments: [ws_bill_addr_sk#29], [ca_address_sk#34], Inner, BuildRight - -(50) CometProject -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ca_address_sk#34] -Arguments: [ws_item_sk#28, ws_ext_sales_price#30], [ws_item_sk#28, ws_ext_sales_price#30] - -(51) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#35, i_item_id#36] - -(52) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#28, ws_ext_sales_price#30] -Right output [2]: [i_item_sk#35, i_item_id#36] -Arguments: [ws_item_sk#28], [i_item_sk#35], Inner, BuildRight - -(53) CometProject -Input [4]: [ws_item_sk#28, ws_ext_sales_price#30, i_item_sk#35, i_item_id#36] -Arguments: [ws_ext_sales_price#30, i_item_id#36], [ws_ext_sales_price#30, i_item_id#36] - -(54) CometHashAggregate -Input [2]: [ws_ext_sales_price#30, i_item_id#36] -Keys [1]: [i_item_id#36] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#30))] - -(55) CometExchange -Input [2]: [i_item_id#36, sum#37] -Arguments: hashpartitioning(i_item_id#36, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(56) CometHashAggregate -Input [2]: [i_item_id#36, sum#37] -Keys [1]: [i_item_id#36] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#30))] - -(57) CometUnion -Child 0 Input [2]: [i_item_id#38, total_sales#39] -Child 1 Input [2]: [i_item_id#26, total_sales#40] -Child 2 Input [2]: [i_item_id#36, total_sales#41] - -(58) CometHashAggregate -Input [2]: [i_item_id#38, total_sales#39] -Keys [1]: [i_item_id#38] -Functions [1]: [partial_sum(total_sales#39)] - -(59) CometExchange -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(60) CometHashAggregate -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Keys [1]: [i_item_id#38] -Functions [1]: [sum(total_sales#39)] - -(61) CometTakeOrderedAndProject -Input [2]: [i_item_id#38, total_sales#44] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#38 ASC NULLS FIRST,total_sales#44 ASC NULLS FIRST], output=[i_item_id#38,total_sales#44]), [i_item_id#38, total_sales#44], 100, 0, [i_item_id#38 ASC NULLS FIRST, total_sales#44 ASC NULLS FIRST], [i_item_id#38, total_sales#44] - -(62) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_id#38, total_sales#44] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (67) -+- * CometColumnarToRow (66) - +- CometProject (65) - +- CometFilter (64) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) - - -(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] -ReadSchema: struct - -(64) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 9)) AND isnotnull(d_date_sk#6)) - -(65) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(66) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(67) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/simplified.txt deleted file mode 100644 index 086da3b776..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,total_sales] - CometHashAggregate [sum,isEmpty] [i_item_id,total_sales,sum(total_sales)] - CometExchange [i_item_id] #1 - CometHashAggregate [total_sales] [i_item_id,sum,isEmpty] - CometUnion [i_item_id,total_sales] - CometHashAggregate [sum] [i_item_id,total_sales,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #7 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_category] - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - 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,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt deleted file mode 100644 index e5e165b5ec..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ /dev/null @@ -1,385 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (62) -+- CometTakeOrderedAndProject (61) - +- CometHashAggregate (60) - +- CometExchange (59) - +- CometHashAggregate (58) - +- CometUnion (57) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- CometHashAggregate (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometFilter (30) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- CometHashAggregate (56) - +- CometExchange (55) - +- CometHashAggregate (54) - +- CometProject (53) - +- CometBroadcastHashJoin (52) - :- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometFilter (44) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (43) - : : +- ReusedExchange (45) - : +- ReusedExchange (48) - +- ReusedExchange (51) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 9)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3], [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#9, ca_gmt_offset#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) - -(11) CometProject -Input [2]: [ca_address_sk#9, ca_gmt_offset#10] -Arguments: [ca_address_sk#9], [ca_address_sk#9] - -(12) CometBroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: [ca_address_sk#9] - -(13) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] -Right output [1]: [ca_address_sk#9] -Arguments: [ss_addr_sk#2], [ca_address_sk#9], Inner, BuildRight - -(14) CometProject -Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#9] -Arguments: [ss_item_sk#1, ss_ext_sales_price#3], [ss_item_sk#1, ss_ext_sales_price#3] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#11, i_item_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#11, i_item_id#12] -Condition : isnotnull(i_item_sk#11) - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_id#13, i_category#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(18) CometFilter -Input [2]: [i_item_id#13, i_category#14] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#14, 50, true, false, true) = Music ) - -(19) CometProject -Input [2]: [i_item_id#13, i_category#14] -Arguments: [i_item_id#15], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#13, 16, true, false, true) AS i_item_id#15] - -(20) CometBroadcastExchange -Input [1]: [i_item_id#15] -Arguments: [i_item_id#15] - -(21) CometBroadcastHashJoin -Left output [2]: [i_item_sk#11, i_item_id#12] -Right output [1]: [i_item_id#15] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#12, 16, true, false, true)], [i_item_id#15], LeftSemi, BuildRight - -(22) CometProject -Input [2]: [i_item_sk#11, i_item_id#12] -Arguments: [i_item_sk#11, i_item_id#16], [i_item_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#12, 16, true, false, true) AS i_item_id#16] - -(23) CometBroadcastExchange -Input [2]: [i_item_sk#11, i_item_id#16] -Arguments: [i_item_sk#11, i_item_id#16] - -(24) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#3] -Right output [2]: [i_item_sk#11, i_item_id#16] -Arguments: [ss_item_sk#1], [i_item_sk#11], Inner, BuildRight - -(25) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#11, i_item_id#16] -Arguments: [ss_ext_sales_price#3, i_item_id#16], [ss_ext_sales_price#3, i_item_id#16] - -(26) CometHashAggregate -Input [2]: [ss_ext_sales_price#3, i_item_id#16] -Keys [1]: [i_item_id#16] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(27) CometExchange -Input [2]: [i_item_id#16, sum#17] -Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [2]: [i_item_id#16, sum#17] -Keys [1]: [i_item_id#16] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Condition : (isnotnull(cs_bill_addr_sk#18) AND isnotnull(cs_item_sk#19)) - -(31) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#23] - -(32) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Right output [1]: [d_date_sk#23] -Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(33) CometProject -Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] -Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] - -(34) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#24] - -(35) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -Right output [1]: [ca_address_sk#24] -Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner, BuildRight - -(36) CometProject -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] -Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] - -(37) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#25, i_item_id#26] - -(38) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] -Right output [2]: [i_item_sk#25, i_item_id#26] -Arguments: [cs_item_sk#19], [i_item_sk#25], Inner, BuildRight - -(39) CometProject -Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_item_id#26] -Arguments: [cs_ext_sales_price#20, i_item_id#26], [cs_ext_sales_price#20, i_item_id#26] - -(40) CometHashAggregate -Input [2]: [cs_ext_sales_price#20, i_item_id#26] -Keys [1]: [i_item_id#26] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] - -(41) CometExchange -Input [2]: [i_item_id#26, sum#27] -Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(42) CometHashAggregate -Input [2]: [i_item_id#26, sum#27] -Keys [1]: [i_item_id#26] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(44) CometFilter -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Condition : (isnotnull(ws_bill_addr_sk#29) AND isnotnull(ws_item_sk#28)) - -(45) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(46) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [ws_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(47) CometProject -Input [5]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31, d_date_sk#33] -Arguments: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30], [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] - -(48) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#34] - -(49) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] -Right output [1]: [ca_address_sk#34] -Arguments: [ws_bill_addr_sk#29], [ca_address_sk#34], Inner, BuildRight - -(50) CometProject -Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ca_address_sk#34] -Arguments: [ws_item_sk#28, ws_ext_sales_price#30], [ws_item_sk#28, ws_ext_sales_price#30] - -(51) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#35, i_item_id#36] - -(52) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#28, ws_ext_sales_price#30] -Right output [2]: [i_item_sk#35, i_item_id#36] -Arguments: [ws_item_sk#28], [i_item_sk#35], Inner, BuildRight - -(53) CometProject -Input [4]: [ws_item_sk#28, ws_ext_sales_price#30, i_item_sk#35, i_item_id#36] -Arguments: [ws_ext_sales_price#30, i_item_id#36], [ws_ext_sales_price#30, i_item_id#36] - -(54) CometHashAggregate -Input [2]: [ws_ext_sales_price#30, i_item_id#36] -Keys [1]: [i_item_id#36] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#30))] - -(55) CometExchange -Input [2]: [i_item_id#36, sum#37] -Arguments: hashpartitioning(i_item_id#36, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(56) CometHashAggregate -Input [2]: [i_item_id#36, sum#37] -Keys [1]: [i_item_id#36] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#30))] - -(57) CometUnion -Child 0 Input [2]: [i_item_id#38, total_sales#39] -Child 1 Input [2]: [i_item_id#26, total_sales#40] -Child 2 Input [2]: [i_item_id#36, total_sales#41] - -(58) CometHashAggregate -Input [2]: [i_item_id#38, total_sales#39] -Keys [1]: [i_item_id#38] -Functions [1]: [partial_sum(total_sales#39)] - -(59) CometExchange -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(60) CometHashAggregate -Input [3]: [i_item_id#38, sum#42, isEmpty#43] -Keys [1]: [i_item_id#38] -Functions [1]: [sum(total_sales#39)] - -(61) CometTakeOrderedAndProject -Input [2]: [i_item_id#38, total_sales#44] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#38 ASC NULLS FIRST,total_sales#44 ASC NULLS FIRST], output=[i_item_id#38,total_sales#44]), [i_item_id#38, total_sales#44], 100, 0, [i_item_id#38 ASC NULLS FIRST, total_sales#44 ASC NULLS FIRST], [i_item_id#38, total_sales#44] - -(62) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_id#38, total_sales#44] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (67) -+- * CometColumnarToRow (66) - +- CometProject (65) - +- CometFilter (64) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) - - -(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#6, d_year#7, d_moy#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] -ReadSchema: struct - -(64) CometFilter -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 9)) AND isnotnull(d_date_sk#6)) - -(65) CometProject -Input [3]: [d_date_sk#6, d_year#7, d_moy#8] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(66) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(67) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/extended.txt deleted file mode 100644 index 36b9a8306d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/extended.txt +++ /dev/null @@ -1,100 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 086da3b776..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,total_sales] - CometHashAggregate [sum,isEmpty] [i_item_id,total_sales,sum(total_sales)] - CometExchange [i_item_id] #1 - CometHashAggregate [total_sales] [i_item_id,sum,isEmpty] - CometUnion [i_item_id,total_sales] - CometHashAggregate [sum] [i_item_id,total_sales,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [i_item_id] #7 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_category] - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - 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,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - 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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_ext_sales_price] - 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_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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/explain.txt deleted file mode 100644 index e50623a19c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/explain.txt +++ /dev/null @@ -1,425 +0,0 @@ -== Physical Plan == -* Project (69) -+- * BroadcastNestedLoopJoin Inner BuildRight (68) - :- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- CometColumnarExchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (33) - : : +- * BroadcastHashJoin Inner BuildRight (32) - : : :- * Project (26) - : : : +- * BroadcastHashJoin Inner BuildRight (25) - : : : :- * Project (20) - : : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : : :- * Project (17) - : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * CometColumnarToRow (7) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometNativeScan parquet spark_catalog.default.store (4) - : : : : : +- BroadcastExchange (15) - : : : : : +- * CometColumnarToRow (14) - : : : : : +- CometProject (13) - : : : : : +- CometFilter (12) - : : : : : +- CometNativeScan parquet spark_catalog.default.promotion (11) - : : : : +- ReusedExchange (18) - : : : +- BroadcastExchange (24) - : : : +- * CometColumnarToRow (23) - : : : +- CometFilter (22) - : : : +- CometNativeScan parquet spark_catalog.default.customer (21) - : : +- BroadcastExchange (31) - : : +- * CometColumnarToRow (30) - : : +- CometProject (29) - : : +- CometFilter (28) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (27) - : +- BroadcastExchange (38) - : +- * CometColumnarToRow (37) - : +- CometProject (36) - : +- CometFilter (35) - : +- CometNativeScan parquet spark_catalog.default.item (34) - +- BroadcastExchange (67) - +- * HashAggregate (66) - +- * CometColumnarToRow (65) - +- CometColumnarExchange (64) - +- * HashAggregate (63) - +- * Project (62) - +- * BroadcastHashJoin Inner BuildRight (61) - :- * Project (59) - : +- * BroadcastHashJoin Inner BuildRight (58) - : :- * Project (56) - : : +- * BroadcastHashJoin Inner BuildRight (55) - : : :- * Project (53) - : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : :- * Project (50) - : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : :- * Filter (47) - : : : : : +- * ColumnarToRow (46) - : : : : : +- Scan parquet spark_catalog.default.store_sales (45) - : : : : +- ReusedExchange (48) - : : : +- ReusedExchange (51) - : : +- ReusedExchange (54) - : +- ReusedExchange (57) - +- ReusedExchange (60) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 7] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(3) Filter [codegen id : 7] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_store_sk#3) AND isnotnull(ss_promo_sk#4)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1)) - -(4) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_gmt_offset#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [s_store_sk#8, s_gmt_offset#9] -Condition : ((isnotnull(s_gmt_offset#9) AND (s_gmt_offset#9 = -5.00)) AND isnotnull(s_store_sk#8)) - -(6) CometProject -Input [2]: [s_store_sk#8, s_gmt_offset#9] -Arguments: [s_store_sk#8], [s_store_sk#8] - -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [s_store_sk#8] - -(8) BroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 7] -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, s_store_sk#8] - -(11) CometNativeScan parquet spark_catalog.default.promotion -Output [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_dmail#11, 1, true, false, true) = Y) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_email#12, 1, true, false, true) = Y)) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#13, 1, true, false, true) = Y)) AND isnotnull(p_promo_sk#10)) - -(13) CometProject -Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Arguments: [p_promo_sk#10], [p_promo_sk#10] - -(14) CometColumnarToRow [codegen id : 2] -Input [1]: [p_promo_sk#10] - -(15) BroadcastExchange -Input [1]: [p_promo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_promo_sk#4] -Right keys [1]: [p_promo_sk#10] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 7] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, p_promo_sk#10] - -(18) ReusedExchange [Reuses operator id: 74] -Output [1]: [d_date_sk#14] - -(19) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 7] -Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#14] - -(21) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#15, c_current_addr_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [c_customer_sk#15, c_current_addr_sk#16] -Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_current_addr_sk#16)) - -(23) CometColumnarToRow [codegen id : 4] -Input [2]: [c_customer_sk#15, c_current_addr_sk#16] - -(24) BroadcastExchange -Input [2]: [c_customer_sk#15, c_current_addr_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(25) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 7] -Output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#16] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#15, c_current_addr_sk#16] - -(27) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#17, ca_gmt_offset#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [ca_address_sk#17, ca_gmt_offset#18] -Condition : ((isnotnull(ca_gmt_offset#18) AND (ca_gmt_offset#18 = -5.00)) AND isnotnull(ca_address_sk#17)) - -(29) CometProject -Input [2]: [ca_address_sk#17, ca_gmt_offset#18] -Arguments: [ca_address_sk#17], [ca_address_sk#17] - -(30) CometColumnarToRow [codegen id : 5] -Input [1]: [ca_address_sk#17] - -(31) BroadcastExchange -Input [1]: [ca_address_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#16] -Right keys [1]: [ca_address_sk#17] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 7] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#5] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#16, ca_address_sk#17] - -(34) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_category#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [i_item_sk#19, i_category#20] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#20, 50, true, false, true) = Jewelry ) AND isnotnull(i_item_sk#19)) - -(36) CometProject -Input [2]: [i_item_sk#19, i_category#20] -Arguments: [i_item_sk#19], [i_item_sk#19] - -(37) CometColumnarToRow [codegen id : 6] -Input [1]: [i_item_sk#19] - -(38) BroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#19] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 7] -Output [1]: [ss_ext_sales_price#5] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#19] - -(41) HashAggregate [codegen id : 7] -Input [1]: [ss_ext_sales_price#5] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum#21] -Results [1]: [sum#22] - -(42) CometColumnarExchange -Input [1]: [sum#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) CometColumnarToRow [codegen id : 15] -Input [1]: [sum#22] - -(44) HashAggregate [codegen id : 15] -Input [1]: [sum#22] -Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#23] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#23,17,2) AS promotions#24] - -(45) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 13] -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] - -(47) Filter [codegen id : 13] -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_store_sk#27) AND isnotnull(ss_customer_sk#26)) AND isnotnull(ss_item_sk#25)) - -(48) ReusedExchange [Reuses operator id: 8] -Output [1]: [s_store_sk#30] - -(49) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#27] -Right keys [1]: [s_store_sk#30] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 13] -Output [4]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] -Input [6]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, s_store_sk#30] - -(51) ReusedExchange [Reuses operator id: 74] -Output [1]: [d_date_sk#31] - -(52) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_date_sk#29] -Right keys [1]: [d_date_sk#31] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 13] -Output [3]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#31] - -(54) ReusedExchange [Reuses operator id: 24] -Output [2]: [c_customer_sk#32, c_current_addr_sk#33] - -(55) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#32] -Join type: Inner -Join condition: None - -(56) Project [codegen id : 13] -Output [3]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#33] -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, c_customer_sk#32, c_current_addr_sk#33] - -(57) ReusedExchange [Reuses operator id: 31] -Output [1]: [ca_address_sk#34] - -(58) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [c_current_addr_sk#33] -Right keys [1]: [ca_address_sk#34] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 13] -Output [2]: [ss_item_sk#25, ss_ext_sales_price#28] -Input [4]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#33, ca_address_sk#34] - -(60) ReusedExchange [Reuses operator id: 38] -Output [1]: [i_item_sk#35] - -(61) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_item_sk#25] -Right keys [1]: [i_item_sk#35] -Join type: Inner -Join condition: None - -(62) Project [codegen id : 13] -Output [1]: [ss_ext_sales_price#28] -Input [3]: [ss_item_sk#25, ss_ext_sales_price#28, i_item_sk#35] - -(63) HashAggregate [codegen id : 13] -Input [1]: [ss_ext_sales_price#28] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#28))] -Aggregate Attributes [1]: [sum#36] -Results [1]: [sum#37] - -(64) CometColumnarExchange -Input [1]: [sum#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(65) CometColumnarToRow [codegen id : 14] -Input [1]: [sum#37] - -(66) HashAggregate [codegen id : 14] -Input [1]: [sum#37] -Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#28))#38] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#28))#38,17,2) AS total#39] - -(67) BroadcastExchange -Input [1]: [total#39] -Arguments: IdentityBroadcastMode, [plan_id=8] - -(68) BroadcastNestedLoopJoin [codegen id : 15] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 15] -Output [3]: [promotions#24, total#39, ((cast(promotions#24 as decimal(15,4)) / cast(total#39 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] -Input [2]: [promotions#24, total#39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (74) -+- * CometColumnarToRow (73) - +- CometProject (72) - +- CometFilter (71) - +- CometNativeScan parquet spark_catalog.default.date_dim (70) - - -(70) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#14, d_year#41, d_moy#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(71) CometFilter -Input [3]: [d_date_sk#14, d_year#41, d_moy#42] -Condition : ((((isnotnull(d_year#41) AND isnotnull(d_moy#42)) AND (d_year#41 = 1998)) AND (d_moy#42 = 11)) AND isnotnull(d_date_sk#14)) - -(72) CometProject -Input [3]: [d_date_sk#14, d_year#41, d_moy#42] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(73) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(74) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 45 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/simplified.txt deleted file mode 100644 index f5bacac9c6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/simplified.txt +++ /dev/null @@ -1,108 +0,0 @@ -WholeStageCodegen (15) - Project [promotions,total] - BroadcastNestedLoopJoin - HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (7) - HashAggregate [ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_gmt_offset] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (14) - HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (13) - HashAggregate [ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk,ss_customer_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [s_store_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [c_customer_sk,c_current_addr_sk] #5 - InputAdapter - ReusedExchange [ca_address_sk] #6 - InputAdapter - ReusedExchange [i_item_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/explain.txt deleted file mode 100644 index 4f090653c8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,397 +0,0 @@ -== Physical Plan == -* Project (65) -+- * BroadcastNestedLoopJoin Inner BuildRight (64) - :- * CometColumnarToRow (41) - : +- CometHashAggregate (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometBroadcastHashJoin (19) - : : : : :- CometProject (14) - : : : : : +- CometBroadcastHashJoin (13) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (6) - : : : : : : +- CometProject (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (3) - : : : : : +- CometBroadcastExchange (12) - : : : : : +- CometProject (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (9) - : : : : +- CometBroadcastExchange (18) - : : : : +- CometProject (17) - : : : : +- CometFilter (16) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (15) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (21) - : : +- CometBroadcastExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (26) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) - +- BroadcastExchange (63) - +- * CometColumnarToRow (62) - +- CometHashAggregate (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometProject (55) - : +- CometBroadcastHashJoin (54) - : :- CometProject (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometProject (49) - : : : +- CometBroadcastHashJoin (48) - : : : :- CometProject (46) - : : : : +- CometBroadcastHashJoin (45) - : : : : :- CometFilter (43) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (42) - : : : : +- ReusedExchange (44) - : : : +- ReusedExchange (47) - : : +- ReusedExchange (50) - : +- ReusedExchange (53) - +- ReusedExchange (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_store_sk#3) AND isnotnull(ss_promo_sk#4)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_gmt_offset#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [s_store_sk#8, s_gmt_offset#9] -Condition : ((isnotnull(s_gmt_offset#9) AND (s_gmt_offset#9 = -5.00)) AND isnotnull(s_store_sk#8)) - -(5) CometProject -Input [2]: [s_store_sk#8, s_gmt_offset#9] -Arguments: [s_store_sk#8], [s_store_sk#8] - -(6) CometBroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: [s_store_sk#8] - -(7) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [s_store_sk#8] -Arguments: [ss_store_sk#3], [s_store_sk#8], Inner, BuildRight - -(8) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, s_store_sk#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6], [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_dmail#11, 1, true, false, true) = Y) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_email#12, 1, true, false, true) = Y)) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#13, 1, true, false, true) = Y)) AND isnotnull(p_promo_sk#10)) - -(11) CometProject -Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Arguments: [p_promo_sk#10], [p_promo_sk#10] - -(12) CometBroadcastExchange -Input [1]: [p_promo_sk#10] -Arguments: [p_promo_sk#10] - -(13) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [p_promo_sk#10] -Arguments: [ss_promo_sk#4], [p_promo_sk#10], Inner, BuildRight - -(14) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, p_promo_sk#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6], [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#14, d_year#15, d_moy#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1998)) AND (d_moy#16 = 11)) AND isnotnull(d_date_sk#14)) - -(17) CometProject -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(18) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(19) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#6], [d_date_sk#14], Inner, BuildRight - -(20) CometProject -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5], [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(c_current_addr_sk#18)) - -(23) CometBroadcastExchange -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [c_customer_sk#17, c_current_addr_sk#18] - -(24) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] -Right output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_customer_sk#2], [c_customer_sk#17], Inner, BuildRight - -(25) CometProject -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18], [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_gmt_offset#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [ca_address_sk#19, ca_gmt_offset#20] -Condition : ((isnotnull(ca_gmt_offset#20) AND (ca_gmt_offset#20 = -5.00)) AND isnotnull(ca_address_sk#19)) - -(28) CometProject -Input [2]: [ca_address_sk#19, ca_gmt_offset#20] -Arguments: [ca_address_sk#19], [ca_address_sk#19] - -(29) CometBroadcastExchange -Input [1]: [ca_address_sk#19] -Arguments: [ca_address_sk#19] - -(30) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18] -Right output [1]: [ca_address_sk#19] -Arguments: [c_current_addr_sk#18], [ca_address_sk#19], Inner, BuildRight - -(31) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18, ca_address_sk#19] -Arguments: [ss_item_sk#1, ss_ext_sales_price#5], [ss_item_sk#1, ss_ext_sales_price#5] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#21, i_category#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [i_item_sk#21, i_category#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#22, 50, true, false, true) = Jewelry ) AND isnotnull(i_item_sk#21)) - -(34) CometProject -Input [2]: [i_item_sk#21, i_category#22] -Arguments: [i_item_sk#21], [i_item_sk#21] - -(35) CometBroadcastExchange -Input [1]: [i_item_sk#21] -Arguments: [i_item_sk#21] - -(36) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#5] -Right output [1]: [i_item_sk#21] -Arguments: [ss_item_sk#1], [i_item_sk#21], Inner, BuildRight - -(37) CometProject -Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#21] -Arguments: [ss_ext_sales_price#5], [ss_ext_sales_price#5] - -(38) CometHashAggregate -Input [1]: [ss_ext_sales_price#5] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(39) CometExchange -Input [1]: [sum#23] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(40) CometHashAggregate -Input [1]: [sum#23] -Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(41) CometColumnarToRow [codegen id : 2] -Input [1]: [promotions#24] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(43) CometFilter -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_store_sk#27) AND isnotnull(ss_customer_sk#26)) AND isnotnull(ss_item_sk#25)) - -(44) ReusedExchange [Reuses operator id: 6] -Output [1]: [s_store_sk#31] - -(45) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Right output [1]: [s_store_sk#31] -Arguments: [ss_store_sk#27], [s_store_sk#31], Inner, BuildRight - -(46) CometProject -Input [6]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, s_store_sk#31] -Arguments: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29], [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] - -(47) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#32] - -(48) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] -Right output [1]: [d_date_sk#32] -Arguments: [ss_sold_date_sk#29], [d_date_sk#32], Inner, BuildRight - -(49) CometProject -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#32] -Arguments: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28], [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] - -(50) ReusedExchange [Reuses operator id: 23] -Output [2]: [c_customer_sk#33, c_current_addr_sk#34] - -(51) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] -Right output [2]: [c_customer_sk#33, c_current_addr_sk#34] -Arguments: [ss_customer_sk#26], [c_customer_sk#33], Inner, BuildRight - -(52) CometProject -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, c_customer_sk#33, c_current_addr_sk#34] -Arguments: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34], [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] - -(53) ReusedExchange [Reuses operator id: 29] -Output [1]: [ca_address_sk#35] - -(54) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] -Right output [1]: [ca_address_sk#35] -Arguments: [c_current_addr_sk#34], [ca_address_sk#35], Inner, BuildRight - -(55) CometProject -Input [4]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34, ca_address_sk#35] -Arguments: [ss_item_sk#25, ss_ext_sales_price#28], [ss_item_sk#25, ss_ext_sales_price#28] - -(56) ReusedExchange [Reuses operator id: 35] -Output [1]: [i_item_sk#36] - -(57) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#25, ss_ext_sales_price#28] -Right output [1]: [i_item_sk#36] -Arguments: [ss_item_sk#25], [i_item_sk#36], Inner, BuildRight - -(58) CometProject -Input [3]: [ss_item_sk#25, ss_ext_sales_price#28, i_item_sk#36] -Arguments: [ss_ext_sales_price#28], [ss_ext_sales_price#28] - -(59) CometHashAggregate -Input [1]: [ss_ext_sales_price#28] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#28))] - -(60) CometExchange -Input [1]: [sum#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(61) CometHashAggregate -Input [1]: [sum#37] -Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] - -(62) CometColumnarToRow [codegen id : 1] -Input [1]: [total#38] - -(63) BroadcastExchange -Input [1]: [total#38] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(64) BroadcastNestedLoopJoin [codegen id : 2] -Join type: Inner -Join condition: None - -(65) Project [codegen id : 2] -Output [3]: [promotions#24, total#38, ((cast(promotions#24 as decimal(15,4)) / cast(total#38 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#39] -Input [2]: [promotions#24, total#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (70) -+- * CometColumnarToRow (69) - +- CometProject (68) - +- CometFilter (67) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) - - -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#14, d_year#15, d_moy#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(67) CometFilter -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1998)) AND (d_moy#16 = 11)) AND isnotnull(d_date_sk#14)) - -(68) CometProject -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(69) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(70) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 42 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/simplified.txt deleted file mode 100644 index 6ebe25b753..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,79 +0,0 @@ -WholeStageCodegen (2) - Project [promotions,total] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [promotions,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange #1 - CometHashAggregate [ss_ext_sales_price] [sum] - CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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_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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk] #3 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - CometBroadcastExchange [p_promo_sk] #4 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk] #7 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_category] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [total,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange #10 - CometHashAggregate [ss_ext_sales_price] [sum] - CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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_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_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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #5 - ReusedExchange [c_customer_sk,c_current_addr_sk] #6 - ReusedExchange [ca_address_sk] #7 - ReusedExchange [i_item_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt deleted file mode 100644 index 4f090653c8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt +++ /dev/null @@ -1,397 +0,0 @@ -== Physical Plan == -* Project (65) -+- * BroadcastNestedLoopJoin Inner BuildRight (64) - :- * CometColumnarToRow (41) - : +- CometHashAggregate (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometBroadcastHashJoin (19) - : : : : :- CometProject (14) - : : : : : +- CometBroadcastHashJoin (13) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (6) - : : : : : : +- CometProject (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (3) - : : : : : +- CometBroadcastExchange (12) - : : : : : +- CometProject (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (9) - : : : : +- CometBroadcastExchange (18) - : : : : +- CometProject (17) - : : : : +- CometFilter (16) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (15) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (21) - : : +- CometBroadcastExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (26) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) - +- BroadcastExchange (63) - +- * CometColumnarToRow (62) - +- CometHashAggregate (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometProject (55) - : +- CometBroadcastHashJoin (54) - : :- CometProject (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometProject (49) - : : : +- CometBroadcastHashJoin (48) - : : : :- CometProject (46) - : : : : +- CometBroadcastHashJoin (45) - : : : : :- CometFilter (43) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (42) - : : : : +- ReusedExchange (44) - : : : +- ReusedExchange (47) - : : +- ReusedExchange (50) - : +- ReusedExchange (53) - +- ReusedExchange (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_store_sk#3) AND isnotnull(ss_promo_sk#4)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_gmt_offset#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [s_store_sk#8, s_gmt_offset#9] -Condition : ((isnotnull(s_gmt_offset#9) AND (s_gmt_offset#9 = -5.00)) AND isnotnull(s_store_sk#8)) - -(5) CometProject -Input [2]: [s_store_sk#8, s_gmt_offset#9] -Arguments: [s_store_sk#8], [s_store_sk#8] - -(6) CometBroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: [s_store_sk#8] - -(7) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [s_store_sk#8] -Arguments: [ss_store_sk#3], [s_store_sk#8], Inner, BuildRight - -(8) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, s_store_sk#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6], [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(10) CometFilter -Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_dmail#11, 1, true, false, true) = Y) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_email#12, 1, true, false, true) = Y)) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#13, 1, true, false, true) = Y)) AND isnotnull(p_promo_sk#10)) - -(11) CometProject -Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] -Arguments: [p_promo_sk#10], [p_promo_sk#10] - -(12) CometBroadcastExchange -Input [1]: [p_promo_sk#10] -Arguments: [p_promo_sk#10] - -(13) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [p_promo_sk#10] -Arguments: [ss_promo_sk#4], [p_promo_sk#10], Inner, BuildRight - -(14) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6, p_promo_sk#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6], [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#14, d_year#15, d_moy#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1998)) AND (d_moy#16 = 11)) AND isnotnull(d_date_sk#14)) - -(17) CometProject -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(18) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(19) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#6], [d_date_sk#14], Inner, BuildRight - -(20) CometProject -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5], [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(c_current_addr_sk#18)) - -(23) CometBroadcastExchange -Input [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [c_customer_sk#17, c_current_addr_sk#18] - -(24) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] -Right output [2]: [c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_customer_sk#2], [c_customer_sk#17], Inner, BuildRight - -(25) CometProject -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#17, c_current_addr_sk#18] -Arguments: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18], [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_gmt_offset#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [ca_address_sk#19, ca_gmt_offset#20] -Condition : ((isnotnull(ca_gmt_offset#20) AND (ca_gmt_offset#20 = -5.00)) AND isnotnull(ca_address_sk#19)) - -(28) CometProject -Input [2]: [ca_address_sk#19, ca_gmt_offset#20] -Arguments: [ca_address_sk#19], [ca_address_sk#19] - -(29) CometBroadcastExchange -Input [1]: [ca_address_sk#19] -Arguments: [ca_address_sk#19] - -(30) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18] -Right output [1]: [ca_address_sk#19] -Arguments: [c_current_addr_sk#18], [ca_address_sk#19], Inner, BuildRight - -(31) CometProject -Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#18, ca_address_sk#19] -Arguments: [ss_item_sk#1, ss_ext_sales_price#5], [ss_item_sk#1, ss_ext_sales_price#5] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#21, i_category#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [i_item_sk#21, i_category#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#22, 50, true, false, true) = Jewelry ) AND isnotnull(i_item_sk#21)) - -(34) CometProject -Input [2]: [i_item_sk#21, i_category#22] -Arguments: [i_item_sk#21], [i_item_sk#21] - -(35) CometBroadcastExchange -Input [1]: [i_item_sk#21] -Arguments: [i_item_sk#21] - -(36) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#1, ss_ext_sales_price#5] -Right output [1]: [i_item_sk#21] -Arguments: [ss_item_sk#1], [i_item_sk#21], Inner, BuildRight - -(37) CometProject -Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#21] -Arguments: [ss_ext_sales_price#5], [ss_ext_sales_price#5] - -(38) CometHashAggregate -Input [1]: [ss_ext_sales_price#5] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] - -(39) CometExchange -Input [1]: [sum#23] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(40) CometHashAggregate -Input [1]: [sum#23] -Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] - -(41) CometColumnarToRow [codegen id : 2] -Input [1]: [promotions#24] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(43) CometFilter -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_store_sk#27) AND isnotnull(ss_customer_sk#26)) AND isnotnull(ss_item_sk#25)) - -(44) ReusedExchange [Reuses operator id: 6] -Output [1]: [s_store_sk#31] - -(45) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Right output [1]: [s_store_sk#31] -Arguments: [ss_store_sk#27], [s_store_sk#31], Inner, BuildRight - -(46) CometProject -Input [6]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, s_store_sk#31] -Arguments: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29], [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] - -(47) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#32] - -(48) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] -Right output [1]: [d_date_sk#32] -Arguments: [ss_sold_date_sk#29], [d_date_sk#32], Inner, BuildRight - -(49) CometProject -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#32] -Arguments: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28], [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] - -(50) ReusedExchange [Reuses operator id: 23] -Output [2]: [c_customer_sk#33, c_current_addr_sk#34] - -(51) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] -Right output [2]: [c_customer_sk#33, c_current_addr_sk#34] -Arguments: [ss_customer_sk#26], [c_customer_sk#33], Inner, BuildRight - -(52) CometProject -Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, c_customer_sk#33, c_current_addr_sk#34] -Arguments: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34], [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] - -(53) ReusedExchange [Reuses operator id: 29] -Output [1]: [ca_address_sk#35] - -(54) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] -Right output [1]: [ca_address_sk#35] -Arguments: [c_current_addr_sk#34], [ca_address_sk#35], Inner, BuildRight - -(55) CometProject -Input [4]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34, ca_address_sk#35] -Arguments: [ss_item_sk#25, ss_ext_sales_price#28], [ss_item_sk#25, ss_ext_sales_price#28] - -(56) ReusedExchange [Reuses operator id: 35] -Output [1]: [i_item_sk#36] - -(57) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#25, ss_ext_sales_price#28] -Right output [1]: [i_item_sk#36] -Arguments: [ss_item_sk#25], [i_item_sk#36], Inner, BuildRight - -(58) CometProject -Input [3]: [ss_item_sk#25, ss_ext_sales_price#28, i_item_sk#36] -Arguments: [ss_ext_sales_price#28], [ss_ext_sales_price#28] - -(59) CometHashAggregate -Input [1]: [ss_ext_sales_price#28] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#28))] - -(60) CometExchange -Input [1]: [sum#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(61) CometHashAggregate -Input [1]: [sum#37] -Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] - -(62) CometColumnarToRow [codegen id : 1] -Input [1]: [total#38] - -(63) BroadcastExchange -Input [1]: [total#38] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(64) BroadcastNestedLoopJoin [codegen id : 2] -Join type: Inner -Join condition: None - -(65) Project [codegen id : 2] -Output [3]: [promotions#24, total#38, ((cast(promotions#24 as decimal(15,4)) / cast(total#38 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#39] -Input [2]: [promotions#24, total#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (70) -+- * CometColumnarToRow (69) - +- CometProject (68) - +- CometFilter (67) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) - - -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#14, d_year#15, d_moy#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(67) CometFilter -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Condition : ((((isnotnull(d_year#15) AND isnotnull(d_moy#16)) AND (d_year#15 = 1998)) AND (d_moy#16 = 11)) AND isnotnull(d_date_sk#14)) - -(68) CometProject -Input [3]: [d_date_sk#14, d_year#15, d_moy#16] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(69) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(70) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 42 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/extended.txt deleted file mode 100644 index 7bae7d5724..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/extended.txt +++ /dev/null @@ -1,88 +0,0 @@ -Project -+- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 77 out of 83 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 6ebe25b753..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/simplified.txt +++ /dev/null @@ -1,79 +0,0 @@ -WholeStageCodegen (2) - Project [promotions,total] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [promotions,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange #1 - CometHashAggregate [ss_ext_sales_price] [sum] - CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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_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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk] #3 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] - CometBroadcastExchange [p_promo_sk] #4 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #6 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk] #7 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_category] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [total,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange #10 - CometHashAggregate [ss_ext_sales_price] [sum] - CometProject [ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk] - CometProject [ss_item_sk,ss_ext_sales_price] - 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_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_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_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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #5 - ReusedExchange [c_customer_sk,c_current_addr_sk] #6 - ReusedExchange [ca_address_sk] #7 - ReusedExchange [i_item_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/explain.txt deleted file mode 100644 index 1d9748055e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometNativeScan parquet spark_catalog.default.warehouse (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometNativeScan parquet spark_catalog.default.ship_mode (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.web_site (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometNativeScan parquet spark_catalog.default.date_dim (19) - - -(1) CometNativeScan parquet spark_catalog.default.web_sales -Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_ship_mode_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Condition : (((isnotnull(ws_warehouse_sk#4) AND isnotnull(ws_ship_mode_sk#3)) AND isnotnull(ws_web_site_sk#2)) AND isnotnull(ws_ship_date_sk#1)) - -(3) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [ws_warehouse_sk#4], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7], [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] - -(8) CometNativeScan parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#8, sm_type#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Condition : isnotnull(sm_ship_mode_sk#8) - -(10) CometProject -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Arguments: [sm_ship_mode_sk#8, sm_type#10], [sm_ship_mode_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, sm_type#9, 30, true, false, true) AS sm_type#10] - -(11) CometBroadcastExchange -Input [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [sm_ship_mode_sk#8, sm_type#10] - -(12) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] -Right output [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [ws_ship_mode_sk#3], [sm_ship_mode_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#10] -Arguments: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10], [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10] - -(14) CometNativeScan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#11, web_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [web_site_sk#11, web_name#12] -Condition : isnotnull(web_site_sk#11) - -(16) CometBroadcastExchange -Input [2]: [web_site_sk#11, web_name#12] -Arguments: [web_site_sk#11, web_name#12] - -(17) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10] -Right output [2]: [web_site_sk#11, web_name#12] -Arguments: [ws_web_site_sk#2], [web_site_sk#11], Inner, BuildRight - -(18) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_site_sk#11, web_name#12] -Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12], [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12] - -(19) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_month_seq#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#13, d_month_seq#14] -Condition : (((isnotnull(d_month_seq#14) AND (d_month_seq#14 >= 1200)) AND (d_month_seq#14 <= 1211)) AND isnotnull(d_date_sk#13)) - -(21) CometProject -Input [2]: [d_date_sk#13, d_month_seq#14] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(23) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12] -Right output [1]: [d_date_sk#13] -Arguments: [ws_ship_date_sk#1], [d_date_sk#13], Inner, BuildRight - -(24) CometProject -Input [6]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12, d_date_sk#13] -Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, _groupingexpression#15], [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#15] - -(25) CometHashAggregate -Input [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, sm_type#10, web_name#12] -Functions [5]: [partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [8]: [_groupingexpression#15, sm_type#10, web_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Arguments: hashpartitioning(_groupingexpression#15, sm_type#10, web_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [8]: [_groupingexpression#15, sm_type#10, web_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Keys [3]: [_groupingexpression#15, sm_type#10, web_name#12] -Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,web_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,web_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, web_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - -(29) CometColumnarToRow [codegen id : 1] -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/simplified.txt deleted file mode 100644 index de80c17349..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 30) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 60) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 90) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [_groupingexpression,sm_type,web_name] #1 - CometHashAggregate [ws_ship_date_sk,ws_sold_date_sk] [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum] - CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] - 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_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_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_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] - CometNativeScan 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 [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [web_site_sk,web_name] #4 - CometFilter [web_site_sk,web_name] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan 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/q62.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_iceberg_compat/explain.txt deleted file mode 100644 index a006a92dd0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_ship_mode_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Condition : (((isnotnull(ws_warehouse_sk#4) AND isnotnull(ws_ship_mode_sk#3)) AND isnotnull(ws_web_site_sk#2)) AND isnotnull(ws_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [ws_warehouse_sk#4], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7], [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#8, sm_type#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Condition : isnotnull(sm_ship_mode_sk#8) - -(10) CometProject -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Arguments: [sm_ship_mode_sk#8, sm_type#10], [sm_ship_mode_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, sm_type#9, 30, true, false, true) AS sm_type#10] - -(11) CometBroadcastExchange -Input [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [sm_ship_mode_sk#8, sm_type#10] - -(12) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] -Right output [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [ws_ship_mode_sk#3], [sm_ship_mode_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#10] -Arguments: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10], [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#11, web_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [web_site_sk#11, web_name#12] -Condition : isnotnull(web_site_sk#11) - -(16) CometBroadcastExchange -Input [2]: [web_site_sk#11, web_name#12] -Arguments: [web_site_sk#11, web_name#12] - -(17) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10] -Right output [2]: [web_site_sk#11, web_name#12] -Arguments: [ws_web_site_sk#2], [web_site_sk#11], Inner, BuildRight - -(18) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_site_sk#11, web_name#12] -Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12], [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_month_seq#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#13, d_month_seq#14] -Condition : (((isnotnull(d_month_seq#14) AND (d_month_seq#14 >= 1200)) AND (d_month_seq#14 <= 1211)) AND isnotnull(d_date_sk#13)) - -(21) CometProject -Input [2]: [d_date_sk#13, d_month_seq#14] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(23) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12] -Right output [1]: [d_date_sk#13] -Arguments: [ws_ship_date_sk#1], [d_date_sk#13], Inner, BuildRight - -(24) CometProject -Input [6]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12, d_date_sk#13] -Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, _groupingexpression#15], [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#15] - -(25) CometHashAggregate -Input [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, sm_type#10, web_name#12] -Functions [5]: [partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [8]: [_groupingexpression#15, sm_type#10, web_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Arguments: hashpartitioning(_groupingexpression#15, sm_type#10, web_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [8]: [_groupingexpression#15, sm_type#10, web_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Keys [3]: [_groupingexpression#15, sm_type#10, web_name#12] -Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,web_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,web_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, web_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - -(29) CometColumnarToRow [codegen id : 1] -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_iceberg_compat/simplified.txt deleted file mode 100644 index dee2e434c7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 30) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 60) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 90) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [_groupingexpression,sm_type,web_name] #1 - CometHashAggregate [ws_ship_date_sk,ws_sold_date_sk] [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum] - CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] - 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_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_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_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 [native_iceberg_compat] 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 [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [web_site_sk,web_name] #4 - CometFilter [web_site_sk,web_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] 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/q62/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt deleted file mode 100644 index a006a92dd0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_ship_mode_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Condition : (((isnotnull(ws_warehouse_sk#4) AND isnotnull(ws_ship_mode_sk#3)) AND isnotnull(ws_web_site_sk#2)) AND isnotnull(ws_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [ws_warehouse_sk#4], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7], [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#8, sm_type#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Condition : isnotnull(sm_ship_mode_sk#8) - -(10) CometProject -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Arguments: [sm_ship_mode_sk#8, sm_type#10], [sm_ship_mode_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, sm_type#9, 30, true, false, true) AS sm_type#10] - -(11) CometBroadcastExchange -Input [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [sm_ship_mode_sk#8, sm_type#10] - -(12) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] -Right output [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [ws_ship_mode_sk#3], [sm_ship_mode_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#10] -Arguments: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10], [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#11, web_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [web_site_sk#11, web_name#12] -Condition : isnotnull(web_site_sk#11) - -(16) CometBroadcastExchange -Input [2]: [web_site_sk#11, web_name#12] -Arguments: [web_site_sk#11, web_name#12] - -(17) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10] -Right output [2]: [web_site_sk#11, web_name#12] -Arguments: [ws_web_site_sk#2], [web_site_sk#11], Inner, BuildRight - -(18) CometProject -Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_site_sk#11, web_name#12] -Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12], [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_month_seq#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#13, d_month_seq#14] -Condition : (((isnotnull(d_month_seq#14) AND (d_month_seq#14 >= 1200)) AND (d_month_seq#14 <= 1211)) AND isnotnull(d_date_sk#13)) - -(21) CometProject -Input [2]: [d_date_sk#13, d_month_seq#14] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(23) CometBroadcastHashJoin -Left output [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12] -Right output [1]: [d_date_sk#13] -Arguments: [ws_ship_date_sk#1], [d_date_sk#13], Inner, BuildRight - -(24) CometProject -Input [6]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#10, web_name#12, d_date_sk#13] -Arguments: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, _groupingexpression#15], [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#15] - -(25) CometHashAggregate -Input [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#10, web_name#12, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, sm_type#10, web_name#12] -Functions [5]: [partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [8]: [_groupingexpression#15, sm_type#10, web_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Arguments: hashpartitioning(_groupingexpression#15, sm_type#10, web_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [8]: [_groupingexpression#15, sm_type#10, web_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Keys [3]: [_groupingexpression#15, sm_type#10, web_name#12] -Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,web_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,web_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, web_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - -(29) CometColumnarToRow [codegen id : 1] -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, web_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/extended.txt deleted file mode 100644 index a5fd7359a3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 28 out of 28 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index dee2e434c7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 30) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 60) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 90) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [_groupingexpression,sm_type,web_name] #1 - CometHashAggregate [ws_ship_date_sk,ws_sold_date_sk] [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum] - CometProject [w_warehouse_name] [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,_groupingexpression] - 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_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_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_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 [native_iceberg_compat] 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 [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [web_site_sk,web_name] #4 - CometFilter [web_site_sk,web_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/explain.txt deleted file mode 100644 index 32f70c3d33..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/explain.txt +++ /dev/null @@ -1,202 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * CometColumnarToRow (26) - +- CometSort (25) - +- CometColumnarExchange (24) - +- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometColumnarExchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * CometColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (8) - : : +- * Filter (7) - : : +- * ColumnarToRow (6) - : : +- Scan parquet spark_catalog.default.store_sales (5) - : +- ReusedExchange (11) - +- BroadcastExchange (17) - +- * CometColumnarToRow (16) - +- CometFilter (15) - +- CometNativeScan parquet spark_catalog.default.store (14) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Books ,Children ,Electronics ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (personal ,portable ,refernece ,self-help )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Women ,Music ,Men ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (accessories ,classical ,fragrances ,pants )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Arguments: [i_item_sk#1, i_manager_id#5], [i_item_sk#1, i_manager_id#5] - -(4) CometColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#1, i_manager_id#5] - -(5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(7) Filter [codegen id : 1] -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) - -(8) BroadcastExchange -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#10] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Input [6]: [i_item_sk#1, i_manager_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(11) ReusedExchange [Reuses operator id: 35] -Output [2]: [d_date_sk#15, d_moy#16] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#13] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#16] -Input [6]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_moy#16] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [1]: [s_store_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [1]: [s_store_sk#17] -Condition : isnotnull(s_store_sk#17) - -(16) CometColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#17] - -(17) BroadcastExchange -Input [1]: [s_store_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#11] -Right keys [1]: [s_store_sk#17] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [3]: [i_manager_id#5, ss_sales_price#12, d_moy#16] -Input [5]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#16, s_store_sk#17] - -(20) HashAggregate [codegen id : 4] -Input [3]: [i_manager_id#5, ss_sales_price#12, d_moy#16] -Keys [2]: [i_manager_id#5, d_moy#16] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [i_manager_id#5, d_moy#16, sum#19] - -(21) CometColumnarExchange -Input [3]: [i_manager_id#5, d_moy#16, sum#19] -Arguments: hashpartitioning(i_manager_id#5, d_moy#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [3]: [i_manager_id#5, d_moy#16, sum#19] - -(23) HashAggregate [codegen id : 5] -Input [3]: [i_manager_id#5, d_moy#16, sum#19] -Keys [2]: [i_manager_id#5, d_moy#16] -Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#20] -Results [3]: [i_manager_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS _w0#22] - -(24) CometColumnarExchange -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: [i_manager_id#5, sum_sales#21, _w0#22], [i_manager_id#5 ASC NULLS FIRST] - -(26) CometColumnarToRow [codegen id : 6] -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] - -(27) Window -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_manager_id#5] - -(28) Filter [codegen id : 7] -Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] -Condition : CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) ELSE false END - -(29) Project [codegen id : 7] -Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] -Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] - -(30) TakeOrderedAndProject -Input [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] -Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometNativeScan parquet spark_catalog.default.date_dim (31) - - -(31) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] -Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(33) CometProject -Input [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] -Arguments: [d_date_sk#15, d_moy#16], [d_date_sk#15, d_moy#16] - -(34) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_moy#16] - -(35) BroadcastExchange -Input [2]: [d_date_sk#15, d_moy#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/simplified.txt deleted file mode 100644 index 2933149226..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/simplified.txt +++ /dev/null @@ -1,53 +0,0 @@ -TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (7) - Project [i_manager_id,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_manager_id] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_manager_id,sum_sales,_w0] - CometColumnarExchange [i_manager_id] #1 - WholeStageCodegen (5) - HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manager_id,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] - Project [i_manager_id,ss_sales_price,d_moy] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_sk,i_manager_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/explain.txt deleted file mode 100644 index e2a0cceb1a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,198 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) - +- CometSort (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Books ,Children ,Electronics ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (personal ,portable ,refernece ,self-help )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Women ,Music ,Men ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (accessories ,classical ,fragrances ,pants )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Arguments: [i_item_sk#1, i_manager_id#5], [i_item_sk#1, i_manager_id#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(7) CometBroadcastHashJoin -Left output [2]: [i_item_sk#1, i_manager_id#5] -Right output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_item_sk#1], [ss_item_sk#10], Inner, BuildRight - -(8) CometProject -Input [6]: [i_item_sk#1, i_manager_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13], [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(11) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] - -(12) CometBroadcastExchange -Input [2]: [d_date_sk#15, d_moy#17] -Arguments: [d_date_sk#15, d_moy#17] - -(13) CometBroadcastHashJoin -Left output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_moy#17] -Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(14) CometProject -Input [6]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_moy#17] -Arguments: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17], [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [s_store_sk#18] -Condition : isnotnull(s_store_sk#18) - -(17) CometBroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: [s_store_sk#18] - -(18) CometBroadcastHashJoin -Left output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17] -Right output [1]: [s_store_sk#18] -Arguments: [ss_store_sk#11], [s_store_sk#18], Inner, BuildRight - -(19) CometProject -Input [5]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17, s_store_sk#18] -Arguments: [i_manager_id#5, ss_sales_price#12, d_moy#17], [i_manager_id#5, ss_sales_price#12, d_moy#17] - -(20) CometHashAggregate -Input [3]: [i_manager_id#5, ss_sales_price#12, d_moy#17] -Keys [2]: [i_manager_id#5, d_moy#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] - -(21) CometExchange -Input [3]: [i_manager_id#5, d_moy#17, sum#19] -Arguments: hashpartitioning(i_manager_id#5, d_moy#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [3]: [i_manager_id#5, d_moy#17, sum#19] -Keys [2]: [i_manager_id#5, d_moy#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] - -(23) CometExchange -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: [i_manager_id#5, sum_sales#20, _w0#21], [i_manager_id#5 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 1] -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] - -(26) Window -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_manager_id#5] - -(27) Filter [codegen id : 2] -Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] -Condition : CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) ELSE false END - -(28) Project [codegen id : 2] -Output [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] -Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] - -(29) TakeOrderedAndProject -Input [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] -Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(32) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_moy#17] - -(34) BroadcastExchange -Input [2]: [d_date_sk#15, d_moy#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/simplified.txt deleted file mode 100644 index 02166879a9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (2) - Project [i_manager_id,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_manager_id] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_manager_id,sum_sales,_w0] - CometExchange [i_manager_id] #1 - CometHashAggregate [d_moy,sum] [i_manager_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_manager_id,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] - CometProject [i_manager_id,ss_sales_price,d_moy] - 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 [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,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_item_sk,i_brand,i_class,i_category,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt deleted file mode 100644 index e2a0cceb1a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt +++ /dev/null @@ -1,198 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) - +- CometSort (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Books ,Children ,Electronics ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (personal ,portable ,refernece ,self-help )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Women ,Music ,Men ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (accessories ,classical ,fragrances ,pants )) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Arguments: [i_item_sk#1, i_manager_id#5], [i_item_sk#1, i_manager_id#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(7) CometBroadcastHashJoin -Left output [2]: [i_item_sk#1, i_manager_id#5] -Right output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_item_sk#1], [ss_item_sk#10], Inner, BuildRight - -(8) CometProject -Input [6]: [i_item_sk#1, i_manager_id#5, ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Arguments: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13], [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(11) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] - -(12) CometBroadcastExchange -Input [2]: [d_date_sk#15, d_moy#17] -Arguments: [d_date_sk#15, d_moy#17] - -(13) CometBroadcastHashJoin -Left output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_moy#17] -Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(14) CometProject -Input [6]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_moy#17] -Arguments: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17], [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [s_store_sk#18] -Condition : isnotnull(s_store_sk#18) - -(17) CometBroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: [s_store_sk#18] - -(18) CometBroadcastHashJoin -Left output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17] -Right output [1]: [s_store_sk#18] -Arguments: [ss_store_sk#11], [s_store_sk#18], Inner, BuildRight - -(19) CometProject -Input [5]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#17, s_store_sk#18] -Arguments: [i_manager_id#5, ss_sales_price#12, d_moy#17], [i_manager_id#5, ss_sales_price#12, d_moy#17] - -(20) CometHashAggregate -Input [3]: [i_manager_id#5, ss_sales_price#12, d_moy#17] -Keys [2]: [i_manager_id#5, d_moy#17] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] - -(21) CometExchange -Input [3]: [i_manager_id#5, d_moy#17, sum#19] -Arguments: hashpartitioning(i_manager_id#5, d_moy#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [3]: [i_manager_id#5, d_moy#17, sum#19] -Keys [2]: [i_manager_id#5, d_moy#17] -Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] - -(23) CometExchange -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: [i_manager_id#5, sum_sales#20, _w0#21], [i_manager_id#5 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 1] -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] - -(26) Window -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_manager_id#5] - -(27) Filter [codegen id : 2] -Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] -Condition : CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) ELSE false END - -(28) Project [codegen id : 2] -Output [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] -Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] - -(29) TakeOrderedAndProject -Input [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] -Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) - -(32) CometProject -Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] -Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#15, d_moy#17] - -(34) BroadcastExchange -Input [2]: [d_date_sk#15, d_moy#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/extended.txt deleted file mode 100644 index 825b1ed81c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 02166879a9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (2) - Project [i_manager_id,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_manager_id] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_manager_id,sum_sales,_w0] - CometExchange [i_manager_id] #1 - CometHashAggregate [d_moy,sum] [i_manager_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_manager_id,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] - CometProject [i_manager_id,ss_sales_price,d_moy] - 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 [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,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_item_sk,i_brand,i_class,i_category,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/explain.txt deleted file mode 100644 index c021738ca6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/explain.txt +++ /dev/null @@ -1,1029 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (180) -+- CometSort (179) - +- CometExchange (178) - +- CometProject (177) - +- CometSortMergeJoin (176) - :- CometSort (105) - : +- CometExchange (104) - : +- CometHashAggregate (103) - : +- CometHashAggregate (102) - : +- CometProject (101) - : +- CometBroadcastHashJoin (100) - : :- CometProject (95) - : : +- CometBroadcastHashJoin (94) - : : :- CometProject (92) - : : : +- CometBroadcastHashJoin (91) - : : : :- CometProject (87) - : : : : +- CometBroadcastHashJoin (86) - : : : : :- CometProject (84) - : : : : : +- CometBroadcastHashJoin (83) - : : : : : :- CometProject (78) - : : : : : : +- CometBroadcastHashJoin (77) - : : : : : : :- CometProject (75) - : : : : : : : +- CometBroadcastHashJoin (74) - : : : : : : : :- CometProject (70) - : : : : : : : : +- CometBroadcastHashJoin (69) - : : : : : : : : :- CometProject (65) - : : : : : : : : : +- CometBroadcastHashJoin (64) - : : : : : : : : : :- CometProject (62) - : : : : : : : : : : +- CometBroadcastHashJoin (61) - : : : : : : : : : : :- CometProject (56) - : : : : : : : : : : : +- CometBroadcastHashJoin (55) - : : : : : : : : : : : :- CometProject (53) - : : : : : : : : : : : : +- CometBroadcastHashJoin (52) - : : : : : : : : : : : : :- CometProject (48) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (47) - : : : : : : : : : : : : : :- CometProject (43) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (42) - : : : : : : : : : : : : : : :- CometProject (37) - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (36) - : : : : : : : : : : : : : : : :- CometProject (32) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (31) - : : : : : : : : : : : : : : : : :- CometSort (12) - : : : : : : : : : : : : : : : : : +- CometColumnarExchange (11) - : : : : : : : : : : : : : : : : : +- * Project (10) - : : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : : : : : : : : : : : : : : :- BroadcastExchange (4) - : : : : : : : : : : : : : : : : : : +- * Filter (3) - : : : : : : : : : : : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (8) - : : : : : : : : : : : : : : : : : +- CometProject (7) - : : : : : : : : : : : : : : : : : +- CometFilter (6) - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (5) - : : : : : : : : : : : : : : : : +- CometSort (30) - : : : : : : : : : : : : : : : : +- CometProject (29) - : : : : : : : : : : : : : : : : +- CometFilter (28) - : : : : : : : : : : : : : : : : +- CometHashAggregate (27) - : : : : : : : : : : : : : : : : +- CometExchange (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometProject (24) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (23) - : : : : : : : : : : : : : : : : :- CometSort (17) - : : : : : : : : : : : : : : : : : +- CometExchange (16) - : : : : : : : : : : : : : : : : : +- CometProject (15) - : : : : : : : : : : : : : : : : : +- CometFilter (14) - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : : : : : : : : : : : +- CometSort (22) - : : : : : : : : : : : : : : : : +- CometExchange (21) - : : : : : : : : : : : : : : : : +- CometProject (20) - : : : : : : : : : : : : : : : : +- CometFilter (19) - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (18) - : : : : : : : : : : : : : : : +- CometBroadcastExchange (35) - : : : : : : : : : : : : : : : +- CometFilter (34) - : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (33) - : : : : : : : : : : : : : : +- CometBroadcastExchange (41) - : : : : : : : : : : : : : : +- CometProject (40) - : : : : : : : : : : : : : : +- CometFilter (39) - : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store (38) - : : : : : : : : : : : : : +- CometBroadcastExchange (46) - : : : : : : : : : : : : : +- CometFilter (45) - : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer (44) - : : : : : : : : : : : : +- CometBroadcastExchange (51) - : : : : : : : : : : : : +- CometFilter (50) - : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (49) - : : : : : : : : : : : +- ReusedExchange (54) - : : : : : : : : : : +- CometBroadcastExchange (60) - : : : : : : : : : : +- CometProject (59) - : : : : : : : : : : +- CometFilter (58) - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (57) - : : : : : : : : : +- ReusedExchange (63) - : : : : : : : : +- CometBroadcastExchange (68) - : : : : : : : : +- CometFilter (67) - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.promotion (66) - : : : : : : : +- CometBroadcastExchange (73) - : : : : : : : +- CometFilter (72) - : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (71) - : : : : : : +- ReusedExchange (76) - : : : : : +- CometBroadcastExchange (82) - : : : : : +- CometProject (81) - : : : : : +- CometFilter (80) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address (79) - : : : : +- ReusedExchange (85) - : : : +- CometBroadcastExchange (90) - : : : +- CometFilter (89) - : : : +- CometNativeScan parquet spark_catalog.default.income_band (88) - : : +- ReusedExchange (93) - : +- CometBroadcastExchange (99) - : +- CometProject (98) - : +- CometFilter (97) - : +- CometNativeScan parquet spark_catalog.default.item (96) - +- CometSort (175) - +- CometExchange (174) - +- CometHashAggregate (173) - +- CometHashAggregate (172) - +- CometProject (171) - +- CometBroadcastHashJoin (170) - :- CometProject (168) - : +- CometBroadcastHashJoin (167) - : :- CometProject (165) - : : +- CometBroadcastHashJoin (164) - : : :- CometProject (162) - : : : +- CometBroadcastHashJoin (161) - : : : :- CometProject (159) - : : : : +- CometBroadcastHashJoin (158) - : : : : :- CometProject (156) - : : : : : +- CometBroadcastHashJoin (155) - : : : : : :- CometProject (153) - : : : : : : +- CometBroadcastHashJoin (152) - : : : : : : :- CometProject (150) - : : : : : : : +- CometBroadcastHashJoin (149) - : : : : : : : :- CometProject (147) - : : : : : : : : +- CometBroadcastHashJoin (146) - : : : : : : : : :- CometProject (144) - : : : : : : : : : +- CometBroadcastHashJoin (143) - : : : : : : : : : :- CometProject (141) - : : : : : : : : : : +- CometBroadcastHashJoin (140) - : : : : : : : : : : :- CometProject (138) - : : : : : : : : : : : +- CometBroadcastHashJoin (137) - : : : : : : : : : : : :- CometProject (135) - : : : : : : : : : : : : +- CometBroadcastHashJoin (134) - : : : : : : : : : : : : :- CometProject (132) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (131) - : : : : : : : : : : : : : :- CometProject (129) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (128) - : : : : : : : : : : : : : : :- CometProject (124) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (123) - : : : : : : : : : : : : : : : :- CometSort (117) - : : : : : : : : : : : : : : : : +- CometColumnarExchange (116) - : : : : : : : : : : : : : : : : +- * Project (115) - : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (114) - : : : : : : : : : : : : : : : : :- BroadcastExchange (109) - : : : : : : : : : : : : : : : : : +- * Filter (108) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (107) - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (106) - : : : : : : : : : : : : : : : : +- * CometColumnarToRow (113) - : : : : : : : : : : : : : : : : +- CometProject (112) - : : : : : : : : : : : : : : : : +- CometFilter (111) - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (110) - : : : : : : : : : : : : : : : +- CometSort (122) - : : : : : : : : : : : : : : : +- CometProject (121) - : : : : : : : : : : : : : : : +- CometFilter (120) - : : : : : : : : : : : : : : : +- CometHashAggregate (119) - : : : : : : : : : : : : : : : +- ReusedExchange (118) - : : : : : : : : : : : : : : +- CometBroadcastExchange (127) - : : : : : : : : : : : : : : +- CometFilter (126) - : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (125) - : : : : : : : : : : : : : +- ReusedExchange (130) - : : : : : : : : : : : : +- ReusedExchange (133) - : : : : : : : : : : : +- ReusedExchange (136) - : : : : : : : : : : +- ReusedExchange (139) - : : : : : : : : : +- ReusedExchange (142) - : : : : : : : : +- ReusedExchange (145) - : : : : : : : +- ReusedExchange (148) - : : : : : : +- ReusedExchange (151) - : : : : : +- ReusedExchange (154) - : : : : +- ReusedExchange (157) - : : : +- ReusedExchange (160) - : : +- ReusedExchange (163) - : +- ReusedExchange (166) - +- ReusedExchange (169) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(3) Filter [codegen id : 1] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) - -(4) BroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=1] - -(5) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) - -(7) CometProject -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] - -(8) CometColumnarToRow -Input [2]: [sr_item_sk#14, sr_ticket_number#15] - -(9) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [ss_item_sk#1, ss_ticket_number#8] -Right keys [2]: [sr_item_sk#14, sr_ticket_number#15] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 2] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] - -(11) CometColumnarExchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(12) CometSort -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] - -(13) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) - -(15) CometProject -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] - -(16) CometExchange -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(17) CometSort -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] - -(18) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(19) CometFilter -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) - -(20) CometProject -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(21) CometExchange -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(22) CometSort -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] - -(23) CometSortMergeJoin -Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner - -(24) CometProject -Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(25) CometHashAggregate -Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Keys [1]: [cs_item_sk#17] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(26) CometExchange -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(27) CometHashAggregate -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Keys [1]: [cs_item_sk#17] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(28) CometFilter -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(29) CometProject -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Arguments: [cs_item_sk#17], [cs_item_sk#17] - -(30) CometSort -Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] - -(31) CometSortMergeJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [1]: [cs_item_sk#17] -Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner - -(32) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(33) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(35) CometBroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: [d_date_sk#32, d_year#33] - -(36) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight - -(37) CometProject -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] - -(38) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name)] -ReadSchema: struct - -(39) CometFilter -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true))) - -(40) CometProject -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37], [s_store_sk#34, s_store_name#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true) AS s_zip#37] - -(41) CometBroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37] - -(42) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight - -(43) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] - -(44) CometNativeScan parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(45) CometFilter -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Condition : (((((isnotnull(c_customer_sk#38) AND isnotnull(c_first_sales_date_sk#43)) AND isnotnull(c_first_shipto_date_sk#42)) AND isnotnull(c_current_cdemo_sk#39)) AND isnotnull(c_current_hdemo_sk#40)) AND isnotnull(c_current_addr_sk#41)) - -(46) CometBroadcastExchange -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(47) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -Right output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_customer_sk#2], [c_customer_sk#38], Inner, BuildRight - -(48) CometProject -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(49) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : isnotnull(d_date_sk#44) - -(51) CometBroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] - -(52) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [c_first_sales_date_sk#43], [d_date_sk#44], Inner, BuildRight - -(53) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43, d_date_sk#44, d_year#45] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] - -(54) ReusedExchange [Reuses operator id: 51] -Output [2]: [d_date_sk#46, d_year#47] - -(55) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [c_first_shipto_date_sk#42], [d_date_sk#46], Inner, BuildRight - -(56) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45, d_date_sk#46, d_year#47] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(57) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#48, cd_marital_status#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(58) CometFilter -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true))) - -(59) CometProject -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Arguments: [cd_demo_sk#48, cd_marital_status#50], [cd_demo_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true) AS cd_marital_status#50] - -(60) CometBroadcastExchange -Input [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [cd_demo_sk#48, cd_marital_status#50] - -(61) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_cdemo_sk#3], [cd_demo_sk#48], Inner, BuildRight - -(62) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] - -(63) ReusedExchange [Reuses operator id: 60] -Output [2]: [cd_demo_sk#51, cd_marital_status#52] - -(64) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -Right output [2]: [cd_demo_sk#51, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#39], [cd_demo_sk#51], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(65) CometProject -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50, cd_demo_sk#51, cd_marital_status#52] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(66) CometNativeScan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(67) CometFilter -Input [1]: [p_promo_sk#53] -Condition : isnotnull(p_promo_sk#53) - -(68) CometBroadcastExchange -Input [1]: [p_promo_sk#53] -Arguments: [p_promo_sk#53] - -(69) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [1]: [p_promo_sk#53] -Arguments: [ss_promo_sk#7], [p_promo_sk#53], Inner, BuildRight - -(70) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, p_promo_sk#53] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(71) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(72) CometFilter -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Condition : (isnotnull(hd_demo_sk#54) AND isnotnull(hd_income_band_sk#55)) - -(73) CometBroadcastExchange -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [hd_demo_sk#54, hd_income_band_sk#55] - -(74) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_hdemo_sk#4], [hd_demo_sk#54], Inner, BuildRight - -(75) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] - -(76) ReusedExchange [Reuses operator id: 73] -Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] - -(77) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -Right output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [c_current_hdemo_sk#40], [hd_demo_sk#56], Inner, BuildRight - -(78) CometProject -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] - -(79) CometNativeScan parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(80) CometFilter -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Condition : isnotnull(ca_address_sk#58) - -(81) CometProject -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ca_address_sk#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#59, 10, true, false, true) AS ca_street_number#63, ca_street_name#60, ca_city#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#62, 10, true, false, true) AS ca_zip#64] - -(82) CometBroadcastExchange -Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(83) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -Right output [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_addr_sk#5], [ca_address_sk#58], Inner, BuildRight - -(84) CometProject -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(85) ReusedExchange [Reuses operator id: 82] -Output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(86) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Right output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [c_current_addr_sk#41], [ca_address_sk#65], Inner, BuildRight - -(87) CometProject -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(88) CometNativeScan parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(89) CometFilter -Input [1]: [ib_income_band_sk#70] -Condition : isnotnull(ib_income_band_sk#70) - -(90) CometBroadcastExchange -Input [1]: [ib_income_band_sk#70] -Arguments: [ib_income_band_sk#70] - -(91) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#70] -Arguments: [hd_income_band_sk#55], [ib_income_band_sk#70], Inner, BuildRight - -(92) CometProject -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#70] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(93) ReusedExchange [Reuses operator id: 90] -Output [1]: [ib_income_band_sk#71] - -(94) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#71] -Arguments: [hd_income_band_sk#57], [ib_income_band_sk#71], Inner, BuildRight - -(95) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#71] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(96) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(97) CometFilter -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Condition : ((((((isnotnull(i_current_price#73) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#74, 20, true, false, true) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#73 >= 64.00)) AND (i_current_price#73 <= 74.00)) AND (i_current_price#73 >= 65.00)) AND (i_current_price#73 <= 79.00)) AND isnotnull(i_item_sk#72)) - -(98) CometProject -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Arguments: [i_item_sk#72, i_product_name#76], [i_item_sk#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#75, 50, true, false, true) AS i_product_name#76] - -(99) CometBroadcastExchange -Input [2]: [i_item_sk#72, i_product_name#76] -Arguments: [i_item_sk#72, i_product_name#76] - -(100) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [2]: [i_item_sk#72, i_product_name#76] -Arguments: [ss_item_sk#1], [i_item_sk#72], Inner, BuildRight - -(101) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] - -(102) CometHashAggregate -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] - -(103) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#77, sum#78, sum#79, sum#80] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] - -(104) CometExchange -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: hashpartitioning(item_sk#82, store_name#83, store_zip#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(105) CometSort -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97], [item_sk#82 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, store_zip#84 ASC NULLS FIRST] - -(106) Scan parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#109), dynamicpruningexpression(ss_sold_date_sk#109 IN dynamicpruning#110)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(107) ColumnarToRow [codegen id : 3] -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(108) Filter [codegen id : 3] -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Condition : (((((((isnotnull(ss_item_sk#98) AND isnotnull(ss_ticket_number#105)) AND isnotnull(ss_store_sk#103)) AND isnotnull(ss_customer_sk#99)) AND isnotnull(ss_cdemo_sk#100)) AND isnotnull(ss_promo_sk#104)) AND isnotnull(ss_hdemo_sk#101)) AND isnotnull(ss_addr_sk#102)) - -(109) BroadcastExchange -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=7] - -(110) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(111) CometFilter -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) - -(112) CometProject -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] - -(113) CometColumnarToRow -Input [2]: [sr_item_sk#111, sr_ticket_number#112] - -(114) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [ss_item_sk#98, ss_ticket_number#105] -Right keys [2]: [sr_item_sk#111, sr_ticket_number#112] -Join type: Inner -Join condition: None - -(115) Project [codegen id : 4] -Output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, sr_item_sk#111, sr_ticket_number#112] - -(116) CometColumnarExchange -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: hashpartitioning(ss_item_sk#98, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(117) CometSort -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98 ASC NULLS FIRST] - -(118) ReusedExchange [Reuses operator id: 26] -Output [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] - -(119) CometHashAggregate -Input [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] -Keys [1]: [cs_item_sk#114] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#118)), sum(((cr_refunded_cash#119 + cr_reversed_charge#120) + cr_store_credit#121))] - -(120) CometFilter -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(121) CometProject -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Arguments: [cs_item_sk#114], [cs_item_sk#114] - -(122) CometSort -Input [1]: [cs_item_sk#114] -Arguments: [cs_item_sk#114], [cs_item_sk#114 ASC NULLS FIRST] - -(123) CometSortMergeJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [1]: [cs_item_sk#114] -Arguments: [ss_item_sk#98], [cs_item_sk#114], Inner - -(124) CometProject -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, cs_item_sk#114] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(125) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(127) CometBroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: [d_date_sk#122, d_year#123] - -(128) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [d_date_sk#122, d_year#123] -Arguments: [ss_sold_date_sk#109], [d_date_sk#122], Inner, BuildRight - -(129) CometProject -Input [13]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, d_date_sk#122, d_year#123] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] - -(130) ReusedExchange [Reuses operator id: 41] -Output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] - -(131) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] -Right output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_store_sk#103], [s_store_sk#124], Inner, BuildRight - -(132) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] - -(133) ReusedExchange [Reuses operator id: 46] -Output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(134) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] -Right output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_customer_sk#99], [c_customer_sk#126], Inner, BuildRight - -(135) CometProject -Input [18]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(136) ReusedExchange [Reuses operator id: 51] -Output [2]: [d_date_sk#132, d_year#133] - -(137) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Right output [2]: [d_date_sk#132, d_year#133] -Arguments: [c_first_sales_date_sk#131], [d_date_sk#132], Inner, BuildRight - -(138) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131, d_date_sk#132, d_year#133] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] - -(139) ReusedExchange [Reuses operator id: 51] -Output [2]: [d_date_sk#134, d_year#135] - -(140) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] -Right output [2]: [d_date_sk#134, d_year#135] -Arguments: [c_first_shipto_date_sk#130], [d_date_sk#134], Inner, BuildRight - -(141) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133, d_date_sk#134, d_year#135] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(142) ReusedExchange [Reuses operator id: 60] -Output [2]: [cd_demo_sk#136, cd_marital_status#50] - -(143) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_cdemo_sk#100], [cd_demo_sk#136], Inner, BuildRight - -(144) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] - -(145) ReusedExchange [Reuses operator id: 60] -Output [2]: [cd_demo_sk#137, cd_marital_status#52] - -(146) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] -Right output [2]: [cd_demo_sk#137, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#127], [cd_demo_sk#137], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(147) CometProject -Input [18]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50, cd_demo_sk#137, cd_marital_status#52] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(148) ReusedExchange [Reuses operator id: 68] -Output [1]: [p_promo_sk#138] - -(149) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [1]: [p_promo_sk#138] -Arguments: [ss_promo_sk#104], [p_promo_sk#138], Inner, BuildRight - -(150) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, p_promo_sk#138] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(151) ReusedExchange [Reuses operator id: 73] -Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] - -(152) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_hdemo_sk#101], [hd_demo_sk#139], Inner, BuildRight - -(153) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] - -(154) ReusedExchange [Reuses operator id: 73] -Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] - -(155) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] -Right output [2]: [hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [c_current_hdemo_sk#128], [hd_demo_sk#141], Inner, BuildRight - -(156) CometProject -Input [15]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] - -(157) ReusedExchange [Reuses operator id: 82] -Output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(158) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] -Right output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_addr_sk#102], [ca_address_sk#143], Inner, BuildRight - -(159) CometProject -Input [18]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(160) ReusedExchange [Reuses operator id: 82] -Output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(161) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Right output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [c_current_addr_sk#129], [ca_address_sk#146], Inner, BuildRight - -(162) CometProject -Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(163) ReusedExchange [Reuses operator id: 90] -Output [1]: [ib_income_band_sk#149] - -(164) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#149] -Arguments: [hd_income_band_sk#140], [ib_income_band_sk#149], Inner, BuildRight - -(165) CometProject -Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#149] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(166) ReusedExchange [Reuses operator id: 90] -Output [1]: [ib_income_band_sk#150] - -(167) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#150] -Arguments: [hd_income_band_sk#142], [ib_income_band_sk#150], Inner, BuildRight - -(168) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#150] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(169) ReusedExchange [Reuses operator id: 99] -Output [2]: [i_item_sk#151, i_product_name#76] - -(170) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [2]: [i_item_sk#151, i_product_name#76] -Arguments: [ss_item_sk#98], [i_item_sk#151], Inner, BuildRight - -(171) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] - -(172) CometHashAggregate -Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#106)), partial_sum(UnscaledValue(ss_list_price#107)), partial_sum(UnscaledValue(ss_coupon_amt#108))] - -(173) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135, count#77, sum#152, sum#153, sum#154] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#106)), sum(UnscaledValue(ss_list_price#107)), sum(UnscaledValue(ss_coupon_amt#108))] - -(174) CometExchange -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: hashpartitioning(item_sk#155, store_name#156, store_zip#157, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(175) CometSort -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162], [item_sk#155 ASC NULLS FIRST, store_name#156 ASC NULLS FIRST, store_zip#157 ASC NULLS FIRST] - -(176) CometSortMergeJoin -Left output [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Right output [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#156, store_zip#157], Inner, (cnt#159 <= cnt#94) - -(177) CometProject -Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -(178) CometExchange -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(179) CometSort -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST] - -(180) CometColumnarToRow [codegen id : 5] -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (184) -+- * CometColumnarToRow (183) - +- CometFilter (182) - +- CometNativeScan parquet spark_catalog.default.date_dim (181) - - -(181) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(182) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(183) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#33] - -(184) BroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] - -Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 -BroadcastExchange (188) -+- * CometColumnarToRow (187) - +- CometFilter (186) - +- CometNativeScan parquet spark_catalog.default.date_dim (185) - - -(185) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(186) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(187) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#122, d_year#123] - -(188) BroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/simplified.txt deleted file mode 100644 index f954616bb5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/simplified.txt +++ /dev/null @@ -1,206 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometExchange [product_name,store_name,cnt] #1 - CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometColumnarExchange [ss_item_sk] #3 - WholeStageCodegen (2) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometNativeScan 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] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [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] #11 - 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] - CometNativeScan 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] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk] - ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometColumnarExchange [ss_item_sk] #20 - WholeStageCodegen (4) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #21 - WholeStageCodegen (3) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet 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 #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - CometBroadcastExchange [d_date_sk,d_year] #23 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/explain.txt deleted file mode 100644 index cbe790cd00..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,1011 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (176) -+- CometSort (175) - +- CometExchange (174) - +- CometProject (173) - +- CometSortMergeJoin (172) - :- CometSort (103) - : +- CometExchange (102) - : +- CometHashAggregate (101) - : +- CometHashAggregate (100) - : +- CometProject (99) - : +- CometBroadcastHashJoin (98) - : :- CometProject (93) - : : +- CometBroadcastHashJoin (92) - : : :- CometProject (90) - : : : +- CometBroadcastHashJoin (89) - : : : :- CometProject (85) - : : : : +- CometBroadcastHashJoin (84) - : : : : :- CometProject (82) - : : : : : +- CometBroadcastHashJoin (81) - : : : : : :- CometProject (76) - : : : : : : +- CometBroadcastHashJoin (75) - : : : : : : :- CometProject (73) - : : : : : : : +- CometBroadcastHashJoin (72) - : : : : : : : :- CometProject (68) - : : : : : : : : +- CometBroadcastHashJoin (67) - : : : : : : : : :- CometProject (63) - : : : : : : : : : +- CometBroadcastHashJoin (62) - : : : : : : : : : :- CometProject (60) - : : : : : : : : : : +- CometBroadcastHashJoin (59) - : : : : : : : : : : :- CometProject (54) - : : : : : : : : : : : +- CometBroadcastHashJoin (53) - : : : : : : : : : : : :- CometProject (51) - : : : : : : : : : : : : +- CometBroadcastHashJoin (50) - : : : : : : : : : : : : :- CometProject (46) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (45) - : : : : : : : : : : : : : :- CometProject (41) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (40) - : : : : : : : : : : : : : : :- CometProject (35) - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (34) - : : : : : : : : : : : : : : : :- CometProject (30) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) - : : : : : : : : : : : : : : : : :- CometSort (10) - : : : : : : : : : : : : : : : : : +- CometExchange (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- CometSort (28) - : : : : : : : : : : : : : : : : +- CometProject (27) - : : : : : : : : : : : : : : : : +- CometFilter (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometExchange (24) - : : : : : : : : : : : : : : : : +- CometHashAggregate (23) - : : : : : : : : : : : : : : : : +- CometProject (22) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) - : : : : : : : : : : : : : : : : :- CometSort (15) - : : : : : : : : : : : : : : : : : +- CometExchange (14) - : : : : : : : : : : : : : : : : : +- CometProject (13) - : : : : : : : : : : : : : : : : : +- CometFilter (12) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (11) - : : : : : : : : : : : : : : : : +- CometSort (20) - : : : : : : : : : : : : : : : : +- CometExchange (19) - : : : : : : : : : : : : : : : : +- CometProject (18) - : : : : : : : : : : : : : : : : +- CometFilter (17) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : : : : : : : : : : : : : : : +- CometBroadcastExchange (33) - : : : : : : : : : : : : : : : +- CometFilter (32) - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - : : : : : : : : : : : : : : +- CometBroadcastExchange (39) - : : : : : : : : : : : : : : +- CometProject (38) - : : : : : : : : : : : : : : +- CometFilter (37) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (36) - : : : : : : : : : : : : : +- CometBroadcastExchange (44) - : : : : : : : : : : : : : +- CometFilter (43) - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (42) - : : : : : : : : : : : : +- CometBroadcastExchange (49) - : : : : : : : : : : : : +- CometFilter (48) - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) - : : : : : : : : : : : +- ReusedExchange (52) - : : : : : : : : : : +- CometBroadcastExchange (58) - : : : : : : : : : : +- CometProject (57) - : : : : : : : : : : +- CometFilter (56) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (55) - : : : : : : : : : +- ReusedExchange (61) - : : : : : : : : +- CometBroadcastExchange (66) - : : : : : : : : +- CometFilter (65) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (64) - : : : : : : : +- CometBroadcastExchange (71) - : : : : : : : +- CometFilter (70) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (69) - : : : : : : +- ReusedExchange (74) - : : : : : +- CometBroadcastExchange (80) - : : : : : +- CometProject (79) - : : : : : +- CometFilter (78) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (77) - : : : : +- ReusedExchange (83) - : : : +- CometBroadcastExchange (88) - : : : +- CometFilter (87) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (86) - : : +- ReusedExchange (91) - : +- CometBroadcastExchange (97) - : +- CometProject (96) - : +- CometFilter (95) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (94) - +- CometSort (171) - +- CometExchange (170) - +- CometHashAggregate (169) - +- CometHashAggregate (168) - +- CometProject (167) - +- CometBroadcastHashJoin (166) - :- CometProject (164) - : +- CometBroadcastHashJoin (163) - : :- CometProject (161) - : : +- CometBroadcastHashJoin (160) - : : :- CometProject (158) - : : : +- CometBroadcastHashJoin (157) - : : : :- CometProject (155) - : : : : +- CometBroadcastHashJoin (154) - : : : : :- CometProject (152) - : : : : : +- CometBroadcastHashJoin (151) - : : : : : :- CometProject (149) - : : : : : : +- CometBroadcastHashJoin (148) - : : : : : : :- CometProject (146) - : : : : : : : +- CometBroadcastHashJoin (145) - : : : : : : : :- CometProject (143) - : : : : : : : : +- CometBroadcastHashJoin (142) - : : : : : : : : :- CometProject (140) - : : : : : : : : : +- CometBroadcastHashJoin (139) - : : : : : : : : : :- CometProject (137) - : : : : : : : : : : +- CometBroadcastHashJoin (136) - : : : : : : : : : : :- CometProject (134) - : : : : : : : : : : : +- CometBroadcastHashJoin (133) - : : : : : : : : : : : :- CometProject (131) - : : : : : : : : : : : : +- CometBroadcastHashJoin (130) - : : : : : : : : : : : : :- CometProject (128) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (127) - : : : : : : : : : : : : : :- CometProject (125) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (124) - : : : : : : : : : : : : : : :- CometProject (120) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (119) - : : : : : : : : : : : : : : : :- CometSort (113) - : : : : : : : : : : : : : : : : +- CometExchange (112) - : : : : : : : : : : : : : : : : +- CometProject (111) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (110) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (106) - : : : : : : : : : : : : : : : : : +- CometFilter (105) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (104) - : : : : : : : : : : : : : : : : +- CometProject (109) - : : : : : : : : : : : : : : : : +- CometFilter (108) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (107) - : : : : : : : : : : : : : : : +- CometSort (118) - : : : : : : : : : : : : : : : +- CometProject (117) - : : : : : : : : : : : : : : : +- CometFilter (116) - : : : : : : : : : : : : : : : +- CometHashAggregate (115) - : : : : : : : : : : : : : : : +- ReusedExchange (114) - : : : : : : : : : : : : : : +- CometBroadcastExchange (123) - : : : : : : : : : : : : : : +- CometFilter (122) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (121) - : : : : : : : : : : : : : +- ReusedExchange (126) - : : : : : : : : : : : : +- ReusedExchange (129) - : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : +- ReusedExchange (141) - : : : : : : : +- ReusedExchange (144) - : : : : : : +- ReusedExchange (147) - : : : : : +- ReusedExchange (150) - : : : : +- ReusedExchange (153) - : : : +- ReusedExchange (156) - : : +- ReusedExchange (159) - : +- ReusedExchange (162) - +- ReusedExchange (165) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) CometFilter -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) - -(3) CometBroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) - -(6) CometProject -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] - -(7) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner, BuildLeft - -(8) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(9) CometExchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometSort -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) - -(13) CometProject -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] - -(14) CometExchange -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(15) CometSort -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(17) CometFilter -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) - -(18) CometProject -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(19) CometExchange -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner - -(22) CometProject -Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(23) CometHashAggregate -Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Keys [1]: [cs_item_sk#17] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(24) CometExchange -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometHashAggregate -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Keys [1]: [cs_item_sk#17] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(26) CometFilter -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(27) CometProject -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Arguments: [cs_item_sk#17], [cs_item_sk#17] - -(28) CometSort -Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] - -(29) CometSortMergeJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [1]: [cs_item_sk#17] -Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner - -(30) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(33) CometBroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: [d_date_sk#32, d_year#33] - -(34) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight - -(35) CometProject -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true))) - -(38) CometProject -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37], [s_store_sk#34, s_store_name#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true) AS s_zip#37] - -(39) CometBroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37] - -(40) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight - -(41) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(43) CometFilter -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Condition : (((((isnotnull(c_customer_sk#38) AND isnotnull(c_first_sales_date_sk#43)) AND isnotnull(c_first_shipto_date_sk#42)) AND isnotnull(c_current_cdemo_sk#39)) AND isnotnull(c_current_hdemo_sk#40)) AND isnotnull(c_current_addr_sk#41)) - -(44) CometBroadcastExchange -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(45) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -Right output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_customer_sk#2], [c_customer_sk#38], Inner, BuildRight - -(46) CometProject -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(48) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : isnotnull(d_date_sk#44) - -(49) CometBroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] - -(50) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [c_first_sales_date_sk#43], [d_date_sk#44], Inner, BuildRight - -(51) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43, d_date_sk#44, d_year#45] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] - -(52) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#46, d_year#47] - -(53) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [c_first_shipto_date_sk#42], [d_date_sk#46], Inner, BuildRight - -(54) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45, d_date_sk#46, d_year#47] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#48, cd_marital_status#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true))) - -(57) CometProject -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Arguments: [cd_demo_sk#48, cd_marital_status#50], [cd_demo_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true) AS cd_marital_status#50] - -(58) CometBroadcastExchange -Input [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [cd_demo_sk#48, cd_marital_status#50] - -(59) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_cdemo_sk#3], [cd_demo_sk#48], Inner, BuildRight - -(60) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] - -(61) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#51, cd_marital_status#52] - -(62) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -Right output [2]: [cd_demo_sk#51, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#39], [cd_demo_sk#51], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(63) CometProject -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50, cd_demo_sk#51, cd_marital_status#52] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(65) CometFilter -Input [1]: [p_promo_sk#53] -Condition : isnotnull(p_promo_sk#53) - -(66) CometBroadcastExchange -Input [1]: [p_promo_sk#53] -Arguments: [p_promo_sk#53] - -(67) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [1]: [p_promo_sk#53] -Arguments: [ss_promo_sk#7], [p_promo_sk#53], Inner, BuildRight - -(68) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, p_promo_sk#53] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(70) CometFilter -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Condition : (isnotnull(hd_demo_sk#54) AND isnotnull(hd_income_band_sk#55)) - -(71) CometBroadcastExchange -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [hd_demo_sk#54, hd_income_band_sk#55] - -(72) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_hdemo_sk#4], [hd_demo_sk#54], Inner, BuildRight - -(73) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] - -(74) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] - -(75) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -Right output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [c_current_hdemo_sk#40], [hd_demo_sk#56], Inner, BuildRight - -(76) CometProject -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(78) CometFilter -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Condition : isnotnull(ca_address_sk#58) - -(79) CometProject -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ca_address_sk#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#59, 10, true, false, true) AS ca_street_number#63, ca_street_name#60, ca_city#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#62, 10, true, false, true) AS ca_zip#64] - -(80) CometBroadcastExchange -Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(81) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -Right output [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_addr_sk#5], [ca_address_sk#58], Inner, BuildRight - -(82) CometProject -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(83) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(84) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Right output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [c_current_addr_sk#41], [ca_address_sk#65], Inner, BuildRight - -(85) CometProject -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(87) CometFilter -Input [1]: [ib_income_band_sk#70] -Condition : isnotnull(ib_income_band_sk#70) - -(88) CometBroadcastExchange -Input [1]: [ib_income_band_sk#70] -Arguments: [ib_income_band_sk#70] - -(89) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#70] -Arguments: [hd_income_band_sk#55], [ib_income_band_sk#70], Inner, BuildRight - -(90) CometProject -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#70] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(91) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#71] - -(92) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#71] -Arguments: [hd_income_band_sk#57], [ib_income_band_sk#71], Inner, BuildRight - -(93) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#71] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(95) CometFilter -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Condition : ((((((isnotnull(i_current_price#73) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#74, 20, true, false, true) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#73 >= 64.00)) AND (i_current_price#73 <= 74.00)) AND (i_current_price#73 >= 65.00)) AND (i_current_price#73 <= 79.00)) AND isnotnull(i_item_sk#72)) - -(96) CometProject -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Arguments: [i_item_sk#72, i_product_name#76], [i_item_sk#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#75, 50, true, false, true) AS i_product_name#76] - -(97) CometBroadcastExchange -Input [2]: [i_item_sk#72, i_product_name#76] -Arguments: [i_item_sk#72, i_product_name#76] - -(98) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [2]: [i_item_sk#72, i_product_name#76] -Arguments: [ss_item_sk#1], [i_item_sk#72], Inner, BuildRight - -(99) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] - -(100) CometHashAggregate -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] - -(101) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#77, sum#78, sum#79, sum#80] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] - -(102) CometExchange -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: hashpartitioning(item_sk#82, store_name#83, store_zip#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(103) CometSort -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97], [item_sk#82 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, store_zip#84 ASC NULLS FIRST] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#109), dynamicpruningexpression(ss_sold_date_sk#109 IN dynamicpruning#110)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(105) CometFilter -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Condition : (((((((isnotnull(ss_item_sk#98) AND isnotnull(ss_ticket_number#105)) AND isnotnull(ss_store_sk#103)) AND isnotnull(ss_customer_sk#99)) AND isnotnull(ss_cdemo_sk#100)) AND isnotnull(ss_promo_sk#104)) AND isnotnull(ss_hdemo_sk#101)) AND isnotnull(ss_addr_sk#102)) - -(106) CometBroadcastExchange -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(108) CometFilter -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) - -(109) CometProject -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] - -(110) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_ticket_number#105], [sr_item_sk#111, sr_ticket_number#112], Inner, BuildLeft - -(111) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(112) CometExchange -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: hashpartitioning(ss_item_sk#98, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(113) CometSort -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98 ASC NULLS FIRST] - -(114) ReusedExchange [Reuses operator id: 24] -Output [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] - -(115) CometHashAggregate -Input [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] -Keys [1]: [cs_item_sk#114] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#118)), sum(((cr_refunded_cash#119 + cr_reversed_charge#120) + cr_store_credit#121))] - -(116) CometFilter -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(117) CometProject -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Arguments: [cs_item_sk#114], [cs_item_sk#114] - -(118) CometSort -Input [1]: [cs_item_sk#114] -Arguments: [cs_item_sk#114], [cs_item_sk#114 ASC NULLS FIRST] - -(119) CometSortMergeJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [1]: [cs_item_sk#114] -Arguments: [ss_item_sk#98], [cs_item_sk#114], Inner - -(120) CometProject -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, cs_item_sk#114] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(121) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(122) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(123) CometBroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: [d_date_sk#122, d_year#123] - -(124) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [d_date_sk#122, d_year#123] -Arguments: [ss_sold_date_sk#109], [d_date_sk#122], Inner, BuildRight - -(125) CometProject -Input [13]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, d_date_sk#122, d_year#123] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] - -(126) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] - -(127) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] -Right output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_store_sk#103], [s_store_sk#124], Inner, BuildRight - -(128) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] - -(129) ReusedExchange [Reuses operator id: 44] -Output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(130) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] -Right output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_customer_sk#99], [c_customer_sk#126], Inner, BuildRight - -(131) CometProject -Input [18]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(132) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#132, d_year#133] - -(133) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Right output [2]: [d_date_sk#132, d_year#133] -Arguments: [c_first_sales_date_sk#131], [d_date_sk#132], Inner, BuildRight - -(134) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131, d_date_sk#132, d_year#133] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] - -(135) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#134, d_year#135] - -(136) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] -Right output [2]: [d_date_sk#134, d_year#135] -Arguments: [c_first_shipto_date_sk#130], [d_date_sk#134], Inner, BuildRight - -(137) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133, d_date_sk#134, d_year#135] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(138) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#136, cd_marital_status#50] - -(139) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_cdemo_sk#100], [cd_demo_sk#136], Inner, BuildRight - -(140) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] - -(141) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#137, cd_marital_status#52] - -(142) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] -Right output [2]: [cd_demo_sk#137, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#127], [cd_demo_sk#137], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(143) CometProject -Input [18]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50, cd_demo_sk#137, cd_marital_status#52] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(144) ReusedExchange [Reuses operator id: 66] -Output [1]: [p_promo_sk#138] - -(145) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [1]: [p_promo_sk#138] -Arguments: [ss_promo_sk#104], [p_promo_sk#138], Inner, BuildRight - -(146) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, p_promo_sk#138] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(147) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] - -(148) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_hdemo_sk#101], [hd_demo_sk#139], Inner, BuildRight - -(149) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] - -(150) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] - -(151) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] -Right output [2]: [hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [c_current_hdemo_sk#128], [hd_demo_sk#141], Inner, BuildRight - -(152) CometProject -Input [15]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] - -(153) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(154) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] -Right output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_addr_sk#102], [ca_address_sk#143], Inner, BuildRight - -(155) CometProject -Input [18]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(156) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(157) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Right output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [c_current_addr_sk#129], [ca_address_sk#146], Inner, BuildRight - -(158) CometProject -Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(159) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#149] - -(160) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#149] -Arguments: [hd_income_band_sk#140], [ib_income_band_sk#149], Inner, BuildRight - -(161) CometProject -Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#149] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(162) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#150] - -(163) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#150] -Arguments: [hd_income_band_sk#142], [ib_income_band_sk#150], Inner, BuildRight - -(164) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#150] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(165) ReusedExchange [Reuses operator id: 97] -Output [2]: [i_item_sk#151, i_product_name#76] - -(166) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [2]: [i_item_sk#151, i_product_name#76] -Arguments: [ss_item_sk#98], [i_item_sk#151], Inner, BuildRight - -(167) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] - -(168) CometHashAggregate -Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#106)), partial_sum(UnscaledValue(ss_list_price#107)), partial_sum(UnscaledValue(ss_coupon_amt#108))] - -(169) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135, count#77, sum#152, sum#153, sum#154] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#106)), sum(UnscaledValue(ss_list_price#107)), sum(UnscaledValue(ss_coupon_amt#108))] - -(170) CometExchange -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: hashpartitioning(item_sk#155, store_name#156, store_zip#157, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(171) CometSort -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162], [item_sk#155 ASC NULLS FIRST, store_name#156 ASC NULLS FIRST, store_zip#157 ASC NULLS FIRST] - -(172) CometSortMergeJoin -Left output [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Right output [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#156, store_zip#157], Inner, (cnt#159 <= cnt#94) - -(173) CometProject -Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -(174) CometExchange -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(175) CometSort -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST] - -(176) CometColumnarToRow [codegen id : 1] -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (180) -+- * CometColumnarToRow (179) - +- CometFilter (178) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) - - -(177) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(178) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(179) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#33] - -(180) BroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 -BroadcastExchange (184) -+- * CometColumnarToRow (183) - +- CometFilter (182) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) - - -(181) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(182) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(183) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#122, d_year#123] - -(184) BroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/simplified.txt deleted file mode 100644 index 43527978d8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,192 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometExchange [product_name,store_name,cnt] #1 - CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #3 - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [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] #11 - 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 [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk] - ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #20 - 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_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] #21 - 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 [native_iceberg_compat] 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 #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - CometBroadcastExchange [d_date_sk,d_year] #23 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt deleted file mode 100644 index cbe790cd00..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt +++ /dev/null @@ -1,1011 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (176) -+- CometSort (175) - +- CometExchange (174) - +- CometProject (173) - +- CometSortMergeJoin (172) - :- CometSort (103) - : +- CometExchange (102) - : +- CometHashAggregate (101) - : +- CometHashAggregate (100) - : +- CometProject (99) - : +- CometBroadcastHashJoin (98) - : :- CometProject (93) - : : +- CometBroadcastHashJoin (92) - : : :- CometProject (90) - : : : +- CometBroadcastHashJoin (89) - : : : :- CometProject (85) - : : : : +- CometBroadcastHashJoin (84) - : : : : :- CometProject (82) - : : : : : +- CometBroadcastHashJoin (81) - : : : : : :- CometProject (76) - : : : : : : +- CometBroadcastHashJoin (75) - : : : : : : :- CometProject (73) - : : : : : : : +- CometBroadcastHashJoin (72) - : : : : : : : :- CometProject (68) - : : : : : : : : +- CometBroadcastHashJoin (67) - : : : : : : : : :- CometProject (63) - : : : : : : : : : +- CometBroadcastHashJoin (62) - : : : : : : : : : :- CometProject (60) - : : : : : : : : : : +- CometBroadcastHashJoin (59) - : : : : : : : : : : :- CometProject (54) - : : : : : : : : : : : +- CometBroadcastHashJoin (53) - : : : : : : : : : : : :- CometProject (51) - : : : : : : : : : : : : +- CometBroadcastHashJoin (50) - : : : : : : : : : : : : :- CometProject (46) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (45) - : : : : : : : : : : : : : :- CometProject (41) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (40) - : : : : : : : : : : : : : : :- CometProject (35) - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (34) - : : : : : : : : : : : : : : : :- CometProject (30) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) - : : : : : : : : : : : : : : : : :- CometSort (10) - : : : : : : : : : : : : : : : : : +- CometExchange (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- CometSort (28) - : : : : : : : : : : : : : : : : +- CometProject (27) - : : : : : : : : : : : : : : : : +- CometFilter (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometExchange (24) - : : : : : : : : : : : : : : : : +- CometHashAggregate (23) - : : : : : : : : : : : : : : : : +- CometProject (22) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) - : : : : : : : : : : : : : : : : :- CometSort (15) - : : : : : : : : : : : : : : : : : +- CometExchange (14) - : : : : : : : : : : : : : : : : : +- CometProject (13) - : : : : : : : : : : : : : : : : : +- CometFilter (12) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (11) - : : : : : : : : : : : : : : : : +- CometSort (20) - : : : : : : : : : : : : : : : : +- CometExchange (19) - : : : : : : : : : : : : : : : : +- CometProject (18) - : : : : : : : : : : : : : : : : +- CometFilter (17) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : : : : : : : : : : : : : : : +- CometBroadcastExchange (33) - : : : : : : : : : : : : : : : +- CometFilter (32) - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - : : : : : : : : : : : : : : +- CometBroadcastExchange (39) - : : : : : : : : : : : : : : +- CometProject (38) - : : : : : : : : : : : : : : +- CometFilter (37) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (36) - : : : : : : : : : : : : : +- CometBroadcastExchange (44) - : : : : : : : : : : : : : +- CometFilter (43) - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (42) - : : : : : : : : : : : : +- CometBroadcastExchange (49) - : : : : : : : : : : : : +- CometFilter (48) - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) - : : : : : : : : : : : +- ReusedExchange (52) - : : : : : : : : : : +- CometBroadcastExchange (58) - : : : : : : : : : : +- CometProject (57) - : : : : : : : : : : +- CometFilter (56) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (55) - : : : : : : : : : +- ReusedExchange (61) - : : : : : : : : +- CometBroadcastExchange (66) - : : : : : : : : +- CometFilter (65) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (64) - : : : : : : : +- CometBroadcastExchange (71) - : : : : : : : +- CometFilter (70) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (69) - : : : : : : +- ReusedExchange (74) - : : : : : +- CometBroadcastExchange (80) - : : : : : +- CometProject (79) - : : : : : +- CometFilter (78) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (77) - : : : : +- ReusedExchange (83) - : : : +- CometBroadcastExchange (88) - : : : +- CometFilter (87) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (86) - : : +- ReusedExchange (91) - : +- CometBroadcastExchange (97) - : +- CometProject (96) - : +- CometFilter (95) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (94) - +- CometSort (171) - +- CometExchange (170) - +- CometHashAggregate (169) - +- CometHashAggregate (168) - +- CometProject (167) - +- CometBroadcastHashJoin (166) - :- CometProject (164) - : +- CometBroadcastHashJoin (163) - : :- CometProject (161) - : : +- CometBroadcastHashJoin (160) - : : :- CometProject (158) - : : : +- CometBroadcastHashJoin (157) - : : : :- CometProject (155) - : : : : +- CometBroadcastHashJoin (154) - : : : : :- CometProject (152) - : : : : : +- CometBroadcastHashJoin (151) - : : : : : :- CometProject (149) - : : : : : : +- CometBroadcastHashJoin (148) - : : : : : : :- CometProject (146) - : : : : : : : +- CometBroadcastHashJoin (145) - : : : : : : : :- CometProject (143) - : : : : : : : : +- CometBroadcastHashJoin (142) - : : : : : : : : :- CometProject (140) - : : : : : : : : : +- CometBroadcastHashJoin (139) - : : : : : : : : : :- CometProject (137) - : : : : : : : : : : +- CometBroadcastHashJoin (136) - : : : : : : : : : : :- CometProject (134) - : : : : : : : : : : : +- CometBroadcastHashJoin (133) - : : : : : : : : : : : :- CometProject (131) - : : : : : : : : : : : : +- CometBroadcastHashJoin (130) - : : : : : : : : : : : : :- CometProject (128) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (127) - : : : : : : : : : : : : : :- CometProject (125) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (124) - : : : : : : : : : : : : : : :- CometProject (120) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (119) - : : : : : : : : : : : : : : : :- CometSort (113) - : : : : : : : : : : : : : : : : +- CometExchange (112) - : : : : : : : : : : : : : : : : +- CometProject (111) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (110) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (106) - : : : : : : : : : : : : : : : : : +- CometFilter (105) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (104) - : : : : : : : : : : : : : : : : +- CometProject (109) - : : : : : : : : : : : : : : : : +- CometFilter (108) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (107) - : : : : : : : : : : : : : : : +- CometSort (118) - : : : : : : : : : : : : : : : +- CometProject (117) - : : : : : : : : : : : : : : : +- CometFilter (116) - : : : : : : : : : : : : : : : +- CometHashAggregate (115) - : : : : : : : : : : : : : : : +- ReusedExchange (114) - : : : : : : : : : : : : : : +- CometBroadcastExchange (123) - : : : : : : : : : : : : : : +- CometFilter (122) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (121) - : : : : : : : : : : : : : +- ReusedExchange (126) - : : : : : : : : : : : : +- ReusedExchange (129) - : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : +- ReusedExchange (141) - : : : : : : : +- ReusedExchange (144) - : : : : : : +- ReusedExchange (147) - : : : : : +- ReusedExchange (150) - : : : : +- ReusedExchange (153) - : : : +- ReusedExchange (156) - : : +- ReusedExchange (159) - : +- ReusedExchange (162) - +- ReusedExchange (165) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) CometFilter -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) - -(3) CometBroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) - -(6) CometProject -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] - -(7) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner, BuildLeft - -(8) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(9) CometExchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometSort -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) - -(13) CometProject -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] - -(14) CometExchange -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(15) CometSort -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(17) CometFilter -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) - -(18) CometProject -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(19) CometExchange -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner - -(22) CometProject -Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(23) CometHashAggregate -Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Keys [1]: [cs_item_sk#17] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(24) CometExchange -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometHashAggregate -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Keys [1]: [cs_item_sk#17] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(26) CometFilter -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(27) CometProject -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Arguments: [cs_item_sk#17], [cs_item_sk#17] - -(28) CometSort -Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] - -(29) CometSortMergeJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [1]: [cs_item_sk#17] -Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner - -(30) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(33) CometBroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: [d_date_sk#32, d_year#33] - -(34) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight - -(35) CometProject -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true))) - -(38) CometProject -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37], [s_store_sk#34, s_store_name#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true) AS s_zip#37] - -(39) CometBroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37] - -(40) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight - -(41) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(43) CometFilter -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Condition : (((((isnotnull(c_customer_sk#38) AND isnotnull(c_first_sales_date_sk#43)) AND isnotnull(c_first_shipto_date_sk#42)) AND isnotnull(c_current_cdemo_sk#39)) AND isnotnull(c_current_hdemo_sk#40)) AND isnotnull(c_current_addr_sk#41)) - -(44) CometBroadcastExchange -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(45) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -Right output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_customer_sk#2], [c_customer_sk#38], Inner, BuildRight - -(46) CometProject -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(48) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : isnotnull(d_date_sk#44) - -(49) CometBroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] - -(50) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [c_first_sales_date_sk#43], [d_date_sk#44], Inner, BuildRight - -(51) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43, d_date_sk#44, d_year#45] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] - -(52) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#46, d_year#47] - -(53) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [c_first_shipto_date_sk#42], [d_date_sk#46], Inner, BuildRight - -(54) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45, d_date_sk#46, d_year#47] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#48, cd_marital_status#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true))) - -(57) CometProject -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Arguments: [cd_demo_sk#48, cd_marital_status#50], [cd_demo_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true) AS cd_marital_status#50] - -(58) CometBroadcastExchange -Input [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [cd_demo_sk#48, cd_marital_status#50] - -(59) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_cdemo_sk#3], [cd_demo_sk#48], Inner, BuildRight - -(60) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] - -(61) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#51, cd_marital_status#52] - -(62) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -Right output [2]: [cd_demo_sk#51, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#39], [cd_demo_sk#51], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(63) CometProject -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50, cd_demo_sk#51, cd_marital_status#52] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(65) CometFilter -Input [1]: [p_promo_sk#53] -Condition : isnotnull(p_promo_sk#53) - -(66) CometBroadcastExchange -Input [1]: [p_promo_sk#53] -Arguments: [p_promo_sk#53] - -(67) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [1]: [p_promo_sk#53] -Arguments: [ss_promo_sk#7], [p_promo_sk#53], Inner, BuildRight - -(68) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, p_promo_sk#53] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(70) CometFilter -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Condition : (isnotnull(hd_demo_sk#54) AND isnotnull(hd_income_band_sk#55)) - -(71) CometBroadcastExchange -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [hd_demo_sk#54, hd_income_band_sk#55] - -(72) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_hdemo_sk#4], [hd_demo_sk#54], Inner, BuildRight - -(73) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] - -(74) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] - -(75) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -Right output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [c_current_hdemo_sk#40], [hd_demo_sk#56], Inner, BuildRight - -(76) CometProject -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(78) CometFilter -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Condition : isnotnull(ca_address_sk#58) - -(79) CometProject -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ca_address_sk#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#59, 10, true, false, true) AS ca_street_number#63, ca_street_name#60, ca_city#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#62, 10, true, false, true) AS ca_zip#64] - -(80) CometBroadcastExchange -Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(81) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -Right output [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_addr_sk#5], [ca_address_sk#58], Inner, BuildRight - -(82) CometProject -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(83) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(84) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Right output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [c_current_addr_sk#41], [ca_address_sk#65], Inner, BuildRight - -(85) CometProject -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(87) CometFilter -Input [1]: [ib_income_band_sk#70] -Condition : isnotnull(ib_income_band_sk#70) - -(88) CometBroadcastExchange -Input [1]: [ib_income_band_sk#70] -Arguments: [ib_income_band_sk#70] - -(89) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#70] -Arguments: [hd_income_band_sk#55], [ib_income_band_sk#70], Inner, BuildRight - -(90) CometProject -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#70] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(91) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#71] - -(92) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#71] -Arguments: [hd_income_band_sk#57], [ib_income_band_sk#71], Inner, BuildRight - -(93) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#71] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(95) CometFilter -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Condition : ((((((isnotnull(i_current_price#73) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#74, 20, true, false, true) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#73 >= 64.00)) AND (i_current_price#73 <= 74.00)) AND (i_current_price#73 >= 65.00)) AND (i_current_price#73 <= 79.00)) AND isnotnull(i_item_sk#72)) - -(96) CometProject -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Arguments: [i_item_sk#72, i_product_name#76], [i_item_sk#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#75, 50, true, false, true) AS i_product_name#76] - -(97) CometBroadcastExchange -Input [2]: [i_item_sk#72, i_product_name#76] -Arguments: [i_item_sk#72, i_product_name#76] - -(98) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [2]: [i_item_sk#72, i_product_name#76] -Arguments: [ss_item_sk#1], [i_item_sk#72], Inner, BuildRight - -(99) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] - -(100) CometHashAggregate -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] - -(101) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#77, sum#78, sum#79, sum#80] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] - -(102) CometExchange -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: hashpartitioning(item_sk#82, store_name#83, store_zip#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(103) CometSort -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97], [item_sk#82 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, store_zip#84 ASC NULLS FIRST] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#109), dynamicpruningexpression(ss_sold_date_sk#109 IN dynamicpruning#110)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(105) CometFilter -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Condition : (((((((isnotnull(ss_item_sk#98) AND isnotnull(ss_ticket_number#105)) AND isnotnull(ss_store_sk#103)) AND isnotnull(ss_customer_sk#99)) AND isnotnull(ss_cdemo_sk#100)) AND isnotnull(ss_promo_sk#104)) AND isnotnull(ss_hdemo_sk#101)) AND isnotnull(ss_addr_sk#102)) - -(106) CometBroadcastExchange -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(108) CometFilter -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) - -(109) CometProject -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] - -(110) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_ticket_number#105], [sr_item_sk#111, sr_ticket_number#112], Inner, BuildLeft - -(111) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(112) CometExchange -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: hashpartitioning(ss_item_sk#98, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(113) CometSort -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98 ASC NULLS FIRST] - -(114) ReusedExchange [Reuses operator id: 24] -Output [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] - -(115) CometHashAggregate -Input [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] -Keys [1]: [cs_item_sk#114] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#118)), sum(((cr_refunded_cash#119 + cr_reversed_charge#120) + cr_store_credit#121))] - -(116) CometFilter -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(117) CometProject -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Arguments: [cs_item_sk#114], [cs_item_sk#114] - -(118) CometSort -Input [1]: [cs_item_sk#114] -Arguments: [cs_item_sk#114], [cs_item_sk#114 ASC NULLS FIRST] - -(119) CometSortMergeJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [1]: [cs_item_sk#114] -Arguments: [ss_item_sk#98], [cs_item_sk#114], Inner - -(120) CometProject -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, cs_item_sk#114] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(121) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(122) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(123) CometBroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: [d_date_sk#122, d_year#123] - -(124) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [d_date_sk#122, d_year#123] -Arguments: [ss_sold_date_sk#109], [d_date_sk#122], Inner, BuildRight - -(125) CometProject -Input [13]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, d_date_sk#122, d_year#123] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] - -(126) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] - -(127) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] -Right output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_store_sk#103], [s_store_sk#124], Inner, BuildRight - -(128) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] - -(129) ReusedExchange [Reuses operator id: 44] -Output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(130) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] -Right output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_customer_sk#99], [c_customer_sk#126], Inner, BuildRight - -(131) CometProject -Input [18]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(132) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#132, d_year#133] - -(133) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Right output [2]: [d_date_sk#132, d_year#133] -Arguments: [c_first_sales_date_sk#131], [d_date_sk#132], Inner, BuildRight - -(134) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131, d_date_sk#132, d_year#133] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] - -(135) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#134, d_year#135] - -(136) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] -Right output [2]: [d_date_sk#134, d_year#135] -Arguments: [c_first_shipto_date_sk#130], [d_date_sk#134], Inner, BuildRight - -(137) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133, d_date_sk#134, d_year#135] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(138) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#136, cd_marital_status#50] - -(139) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_cdemo_sk#100], [cd_demo_sk#136], Inner, BuildRight - -(140) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] - -(141) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#137, cd_marital_status#52] - -(142) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] -Right output [2]: [cd_demo_sk#137, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#127], [cd_demo_sk#137], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(143) CometProject -Input [18]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50, cd_demo_sk#137, cd_marital_status#52] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(144) ReusedExchange [Reuses operator id: 66] -Output [1]: [p_promo_sk#138] - -(145) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [1]: [p_promo_sk#138] -Arguments: [ss_promo_sk#104], [p_promo_sk#138], Inner, BuildRight - -(146) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, p_promo_sk#138] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(147) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] - -(148) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_hdemo_sk#101], [hd_demo_sk#139], Inner, BuildRight - -(149) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] - -(150) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] - -(151) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] -Right output [2]: [hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [c_current_hdemo_sk#128], [hd_demo_sk#141], Inner, BuildRight - -(152) CometProject -Input [15]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] - -(153) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(154) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] -Right output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_addr_sk#102], [ca_address_sk#143], Inner, BuildRight - -(155) CometProject -Input [18]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(156) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(157) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Right output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [c_current_addr_sk#129], [ca_address_sk#146], Inner, BuildRight - -(158) CometProject -Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(159) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#149] - -(160) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#149] -Arguments: [hd_income_band_sk#140], [ib_income_band_sk#149], Inner, BuildRight - -(161) CometProject -Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#149] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(162) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#150] - -(163) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#150] -Arguments: [hd_income_band_sk#142], [ib_income_band_sk#150], Inner, BuildRight - -(164) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#150] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(165) ReusedExchange [Reuses operator id: 97] -Output [2]: [i_item_sk#151, i_product_name#76] - -(166) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [2]: [i_item_sk#151, i_product_name#76] -Arguments: [ss_item_sk#98], [i_item_sk#151], Inner, BuildRight - -(167) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] - -(168) CometHashAggregate -Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#106)), partial_sum(UnscaledValue(ss_list_price#107)), partial_sum(UnscaledValue(ss_coupon_amt#108))] - -(169) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135, count#77, sum#152, sum#153, sum#154] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#106)), sum(UnscaledValue(ss_list_price#107)), sum(UnscaledValue(ss_coupon_amt#108))] - -(170) CometExchange -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: hashpartitioning(item_sk#155, store_name#156, store_zip#157, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(171) CometSort -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162], [item_sk#155 ASC NULLS FIRST, store_name#156 ASC NULLS FIRST, store_zip#157 ASC NULLS FIRST] - -(172) CometSortMergeJoin -Left output [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Right output [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#156, store_zip#157], Inner, (cnt#159 <= cnt#94) - -(173) CometProject -Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -(174) CometExchange -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(175) CometSort -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST] - -(176) CometColumnarToRow [codegen id : 1] -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (180) -+- * CometColumnarToRow (179) - +- CometFilter (178) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) - - -(177) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(178) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(179) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#33] - -(180) BroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 -BroadcastExchange (184) -+- * CometColumnarToRow (183) - +- CometFilter (182) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) - - -(181) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(182) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(183) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#122, d_year#123] - -(184) BroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/extended.txt deleted file mode 100644 index 059acab385..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/extended.txt +++ /dev/null @@ -1,247 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 43527978d8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt +++ /dev/null @@ -1,192 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometExchange [product_name,store_name,cnt] #1 - CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #3 - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [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] #11 - 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 [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk] - ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #20 - 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_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] #21 - 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 [native_iceberg_compat] 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 #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - CometBroadcastExchange [d_date_sk,d_year] #23 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/explain.txt deleted file mode 100644 index 7cbfb795e8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/explain.txt +++ /dev/null @@ -1,286 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * CometColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.store (1) - : : +- BroadcastExchange (15) - : : +- * Filter (14) - : : +- * HashAggregate (13) - : : +- * CometColumnarToRow (12) - : : +- CometColumnarExchange (11) - : : +- * HashAggregate (10) - : : +- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : +- ReusedExchange (7) - : +- BroadcastExchange (22) - : +- * CometColumnarToRow (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometNativeScan parquet spark_catalog.default.item (18) - +- BroadcastExchange (40) - +- * Filter (39) - +- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * HashAggregate (34) - +- * CometColumnarToRow (33) - +- CometColumnarExchange (32) - +- * HashAggregate (31) - +- * Project (30) - +- * BroadcastHashJoin Inner BuildRight (29) - :- * Filter (27) - : +- * ColumnarToRow (26) - : +- Scan parquet spark_catalog.default.store_sales (25) - +- ReusedExchange (28) - - -(1) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#1, s_store_name#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [s_store_sk#1, s_store_name#2] -Condition : isnotnull(s_store_sk#1) - -(3) CometColumnarToRow [codegen id : 9] -Input [2]: [s_store_sk#1, s_store_name#2] - -(4) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] - -(6) Filter [codegen id : 2] -Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3)) - -(7) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#8] - -(8) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] -Join type: Inner -Join condition: None - -(9) Project [codegen id : 2] -Output [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] -Input [5]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] - -(10) HashAggregate [codegen id : 2] -Input [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] -Keys [2]: [ss_store_sk#4, ss_item_sk#3] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#5))] -Aggregate Attributes [1]: [sum#9] -Results [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] - -(11) CometColumnarExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(12) CometColumnarToRow [codegen id : 3] -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] - -(13) HashAggregate [codegen id : 3] -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -Keys [2]: [ss_store_sk#4, ss_item_sk#3] -Functions [1]: [sum(UnscaledValue(ss_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#5))#11] -Results [3]: [ss_store_sk#4, ss_item_sk#3, MakeDecimal(sum(UnscaledValue(ss_sales_price#5))#11,17,2) AS revenue#12] - -(14) Filter [codegen id : 3] -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] -Condition : isnotnull(revenue#12) - -(15) BroadcastExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [s_store_sk#1] -Right keys [1]: [ss_store_sk#4] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 9] -Output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] -Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] - -(18) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(19) CometFilter -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Condition : isnotnull(i_item_sk#13) - -(20) CometProject -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Arguments: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18], [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#17, 50, true, false, true) AS i_brand#18] - -(21) CometColumnarToRow [codegen id : 4] -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] - -(22) BroadcastExchange -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(23) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#3] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 9] -Output [7]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] -Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12, i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] - -(25) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(26) ColumnarToRow [codegen id : 6] -Input [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] - -(27) Filter [codegen id : 6] -Input [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_store_sk#20) - -(28) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#23] - -(29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#22] -Right keys [1]: [d_date_sk#23] -Join type: Inner -Join condition: None - -(30) Project [codegen id : 6] -Output [3]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21] -Input [5]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22, d_date_sk#23] - -(31) HashAggregate [codegen id : 6] -Input [3]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21] -Keys [2]: [ss_store_sk#20, ss_item_sk#19] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#21))] -Aggregate Attributes [1]: [sum#24] -Results [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] - -(32) CometColumnarExchange -Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] -Arguments: hashpartitioning(ss_store_sk#20, ss_item_sk#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(33) CometColumnarToRow [codegen id : 7] -Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] - -(34) HashAggregate [codegen id : 7] -Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] -Keys [2]: [ss_store_sk#20, ss_item_sk#19] -Functions [1]: [sum(UnscaledValue(ss_sales_price#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#21))#26] -Results [2]: [ss_store_sk#20, MakeDecimal(sum(UnscaledValue(ss_sales_price#21))#26,17,2) AS revenue#27] - -(35) HashAggregate [codegen id : 7] -Input [2]: [ss_store_sk#20, revenue#27] -Keys [1]: [ss_store_sk#20] -Functions [1]: [partial_avg(revenue#27)] -Aggregate Attributes [2]: [sum#28, count#29] -Results [3]: [ss_store_sk#20, sum#30, count#31] - -(36) CometColumnarExchange -Input [3]: [ss_store_sk#20, sum#30, count#31] -Arguments: hashpartitioning(ss_store_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(37) CometColumnarToRow [codegen id : 8] -Input [3]: [ss_store_sk#20, sum#30, count#31] - -(38) HashAggregate [codegen id : 8] -Input [3]: [ss_store_sk#20, sum#30, count#31] -Keys [1]: [ss_store_sk#20] -Functions [1]: [avg(revenue#27)] -Aggregate Attributes [1]: [avg(revenue#27)#32] -Results [2]: [ss_store_sk#20, avg(revenue#27)#32 AS ave#33] - -(39) Filter [codegen id : 8] -Input [2]: [ss_store_sk#20, ave#33] -Condition : isnotnull(ave#33) - -(40) BroadcastExchange -Input [2]: [ss_store_sk#20, ave#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [ss_store_sk#20] -Join type: Inner -Join condition: (cast(revenue#12 as decimal(23,7)) <= (0.1 * ave#33)) - -(42) Project [codegen id : 9] -Output [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#18] -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18, ss_store_sk#20, ave#33] - -(43) TakeOrderedAndProject -Input [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#18] -Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#14 ASC NULLS FIRST], [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#18] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometNativeScan parquet spark_catalog.default.date_dim (44) - - -(44) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [2]: [d_date_sk#8, d_month_seq#34] -Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1176)) AND (d_month_seq#34 <= 1187)) AND isnotnull(d_date_sk#8)) - -(46) CometProject -Input [2]: [d_date_sk#8, d_month_seq#34] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(48) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/simplified.txt deleted file mode 100644 index 2695c9fb89..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - WholeStageCodegen (9) - Project [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] - Project [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [s_store_name,ss_store_sk,ss_item_sk,revenue] - BroadcastHashJoin [s_store_sk,ss_store_sk] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name] - InputAdapter - BroadcastExchange #1 - WholeStageCodegen (3) - Filter [revenue] - HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_store_sk,ss_item_sk] #2 - WholeStageCodegen (2) - HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_brand] [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) - Filter [ave] - HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_store_sk] #6 - WholeStageCodegen (7) - HashAggregate [ss_store_sk,revenue] [sum,count,sum,count] - HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_store_sk,ss_item_sk] #7 - WholeStageCodegen (6) - HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_store_sk,ss_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/explain.txt deleted file mode 100644 index 8b38832289..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,264 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometProject (38) - +- CometBroadcastHashJoin (37) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (1) - : : +- CometBroadcastExchange (15) - : : +- CometFilter (14) - : : +- CometHashAggregate (13) - : : +- CometExchange (12) - : : +- CometHashAggregate (11) - : : +- CometProject (10) - : : +- CometBroadcastHashJoin (9) - : : :- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : +- CometBroadcastExchange (8) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (5) - : +- CometBroadcastExchange (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (18) - +- CometBroadcastExchange (36) - +- CometFilter (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometHashAggregate (31) - +- CometExchange (30) - +- CometHashAggregate (29) - +- CometProject (28) - +- CometBroadcastHashJoin (27) - :- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (24) - +- ReusedExchange (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#1, s_store_name#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [s_store_sk#1, s_store_name#2] -Condition : isnotnull(s_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [d_date_sk#8, d_month_seq#9] -Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_month_seq#9 <= 1187)) AND isnotnull(d_date_sk#8)) - -(7) CometProject -Input [2]: [d_date_sk#8, d_month_seq#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(8) CometBroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: [d_date_sk#8] - -(9) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [d_date_sk#8] -Arguments: [ss_sold_date_sk#6], [d_date_sk#8], Inner, BuildRight - -(10) CometProject -Input [5]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] -Arguments: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5], [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] - -(11) CometHashAggregate -Input [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] -Keys [2]: [ss_store_sk#4, ss_item_sk#3] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#5))] - -(12) CometExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(13) CometHashAggregate -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -Keys [2]: [ss_store_sk#4, ss_item_sk#3] -Functions [1]: [sum(UnscaledValue(ss_sales_price#5))] - -(14) CometFilter -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] -Condition : isnotnull(revenue#11) - -(15) CometBroadcastExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] -Arguments: [ss_store_sk#4, ss_item_sk#3, revenue#11] - -(16) CometBroadcastHashJoin -Left output [2]: [s_store_sk#1, s_store_name#2] -Right output [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] -Arguments: [s_store_sk#1], [ss_store_sk#4], Inner, BuildRight - -(17) CometProject -Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] -Arguments: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11], [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(19) CometFilter -Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -Condition : isnotnull(i_item_sk#12) - -(20) CometProject -Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -Arguments: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17], [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#16, 50, true, false, true) AS i_brand#17] - -(21) CometBroadcastExchange -Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(22) CometBroadcastHashJoin -Left output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] -Right output [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: [ss_item_sk#3], [i_item_sk#12], Inner, BuildRight - -(23) CometProject -Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11, i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17], [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#21), dynamicpruningexpression(ss_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] -Condition : isnotnull(ss_store_sk#19) - -(26) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#23] - -(27) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] -Right output [1]: [d_date_sk#23] -Arguments: [ss_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(28) CometProject -Input [5]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21, d_date_sk#23] -Arguments: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20], [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] - -(29) CometHashAggregate -Input [3]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] -Keys [2]: [ss_store_sk#19, ss_item_sk#18] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#20))] - -(30) CometExchange -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#24] -Arguments: hashpartitioning(ss_store_sk#19, ss_item_sk#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(31) CometHashAggregate -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#24] -Keys [2]: [ss_store_sk#19, ss_item_sk#18] -Functions [1]: [sum(UnscaledValue(ss_sales_price#20))] - -(32) CometHashAggregate -Input [2]: [ss_store_sk#19, revenue#25] -Keys [1]: [ss_store_sk#19] -Functions [1]: [partial_avg(revenue#25)] - -(33) CometExchange -Input [3]: [ss_store_sk#19, sum#26, count#27] -Arguments: hashpartitioning(ss_store_sk#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(34) CometHashAggregate -Input [3]: [ss_store_sk#19, sum#26, count#27] -Keys [1]: [ss_store_sk#19] -Functions [1]: [avg(revenue#25)] - -(35) CometFilter -Input [2]: [ss_store_sk#19, ave#28] -Condition : isnotnull(ave#28) - -(36) CometBroadcastExchange -Input [2]: [ss_store_sk#19, ave#28] -Arguments: [ss_store_sk#19, ave#28] - -(37) CometBroadcastHashJoin -Left output [7]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Right output [2]: [ss_store_sk#19, ave#28] -Arguments: [ss_store_sk#4], [ss_store_sk#19], Inner, (cast(revenue#11 as decimal(23,7)) <= (0.1 * ave#28)), BuildRight - -(38) CometProject -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17, ss_store_sk#19, ave#28] -Arguments: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(39) CometTakeOrderedAndProject -Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#2 ASC NULLS FIRST,i_item_desc#13 ASC NULLS FIRST], output=[s_store_name#2,i_item_desc#13,revenue#11,i_current_price#14,i_wholesale_cost#15,i_brand#17]), [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17], 100, 0, [s_store_name#2 ASC NULLS FIRST, i_item_desc#13 ASC NULLS FIRST], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(40) CometColumnarToRow [codegen id : 1] -Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#8, d_month_seq#9] -Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_month_seq#9 <= 1187)) AND isnotnull(d_date_sk#8)) - -(43) CometProject -Input [2]: [d_date_sk#8, d_month_seq#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(45) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/simplified.txt deleted file mode 100644 index cd1c84b6f2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,51 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - CometProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - CometBroadcastHashJoin [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand,ss_store_sk,ave] - CometProject [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometBroadcastHashJoin [s_store_name,ss_store_sk,ss_item_sk,revenue,i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometProject [s_store_name,ss_store_sk,ss_item_sk,revenue] - CometBroadcastHashJoin [s_store_sk,s_store_name,ss_store_sk,ss_item_sk,revenue] - CometFilter [s_store_sk,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name] - CometBroadcastExchange [ss_store_sk,ss_item_sk,revenue] #1 - CometFilter [ss_store_sk,ss_item_sk,revenue] - CometHashAggregate [sum] [ss_store_sk,ss_item_sk,revenue,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_store_sk,ss_item_sk] #2 - CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] - CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] #5 - CometProject [i_brand] [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometBroadcastExchange [ss_store_sk,ave] #6 - CometFilter [ss_store_sk,ave] - CometHashAggregate [sum,count] [ss_store_sk,ave,avg(revenue)] - CometExchange [ss_store_sk] #7 - CometHashAggregate [revenue] [ss_store_sk,sum,count] - CometHashAggregate [ss_item_sk,sum] [ss_store_sk,revenue,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_store_sk,ss_item_sk] #8 - CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] - CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - 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 [native_iceberg_compat] 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/q65/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt deleted file mode 100644 index 8b38832289..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt +++ /dev/null @@ -1,264 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometProject (38) - +- CometBroadcastHashJoin (37) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (1) - : : +- CometBroadcastExchange (15) - : : +- CometFilter (14) - : : +- CometHashAggregate (13) - : : +- CometExchange (12) - : : +- CometHashAggregate (11) - : : +- CometProject (10) - : : +- CometBroadcastHashJoin (9) - : : :- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : +- CometBroadcastExchange (8) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (5) - : +- CometBroadcastExchange (21) - : +- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (18) - +- CometBroadcastExchange (36) - +- CometFilter (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometHashAggregate (31) - +- CometExchange (30) - +- CometHashAggregate (29) - +- CometProject (28) - +- CometBroadcastHashJoin (27) - :- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (24) - +- ReusedExchange (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#1, s_store_name#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [s_store_sk#1, s_store_name#2] -Condition : isnotnull(s_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [d_date_sk#8, d_month_seq#9] -Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_month_seq#9 <= 1187)) AND isnotnull(d_date_sk#8)) - -(7) CometProject -Input [2]: [d_date_sk#8, d_month_seq#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(8) CometBroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: [d_date_sk#8] - -(9) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] -Right output [1]: [d_date_sk#8] -Arguments: [ss_sold_date_sk#6], [d_date_sk#8], Inner, BuildRight - -(10) CometProject -Input [5]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] -Arguments: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5], [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] - -(11) CometHashAggregate -Input [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] -Keys [2]: [ss_store_sk#4, ss_item_sk#3] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#5))] - -(12) CometExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(13) CometHashAggregate -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] -Keys [2]: [ss_store_sk#4, ss_item_sk#3] -Functions [1]: [sum(UnscaledValue(ss_sales_price#5))] - -(14) CometFilter -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] -Condition : isnotnull(revenue#11) - -(15) CometBroadcastExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] -Arguments: [ss_store_sk#4, ss_item_sk#3, revenue#11] - -(16) CometBroadcastHashJoin -Left output [2]: [s_store_sk#1, s_store_name#2] -Right output [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] -Arguments: [s_store_sk#1], [ss_store_sk#4], Inner, BuildRight - -(17) CometProject -Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] -Arguments: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11], [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(19) CometFilter -Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -Condition : isnotnull(i_item_sk#12) - -(20) CometProject -Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] -Arguments: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17], [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#16, 50, true, false, true) AS i_brand#17] - -(21) CometBroadcastExchange -Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(22) CometBroadcastHashJoin -Left output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] -Right output [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: [ss_item_sk#3], [i_item_sk#12], Inner, BuildRight - -(23) CometProject -Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11, i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17], [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#21), dynamicpruningexpression(ss_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] -Condition : isnotnull(ss_store_sk#19) - -(26) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#23] - -(27) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] -Right output [1]: [d_date_sk#23] -Arguments: [ss_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(28) CometProject -Input [5]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21, d_date_sk#23] -Arguments: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20], [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] - -(29) CometHashAggregate -Input [3]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] -Keys [2]: [ss_store_sk#19, ss_item_sk#18] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#20))] - -(30) CometExchange -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#24] -Arguments: hashpartitioning(ss_store_sk#19, ss_item_sk#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(31) CometHashAggregate -Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#24] -Keys [2]: [ss_store_sk#19, ss_item_sk#18] -Functions [1]: [sum(UnscaledValue(ss_sales_price#20))] - -(32) CometHashAggregate -Input [2]: [ss_store_sk#19, revenue#25] -Keys [1]: [ss_store_sk#19] -Functions [1]: [partial_avg(revenue#25)] - -(33) CometExchange -Input [3]: [ss_store_sk#19, sum#26, count#27] -Arguments: hashpartitioning(ss_store_sk#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(34) CometHashAggregate -Input [3]: [ss_store_sk#19, sum#26, count#27] -Keys [1]: [ss_store_sk#19] -Functions [1]: [avg(revenue#25)] - -(35) CometFilter -Input [2]: [ss_store_sk#19, ave#28] -Condition : isnotnull(ave#28) - -(36) CometBroadcastExchange -Input [2]: [ss_store_sk#19, ave#28] -Arguments: [ss_store_sk#19, ave#28] - -(37) CometBroadcastHashJoin -Left output [7]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Right output [2]: [ss_store_sk#19, ave#28] -Arguments: [ss_store_sk#4], [ss_store_sk#19], Inner, (cast(revenue#11 as decimal(23,7)) <= (0.1 * ave#28)), BuildRight - -(38) CometProject -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17, ss_store_sk#19, ave#28] -Arguments: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(39) CometTakeOrderedAndProject -Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#2 ASC NULLS FIRST,i_item_desc#13 ASC NULLS FIRST], output=[s_store_name#2,i_item_desc#13,revenue#11,i_current_price#14,i_wholesale_cost#15,i_brand#17]), [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17], 100, 0, [s_store_name#2 ASC NULLS FIRST, i_item_desc#13 ASC NULLS FIRST], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -(40) CometColumnarToRow [codegen id : 1] -Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_month_seq#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#8, d_month_seq#9] -Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_month_seq#9 <= 1187)) AND isnotnull(d_date_sk#8)) - -(43) CometProject -Input [2]: [d_date_sk#8, d_month_seq#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(45) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/extended.txt deleted file mode 100644 index 1fc09bd2e3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/extended.txt +++ /dev/null @@ -1,52 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 45 out of 48 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index cd1c84b6f2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt +++ /dev/null @@ -1,51 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - CometProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - CometBroadcastHashJoin [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand,ss_store_sk,ave] - CometProject [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometBroadcastHashJoin [s_store_name,ss_store_sk,ss_item_sk,revenue,i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometProject [s_store_name,ss_store_sk,ss_item_sk,revenue] - CometBroadcastHashJoin [s_store_sk,s_store_name,ss_store_sk,ss_item_sk,revenue] - CometFilter [s_store_sk,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name] - CometBroadcastExchange [ss_store_sk,ss_item_sk,revenue] #1 - CometFilter [ss_store_sk,ss_item_sk,revenue] - CometHashAggregate [sum] [ss_store_sk,ss_item_sk,revenue,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_store_sk,ss_item_sk] #2 - CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] - CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] #5 - CometProject [i_brand] [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometBroadcastExchange [ss_store_sk,ave] #6 - CometFilter [ss_store_sk,ave] - CometHashAggregate [sum,count] [ss_store_sk,ave,avg(revenue)] - CometExchange [ss_store_sk] #7 - CometHashAggregate [revenue] [ss_store_sk,sum,count] - CometHashAggregate [ss_item_sk,sum] [ss_store_sk,revenue,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_store_sk,ss_item_sk] #8 - CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] - CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - 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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/explain.txt deleted file mode 100644 index 56c5025634..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/explain.txt +++ /dev/null @@ -1,349 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (56) -+- * HashAggregate (55) - +- * CometColumnarToRow (54) - +- CometColumnarExchange (53) - +- * HashAggregate (52) - +- Union (51) - :- * HashAggregate (31) - : +- * CometColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- * HashAggregate (28) - : +- * Project (27) - : +- * BroadcastHashJoin Inner BuildRight (26) - : :- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.warehouse (4) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (18) - : : +- * CometColumnarToRow (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.time_dim (14) - : +- BroadcastExchange (25) - : +- * CometColumnarToRow (24) - : +- CometProject (23) - : +- CometFilter (22) - : +- CometNativeScan parquet spark_catalog.default.ship_mode (21) - +- * HashAggregate (50) - +- * CometColumnarToRow (49) - +- CometColumnarExchange (48) - +- * HashAggregate (47) - +- * Project (46) - +- * BroadcastHashJoin Inner BuildRight (45) - :- * Project (43) - : +- * BroadcastHashJoin Inner BuildRight (42) - : :- * Project (40) - : : +- * BroadcastHashJoin Inner BuildRight (39) - : : :- * Project (37) - : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : :- * Filter (34) - : : : : +- * ColumnarToRow (33) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (32) - : : : +- ReusedExchange (35) - : : +- ReusedExchange (38) - : +- ReusedExchange (41) - +- ReusedExchange (44) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_ship_mode_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] - -(3) Filter [codegen id : 5] -Input [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Condition : ((isnotnull(ws_warehouse_sk#3) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_ship_mode_sk#2)) - -(4) CometNativeScan parquet spark_catalog.default.warehouse -Output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(5) CometFilter -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Condition : isnotnull(w_warehouse_sk#9) - -(6) CometProject -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Arguments: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15], [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, w_state#14, 2, true, false, true) AS w_state#16, w_country#15] - -(7) CometColumnarToRow [codegen id : 1] -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] - -(8) BroadcastExchange -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_warehouse_sk#3] -Right keys [1]: [w_warehouse_sk#9] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 5] -Output [12]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] - -(11) ReusedExchange [Reuses operator id: 60] -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_sold_date_sk#7] -Right keys [1]: [d_date_sk#17] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [13]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Input [15]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_date_sk#17, d_year#18, d_moy#19] - -(14) CometNativeScan parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#20, t_time#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_time), GreaterThanOrEqual(t_time,30838), LessThanOrEqual(t_time,59638), IsNotNull(t_time_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [t_time_sk#20, t_time#21] -Condition : (((isnotnull(t_time#21) AND (t_time#21 >= 30838)) AND (t_time#21 <= 59638)) AND isnotnull(t_time_sk#20)) - -(16) CometProject -Input [2]: [t_time_sk#20, t_time#21] -Arguments: [t_time_sk#20], [t_time_sk#20] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [t_time_sk#20] - -(18) BroadcastExchange -Input [1]: [t_time_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_sold_time_sk#1] -Right keys [1]: [t_time_sk#20] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [12]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, t_time_sk#20] - -(21) CometNativeScan parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, sm_carrier#23, 20, true, false, true) IN (DHL ,BARIAN ) AND isnotnull(sm_ship_mode_sk#22)) - -(23) CometProject -Input [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Arguments: [sm_ship_mode_sk#22], [sm_ship_mode_sk#22] - -(24) CometColumnarToRow [codegen id : 4] -Input [1]: [sm_ship_mode_sk#22] - -(25) BroadcastExchange -Input [1]: [sm_ship_mode_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ws_ship_mode_sk#2] -Right keys [1]: [sm_ship_mode_sk#22] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 5] -Output [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Input [13]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, sm_ship_mode_sk#22] - -(28) HashAggregate [codegen id : 5] -Input [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] -Functions [24]: [partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] -Aggregate Attributes [48]: [sum#24, isEmpty#25, sum#26, isEmpty#27, sum#28, isEmpty#29, sum#30, isEmpty#31, sum#32, isEmpty#33, sum#34, isEmpty#35, sum#36, isEmpty#37, sum#38, isEmpty#39, sum#40, isEmpty#41, sum#42, isEmpty#43, sum#44, isEmpty#45, sum#46, isEmpty#47, sum#48, isEmpty#49, sum#50, isEmpty#51, sum#52, isEmpty#53, sum#54, isEmpty#55, sum#56, isEmpty#57, sum#58, isEmpty#59, sum#60, isEmpty#61, sum#62, isEmpty#63, sum#64, isEmpty#65, sum#66, isEmpty#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Results [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#72, isEmpty#73, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] - -(29) CometColumnarExchange -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#72, isEmpty#73, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 6] -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#72, isEmpty#73, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] - -(31) HashAggregate [codegen id : 6] -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#72, isEmpty#73, sum#74, isEmpty#75, sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87, sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] -Functions [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#120, sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#121, sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#122, sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#123, sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#124, sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#125, sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#126, sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#127, sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#128, sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#129, sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#130, sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#131, sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#132, sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#133, sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#134, sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#135, sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#136, sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#137, sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#138, sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#139, sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#140, sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#141, sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#142, sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#143] -Results [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, DHL,BARIAN AS ship_carriers#144, d_year#18 AS year#145, sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#120 AS jan_sales#146, sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#121 AS feb_sales#147, sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#122 AS mar_sales#148, sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#123 AS apr_sales#149, sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#124 AS may_sales#150, sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#125 AS jun_sales#151, sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#126 AS jul_sales#152, sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#127 AS aug_sales#153, sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#128 AS sep_sales#154, sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#129 AS oct_sales#155, sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#130 AS nov_sales#156, sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#131 AS dec_sales#157, sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#132 AS jan_net#158, sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#133 AS feb_net#159, sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#134 AS mar_net#160, sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#135 AS apr_net#161, sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#136 AS may_net#162, sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#137 AS jun_net#163, sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#138 AS jul_net#164, sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#139 AS aug_net#165, sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#140 AS sep_net#166, sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#141 AS oct_net#167, sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#142 AS nov_net#168, sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#143 AS dec_net#169] - -(32) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_warehouse_sk#172, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#176), dynamicpruningexpression(cs_sold_date_sk#176 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] -ReadSchema: struct - -(33) ColumnarToRow [codegen id : 11] -Input [7]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_warehouse_sk#172, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176] - -(34) Filter [codegen id : 11] -Input [7]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_warehouse_sk#172, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176] -Condition : ((isnotnull(cs_warehouse_sk#172) AND isnotnull(cs_sold_time_sk#170)) AND isnotnull(cs_ship_mode_sk#171)) - -(35) ReusedExchange [Reuses operator id: 8] -Output [7]: [w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] - -(36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_warehouse_sk#172] -Right keys [1]: [w_warehouse_sk#177] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 11] -Output [12]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] -Input [14]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_warehouse_sk#172, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176, w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] - -(38) ReusedExchange [Reuses operator id: 60] -Output [3]: [d_date_sk#184, d_year#185, d_moy#186] - -(39) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#176] -Right keys [1]: [d_date_sk#184] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 11] -Output [13]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -Input [15]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, cs_sold_date_sk#176, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_date_sk#184, d_year#185, d_moy#186] - -(41) ReusedExchange [Reuses operator id: 18] -Output [1]: [t_time_sk#187] - -(42) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_time_sk#170] -Right keys [1]: [t_time_sk#187] -Join type: Inner -Join condition: None - -(43) Project [codegen id : 11] -Output [12]: [cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -Input [14]: [cs_sold_time_sk#170, cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, t_time_sk#187] - -(44) ReusedExchange [Reuses operator id: 25] -Output [1]: [sm_ship_mode_sk#188] - -(45) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_ship_mode_sk#171] -Right keys [1]: [sm_ship_mode_sk#188] -Join type: Inner -Join condition: None - -(46) Project [codegen id : 11] -Output [11]: [cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -Input [13]: [cs_ship_mode_sk#171, cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, sm_ship_mode_sk#188] - -(47) HashAggregate [codegen id : 11] -Input [11]: [cs_quantity#173, cs_sales_price#174, cs_net_paid_inc_tax#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] -Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] -Functions [24]: [partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)] -Aggregate Attributes [48]: [sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236] -Results [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] - -(48) CometColumnarExchange -Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] -Arguments: hashpartitioning(w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(49) CometColumnarToRow [codegen id : 12] -Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] - -(50) HashAggregate [codegen id : 12] -Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] -Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] -Functions [24]: [sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#286, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#287, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#288, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#289, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#301, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#302, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#303, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#304, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#305, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#306, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#307, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#308] -Results [32]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, DHL,BARIAN AS ship_carriers#309, d_year#185 AS year#310, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#285 AS jan_sales#311, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#286 AS feb_sales#312, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#287 AS mar_sales#313, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#288 AS apr_sales#314, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#289 AS may_sales#315, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#290 AS jun_sales#316, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#291 AS jul_sales#317, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#292 AS aug_sales#318, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#293 AS sep_sales#319, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#294 AS oct_sales#320, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#295 AS nov_sales#321, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#174 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#296 AS dec_sales#322, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#297 AS jan_net#323, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#298 AS feb_net#324, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#299 AS mar_net#325, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#300 AS apr_net#326, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#301 AS may_net#327, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#302 AS jun_net#328, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#303 AS jul_net#329, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#304 AS aug_net#330, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#305 AS sep_net#331, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#306 AS oct_net#332, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#307 AS nov_net#333, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#175 * cast(cs_quantity#173 as decimal(10,0))) ELSE 0.00 END)#308 AS dec_net#334] - -(51) Union - -(52) HashAggregate [codegen id : 13] -Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, jan_sales#146, feb_sales#147, mar_sales#148, apr_sales#149, may_sales#150, jun_sales#151, jul_sales#152, aug_sales#153, sep_sales#154, oct_sales#155, nov_sales#156, dec_sales#157, jan_net#158, feb_net#159, mar_net#160, apr_net#161, may_net#162, jun_net#163, jul_net#164, aug_net#165, sep_net#166, oct_net#167, nov_net#168, dec_net#169] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145] -Functions [36]: [partial_sum(jan_sales#146), partial_sum(feb_sales#147), partial_sum(mar_sales#148), partial_sum(apr_sales#149), partial_sum(may_sales#150), partial_sum(jun_sales#151), partial_sum(jul_sales#152), partial_sum(aug_sales#153), partial_sum(sep_sales#154), partial_sum(oct_sales#155), partial_sum(nov_sales#156), partial_sum(dec_sales#157), partial_sum((jan_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#157 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#158), partial_sum(feb_net#159), partial_sum(mar_net#160), partial_sum(apr_net#161), partial_sum(may_net#162), partial_sum(jun_net#163), partial_sum(jul_net#164), partial_sum(aug_net#165), partial_sum(sep_net#166), partial_sum(oct_net#167), partial_sum(nov_net#168), partial_sum(dec_net#169)] -Aggregate Attributes [72]: [sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358, sum#359, isEmpty#360, sum#361, isEmpty#362, sum#363, isEmpty#364, sum#365, isEmpty#366, sum#367, isEmpty#368, sum#369, isEmpty#370, sum#371, isEmpty#372, sum#373, isEmpty#374, sum#375, isEmpty#376, sum#377, isEmpty#378, sum#379, isEmpty#380, sum#381, isEmpty#382, sum#383, isEmpty#384, sum#385, isEmpty#386, sum#387, isEmpty#388, sum#389, isEmpty#390, sum#391, isEmpty#392, sum#393, isEmpty#394, sum#395, isEmpty#396, sum#397, isEmpty#398, sum#399, isEmpty#400, sum#401, isEmpty#402, sum#403, isEmpty#404, sum#405, isEmpty#406] -Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] - -(53) CometColumnarExchange -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(54) CometColumnarToRow [codegen id : 14] -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] - -(55) HashAggregate [codegen id : 14] -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145] -Functions [36]: [sum(jan_sales#146), sum(feb_sales#147), sum(mar_sales#148), sum(apr_sales#149), sum(may_sales#150), sum(jun_sales#151), sum(jul_sales#152), sum(aug_sales#153), sum(sep_sales#154), sum(oct_sales#155), sum(nov_sales#156), sum(dec_sales#157), sum((jan_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#157 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#158), sum(feb_net#159), sum(mar_net#160), sum(apr_net#161), sum(may_net#162), sum(jun_net#163), sum(jul_net#164), sum(aug_net#165), sum(sep_net#166), sum(oct_net#167), sum(nov_net#168), sum(dec_net#169)] -Aggregate Attributes [36]: [sum(jan_sales#146)#479, sum(feb_sales#147)#480, sum(mar_sales#148)#481, sum(apr_sales#149)#482, sum(may_sales#150)#483, sum(jun_sales#151)#484, sum(jul_sales#152)#485, sum(aug_sales#153)#486, sum(sep_sales#154)#487, sum(oct_sales#155)#488, sum(nov_sales#156)#489, sum(dec_sales#157)#490, sum((jan_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#491, sum((feb_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#492, sum((mar_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#493, sum((apr_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#494, sum((may_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#495, sum((jun_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#496, sum((jul_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#497, sum((aug_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#498, sum((sep_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#499, sum((oct_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#500, sum((nov_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#501, sum((dec_sales#157 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#502, sum(jan_net#158)#503, sum(feb_net#159)#504, sum(mar_net#160)#505, sum(apr_net#161)#506, sum(may_net#162)#507, sum(jun_net#163)#508, sum(jul_net#164)#509, sum(aug_net#165)#510, sum(sep_net#166)#511, sum(oct_net#167)#512, sum(nov_net#168)#513, sum(dec_net#169)#514] -Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, sum(jan_sales#146)#479 AS jan_sales#515, sum(feb_sales#147)#480 AS feb_sales#516, sum(mar_sales#148)#481 AS mar_sales#517, sum(apr_sales#149)#482 AS apr_sales#518, sum(may_sales#150)#483 AS may_sales#519, sum(jun_sales#151)#484 AS jun_sales#520, sum(jul_sales#152)#485 AS jul_sales#521, sum(aug_sales#153)#486 AS aug_sales#522, sum(sep_sales#154)#487 AS sep_sales#523, sum(oct_sales#155)#488 AS oct_sales#524, sum(nov_sales#156)#489 AS nov_sales#525, sum(dec_sales#157)#490 AS dec_sales#526, sum((jan_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#491 AS jan_sales_per_sq_foot#527, sum((feb_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#492 AS feb_sales_per_sq_foot#528, sum((mar_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#493 AS mar_sales_per_sq_foot#529, sum((apr_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#494 AS apr_sales_per_sq_foot#530, sum((may_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#495 AS may_sales_per_sq_foot#531, sum((jun_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#496 AS jun_sales_per_sq_foot#532, sum((jul_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#497 AS jul_sales_per_sq_foot#533, sum((aug_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#498 AS aug_sales_per_sq_foot#534, sum((sep_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#499 AS sep_sales_per_sq_foot#535, sum((oct_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#500 AS oct_sales_per_sq_foot#536, sum((nov_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#501 AS nov_sales_per_sq_foot#537, sum((dec_sales#157 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#502 AS dec_sales_per_sq_foot#538, sum(jan_net#158)#503 AS jan_net#539, sum(feb_net#159)#504 AS feb_net#540, sum(mar_net#160)#505 AS mar_net#541, sum(apr_net#161)#506 AS apr_net#542, sum(may_net#162)#507 AS may_net#543, sum(jun_net#163)#508 AS jun_net#544, sum(jul_net#164)#509 AS jul_net#545, sum(aug_net#165)#510 AS aug_net#546, sum(sep_net#166)#511 AS sep_net#547, sum(oct_net#167)#512 AS oct_net#548, sum(nov_net#168)#513 AS nov_net#549, sum(dec_net#169)#514 AS dec_net#550] - -(56) TakeOrderedAndProject -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, jan_sales#515, feb_sales#516, mar_sales#517, apr_sales#518, may_sales#519, jun_sales#520, jul_sales#521, aug_sales#522, sep_sales#523, oct_sales#524, nov_sales#525, dec_sales#526, jan_sales_per_sq_foot#527, feb_sales_per_sq_foot#528, mar_sales_per_sq_foot#529, apr_sales_per_sq_foot#530, may_sales_per_sq_foot#531, jun_sales_per_sq_foot#532, jul_sales_per_sq_foot#533, aug_sales_per_sq_foot#534, sep_sales_per_sq_foot#535, oct_sales_per_sq_foot#536, nov_sales_per_sq_foot#537, dec_sales_per_sq_foot#538, jan_net#539, feb_net#540, mar_net#541, apr_net#542, may_net#543, jun_net#544, jul_net#545, aug_net#546, sep_net#547, oct_net#548, nov_net#549, dec_net#550] -Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#144, year#145, jan_sales#515, feb_sales#516, mar_sales#517, apr_sales#518, may_sales#519, jun_sales#520, jul_sales#521, aug_sales#522, sep_sales#523, oct_sales#524, nov_sales#525, dec_sales#526, jan_sales_per_sq_foot#527, feb_sales_per_sq_foot#528, mar_sales_per_sq_foot#529, apr_sales_per_sq_foot#530, may_sales_per_sq_foot#531, jun_sales_per_sq_foot#532, jul_sales_per_sq_foot#533, aug_sales_per_sq_foot#534, sep_sales_per_sq_foot#535, oct_sales_per_sq_foot#536, nov_sales_per_sq_foot#537, dec_sales_per_sq_foot#538, jan_net#539, feb_net#540, mar_net#541, apr_net#542, may_net#543, jun_net#544, jul_net#545, aug_net#546, sep_net#547, oct_net#548, nov_net#549, dec_net#550] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (60) -+- * CometColumnarToRow (59) - +- CometFilter (58) - +- CometNativeScan parquet spark_catalog.default.date_dim (57) - - -(57) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(59) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] - -(60) BroadcastExchange -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#176 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/simplified.txt deleted file mode 100644 index 3a1f053d60..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - WholeStageCodegen (14) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,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,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum((jan_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((feb_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((mar_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((apr_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((may_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jun_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jul_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((aug_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((sep_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((oct_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((nov_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((dec_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net),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_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,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,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - WholeStageCodegen (13) - HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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,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,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] - InputAdapter - Union - WholeStageCodegen (6) - 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 (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - WholeStageCodegen (5) - HashAggregate [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] [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,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] - Project [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] - BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] - Project [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] - BroadcastHashJoin [ws_sold_time_sk,t_time_sk] - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] - Filter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [w_state] [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [t_time_sk] - CometFilter [t_time_sk,t_time] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [sm_ship_mode_sk] - CometFilter [sm_ship_mode_sk,sm_carrier] - CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] - WholeStageCodegen (12) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #7 - WholeStageCodegen (11) - HashAggregate [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] [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,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] - Project [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] - BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] - Project [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] - BroadcastHashJoin [cs_sold_time_sk,t_time_sk] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] - Filter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #3 - InputAdapter - ReusedExchange [t_time_sk] #5 - InputAdapter - ReusedExchange [sm_ship_mode_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/explain.txt deleted file mode 100644 index 7599e9ecd3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,317 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (51) -+- CometTakeOrderedAndProject (50) - +- CometHashAggregate (49) - +- CometExchange (48) - +- CometHashAggregate (47) - +- CometUnion (46) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (14) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode (20) - +- CometHashAggregate (45) - +- CometExchange (44) - +- CometHashAggregate (43) - +- CometProject (42) - +- CometBroadcastHashJoin (41) - :- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometFilter (30) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_ship_mode_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Condition : ((isnotnull(ws_warehouse_sk#3) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_ship_mode_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Condition : isnotnull(w_warehouse_sk#9) - -(5) CometProject -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Arguments: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15], [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, w_state#14, 2, true, false, true) AS w_state#16, w_country#15] - -(6) CometBroadcastExchange -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Arguments: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] - -(7) CometBroadcastHashJoin -Left output [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Right output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Arguments: [ws_warehouse_sk#3], [w_warehouse_sk#9], Inner, BuildRight - -(8) CometProject -Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Arguments: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15], [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [d_date_sk#17, d_year#18, d_moy#19] - -(12) CometBroadcastHashJoin -Left output [12]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Right output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [ws_sold_date_sk#7], [d_date_sk#17], Inner, BuildRight - -(13) CometProject -Input [15]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_date_sk#17, d_year#18, d_moy#19] -Arguments: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19], [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#20, t_time#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_time), GreaterThanOrEqual(t_time,30838), LessThanOrEqual(t_time,59638), IsNotNull(t_time_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [t_time_sk#20, t_time#21] -Condition : (((isnotnull(t_time#21) AND (t_time#21 >= 30838)) AND (t_time#21 <= 59638)) AND isnotnull(t_time_sk#20)) - -(16) CometProject -Input [2]: [t_time_sk#20, t_time#21] -Arguments: [t_time_sk#20], [t_time_sk#20] - -(17) CometBroadcastExchange -Input [1]: [t_time_sk#20] -Arguments: [t_time_sk#20] - -(18) CometBroadcastHashJoin -Left output [13]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Right output [1]: [t_time_sk#20] -Arguments: [ws_sold_time_sk#1], [t_time_sk#20], Inner, BuildRight - -(19) CometProject -Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, t_time_sk#20] -Arguments: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19], [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, sm_carrier#23, 20, true, false, true) IN (DHL ,BARIAN ) AND isnotnull(sm_ship_mode_sk#22)) - -(22) CometProject -Input [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Arguments: [sm_ship_mode_sk#22], [sm_ship_mode_sk#22] - -(23) CometBroadcastExchange -Input [1]: [sm_ship_mode_sk#22] -Arguments: [sm_ship_mode_sk#22] - -(24) CometBroadcastHashJoin -Left output [12]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Right output [1]: [sm_ship_mode_sk#22] -Arguments: [ws_ship_mode_sk#2], [sm_ship_mode_sk#22], Inner, BuildRight - -(25) CometProject -Input [13]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, sm_ship_mode_sk#22] -Arguments: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19], [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] - -(26) CometHashAggregate -Input [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] -Functions [24]: [partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] - -(27) CometExchange -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#24, isEmpty#25, sum#26, isEmpty#27, sum#28, isEmpty#29, sum#30, isEmpty#31, sum#32, isEmpty#33, sum#34, isEmpty#35, sum#36, isEmpty#37, sum#38, isEmpty#39, sum#40, isEmpty#41, sum#42, isEmpty#43, sum#44, isEmpty#45, sum#46, isEmpty#47, sum#48, isEmpty#49, sum#50, isEmpty#51, sum#52, isEmpty#53, sum#54, isEmpty#55, sum#56, isEmpty#57, sum#58, isEmpty#59, sum#60, isEmpty#61, sum#62, isEmpty#63, sum#64, isEmpty#65, sum#66, isEmpty#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#24, isEmpty#25, sum#26, isEmpty#27, sum#28, isEmpty#29, sum#30, isEmpty#31, sum#32, isEmpty#33, sum#34, isEmpty#35, sum#36, isEmpty#37, sum#38, isEmpty#39, sum#40, isEmpty#41, sum#42, isEmpty#43, sum#44, isEmpty#45, sum#46, isEmpty#47, sum#48, isEmpty#49, sum#50, isEmpty#51, sum#52, isEmpty#53, sum#54, isEmpty#55, sum#56, isEmpty#57, sum#58, isEmpty#59, sum#60, isEmpty#61, sum#62, isEmpty#63, sum#64, isEmpty#65, sum#66, isEmpty#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] -Functions [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#78), dynamicpruningexpression(cs_sold_date_sk#78 IN dynamicpruning#79)] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] -ReadSchema: struct - -(30) CometFilter -Input [7]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78] -Condition : ((isnotnull(cs_warehouse_sk#74) AND isnotnull(cs_sold_time_sk#72)) AND isnotnull(cs_ship_mode_sk#73)) - -(31) ReusedExchange [Reuses operator id: 6] -Output [7]: [w_warehouse_sk#80, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] - -(32) CometBroadcastHashJoin -Left output [7]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78] -Right output [7]: [w_warehouse_sk#80, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] -Arguments: [cs_warehouse_sk#74], [w_warehouse_sk#80], Inner, BuildRight - -(33) CometProject -Input [14]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_sk#80, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] -Arguments: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86], [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] - -(34) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#87, d_year#88, d_moy#89] - -(35) CometBroadcastHashJoin -Left output [12]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] -Right output [3]: [d_date_sk#87, d_year#88, d_moy#89] -Arguments: [cs_sold_date_sk#78], [d_date_sk#87], Inner, BuildRight - -(36) CometProject -Input [15]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_date_sk#87, d_year#88, d_moy#89] -Arguments: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89], [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] - -(37) ReusedExchange [Reuses operator id: 17] -Output [1]: [t_time_sk#90] - -(38) CometBroadcastHashJoin -Left output [13]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] -Right output [1]: [t_time_sk#90] -Arguments: [cs_sold_time_sk#72], [t_time_sk#90], Inner, BuildRight - -(39) CometProject -Input [14]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89, t_time_sk#90] -Arguments: [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89], [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] - -(40) ReusedExchange [Reuses operator id: 23] -Output [1]: [sm_ship_mode_sk#91] - -(41) CometBroadcastHashJoin -Left output [12]: [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] -Right output [1]: [sm_ship_mode_sk#91] -Arguments: [cs_ship_mode_sk#73], [sm_ship_mode_sk#91], Inner, BuildRight - -(42) CometProject -Input [13]: [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89, sm_ship_mode_sk#91] -Arguments: [cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89], [cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] - -(43) CometHashAggregate -Input [11]: [cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] -Keys [7]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88] -Functions [24]: [partial_sum(CASE WHEN (d_moy#89 = 1) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 2) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 3) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 4) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 5) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 6) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 7) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 8) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 9) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 10) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 11) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 12) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 1) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 2) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 3) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 4) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 5) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 6) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 7) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 8) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 9) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 10) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 11) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 12) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END)] - -(44) CometExchange -Input [55]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Arguments: hashpartitioning(w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(45) CometHashAggregate -Input [55]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Keys [7]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88] -Functions [24]: [sum(CASE WHEN (d_moy#89 = 1) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 2) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 3) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 4) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 5) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 6) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 7) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 8) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 9) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 10) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 11) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 12) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 1) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 2) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 3) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 4) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 5) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 6) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 7) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 8) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 9) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 10) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 11) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 12) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END)] - -(46) CometUnion -Child 0 Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#142, feb_sales#143, mar_sales#144, apr_sales#145, may_sales#146, jun_sales#147, jul_sales#148, aug_sales#149, sep_sales#150, oct_sales#151, nov_sales#152, dec_sales#153, jan_net#154, feb_net#155, mar_net#156, apr_net#157, may_net#158, jun_net#159, jul_net#160, aug_net#161, sep_net#162, oct_net#163, nov_net#164, dec_net#165] -Child 1 Input [32]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, ship_carriers#166, year#167, jan_sales#168, feb_sales#169, mar_sales#170, apr_sales#171, may_sales#172, jun_sales#173, jul_sales#174, aug_sales#175, sep_sales#176, oct_sales#177, nov_sales#178, dec_sales#179, jan_net#180, feb_net#181, mar_net#182, apr_net#183, may_net#184, jun_net#185, jul_net#186, aug_net#187, sep_net#188, oct_net#189, nov_net#190, dec_net#191] - -(47) CometHashAggregate -Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#142, feb_sales#143, mar_sales#144, apr_sales#145, may_sales#146, jun_sales#147, jul_sales#148, aug_sales#149, sep_sales#150, oct_sales#151, nov_sales#152, dec_sales#153, jan_net#154, feb_net#155, mar_net#156, apr_net#157, may_net#158, jun_net#159, jul_net#160, aug_net#161, sep_net#162, oct_net#163, nov_net#164, dec_net#165] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141] -Functions [36]: [partial_sum(jan_sales#142), partial_sum(feb_sales#143), partial_sum(mar_sales#144), partial_sum(apr_sales#145), partial_sum(may_sales#146), partial_sum(jun_sales#147), partial_sum(jul_sales#148), partial_sum(aug_sales#149), partial_sum(sep_sales#150), partial_sum(oct_sales#151), partial_sum(nov_sales#152), partial_sum(dec_sales#153), partial_sum((jan_sales#142 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#143 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#144 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#154), partial_sum(feb_net#155), partial_sum(mar_net#156), partial_sum(apr_net#157), partial_sum(may_net#158), partial_sum(jun_net#159), partial_sum(jul_net#160), partial_sum(aug_net#161), partial_sum(sep_net#162), partial_sum(oct_net#163), partial_sum(nov_net#164), partial_sum(dec_net#165)] - -(48) CometExchange -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211, sum#212, isEmpty#213, sum#214, isEmpty#215, sum#216, isEmpty#217, sum#218, isEmpty#219, sum#220, isEmpty#221, sum#222, isEmpty#223, sum#224, isEmpty#225, sum#226, isEmpty#227, sum#228, isEmpty#229, sum#230, isEmpty#231, sum#232, isEmpty#233, sum#234, isEmpty#235, sum#236, isEmpty#237, sum#238, isEmpty#239, sum#240, isEmpty#241, sum#242, isEmpty#243, sum#244, isEmpty#245, sum#246, isEmpty#247, sum#248, isEmpty#249, sum#250, isEmpty#251, sum#252, isEmpty#253, sum#254, isEmpty#255, sum#256, isEmpty#257, sum#258, isEmpty#259, sum#260, isEmpty#261, sum#262, isEmpty#263] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211, sum#212, isEmpty#213, sum#214, isEmpty#215, sum#216, isEmpty#217, sum#218, isEmpty#219, sum#220, isEmpty#221, sum#222, isEmpty#223, sum#224, isEmpty#225, sum#226, isEmpty#227, sum#228, isEmpty#229, sum#230, isEmpty#231, sum#232, isEmpty#233, sum#234, isEmpty#235, sum#236, isEmpty#237, sum#238, isEmpty#239, sum#240, isEmpty#241, sum#242, isEmpty#243, sum#244, isEmpty#245, sum#246, isEmpty#247, sum#248, isEmpty#249, sum#250, isEmpty#251, sum#252, isEmpty#253, sum#254, isEmpty#255, sum#256, isEmpty#257, sum#258, isEmpty#259, sum#260, isEmpty#261, sum#262, isEmpty#263] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141] -Functions [36]: [sum(jan_sales#142), sum(feb_sales#143), sum(mar_sales#144), sum(apr_sales#145), sum(may_sales#146), sum(jun_sales#147), sum(jul_sales#148), sum(aug_sales#149), sum(sep_sales#150), sum(oct_sales#151), sum(nov_sales#152), sum(dec_sales#153), sum((jan_sales#142 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#143 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#144 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#154), sum(feb_net#155), sum(mar_net#156), sum(apr_net#157), sum(may_net#158), sum(jun_net#159), sum(jul_net#160), sum(aug_net#161), sum(sep_net#162), sum(oct_net#163), sum(nov_net#164), sum(dec_net#165)] - -(50) CometTakeOrderedAndProject -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#10 ASC NULLS FIRST], output=[w_warehouse_name#10,w_warehouse_sq_ft#11,w_city#12,w_county#13,w_state#16,w_country#15,ship_carriers#140,year#141,jan_sales#264,feb_sales#265,mar_sales#266,apr_sales#267,may_sales#268,jun_sales#269,jul_sales#270,aug_sales#271,sep_sales#272,oct_sales#273,nov_sales#274,dec_sales#275,jan_sales_per_sq_foot#276,feb_sales_per_sq_foot#277,mar_sales_per_sq_foot#278,apr_sales_per_sq_foot#279,may_sales_per_sq_foot#280,jun_sales_per_sq_foot#281,jul_sales_per_sq_foot#282,aug_sales_per_sq_foot#283,sep_sales_per_sq_foot#284,oct_sales_per_sq_foot#285,nov_sales_per_sq_foot#286,dec_sales_per_sq_foot#287,jan_net#288,feb_net#289,mar_net#290,apr_net#291,may_net#292,jun_net#293,jul_net#294,aug_net#295,sep_net#296,oct_net#297,nov_net#298,dec_net#299]), [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299], 100, 0, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299] - -(51) CometColumnarToRow [codegen id : 1] -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometFilter (53) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) - - -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(54) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] - -(55) BroadcastExchange -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#78 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/simplified.txt deleted file mode 100644 index bc6bde24b2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - CometHashAggregate [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,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,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(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum((jan_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((feb_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((mar_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((apr_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((may_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jun_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jul_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((aug_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((sep_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((oct_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((nov_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((dec_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net)] - CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - CometHashAggregate [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] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,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,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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] - CometHashAggregate [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] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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,d_year,sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END)] - CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - CometHashAggregate [d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [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] - 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,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,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_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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 - CometProject [w_state] [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [t_time_sk] #6 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_time] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_time] - CometBroadcastExchange [sm_ship_mode_sk] #7 - CometProject [sm_ship_mode_sk] - CometFilter [sm_ship_mode_sk,sm_carrier] - CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] - CometHashAggregate [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] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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,d_year,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)] - CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #8 - CometHashAggregate [d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [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] - 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,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,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_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_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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year,d_moy] #5 - ReusedExchange [t_time_sk] #6 - ReusedExchange [sm_ship_mode_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt deleted file mode 100644 index 7599e9ecd3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt +++ /dev/null @@ -1,317 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (51) -+- CometTakeOrderedAndProject (50) - +- CometHashAggregate (49) - +- CometExchange (48) - +- CometHashAggregate (47) - +- CometUnion (46) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (14) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode (20) - +- CometHashAggregate (45) - +- CometExchange (44) - +- CometHashAggregate (43) - +- CometProject (42) - +- CometBroadcastHashJoin (41) - :- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometFilter (30) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_ship_mode_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Condition : ((isnotnull(ws_warehouse_sk#3) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_ship_mode_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Condition : isnotnull(w_warehouse_sk#9) - -(5) CometProject -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Arguments: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15], [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, w_state#14, 2, true, false, true) AS w_state#16, w_country#15] - -(6) CometBroadcastExchange -Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Arguments: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] - -(7) CometBroadcastHashJoin -Left output [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] -Right output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Arguments: [ws_warehouse_sk#3], [w_warehouse_sk#9], Inner, BuildRight - -(8) CometProject -Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Arguments: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15], [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [d_date_sk#17, d_year#18, d_moy#19] - -(12) CometBroadcastHashJoin -Left output [12]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15] -Right output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: [ws_sold_date_sk#7], [d_date_sk#17], Inner, BuildRight - -(13) CometProject -Input [15]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_date_sk#17, d_year#18, d_moy#19] -Arguments: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19], [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#20, t_time#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_time), GreaterThanOrEqual(t_time,30838), LessThanOrEqual(t_time,59638), IsNotNull(t_time_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [t_time_sk#20, t_time#21] -Condition : (((isnotnull(t_time#21) AND (t_time#21 >= 30838)) AND (t_time#21 <= 59638)) AND isnotnull(t_time_sk#20)) - -(16) CometProject -Input [2]: [t_time_sk#20, t_time#21] -Arguments: [t_time_sk#20], [t_time_sk#20] - -(17) CometBroadcastExchange -Input [1]: [t_time_sk#20] -Arguments: [t_time_sk#20] - -(18) CometBroadcastHashJoin -Left output [13]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Right output [1]: [t_time_sk#20] -Arguments: [ws_sold_time_sk#1], [t_time_sk#20], Inner, BuildRight - -(19) CometProject -Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, t_time_sk#20] -Arguments: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19], [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, sm_carrier#23, 20, true, false, true) IN (DHL ,BARIAN ) AND isnotnull(sm_ship_mode_sk#22)) - -(22) CometProject -Input [2]: [sm_ship_mode_sk#22, sm_carrier#23] -Arguments: [sm_ship_mode_sk#22], [sm_ship_mode_sk#22] - -(23) CometBroadcastExchange -Input [1]: [sm_ship_mode_sk#22] -Arguments: [sm_ship_mode_sk#22] - -(24) CometBroadcastHashJoin -Left output [12]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Right output [1]: [sm_ship_mode_sk#22] -Arguments: [ws_ship_mode_sk#2], [sm_ship_mode_sk#22], Inner, BuildRight - -(25) CometProject -Input [13]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19, sm_ship_mode_sk#22] -Arguments: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19], [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] - -(26) CometHashAggregate -Input [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, d_moy#19] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] -Functions [24]: [partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] - -(27) CometExchange -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#24, isEmpty#25, sum#26, isEmpty#27, sum#28, isEmpty#29, sum#30, isEmpty#31, sum#32, isEmpty#33, sum#34, isEmpty#35, sum#36, isEmpty#37, sum#38, isEmpty#39, sum#40, isEmpty#41, sum#42, isEmpty#43, sum#44, isEmpty#45, sum#46, isEmpty#47, sum#48, isEmpty#49, sum#50, isEmpty#51, sum#52, isEmpty#53, sum#54, isEmpty#55, sum#56, isEmpty#57, sum#58, isEmpty#59, sum#60, isEmpty#61, sum#62, isEmpty#63, sum#64, isEmpty#65, sum#66, isEmpty#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18, sum#24, isEmpty#25, sum#26, isEmpty#27, sum#28, isEmpty#29, sum#30, isEmpty#31, sum#32, isEmpty#33, sum#34, isEmpty#35, sum#36, isEmpty#37, sum#38, isEmpty#39, sum#40, isEmpty#41, sum#42, isEmpty#43, sum#44, isEmpty#45, sum#46, isEmpty#47, sum#48, isEmpty#49, sum#50, isEmpty#51, sum#52, isEmpty#53, sum#54, isEmpty#55, sum#56, isEmpty#57, sum#58, isEmpty#59, sum#60, isEmpty#61, sum#62, isEmpty#63, sum#64, isEmpty#65, sum#66, isEmpty#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, d_year#18] -Functions [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#78), dynamicpruningexpression(cs_sold_date_sk#78 IN dynamicpruning#79)] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] -ReadSchema: struct - -(30) CometFilter -Input [7]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78] -Condition : ((isnotnull(cs_warehouse_sk#74) AND isnotnull(cs_sold_time_sk#72)) AND isnotnull(cs_ship_mode_sk#73)) - -(31) ReusedExchange [Reuses operator id: 6] -Output [7]: [w_warehouse_sk#80, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] - -(32) CometBroadcastHashJoin -Left output [7]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78] -Right output [7]: [w_warehouse_sk#80, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] -Arguments: [cs_warehouse_sk#74], [w_warehouse_sk#80], Inner, BuildRight - -(33) CometProject -Input [14]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_warehouse_sk#74, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_sk#80, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] -Arguments: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86], [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] - -(34) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#87, d_year#88, d_moy#89] - -(35) CometBroadcastHashJoin -Left output [12]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86] -Right output [3]: [d_date_sk#87, d_year#88, d_moy#89] -Arguments: [cs_sold_date_sk#78], [d_date_sk#87], Inner, BuildRight - -(36) CometProject -Input [15]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, cs_sold_date_sk#78, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_date_sk#87, d_year#88, d_moy#89] -Arguments: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89], [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] - -(37) ReusedExchange [Reuses operator id: 17] -Output [1]: [t_time_sk#90] - -(38) CometBroadcastHashJoin -Left output [13]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] -Right output [1]: [t_time_sk#90] -Arguments: [cs_sold_time_sk#72], [t_time_sk#90], Inner, BuildRight - -(39) CometProject -Input [14]: [cs_sold_time_sk#72, cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89, t_time_sk#90] -Arguments: [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89], [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] - -(40) ReusedExchange [Reuses operator id: 23] -Output [1]: [sm_ship_mode_sk#91] - -(41) CometBroadcastHashJoin -Left output [12]: [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] -Right output [1]: [sm_ship_mode_sk#91] -Arguments: [cs_ship_mode_sk#73], [sm_ship_mode_sk#91], Inner, BuildRight - -(42) CometProject -Input [13]: [cs_ship_mode_sk#73, cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89, sm_ship_mode_sk#91] -Arguments: [cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89], [cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] - -(43) CometHashAggregate -Input [11]: [cs_quantity#75, cs_sales_price#76, cs_net_paid_inc_tax#77, w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, d_moy#89] -Keys [7]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88] -Functions [24]: [partial_sum(CASE WHEN (d_moy#89 = 1) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 2) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 3) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 4) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 5) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 6) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 7) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 8) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 9) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 10) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 11) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 12) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 1) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 2) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 3) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 4) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 5) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 6) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 7) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 8) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 9) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 10) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 11) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#89 = 12) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END)] - -(44) CometExchange -Input [55]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Arguments: hashpartitioning(w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(45) CometHashAggregate -Input [55]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Keys [7]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, d_year#88] -Functions [24]: [sum(CASE WHEN (d_moy#89 = 1) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 2) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 3) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 4) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 5) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 6) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 7) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 8) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 9) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 10) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 11) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 12) THEN (cs_sales_price#76 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 1) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 2) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 3) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 4) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 5) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 6) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 7) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 8) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 9) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 10) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 11) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#89 = 12) THEN (cs_net_paid_inc_tax#77 * cast(cs_quantity#75 as decimal(10,0))) ELSE 0.00 END)] - -(46) CometUnion -Child 0 Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#142, feb_sales#143, mar_sales#144, apr_sales#145, may_sales#146, jun_sales#147, jul_sales#148, aug_sales#149, sep_sales#150, oct_sales#151, nov_sales#152, dec_sales#153, jan_net#154, feb_net#155, mar_net#156, apr_net#157, may_net#158, jun_net#159, jul_net#160, aug_net#161, sep_net#162, oct_net#163, nov_net#164, dec_net#165] -Child 1 Input [32]: [w_warehouse_name#81, w_warehouse_sq_ft#82, w_city#83, w_county#84, w_state#85, w_country#86, ship_carriers#166, year#167, jan_sales#168, feb_sales#169, mar_sales#170, apr_sales#171, may_sales#172, jun_sales#173, jul_sales#174, aug_sales#175, sep_sales#176, oct_sales#177, nov_sales#178, dec_sales#179, jan_net#180, feb_net#181, mar_net#182, apr_net#183, may_net#184, jun_net#185, jul_net#186, aug_net#187, sep_net#188, oct_net#189, nov_net#190, dec_net#191] - -(47) CometHashAggregate -Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#142, feb_sales#143, mar_sales#144, apr_sales#145, may_sales#146, jun_sales#147, jul_sales#148, aug_sales#149, sep_sales#150, oct_sales#151, nov_sales#152, dec_sales#153, jan_net#154, feb_net#155, mar_net#156, apr_net#157, may_net#158, jun_net#159, jul_net#160, aug_net#161, sep_net#162, oct_net#163, nov_net#164, dec_net#165] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141] -Functions [36]: [partial_sum(jan_sales#142), partial_sum(feb_sales#143), partial_sum(mar_sales#144), partial_sum(apr_sales#145), partial_sum(may_sales#146), partial_sum(jun_sales#147), partial_sum(jul_sales#148), partial_sum(aug_sales#149), partial_sum(sep_sales#150), partial_sum(oct_sales#151), partial_sum(nov_sales#152), partial_sum(dec_sales#153), partial_sum((jan_sales#142 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#143 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#144 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#154), partial_sum(feb_net#155), partial_sum(mar_net#156), partial_sum(apr_net#157), partial_sum(may_net#158), partial_sum(jun_net#159), partial_sum(jul_net#160), partial_sum(aug_net#161), partial_sum(sep_net#162), partial_sum(oct_net#163), partial_sum(nov_net#164), partial_sum(dec_net#165)] - -(48) CometExchange -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211, sum#212, isEmpty#213, sum#214, isEmpty#215, sum#216, isEmpty#217, sum#218, isEmpty#219, sum#220, isEmpty#221, sum#222, isEmpty#223, sum#224, isEmpty#225, sum#226, isEmpty#227, sum#228, isEmpty#229, sum#230, isEmpty#231, sum#232, isEmpty#233, sum#234, isEmpty#235, sum#236, isEmpty#237, sum#238, isEmpty#239, sum#240, isEmpty#241, sum#242, isEmpty#243, sum#244, isEmpty#245, sum#246, isEmpty#247, sum#248, isEmpty#249, sum#250, isEmpty#251, sum#252, isEmpty#253, sum#254, isEmpty#255, sum#256, isEmpty#257, sum#258, isEmpty#259, sum#260, isEmpty#261, sum#262, isEmpty#263] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211, sum#212, isEmpty#213, sum#214, isEmpty#215, sum#216, isEmpty#217, sum#218, isEmpty#219, sum#220, isEmpty#221, sum#222, isEmpty#223, sum#224, isEmpty#225, sum#226, isEmpty#227, sum#228, isEmpty#229, sum#230, isEmpty#231, sum#232, isEmpty#233, sum#234, isEmpty#235, sum#236, isEmpty#237, sum#238, isEmpty#239, sum#240, isEmpty#241, sum#242, isEmpty#243, sum#244, isEmpty#245, sum#246, isEmpty#247, sum#248, isEmpty#249, sum#250, isEmpty#251, sum#252, isEmpty#253, sum#254, isEmpty#255, sum#256, isEmpty#257, sum#258, isEmpty#259, sum#260, isEmpty#261, sum#262, isEmpty#263] -Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141] -Functions [36]: [sum(jan_sales#142), sum(feb_sales#143), sum(mar_sales#144), sum(apr_sales#145), sum(may_sales#146), sum(jun_sales#147), sum(jul_sales#148), sum(aug_sales#149), sum(sep_sales#150), sum(oct_sales#151), sum(nov_sales#152), sum(dec_sales#153), sum((jan_sales#142 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#143 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#144 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#154), sum(feb_net#155), sum(mar_net#156), sum(apr_net#157), sum(may_net#158), sum(jun_net#159), sum(jul_net#160), sum(aug_net#161), sum(sep_net#162), sum(oct_net#163), sum(nov_net#164), sum(dec_net#165)] - -(50) CometTakeOrderedAndProject -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[w_warehouse_name#10 ASC NULLS FIRST], output=[w_warehouse_name#10,w_warehouse_sq_ft#11,w_city#12,w_county#13,w_state#16,w_country#15,ship_carriers#140,year#141,jan_sales#264,feb_sales#265,mar_sales#266,apr_sales#267,may_sales#268,jun_sales#269,jul_sales#270,aug_sales#271,sep_sales#272,oct_sales#273,nov_sales#274,dec_sales#275,jan_sales_per_sq_foot#276,feb_sales_per_sq_foot#277,mar_sales_per_sq_foot#278,apr_sales_per_sq_foot#279,may_sales_per_sq_foot#280,jun_sales_per_sq_foot#281,jul_sales_per_sq_foot#282,aug_sales_per_sq_foot#283,sep_sales_per_sq_foot#284,oct_sales_per_sq_foot#285,nov_sales_per_sq_foot#286,dec_sales_per_sq_foot#287,jan_net#288,feb_net#289,mar_net#290,apr_net#291,may_net#292,jun_net#293,jul_net#294,aug_net#295,sep_net#296,oct_net#297,nov_net#298,dec_net#299]), [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299], 100, 0, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299] - -(51) CometColumnarToRow [codegen id : 1] -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#16, w_country#15, ship_carriers#140, year#141, jan_sales#264, feb_sales#265, mar_sales#266, apr_sales#267, may_sales#268, jun_sales#269, jul_sales#270, aug_sales#271, sep_sales#272, oct_sales#273, nov_sales#274, dec_sales#275, jan_sales_per_sq_foot#276, feb_sales_per_sq_foot#277, mar_sales_per_sq_foot#278, apr_sales_per_sq_foot#279, may_sales_per_sq_foot#280, jun_sales_per_sq_foot#281, jul_sales_per_sq_foot#282, aug_sales_per_sq_foot#283, sep_sales_per_sq_foot#284, oct_sales_per_sq_foot#285, nov_sales_per_sq_foot#286, dec_sales_per_sq_foot#287, jan_net#288, feb_net#289, mar_net#290, apr_net#291, may_net#292, jun_net#293, jul_net#294, aug_net#295, sep_net#296, oct_net#297, nov_net#298, dec_net#299] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometFilter (53) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) - - -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2001)) AND isnotnull(d_date_sk#17)) - -(54) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] - -(55) BroadcastExchange -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#78 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/extended.txt deleted file mode 100644 index ca39e001b0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/extended.txt +++ /dev/null @@ -1,70 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - -Comet accelerated 63 out of 66 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index bc6bde24b2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net] - CometHashAggregate [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,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,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(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum((jan_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((feb_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((mar_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((apr_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((may_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jun_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jul_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((aug_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((sep_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((oct_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((nov_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((dec_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net)] - CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 - CometHashAggregate [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] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,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,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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] - CometHashAggregate [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] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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,d_year,sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END)] - CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 - CometHashAggregate [d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [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] - 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,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,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_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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 - CometProject [w_state] [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometFilter [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [t_time_sk] #6 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_time] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_time] - CometBroadcastExchange [sm_ship_mode_sk] #7 - CometProject [sm_ship_mode_sk] - CometFilter [sm_ship_mode_sk,sm_carrier] - CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] - CometHashAggregate [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] [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,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,d_year,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)] - CometExchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #8 - CometHashAggregate [d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [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] - 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,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,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_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_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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year,d_moy] #5 - ReusedExchange [t_time_sk] #6 - ReusedExchange [sm_ship_mode_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/explain.txt deleted file mode 100644 index 35fe4c82c3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/explain.txt +++ /dev/null @@ -1,207 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (31) -+- * Filter (30) - +- Window (29) - +- * CometColumnarToRow (28) - +- CometSort (27) - +- CometColumnarExchange (26) - +- * HashAggregate (25) - +- * CometColumnarToRow (24) - +- CometColumnarExchange (23) - +- * HashAggregate (22) - +- * Expand (21) - +- * Project (20) - +- * BroadcastHashJoin Inner BuildRight (19) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) - : +- CometProject (9) - : +- CometFilter (8) - : +- CometNativeScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (18) - +- * CometColumnarToRow (17) - +- CometProject (16) - +- CometFilter (15) - +- CometNativeScan parquet spark_catalog.default.item (14) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(4) ReusedExchange [Reuses operator id: 36] -Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_store_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [s_store_sk#11, s_store_id#12] -Condition : isnotnull(s_store_sk#11) - -(9) CometProject -Input [2]: [s_store_sk#11, s_store_id#12] -Arguments: [s_store_sk#11, s_store_id#13], [s_store_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#12, 16, true, false, true) AS s_store_id#13] - -(10) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#11, s_store_id#13] - -(11) BroadcastExchange -Input [2]: [s_store_sk#11, s_store_id#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#13] - -(14) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -Condition : isnotnull(i_item_sk#14) - -(16) CometProject -Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -Arguments: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22], [i_item_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#15, 50, true, false, true) AS i_brand#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#16, 50, true, false, true) AS i_class#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#17, 50, true, false, true) AS i_category#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#18, 50, true, false, true) AS i_product_name#22] - -(17) CometColumnarToRow [codegen id : 3] -Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] - -(18) BroadcastExchange -Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#14] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [10]: [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] - -(21) Expand [codegen id : 4] -Input [10]: [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] -Arguments: [[ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, 0], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, null, 1], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, null, null, 3], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, null, null, null, 7], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, i_product_name#22, null, null, null, null, 15], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, i_brand#19, null, null, null, null, null, 31], [ss_quantity#3, ss_sales_price#4, i_category#21, i_class#20, null, null, null, null, null, null, 63], [ss_quantity#3, ss_sales_price#4, i_category#21, null, null, null, null, null, null, null, 127], [ss_quantity#3, ss_sales_price#4, null, null, null, null, null, null, null, null, 255]], [ss_quantity#3, ss_sales_price#4, i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] - -(22) HashAggregate [codegen id : 4] -Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] -Keys [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] -Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [2]: [sum#32, isEmpty#33] -Results [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, sum#34, isEmpty#35] - -(23) CometColumnarExchange -Input [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, sum#34, isEmpty#35] -Arguments: hashpartitioning(i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometColumnarToRow [codegen id : 5] -Input [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, sum#34, isEmpty#35] - -(25) HashAggregate [codegen id : 5] -Input [11]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31, sum#34, isEmpty#35] -Keys [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, spark_grouping_id#31] -Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#36] -Results [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#36 AS sumsales#37] - -(26) CometColumnarExchange -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] -Arguments: hashpartitioning(i_category#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometSort -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] -Arguments: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37], [i_category#23 ASC NULLS FIRST, sumsales#37 DESC NULLS LAST] - -(28) CometColumnarToRow [codegen id : 6] -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] - -(29) Window -Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37] -Arguments: [rank(sumsales#37) windowspecdefinition(i_category#23, sumsales#37 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#38], [i_category#23], [sumsales#37 DESC NULLS LAST] - -(30) Filter [codegen id : 7] -Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37, rk#38] -Condition : (rk#38 <= 100) - -(31) TakeOrderedAndProject -Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37, rk#38] -Arguments: 100, [i_category#23 ASC NULLS FIRST, i_class#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_product_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_qoy#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, s_store_id#30 ASC NULLS FIRST, sumsales#37 ASC NULLS FIRST, rk#38 ASC NULLS FIRST], [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#37, rk#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.date_dim (32) - - -(32) CometNativeScan parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#39, d_year#8, d_moy#9, d_qoy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#39, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#39) AND (d_month_seq#39 >= 1200)) AND (d_month_seq#39 <= 1211)) AND isnotnull(d_date_sk#7)) - -(34) CometProject -Input [5]: [d_date_sk#7, d_month_seq#39, d_year#8, d_moy#9, d_qoy#10] -Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(35) CometColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(36) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/simplified.txt deleted file mode 100644 index 119660fde5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (7) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [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] [sum,isEmpty,sum,isEmpty] - Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometNativeScan 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/q67.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/explain.txt deleted file mode 100644 index df94b5d986..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,203 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (30) -+- * Filter (29) - +- Window (28) - +- * CometColumnarToRow (27) - +- CometSort (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExpand (21) - +- CometProject (20) - +- CometBroadcastHashJoin (19) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - +- CometBroadcastExchange (18) - +- CometProject (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_month_seq#8 <= 1211)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(6) CometBroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Right output [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11], [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_store_id#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#12, s_store_id#13] -Condition : isnotnull(s_store_sk#12) - -(11) CometProject -Input [2]: [s_store_sk#12, s_store_id#13] -Arguments: [s_store_sk#12, s_store_id#14], [s_store_sk#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#13, 16, true, false, true) AS s_store_id#14] - -(12) CometBroadcastExchange -Input [2]: [s_store_sk#12, s_store_id#14] -Arguments: [s_store_sk#12, s_store_id#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] -Right output [2]: [s_store_sk#12, s_store_id#14] -Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_sk#12, s_store_id#14] -Arguments: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14], [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Condition : isnotnull(i_item_sk#15) - -(17) CometProject -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#16, 50, true, false, true) AS i_brand#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#17, 50, true, false, true) AS i_class#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true) AS i_category#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#19, 50, true, false, true) AS i_product_name#23] - -(18) CometBroadcastExchange -Input [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] -Right output [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight - -(20) CometProject -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] - -(21) CometExpand -Input [10]: [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -Arguments: [[ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 0], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, null, 1], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, null, null, 3], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, null, null, null, 7], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, null, null, null, null, 15], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, null, null, null, null, null, 31], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, null, null, null, null, null, null, 63], [ss_quantity#3, ss_sales_price#4, i_category#22, null, null, null, null, null, null, null, 127], [ss_quantity#3, ss_sales_price#4, null, null, null, null, null, null, null, null, 255]], [ss_quantity#3, ss_sales_price#4, i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] - -(22) CometHashAggregate -Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] -Keys [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] -Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(23) CometExchange -Input [11]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32, sum#33, isEmpty#34] -Arguments: hashpartitioning(i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [11]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32, sum#33, isEmpty#34] -Keys [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] -Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(25) CometExchange -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: hashpartitioning(i_category#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(26) CometSort -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35], [i_category#24 ASC NULLS FIRST, sumsales#35 DESC NULLS LAST] - -(27) CometColumnarToRow [codegen id : 1] -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] - -(28) Window -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [rank(sumsales#35) windowspecdefinition(i_category#24, sumsales#35 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#36], [i_category#24], [sumsales#35 DESC NULLS LAST] - -(29) Filter [codegen id : 2] -Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] -Condition : (rk#36 <= 100) - -(30) TakeOrderedAndProject -Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] -Arguments: 100, [i_category#24 ASC NULLS FIRST, i_class#25 ASC NULLS FIRST, i_brand#26 ASC NULLS FIRST, i_product_name#27 ASC NULLS FIRST, d_year#28 ASC NULLS FIRST, d_qoy#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, s_store_id#31 ASC NULLS FIRST, sumsales#35 ASC NULLS FIRST, rk#36 ASC NULLS FIRST], [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_month_seq#8 <= 1211)) AND isnotnull(d_date_sk#7)) - -(33) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(34) CometColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(35) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/simplified.txt deleted file mode 100644 index e85c243077..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,42 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (2) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometExchange [i_category] #1 - CometHashAggregate [spark_grouping_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 - CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] - 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,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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] 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/q67/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt deleted file mode 100644 index df94b5d986..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt +++ /dev/null @@ -1,203 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (30) -+- * Filter (29) - +- Window (28) - +- * CometColumnarToRow (27) - +- CometSort (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExpand (21) - +- CometProject (20) - +- CometBroadcastHashJoin (19) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - +- CometBroadcastExchange (18) - +- CometProject (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_month_seq#8 <= 1211)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(6) CometBroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Right output [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11], [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_store_id#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#12, s_store_id#13] -Condition : isnotnull(s_store_sk#12) - -(11) CometProject -Input [2]: [s_store_sk#12, s_store_id#13] -Arguments: [s_store_sk#12, s_store_id#14], [s_store_sk#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#13, 16, true, false, true) AS s_store_id#14] - -(12) CometBroadcastExchange -Input [2]: [s_store_sk#12, s_store_id#14] -Arguments: [s_store_sk#12, s_store_id#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] -Right output [2]: [s_store_sk#12, s_store_id#14] -Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_sk#12, s_store_id#14] -Arguments: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14], [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Condition : isnotnull(i_item_sk#15) - -(17) CometProject -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#16, 50, true, false, true) AS i_brand#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#17, 50, true, false, true) AS i_class#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true) AS i_category#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#19, 50, true, false, true) AS i_product_name#23] - -(18) CometBroadcastExchange -Input [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] -Right output [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight - -(20) CometProject -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] - -(21) CometExpand -Input [10]: [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -Arguments: [[ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 0], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, null, 1], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, null, null, 3], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, null, null, null, 7], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, i_product_name#23, null, null, null, null, 15], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, i_brand#20, null, null, null, null, null, 31], [ss_quantity#3, ss_sales_price#4, i_category#22, i_class#21, null, null, null, null, null, null, 63], [ss_quantity#3, ss_sales_price#4, i_category#22, null, null, null, null, null, null, null, 127], [ss_quantity#3, ss_sales_price#4, null, null, null, null, null, null, null, null, 255]], [ss_quantity#3, ss_sales_price#4, i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] - -(22) CometHashAggregate -Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] -Keys [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] -Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(23) CometExchange -Input [11]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32, sum#33, isEmpty#34] -Arguments: hashpartitioning(i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [11]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32, sum#33, isEmpty#34] -Keys [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, spark_grouping_id#32] -Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(25) CometExchange -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: hashpartitioning(i_category#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(26) CometSort -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35], [i_category#24 ASC NULLS FIRST, sumsales#35 DESC NULLS LAST] - -(27) CometColumnarToRow [codegen id : 1] -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] - -(28) Window -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [rank(sumsales#35) windowspecdefinition(i_category#24, sumsales#35 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#36], [i_category#24], [sumsales#35 DESC NULLS LAST] - -(29) Filter [codegen id : 2] -Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] -Condition : (rk#36 <= 100) - -(30) TakeOrderedAndProject -Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] -Arguments: 100, [i_category#24 ASC NULLS FIRST, i_class#25 ASC NULLS FIRST, i_brand#26 ASC NULLS FIRST, i_product_name#27 ASC NULLS FIRST, d_year#28 ASC NULLS FIRST, d_qoy#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, s_store_id#31 ASC NULLS FIRST, sumsales#35 ASC NULLS FIRST, rk#36 ASC NULLS FIRST], [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1200)) AND (d_month_seq#8 <= 1211)) AND isnotnull(d_date_sk#7)) - -(33) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(34) CometColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(35) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/extended.txt deleted file mode 100644 index 708d91d578..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/extended.txt +++ /dev/null @@ -1,38 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index e85c243077..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt +++ /dev/null @@ -1,42 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (2) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometExchange [i_category] #1 - CometHashAggregate [spark_grouping_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 - CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] - 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,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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/explain.txt deleted file mode 100644 index 7da946625e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/explain.txt +++ /dev/null @@ -1,267 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (41) -+- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * HashAggregate (30) - : : +- * CometColumnarToRow (29) - : : +- CometColumnarExchange (28) - : : +- * HashAggregate (27) - : : +- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (20) - : : : +- * BroadcastHashJoin Inner BuildRight (19) - : : : :- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Project (6) - : : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : : :- * Filter (3) - : : : : : : +- * ColumnarToRow (2) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : +- ReusedExchange (4) - : : : : +- BroadcastExchange (11) - : : : : +- * CometColumnarToRow (10) - : : : : +- CometProject (9) - : : : : +- CometFilter (8) - : : : : +- CometNativeScan parquet spark_catalog.default.store (7) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometFilter (15) - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) - : : +- BroadcastExchange (24) - : : +- * CometColumnarToRow (23) - : : +- CometFilter (22) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (21) - : +- BroadcastExchange (35) - : +- * CometColumnarToRow (34) - : +- CometProject (33) - : +- CometFilter (32) - : +- CometNativeScan parquet spark_catalog.default.customer (31) - +- ReusedExchange (38) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] - -(3) Filter [codegen id : 5] -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 46] -Output [1]: [d_date_sk#11] - -(5) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 5] -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Input [10]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9, d_date_sk#11] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_city#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [s_store_sk#12, s_city#13] -Condition : (s_city#13 IN (Midway,Fairview) AND isnotnull(s_store_sk#12)) - -(9) CometProject -Input [2]: [s_store_sk#12, s_city#13] -Arguments: [s_store_sk#12], [s_store_sk#12] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#12] - -(11) BroadcastExchange -Input [1]: [s_store_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#12] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, s_store_sk#12] - -(14) CometNativeScan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Condition : (((hd_dep_count#15 = 4) OR (hd_vehicle_count#16 = 3)) AND isnotnull(hd_demo_sk#14)) - -(16) CometProject -Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Arguments: [hd_demo_sk#14], [hd_demo_sk#14] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#14] - -(18) BroadcastExchange -Input [1]: [hd_demo_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#14] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, hd_demo_sk#14] - -(21) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#17, ca_city#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [ca_address_sk#17, ca_city#18] -Condition : (isnotnull(ca_address_sk#17) AND isnotnull(ca_city#18)) - -(23) CometColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#17, ca_city#18] - -(24) BroadcastExchange -Input [2]: [ca_address_sk#17, ca_city#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#17] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 5] -Output [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#18] -Input [8]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_address_sk#17, ca_city#18] - -(27) HashAggregate [codegen id : 5] -Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#18] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#6)), partial_sum(UnscaledValue(ss_ext_list_price#7)), partial_sum(UnscaledValue(ss_ext_tax#8))] -Aggregate Attributes [3]: [sum#19, sum#20, sum#21] -Results [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] - -(28) CometColumnarExchange -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(29) CometColumnarToRow [codegen id : 8] -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] - -(30) HashAggregate [codegen id : 8] -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#6)), sum(UnscaledValue(ss_ext_list_price#7)), sum(UnscaledValue(ss_ext_tax#8))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#6))#25, sum(UnscaledValue(ss_ext_list_price#7))#26, sum(UnscaledValue(ss_ext_tax#8))#27] -Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#18 AS bought_city#28, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#25,17,2) AS extended_price#29, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#7))#26,17,2) AS list_price#30, MakeDecimal(sum(UnscaledValue(ss_ext_tax#8))#27,17,2) AS extended_tax#31] - -(31) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(32) CometFilter -Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#33)) - -(33) CometProject -Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] -Arguments: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37], [c_customer_sk#32, c_current_addr_sk#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#34, 20, true, false, true) AS c_first_name#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#35, 30, true, false, true) AS c_last_name#37] - -(34) CometColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37] - -(35) BroadcastExchange -Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(36) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#32] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 8] -Output [8]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#36, c_last_name#37] -Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_customer_sk#32, c_current_addr_sk#33, c_first_name#36, c_last_name#37] - -(38) ReusedExchange [Reuses operator id: 24] -Output [2]: [ca_address_sk#38, ca_city#39] - -(39) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [c_current_addr_sk#33] -Right keys [1]: [ca_address_sk#38] -Join type: Inner -Join condition: NOT (ca_city#39 = bought_city#28) - -(40) Project [codegen id : 8] -Output [8]: [c_last_name#37, c_first_name#36, ca_city#39, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] -Input [10]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#36, c_last_name#37, ca_address_sk#38, ca_city#39] - -(41) TakeOrderedAndProject -Input [8]: [c_last_name#37, c_first_name#36, ca_city#39, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] -Arguments: 100, [c_last_name#37 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#39, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometNativeScan parquet spark_catalog.default.date_dim (42) - - -(42) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#40, d_dom#41] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [3]: [d_date_sk#11, d_year#40, d_dom#41] -Condition : ((((isnotnull(d_dom#41) AND (d_dom#41 >= 1)) AND (d_dom#41 <= 2)) AND d_year#40 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) - -(44) CometProject -Input [3]: [d_date_sk#11, d_year#40, d_dom#41] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(45) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(46) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/simplified.txt deleted file mode 100644 index 05b883b1e6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/simplified.txt +++ /dev/null @@ -1,67 +0,0 @@ -TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_city,extended_price,extended_tax,list_price] - WholeStageCodegen (8) - Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] - Project [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax)),bought_city,extended_price,list_price,extended_tax,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - WholeStageCodegen (5) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [sum,sum,sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_city] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - InputAdapter - ReusedExchange [ca_address_sk,ca_city] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/explain.txt deleted file mode 100644 index bc17101a84..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,256 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (39) -+- CometTakeOrderedAndProject (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (34) - : +- CometBroadcastHashJoin (33) - : :- CometHashAggregate (28) - : : +- CometExchange (27) - : : +- CometHashAggregate (26) - : : +- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (20) - : : : +- CometBroadcastHashJoin (19) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometProject (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : : : +- CometBroadcastExchange (18) - : : : +- CometProject (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - : : +- CometBroadcastExchange (23) - : : +- CometFilter (22) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (21) - : +- CometBroadcastExchange (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (29) - +- ReusedExchange (35) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_dom#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AND d_year#12 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) - -(5) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(7) CometBroadcastHashJoin -Left output [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(8) CometProject -Input [10]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9, d_date_sk#11] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_city#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#14, s_city#15] -Condition : (s_city#15 IN (Midway,Fairview) AND isnotnull(s_store_sk#14)) - -(11) CometProject -Input [2]: [s_store_sk#14, s_city#15] -Arguments: [s_store_sk#14], [s_store_sk#14] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#14] -Arguments: [s_store_sk#14] - -(13) CometBroadcastHashJoin -Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Right output [1]: [s_store_sk#14] -Arguments: [ss_store_sk#4], [s_store_sk#14], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, s_store_sk#14] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Condition : (((hd_dep_count#17 = 4) OR (hd_vehicle_count#18 = 3)) AND isnotnull(hd_demo_sk#16)) - -(17) CometProject -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Arguments: [hd_demo_sk#16], [hd_demo_sk#16] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#16] -Arguments: [hd_demo_sk#16] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Right output [1]: [hd_demo_sk#16] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, hd_demo_sk#16] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_city#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [ca_address_sk#19, ca_city#20] -Condition : (isnotnull(ca_address_sk#19) AND isnotnull(ca_city#20)) - -(23) CometBroadcastExchange -Input [2]: [ca_address_sk#19, ca_city#20] -Arguments: [ca_address_sk#19, ca_city#20] - -(24) CometBroadcastHashJoin -Left output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Right output [2]: [ca_address_sk#19, ca_city#20] -Arguments: [ss_addr_sk#3], [ca_address_sk#19], Inner, BuildRight - -(25) CometProject -Input [8]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_address_sk#19, ca_city#20] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20] - -(26) CometHashAggregate -Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#6)), partial_sum(UnscaledValue(ss_ext_list_price#7)), partial_sum(UnscaledValue(ss_ext_tax#8))] - -(27) CometExchange -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#21, sum#22, sum#23] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#21, sum#22, sum#23] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#6)), sum(UnscaledValue(ss_ext_list_price#7)), sum(UnscaledValue(ss_ext_tax#8))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] -Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_current_addr_sk#25)) - -(31) CometProject -Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29], [c_customer_sk#24, c_current_addr_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#26, 20, true, false, true) AS c_first_name#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#27, 30, true, false, true) AS c_last_name#29] - -(32) CometBroadcastExchange -Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Arguments: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] - -(33) CometBroadcastHashJoin -Left output [6]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#30, extended_price#31, list_price#32, extended_tax#33] -Right output [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Arguments: [ss_customer_sk#1], [c_customer_sk#24], Inner, BuildRight - -(34) CometProject -Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Arguments: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29], [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29] - -(35) ReusedExchange [Reuses operator id: 23] -Output [2]: [ca_address_sk#34, ca_city#35] - -(36) CometBroadcastHashJoin -Left output [8]: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Right output [2]: [ca_address_sk#34, ca_city#35] -Arguments: [c_current_addr_sk#25], [ca_address_sk#34], Inner, NOT (ca_city#35 = bought_city#30), BuildRight - -(37) CometProject -Input [10]: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29, ca_address_sk#34, ca_city#35] -Arguments: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32], [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] - -(38) CometTakeOrderedAndProject -Input [8]: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#29 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#29,c_first_name#28,ca_city#35,bought_city#30,ss_ticket_number#5,extended_price#31,extended_tax#33,list_price#32]), [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32], 100, 0, [c_last_name#29 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] - -(39) CometColumnarToRow [codegen id : 1] -Input [8]: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) - - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_dom#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(41) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AND d_year#12 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) - -(42) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(43) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(44) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/simplified.txt deleted file mode 100644 index fdddd82de0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] - CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] - CometBroadcastHashJoin [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] - CometProject [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometHashAggregate [ss_addr_sk,sum,sum,sum] [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,ca_city,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax))] - CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - CometHashAggregate [ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] - 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_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_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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [ca_address_sk,ca_city] #6 - CometFilter [ca_address_sk,ca_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - ReusedExchange [ca_address_sk,ca_city] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt deleted file mode 100644 index bc17101a84..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt +++ /dev/null @@ -1,256 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (39) -+- CometTakeOrderedAndProject (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (34) - : +- CometBroadcastHashJoin (33) - : :- CometHashAggregate (28) - : : +- CometExchange (27) - : : +- CometHashAggregate (26) - : : +- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (20) - : : : +- CometBroadcastHashJoin (19) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometProject (5) - : : : : : +- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : : : +- CometBroadcastExchange (18) - : : : +- CometProject (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - : : +- CometBroadcastExchange (23) - : : +- CometFilter (22) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (21) - : +- CometBroadcastExchange (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (29) - +- ReusedExchange (35) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Condition : (((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_dom#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AND d_year#12 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) - -(5) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(7) CometBroadcastHashJoin -Left output [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(8) CometProject -Input [10]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9, d_date_sk#11] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_city#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#14, s_city#15] -Condition : (s_city#15 IN (Midway,Fairview) AND isnotnull(s_store_sk#14)) - -(11) CometProject -Input [2]: [s_store_sk#14, s_city#15] -Arguments: [s_store_sk#14], [s_store_sk#14] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#14] -Arguments: [s_store_sk#14] - -(13) CometBroadcastHashJoin -Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Right output [1]: [s_store_sk#14] -Arguments: [ss_store_sk#4], [s_store_sk#14], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, s_store_sk#14] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Condition : (((hd_dep_count#17 = 4) OR (hd_vehicle_count#18 = 3)) AND isnotnull(hd_demo_sk#16)) - -(17) CometProject -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Arguments: [hd_demo_sk#16], [hd_demo_sk#16] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#16] -Arguments: [hd_demo_sk#16] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Right output [1]: [hd_demo_sk#16] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, hd_demo_sk#16] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_city#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [ca_address_sk#19, ca_city#20] -Condition : (isnotnull(ca_address_sk#19) AND isnotnull(ca_city#20)) - -(23) CometBroadcastExchange -Input [2]: [ca_address_sk#19, ca_city#20] -Arguments: [ca_address_sk#19, ca_city#20] - -(24) CometBroadcastHashJoin -Left output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] -Right output [2]: [ca_address_sk#19, ca_city#20] -Arguments: [ss_addr_sk#3], [ca_address_sk#19], Inner, BuildRight - -(25) CometProject -Input [8]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_address_sk#19, ca_city#20] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20] - -(26) CometHashAggregate -Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#20] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#6)), partial_sum(UnscaledValue(ss_ext_list_price#7)), partial_sum(UnscaledValue(ss_ext_tax#8))] - -(27) CometExchange -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#21, sum#22, sum#23] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(28) CometHashAggregate -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#21, sum#22, sum#23] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#6)), sum(UnscaledValue(ss_ext_list_price#7)), sum(UnscaledValue(ss_ext_tax#8))] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] -Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_current_addr_sk#25)) - -(31) CometProject -Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29], [c_customer_sk#24, c_current_addr_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#26, 20, true, false, true) AS c_first_name#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#27, 30, true, false, true) AS c_last_name#29] - -(32) CometBroadcastExchange -Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Arguments: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] - -(33) CometBroadcastHashJoin -Left output [6]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#30, extended_price#31, list_price#32, extended_tax#33] -Right output [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Arguments: [ss_customer_sk#1], [c_customer_sk#24], Inner, BuildRight - -(34) CometProject -Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Arguments: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29], [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29] - -(35) ReusedExchange [Reuses operator id: 23] -Output [2]: [ca_address_sk#34, ca_city#35] - -(36) CometBroadcastHashJoin -Left output [8]: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -Right output [2]: [ca_address_sk#34, ca_city#35] -Arguments: [c_current_addr_sk#25], [ca_address_sk#34], Inner, NOT (ca_city#35 = bought_city#30), BuildRight - -(37) CometProject -Input [10]: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29, ca_address_sk#34, ca_city#35] -Arguments: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32], [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] - -(38) CometTakeOrderedAndProject -Input [8]: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#29 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#29,c_first_name#28,ca_city#35,bought_city#30,ss_ticket_number#5,extended_price#31,extended_tax#33,list_price#32]), [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32], 100, 0, [c_last_name#29 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] - -(39) CometColumnarToRow [codegen id : 1] -Input [8]: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) - - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_dom#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(41) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AND d_year#12 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) - -(42) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_dom#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(43) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(44) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/extended.txt deleted file mode 100644 index c842000e11..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/extended.txt +++ /dev/null @@ -1,49 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index fdddd82de0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] - CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] - CometBroadcastHashJoin [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] - CometProject [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometHashAggregate [ss_addr_sk,sum,sum,sum] [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,ca_city,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax))] - CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 - CometHashAggregate [ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] - 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_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_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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [ca_address_sk,ca_city] #6 - CometFilter [ca_address_sk,ca_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] #7 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - ReusedExchange [ca_address_sk,ca_city] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/explain.txt deleted file mode 100644 index 3641cb51df..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/explain.txt +++ /dev/null @@ -1,290 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (44) -+- * HashAggregate (43) - +- * CometColumnarToRow (42) - +- CometColumnarExchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (32) - : +- * BroadcastHashJoin Inner BuildRight (31) - : :- * Project (25) - : : +- * BroadcastHashJoin LeftAnti BuildRight (24) - : : :- * BroadcastHashJoin LeftAnti BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (30) - : +- * CometColumnarToRow (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometNativeScan parquet spark_catalog.default.customer_address (26) - +- BroadcastExchange (37) - +- * CometColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometNativeScan parquet spark_catalog.default.customer_demographics (33) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(4) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] - -(6) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#7] - -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#4] -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] - -(9) BroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#4] -Join type: LeftSemi -Join condition: None - -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#6)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] - -(13) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#7] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#8] -Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#7] - -(16) BroadcastExchange -Input [1]: [ws_bill_customer_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(17) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#8] -Join type: LeftAnti -Join condition: None - -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#10, cs_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#11), dynamicpruningexpression(cs_sold_date_sk#11 IN dynamicpruning#6)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#10, cs_sold_date_sk#11] - -(20) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#7] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#11] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#10] -Input [3]: [cs_ship_customer_sk#10, cs_sold_date_sk#11, d_date_sk#7] - -(23) BroadcastExchange -Input [1]: [cs_ship_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_ship_customer_sk#10] -Join type: LeftAnti -Join condition: None - -(25) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(26) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#12, ca_state#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [ca_address_sk#12, ca_state#13] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#13, 2, true, false, true) IN (KY,GA,NM) AND isnotnull(ca_address_sk#12)) - -(28) CometProject -Input [2]: [ca_address_sk#12, ca_state#13] -Arguments: [ca_address_sk#12], [ca_address_sk#12] - -(29) CometColumnarToRow [codegen id : 7] -Input [1]: [ca_address_sk#12] - -(30) BroadcastExchange -Input [1]: [ca_address_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#12] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 9] -Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#12] - -(33) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#14, cd_gender#15, cd_marital_status#16, cd_education_status#17, cd_purchase_estimate#18, cd_credit_rating#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(34) CometFilter -Input [6]: [cd_demo_sk#14, cd_gender#15, cd_marital_status#16, cd_education_status#17, cd_purchase_estimate#18, cd_credit_rating#19] -Condition : isnotnull(cd_demo_sk#14) - -(35) CometProject -Input [6]: [cd_demo_sk#14, cd_gender#15, cd_marital_status#16, cd_education_status#17, cd_purchase_estimate#18, cd_credit_rating#19] -Arguments: [cd_demo_sk#14, cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23], [cd_demo_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#15, 1, true, false, true) AS cd_gender#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#16, 1, true, false, true) AS cd_marital_status#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#17, 20, true, false, true) AS cd_education_status#22, cd_purchase_estimate#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#19, 10, true, false, true) AS cd_credit_rating#23] - -(36) CometColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#14, cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23] - -(37) BroadcastExchange -Input [6]: [cd_demo_sk#14, cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(38) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#14] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 9] -Output [5]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23] -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#14, cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23] - -(40) HashAggregate [codegen id : 9] -Input [5]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23] -Keys [5]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#24] -Results [6]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23, count#25] - -(41) CometColumnarExchange -Input [6]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23, count#25] -Arguments: hashpartitioning(cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(42) CometColumnarToRow [codegen id : 10] -Input [6]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23, count#25] - -(43) HashAggregate [codegen id : 10] -Input [6]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23, count#25] -Keys [5]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cd_purchase_estimate#18, cd_credit_rating#23] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#26] -Results [8]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, count(1)#26 AS cnt1#27, cd_purchase_estimate#18, count(1)#26 AS cnt2#28, cd_credit_rating#23, count(1)#26 AS cnt3#29] - -(44) TakeOrderedAndProject -Input [8]: [cd_gender#20, cd_marital_status#21, cd_education_status#22, cnt1#27, cd_purchase_estimate#18, cnt2#28, cd_credit_rating#23, cnt3#29] -Arguments: 100, [cd_gender#20 ASC NULLS FIRST, cd_marital_status#21 ASC NULLS FIRST, cd_education_status#22 ASC NULLS FIRST, cd_purchase_estimate#18 ASC NULLS FIRST, cd_credit_rating#23 ASC NULLS FIRST], [cd_gender#20, cd_marital_status#21, cd_education_status#22, cnt1#27, cd_purchase_estimate#18, cnt2#28, cd_credit_rating#23, cnt3#29] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometNativeScan parquet spark_catalog.default.date_dim (45) - - -(45) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#30, d_moy#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [3]: [d_date_sk#7, d_year#30, d_moy#31] -Condition : (((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2001)) AND (d_moy#31 >= 4)) AND (d_moy#31 <= 6)) AND isnotnull(d_date_sk#7)) - -(47) CometProject -Input [3]: [d_date_sk#7, d_year#30, d_moy#31] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(49) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/simplified.txt deleted file mode 100644 index f8868e38be..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/simplified.txt +++ /dev/null @@ -1,76 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (10) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (9) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometNativeScan 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/q69.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/explain.txt deleted file mode 100644 index 2986d9c0bf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,301 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (46) -+- * HashAggregate (45) - +- * CometColumnarToRow (44) - +- CometColumnarExchange (43) - +- * HashAggregate (42) - +- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (34) - : +- * BroadcastHashJoin Inner BuildRight (33) - : :- * Project (27) - : : +- * BroadcastHashJoin LeftAnti BuildRight (26) - : : :- * BroadcastHashJoin LeftAnti BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (32) - : +- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (28) - +- BroadcastExchange (39) - +- * CometColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (35) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2001)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 6)) AND isnotnull(d_date_sk#7)) - -(6) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#4], [ss_customer_sk#4] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: [ss_customer_sk#4] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ss_customer_sk#4] -Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#7] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Right output [1]: [d_date_sk#7] -Arguments: [ws_sold_date_sk#11], [d_date_sk#7], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#7] -Arguments: [ws_bill_customer_sk#10], [ws_bill_customer_sk#10] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#10] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#10] -Join type: LeftAnti -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#15)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#7] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -Right output [1]: [d_date_sk#7] -Arguments: [cs_sold_date_sk#14], [d_date_sk#7], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#7] -Arguments: [cs_ship_customer_sk#13], [cs_ship_customer_sk#13] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#13] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_ship_customer_sk#13] -Join type: LeftAnti -Join condition: None - -(27) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(29) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#17, 2, true, false, true) IN (KY,GA,NM) AND isnotnull(ca_address_sk#16)) - -(30) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(31) CometColumnarToRow [codegen id : 3] -Input [1]: [ca_address_sk#16] - -(32) BroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(33) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#16] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 5] -Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#16] - -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(36) CometFilter -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] -Condition : isnotnull(cd_demo_sk#18) - -(37) CometProject -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] -Arguments: [cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27], [cd_demo_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#19, 1, true, false, true) AS cd_gender#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) AS cd_marital_status#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) AS cd_education_status#26, cd_purchase_estimate#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#23, 10, true, false, true) AS cd_credit_rating#27] - -(38) CometColumnarToRow [codegen id : 4] -Input [6]: [cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] - -(39) BroadcastExchange -Input [6]: [cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(40) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 5] -Output [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] - -(42) HashAggregate [codegen id : 5] -Input [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] -Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#28] -Results [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27, count#29] - -(43) CometColumnarExchange -Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27, count#29] -Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(44) CometColumnarToRow [codegen id : 6] -Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27, count#29] - -(45) HashAggregate [codegen id : 6] -Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27, count#29] -Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#30] -Results [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#30 AS cnt1#31, cd_purchase_estimate#22, count(1)#30 AS cnt2#32, cd_credit_rating#27, count(1)#30 AS cnt3#33] - -(46) TakeOrderedAndProject -Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#31, cd_purchase_estimate#22, cnt2#32, cd_credit_rating#27, cnt3#33] -Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#22 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#31, cd_purchase_estimate#22, cnt2#32, cd_credit_rating#27, cnt3#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (51) -+- * CometColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) - - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] -ReadSchema: struct - -(48) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2001)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 6)) AND isnotnull(d_date_sk#7)) - -(49) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(50) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(51) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/simplified.txt deleted file mode 100644 index 68bf32c40b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,72 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (6) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (5) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometScan [native_iceberg_compat] 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/q69/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt deleted file mode 100644 index 2986d9c0bf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt +++ /dev/null @@ -1,301 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (46) -+- * HashAggregate (45) - +- * CometColumnarToRow (44) - +- CometColumnarExchange (43) - +- * HashAggregate (42) - +- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (34) - : +- * BroadcastHashJoin Inner BuildRight (33) - : :- * Project (27) - : : +- * BroadcastHashJoin LeftAnti BuildRight (26) - : : :- * BroadcastHashJoin LeftAnti BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (32) - : +- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometFilter (29) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (28) - +- BroadcastExchange (39) - +- * CometColumnarToRow (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (35) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2001)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 6)) AND isnotnull(d_date_sk#7)) - -(6) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#4], [ss_customer_sk#4] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: [ss_customer_sk#4] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ss_customer_sk#4] -Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#7] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Right output [1]: [d_date_sk#7] -Arguments: [ws_sold_date_sk#11], [d_date_sk#7], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#7] -Arguments: [ws_bill_customer_sk#10], [ws_bill_customer_sk#10] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#10] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#10] -Join type: LeftAnti -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#15)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#7] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -Right output [1]: [d_date_sk#7] -Arguments: [cs_sold_date_sk#14], [d_date_sk#7], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#7] -Arguments: [cs_ship_customer_sk#13], [cs_ship_customer_sk#13] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#13] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_ship_customer_sk#13] -Join type: LeftAnti -Join condition: None - -(27) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(29) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#17, 2, true, false, true) IN (KY,GA,NM) AND isnotnull(ca_address_sk#16)) - -(30) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(31) CometColumnarToRow [codegen id : 3] -Input [1]: [ca_address_sk#16] - -(32) BroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(33) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#16] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 5] -Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#16] - -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(36) CometFilter -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] -Condition : isnotnull(cd_demo_sk#18) - -(37) CometProject -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] -Arguments: [cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27], [cd_demo_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#19, 1, true, false, true) AS cd_gender#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) AS cd_marital_status#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) AS cd_education_status#26, cd_purchase_estimate#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#23, 10, true, false, true) AS cd_credit_rating#27] - -(38) CometColumnarToRow [codegen id : 4] -Input [6]: [cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] - -(39) BroadcastExchange -Input [6]: [cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(40) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 5] -Output [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] - -(42) HashAggregate [codegen id : 5] -Input [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] -Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#28] -Results [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27, count#29] - -(43) CometColumnarExchange -Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27, count#29] -Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(44) CometColumnarToRow [codegen id : 6] -Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27, count#29] - -(45) HashAggregate [codegen id : 6] -Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27, count#29] -Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#22, cd_credit_rating#27] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#30] -Results [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#30 AS cnt1#31, cd_purchase_estimate#22, count(1)#30 AS cnt2#32, cd_credit_rating#27, count(1)#30 AS cnt3#33] - -(46) TakeOrderedAndProject -Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#31, cd_purchase_estimate#22, cnt2#32, cd_credit_rating#27, cnt3#33] -Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#22 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#31, cd_purchase_estimate#22, cnt2#32, cd_credit_rating#27, cnt3#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (51) -+- * CometColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) - - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] -ReadSchema: struct - -(48) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2001)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 6)) AND isnotnull(d_date_sk#7)) - -(49) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(50) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(51) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/extended.txt deleted file mode 100644 index b4fbb455ef..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/extended.txt +++ /dev/null @@ -1,62 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 68bf32c40b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/simplified.txt +++ /dev/null @@ -1,72 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] - WholeStageCodegen (6) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 - WholeStageCodegen (5) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/explain.txt deleted file mode 100644 index 1a5a7efc91..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (32) -+- * HashAggregate (31) - +- * CometColumnarToRow (30) - +- CometColumnarExchange (29) - +- * HashAggregate (28) - +- * Project (27) - +- * BroadcastHashJoin Inner BuildRight (26) - :- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.item (14) - +- BroadcastExchange (25) - +- * CometColumnarToRow (24) - +- CometProject (23) - +- CometFilter (22) - +- CometNativeScan parquet spark_catalog.default.promotion (21) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(3) Filter [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(4) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = S)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#10)) - -(6) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [cd_demo_sk#10] - -(8) BroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 5] -Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] - -(11) ReusedExchange [Reuses operator id: 37] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] - -(14) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#15, i_item_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [i_item_sk#15, i_item_id#16] -Condition : isnotnull(i_item_sk#15) - -(16) CometProject -Input [2]: [i_item_sk#15, i_item_id#16] -Arguments: [i_item_sk#15, i_item_id#17], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#16, 16, true, false, true) AS i_item_id#17] - -(17) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#15, i_item_id#17] - -(18) BroadcastExchange -Input [2]: [i_item_sk#15, i_item_id#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [6]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#15, i_item_id#17] - -(21) CometNativeScan parquet spark_catalog.default.promotion -Output [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_email#19, 1, true, false, true) = N) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_event#20, 1, true, false, true) = N)) AND isnotnull(p_promo_sk#18)) - -(23) CometProject -Input [3]: [p_promo_sk#18, p_channel_email#19, p_channel_event#20] -Arguments: [p_promo_sk#18], [p_promo_sk#18] - -(24) CometColumnarToRow [codegen id : 4] -Input [1]: [p_promo_sk#18] - -(25) BroadcastExchange -Input [1]: [p_promo_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_promo_sk#3] -Right keys [1]: [p_promo_sk#18] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 5] -Output [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] -Input [7]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17, p_promo_sk#18] - -(28) HashAggregate [codegen id : 5] -Input [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#17] -Keys [1]: [i_item_id#17] -Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [8]: [sum#21, count#22, sum#23, count#24, sum#25, count#26, sum#27, count#28] -Results [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] - -(29) CometColumnarExchange -Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] -Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 6] -Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] - -(31) HashAggregate [codegen id : 6] -Input [9]: [i_item_id#17, sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35, count#36] -Keys [1]: [i_item_id#17] -Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] -Aggregate Attributes [4]: [avg(ss_quantity#4)#37, avg(UnscaledValue(ss_list_price#5))#38, avg(UnscaledValue(ss_coupon_amt#7))#39, avg(UnscaledValue(ss_sales_price#6))#40] -Results [5]: [i_item_id#17, avg(ss_quantity#4)#37 AS agg1#41, cast((avg(UnscaledValue(ss_list_price#5))#38 / 100.0) as decimal(11,6)) AS agg2#42, cast((avg(UnscaledValue(ss_coupon_amt#7))#39 / 100.0) as decimal(11,6)) AS agg3#43, cast((avg(UnscaledValue(ss_sales_price#6))#40 / 100.0) as decimal(11,6)) AS agg4#44] - -(32) TakeOrderedAndProject -Input [5]: [i_item_id#17, agg1#41, agg2#42, agg3#43, agg4#44] -Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#41, agg2#42, agg3#43, agg4#44] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometNativeScan parquet spark_catalog.default.date_dim (33) - - -(33) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [d_date_sk#14, d_year#45] -Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2000)) AND isnotnull(d_date_sk#14)) - -(35) CometProject -Input [2]: [d_date_sk#14, d_year#45] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(37) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/simplified.txt deleted file mode 100644 index bb670b4a73..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - WholeStageCodegen (6) - HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - BroadcastHashJoin [ss_promo_sk,p_promo_sk] - Project [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Filter [ss_cdemo_sk,ss_item_sk,ss_promo_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometNativeScan 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/q7.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/explain.txt deleted file mode 100644 index 0e32283618..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,212 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (31) -+- CometTakeOrderedAndProject (30) - +- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = S)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#16, i_item_id#17] -Condition : isnotnull(i_item_sk#16) - -(17) CometProject -Input [2]: [i_item_sk#16, i_item_id#17] -Arguments: [i_item_sk#16, i_item_id#18], [i_item_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#17, 16, true, false, true) AS i_item_id#18] - -(18) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_id#18] -Arguments: [i_item_sk#16, i_item_id#18] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Right output [2]: [i_item_sk#16, i_item_id#18] -Arguments: [ss_item_sk#1], [i_item_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#16, i_item_id#18] -Arguments: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18], [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_email#20, 1, true, false, true) = N) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_event#21, 1, true, false, true) = N)) AND isnotnull(p_promo_sk#19)) - -(23) CometProject -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Arguments: [p_promo_sk#19], [p_promo_sk#19] - -(24) CometBroadcastExchange -Input [1]: [p_promo_sk#19] -Arguments: [p_promo_sk#19] - -(25) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] -Right output [1]: [p_promo_sk#19] -Arguments: [ss_promo_sk#3], [p_promo_sk#19], Inner, BuildRight - -(26) CometProject -Input [7]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18, p_promo_sk#19] -Arguments: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] - -(27) CometHashAggregate -Input [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] - -(28) CometExchange -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Keys [1]: [i_item_id#18] -Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] - -(30) CometTakeOrderedAndProject -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#18 ASC NULLS FIRST], output=[i_item_id#18,agg1#30,agg2#31,agg3#32,agg4#33]), [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33], 100, 0, [i_item_id#18 ASC NULLS FIRST], [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(34) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(36) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/simplified.txt deleted file mode 100644 index a15bcd5c0d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,agg1,agg2,agg3,agg4,avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price))] - CometExchange [i_item_id] #1 - CometHashAggregate [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,sum,count,sum,count,sum,count,sum,count] - CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - 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,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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [p_promo_sk] #6 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometScan [native_iceberg_compat] 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/q7/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt deleted file mode 100644 index 0e32283618..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt +++ /dev/null @@ -1,212 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (31) -+- CometTakeOrderedAndProject (30) - +- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = S)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [i_item_sk#16, i_item_id#17] -Condition : isnotnull(i_item_sk#16) - -(17) CometProject -Input [2]: [i_item_sk#16, i_item_id#17] -Arguments: [i_item_sk#16, i_item_id#18], [i_item_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#17, 16, true, false, true) AS i_item_id#18] - -(18) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_id#18] -Arguments: [i_item_sk#16, i_item_id#18] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Right output [2]: [i_item_sk#16, i_item_id#18] -Arguments: [ss_item_sk#1], [i_item_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#16, i_item_id#18] -Arguments: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18], [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_email#20, 1, true, false, true) = N) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_event#21, 1, true, false, true) = N)) AND isnotnull(p_promo_sk#19)) - -(23) CometProject -Input [3]: [p_promo_sk#19, p_channel_email#20, p_channel_event#21] -Arguments: [p_promo_sk#19], [p_promo_sk#19] - -(24) CometBroadcastExchange -Input [1]: [p_promo_sk#19] -Arguments: [p_promo_sk#19] - -(25) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] -Right output [1]: [p_promo_sk#19] -Arguments: [ss_promo_sk#3], [p_promo_sk#19], Inner, BuildRight - -(26) CometProject -Input [7]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18, p_promo_sk#19] -Arguments: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] - -(27) CometHashAggregate -Input [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18] -Keys [1]: [i_item_id#18] -Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] - -(28) CometExchange -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [9]: [i_item_id#18, sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] -Keys [1]: [i_item_id#18] -Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] - -(30) CometTakeOrderedAndProject -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#18 ASC NULLS FIRST], output=[i_item_id#18,agg1#30,agg2#31,agg3#32,agg4#33]), [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33], 100, 0, [i_item_id#18 ASC NULLS FIRST], [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [i_item_id#18, agg1#30, agg2#31, agg3#32, agg4#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2000)) AND isnotnull(d_date_sk#14)) - -(34) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(36) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/extended.txt deleted file mode 100644 index 655f651a41..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/extended.txt +++ /dev/null @@ -1,39 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index a15bcd5c0d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,agg1,agg2,agg3,agg4,avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price))] - CometExchange [i_item_id] #1 - CometHashAggregate [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,sum,count,sum,count,sum,count,sum,count] - CometProject [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] - 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,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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [p_promo_sk] #6 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_email,p_channel_event] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/explain.txt deleted file mode 100644 index f185af2c42..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/explain.txt +++ /dev/null @@ -1,300 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (47) -+- * Project (46) - +- Window (45) - +- * CometColumnarToRow (44) - +- CometSort (43) - +- CometColumnarExchange (42) - +- * HashAggregate (41) - +- * CometColumnarToRow (40) - +- CometColumnarExchange (39) - +- * HashAggregate (38) - +- * Expand (37) - +- * Project (36) - +- * BroadcastHashJoin Inner BuildRight (35) - :- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.store_sales (1) - : +- ReusedExchange (4) - +- BroadcastExchange (34) - +- * Project (33) - +- * BroadcastHashJoin LeftSemi BuildRight (32) - :- * CometColumnarToRow (9) - : +- CometFilter (8) - : +- CometNativeScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (31) - +- * Project (30) - +- * Filter (29) - +- Window (28) - +- * Sort (27) - +- * HashAggregate (26) - +- * CometColumnarToRow (25) - +- CometColumnarExchange (24) - +- * HashAggregate (23) - +- * Project (22) - +- * BroadcastHashJoin Inner BuildRight (21) - :- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet spark_catalog.default.store_sales (10) - : +- BroadcastExchange (17) - : +- * CometColumnarToRow (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometNativeScan parquet spark_catalog.default.store (13) - +- ReusedExchange (20) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 8] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] - -(3) Filter [codegen id : 8] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(4) ReusedExchange [Reuses operator id: 52] -Output [1]: [d_date_sk#5] - -(5) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 8] -Output [2]: [ss_store_sk#1, ss_net_profit#2] -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#6, s_county#7, s_state#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Condition : isnotnull(s_store_sk#6) - -(9) CometColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#6, s_county#7, s_state#8] - -(10) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] - -(12) Filter [codegen id : 4] -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_store_sk#9) - -(13) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#6, s_state#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [s_store_sk#6, s_state#8] -Condition : isnotnull(s_store_sk#6) - -(15) CometProject -Input [2]: [s_store_sk#6, s_state#8] -Arguments: [s_store_sk#6, s_state#12], [s_store_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true) AS s_state#12] - -(16) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#6, s_state#12] - -(17) BroadcastExchange -Input [2]: [s_store_sk#6, s_state#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#9] -Right keys [1]: [s_store_sk#6] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#12] -Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#6, s_state#12] - -(20) ReusedExchange [Reuses operator id: 52] -Output [1]: [d_date_sk#5] - -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 4] -Output [2]: [ss_net_profit#10, s_state#12] -Input [4]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#12, d_date_sk#5] - -(23) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#10, s_state#12] -Keys [1]: [s_state#12] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum#13] -Results [2]: [s_state#12, sum#14] - -(24) CometColumnarExchange -Input [2]: [s_state#12, sum#14] -Arguments: hashpartitioning(s_state#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(25) CometColumnarToRow [codegen id : 5] -Input [2]: [s_state#12, sum#14] - -(26) HashAggregate [codegen id : 5] -Input [2]: [s_state#12, sum#14] -Keys [1]: [s_state#12] -Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#15] -Results [3]: [s_state#12, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#15,17,2) AS _w0#16, s_state#12] - -(27) Sort [codegen id : 5] -Input [3]: [s_state#12, _w0#16, s_state#12] -Arguments: [s_state#12 ASC NULLS FIRST, _w0#16 DESC NULLS LAST], false, 0 - -(28) Window -Input [3]: [s_state#12, _w0#16, s_state#12] -Arguments: [rank(_w0#16) windowspecdefinition(s_state#12, _w0#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#17], [s_state#12], [_w0#16 DESC NULLS LAST] - -(29) Filter [codegen id : 6] -Input [4]: [s_state#12, _w0#16, s_state#12, ranking#17] -Condition : (ranking#17 <= 5) - -(30) Project [codegen id : 6] -Output [1]: [s_state#12] -Input [4]: [s_state#12, _w0#16, s_state#12, ranking#17] - -(31) BroadcastExchange -Input [1]: [s_state#12] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true)] -Right keys [1]: [s_state#12] -Join type: LeftSemi -Join condition: None - -(33) Project [codegen id : 7] -Output [3]: [s_store_sk#6, s_county#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true) AS s_state#18] -Input [3]: [s_store_sk#6, s_county#7, s_state#8] - -(34) BroadcastExchange -Input [3]: [s_store_sk#6, s_county#7, s_state#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#6] -Join type: Inner -Join condition: None - -(36) Project [codegen id : 8] -Output [3]: [ss_net_profit#2, s_state#18, s_county#7] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#18] - -(37) Expand [codegen id : 8] -Input [3]: [ss_net_profit#2, s_state#18, s_county#7] -Arguments: [[ss_net_profit#2, s_state#18, s_county#7, 0], [ss_net_profit#2, s_state#18, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#19, s_county#20, spark_grouping_id#21] - -(38) HashAggregate [codegen id : 8] -Input [4]: [ss_net_profit#2, s_state#19, s_county#20, spark_grouping_id#21] -Keys [3]: [s_state#19, s_county#20, spark_grouping_id#21] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] - -(39) CometColumnarExchange -Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] -Arguments: hashpartitioning(s_state#19, s_county#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(40) CometColumnarToRow [codegen id : 9] -Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] - -(41) HashAggregate [codegen id : 9] -Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] -Keys [3]: [s_state#19, s_county#20, spark_grouping_id#21] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) AS total_sum#25, s_state#19, s_county#20, (cast((shiftright(spark_grouping_id#21, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint)) AS lochierarchy#26, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) AS _w0#27, (cast((shiftright(spark_grouping_id#21, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint)) AS _w1#28, CASE WHEN (cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint) = 0) THEN s_state#19 END AS _w2#29] - -(42) CometColumnarExchange -Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] -Arguments: hashpartitioning(_w1#28, _w2#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) CometSort -Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] -Arguments: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29], [_w1#28 ASC NULLS FIRST, _w2#29 ASC NULLS FIRST, _w0#27 DESC NULLS LAST] - -(44) CometColumnarToRow [codegen id : 10] -Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] - -(45) Window -Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] -Arguments: [rank(_w0#27) windowspecdefinition(_w1#28, _w2#29, _w0#27 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#30], [_w1#28, _w2#29], [_w0#27 DESC NULLS LAST] - -(46) Project [codegen id : 11] -Output [5]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, rank_within_parent#30] -Input [8]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29, rank_within_parent#30] - -(47) TakeOrderedAndProject -Input [5]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, rank_within_parent#30] -Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN s_state#19 END ASC NULLS FIRST, rank_within_parent#30 ASC NULLS FIRST], [total_sum#25, s_state#19, s_county#20, lochierarchy#26, rank_within_parent#30] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometNativeScan parquet spark_catalog.default.date_dim (48) - - -(48) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#31] -Condition : (((isnotnull(d_month_seq#31) AND (d_month_seq#31 >= 1200)) AND (d_month_seq#31 <= 1211)) AND isnotnull(d_date_sk#5)) - -(50) CometProject -Input [2]: [d_date_sk#5, d_month_seq#31] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(52) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/simplified.txt deleted file mode 100644 index 62c46296ca..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/simplified.txt +++ /dev/null @@ -1,79 +0,0 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (11) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (9) - HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (8) - HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [ss_net_profit,s_state,s_county] - Project [ss_net_profit,s_state,s_county] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WholeStageCodegen (5) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #6 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/explain.txt deleted file mode 100644 index d3610af0f8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,304 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- CometColumnarExchange (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Expand (38) - +- * Project (37) - +- * BroadcastHashJoin Inner BuildRight (36) - :- * CometColumnarToRow (9) - : +- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- BroadcastExchange (35) - +- * Project (34) - +- * BroadcastHashJoin LeftSemi BuildRight (33) - :- * CometColumnarToRow (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) - +- BroadcastExchange (32) - +- * Project (31) - +- * Filter (30) - +- Window (29) - +- * CometColumnarToRow (28) - +- CometSort (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometBroadcastHashJoin (22) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - +- ReusedExchange (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] - -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ss_store_sk#1, ss_net_profit#2] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#7, s_county#8, s_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Condition : isnotnull(s_store_sk#7) - -(12) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_store_sk#10) - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#7, s_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#7, s_state#9] -Condition : isnotnull(s_store_sk#7) - -(17) CometProject -Input [2]: [s_store_sk#7, s_state#9] -Arguments: [s_store_sk#7, s_state#14], [s_store_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#14] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#7, s_state#14] -Arguments: [s_store_sk#7, s_state#14] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Right output [2]: [s_store_sk#7, s_state#14] -Arguments: [ss_store_sk#10], [s_store_sk#7], Inner, BuildRight - -(20) CometProject -Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#7, s_state#14] -Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#14], [ss_net_profit#11, ss_sold_date_sk#12, s_state#14] - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#5] - -(22) CometBroadcastHashJoin -Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#14] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#12], [d_date_sk#5], Inner, BuildRight - -(23) CometProject -Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#14, d_date_sk#5] -Arguments: [ss_net_profit#11, s_state#14], [ss_net_profit#11, s_state#14] - -(24) CometHashAggregate -Input [2]: [ss_net_profit#11, s_state#14] -Keys [1]: [s_state#14] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] - -(25) CometExchange -Input [2]: [s_state#14, sum#15] -Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(26) CometHashAggregate -Input [2]: [s_state#14, sum#15] -Keys [1]: [s_state#14] -Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] - -(27) CometSort -Input [3]: [s_state#14, _w0#16, s_state#14] -Arguments: [s_state#14, _w0#16, s_state#14], [s_state#14 ASC NULLS FIRST, _w0#16 DESC NULLS LAST] - -(28) CometColumnarToRow [codegen id : 1] -Input [3]: [s_state#14, _w0#16, s_state#14] - -(29) Window -Input [3]: [s_state#14, _w0#16, s_state#14] -Arguments: [rank(_w0#16) windowspecdefinition(s_state#14, _w0#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#17], [s_state#14], [_w0#16 DESC NULLS LAST] - -(30) Filter [codegen id : 2] -Input [4]: [s_state#14, _w0#16, s_state#14, ranking#17] -Condition : (ranking#17 <= 5) - -(31) Project [codegen id : 2] -Output [1]: [s_state#14] -Input [4]: [s_state#14, _w0#16, s_state#14, ranking#17] - -(32) BroadcastExchange -Input [1]: [s_state#14] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] - -(33) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)] -Right keys [1]: [s_state#14] -Join type: LeftSemi -Join condition: None - -(34) Project [codegen id : 3] -Output [3]: [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#18] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(35) BroadcastExchange -Input [3]: [s_store_sk#7, s_county#8, s_state#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(36) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_state#18, s_county#8] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#18] - -(38) Expand [codegen id : 4] -Input [3]: [ss_net_profit#2, s_state#18, s_county#8] -Arguments: [[ss_net_profit#2, s_state#18, s_county#8, 0], [ss_net_profit#2, s_state#18, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#19, s_county#20, spark_grouping_id#21] - -(39) HashAggregate [codegen id : 4] -Input [4]: [ss_net_profit#2, s_state#19, s_county#20, spark_grouping_id#21] -Keys [3]: [s_state#19, s_county#20, spark_grouping_id#21] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] - -(40) CometColumnarExchange -Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] -Arguments: hashpartitioning(s_state#19, s_county#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 5] -Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] - -(42) HashAggregate [codegen id : 5] -Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] -Keys [3]: [s_state#19, s_county#20, spark_grouping_id#21] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) AS total_sum#25, s_state#19, s_county#20, (cast((shiftright(spark_grouping_id#21, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint)) AS lochierarchy#26, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) AS _w0#27, (cast((shiftright(spark_grouping_id#21, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint)) AS _w1#28, CASE WHEN (cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint) = 0) THEN s_state#19 END AS _w2#29] - -(43) CometColumnarExchange -Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] -Arguments: hashpartitioning(_w1#28, _w2#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(44) CometSort -Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] -Arguments: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29], [_w1#28 ASC NULLS FIRST, _w2#29 ASC NULLS FIRST, _w0#27 DESC NULLS LAST] - -(45) CometColumnarToRow [codegen id : 6] -Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] - -(46) Window -Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] -Arguments: [rank(_w0#27) windowspecdefinition(_w1#28, _w2#29, _w0#27 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#30], [_w1#28, _w2#29], [_w0#27 DESC NULLS LAST] - -(47) Project [codegen id : 7] -Output [5]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, rank_within_parent#30] -Input [8]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29, rank_within_parent#30] - -(48) TakeOrderedAndProject -Input [5]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, rank_within_parent#30] -Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN s_state#19 END ASC NULLS FIRST, rank_within_parent#30 ASC NULLS FIRST], [total_sum#25, s_state#19, s_county#20, lochierarchy#26, rank_within_parent#30] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(51) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(53) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/simplified.txt deleted file mode 100644 index d6ba61a844..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (7) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (5) - HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [ss_net_profit,s_state,s_county] - Project [ss_net_profit,s_state,s_county] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [s_state,_w0] - CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] - CometExchange [s_state] #7 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - 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,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] 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/q70/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt deleted file mode 100644 index d3610af0f8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ /dev/null @@ -1,304 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- CometColumnarExchange (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Expand (38) - +- * Project (37) - +- * BroadcastHashJoin Inner BuildRight (36) - :- * CometColumnarToRow (9) - : +- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- BroadcastExchange (35) - +- * Project (34) - +- * BroadcastHashJoin LeftSemi BuildRight (33) - :- * CometColumnarToRow (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) - +- BroadcastExchange (32) - +- * Project (31) - +- * Filter (30) - +- Window (29) - +- * CometColumnarToRow (28) - +- CometSort (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometBroadcastHashJoin (22) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - +- ReusedExchange (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] - -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ss_store_sk#1, ss_net_profit#2] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#7, s_county#8, s_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Condition : isnotnull(s_store_sk#7) - -(12) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_store_sk#10) - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#7, s_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#7, s_state#9] -Condition : isnotnull(s_store_sk#7) - -(17) CometProject -Input [2]: [s_store_sk#7, s_state#9] -Arguments: [s_store_sk#7, s_state#14], [s_store_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#14] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#7, s_state#14] -Arguments: [s_store_sk#7, s_state#14] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Right output [2]: [s_store_sk#7, s_state#14] -Arguments: [ss_store_sk#10], [s_store_sk#7], Inner, BuildRight - -(20) CometProject -Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#7, s_state#14] -Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#14], [ss_net_profit#11, ss_sold_date_sk#12, s_state#14] - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#5] - -(22) CometBroadcastHashJoin -Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#14] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#12], [d_date_sk#5], Inner, BuildRight - -(23) CometProject -Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#14, d_date_sk#5] -Arguments: [ss_net_profit#11, s_state#14], [ss_net_profit#11, s_state#14] - -(24) CometHashAggregate -Input [2]: [ss_net_profit#11, s_state#14] -Keys [1]: [s_state#14] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] - -(25) CometExchange -Input [2]: [s_state#14, sum#15] -Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(26) CometHashAggregate -Input [2]: [s_state#14, sum#15] -Keys [1]: [s_state#14] -Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] - -(27) CometSort -Input [3]: [s_state#14, _w0#16, s_state#14] -Arguments: [s_state#14, _w0#16, s_state#14], [s_state#14 ASC NULLS FIRST, _w0#16 DESC NULLS LAST] - -(28) CometColumnarToRow [codegen id : 1] -Input [3]: [s_state#14, _w0#16, s_state#14] - -(29) Window -Input [3]: [s_state#14, _w0#16, s_state#14] -Arguments: [rank(_w0#16) windowspecdefinition(s_state#14, _w0#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#17], [s_state#14], [_w0#16 DESC NULLS LAST] - -(30) Filter [codegen id : 2] -Input [4]: [s_state#14, _w0#16, s_state#14, ranking#17] -Condition : (ranking#17 <= 5) - -(31) Project [codegen id : 2] -Output [1]: [s_state#14] -Input [4]: [s_state#14, _w0#16, s_state#14, ranking#17] - -(32) BroadcastExchange -Input [1]: [s_state#14] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] - -(33) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)] -Right keys [1]: [s_state#14] -Join type: LeftSemi -Join condition: None - -(34) Project [codegen id : 3] -Output [3]: [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#18] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(35) BroadcastExchange -Input [3]: [s_store_sk#7, s_county#8, s_state#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(36) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_state#18, s_county#8] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#18] - -(38) Expand [codegen id : 4] -Input [3]: [ss_net_profit#2, s_state#18, s_county#8] -Arguments: [[ss_net_profit#2, s_state#18, s_county#8, 0], [ss_net_profit#2, s_state#18, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#19, s_county#20, spark_grouping_id#21] - -(39) HashAggregate [codegen id : 4] -Input [4]: [ss_net_profit#2, s_state#19, s_county#20, spark_grouping_id#21] -Keys [3]: [s_state#19, s_county#20, spark_grouping_id#21] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] - -(40) CometColumnarExchange -Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] -Arguments: hashpartitioning(s_state#19, s_county#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 5] -Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] - -(42) HashAggregate [codegen id : 5] -Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] -Keys [3]: [s_state#19, s_county#20, spark_grouping_id#21] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) AS total_sum#25, s_state#19, s_county#20, (cast((shiftright(spark_grouping_id#21, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint)) AS lochierarchy#26, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) AS _w0#27, (cast((shiftright(spark_grouping_id#21, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint)) AS _w1#28, CASE WHEN (cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint) = 0) THEN s_state#19 END AS _w2#29] - -(43) CometColumnarExchange -Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] -Arguments: hashpartitioning(_w1#28, _w2#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(44) CometSort -Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] -Arguments: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29], [_w1#28 ASC NULLS FIRST, _w2#29 ASC NULLS FIRST, _w0#27 DESC NULLS LAST] - -(45) CometColumnarToRow [codegen id : 6] -Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] - -(46) Window -Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] -Arguments: [rank(_w0#27) windowspecdefinition(_w1#28, _w2#29, _w0#27 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#30], [_w1#28, _w2#29], [_w0#27 DESC NULLS LAST] - -(47) Project [codegen id : 7] -Output [5]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, rank_within_parent#30] -Input [8]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29, rank_within_parent#30] - -(48) TakeOrderedAndProject -Input [5]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, rank_within_parent#30] -Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN s_state#19 END ASC NULLS FIRST, rank_within_parent#30 ASC NULLS FIRST], [total_sum#25, s_state#19, s_county#20, lochierarchy#26, rank_within_parent#30] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(51) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(53) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/extended.txt deleted file mode 100644 index 6a918ac127..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/extended.txt +++ /dev/null @@ -1,60 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 34 out of 52 eligible operators (65%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d6ba61a844..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (7) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (5) - HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [ss_net_profit,s_state,s_county] - Project [ss_net_profit,s_state,s_county] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [s_state,_w0] - CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] - CometExchange [s_state] #7 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - 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,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/explain.txt deleted file mode 100644 index 7c4f959123..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/explain.txt +++ /dev/null @@ -1,262 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometSort (39) - +- CometColumnarExchange (38) - +- * HashAggregate (37) - +- * CometColumnarToRow (36) - +- CometColumnarExchange (35) - +- * HashAggregate (34) - +- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * Project (26) - : +- * BroadcastHashJoin Inner BuildLeft (25) - : :- BroadcastExchange (5) - : : +- * CometColumnarToRow (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometNativeScan parquet spark_catalog.default.item (1) - : +- Union (24) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * Filter (8) - : : : +- * ColumnarToRow (7) - : : : +- Scan parquet spark_catalog.default.web_sales (6) - : : +- ReusedExchange (9) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Filter (14) - : : : +- * ColumnarToRow (13) - : : : +- Scan parquet spark_catalog.default.catalog_sales (12) - : : +- ReusedExchange (15) - : +- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * Filter (20) - : : +- * ColumnarToRow (19) - : : +- Scan parquet spark_catalog.default.store_sales (18) - : +- ReusedExchange (21) - +- BroadcastExchange (31) - +- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometNativeScan parquet spark_catalog.default.time_dim (27) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Condition : ((isnotnull(i_manager_id#4) AND (i_manager_id#4 = 1)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Arguments: [i_item_sk#1, i_brand_id#2, i_brand#5], [i_item_sk#1, i_brand_id#2, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#3, 50, true, false, true) AS i_brand#5] - -(4) CometColumnarToRow [codegen id : 1] -Input [3]: [i_item_sk#1, i_brand_id#2, i_brand#5] - -(5) BroadcastExchange -Input [3]: [i_item_sk#1, i_brand_id#2, i_brand#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(6) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_time_sk)] -ReadSchema: struct - -(7) ColumnarToRow [codegen id : 3] -Input [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] - -(8) Filter [codegen id : 3] -Input [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Condition : (isnotnull(ws_item_sk#7) AND isnotnull(ws_sold_time_sk#6)) - -(9) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#11] - -(10) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 3] -Output [3]: [ws_ext_sales_price#8 AS ext_price#12, ws_item_sk#7 AS sold_item_sk#13, ws_sold_time_sk#6 AS time_sk#14] -Input [5]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9, d_date_sk#11] - -(12) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_sold_time_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_time_sk)] -ReadSchema: struct - -(13) ColumnarToRow [codegen id : 5] -Input [4]: [cs_sold_time_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] - -(14) Filter [codegen id : 5] -Input [4]: [cs_sold_time_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18] -Condition : (isnotnull(cs_item_sk#16) AND isnotnull(cs_sold_time_sk#15)) - -(15) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#19] - -(16) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#19] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 5] -Output [3]: [cs_ext_sales_price#17 AS ext_price#20, cs_item_sk#16 AS sold_item_sk#21, cs_sold_time_sk#15 AS time_sk#22] -Input [5]: [cs_sold_time_sk#15, cs_item_sk#16, cs_ext_sales_price#17, cs_sold_date_sk#18, d_date_sk#19] - -(18) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 7] -Input [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] - -(20) Filter [codegen id : 7] -Input [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : (isnotnull(ss_item_sk#24) AND isnotnull(ss_sold_time_sk#23)) - -(21) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#27] - -(22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: None - -(23) Project [codegen id : 7] -Output [3]: [ss_ext_sales_price#25 AS ext_price#28, ss_item_sk#24 AS sold_item_sk#29, ss_sold_time_sk#23 AS time_sk#30] -Input [5]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26, d_date_sk#27] - -(24) Union - -(25) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [sold_item_sk#13] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 9] -Output [4]: [i_brand_id#2, i_brand#5, ext_price#12, time_sk#14] -Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#5, ext_price#12, sold_item_sk#13, time_sk#14] - -(27) CometNativeScan parquet spark_catalog.default.time_dim -Output [4]: [t_time_sk#31, t_hour#32, t_minute#33, t_meal_time#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_time_sk)] -ReadSchema: struct - -(28) CometFilter -Input [4]: [t_time_sk#31, t_hour#32, t_minute#33, t_meal_time#34] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, t_meal_time#34, 20, true, false, true) = breakfast ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, t_meal_time#34, 20, true, false, true) = dinner )) AND isnotnull(t_time_sk#31)) - -(29) CometProject -Input [4]: [t_time_sk#31, t_hour#32, t_minute#33, t_meal_time#34] -Arguments: [t_time_sk#31, t_hour#32, t_minute#33], [t_time_sk#31, t_hour#32, t_minute#33] - -(30) CometColumnarToRow [codegen id : 8] -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] - -(31) BroadcastExchange -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [time_sk#14] -Right keys [1]: [t_time_sk#31] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 9] -Output [5]: [i_brand_id#2, i_brand#5, ext_price#12, t_hour#32, t_minute#33] -Input [7]: [i_brand_id#2, i_brand#5, ext_price#12, time_sk#14, t_time_sk#31, t_hour#32, t_minute#33] - -(34) HashAggregate [codegen id : 9] -Input [5]: [i_brand_id#2, i_brand#5, ext_price#12, t_hour#32, t_minute#33] -Keys [4]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33] -Functions [1]: [partial_sum(UnscaledValue(ext_price#12))] -Aggregate Attributes [1]: [sum#35] -Results [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] - -(35) CometColumnarExchange -Input [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] -Arguments: hashpartitioning(i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(36) CometColumnarToRow [codegen id : 10] -Input [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] - -(37) HashAggregate [codegen id : 10] -Input [5]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33, sum#36] -Keys [4]: [i_brand#5, i_brand_id#2, t_hour#32, t_minute#33] -Functions [1]: [sum(UnscaledValue(ext_price#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#12))#37] -Results [5]: [i_brand_id#2 AS brand_id#38, i_brand#5 AS brand#39, t_hour#32, t_minute#33, MakeDecimal(sum(UnscaledValue(ext_price#12))#37,17,2) AS ext_price#40] - -(38) CometColumnarExchange -Input [5]: [brand_id#38, brand#39, t_hour#32, t_minute#33, ext_price#40] -Arguments: rangepartitioning(ext_price#40 DESC NULLS LAST, brand_id#38 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(39) CometSort -Input [5]: [brand_id#38, brand#39, t_hour#32, t_minute#33, ext_price#40] -Arguments: [brand_id#38, brand#39, t_hour#32, t_minute#33, ext_price#40], [ext_price#40 DESC NULLS LAST, brand_id#38 ASC NULLS FIRST] - -(40) CometColumnarToRow [codegen id : 11] -Input [5]: [brand_id#38, brand#39, t_hour#32, t_minute#33, ext_price#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 6 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometNativeScan parquet spark_catalog.default.date_dim (41) - - -(41) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#41, d_moy#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#11, d_year#41, d_moy#42] -Condition : ((((isnotnull(d_moy#42) AND isnotnull(d_year#41)) AND (d_moy#42 = 11)) AND (d_year#41 = 1999)) AND isnotnull(d_date_sk#11)) - -(43) CometProject -Input [3]: [d_date_sk#11, d_year#41, d_moy#42] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(45) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 12 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#10 - -Subquery:3 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#10 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/simplified.txt deleted file mode 100644 index 838a3e6604..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometColumnarExchange [ext_price,brand_id] #1 - WholeStageCodegen (10) - HashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] [sum(UnscaledValue(ext_price)),brand_id,brand,ext_price,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand,i_brand_id,t_hour,t_minute] #2 - WholeStageCodegen (9) - HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] - Project [i_brand_id,i_brand,ext_price,t_hour,t_minute] - BroadcastHashJoin [time_sk,t_time_sk] - Project [i_brand_id,i_brand,ext_price,time_sk] - BroadcastHashJoin [i_item_sk,sold_item_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - InputAdapter - Union - WholeStageCodegen (3) - Project [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk,ws_sold_time_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (5) - Project [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk,cs_sold_time_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (7) - Project [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_sold_time_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] - CometNativeScan 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/q71.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/explain.txt deleted file mode 100644 index bb217d6417..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,251 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (37) -+- CometSort (36) - +- CometExchange (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometBroadcastExchange (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : +- CometUnion (23) - : :- CometProject (12) - : : +- CometBroadcastHashJoin (11) - : : :- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (5) - : : +- CometBroadcastExchange (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (7) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (13) - : : +- ReusedExchange (15) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (19) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (18) - : +- ReusedExchange (20) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Condition : ((isnotnull(i_manager_id#4) AND (i_manager_id#4 = 1)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Arguments: [i_item_sk#1, i_brand_id#2, i_brand#5], [i_item_sk#1, i_brand_id#2, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#3, 50, true, false, true) AS i_brand#5] - -(4) CometBroadcastExchange -Input [3]: [i_item_sk#1, i_brand_id#2, i_brand#5] -Arguments: [i_item_sk#1, i_brand_id#2, i_brand#5] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_time_sk)] -ReadSchema: struct - -(6) CometFilter -Input [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Condition : (isnotnull(ws_item_sk#7) AND isnotnull(ws_sold_time_sk#6)) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_moy#13) AND isnotnull(d_year#12)) AND (d_moy#13 = 11)) AND (d_year#12 = 1999)) AND isnotnull(d_date_sk#11)) - -(9) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(10) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(11) CometBroadcastHashJoin -Left output [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [ws_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(12) CometProject -Input [5]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9, d_date_sk#11] -Arguments: [ext_price#14, sold_item_sk#15, time_sk#16], [ws_ext_sales_price#8 AS ext_price#14, ws_item_sk#7 AS sold_item_sk#15, ws_sold_time_sk#6 AS time_sk#16] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#21)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_time_sk)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#18) AND isnotnull(cs_sold_time_sk#17)) - -(15) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#22] - -(16) CometBroadcastHashJoin -Left output [4]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Right output [1]: [d_date_sk#22] -Arguments: [cs_sold_date_sk#20], [d_date_sk#22], Inner, BuildRight - -(17) CometProject -Input [5]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] -Arguments: [ext_price#23, sold_item_sk#24, time_sk#25], [cs_ext_sales_price#19 AS ext_price#23, cs_item_sk#18 AS sold_item_sk#24, cs_sold_time_sk#17 AS time_sk#25] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] -ReadSchema: struct - -(19) CometFilter -Input [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Condition : (isnotnull(ss_item_sk#27) AND isnotnull(ss_sold_time_sk#26)) - -(20) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#31] - -(21) CometBroadcastHashJoin -Left output [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Right output [1]: [d_date_sk#31] -Arguments: [ss_sold_date_sk#29], [d_date_sk#31], Inner, BuildRight - -(22) CometProject -Input [5]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#31] -Arguments: [ext_price#32, sold_item_sk#33, time_sk#34], [ss_ext_sales_price#28 AS ext_price#32, ss_item_sk#27 AS sold_item_sk#33, ss_sold_time_sk#26 AS time_sk#34] - -(23) CometUnion -Child 0 Input [3]: [ext_price#14, sold_item_sk#15, time_sk#16] -Child 1 Input [3]: [ext_price#23, sold_item_sk#24, time_sk#25] -Child 2 Input [3]: [ext_price#32, sold_item_sk#33, time_sk#34] - -(24) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand_id#2, i_brand#5] -Right output [3]: [ext_price#14, sold_item_sk#15, time_sk#16] -Arguments: [i_item_sk#1], [sold_item_sk#15], Inner, BuildLeft - -(25) CometProject -Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#5, ext_price#14, sold_item_sk#15, time_sk#16] -Arguments: [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16], [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [4]: [t_time_sk#35, t_hour#36, t_minute#37, t_meal_time#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_time_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [t_time_sk#35, t_hour#36, t_minute#37, t_meal_time#38] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, t_meal_time#38, 20, true, false, true) = breakfast ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, t_meal_time#38, 20, true, false, true) = dinner )) AND isnotnull(t_time_sk#35)) - -(28) CometProject -Input [4]: [t_time_sk#35, t_hour#36, t_minute#37, t_meal_time#38] -Arguments: [t_time_sk#35, t_hour#36, t_minute#37], [t_time_sk#35, t_hour#36, t_minute#37] - -(29) CometBroadcastExchange -Input [3]: [t_time_sk#35, t_hour#36, t_minute#37] -Arguments: [t_time_sk#35, t_hour#36, t_minute#37] - -(30) CometBroadcastHashJoin -Left output [4]: [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16] -Right output [3]: [t_time_sk#35, t_hour#36, t_minute#37] -Arguments: [time_sk#16], [t_time_sk#35], Inner, BuildRight - -(31) CometProject -Input [7]: [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16, t_time_sk#35, t_hour#36, t_minute#37] -Arguments: [i_brand_id#2, i_brand#5, ext_price#14, t_hour#36, t_minute#37], [i_brand_id#2, i_brand#5, ext_price#14, t_hour#36, t_minute#37] - -(32) CometHashAggregate -Input [5]: [i_brand_id#2, i_brand#5, ext_price#14, t_hour#36, t_minute#37] -Keys [4]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37] -Functions [1]: [partial_sum(UnscaledValue(ext_price#14))] - -(33) CometExchange -Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] -Arguments: hashpartitioning(i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(34) CometHashAggregate -Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] -Keys [4]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37] -Functions [1]: [sum(UnscaledValue(ext_price#14))] - -(35) CometExchange -Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] -Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(36) CometSort -Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] -Arguments: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42], [ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST] - -(37) CometColumnarToRow [codegen id : 1] -Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (42) -+- * CometColumnarToRow (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) - - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(39) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_moy#13) AND isnotnull(d_year#12)) AND (d_moy#13 = 11)) AND (d_year#12 = 1999)) AND isnotnull(d_date_sk#11)) - -(40) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(41) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(42) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#10 - -Subquery:3 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#10 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/simplified.txt deleted file mode 100644 index 63e7c353ac..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometExchange [ext_price,brand_id] #1 - CometHashAggregate [sum] [brand_id,brand,t_hour,t_minute,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ext_price))] - CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 - CometHashAggregate [ext_price] [i_brand,i_brand_id,t_hour,t_minute,sum] - CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] - 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,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - 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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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_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 [native_iceberg_compat] 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_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 [native_iceberg_compat] 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 [t_time_sk,t_hour,t_minute] #6 - CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] - CometScan [native_iceberg_compat] 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/q71/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt deleted file mode 100644 index bb217d6417..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt +++ /dev/null @@ -1,251 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (37) -+- CometSort (36) - +- CometExchange (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometBroadcastExchange (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : +- CometUnion (23) - : :- CometProject (12) - : : +- CometBroadcastHashJoin (11) - : : :- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (5) - : : +- CometBroadcastExchange (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (7) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (13) - : : +- ReusedExchange (15) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (19) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (18) - : +- ReusedExchange (20) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (26) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Condition : ((isnotnull(i_manager_id#4) AND (i_manager_id#4 = 1)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] -Arguments: [i_item_sk#1, i_brand_id#2, i_brand#5], [i_item_sk#1, i_brand_id#2, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#3, 50, true, false, true) AS i_brand#5] - -(4) CometBroadcastExchange -Input [3]: [i_item_sk#1, i_brand_id#2, i_brand#5] -Arguments: [i_item_sk#1, i_brand_id#2, i_brand#5] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_time_sk)] -ReadSchema: struct - -(6) CometFilter -Input [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Condition : (isnotnull(ws_item_sk#7) AND isnotnull(ws_sold_time_sk#6)) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_moy#13) AND isnotnull(d_year#12)) AND (d_moy#13 = 11)) AND (d_year#12 = 1999)) AND isnotnull(d_date_sk#11)) - -(9) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(10) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(11) CometBroadcastHashJoin -Left output [4]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [ws_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(12) CometProject -Input [5]: [ws_sold_time_sk#6, ws_item_sk#7, ws_ext_sales_price#8, ws_sold_date_sk#9, d_date_sk#11] -Arguments: [ext_price#14, sold_item_sk#15, time_sk#16], [ws_ext_sales_price#8 AS ext_price#14, ws_item_sk#7 AS sold_item_sk#15, ws_sold_time_sk#6 AS time_sk#16] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#21)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_time_sk)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#18) AND isnotnull(cs_sold_time_sk#17)) - -(15) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#22] - -(16) CometBroadcastHashJoin -Left output [4]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] -Right output [1]: [d_date_sk#22] -Arguments: [cs_sold_date_sk#20], [d_date_sk#22], Inner, BuildRight - -(17) CometProject -Input [5]: [cs_sold_time_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] -Arguments: [ext_price#23, sold_item_sk#24, time_sk#25], [cs_ext_sales_price#19 AS ext_price#23, cs_item_sk#18 AS sold_item_sk#24, cs_sold_time_sk#17 AS time_sk#25] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] -ReadSchema: struct - -(19) CometFilter -Input [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Condition : (isnotnull(ss_item_sk#27) AND isnotnull(ss_sold_time_sk#26)) - -(20) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#31] - -(21) CometBroadcastHashJoin -Left output [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Right output [1]: [d_date_sk#31] -Arguments: [ss_sold_date_sk#29], [d_date_sk#31], Inner, BuildRight - -(22) CometProject -Input [5]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#31] -Arguments: [ext_price#32, sold_item_sk#33, time_sk#34], [ss_ext_sales_price#28 AS ext_price#32, ss_item_sk#27 AS sold_item_sk#33, ss_sold_time_sk#26 AS time_sk#34] - -(23) CometUnion -Child 0 Input [3]: [ext_price#14, sold_item_sk#15, time_sk#16] -Child 1 Input [3]: [ext_price#23, sold_item_sk#24, time_sk#25] -Child 2 Input [3]: [ext_price#32, sold_item_sk#33, time_sk#34] - -(24) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand_id#2, i_brand#5] -Right output [3]: [ext_price#14, sold_item_sk#15, time_sk#16] -Arguments: [i_item_sk#1], [sold_item_sk#15], Inner, BuildLeft - -(25) CometProject -Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#5, ext_price#14, sold_item_sk#15, time_sk#16] -Arguments: [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16], [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [4]: [t_time_sk#35, t_hour#36, t_minute#37, t_meal_time#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_time_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [t_time_sk#35, t_hour#36, t_minute#37, t_meal_time#38] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, t_meal_time#38, 20, true, false, true) = breakfast ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, t_meal_time#38, 20, true, false, true) = dinner )) AND isnotnull(t_time_sk#35)) - -(28) CometProject -Input [4]: [t_time_sk#35, t_hour#36, t_minute#37, t_meal_time#38] -Arguments: [t_time_sk#35, t_hour#36, t_minute#37], [t_time_sk#35, t_hour#36, t_minute#37] - -(29) CometBroadcastExchange -Input [3]: [t_time_sk#35, t_hour#36, t_minute#37] -Arguments: [t_time_sk#35, t_hour#36, t_minute#37] - -(30) CometBroadcastHashJoin -Left output [4]: [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16] -Right output [3]: [t_time_sk#35, t_hour#36, t_minute#37] -Arguments: [time_sk#16], [t_time_sk#35], Inner, BuildRight - -(31) CometProject -Input [7]: [i_brand_id#2, i_brand#5, ext_price#14, time_sk#16, t_time_sk#35, t_hour#36, t_minute#37] -Arguments: [i_brand_id#2, i_brand#5, ext_price#14, t_hour#36, t_minute#37], [i_brand_id#2, i_brand#5, ext_price#14, t_hour#36, t_minute#37] - -(32) CometHashAggregate -Input [5]: [i_brand_id#2, i_brand#5, ext_price#14, t_hour#36, t_minute#37] -Keys [4]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37] -Functions [1]: [partial_sum(UnscaledValue(ext_price#14))] - -(33) CometExchange -Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] -Arguments: hashpartitioning(i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(34) CometHashAggregate -Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] -Keys [4]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37] -Functions [1]: [sum(UnscaledValue(ext_price#14))] - -(35) CometExchange -Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] -Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(36) CometSort -Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] -Arguments: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42], [ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST] - -(37) CometColumnarToRow [codegen id : 1] -Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (42) -+- * CometColumnarToRow (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) - - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(39) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_moy#13) AND isnotnull(d_year#12)) AND (d_moy#13 = 11)) AND (d_year#12 = 1999)) AND isnotnull(d_date_sk#11)) - -(40) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(41) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(42) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#10 - -Subquery:3 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#10 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/extended.txt deleted file mode 100644 index 25ea15c536..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/extended.txt +++ /dev/null @@ -1,53 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometUnion - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - -Comet accelerated 45 out of 49 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 63e7c353ac..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometExchange [ext_price,brand_id] #1 - CometHashAggregate [sum] [brand_id,brand,t_hour,t_minute,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ext_price))] - CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 - CometHashAggregate [ext_price] [i_brand,i_brand_id,t_hour,t_minute,sum] - CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] - 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,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - 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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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_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 [native_iceberg_compat] 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_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 [native_iceberg_compat] 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 [t_time_sk,t_hour,t_minute] #6 - CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/explain.txt deleted file mode 100644 index ed3a4a7aea..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/explain.txt +++ /dev/null @@ -1,428 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (70) -+- CometTakeOrderedAndProject (69) - +- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometProject (65) - +- CometSortMergeJoin (64) - :- CometSort (58) - : +- CometColumnarExchange (57) - : +- * Project (56) - : +- * BroadcastHashJoin LeftOuter BuildRight (55) - : :- * Project (50) - : : +- * BroadcastHashJoin Inner BuildRight (49) - : : :- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (28) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Project (15) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : : :- * Project (9) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : : : : :- * Filter (3) - : : : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * CometColumnarToRow (6) - : : : : : : : : : +- CometFilter (5) - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory (4) - : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * CometColumnarToRow (12) - : : : : : : : : +- CometFilter (11) - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * CometColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) - : : : : : : +- BroadcastExchange (26) - : : : : : : +- * CometColumnarToRow (25) - : : : : : : +- CometProject (24) - : : : : : : +- CometFilter (23) - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (22) - : : : : : +- BroadcastExchange (33) - : : : : : +- * CometColumnarToRow (32) - : : : : : +- CometProject (31) - : : : : : +- CometFilter (30) - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (29) - : : : : +- ReusedExchange (36) - : : : +- BroadcastExchange (42) - : : : +- * CometColumnarToRow (41) - : : : +- CometFilter (40) - : : : +- CometNativeScan parquet spark_catalog.default.date_dim (39) - : : +- BroadcastExchange (48) - : : +- * CometColumnarToRow (47) - : : +- CometFilter (46) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (45) - : +- BroadcastExchange (54) - : +- * CometColumnarToRow (53) - : +- CometFilter (52) - : +- CometNativeScan parquet spark_catalog.default.promotion (51) - +- CometSort (63) - +- CometExchange (62) - +- CometProject (61) - +- CometFilter (60) - +- CometNativeScan parquet spark_catalog.default.catalog_returns (59) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 10] -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] - -(3) Filter [codegen id : 10] -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) - -(4) CometNativeScan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(inv_date_sk#13)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) - -(6) CometColumnarToRow [codegen id : 1] -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(7) BroadcastExchange -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [inv_item_sk#10] -Join type: Inner -Join condition: (inv_quantity_on_hand#12 < cs_quantity#7) - -(9) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(10) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Condition : isnotnull(w_warehouse_sk#14) - -(12) CometColumnarToRow [codegen id : 2] -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] - -(13) BroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_warehouse_sk#11] -Right keys [1]: [w_warehouse_sk#14] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] - -(16) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_desc#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [i_item_sk#16, i_item_desc#17] -Condition : isnotnull(i_item_sk#16) - -(18) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#16, i_item_desc#17] - -(19) BroadcastExchange -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(20) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#16] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 10] -Output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] - -(22) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#18, cd_marital_status#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(23) CometFilter -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = D) AND isnotnull(cd_demo_sk#18)) - -(24) CometProject -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Arguments: [cd_demo_sk#18], [cd_demo_sk#18] - -(25) CometColumnarToRow [codegen id : 4] -Input [1]: [cd_demo_sk#18] - -(26) BroadcastExchange -Input [1]: [cd_demo_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(27) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] - -(29) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_buy_potential#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#21, 15, true, false, true) = >10000 ) AND isnotnull(hd_demo_sk#20)) - -(31) CometProject -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Arguments: [hd_demo_sk#20], [hd_demo_sk#20] - -(32) CometColumnarToRow [codegen id : 5] -Input [1]: [hd_demo_sk#20] - -(33) BroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] - -(36) ReusedExchange [Reuses operator id: 75] -Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#8] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] - -(39) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_week_seq#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(40) CometFilter -Input [2]: [d_date_sk#25, d_week_seq#26] -Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) - -(41) CometColumnarToRow [codegen id : 7] -Input [2]: [d_date_sk#25, d_week_seq#26] - -(42) BroadcastExchange -Input [2]: [d_date_sk#25, d_week_seq#26] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] - -(43) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [d_week_seq#24, inv_date_sk#13] -Right keys [2]: [d_week_seq#26, d_date_sk#25] -Join type: Inner -Join condition: None - -(44) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#25, d_week_seq#26] - -(45) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#27, d_date#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [d_date_sk#27, d_date#28] -Condition : (isnotnull(d_date#28) AND isnotnull(d_date_sk#27)) - -(47) CometColumnarToRow [codegen id : 8] -Input [2]: [d_date_sk#27, d_date#28] - -(48) BroadcastExchange -Input [2]: [d_date_sk#27, d_date#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: (d_date#28 > date_add(d_date#23, 5)) - -(50) Project [codegen id : 10] -Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#27, d_date#28] - -(51) CometNativeScan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(52) CometFilter -Input [1]: [p_promo_sk#29] -Condition : isnotnull(p_promo_sk#29) - -(53) CometColumnarToRow [codegen id : 9] -Input [1]: [p_promo_sk#29] - -(54) BroadcastExchange -Input [1]: [p_promo_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(55) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_promo_sk#5] -Right keys [1]: [p_promo_sk#29] -Join type: LeftOuter -Join condition: None - -(56) Project [codegen id : 10] -Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] - -(57) CometColumnarExchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(58) CometSort -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] - -(59) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(60) CometFilter -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Condition : (isnotnull(cr_item_sk#30) AND isnotnull(cr_order_number#31)) - -(61) CometProject -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30, cr_order_number#31] - -(62) CometExchange -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(63) CometSort -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST] - -(64) CometSortMergeJoin -Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#30, cr_order_number#31], LeftOuter - -(65) CometProject -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#30, cr_order_number#31] -Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(66) CometHashAggregate -Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [partial_count(1)] - -(67) CometExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(68) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [count(1)] - -(69) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#36 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#34,promo#35,total_cnt#36]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36], 100, 0, [total_cnt#36 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] - -(70) CometColumnarToRow [codegen id : 11] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (75) -+- * CometColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometNativeScan parquet spark_catalog.default.date_dim (71) - - -(71) CometNativeScan parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(72) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] -Condition : ((((isnotnull(d_year#37) AND (d_year#37 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(73) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(74) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(75) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/simplified.txt deleted file mode 100644 index 025217a607..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/simplified.txt +++ /dev/null @@ -1,107 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] - CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] - CometProject [w_warehouse_name,i_item_desc,d_week_seq] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometColumnarExchange [cs_item_sk,cs_order_number] #2 - WholeStageCodegen (10) - Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [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] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometNativeScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometNativeScan 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/q72.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/explain.txt deleted file mode 100644 index bf5bcb475b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,402 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (64) -+- CometTakeOrderedAndProject (63) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometSortMergeJoin (58) - :- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (45) - : : +- CometBroadcastHashJoin (44) - : : :- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (35) - : : : : +- CometBroadcastHashJoin (34) - : : : : :- CometProject (29) - : : : : : +- CometBroadcastHashJoin (28) - : : : : : :- CometProject (23) - : : : : : : +- CometBroadcastHashJoin (22) - : : : : : : :- CometProject (17) - : : : : : : : +- CometBroadcastHashJoin (16) - : : : : : : : :- CometProject (12) - : : : : : : : : +- CometBroadcastHashJoin (11) - : : : : : : : : :- CometProject (7) - : : : : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : : : : :- CometFilter (2) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- CometBroadcastExchange (5) - : : : : : : : : : +- CometFilter (4) - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (3) - : : : : : : : : +- CometBroadcastExchange (10) - : : : : : : : : +- CometFilter (9) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : : : : : : : +- CometBroadcastExchange (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (13) - : : : : : : +- CometBroadcastExchange (21) - : : : : : : +- CometProject (20) - : : : : : : +- CometFilter (19) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (18) - : : : : : +- CometBroadcastExchange (27) - : : : : : +- CometProject (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (24) - : : : : +- CometBroadcastExchange (33) - : : : : +- CometProject (32) - : : : : +- CometFilter (31) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - : : : +- CometBroadcastExchange (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (36) - : : +- CometBroadcastExchange (43) - : : +- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - : +- CometBroadcastExchange (48) - : +- CometFilter (47) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (46) - +- CometSort (57) - +- CometExchange (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (53) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#13)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) - -(5) CometBroadcastExchange -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(6) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Right output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_item_sk#4], [inv_item_sk#10], Inner, (inv_quantity_on_hand#12 < cs_quantity#7), BuildRight - -(7) CometProject -Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Condition : isnotnull(w_warehouse_sk#14) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [w_warehouse_sk#14, w_warehouse_name#15] - -(11) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -Right output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [inv_warehouse_sk#11], [w_warehouse_sk#14], Inner, BuildRight - -(12) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_desc#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [i_item_sk#16, i_item_desc#17] -Condition : isnotnull(i_item_sk#16) - -(15) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [i_item_sk#16, i_item_desc#17] - -(16) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -Right output [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [cs_item_sk#4], [i_item_sk#16], Inner, BuildRight - -(17) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#18, cd_marital_status#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = D) AND isnotnull(cd_demo_sk#18)) - -(20) CometProject -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Arguments: [cd_demo_sk#18], [cd_demo_sk#18] - -(21) CometBroadcastExchange -Input [1]: [cd_demo_sk#18] -Arguments: [cd_demo_sk#18] - -(22) CometBroadcastHashJoin -Left output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [cd_demo_sk#18] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#18], Inner, BuildRight - -(23) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] -Arguments: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_buy_potential#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#21, 15, true, false, true) = >10000 ) AND isnotnull(hd_demo_sk#20)) - -(26) CometProject -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Arguments: [hd_demo_sk#20], [hd_demo_sk#20] - -(27) CometBroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: [hd_demo_sk#20] - -(28) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [hd_demo_sk#20] -Arguments: [cs_bill_hdemo_sk#3], [hd_demo_sk#20], Inner, BuildRight - -(29) CometProject -Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(31) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(32) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(33) CometBroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24] - -(34) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_sold_date_sk#8], [d_date_sk#22], Inner, BuildRight - -(35) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_week_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [d_date_sk#26, d_week_seq#27] -Condition : (isnotnull(d_week_seq#27) AND isnotnull(d_date_sk#26)) - -(38) CometBroadcastExchange -Input [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_date_sk#26, d_week_seq#27] - -(39) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_week_seq#24, inv_date_sk#13], [d_week_seq#27, d_date_sk#26], Inner, BuildRight - -(40) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#26, d_week_seq#27] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_date#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#28, d_date#29] -Condition : (isnotnull(d_date#29) AND isnotnull(d_date_sk#28)) - -(43) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_date#29] -Arguments: [d_date_sk#28, d_date#29] - -(44) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#28, d_date#29] -Arguments: [cs_ship_date_sk#1], [d_date_sk#28], Inner, (d_date#29 > date_add(d_date#23, 5)), BuildRight - -(45) CometProject -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#28, d_date#29] -Arguments: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(47) CometFilter -Input [1]: [p_promo_sk#30] -Condition : isnotnull(p_promo_sk#30) - -(48) CometBroadcastExchange -Input [1]: [p_promo_sk#30] -Arguments: [p_promo_sk#30] - -(49) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [1]: [p_promo_sk#30] -Arguments: [cs_promo_sk#5], [p_promo_sk#30], LeftOuter, BuildRight - -(50) CometProject -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#30] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(51) CometExchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(52) CometSort -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Condition : (isnotnull(cr_item_sk#31) AND isnotnull(cr_order_number#32)) - -(55) CometProject -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31, cr_order_number#32] - -(56) CometExchange -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(57) CometSort -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#31, cr_order_number#32], LeftOuter - -(59) CometProject -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#31, cr_order_number#32] -Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(60) CometHashAggregate -Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [partial_count(1)] - -(61) CometExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(62) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [count(1)] - -(63) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, 0, [total_cnt#37 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -(64) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (69) -+- * CometColumnarToRow (68) - +- CometProject (67) - +- CometFilter (66) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) - - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(67) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(68) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(69) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/simplified.txt deleted file mode 100644 index 678f8c67df..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] - CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] - CometProject [w_warehouse_name,i_item_desc,d_week_seq] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometExchange [cs_item_sk,cs_order_number] #2 - CometProject [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,p_promo_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_date] - 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 [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_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_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_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_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 [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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_desc] #6 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange [cd_demo_sk] #7 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange [hd_demo_sk] #8 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [d_date_sk,d_week_seq] #10 - CometFilter [d_date_sk,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [d_date_sk,d_date] #11 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [p_promo_sk] #12 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt deleted file mode 100644 index bf5bcb475b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt +++ /dev/null @@ -1,402 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (64) -+- CometTakeOrderedAndProject (63) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometSortMergeJoin (58) - :- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (45) - : : +- CometBroadcastHashJoin (44) - : : :- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (35) - : : : : +- CometBroadcastHashJoin (34) - : : : : :- CometProject (29) - : : : : : +- CometBroadcastHashJoin (28) - : : : : : :- CometProject (23) - : : : : : : +- CometBroadcastHashJoin (22) - : : : : : : :- CometProject (17) - : : : : : : : +- CometBroadcastHashJoin (16) - : : : : : : : :- CometProject (12) - : : : : : : : : +- CometBroadcastHashJoin (11) - : : : : : : : : :- CometProject (7) - : : : : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : : : : :- CometFilter (2) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- CometBroadcastExchange (5) - : : : : : : : : : +- CometFilter (4) - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (3) - : : : : : : : : +- CometBroadcastExchange (10) - : : : : : : : : +- CometFilter (9) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : : : : : : : +- CometBroadcastExchange (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (13) - : : : : : : +- CometBroadcastExchange (21) - : : : : : : +- CometProject (20) - : : : : : : +- CometFilter (19) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (18) - : : : : : +- CometBroadcastExchange (27) - : : : : : +- CometProject (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (24) - : : : : +- CometBroadcastExchange (33) - : : : : +- CometProject (32) - : : : : +- CometFilter (31) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - : : : +- CometBroadcastExchange (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (36) - : : +- CometBroadcastExchange (43) - : : +- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - : +- CometBroadcastExchange (48) - : +- CometFilter (47) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (46) - +- CometSort (57) - +- CometExchange (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (53) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#13)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) - -(5) CometBroadcastExchange -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(6) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Right output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_item_sk#4], [inv_item_sk#10], Inner, (inv_quantity_on_hand#12 < cs_quantity#7), BuildRight - -(7) CometProject -Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Condition : isnotnull(w_warehouse_sk#14) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [w_warehouse_sk#14, w_warehouse_name#15] - -(11) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -Right output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [inv_warehouse_sk#11], [w_warehouse_sk#14], Inner, BuildRight - -(12) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_desc#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [i_item_sk#16, i_item_desc#17] -Condition : isnotnull(i_item_sk#16) - -(15) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [i_item_sk#16, i_item_desc#17] - -(16) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -Right output [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [cs_item_sk#4], [i_item_sk#16], Inner, BuildRight - -(17) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#18, cd_marital_status#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = D) AND isnotnull(cd_demo_sk#18)) - -(20) CometProject -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Arguments: [cd_demo_sk#18], [cd_demo_sk#18] - -(21) CometBroadcastExchange -Input [1]: [cd_demo_sk#18] -Arguments: [cd_demo_sk#18] - -(22) CometBroadcastHashJoin -Left output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [cd_demo_sk#18] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#18], Inner, BuildRight - -(23) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] -Arguments: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_buy_potential#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#21, 15, true, false, true) = >10000 ) AND isnotnull(hd_demo_sk#20)) - -(26) CometProject -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Arguments: [hd_demo_sk#20], [hd_demo_sk#20] - -(27) CometBroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: [hd_demo_sk#20] - -(28) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [hd_demo_sk#20] -Arguments: [cs_bill_hdemo_sk#3], [hd_demo_sk#20], Inner, BuildRight - -(29) CometProject -Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(31) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(32) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(33) CometBroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24] - -(34) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_sold_date_sk#8], [d_date_sk#22], Inner, BuildRight - -(35) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_week_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [d_date_sk#26, d_week_seq#27] -Condition : (isnotnull(d_week_seq#27) AND isnotnull(d_date_sk#26)) - -(38) CometBroadcastExchange -Input [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_date_sk#26, d_week_seq#27] - -(39) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_week_seq#24, inv_date_sk#13], [d_week_seq#27, d_date_sk#26], Inner, BuildRight - -(40) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#26, d_week_seq#27] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_date#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#28, d_date#29] -Condition : (isnotnull(d_date#29) AND isnotnull(d_date_sk#28)) - -(43) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_date#29] -Arguments: [d_date_sk#28, d_date#29] - -(44) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#28, d_date#29] -Arguments: [cs_ship_date_sk#1], [d_date_sk#28], Inner, (d_date#29 > date_add(d_date#23, 5)), BuildRight - -(45) CometProject -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#28, d_date#29] -Arguments: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(47) CometFilter -Input [1]: [p_promo_sk#30] -Condition : isnotnull(p_promo_sk#30) - -(48) CometBroadcastExchange -Input [1]: [p_promo_sk#30] -Arguments: [p_promo_sk#30] - -(49) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [1]: [p_promo_sk#30] -Arguments: [cs_promo_sk#5], [p_promo_sk#30], LeftOuter, BuildRight - -(50) CometProject -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#30] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(51) CometExchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(52) CometSort -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Condition : (isnotnull(cr_item_sk#31) AND isnotnull(cr_order_number#32)) - -(55) CometProject -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31, cr_order_number#32] - -(56) CometExchange -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(57) CometSort -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#31, cr_order_number#32], LeftOuter - -(59) CometProject -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#31, cr_order_number#32] -Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(60) CometHashAggregate -Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [partial_count(1)] - -(61) CometExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(62) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [count(1)] - -(63) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, 0, [total_cnt#37 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -(64) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (69) -+- * CometColumnarToRow (68) - +- CometProject (67) - +- CometFilter (66) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) - - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(67) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(68) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(69) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/extended.txt deleted file mode 100644 index 9000d27d1b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/extended.txt +++ /dev/null @@ -1,72 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 678f8c67df..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] - CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] - CometProject [w_warehouse_name,i_item_desc,d_week_seq] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometExchange [cs_item_sk,cs_order_number] #2 - CometProject [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,p_promo_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_date] - 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 [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_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_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_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_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 [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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_desc] #6 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange [cd_demo_sk] #7 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange [hd_demo_sk] #8 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [d_date_sk,d_week_seq] #10 - CometFilter [d_date_sk,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [d_date_sk,d_date] #11 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [p_promo_sk] #12 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/explain.txt deleted file mode 100644 index 7fe1aa758c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/explain.txt +++ /dev/null @@ -1,231 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (35) -+- CometSort (34) - +- CometColumnarExchange (33) - +- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * Filter (25) - : +- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (30) - +- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometNativeScan parquet spark_catalog.default.customer (26) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 40] -Output [1]: [d_date_sk#7] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_county#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_county, [Bronx County,Franklin Parish,Orange County,Williamson County]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [s_store_sk#8, s_county#9] -Condition : (s_county#9 IN (Williamson County,Franklin Parish,Bronx County,Orange County) AND isnotnull(s_store_sk#8)) - -(9) CometProject -Input [2]: [s_store_sk#8, s_county#9] -Arguments: [s_store_sk#8], [s_store_sk#8] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#8] - -(11) BroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] - -(14) CometNativeScan parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : ((((isnotnull(hd_vehicle_count#13) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#11, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#11, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#12 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#13 as double)))))) > 1.0) END) AND isnotnull(hd_demo_sk#10)) - -(16) CometProject -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Arguments: [hd_demo_sk#10], [hd_demo_sk#10] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#10] - -(18) BroadcastExchange -Input [1]: [hd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#10] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [2]: [ss_customer_sk#1, ss_ticket_number#4] -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] - -(21) HashAggregate [codegen id : 4] -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#14] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(22) CometColumnarExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(24) HashAggregate [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#16] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] - -(25) Filter [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 1) AND (cnt#17 <= 5)) - -(26) CometNativeScan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(27) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(28) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] - -(29) CometColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(30) BroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#18] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 6] -Output [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(33) CometColumnarExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(34) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [cnt#17 DESC NULLS LAST] - -(35) CometColumnarToRow [codegen id : 7] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (40) -+- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometNativeScan parquet spark_catalog.default.date_dim (36) - - -(36) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#27, d_dom#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [d_date_sk#7, d_year#27, d_dom#28] -Condition : ((((isnotnull(d_dom#28) AND (d_dom#28 >= 1)) AND (d_dom#28 <= 2)) AND d_year#27 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(38) CometProject -Input [3]: [d_date_sk#7, d_year#27, d_dom#28] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(39) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(40) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/simplified.txt deleted file mode 100644 index 4d2a6a74ab..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/simplified.txt +++ /dev/null @@ -1,59 +0,0 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [cnt] #1 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometNativeScan 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/q73.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/explain.txt deleted file mode 100644 index 346e110371..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometSort (32) - +- CometExchange (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (24) - : +- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_county#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_county, [Bronx County,Franklin Parish,Orange County,Williamson County]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#10, s_county#11] -Condition : (s_county#11 IN (Williamson County,Franklin Parish,Bronx County,Orange County) AND isnotnull(s_store_sk#10)) - -(11) CometProject -Input [2]: [s_store_sk#10, s_county#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : ((((isnotnull(hd_vehicle_count#15) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#14 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#15 as double)))))) > 1.0) END) AND isnotnull(hd_demo_sk#12)) - -(17) CometProject -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#12], [hd_demo_sk#12] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#12] -Arguments: [hd_demo_sk#12] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Right output [1]: [hd_demo_sk#12] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight - -(20) CometProject -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] -Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] - -(21) CometHashAggregate -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] - -(22) CometExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] - -(24) CometFilter -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 1) AND (cnt#17 <= 5)) - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(27) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(29) CometBroadcastHashJoin -Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Right output [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -(31) CometExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(32) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [cnt#17 DESC NULLS LAST] - -(33) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(36) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(37) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(38) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/simplified.txt deleted file mode 100644 index 227dd00f2b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometExchange [cnt] #1 - CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] - CometExchange [ss_ticket_number,ss_customer_sk] #2 - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - 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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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/q73/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt deleted file mode 100644 index 346e110371..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometSort (32) - +- CometExchange (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (24) - : +- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_county#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [In(s_county, [Bronx County,Franklin Parish,Orange County,Williamson County]), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#10, s_county#11] -Condition : (s_county#11 IN (Williamson County,Franklin Parish,Bronx County,Orange County) AND isnotnull(s_store_sk#10)) - -(11) CometProject -Input [2]: [s_store_sk#10, s_county#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : ((((isnotnull(hd_vehicle_count#15) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#14 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#15 as double)))))) > 1.0) END) AND isnotnull(hd_demo_sk#12)) - -(17) CometProject -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#12], [hd_demo_sk#12] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#12] -Arguments: [hd_demo_sk#12] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Right output [1]: [hd_demo_sk#12] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight - -(20) CometProject -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] -Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] - -(21) CometHashAggregate -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] - -(22) CometExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] - -(24) CometFilter -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 1) AND (cnt#17 <= 5)) - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(27) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(29) CometBroadcastHashJoin -Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Right output [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -(31) CometExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(32) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [cnt#17 DESC NULLS LAST] - -(33) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : ((((isnotnull(d_dom#9) AND (d_dom#9 >= 1)) AND (d_dom#9 <= 2)) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(36) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(37) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(38) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/extended.txt deleted file mode 100644 index 3d9d23d2cc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/extended.txt +++ /dev/null @@ -1,41 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 227dd00f2b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometExchange [cnt] #1 - CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] - CometExchange [ss_ticket_number,ss_customer_sk] #2 - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - 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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/explain.txt deleted file mode 100644 index c019668fc8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/explain.txt +++ /dev/null @@ -1,513 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (79) -+- * Project (78) - +- * BroadcastHashJoin Inner BuildRight (77) - :- * Project (58) - : +- * BroadcastHashJoin Inner BuildRight (57) - : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- * HashAggregate (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * Project (28) - : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (26) - : : : +- * Filter (25) - : : : +- * ColumnarToRow (24) - : : : +- Scan parquet spark_catalog.default.store_sales (23) - : : +- ReusedExchange (29) - : +- BroadcastExchange (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- * CometColumnarToRow (53) - : +- CometColumnarExchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (41) - : : : +- CometProject (40) - : : : +- CometFilter (39) - : : : +- CometNativeScan parquet spark_catalog.default.customer (38) - : : +- BroadcastExchange (45) - : : +- * Filter (44) - : : +- * ColumnarToRow (43) - : : +- Scan parquet spark_catalog.default.web_sales (42) - : +- ReusedExchange (48) - +- BroadcastExchange (76) - +- * HashAggregate (75) - +- * CometColumnarToRow (74) - +- CometColumnarExchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * CometColumnarToRow (62) - : : +- CometProject (61) - : : +- CometFilter (60) - : : +- CometNativeScan parquet spark_catalog.default.customer (59) - : +- BroadcastExchange (66) - : +- * Filter (65) - : +- * ColumnarToRow (64) - : +- Scan parquet spark_catalog.default.web_sales (63) - +- ReusedExchange (69) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#7] - -(4) CometColumnarToRow [codegen id : 3] -Input [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] - -(5) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(7) Filter [codegen id : 1] -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Condition : isnotnull(ss_customer_sk#8) - -(8) BroadcastExchange -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#8] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(11) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#12, d_year#13] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#10] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] - -(14) HashAggregate [codegen id : 3] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] -Aggregate Attributes [1]: [sum#14] -Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] - -(15) CometColumnarExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 16] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] - -(17) HashAggregate [codegen id : 16] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#9))#16] -Results [2]: [c_customer_id#5 AS customer_id#17, MakeDecimal(sum(UnscaledValue(ss_net_paid#9))#16,17,2) AS year_total#18] - -(18) Filter [codegen id : 16] -Input [2]: [customer_id#17, year_total#18] -Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) - -(19) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(20) CometFilter -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Condition : (isnotnull(c_customer_sk#19) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#20, 16, true, false, true))) - -(21) CometProject -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Arguments: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#20, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#21, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#22, 30, true, false, true) AS c_last_name#7] - -(22) CometColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7] - -(23) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] - -(25) Filter [codegen id : 4] -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_customer_sk#23) - -(26) BroadcastExchange -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#19] -Right keys [1]: [ss_customer_sk#23] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25] -Input [7]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] - -(29) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#27, d_year#28] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#25] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25, d_date_sk#27, d_year#28] - -(32) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#24))] -Aggregate Attributes [1]: [sum#29] -Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] - -(33) CometColumnarExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 7] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] - -(35) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] -Functions [1]: [sum(UnscaledValue(ss_net_paid#24))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#24))#16] -Results [4]: [c_customer_id#5 AS customer_id#31, c_first_name#6 AS customer_first_name#32, c_last_name#7 AS customer_last_name#33, MakeDecimal(sum(UnscaledValue(ss_net_paid#24))#16,17,2) AS year_total#34] - -(36) BroadcastExchange -Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(37) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#31] -Join type: Inner -Join condition: None - -(38) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(39) CometFilter -Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Condition : (isnotnull(c_customer_sk#35) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#36, 16, true, false, true))) - -(40) CometProject -Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Arguments: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#36, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#37, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#38, 30, true, false, true) AS c_last_name#41] - -(41) CometColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41] - -(42) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] - -(44) Filter [codegen id : 8] -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_bill_customer_sk#42) - -(45) BroadcastExchange -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -(46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#35] -Right keys [1]: [ws_bill_customer_sk#42] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 10] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] -Input [7]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] - -(48) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#45, d_year#46] - -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#45] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 10] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#46] -Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#45, d_year#46] - -(51) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#46] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] -Aggregate Attributes [1]: [sum#47] -Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] - -(52) CometColumnarExchange -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] -Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(53) CometColumnarToRow [codegen id : 11] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] - -(54) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46] -Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#49] -Results [2]: [c_customer_id#39 AS customer_id#50, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#49,17,2) AS year_total#51] - -(55) Filter [codegen id : 11] -Input [2]: [customer_id#50, year_total#51] -Condition : (isnotnull(year_total#51) AND (year_total#51 > 0.00)) - -(56) BroadcastExchange -Input [2]: [customer_id#50, year_total#51] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(57) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#50] -Join type: Inner -Join condition: None - -(58) Project [codegen id : 16] -Output [7]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#51] -Input [8]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, customer_id#50, year_total#51] - -(59) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(60) CometFilter -Input [4]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55] -Condition : (isnotnull(c_customer_sk#52) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#53, 16, true, false, true))) - -(61) CometProject -Input [4]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55] -Arguments: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#52, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#53, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#54, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#55, 30, true, false, true) AS c_last_name#41] - -(62) CometColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41] - -(63) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#26)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(64) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] - -(65) Filter [codegen id : 12] -Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] -Condition : isnotnull(ws_bill_customer_sk#56) - -(66) BroadcastExchange -Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#52] -Right keys [1]: [ws_bill_customer_sk#56] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 14] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, ws_sold_date_sk#58] -Input [7]: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] - -(69) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#59, d_year#60] - -(70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#58] -Right keys [1]: [d_date_sk#59] -Join type: Inner -Join condition: None - -(71) Project [codegen id : 14] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, d_year#60] -Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, ws_sold_date_sk#58, d_date_sk#59, d_year#60] - -(72) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, d_year#60] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#57))] -Aggregate Attributes [1]: [sum#61] -Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] - -(73) CometColumnarExchange -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] -Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(74) CometColumnarToRow [codegen id : 15] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] - -(75) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60] -Functions [1]: [sum(UnscaledValue(ws_net_paid#57))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#57))#49] -Results [2]: [c_customer_id#39 AS customer_id#63, MakeDecimal(sum(UnscaledValue(ws_net_paid#57))#49,17,2) AS year_total#64] - -(76) BroadcastExchange -Input [2]: [customer_id#63, year_total#64] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] - -(77) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#63] -Join type: Inner -Join condition: (CASE WHEN (year_total#51 > 0.00) THEN (year_total#64 / year_total#51) END > CASE WHEN (year_total#18 > 0.00) THEN (year_total#34 / year_total#18) END) - -(78) Project [codegen id : 16] -Output [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] -Input [9]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#51, customer_id#63, year_total#64] - -(79) TakeOrderedAndProject -Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] -Arguments: 100, [customer_id#31 ASC NULLS FIRST, customer_id#31 ASC NULLS FIRST, customer_id#31 ASC NULLS FIRST], [customer_id#31, customer_first_name#32, customer_last_name#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometFilter (81) - +- CometNativeScan parquet spark_catalog.default.date_dim (80) - - -(80) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(81) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(82) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(83) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 -BroadcastExchange (87) -+- * CometColumnarToRow (86) - +- CometFilter (85) - +- CometNativeScan parquet spark_catalog.default.date_dim (84) - - -(84) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#27, d_year#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(85) CometFilter -Input [2]: [d_date_sk#27, d_year#28] -Condition : (((isnotnull(d_year#28) AND (d_year#28 = 2002)) AND d_year#28 IN (2001,2002)) AND isnotnull(d_date_sk#27)) - -(86) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#28] - -(87) BroadcastExchange -Input [2]: [d_date_sk#27, d_year#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] - -Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#11 - -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#26 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/simplified.txt deleted file mode 100644 index f488e18c7c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/simplified.txt +++ /dev/null @@ -1,130 +0,0 @@ -TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - WholeStageCodegen (16) - Project [customer_id,customer_first_name,customer_last_name] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/explain.txt deleted file mode 100644 index 649b5dc4d5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,470 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometProject (53) - : +- CometBroadcastHashJoin (52) - : :- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (51) - : +- CometFilter (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : +- CometBroadcastExchange (41) - : : +- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) - : +- ReusedExchange (44) - +- CometBroadcastExchange (68) - +- CometHashAggregate (67) - +- CometExchange (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometFilter (55) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (62) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Condition : isnotnull(ss_customer_sk#8) - -(6) CometBroadcastExchange -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_sk#1], [ss_customer_sk#8], Inner, BuildRight - -(8) CometProject -Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(12) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#10], [d_date_sk#12], Inner, BuildRight - -(13) CometProject -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] - -(14) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] - -(15) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] - -(17) CometFilter -Input [2]: [customer_id#15, year_total#16] -Condition : (isnotnull(year_total#16) AND (year_total#16 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true))) - -(20) CometProject -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Arguments: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#19, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#20, 30, true, false, true) AS c_last_name#7] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_customer_sk#21) - -(23) CometBroadcastExchange -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] - -(24) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [c_customer_sk#17], [ss_customer_sk#21], Inner, BuildRight - -(25) CometProject -Input [7]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: [d_date_sk#25, d_year#26] - -(29) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] -Right output [2]: [d_date_sk#25, d_year#26] -Arguments: [ss_sold_date_sk#23], [d_date_sk#25], Inner, BuildRight - -(30) CometProject -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23, d_date_sk#25, d_year#26] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] - -(31) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] - -(32) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] -Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] - -(34) CometBroadcastExchange -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#15, year_total#16] -Right output [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#15], [customer_id#28], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true))) - -(38) CometProject -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Arguments: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#34, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#35, 30, true, false, true) AS c_last_name#38] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(ws_sold_date_sk#41 IN dynamicpruning#42)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Condition : isnotnull(ws_bill_customer_sk#39) - -(41) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] - -(42) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38] -Right output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [c_customer_sk#32], [ws_bill_customer_sk#39], Inner, BuildRight - -(43) CometProject -Input [7]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#43, d_year#44] - -(45) CometBroadcastHashJoin -Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] -Right output [2]: [d_date_sk#43, d_year#44] -Arguments: [ws_sold_date_sk#41], [d_date_sk#43], Inner, BuildRight - -(46) CometProject -Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41, d_date_sk#43, d_year#44] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] - -(47) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#40))] - -(48) CometExchange -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] -Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] - -(50) CometFilter -Input [2]: [customer_id#46, year_total#47] -Condition : (isnotnull(year_total#47) AND (year_total#47 > 0.00)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#46, year_total#47] -Arguments: [customer_id#46, year_total#47] - -(52) CometBroadcastHashJoin -Left output [6]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Right output [2]: [customer_id#46, year_total#47] -Arguments: [customer_id#15], [customer_id#46], Inner, BuildRight - -(53) CometProject -Input [8]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#46, year_total#47] -Arguments: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47], [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Condition : (isnotnull(c_customer_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true))) - -(56) CometProject -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Arguments: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#50, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#51, 30, true, false, true) AS c_last_name#38] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#55)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Condition : isnotnull(ws_bill_customer_sk#52) - -(59) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] - -(60) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38] -Right output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [c_customer_sk#48], [ws_bill_customer_sk#52], Inner, BuildRight - -(61) CometProject -Input [7]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#56, d_year#57] - -(63) CometBroadcastHashJoin -Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] -Right output [2]: [d_date_sk#56, d_year#57] -Arguments: [ws_sold_date_sk#54], [d_date_sk#56], Inner, BuildRight - -(64) CometProject -Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54, d_date_sk#56, d_year#57] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] - -(65) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#53))] - -(66) CometExchange -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] -Functions [1]: [sum(UnscaledValue(ws_net_paid#53))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#59, year_total#60] -Arguments: [customer_id#59, year_total#60] - -(69) CometBroadcastHashJoin -Left output [7]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] -Right output [2]: [customer_id#59, year_total#60] -Arguments: [customer_id#15], [customer_id#59], Inner, (CASE WHEN (year_total#47 > 0.00) THEN (year_total#60 / year_total#47) END > CASE WHEN (year_total#16 > 0.00) THEN (year_total#31 / year_total#16) END), BuildRight - -(70) CometProject -Input [9]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47, customer_id#59, year_total#60] -Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30], [customer_id#28, customer_first_name#29, customer_last_name#30] - -(71) CometTakeOrderedAndProject -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#28 ASC NULLS FIRST,customer_id#28 ASC NULLS FIRST,customer_id#28 ASC NULLS FIRST], output=[customer_id#28,customer_first_name#29,customer_last_name#30]), [customer_id#28, customer_first_name#29, customer_last_name#30], 100, 0, [customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] - -(72) CometColumnarToRow [codegen id : 1] -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(75) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(76) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) - -(79) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#25, d_year#26] - -(80) BroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#41 IN dynamicpruning#11 - -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#24 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/simplified.txt deleted file mode 100644 index 72e39422e0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - CometProject [customer_id,customer_first_name,customer_last_name] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 - CometHashAggregate [d_year,sum] [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt deleted file mode 100644 index 649b5dc4d5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt +++ /dev/null @@ -1,470 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometProject (53) - : +- CometBroadcastHashJoin (52) - : :- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (51) - : +- CometFilter (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : +- CometBroadcastExchange (41) - : : +- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) - : +- ReusedExchange (44) - +- CometBroadcastExchange (68) - +- CometHashAggregate (67) - +- CometExchange (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometFilter (55) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (62) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Condition : isnotnull(ss_customer_sk#8) - -(6) CometBroadcastExchange -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_sk#1], [ss_customer_sk#8], Inner, BuildRight - -(8) CometProject -Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(12) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#10], [d_date_sk#12], Inner, BuildRight - -(13) CometProject -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] - -(14) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] - -(15) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] - -(17) CometFilter -Input [2]: [customer_id#15, year_total#16] -Condition : (isnotnull(year_total#16) AND (year_total#16 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true))) - -(20) CometProject -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Arguments: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#19, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#20, 30, true, false, true) AS c_last_name#7] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_customer_sk#21) - -(23) CometBroadcastExchange -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] - -(24) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [c_customer_sk#17], [ss_customer_sk#21], Inner, BuildRight - -(25) CometProject -Input [7]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: [d_date_sk#25, d_year#26] - -(29) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] -Right output [2]: [d_date_sk#25, d_year#26] -Arguments: [ss_sold_date_sk#23], [d_date_sk#25], Inner, BuildRight - -(30) CometProject -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23, d_date_sk#25, d_year#26] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] - -(31) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] - -(32) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] -Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] - -(34) CometBroadcastExchange -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#15, year_total#16] -Right output [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#15], [customer_id#28], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true))) - -(38) CometProject -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Arguments: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#34, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#35, 30, true, false, true) AS c_last_name#38] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(ws_sold_date_sk#41 IN dynamicpruning#42)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Condition : isnotnull(ws_bill_customer_sk#39) - -(41) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] - -(42) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38] -Right output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [c_customer_sk#32], [ws_bill_customer_sk#39], Inner, BuildRight - -(43) CometProject -Input [7]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#43, d_year#44] - -(45) CometBroadcastHashJoin -Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] -Right output [2]: [d_date_sk#43, d_year#44] -Arguments: [ws_sold_date_sk#41], [d_date_sk#43], Inner, BuildRight - -(46) CometProject -Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41, d_date_sk#43, d_year#44] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] - -(47) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#40))] - -(48) CometExchange -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] -Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] - -(50) CometFilter -Input [2]: [customer_id#46, year_total#47] -Condition : (isnotnull(year_total#47) AND (year_total#47 > 0.00)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#46, year_total#47] -Arguments: [customer_id#46, year_total#47] - -(52) CometBroadcastHashJoin -Left output [6]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Right output [2]: [customer_id#46, year_total#47] -Arguments: [customer_id#15], [customer_id#46], Inner, BuildRight - -(53) CometProject -Input [8]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#46, year_total#47] -Arguments: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47], [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Condition : (isnotnull(c_customer_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true))) - -(56) CometProject -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Arguments: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#50, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#51, 30, true, false, true) AS c_last_name#38] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#55)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Condition : isnotnull(ws_bill_customer_sk#52) - -(59) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] - -(60) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38] -Right output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [c_customer_sk#48], [ws_bill_customer_sk#52], Inner, BuildRight - -(61) CometProject -Input [7]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#56, d_year#57] - -(63) CometBroadcastHashJoin -Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] -Right output [2]: [d_date_sk#56, d_year#57] -Arguments: [ws_sold_date_sk#54], [d_date_sk#56], Inner, BuildRight - -(64) CometProject -Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54, d_date_sk#56, d_year#57] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] - -(65) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#53))] - -(66) CometExchange -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] -Functions [1]: [sum(UnscaledValue(ws_net_paid#53))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#59, year_total#60] -Arguments: [customer_id#59, year_total#60] - -(69) CometBroadcastHashJoin -Left output [7]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] -Right output [2]: [customer_id#59, year_total#60] -Arguments: [customer_id#15], [customer_id#59], Inner, (CASE WHEN (year_total#47 > 0.00) THEN (year_total#60 / year_total#47) END > CASE WHEN (year_total#16 > 0.00) THEN (year_total#31 / year_total#16) END), BuildRight - -(70) CometProject -Input [9]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47, customer_id#59, year_total#60] -Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30], [customer_id#28, customer_first_name#29, customer_last_name#30] - -(71) CometTakeOrderedAndProject -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#28 ASC NULLS FIRST,customer_id#28 ASC NULLS FIRST,customer_id#28 ASC NULLS FIRST], output=[customer_id#28,customer_first_name#29,customer_last_name#30]), [customer_id#28, customer_first_name#29, customer_last_name#30], 100, 0, [customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] - -(72) CometColumnarToRow [codegen id : 1] -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(75) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(76) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) - -(79) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#25, d_year#26] - -(80) BroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#41 IN dynamicpruning#11 - -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#24 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/extended.txt deleted file mode 100644 index 476c7be954..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/extended.txt +++ /dev/null @@ -1,90 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 72e39422e0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - CometProject [customer_id,customer_first_name,customer_last_name] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 - CometHashAggregate [d_year,sum] [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/explain.txt deleted file mode 100644 index abecd7ac24..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/explain.txt +++ /dev/null @@ -1,754 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (127) -+- CometTakeOrderedAndProject (126) - +- CometProject (125) - +- CometSortMergeJoin (124) - :- CometSort (68) - : +- CometExchange (67) - : +- CometFilter (66) - : +- CometHashAggregate (65) - : +- CometExchange (64) - : +- CometHashAggregate (63) - : +- CometHashAggregate (62) - : +- CometExchange (61) - : +- CometHashAggregate (60) - : +- CometUnion (59) - : :- CometProject (22) - : : +- CometSortMergeJoin (21) - : : :- CometSort (15) - : : : +- CometColumnarExchange (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : : +- ReusedExchange (11) - : : +- CometSort (20) - : : +- CometExchange (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (40) - : : +- CometSortMergeJoin (39) - : : :- CometSort (33) - : : : +- CometColumnarExchange (32) - : : : +- * Project (31) - : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : :- * Project (28) - : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : :- * Filter (25) - : : : : : +- * ColumnarToRow (24) - : : : : : +- Scan parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (26) - : : : +- ReusedExchange (29) - : : +- CometSort (38) - : : +- CometExchange (37) - : : +- CometProject (36) - : : +- CometFilter (35) - : : +- CometNativeScan parquet spark_catalog.default.store_returns (34) - : +- CometProject (58) - : +- CometSortMergeJoin (57) - : :- CometSort (51) - : : +- CometColumnarExchange (50) - : : +- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * Project (46) - : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : :- * Filter (43) - : : : : +- * ColumnarToRow (42) - : : : : +- Scan parquet spark_catalog.default.web_sales (41) - : : : +- ReusedExchange (44) - : : +- ReusedExchange (47) - : +- CometSort (56) - : +- CometExchange (55) - : +- CometProject (54) - : +- CometFilter (53) - : +- CometNativeScan parquet spark_catalog.default.web_returns (52) - +- CometSort (123) - +- CometExchange (122) - +- CometFilter (121) - +- CometHashAggregate (120) - +- CometExchange (119) - +- CometHashAggregate (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometUnion (114) - :- CometProject (83) - : +- CometSortMergeJoin (82) - : :- CometSort (79) - : : +- CometColumnarExchange (78) - : : +- * Project (77) - : : +- * BroadcastHashJoin Inner BuildRight (76) - : : :- * Project (74) - : : : +- * BroadcastHashJoin Inner BuildRight (73) - : : : :- * Filter (71) - : : : : +- * ColumnarToRow (70) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (69) - : : : +- ReusedExchange (72) - : : +- ReusedExchange (75) - : +- CometSort (81) - : +- ReusedExchange (80) - :- CometProject (98) - : +- CometSortMergeJoin (97) - : :- CometSort (94) - : : +- CometColumnarExchange (93) - : : +- * Project (92) - : : +- * BroadcastHashJoin Inner BuildRight (91) - : : :- * Project (89) - : : : +- * BroadcastHashJoin Inner BuildRight (88) - : : : :- * Filter (86) - : : : : +- * ColumnarToRow (85) - : : : : +- Scan parquet spark_catalog.default.store_sales (84) - : : : +- ReusedExchange (87) - : : +- ReusedExchange (90) - : +- CometSort (96) - : +- ReusedExchange (95) - +- CometProject (113) - +- CometSortMergeJoin (112) - :- CometSort (109) - : +- CometColumnarExchange (108) - : +- * Project (107) - : +- * BroadcastHashJoin Inner BuildRight (106) - : :- * Project (104) - : : +- * BroadcastHashJoin Inner BuildRight (103) - : : :- * Filter (101) - : : : +- * ColumnarToRow (100) - : : : +- Scan parquet spark_catalog.default.web_sales (99) - : : +- ReusedExchange (102) - : +- ReusedExchange (105) - +- CometSort (111) - +- ReusedExchange (110) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] - -(3) Filter [codegen id : 3] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) = Books ) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) - -(6) CometProject -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(7) CometColumnarToRow [codegen id : 1] -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(8) BroadcastExchange -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(11) ReusedExchange [Reuses operator id: 131] -Output [2]: [d_date_sk#13, d_year#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] - -(14) CometColumnarExchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(15) CometSort -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] - -(16) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) - -(18) CometProject -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(19) CometExchange -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter - -(22) CometProject -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21], [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] - -(23) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 6] -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] - -(25) Filter [codegen id : 6] -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#22) - -(26) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#22] -Right keys [1]: [i_item_sk#27] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] -Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] - -(29) ReusedExchange [Reuses operator id: 131] -Output [2]: [d_date_sk#32, d_year#33] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#32] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_date_sk#32, d_year#33] - -(32) CometColumnarExchange -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(33) CometSort -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33], [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] - -(34) CometNativeScan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37, sr_returned_date_sk#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [5]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37, sr_returned_date_sk#38] -Condition : (isnotnull(sr_ticket_number#35) AND isnotnull(sr_item_sk#34)) - -(36) CometProject -Input [5]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37, sr_returned_date_sk#38] -Arguments: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37], [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] - -(37) CometExchange -Input [4]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: hashpartitioning(sr_ticket_number#35, sr_item_sk#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(38) CometSort -Input [4]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37], [sr_ticket_number#35 ASC NULLS FIRST, sr_item_sk#34 ASC NULLS FIRST] - -(39) CometSortMergeJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Right output [4]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#35, sr_item_sk#34], LeftOuter - -(40) CometProject -Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33, sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: [d_year#33, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, sales_cnt#39, sales_amt#40], [d_year#33, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, (ss_quantity#24 - coalesce(sr_return_quantity#36, 0)) AS sales_cnt#39, (ss_ext_sales_price#25 - coalesce(sr_return_amt#37, 0.00)) AS sales_amt#40] - -(41) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(42) ColumnarToRow [codegen id : 9] -Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] - -(43) Filter [codegen id : 9] -Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -Condition : isnotnull(ws_item_sk#41) - -(44) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] - -(45) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#41] -Right keys [1]: [i_item_sk#46] -Join type: Inner -Join condition: None - -(46) Project [codegen id : 9] -Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] -Input [10]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] - -(47) ReusedExchange [Reuses operator id: 131] -Output [2]: [d_date_sk#51, d_year#52] - -(48) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#45] -Right keys [1]: [d_date_sk#51] -Join type: Inner -Join condition: None - -(49) Project [codegen id : 9] -Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Input [11]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_date_sk#51, d_year#52] - -(50) CometColumnarExchange -Input [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Arguments: hashpartitioning(ws_order_number#42, ws_item_sk#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(51) CometSort -Input [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Arguments: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52], [ws_order_number#42 ASC NULLS FIRST, ws_item_sk#41 ASC NULLS FIRST] - -(52) CometNativeScan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56, wr_returned_date_sk#57] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(53) CometFilter -Input [5]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56, wr_returned_date_sk#57] -Condition : (isnotnull(wr_order_number#54) AND isnotnull(wr_item_sk#53)) - -(54) CometProject -Input [5]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56, wr_returned_date_sk#57] -Arguments: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56], [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] - -(55) CometExchange -Input [4]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: hashpartitioning(wr_order_number#54, wr_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(56) CometSort -Input [4]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56], [wr_order_number#54 ASC NULLS FIRST, wr_item_sk#53 ASC NULLS FIRST] - -(57) CometSortMergeJoin -Left output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Right output [4]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: [ws_order_number#42, ws_item_sk#41], [wr_order_number#54, wr_item_sk#53], LeftOuter - -(58) CometProject -Input [13]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52, wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: [d_year#52, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, sales_cnt#58, sales_amt#59], [d_year#52, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, (ws_quantity#43 - coalesce(wr_return_quantity#55, 0)) AS sales_cnt#58, (ws_ext_sales_price#44 - coalesce(wr_return_amt#56, 0.00)) AS sales_amt#59] - -(59) CometUnion -Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#33, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, sales_cnt#39, sales_amt#40] -Child 2 Input [7]: [d_year#52, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, sales_cnt#58, sales_amt#59] - -(60) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(61) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(62) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(63) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(64) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#60, sum#61] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(65) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#60, sum#61] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(66) CometFilter -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Condition : isnotnull(sales_cnt#62) - -(67) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(68) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] - -(69) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#68), dynamicpruningexpression(cs_sold_date_sk#68 IN dynamicpruning#69)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(70) ColumnarToRow [codegen id : 12] -Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] - -(71) Filter [codegen id : 12] -Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] -Condition : isnotnull(cs_item_sk#64) - -(72) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] - -(73) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cs_item_sk#64] -Right keys [1]: [i_item_sk#70] -Join type: Inner -Join condition: None - -(74) Project [codegen id : 12] -Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -Input [10]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] - -(75) ReusedExchange [Reuses operator id: 135] -Output [2]: [d_date_sk#75, d_year#76] - -(76) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cs_sold_date_sk#68] -Right keys [1]: [d_date_sk#75] -Join type: Inner -Join condition: None - -(77) Project [codegen id : 12] -Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Input [11]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_date_sk#75, d_year#76] - -(78) CometColumnarExchange -Input [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Arguments: hashpartitioning(cs_order_number#65, cs_item_sk#64, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(79) CometSort -Input [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Arguments: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76], [cs_order_number#65 ASC NULLS FIRST, cs_item_sk#64 ASC NULLS FIRST] - -(80) ReusedExchange [Reuses operator id: 19] -Output [4]: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] - -(81) CometSort -Input [4]: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] -Arguments: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80], [cr_order_number#78 ASC NULLS FIRST, cr_item_sk#77 ASC NULLS FIRST] - -(82) CometSortMergeJoin -Left output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Right output [4]: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] -Arguments: [cs_order_number#65, cs_item_sk#64], [cr_order_number#78, cr_item_sk#77], LeftOuter - -(83) CometProject -Input [13]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76, cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] -Arguments: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21], [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, (cs_quantity#66 - coalesce(cr_return_quantity#79, 0)) AS sales_cnt#20, (cs_ext_sales_price#67 - coalesce(cr_return_amount#80, 0.00)) AS sales_amt#21] - -(84) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#85), dynamicpruningexpression(ss_sold_date_sk#85 IN dynamicpruning#69)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(85) ColumnarToRow [codegen id : 15] -Input [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85] - -(86) Filter [codegen id : 15] -Input [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85] -Condition : isnotnull(ss_item_sk#81) - -(87) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] - -(88) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ss_item_sk#81] -Right keys [1]: [i_item_sk#86] -Join type: Inner -Join condition: None - -(89) Project [codegen id : 15] -Output [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] -Input [10]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] - -(90) ReusedExchange [Reuses operator id: 135] -Output [2]: [d_date_sk#91, d_year#92] - -(91) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ss_sold_date_sk#85] -Right keys [1]: [d_date_sk#91] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 15] -Output [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92] -Input [11]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_date_sk#91, d_year#92] - -(93) CometColumnarExchange -Input [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92] -Arguments: hashpartitioning(ss_ticket_number#82, ss_item_sk#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(94) CometSort -Input [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92] -Arguments: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92], [ss_ticket_number#82 ASC NULLS FIRST, ss_item_sk#81 ASC NULLS FIRST] - -(95) ReusedExchange [Reuses operator id: 37] -Output [4]: [sr_item_sk#93, sr_ticket_number#94, sr_return_quantity#95, sr_return_amt#96] - -(96) CometSort -Input [4]: [sr_item_sk#93, sr_ticket_number#94, sr_return_quantity#95, sr_return_amt#96] -Arguments: [sr_item_sk#93, sr_ticket_number#94, sr_return_quantity#95, sr_return_amt#96], [sr_ticket_number#94 ASC NULLS FIRST, sr_item_sk#93 ASC NULLS FIRST] - -(97) CometSortMergeJoin -Left output [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92] -Right output [4]: [sr_item_sk#93, sr_ticket_number#94, sr_return_quantity#95, sr_return_amt#96] -Arguments: [ss_ticket_number#82, ss_item_sk#81], [sr_ticket_number#94, sr_item_sk#93], LeftOuter - -(98) CometProject -Input [13]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92, sr_item_sk#93, sr_ticket_number#94, sr_return_quantity#95, sr_return_amt#96] -Arguments: [d_year#92, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, sales_cnt#39, sales_amt#40], [d_year#92, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, (ss_quantity#83 - coalesce(sr_return_quantity#95, 0)) AS sales_cnt#39, (ss_ext_sales_price#84 - coalesce(sr_return_amt#96, 0.00)) AS sales_amt#40] - -(99) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#101), dynamicpruningexpression(ws_sold_date_sk#101 IN dynamicpruning#69)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(100) ColumnarToRow [codegen id : 18] -Input [5]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101] - -(101) Filter [codegen id : 18] -Input [5]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101] -Condition : isnotnull(ws_item_sk#97) - -(102) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#102, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] - -(103) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_item_sk#97] -Right keys [1]: [i_item_sk#102] -Join type: Inner -Join condition: None - -(104) Project [codegen id : 18] -Output [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] -Input [10]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_item_sk#102, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] - -(105) ReusedExchange [Reuses operator id: 135] -Output [2]: [d_date_sk#107, d_year#108] - -(106) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#101] -Right keys [1]: [d_date_sk#107] -Join type: Inner -Join condition: None - -(107) Project [codegen id : 18] -Output [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108] -Input [11]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_date_sk#107, d_year#108] - -(108) CometColumnarExchange -Input [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108] -Arguments: hashpartitioning(ws_order_number#98, ws_item_sk#97, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(109) CometSort -Input [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108] -Arguments: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108], [ws_order_number#98 ASC NULLS FIRST, ws_item_sk#97 ASC NULLS FIRST] - -(110) ReusedExchange [Reuses operator id: 55] -Output [4]: [wr_item_sk#109, wr_order_number#110, wr_return_quantity#111, wr_return_amt#112] - -(111) CometSort -Input [4]: [wr_item_sk#109, wr_order_number#110, wr_return_quantity#111, wr_return_amt#112] -Arguments: [wr_item_sk#109, wr_order_number#110, wr_return_quantity#111, wr_return_amt#112], [wr_order_number#110 ASC NULLS FIRST, wr_item_sk#109 ASC NULLS FIRST] - -(112) CometSortMergeJoin -Left output [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108] -Right output [4]: [wr_item_sk#109, wr_order_number#110, wr_return_quantity#111, wr_return_amt#112] -Arguments: [ws_order_number#98, ws_item_sk#97], [wr_order_number#110, wr_item_sk#109], LeftOuter - -(113) CometProject -Input [13]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108, wr_item_sk#109, wr_order_number#110, wr_return_quantity#111, wr_return_amt#112] -Arguments: [d_year#108, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, sales_cnt#58, sales_amt#59], [d_year#108, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, (ws_quantity#99 - coalesce(wr_return_quantity#111, 0)) AS sales_cnt#58, (ws_ext_sales_price#100 - coalesce(wr_return_amt#112, 0.00)) AS sales_amt#59] - -(114) CometUnion -Child 0 Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#92, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, sales_cnt#39, sales_amt#40] -Child 2 Input [7]: [d_year#108, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, sales_cnt#58, sales_amt#59] - -(115) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Functions: [] - -(116) CometExchange -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(117) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Functions: [] - -(118) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(119) CometExchange -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sum#60, sum#113] -Arguments: hashpartitioning(d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(120) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sum#60, sum#113] -Keys [5]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(121) CometFilter -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115] -Condition : isnotnull(sales_cnt#114) - -(122) CometExchange -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115] -Arguments: hashpartitioning(i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=16] - -(123) CometSort -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115] -Arguments: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115], [i_brand_id#71 ASC NULLS FIRST, i_class_id#72 ASC NULLS FIRST, i_category_id#73 ASC NULLS FIRST, i_manufact_id#74 ASC NULLS FIRST] - -(124) CometSortMergeJoin -Left output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Right output [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115] -Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74], Inner, ((cast(sales_cnt#62 as decimal(17,2)) / cast(sales_cnt#114 as decimal(17,2))) < 0.90000000000000000000) - -(125) CometProject -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115] -Arguments: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121], [d_year#76 AS prev_year#116, d_year#14 AS year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#114 AS prev_yr_cnt#118, sales_cnt#62 AS curr_yr_cnt#119, (sales_cnt#62 - sales_cnt#114) AS sales_cnt_diff#120, (sales_amt#63 - sales_amt#115) AS sales_amt_diff#121] - -(126) CometTakeOrderedAndProject -Input [10]: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#120 ASC NULLS FIRST], output=[prev_year#116,year#117,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#118,curr_yr_cnt#119,sales_cnt_diff#120,sales_amt_diff#121]), [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121], 100, 0, [sales_cnt_diff#120 ASC NULLS FIRST], [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] - -(127) CometColumnarToRow [codegen id : 19] -Input [10]: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (131) -+- * CometColumnarToRow (130) - +- CometFilter (129) - +- CometNativeScan parquet spark_catalog.default.date_dim (128) - - -(128) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(129) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(130) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_year#14] - -(131) BroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#6 - -Subquery:4 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#68 IN dynamicpruning#69 -BroadcastExchange (135) -+- * CometColumnarToRow (134) - +- CometFilter (133) - +- CometNativeScan parquet spark_catalog.default.date_dim (132) - - -(132) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#75, d_year#76] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(133) CometFilter -Input [2]: [d_date_sk#75, d_year#76] -Condition : ((isnotnull(d_year#76) AND (d_year#76 = 2001)) AND isnotnull(d_date_sk#75)) - -(134) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#75, d_year#76] - -(135) BroadcastExchange -Input [2]: [d_date_sk#75, d_year#76] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] - -Subquery:5 Hosting operator id = 84 Hosting Expression = ss_sold_date_sk#85 IN dynamicpruning#69 - -Subquery:6 Hosting operator id = 99 Hosting Expression = ws_sold_date_sk#101 IN dynamicpruning#69 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/simplified.txt deleted file mode 100644 index aaf98f4b09..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/simplified.txt +++ /dev/null @@ -1,173 +0,0 @@ -WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometColumnarExchange [cs_order_number,cs_item_sk] #4 - WholeStageCodegen (3) - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [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] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #7 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 - WholeStageCodegen (6) - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #9 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometColumnarExchange [ws_order_number,ws_item_sk] #10 - WholeStageCodegen (9) - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #11 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometColumnarExchange [cs_order_number,cs_item_sk] #15 - WholeStageCodegen (12) - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 #16 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 - WholeStageCodegen (15) - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometColumnarExchange [ws_order_number,ws_item_sk] #18 - WholeStageCodegen (18) - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/explain.txt deleted file mode 100644 index 7afb1e3be6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,742 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (124) -+- CometTakeOrderedAndProject (123) - +- CometProject (122) - +- CometSortMergeJoin (121) - :- CometSort (66) - : +- CometExchange (65) - : +- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometHashAggregate (60) - : +- CometExchange (59) - : +- CometHashAggregate (58) - : +- CometUnion (57) - : :- CometProject (22) - : : +- CometSortMergeJoin (21) - : : :- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometSort (20) - : : +- CometExchange (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (39) - : : +- CometSortMergeJoin (38) - : : :- CometSort (32) - : : : +- CometExchange (31) - : : : +- CometProject (30) - : : : +- CometBroadcastHashJoin (29) - : : : :- CometProject (27) - : : : : +- CometBroadcastHashJoin (26) - : : : : :- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (25) - : : : +- ReusedExchange (28) - : : +- CometSort (37) - : : +- CometExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (33) - : +- CometProject (56) - : +- CometSortMergeJoin (55) - : :- CometSort (49) - : : +- CometExchange (48) - : : +- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometProject (44) - : : : +- CometBroadcastHashJoin (43) - : : : :- CometFilter (41) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) - : : : +- ReusedExchange (42) - : : +- ReusedExchange (45) - : +- CometSort (54) - : +- CometExchange (53) - : +- CometProject (52) - : +- CometFilter (51) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (50) - +- CometSort (120) - +- CometExchange (119) - +- CometFilter (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometHashAggregate (114) - +- CometExchange (113) - +- CometHashAggregate (112) - +- CometUnion (111) - :- CometProject (82) - : +- CometSortMergeJoin (81) - : :- CometSort (78) - : : +- CometExchange (77) - : : +- CometProject (76) - : : +- CometBroadcastHashJoin (75) - : : :- CometProject (71) - : : : +- CometBroadcastHashJoin (70) - : : : :- CometFilter (68) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) - : : : +- ReusedExchange (69) - : : +- CometBroadcastExchange (74) - : : +- CometFilter (73) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - : +- CometSort (80) - : +- ReusedExchange (79) - :- CometProject (96) - : +- CometSortMergeJoin (95) - : :- CometSort (92) - : : +- CometExchange (91) - : : +- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (87) - : : : +- CometBroadcastHashJoin (86) - : : : :- CometFilter (84) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (83) - : : : +- ReusedExchange (85) - : : +- ReusedExchange (88) - : +- CometSort (94) - : +- ReusedExchange (93) - +- CometProject (110) - +- CometSortMergeJoin (109) - :- CometSort (106) - : +- CometExchange (105) - : +- CometProject (104) - : +- CometBroadcastHashJoin (103) - : :- CometProject (101) - : : +- CometBroadcastHashJoin (100) - : : :- CometFilter (98) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (97) - : : +- ReusedExchange (99) - : +- ReusedExchange (102) - +- CometSort (108) - +- ReusedExchange (107) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) = Books ) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) - -(5) CometProject -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(6) CometBroadcastExchange -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(7) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Right output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: [d_date_sk#13, d_year#14] - -(12) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Right output [2]: [d_date_sk#13, d_year#14] -Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight - -(13) CometProject -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] - -(14) CometExchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(15) CometSort -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) - -(18) CometProject -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(19) CometExchange -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter - -(22) CometProject -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21], [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(24) CometFilter -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#22) - -(25) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(26) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Right output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22], [i_item_sk#28], Inner, BuildRight - -(27) CometProject -Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(28) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#33, d_year#34] - -(29) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Right output [2]: [d_date_sk#33, d_year#34] -Arguments: [ss_sold_date_sk#26], [d_date_sk#33], Inner, BuildRight - -(30) CometProject -Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_date_sk#33, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] - -(31) CometExchange -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometSort -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) - -(35) CometProject -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] - -(36) CometExchange -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(37) CometSort -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] - -(38) CometSortMergeJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter - -(39) CometProject -Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#47)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(41) CometFilter -Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Condition : isnotnull(ws_item_sk#42) - -(42) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(43) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Right output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42], [i_item_sk#48], Inner, BuildRight - -(44) CometProject -Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(45) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#53, d_year#54] - -(46) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [ws_sold_date_sk#46], [d_date_sk#53], Inner, BuildRight - -(47) CometProject -Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] - -(48) CometExchange -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(49) CometSort -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) - -(52) CometProject -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] - -(53) CometExchange -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(54) CometSort -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] - -(55) CometSortMergeJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter - -(56) CometProject -Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] - -(57) CometUnion -Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] - -(58) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(59) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(60) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(61) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(62) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(63) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(64) CometFilter -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Condition : isnotnull(sales_cnt#64) - -(65) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(66) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(68) CometFilter -Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Condition : isnotnull(cs_item_sk#66) - -(69) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(70) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Right output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66], [i_item_sk#72], Inner, BuildRight - -(71) CometProject -Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(74) CometBroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: [d_date_sk#77, d_year#78] - -(75) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Right output [2]: [d_date_sk#77, d_year#78] -Arguments: [cs_sold_date_sk#70], [d_date_sk#77], Inner, BuildRight - -(76) CometProject -Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] - -(77) CometExchange -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(78) CometSort -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] - -(79) ReusedExchange [Reuses operator id: 19] -Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] - -(80) CometSort -Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] - -(81) CometSortMergeJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter - -(82) CometProject -Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#20, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#21] - -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(84) CometFilter -Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Condition : isnotnull(ss_item_sk#83) - -(85) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] - -(86) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Right output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Arguments: [ss_item_sk#83], [i_item_sk#89], Inner, BuildRight - -(87) CometProject -Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] - -(88) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#94, d_year#95] - -(89) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Right output [2]: [d_date_sk#94, d_year#95] -Arguments: [ss_sold_date_sk#87], [d_date_sk#94], Inner, BuildRight - -(90) CometProject -Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_date_sk#94, d_year#95] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] - -(91) CometExchange -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(92) CometSort -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_ticket_number#84 ASC NULLS FIRST, ss_item_sk#83 ASC NULLS FIRST] - -(93) ReusedExchange [Reuses operator id: 36] -Output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] - -(94) CometSort -Input [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99], [sr_ticket_number#97 ASC NULLS FIRST, sr_item_sk#96 ASC NULLS FIRST] - -(95) CometSortMergeJoin -Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Right output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [ss_ticket_number#84, ss_item_sk#83], [sr_ticket_number#97, sr_item_sk#96], LeftOuter - -(96) CometProject -Input [13]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95, sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41], [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, (ss_quantity#85 - coalesce(sr_return_quantity#98, 0)) AS sales_cnt#40, (ss_ext_sales_price#86 - coalesce(sr_return_amt#99, 0.00)) AS sales_amt#41] - -(97) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_sold_date_sk#104 IN dynamicpruning#105)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(98) CometFilter -Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Condition : isnotnull(ws_item_sk#100) - -(99) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] - -(100) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Right output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Arguments: [ws_item_sk#100], [i_item_sk#106], Inner, BuildRight - -(101) CometProject -Input [10]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] - -(102) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#111, d_year#112] - -(103) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Right output [2]: [d_date_sk#111, d_year#112] -Arguments: [ws_sold_date_sk#104], [d_date_sk#111], Inner, BuildRight - -(104) CometProject -Input [11]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] - -(105) CometExchange -Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(106) CometSort -Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_order_number#101 ASC NULLS FIRST, ws_item_sk#100 ASC NULLS FIRST] - -(107) ReusedExchange [Reuses operator id: 53] -Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] - -(108) CometSort -Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116], [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST] - -(109) CometSortMergeJoin -Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Right output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [ws_order_number#101, ws_item_sk#100], [wr_order_number#114, wr_item_sk#113], LeftOuter - -(110) CometProject -Input [13]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61], [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#102 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#60, (ws_ext_sales_price#103 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#61] - -(111) CometUnion -Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61] - -(112) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Functions: [] - -(113) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] - -(114) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Functions: [] - -(115) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(116) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(117) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(118) CometFilter -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Condition : isnotnull(sales_cnt#118) - -(119) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(120) CometSort -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] - -(121) CometSortMergeJoin -Left output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Right output [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], Inner, ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#118 as decimal(17,2))) < 0.90000000000000000000) - -(122) CometProject -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], [d_year#78 AS prev_year#120, d_year#14 AS year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#118 AS prev_yr_cnt#122, sales_cnt#64 AS curr_yr_cnt#123, (sales_cnt#64 - sales_cnt#118) AS sales_cnt_diff#124, (sales_amt#65 - sales_amt#119) AS sales_amt_diff#125] - -(123) CometTakeOrderedAndProject -Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#124 ASC NULLS FIRST], output=[prev_year#120,year#121,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#122,curr_yr_cnt#123,sales_cnt_diff#124,sales_amt_diff#125]), [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], 100, 0, [sales_cnt_diff#124 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] - -(124) CometColumnarToRow [codegen id : 1] -Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (128) -+- * CometColumnarToRow (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) - - -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(127) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_year#14] - -(128) BroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 - -Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 -BroadcastExchange (132) -+- * CometColumnarToRow (131) - +- CometFilter (130) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) - - -(129) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(130) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(131) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#77, d_year#78] - -(132) BroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] - -Subquery:5 Hosting operator id = 83 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 - -Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/simplified.txt deleted file mode 100644 index d0520c8b63..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,144 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #4 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange [d_date_sk,d_year] #7 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #8 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #9 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #10 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #11 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #12 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #16 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_year] #18 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #19 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #20 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt deleted file mode 100644 index 7afb1e3be6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt +++ /dev/null @@ -1,742 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (124) -+- CometTakeOrderedAndProject (123) - +- CometProject (122) - +- CometSortMergeJoin (121) - :- CometSort (66) - : +- CometExchange (65) - : +- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometHashAggregate (60) - : +- CometExchange (59) - : +- CometHashAggregate (58) - : +- CometUnion (57) - : :- CometProject (22) - : : +- CometSortMergeJoin (21) - : : :- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometSort (20) - : : +- CometExchange (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (39) - : : +- CometSortMergeJoin (38) - : : :- CometSort (32) - : : : +- CometExchange (31) - : : : +- CometProject (30) - : : : +- CometBroadcastHashJoin (29) - : : : :- CometProject (27) - : : : : +- CometBroadcastHashJoin (26) - : : : : :- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (25) - : : : +- ReusedExchange (28) - : : +- CometSort (37) - : : +- CometExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (33) - : +- CometProject (56) - : +- CometSortMergeJoin (55) - : :- CometSort (49) - : : +- CometExchange (48) - : : +- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometProject (44) - : : : +- CometBroadcastHashJoin (43) - : : : :- CometFilter (41) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) - : : : +- ReusedExchange (42) - : : +- ReusedExchange (45) - : +- CometSort (54) - : +- CometExchange (53) - : +- CometProject (52) - : +- CometFilter (51) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (50) - +- CometSort (120) - +- CometExchange (119) - +- CometFilter (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometHashAggregate (114) - +- CometExchange (113) - +- CometHashAggregate (112) - +- CometUnion (111) - :- CometProject (82) - : +- CometSortMergeJoin (81) - : :- CometSort (78) - : : +- CometExchange (77) - : : +- CometProject (76) - : : +- CometBroadcastHashJoin (75) - : : :- CometProject (71) - : : : +- CometBroadcastHashJoin (70) - : : : :- CometFilter (68) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) - : : : +- ReusedExchange (69) - : : +- CometBroadcastExchange (74) - : : +- CometFilter (73) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - : +- CometSort (80) - : +- ReusedExchange (79) - :- CometProject (96) - : +- CometSortMergeJoin (95) - : :- CometSort (92) - : : +- CometExchange (91) - : : +- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (87) - : : : +- CometBroadcastHashJoin (86) - : : : :- CometFilter (84) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (83) - : : : +- ReusedExchange (85) - : : +- ReusedExchange (88) - : +- CometSort (94) - : +- ReusedExchange (93) - +- CometProject (110) - +- CometSortMergeJoin (109) - :- CometSort (106) - : +- CometExchange (105) - : +- CometProject (104) - : +- CometBroadcastHashJoin (103) - : :- CometProject (101) - : : +- CometBroadcastHashJoin (100) - : : :- CometFilter (98) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (97) - : : +- ReusedExchange (99) - : +- ReusedExchange (102) - +- CometSort (108) - +- ReusedExchange (107) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) = Books ) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) - -(5) CometProject -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(6) CometBroadcastExchange -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(7) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Right output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: [d_date_sk#13, d_year#14] - -(12) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Right output [2]: [d_date_sk#13, d_year#14] -Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight - -(13) CometProject -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] - -(14) CometExchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(15) CometSort -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) - -(18) CometProject -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(19) CometExchange -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter - -(22) CometProject -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21], [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(24) CometFilter -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#22) - -(25) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(26) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Right output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22], [i_item_sk#28], Inner, BuildRight - -(27) CometProject -Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(28) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#33, d_year#34] - -(29) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Right output [2]: [d_date_sk#33, d_year#34] -Arguments: [ss_sold_date_sk#26], [d_date_sk#33], Inner, BuildRight - -(30) CometProject -Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_date_sk#33, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] - -(31) CometExchange -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometSort -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) - -(35) CometProject -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] - -(36) CometExchange -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(37) CometSort -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] - -(38) CometSortMergeJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter - -(39) CometProject -Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#47)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(41) CometFilter -Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Condition : isnotnull(ws_item_sk#42) - -(42) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(43) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Right output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42], [i_item_sk#48], Inner, BuildRight - -(44) CometProject -Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(45) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#53, d_year#54] - -(46) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [ws_sold_date_sk#46], [d_date_sk#53], Inner, BuildRight - -(47) CometProject -Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] - -(48) CometExchange -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(49) CometSort -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) - -(52) CometProject -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] - -(53) CometExchange -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(54) CometSort -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] - -(55) CometSortMergeJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter - -(56) CometProject -Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] - -(57) CometUnion -Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] - -(58) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(59) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(60) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(61) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(62) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(63) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(64) CometFilter -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Condition : isnotnull(sales_cnt#64) - -(65) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(66) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(68) CometFilter -Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Condition : isnotnull(cs_item_sk#66) - -(69) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(70) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Right output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66], [i_item_sk#72], Inner, BuildRight - -(71) CometProject -Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(74) CometBroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: [d_date_sk#77, d_year#78] - -(75) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Right output [2]: [d_date_sk#77, d_year#78] -Arguments: [cs_sold_date_sk#70], [d_date_sk#77], Inner, BuildRight - -(76) CometProject -Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] - -(77) CometExchange -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(78) CometSort -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] - -(79) ReusedExchange [Reuses operator id: 19] -Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] - -(80) CometSort -Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] - -(81) CometSortMergeJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter - -(82) CometProject -Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#20, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#21] - -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(84) CometFilter -Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Condition : isnotnull(ss_item_sk#83) - -(85) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] - -(86) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Right output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Arguments: [ss_item_sk#83], [i_item_sk#89], Inner, BuildRight - -(87) CometProject -Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] - -(88) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#94, d_year#95] - -(89) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Right output [2]: [d_date_sk#94, d_year#95] -Arguments: [ss_sold_date_sk#87], [d_date_sk#94], Inner, BuildRight - -(90) CometProject -Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_date_sk#94, d_year#95] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] - -(91) CometExchange -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(92) CometSort -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_ticket_number#84 ASC NULLS FIRST, ss_item_sk#83 ASC NULLS FIRST] - -(93) ReusedExchange [Reuses operator id: 36] -Output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] - -(94) CometSort -Input [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99], [sr_ticket_number#97 ASC NULLS FIRST, sr_item_sk#96 ASC NULLS FIRST] - -(95) CometSortMergeJoin -Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Right output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [ss_ticket_number#84, ss_item_sk#83], [sr_ticket_number#97, sr_item_sk#96], LeftOuter - -(96) CometProject -Input [13]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95, sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41], [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, (ss_quantity#85 - coalesce(sr_return_quantity#98, 0)) AS sales_cnt#40, (ss_ext_sales_price#86 - coalesce(sr_return_amt#99, 0.00)) AS sales_amt#41] - -(97) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_sold_date_sk#104 IN dynamicpruning#105)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(98) CometFilter -Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Condition : isnotnull(ws_item_sk#100) - -(99) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] - -(100) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Right output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Arguments: [ws_item_sk#100], [i_item_sk#106], Inner, BuildRight - -(101) CometProject -Input [10]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] - -(102) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#111, d_year#112] - -(103) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Right output [2]: [d_date_sk#111, d_year#112] -Arguments: [ws_sold_date_sk#104], [d_date_sk#111], Inner, BuildRight - -(104) CometProject -Input [11]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] - -(105) CometExchange -Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(106) CometSort -Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_order_number#101 ASC NULLS FIRST, ws_item_sk#100 ASC NULLS FIRST] - -(107) ReusedExchange [Reuses operator id: 53] -Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] - -(108) CometSort -Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116], [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST] - -(109) CometSortMergeJoin -Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Right output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [ws_order_number#101, ws_item_sk#100], [wr_order_number#114, wr_item_sk#113], LeftOuter - -(110) CometProject -Input [13]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61], [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#102 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#60, (ws_ext_sales_price#103 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#61] - -(111) CometUnion -Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61] - -(112) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Functions: [] - -(113) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] - -(114) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Functions: [] - -(115) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(116) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(117) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(118) CometFilter -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Condition : isnotnull(sales_cnt#118) - -(119) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(120) CometSort -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] - -(121) CometSortMergeJoin -Left output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Right output [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], Inner, ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#118 as decimal(17,2))) < 0.90000000000000000000) - -(122) CometProject -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], [d_year#78 AS prev_year#120, d_year#14 AS year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#118 AS prev_yr_cnt#122, sales_cnt#64 AS curr_yr_cnt#123, (sales_cnt#64 - sales_cnt#118) AS sales_cnt_diff#124, (sales_amt#65 - sales_amt#119) AS sales_amt_diff#125] - -(123) CometTakeOrderedAndProject -Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#124 ASC NULLS FIRST], output=[prev_year#120,year#121,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#122,curr_yr_cnt#123,sales_cnt_diff#124,sales_amt_diff#125]), [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], 100, 0, [sales_cnt_diff#124 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] - -(124) CometColumnarToRow [codegen id : 1] -Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (128) -+- * CometColumnarToRow (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) - - -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(127) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_year#14] - -(128) BroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 - -Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 -BroadcastExchange (132) -+- * CometColumnarToRow (131) - +- CometFilter (130) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) - - -(129) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(130) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(131) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#77, d_year#78] - -(132) BroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] - -Subquery:5 Hosting operator id = 83 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 - -Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/extended.txt deleted file mode 100644 index 22ccc17641..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/extended.txt +++ /dev/null @@ -1,172 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : +- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - +- CometSort - +- CometExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - -Comet accelerated 159 out of 167 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d0520c8b63..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/simplified.txt +++ /dev/null @@ -1,144 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #4 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange [d_date_sk,d_year] #7 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #8 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #9 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #10 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #11 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #12 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #16 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_year] #18 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #19 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #20 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_datafusion/explain.txt deleted file mode 100644 index f04f5a8a5a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_datafusion/explain.txt +++ /dev/null @@ -1,200 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (35) -+- CometTakeOrderedAndProject (34) - +- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometUnion (30) - :- CometProject (13) - : +- CometBroadcastHashJoin (12) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometNativeScan parquet spark_catalog.default.item (3) - : +- CometBroadcastExchange (11) - : +- CometFilter (10) - : +- CometNativeScan parquet spark_catalog.default.date_dim (9) - :- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometFilter (15) - : : : +- CometNativeScan parquet spark_catalog.default.web_sales (14) - : : +- ReusedExchange (16) - : +- ReusedExchange (19) - +- CometProject (29) - +- CometBroadcastHashJoin (28) - :- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometFilter (23) - : : +- CometNativeScan parquet spark_catalog.default.catalog_sales (22) - : +- ReusedExchange (24) - +- ReusedExchange (27) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ss_sold_date_sk#4)] -PushedFilters: [IsNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_category#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_category#6] -Condition : isnotnull(i_item_sk#5) - -(5) CometProject -Input [2]: [i_item_sk#5, i_category#6] -Arguments: [i_item_sk#5, i_category#7], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#6, 50, true, false, true) AS i_category#7] - -(6) CometBroadcastExchange -Input [2]: [i_item_sk#5, i_category#7] -Arguments: [i_item_sk#5, i_category#7] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [2]: [i_item_sk#5, i_category#7] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_item_sk#5, i_category#7] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7], [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7] - -(9) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Condition : isnotnull(d_date_sk#8) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [d_date_sk#8, d_year#9, d_qoy#10] - -(12) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7] -Right output [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [ss_sold_date_sk#4], [d_date_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7, d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13], [store AS channel#11, ss_store_sk#2 AS col_name#12, d_year#9, d_qoy#10, i_category#7, ss_ext_sales_price#3 AS ext_sales_price#13] - -(14) CometNativeScan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(ws_sold_date_sk#17)] -PushedFilters: [IsNull(ws_ship_customer_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Condition : (isnull(ws_ship_customer_sk#15) AND isnotnull(ws_item_sk#14)) - -(16) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#18, i_category#19] - -(17) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Right output [2]: [i_item_sk#18, i_category#19] -Arguments: [ws_item_sk#14], [i_item_sk#18], Inner, BuildRight - -(18) CometProject -Input [6]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_item_sk#18, i_category#19] -Arguments: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19], [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19] - -(19) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#20, d_year#21, d_qoy#22] - -(20) CometBroadcastHashJoin -Left output [4]: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19] -Right output [3]: [d_date_sk#20, d_year#21, d_qoy#22] -Arguments: [ws_sold_date_sk#17], [d_date_sk#20], Inner, BuildRight - -(21) CometProject -Input [7]: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19, d_date_sk#20, d_year#21, d_qoy#22] -Arguments: [channel#23, col_name#24, d_year#21, d_qoy#22, i_category#19, ext_sales_price#25], [web AS channel#23, ws_ship_customer_sk#15 AS col_name#24, d_year#21, d_qoy#22, i_category#19, ws_ext_sales_price#16 AS ext_sales_price#25] - -(22) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(cs_sold_date_sk#29)] -PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(23) CometFilter -Input [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Condition : (isnull(cs_ship_addr_sk#26) AND isnotnull(cs_item_sk#27)) - -(24) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#30, i_category#31] - -(25) CometBroadcastHashJoin -Left output [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Right output [2]: [i_item_sk#30, i_category#31] -Arguments: [cs_item_sk#27], [i_item_sk#30], Inner, BuildRight - -(26) CometProject -Input [6]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29, i_item_sk#30, i_category#31] -Arguments: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31], [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31] - -(27) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#32, d_year#33, d_qoy#34] - -(28) CometBroadcastHashJoin -Left output [4]: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31] -Right output [3]: [d_date_sk#32, d_year#33, d_qoy#34] -Arguments: [cs_sold_date_sk#29], [d_date_sk#32], Inner, BuildRight - -(29) CometProject -Input [7]: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31, d_date_sk#32, d_year#33, d_qoy#34] -Arguments: [channel#35, col_name#36, d_year#33, d_qoy#34, i_category#31, ext_sales_price#37], [catalog AS channel#35, cs_ship_addr_sk#26 AS col_name#36, d_year#33, d_qoy#34, i_category#31, cs_ext_sales_price#28 AS ext_sales_price#37] - -(30) CometUnion -Child 0 Input [6]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13] -Child 1 Input [6]: [channel#23, col_name#24, d_year#21, d_qoy#22, i_category#19, ext_sales_price#25] -Child 2 Input [6]: [channel#35, col_name#36, d_year#33, d_qoy#34, i_category#31, ext_sales_price#37] - -(31) CometHashAggregate -Input [6]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13] -Keys [5]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7] -Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#13))] - -(32) CometExchange -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] -Arguments: hashpartitioning(channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(33) CometHashAggregate -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] -Keys [5]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7] -Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#13))] - -(34) CometTakeOrderedAndProject -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#11 ASC NULLS FIRST,col_name#12 ASC NULLS FIRST,d_year#9 ASC NULLS FIRST,d_qoy#10 ASC NULLS FIRST,i_category#7 ASC NULLS FIRST], output=[channel#11,col_name#12,d_year#9,d_qoy#10,i_category#7,sales_cnt#40,sales_amt#41]), [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41], 100, 0, [channel#11 ASC NULLS FIRST, col_name#12 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#7 ASC NULLS FIRST], [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] - -(35) CometColumnarToRow [codegen id : 1] -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_datafusion/simplified.txt deleted file mode 100644 index abb482c3d5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_datafusion/simplified.txt +++ /dev/null @@ -1,37 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] - CometHashAggregate [count,sum] [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt,count(1),sum(UnscaledValue(ext_sales_price))] - CometExchange [channel,col_name,d_year,d_qoy,i_category] #1 - CometHashAggregate [ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,count,sum] - 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_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,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] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category] #2 - CometProject [i_category] [i_item_sk,i_category] - CometFilter [i_item_sk,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan 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_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,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] - CometNativeScan 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_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_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] - CometNativeScan 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/q76.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_iceberg_compat/explain.txt deleted file mode 100644 index d862a6cb29..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,200 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (35) -+- CometTakeOrderedAndProject (34) - +- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometUnion (30) - :- CometProject (13) - : +- CometBroadcastHashJoin (12) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : +- CometBroadcastExchange (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - :- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometFilter (15) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (14) - : : +- ReusedExchange (16) - : +- ReusedExchange (19) - +- CometProject (29) - +- CometBroadcastHashJoin (28) - :- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometFilter (23) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (22) - : +- ReusedExchange (24) - +- ReusedExchange (27) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4)] -PushedFilters: [IsNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_category#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_category#6] -Condition : isnotnull(i_item_sk#5) - -(5) CometProject -Input [2]: [i_item_sk#5, i_category#6] -Arguments: [i_item_sk#5, i_category#7], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#6, 50, true, false, true) AS i_category#7] - -(6) CometBroadcastExchange -Input [2]: [i_item_sk#5, i_category#7] -Arguments: [i_item_sk#5, i_category#7] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [2]: [i_item_sk#5, i_category#7] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_item_sk#5, i_category#7] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7], [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Condition : isnotnull(d_date_sk#8) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [d_date_sk#8, d_year#9, d_qoy#10] - -(12) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7] -Right output [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [ss_sold_date_sk#4], [d_date_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7, d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13], [store AS channel#11, ss_store_sk#2 AS col_name#12, d_year#9, d_qoy#10, i_category#7, ss_ext_sales_price#3 AS ext_sales_price#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#17)] -PushedFilters: [IsNull(ws_ship_customer_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Condition : (isnull(ws_ship_customer_sk#15) AND isnotnull(ws_item_sk#14)) - -(16) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#18, i_category#19] - -(17) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Right output [2]: [i_item_sk#18, i_category#19] -Arguments: [ws_item_sk#14], [i_item_sk#18], Inner, BuildRight - -(18) CometProject -Input [6]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_item_sk#18, i_category#19] -Arguments: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19], [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19] - -(19) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#20, d_year#21, d_qoy#22] - -(20) CometBroadcastHashJoin -Left output [4]: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19] -Right output [3]: [d_date_sk#20, d_year#21, d_qoy#22] -Arguments: [ws_sold_date_sk#17], [d_date_sk#20], Inner, BuildRight - -(21) CometProject -Input [7]: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19, d_date_sk#20, d_year#21, d_qoy#22] -Arguments: [channel#23, col_name#24, d_year#21, d_qoy#22, i_category#19, ext_sales_price#25], [web AS channel#23, ws_ship_customer_sk#15 AS col_name#24, d_year#21, d_qoy#22, i_category#19, ws_ext_sales_price#16 AS ext_sales_price#25] - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#29)] -PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(23) CometFilter -Input [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Condition : (isnull(cs_ship_addr_sk#26) AND isnotnull(cs_item_sk#27)) - -(24) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#30, i_category#31] - -(25) CometBroadcastHashJoin -Left output [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Right output [2]: [i_item_sk#30, i_category#31] -Arguments: [cs_item_sk#27], [i_item_sk#30], Inner, BuildRight - -(26) CometProject -Input [6]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29, i_item_sk#30, i_category#31] -Arguments: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31], [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31] - -(27) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#32, d_year#33, d_qoy#34] - -(28) CometBroadcastHashJoin -Left output [4]: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31] -Right output [3]: [d_date_sk#32, d_year#33, d_qoy#34] -Arguments: [cs_sold_date_sk#29], [d_date_sk#32], Inner, BuildRight - -(29) CometProject -Input [7]: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31, d_date_sk#32, d_year#33, d_qoy#34] -Arguments: [channel#35, col_name#36, d_year#33, d_qoy#34, i_category#31, ext_sales_price#37], [catalog AS channel#35, cs_ship_addr_sk#26 AS col_name#36, d_year#33, d_qoy#34, i_category#31, cs_ext_sales_price#28 AS ext_sales_price#37] - -(30) CometUnion -Child 0 Input [6]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13] -Child 1 Input [6]: [channel#23, col_name#24, d_year#21, d_qoy#22, i_category#19, ext_sales_price#25] -Child 2 Input [6]: [channel#35, col_name#36, d_year#33, d_qoy#34, i_category#31, ext_sales_price#37] - -(31) CometHashAggregate -Input [6]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13] -Keys [5]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7] -Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#13))] - -(32) CometExchange -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] -Arguments: hashpartitioning(channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(33) CometHashAggregate -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] -Keys [5]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7] -Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#13))] - -(34) CometTakeOrderedAndProject -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#11 ASC NULLS FIRST,col_name#12 ASC NULLS FIRST,d_year#9 ASC NULLS FIRST,d_qoy#10 ASC NULLS FIRST,i_category#7 ASC NULLS FIRST], output=[channel#11,col_name#12,d_year#9,d_qoy#10,i_category#7,sales_cnt#40,sales_amt#41]), [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41], 100, 0, [channel#11 ASC NULLS FIRST, col_name#12 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#7 ASC NULLS FIRST], [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] - -(35) CometColumnarToRow [codegen id : 1] -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_iceberg_compat/simplified.txt deleted file mode 100644 index b5449831cb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,37 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] - CometHashAggregate [count,sum] [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt,count(1),sum(UnscaledValue(ext_sales_price))] - CometExchange [channel,col_name,d_year,d_qoy,i_category] #1 - CometHashAggregate [ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,count,sum] - 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_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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category] #2 - CometProject [i_category] [i_item_sk,i_category] - CometFilter [i_item_sk,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] 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_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,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 [native_iceberg_compat] 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_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_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 [native_iceberg_compat] 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/q76/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt deleted file mode 100644 index d862a6cb29..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt +++ /dev/null @@ -1,200 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (35) -+- CometTakeOrderedAndProject (34) - +- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometUnion (30) - :- CometProject (13) - : +- CometBroadcastHashJoin (12) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : +- CometBroadcastExchange (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - :- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometFilter (15) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (14) - : : +- ReusedExchange (16) - : +- ReusedExchange (19) - +- CometProject (29) - +- CometBroadcastHashJoin (28) - :- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometFilter (23) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (22) - : +- ReusedExchange (24) - +- ReusedExchange (27) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4)] -PushedFilters: [IsNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Condition : (isnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_category#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_category#6] -Condition : isnotnull(i_item_sk#5) - -(5) CometProject -Input [2]: [i_item_sk#5, i_category#6] -Arguments: [i_item_sk#5, i_category#7], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#6, 50, true, false, true) AS i_category#7] - -(6) CometBroadcastExchange -Input [2]: [i_item_sk#5, i_category#7] -Arguments: [i_item_sk#5, i_category#7] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] -Right output [2]: [i_item_sk#5, i_category#7] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_item_sk#5, i_category#7] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7], [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Condition : isnotnull(d_date_sk#8) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [d_date_sk#8, d_year#9, d_qoy#10] - -(12) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7] -Right output [3]: [d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [ss_sold_date_sk#4], [d_date_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#7, d_date_sk#8, d_year#9, d_qoy#10] -Arguments: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13], [store AS channel#11, ss_store_sk#2 AS col_name#12, d_year#9, d_qoy#10, i_category#7, ss_ext_sales_price#3 AS ext_sales_price#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#17)] -PushedFilters: [IsNull(ws_ship_customer_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Condition : (isnull(ws_ship_customer_sk#15) AND isnotnull(ws_item_sk#14)) - -(16) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#18, i_category#19] - -(17) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17] -Right output [2]: [i_item_sk#18, i_category#19] -Arguments: [ws_item_sk#14], [i_item_sk#18], Inner, BuildRight - -(18) CometProject -Input [6]: [ws_item_sk#14, ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_item_sk#18, i_category#19] -Arguments: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19], [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19] - -(19) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#20, d_year#21, d_qoy#22] - -(20) CometBroadcastHashJoin -Left output [4]: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19] -Right output [3]: [d_date_sk#20, d_year#21, d_qoy#22] -Arguments: [ws_sold_date_sk#17], [d_date_sk#20], Inner, BuildRight - -(21) CometProject -Input [7]: [ws_ship_customer_sk#15, ws_ext_sales_price#16, ws_sold_date_sk#17, i_category#19, d_date_sk#20, d_year#21, d_qoy#22] -Arguments: [channel#23, col_name#24, d_year#21, d_qoy#22, i_category#19, ext_sales_price#25], [web AS channel#23, ws_ship_customer_sk#15 AS col_name#24, d_year#21, d_qoy#22, i_category#19, ws_ext_sales_price#16 AS ext_sales_price#25] - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#29)] -PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(23) CometFilter -Input [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Condition : (isnull(cs_ship_addr_sk#26) AND isnotnull(cs_item_sk#27)) - -(24) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#30, i_category#31] - -(25) CometBroadcastHashJoin -Left output [4]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29] -Right output [2]: [i_item_sk#30, i_category#31] -Arguments: [cs_item_sk#27], [i_item_sk#30], Inner, BuildRight - -(26) CometProject -Input [6]: [cs_ship_addr_sk#26, cs_item_sk#27, cs_ext_sales_price#28, cs_sold_date_sk#29, i_item_sk#30, i_category#31] -Arguments: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31], [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31] - -(27) ReusedExchange [Reuses operator id: 11] -Output [3]: [d_date_sk#32, d_year#33, d_qoy#34] - -(28) CometBroadcastHashJoin -Left output [4]: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31] -Right output [3]: [d_date_sk#32, d_year#33, d_qoy#34] -Arguments: [cs_sold_date_sk#29], [d_date_sk#32], Inner, BuildRight - -(29) CometProject -Input [7]: [cs_ship_addr_sk#26, cs_ext_sales_price#28, cs_sold_date_sk#29, i_category#31, d_date_sk#32, d_year#33, d_qoy#34] -Arguments: [channel#35, col_name#36, d_year#33, d_qoy#34, i_category#31, ext_sales_price#37], [catalog AS channel#35, cs_ship_addr_sk#26 AS col_name#36, d_year#33, d_qoy#34, i_category#31, cs_ext_sales_price#28 AS ext_sales_price#37] - -(30) CometUnion -Child 0 Input [6]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13] -Child 1 Input [6]: [channel#23, col_name#24, d_year#21, d_qoy#22, i_category#19, ext_sales_price#25] -Child 2 Input [6]: [channel#35, col_name#36, d_year#33, d_qoy#34, i_category#31, ext_sales_price#37] - -(31) CometHashAggregate -Input [6]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, ext_sales_price#13] -Keys [5]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7] -Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#13))] - -(32) CometExchange -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] -Arguments: hashpartitioning(channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(33) CometHashAggregate -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] -Keys [5]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7] -Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#13))] - -(34) CometTakeOrderedAndProject -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#11 ASC NULLS FIRST,col_name#12 ASC NULLS FIRST,d_year#9 ASC NULLS FIRST,d_qoy#10 ASC NULLS FIRST,i_category#7 ASC NULLS FIRST], output=[channel#11,col_name#12,d_year#9,d_qoy#10,i_category#7,sales_cnt#40,sales_amt#41]), [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41], 100, 0, [channel#11 ASC NULLS FIRST, col_name#12 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#7 ASC NULLS FIRST], [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] - -(35) CometColumnarToRow [codegen id : 1] -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/extended.txt deleted file mode 100644 index b56a6590db..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/extended.txt +++ /dev/null @@ -1,47 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 44 out of 44 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index b5449831cb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/simplified.txt +++ /dev/null @@ -1,37 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] - CometHashAggregate [count,sum] [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt,count(1),sum(UnscaledValue(ext_sales_price))] - CometExchange [channel,col_name,d_year,d_qoy,i_category] #1 - CometHashAggregate [ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,count,sum] - 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_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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastExchange [i_item_sk,i_category] #2 - CometProject [i_category] [i_item_sk,i_category] - CometFilter [i_item_sk,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_category] - CometBroadcastExchange [d_date_sk,d_year,d_qoy] #3 - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] 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_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,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 [native_iceberg_compat] 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_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_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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/explain.txt deleted file mode 100644 index ae530b4900..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/explain.txt +++ /dev/null @@ -1,575 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (92) -+- * HashAggregate (91) - +- * CometColumnarToRow (90) - +- CometColumnarExchange (89) - +- * HashAggregate (88) - +- * Expand (87) - +- Union (86) - :- * Project (32) - : +- * BroadcastHashJoin LeftOuter BuildRight (31) - : :- * HashAggregate (16) - : : +- * CometColumnarToRow (15) - : : +- CometColumnarExchange (14) - : : +- * HashAggregate (13) - : : +- * Project (12) - : : +- * BroadcastHashJoin Inner BuildRight (11) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (10) - : : +- * CometColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (30) - : +- * HashAggregate (29) - : +- * CometColumnarToRow (28) - : +- CometColumnarExchange (27) - : +- * HashAggregate (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Filter (19) - : : : +- * ColumnarToRow (18) - : : : +- Scan parquet spark_catalog.default.store_returns (17) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - :- * Project (53) - : +- * BroadcastNestedLoopJoin Inner BuildLeft (52) - : :- BroadcastExchange (42) - : : +- * HashAggregate (41) - : : +- * CometColumnarToRow (40) - : : +- CometColumnarExchange (39) - : : +- * HashAggregate (38) - : : +- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * ColumnarToRow (34) - : : : +- Scan parquet spark_catalog.default.catalog_sales (33) - : : +- ReusedExchange (35) - : +- * HashAggregate (51) - : +- * CometColumnarToRow (50) - : +- CometColumnarExchange (49) - : +- * HashAggregate (48) - : +- * Project (47) - : +- * BroadcastHashJoin Inner BuildRight (46) - : :- * ColumnarToRow (44) - : : +- Scan parquet spark_catalog.default.catalog_returns (43) - : +- ReusedExchange (45) - +- * Project (85) - +- * BroadcastHashJoin LeftOuter BuildRight (84) - :- * HashAggregate (69) - : +- * CometColumnarToRow (68) - : +- CometColumnarExchange (67) - : +- * HashAggregate (66) - : +- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (59) - : : +- * BroadcastHashJoin Inner BuildRight (58) - : : :- * Filter (56) - : : : +- * ColumnarToRow (55) - : : : +- Scan parquet spark_catalog.default.web_sales (54) - : : +- ReusedExchange (57) - : +- BroadcastExchange (63) - : +- * CometColumnarToRow (62) - : +- CometFilter (61) - : +- CometNativeScan parquet spark_catalog.default.web_page (60) - +- BroadcastExchange (83) - +- * HashAggregate (82) - +- * CometColumnarToRow (81) - +- CometColumnarExchange (80) - +- * HashAggregate (79) - +- * Project (78) - +- * BroadcastHashJoin Inner BuildRight (77) - :- * Project (75) - : +- * BroadcastHashJoin Inner BuildRight (74) - : :- * Filter (72) - : : +- * ColumnarToRow (71) - : : +- Scan parquet spark_catalog.default.web_returns (70) - : +- ReusedExchange (73) - +- ReusedExchange (76) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 3] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(4) ReusedExchange [Reuses operator id: 97] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [1]: [s_store_sk#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [1]: [s_store_sk#7] -Condition : isnotnull(s_store_sk#7) - -(9) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#7] - -(10) BroadcastExchange -Input [1]: [s_store_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None - -(12) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] - -(13) HashAggregate [codegen id : 3] -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Keys [1]: [s_store_sk#7] -Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum#8, sum#9] -Results [3]: [s_store_sk#7, sum#10, sum#11] - -(14) CometColumnarExchange -Input [3]: [s_store_sk#7, sum#10, sum#11] -Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(15) CometColumnarToRow [codegen id : 8] -Input [3]: [s_store_sk#7, sum#10, sum#11] - -(16) HashAggregate [codegen id : 8] -Input [3]: [s_store_sk#7, sum#10, sum#11] -Keys [1]: [s_store_sk#7] -Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#12, sum(UnscaledValue(ss_net_profit#3))#13] -Results [3]: [s_store_sk#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS sales#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#13,17,2) AS profit#15] - -(17) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 6] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] - -(19) Filter [codegen id : 6] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_store_sk#16) - -(20) ReusedExchange [Reuses operator id: 97] -Output [1]: [d_date_sk#20] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_returned_date_sk#19] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] -Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#20] - -(23) ReusedExchange [Reuses operator id: 10] -Output [1]: [s_store_sk#21] - -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_store_sk#16] -Right keys [1]: [s_store_sk#21] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#21] - -(26) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] -Keys [1]: [s_store_sk#21] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum#22, sum#23] -Results [3]: [s_store_sk#21, sum#24, sum#25] - -(27) CometColumnarExchange -Input [3]: [s_store_sk#21, sum#24, sum#25] -Arguments: hashpartitioning(s_store_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#21, sum#24, sum#25] - -(29) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#21, sum#24, sum#25] -Keys [1]: [s_store_sk#21] -Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#17))#26, sum(UnscaledValue(sr_net_loss#18))#27] -Results [3]: [s_store_sk#21, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#26,17,2) AS returns#28, MakeDecimal(sum(UnscaledValue(sr_net_loss#18))#27,17,2) AS profit_loss#29] - -(30) BroadcastExchange -Input [3]: [s_store_sk#21, returns#28, profit_loss#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [s_store_sk#7] -Right keys [1]: [s_store_sk#21] -Join type: LeftOuter -Join condition: None - -(32) Project [codegen id : 8] -Output [5]: [sales#14, coalesce(returns#28, 0.00) AS returns#30, (profit#15 - coalesce(profit_loss#29, 0.00)) AS profit#31, store channel AS channel#32, s_store_sk#7 AS id#33] -Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#21, returns#28, profit_loss#29] - -(33) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#5)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37] - -(35) ReusedExchange [Reuses operator id: 97] -Output [1]: [d_date_sk#38] - -(36) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#37] -Right keys [1]: [d_date_sk#38] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] -Input [5]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37, d_date_sk#38] - -(38) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] -Keys [1]: [cs_call_center_sk#34] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#35)), partial_sum(UnscaledValue(cs_net_profit#36))] -Aggregate Attributes [2]: [sum#39, sum#40] -Results [3]: [cs_call_center_sk#34, sum#41, sum#42] - -(39) CometColumnarExchange -Input [3]: [cs_call_center_sk#34, sum#41, sum#42] -Arguments: hashpartitioning(cs_call_center_sk#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(40) CometColumnarToRow [codegen id : 11] -Input [3]: [cs_call_center_sk#34, sum#41, sum#42] - -(41) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#34, sum#41, sum#42] -Keys [1]: [cs_call_center_sk#34] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#35)), sum(UnscaledValue(cs_net_profit#36))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#35))#43, sum(UnscaledValue(cs_net_profit#36))#44] -Results [3]: [cs_call_center_sk#34, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#35))#43,17,2) AS sales#45, MakeDecimal(sum(UnscaledValue(cs_net_profit#36))#44,17,2) AS profit#46] - -(42) BroadcastExchange -Input [3]: [cs_call_center_sk#34, sales#45, profit#46] -Arguments: IdentityBroadcastMode, [plan_id=6] - -(43) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#49), dynamicpruningexpression(cr_returned_date_sk#49 IN dynamicpruning#5)] -ReadSchema: struct - -(44) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49] - -(45) ReusedExchange [Reuses operator id: 97] -Output [1]: [d_date_sk#50] - -(46) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#49] -Right keys [1]: [d_date_sk#50] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 13] -Output [2]: [cr_return_amount#47, cr_net_loss#48] -Input [4]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49, d_date_sk#50] - -(48) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#47, cr_net_loss#48] -Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#47)), partial_sum(UnscaledValue(cr_net_loss#48))] -Aggregate Attributes [2]: [sum#51, sum#52] -Results [2]: [sum#53, sum#54] - -(49) CometColumnarExchange -Input [2]: [sum#53, sum#54] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(50) CometColumnarToRow -Input [2]: [sum#53, sum#54] - -(51) HashAggregate -Input [2]: [sum#53, sum#54] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#47)), sum(UnscaledValue(cr_net_loss#48))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#47))#55, sum(UnscaledValue(cr_net_loss#48))#56] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#47))#55,17,2) AS returns#57, MakeDecimal(sum(UnscaledValue(cr_net_loss#48))#56,17,2) AS profit_loss#58] - -(52) BroadcastNestedLoopJoin [codegen id : 14] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 14] -Output [5]: [sales#45, returns#57, (profit#46 - profit_loss#58) AS profit#59, catalog channel AS channel#60, cs_call_center_sk#34 AS id#61] -Input [5]: [cs_call_center_sk#34, sales#45, profit#46, returns#57, profit_loss#58] - -(54) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] - -(56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] -Condition : isnotnull(ws_web_page_sk#62) - -(57) ReusedExchange [Reuses operator id: 97] -Output [1]: [d_date_sk#66] - -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#65] -Right keys [1]: [d_date_sk#66] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64] -Input [5]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65, d_date_sk#66] - -(60) CometNativeScan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(61) CometFilter -Input [1]: [wp_web_page_sk#67] -Condition : isnotnull(wp_web_page_sk#67) - -(62) CometColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#67] - -(63) BroadcastExchange -Input [1]: [wp_web_page_sk#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#62] -Right keys [1]: [wp_web_page_sk#67] -Join type: Inner -Join condition: None - -(65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] -Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] - -(66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] -Keys [1]: [wp_web_page_sk#67] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#63)), partial_sum(UnscaledValue(ws_net_profit#64))] -Aggregate Attributes [2]: [sum#68, sum#69] -Results [3]: [wp_web_page_sk#67, sum#70, sum#71] - -(67) CometColumnarExchange -Input [3]: [wp_web_page_sk#67, sum#70, sum#71] -Arguments: hashpartitioning(wp_web_page_sk#67, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(68) CometColumnarToRow [codegen id : 22] -Input [3]: [wp_web_page_sk#67, sum#70, sum#71] - -(69) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#67, sum#70, sum#71] -Keys [1]: [wp_web_page_sk#67] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#63)), sum(UnscaledValue(ws_net_profit#64))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#63))#72, sum(UnscaledValue(ws_net_profit#64))#73] -Results [3]: [wp_web_page_sk#67, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#63))#72,17,2) AS sales#74, MakeDecimal(sum(UnscaledValue(ws_net_profit#64))#73,17,2) AS profit#75] - -(70) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#79), dynamicpruningexpression(wr_returned_date_sk#79 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_web_page_sk)] -ReadSchema: struct - -(71) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] - -(72) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] -Condition : isnotnull(wr_web_page_sk#76) - -(73) ReusedExchange [Reuses operator id: 97] -Output [1]: [d_date_sk#80] - -(74) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#79] -Right keys [1]: [d_date_sk#80] -Join type: Inner -Join condition: None - -(75) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78] -Input [5]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79, d_date_sk#80] - -(76) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#81] - -(77) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#76] -Right keys [1]: [wp_web_page_sk#81] -Join type: Inner -Join condition: None - -(78) Project [codegen id : 20] -Output [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] -Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] - -(79) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] -Keys [1]: [wp_web_page_sk#81] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#77)), partial_sum(UnscaledValue(wr_net_loss#78))] -Aggregate Attributes [2]: [sum#82, sum#83] -Results [3]: [wp_web_page_sk#81, sum#84, sum#85] - -(80) CometColumnarExchange -Input [3]: [wp_web_page_sk#81, sum#84, sum#85] -Arguments: hashpartitioning(wp_web_page_sk#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(81) CometColumnarToRow [codegen id : 21] -Input [3]: [wp_web_page_sk#81, sum#84, sum#85] - -(82) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#81, sum#84, sum#85] -Keys [1]: [wp_web_page_sk#81] -Functions [2]: [sum(UnscaledValue(wr_return_amt#77)), sum(UnscaledValue(wr_net_loss#78))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#77))#86, sum(UnscaledValue(wr_net_loss#78))#87] -Results [3]: [wp_web_page_sk#81, MakeDecimal(sum(UnscaledValue(wr_return_amt#77))#86,17,2) AS returns#88, MakeDecimal(sum(UnscaledValue(wr_net_loss#78))#87,17,2) AS profit_loss#89] - -(83) BroadcastExchange -Input [3]: [wp_web_page_sk#81, returns#88, profit_loss#89] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -(84) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#67] -Right keys [1]: [wp_web_page_sk#81] -Join type: LeftOuter -Join condition: None - -(85) Project [codegen id : 22] -Output [5]: [sales#74, coalesce(returns#88, 0.00) AS returns#90, (profit#75 - coalesce(profit_loss#89, 0.00)) AS profit#91, web channel AS channel#92, wp_web_page_sk#67 AS id#93] -Input [6]: [wp_web_page_sk#67, sales#74, profit#75, wp_web_page_sk#81, returns#88, profit_loss#89] - -(86) Union - -(87) Expand [codegen id : 23] -Input [5]: [sales#14, returns#30, profit#31, channel#32, id#33] -Arguments: [[sales#14, returns#30, profit#31, channel#32, id#33, 0], [sales#14, returns#30, profit#31, channel#32, null, 1], [sales#14, returns#30, profit#31, null, null, 3]], [sales#14, returns#30, profit#31, channel#94, id#95, spark_grouping_id#96] - -(88) HashAggregate [codegen id : 23] -Input [6]: [sales#14, returns#30, profit#31, channel#94, id#95, spark_grouping_id#96] -Keys [3]: [channel#94, id#95, spark_grouping_id#96] -Functions [3]: [partial_sum(sales#14), partial_sum(returns#30), partial_sum(profit#31)] -Aggregate Attributes [6]: [sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Results [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] - -(89) CometColumnarExchange -Input [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Arguments: hashpartitioning(channel#94, id#95, spark_grouping_id#96, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(90) CometColumnarToRow [codegen id : 24] -Input [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] - -(91) HashAggregate [codegen id : 24] -Input [9]: [channel#94, id#95, spark_grouping_id#96, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Keys [3]: [channel#94, id#95, spark_grouping_id#96] -Functions [3]: [sum(sales#14), sum(returns#30), sum(profit#31)] -Aggregate Attributes [3]: [sum(sales#14)#109, sum(returns#30)#110, sum(profit#31)#111] -Results [5]: [channel#94, id#95, sum(sales#14)#109 AS sales#112, sum(returns#30)#110 AS returns#113, sum(profit#31)#111 AS profit#114] - -(92) TakeOrderedAndProject -Input [5]: [channel#94, id#95, sales#112, returns#113, profit#114] -Arguments: 100, [channel#94 ASC NULLS FIRST, id#95 ASC NULLS FIRST], [channel#94, id#95, sales#112, returns#113, profit#114] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (97) -+- * CometColumnarToRow (96) - +- CometProject (95) - +- CometFilter (94) - +- CometNativeScan parquet spark_catalog.default.date_dim (93) - - -(93) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#115] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(94) CometFilter -Input [2]: [d_date_sk#6, d_date#115] -Condition : (((isnotnull(d_date#115) AND (d_date#115 >= 2000-08-03)) AND (d_date#115 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) - -(95) CometProject -Input [2]: [d_date_sk#6, d_date#115] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(96) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(97) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#49 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 70 Hosting Expression = wr_returned_date_sk#79 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/simplified.txt deleted file mode 100644 index 4256e90759..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/simplified.txt +++ /dev/null @@ -1,150 +0,0 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (24) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (23) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (8) - Project [sales,returns,profit,profit_loss,s_store_sk] - BroadcastHashJoin [s_store_sk,s_store_sk] - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #2 - WholeStageCodegen (3) - HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,s_store_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #6 - WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - Project [sr_return_amt,sr_net_loss,s_store_sk] - BroadcastHashJoin [sr_store_sk,s_store_sk] - Project [sr_store_sk,sr_return_amt,sr_net_loss] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [s_store_sk] #4 - WholeStageCodegen (14) - Project [sales,returns,profit,profit_loss,cs_call_center_sk] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) - HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_call_center_sk] #8 - WholeStageCodegen (10) - HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (13) - HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - Project [cr_return_amount,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (22) - Project [sales,returns,profit,profit_loss,wp_web_page_sk] - BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wp_web_page_sk] #10 - WholeStageCodegen (17) - HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_web_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (21) - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wp_web_page_sk] #13 - WholeStageCodegen (20) - HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - Project [wr_return_amt,wr_net_loss,wp_web_page_sk] - BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] - Project [wr_web_page_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Filter [wr_web_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - ReusedExchange [wp_web_page_sk] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/explain.txt deleted file mode 100644 index bb7ed0a50c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,518 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (85) -+- * HashAggregate (84) - +- * CometColumnarToRow (83) - +- CometColumnarExchange (82) - +- * HashAggregate (81) - +- * Expand (80) - +- Union (79) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometHashAggregate (16) - : : +- CometExchange (15) - : : +- CometHashAggregate (14) - : : +- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (28) - : +- CometHashAggregate (27) - : +- CometExchange (26) - : +- CometHashAggregate (25) - : +- CometProject (24) - : +- CometBroadcastHashJoin (23) - : :- CometProject (21) - : : +- CometBroadcastHashJoin (20) - : : :- CometFilter (18) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (17) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - :- * Project (50) - : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) - : :- BroadcastExchange (40) - : : +- * CometColumnarToRow (39) - : : +- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (32) - : : +- ReusedExchange (33) - : +- * CometColumnarToRow (48) - : +- CometHashAggregate (47) - : +- CometExchange (46) - : +- CometHashAggregate (45) - : +- CometProject (44) - : +- CometBroadcastHashJoin (43) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (41) - : +- ReusedExchange (42) - +- * CometColumnarToRow (78) - +- CometProject (77) - +- CometBroadcastHashJoin (76) - :- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (55) - : : +- CometBroadcastHashJoin (54) - : : :- CometFilter (52) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (51) - : : +- ReusedExchange (53) - : +- CometBroadcastExchange (58) - : +- CometFilter (57) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (56) - +- CometBroadcastExchange (75) - +- CometHashAggregate (74) - +- CometExchange (73) - +- CometHashAggregate (72) - +- CometProject (71) - +- CometBroadcastHashJoin (70) - :- CometProject (68) - : +- CometBroadcastHashJoin (67) - : :- CometFilter (65) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (64) - : +- ReusedExchange (66) - +- ReusedExchange (69) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3], [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [1]: [s_store_sk#8] -Condition : isnotnull(s_store_sk#8) - -(11) CometBroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: [s_store_sk#8] - -(12) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -Right output [1]: [s_store_sk#8] -Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight - -(13) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] - -(14) CometHashAggregate -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Keys [1]: [s_store_sk#8] -Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] - -(15) CometExchange -Input [3]: [s_store_sk#8, sum#9, sum#10] -Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [3]: [s_store_sk#8, sum#9, sum#10] -Keys [1]: [s_store_sk#8] -Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(18) CometFilter -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Condition : isnotnull(sr_store_sk#11) - -(19) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#16] - -(20) CometBroadcastHashJoin -Left output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Right output [1]: [d_date_sk#16] -Arguments: [sr_returned_date_sk#14], [d_date_sk#16], Inner, BuildRight - -(21) CometProject -Input [5]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14, d_date_sk#16] -Arguments: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13], [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] - -(22) ReusedExchange [Reuses operator id: 11] -Output [1]: [s_store_sk#17] - -(23) CometBroadcastHashJoin -Left output [3]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] -Right output [1]: [s_store_sk#17] -Arguments: [sr_store_sk#11], [s_store_sk#17], Inner, BuildRight - -(24) CometProject -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Arguments: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17], [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] - -(25) CometHashAggregate -Input [3]: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Keys [1]: [s_store_sk#17] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#12)), partial_sum(UnscaledValue(sr_net_loss#13))] - -(26) CometExchange -Input [3]: [s_store_sk#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(27) CometHashAggregate -Input [3]: [s_store_sk#17, sum#18, sum#19] -Keys [1]: [s_store_sk#17] -Functions [2]: [sum(UnscaledValue(sr_return_amt#12)), sum(UnscaledValue(sr_net_loss#13))] - -(28) CometBroadcastExchange -Input [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#17, returns#20, profit_loss#21] - -(29) CometBroadcastHashJoin -Left output [3]: [s_store_sk#8, sales#22, profit#23] -Right output [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight - -(30) CometProject -Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] -Arguments: [sales#22, returns#24, profit#25, channel#26, id#27], [sales#22, coalesce(returns#20, 0.00) AS returns#24, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#25, store channel AS channel#26, s_store_sk#8 AS id#27] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [sales#22, returns#24, profit#25, channel#26, id#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] -ReadSchema: struct - -(33) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(34) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [cs_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(35) CometProject -Input [5]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31, d_date_sk#33] -Arguments: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30], [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] - -(36) CometHashAggregate -Input [3]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#29)), partial_sum(UnscaledValue(cs_net_profit#30))] - -(37) CometExchange -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Arguments: hashpartitioning(cs_call_center_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] - -(39) CometColumnarToRow [codegen id : 2] -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] - -(40) BroadcastExchange -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] -Arguments: IdentityBroadcastMode, [plan_id=4] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] -ReadSchema: struct - -(42) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#42] - -(43) CometBroadcastHashJoin -Left output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Right output [1]: [d_date_sk#42] -Arguments: [cr_returned_date_sk#40], [d_date_sk#42], Inner, BuildRight - -(44) CometProject -Input [4]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40, d_date_sk#42] -Arguments: [cr_return_amount#38, cr_net_loss#39], [cr_return_amount#38, cr_net_loss#39] - -(45) CometHashAggregate -Input [2]: [cr_return_amount#38, cr_net_loss#39] -Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] - -(46) CometExchange -Input [2]: [sum#43, sum#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometHashAggregate -Input [2]: [sum#43, sum#44] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] - -(48) CometColumnarToRow -Input [2]: [returns#45, profit_loss#46] - -(49) BroadcastNestedLoopJoin [codegen id : 3] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 3] -Output [5]: [sales#36, returns#45, (profit#37 - profit_loss#46) AS profit#47, catalog channel AS channel#48, cs_call_center_sk#28 AS id#49] -Input [5]: [cs_call_center_sk#28, sales#36, profit#37, returns#45, profit_loss#46] - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] -PushedFilters: [IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(52) CometFilter -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Condition : isnotnull(ws_web_page_sk#50) - -(53) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#55] - -(54) CometBroadcastHashJoin -Left output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Right output [1]: [d_date_sk#55] -Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight - -(55) CometProject -Input [5]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53, d_date_sk#55] -Arguments: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52], [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(57) CometFilter -Input [1]: [wp_web_page_sk#56] -Condition : isnotnull(wp_web_page_sk#56) - -(58) CometBroadcastExchange -Input [1]: [wp_web_page_sk#56] -Arguments: [wp_web_page_sk#56] - -(59) CometBroadcastHashJoin -Left output [3]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] -Right output [1]: [wp_web_page_sk#56] -Arguments: [ws_web_page_sk#50], [wp_web_page_sk#56], Inner, BuildRight - -(60) CometProject -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Arguments: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56], [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] - -(61) CometHashAggregate -Input [3]: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#51)), partial_sum(UnscaledValue(ws_net_profit#52))] - -(62) CometExchange -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Arguments: hashpartitioning(wp_web_page_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(63) CometHashAggregate -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#51)), sum(UnscaledValue(ws_net_profit#52))] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] -PushedFilters: [IsNotNull(wr_web_page_sk)] -ReadSchema: struct - -(65) CometFilter -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Condition : isnotnull(wr_web_page_sk#59) - -(66) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#64] - -(67) CometBroadcastHashJoin -Left output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Right output [1]: [d_date_sk#64] -Arguments: [wr_returned_date_sk#62], [d_date_sk#64], Inner, BuildRight - -(68) CometProject -Input [5]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62, d_date_sk#64] -Arguments: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61], [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] - -(69) ReusedExchange [Reuses operator id: 58] -Output [1]: [wp_web_page_sk#65] - -(70) CometBroadcastHashJoin -Left output [3]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] -Right output [1]: [wp_web_page_sk#65] -Arguments: [wr_web_page_sk#59], [wp_web_page_sk#65], Inner, BuildRight - -(71) CometProject -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Arguments: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65], [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] - -(72) CometHashAggregate -Input [3]: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#60)), partial_sum(UnscaledValue(wr_net_loss#61))] - -(73) CometExchange -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Arguments: hashpartitioning(wp_web_page_sk#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(74) CometHashAggregate -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [sum(UnscaledValue(wr_return_amt#60)), sum(UnscaledValue(wr_net_loss#61))] - -(75) CometBroadcastExchange -Input [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#65, returns#68, profit_loss#69] - -(76) CometBroadcastHashJoin -Left output [3]: [wp_web_page_sk#56, sales#70, profit#71] -Right output [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight - -(77) CometProject -Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [sales#70, returns#72, profit#73, channel#74, id#75], [sales#70, coalesce(returns#68, 0.00) AS returns#72, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#73, web channel AS channel#74, wp_web_page_sk#56 AS id#75] - -(78) CometColumnarToRow [codegen id : 4] -Input [5]: [sales#70, returns#72, profit#73, channel#74, id#75] - -(79) Union - -(80) Expand [codegen id : 5] -Input [5]: [sales#22, returns#24, profit#25, channel#26, id#27] -Arguments: [[sales#22, returns#24, profit#25, channel#26, id#27, 0], [sales#22, returns#24, profit#25, channel#26, null, 1], [sales#22, returns#24, profit#25, null, null, 3]], [sales#22, returns#24, profit#25, channel#76, id#77, spark_grouping_id#78] - -(81) HashAggregate [codegen id : 5] -Input [6]: [sales#22, returns#24, profit#25, channel#76, id#77, spark_grouping_id#78] -Keys [3]: [channel#76, id#77, spark_grouping_id#78] -Functions [3]: [partial_sum(sales#22), partial_sum(returns#24), partial_sum(profit#25)] -Aggregate Attributes [6]: [sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84] -Results [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] - -(82) CometColumnarExchange -Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] -Arguments: hashpartitioning(channel#76, id#77, spark_grouping_id#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(83) CometColumnarToRow [codegen id : 6] -Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] - -(84) HashAggregate [codegen id : 6] -Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] -Keys [3]: [channel#76, id#77, spark_grouping_id#78] -Functions [3]: [sum(sales#22), sum(returns#24), sum(profit#25)] -Aggregate Attributes [3]: [sum(sales#22)#91, sum(returns#24)#92, sum(profit#25)#93] -Results [5]: [channel#76, id#77, sum(sales#22)#91 AS sales#94, sum(returns#24)#92 AS returns#95, sum(profit#25)#93 AS profit#96] - -(85) TakeOrderedAndProject -Input [5]: [channel#76, id#77, sales#94, returns#95, profit#96] -Arguments: 100, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#94, returns#95, profit#96] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (90) -+- * CometColumnarToRow (89) - +- CometProject (88) - +- CometFilter (87) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (86) - - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(87) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) - -(88) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(89) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(90) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#14 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 64 Hosting Expression = wr_returned_date_sk#62 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/simplified.txt deleted file mode 100644 index a1243769e5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,111 +0,0 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (6) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (5) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [returns,profit,profit_loss,s_store_sk] [sales,returns,profit,channel,id] - CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [s_store_sk,sales,profit,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] - CometExchange [s_store_sk] #2 - CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk] #5 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [s_store_sk,returns,profit_loss] #6 - CometHashAggregate [sum,sum] [s_store_sk,returns,profit_loss,sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss))] - CometExchange [s_store_sk] #7 - CometHashAggregate [sr_return_amt,sr_net_loss] [s_store_sk,sum,sum] - CometProject [sr_return_amt,sr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [s_store_sk] #5 - WholeStageCodegen (3) - Project [sales,returns,profit,profit_loss,cs_call_center_sk] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [cs_call_center_sk,sales,profit,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] - CometExchange [cs_call_center_sk] #9 - CometHashAggregate [cs_ext_sales_price,cs_net_profit] [cs_call_center_sk,sum,sum] - CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [returns,profit_loss,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] - CometExchange #10 - CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] - CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [returns,profit,profit_loss,wp_web_page_sk] [sales,returns,profit,channel,id] - CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [wp_web_page_sk,sales,profit,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] - CometExchange [wp_web_page_sk] #11 - CometHashAggregate [ws_ext_sales_price,ws_net_profit] [wp_web_page_sk,sum,sum] - CometProject [ws_ext_sales_price,ws_net_profit,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_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 [native_iceberg_compat] 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 [wp_web_page_sk] #12 - CometFilter [wp_web_page_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [wp_web_page_sk,returns,profit_loss] #13 - CometHashAggregate [sum,sum] [wp_web_page_sk,returns,profit_loss,sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss))] - CometExchange [wp_web_page_sk] #14 - CometHashAggregate [wr_return_amt,wr_net_loss] [wp_web_page_sk,sum,sum] - CometProject [wr_return_amt,wr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [wp_web_page_sk] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt deleted file mode 100644 index bb7ed0a50c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt +++ /dev/null @@ -1,518 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (85) -+- * HashAggregate (84) - +- * CometColumnarToRow (83) - +- CometColumnarExchange (82) - +- * HashAggregate (81) - +- * Expand (80) - +- Union (79) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometHashAggregate (16) - : : +- CometExchange (15) - : : +- CometHashAggregate (14) - : : +- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (28) - : +- CometHashAggregate (27) - : +- CometExchange (26) - : +- CometHashAggregate (25) - : +- CometProject (24) - : +- CometBroadcastHashJoin (23) - : :- CometProject (21) - : : +- CometBroadcastHashJoin (20) - : : :- CometFilter (18) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (17) - : : +- ReusedExchange (19) - : +- ReusedExchange (22) - :- * Project (50) - : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) - : :- BroadcastExchange (40) - : : +- * CometColumnarToRow (39) - : : +- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (32) - : : +- ReusedExchange (33) - : +- * CometColumnarToRow (48) - : +- CometHashAggregate (47) - : +- CometExchange (46) - : +- CometHashAggregate (45) - : +- CometProject (44) - : +- CometBroadcastHashJoin (43) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (41) - : +- ReusedExchange (42) - +- * CometColumnarToRow (78) - +- CometProject (77) - +- CometBroadcastHashJoin (76) - :- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (55) - : : +- CometBroadcastHashJoin (54) - : : :- CometFilter (52) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (51) - : : +- ReusedExchange (53) - : +- CometBroadcastExchange (58) - : +- CometFilter (57) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (56) - +- CometBroadcastExchange (75) - +- CometHashAggregate (74) - +- CometExchange (73) - +- CometHashAggregate (72) - +- CometProject (71) - +- CometBroadcastHashJoin (70) - :- CometProject (68) - : +- CometBroadcastHashJoin (67) - : :- CometFilter (65) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (64) - : +- ReusedExchange (66) - +- ReusedExchange (69) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3], [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [1]: [s_store_sk#8] -Condition : isnotnull(s_store_sk#8) - -(11) CometBroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: [s_store_sk#8] - -(12) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -Right output [1]: [s_store_sk#8] -Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight - -(13) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] - -(14) CometHashAggregate -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Keys [1]: [s_store_sk#8] -Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] - -(15) CometExchange -Input [3]: [s_store_sk#8, sum#9, sum#10] -Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [3]: [s_store_sk#8, sum#9, sum#10] -Keys [1]: [s_store_sk#8] -Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(18) CometFilter -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Condition : isnotnull(sr_store_sk#11) - -(19) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#16] - -(20) CometBroadcastHashJoin -Left output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Right output [1]: [d_date_sk#16] -Arguments: [sr_returned_date_sk#14], [d_date_sk#16], Inner, BuildRight - -(21) CometProject -Input [5]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14, d_date_sk#16] -Arguments: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13], [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] - -(22) ReusedExchange [Reuses operator id: 11] -Output [1]: [s_store_sk#17] - -(23) CometBroadcastHashJoin -Left output [3]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] -Right output [1]: [s_store_sk#17] -Arguments: [sr_store_sk#11], [s_store_sk#17], Inner, BuildRight - -(24) CometProject -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Arguments: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17], [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] - -(25) CometHashAggregate -Input [3]: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Keys [1]: [s_store_sk#17] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#12)), partial_sum(UnscaledValue(sr_net_loss#13))] - -(26) CometExchange -Input [3]: [s_store_sk#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(27) CometHashAggregate -Input [3]: [s_store_sk#17, sum#18, sum#19] -Keys [1]: [s_store_sk#17] -Functions [2]: [sum(UnscaledValue(sr_return_amt#12)), sum(UnscaledValue(sr_net_loss#13))] - -(28) CometBroadcastExchange -Input [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#17, returns#20, profit_loss#21] - -(29) CometBroadcastHashJoin -Left output [3]: [s_store_sk#8, sales#22, profit#23] -Right output [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight - -(30) CometProject -Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] -Arguments: [sales#22, returns#24, profit#25, channel#26, id#27], [sales#22, coalesce(returns#20, 0.00) AS returns#24, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#25, store channel AS channel#26, s_store_sk#8 AS id#27] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [sales#22, returns#24, profit#25, channel#26, id#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] -ReadSchema: struct - -(33) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(34) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [cs_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(35) CometProject -Input [5]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31, d_date_sk#33] -Arguments: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30], [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] - -(36) CometHashAggregate -Input [3]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#29)), partial_sum(UnscaledValue(cs_net_profit#30))] - -(37) CometExchange -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Arguments: hashpartitioning(cs_call_center_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] - -(39) CometColumnarToRow [codegen id : 2] -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] - -(40) BroadcastExchange -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] -Arguments: IdentityBroadcastMode, [plan_id=4] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] -ReadSchema: struct - -(42) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#42] - -(43) CometBroadcastHashJoin -Left output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Right output [1]: [d_date_sk#42] -Arguments: [cr_returned_date_sk#40], [d_date_sk#42], Inner, BuildRight - -(44) CometProject -Input [4]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40, d_date_sk#42] -Arguments: [cr_return_amount#38, cr_net_loss#39], [cr_return_amount#38, cr_net_loss#39] - -(45) CometHashAggregate -Input [2]: [cr_return_amount#38, cr_net_loss#39] -Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] - -(46) CometExchange -Input [2]: [sum#43, sum#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometHashAggregate -Input [2]: [sum#43, sum#44] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] - -(48) CometColumnarToRow -Input [2]: [returns#45, profit_loss#46] - -(49) BroadcastNestedLoopJoin [codegen id : 3] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 3] -Output [5]: [sales#36, returns#45, (profit#37 - profit_loss#46) AS profit#47, catalog channel AS channel#48, cs_call_center_sk#28 AS id#49] -Input [5]: [cs_call_center_sk#28, sales#36, profit#37, returns#45, profit_loss#46] - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] -PushedFilters: [IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(52) CometFilter -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Condition : isnotnull(ws_web_page_sk#50) - -(53) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#55] - -(54) CometBroadcastHashJoin -Left output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Right output [1]: [d_date_sk#55] -Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight - -(55) CometProject -Input [5]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53, d_date_sk#55] -Arguments: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52], [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(57) CometFilter -Input [1]: [wp_web_page_sk#56] -Condition : isnotnull(wp_web_page_sk#56) - -(58) CometBroadcastExchange -Input [1]: [wp_web_page_sk#56] -Arguments: [wp_web_page_sk#56] - -(59) CometBroadcastHashJoin -Left output [3]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] -Right output [1]: [wp_web_page_sk#56] -Arguments: [ws_web_page_sk#50], [wp_web_page_sk#56], Inner, BuildRight - -(60) CometProject -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Arguments: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56], [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] - -(61) CometHashAggregate -Input [3]: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#51)), partial_sum(UnscaledValue(ws_net_profit#52))] - -(62) CometExchange -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Arguments: hashpartitioning(wp_web_page_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(63) CometHashAggregate -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#51)), sum(UnscaledValue(ws_net_profit#52))] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] -PushedFilters: [IsNotNull(wr_web_page_sk)] -ReadSchema: struct - -(65) CometFilter -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Condition : isnotnull(wr_web_page_sk#59) - -(66) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#64] - -(67) CometBroadcastHashJoin -Left output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Right output [1]: [d_date_sk#64] -Arguments: [wr_returned_date_sk#62], [d_date_sk#64], Inner, BuildRight - -(68) CometProject -Input [5]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62, d_date_sk#64] -Arguments: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61], [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] - -(69) ReusedExchange [Reuses operator id: 58] -Output [1]: [wp_web_page_sk#65] - -(70) CometBroadcastHashJoin -Left output [3]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] -Right output [1]: [wp_web_page_sk#65] -Arguments: [wr_web_page_sk#59], [wp_web_page_sk#65], Inner, BuildRight - -(71) CometProject -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Arguments: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65], [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] - -(72) CometHashAggregate -Input [3]: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#60)), partial_sum(UnscaledValue(wr_net_loss#61))] - -(73) CometExchange -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Arguments: hashpartitioning(wp_web_page_sk#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(74) CometHashAggregate -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [sum(UnscaledValue(wr_return_amt#60)), sum(UnscaledValue(wr_net_loss#61))] - -(75) CometBroadcastExchange -Input [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#65, returns#68, profit_loss#69] - -(76) CometBroadcastHashJoin -Left output [3]: [wp_web_page_sk#56, sales#70, profit#71] -Right output [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight - -(77) CometProject -Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [sales#70, returns#72, profit#73, channel#74, id#75], [sales#70, coalesce(returns#68, 0.00) AS returns#72, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#73, web channel AS channel#74, wp_web_page_sk#56 AS id#75] - -(78) CometColumnarToRow [codegen id : 4] -Input [5]: [sales#70, returns#72, profit#73, channel#74, id#75] - -(79) Union - -(80) Expand [codegen id : 5] -Input [5]: [sales#22, returns#24, profit#25, channel#26, id#27] -Arguments: [[sales#22, returns#24, profit#25, channel#26, id#27, 0], [sales#22, returns#24, profit#25, channel#26, null, 1], [sales#22, returns#24, profit#25, null, null, 3]], [sales#22, returns#24, profit#25, channel#76, id#77, spark_grouping_id#78] - -(81) HashAggregate [codegen id : 5] -Input [6]: [sales#22, returns#24, profit#25, channel#76, id#77, spark_grouping_id#78] -Keys [3]: [channel#76, id#77, spark_grouping_id#78] -Functions [3]: [partial_sum(sales#22), partial_sum(returns#24), partial_sum(profit#25)] -Aggregate Attributes [6]: [sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84] -Results [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] - -(82) CometColumnarExchange -Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] -Arguments: hashpartitioning(channel#76, id#77, spark_grouping_id#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(83) CometColumnarToRow [codegen id : 6] -Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] - -(84) HashAggregate [codegen id : 6] -Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] -Keys [3]: [channel#76, id#77, spark_grouping_id#78] -Functions [3]: [sum(sales#22), sum(returns#24), sum(profit#25)] -Aggregate Attributes [3]: [sum(sales#22)#91, sum(returns#24)#92, sum(profit#25)#93] -Results [5]: [channel#76, id#77, sum(sales#22)#91 AS sales#94, sum(returns#24)#92 AS returns#95, sum(profit#25)#93 AS profit#96] - -(85) TakeOrderedAndProject -Input [5]: [channel#76, id#77, sales#94, returns#95, profit#96] -Arguments: 100, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#94, returns#95, profit#96] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (90) -+- * CometColumnarToRow (89) - +- CometProject (88) - +- CometFilter (87) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (86) - - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(87) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) - -(88) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(89) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(90) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#14 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 64 Hosting Expression = wr_returned_date_sk#62 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/extended.txt deleted file mode 100644 index ed8a9e38ca..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/extended.txt +++ /dev/null @@ -1,117 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : :- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - -Comet accelerated 94 out of 109 eligible operators (86%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index a1243769e5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/simplified.txt +++ /dev/null @@ -1,111 +0,0 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (6) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (5) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [returns,profit,profit_loss,s_store_sk] [sales,returns,profit,channel,id] - CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [s_store_sk,sales,profit,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] - CometExchange [s_store_sk] #2 - CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk] #5 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [s_store_sk,returns,profit_loss] #6 - CometHashAggregate [sum,sum] [s_store_sk,returns,profit_loss,sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss))] - CometExchange [s_store_sk] #7 - CometHashAggregate [sr_return_amt,sr_net_loss] [s_store_sk,sum,sum] - CometProject [sr_return_amt,sr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [s_store_sk] #5 - WholeStageCodegen (3) - Project [sales,returns,profit,profit_loss,cs_call_center_sk] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [cs_call_center_sk,sales,profit,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] - CometExchange [cs_call_center_sk] #9 - CometHashAggregate [cs_ext_sales_price,cs_net_profit] [cs_call_center_sk,sum,sum] - CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [returns,profit_loss,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] - CometExchange #10 - CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] - CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [returns,profit,profit_loss,wp_web_page_sk] [sales,returns,profit,channel,id] - CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [wp_web_page_sk,sales,profit,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] - CometExchange [wp_web_page_sk] #11 - CometHashAggregate [ws_ext_sales_price,ws_net_profit] [wp_web_page_sk,sum,sum] - CometProject [ws_ext_sales_price,ws_net_profit,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_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 [native_iceberg_compat] 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 [wp_web_page_sk] #12 - CometFilter [wp_web_page_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [wp_web_page_sk,returns,profit_loss] #13 - CometHashAggregate [sum,sum] [wp_web_page_sk,returns,profit_loss,sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss))] - CometExchange [wp_web_page_sk] #14 - CometHashAggregate [wr_return_amt,wr_net_loss] [wp_web_page_sk,sum,sum] - CometProject [wr_return_amt,wr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [wp_web_page_sk] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/explain.txt deleted file mode 100644 index 4dc7ac380a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/explain.txt +++ /dev/null @@ -1,417 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (70) -+- * Project (69) - +- * CometColumnarToRow (68) - +- CometSortMergeJoin (67) - :- CometProject (45) - : +- CometSortMergeJoin (44) - : :- CometSort (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometFilter (12) - : : : +- CometSortMergeJoin (11) - : : : :- CometSort (5) - : : : : +- CometColumnarExchange (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (10) - : : : +- CometExchange (9) - : : : +- CometProject (8) - : : : +- CometFilter (7) - : : : +- CometNativeScan parquet spark_catalog.default.store_returns (6) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (14) - : +- CometSort (43) - : +- CometFilter (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometFilter (34) - : : +- CometSortMergeJoin (33) - : : :- CometSort (27) - : : : +- CometColumnarExchange (26) - : : : +- * Filter (25) - : : : +- * ColumnarToRow (24) - : : : +- Scan parquet spark_catalog.default.web_sales (23) - : : +- CometSort (32) - : : +- CometExchange (31) - : : +- CometProject (30) - : : +- CometFilter (29) - : : +- CometNativeScan parquet spark_catalog.default.web_returns (28) - : +- ReusedExchange (36) - +- CometSort (66) - +- CometFilter (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometFilter (57) - : +- CometSortMergeJoin (56) - : :- CometSort (50) - : : +- CometColumnarExchange (49) - : : +- * Filter (48) - : : +- * ColumnarToRow (47) - : : +- Scan parquet spark_catalog.default.catalog_sales (46) - : +- CometSort (55) - : +- CometExchange (54) - : +- CometProject (53) - : +- CometFilter (52) - : +- CometNativeScan parquet spark_catalog.default.catalog_returns (51) - +- ReusedExchange (59) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(3) Filter [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_customer_sk#2)) - -(4) CometColumnarExchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_ticket_number#3, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_ticket_number#3 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) - -(8) CometProject -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] - -(9) CometExchange -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter - -(12) CometFilter -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Condition : isnull(sr_ticket_number#10) - -(13) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(14) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(17) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] - -(19) CometHashAggregate -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] - -(20) CometExchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(21) CometHashAggregate -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] - -(22) CometSort -Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] - -(23) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 2] -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(25) Filter [codegen id : 2] -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) - -(26) CometColumnarExchange -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometSort -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] - -(28) CometNativeScan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#28, wr_order_number#29, wr_returned_date_sk#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(29) CometFilter -Input [3]: [wr_item_sk#28, wr_order_number#29, wr_returned_date_sk#30] -Condition : (isnotnull(wr_order_number#29) AND isnotnull(wr_item_sk#28)) - -(30) CometProject -Input [3]: [wr_item_sk#28, wr_order_number#29, wr_returned_date_sk#30] -Arguments: [wr_item_sk#28, wr_order_number#29], [wr_item_sk#28, wr_order_number#29] - -(31) CometExchange -Input [2]: [wr_item_sk#28, wr_order_number#29] -Arguments: hashpartitioning(wr_order_number#29, wr_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(32) CometSort -Input [2]: [wr_item_sk#28, wr_order_number#29] -Arguments: [wr_item_sk#28, wr_order_number#29], [wr_order_number#29 ASC NULLS FIRST, wr_item_sk#28 ASC NULLS FIRST] - -(33) CometSortMergeJoin -Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [wr_item_sk#28, wr_order_number#29] -Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#29, wr_item_sk#28], LeftOuter - -(34) CometFilter -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#28, wr_order_number#29] -Condition : isnull(wr_order_number#29) - -(35) CometProject -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#28, wr_order_number#29] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(36) ReusedExchange [Reuses operator id: 16] -Output [2]: [d_date_sk#31, d_year#32] - -(37) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [d_date_sk#31, d_year#32] -Arguments: [ws_sold_date_sk#27], [d_date_sk#31], Inner, BuildRight - -(38) CometProject -Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#31, d_year#32] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#32], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#32] - -(39) CometHashAggregate -Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#32] -Keys [3]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] - -(40) CometExchange -Input [6]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22, sum#33, sum#34, sum#35] -Arguments: hashpartitioning(d_year#32, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(41) CometHashAggregate -Input [6]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22, sum#33, sum#34, sum#35] -Keys [3]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] - -(42) CometFilter -Input [6]: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Condition : (coalesce(ws_qty#38, 0) > 0) - -(43) CometSort -Input [6]: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Arguments: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40], [ws_sold_year#36 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#37 ASC NULLS FIRST] - -(44) CometSortMergeJoin -Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Right output [6]: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37], Inner - -(45) CometProject -Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40] - -(46) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#47), dynamicpruningexpression(cs_sold_date_sk#47 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(47) ColumnarToRow [codegen id : 3] -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] - -(48) Filter [codegen id : 3] -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Condition : (isnotnull(cs_item_sk#42) AND isnotnull(cs_bill_customer_sk#41)) - -(49) CometColumnarExchange -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Arguments: hashpartitioning(cs_order_number#43, cs_item_sk#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(50) CometSort -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Arguments: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47], [cs_order_number#43 ASC NULLS FIRST, cs_item_sk#42 ASC NULLS FIRST] - -(51) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#48, cr_order_number#49, cr_returned_date_sk#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(52) CometFilter -Input [3]: [cr_item_sk#48, cr_order_number#49, cr_returned_date_sk#50] -Condition : (isnotnull(cr_order_number#49) AND isnotnull(cr_item_sk#48)) - -(53) CometProject -Input [3]: [cr_item_sk#48, cr_order_number#49, cr_returned_date_sk#50] -Arguments: [cr_item_sk#48, cr_order_number#49], [cr_item_sk#48, cr_order_number#49] - -(54) CometExchange -Input [2]: [cr_item_sk#48, cr_order_number#49] -Arguments: hashpartitioning(cr_order_number#49, cr_item_sk#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(55) CometSort -Input [2]: [cr_item_sk#48, cr_order_number#49] -Arguments: [cr_item_sk#48, cr_order_number#49], [cr_order_number#49 ASC NULLS FIRST, cr_item_sk#48 ASC NULLS FIRST] - -(56) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Right output [2]: [cr_item_sk#48, cr_order_number#49] -Arguments: [cs_order_number#43, cs_item_sk#42], [cr_order_number#49, cr_item_sk#48], LeftOuter - -(57) CometFilter -Input [9]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47, cr_item_sk#48, cr_order_number#49] -Condition : isnull(cr_order_number#49) - -(58) CometProject -Input [9]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47, cr_item_sk#48, cr_order_number#49] -Arguments: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47], [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] - -(59) ReusedExchange [Reuses operator id: 16] -Output [2]: [d_date_sk#51, d_year#52] - -(60) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Right output [2]: [d_date_sk#51, d_year#52] -Arguments: [cs_sold_date_sk#47], [d_date_sk#51], Inner, BuildRight - -(61) CometProject -Input [8]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47, d_date_sk#51, d_year#52] -Arguments: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, d_year#52], [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, d_year#52] - -(62) CometHashAggregate -Input [6]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, d_year#52] -Keys [3]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41] -Functions [3]: [partial_sum(cs_quantity#44), partial_sum(UnscaledValue(cs_wholesale_cost#45)), partial_sum(UnscaledValue(cs_sales_price#46))] - -(63) CometExchange -Input [6]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41, sum#53, sum#54, sum#55] -Arguments: hashpartitioning(d_year#52, cs_item_sk#42, cs_bill_customer_sk#41, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(64) CometHashAggregate -Input [6]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41, sum#53, sum#54, sum#55] -Keys [3]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41] -Functions [3]: [sum(cs_quantity#44), sum(UnscaledValue(cs_wholesale_cost#45)), sum(UnscaledValue(cs_sales_price#46))] - -(65) CometFilter -Input [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] -Condition : (coalesce(cs_qty#58, 0) > 0) - -(66) CometSort -Input [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] -Arguments: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60], [cs_sold_year#56 ASC NULLS FIRST, cs_item_sk#42 ASC NULLS FIRST, cs_customer_sk#57 ASC NULLS FIRST] - -(67) CometSortMergeJoin -Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40] -Right output [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57], Inner - -(68) CometColumnarToRow [codegen id : 4] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40, cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] - -(69) Project [codegen id : 4] -Output [12]: [round((cast(ss_qty#18 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#38 + cs_qty#58), 1) as double)))), 2) AS ratio#61, ss_qty#18 AS store_qty#62, ss_wc#19 AS store_wholesale_cost#63, ss_sp#20 AS store_sales_price#64, (coalesce(ws_qty#38, 0) + coalesce(cs_qty#58, 0)) AS other_chan_qty#65, (coalesce(ws_wc#39, 0.00) + coalesce(cs_wc#59, 0.00)) AS other_chan_wholesale_cost#66, (coalesce(ws_sp#40, 0.00) + coalesce(cs_sp#60, 0.00)) AS other_chan_sales_price#67, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, cs_qty#58] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40, cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] - -(70) TakeOrderedAndProject -Input [12]: [ratio#61, store_qty#62, store_wholesale_cost#63, store_sales_price#64, other_chan_qty#65, other_chan_wholesale_cost#66, other_chan_sales_price#67, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, cs_qty#58] -Arguments: 100, [ratio#61 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#65 ASC NULLS FIRST, other_chan_wholesale_cost#66 ASC NULLS FIRST, other_chan_sales_price#67 ASC NULLS FIRST, round((cast(ss_qty#18 as double) / cast(coalesce((ws_qty#38 + cs_qty#58), 1) as double)), 2) ASC NULLS FIRST], [ratio#61, store_qty#62, store_wholesale_cost#63, store_sales_price#64, other_chan_qty#65, other_chan_wholesale_cost#66, other_chan_sales_price#67] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (74) -+- * CometColumnarToRow (73) - +- CometFilter (72) - +- CometNativeScan parquet spark_catalog.default.date_dim (71) - - -(71) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(72) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(73) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(74) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#47 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/simplified.txt deleted file mode 100644 index ebd0840eec..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/simplified.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (4) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - CometColumnarToRow - InputAdapter - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] - CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #2 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #5 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] - CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometColumnarExchange [ws_order_number,ws_item_sk] #7 - WholeStageCodegen (2) - Filter [ws_item_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_order_number,wr_item_sk] #8 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] - CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometColumnarExchange [cs_order_number,cs_item_sk] #10 - WholeStageCodegen (3) - Filter [cs_item_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_order_number,cr_item_sk] #11 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/explain.txt deleted file mode 100644 index 3d3ea5ab86..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,405 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (67) -+- * Project (66) - +- * CometColumnarToRow (65) - +- CometSortMergeJoin (64) - :- CometProject (43) - : +- CometSortMergeJoin (42) - : :- CometSort (21) - : : +- CometHashAggregate (20) - : : +- CometExchange (19) - : : +- CometHashAggregate (18) - : : +- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : +- CometBroadcastExchange (15) - : : +- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : +- CometSort (41) - : +- CometFilter (40) - : +- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (33) - : : +- CometFilter (32) - : : +- CometSortMergeJoin (31) - : : :- CometSort (25) - : : : +- CometExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (22) - : : +- CometSort (30) - : : +- CometExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (26) - : +- ReusedExchange (34) - +- CometSort (63) - +- CometFilter (62) - +- CometHashAggregate (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometProject (55) - : +- CometFilter (54) - : +- CometSortMergeJoin (53) - : :- CometSort (47) - : : +- CometExchange (46) - : : +- CometFilter (45) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (44) - : +- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometFilter (49) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (48) - +- ReusedExchange (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_customer_sk#2)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_ticket_number#3, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_ticket_number#3 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) - -(7) CometProject -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] - -(8) CometExchange -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter - -(11) CometFilter -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Condition : isnull(sr_ticket_number#10) - -(12) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(15) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(16) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight - -(17) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] - -(18) CometHashAggregate -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] - -(19) CometExchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometHashAggregate -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] - -(21) CometSort -Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(23) CometFilter -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) - -(24) CometExchange -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometSort -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Condition : (isnotnull(wr_order_number#30) AND isnotnull(wr_item_sk#29)) - -(28) CometProject -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_item_sk#29, wr_order_number#30] - -(29) CometExchange -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: hashpartitioning(wr_order_number#30, wr_item_sk#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(30) CometSort -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_order_number#30 ASC NULLS FIRST, wr_item_sk#29 ASC NULLS FIRST] - -(31) CometSortMergeJoin -Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#30, wr_item_sk#29], LeftOuter - -(32) CometFilter -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Condition : isnull(wr_order_number#30) - -(33) CometProject -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(34) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#32, d_year#33] - -(35) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ws_sold_date_sk#27], [d_date_sk#32], Inner, BuildRight - -(36) CometProject -Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#32, d_year#33] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] - -(37) CometHashAggregate -Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] - -(38) CometExchange -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Arguments: hashpartitioning(d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(39) CometHashAggregate -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] - -(40) CometFilter -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Condition : (coalesce(ws_qty#39, 0) > 0) - -(41) CometSort -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41], [ws_sold_year#37 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#38 ASC NULLS FIRST] - -(42) CometSortMergeJoin -Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Right output [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38], Inner - -(43) CometProject -Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#48), dynamicpruningexpression(cs_sold_date_sk#48 IN dynamicpruning#49)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(45) CometFilter -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Condition : (isnotnull(cs_item_sk#43) AND isnotnull(cs_bill_customer_sk#42)) - -(46) CometExchange -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(47) CometSort -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST] - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Condition : (isnotnull(cr_order_number#51) AND isnotnull(cr_item_sk#50)) - -(50) CometProject -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_item_sk#50, cr_order_number#51] - -(51) CometExchange -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(52) CometSort -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST] - -(53) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cs_order_number#44, cs_item_sk#43], [cr_order_number#51, cr_item_sk#50], LeftOuter - -(54) CometFilter -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Condition : isnull(cr_order_number#51) - -(55) CometProject -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] - -(56) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#53, d_year#54] - -(57) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [cs_sold_date_sk#48], [d_date_sk#53], Inner, BuildRight - -(58) CometProject -Input [8]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, d_date_sk#53, d_year#54] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] - -(59) CometHashAggregate -Input [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [partial_sum(cs_quantity#45), partial_sum(UnscaledValue(cs_wholesale_cost#46)), partial_sum(UnscaledValue(cs_sales_price#47))] - -(60) CometExchange -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Arguments: hashpartitioning(d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(61) CometHashAggregate -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [sum(cs_quantity#45), sum(UnscaledValue(cs_wholesale_cost#46)), sum(UnscaledValue(cs_sales_price#47))] - -(62) CometFilter -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Condition : (coalesce(cs_qty#60, 0) > 0) - -(63) CometSort -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62], [cs_sold_year#58 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST, cs_customer_sk#59 ASC NULLS FIRST] - -(64) CometSortMergeJoin -Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] -Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner - -(65) CometColumnarToRow [codegen id : 1] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(66) Project [codegen id : 1] -Output [12]: [round((cast(ss_qty#18 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)))), 2) AS ratio#63, ss_qty#18 AS store_qty#64, ss_wc#19 AS store_wholesale_cost#65, ss_sp#20 AS store_sales_price#66, (coalesce(ws_qty#39, 0) + coalesce(cs_qty#60, 0)) AS other_chan_qty#67, (coalesce(ws_wc#40, 0.00) + coalesce(cs_wc#61, 0.00)) AS other_chan_wholesale_cost#68, (coalesce(ws_sp#41, 0.00) + coalesce(cs_sp#62, 0.00)) AS other_chan_sales_price#69, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, cs_qty#60] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(67) TakeOrderedAndProject -Input [12]: [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, cs_qty#60] -Arguments: 100, [ratio#63 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#67 ASC NULLS FIRST, other_chan_wholesale_cost#68 ASC NULLS FIRST, other_chan_sales_price#69 ASC NULLS FIRST, round((cast(ss_qty#18 as double) / cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)), 2) ASC NULLS FIRST], [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) - - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(70) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(71) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/simplified.txt deleted file mode 100644 index 77c05217f4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,78 +0,0 @@ -TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (1) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - CometColumnarToRow - InputAdapter - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] - CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #5 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] - CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometExchange [ws_order_number,ws_item_sk] #7 - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_order_number,wr_item_sk] #8 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] - CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometExchange [cs_order_number,cs_item_sk] #10 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_order_number,cr_item_sk] #11 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt deleted file mode 100644 index 3d3ea5ab86..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt +++ /dev/null @@ -1,405 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (67) -+- * Project (66) - +- * CometColumnarToRow (65) - +- CometSortMergeJoin (64) - :- CometProject (43) - : +- CometSortMergeJoin (42) - : :- CometSort (21) - : : +- CometHashAggregate (20) - : : +- CometExchange (19) - : : +- CometHashAggregate (18) - : : +- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : +- CometBroadcastExchange (15) - : : +- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : +- CometSort (41) - : +- CometFilter (40) - : +- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (33) - : : +- CometFilter (32) - : : +- CometSortMergeJoin (31) - : : :- CometSort (25) - : : : +- CometExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (22) - : : +- CometSort (30) - : : +- CometExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (26) - : +- ReusedExchange (34) - +- CometSort (63) - +- CometFilter (62) - +- CometHashAggregate (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometProject (55) - : +- CometFilter (54) - : +- CometSortMergeJoin (53) - : :- CometSort (47) - : : +- CometExchange (46) - : : +- CometFilter (45) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (44) - : +- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometFilter (49) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (48) - +- ReusedExchange (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_customer_sk#2)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_ticket_number#3, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_ticket_number#3 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) - -(7) CometProject -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] - -(8) CometExchange -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter - -(11) CometFilter -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Condition : isnull(sr_ticket_number#10) - -(12) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(15) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(16) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight - -(17) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] - -(18) CometHashAggregate -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] - -(19) CometExchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometHashAggregate -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] - -(21) CometSort -Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(23) CometFilter -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) - -(24) CometExchange -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometSort -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Condition : (isnotnull(wr_order_number#30) AND isnotnull(wr_item_sk#29)) - -(28) CometProject -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_item_sk#29, wr_order_number#30] - -(29) CometExchange -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: hashpartitioning(wr_order_number#30, wr_item_sk#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(30) CometSort -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_order_number#30 ASC NULLS FIRST, wr_item_sk#29 ASC NULLS FIRST] - -(31) CometSortMergeJoin -Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#30, wr_item_sk#29], LeftOuter - -(32) CometFilter -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Condition : isnull(wr_order_number#30) - -(33) CometProject -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(34) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#32, d_year#33] - -(35) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ws_sold_date_sk#27], [d_date_sk#32], Inner, BuildRight - -(36) CometProject -Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#32, d_year#33] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] - -(37) CometHashAggregate -Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] - -(38) CometExchange -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Arguments: hashpartitioning(d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(39) CometHashAggregate -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] - -(40) CometFilter -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Condition : (coalesce(ws_qty#39, 0) > 0) - -(41) CometSort -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41], [ws_sold_year#37 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#38 ASC NULLS FIRST] - -(42) CometSortMergeJoin -Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Right output [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38], Inner - -(43) CometProject -Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#48), dynamicpruningexpression(cs_sold_date_sk#48 IN dynamicpruning#49)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(45) CometFilter -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Condition : (isnotnull(cs_item_sk#43) AND isnotnull(cs_bill_customer_sk#42)) - -(46) CometExchange -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(47) CometSort -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST] - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Condition : (isnotnull(cr_order_number#51) AND isnotnull(cr_item_sk#50)) - -(50) CometProject -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_item_sk#50, cr_order_number#51] - -(51) CometExchange -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(52) CometSort -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST] - -(53) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cs_order_number#44, cs_item_sk#43], [cr_order_number#51, cr_item_sk#50], LeftOuter - -(54) CometFilter -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Condition : isnull(cr_order_number#51) - -(55) CometProject -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] - -(56) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#53, d_year#54] - -(57) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [cs_sold_date_sk#48], [d_date_sk#53], Inner, BuildRight - -(58) CometProject -Input [8]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, d_date_sk#53, d_year#54] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] - -(59) CometHashAggregate -Input [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [partial_sum(cs_quantity#45), partial_sum(UnscaledValue(cs_wholesale_cost#46)), partial_sum(UnscaledValue(cs_sales_price#47))] - -(60) CometExchange -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Arguments: hashpartitioning(d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(61) CometHashAggregate -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [sum(cs_quantity#45), sum(UnscaledValue(cs_wholesale_cost#46)), sum(UnscaledValue(cs_sales_price#47))] - -(62) CometFilter -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Condition : (coalesce(cs_qty#60, 0) > 0) - -(63) CometSort -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62], [cs_sold_year#58 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST, cs_customer_sk#59 ASC NULLS FIRST] - -(64) CometSortMergeJoin -Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] -Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner - -(65) CometColumnarToRow [codegen id : 1] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(66) Project [codegen id : 1] -Output [12]: [round((cast(ss_qty#18 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)))), 2) AS ratio#63, ss_qty#18 AS store_qty#64, ss_wc#19 AS store_wholesale_cost#65, ss_sp#20 AS store_sales_price#66, (coalesce(ws_qty#39, 0) + coalesce(cs_qty#60, 0)) AS other_chan_qty#67, (coalesce(ws_wc#40, 0.00) + coalesce(cs_wc#61, 0.00)) AS other_chan_wholesale_cost#68, (coalesce(ws_sp#41, 0.00) + coalesce(cs_sp#62, 0.00)) AS other_chan_sales_price#69, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, cs_qty#60] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(67) TakeOrderedAndProject -Input [12]: [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, cs_qty#60] -Arguments: 100, [ratio#63 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#67 ASC NULLS FIRST, other_chan_wholesale_cost#68 ASC NULLS FIRST, other_chan_sales_price#69 ASC NULLS FIRST, round((cast(ss_qty#18 as double) / cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)), 2) ASC NULLS FIRST], [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) - - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(70) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(71) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/extended.txt deleted file mode 100644 index 9d5ba53be1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/extended.txt +++ /dev/null @@ -1,80 +0,0 @@ -TakeOrderedAndProject -+- Project [COMET: Comet does not support Spark's BigDecimal rounding] - +- CometColumnarToRow - +- CometSortMergeJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 77c05217f4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/simplified.txt +++ /dev/null @@ -1,78 +0,0 @@ -TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (1) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - CometColumnarToRow - InputAdapter - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] - CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #5 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] - CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometExchange [ws_order_number,ws_item_sk] #7 - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_order_number,wr_item_sk] #8 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] - CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometExchange [cs_order_number,cs_item_sk] #10 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_order_number,cr_item_sk] #11 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/explain.txt deleted file mode 100644 index 78ed64abe7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (32) -+- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (29) - +- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometNativeScan parquet spark_catalog.default.customer (25) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] - -(3) Filter [codegen id : 4] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 37] -Output [1]: [d_date_sk#10] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#11, s_number_employees#12, s_city#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_number_employees), GreaterThanOrEqual(s_number_employees,200), LessThanOrEqual(s_number_employees,295), IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [s_store_sk#11, s_number_employees#12, s_city#13] -Condition : (((isnotnull(s_number_employees#12) AND (s_number_employees#12 >= 200)) AND (s_number_employees#12 <= 295)) AND isnotnull(s_store_sk#11)) - -(9) CometProject -Input [3]: [s_store_sk#11, s_number_employees#12, s_city#13] -Arguments: [s_store_sk#11, s_city#13], [s_store_sk#11, s_city#13] - -(10) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#11, s_city#13] - -(11) BroadcastExchange -Input [2]: [s_store_sk#11, s_city#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#11, s_city#13] - -(14) CometNativeScan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,6),GreaterThan(hd_vehicle_count,2)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Condition : (((hd_dep_count#15 = 6) OR (hd_vehicle_count#16 > 2)) AND isnotnull(hd_demo_sk#14)) - -(16) CometProject -Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] -Arguments: [hd_demo_sk#14], [hd_demo_sk#14] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#14] - -(18) BroadcastExchange -Input [1]: [hd_demo_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#14] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13, hd_demo_sk#14] - -(21) HashAggregate [codegen id : 4] -Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13] -Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum#17, sum#18] -Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] - -(22) CometColumnarExchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 6] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] - -(24) HashAggregate [codegen id : 6] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13] -Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#6))#21, sum(UnscaledValue(ss_net_profit#7))#22] -Results [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#21,17,2) AS amt#23, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#22,17,2) AS profit#24] - -(25) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Condition : isnotnull(c_customer_sk#25) - -(27) CometProject -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_customer_sk#25, c_first_name#28, c_last_name#29], [c_customer_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#26, 20, true, false, true) AS c_first_name#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#27, 30, true, false, true) AS c_last_name#29] - -(28) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#25, c_first_name#28, c_last_name#29] - -(29) BroadcastExchange -Input [3]: [c_customer_sk#25, c_first_name#28, c_last_name#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#25] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [7]: [c_last_name#29, c_first_name#28, substr(s_city#13, 1, 30) AS substr(s_city, 1, 30)#30, ss_ticket_number#5, amt#23, profit#24, s_city#13] -Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, amt#23, profit#24, c_customer_sk#25, c_first_name#28, c_last_name#29] - -(32) TakeOrderedAndProject -Input [7]: [c_last_name#29, c_first_name#28, substr(s_city, 1, 30)#30, ss_ticket_number#5, amt#23, profit#24, s_city#13] -Arguments: 100, [c_last_name#29 ASC NULLS FIRST, c_first_name#28 ASC NULLS FIRST, substr(s_city#13, 1, 30) ASC NULLS FIRST, profit#24 ASC NULLS FIRST], [c_last_name#29, c_first_name#28, substr(s_city, 1, 30)#30, ss_ticket_number#5, amt#23, profit#24] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometNativeScan parquet spark_catalog.default.date_dim (33) - - -(33) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#31, d_dow#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [3]: [d_date_sk#10, d_year#31, d_dow#32] -Condition : (((isnotnull(d_dow#32) AND (d_dow#32 = 1)) AND d_year#31 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(35) CometProject -Input [3]: [d_date_sk#10, d_year#31, d_dow#32] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(37) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/simplified.txt deleted file mode 100644 index 1ee7a286a4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, 30),ss_ticket_number,amt] - WholeStageCodegen (6) - Project [c_last_name,c_first_name,s_city,ss_ticket_number,amt,profit] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),amt,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] - Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk,s_city] - CometFilter [s_store_sk,s_number_employees,s_city] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan 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/q79.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/explain.txt deleted file mode 100644 index 2d2c6dc938..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,212 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (31) -+- CometTakeOrderedAndProject (30) - +- CometProject (29) - +- CometBroadcastHashJoin (28) - :- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(5) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#13, s_number_employees#14, s_city#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_number_employees), GreaterThanOrEqual(s_number_employees,200), LessThanOrEqual(s_number_employees,295), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [s_store_sk#13, s_number_employees#14, s_city#15] -Condition : (((isnotnull(s_number_employees#14) AND (s_number_employees#14 >= 200)) AND (s_number_employees#14 <= 295)) AND isnotnull(s_store_sk#13)) - -(11) CometProject -Input [3]: [s_store_sk#13, s_number_employees#14, s_city#15] -Arguments: [s_store_sk#13, s_city#15], [s_store_sk#13, s_city#15] - -(12) CometBroadcastExchange -Input [2]: [s_store_sk#13, s_city#15] -Arguments: [s_store_sk#13, s_city#15] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [2]: [s_store_sk#13, s_city#15] -Arguments: [ss_store_sk#4], [s_store_sk#13], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#13, s_city#15] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,6),GreaterThan(hd_vehicle_count,2)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Condition : (((hd_dep_count#17 = 6) OR (hd_vehicle_count#18 > 2)) AND isnotnull(hd_demo_sk#16)) - -(17) CometProject -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Arguments: [hd_demo_sk#16], [hd_demo_sk#16] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#16] -Arguments: [hd_demo_sk#16] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] -Right output [1]: [hd_demo_sk#16] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15, hd_demo_sk#16] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] - -(21) CometHashAggregate -Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15] -Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] - -(22) CometExchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#19, sum#20] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#19, sum#20] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15] -Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(25) CometFilter -Input [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] -Condition : isnotnull(c_customer_sk#21) - -(26) CometProject -Input [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] -Arguments: [c_customer_sk#21, c_first_name#24, c_last_name#25], [c_customer_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#22, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#23, 30, true, false, true) AS c_last_name#25] - -(27) CometBroadcastExchange -Input [3]: [c_customer_sk#21, c_first_name#24, c_last_name#25] -Arguments: [c_customer_sk#21, c_first_name#24, c_last_name#25] - -(28) CometBroadcastHashJoin -Left output [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#26, profit#27] -Right output [3]: [c_customer_sk#21, c_first_name#24, c_last_name#25] -Arguments: [ss_customer_sk#1], [c_customer_sk#21], Inner, BuildRight - -(29) CometProject -Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#26, profit#27, c_customer_sk#21, c_first_name#24, c_last_name#25] -Arguments: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15], [c_last_name#25, c_first_name#24, substr(s_city#15, 1, 30) AS substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15] - -(30) CometTakeOrderedAndProject -Input [7]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#25 ASC NULLS FIRST,c_first_name#24 ASC NULLS FIRST,substr(s_city#15, 1, 30) ASC NULLS FIRST,profit#27 ASC NULLS FIRST], output=[c_last_name#25,c_first_name#24,substr(s_city, 1, 30)#28,ss_ticket_number#5,amt#26,profit#27]), [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27], 100, 0, [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, substr(s_city#15, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27] - -(31) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(34) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(36) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/simplified.txt deleted file mode 100644 index 5b252a906b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_city] [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit] - CometProject [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit,s_city] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,s_city,amt,profit,c_customer_sk,c_first_name,c_last_name] - CometHashAggregate [ss_addr_sk,sum,sum] [ss_ticket_number,ss_customer_sk,s_city,amt,profit,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] - CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 - CometHashAggregate [ss_coupon_amt,ss_net_profit] [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] - CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - 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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [s_store_sk,s_city] #4 - CometProject [s_store_sk,s_city] - CometFilter [s_store_sk,s_number_employees,s_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #6 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] 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/q79/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt deleted file mode 100644 index 2d2c6dc938..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt +++ /dev/null @@ -1,212 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (31) -+- CometTakeOrderedAndProject (30) - +- CometProject (29) - +- CometBroadcastHashJoin (28) - :- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_store_sk#4) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(5) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#13, s_number_employees#14, s_city#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_number_employees), GreaterThanOrEqual(s_number_employees,200), LessThanOrEqual(s_number_employees,295), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [s_store_sk#13, s_number_employees#14, s_city#15] -Condition : (((isnotnull(s_number_employees#14) AND (s_number_employees#14 >= 200)) AND (s_number_employees#14 <= 295)) AND isnotnull(s_store_sk#13)) - -(11) CometProject -Input [3]: [s_store_sk#13, s_number_employees#14, s_city#15] -Arguments: [s_store_sk#13, s_city#15], [s_store_sk#13, s_city#15] - -(12) CometBroadcastExchange -Input [2]: [s_store_sk#13, s_city#15] -Arguments: [s_store_sk#13, s_city#15] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] -Right output [2]: [s_store_sk#13, s_city#15] -Arguments: [ss_store_sk#4], [s_store_sk#13], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#13, s_city#15] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15], [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(EqualTo(hd_dep_count,6),GreaterThan(hd_vehicle_count,2)), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Condition : (((hd_dep_count#17 = 6) OR (hd_vehicle_count#18 > 2)) AND isnotnull(hd_demo_sk#16)) - -(17) CometProject -Input [3]: [hd_demo_sk#16, hd_dep_count#17, hd_vehicle_count#18] -Arguments: [hd_demo_sk#16], [hd_demo_sk#16] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#16] -Arguments: [hd_demo_sk#16] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] -Right output [1]: [hd_demo_sk#16] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15, hd_demo_sk#16] -Arguments: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15], [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] - -(21) CometHashAggregate -Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#15] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15] -Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] - -(22) CometExchange -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#19, sum#20] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#19, sum#20] -Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15] -Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(25) CometFilter -Input [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] -Condition : isnotnull(c_customer_sk#21) - -(26) CometProject -Input [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] -Arguments: [c_customer_sk#21, c_first_name#24, c_last_name#25], [c_customer_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#22, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#23, 30, true, false, true) AS c_last_name#25] - -(27) CometBroadcastExchange -Input [3]: [c_customer_sk#21, c_first_name#24, c_last_name#25] -Arguments: [c_customer_sk#21, c_first_name#24, c_last_name#25] - -(28) CometBroadcastHashJoin -Left output [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#26, profit#27] -Right output [3]: [c_customer_sk#21, c_first_name#24, c_last_name#25] -Arguments: [ss_customer_sk#1], [c_customer_sk#21], Inner, BuildRight - -(29) CometProject -Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#26, profit#27, c_customer_sk#21, c_first_name#24, c_last_name#25] -Arguments: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15], [c_last_name#25, c_first_name#24, substr(s_city#15, 1, 30) AS substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15] - -(30) CometTakeOrderedAndProject -Input [7]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#25 ASC NULLS FIRST,c_first_name#24 ASC NULLS FIRST,substr(s_city#15, 1, 30) ASC NULLS FIRST,profit#27 ASC NULLS FIRST], output=[c_last_name#25,c_first_name#24,substr(s_city, 1, 30)#28,ss_ticket_number#5,amt#26,profit#27]), [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27], 100, 0, [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, substr(s_city#15, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27] - -(31) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#10, d_year#11, d_dow#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) - -(34) CometProject -Input [3]: [d_date_sk#10, d_year#11, d_dow#12] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(36) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/extended.txt deleted file mode 100644 index 012403275a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/extended.txt +++ /dev/null @@ -1,39 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 5b252a906b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_city] [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit] - CometProject [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit,s_city] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,s_city,amt,profit,c_customer_sk,c_first_name,c_last_name] - CometHashAggregate [ss_addr_sk,sum,sum] [ss_ticket_number,ss_customer_sk,s_city,amt,profit,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] - CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 - CometHashAggregate [ss_coupon_amt,ss_net_profit] [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] - CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] - 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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dow] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] - CometBroadcastExchange [s_store_sk,s_city] #4 - CometProject [s_store_sk,s_city] - CometFilter [s_store_sk,s_number_employees,s_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] - CometBroadcastExchange [hd_demo_sk] #5 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #6 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/explain.txt deleted file mode 100644 index 754a8871d1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/explain.txt +++ /dev/null @@ -1,284 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (44) -+- * HashAggregate (43) - +- * CometColumnarToRow (42) - +- CometColumnarExchange (41) - +- * HashAggregate (40) - +- * Project (39) - +- * BroadcastHashJoin Inner BuildRight (38) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) - : +- CometProject (9) - : +- CometFilter (8) - : +- CometNativeScan parquet spark_catalog.default.store (7) - +- BroadcastExchange (37) - +- * CometColumnarToRow (36) - +- CometHashAggregate (35) - +- CometExchange (34) - +- CometHashAggregate (33) - +- CometBroadcastHashJoin (32) - :- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.customer_address (14) - +- CometBroadcastExchange (31) - +- CometProject (30) - +- CometFilter (29) - +- CometHashAggregate (28) - +- CometExchange (27) - +- CometHashAggregate (26) - +- CometProject (25) - +- CometBroadcastHashJoin (24) - :- CometProject (19) - : +- CometFilter (18) - : +- CometNativeScan parquet spark_catalog.default.customer_address (17) - +- CometBroadcastExchange (23) - +- CometProject (22) - +- CometFilter (21) - +- CometNativeScan parquet spark_catalog.default.customer (20) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] - -(3) Filter [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(4) ReusedExchange [Reuses operator id: 49] -Output [1]: [d_date_sk#5] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [2]: [ss_store_sk#1, ss_net_profit#2] -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#6, s_store_name#7, s_zip#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [s_store_sk#6, s_store_name#7, s_zip#8] -Condition : (isnotnull(s_store_sk#6) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#8, 10, true, false, true))) - -(9) CometProject -Input [3]: [s_store_sk#6, s_store_name#7, s_zip#8] -Arguments: [s_store_sk#6, s_store_name#7, s_zip#9], [s_store_sk#6, s_store_name#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#8, 10, true, false, true) AS s_zip#9] - -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [s_store_sk#6, s_store_name#7, s_zip#9] - -(11) BroadcastExchange -Input [3]: [s_store_sk#6, s_store_name#7, s_zip#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#6] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_store_name#7, s_zip#9] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_store_name#7, s_zip#9] - -(14) CometNativeScan parquet spark_catalog.default.customer_address -Output [1]: [ca_zip#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -ReadSchema: struct - -(15) CometFilter -Input [1]: [ca_zip#10] -Condition : (substr(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#10, 10, true, false, true), 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#10, 10, true, false, true), 1, 5))) - -(16) CometProject -Input [1]: [ca_zip#10] -Arguments: [ca_zip#11], [substr(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#10, 10, true, false, true), 1, 5) AS ca_zip#11] - -(17) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#12, ca_zip#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [ca_address_sk#12, ca_zip#13] -Condition : isnotnull(ca_address_sk#12) - -(19) CometProject -Input [2]: [ca_address_sk#12, ca_zip#13] -Arguments: [ca_address_sk#12, ca_zip#14], [ca_address_sk#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#13, 10, true, false, true) AS ca_zip#14] - -(20) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#16, 1, true, false, true) = Y) AND isnotnull(c_current_addr_sk#15)) - -(22) CometProject -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] -Arguments: [c_current_addr_sk#15], [c_current_addr_sk#15] - -(23) CometBroadcastExchange -Input [1]: [c_current_addr_sk#15] -Arguments: [c_current_addr_sk#15] - -(24) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#12, ca_zip#14] -Right output [1]: [c_current_addr_sk#15] -Arguments: [ca_address_sk#12], [c_current_addr_sk#15], Inner, BuildRight - -(25) CometProject -Input [3]: [ca_address_sk#12, ca_zip#14, c_current_addr_sk#15] -Arguments: [ca_zip#14], [ca_zip#14] - -(26) CometHashAggregate -Input [1]: [ca_zip#14] -Keys [1]: [ca_zip#14] -Functions [1]: [partial_count(1)] - -(27) CometExchange -Input [2]: [ca_zip#14, count#17] -Arguments: hashpartitioning(ca_zip#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [2]: [ca_zip#14, count#17] -Keys [1]: [ca_zip#14] -Functions [1]: [count(1)] - -(29) CometFilter -Input [2]: [ca_zip#18, cnt#19] -Condition : (cnt#19 > 10) - -(30) CometProject -Input [2]: [ca_zip#18, cnt#19] -Arguments: [ca_zip#18], [ca_zip#18] - -(31) CometBroadcastExchange -Input [1]: [ca_zip#18] -Arguments: [ca_zip#18] - -(32) CometBroadcastHashJoin -Left output [1]: [ca_zip#11] -Right output [1]: [ca_zip#18] -Arguments: [coalesce(ca_zip#11, ), isnull(ca_zip#11)], [coalesce(ca_zip#18, ), isnull(ca_zip#18)], LeftSemi, BuildRight - -(33) CometHashAggregate -Input [1]: [ca_zip#11] -Keys [1]: [ca_zip#11] -Functions: [] - -(34) CometExchange -Input [1]: [ca_zip#11] -Arguments: hashpartitioning(ca_zip#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(35) CometHashAggregate -Input [1]: [ca_zip#11] -Keys [1]: [ca_zip#11] -Functions: [] - -(36) CometColumnarToRow [codegen id : 3] -Input [1]: [ca_zip#11] - -(37) BroadcastExchange -Input [1]: [ca_zip#11] -Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [substr(s_zip#9, 1, 2)] -Right keys [1]: [substr(ca_zip#11, 1, 2)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 4] -Output [2]: [ss_net_profit#2, s_store_name#7] -Input [4]: [ss_net_profit#2, s_store_name#7, s_zip#9, ca_zip#11] - -(40) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#2, s_store_name#7] -Keys [1]: [s_store_name#7] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#20] -Results [2]: [s_store_name#7, sum#21] - -(41) CometColumnarExchange -Input [2]: [s_store_name#7, sum#21] -Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(42) CometColumnarToRow [codegen id : 5] -Input [2]: [s_store_name#7, sum#21] - -(43) HashAggregate [codegen id : 5] -Input [2]: [s_store_name#7, sum#21] -Keys [1]: [s_store_name#7] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22] -Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS sum(ss_net_profit)#23] - -(44) TakeOrderedAndProject -Input [2]: [s_store_name#7, sum(ss_net_profit)#23] -Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometNativeScan parquet spark_catalog.default.date_dim (45) - - -(45) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#24, d_qoy#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [3]: [d_date_sk#5, d_year#24, d_qoy#25] -Condition : ((((isnotnull(d_qoy#25) AND isnotnull(d_year#24)) AND (d_qoy#25 = 2)) AND (d_year#24 = 1998)) AND isnotnull(d_date_sk#5)) - -(47) CometProject -Input [3]: [d_date_sk#5, d_year#24, d_qoy#25] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(49) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/simplified.txt deleted file mode 100644 index 4df6d8f659..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/simplified.txt +++ /dev/null @@ -1,63 +0,0 @@ -TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - WholeStageCodegen (5) - HashAggregate [s_store_name,sum] [sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit),sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_name] #1 - WholeStageCodegen (4) - HashAggregate [s_store_name,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_store_name] - BroadcastHashJoin [s_zip,ca_zip] - Project [ss_net_profit,s_store_name,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [ca_zip] - CometExchange [ca_zip] #5 - CometHashAggregate [ca_zip] - CometBroadcastHashJoin [ca_zip,ca_zip] - CometProject [ca_zip] [ca_zip] - CometFilter [ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_zip] - CometBroadcastExchange [ca_zip] #6 - CometProject [ca_zip] - CometFilter [ca_zip,cnt] - CometHashAggregate [count] [ca_zip,cnt,ca_zip,count(1)] - CometExchange [ca_zip] #7 - CometHashAggregate [ca_zip,count] - CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [c_current_addr_sk] #8 - CometProject [c_current_addr_sk] - CometFilter [c_current_addr_sk,c_preferred_cust_flag] - CometNativeScan 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/q8.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/explain.txt deleted file mode 100644 index 075f17ea3d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,284 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - +- CometBroadcastExchange (37) - +- CometHashAggregate (36) - +- CometExchange (35) - +- CometHashAggregate (34) - +- CometBroadcastHashJoin (33) - :- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (15) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (18) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) AND (d_year#6 = 1998)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#8, s_store_name#9, s_zip#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [s_store_sk#8, s_store_name#9, s_zip#10] -Condition : (isnotnull(s_store_sk#8) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#10, 10, true, false, true))) - -(11) CometProject -Input [3]: [s_store_sk#8, s_store_name#9, s_zip#10] -Arguments: [s_store_sk#8, s_store_name#9, s_zip#11], [s_store_sk#8, s_store_name#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#10, 10, true, false, true) AS s_zip#11] - -(12) CometBroadcastExchange -Input [3]: [s_store_sk#8, s_store_name#9, s_zip#11] -Arguments: [s_store_sk#8, s_store_name#9, s_zip#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_store_sk#1, ss_net_profit#2] -Right output [3]: [s_store_sk#8, s_store_name#9, s_zip#11] -Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_store_name#9, s_zip#11] -Arguments: [ss_net_profit#2, s_store_name#9, s_zip#11], [ss_net_profit#2, s_store_name#9, s_zip#11] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [1]: [ca_zip#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -ReadSchema: struct - -(16) CometFilter -Input [1]: [ca_zip#12] -Condition : (substr(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#12, 10, true, false, true), 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#12, 10, true, false, true), 1, 5))) - -(17) CometProject -Input [1]: [ca_zip#12] -Arguments: [ca_zip#13], [substr(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#12, 10, true, false, true), 1, 5) AS ca_zip#13] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#14, ca_zip#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [ca_address_sk#14, ca_zip#15] -Condition : isnotnull(ca_address_sk#14) - -(20) CometProject -Input [2]: [ca_address_sk#14, ca_zip#15] -Arguments: [ca_address_sk#14, ca_zip#16], [ca_address_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#15, 10, true, false, true) AS ca_zip#16] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#18, 1, true, false, true) = Y) AND isnotnull(c_current_addr_sk#17)) - -(23) CometProject -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -Arguments: [c_current_addr_sk#17], [c_current_addr_sk#17] - -(24) CometBroadcastExchange -Input [1]: [c_current_addr_sk#17] -Arguments: [c_current_addr_sk#17] - -(25) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#14, ca_zip#16] -Right output [1]: [c_current_addr_sk#17] -Arguments: [ca_address_sk#14], [c_current_addr_sk#17], Inner, BuildRight - -(26) CometProject -Input [3]: [ca_address_sk#14, ca_zip#16, c_current_addr_sk#17] -Arguments: [ca_zip#16], [ca_zip#16] - -(27) CometHashAggregate -Input [1]: [ca_zip#16] -Keys [1]: [ca_zip#16] -Functions [1]: [partial_count(1)] - -(28) CometExchange -Input [2]: [ca_zip#16, count#19] -Arguments: hashpartitioning(ca_zip#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [2]: [ca_zip#16, count#19] -Keys [1]: [ca_zip#16] -Functions [1]: [count(1)] - -(30) CometFilter -Input [2]: [ca_zip#20, cnt#21] -Condition : (cnt#21 > 10) - -(31) CometProject -Input [2]: [ca_zip#20, cnt#21] -Arguments: [ca_zip#20], [ca_zip#20] - -(32) CometBroadcastExchange -Input [1]: [ca_zip#20] -Arguments: [ca_zip#20] - -(33) CometBroadcastHashJoin -Left output [1]: [ca_zip#13] -Right output [1]: [ca_zip#20] -Arguments: [coalesce(ca_zip#13, ), isnull(ca_zip#13)], [coalesce(ca_zip#20, ), isnull(ca_zip#20)], LeftSemi, BuildRight - -(34) CometHashAggregate -Input [1]: [ca_zip#13] -Keys [1]: [ca_zip#13] -Functions: [] - -(35) CometExchange -Input [1]: [ca_zip#13] -Arguments: hashpartitioning(ca_zip#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(36) CometHashAggregate -Input [1]: [ca_zip#13] -Keys [1]: [ca_zip#13] -Functions: [] - -(37) CometBroadcastExchange -Input [1]: [ca_zip#13] -Arguments: [ca_zip#13] - -(38) CometBroadcastHashJoin -Left output [3]: [ss_net_profit#2, s_store_name#9, s_zip#11] -Right output [1]: [ca_zip#13] -Arguments: [substr(s_zip#11, 1, 2)], [substr(ca_zip#13, 1, 2)], Inner, BuildRight - -(39) CometProject -Input [4]: [ss_net_profit#2, s_store_name#9, s_zip#11, ca_zip#13] -Arguments: [ss_net_profit#2, s_store_name#9], [ss_net_profit#2, s_store_name#9] - -(40) CometHashAggregate -Input [2]: [ss_net_profit#2, s_store_name#9] -Keys [1]: [s_store_name#9] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] - -(41) CometExchange -Input [2]: [s_store_name#9, sum#22] -Arguments: hashpartitioning(s_store_name#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(42) CometHashAggregate -Input [2]: [s_store_name#9, sum#22] -Keys [1]: [s_store_name#9] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] - -(43) CometTakeOrderedAndProject -Input [2]: [s_store_name#9, sum(ss_net_profit)#23] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#9 ASC NULLS FIRST], output=[s_store_name#9,sum(ss_net_profit)#23]), [s_store_name#9, sum(ss_net_profit)#23], 100, 0, [s_store_name#9 ASC NULLS FIRST], [s_store_name#9, sum(ss_net_profit)#23] - -(44) CometColumnarToRow [codegen id : 1] -Input [2]: [s_store_name#9, sum(ss_net_profit)#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) AND (d_year#6 = 1998)) AND isnotnull(d_date_sk#5)) - -(47) CometProject -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(49) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/simplified.txt deleted file mode 100644 index d1542ab607..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - CometHashAggregate [sum] [s_store_name,sum(ss_net_profit),sum(UnscaledValue(ss_net_profit))] - CometExchange [s_store_name] #1 - CometHashAggregate [ss_net_profit] [s_store_name,sum] - CometProject [ss_net_profit,s_store_name] - CometBroadcastHashJoin [ss_net_profit,s_store_name,s_zip,ca_zip] - CometProject [ss_net_profit,s_store_name,s_zip] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] - CometProject [ss_store_sk,ss_net_profit] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [ca_zip] #5 - CometHashAggregate [ca_zip] - CometExchange [ca_zip] #6 - CometHashAggregate [ca_zip] - CometBroadcastHashJoin [ca_zip,ca_zip] - CometProject [ca_zip] [ca_zip] - CometFilter [ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_zip] - CometBroadcastExchange [ca_zip] #7 - CometProject [ca_zip] - CometFilter [ca_zip,cnt] - CometHashAggregate [count] [ca_zip,cnt,ca_zip,count(1)] - CometExchange [ca_zip] #8 - CometHashAggregate [ca_zip,count] - CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [c_current_addr_sk] #9 - CometProject [c_current_addr_sk] - CometFilter [c_current_addr_sk,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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/q8/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt deleted file mode 100644 index 075f17ea3d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt +++ /dev/null @@ -1,284 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometBroadcastHashJoin (38) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - +- CometBroadcastExchange (37) - +- CometHashAggregate (36) - +- CometExchange (35) - +- CometHashAggregate (34) - +- CometBroadcastHashJoin (33) - :- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (15) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (20) - : +- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (18) - +- CometBroadcastExchange (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (21) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) AND (d_year#6 = 1998)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#8, s_store_name#9, s_zip#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [s_store_sk#8, s_store_name#9, s_zip#10] -Condition : (isnotnull(s_store_sk#8) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#10, 10, true, false, true))) - -(11) CometProject -Input [3]: [s_store_sk#8, s_store_name#9, s_zip#10] -Arguments: [s_store_sk#8, s_store_name#9, s_zip#11], [s_store_sk#8, s_store_name#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#10, 10, true, false, true) AS s_zip#11] - -(12) CometBroadcastExchange -Input [3]: [s_store_sk#8, s_store_name#9, s_zip#11] -Arguments: [s_store_sk#8, s_store_name#9, s_zip#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_store_sk#1, ss_net_profit#2] -Right output [3]: [s_store_sk#8, s_store_name#9, s_zip#11] -Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_store_name#9, s_zip#11] -Arguments: [ss_net_profit#2, s_store_name#9, s_zip#11], [ss_net_profit#2, s_store_name#9, s_zip#11] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [1]: [ca_zip#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -ReadSchema: struct - -(16) CometFilter -Input [1]: [ca_zip#12] -Condition : (substr(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#12, 10, true, false, true), 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#12, 10, true, false, true), 1, 5))) - -(17) CometProject -Input [1]: [ca_zip#12] -Arguments: [ca_zip#13], [substr(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#12, 10, true, false, true), 1, 5) AS ca_zip#13] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#14, ca_zip#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [ca_address_sk#14, ca_zip#15] -Condition : isnotnull(ca_address_sk#14) - -(20) CometProject -Input [2]: [ca_address_sk#14, ca_zip#15] -Arguments: [ca_address_sk#14, ca_zip#16], [ca_address_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#15, 10, true, false, true) AS ca_zip#16] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#18, 1, true, false, true) = Y) AND isnotnull(c_current_addr_sk#17)) - -(23) CometProject -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -Arguments: [c_current_addr_sk#17], [c_current_addr_sk#17] - -(24) CometBroadcastExchange -Input [1]: [c_current_addr_sk#17] -Arguments: [c_current_addr_sk#17] - -(25) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#14, ca_zip#16] -Right output [1]: [c_current_addr_sk#17] -Arguments: [ca_address_sk#14], [c_current_addr_sk#17], Inner, BuildRight - -(26) CometProject -Input [3]: [ca_address_sk#14, ca_zip#16, c_current_addr_sk#17] -Arguments: [ca_zip#16], [ca_zip#16] - -(27) CometHashAggregate -Input [1]: [ca_zip#16] -Keys [1]: [ca_zip#16] -Functions [1]: [partial_count(1)] - -(28) CometExchange -Input [2]: [ca_zip#16, count#19] -Arguments: hashpartitioning(ca_zip#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [2]: [ca_zip#16, count#19] -Keys [1]: [ca_zip#16] -Functions [1]: [count(1)] - -(30) CometFilter -Input [2]: [ca_zip#20, cnt#21] -Condition : (cnt#21 > 10) - -(31) CometProject -Input [2]: [ca_zip#20, cnt#21] -Arguments: [ca_zip#20], [ca_zip#20] - -(32) CometBroadcastExchange -Input [1]: [ca_zip#20] -Arguments: [ca_zip#20] - -(33) CometBroadcastHashJoin -Left output [1]: [ca_zip#13] -Right output [1]: [ca_zip#20] -Arguments: [coalesce(ca_zip#13, ), isnull(ca_zip#13)], [coalesce(ca_zip#20, ), isnull(ca_zip#20)], LeftSemi, BuildRight - -(34) CometHashAggregate -Input [1]: [ca_zip#13] -Keys [1]: [ca_zip#13] -Functions: [] - -(35) CometExchange -Input [1]: [ca_zip#13] -Arguments: hashpartitioning(ca_zip#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(36) CometHashAggregate -Input [1]: [ca_zip#13] -Keys [1]: [ca_zip#13] -Functions: [] - -(37) CometBroadcastExchange -Input [1]: [ca_zip#13] -Arguments: [ca_zip#13] - -(38) CometBroadcastHashJoin -Left output [3]: [ss_net_profit#2, s_store_name#9, s_zip#11] -Right output [1]: [ca_zip#13] -Arguments: [substr(s_zip#11, 1, 2)], [substr(ca_zip#13, 1, 2)], Inner, BuildRight - -(39) CometProject -Input [4]: [ss_net_profit#2, s_store_name#9, s_zip#11, ca_zip#13] -Arguments: [ss_net_profit#2, s_store_name#9], [ss_net_profit#2, s_store_name#9] - -(40) CometHashAggregate -Input [2]: [ss_net_profit#2, s_store_name#9] -Keys [1]: [s_store_name#9] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] - -(41) CometExchange -Input [2]: [s_store_name#9, sum#22] -Arguments: hashpartitioning(s_store_name#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(42) CometHashAggregate -Input [2]: [s_store_name#9, sum#22] -Keys [1]: [s_store_name#9] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] - -(43) CometTakeOrderedAndProject -Input [2]: [s_store_name#9, sum(ss_net_profit)#23] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#9 ASC NULLS FIRST], output=[s_store_name#9,sum(ss_net_profit)#23]), [s_store_name#9, sum(ss_net_profit)#23], 100, 0, [s_store_name#9 ASC NULLS FIRST], [s_store_name#9, sum(ss_net_profit)#23] - -(44) CometColumnarToRow [codegen id : 1] -Input [2]: [s_store_name#9, sum(ss_net_profit)#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) - - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 2)) AND (d_year#6 = 1998)) AND isnotnull(d_date_sk#5)) - -(47) CometProject -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(48) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(49) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/extended.txt deleted file mode 100644 index 8e04bbeba7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/extended.txt +++ /dev/null @@ -1,52 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d1542ab607..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - CometHashAggregate [sum] [s_store_name,sum(ss_net_profit),sum(UnscaledValue(ss_net_profit))] - CometExchange [s_store_name] #1 - CometHashAggregate [ss_net_profit] [s_store_name,sum] - CometProject [ss_net_profit,s_store_name] - CometBroadcastHashJoin [ss_net_profit,s_store_name,s_zip,ca_zip] - CometProject [ss_net_profit,s_store_name,s_zip] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_store_name,s_zip] - CometProject [ss_store_sk,ss_net_profit] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #4 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [ca_zip] #5 - CometHashAggregate [ca_zip] - CometExchange [ca_zip] #6 - CometHashAggregate [ca_zip] - CometBroadcastHashJoin [ca_zip,ca_zip] - CometProject [ca_zip] [ca_zip] - CometFilter [ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_zip] - CometBroadcastExchange [ca_zip] #7 - CometProject [ca_zip] - CometFilter [ca_zip,cnt] - CometHashAggregate [count] [ca_zip,cnt,ca_zip,count(1)] - CometExchange [ca_zip] #8 - CometHashAggregate [ca_zip,count] - CometProject [ca_zip] - CometBroadcastHashJoin [ca_address_sk,ca_zip,c_current_addr_sk] - CometProject [ca_zip] [ca_address_sk,ca_zip] - CometFilter [ca_address_sk,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] - CometBroadcastExchange [c_current_addr_sk] #9 - CometProject [c_current_addr_sk] - CometFilter [c_current_addr_sk,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/explain.txt deleted file mode 100644 index f1e1a2ace1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/explain.txt +++ /dev/null @@ -1,623 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (106) -+- CometTakeOrderedAndProject (105) - +- CometHashAggregate (104) - +- CometExchange (103) - +- CometHashAggregate (102) - +- CometExpand (101) - +- CometUnion (100) - :- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometProject (18) - : : : : +- CometBroadcastHashJoin (17) - : : : : :- CometProject (12) - : : : : : +- CometSortMergeJoin (11) - : : : : : :- CometSort (5) - : : : : : : +- CometColumnarExchange (4) - : : : : : : +- * Filter (3) - : : : : : : +- * ColumnarToRow (2) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : +- CometSort (10) - : : : : : +- CometExchange (9) - : : : : : +- CometProject (8) - : : : : : +- CometFilter (7) - : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (6) - : : : : +- CometBroadcastExchange (16) - : : : : +- CometProject (15) - : : : : +- CometFilter (14) - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (13) - : : : +- CometBroadcastExchange (22) - : : : +- CometProject (21) - : : : +- CometFilter (20) - : : : +- CometNativeScan parquet spark_catalog.default.store (19) - : : +- CometBroadcastExchange (28) - : : +- CometProject (27) - : : +- CometFilter (26) - : : +- CometNativeScan parquet spark_catalog.default.item (25) - : +- CometBroadcastExchange (34) - : +- CometProject (33) - : +- CometFilter (32) - : +- CometNativeScan parquet spark_catalog.default.promotion (31) - :- CometHashAggregate (69) - : +- CometExchange (68) - : +- CometHashAggregate (67) - : +- CometProject (66) - : +- CometBroadcastHashJoin (65) - : :- CometProject (63) - : : +- CometBroadcastHashJoin (62) - : : :- CometProject (60) - : : : +- CometBroadcastHashJoin (59) - : : : :- CometProject (54) - : : : : +- CometBroadcastHashJoin (53) - : : : : :- CometProject (51) - : : : : : +- CometSortMergeJoin (50) - : : : : : :- CometSort (44) - : : : : : : +- CometColumnarExchange (43) - : : : : : : +- * Filter (42) - : : : : : : +- * ColumnarToRow (41) - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (40) - : : : : : +- CometSort (49) - : : : : : +- CometExchange (48) - : : : : : +- CometProject (47) - : : : : : +- CometFilter (46) - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (45) - : : : : +- ReusedExchange (52) - : : : +- CometBroadcastExchange (58) - : : : +- CometProject (57) - : : : +- CometFilter (56) - : : : +- CometNativeScan parquet spark_catalog.default.catalog_page (55) - : : +- ReusedExchange (61) - : +- ReusedExchange (64) - +- CometHashAggregate (99) - +- CometExchange (98) - +- CometHashAggregate (97) - +- CometProject (96) - +- CometBroadcastHashJoin (95) - :- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (84) - : : : +- CometBroadcastHashJoin (83) - : : : :- CometProject (81) - : : : : +- CometSortMergeJoin (80) - : : : : :- CometSort (74) - : : : : : +- CometColumnarExchange (73) - : : : : : +- * Filter (72) - : : : : : +- * ColumnarToRow (71) - : : : : : +- Scan parquet spark_catalog.default.web_sales (70) - : : : : +- CometSort (79) - : : : : +- CometExchange (78) - : : : : +- CometProject (77) - : : : : +- CometFilter (76) - : : : : +- CometNativeScan parquet spark_catalog.default.web_returns (75) - : : : +- ReusedExchange (82) - : : +- CometBroadcastExchange (88) - : : +- CometProject (87) - : : +- CometFilter (86) - : : +- CometNativeScan parquet spark_catalog.default.web_site (85) - : +- ReusedExchange (91) - +- ReusedExchange (94) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] - -(3) Filter [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(4) CometColumnarExchange -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(7) CometFilter -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) - -(8) CometProject -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(9) CometExchange -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter - -(12) CometProject -Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] - -(13) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) - -(15) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] - -(19) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_store_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [s_store_sk#16, s_store_id#17] -Condition : isnotnull(s_store_sk#16) - -(21) CometProject -Input [2]: [s_store_sk#16, s_store_id#17] -Arguments: [s_store_sk#16, s_store_id#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#17, 16, true, false, true) AS s_store_id#18] - -(22) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_store_id#18] -Arguments: [s_store_sk#16, s_store_id#18] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -Right output [2]: [s_store_sk#16, s_store_id#18] -Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight - -(24) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#18] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(25) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_current_price#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [i_item_sk#19, i_current_price#20] -Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) - -(27) CometProject -Input [2]: [i_item_sk#19, i_current_price#20] -Arguments: [i_item_sk#19], [i_item_sk#19] - -(28) CometBroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: [i_item_sk#19] - -(29) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [i_item_sk#19] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, i_item_sk#19] -Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(31) CometNativeScan parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#21, p_channel_tv#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#22, 1, true, false, true) = N) AND isnotnull(p_promo_sk#21)) - -(33) CometProject -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Arguments: [p_promo_sk#21], [p_promo_sk#21] - -(34) CometBroadcastExchange -Input [1]: [p_promo_sk#21] -Arguments: [p_promo_sk#21] - -(35) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [p_promo_sk#21] -Arguments: [ss_promo_sk#3], [p_promo_sk#21], Inner, BuildRight - -(36) CometProject -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, p_promo_sk#21] -Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(37) CometHashAggregate -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Keys [1]: [s_store_id#18] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(38) CometExchange -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(39) CometHashAggregate -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [s_store_id#18] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(40) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(41) ColumnarToRow [codegen id : 2] -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] - -(42) Filter [codegen id : 2] -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) - -(43) CometColumnarExchange -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(44) CometSort -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] - -(45) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38, cr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(46) CometFilter -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38, cr_returned_date_sk#39] -Condition : (isnotnull(cr_item_sk#35) AND isnotnull(cr_order_number#36)) - -(47) CometProject -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38, cr_returned_date_sk#39] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38], [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] - -(48) CometExchange -Input [4]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: hashpartitioning(cr_item_sk#35, cr_order_number#36, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(49) CometSort -Input [4]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38], [cr_item_sk#35 ASC NULLS FIRST, cr_order_number#36 ASC NULLS FIRST] - -(50) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#35, cr_order_number#36], LeftOuter - -(51) CometProject -Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38] - -(52) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#40] - -(53) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38] -Right output [1]: [d_date_sk#40] -Arguments: [cs_sold_date_sk#34], [d_date_sk#40], Inner, BuildRight - -(54) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38, d_date_sk#40] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38] - -(55) CometNativeScan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#42] -Condition : isnotnull(cp_catalog_page_sk#41) - -(57) CometProject -Input [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#42] -Arguments: [cp_catalog_page_sk#41, cp_catalog_page_id#43], [cp_catalog_page_sk#41, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#42, 16, true, false, true) AS cp_catalog_page_id#43] - -(58) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#43] -Arguments: [cp_catalog_page_sk#41, cp_catalog_page_id#43] - -(59) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38] -Right output [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#43] -Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#41], Inner, BuildRight - -(60) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_sk#41, cp_catalog_page_id#43] -Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] - -(61) ReusedExchange [Reuses operator id: 28] -Output [1]: [i_item_sk#44] - -(62) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] -Right output [1]: [i_item_sk#44] -Arguments: [cs_item_sk#29], [i_item_sk#44], Inner, BuildRight - -(63) CometProject -Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43, i_item_sk#44] -Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] - -(64) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#45] - -(65) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] -Right output [1]: [p_promo_sk#45] -Arguments: [cs_promo_sk#30], [p_promo_sk#45], Inner, BuildRight - -(66) CometProject -Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43, p_promo_sk#45] -Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] - -(67) CometHashAggregate -Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] -Keys [1]: [cp_catalog_page_id#43] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#37 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#38 as decimal(12,2)), 0.00)))] - -(68) CometExchange -Input [6]: [cp_catalog_page_id#43, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] -Arguments: hashpartitioning(cp_catalog_page_id#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(69) CometHashAggregate -Input [6]: [cp_catalog_page_id#43, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] -Keys [1]: [cp_catalog_page_id#43] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#37 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#38 as decimal(12,2)), 0.00)))] - -(70) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#57), dynamicpruningexpression(ws_sold_date_sk#57 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] -ReadSchema: struct - -(71) ColumnarToRow [codegen id : 3] -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] - -(72) Filter [codegen id : 3] -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Condition : ((isnotnull(ws_web_site_sk#52) AND isnotnull(ws_item_sk#51)) AND isnotnull(ws_promo_sk#53)) - -(73) CometColumnarExchange -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Arguments: hashpartitioning(ws_item_sk#51, ws_order_number#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(74) CometSort -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Arguments: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57], [ws_item_sk#51 ASC NULLS FIRST, ws_order_number#54 ASC NULLS FIRST] - -(75) CometNativeScan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] -ReadSchema: struct - -(76) CometFilter -Input [5]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Condition : (isnotnull(wr_item_sk#58) AND isnotnull(wr_order_number#59)) - -(77) CometProject -Input [5]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Arguments: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61], [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] - -(78) CometExchange -Input [4]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: hashpartitioning(wr_item_sk#58, wr_order_number#59, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(79) CometSort -Input [4]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61], [wr_item_sk#58 ASC NULLS FIRST, wr_order_number#59 ASC NULLS FIRST] - -(80) CometSortMergeJoin -Left output [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Right output [4]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: [ws_item_sk#51, ws_order_number#54], [wr_item_sk#58, wr_order_number#59], LeftOuter - -(81) CometProject -Input [11]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61], [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61] - -(82) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#63] - -(83) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61] -Right output [1]: [d_date_sk#63] -Arguments: [ws_sold_date_sk#57], [d_date_sk#63], Inner, BuildRight - -(84) CometProject -Input [9]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61, d_date_sk#63] -Arguments: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61], [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61] - -(85) CometNativeScan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#64, web_site_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(86) CometFilter -Input [2]: [web_site_sk#64, web_site_id#65] -Condition : isnotnull(web_site_sk#64) - -(87) CometProject -Input [2]: [web_site_sk#64, web_site_id#65] -Arguments: [web_site_sk#64, web_site_id#66], [web_site_sk#64, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#65, 16, true, false, true) AS web_site_id#66] - -(88) CometBroadcastExchange -Input [2]: [web_site_sk#64, web_site_id#66] -Arguments: [web_site_sk#64, web_site_id#66] - -(89) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61] -Right output [2]: [web_site_sk#64, web_site_id#66] -Arguments: [ws_web_site_sk#52], [web_site_sk#64], Inner, BuildRight - -(90) CometProject -Input [9]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_sk#64, web_site_id#66] -Arguments: [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66], [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] - -(91) ReusedExchange [Reuses operator id: 28] -Output [1]: [i_item_sk#67] - -(92) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] -Right output [1]: [i_item_sk#67] -Arguments: [ws_item_sk#51], [i_item_sk#67], Inner, BuildRight - -(93) CometProject -Input [8]: [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66, i_item_sk#67] -Arguments: [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66], [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] - -(94) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#68] - -(95) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] -Right output [1]: [p_promo_sk#68] -Arguments: [ws_promo_sk#53], [p_promo_sk#68], Inner, BuildRight - -(96) CometProject -Input [7]: [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66, p_promo_sk#68] -Arguments: [ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66], [ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] - -(97) CometHashAggregate -Input [5]: [ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] -Keys [1]: [web_site_id#66] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#55)), partial_sum(coalesce(cast(wr_return_amt#60 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#56 - coalesce(cast(wr_net_loss#61 as decimal(12,2)), 0.00)))] - -(98) CometExchange -Input [6]: [web_site_id#66, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Arguments: hashpartitioning(web_site_id#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(99) CometHashAggregate -Input [6]: [web_site_id#66, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Keys [1]: [web_site_id#66] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#55)), sum(coalesce(cast(wr_return_amt#60 as decimal(12,2)), 0.00)), sum((ws_net_profit#56 - coalesce(cast(wr_net_loss#61 as decimal(12,2)), 0.00)))] - -(100) CometUnion -Child 0 Input [5]: [sales#74, returns#75, profit#76, channel#77, id#78] -Child 1 Input [5]: [sales#79, returns#80, profit#81, channel#82, id#83] -Child 2 Input [5]: [sales#84, returns#85, profit#86, channel#87, id#88] - -(101) CometExpand -Input [5]: [sales#74, returns#75, profit#76, channel#77, id#78] -Arguments: [[sales#74, returns#75, profit#76, channel#77, id#78, 0], [sales#74, returns#75, profit#76, channel#77, null, 1], [sales#74, returns#75, profit#76, null, null, 3]], [sales#74, returns#75, profit#76, channel#89, id#90, spark_grouping_id#91] - -(102) CometHashAggregate -Input [6]: [sales#74, returns#75, profit#76, channel#89, id#90, spark_grouping_id#91] -Keys [3]: [channel#89, id#90, spark_grouping_id#91] -Functions [3]: [partial_sum(sales#74), partial_sum(returns#75), partial_sum(profit#76)] - -(103) CometExchange -Input [9]: [channel#89, id#90, spark_grouping_id#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97] -Arguments: hashpartitioning(channel#89, id#90, spark_grouping_id#91, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(104) CometHashAggregate -Input [9]: [channel#89, id#90, spark_grouping_id#91, sum#92, isEmpty#93, sum#94, isEmpty#95, sum#96, isEmpty#97] -Keys [3]: [channel#89, id#90, spark_grouping_id#91] -Functions [3]: [sum(sales#74), sum(returns#75), sum(profit#76)] - -(105) CometTakeOrderedAndProject -Input [5]: [channel#89, id#90, sales#98, returns#99, profit#100] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#89 ASC NULLS FIRST,id#90 ASC NULLS FIRST], output=[channel#89,id#90,sales#98,returns#99,profit#100]), [channel#89, id#90, sales#98, returns#99, profit#100], 100, 0, [channel#89 ASC NULLS FIRST, id#90 ASC NULLS FIRST], [channel#89, id#90, sales#98, returns#99, profit#100] - -(106) CometColumnarToRow [codegen id : 4] -Input [5]: [channel#89, id#90, sales#98, returns#99, profit#100] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (111) -+- * CometColumnarToRow (110) - +- CometProject (109) - +- CometFilter (108) - +- CometNativeScan parquet spark_catalog.default.date_dim (107) - - -(107) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] -ReadSchema: struct - -(108) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) - -(109) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(110) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(111) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#57 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/simplified.txt deleted file mode 100644 index 09e9224cef..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/simplified.txt +++ /dev/null @@ -1,124 +0,0 @@ -WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id,spark_grouping_id] #1 - CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] - CometUnion [sales,returns,profit,channel,id] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),s_store_id] - CometExchange [s_store_id] #2 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk,ss_ticket_number] #3 - WholeStageCodegen (1) - Filter [ss_store_sk,ss_item_sk,ss_promo_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #5 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #7 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #9 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,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))),cp_catalog_page_id] - CometExchange [cp_catalog_page_id] #10 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometColumnarExchange [cs_item_sk,cs_order_number] #11 - WholeStageCodegen (2) - Filter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),web_site_id] - CometExchange [web_site_id] #14 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometColumnarExchange [ws_item_sk,ws_order_number] #15 - WholeStageCodegen (3) - Filter [ws_web_site_sk,ws_item_sk,ws_promo_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #16 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [web_site_sk,web_site_id] #17 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/explain.txt deleted file mode 100644 index f0a163bf69..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,611 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (103) -+- CometTakeOrderedAndProject (102) - +- CometHashAggregate (101) - +- CometExchange (100) - +- CometHashAggregate (99) - +- CometExpand (98) - +- CometUnion (97) - :- CometHashAggregate (38) - : +- CometExchange (37) - : +- CometHashAggregate (36) - : +- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometProject (29) - : : +- CometBroadcastHashJoin (28) - : : :- CometProject (23) - : : : +- CometBroadcastHashJoin (22) - : : : :- CometProject (17) - : : : : +- CometBroadcastHashJoin (16) - : : : : :- CometProject (11) - : : : : : +- CometSortMergeJoin (10) - : : : : : :- CometSort (4) - : : : : : : +- CometExchange (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : +- CometSort (9) - : : : : : +- CometExchange (8) - : : : : : +- CometProject (7) - : : : : : +- CometFilter (6) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : +- CometBroadcastExchange (15) - : : : : +- CometProject (14) - : : : : +- CometFilter (13) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) - : : : +- CometBroadcastExchange (21) - : : : +- CometProject (20) - : : : +- CometFilter (19) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) - : : +- CometBroadcastExchange (27) - : : +- CometProject (26) - : : +- CometFilter (25) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - : +- CometBroadcastExchange (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - :- CometHashAggregate (67) - : +- CometExchange (66) - : +- CometHashAggregate (65) - : +- CometProject (64) - : +- CometBroadcastHashJoin (63) - : :- CometProject (61) - : : +- CometBroadcastHashJoin (60) - : : :- CometProject (58) - : : : +- CometBroadcastHashJoin (57) - : : : :- CometProject (52) - : : : : +- CometBroadcastHashJoin (51) - : : : : :- CometProject (49) - : : : : : +- CometSortMergeJoin (48) - : : : : : :- CometSort (42) - : : : : : : +- CometExchange (41) - : : : : : : +- CometFilter (40) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) - : : : : : +- CometSort (47) - : : : : : +- CometExchange (46) - : : : : : +- CometProject (45) - : : : : : +- CometFilter (44) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) - : : : : +- ReusedExchange (50) - : : : +- CometBroadcastExchange (56) - : : : +- CometProject (55) - : : : +- CometFilter (54) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) - : : +- ReusedExchange (59) - : +- ReusedExchange (62) - +- CometHashAggregate (96) - +- CometExchange (95) - +- CometHashAggregate (94) - +- CometProject (93) - +- CometBroadcastHashJoin (92) - :- CometProject (90) - : +- CometBroadcastHashJoin (89) - : :- CometProject (87) - : : +- CometBroadcastHashJoin (86) - : : :- CometProject (81) - : : : +- CometBroadcastHashJoin (80) - : : : :- CometProject (78) - : : : : +- CometSortMergeJoin (77) - : : : : :- CometSort (71) - : : : : : +- CometExchange (70) - : : : : : +- CometFilter (69) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) - : : : : +- CometSort (76) - : : : : +- CometExchange (75) - : : : : +- CometProject (74) - : : : : +- CometFilter (73) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) - : : : +- ReusedExchange (79) - : : +- CometBroadcastExchange (85) - : : +- CometProject (84) - : : +- CometFilter (83) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) - : +- ReusedExchange (88) - +- ReusedExchange (91) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) - -(7) CometProject -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(8) CometExchange -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter - -(11) CometProject -Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) - -(14) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(15) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(16) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight - -(17) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_store_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [s_store_sk#16, s_store_id#17] -Condition : isnotnull(s_store_sk#16) - -(20) CometProject -Input [2]: [s_store_sk#16, s_store_id#17] -Arguments: [s_store_sk#16, s_store_id#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#17, 16, true, false, true) AS s_store_id#18] - -(21) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_store_id#18] -Arguments: [s_store_sk#16, s_store_id#18] - -(22) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -Right output [2]: [s_store_sk#16, s_store_id#18] -Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight - -(23) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#18] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_current_price#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [i_item_sk#19, i_current_price#20] -Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) - -(26) CometProject -Input [2]: [i_item_sk#19, i_current_price#20] -Arguments: [i_item_sk#19], [i_item_sk#19] - -(27) CometBroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: [i_item_sk#19] - -(28) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [i_item_sk#19] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(29) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, i_item_sk#19] -Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#21, p_channel_tv#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#22, 1, true, false, true) = N) AND isnotnull(p_promo_sk#21)) - -(32) CometProject -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Arguments: [p_promo_sk#21], [p_promo_sk#21] - -(33) CometBroadcastExchange -Input [1]: [p_promo_sk#21] -Arguments: [p_promo_sk#21] - -(34) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [p_promo_sk#21] -Arguments: [ss_promo_sk#3], [p_promo_sk#21], Inner, BuildRight - -(35) CometProject -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, p_promo_sk#21] -Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(36) CometHashAggregate -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Keys [1]: [s_store_id#18] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(37) CometExchange -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [s_store_id#18] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(40) CometFilter -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) - -(41) CometExchange -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(42) CometSort -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(44) CometFilter -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) - -(45) CometProject -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] - -(46) CometExchange -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometSort -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] - -(48) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter - -(49) CometProject -Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] - -(50) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#41] - -(51) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -Right output [1]: [d_date_sk#41] -Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight - -(52) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(54) CometFilter -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Condition : isnotnull(cp_catalog_page_sk#42) - -(55) CometProject -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#43, 16, true, false, true) AS cp_catalog_page_id#44] - -(56) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] - -(57) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight - -(58) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(59) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#45] - -(60) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [i_item_sk#45] -Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight - -(61) CometProject -Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] -Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(62) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#46] - -(63) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [p_promo_sk#46] -Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight - -(64) CometProject -Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] -Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(65) CometHashAggregate -Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(66) CometExchange -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(67) CometHashAggregate -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] -ReadSchema: struct - -(69) CometFilter -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) - -(70) CometExchange -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(71) CometSort -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] -ReadSchema: struct - -(73) CometFilter -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) - -(74) CometProject -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] - -(75) CometExchange -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(76) CometSort -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] - -(77) CometSortMergeJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter - -(78) CometProject -Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] - -(79) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#65] - -(80) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -Right output [1]: [d_date_sk#65] -Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(81) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] - -(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#66, web_site_id#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(83) CometFilter -Input [2]: [web_site_sk#66, web_site_id#67] -Condition : isnotnull(web_site_sk#66) - -(84) CometProject -Input [2]: [web_site_sk#66, web_site_id#67] -Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#67, 16, true, false, true) AS web_site_id#68] - -(85) CometBroadcastExchange -Input [2]: [web_site_sk#66, web_site_id#68] -Arguments: [web_site_sk#66, web_site_id#68] - -(86) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -Right output [2]: [web_site_sk#66, web_site_id#68] -Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight - -(87) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] -Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(88) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#69] - -(89) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [i_item_sk#69] -Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight - -(90) CometProject -Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] -Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(91) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#70] - -(92) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [p_promo_sk#70] -Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight - -(93) CometProject -Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] -Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(94) CometHashAggregate -Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Keys [1]: [web_site_id#68] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(95) CometExchange -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(96) CometHashAggregate -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Keys [1]: [web_site_id#68] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(97) CometUnion -Child 0 Input [5]: [sales#76, returns#77, profit#78, channel#79, id#80] -Child 1 Input [5]: [sales#81, returns#82, profit#83, channel#84, id#85] -Child 2 Input [5]: [sales#86, returns#87, profit#88, channel#89, id#90] - -(98) CometExpand -Input [5]: [sales#76, returns#77, profit#78, channel#79, id#80] -Arguments: [[sales#76, returns#77, profit#78, channel#79, id#80, 0], [sales#76, returns#77, profit#78, channel#79, null, 1], [sales#76, returns#77, profit#78, null, null, 3]], [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] - -(99) CometHashAggregate -Input [6]: [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] -Keys [3]: [channel#91, id#92, spark_grouping_id#93] -Functions [3]: [partial_sum(sales#76), partial_sum(returns#77), partial_sum(profit#78)] - -(100) CometExchange -Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] -Arguments: hashpartitioning(channel#91, id#92, spark_grouping_id#93, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(101) CometHashAggregate -Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] -Keys [3]: [channel#91, id#92, spark_grouping_id#93] -Functions [3]: [sum(sales#76), sum(returns#77), sum(profit#78)] - -(102) CometTakeOrderedAndProject -Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#91 ASC NULLS FIRST,id#92 ASC NULLS FIRST], output=[channel#91,id#92,sales#100,returns#101,profit#102]), [channel#91, id#92, sales#100, returns#101, profit#102], 100, 0, [channel#91 ASC NULLS FIRST, id#92 ASC NULLS FIRST], [channel#91, id#92, sales#100, returns#101, profit#102] - -(103) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (108) -+- * CometColumnarToRow (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) - - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) - -(106) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(107) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(108) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/simplified.txt deleted file mode 100644 index 198646e0fd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,115 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id,spark_grouping_id] #1 - CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] - CometUnion [sales,returns,profit,channel,id] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),s_store_id] - CometExchange [s_store_id] #2 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #3 - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #5 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #7 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #9 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,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))),cp_catalog_page_id] - CometExchange [cp_catalog_page_id] #10 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #11 - 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 [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),web_site_id] - CometExchange [web_site_id] #14 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #15 - 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 [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #16 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [web_site_sk,web_site_id] #17 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt deleted file mode 100644 index f0a163bf69..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt +++ /dev/null @@ -1,611 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (103) -+- CometTakeOrderedAndProject (102) - +- CometHashAggregate (101) - +- CometExchange (100) - +- CometHashAggregate (99) - +- CometExpand (98) - +- CometUnion (97) - :- CometHashAggregate (38) - : +- CometExchange (37) - : +- CometHashAggregate (36) - : +- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometProject (29) - : : +- CometBroadcastHashJoin (28) - : : :- CometProject (23) - : : : +- CometBroadcastHashJoin (22) - : : : :- CometProject (17) - : : : : +- CometBroadcastHashJoin (16) - : : : : :- CometProject (11) - : : : : : +- CometSortMergeJoin (10) - : : : : : :- CometSort (4) - : : : : : : +- CometExchange (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : +- CometSort (9) - : : : : : +- CometExchange (8) - : : : : : +- CometProject (7) - : : : : : +- CometFilter (6) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : +- CometBroadcastExchange (15) - : : : : +- CometProject (14) - : : : : +- CometFilter (13) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) - : : : +- CometBroadcastExchange (21) - : : : +- CometProject (20) - : : : +- CometFilter (19) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) - : : +- CometBroadcastExchange (27) - : : +- CometProject (26) - : : +- CometFilter (25) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - : +- CometBroadcastExchange (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - :- CometHashAggregate (67) - : +- CometExchange (66) - : +- CometHashAggregate (65) - : +- CometProject (64) - : +- CometBroadcastHashJoin (63) - : :- CometProject (61) - : : +- CometBroadcastHashJoin (60) - : : :- CometProject (58) - : : : +- CometBroadcastHashJoin (57) - : : : :- CometProject (52) - : : : : +- CometBroadcastHashJoin (51) - : : : : :- CometProject (49) - : : : : : +- CometSortMergeJoin (48) - : : : : : :- CometSort (42) - : : : : : : +- CometExchange (41) - : : : : : : +- CometFilter (40) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) - : : : : : +- CometSort (47) - : : : : : +- CometExchange (46) - : : : : : +- CometProject (45) - : : : : : +- CometFilter (44) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) - : : : : +- ReusedExchange (50) - : : : +- CometBroadcastExchange (56) - : : : +- CometProject (55) - : : : +- CometFilter (54) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) - : : +- ReusedExchange (59) - : +- ReusedExchange (62) - +- CometHashAggregate (96) - +- CometExchange (95) - +- CometHashAggregate (94) - +- CometProject (93) - +- CometBroadcastHashJoin (92) - :- CometProject (90) - : +- CometBroadcastHashJoin (89) - : :- CometProject (87) - : : +- CometBroadcastHashJoin (86) - : : :- CometProject (81) - : : : +- CometBroadcastHashJoin (80) - : : : :- CometProject (78) - : : : : +- CometSortMergeJoin (77) - : : : : :- CometSort (71) - : : : : : +- CometExchange (70) - : : : : : +- CometFilter (69) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) - : : : : +- CometSort (76) - : : : : +- CometExchange (75) - : : : : +- CometProject (74) - : : : : +- CometFilter (73) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) - : : : +- ReusedExchange (79) - : : +- CometBroadcastExchange (85) - : : +- CometProject (84) - : : +- CometFilter (83) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) - : +- ReusedExchange (88) - +- ReusedExchange (91) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) - -(7) CometProject -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(8) CometExchange -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter - -(11) CometProject -Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) - -(14) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(15) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(16) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight - -(17) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_store_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [s_store_sk#16, s_store_id#17] -Condition : isnotnull(s_store_sk#16) - -(20) CometProject -Input [2]: [s_store_sk#16, s_store_id#17] -Arguments: [s_store_sk#16, s_store_id#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#17, 16, true, false, true) AS s_store_id#18] - -(21) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_store_id#18] -Arguments: [s_store_sk#16, s_store_id#18] - -(22) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -Right output [2]: [s_store_sk#16, s_store_id#18] -Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight - -(23) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#18] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_current_price#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [i_item_sk#19, i_current_price#20] -Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) - -(26) CometProject -Input [2]: [i_item_sk#19, i_current_price#20] -Arguments: [i_item_sk#19], [i_item_sk#19] - -(27) CometBroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: [i_item_sk#19] - -(28) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [i_item_sk#19] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(29) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, i_item_sk#19] -Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#21, p_channel_tv#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#22, 1, true, false, true) = N) AND isnotnull(p_promo_sk#21)) - -(32) CometProject -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Arguments: [p_promo_sk#21], [p_promo_sk#21] - -(33) CometBroadcastExchange -Input [1]: [p_promo_sk#21] -Arguments: [p_promo_sk#21] - -(34) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [p_promo_sk#21] -Arguments: [ss_promo_sk#3], [p_promo_sk#21], Inner, BuildRight - -(35) CometProject -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, p_promo_sk#21] -Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(36) CometHashAggregate -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Keys [1]: [s_store_id#18] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(37) CometExchange -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [s_store_id#18] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(40) CometFilter -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) - -(41) CometExchange -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(42) CometSort -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(44) CometFilter -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) - -(45) CometProject -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] - -(46) CometExchange -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometSort -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] - -(48) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter - -(49) CometProject -Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] - -(50) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#41] - -(51) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -Right output [1]: [d_date_sk#41] -Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight - -(52) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(54) CometFilter -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Condition : isnotnull(cp_catalog_page_sk#42) - -(55) CometProject -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#43, 16, true, false, true) AS cp_catalog_page_id#44] - -(56) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] - -(57) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight - -(58) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(59) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#45] - -(60) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [i_item_sk#45] -Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight - -(61) CometProject -Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] -Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(62) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#46] - -(63) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [p_promo_sk#46] -Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight - -(64) CometProject -Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] -Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(65) CometHashAggregate -Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(66) CometExchange -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(67) CometHashAggregate -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] -ReadSchema: struct - -(69) CometFilter -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) - -(70) CometExchange -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(71) CometSort -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] -ReadSchema: struct - -(73) CometFilter -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) - -(74) CometProject -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] - -(75) CometExchange -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(76) CometSort -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] - -(77) CometSortMergeJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter - -(78) CometProject -Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] - -(79) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#65] - -(80) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -Right output [1]: [d_date_sk#65] -Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(81) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] - -(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#66, web_site_id#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(83) CometFilter -Input [2]: [web_site_sk#66, web_site_id#67] -Condition : isnotnull(web_site_sk#66) - -(84) CometProject -Input [2]: [web_site_sk#66, web_site_id#67] -Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#67, 16, true, false, true) AS web_site_id#68] - -(85) CometBroadcastExchange -Input [2]: [web_site_sk#66, web_site_id#68] -Arguments: [web_site_sk#66, web_site_id#68] - -(86) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -Right output [2]: [web_site_sk#66, web_site_id#68] -Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight - -(87) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] -Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(88) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#69] - -(89) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [i_item_sk#69] -Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight - -(90) CometProject -Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] -Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(91) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#70] - -(92) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [p_promo_sk#70] -Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight - -(93) CometProject -Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] -Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(94) CometHashAggregate -Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Keys [1]: [web_site_id#68] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(95) CometExchange -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(96) CometHashAggregate -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Keys [1]: [web_site_id#68] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(97) CometUnion -Child 0 Input [5]: [sales#76, returns#77, profit#78, channel#79, id#80] -Child 1 Input [5]: [sales#81, returns#82, profit#83, channel#84, id#85] -Child 2 Input [5]: [sales#86, returns#87, profit#88, channel#89, id#90] - -(98) CometExpand -Input [5]: [sales#76, returns#77, profit#78, channel#79, id#80] -Arguments: [[sales#76, returns#77, profit#78, channel#79, id#80, 0], [sales#76, returns#77, profit#78, channel#79, null, 1], [sales#76, returns#77, profit#78, null, null, 3]], [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] - -(99) CometHashAggregate -Input [6]: [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] -Keys [3]: [channel#91, id#92, spark_grouping_id#93] -Functions [3]: [partial_sum(sales#76), partial_sum(returns#77), partial_sum(profit#78)] - -(100) CometExchange -Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] -Arguments: hashpartitioning(channel#91, id#92, spark_grouping_id#93, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(101) CometHashAggregate -Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] -Keys [3]: [channel#91, id#92, spark_grouping_id#93] -Functions [3]: [sum(sales#76), sum(returns#77), sum(profit#78)] - -(102) CometTakeOrderedAndProject -Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#91 ASC NULLS FIRST,id#92 ASC NULLS FIRST], output=[channel#91,id#92,sales#100,returns#101,profit#102]), [channel#91, id#92, sales#100, returns#101, profit#102], 100, 0, [channel#91 ASC NULLS FIRST, id#92 ASC NULLS FIRST], [channel#91, id#92, sales#100, returns#101, profit#102] - -(103) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (108) -+- * CometColumnarToRow (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) - - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) - -(106) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(107) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(108) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/extended.txt deleted file mode 100644 index ae6fc19b60..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/extended.txt +++ /dev/null @@ -1,131 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - -Comet accelerated 123 out of 127 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 198646e0fd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt +++ /dev/null @@ -1,115 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id,spark_grouping_id] #1 - CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] - CometUnion [sales,returns,profit,channel,id] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),s_store_id] - CometExchange [s_store_id] #2 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #3 - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #5 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #7 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #9 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,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))),cp_catalog_page_id] - CometExchange [cp_catalog_page_id] #10 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #11 - 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 [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),web_site_id] - CometExchange [web_site_id] #14 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #15 - 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 [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #16 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [web_site_sk,web_site_id] #17 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/explain.txt deleted file mode 100644 index 63856aaca5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/explain.txt +++ /dev/null @@ -1,346 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (54) -+- * Project (53) - +- * BroadcastHashJoin Inner BuildRight (52) - :- * Project (46) - : +- * BroadcastHashJoin Inner BuildRight (45) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- * HashAggregate (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.catalog_returns (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (11) - : : : +- * CometColumnarToRow (10) - : : : +- CometProject (9) - : : : +- CometFilter (8) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (7) - : : +- BroadcastExchange (37) - : : +- * Filter (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * HashAggregate (31) - : : +- * CometColumnarToRow (30) - : : +- CometColumnarExchange (29) - : : +- * HashAggregate (28) - : : +- * Project (27) - : : +- * BroadcastHashJoin Inner BuildRight (26) - : : :- * Project (24) - : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : :- * Filter (21) - : : : : +- * ColumnarToRow (20) - : : : : +- Scan parquet spark_catalog.default.catalog_returns (19) - : : : +- ReusedExchange (22) - : : +- ReusedExchange (25) - : +- BroadcastExchange (44) - : +- * CometColumnarToRow (43) - : +- CometProject (42) - : +- CometFilter (41) - : +- CometNativeScan parquet spark_catalog.default.customer (40) - +- BroadcastExchange (51) - +- * CometColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometNativeScan parquet spark_catalog.default.customer_address (47) - - -(1) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] - -(3) Filter [codegen id : 3] -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Condition : (isnotnull(cr_returning_addr_sk#2) AND isnotnull(cr_returning_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#7, ca_state#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [ca_address_sk#7, ca_state#8] -Condition : (isnotnull(ca_address_sk#7) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#8, 2, true, false, true))) - -(9) CometProject -Input [2]: [ca_address_sk#7, ca_state#8] -Arguments: [ca_address_sk#7, ca_state#9], [ca_address_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#8, 2, true, false, true) AS ca_state#9] - -(10) CometColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#7, ca_state#9] - -(11) BroadcastExchange -Input [2]: [ca_address_sk#7, ca_state#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cr_returning_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#7, ca_state#9] - -(14) HashAggregate [codegen id : 3] -Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] -Keys [2]: [cr_returning_customer_sk#1, ca_state#9] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum#10] -Results [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] - -(15) CometColumnarExchange -Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 11] -Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] - -(17) HashAggregate [codegen id : 11] -Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#11] -Keys [2]: [cr_returning_customer_sk#1, ca_state#9] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#12] -Results [3]: [cr_returning_customer_sk#1 AS ctr_customer_sk#13, ca_state#9 AS ctr_state#14, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#12,17,2) AS ctr_total_return#15] - -(18) Filter [codegen id : 11] -Input [3]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15] -Condition : isnotnull(ctr_total_return#15) - -(19) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cr_returning_addr_sk)] -ReadSchema: struct - -(20) ColumnarToRow [codegen id : 6] -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] - -(21) Filter [codegen id : 6] -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Condition : isnotnull(cr_returning_addr_sk#2) - -(22) ReusedExchange [Reuses operator id: 59] -Output [1]: [d_date_sk#6] - -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] - -(25) ReusedExchange [Reuses operator id: 11] -Output [2]: [ca_address_sk#7, ca_state#9] - -(26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returning_addr_sk#2] -Right keys [1]: [ca_address_sk#7] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#7, ca_state#9] - -(28) HashAggregate [codegen id : 6] -Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#9] -Keys [2]: [cr_returning_customer_sk#1, ca_state#9] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum#16] -Results [3]: [cr_returning_customer_sk#1, ca_state#9, sum#17] - -(29) CometColumnarExchange -Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#17] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(30) CometColumnarToRow [codegen id : 7] -Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#17] - -(31) HashAggregate [codegen id : 7] -Input [3]: [cr_returning_customer_sk#1, ca_state#9, sum#17] -Keys [2]: [cr_returning_customer_sk#1, ca_state#9] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#12] -Results [2]: [ca_state#9 AS ctr_state#14, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#12,17,2) AS ctr_total_return#15] - -(32) HashAggregate [codegen id : 7] -Input [2]: [ctr_state#14, ctr_total_return#15] -Keys [1]: [ctr_state#14] -Functions [1]: [partial_avg(ctr_total_return#15)] -Aggregate Attributes [2]: [sum#18, count#19] -Results [3]: [ctr_state#14, sum#20, count#21] - -(33) CometColumnarExchange -Input [3]: [ctr_state#14, sum#20, count#21] -Arguments: hashpartitioning(ctr_state#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 8] -Input [3]: [ctr_state#14, sum#20, count#21] - -(35) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#14, sum#20, count#21] -Keys [1]: [ctr_state#14] -Functions [1]: [avg(ctr_total_return#15)] -Aggregate Attributes [1]: [avg(ctr_total_return#15)#22] -Results [2]: [(avg(ctr_total_return#15)#22 * 1.2) AS (avg(ctr_total_return) * 1.2)#23, ctr_state#14 AS ctr_state#14#24] - -(36) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#14#24] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) - -(37) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_state#14#24] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] - -(38) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_state#14] -Right keys [1]: [ctr_state#14#24] -Join type: Inner -Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#23) - -(39) Project [codegen id : 11] -Output [2]: [ctr_customer_sk#13, ctr_total_return#15] -Input [5]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#23, ctr_state#14#24] - -(40) CometNativeScan parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(41) CometFilter -Input [6]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_current_addr_sk#27)) - -(42) CometProject -Input [6]: [c_customer_sk#25, c_customer_id#26, c_current_addr_sk#27, c_salutation#28, c_first_name#29, c_last_name#30] -Arguments: [c_customer_sk#25, c_customer_id#31, c_current_addr_sk#27, c_salutation#32, c_first_name#33, c_last_name#34], [c_customer_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true) AS c_customer_id#31, c_current_addr_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#28, 10, true, false, true) AS c_salutation#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#29, 20, true, false, true) AS c_first_name#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#30, 30, true, false, true) AS c_last_name#34] - -(43) CometColumnarToRow [codegen id : 9] -Input [6]: [c_customer_sk#25, c_customer_id#31, c_current_addr_sk#27, c_salutation#32, c_first_name#33, c_last_name#34] - -(44) BroadcastExchange -Input [6]: [c_customer_sk#25, c_customer_id#31, c_current_addr_sk#27, c_salutation#32, c_first_name#33, c_last_name#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(45) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ctr_customer_sk#13] -Right keys [1]: [c_customer_sk#25] -Join type: Inner -Join condition: None - -(46) Project [codegen id : 11] -Output [6]: [ctr_total_return#15, c_customer_id#31, c_current_addr_sk#27, c_salutation#32, c_first_name#33, c_last_name#34] -Input [8]: [ctr_customer_sk#13, ctr_total_return#15, c_customer_sk#25, c_customer_id#31, c_current_addr_sk#27, c_salutation#32, c_first_name#33, c_last_name#34] - -(47) CometNativeScan parquet spark_catalog.default.customer_address -Output [12]: [ca_address_sk#35, ca_street_number#36, ca_street_name#37, ca_street_type#38, ca_suite_number#39, ca_city#40, ca_county#41, ca_state#42, ca_zip#43, ca_country#44, ca_gmt_offset#45, ca_location_type#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(48) CometFilter -Input [12]: [ca_address_sk#35, ca_street_number#36, ca_street_name#37, ca_street_type#38, ca_suite_number#39, ca_city#40, ca_county#41, ca_state#42, ca_zip#43, ca_country#44, ca_gmt_offset#45, ca_location_type#46] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#42, 2, true, false, true) = GA) AND isnotnull(ca_address_sk#35)) - -(49) CometProject -Input [12]: [ca_address_sk#35, ca_street_number#36, ca_street_name#37, ca_street_type#38, ca_suite_number#39, ca_city#40, ca_county#41, ca_state#42, ca_zip#43, ca_country#44, ca_gmt_offset#45, ca_location_type#46] -Arguments: [ca_address_sk#35, ca_street_number#47, ca_street_name#37, ca_street_type#48, ca_suite_number#49, ca_city#40, ca_county#41, ca_state#50, ca_zip#51, ca_country#44, ca_gmt_offset#45, ca_location_type#52], [ca_address_sk#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#36, 10, true, false, true) AS ca_street_number#47, ca_street_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_type#38, 15, true, false, true) AS ca_street_type#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_suite_number#39, 10, true, false, true) AS ca_suite_number#49, ca_city#40, ca_county#41, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#42, 2, true, false, true) AS ca_state#50, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#43, 10, true, false, true) AS ca_zip#51, ca_country#44, ca_gmt_offset#45, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_location_type#46, 20, true, false, true) AS ca_location_type#52] - -(50) CometColumnarToRow [codegen id : 10] -Input [12]: [ca_address_sk#35, ca_street_number#47, ca_street_name#37, ca_street_type#48, ca_suite_number#49, ca_city#40, ca_county#41, ca_state#50, ca_zip#51, ca_country#44, ca_gmt_offset#45, ca_location_type#52] - -(51) BroadcastExchange -Input [12]: [ca_address_sk#35, ca_street_number#47, ca_street_name#37, ca_street_type#48, ca_suite_number#49, ca_city#40, ca_county#41, ca_state#50, ca_zip#51, ca_country#44, ca_gmt_offset#45, ca_location_type#52] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(52) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#27] -Right keys [1]: [ca_address_sk#35] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 11] -Output [16]: [c_customer_id#31, c_salutation#32, c_first_name#33, c_last_name#34, ca_street_number#47, ca_street_name#37, ca_street_type#48, ca_suite_number#49, ca_city#40, ca_county#41, ca_state#50, ca_zip#51, ca_country#44, ca_gmt_offset#45, ca_location_type#52, ctr_total_return#15] -Input [18]: [ctr_total_return#15, c_customer_id#31, c_current_addr_sk#27, c_salutation#32, c_first_name#33, c_last_name#34, ca_address_sk#35, ca_street_number#47, ca_street_name#37, ca_street_type#48, ca_suite_number#49, ca_city#40, ca_county#41, ca_state#50, ca_zip#51, ca_country#44, ca_gmt_offset#45, ca_location_type#52] - -(54) TakeOrderedAndProject -Input [16]: [c_customer_id#31, c_salutation#32, c_first_name#33, c_last_name#34, ca_street_number#47, ca_street_name#37, ca_street_type#48, ca_suite_number#49, ca_city#40, ca_county#41, ca_state#50, ca_zip#51, ca_country#44, ca_gmt_offset#45, ca_location_type#52, ctr_total_return#15] -Arguments: 100, [c_customer_id#31 ASC NULLS FIRST, c_salutation#32 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, c_last_name#34 ASC NULLS FIRST, ca_street_number#47 ASC NULLS FIRST, ca_street_name#37 ASC NULLS FIRST, ca_street_type#48 ASC NULLS FIRST, ca_suite_number#49 ASC NULLS FIRST, ca_city#40 ASC NULLS FIRST, ca_county#41 ASC NULLS FIRST, ca_state#50 ASC NULLS FIRST, ca_zip#51 ASC NULLS FIRST, ca_country#44 ASC NULLS FIRST, ca_gmt_offset#45 ASC NULLS FIRST, ca_location_type#52 ASC NULLS FIRST, ctr_total_return#15 ASC NULLS FIRST], [c_customer_id#31, c_salutation#32, c_first_name#33, c_last_name#34, ca_street_number#47, ca_street_name#37, ca_street_type#48, ca_suite_number#49, ca_city#40, ca_county#41, ca_state#50, ca_zip#51, ca_country#44, ca_gmt_offset#45, ca_location_type#52, ctr_total_return#15] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (59) -+- * CometColumnarToRow (58) - +- CometProject (57) - +- CometFilter (56) - +- CometNativeScan parquet spark_catalog.default.date_dim (55) - - -(55) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date_sk#6, d_year#53] -Condition : ((isnotnull(d_year#53) AND (d_year#53 = 2000)) AND isnotnull(d_date_sk#6)) - -(57) CometProject -Input [2]: [d_date_sk#6, d_year#53] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(58) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(59) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 19 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/simplified.txt deleted file mode 100644 index af74a164cc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/simplified.txt +++ /dev/null @@ -1,86 +0,0 @@ -TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_total_return] - WholeStageCodegen (11) - Project [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_total_return] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cr_returning_customer_sk,ca_state] #1 - WholeStageCodegen (3) - HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] - Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Filter [cr_returning_addr_sk,cr_returning_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (8) - Filter [(avg(ctr_total_return) * 1.2)] - HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),ctr_state,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ctr_state] #5 - WholeStageCodegen (7) - HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cr_returning_customer_sk,ca_state] #6 - WholeStageCodegen (6) - HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] - Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Filter [cr_returning_addr_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [ca_address_sk,ca_state] #3 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometNativeScan 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 #8 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [ca_street_number,ca_street_type,ca_suite_number,ca_state,ca_zip,ca_location_type] [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] - 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] - CometNativeScan 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/q81.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/explain.txt deleted file mode 100644 index 8b66e63cf3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,318 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (50) -+- CometTakeOrderedAndProject (49) - +- CometProject (48) - +- CometBroadcastHashJoin (47) - :- CometProject (42) - : +- CometBroadcastHashJoin (41) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometFilter (18) - : : : +- CometHashAggregate (17) - : : : +- CometExchange (16) - : : : +- CometHashAggregate (15) - : : : +- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : : +- CometBroadcastExchange (34) - : : +- CometFilter (33) - : : +- CometHashAggregate (32) - : : +- CometExchange (31) - : : +- CometHashAggregate (30) - : : +- CometHashAggregate (29) - : : +- CometExchange (28) - : : +- CometHashAggregate (27) - : : +- CometProject (26) - : : +- CometBroadcastHashJoin (25) - : : :- CometProject (23) - : : : +- CometBroadcastHashJoin (22) - : : : :- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (19) - : : : +- ReusedExchange (21) - : : +- ReusedExchange (24) - : +- CometBroadcastExchange (40) - : +- CometProject (39) - : +- CometFilter (38) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) - +- CometBroadcastExchange (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (43) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Condition : (isnotnull(cr_returning_addr_sk#2) AND isnotnull(cr_returning_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [cr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] -Arguments: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3], [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#8, ca_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#8, ca_state#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#9, 2, true, false, true))) - -(11) CometProject -Input [2]: [ca_address_sk#8, ca_state#9] -Arguments: [ca_address_sk#8, ca_state#10], [ca_address_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#9, 2, true, false, true) AS ca_state#10] - -(12) CometBroadcastExchange -Input [2]: [ca_address_sk#8, ca_state#10] -Arguments: [ca_address_sk#8, ca_state#10] - -(13) CometBroadcastHashJoin -Left output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -Right output [2]: [ca_address_sk#8, ca_state#10] -Arguments: [cr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight - -(14) CometProject -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#8, ca_state#10] -Arguments: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10], [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] - -(15) CometHashAggregate -Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] -Keys [2]: [cr_returning_customer_sk#1, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] - -(16) CometExchange -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#11] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#11] -Keys [2]: [cr_returning_customer_sk#1, ca_state#10] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] - -(18) CometFilter -Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Condition : isnotnull(ctr_total_return#14) - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(cr_returning_addr_sk)] -ReadSchema: struct - -(20) CometFilter -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Condition : isnotnull(cr_returning_addr_sk#2) - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#6] - -(22) CometBroadcastHashJoin -Left output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [cr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(23) CometProject -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] -Arguments: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3], [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] - -(24) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#8, ca_state#10] - -(25) CometBroadcastHashJoin -Left output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -Right output [2]: [ca_address_sk#8, ca_state#10] -Arguments: [cr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight - -(26) CometProject -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#8, ca_state#10] -Arguments: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10], [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] - -(27) CometHashAggregate -Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] -Keys [2]: [cr_returning_customer_sk#1, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] - -(28) CometExchange -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#16] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#16] -Keys [2]: [cr_returning_customer_sk#1, ca_state#10] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] - -(30) CometHashAggregate -Input [2]: [ctr_state#13, ctr_total_return#14] -Keys [1]: [ctr_state#13] -Functions [1]: [partial_avg(ctr_total_return#14)] - -(31) CometExchange -Input [3]: [ctr_state#13, sum#17, count#18] -Arguments: hashpartitioning(ctr_state#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometHashAggregate -Input [3]: [ctr_state#13, sum#17, count#18] -Keys [1]: [ctr_state#13] -Functions [1]: [avg(ctr_total_return#14)] - -(33) CometFilter -Input [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#13#20] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#19) - -(34) CometBroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#13#20] -Arguments: [(avg(ctr_total_return) * 1.2)#19, ctr_state#13#20] - -(35) CometBroadcastHashJoin -Left output [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Right output [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#13#20] -Arguments: [ctr_state#13], [ctr_state#13#20], Inner, (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#19), BuildRight - -(36) CometProject -Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#19, ctr_state#13#20] -Arguments: [ctr_customer_sk#12, ctr_total_return#14], [ctr_customer_sk#12, ctr_total_return#14] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(38) CometFilter -Input [6]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26] -Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_current_addr_sk#23)) - -(39) CometProject -Input [6]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26] -Arguments: [c_customer_sk#21, c_customer_id#27, c_current_addr_sk#23, c_salutation#28, c_first_name#29, c_last_name#30], [c_customer_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#22, 16, true, false, true) AS c_customer_id#27, c_current_addr_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#24, 10, true, false, true) AS c_salutation#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#25, 20, true, false, true) AS c_first_name#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#26, 30, true, false, true) AS c_last_name#30] - -(40) CometBroadcastExchange -Input [6]: [c_customer_sk#21, c_customer_id#27, c_current_addr_sk#23, c_salutation#28, c_first_name#29, c_last_name#30] -Arguments: [c_customer_sk#21, c_customer_id#27, c_current_addr_sk#23, c_salutation#28, c_first_name#29, c_last_name#30] - -(41) CometBroadcastHashJoin -Left output [2]: [ctr_customer_sk#12, ctr_total_return#14] -Right output [6]: [c_customer_sk#21, c_customer_id#27, c_current_addr_sk#23, c_salutation#28, c_first_name#29, c_last_name#30] -Arguments: [ctr_customer_sk#12], [c_customer_sk#21], Inner, BuildRight - -(42) CometProject -Input [8]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#21, c_customer_id#27, c_current_addr_sk#23, c_salutation#28, c_first_name#29, c_last_name#30] -Arguments: [ctr_total_return#14, c_customer_id#27, c_current_addr_sk#23, c_salutation#28, c_first_name#29, c_last_name#30], [ctr_total_return#14, c_customer_id#27, c_current_addr_sk#23, c_salutation#28, c_first_name#29, c_last_name#30] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [12]: [ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(44) CometFilter -Input [12]: [ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#38, 2, true, false, true) = GA) AND isnotnull(ca_address_sk#31)) - -(45) CometProject -Input [12]: [ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] -Arguments: [ca_address_sk#31, ca_street_number#43, ca_street_name#33, ca_street_type#44, ca_suite_number#45, ca_city#36, ca_county#37, ca_state#46, ca_zip#47, ca_country#40, ca_gmt_offset#41, ca_location_type#48], [ca_address_sk#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#32, 10, true, false, true) AS ca_street_number#43, ca_street_name#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_type#34, 15, true, false, true) AS ca_street_type#44, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_suite_number#35, 10, true, false, true) AS ca_suite_number#45, ca_city#36, ca_county#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#38, 2, true, false, true) AS ca_state#46, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#39, 10, true, false, true) AS ca_zip#47, ca_country#40, ca_gmt_offset#41, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_location_type#42, 20, true, false, true) AS ca_location_type#48] - -(46) CometBroadcastExchange -Input [12]: [ca_address_sk#31, ca_street_number#43, ca_street_name#33, ca_street_type#44, ca_suite_number#45, ca_city#36, ca_county#37, ca_state#46, ca_zip#47, ca_country#40, ca_gmt_offset#41, ca_location_type#48] -Arguments: [ca_address_sk#31, ca_street_number#43, ca_street_name#33, ca_street_type#44, ca_suite_number#45, ca_city#36, ca_county#37, ca_state#46, ca_zip#47, ca_country#40, ca_gmt_offset#41, ca_location_type#48] - -(47) CometBroadcastHashJoin -Left output [6]: [ctr_total_return#14, c_customer_id#27, c_current_addr_sk#23, c_salutation#28, c_first_name#29, c_last_name#30] -Right output [12]: [ca_address_sk#31, ca_street_number#43, ca_street_name#33, ca_street_type#44, ca_suite_number#45, ca_city#36, ca_county#37, ca_state#46, ca_zip#47, ca_country#40, ca_gmt_offset#41, ca_location_type#48] -Arguments: [c_current_addr_sk#23], [ca_address_sk#31], Inner, BuildRight - -(48) CometProject -Input [18]: [ctr_total_return#14, c_customer_id#27, c_current_addr_sk#23, c_salutation#28, c_first_name#29, c_last_name#30, ca_address_sk#31, ca_street_number#43, ca_street_name#33, ca_street_type#44, ca_suite_number#45, ca_city#36, ca_county#37, ca_state#46, ca_zip#47, ca_country#40, ca_gmt_offset#41, ca_location_type#48] -Arguments: [c_customer_id#27, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#43, ca_street_name#33, ca_street_type#44, ca_suite_number#45, ca_city#36, ca_county#37, ca_state#46, ca_zip#47, ca_country#40, ca_gmt_offset#41, ca_location_type#48, ctr_total_return#14], [c_customer_id#27, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#43, ca_street_name#33, ca_street_type#44, ca_suite_number#45, ca_city#36, ca_county#37, ca_state#46, ca_zip#47, ca_country#40, ca_gmt_offset#41, ca_location_type#48, ctr_total_return#14] - -(49) CometTakeOrderedAndProject -Input [16]: [c_customer_id#27, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#43, ca_street_name#33, ca_street_type#44, ca_suite_number#45, ca_city#36, ca_county#37, ca_state#46, ca_zip#47, ca_country#40, ca_gmt_offset#41, ca_location_type#48, ctr_total_return#14] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#27 ASC NULLS FIRST,c_salutation#28 ASC NULLS FIRST,c_first_name#29 ASC NULLS FIRST,c_last_name#30 ASC NULLS FIRST,ca_street_number#43 ASC NULLS FIRST,ca_street_name#33 ASC NULLS FIRST,ca_street_type#44 ASC NULLS FIRST,ca_suite_number#45 ASC NULLS FIRST,ca_city#36 ASC NULLS FIRST,ca_county#37 ASC NULLS FIRST,ca_state#46 ASC NULLS FIRST,ca_zip#47 ASC NULLS FIRST,ca_country#40 ASC NULLS FIRST,ca_gmt_offset#41 ASC NULLS FIRST,ca_location_type#48 ASC NULLS FIRST,ctr_total_return#14 ASC NULLS FIRST], output=[c_customer_id#27,c_salutation#28,c_first_name#29,c_last_name#30,ca_street_number#43,ca_street_name#33,ca_street_type#44,ca_suite_number#45,ca_city#36,ca_county#37,ca_state#46,ca_zip#47,ca_country#40,ca_gmt_offset#41,ca_location_type#48,ctr_total_return#14]), [c_customer_id#27, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#43, ca_street_name#33, ca_street_type#44, ca_suite_number#45, ca_city#36, ca_county#37, ca_state#46, ca_zip#47, ca_country#40, ca_gmt_offset#41, ca_location_type#48, ctr_total_return#14], 100, 0, [c_customer_id#27 ASC NULLS FIRST, c_salutation#28 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, c_last_name#30 ASC NULLS FIRST, ca_street_number#43 ASC NULLS FIRST, ca_street_name#33 ASC NULLS FIRST, ca_street_type#44 ASC NULLS FIRST, ca_suite_number#45 ASC NULLS FIRST, ca_city#36 ASC NULLS FIRST, ca_county#37 ASC NULLS FIRST, ca_state#46 ASC NULLS FIRST, ca_zip#47 ASC NULLS FIRST, ca_country#40 ASC NULLS FIRST, ca_gmt_offset#41 ASC NULLS FIRST, ca_location_type#48 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#27, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#43, ca_street_name#33, ca_street_type#44, ca_suite_number#45, ca_city#36, ca_county#37, ca_state#46, ca_zip#47, ca_country#40, ca_gmt_offset#41, ca_location_type#48, ctr_total_return#14] - -(50) CometColumnarToRow [codegen id : 1] -Input [16]: [c_customer_id#27, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#43, ca_street_name#33, ca_street_type#44, ca_suite_number#45, ca_city#36, ca_county#37, ca_state#46, ca_zip#47, ca_country#40, ca_gmt_offset#41, ca_location_type#48, ctr_total_return#14] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) - - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(52) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(53) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(54) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(55) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 19 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/simplified.txt deleted file mode 100644 index a3a981dc54..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_total_return] - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_total_return] - CometBroadcastHashJoin [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,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] - CometProject [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometBroadcastHashJoin [ctr_customer_sk,ctr_total_return,c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometProject [ctr_customer_sk,ctr_total_return] - CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] - CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] - CometHashAggregate [sum] [ctr_customer_sk,ctr_state,ctr_total_return,cr_returning_customer_sk,ca_state,sum(UnscaledValue(cr_return_amt_inc_tax))] - CometExchange [cr_returning_customer_sk,ca_state] #1 - CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] - CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - 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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 - CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] - CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_state,avg(ctr_total_return),ctr_state] - CometExchange [ctr_state] #6 - CometHashAggregate [ctr_total_return] [ctr_state,sum,count] - CometHashAggregate [cr_returning_customer_sk,sum] [ctr_state,ctr_total_return,ca_state,sum(UnscaledValue(cr_return_amt_inc_tax))] - CometExchange [cr_returning_customer_sk,ca_state] #7 - CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] - CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - 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_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 [native_iceberg_compat] 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 - ReusedExchange [ca_address_sk,ca_state] #4 - CometBroadcastExchange [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] #8 - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometBroadcastExchange [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] #9 - CometProject [ca_street_number,ca_street_type,ca_suite_number,ca_state,ca_zip,ca_location_type] [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] - 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 [native_iceberg_compat] 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/q81/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt deleted file mode 100644 index 8b66e63cf3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt +++ /dev/null @@ -1,318 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (50) -+- CometTakeOrderedAndProject (49) - +- CometProject (48) - +- CometBroadcastHashJoin (47) - :- CometProject (42) - : +- CometBroadcastHashJoin (41) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometFilter (18) - : : : +- CometHashAggregate (17) - : : : +- CometExchange (16) - : : : +- CometHashAggregate (15) - : : : +- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (9) - : : +- CometBroadcastExchange (34) - : : +- CometFilter (33) - : : +- CometHashAggregate (32) - : : +- CometExchange (31) - : : +- CometHashAggregate (30) - : : +- CometHashAggregate (29) - : : +- CometExchange (28) - : : +- CometHashAggregate (27) - : : +- CometProject (26) - : : +- CometBroadcastHashJoin (25) - : : :- CometProject (23) - : : : +- CometBroadcastHashJoin (22) - : : : :- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (19) - : : : +- ReusedExchange (21) - : : +- ReusedExchange (24) - : +- CometBroadcastExchange (40) - : +- CometProject (39) - : +- CometFilter (38) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) - +- CometBroadcastExchange (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (43) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Condition : (isnotnull(cr_returning_addr_sk#2) AND isnotnull(cr_returning_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [cr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] -Arguments: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3], [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#8, ca_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [ca_address_sk#8, ca_state#9] -Condition : (isnotnull(ca_address_sk#8) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#9, 2, true, false, true))) - -(11) CometProject -Input [2]: [ca_address_sk#8, ca_state#9] -Arguments: [ca_address_sk#8, ca_state#10], [ca_address_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#9, 2, true, false, true) AS ca_state#10] - -(12) CometBroadcastExchange -Input [2]: [ca_address_sk#8, ca_state#10] -Arguments: [ca_address_sk#8, ca_state#10] - -(13) CometBroadcastHashJoin -Left output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -Right output [2]: [ca_address_sk#8, ca_state#10] -Arguments: [cr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight - -(14) CometProject -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#8, ca_state#10] -Arguments: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10], [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] - -(15) CometHashAggregate -Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] -Keys [2]: [cr_returning_customer_sk#1, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] - -(16) CometExchange -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#11] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#11] -Keys [2]: [cr_returning_customer_sk#1, ca_state#10] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] - -(18) CometFilter -Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Condition : isnotnull(ctr_total_return#14) - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(cr_returning_addr_sk)] -ReadSchema: struct - -(20) CometFilter -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Condition : isnotnull(cr_returning_addr_sk#2) - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#6] - -(22) CometBroadcastHashJoin -Left output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [cr_returned_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(23) CometProject -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4, d_date_sk#6] -Arguments: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3], [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] - -(24) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#8, ca_state#10] - -(25) CometBroadcastHashJoin -Left output [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -Right output [2]: [ca_address_sk#8, ca_state#10] -Arguments: [cr_returning_addr_sk#2], [ca_address_sk#8], Inner, BuildRight - -(26) CometProject -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#8, ca_state#10] -Arguments: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10], [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] - -(27) CometHashAggregate -Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] -Keys [2]: [cr_returning_customer_sk#1, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] - -(28) CometExchange -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#16] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#16] -Keys [2]: [cr_returning_customer_sk#1, ca_state#10] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] - -(30) CometHashAggregate -Input [2]: [ctr_state#13, ctr_total_return#14] -Keys [1]: [ctr_state#13] -Functions [1]: [partial_avg(ctr_total_return#14)] - -(31) CometExchange -Input [3]: [ctr_state#13, sum#17, count#18] -Arguments: hashpartitioning(ctr_state#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometHashAggregate -Input [3]: [ctr_state#13, sum#17, count#18] -Keys [1]: [ctr_state#13] -Functions [1]: [avg(ctr_total_return#14)] - -(33) CometFilter -Input [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#13#20] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#19) - -(34) CometBroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#13#20] -Arguments: [(avg(ctr_total_return) * 1.2)#19, ctr_state#13#20] - -(35) CometBroadcastHashJoin -Left output [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] -Right output [2]: [(avg(ctr_total_return) * 1.2)#19, ctr_state#13#20] -Arguments: [ctr_state#13], [ctr_state#13#20], Inner, (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#19), BuildRight - -(36) CometProject -Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#19, ctr_state#13#20] -Arguments: [ctr_customer_sk#12, ctr_total_return#14], [ctr_customer_sk#12, ctr_total_return#14] - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(38) CometFilter -Input [6]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26] -Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_current_addr_sk#23)) - -(39) CometProject -Input [6]: [c_customer_sk#21, c_customer_id#22, c_current_addr_sk#23, c_salutation#24, c_first_name#25, c_last_name#26] -Arguments: [c_customer_sk#21, c_customer_id#27, c_current_addr_sk#23, c_salutation#28, c_first_name#29, c_last_name#30], [c_customer_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#22, 16, true, false, true) AS c_customer_id#27, c_current_addr_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#24, 10, true, false, true) AS c_salutation#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#25, 20, true, false, true) AS c_first_name#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#26, 30, true, false, true) AS c_last_name#30] - -(40) CometBroadcastExchange -Input [6]: [c_customer_sk#21, c_customer_id#27, c_current_addr_sk#23, c_salutation#28, c_first_name#29, c_last_name#30] -Arguments: [c_customer_sk#21, c_customer_id#27, c_current_addr_sk#23, c_salutation#28, c_first_name#29, c_last_name#30] - -(41) CometBroadcastHashJoin -Left output [2]: [ctr_customer_sk#12, ctr_total_return#14] -Right output [6]: [c_customer_sk#21, c_customer_id#27, c_current_addr_sk#23, c_salutation#28, c_first_name#29, c_last_name#30] -Arguments: [ctr_customer_sk#12], [c_customer_sk#21], Inner, BuildRight - -(42) CometProject -Input [8]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#21, c_customer_id#27, c_current_addr_sk#23, c_salutation#28, c_first_name#29, c_last_name#30] -Arguments: [ctr_total_return#14, c_customer_id#27, c_current_addr_sk#23, c_salutation#28, c_first_name#29, c_last_name#30], [ctr_total_return#14, c_customer_id#27, c_current_addr_sk#23, c_salutation#28, c_first_name#29, c_last_name#30] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [12]: [ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(44) CometFilter -Input [12]: [ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#38, 2, true, false, true) = GA) AND isnotnull(ca_address_sk#31)) - -(45) CometProject -Input [12]: [ca_address_sk#31, ca_street_number#32, ca_street_name#33, ca_street_type#34, ca_suite_number#35, ca_city#36, ca_county#37, ca_state#38, ca_zip#39, ca_country#40, ca_gmt_offset#41, ca_location_type#42] -Arguments: [ca_address_sk#31, ca_street_number#43, ca_street_name#33, ca_street_type#44, ca_suite_number#45, ca_city#36, ca_county#37, ca_state#46, ca_zip#47, ca_country#40, ca_gmt_offset#41, ca_location_type#48], [ca_address_sk#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#32, 10, true, false, true) AS ca_street_number#43, ca_street_name#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_type#34, 15, true, false, true) AS ca_street_type#44, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_suite_number#35, 10, true, false, true) AS ca_suite_number#45, ca_city#36, ca_county#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#38, 2, true, false, true) AS ca_state#46, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#39, 10, true, false, true) AS ca_zip#47, ca_country#40, ca_gmt_offset#41, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_location_type#42, 20, true, false, true) AS ca_location_type#48] - -(46) CometBroadcastExchange -Input [12]: [ca_address_sk#31, ca_street_number#43, ca_street_name#33, ca_street_type#44, ca_suite_number#45, ca_city#36, ca_county#37, ca_state#46, ca_zip#47, ca_country#40, ca_gmt_offset#41, ca_location_type#48] -Arguments: [ca_address_sk#31, ca_street_number#43, ca_street_name#33, ca_street_type#44, ca_suite_number#45, ca_city#36, ca_county#37, ca_state#46, ca_zip#47, ca_country#40, ca_gmt_offset#41, ca_location_type#48] - -(47) CometBroadcastHashJoin -Left output [6]: [ctr_total_return#14, c_customer_id#27, c_current_addr_sk#23, c_salutation#28, c_first_name#29, c_last_name#30] -Right output [12]: [ca_address_sk#31, ca_street_number#43, ca_street_name#33, ca_street_type#44, ca_suite_number#45, ca_city#36, ca_county#37, ca_state#46, ca_zip#47, ca_country#40, ca_gmt_offset#41, ca_location_type#48] -Arguments: [c_current_addr_sk#23], [ca_address_sk#31], Inner, BuildRight - -(48) CometProject -Input [18]: [ctr_total_return#14, c_customer_id#27, c_current_addr_sk#23, c_salutation#28, c_first_name#29, c_last_name#30, ca_address_sk#31, ca_street_number#43, ca_street_name#33, ca_street_type#44, ca_suite_number#45, ca_city#36, ca_county#37, ca_state#46, ca_zip#47, ca_country#40, ca_gmt_offset#41, ca_location_type#48] -Arguments: [c_customer_id#27, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#43, ca_street_name#33, ca_street_type#44, ca_suite_number#45, ca_city#36, ca_county#37, ca_state#46, ca_zip#47, ca_country#40, ca_gmt_offset#41, ca_location_type#48, ctr_total_return#14], [c_customer_id#27, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#43, ca_street_name#33, ca_street_type#44, ca_suite_number#45, ca_city#36, ca_county#37, ca_state#46, ca_zip#47, ca_country#40, ca_gmt_offset#41, ca_location_type#48, ctr_total_return#14] - -(49) CometTakeOrderedAndProject -Input [16]: [c_customer_id#27, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#43, ca_street_name#33, ca_street_type#44, ca_suite_number#45, ca_city#36, ca_county#37, ca_state#46, ca_zip#47, ca_country#40, ca_gmt_offset#41, ca_location_type#48, ctr_total_return#14] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#27 ASC NULLS FIRST,c_salutation#28 ASC NULLS FIRST,c_first_name#29 ASC NULLS FIRST,c_last_name#30 ASC NULLS FIRST,ca_street_number#43 ASC NULLS FIRST,ca_street_name#33 ASC NULLS FIRST,ca_street_type#44 ASC NULLS FIRST,ca_suite_number#45 ASC NULLS FIRST,ca_city#36 ASC NULLS FIRST,ca_county#37 ASC NULLS FIRST,ca_state#46 ASC NULLS FIRST,ca_zip#47 ASC NULLS FIRST,ca_country#40 ASC NULLS FIRST,ca_gmt_offset#41 ASC NULLS FIRST,ca_location_type#48 ASC NULLS FIRST,ctr_total_return#14 ASC NULLS FIRST], output=[c_customer_id#27,c_salutation#28,c_first_name#29,c_last_name#30,ca_street_number#43,ca_street_name#33,ca_street_type#44,ca_suite_number#45,ca_city#36,ca_county#37,ca_state#46,ca_zip#47,ca_country#40,ca_gmt_offset#41,ca_location_type#48,ctr_total_return#14]), [c_customer_id#27, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#43, ca_street_name#33, ca_street_type#44, ca_suite_number#45, ca_city#36, ca_county#37, ca_state#46, ca_zip#47, ca_country#40, ca_gmt_offset#41, ca_location_type#48, ctr_total_return#14], 100, 0, [c_customer_id#27 ASC NULLS FIRST, c_salutation#28 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, c_last_name#30 ASC NULLS FIRST, ca_street_number#43 ASC NULLS FIRST, ca_street_name#33 ASC NULLS FIRST, ca_street_type#44 ASC NULLS FIRST, ca_suite_number#45 ASC NULLS FIRST, ca_city#36 ASC NULLS FIRST, ca_county#37 ASC NULLS FIRST, ca_state#46 ASC NULLS FIRST, ca_zip#47 ASC NULLS FIRST, ca_country#40 ASC NULLS FIRST, ca_gmt_offset#41 ASC NULLS FIRST, ca_location_type#48 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#27, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#43, ca_street_name#33, ca_street_type#44, ca_suite_number#45, ca_city#36, ca_county#37, ca_state#46, ca_zip#47, ca_country#40, ca_gmt_offset#41, ca_location_type#48, ctr_total_return#14] - -(50) CometColumnarToRow [codegen id : 1] -Input [16]: [c_customer_id#27, c_salutation#28, c_first_name#29, c_last_name#30, ca_street_number#43, ca_street_name#33, ca_street_type#44, ca_suite_number#45, ca_city#36, ca_county#37, ca_state#46, ca_zip#47, ca_country#40, ca_gmt_offset#41, ca_location_type#48, ctr_total_return#14] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometProject (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) - - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_year#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(52) CometFilter -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) - -(53) CometProject -Input [2]: [d_date_sk#6, d_year#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(54) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(55) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 19 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/extended.txt deleted file mode 100644 index 9708802e79..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index a3a981dc54..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_total_return] - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,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,ctr_total_return] - CometBroadcastHashJoin [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,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] - CometProject [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometBroadcastHashJoin [ctr_customer_sk,ctr_total_return,c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometProject [ctr_customer_sk,ctr_total_return] - CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] - CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] - CometHashAggregate [sum] [ctr_customer_sk,ctr_state,ctr_total_return,cr_returning_customer_sk,ca_state,sum(UnscaledValue(cr_return_amt_inc_tax))] - CometExchange [cr_returning_customer_sk,ca_state] #1 - CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] - CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - 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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [ca_address_sk,ca_state] #4 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 - CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] - CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_state,avg(ctr_total_return),ctr_state] - CometExchange [ctr_state] #6 - CometHashAggregate [ctr_total_return] [ctr_state,sum,count] - CometHashAggregate [cr_returning_customer_sk,sum] [ctr_state,ctr_total_return,ca_state,sum(UnscaledValue(cr_return_amt_inc_tax))] - CometExchange [cr_returning_customer_sk,ca_state] #7 - CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] - CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - 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_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 [native_iceberg_compat] 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 - ReusedExchange [ca_address_sk,ca_state] #4 - CometBroadcastExchange [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] #8 - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometBroadcastExchange [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] #9 - CometProject [ca_street_number,ca_street_type,ca_suite_number,ca_state,ca_zip,ca_location_type] [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] - 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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/explain.txt deleted file mode 100644 index 93da88fcc1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/explain.txt +++ /dev/null @@ -1,181 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometTakeOrderedAndProject (25) - +- CometHashAggregate (24) - +- CometColumnarExchange (23) - +- * HashAggregate (22) - +- * Project (21) - +- * BroadcastHashJoin Inner BuildLeft (20) - :- BroadcastExchange (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildRight (10) - : : :- * CometColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (9) - : : +- * Project (8) - : : +- * Filter (7) - : : +- * ColumnarToRow (6) - : : +- Scan parquet spark_catalog.default.inventory (5) - : +- ReusedExchange (12) - +- * CometColumnarToRow (19) - +- CometProject (18) - +- CometFilter (17) - +- CometNativeScan parquet spark_catalog.default.store_sales (16) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,62.00), LessThanOrEqual(i_current_price,92.00), In(i_manufact_id, [129,270,423,821]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 62.00)) AND (i_current_price#4 <= 92.00)) AND i_manufact_id#5 IN (129,270,821,423)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#2, 16, true, false, true) AS i_item_id#6, i_item_desc#3, i_current_price#4] - -(4) CometColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(5) Scan parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#9), dynamicpruningexpression(inv_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] - -(7) Filter [codegen id : 1] -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Condition : (((isnotnull(inv_quantity_on_hand#8) AND (inv_quantity_on_hand#8 >= 100)) AND (inv_quantity_on_hand#8 <= 500)) AND isnotnull(inv_item_sk#7)) - -(8) Project [codegen id : 1] -Output [2]: [inv_item_sk#7, inv_date_sk#9] -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] - -(9) BroadcastExchange -Input [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [inv_item_sk#7] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 3] -Output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] - -(12) ReusedExchange [Reuses operator id: 31] -Output [1]: [d_date_sk#11] - -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [inv_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 3] -Output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] - -(15) BroadcastExchange -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) CometNativeScan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#12, ss_sold_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [ss_item_sk#12, ss_sold_date_sk#13] -Condition : isnotnull(ss_item_sk#12) - -(18) CometProject -Input [2]: [ss_item_sk#12, ss_sold_date_sk#13] -Arguments: [ss_item_sk#12], [ss_item_sk#12] - -(19) CometColumnarToRow -Input [1]: [ss_item_sk#12] - -(20) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#12] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 4] -Output [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, ss_item_sk#12] - -(22) HashAggregate [codegen id : 4] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] -Aggregate Attributes: [] -Results [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -(23) CometColumnarExchange -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(25) CometTakeOrderedAndProject -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(26) CometColumnarToRow [codegen id : 5] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometNativeScan parquet spark_catalog.default.date_dim (27) - - -(27) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#11, d_date#14] -Condition : (((isnotnull(d_date#14) AND (d_date#14 >= 2000-05-25)) AND (d_date#14 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) - -(29) CometProject -Input [2]: [d_date_sk#11, d_date#14] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(31) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/simplified.txt deleted file mode 100644 index b60bf92c5b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometColumnarExchange [i_item_id,i_item_desc,i_current_price] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_current_price] - Project [i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [i_item_sk,ss_item_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (3) - Project [i_item_sk,i_item_id,i_item_desc,i_current_price] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] - BroadcastHashJoin [i_item_sk,inv_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Project [inv_item_sk,inv_date_sk] - Filter [inv_quantity_on_hand,inv_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #4 - CometColumnarToRow - InputAdapter - CometProject [ss_item_sk] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometNativeScan 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/q82.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/explain.txt deleted file mode 100644 index 698608e16d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometTakeOrderedAndProject (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (4) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - +- CometProject (19) - +- CometFilter (18) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (17) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,62.00), LessThanOrEqual(i_current_price,92.00), In(i_manufact_id, [129,270,423,821]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 62.00)) AND (i_current_price#4 <= 92.00)) AND i_manufact_id#5 IN (129,270,821,423)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#2, 16, true, false, true) AS i_item_id#6, i_item_desc#3, i_current_price#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#9), dynamicpruningexpression(inv_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Condition : (((isnotnull(inv_quantity_on_hand#8) AND (inv_quantity_on_hand#8 >= 100)) AND (inv_quantity_on_hand#8 <= 500)) AND isnotnull(inv_item_sk#7)) - -(6) CometProject -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9], [inv_item_sk#7, inv_date_sk#9] - -(7) CometBroadcastExchange -Input [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1], [inv_item_sk#7], Inner, BuildRight - -(9) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#12 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) - -(12) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(14) CometBroadcastHashJoin -Left output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [inv_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(15) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(16) CometBroadcastExchange -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#13, ss_sold_date_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [ss_item_sk#13, ss_sold_date_sk#14] -Condition : isnotnull(ss_item_sk#13) - -(19) CometProject -Input [2]: [ss_item_sk#13, ss_sold_date_sk#14] -Arguments: [ss_item_sk#13], [ss_item_sk#13] - -(20) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [1]: [ss_item_sk#13] -Arguments: [i_item_sk#1], [ss_item_sk#13], Inner, BuildLeft - -(21) CometProject -Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, ss_item_sk#13] -Arguments: [i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(22) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(23) CometExchange -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(25) CometTakeOrderedAndProject -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(26) CometColumnarToRow [codegen id : 1] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#12 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) - -(29) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(31) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/simplified.txt deleted file mode 100644 index 28d160a4a8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometExchange [i_item_id,i_item_desc,i_current_price] #1 - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometProject [i_item_id,i_item_desc,i_current_price] - 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 [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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [ss_item_sk] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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/q82/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt deleted file mode 100644 index 698608e16d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometTakeOrderedAndProject (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (4) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - +- CometProject (19) - +- CometFilter (18) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (17) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,62.00), LessThanOrEqual(i_current_price,92.00), In(i_manufact_id, [129,270,423,821]), IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 62.00)) AND (i_current_price#4 <= 92.00)) AND i_manufact_id#5 IN (129,270,821,423)) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#2, 16, true, false, true) AS i_item_id#6, i_item_desc#3, i_current_price#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#9), dynamicpruningexpression(inv_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Condition : (((isnotnull(inv_quantity_on_hand#8) AND (inv_quantity_on_hand#8 >= 100)) AND (inv_quantity_on_hand#8 <= 500)) AND isnotnull(inv_item_sk#7)) - -(6) CometProject -Input [3]: [inv_item_sk#7, inv_quantity_on_hand#8, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9], [inv_item_sk#7, inv_date_sk#9] - -(7) CometBroadcastExchange -Input [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [inv_item_sk#7, inv_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [2]: [inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1], [inv_item_sk#7], Inner, BuildRight - -(9) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_item_sk#7, inv_date_sk#9] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#12 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) - -(12) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(14) CometBroadcastHashJoin -Left output [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [inv_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(15) CometProject -Input [6]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, inv_date_sk#9, d_date_sk#11] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(16) CometBroadcastExchange -Input [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#13, ss_sold_date_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [ss_item_sk#13, ss_sold_date_sk#14] -Condition : isnotnull(ss_item_sk#13) - -(19) CometProject -Input [2]: [ss_item_sk#13, ss_sold_date_sk#14] -Arguments: [ss_item_sk#13], [ss_item_sk#13] - -(20) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4] -Right output [1]: [ss_item_sk#13] -Arguments: [i_item_sk#1], [ss_item_sk#13], Inner, BuildLeft - -(21) CometProject -Input [5]: [i_item_sk#1, i_item_id#6, i_item_desc#3, i_current_price#4, ss_item_sk#13] -Arguments: [i_item_id#6, i_item_desc#3, i_current_price#4], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(22) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(23) CometExchange -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: hashpartitioning(i_item_id#6, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Keys [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Functions: [] - -(25) CometTakeOrderedAndProject -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#6 ASC NULLS FIRST], output=[i_item_id#6,i_item_desc#3,i_current_price#4]), [i_item_id#6, i_item_desc#3, i_current_price#4], 100, 0, [i_item_id#6 ASC NULLS FIRST], [i_item_id#6, i_item_desc#3, i_current_price#4] - -(26) CometColumnarToRow [codegen id : 1] -Input [3]: [i_item_id#6, i_item_desc#3, i_current_price#4] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = inv_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#12 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) - -(29) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(31) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/extended.txt deleted file mode 100644 index 9fbb7893b8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/extended.txt +++ /dev/null @@ -1,34 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 28d160a4a8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometExchange [i_item_id,i_item_desc,i_current_price] #1 - CometHashAggregate [i_item_id,i_item_desc,i_current_price] - CometProject [i_item_id,i_item_desc,i_current_price] - 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 [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,i_item_id,i_item_desc,i_current_price,inv_item_sk,inv_date_sk] - CometProject [i_item_id] [i_item_sk,i_item_id,i_item_desc,i_current_price] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] - CometBroadcastExchange [inv_item_sk,inv_date_sk] #3 - CometProject [inv_item_sk,inv_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [ss_item_sk] - CometFilter [ss_item_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/explain.txt deleted file mode 100644 index 8af6504881..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/explain.txt +++ /dev/null @@ -1,439 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (50) -+- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * HashAggregate (17) - : : +- * CometColumnarToRow (16) - : : +- CometColumnarExchange (15) - : : +- * HashAggregate (14) - : : +- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_returns (1) - : : : +- BroadcastExchange (8) - : : : +- * CometColumnarToRow (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : +- ReusedExchange (11) - : +- BroadcastExchange (31) - : +- * HashAggregate (30) - : +- * CometColumnarToRow (29) - : +- CometColumnarExchange (28) - : +- * HashAggregate (27) - : +- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * Filter (20) - : : : +- * ColumnarToRow (19) - : : : +- Scan parquet spark_catalog.default.catalog_returns (18) - : : +- ReusedExchange (21) - : +- ReusedExchange (24) - +- BroadcastExchange (47) - +- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Filter (36) - : : +- * ColumnarToRow (35) - : : +- Scan parquet spark_catalog.default.web_returns (34) - : +- ReusedExchange (37) - +- ReusedExchange (40) - - -(1) Scan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(sr_returned_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(sr_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Condition : isnotnull(sr_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_item_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [i_item_sk#5, i_item_id#6] -Condition : (isnotnull(i_item_sk#5) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true))) - -(6) CometProject -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#7] - -(7) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_sk#5, i_item_id#7] - -(8) BroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [sr_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7] -Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#5, i_item_id#7] - -(11) ReusedExchange [Reuses operator id: 64] -Output [1]: [d_date_sk#8] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [sr_returned_date_sk#3] -Right keys [1]: [d_date_sk#8] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [2]: [sr_return_quantity#2, i_item_id#7] -Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7, d_date_sk#8] - -(14) HashAggregate [codegen id : 3] -Input [2]: [sr_return_quantity#2, i_item_id#7] -Keys [1]: [i_item_id#7] -Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#9] -Results [2]: [i_item_id#7, sum#10] - -(15) CometColumnarExchange -Input [2]: [i_item_id#7, sum#10] -Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 12] -Input [2]: [i_item_id#7, sum#10] - -(17) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#7, sum#10] -Keys [1]: [i_item_id#7] -Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#11] -Results [2]: [i_item_id#7 AS item_id#12, sum(sr_return_quantity#2)#11 AS sr_item_qty#13] - -(18) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#16), dynamicpruningexpression(cr_returned_date_sk#16 IN dynamicpruning#17)] -PushedFilters: [IsNotNull(cr_item_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 6] -Input [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] - -(20) Filter [codegen id : 6] -Input [3]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16] -Condition : isnotnull(cr_item_sk#14) - -(21) ReusedExchange [Reuses operator id: 8] -Output [2]: [i_item_sk#18, i_item_id#19] - -(22) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_item_sk#14] -Right keys [1]: [i_item_sk#18] -Join type: Inner -Join condition: None - -(23) Project [codegen id : 6] -Output [3]: [cr_return_quantity#15, cr_returned_date_sk#16, i_item_id#19] -Input [5]: [cr_item_sk#14, cr_return_quantity#15, cr_returned_date_sk#16, i_item_sk#18, i_item_id#19] - -(24) ReusedExchange [Reuses operator id: 75] -Output [1]: [d_date_sk#20] - -(25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returned_date_sk#16] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 6] -Output [2]: [cr_return_quantity#15, i_item_id#19] -Input [4]: [cr_return_quantity#15, cr_returned_date_sk#16, i_item_id#19, d_date_sk#20] - -(27) HashAggregate [codegen id : 6] -Input [2]: [cr_return_quantity#15, i_item_id#19] -Keys [1]: [i_item_id#19] -Functions [1]: [partial_sum(cr_return_quantity#15)] -Aggregate Attributes [1]: [sum#21] -Results [2]: [i_item_id#19, sum#22] - -(28) CometColumnarExchange -Input [2]: [i_item_id#19, sum#22] -Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(29) CometColumnarToRow [codegen id : 7] -Input [2]: [i_item_id#19, sum#22] - -(30) HashAggregate [codegen id : 7] -Input [2]: [i_item_id#19, sum#22] -Keys [1]: [i_item_id#19] -Functions [1]: [sum(cr_return_quantity#15)] -Aggregate Attributes [1]: [sum(cr_return_quantity#15)#23] -Results [2]: [i_item_id#19 AS item_id#24, sum(cr_return_quantity#15)#23 AS cr_item_qty#25] - -(31) BroadcastExchange -Input [2]: [item_id#24, cr_item_qty#25] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [item_id#12] -Right keys [1]: [item_id#24] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 12] -Output [3]: [item_id#12, sr_item_qty#13, cr_item_qty#25] -Input [4]: [item_id#12, sr_item_qty#13, item_id#24, cr_item_qty#25] - -(34) Scan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#26, wr_return_quantity#27, wr_returned_date_sk#28] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#28), dynamicpruningexpression(wr_returned_date_sk#28 IN dynamicpruning#17)] -PushedFilters: [IsNotNull(wr_item_sk)] -ReadSchema: struct - -(35) ColumnarToRow [codegen id : 10] -Input [3]: [wr_item_sk#26, wr_return_quantity#27, wr_returned_date_sk#28] - -(36) Filter [codegen id : 10] -Input [3]: [wr_item_sk#26, wr_return_quantity#27, wr_returned_date_sk#28] -Condition : isnotnull(wr_item_sk#26) - -(37) ReusedExchange [Reuses operator id: 8] -Output [2]: [i_item_sk#29, i_item_id#30] - -(38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [wr_item_sk#26] -Right keys [1]: [i_item_sk#29] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 10] -Output [3]: [wr_return_quantity#27, wr_returned_date_sk#28, i_item_id#30] -Input [5]: [wr_item_sk#26, wr_return_quantity#27, wr_returned_date_sk#28, i_item_sk#29, i_item_id#30] - -(40) ReusedExchange [Reuses operator id: 75] -Output [1]: [d_date_sk#31] - -(41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [wr_returned_date_sk#28] -Right keys [1]: [d_date_sk#31] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 10] -Output [2]: [wr_return_quantity#27, i_item_id#30] -Input [4]: [wr_return_quantity#27, wr_returned_date_sk#28, i_item_id#30, d_date_sk#31] - -(43) HashAggregate [codegen id : 10] -Input [2]: [wr_return_quantity#27, i_item_id#30] -Keys [1]: [i_item_id#30] -Functions [1]: [partial_sum(wr_return_quantity#27)] -Aggregate Attributes [1]: [sum#32] -Results [2]: [i_item_id#30, sum#33] - -(44) CometColumnarExchange -Input [2]: [i_item_id#30, sum#33] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 11] -Input [2]: [i_item_id#30, sum#33] - -(46) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#30, sum#33] -Keys [1]: [i_item_id#30] -Functions [1]: [sum(wr_return_quantity#27)] -Aggregate Attributes [1]: [sum(wr_return_quantity#27)#34] -Results [2]: [i_item_id#30 AS item_id#35, sum(wr_return_quantity#27)#34 AS wr_item_qty#36] - -(47) BroadcastExchange -Input [2]: [item_id#35, wr_item_qty#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] - -(48) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [item_id#12] -Right keys [1]: [item_id#35] -Join type: Inner -Join condition: None - -(49) Project [codegen id : 12] -Output [8]: [item_id#12, sr_item_qty#13, (((cast(sr_item_qty#13 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#13 + cr_item_qty#25) + wr_item_qty#36) as double)))) / 3.0) * 100.0) AS sr_dev#37, cr_item_qty#25, (((cast(cr_item_qty#25 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#13 + cr_item_qty#25) + wr_item_qty#36) as double)))) / 3.0) * 100.0) AS cr_dev#38, wr_item_qty#36, (((cast(wr_item_qty#36 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#13 + cr_item_qty#25) + wr_item_qty#36) as double)))) / 3.0) * 100.0) AS wr_dev#39, (cast(((sr_item_qty#13 + cr_item_qty#25) + wr_item_qty#36) as decimal(20,0)) / 3.0) AS average#40] -Input [5]: [item_id#12, sr_item_qty#13, cr_item_qty#25, item_id#35, wr_item_qty#36] - -(50) TakeOrderedAndProject -Input [8]: [item_id#12, sr_item_qty#13, sr_dev#37, cr_item_qty#25, cr_dev#38, wr_item_qty#36, wr_dev#39, average#40] -Arguments: 100, [item_id#12 ASC NULLS FIRST, sr_item_qty#13 ASC NULLS FIRST], [item_id#12, sr_item_qty#13, sr_dev#37, cr_item_qty#25, cr_dev#38, wr_item_qty#36, wr_dev#39, average#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (64) -+- * CometColumnarToRow (63) - +- CometProject (62) - +- CometBroadcastHashJoin (61) - :- CometFilter (52) - : +- CometNativeScan parquet spark_catalog.default.date_dim (51) - +- CometBroadcastExchange (60) - +- CometProject (59) - +- CometBroadcastHashJoin (58) - :- CometNativeScan parquet spark_catalog.default.date_dim (53) - +- CometBroadcastExchange (57) - +- CometProject (56) - +- CometFilter (55) - +- CometNativeScan parquet spark_catalog.default.date_dim (54) - - -(51) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#41] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(52) CometFilter -Input [2]: [d_date_sk#8, d_date#41] -Condition : isnotnull(d_date_sk#8) - -(53) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date#41, d_week_seq#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(54) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date#41, d_week_seq#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(55) CometFilter -Input [2]: [d_date#41, d_week_seq#42] -Condition : cast(d_date#41 as string) IN (2000-06-30,2000-09-27,2000-11-17) - -(56) CometProject -Input [2]: [d_date#41, d_week_seq#42] -Arguments: [d_week_seq#42#43], [d_week_seq#42 AS d_week_seq#42#43] - -(57) CometBroadcastExchange -Input [1]: [d_week_seq#42#43] -Arguments: [d_week_seq#42#43] - -(58) CometBroadcastHashJoin -Left output [2]: [d_date#41, d_week_seq#42] -Right output [1]: [d_week_seq#42#43] -Arguments: [d_week_seq#42], [d_week_seq#42#43], LeftSemi, BuildRight - -(59) CometProject -Input [2]: [d_date#41, d_week_seq#42] -Arguments: [d_date#41#44], [d_date#41 AS d_date#41#44] - -(60) CometBroadcastExchange -Input [1]: [d_date#41#44] -Arguments: [d_date#41#44] - -(61) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#41] -Right output [1]: [d_date#41#44] -Arguments: [d_date#41], [d_date#41#44], LeftSemi, BuildRight - -(62) CometProject -Input [2]: [d_date_sk#8, d_date#41] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(63) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(64) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 18 Hosting Expression = cr_returned_date_sk#16 IN dynamicpruning#17 -BroadcastExchange (75) -+- * CometColumnarToRow (74) - +- CometProject (73) - +- CometBroadcastHashJoin (72) - :- CometFilter (66) - : +- CometNativeScan parquet spark_catalog.default.date_dim (65) - +- CometBroadcastExchange (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometNativeScan parquet spark_catalog.default.date_dim (67) - +- ReusedExchange (68) - - -(65) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_date#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(66) CometFilter -Input [2]: [d_date_sk#20, d_date#45] -Condition : isnotnull(d_date_sk#20) - -(67) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date#41, d_week_seq#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(68) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_week_seq#42#46] - -(69) CometBroadcastHashJoin -Left output [2]: [d_date#41, d_week_seq#42] -Right output [1]: [d_week_seq#42#46] -Arguments: [d_week_seq#42], [d_week_seq#42#46], LeftSemi, BuildRight - -(70) CometProject -Input [2]: [d_date#41, d_week_seq#42] -Arguments: [d_date#41], [d_date#41] - -(71) CometBroadcastExchange -Input [1]: [d_date#41] -Arguments: [d_date#41] - -(72) CometBroadcastHashJoin -Left output [2]: [d_date_sk#20, d_date#45] -Right output [1]: [d_date#41] -Arguments: [d_date#45], [d_date#41], LeftSemi, BuildRight - -(73) CometProject -Input [2]: [d_date_sk#20, d_date#45] -Arguments: [d_date_sk#20], [d_date_sk#20] - -(74) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#20] - -(75) BroadcastExchange -Input [1]: [d_date_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:3 Hosting operator id = 34 Hosting Expression = wr_returned_date_sk#28 IN dynamicpruning#17 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/simplified.txt deleted file mode 100644 index 1e0e8fea70..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/simplified.txt +++ /dev/null @@ -1,104 +0,0 @@ -TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - WholeStageCodegen (12) - Project [item_id,sr_item_qty,cr_item_qty,wr_item_qty] - BroadcastHashJoin [item_id,item_id] - Project [item_id,sr_item_qty,cr_item_qty] - BroadcastHashJoin [item_id,item_id] - HashAggregate [i_item_id,sum] [sum(sr_return_quantity),item_id,sr_item_qty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (3) - HashAggregate [i_item_id,sr_return_quantity] [sum,sum] - Project [sr_return_quantity,i_item_id] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Project [sr_return_quantity,sr_returned_date_sk,i_item_id] - BroadcastHashJoin [sr_item_sk,i_item_sk] - Filter [sr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] [d_week_seq] - CometFilter [d_date,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - HashAggregate [i_item_id,sum] [sum(cr_return_quantity),item_id,cr_item_qty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #7 - WholeStageCodegen (6) - HashAggregate [i_item_id,cr_return_quantity] [sum,sum] - Project [cr_return_quantity,i_item_id] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Project [cr_return_quantity,cr_returned_date_sk,i_item_id] - BroadcastHashJoin [cr_item_sk,i_item_sk] - Filter [cr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #9 - CometProject [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] - ReusedExchange [d_week_seq] #4 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (11) - HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #11 - WholeStageCodegen (10) - HashAggregate [i_item_id,wr_return_quantity] [sum,sum] - Project [wr_return_quantity,i_item_id] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Project [wr_return_quantity,wr_returned_date_sk,i_item_id] - BroadcastHashJoin [wr_item_sk,i_item_sk] - Filter [wr_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - InputAdapter - ReusedExchange [d_date_sk] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/explain.txt deleted file mode 100644 index a700acceb7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,453 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (54) -+- CometTakeOrderedAndProject (53) - +- CometProject (52) - +- CometBroadcastHashJoin (51) - :- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometHashAggregate (19) - : : +- CometExchange (18) - : : +- CometHashAggregate (17) - : : +- CometProject (16) - : : +- CometBroadcastHashJoin (15) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (14) - : : +- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- ReusedExchange (11) - : +- CometBroadcastExchange (36) - : +- CometHashAggregate (35) - : +- CometExchange (34) - : +- CometHashAggregate (33) - : +- CometProject (32) - : +- CometBroadcastHashJoin (31) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometFilter (21) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (20) - : : +- ReusedExchange (22) - : +- CometBroadcastExchange (30) - : +- CometProject (29) - : +- CometBroadcastHashJoin (28) - : :- CometFilter (26) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - : +- ReusedExchange (27) - +- CometBroadcastExchange (50) - +- CometHashAggregate (49) - +- CometExchange (48) - +- CometHashAggregate (47) - +- CometProject (46) - +- CometBroadcastHashJoin (45) - :- CometProject (43) - : +- CometBroadcastHashJoin (42) - : :- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (39) - : +- ReusedExchange (41) - +- ReusedExchange (44) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(sr_returned_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(sr_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Condition : isnotnull(sr_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_item_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_item_id#6] -Condition : (isnotnull(i_item_sk#5) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true))) - -(5) CometProject -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#7] - -(6) CometBroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#7] -Arguments: [i_item_sk#5, i_item_id#7] - -(7) CometBroadcastHashJoin -Left output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Right output [2]: [i_item_sk#5, i_item_id#7] -Arguments: [sr_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#5, i_item_id#7] -Arguments: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7], [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(11) ReusedExchange [Reuses operator id: 64] -Output [1]: [d_date#9#10] - -(12) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#9#10] -Arguments: [d_date#9], [d_date#9#10], LeftSemi, BuildRight - -(13) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: [d_date_sk#8] - -(15) CometBroadcastHashJoin -Left output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7] -Right output [1]: [d_date_sk#8] -Arguments: [sr_returned_date_sk#3], [d_date_sk#8], Inner, BuildRight - -(16) CometProject -Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7, d_date_sk#8] -Arguments: [sr_return_quantity#2, i_item_id#7], [sr_return_quantity#2, i_item_id#7] - -(17) CometHashAggregate -Input [2]: [sr_return_quantity#2, i_item_id#7] -Keys [1]: [i_item_id#7] -Functions [1]: [partial_sum(sr_return_quantity#2)] - -(18) CometExchange -Input [2]: [i_item_id#7, sum#11] -Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [2]: [i_item_id#7, sum#11] -Keys [1]: [i_item_id#7] -Functions [1]: [sum(sr_return_quantity#2)] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#14), dynamicpruningexpression(cr_returned_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(cr_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] -Condition : isnotnull(cr_item_sk#12) - -(22) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#16, i_item_id#17] - -(23) CometBroadcastHashJoin -Left output [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] -Right output [2]: [i_item_sk#16, i_item_id#17] -Arguments: [cr_item_sk#12], [i_item_sk#16], Inner, BuildRight - -(24) CometProject -Input [5]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14, i_item_sk#16, i_item_id#17] -Arguments: [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17], [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#18, d_date#19] -Condition : isnotnull(d_date_sk#18) - -(27) ReusedExchange [Reuses operator id: 75] -Output [1]: [d_date#9] - -(28) CometBroadcastHashJoin -Left output [2]: [d_date_sk#18, d_date#19] -Right output [1]: [d_date#9] -Arguments: [d_date#19], [d_date#9], LeftSemi, BuildRight - -(29) CometProject -Input [2]: [d_date_sk#18, d_date#19] -Arguments: [d_date_sk#18], [d_date_sk#18] - -(30) CometBroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: [d_date_sk#18] - -(31) CometBroadcastHashJoin -Left output [3]: [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17] -Right output [1]: [d_date_sk#18] -Arguments: [cr_returned_date_sk#14], [d_date_sk#18], Inner, BuildRight - -(32) CometProject -Input [4]: [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17, d_date_sk#18] -Arguments: [cr_return_quantity#13, i_item_id#17], [cr_return_quantity#13, i_item_id#17] - -(33) CometHashAggregate -Input [2]: [cr_return_quantity#13, i_item_id#17] -Keys [1]: [i_item_id#17] -Functions [1]: [partial_sum(cr_return_quantity#13)] - -(34) CometExchange -Input [2]: [i_item_id#17, sum#20] -Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(35) CometHashAggregate -Input [2]: [i_item_id#17, sum#20] -Keys [1]: [i_item_id#17] -Functions [1]: [sum(cr_return_quantity#13)] - -(36) CometBroadcastExchange -Input [2]: [item_id#21, cr_item_qty#22] -Arguments: [item_id#21, cr_item_qty#22] - -(37) CometBroadcastHashJoin -Left output [2]: [item_id#23, sr_item_qty#24] -Right output [2]: [item_id#21, cr_item_qty#22] -Arguments: [item_id#23], [item_id#21], Inner, BuildRight - -(38) CometProject -Input [4]: [item_id#23, sr_item_qty#24, item_id#21, cr_item_qty#22] -Arguments: [item_id#23, sr_item_qty#24, cr_item_qty#22], [item_id#23, sr_item_qty#24, cr_item_qty#22] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#27), dynamicpruningexpression(wr_returned_date_sk#27 IN dynamicpruning#28)] -PushedFilters: [IsNotNull(wr_item_sk)] -ReadSchema: struct - -(40) CometFilter -Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] -Condition : isnotnull(wr_item_sk#25) - -(41) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#29, i_item_id#30] - -(42) CometBroadcastHashJoin -Left output [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] -Right output [2]: [i_item_sk#29, i_item_id#30] -Arguments: [wr_item_sk#25], [i_item_sk#29], Inner, BuildRight - -(43) CometProject -Input [5]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27, i_item_sk#29, i_item_id#30] -Arguments: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30], [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30] - -(44) ReusedExchange [Reuses operator id: 30] -Output [1]: [d_date_sk#31] - -(45) CometBroadcastHashJoin -Left output [3]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30] -Right output [1]: [d_date_sk#31] -Arguments: [wr_returned_date_sk#27], [d_date_sk#31], Inner, BuildRight - -(46) CometProject -Input [4]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30, d_date_sk#31] -Arguments: [wr_return_quantity#26, i_item_id#30], [wr_return_quantity#26, i_item_id#30] - -(47) CometHashAggregate -Input [2]: [wr_return_quantity#26, i_item_id#30] -Keys [1]: [i_item_id#30] -Functions [1]: [partial_sum(wr_return_quantity#26)] - -(48) CometExchange -Input [2]: [i_item_id#30, sum#32] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [2]: [i_item_id#30, sum#32] -Keys [1]: [i_item_id#30] -Functions [1]: [sum(wr_return_quantity#26)] - -(50) CometBroadcastExchange -Input [2]: [item_id#33, wr_item_qty#34] -Arguments: [item_id#33, wr_item_qty#34] - -(51) CometBroadcastHashJoin -Left output [3]: [item_id#23, sr_item_qty#24, cr_item_qty#22] -Right output [2]: [item_id#33, wr_item_qty#34] -Arguments: [item_id#23], [item_id#33], Inner, BuildRight - -(52) CometProject -Input [5]: [item_id#23, sr_item_qty#24, cr_item_qty#22, item_id#33, wr_item_qty#34] -Arguments: [item_id#23, sr_item_qty#24, sr_dev#35, cr_item_qty#22, cr_dev#36, wr_item_qty#34, wr_dev#37, average#38], [item_id#23, sr_item_qty#24, (((cast(sr_item_qty#24 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#24 + cr_item_qty#22) + wr_item_qty#34) as double)))) / 3.0) * 100.0) AS sr_dev#35, cr_item_qty#22, (((cast(cr_item_qty#22 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#24 + cr_item_qty#22) + wr_item_qty#34) as double)))) / 3.0) * 100.0) AS cr_dev#36, wr_item_qty#34, (((cast(wr_item_qty#34 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#24 + cr_item_qty#22) + wr_item_qty#34) as double)))) / 3.0) * 100.0) AS wr_dev#37, (cast(((sr_item_qty#24 + cr_item_qty#22) + wr_item_qty#34) as decimal(20,0)) / 3.0) AS average#38] - -(53) CometTakeOrderedAndProject -Input [8]: [item_id#23, sr_item_qty#24, sr_dev#35, cr_item_qty#22, cr_dev#36, wr_item_qty#34, wr_dev#37, average#38] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_id#23 ASC NULLS FIRST,sr_item_qty#24 ASC NULLS FIRST], output=[item_id#23,sr_item_qty#24,sr_dev#35,cr_item_qty#22,cr_dev#36,wr_item_qty#34,wr_dev#37,average#38]), [item_id#23, sr_item_qty#24, sr_dev#35, cr_item_qty#22, cr_dev#36, wr_item_qty#34, wr_dev#37, average#38], 100, 0, [item_id#23 ASC NULLS FIRST, sr_item_qty#24 ASC NULLS FIRST], [item_id#23, sr_item_qty#24, sr_dev#35, cr_item_qty#22, cr_dev#36, wr_item_qty#34, wr_dev#37, average#38] - -(54) CometColumnarToRow [codegen id : 1] -Input [8]: [item_id#23, sr_item_qty#24, sr_dev#35, cr_item_qty#22, cr_dev#36, wr_item_qty#34, wr_dev#37, average#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (68) -+- * CometColumnarToRow (67) - +- CometProject (66) - +- CometBroadcastHashJoin (65) - :- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometBroadcastExchange (64) - +- CometProject (63) - +- CometBroadcastHashJoin (62) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (57) - +- CometBroadcastExchange (61) - +- CometProject (60) - +- CometFilter (59) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) - - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#9, d_week_seq#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#9, d_week_seq#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(59) CometFilter -Input [2]: [d_date#9, d_week_seq#39] -Condition : cast(d_date#9 as string) IN (2000-06-30,2000-09-27,2000-11-17) - -(60) CometProject -Input [2]: [d_date#9, d_week_seq#39] -Arguments: [d_week_seq#39#40], [d_week_seq#39 AS d_week_seq#39#40] - -(61) CometBroadcastExchange -Input [1]: [d_week_seq#39#40] -Arguments: [d_week_seq#39#40] - -(62) CometBroadcastHashJoin -Left output [2]: [d_date#9, d_week_seq#39] -Right output [1]: [d_week_seq#39#40] -Arguments: [d_week_seq#39], [d_week_seq#39#40], LeftSemi, BuildRight - -(63) CometProject -Input [2]: [d_date#9, d_week_seq#39] -Arguments: [d_date#9#10], [d_date#9 AS d_date#9#10] - -(64) CometBroadcastExchange -Input [1]: [d_date#9#10] -Arguments: [d_date#9#10] - -(65) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#9#10] -Arguments: [d_date#9], [d_date#9#10], LeftSemi, BuildRight - -(66) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(67) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(68) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 20 Hosting Expression = cr_returned_date_sk#14 IN dynamicpruning#15 -BroadcastExchange (79) -+- * CometColumnarToRow (78) - +- CometProject (77) - +- CometBroadcastHashJoin (76) - :- CometFilter (70) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (69) - +- CometBroadcastExchange (75) - +- CometProject (74) - +- CometBroadcastHashJoin (73) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (71) - +- ReusedExchange (72) - - -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(70) CometFilter -Input [2]: [d_date_sk#18, d_date#19] -Condition : isnotnull(d_date_sk#18) - -(71) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#9, d_week_seq#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(72) ReusedExchange [Reuses operator id: 61] -Output [1]: [d_week_seq#39#41] - -(73) CometBroadcastHashJoin -Left output [2]: [d_date#9, d_week_seq#39] -Right output [1]: [d_week_seq#39#41] -Arguments: [d_week_seq#39], [d_week_seq#39#41], LeftSemi, BuildRight - -(74) CometProject -Input [2]: [d_date#9, d_week_seq#39] -Arguments: [d_date#9], [d_date#9] - -(75) CometBroadcastExchange -Input [1]: [d_date#9] -Arguments: [d_date#9] - -(76) CometBroadcastHashJoin -Left output [2]: [d_date_sk#18, d_date#19] -Right output [1]: [d_date#9] -Arguments: [d_date#19], [d_date#9], LeftSemi, BuildRight - -(77) CometProject -Input [2]: [d_date_sk#18, d_date#19] -Arguments: [d_date_sk#18], [d_date_sk#18] - -(78) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#18] - -(79) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:3 Hosting operator id = 39 Hosting Expression = wr_returned_date_sk#27 IN dynamicpruning#15 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/simplified.txt deleted file mode 100644 index bff9c86587..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,88 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - CometProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - CometBroadcastHashJoin [item_id,sr_item_qty,cr_item_qty,item_id,wr_item_qty] - CometProject [item_id,sr_item_qty,cr_item_qty] - CometBroadcastHashJoin [item_id,sr_item_qty,item_id,cr_item_qty] - CometHashAggregate [sum] [item_id,sr_item_qty,i_item_id,sum(sr_return_quantity)] - CometExchange [i_item_id] #1 - CometHashAggregate [sr_return_quantity] [i_item_id,sum] - CometProject [sr_return_quantity,i_item_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #3 - CometBroadcastExchange [item_id,cr_item_qty] #7 - CometHashAggregate [sum] [item_id,cr_item_qty,i_item_id,sum(cr_return_quantity)] - CometExchange [i_item_id] #8 - CometHashAggregate [cr_return_quantity] [i_item_id,sum] - CometProject [cr_return_quantity,i_item_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #10 - CometProject [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - ReusedExchange [d_week_seq] #4 - ReusedExchange [i_item_sk,i_item_id] #5 - CometBroadcastExchange [d_date_sk] #11 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #10 - CometBroadcastExchange [item_id,wr_item_qty] #12 - CometHashAggregate [sum] [item_id,wr_item_qty,i_item_id,sum(wr_return_quantity)] - CometExchange [i_item_id] #13 - CometHashAggregate [wr_return_quantity] [i_item_id,sum] - CometProject [wr_return_quantity,i_item_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [i_item_sk,i_item_id] #5 - ReusedExchange [d_date_sk] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt deleted file mode 100644 index a700acceb7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ /dev/null @@ -1,453 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (54) -+- CometTakeOrderedAndProject (53) - +- CometProject (52) - +- CometBroadcastHashJoin (51) - :- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometHashAggregate (19) - : : +- CometExchange (18) - : : +- CometHashAggregate (17) - : : +- CometProject (16) - : : +- CometBroadcastHashJoin (15) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : +- CometBroadcastExchange (14) - : : +- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- ReusedExchange (11) - : +- CometBroadcastExchange (36) - : +- CometHashAggregate (35) - : +- CometExchange (34) - : +- CometHashAggregate (33) - : +- CometProject (32) - : +- CometBroadcastHashJoin (31) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometFilter (21) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (20) - : : +- ReusedExchange (22) - : +- CometBroadcastExchange (30) - : +- CometProject (29) - : +- CometBroadcastHashJoin (28) - : :- CometFilter (26) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - : +- ReusedExchange (27) - +- CometBroadcastExchange (50) - +- CometHashAggregate (49) - +- CometExchange (48) - +- CometHashAggregate (47) - +- CometProject (46) - +- CometBroadcastHashJoin (45) - :- CometProject (43) - : +- CometBroadcastHashJoin (42) - : :- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (39) - : +- ReusedExchange (41) - +- ReusedExchange (44) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(sr_returned_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(sr_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Condition : isnotnull(sr_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_item_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_item_id#6] -Condition : (isnotnull(i_item_sk#5) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true))) - -(5) CometProject -Input [2]: [i_item_sk#5, i_item_id#6] -Arguments: [i_item_sk#5, i_item_id#7], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#7] - -(6) CometBroadcastExchange -Input [2]: [i_item_sk#5, i_item_id#7] -Arguments: [i_item_sk#5, i_item_id#7] - -(7) CometBroadcastHashJoin -Left output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] -Right output [2]: [i_item_sk#5, i_item_id#7] -Arguments: [sr_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#5, i_item_id#7] -Arguments: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7], [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(11) ReusedExchange [Reuses operator id: 64] -Output [1]: [d_date#9#10] - -(12) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#9#10] -Arguments: [d_date#9], [d_date#9#10], LeftSemi, BuildRight - -(13) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: [d_date_sk#8] - -(15) CometBroadcastHashJoin -Left output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7] -Right output [1]: [d_date_sk#8] -Arguments: [sr_returned_date_sk#3], [d_date_sk#8], Inner, BuildRight - -(16) CometProject -Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#7, d_date_sk#8] -Arguments: [sr_return_quantity#2, i_item_id#7], [sr_return_quantity#2, i_item_id#7] - -(17) CometHashAggregate -Input [2]: [sr_return_quantity#2, i_item_id#7] -Keys [1]: [i_item_id#7] -Functions [1]: [partial_sum(sr_return_quantity#2)] - -(18) CometExchange -Input [2]: [i_item_id#7, sum#11] -Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [2]: [i_item_id#7, sum#11] -Keys [1]: [i_item_id#7] -Functions [1]: [sum(sr_return_quantity#2)] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#14), dynamicpruningexpression(cr_returned_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(cr_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] -Condition : isnotnull(cr_item_sk#12) - -(22) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#16, i_item_id#17] - -(23) CometBroadcastHashJoin -Left output [3]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14] -Right output [2]: [i_item_sk#16, i_item_id#17] -Arguments: [cr_item_sk#12], [i_item_sk#16], Inner, BuildRight - -(24) CometProject -Input [5]: [cr_item_sk#12, cr_return_quantity#13, cr_returned_date_sk#14, i_item_sk#16, i_item_id#17] -Arguments: [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17], [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#18, d_date#19] -Condition : isnotnull(d_date_sk#18) - -(27) ReusedExchange [Reuses operator id: 75] -Output [1]: [d_date#9] - -(28) CometBroadcastHashJoin -Left output [2]: [d_date_sk#18, d_date#19] -Right output [1]: [d_date#9] -Arguments: [d_date#19], [d_date#9], LeftSemi, BuildRight - -(29) CometProject -Input [2]: [d_date_sk#18, d_date#19] -Arguments: [d_date_sk#18], [d_date_sk#18] - -(30) CometBroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: [d_date_sk#18] - -(31) CometBroadcastHashJoin -Left output [3]: [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17] -Right output [1]: [d_date_sk#18] -Arguments: [cr_returned_date_sk#14], [d_date_sk#18], Inner, BuildRight - -(32) CometProject -Input [4]: [cr_return_quantity#13, cr_returned_date_sk#14, i_item_id#17, d_date_sk#18] -Arguments: [cr_return_quantity#13, i_item_id#17], [cr_return_quantity#13, i_item_id#17] - -(33) CometHashAggregate -Input [2]: [cr_return_quantity#13, i_item_id#17] -Keys [1]: [i_item_id#17] -Functions [1]: [partial_sum(cr_return_quantity#13)] - -(34) CometExchange -Input [2]: [i_item_id#17, sum#20] -Arguments: hashpartitioning(i_item_id#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(35) CometHashAggregate -Input [2]: [i_item_id#17, sum#20] -Keys [1]: [i_item_id#17] -Functions [1]: [sum(cr_return_quantity#13)] - -(36) CometBroadcastExchange -Input [2]: [item_id#21, cr_item_qty#22] -Arguments: [item_id#21, cr_item_qty#22] - -(37) CometBroadcastHashJoin -Left output [2]: [item_id#23, sr_item_qty#24] -Right output [2]: [item_id#21, cr_item_qty#22] -Arguments: [item_id#23], [item_id#21], Inner, BuildRight - -(38) CometProject -Input [4]: [item_id#23, sr_item_qty#24, item_id#21, cr_item_qty#22] -Arguments: [item_id#23, sr_item_qty#24, cr_item_qty#22], [item_id#23, sr_item_qty#24, cr_item_qty#22] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#27), dynamicpruningexpression(wr_returned_date_sk#27 IN dynamicpruning#28)] -PushedFilters: [IsNotNull(wr_item_sk)] -ReadSchema: struct - -(40) CometFilter -Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] -Condition : isnotnull(wr_item_sk#25) - -(41) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#29, i_item_id#30] - -(42) CometBroadcastHashJoin -Left output [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] -Right output [2]: [i_item_sk#29, i_item_id#30] -Arguments: [wr_item_sk#25], [i_item_sk#29], Inner, BuildRight - -(43) CometProject -Input [5]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27, i_item_sk#29, i_item_id#30] -Arguments: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30], [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30] - -(44) ReusedExchange [Reuses operator id: 30] -Output [1]: [d_date_sk#31] - -(45) CometBroadcastHashJoin -Left output [3]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30] -Right output [1]: [d_date_sk#31] -Arguments: [wr_returned_date_sk#27], [d_date_sk#31], Inner, BuildRight - -(46) CometProject -Input [4]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30, d_date_sk#31] -Arguments: [wr_return_quantity#26, i_item_id#30], [wr_return_quantity#26, i_item_id#30] - -(47) CometHashAggregate -Input [2]: [wr_return_quantity#26, i_item_id#30] -Keys [1]: [i_item_id#30] -Functions [1]: [partial_sum(wr_return_quantity#26)] - -(48) CometExchange -Input [2]: [i_item_id#30, sum#32] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [2]: [i_item_id#30, sum#32] -Keys [1]: [i_item_id#30] -Functions [1]: [sum(wr_return_quantity#26)] - -(50) CometBroadcastExchange -Input [2]: [item_id#33, wr_item_qty#34] -Arguments: [item_id#33, wr_item_qty#34] - -(51) CometBroadcastHashJoin -Left output [3]: [item_id#23, sr_item_qty#24, cr_item_qty#22] -Right output [2]: [item_id#33, wr_item_qty#34] -Arguments: [item_id#23], [item_id#33], Inner, BuildRight - -(52) CometProject -Input [5]: [item_id#23, sr_item_qty#24, cr_item_qty#22, item_id#33, wr_item_qty#34] -Arguments: [item_id#23, sr_item_qty#24, sr_dev#35, cr_item_qty#22, cr_dev#36, wr_item_qty#34, wr_dev#37, average#38], [item_id#23, sr_item_qty#24, (((cast(sr_item_qty#24 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#24 + cr_item_qty#22) + wr_item_qty#34) as double)))) / 3.0) * 100.0) AS sr_dev#35, cr_item_qty#22, (((cast(cr_item_qty#22 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#24 + cr_item_qty#22) + wr_item_qty#34) as double)))) / 3.0) * 100.0) AS cr_dev#36, wr_item_qty#34, (((cast(wr_item_qty#34 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(((sr_item_qty#24 + cr_item_qty#22) + wr_item_qty#34) as double)))) / 3.0) * 100.0) AS wr_dev#37, (cast(((sr_item_qty#24 + cr_item_qty#22) + wr_item_qty#34) as decimal(20,0)) / 3.0) AS average#38] - -(53) CometTakeOrderedAndProject -Input [8]: [item_id#23, sr_item_qty#24, sr_dev#35, cr_item_qty#22, cr_dev#36, wr_item_qty#34, wr_dev#37, average#38] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_id#23 ASC NULLS FIRST,sr_item_qty#24 ASC NULLS FIRST], output=[item_id#23,sr_item_qty#24,sr_dev#35,cr_item_qty#22,cr_dev#36,wr_item_qty#34,wr_dev#37,average#38]), [item_id#23, sr_item_qty#24, sr_dev#35, cr_item_qty#22, cr_dev#36, wr_item_qty#34, wr_dev#37, average#38], 100, 0, [item_id#23 ASC NULLS FIRST, sr_item_qty#24 ASC NULLS FIRST], [item_id#23, sr_item_qty#24, sr_dev#35, cr_item_qty#22, cr_dev#36, wr_item_qty#34, wr_dev#37, average#38] - -(54) CometColumnarToRow [codegen id : 1] -Input [8]: [item_id#23, sr_item_qty#24, sr_dev#35, cr_item_qty#22, cr_dev#36, wr_item_qty#34, wr_dev#37, average#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (68) -+- * CometColumnarToRow (67) - +- CometProject (66) - +- CometBroadcastHashJoin (65) - :- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometBroadcastExchange (64) - +- CometProject (63) - +- CometBroadcastHashJoin (62) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (57) - +- CometBroadcastExchange (61) - +- CometProject (60) - +- CometFilter (59) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) - - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#8, d_date#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date_sk#8, d_date#9] -Condition : isnotnull(d_date_sk#8) - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#9, d_week_seq#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#9, d_week_seq#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(59) CometFilter -Input [2]: [d_date#9, d_week_seq#39] -Condition : cast(d_date#9 as string) IN (2000-06-30,2000-09-27,2000-11-17) - -(60) CometProject -Input [2]: [d_date#9, d_week_seq#39] -Arguments: [d_week_seq#39#40], [d_week_seq#39 AS d_week_seq#39#40] - -(61) CometBroadcastExchange -Input [1]: [d_week_seq#39#40] -Arguments: [d_week_seq#39#40] - -(62) CometBroadcastHashJoin -Left output [2]: [d_date#9, d_week_seq#39] -Right output [1]: [d_week_seq#39#40] -Arguments: [d_week_seq#39], [d_week_seq#39#40], LeftSemi, BuildRight - -(63) CometProject -Input [2]: [d_date#9, d_week_seq#39] -Arguments: [d_date#9#10], [d_date#9 AS d_date#9#10] - -(64) CometBroadcastExchange -Input [1]: [d_date#9#10] -Arguments: [d_date#9#10] - -(65) CometBroadcastHashJoin -Left output [2]: [d_date_sk#8, d_date#9] -Right output [1]: [d_date#9#10] -Arguments: [d_date#9], [d_date#9#10], LeftSemi, BuildRight - -(66) CometProject -Input [2]: [d_date_sk#8, d_date#9] -Arguments: [d_date_sk#8], [d_date_sk#8] - -(67) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#8] - -(68) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 20 Hosting Expression = cr_returned_date_sk#14 IN dynamicpruning#15 -BroadcastExchange (79) -+- * CometColumnarToRow (78) - +- CometProject (77) - +- CometBroadcastHashJoin (76) - :- CometFilter (70) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (69) - +- CometBroadcastExchange (75) - +- CometProject (74) - +- CometBroadcastHashJoin (73) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (71) - +- ReusedExchange (72) - - -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#18, d_date#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(70) CometFilter -Input [2]: [d_date_sk#18, d_date#19] -Condition : isnotnull(d_date_sk#18) - -(71) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#9, d_week_seq#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(72) ReusedExchange [Reuses operator id: 61] -Output [1]: [d_week_seq#39#41] - -(73) CometBroadcastHashJoin -Left output [2]: [d_date#9, d_week_seq#39] -Right output [1]: [d_week_seq#39#41] -Arguments: [d_week_seq#39], [d_week_seq#39#41], LeftSemi, BuildRight - -(74) CometProject -Input [2]: [d_date#9, d_week_seq#39] -Arguments: [d_date#9], [d_date#9] - -(75) CometBroadcastExchange -Input [1]: [d_date#9] -Arguments: [d_date#9] - -(76) CometBroadcastHashJoin -Left output [2]: [d_date_sk#18, d_date#19] -Right output [1]: [d_date#9] -Arguments: [d_date#19], [d_date#9], LeftSemi, BuildRight - -(77) CometProject -Input [2]: [d_date_sk#18, d_date#19] -Arguments: [d_date_sk#18], [d_date_sk#18] - -(78) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#18] - -(79) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:3 Hosting operator id = 39 Hosting Expression = wr_returned_date_sk#27 IN dynamicpruning#15 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/extended.txt deleted file mode 100644 index d325825292..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/extended.txt +++ /dev/null @@ -1,119 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 109 out of 114 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index bff9c86587..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ /dev/null @@ -1,88 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - CometProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] - CometBroadcastHashJoin [item_id,sr_item_qty,cr_item_qty,item_id,wr_item_qty] - CometProject [item_id,sr_item_qty,cr_item_qty] - CometBroadcastHashJoin [item_id,sr_item_qty,item_id,cr_item_qty] - CometHashAggregate [sum] [item_id,sr_item_qty,i_item_id,sum(sr_return_quantity)] - CometExchange [i_item_id] #1 - CometHashAggregate [sr_return_quantity] [i_item_id,sum] - CometProject [sr_return_quantity,i_item_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #3 - CometProject [d_date] [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [d_week_seq] #4 - CometProject [d_week_seq] [d_week_seq] - CometFilter [d_date,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - CometBroadcastExchange [i_item_sk,i_item_id] #5 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #3 - CometBroadcastExchange [item_id,cr_item_qty] #7 - CometHashAggregate [sum] [item_id,cr_item_qty,i_item_id,sum(cr_return_quantity)] - CometExchange [i_item_id] #8 - CometHashAggregate [cr_return_quantity] [i_item_id,sum] - CometProject [cr_return_quantity,i_item_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date] #10 - CometProject [d_date] - CometBroadcastHashJoin [d_date,d_week_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date,d_week_seq] - ReusedExchange [d_week_seq] #4 - ReusedExchange [i_item_sk,i_item_id] #5 - CometBroadcastExchange [d_date_sk] #11 - CometProject [d_date_sk] - CometBroadcastHashJoin [d_date_sk,d_date,d_date] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date] #10 - CometBroadcastExchange [item_id,wr_item_qty] #12 - CometHashAggregate [sum] [item_id,wr_item_qty,i_item_id,sum(wr_return_quantity)] - CometExchange [i_item_id] #13 - CometHashAggregate [wr_return_quantity] [i_item_id,sum] - CometProject [wr_return_quantity,i_item_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [i_item_sk,i_item_id] #5 - ReusedExchange [d_date_sk] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_datafusion/explain.txt deleted file mode 100644 index 63140cfc2d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_datafusion/explain.txt +++ /dev/null @@ -1,190 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometTakeOrderedAndProject (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometFilter (11) - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (10) - : : +- CometBroadcastExchange (17) - : : +- CometFilter (16) - : : +- CometNativeScan parquet spark_catalog.default.household_demographics (15) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometNativeScan parquet spark_catalog.default.income_band (20) - +- CometProject (29) - +- CometFilter (28) - +- CometNativeScan parquet spark_catalog.default.store_returns (27) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Condition : ((isnotnull(c_current_addr_sk#4) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_current_hdemo_sk#3)) - -(3) CometProject -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Arguments: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#1, 16, true, false, true) AS c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#5, 20, true, false, true) AS c_first_name#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#6, 30, true, false, true) AS c_last_name#9] - -(4) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#10, ca_city#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_city), EqualTo(ca_city,Edgewood), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [ca_address_sk#10, ca_city#11] -Condition : ((isnotnull(ca_city#11) AND (ca_city#11 = Edgewood)) AND isnotnull(ca_address_sk#10)) - -(6) CometProject -Input [2]: [ca_address_sk#10, ca_city#11] -Arguments: [ca_address_sk#10], [ca_address_sk#10] - -(7) CometBroadcastExchange -Input [1]: [ca_address_sk#10] -Arguments: [ca_address_sk#10] - -(8) CometBroadcastHashJoin -Left output [6]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9] -Right output [1]: [ca_address_sk#10] -Arguments: [c_current_addr_sk#4], [ca_address_sk#10], Inner, BuildRight - -(9) CometProject -Input [7]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9, ca_address_sk#10] -Arguments: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9], [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9] - -(10) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(11) CometFilter -Input [1]: [cd_demo_sk#12] -Condition : isnotnull(cd_demo_sk#12) - -(12) CometBroadcastExchange -Input [1]: [cd_demo_sk#12] -Arguments: [cd_demo_sk#12] - -(13) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9] -Right output [1]: [cd_demo_sk#12] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#12], Inner, BuildRight - -(14) CometProject -Input [6]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Arguments: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12], [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(15) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Condition : (isnotnull(hd_demo_sk#13) AND isnotnull(hd_income_band_sk#14)) - -(17) CometBroadcastExchange -Input [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [hd_demo_sk#13, hd_income_band_sk#14] - -(18) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Right output [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [c_current_hdemo_sk#3], [hd_demo_sk#13], Inner, BuildRight - -(19) CometProject -Input [7]: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14], [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14] - -(20) CometNativeScan parquet spark_catalog.default.income_band -Output [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_lower_bound), IsNotNull(ib_upper_bound), GreaterThanOrEqual(ib_lower_bound,38128), LessThanOrEqual(ib_upper_bound,88128), IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Condition : ((((isnotnull(ib_lower_bound#16) AND isnotnull(ib_upper_bound#17)) AND (ib_lower_bound#16 >= 38128)) AND (ib_upper_bound#17 <= 88128)) AND isnotnull(ib_income_band_sk#15)) - -(22) CometProject -Input [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Arguments: [ib_income_band_sk#15], [ib_income_band_sk#15] - -(23) CometBroadcastExchange -Input [1]: [ib_income_band_sk#15] -Arguments: [ib_income_band_sk#15] - -(24) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14] -Right output [1]: [ib_income_band_sk#15] -Arguments: [hd_income_band_sk#14], [ib_income_band_sk#15], Inner, BuildRight - -(25) CometProject -Input [6]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14, ib_income_band_sk#15] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12], [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(26) CometBroadcastExchange -Input [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(27) CometNativeScan parquet spark_catalog.default.store_returns -Output [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_cdemo_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_cdemo_sk#18) - -(29) CometProject -Input [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Arguments: [sr_cdemo_sk#18], [sr_cdemo_sk#18] - -(30) CometBroadcastHashJoin -Left output [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Right output [1]: [sr_cdemo_sk#18] -Arguments: [cd_demo_sk#12], [sr_cdemo_sk#18], Inner, BuildLeft - -(31) CometProject -Input [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, sr_cdemo_sk#18] -Arguments: [customer_id#20, customername#21, c_customer_id#7], [c_customer_id#7 AS customer_id#20, concat(c_last_name#9, , , c_first_name#8) AS customername#21, c_customer_id#7] - -(32) CometTakeOrderedAndProject -Input [3]: [customer_id#20, customername#21, c_customer_id#7] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#7 ASC NULLS FIRST], output=[customer_id#20,customername#21]), [customer_id#20, customername#21], 100, 0, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [customer_id#20, customername#21] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_datafusion/simplified.txt deleted file mode 100644 index 9df82b0e98..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_datafusion/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id] [customer_id,customername] - CometProject [c_last_name,c_first_name] [customer_id,customername,c_customer_id] - 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 [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_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_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_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometNativeScan 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 [ca_address_sk] #2 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_city] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [cd_demo_sk] #3 - CometFilter [cd_demo_sk] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange [ib_income_band_sk] #5 - CometProject [ib_income_band_sk] - CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk,sr_returned_date_sk] - CometNativeScan 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/q84.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/explain.txt deleted file mode 100644 index 5bafdc5586..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,190 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometTakeOrderedAndProject (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometFilter (11) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (10) - : : +- CometBroadcastExchange (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (20) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (27) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Condition : ((isnotnull(c_current_addr_sk#4) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_current_hdemo_sk#3)) - -(3) CometProject -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Arguments: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#1, 16, true, false, true) AS c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#5, 20, true, false, true) AS c_first_name#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#6, 30, true, false, true) AS c_last_name#9] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#10, ca_city#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_city), EqualTo(ca_city,Edgewood), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [ca_address_sk#10, ca_city#11] -Condition : ((isnotnull(ca_city#11) AND (ca_city#11 = Edgewood)) AND isnotnull(ca_address_sk#10)) - -(6) CometProject -Input [2]: [ca_address_sk#10, ca_city#11] -Arguments: [ca_address_sk#10], [ca_address_sk#10] - -(7) CometBroadcastExchange -Input [1]: [ca_address_sk#10] -Arguments: [ca_address_sk#10] - -(8) CometBroadcastHashJoin -Left output [6]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9] -Right output [1]: [ca_address_sk#10] -Arguments: [c_current_addr_sk#4], [ca_address_sk#10], Inner, BuildRight - -(9) CometProject -Input [7]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9, ca_address_sk#10] -Arguments: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9], [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(11) CometFilter -Input [1]: [cd_demo_sk#12] -Condition : isnotnull(cd_demo_sk#12) - -(12) CometBroadcastExchange -Input [1]: [cd_demo_sk#12] -Arguments: [cd_demo_sk#12] - -(13) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9] -Right output [1]: [cd_demo_sk#12] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#12], Inner, BuildRight - -(14) CometProject -Input [6]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Arguments: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12], [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Condition : (isnotnull(hd_demo_sk#13) AND isnotnull(hd_income_band_sk#14)) - -(17) CometBroadcastExchange -Input [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [hd_demo_sk#13, hd_income_band_sk#14] - -(18) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Right output [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [c_current_hdemo_sk#3], [hd_demo_sk#13], Inner, BuildRight - -(19) CometProject -Input [7]: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14], [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band -Output [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_lower_bound), IsNotNull(ib_upper_bound), GreaterThanOrEqual(ib_lower_bound,38128), LessThanOrEqual(ib_upper_bound,88128), IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Condition : ((((isnotnull(ib_lower_bound#16) AND isnotnull(ib_upper_bound#17)) AND (ib_lower_bound#16 >= 38128)) AND (ib_upper_bound#17 <= 88128)) AND isnotnull(ib_income_band_sk#15)) - -(22) CometProject -Input [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Arguments: [ib_income_band_sk#15], [ib_income_band_sk#15] - -(23) CometBroadcastExchange -Input [1]: [ib_income_band_sk#15] -Arguments: [ib_income_band_sk#15] - -(24) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14] -Right output [1]: [ib_income_band_sk#15] -Arguments: [hd_income_band_sk#14], [ib_income_band_sk#15], Inner, BuildRight - -(25) CometProject -Input [6]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14, ib_income_band_sk#15] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12], [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(26) CometBroadcastExchange -Input [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_cdemo_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_cdemo_sk#18) - -(29) CometProject -Input [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Arguments: [sr_cdemo_sk#18], [sr_cdemo_sk#18] - -(30) CometBroadcastHashJoin -Left output [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Right output [1]: [sr_cdemo_sk#18] -Arguments: [cd_demo_sk#12], [sr_cdemo_sk#18], Inner, BuildLeft - -(31) CometProject -Input [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, sr_cdemo_sk#18] -Arguments: [customer_id#20, customername#21, c_customer_id#7], [c_customer_id#7 AS customer_id#20, concat(c_last_name#9, , , c_first_name#8) AS customername#21, c_customer_id#7] - -(32) CometTakeOrderedAndProject -Input [3]: [customer_id#20, customername#21, c_customer_id#7] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#7 ASC NULLS FIRST], output=[customer_id#20,customername#21]), [customer_id#20, customername#21], 100, 0, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [customer_id#20, customername#21] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/simplified.txt deleted file mode 100644 index e43557c27d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id] [customer_id,customername] - CometProject [c_last_name,c_first_name] [customer_id,customername,c_customer_id] - 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 [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_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_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_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] 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 [ca_address_sk] #2 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [cd_demo_sk] #3 - CometFilter [cd_demo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange [ib_income_band_sk] #5 - CometProject [ib_income_band_sk] - CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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/q84/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt deleted file mode 100644 index 5bafdc5586..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt +++ /dev/null @@ -1,190 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometTakeOrderedAndProject (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (4) - : : : +- CometBroadcastExchange (12) - : : : +- CometFilter (11) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (10) - : : +- CometBroadcastExchange (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (20) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (27) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Condition : ((isnotnull(c_current_addr_sk#4) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_current_hdemo_sk#3)) - -(3) CometProject -Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] -Arguments: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9], [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#1, 16, true, false, true) AS c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#5, 20, true, false, true) AS c_first_name#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#6, 30, true, false, true) AS c_last_name#9] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#10, ca_city#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_city), EqualTo(ca_city,Edgewood), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [ca_address_sk#10, ca_city#11] -Condition : ((isnotnull(ca_city#11) AND (ca_city#11 = Edgewood)) AND isnotnull(ca_address_sk#10)) - -(6) CometProject -Input [2]: [ca_address_sk#10, ca_city#11] -Arguments: [ca_address_sk#10], [ca_address_sk#10] - -(7) CometBroadcastExchange -Input [1]: [ca_address_sk#10] -Arguments: [ca_address_sk#10] - -(8) CometBroadcastHashJoin -Left output [6]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9] -Right output [1]: [ca_address_sk#10] -Arguments: [c_current_addr_sk#4], [ca_address_sk#10], Inner, BuildRight - -(9) CometProject -Input [7]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#8, c_last_name#9, ca_address_sk#10] -Arguments: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9], [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(11) CometFilter -Input [1]: [cd_demo_sk#12] -Condition : isnotnull(cd_demo_sk#12) - -(12) CometBroadcastExchange -Input [1]: [cd_demo_sk#12] -Arguments: [cd_demo_sk#12] - -(13) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9] -Right output [1]: [cd_demo_sk#12] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#12], Inner, BuildRight - -(14) CometProject -Input [6]: [c_customer_id#7, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Arguments: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12], [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Condition : (isnotnull(hd_demo_sk#13) AND isnotnull(hd_income_band_sk#14)) - -(17) CometBroadcastExchange -Input [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [hd_demo_sk#13, hd_income_band_sk#14] - -(18) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Right output [2]: [hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [c_current_hdemo_sk#3], [hd_demo_sk#13], Inner, BuildRight - -(19) CometProject -Input [7]: [c_customer_id#7, c_current_hdemo_sk#3, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_demo_sk#13, hd_income_band_sk#14] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14], [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band -Output [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_lower_bound), IsNotNull(ib_upper_bound), GreaterThanOrEqual(ib_lower_bound,38128), LessThanOrEqual(ib_upper_bound,88128), IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Condition : ((((isnotnull(ib_lower_bound#16) AND isnotnull(ib_upper_bound#17)) AND (ib_lower_bound#16 >= 38128)) AND (ib_upper_bound#17 <= 88128)) AND isnotnull(ib_income_band_sk#15)) - -(22) CometProject -Input [3]: [ib_income_band_sk#15, ib_lower_bound#16, ib_upper_bound#17] -Arguments: [ib_income_band_sk#15], [ib_income_band_sk#15] - -(23) CometBroadcastExchange -Input [1]: [ib_income_band_sk#15] -Arguments: [ib_income_band_sk#15] - -(24) CometBroadcastHashJoin -Left output [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14] -Right output [1]: [ib_income_band_sk#15] -Arguments: [hd_income_band_sk#14], [ib_income_band_sk#15], Inner, BuildRight - -(25) CometProject -Input [6]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, hd_income_band_sk#14, ib_income_band_sk#15] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12], [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(26) CometBroadcastExchange -Input [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Arguments: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_cdemo_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_cdemo_sk#18) - -(29) CometProject -Input [2]: [sr_cdemo_sk#18, sr_returned_date_sk#19] -Arguments: [sr_cdemo_sk#18], [sr_cdemo_sk#18] - -(30) CometBroadcastHashJoin -Left output [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] -Right output [1]: [sr_cdemo_sk#18] -Arguments: [cd_demo_sk#12], [sr_cdemo_sk#18], Inner, BuildLeft - -(31) CometProject -Input [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, sr_cdemo_sk#18] -Arguments: [customer_id#20, customername#21, c_customer_id#7], [c_customer_id#7 AS customer_id#20, concat(c_last_name#9, , , c_first_name#8) AS customername#21, c_customer_id#7] - -(32) CometTakeOrderedAndProject -Input [3]: [customer_id#20, customername#21, c_customer_id#7] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#7 ASC NULLS FIRST], output=[customer_id#20,customername#21]), [customer_id#20, customername#21], 100, 0, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [customer_id#20, customername#21] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/extended.txt deleted file mode 100644 index 5fce933878..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/extended.txt +++ /dev/null @@ -1,35 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - -Comet accelerated 32 out of 32 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index e43557c27d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [c_customer_id] [customer_id,customername] - CometProject [c_last_name,c_first_name] [customer_id,customername,c_customer_id] - 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 [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_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_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_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] 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 [ca_address_sk] #2 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - CometBroadcastExchange [cd_demo_sk] #3 - CometFilter [cd_demo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #4 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - CometBroadcastExchange [ib_income_band_sk] #5 - CometProject [ib_income_band_sk] - CometFilter [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] - CometProject [sr_cdemo_sk] - CometFilter [sr_cdemo_sk,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/explain.txt deleted file mode 100644 index 204cd4ea7a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/explain.txt +++ /dev/null @@ -1,329 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (52) -+- * HashAggregate (51) - +- * CometColumnarToRow (50) - +- CometColumnarExchange (49) - +- * HashAggregate (48) - +- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) - : : :- * Project (30) - : : : +- * BroadcastHashJoin Inner BuildRight (29) - : : : :- * Project (23) - : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : :- * Project (16) - : : : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : : : :- BroadcastExchange (4) - : : : : : : : +- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : : : : +- * CometColumnarToRow (8) - : : : : : : +- CometProject (7) - : : : : : : +- CometFilter (6) - : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns (5) - : : : : : +- BroadcastExchange (14) - : : : : : +- * CometColumnarToRow (13) - : : : : : +- CometFilter (12) - : : : : : +- CometNativeScan parquet spark_catalog.default.web_page (11) - : : : : +- BroadcastExchange (21) - : : : : +- * CometColumnarToRow (20) - : : : : +- CometProject (19) - : : : : +- CometFilter (18) - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (17) - : : : +- BroadcastExchange (28) - : : : +- * CometColumnarToRow (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (24) - : : +- BroadcastExchange (35) - : : +- * CometColumnarToRow (34) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (31) - : +- ReusedExchange (38) - +- BroadcastExchange (45) - +- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometNativeScan parquet spark_catalog.default.reason (41) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_page_sk), Or(Or(And(GreaterThanOrEqual(ws_sales_price,100.00),LessThanOrEqual(ws_sales_price,150.00)),And(GreaterThanOrEqual(ws_sales_price,50.00),LessThanOrEqual(ws_sales_price,100.00))),And(GreaterThanOrEqual(ws_sales_price,150.00),LessThanOrEqual(ws_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ws_net_profit,100.00),LessThanOrEqual(ws_net_profit,200.00)),And(GreaterThanOrEqual(ws_net_profit,150.00),LessThanOrEqual(ws_net_profit,300.00))),And(GreaterThanOrEqual(ws_net_profit,50.00),LessThanOrEqual(ws_net_profit,250.00)))] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] - -(3) Filter [codegen id : 1] -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((((isnotnull(ws_item_sk#1) AND isnotnull(ws_order_number#3)) AND isnotnull(ws_web_page_sk#2)) AND ((((ws_sales_price#5 >= 100.00) AND (ws_sales_price#5 <= 150.00)) OR ((ws_sales_price#5 >= 50.00) AND (ws_sales_price#5 <= 100.00))) OR ((ws_sales_price#5 >= 150.00) AND (ws_sales_price#5 <= 200.00)))) AND ((((ws_net_profit#6 >= 100.00) AND (ws_net_profit#6 <= 200.00)) OR ((ws_net_profit#6 >= 150.00) AND (ws_net_profit#6 <= 300.00))) OR ((ws_net_profit#6 >= 50.00) AND (ws_net_profit#6 <= 250.00)))) - -(4) BroadcastExchange -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[2, int, false] as bigint) & 4294967295))),false), [plan_id=1] - -(5) CometNativeScan parquet spark_catalog.default.web_returns -Output [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number), IsNotNull(wr_refunded_cdemo_sk), IsNotNull(wr_returning_cdemo_sk), IsNotNull(wr_refunded_addr_sk), IsNotNull(wr_reason_sk)] -ReadSchema: struct - -(6) CometFilter -Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Condition : (((((isnotnull(wr_item_sk#9) AND isnotnull(wr_order_number#14)) AND isnotnull(wr_refunded_cdemo_sk#10)) AND isnotnull(wr_returning_cdemo_sk#12)) AND isnotnull(wr_refunded_addr_sk#11)) AND isnotnull(wr_reason_sk#13)) - -(7) CometProject -Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Arguments: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16], [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] - -(8) CometColumnarToRow -Input [8]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] - -(9) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [ws_item_sk#1, ws_order_number#3] -Right keys [2]: [wr_item_sk#9, wr_order_number#14] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 8] -Output [11]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Input [15]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] - -(11) CometNativeScan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(12) CometFilter -Input [1]: [wp_web_page_sk#18] -Condition : isnotnull(wp_web_page_sk#18) - -(13) CometColumnarToRow [codegen id : 2] -Input [1]: [wp_web_page_sk#18] - -(14) BroadcastExchange -Input [1]: [wp_web_page_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(15) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_web_page_sk#2] -Right keys [1]: [wp_web_page_sk#18] -Join type: Inner -Join condition: None - -(16) Project [codegen id : 8] -Output [10]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Input [12]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, wp_web_page_sk#18] - -(17) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(18) CometFilter -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Condition : (((isnotnull(cd_demo_sk#19) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true))) AND ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = Advanced Degree )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = S) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = College ))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = W) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = 2 yr Degree )))) - -(19) CometProject -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) AS cd_marital_status#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) AS cd_education_status#23] - -(20) CometColumnarToRow [codegen id : 3] -Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] - -(21) BroadcastExchange -Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(22) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [wr_refunded_cdemo_sk#10] -Right keys [1]: [cd_demo_sk#19] -Join type: Inner -Join condition: ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) - -(23) Project [codegen id : 8] -Output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23] -Input [13]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] - -(24) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(25) CometFilter -Input [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Condition : ((isnotnull(cd_demo_sk#24) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#26, 20, true, false, true))) - -(26) CometProject -Input [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Arguments: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28], [cd_demo_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#26, 20, true, false, true) AS cd_education_status#28] - -(27) CometColumnarToRow [codegen id : 4] -Input [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] - -(28) BroadcastExchange -Input [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, string, true], input[2, string, true]),false), [plan_id=4] - -(29) BroadcastHashJoin [codegen id : 8] -Left keys [3]: [wr_returning_cdemo_sk#12, cd_marital_status#22, cd_education_status#23] -Right keys [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Join type: Inner -Join condition: None - -(30) Project [codegen id : 8] -Output [7]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Input [13]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23, cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] - -(31) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Condition : (((isnotnull(ca_country#31) AND (ca_country#31 = United States)) AND isnotnull(ca_address_sk#29)) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) IN (IN,OH,NJ) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) IN (WI,CT,KY)) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) IN (LA,IA,AR))) - -(33) CometProject -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Arguments: [ca_address_sk#29, ca_state#32], [ca_address_sk#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) AS ca_state#32] - -(34) CometColumnarToRow [codegen id : 5] -Input [2]: [ca_address_sk#29, ca_state#32] - -(35) BroadcastExchange -Input [2]: [ca_address_sk#29, ca_state#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(36) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [wr_refunded_addr_sk#11] -Right keys [1]: [ca_address_sk#29] -Join type: Inner -Join condition: ((((ca_state#32 IN (IN,OH,NJ) AND (ws_net_profit#6 >= 100.00)) AND (ws_net_profit#6 <= 200.00)) OR ((ca_state#32 IN (WI,CT,KY) AND (ws_net_profit#6 >= 150.00)) AND (ws_net_profit#6 <= 300.00))) OR ((ca_state#32 IN (LA,IA,AR) AND (ws_net_profit#6 >= 50.00)) AND (ws_net_profit#6 <= 250.00))) - -(37) Project [codegen id : 8] -Output [5]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Input [9]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, ca_address_sk#29, ca_state#32] - -(38) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#33] - -(39) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_date_sk#7] -Right keys [1]: [d_date_sk#33] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 8] -Output [4]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Input [6]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, d_date_sk#33] - -(41) CometNativeScan parquet spark_catalog.default.reason -Output [2]: [r_reason_sk#34, r_reason_desc#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [r_reason_sk#34, r_reason_desc#35] -Condition : isnotnull(r_reason_sk#34) - -(43) CometProject -Input [2]: [r_reason_sk#34, r_reason_desc#35] -Arguments: [r_reason_sk#34, r_reason_desc#36], [r_reason_sk#34, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, r_reason_desc#35, 100, true, false, true) AS r_reason_desc#36] - -(44) CometColumnarToRow [codegen id : 7] -Input [2]: [r_reason_sk#34, r_reason_desc#36] - -(45) BroadcastExchange -Input [2]: [r_reason_sk#34, r_reason_desc#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(46) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [wr_reason_sk#13] -Right keys [1]: [r_reason_sk#34] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 8] -Output [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#36] -Input [6]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, r_reason_sk#34, r_reason_desc#36] - -(48) HashAggregate [codegen id : 8] -Input [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#36] -Keys [1]: [r_reason_desc#36] -Functions [3]: [partial_avg(ws_quantity#4), partial_avg(UnscaledValue(wr_refunded_cash#16)), partial_avg(UnscaledValue(wr_fee#15))] -Aggregate Attributes [6]: [sum#37, count#38, sum#39, count#40, sum#41, count#42] -Results [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, count#48] - -(49) CometColumnarExchange -Input [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, count#48] -Arguments: hashpartitioning(r_reason_desc#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(50) CometColumnarToRow [codegen id : 9] -Input [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, count#48] - -(51) HashAggregate [codegen id : 9] -Input [7]: [r_reason_desc#36, sum#43, count#44, sum#45, count#46, sum#47, count#48] -Keys [1]: [r_reason_desc#36] -Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg(UnscaledValue(wr_fee#15))] -Aggregate Attributes [3]: [avg(ws_quantity#4)#49, avg(UnscaledValue(wr_refunded_cash#16))#50, avg(UnscaledValue(wr_fee#15))#51] -Results [4]: [substr(r_reason_desc#36, 1, 20) AS substr(r_reason_desc, 1, 20)#52, avg(ws_quantity#4)#49 AS avg(ws_quantity)#53, cast((avg(UnscaledValue(wr_refunded_cash#16))#50 / 100.0) as decimal(11,6)) AS avg(wr_refunded_cash)#54, cast((avg(UnscaledValue(wr_fee#15))#51 / 100.0) as decimal(11,6)) AS avg(wr_fee)#55] - -(52) TakeOrderedAndProject -Input [4]: [substr(r_reason_desc, 1, 20)#52, avg(ws_quantity)#53, avg(wr_refunded_cash)#54, avg(wr_fee)#55] -Arguments: 100, [substr(r_reason_desc, 1, 20)#52 ASC NULLS FIRST, avg(ws_quantity)#53 ASC NULLS FIRST, avg(wr_refunded_cash)#54 ASC NULLS FIRST, avg(wr_fee)#55 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#52, avg(ws_quantity)#53, avg(wr_refunded_cash)#54, avg(wr_fee)#55] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometNativeScan parquet spark_catalog.default.date_dim (53) - - -(53) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [2]: [d_date_sk#33, d_year#56] -Condition : ((isnotnull(d_year#56) AND (d_year#56 = 2000)) AND isnotnull(d_date_sk#33)) - -(55) CometProject -Input [2]: [d_date_sk#33, d_year#56] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(56) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#33] - -(57) BroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/simplified.txt deleted file mode 100644 index 68d46e608d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/simplified.txt +++ /dev/null @@ -1,83 +0,0 @@ -TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] - WholeStageCodegen (9) - HashAggregate [r_reason_desc,sum,count,sum,count,sum,count] [avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee)),substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [r_reason_desc] #1 - WholeStageCodegen (8) - HashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - BroadcastHashJoin [wr_reason_sk,r_reason_sk] - Project [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] - Project [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] - BroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] - Project [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] - BroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] - Project [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] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [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] - BroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - 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_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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [r_reason_desc] [r_reason_sk,r_reason_desc] - CometFilter [r_reason_sk,r_reason_desc] - CometNativeScan 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/q85.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/explain.txt deleted file mode 100644 index c263f03d76..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,309 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (48) -+- CometTakeOrderedAndProject (47) - +- CometHashAggregate (46) - +- CometExchange (45) - +- CometHashAggregate (44) - +- CometProject (43) - +- CometBroadcastHashJoin (42) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (19) - : : : : +- CometBroadcastHashJoin (18) - : : : : :- CometProject (13) - : : : : : +- CometBroadcastHashJoin (12) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometBroadcastExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (4) - : : : : : +- CometBroadcastExchange (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (9) - : : : : +- CometBroadcastExchange (17) - : : : : +- CometProject (16) - : : : : +- CometFilter (15) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (14) - : : : +- CometBroadcastExchange (23) - : : : +- CometProject (22) - : : : +- CometFilter (21) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (20) - : : +- CometBroadcastExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (26) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - +- CometBroadcastExchange (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (38) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_page_sk), Or(Or(And(GreaterThanOrEqual(ws_sales_price,100.00),LessThanOrEqual(ws_sales_price,150.00)),And(GreaterThanOrEqual(ws_sales_price,50.00),LessThanOrEqual(ws_sales_price,100.00))),And(GreaterThanOrEqual(ws_sales_price,150.00),LessThanOrEqual(ws_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ws_net_profit,100.00),LessThanOrEqual(ws_net_profit,200.00)),And(GreaterThanOrEqual(ws_net_profit,150.00),LessThanOrEqual(ws_net_profit,300.00))),And(GreaterThanOrEqual(ws_net_profit,50.00),LessThanOrEqual(ws_net_profit,250.00)))] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((((isnotnull(ws_item_sk#1) AND isnotnull(ws_order_number#3)) AND isnotnull(ws_web_page_sk#2)) AND ((((ws_sales_price#5 >= 100.00) AND (ws_sales_price#5 <= 150.00)) OR ((ws_sales_price#5 >= 50.00) AND (ws_sales_price#5 <= 100.00))) OR ((ws_sales_price#5 >= 150.00) AND (ws_sales_price#5 <= 200.00)))) AND ((((ws_net_profit#6 >= 100.00) AND (ws_net_profit#6 <= 200.00)) OR ((ws_net_profit#6 >= 150.00) AND (ws_net_profit#6 <= 300.00))) OR ((ws_net_profit#6 >= 50.00) AND (ws_net_profit#6 <= 250.00)))) - -(3) CometBroadcastExchange -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Arguments: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number), IsNotNull(wr_refunded_cdemo_sk), IsNotNull(wr_returning_cdemo_sk), IsNotNull(wr_refunded_addr_sk), IsNotNull(wr_reason_sk)] -ReadSchema: struct - -(5) CometFilter -Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Condition : (((((isnotnull(wr_item_sk#9) AND isnotnull(wr_order_number#14)) AND isnotnull(wr_refunded_cdemo_sk#10)) AND isnotnull(wr_returning_cdemo_sk#12)) AND isnotnull(wr_refunded_addr_sk#11)) AND isnotnull(wr_reason_sk#13)) - -(6) CometProject -Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Arguments: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16], [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] - -(7) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Right output [8]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] -Arguments: [ws_item_sk#1, ws_order_number#3], [wr_item_sk#9, wr_order_number#14], Inner, BuildLeft - -(8) CometProject -Input [15]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] -Arguments: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(10) CometFilter -Input [1]: [wp_web_page_sk#18] -Condition : isnotnull(wp_web_page_sk#18) - -(11) CometBroadcastExchange -Input [1]: [wp_web_page_sk#18] -Arguments: [wp_web_page_sk#18] - -(12) CometBroadcastHashJoin -Left output [11]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [1]: [wp_web_page_sk#18] -Arguments: [ws_web_page_sk#2], [wp_web_page_sk#18], Inner, BuildRight - -(13) CometProject -Input [12]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, wp_web_page_sk#18] -Arguments: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Condition : (((isnotnull(cd_demo_sk#19) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true))) AND ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = Advanced Degree )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = S) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = College ))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = W) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = 2 yr Degree )))) - -(16) CometProject -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) AS cd_marital_status#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) AS cd_education_status#23] - -(17) CometBroadcastExchange -Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] - -(18) CometBroadcastHashJoin -Left output [10]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [wr_refunded_cdemo_sk#10], [cd_demo_sk#19], Inner, ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))), BuildRight - -(19) CometProject -Input [13]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23], [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Condition : ((isnotnull(cd_demo_sk#24) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#26, 20, true, false, true))) - -(22) CometProject -Input [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Arguments: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28], [cd_demo_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#26, 20, true, false, true) AS cd_education_status#28] - -(23) CometBroadcastExchange -Input [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Arguments: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] - -(24) CometBroadcastHashJoin -Left output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23] -Right output [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Arguments: [wr_returning_cdemo_sk#12, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28], Inner, BuildRight - -(25) CometProject -Input [13]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23, cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Arguments: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Condition : (((isnotnull(ca_country#31) AND (ca_country#31 = United States)) AND isnotnull(ca_address_sk#29)) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) IN (IN,OH,NJ) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) IN (WI,CT,KY)) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) IN (LA,IA,AR))) - -(28) CometProject -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Arguments: [ca_address_sk#29, ca_state#32], [ca_address_sk#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) AS ca_state#32] - -(29) CometBroadcastExchange -Input [2]: [ca_address_sk#29, ca_state#32] -Arguments: [ca_address_sk#29, ca_state#32] - -(30) CometBroadcastHashJoin -Left output [7]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [2]: [ca_address_sk#29, ca_state#32] -Arguments: [wr_refunded_addr_sk#11], [ca_address_sk#29], Inner, ((((ca_state#32 IN (IN,OH,NJ) AND (ws_net_profit#6 >= 100.00)) AND (ws_net_profit#6 <= 200.00)) OR ((ca_state#32 IN (WI,CT,KY) AND (ws_net_profit#6 >= 150.00)) AND (ws_net_profit#6 <= 300.00))) OR ((ca_state#32 IN (LA,IA,AR) AND (ws_net_profit#6 >= 50.00)) AND (ws_net_profit#6 <= 250.00))), BuildRight - -(31) CometProject -Input [9]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, ca_address_sk#29, ca_state#32] -Arguments: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#33, d_year#34] -Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2000)) AND isnotnull(d_date_sk#33)) - -(34) CometProject -Input [2]: [d_date_sk#33, d_year#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(35) CometBroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: [d_date_sk#33] - -(36) CometBroadcastHashJoin -Left output [5]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [1]: [d_date_sk#33] -Arguments: [ws_sold_date_sk#7], [d_date_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, d_date_sk#33] -Arguments: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Output [2]: [r_reason_sk#35, r_reason_desc#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk)] -ReadSchema: struct - -(39) CometFilter -Input [2]: [r_reason_sk#35, r_reason_desc#36] -Condition : isnotnull(r_reason_sk#35) - -(40) CometProject -Input [2]: [r_reason_sk#35, r_reason_desc#36] -Arguments: [r_reason_sk#35, r_reason_desc#37], [r_reason_sk#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, r_reason_desc#36, 100, true, false, true) AS r_reason_desc#37] - -(41) CometBroadcastExchange -Input [2]: [r_reason_sk#35, r_reason_desc#37] -Arguments: [r_reason_sk#35, r_reason_desc#37] - -(42) CometBroadcastHashJoin -Left output [4]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [2]: [r_reason_sk#35, r_reason_desc#37] -Arguments: [wr_reason_sk#13], [r_reason_sk#35], Inner, BuildRight - -(43) CometProject -Input [6]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, r_reason_sk#35, r_reason_desc#37] -Arguments: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#37], [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#37] - -(44) CometHashAggregate -Input [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#37] -Keys [1]: [r_reason_desc#37] -Functions [3]: [partial_avg(ws_quantity#4), partial_avg(UnscaledValue(wr_refunded_cash#16)), partial_avg(UnscaledValue(wr_fee#15))] - -(45) CometExchange -Input [7]: [r_reason_desc#37, sum#38, count#39, sum#40, count#41, sum#42, count#43] -Arguments: hashpartitioning(r_reason_desc#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(46) CometHashAggregate -Input [7]: [r_reason_desc#37, sum#38, count#39, sum#40, count#41, sum#42, count#43] -Keys [1]: [r_reason_desc#37] -Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg(UnscaledValue(wr_fee#15))] - -(47) CometTakeOrderedAndProject -Input [4]: [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(r_reason_desc, 1, 20)#44 ASC NULLS FIRST,avg(ws_quantity)#45 ASC NULLS FIRST,avg(wr_refunded_cash)#46 ASC NULLS FIRST,avg(wr_fee)#47 ASC NULLS FIRST], output=[substr(r_reason_desc, 1, 20)#44,avg(ws_quantity)#45,avg(wr_refunded_cash)#46,avg(wr_fee)#47]), [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47], 100, 0, [substr(r_reason_desc, 1, 20)#44 ASC NULLS FIRST, avg(ws_quantity)#45 ASC NULLS FIRST, avg(wr_refunded_cash)#46 ASC NULLS FIRST, avg(wr_fee)#47 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47] - -(48) CometColumnarToRow [codegen id : 1] -Input [4]: [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [2]: [d_date_sk#33, d_year#34] -Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2000)) AND isnotnull(d_date_sk#33)) - -(51) CometProject -Input [2]: [d_date_sk#33, d_year#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#33] - -(53) BroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/simplified.txt deleted file mode 100644 index dfcca32f03..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,58 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] - CometHashAggregate [sum,count,sum,count,sum,count] [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),r_reason_desc,avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee))] - CometExchange [r_reason_desc] #1 - CometHashAggregate [ws_quantity,wr_refunded_cash,wr_fee] [r_reason_desc,sum,count,sum,count,sum,count] - CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - 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_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 [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 [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 [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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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_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 [native_iceberg_compat] 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 [wp_web_page_sk] #4 - CometFilter [wp_web_page_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [ca_address_sk,ca_state] #7 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #8 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 - CometProject [r_reason_desc] [r_reason_sk,r_reason_desc] - CometFilter [r_reason_sk,r_reason_desc] - CometScan [native_iceberg_compat] 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/q85/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt deleted file mode 100644 index c263f03d76..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt +++ /dev/null @@ -1,309 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (48) -+- CometTakeOrderedAndProject (47) - +- CometHashAggregate (46) - +- CometExchange (45) - +- CometHashAggregate (44) - +- CometProject (43) - +- CometBroadcastHashJoin (42) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (19) - : : : : +- CometBroadcastHashJoin (18) - : : : : :- CometProject (13) - : : : : : +- CometBroadcastHashJoin (12) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometBroadcastExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (4) - : : : : : +- CometBroadcastExchange (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (9) - : : : : +- CometBroadcastExchange (17) - : : : : +- CometProject (16) - : : : : +- CometFilter (15) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (14) - : : : +- CometBroadcastExchange (23) - : : : +- CometProject (22) - : : : +- CometFilter (21) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (20) - : : +- CometBroadcastExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (26) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) - +- CometBroadcastExchange (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (38) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_page_sk), Or(Or(And(GreaterThanOrEqual(ws_sales_price,100.00),LessThanOrEqual(ws_sales_price,150.00)),And(GreaterThanOrEqual(ws_sales_price,50.00),LessThanOrEqual(ws_sales_price,100.00))),And(GreaterThanOrEqual(ws_sales_price,150.00),LessThanOrEqual(ws_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ws_net_profit,100.00),LessThanOrEqual(ws_net_profit,200.00)),And(GreaterThanOrEqual(ws_net_profit,150.00),LessThanOrEqual(ws_net_profit,300.00))),And(GreaterThanOrEqual(ws_net_profit,50.00),LessThanOrEqual(ws_net_profit,250.00)))] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((((isnotnull(ws_item_sk#1) AND isnotnull(ws_order_number#3)) AND isnotnull(ws_web_page_sk#2)) AND ((((ws_sales_price#5 >= 100.00) AND (ws_sales_price#5 <= 150.00)) OR ((ws_sales_price#5 >= 50.00) AND (ws_sales_price#5 <= 100.00))) OR ((ws_sales_price#5 >= 150.00) AND (ws_sales_price#5 <= 200.00)))) AND ((((ws_net_profit#6 >= 100.00) AND (ws_net_profit#6 <= 200.00)) OR ((ws_net_profit#6 >= 150.00) AND (ws_net_profit#6 <= 300.00))) OR ((ws_net_profit#6 >= 50.00) AND (ws_net_profit#6 <= 250.00)))) - -(3) CometBroadcastExchange -Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Arguments: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number), IsNotNull(wr_refunded_cdemo_sk), IsNotNull(wr_returning_cdemo_sk), IsNotNull(wr_refunded_addr_sk), IsNotNull(wr_reason_sk)] -ReadSchema: struct - -(5) CometFilter -Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Condition : (((((isnotnull(wr_item_sk#9) AND isnotnull(wr_order_number#14)) AND isnotnull(wr_refunded_cdemo_sk#10)) AND isnotnull(wr_returning_cdemo_sk#12)) AND isnotnull(wr_refunded_addr_sk#11)) AND isnotnull(wr_reason_sk#13)) - -(6) CometProject -Input [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] -Arguments: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16], [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] - -(7) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Right output [8]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] -Arguments: [ws_item_sk#1, ws_order_number#3], [wr_item_sk#9, wr_order_number#14], Inner, BuildLeft - -(8) CometProject -Input [15]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] -Arguments: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(10) CometFilter -Input [1]: [wp_web_page_sk#18] -Condition : isnotnull(wp_web_page_sk#18) - -(11) CometBroadcastExchange -Input [1]: [wp_web_page_sk#18] -Arguments: [wp_web_page_sk#18] - -(12) CometBroadcastHashJoin -Left output [11]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [1]: [wp_web_page_sk#18] -Arguments: [ws_web_page_sk#2], [wp_web_page_sk#18], Inner, BuildRight - -(13) CometProject -Input [12]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, wp_web_page_sk#18] -Arguments: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Condition : (((isnotnull(cd_demo_sk#19) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true))) AND ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = Advanced Degree )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = S) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = College ))) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) = W) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) = 2 yr Degree )))) - -(16) CometProject -Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) AS cd_marital_status#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) AS cd_education_status#23] - -(17) CometBroadcastExchange -Input [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] - -(18) CometBroadcastHashJoin -Left output [10]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [3]: [cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [wr_refunded_cdemo_sk#10], [cd_demo_sk#19], Inner, ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))), BuildRight - -(19) CometProject -Input [13]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_demo_sk#19, cd_marital_status#22, cd_education_status#23] -Arguments: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23], [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Condition : ((isnotnull(cd_demo_sk#24) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#26, 20, true, false, true))) - -(22) CometProject -Input [3]: [cd_demo_sk#24, cd_marital_status#25, cd_education_status#26] -Arguments: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28], [cd_demo_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#26, 20, true, false, true) AS cd_education_status#28] - -(23) CometBroadcastExchange -Input [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Arguments: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] - -(24) CometBroadcastHashJoin -Left output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23] -Right output [3]: [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Arguments: [wr_returning_cdemo_sk#12, cd_marital_status#22, cd_education_status#23], [cd_demo_sk#24, cd_marital_status#27, cd_education_status#28], Inner, BuildRight - -(25) CometProject -Input [13]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#22, cd_education_status#23, cd_demo_sk#24, cd_marital_status#27, cd_education_status#28] -Arguments: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Condition : (((isnotnull(ca_country#31) AND (ca_country#31 = United States)) AND isnotnull(ca_address_sk#29)) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) IN (IN,OH,NJ) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) IN (WI,CT,KY)) OR staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) IN (LA,IA,AR))) - -(28) CometProject -Input [3]: [ca_address_sk#29, ca_state#30, ca_country#31] -Arguments: [ca_address_sk#29, ca_state#32], [ca_address_sk#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#30, 2, true, false, true) AS ca_state#32] - -(29) CometBroadcastExchange -Input [2]: [ca_address_sk#29, ca_state#32] -Arguments: [ca_address_sk#29, ca_state#32] - -(30) CometBroadcastHashJoin -Left output [7]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [2]: [ca_address_sk#29, ca_state#32] -Arguments: [wr_refunded_addr_sk#11], [ca_address_sk#29], Inner, ((((ca_state#32 IN (IN,OH,NJ) AND (ws_net_profit#6 >= 100.00)) AND (ws_net_profit#6 <= 200.00)) OR ((ca_state#32 IN (WI,CT,KY) AND (ws_net_profit#6 >= 150.00)) AND (ws_net_profit#6 <= 300.00))) OR ((ca_state#32 IN (LA,IA,AR) AND (ws_net_profit#6 >= 50.00)) AND (ws_net_profit#6 <= 250.00))), BuildRight - -(31) CometProject -Input [9]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, ca_address_sk#29, ca_state#32] -Arguments: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#33, d_year#34] -Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2000)) AND isnotnull(d_date_sk#33)) - -(34) CometProject -Input [2]: [d_date_sk#33, d_year#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(35) CometBroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: [d_date_sk#33] - -(36) CometBroadcastHashJoin -Left output [5]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [1]: [d_date_sk#33] -Arguments: [ws_sold_date_sk#7], [d_date_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, d_date_sk#33] -Arguments: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Output [2]: [r_reason_sk#35, r_reason_desc#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk)] -ReadSchema: struct - -(39) CometFilter -Input [2]: [r_reason_sk#35, r_reason_desc#36] -Condition : isnotnull(r_reason_sk#35) - -(40) CometProject -Input [2]: [r_reason_sk#35, r_reason_desc#36] -Arguments: [r_reason_sk#35, r_reason_desc#37], [r_reason_sk#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, r_reason_desc#36, 100, true, false, true) AS r_reason_desc#37] - -(41) CometBroadcastExchange -Input [2]: [r_reason_sk#35, r_reason_desc#37] -Arguments: [r_reason_sk#35, r_reason_desc#37] - -(42) CometBroadcastHashJoin -Left output [4]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] -Right output [2]: [r_reason_sk#35, r_reason_desc#37] -Arguments: [wr_reason_sk#13], [r_reason_sk#35], Inner, BuildRight - -(43) CometProject -Input [6]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, r_reason_sk#35, r_reason_desc#37] -Arguments: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#37], [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#37] - -(44) CometHashAggregate -Input [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#37] -Keys [1]: [r_reason_desc#37] -Functions [3]: [partial_avg(ws_quantity#4), partial_avg(UnscaledValue(wr_refunded_cash#16)), partial_avg(UnscaledValue(wr_fee#15))] - -(45) CometExchange -Input [7]: [r_reason_desc#37, sum#38, count#39, sum#40, count#41, sum#42, count#43] -Arguments: hashpartitioning(r_reason_desc#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(46) CometHashAggregate -Input [7]: [r_reason_desc#37, sum#38, count#39, sum#40, count#41, sum#42, count#43] -Keys [1]: [r_reason_desc#37] -Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg(UnscaledValue(wr_fee#15))] - -(47) CometTakeOrderedAndProject -Input [4]: [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(r_reason_desc, 1, 20)#44 ASC NULLS FIRST,avg(ws_quantity)#45 ASC NULLS FIRST,avg(wr_refunded_cash)#46 ASC NULLS FIRST,avg(wr_fee)#47 ASC NULLS FIRST], output=[substr(r_reason_desc, 1, 20)#44,avg(ws_quantity)#45,avg(wr_refunded_cash)#46,avg(wr_fee)#47]), [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47], 100, 0, [substr(r_reason_desc, 1, 20)#44 ASC NULLS FIRST, avg(ws_quantity)#45 ASC NULLS FIRST, avg(wr_refunded_cash)#46 ASC NULLS FIRST, avg(wr_fee)#47 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47] - -(48) CometColumnarToRow [codegen id : 1] -Input [4]: [substr(r_reason_desc, 1, 20)#44, avg(ws_quantity)#45, avg(wr_refunded_cash)#46, avg(wr_fee)#47] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#33, d_year#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [2]: [d_date_sk#33, d_year#34] -Condition : ((isnotnull(d_year#34) AND (d_year#34 = 2000)) AND isnotnull(d_date_sk#33)) - -(51) CometProject -Input [2]: [d_date_sk#33, d_year#34] -Arguments: [d_date_sk#33], [d_date_sk#33] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#33] - -(53) BroadcastExchange -Input [1]: [d_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/extended.txt deleted file mode 100644 index e2383394d1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/extended.txt +++ /dev/null @@ -1,56 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason - -Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index dfcca32f03..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/simplified.txt +++ /dev/null @@ -1,58 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] - CometHashAggregate [sum,count,sum,count,sum,count] [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),r_reason_desc,avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee))] - CometExchange [r_reason_desc] #1 - CometHashAggregate [ws_quantity,wr_refunded_cash,wr_fee] [r_reason_desc,sum,count,sum,count,sum,count] - CometProject [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] - 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_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 [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 [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 [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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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_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 [native_iceberg_compat] 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 [wp_web_page_sk] #4 - CometFilter [wp_web_page_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #5 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #6 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [ca_address_sk,ca_state] #7 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #8 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [r_reason_sk,r_reason_desc] #9 - CometProject [r_reason_desc] [r_reason_sk,r_reason_desc] - CometFilter [r_reason_sk,r_reason_desc] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/explain.txt deleted file mode 100644 index 3f745e99c5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (17) - +- CometColumnarExchange (16) - +- * HashAggregate (15) - +- * Expand (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (6) - : +- * BroadcastHashJoin Inner BuildRight (5) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.web_sales (1) - : +- ReusedExchange (4) - +- BroadcastExchange (11) - +- * CometColumnarToRow (10) - +- CometProject (9) - +- CometFilter (8) - +- CometNativeScan parquet spark_catalog.default.item (7) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(4) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#5] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [2]: [ws_item_sk#1, ws_net_paid#2] -Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] - -(7) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#6, i_class#7, i_category#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Condition : isnotnull(i_item_sk#6) - -(9) CometProject -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Arguments: [i_item_sk#6, i_class#9, i_category#10], [i_item_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#7, 50, true, false, true) AS i_class#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#8, 50, true, false, true) AS i_category#10] - -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#6, i_class#9, i_category#10] - -(11) BroadcastExchange -Input [3]: [i_item_sk#6, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [3]: [ws_net_paid#2, i_category#10, i_class#9] -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#6, i_class#9, i_category#10] - -(14) Expand [codegen id : 3] -Input [3]: [ws_net_paid#2, i_category#10, i_class#9] -Arguments: [[ws_net_paid#2, i_category#10, i_class#9, 0], [ws_net_paid#2, i_category#10, null, 1], [ws_net_paid#2, null, null, 3]], [ws_net_paid#2, i_category#11, i_class#12, spark_grouping_id#13] - -(15) HashAggregate [codegen id : 3] -Input [4]: [ws_net_paid#2, i_category#11, i_class#12, spark_grouping_id#13] -Keys [3]: [i_category#11, i_class#12, spark_grouping_id#13] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum#14] -Results [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] - -(16) CometColumnarExchange -Input [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] -Arguments: hashpartitioning(i_category#11, i_class#12, spark_grouping_id#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(17) CometColumnarToRow [codegen id : 4] -Input [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] - -(18) HashAggregate [codegen id : 4] -Input [4]: [i_category#11, i_class#12, spark_grouping_id#13, sum#15] -Keys [3]: [i_category#11, i_class#12, spark_grouping_id#13] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#16] -Results [7]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#16,17,2) AS total_sum#17, i_category#11, i_class#12, (cast((shiftright(spark_grouping_id#13, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#13, 0) & 1) as tinyint)) AS lochierarchy#18, MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#16,17,2) AS _w0#19, (cast((shiftright(spark_grouping_id#13, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#13, 0) & 1) as tinyint)) AS _w1#20, CASE WHEN (cast((shiftright(spark_grouping_id#13, 0) & 1) as tinyint) = 0) THEN i_category#11 END AS _w2#21] - -(19) CometColumnarExchange -Input [7]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21] -Arguments: hashpartitioning(_w1#20, _w2#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(20) CometSort -Input [7]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21] -Arguments: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21], [_w1#20 ASC NULLS FIRST, _w2#21 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] - -(21) CometColumnarToRow [codegen id : 5] -Input [7]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21] - -(22) Window -Input [7]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21] -Arguments: [rank(_w0#19) windowspecdefinition(_w1#20, _w2#21, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#22], [_w1#20, _w2#21], [_w0#19 DESC NULLS LAST] - -(23) Project [codegen id : 6] -Output [5]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, rank_within_parent#22] -Input [8]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, _w0#19, _w1#20, _w2#21, rank_within_parent#22] - -(24) TakeOrderedAndProject -Input [5]: [total_sum#17, i_category#11, i_class#12, lochierarchy#18, rank_within_parent#22] -Arguments: 100, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0) THEN i_category#11 END ASC NULLS FIRST, rank_within_parent#22 ASC NULLS FIRST], [total_sum#17, i_category#11, i_class#12, lochierarchy#18, rank_within_parent#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometNativeScan parquet spark_catalog.default.date_dim (25) - - -(25) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#23] -Condition : (((isnotnull(d_month_seq#23) AND (d_month_seq#23 >= 1200)) AND (d_month_seq#23 <= 1211)) AND isnotnull(d_date_sk#5)) - -(27) CometProject -Input [2]: [d_date_sk#5, d_month_seq#23] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(29) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/simplified.txt deleted file mode 100644 index 0b24fe234a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/simplified.txt +++ /dev/null @@ -1,44 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (6) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,spark_grouping_id] #2 - WholeStageCodegen (3) - HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] - Expand [ws_net_paid,i_category,i_class] - Project [ws_net_paid,i_category,i_class] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan 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/q86.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/explain.txt deleted file mode 100644 index 1af5e2612c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,169 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometExchange (19) - +- CometHashAggregate (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometExpand (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] -Arguments: [ws_item_sk#1, ws_net_paid#2], [ws_item_sk#1, ws_net_paid#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#7, i_class#8, i_category#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Condition : isnotnull(i_item_sk#7) - -(11) CometProject -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Arguments: [i_item_sk#7, i_class#10, i_category#11], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#8, 50, true, false, true) AS i_class#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [i_item_sk#7, i_class#10, i_category#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#1, ws_net_paid#2] -Right output [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_net_paid#2, i_category#11, i_class#10], [ws_net_paid#2, i_category#11, i_class#10] - -(15) CometExpand -Input [3]: [ws_net_paid#2, i_category#11, i_class#10] -Arguments: [[ws_net_paid#2, i_category#11, i_class#10, 0], [ws_net_paid#2, i_category#11, null, 1], [ws_net_paid#2, null, null, 3]], [ws_net_paid#2, i_category#12, i_class#13, spark_grouping_id#14] - -(16) CometHashAggregate -Input [4]: [ws_net_paid#2, i_category#12, i_class#13, spark_grouping_id#14] -Keys [3]: [i_category#12, i_class#13, spark_grouping_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] - -(17) CometExchange -Input [4]: [i_category#12, i_class#13, spark_grouping_id#14, sum#15] -Arguments: hashpartitioning(i_category#12, i_class#13, spark_grouping_id#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [4]: [i_category#12, i_class#13, spark_grouping_id#14, sum#15] -Keys [3]: [i_category#12, i_class#13, spark_grouping_id#14] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] - -(19) CometExchange -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -Arguments: hashpartitioning(_w1#19, _w2#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -Arguments: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20], [_w1#19 ASC NULLS FIRST, _w2#20 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] - -(21) CometColumnarToRow [codegen id : 1] -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] - -(22) Window -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -Arguments: [rank(_w0#18) windowspecdefinition(_w1#19, _w2#20, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#21], [_w1#19, _w2#20], [_w0#18 DESC NULLS LAST] - -(23) Project [codegen id : 2] -Output [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] -Input [8]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20, rank_within_parent#21] - -(24) TakeOrderedAndProject -Input [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] -Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#21 ASC NULLS FIRST], [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(27) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(29) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/simplified.txt deleted file mode 100644 index 29e2d72920..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (2) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometExchange [_w1,_w2] #1 - CometHashAggregate [sum] [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ws_net_paid)),spark_grouping_id] - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] - 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,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] 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/q86/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt deleted file mode 100644 index 1af5e2612c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt +++ /dev/null @@ -1,169 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometExchange (19) - +- CometHashAggregate (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometExpand (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] -Arguments: [ws_item_sk#1, ws_net_paid#2], [ws_item_sk#1, ws_net_paid#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#7, i_class#8, i_category#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Condition : isnotnull(i_item_sk#7) - -(11) CometProject -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Arguments: [i_item_sk#7, i_class#10, i_category#11], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#8, 50, true, false, true) AS i_class#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [i_item_sk#7, i_class#10, i_category#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#1, ws_net_paid#2] -Right output [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_net_paid#2, i_category#11, i_class#10], [ws_net_paid#2, i_category#11, i_class#10] - -(15) CometExpand -Input [3]: [ws_net_paid#2, i_category#11, i_class#10] -Arguments: [[ws_net_paid#2, i_category#11, i_class#10, 0], [ws_net_paid#2, i_category#11, null, 1], [ws_net_paid#2, null, null, 3]], [ws_net_paid#2, i_category#12, i_class#13, spark_grouping_id#14] - -(16) CometHashAggregate -Input [4]: [ws_net_paid#2, i_category#12, i_class#13, spark_grouping_id#14] -Keys [3]: [i_category#12, i_class#13, spark_grouping_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] - -(17) CometExchange -Input [4]: [i_category#12, i_class#13, spark_grouping_id#14, sum#15] -Arguments: hashpartitioning(i_category#12, i_class#13, spark_grouping_id#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [4]: [i_category#12, i_class#13, spark_grouping_id#14, sum#15] -Keys [3]: [i_category#12, i_class#13, spark_grouping_id#14] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] - -(19) CometExchange -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -Arguments: hashpartitioning(_w1#19, _w2#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -Arguments: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20], [_w1#19 ASC NULLS FIRST, _w2#20 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] - -(21) CometColumnarToRow [codegen id : 1] -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] - -(22) Window -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -Arguments: [rank(_w0#18) windowspecdefinition(_w1#19, _w2#20, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#21], [_w1#19, _w2#20], [_w0#18 DESC NULLS LAST] - -(23) Project [codegen id : 2] -Output [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] -Input [8]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20, rank_within_parent#21] - -(24) TakeOrderedAndProject -Input [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] -Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#21 ASC NULLS FIRST], [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) - - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(27) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(29) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/extended.txt deleted file mode 100644 index dfc810b108..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/extended.txt +++ /dev/null @@ -1,32 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 29e2d72920..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (2) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometExchange [_w1,_w2] #1 - CometHashAggregate [sum] [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ws_net_paid)),spark_grouping_id] - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] - 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,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/explain.txt deleted file mode 100644 index b3086b072a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/explain.txt +++ /dev/null @@ -1,336 +0,0 @@ -== Physical Plan == -* HashAggregate (52) -+- * CometColumnarToRow (51) - +- CometColumnarExchange (50) - +- * HashAggregate (49) - +- * Project (48) - +- * BroadcastHashJoin LeftAnti BuildRight (47) - :- * BroadcastHashJoin LeftAnti BuildRight (32) - : :- * CometColumnarToRow (17) - : : +- CometHashAggregate (16) - : : +- CometColumnarExchange (15) - : : +- * HashAggregate (14) - : : +- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.customer (7) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometHashAggregate (29) - : +- CometColumnarExchange (28) - : +- * HashAggregate (27) - : +- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * Filter (20) - : : : +- * ColumnarToRow (19) - : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (21) - : +- ReusedExchange (24) - +- BroadcastExchange (46) - +- * CometColumnarToRow (45) - +- CometHashAggregate (44) - +- CometColumnarExchange (43) - +- * HashAggregate (42) - +- * Project (41) - +- * BroadcastHashJoin Inner BuildRight (40) - :- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * Filter (35) - : : +- * ColumnarToRow (34) - : : +- Scan parquet spark_catalog.default.web_sales (33) - : +- ReusedExchange (36) - +- ReusedExchange (39) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sold_date_sk#2 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] - -(3) Filter [codegen id : 3] -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Condition : isnotnull(ss_customer_sk#1) - -(4) ReusedExchange [Reuses operator id: 57] -Output [2]: [d_date_sk#4, d_date#5] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#2] -Right keys [1]: [d_date_sk#4] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [2]: [ss_customer_sk#1, d_date#5] -Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] - -(7) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Condition : isnotnull(c_customer_sk#6) - -(9) CometProject -Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] -Arguments: [c_customer_sk#6, c_first_name#9, c_last_name#10], [c_customer_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#7, 20, true, false, true) AS c_first_name#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#8, 30, true, false, true) AS c_last_name#10] - -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] - -(11) BroadcastExchange -Input [3]: [c_customer_sk#6, c_first_name#9, c_last_name#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#6] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [3]: [c_last_name#10, c_first_name#9, d_date#5] -Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#6, c_first_name#9, c_last_name#10] - -(14) HashAggregate [codegen id : 3] -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#5] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#10, c_first_name#9, d_date#5] - -(15) CometColumnarExchange -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] -Arguments: hashpartitioning(c_last_name#10, c_first_name#9, d_date#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometHashAggregate -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] -Keys [3]: [c_last_name#10, c_first_name#9, d_date#5] -Functions: [] - -(17) CometColumnarToRow [codegen id : 12] -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] - -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#12), dynamicpruningexpression(cs_sold_date_sk#12 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] - -(20) Filter [codegen id : 6] -Input [2]: [cs_bill_customer_sk#11, cs_sold_date_sk#12] -Condition : isnotnull(cs_bill_customer_sk#11) - -(21) ReusedExchange [Reuses operator id: 57] -Output [2]: [d_date_sk#13, d_date#14] - -(22) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#12] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(23) Project [codegen id : 6] -Output [2]: [cs_bill_customer_sk#11, d_date#14] -Input [4]: [cs_bill_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13, d_date#14] - -(24) ReusedExchange [Reuses operator id: 11] -Output [3]: [c_customer_sk#15, c_first_name#16, c_last_name#17] - -(25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_bill_customer_sk#11] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 6] -Output [3]: [c_last_name#17, c_first_name#16, d_date#14] -Input [5]: [cs_bill_customer_sk#11, d_date#14, c_customer_sk#15, c_first_name#16, c_last_name#17] - -(27) HashAggregate [codegen id : 6] -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#17, c_first_name#16, d_date#14] - -(28) CometColumnarExchange -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, d_date#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(29) CometHashAggregate -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Keys [3]: [c_last_name#17, c_first_name#16, d_date#14] -Functions: [] - -(30) CometColumnarToRow [codegen id : 7] -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] - -(31) BroadcastExchange -Input [3]: [c_last_name#17, c_first_name#16, d_date#14] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#17, ), isnull(c_last_name#17), coalesce(c_first_name#16, ), isnull(c_first_name#16), coalesce(d_date#14, 1970-01-01), isnull(d_date#14)] -Join type: LeftAnti -Join condition: None - -(33) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#19), dynamicpruningexpression(ws_sold_date_sk#19 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 10] -Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] - -(35) Filter [codegen id : 10] -Input [2]: [ws_bill_customer_sk#18, ws_sold_date_sk#19] -Condition : isnotnull(ws_bill_customer_sk#18) - -(36) ReusedExchange [Reuses operator id: 57] -Output [2]: [d_date_sk#20, d_date#21] - -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#19] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 10] -Output [2]: [ws_bill_customer_sk#18, d_date#21] -Input [4]: [ws_bill_customer_sk#18, ws_sold_date_sk#19, d_date_sk#20, d_date#21] - -(39) ReusedExchange [Reuses operator id: 11] -Output [3]: [c_customer_sk#22, c_first_name#23, c_last_name#24] - -(40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_bill_customer_sk#18] -Right keys [1]: [c_customer_sk#22] -Join type: Inner -Join condition: None - -(41) Project [codegen id : 10] -Output [3]: [c_last_name#24, c_first_name#23, d_date#21] -Input [5]: [ws_bill_customer_sk#18, d_date#21, c_customer_sk#22, c_first_name#23, c_last_name#24] - -(42) HashAggregate [codegen id : 10] -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] -Functions: [] -Aggregate Attributes: [] -Results [3]: [c_last_name#24, c_first_name#23, d_date#21] - -(43) CometColumnarExchange -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: hashpartitioning(c_last_name#24, c_first_name#23, d_date#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(44) CometHashAggregate -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] -Functions: [] - -(45) CometColumnarToRow [codegen id : 11] -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] - -(46) BroadcastExchange -Input [3]: [c_last_name#24, c_first_name#23, d_date#21] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=6] - -(47) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#24, ), isnull(c_last_name#24), coalesce(c_first_name#23, ), isnull(c_first_name#23), coalesce(d_date#21, 1970-01-01), isnull(d_date#21)] -Join type: LeftAnti -Join condition: None - -(48) Project [codegen id : 12] -Output: [] -Input [3]: [c_last_name#10, c_first_name#9, d_date#5] - -(49) HashAggregate [codegen id : 12] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#25] -Results [1]: [count#26] - -(50) CometColumnarExchange -Input [1]: [count#26] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(51) CometColumnarToRow [codegen id : 13] -Input [1]: [count#26] - -(52) HashAggregate [codegen id : 13] -Input [1]: [count#26] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#27] -Results [1]: [count(1)#27 AS count(1)#28] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (57) -+- * CometColumnarToRow (56) - +- CometProject (55) - +- CometFilter (54) - +- CometNativeScan parquet spark_catalog.default.date_dim (53) - - -(53) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] -Condition : (((isnotnull(d_month_seq#29) AND (d_month_seq#29 >= 1200)) AND (d_month_seq#29 <= 1211)) AND isnotnull(d_date_sk#4)) - -(55) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(56) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] - -(57) BroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#12 IN dynamicpruning#3 - -Subquery:3 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#19 IN dynamicpruning#3 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/simplified.txt deleted file mode 100644 index afc302e779..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/simplified.txt +++ /dev/null @@ -1,86 +0,0 @@ -WholeStageCodegen (13) - HashAggregate [count] [count(1),count(1),count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (12) - HashAggregate [count,count] - Project - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Project [ss_customer_sk,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #6 - WholeStageCodegen (6) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [cs_bill_customer_sk,d_date] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometColumnarExchange [c_last_name,c_first_name,d_date] #8 - WholeStageCodegen (10) - HashAggregate [c_last_name,c_first_name,d_date] - Project [c_last_name,c_first_name,d_date] - BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] - Project [ws_bill_customer_sk,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/explain.txt deleted file mode 100644 index fc6b12b834..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,327 +0,0 @@ -== Physical Plan == -* HashAggregate (51) -+- * CometColumnarToRow (50) - +- CometColumnarExchange (49) - +- * HashAggregate (48) - +- * Project (47) - +- * BroadcastHashJoin LeftAnti BuildRight (46) - :- * BroadcastHashJoin LeftAnti BuildRight (32) - : :- * CometColumnarToRow (18) - : : +- CometHashAggregate (17) - : : +- CometExchange (16) - : : +- CometHashAggregate (15) - : : +- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometHashAggregate (29) - : +- CometExchange (28) - : +- CometHashAggregate (27) - : +- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (19) - : : +- ReusedExchange (21) - : +- ReusedExchange (24) - +- BroadcastExchange (45) - +- * CometColumnarToRow (44) - +- CometHashAggregate (43) - +- CometExchange (42) - +- CometHashAggregate (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : +- ReusedExchange (35) - +- ReusedExchange (38) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sold_date_sk#2 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Condition : isnotnull(ss_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(5) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: [d_date_sk#4, d_date#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Right output [2]: [d_date_sk#4, d_date#5] -Arguments: [ss_sold_date_sk#2], [d_date_sk#4], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] -Arguments: [ss_customer_sk#1, d_date#5], [ss_customer_sk#1, d_date#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Condition : isnotnull(c_customer_sk#7) - -(11) CometProject -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11], [c_customer_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#8, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#9, 30, true, false, true) AS c_last_name#11] - -(12) CometBroadcastExchange -Input [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, d_date#5] -Right output [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [ss_customer_sk#1], [c_customer_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_last_name#11, c_first_name#10, d_date#5], [c_last_name#11, c_first_name#10, d_date#5] - -(15) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(16) CometExchange -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(c_last_name#11, c_first_name#10, d_date#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(18) CometColumnarToRow [codegen id : 3] -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Condition : isnotnull(cs_bill_customer_sk#12) - -(21) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#15, d_date#16] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_date#16] -Arguments: [cs_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(23) CometProject -Input [4]: [cs_bill_customer_sk#12, cs_sold_date_sk#13, d_date_sk#15, d_date#16] -Arguments: [cs_bill_customer_sk#12, d_date#16], [cs_bill_customer_sk#12, d_date#16] - -(24) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] - -(25) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, d_date#16] -Right output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [cs_bill_customer_sk#12], [c_customer_sk#17], Inner, BuildRight - -(26) CometProject -Input [5]: [cs_bill_customer_sk#12, d_date#16, c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [c_last_name#19, c_first_name#18, d_date#16], [c_last_name#19, c_first_name#18, d_date#16] - -(27) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(28) CometExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: hashpartitioning(c_last_name#19, c_first_name#18, d_date#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(30) CometColumnarToRow [codegen id : 1] -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] - -(31) BroadcastExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=3] - -(32) BroadcastHashJoin [codegen id : 3] -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 1970-01-01), isnull(d_date#16)] -Join type: LeftAnti -Join condition: None - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#21), dynamicpruningexpression(ws_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Condition : isnotnull(ws_bill_customer_sk#20) - -(35) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#23, d_date#24] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Right output [2]: [d_date_sk#23, d_date#24] -Arguments: [ws_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(37) CometProject -Input [4]: [ws_bill_customer_sk#20, ws_sold_date_sk#21, d_date_sk#23, d_date#24] -Arguments: [ws_bill_customer_sk#20, d_date#24], [ws_bill_customer_sk#20, d_date#24] - -(38) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] - -(39) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, d_date#24] -Right output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [ws_bill_customer_sk#20], [c_customer_sk#25], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_bill_customer_sk#20, d_date#24, c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_last_name#27, c_first_name#26, d_date#24], [c_last_name#27, c_first_name#26, d_date#24] - -(41) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(42) CometExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: hashpartitioning(c_last_name#27, c_first_name#26, d_date#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(43) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(44) CometColumnarToRow [codegen id : 2] -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] - -(45) BroadcastExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 3] -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#27, ), isnull(c_last_name#27), coalesce(c_first_name#26, ), isnull(c_first_name#26), coalesce(d_date#24, 1970-01-01), isnull(d_date#24)] -Join type: LeftAnti -Join condition: None - -(47) Project [codegen id : 3] -Output: [] -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] - -(48) HashAggregate [codegen id : 3] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#28] -Results [1]: [count#29] - -(49) CometColumnarExchange -Input [1]: [count#29] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(50) CometColumnarToRow [codegen id : 4] -Input [1]: [count#29] - -(51) HashAggregate [codegen id : 4] -Input [1]: [count#29] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#30] -Results [1]: [count(1)#30 AS count(1)#31] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (56) -+- * CometColumnarToRow (55) - +- CometProject (54) - +- CometFilter (53) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) - - -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(54) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(55) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] - -(56) BroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#3 - -Subquery:3 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#21 IN dynamicpruning#3 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/simplified.txt deleted file mode 100644 index f687139735..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -WholeStageCodegen (4) - HashAggregate [count] [count(1),count(1),count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (3) - HashAggregate [count,count] - Project - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #2 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] - CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_customer_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #4 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #7 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] - CometFilter [cs_bill_customer_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #9 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_bill_customer_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt deleted file mode 100644 index fc6b12b834..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt +++ /dev/null @@ -1,327 +0,0 @@ -== Physical Plan == -* HashAggregate (51) -+- * CometColumnarToRow (50) - +- CometColumnarExchange (49) - +- * HashAggregate (48) - +- * Project (47) - +- * BroadcastHashJoin LeftAnti BuildRight (46) - :- * BroadcastHashJoin LeftAnti BuildRight (32) - : :- * CometColumnarToRow (18) - : : +- CometHashAggregate (17) - : : +- CometExchange (16) - : : +- CometHashAggregate (15) - : : +- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometHashAggregate (29) - : +- CometExchange (28) - : +- CometHashAggregate (27) - : +- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometFilter (20) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (19) - : : +- ReusedExchange (21) - : +- ReusedExchange (24) - +- BroadcastExchange (45) - +- * CometColumnarToRow (44) - +- CometHashAggregate (43) - +- CometExchange (42) - +- CometHashAggregate (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : +- ReusedExchange (35) - +- ReusedExchange (38) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sold_date_sk#2 IN dynamicpruning#3)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Condition : isnotnull(ss_customer_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(5) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: [d_date_sk#4, d_date#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] -Right output [2]: [d_date_sk#4, d_date#5] -Arguments: [ss_sold_date_sk#2], [d_date_sk#4], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_customer_sk#1, ss_sold_date_sk#2, d_date_sk#4, d_date#5] -Arguments: [ss_customer_sk#1, d_date#5], [ss_customer_sk#1, d_date#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Condition : isnotnull(c_customer_sk#7) - -(11) CometProject -Input [3]: [c_customer_sk#7, c_first_name#8, c_last_name#9] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11], [c_customer_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#8, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#9, 30, true, false, true) AS c_last_name#11] - -(12) CometBroadcastExchange -Input [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_customer_sk#7, c_first_name#10, c_last_name#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#1, d_date#5] -Right output [3]: [c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [ss_customer_sk#1], [c_customer_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#7, c_first_name#10, c_last_name#11] -Arguments: [c_last_name#11, c_first_name#10, d_date#5], [c_last_name#11, c_first_name#10, d_date#5] - -(15) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(16) CometExchange -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(c_last_name#11, c_first_name#10, d_date#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Keys [3]: [c_last_name#11, c_first_name#10, d_date#5] -Functions: [] - -(18) CometColumnarToRow [codegen id : 3] -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#14)] -PushedFilters: [IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Condition : isnotnull(cs_bill_customer_sk#12) - -(21) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#15, d_date#16] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, cs_sold_date_sk#13] -Right output [2]: [d_date_sk#15, d_date#16] -Arguments: [cs_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(23) CometProject -Input [4]: [cs_bill_customer_sk#12, cs_sold_date_sk#13, d_date_sk#15, d_date#16] -Arguments: [cs_bill_customer_sk#12, d_date#16], [cs_bill_customer_sk#12, d_date#16] - -(24) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] - -(25) CometBroadcastHashJoin -Left output [2]: [cs_bill_customer_sk#12, d_date#16] -Right output [3]: [c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [cs_bill_customer_sk#12], [c_customer_sk#17], Inner, BuildRight - -(26) CometProject -Input [5]: [cs_bill_customer_sk#12, d_date#16, c_customer_sk#17, c_first_name#18, c_last_name#19] -Arguments: [c_last_name#19, c_first_name#18, d_date#16], [c_last_name#19, c_first_name#18, d_date#16] - -(27) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(28) CometExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: hashpartitioning(c_last_name#19, c_first_name#18, d_date#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Keys [3]: [c_last_name#19, c_first_name#18, d_date#16] -Functions: [] - -(30) CometColumnarToRow [codegen id : 1] -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] - -(31) BroadcastExchange -Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=3] - -(32) BroadcastHashJoin [codegen id : 3] -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 1970-01-01), isnull(d_date#16)] -Join type: LeftAnti -Join condition: None - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#21), dynamicpruningexpression(ws_sold_date_sk#21 IN dynamicpruning#22)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Condition : isnotnull(ws_bill_customer_sk#20) - -(35) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#23, d_date#24] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, ws_sold_date_sk#21] -Right output [2]: [d_date_sk#23, d_date#24] -Arguments: [ws_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight - -(37) CometProject -Input [4]: [ws_bill_customer_sk#20, ws_sold_date_sk#21, d_date_sk#23, d_date#24] -Arguments: [ws_bill_customer_sk#20, d_date#24], [ws_bill_customer_sk#20, d_date#24] - -(38) ReusedExchange [Reuses operator id: 12] -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] - -(39) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#20, d_date#24] -Right output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [ws_bill_customer_sk#20], [c_customer_sk#25], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_bill_customer_sk#20, d_date#24, c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_last_name#27, c_first_name#26, d_date#24], [c_last_name#27, c_first_name#26, d_date#24] - -(41) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(42) CometExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: hashpartitioning(c_last_name#27, c_first_name#26, d_date#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(43) CometHashAggregate -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Keys [3]: [c_last_name#27, c_first_name#26, d_date#24] -Functions: [] - -(44) CometColumnarToRow [codegen id : 2] -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] - -(45) BroadcastExchange -Input [3]: [c_last_name#27, c_first_name#26, d_date#24] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 3] -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#27, ), isnull(c_last_name#27), coalesce(c_first_name#26, ), isnull(c_first_name#26), coalesce(d_date#24, 1970-01-01), isnull(d_date#24)] -Join type: LeftAnti -Join condition: None - -(47) Project [codegen id : 3] -Output: [] -Input [3]: [c_last_name#11, c_first_name#10, d_date#5] - -(48) HashAggregate [codegen id : 3] -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#28] -Results [1]: [count#29] - -(49) CometColumnarExchange -Input [1]: [count#29] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(50) CometColumnarToRow [codegen id : 4] -Input [1]: [count#29] - -(51) HashAggregate [codegen id : 4] -Input [1]: [count#29] -Keys: [] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#30] -Results [1]: [count(1)#30 AS count(1)#31] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 -BroadcastExchange (56) -+- * CometColumnarToRow (55) - +- CometProject (54) - +- CometFilter (53) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (52) - - -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#4)) - -(54) CometProject -Input [3]: [d_date_sk#4, d_date#5, d_month_seq#6] -Arguments: [d_date_sk#4, d_date#5], [d_date_sk#4, d_date#5] - -(55) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] - -(56) BroadcastExchange -Input [2]: [d_date_sk#4, d_date#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 19 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#3 - -Subquery:3 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#21 IN dynamicpruning#3 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/extended.txt deleted file mode 100644 index ddee139acf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/extended.txt +++ /dev/null @@ -1,73 +0,0 @@ -HashAggregate -+- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] - : :- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 55 out of 66 eligible operators (83%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index f687139735..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -WholeStageCodegen (4) - HashAggregate [count] [count(1),count(1),count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (3) - HashAggregate [count,count] - Project - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #2 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - CometBroadcastHashJoin [ss_customer_sk,d_date,c_customer_sk,c_first_name,c_last_name] - CometProject [ss_customer_sk,d_date] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_customer_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #4 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #5 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #7 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,cs_sold_date_sk,d_date_sk,d_date] - CometFilter [cs_bill_customer_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [c_last_name,c_first_name,d_date] - CometExchange [c_last_name,c_first_name,d_date] #9 - CometHashAggregate [c_last_name,c_first_name,d_date] - CometProject [c_last_name,c_first_name,d_date] - 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_bill_customer_sk,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_bill_customer_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - ReusedExchange [c_customer_sk,c_first_name,c_last_name] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_datafusion/explain.txt deleted file mode 100644 index 25b4c305f2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_datafusion/explain.txt +++ /dev/null @@ -1,927 +0,0 @@ -== Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (172) -:- * BroadcastNestedLoopJoin Inner BuildRight (151) -: :- * BroadcastNestedLoopJoin Inner BuildRight (130) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (109) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) -: : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) -: : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : : : : : :- * CometColumnarToRow (25) -: : : : : : : +- CometHashAggregate (24) -: : : : : : : +- CometExchange (23) -: : : : : : : +- CometHashAggregate (22) -: : : : : : : +- CometProject (21) -: : : : : : : +- CometBroadcastHashJoin (20) -: : : : : : : :- CometProject (15) -: : : : : : : : +- CometBroadcastHashJoin (14) -: : : : : : : : :- CometProject (9) -: : : : : : : : : +- CometBroadcastHashJoin (8) -: : : : : : : : : :- CometProject (3) -: : : : : : : : : : +- CometFilter (2) -: : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) -: : : : : : : : : +- CometBroadcastExchange (7) -: : : : : : : : : +- CometProject (6) -: : : : : : : : : +- CometFilter (5) -: : : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (4) -: : : : : : : : +- CometBroadcastExchange (13) -: : : : : : : : +- CometProject (12) -: : : : : : : : +- CometFilter (11) -: : : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (10) -: : : : : : : +- CometBroadcastExchange (19) -: : : : : : : +- CometProject (18) -: : : : : : : +- CometFilter (17) -: : : : : : : +- CometNativeScan parquet spark_catalog.default.store (16) -: : : : : : +- BroadcastExchange (45) -: : : : : : +- * CometColumnarToRow (44) -: : : : : : +- CometHashAggregate (43) -: : : : : : +- CometExchange (42) -: : : : : : +- CometHashAggregate (41) -: : : : : : +- CometProject (40) -: : : : : : +- CometBroadcastHashJoin (39) -: : : : : : :- CometProject (37) -: : : : : : : +- CometBroadcastHashJoin (36) -: : : : : : : :- CometProject (31) -: : : : : : : : +- CometBroadcastHashJoin (30) -: : : : : : : : :- CometProject (28) -: : : : : : : : : +- CometFilter (27) -: : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (26) -: : : : : : : : +- ReusedExchange (29) -: : : : : : : +- CometBroadcastExchange (35) -: : : : : : : +- CometProject (34) -: : : : : : : +- CometFilter (33) -: : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (32) -: : : : : : +- ReusedExchange (38) -: : : : : +- BroadcastExchange (66) -: : : : : +- * CometColumnarToRow (65) -: : : : : +- CometHashAggregate (64) -: : : : : +- CometExchange (63) -: : : : : +- CometHashAggregate (62) -: : : : : +- CometProject (61) -: : : : : +- CometBroadcastHashJoin (60) -: : : : : :- CometProject (58) -: : : : : : +- CometBroadcastHashJoin (57) -: : : : : : :- CometProject (52) -: : : : : : : +- CometBroadcastHashJoin (51) -: : : : : : : :- CometProject (49) -: : : : : : : : +- CometFilter (48) -: : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (47) -: : : : : : : +- ReusedExchange (50) -: : : : : : +- CometBroadcastExchange (56) -: : : : : : +- CometProject (55) -: : : : : : +- CometFilter (54) -: : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (53) -: : : : : +- ReusedExchange (59) -: : : : +- BroadcastExchange (87) -: : : : +- * CometColumnarToRow (86) -: : : : +- CometHashAggregate (85) -: : : : +- CometExchange (84) -: : : : +- CometHashAggregate (83) -: : : : +- CometProject (82) -: : : : +- CometBroadcastHashJoin (81) -: : : : :- CometProject (79) -: : : : : +- CometBroadcastHashJoin (78) -: : : : : :- CometProject (73) -: : : : : : +- CometBroadcastHashJoin (72) -: : : : : : :- CometProject (70) -: : : : : : : +- CometFilter (69) -: : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (68) -: : : : : : +- ReusedExchange (71) -: : : : : +- CometBroadcastExchange (77) -: : : : : +- CometProject (76) -: : : : : +- CometFilter (75) -: : : : : +- CometNativeScan parquet spark_catalog.default.time_dim (74) -: : : : +- ReusedExchange (80) -: : : +- BroadcastExchange (108) -: : : +- * CometColumnarToRow (107) -: : : +- CometHashAggregate (106) -: : : +- CometExchange (105) -: : : +- CometHashAggregate (104) -: : : +- CometProject (103) -: : : +- CometBroadcastHashJoin (102) -: : : :- CometProject (100) -: : : : +- CometBroadcastHashJoin (99) -: : : : :- CometProject (94) -: : : : : +- CometBroadcastHashJoin (93) -: : : : : :- CometProject (91) -: : : : : : +- CometFilter (90) -: : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (89) -: : : : : +- ReusedExchange (92) -: : : : +- CometBroadcastExchange (98) -: : : : +- CometProject (97) -: : : : +- CometFilter (96) -: : : : +- CometNativeScan parquet spark_catalog.default.time_dim (95) -: : : +- ReusedExchange (101) -: : +- BroadcastExchange (129) -: : +- * CometColumnarToRow (128) -: : +- CometHashAggregate (127) -: : +- CometExchange (126) -: : +- CometHashAggregate (125) -: : +- CometProject (124) -: : +- CometBroadcastHashJoin (123) -: : :- CometProject (121) -: : : +- CometBroadcastHashJoin (120) -: : : :- CometProject (115) -: : : : +- CometBroadcastHashJoin (114) -: : : : :- CometProject (112) -: : : : : +- CometFilter (111) -: : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (110) -: : : : +- ReusedExchange (113) -: : : +- CometBroadcastExchange (119) -: : : +- CometProject (118) -: : : +- CometFilter (117) -: : : +- CometNativeScan parquet spark_catalog.default.time_dim (116) -: : +- ReusedExchange (122) -: +- BroadcastExchange (150) -: +- * CometColumnarToRow (149) -: +- CometHashAggregate (148) -: +- CometExchange (147) -: +- CometHashAggregate (146) -: +- CometProject (145) -: +- CometBroadcastHashJoin (144) -: :- CometProject (142) -: : +- CometBroadcastHashJoin (141) -: : :- CometProject (136) -: : : +- CometBroadcastHashJoin (135) -: : : :- CometProject (133) -: : : : +- CometFilter (132) -: : : : +- CometNativeScan parquet spark_catalog.default.store_sales (131) -: : : +- ReusedExchange (134) -: : +- CometBroadcastExchange (140) -: : +- CometProject (139) -: : +- CometFilter (138) -: : +- CometNativeScan parquet spark_catalog.default.time_dim (137) -: +- ReusedExchange (143) -+- BroadcastExchange (171) - +- * CometColumnarToRow (170) - +- CometHashAggregate (169) - +- CometExchange (168) - +- CometHashAggregate (167) - +- CometProject (166) - +- CometBroadcastHashJoin (165) - :- CometProject (163) - : +- CometBroadcastHashJoin (162) - : :- CometProject (157) - : : +- CometBroadcastHashJoin (156) - : : :- CometProject (154) - : : : +- CometFilter (153) - : : : +- CometNativeScan parquet spark_catalog.default.store_sales (152) - : : +- ReusedExchange (155) - : +- CometBroadcastExchange (161) - : +- CometProject (160) - : +- CometFilter (159) - : +- CometNativeScan parquet spark_catalog.default.time_dim (158) - +- ReusedExchange (164) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(3) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(4) CometNativeScan parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(Or(And(EqualTo(hd_dep_count,4),LessThanOrEqual(hd_vehicle_count,6)),And(EqualTo(hd_dep_count,2),LessThanOrEqual(hd_vehicle_count,4))),And(EqualTo(hd_dep_count,0),LessThanOrEqual(hd_vehicle_count,2))), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Condition : (((((hd_dep_count#6 = 4) AND (hd_vehicle_count#7 <= 6)) OR ((hd_dep_count#6 = 2) AND (hd_vehicle_count#7 <= 4))) OR ((hd_dep_count#6 = 0) AND (hd_vehicle_count#7 <= 2))) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] -Arguments: [ss_sold_time_sk#1, ss_store_sk#3], [ss_sold_time_sk#1, ss_store_sk#3] - -(10) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,8), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Condition : ((((isnotnull(t_hour#9) AND isnotnull(t_minute#10)) AND (t_hour#9 = 8)) AND (t_minute#10 >= 30)) AND isnotnull(t_time_sk#8)) - -(12) CometProject -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Arguments: [t_time_sk#8], [t_time_sk#8] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#8] -Arguments: [t_time_sk#8] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Right output [1]: [t_time_sk#8] -Arguments: [ss_sold_time_sk#1], [t_time_sk#8], Inner, BuildRight - -(15) CometProject -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#8] -Arguments: [ss_store_sk#3], [ss_store_sk#3] - -(16) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_store_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [s_store_sk#11, s_store_name#12] -Condition : ((isnotnull(s_store_name#12) AND (s_store_name#12 = ese)) AND isnotnull(s_store_sk#11)) - -(18) CometProject -Input [2]: [s_store_sk#11, s_store_name#12] -Arguments: [s_store_sk#11], [s_store_sk#11] - -(19) CometBroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: [s_store_sk#11] - -(20) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#3] -Right output [1]: [s_store_sk#11] -Arguments: [ss_store_sk#3], [s_store_sk#11], Inner, BuildRight - -(21) CometProject -Input [2]: [ss_store_sk#3, s_store_sk#11] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#13] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#13] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 8] -Input [1]: [h8_30_to_9#14] - -(26) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Condition : ((isnotnull(ss_hdemo_sk#16) AND isnotnull(ss_sold_time_sk#15)) AND isnotnull(ss_store_sk#17)) - -(28) CometProject -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Arguments: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17], [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] - -(29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#19] - -(30) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] -Right output [1]: [hd_demo_sk#19] -Arguments: [ss_hdemo_sk#16], [hd_demo_sk#19], Inner, BuildRight - -(31) CometProject -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, hd_demo_sk#19] -Arguments: [ss_sold_time_sk#15, ss_store_sk#17], [ss_sold_time_sk#15, ss_store_sk#17] - -(32) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Condition : ((((isnotnull(t_hour#21) AND isnotnull(t_minute#22)) AND (t_hour#21 = 9)) AND (t_minute#22 < 30)) AND isnotnull(t_time_sk#20)) - -(34) CometProject -Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Arguments: [t_time_sk#20], [t_time_sk#20] - -(35) CometBroadcastExchange -Input [1]: [t_time_sk#20] -Arguments: [t_time_sk#20] - -(36) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#15, ss_store_sk#17] -Right output [1]: [t_time_sk#20] -Arguments: [ss_sold_time_sk#15], [t_time_sk#20], Inner, BuildRight - -(37) CometProject -Input [3]: [ss_sold_time_sk#15, ss_store_sk#17, t_time_sk#20] -Arguments: [ss_store_sk#17], [ss_store_sk#17] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#23] - -(39) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#17] -Right output [1]: [s_store_sk#23] -Arguments: [ss_store_sk#17], [s_store_sk#23], Inner, BuildRight - -(40) CometProject -Input [2]: [ss_store_sk#17, s_store_sk#23] - -(41) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(42) CometExchange -Input [1]: [count#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(43) CometHashAggregate -Input [1]: [count#24] -Keys: [] -Functions [1]: [count(1)] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [h9_to_9_30#25] - -(45) BroadcastExchange -Input [1]: [h9_to_9_30#25] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(46) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(47) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(48) CometFilter -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_hdemo_sk#27) AND isnotnull(ss_sold_time_sk#26)) AND isnotnull(ss_store_sk#28)) - -(49) CometProject -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Arguments: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28], [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] - -(50) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#30] - -(51) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] -Right output [1]: [hd_demo_sk#30] -Arguments: [ss_hdemo_sk#27], [hd_demo_sk#30], Inner, BuildRight - -(52) CometProject -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, hd_demo_sk#30] -Arguments: [ss_sold_time_sk#26, ss_store_sk#28], [ss_sold_time_sk#26, ss_store_sk#28] - -(53) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Condition : ((((isnotnull(t_hour#32) AND isnotnull(t_minute#33)) AND (t_hour#32 = 9)) AND (t_minute#33 >= 30)) AND isnotnull(t_time_sk#31)) - -(55) CometProject -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Arguments: [t_time_sk#31], [t_time_sk#31] - -(56) CometBroadcastExchange -Input [1]: [t_time_sk#31] -Arguments: [t_time_sk#31] - -(57) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#26, ss_store_sk#28] -Right output [1]: [t_time_sk#31] -Arguments: [ss_sold_time_sk#26], [t_time_sk#31], Inner, BuildRight - -(58) CometProject -Input [3]: [ss_sold_time_sk#26, ss_store_sk#28, t_time_sk#31] -Arguments: [ss_store_sk#28], [ss_store_sk#28] - -(59) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#34] - -(60) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#28] -Right output [1]: [s_store_sk#34] -Arguments: [ss_store_sk#28], [s_store_sk#34], Inner, BuildRight - -(61) CometProject -Input [2]: [ss_store_sk#28, s_store_sk#34] - -(62) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(63) CometExchange -Input [1]: [count#35] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(64) CometHashAggregate -Input [1]: [count#35] -Keys: [] -Functions [1]: [count(1)] - -(65) CometColumnarToRow [codegen id : 2] -Input [1]: [h9_30_to_10#36] - -(66) BroadcastExchange -Input [1]: [h9_30_to_10#36] -Arguments: IdentityBroadcastMode, [plan_id=5] - -(67) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(68) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(69) CometFilter -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Condition : ((isnotnull(ss_hdemo_sk#38) AND isnotnull(ss_sold_time_sk#37)) AND isnotnull(ss_store_sk#39)) - -(70) CometProject -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Arguments: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39], [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] - -(71) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#41] - -(72) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] -Right output [1]: [hd_demo_sk#41] -Arguments: [ss_hdemo_sk#38], [hd_demo_sk#41], Inner, BuildRight - -(73) CometProject -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, hd_demo_sk#41] -Arguments: [ss_sold_time_sk#37, ss_store_sk#39], [ss_sold_time_sk#37, ss_store_sk#39] - -(74) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(75) CometFilter -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Condition : ((((isnotnull(t_hour#43) AND isnotnull(t_minute#44)) AND (t_hour#43 = 10)) AND (t_minute#44 < 30)) AND isnotnull(t_time_sk#42)) - -(76) CometProject -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Arguments: [t_time_sk#42], [t_time_sk#42] - -(77) CometBroadcastExchange -Input [1]: [t_time_sk#42] -Arguments: [t_time_sk#42] - -(78) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#37, ss_store_sk#39] -Right output [1]: [t_time_sk#42] -Arguments: [ss_sold_time_sk#37], [t_time_sk#42], Inner, BuildRight - -(79) CometProject -Input [3]: [ss_sold_time_sk#37, ss_store_sk#39, t_time_sk#42] -Arguments: [ss_store_sk#39], [ss_store_sk#39] - -(80) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#45] - -(81) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#39] -Right output [1]: [s_store_sk#45] -Arguments: [ss_store_sk#39], [s_store_sk#45], Inner, BuildRight - -(82) CometProject -Input [2]: [ss_store_sk#39, s_store_sk#45] - -(83) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(84) CometExchange -Input [1]: [count#46] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(85) CometHashAggregate -Input [1]: [count#46] -Keys: [] -Functions [1]: [count(1)] - -(86) CometColumnarToRow [codegen id : 3] -Input [1]: [h10_to_10_30#47] - -(87) BroadcastExchange -Input [1]: [h10_to_10_30#47] -Arguments: IdentityBroadcastMode, [plan_id=7] - -(88) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(89) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(90) CometFilter -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Condition : ((isnotnull(ss_hdemo_sk#49) AND isnotnull(ss_sold_time_sk#48)) AND isnotnull(ss_store_sk#50)) - -(91) CometProject -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Arguments: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50], [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] - -(92) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#52] - -(93) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] -Right output [1]: [hd_demo_sk#52] -Arguments: [ss_hdemo_sk#49], [hd_demo_sk#52], Inner, BuildRight - -(94) CometProject -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, hd_demo_sk#52] -Arguments: [ss_sold_time_sk#48, ss_store_sk#50], [ss_sold_time_sk#48, ss_store_sk#50] - -(95) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(96) CometFilter -Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Condition : ((((isnotnull(t_hour#54) AND isnotnull(t_minute#55)) AND (t_hour#54 = 10)) AND (t_minute#55 >= 30)) AND isnotnull(t_time_sk#53)) - -(97) CometProject -Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Arguments: [t_time_sk#53], [t_time_sk#53] - -(98) CometBroadcastExchange -Input [1]: [t_time_sk#53] -Arguments: [t_time_sk#53] - -(99) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#48, ss_store_sk#50] -Right output [1]: [t_time_sk#53] -Arguments: [ss_sold_time_sk#48], [t_time_sk#53], Inner, BuildRight - -(100) CometProject -Input [3]: [ss_sold_time_sk#48, ss_store_sk#50, t_time_sk#53] -Arguments: [ss_store_sk#50], [ss_store_sk#50] - -(101) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#56] - -(102) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#50] -Right output [1]: [s_store_sk#56] -Arguments: [ss_store_sk#50], [s_store_sk#56], Inner, BuildRight - -(103) CometProject -Input [2]: [ss_store_sk#50, s_store_sk#56] - -(104) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(105) CometExchange -Input [1]: [count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(106) CometHashAggregate -Input [1]: [count#57] -Keys: [] -Functions [1]: [count(1)] - -(107) CometColumnarToRow [codegen id : 4] -Input [1]: [h10_30_to_11#58] - -(108) BroadcastExchange -Input [1]: [h10_30_to_11#58] -Arguments: IdentityBroadcastMode, [plan_id=9] - -(109) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(110) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(111) CometFilter -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Condition : ((isnotnull(ss_hdemo_sk#60) AND isnotnull(ss_sold_time_sk#59)) AND isnotnull(ss_store_sk#61)) - -(112) CometProject -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Arguments: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61], [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] - -(113) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#63] - -(114) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] -Right output [1]: [hd_demo_sk#63] -Arguments: [ss_hdemo_sk#60], [hd_demo_sk#63], Inner, BuildRight - -(115) CometProject -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, hd_demo_sk#63] -Arguments: [ss_sold_time_sk#59, ss_store_sk#61], [ss_sold_time_sk#59, ss_store_sk#61] - -(116) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(117) CometFilter -Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Condition : ((((isnotnull(t_hour#65) AND isnotnull(t_minute#66)) AND (t_hour#65 = 11)) AND (t_minute#66 < 30)) AND isnotnull(t_time_sk#64)) - -(118) CometProject -Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Arguments: [t_time_sk#64], [t_time_sk#64] - -(119) CometBroadcastExchange -Input [1]: [t_time_sk#64] -Arguments: [t_time_sk#64] - -(120) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#59, ss_store_sk#61] -Right output [1]: [t_time_sk#64] -Arguments: [ss_sold_time_sk#59], [t_time_sk#64], Inner, BuildRight - -(121) CometProject -Input [3]: [ss_sold_time_sk#59, ss_store_sk#61, t_time_sk#64] -Arguments: [ss_store_sk#61], [ss_store_sk#61] - -(122) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#67] - -(123) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#61] -Right output [1]: [s_store_sk#67] -Arguments: [ss_store_sk#61], [s_store_sk#67], Inner, BuildRight - -(124) CometProject -Input [2]: [ss_store_sk#61, s_store_sk#67] - -(125) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(126) CometExchange -Input [1]: [count#68] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(127) CometHashAggregate -Input [1]: [count#68] -Keys: [] -Functions [1]: [count(1)] - -(128) CometColumnarToRow [codegen id : 5] -Input [1]: [h11_to_11_30#69] - -(129) BroadcastExchange -Input [1]: [h11_to_11_30#69] -Arguments: IdentityBroadcastMode, [plan_id=11] - -(130) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(131) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(132) CometFilter -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Condition : ((isnotnull(ss_hdemo_sk#71) AND isnotnull(ss_sold_time_sk#70)) AND isnotnull(ss_store_sk#72)) - -(133) CometProject -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Arguments: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72], [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] - -(134) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#74] - -(135) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] -Right output [1]: [hd_demo_sk#74] -Arguments: [ss_hdemo_sk#71], [hd_demo_sk#74], Inner, BuildRight - -(136) CometProject -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, hd_demo_sk#74] -Arguments: [ss_sold_time_sk#70, ss_store_sk#72], [ss_sold_time_sk#70, ss_store_sk#72] - -(137) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(138) CometFilter -Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Condition : ((((isnotnull(t_hour#76) AND isnotnull(t_minute#77)) AND (t_hour#76 = 11)) AND (t_minute#77 >= 30)) AND isnotnull(t_time_sk#75)) - -(139) CometProject -Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Arguments: [t_time_sk#75], [t_time_sk#75] - -(140) CometBroadcastExchange -Input [1]: [t_time_sk#75] -Arguments: [t_time_sk#75] - -(141) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#70, ss_store_sk#72] -Right output [1]: [t_time_sk#75] -Arguments: [ss_sold_time_sk#70], [t_time_sk#75], Inner, BuildRight - -(142) CometProject -Input [3]: [ss_sold_time_sk#70, ss_store_sk#72, t_time_sk#75] -Arguments: [ss_store_sk#72], [ss_store_sk#72] - -(143) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#78] - -(144) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#72] -Right output [1]: [s_store_sk#78] -Arguments: [ss_store_sk#72], [s_store_sk#78], Inner, BuildRight - -(145) CometProject -Input [2]: [ss_store_sk#72, s_store_sk#78] - -(146) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(147) CometExchange -Input [1]: [count#79] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(148) CometHashAggregate -Input [1]: [count#79] -Keys: [] -Functions [1]: [count(1)] - -(149) CometColumnarToRow [codegen id : 6] -Input [1]: [h11_30_to_12#80] - -(150) BroadcastExchange -Input [1]: [h11_30_to_12#80] -Arguments: IdentityBroadcastMode, [plan_id=13] - -(151) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(152) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(153) CometFilter -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Condition : ((isnotnull(ss_hdemo_sk#82) AND isnotnull(ss_sold_time_sk#81)) AND isnotnull(ss_store_sk#83)) - -(154) CometProject -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Arguments: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83], [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] - -(155) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#85] - -(156) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] -Right output [1]: [hd_demo_sk#85] -Arguments: [ss_hdemo_sk#82], [hd_demo_sk#85], Inner, BuildRight - -(157) CometProject -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, hd_demo_sk#85] -Arguments: [ss_sold_time_sk#81, ss_store_sk#83], [ss_sold_time_sk#81, ss_store_sk#83] - -(158) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(159) CometFilter -Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Condition : ((((isnotnull(t_hour#87) AND isnotnull(t_minute#88)) AND (t_hour#87 = 12)) AND (t_minute#88 < 30)) AND isnotnull(t_time_sk#86)) - -(160) CometProject -Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Arguments: [t_time_sk#86], [t_time_sk#86] - -(161) CometBroadcastExchange -Input [1]: [t_time_sk#86] -Arguments: [t_time_sk#86] - -(162) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#81, ss_store_sk#83] -Right output [1]: [t_time_sk#86] -Arguments: [ss_sold_time_sk#81], [t_time_sk#86], Inner, BuildRight - -(163) CometProject -Input [3]: [ss_sold_time_sk#81, ss_store_sk#83, t_time_sk#86] -Arguments: [ss_store_sk#83], [ss_store_sk#83] - -(164) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#89] - -(165) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#83] -Right output [1]: [s_store_sk#89] -Arguments: [ss_store_sk#83], [s_store_sk#89], Inner, BuildRight - -(166) CometProject -Input [2]: [ss_store_sk#83, s_store_sk#89] - -(167) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(168) CometExchange -Input [1]: [count#90] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(169) CometHashAggregate -Input [1]: [count#90] -Keys: [] -Functions [1]: [count(1)] - -(170) CometColumnarToRow [codegen id : 7] -Input [1]: [h12_to_12_30#91] - -(171) BroadcastExchange -Input [1]: [h12_to_12_30#91] -Arguments: IdentityBroadcastMode, [plan_id=15] - -(172) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_datafusion/simplified.txt deleted file mode 100644 index cef8e0e760..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_datafusion/simplified.txt +++ /dev/null @@ -1,195 +0,0 @@ -WholeStageCodegen (8) - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h8_30_to_9,count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_to_9_30,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan 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 [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_30_to_10,count(1)] - CometExchange #9 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan 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 [t_time_sk] #10 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_to_10_30,count(1)] - CometExchange #12 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan 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 [t_time_sk] #13 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_30_to_11,count(1)] - CometExchange #15 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan 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 [t_time_sk] #16 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_to_11_30,count(1)] - CometExchange #18 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan 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 [t_time_sk] #19 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_30_to_12,count(1)] - CometExchange #21 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan 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 [t_time_sk] #22 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #23 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h12_to_12_30,count(1)] - CometExchange #24 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan 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 [t_time_sk] #25 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan 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/q88.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_iceberg_compat/explain.txt deleted file mode 100644 index 2e49d20af4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,927 +0,0 @@ -== Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (172) -:- * BroadcastNestedLoopJoin Inner BuildRight (151) -: :- * BroadcastNestedLoopJoin Inner BuildRight (130) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (109) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) -: : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) -: : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : : : : : :- * CometColumnarToRow (25) -: : : : : : : +- CometHashAggregate (24) -: : : : : : : +- CometExchange (23) -: : : : : : : +- CometHashAggregate (22) -: : : : : : : +- CometProject (21) -: : : : : : : +- CometBroadcastHashJoin (20) -: : : : : : : :- CometProject (15) -: : : : : : : : +- CometBroadcastHashJoin (14) -: : : : : : : : :- CometProject (9) -: : : : : : : : : +- CometBroadcastHashJoin (8) -: : : : : : : : : :- CometProject (3) -: : : : : : : : : : +- CometFilter (2) -: : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) -: : : : : : : : : +- CometBroadcastExchange (7) -: : : : : : : : : +- CometProject (6) -: : : : : : : : : +- CometFilter (5) -: : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (4) -: : : : : : : : +- CometBroadcastExchange (13) -: : : : : : : : +- CometProject (12) -: : : : : : : : +- CometFilter (11) -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (10) -: : : : : : : +- CometBroadcastExchange (19) -: : : : : : : +- CometProject (18) -: : : : : : : +- CometFilter (17) -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (16) -: : : : : : +- BroadcastExchange (45) -: : : : : : +- * CometColumnarToRow (44) -: : : : : : +- CometHashAggregate (43) -: : : : : : +- CometExchange (42) -: : : : : : +- CometHashAggregate (41) -: : : : : : +- CometProject (40) -: : : : : : +- CometBroadcastHashJoin (39) -: : : : : : :- CometProject (37) -: : : : : : : +- CometBroadcastHashJoin (36) -: : : : : : : :- CometProject (31) -: : : : : : : : +- CometBroadcastHashJoin (30) -: : : : : : : : :- CometProject (28) -: : : : : : : : : +- CometFilter (27) -: : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (26) -: : : : : : : : +- ReusedExchange (29) -: : : : : : : +- CometBroadcastExchange (35) -: : : : : : : +- CometProject (34) -: : : : : : : +- CometFilter (33) -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (32) -: : : : : : +- ReusedExchange (38) -: : : : : +- BroadcastExchange (66) -: : : : : +- * CometColumnarToRow (65) -: : : : : +- CometHashAggregate (64) -: : : : : +- CometExchange (63) -: : : : : +- CometHashAggregate (62) -: : : : : +- CometProject (61) -: : : : : +- CometBroadcastHashJoin (60) -: : : : : :- CometProject (58) -: : : : : : +- CometBroadcastHashJoin (57) -: : : : : : :- CometProject (52) -: : : : : : : +- CometBroadcastHashJoin (51) -: : : : : : : :- CometProject (49) -: : : : : : : : +- CometFilter (48) -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (47) -: : : : : : : +- ReusedExchange (50) -: : : : : : +- CometBroadcastExchange (56) -: : : : : : +- CometProject (55) -: : : : : : +- CometFilter (54) -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (53) -: : : : : +- ReusedExchange (59) -: : : : +- BroadcastExchange (87) -: : : : +- * CometColumnarToRow (86) -: : : : +- CometHashAggregate (85) -: : : : +- CometExchange (84) -: : : : +- CometHashAggregate (83) -: : : : +- CometProject (82) -: : : : +- CometBroadcastHashJoin (81) -: : : : :- CometProject (79) -: : : : : +- CometBroadcastHashJoin (78) -: : : : : :- CometProject (73) -: : : : : : +- CometBroadcastHashJoin (72) -: : : : : : :- CometProject (70) -: : : : : : : +- CometFilter (69) -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (68) -: : : : : : +- ReusedExchange (71) -: : : : : +- CometBroadcastExchange (77) -: : : : : +- CometProject (76) -: : : : : +- CometFilter (75) -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (74) -: : : : +- ReusedExchange (80) -: : : +- BroadcastExchange (108) -: : : +- * CometColumnarToRow (107) -: : : +- CometHashAggregate (106) -: : : +- CometExchange (105) -: : : +- CometHashAggregate (104) -: : : +- CometProject (103) -: : : +- CometBroadcastHashJoin (102) -: : : :- CometProject (100) -: : : : +- CometBroadcastHashJoin (99) -: : : : :- CometProject (94) -: : : : : +- CometBroadcastHashJoin (93) -: : : : : :- CometProject (91) -: : : : : : +- CometFilter (90) -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (89) -: : : : : +- ReusedExchange (92) -: : : : +- CometBroadcastExchange (98) -: : : : +- CometProject (97) -: : : : +- CometFilter (96) -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (95) -: : : +- ReusedExchange (101) -: : +- BroadcastExchange (129) -: : +- * CometColumnarToRow (128) -: : +- CometHashAggregate (127) -: : +- CometExchange (126) -: : +- CometHashAggregate (125) -: : +- CometProject (124) -: : +- CometBroadcastHashJoin (123) -: : :- CometProject (121) -: : : +- CometBroadcastHashJoin (120) -: : : :- CometProject (115) -: : : : +- CometBroadcastHashJoin (114) -: : : : :- CometProject (112) -: : : : : +- CometFilter (111) -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (110) -: : : : +- ReusedExchange (113) -: : : +- CometBroadcastExchange (119) -: : : +- CometProject (118) -: : : +- CometFilter (117) -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (116) -: : +- ReusedExchange (122) -: +- BroadcastExchange (150) -: +- * CometColumnarToRow (149) -: +- CometHashAggregate (148) -: +- CometExchange (147) -: +- CometHashAggregate (146) -: +- CometProject (145) -: +- CometBroadcastHashJoin (144) -: :- CometProject (142) -: : +- CometBroadcastHashJoin (141) -: : :- CometProject (136) -: : : +- CometBroadcastHashJoin (135) -: : : :- CometProject (133) -: : : : +- CometFilter (132) -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (131) -: : : +- ReusedExchange (134) -: : +- CometBroadcastExchange (140) -: : +- CometProject (139) -: : +- CometFilter (138) -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (137) -: +- ReusedExchange (143) -+- BroadcastExchange (171) - +- * CometColumnarToRow (170) - +- CometHashAggregate (169) - +- CometExchange (168) - +- CometHashAggregate (167) - +- CometProject (166) - +- CometBroadcastHashJoin (165) - :- CometProject (163) - : +- CometBroadcastHashJoin (162) - : :- CometProject (157) - : : +- CometBroadcastHashJoin (156) - : : :- CometProject (154) - : : : +- CometFilter (153) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (152) - : : +- ReusedExchange (155) - : +- CometBroadcastExchange (161) - : +- CometProject (160) - : +- CometFilter (159) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (158) - +- ReusedExchange (164) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(3) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(Or(And(EqualTo(hd_dep_count,4),LessThanOrEqual(hd_vehicle_count,6)),And(EqualTo(hd_dep_count,2),LessThanOrEqual(hd_vehicle_count,4))),And(EqualTo(hd_dep_count,0),LessThanOrEqual(hd_vehicle_count,2))), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Condition : (((((hd_dep_count#6 = 4) AND (hd_vehicle_count#7 <= 6)) OR ((hd_dep_count#6 = 2) AND (hd_vehicle_count#7 <= 4))) OR ((hd_dep_count#6 = 0) AND (hd_vehicle_count#7 <= 2))) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] -Arguments: [ss_sold_time_sk#1, ss_store_sk#3], [ss_sold_time_sk#1, ss_store_sk#3] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,8), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Condition : ((((isnotnull(t_hour#9) AND isnotnull(t_minute#10)) AND (t_hour#9 = 8)) AND (t_minute#10 >= 30)) AND isnotnull(t_time_sk#8)) - -(12) CometProject -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Arguments: [t_time_sk#8], [t_time_sk#8] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#8] -Arguments: [t_time_sk#8] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Right output [1]: [t_time_sk#8] -Arguments: [ss_sold_time_sk#1], [t_time_sk#8], Inner, BuildRight - -(15) CometProject -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#8] -Arguments: [ss_store_sk#3], [ss_store_sk#3] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_store_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [s_store_sk#11, s_store_name#12] -Condition : ((isnotnull(s_store_name#12) AND (s_store_name#12 = ese)) AND isnotnull(s_store_sk#11)) - -(18) CometProject -Input [2]: [s_store_sk#11, s_store_name#12] -Arguments: [s_store_sk#11], [s_store_sk#11] - -(19) CometBroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: [s_store_sk#11] - -(20) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#3] -Right output [1]: [s_store_sk#11] -Arguments: [ss_store_sk#3], [s_store_sk#11], Inner, BuildRight - -(21) CometProject -Input [2]: [ss_store_sk#3, s_store_sk#11] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#13] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#13] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 8] -Input [1]: [h8_30_to_9#14] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Condition : ((isnotnull(ss_hdemo_sk#16) AND isnotnull(ss_sold_time_sk#15)) AND isnotnull(ss_store_sk#17)) - -(28) CometProject -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Arguments: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17], [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] - -(29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#19] - -(30) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] -Right output [1]: [hd_demo_sk#19] -Arguments: [ss_hdemo_sk#16], [hd_demo_sk#19], Inner, BuildRight - -(31) CometProject -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, hd_demo_sk#19] -Arguments: [ss_sold_time_sk#15, ss_store_sk#17], [ss_sold_time_sk#15, ss_store_sk#17] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Condition : ((((isnotnull(t_hour#21) AND isnotnull(t_minute#22)) AND (t_hour#21 = 9)) AND (t_minute#22 < 30)) AND isnotnull(t_time_sk#20)) - -(34) CometProject -Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Arguments: [t_time_sk#20], [t_time_sk#20] - -(35) CometBroadcastExchange -Input [1]: [t_time_sk#20] -Arguments: [t_time_sk#20] - -(36) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#15, ss_store_sk#17] -Right output [1]: [t_time_sk#20] -Arguments: [ss_sold_time_sk#15], [t_time_sk#20], Inner, BuildRight - -(37) CometProject -Input [3]: [ss_sold_time_sk#15, ss_store_sk#17, t_time_sk#20] -Arguments: [ss_store_sk#17], [ss_store_sk#17] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#23] - -(39) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#17] -Right output [1]: [s_store_sk#23] -Arguments: [ss_store_sk#17], [s_store_sk#23], Inner, BuildRight - -(40) CometProject -Input [2]: [ss_store_sk#17, s_store_sk#23] - -(41) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(42) CometExchange -Input [1]: [count#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(43) CometHashAggregate -Input [1]: [count#24] -Keys: [] -Functions [1]: [count(1)] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [h9_to_9_30#25] - -(45) BroadcastExchange -Input [1]: [h9_to_9_30#25] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(46) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(48) CometFilter -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_hdemo_sk#27) AND isnotnull(ss_sold_time_sk#26)) AND isnotnull(ss_store_sk#28)) - -(49) CometProject -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Arguments: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28], [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] - -(50) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#30] - -(51) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] -Right output [1]: [hd_demo_sk#30] -Arguments: [ss_hdemo_sk#27], [hd_demo_sk#30], Inner, BuildRight - -(52) CometProject -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, hd_demo_sk#30] -Arguments: [ss_sold_time_sk#26, ss_store_sk#28], [ss_sold_time_sk#26, ss_store_sk#28] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Condition : ((((isnotnull(t_hour#32) AND isnotnull(t_minute#33)) AND (t_hour#32 = 9)) AND (t_minute#33 >= 30)) AND isnotnull(t_time_sk#31)) - -(55) CometProject -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Arguments: [t_time_sk#31], [t_time_sk#31] - -(56) CometBroadcastExchange -Input [1]: [t_time_sk#31] -Arguments: [t_time_sk#31] - -(57) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#26, ss_store_sk#28] -Right output [1]: [t_time_sk#31] -Arguments: [ss_sold_time_sk#26], [t_time_sk#31], Inner, BuildRight - -(58) CometProject -Input [3]: [ss_sold_time_sk#26, ss_store_sk#28, t_time_sk#31] -Arguments: [ss_store_sk#28], [ss_store_sk#28] - -(59) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#34] - -(60) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#28] -Right output [1]: [s_store_sk#34] -Arguments: [ss_store_sk#28], [s_store_sk#34], Inner, BuildRight - -(61) CometProject -Input [2]: [ss_store_sk#28, s_store_sk#34] - -(62) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(63) CometExchange -Input [1]: [count#35] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(64) CometHashAggregate -Input [1]: [count#35] -Keys: [] -Functions [1]: [count(1)] - -(65) CometColumnarToRow [codegen id : 2] -Input [1]: [h9_30_to_10#36] - -(66) BroadcastExchange -Input [1]: [h9_30_to_10#36] -Arguments: IdentityBroadcastMode, [plan_id=5] - -(67) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(69) CometFilter -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Condition : ((isnotnull(ss_hdemo_sk#38) AND isnotnull(ss_sold_time_sk#37)) AND isnotnull(ss_store_sk#39)) - -(70) CometProject -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Arguments: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39], [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] - -(71) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#41] - -(72) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] -Right output [1]: [hd_demo_sk#41] -Arguments: [ss_hdemo_sk#38], [hd_demo_sk#41], Inner, BuildRight - -(73) CometProject -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, hd_demo_sk#41] -Arguments: [ss_sold_time_sk#37, ss_store_sk#39], [ss_sold_time_sk#37, ss_store_sk#39] - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(75) CometFilter -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Condition : ((((isnotnull(t_hour#43) AND isnotnull(t_minute#44)) AND (t_hour#43 = 10)) AND (t_minute#44 < 30)) AND isnotnull(t_time_sk#42)) - -(76) CometProject -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Arguments: [t_time_sk#42], [t_time_sk#42] - -(77) CometBroadcastExchange -Input [1]: [t_time_sk#42] -Arguments: [t_time_sk#42] - -(78) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#37, ss_store_sk#39] -Right output [1]: [t_time_sk#42] -Arguments: [ss_sold_time_sk#37], [t_time_sk#42], Inner, BuildRight - -(79) CometProject -Input [3]: [ss_sold_time_sk#37, ss_store_sk#39, t_time_sk#42] -Arguments: [ss_store_sk#39], [ss_store_sk#39] - -(80) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#45] - -(81) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#39] -Right output [1]: [s_store_sk#45] -Arguments: [ss_store_sk#39], [s_store_sk#45], Inner, BuildRight - -(82) CometProject -Input [2]: [ss_store_sk#39, s_store_sk#45] - -(83) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(84) CometExchange -Input [1]: [count#46] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(85) CometHashAggregate -Input [1]: [count#46] -Keys: [] -Functions [1]: [count(1)] - -(86) CometColumnarToRow [codegen id : 3] -Input [1]: [h10_to_10_30#47] - -(87) BroadcastExchange -Input [1]: [h10_to_10_30#47] -Arguments: IdentityBroadcastMode, [plan_id=7] - -(88) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(89) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(90) CometFilter -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Condition : ((isnotnull(ss_hdemo_sk#49) AND isnotnull(ss_sold_time_sk#48)) AND isnotnull(ss_store_sk#50)) - -(91) CometProject -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Arguments: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50], [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] - -(92) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#52] - -(93) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] -Right output [1]: [hd_demo_sk#52] -Arguments: [ss_hdemo_sk#49], [hd_demo_sk#52], Inner, BuildRight - -(94) CometProject -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, hd_demo_sk#52] -Arguments: [ss_sold_time_sk#48, ss_store_sk#50], [ss_sold_time_sk#48, ss_store_sk#50] - -(95) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(96) CometFilter -Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Condition : ((((isnotnull(t_hour#54) AND isnotnull(t_minute#55)) AND (t_hour#54 = 10)) AND (t_minute#55 >= 30)) AND isnotnull(t_time_sk#53)) - -(97) CometProject -Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Arguments: [t_time_sk#53], [t_time_sk#53] - -(98) CometBroadcastExchange -Input [1]: [t_time_sk#53] -Arguments: [t_time_sk#53] - -(99) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#48, ss_store_sk#50] -Right output [1]: [t_time_sk#53] -Arguments: [ss_sold_time_sk#48], [t_time_sk#53], Inner, BuildRight - -(100) CometProject -Input [3]: [ss_sold_time_sk#48, ss_store_sk#50, t_time_sk#53] -Arguments: [ss_store_sk#50], [ss_store_sk#50] - -(101) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#56] - -(102) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#50] -Right output [1]: [s_store_sk#56] -Arguments: [ss_store_sk#50], [s_store_sk#56], Inner, BuildRight - -(103) CometProject -Input [2]: [ss_store_sk#50, s_store_sk#56] - -(104) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(105) CometExchange -Input [1]: [count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(106) CometHashAggregate -Input [1]: [count#57] -Keys: [] -Functions [1]: [count(1)] - -(107) CometColumnarToRow [codegen id : 4] -Input [1]: [h10_30_to_11#58] - -(108) BroadcastExchange -Input [1]: [h10_30_to_11#58] -Arguments: IdentityBroadcastMode, [plan_id=9] - -(109) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(110) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(111) CometFilter -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Condition : ((isnotnull(ss_hdemo_sk#60) AND isnotnull(ss_sold_time_sk#59)) AND isnotnull(ss_store_sk#61)) - -(112) CometProject -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Arguments: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61], [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] - -(113) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#63] - -(114) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] -Right output [1]: [hd_demo_sk#63] -Arguments: [ss_hdemo_sk#60], [hd_demo_sk#63], Inner, BuildRight - -(115) CometProject -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, hd_demo_sk#63] -Arguments: [ss_sold_time_sk#59, ss_store_sk#61], [ss_sold_time_sk#59, ss_store_sk#61] - -(116) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(117) CometFilter -Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Condition : ((((isnotnull(t_hour#65) AND isnotnull(t_minute#66)) AND (t_hour#65 = 11)) AND (t_minute#66 < 30)) AND isnotnull(t_time_sk#64)) - -(118) CometProject -Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Arguments: [t_time_sk#64], [t_time_sk#64] - -(119) CometBroadcastExchange -Input [1]: [t_time_sk#64] -Arguments: [t_time_sk#64] - -(120) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#59, ss_store_sk#61] -Right output [1]: [t_time_sk#64] -Arguments: [ss_sold_time_sk#59], [t_time_sk#64], Inner, BuildRight - -(121) CometProject -Input [3]: [ss_sold_time_sk#59, ss_store_sk#61, t_time_sk#64] -Arguments: [ss_store_sk#61], [ss_store_sk#61] - -(122) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#67] - -(123) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#61] -Right output [1]: [s_store_sk#67] -Arguments: [ss_store_sk#61], [s_store_sk#67], Inner, BuildRight - -(124) CometProject -Input [2]: [ss_store_sk#61, s_store_sk#67] - -(125) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(126) CometExchange -Input [1]: [count#68] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(127) CometHashAggregate -Input [1]: [count#68] -Keys: [] -Functions [1]: [count(1)] - -(128) CometColumnarToRow [codegen id : 5] -Input [1]: [h11_to_11_30#69] - -(129) BroadcastExchange -Input [1]: [h11_to_11_30#69] -Arguments: IdentityBroadcastMode, [plan_id=11] - -(130) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(131) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(132) CometFilter -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Condition : ((isnotnull(ss_hdemo_sk#71) AND isnotnull(ss_sold_time_sk#70)) AND isnotnull(ss_store_sk#72)) - -(133) CometProject -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Arguments: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72], [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] - -(134) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#74] - -(135) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] -Right output [1]: [hd_demo_sk#74] -Arguments: [ss_hdemo_sk#71], [hd_demo_sk#74], Inner, BuildRight - -(136) CometProject -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, hd_demo_sk#74] -Arguments: [ss_sold_time_sk#70, ss_store_sk#72], [ss_sold_time_sk#70, ss_store_sk#72] - -(137) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(138) CometFilter -Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Condition : ((((isnotnull(t_hour#76) AND isnotnull(t_minute#77)) AND (t_hour#76 = 11)) AND (t_minute#77 >= 30)) AND isnotnull(t_time_sk#75)) - -(139) CometProject -Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Arguments: [t_time_sk#75], [t_time_sk#75] - -(140) CometBroadcastExchange -Input [1]: [t_time_sk#75] -Arguments: [t_time_sk#75] - -(141) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#70, ss_store_sk#72] -Right output [1]: [t_time_sk#75] -Arguments: [ss_sold_time_sk#70], [t_time_sk#75], Inner, BuildRight - -(142) CometProject -Input [3]: [ss_sold_time_sk#70, ss_store_sk#72, t_time_sk#75] -Arguments: [ss_store_sk#72], [ss_store_sk#72] - -(143) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#78] - -(144) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#72] -Right output [1]: [s_store_sk#78] -Arguments: [ss_store_sk#72], [s_store_sk#78], Inner, BuildRight - -(145) CometProject -Input [2]: [ss_store_sk#72, s_store_sk#78] - -(146) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(147) CometExchange -Input [1]: [count#79] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(148) CometHashAggregate -Input [1]: [count#79] -Keys: [] -Functions [1]: [count(1)] - -(149) CometColumnarToRow [codegen id : 6] -Input [1]: [h11_30_to_12#80] - -(150) BroadcastExchange -Input [1]: [h11_30_to_12#80] -Arguments: IdentityBroadcastMode, [plan_id=13] - -(151) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(152) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(153) CometFilter -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Condition : ((isnotnull(ss_hdemo_sk#82) AND isnotnull(ss_sold_time_sk#81)) AND isnotnull(ss_store_sk#83)) - -(154) CometProject -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Arguments: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83], [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] - -(155) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#85] - -(156) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] -Right output [1]: [hd_demo_sk#85] -Arguments: [ss_hdemo_sk#82], [hd_demo_sk#85], Inner, BuildRight - -(157) CometProject -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, hd_demo_sk#85] -Arguments: [ss_sold_time_sk#81, ss_store_sk#83], [ss_sold_time_sk#81, ss_store_sk#83] - -(158) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(159) CometFilter -Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Condition : ((((isnotnull(t_hour#87) AND isnotnull(t_minute#88)) AND (t_hour#87 = 12)) AND (t_minute#88 < 30)) AND isnotnull(t_time_sk#86)) - -(160) CometProject -Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Arguments: [t_time_sk#86], [t_time_sk#86] - -(161) CometBroadcastExchange -Input [1]: [t_time_sk#86] -Arguments: [t_time_sk#86] - -(162) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#81, ss_store_sk#83] -Right output [1]: [t_time_sk#86] -Arguments: [ss_sold_time_sk#81], [t_time_sk#86], Inner, BuildRight - -(163) CometProject -Input [3]: [ss_sold_time_sk#81, ss_store_sk#83, t_time_sk#86] -Arguments: [ss_store_sk#83], [ss_store_sk#83] - -(164) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#89] - -(165) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#83] -Right output [1]: [s_store_sk#89] -Arguments: [ss_store_sk#83], [s_store_sk#89], Inner, BuildRight - -(166) CometProject -Input [2]: [ss_store_sk#83, s_store_sk#89] - -(167) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(168) CometExchange -Input [1]: [count#90] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(169) CometHashAggregate -Input [1]: [count#90] -Keys: [] -Functions [1]: [count(1)] - -(170) CometColumnarToRow [codegen id : 7] -Input [1]: [h12_to_12_30#91] - -(171) BroadcastExchange -Input [1]: [h12_to_12_30#91] -Arguments: IdentityBroadcastMode, [plan_id=15] - -(172) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_iceberg_compat/simplified.txt deleted file mode 100644 index e3923fb3ea..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,195 +0,0 @@ -WholeStageCodegen (8) - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h8_30_to_9,count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_to_9_30,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_30_to_10,count(1)] - CometExchange #9 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #10 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_to_10_30,count(1)] - CometExchange #12 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #13 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_30_to_11,count(1)] - CometExchange #15 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #16 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_to_11_30,count(1)] - CometExchange #18 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #19 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_30_to_12,count(1)] - CometExchange #21 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #22 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #23 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h12_to_12_30,count(1)] - CometExchange #24 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #25 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] 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/q88/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt deleted file mode 100644 index 2e49d20af4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt +++ /dev/null @@ -1,927 +0,0 @@ -== Physical Plan == -* BroadcastNestedLoopJoin Inner BuildRight (172) -:- * BroadcastNestedLoopJoin Inner BuildRight (151) -: :- * BroadcastNestedLoopJoin Inner BuildRight (130) -: : :- * BroadcastNestedLoopJoin Inner BuildRight (109) -: : : :- * BroadcastNestedLoopJoin Inner BuildRight (88) -: : : : :- * BroadcastNestedLoopJoin Inner BuildRight (67) -: : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (46) -: : : : : : :- * CometColumnarToRow (25) -: : : : : : : +- CometHashAggregate (24) -: : : : : : : +- CometExchange (23) -: : : : : : : +- CometHashAggregate (22) -: : : : : : : +- CometProject (21) -: : : : : : : +- CometBroadcastHashJoin (20) -: : : : : : : :- CometProject (15) -: : : : : : : : +- CometBroadcastHashJoin (14) -: : : : : : : : :- CometProject (9) -: : : : : : : : : +- CometBroadcastHashJoin (8) -: : : : : : : : : :- CometProject (3) -: : : : : : : : : : +- CometFilter (2) -: : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) -: : : : : : : : : +- CometBroadcastExchange (7) -: : : : : : : : : +- CometProject (6) -: : : : : : : : : +- CometFilter (5) -: : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (4) -: : : : : : : : +- CometBroadcastExchange (13) -: : : : : : : : +- CometProject (12) -: : : : : : : : +- CometFilter (11) -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (10) -: : : : : : : +- CometBroadcastExchange (19) -: : : : : : : +- CometProject (18) -: : : : : : : +- CometFilter (17) -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (16) -: : : : : : +- BroadcastExchange (45) -: : : : : : +- * CometColumnarToRow (44) -: : : : : : +- CometHashAggregate (43) -: : : : : : +- CometExchange (42) -: : : : : : +- CometHashAggregate (41) -: : : : : : +- CometProject (40) -: : : : : : +- CometBroadcastHashJoin (39) -: : : : : : :- CometProject (37) -: : : : : : : +- CometBroadcastHashJoin (36) -: : : : : : : :- CometProject (31) -: : : : : : : : +- CometBroadcastHashJoin (30) -: : : : : : : : :- CometProject (28) -: : : : : : : : : +- CometFilter (27) -: : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (26) -: : : : : : : : +- ReusedExchange (29) -: : : : : : : +- CometBroadcastExchange (35) -: : : : : : : +- CometProject (34) -: : : : : : : +- CometFilter (33) -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (32) -: : : : : : +- ReusedExchange (38) -: : : : : +- BroadcastExchange (66) -: : : : : +- * CometColumnarToRow (65) -: : : : : +- CometHashAggregate (64) -: : : : : +- CometExchange (63) -: : : : : +- CometHashAggregate (62) -: : : : : +- CometProject (61) -: : : : : +- CometBroadcastHashJoin (60) -: : : : : :- CometProject (58) -: : : : : : +- CometBroadcastHashJoin (57) -: : : : : : :- CometProject (52) -: : : : : : : +- CometBroadcastHashJoin (51) -: : : : : : : :- CometProject (49) -: : : : : : : : +- CometFilter (48) -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (47) -: : : : : : : +- ReusedExchange (50) -: : : : : : +- CometBroadcastExchange (56) -: : : : : : +- CometProject (55) -: : : : : : +- CometFilter (54) -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (53) -: : : : : +- ReusedExchange (59) -: : : : +- BroadcastExchange (87) -: : : : +- * CometColumnarToRow (86) -: : : : +- CometHashAggregate (85) -: : : : +- CometExchange (84) -: : : : +- CometHashAggregate (83) -: : : : +- CometProject (82) -: : : : +- CometBroadcastHashJoin (81) -: : : : :- CometProject (79) -: : : : : +- CometBroadcastHashJoin (78) -: : : : : :- CometProject (73) -: : : : : : +- CometBroadcastHashJoin (72) -: : : : : : :- CometProject (70) -: : : : : : : +- CometFilter (69) -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (68) -: : : : : : +- ReusedExchange (71) -: : : : : +- CometBroadcastExchange (77) -: : : : : +- CometProject (76) -: : : : : +- CometFilter (75) -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (74) -: : : : +- ReusedExchange (80) -: : : +- BroadcastExchange (108) -: : : +- * CometColumnarToRow (107) -: : : +- CometHashAggregate (106) -: : : +- CometExchange (105) -: : : +- CometHashAggregate (104) -: : : +- CometProject (103) -: : : +- CometBroadcastHashJoin (102) -: : : :- CometProject (100) -: : : : +- CometBroadcastHashJoin (99) -: : : : :- CometProject (94) -: : : : : +- CometBroadcastHashJoin (93) -: : : : : :- CometProject (91) -: : : : : : +- CometFilter (90) -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (89) -: : : : : +- ReusedExchange (92) -: : : : +- CometBroadcastExchange (98) -: : : : +- CometProject (97) -: : : : +- CometFilter (96) -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (95) -: : : +- ReusedExchange (101) -: : +- BroadcastExchange (129) -: : +- * CometColumnarToRow (128) -: : +- CometHashAggregate (127) -: : +- CometExchange (126) -: : +- CometHashAggregate (125) -: : +- CometProject (124) -: : +- CometBroadcastHashJoin (123) -: : :- CometProject (121) -: : : +- CometBroadcastHashJoin (120) -: : : :- CometProject (115) -: : : : +- CometBroadcastHashJoin (114) -: : : : :- CometProject (112) -: : : : : +- CometFilter (111) -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (110) -: : : : +- ReusedExchange (113) -: : : +- CometBroadcastExchange (119) -: : : +- CometProject (118) -: : : +- CometFilter (117) -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (116) -: : +- ReusedExchange (122) -: +- BroadcastExchange (150) -: +- * CometColumnarToRow (149) -: +- CometHashAggregate (148) -: +- CometExchange (147) -: +- CometHashAggregate (146) -: +- CometProject (145) -: +- CometBroadcastHashJoin (144) -: :- CometProject (142) -: : +- CometBroadcastHashJoin (141) -: : :- CometProject (136) -: : : +- CometBroadcastHashJoin (135) -: : : :- CometProject (133) -: : : : +- CometFilter (132) -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (131) -: : : +- ReusedExchange (134) -: : +- CometBroadcastExchange (140) -: : +- CometProject (139) -: : +- CometFilter (138) -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (137) -: +- ReusedExchange (143) -+- BroadcastExchange (171) - +- * CometColumnarToRow (170) - +- CometHashAggregate (169) - +- CometExchange (168) - +- CometHashAggregate (167) - +- CometProject (166) - +- CometBroadcastHashJoin (165) - :- CometProject (163) - : +- CometBroadcastHashJoin (162) - : :- CometProject (157) - : : +- CometBroadcastHashJoin (156) - : : :- CometProject (154) - : : : +- CometFilter (153) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (152) - : : +- ReusedExchange (155) - : +- CometBroadcastExchange (161) - : +- CometProject (160) - : +- CometFilter (159) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (158) - +- ReusedExchange (164) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(3) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [Or(Or(And(EqualTo(hd_dep_count,4),LessThanOrEqual(hd_vehicle_count,6)),And(EqualTo(hd_dep_count,2),LessThanOrEqual(hd_vehicle_count,4))),And(EqualTo(hd_dep_count,0),LessThanOrEqual(hd_vehicle_count,2))), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Condition : (((((hd_dep_count#6 = 4) AND (hd_vehicle_count#7 <= 6)) OR ((hd_dep_count#6 = 2) AND (hd_vehicle_count#7 <= 4))) OR ((hd_dep_count#6 = 0) AND (hd_vehicle_count#7 <= 2))) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] -Arguments: [ss_sold_time_sk#1, ss_store_sk#3], [ss_sold_time_sk#1, ss_store_sk#3] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,8), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Condition : ((((isnotnull(t_hour#9) AND isnotnull(t_minute#10)) AND (t_hour#9 = 8)) AND (t_minute#10 >= 30)) AND isnotnull(t_time_sk#8)) - -(12) CometProject -Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] -Arguments: [t_time_sk#8], [t_time_sk#8] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#8] -Arguments: [t_time_sk#8] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Right output [1]: [t_time_sk#8] -Arguments: [ss_sold_time_sk#1], [t_time_sk#8], Inner, BuildRight - -(15) CometProject -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#8] -Arguments: [ss_store_sk#3], [ss_store_sk#3] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_store_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [s_store_sk#11, s_store_name#12] -Condition : ((isnotnull(s_store_name#12) AND (s_store_name#12 = ese)) AND isnotnull(s_store_sk#11)) - -(18) CometProject -Input [2]: [s_store_sk#11, s_store_name#12] -Arguments: [s_store_sk#11], [s_store_sk#11] - -(19) CometBroadcastExchange -Input [1]: [s_store_sk#11] -Arguments: [s_store_sk#11] - -(20) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#3] -Right output [1]: [s_store_sk#11] -Arguments: [ss_store_sk#3], [s_store_sk#11], Inner, BuildRight - -(21) CometProject -Input [2]: [ss_store_sk#3, s_store_sk#11] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#13] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#13] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 8] -Input [1]: [h8_30_to_9#14] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Condition : ((isnotnull(ss_hdemo_sk#16) AND isnotnull(ss_sold_time_sk#15)) AND isnotnull(ss_store_sk#17)) - -(28) CometProject -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, ss_sold_date_sk#18] -Arguments: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17], [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] - -(29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#19] - -(30) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17] -Right output [1]: [hd_demo_sk#19] -Arguments: [ss_hdemo_sk#16], [hd_demo_sk#19], Inner, BuildRight - -(31) CometProject -Input [4]: [ss_sold_time_sk#15, ss_hdemo_sk#16, ss_store_sk#17, hd_demo_sk#19] -Arguments: [ss_sold_time_sk#15, ss_store_sk#17], [ss_sold_time_sk#15, ss_store_sk#17] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(33) CometFilter -Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Condition : ((((isnotnull(t_hour#21) AND isnotnull(t_minute#22)) AND (t_hour#21 = 9)) AND (t_minute#22 < 30)) AND isnotnull(t_time_sk#20)) - -(34) CometProject -Input [3]: [t_time_sk#20, t_hour#21, t_minute#22] -Arguments: [t_time_sk#20], [t_time_sk#20] - -(35) CometBroadcastExchange -Input [1]: [t_time_sk#20] -Arguments: [t_time_sk#20] - -(36) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#15, ss_store_sk#17] -Right output [1]: [t_time_sk#20] -Arguments: [ss_sold_time_sk#15], [t_time_sk#20], Inner, BuildRight - -(37) CometProject -Input [3]: [ss_sold_time_sk#15, ss_store_sk#17, t_time_sk#20] -Arguments: [ss_store_sk#17], [ss_store_sk#17] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#23] - -(39) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#17] -Right output [1]: [s_store_sk#23] -Arguments: [ss_store_sk#17], [s_store_sk#23], Inner, BuildRight - -(40) CometProject -Input [2]: [ss_store_sk#17, s_store_sk#23] - -(41) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(42) CometExchange -Input [1]: [count#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(43) CometHashAggregate -Input [1]: [count#24] -Keys: [] -Functions [1]: [count(1)] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [h9_to_9_30#25] - -(45) BroadcastExchange -Input [1]: [h9_to_9_30#25] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(46) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(48) CometFilter -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Condition : ((isnotnull(ss_hdemo_sk#27) AND isnotnull(ss_sold_time_sk#26)) AND isnotnull(ss_store_sk#28)) - -(49) CometProject -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, ss_sold_date_sk#29] -Arguments: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28], [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] - -(50) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#30] - -(51) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28] -Right output [1]: [hd_demo_sk#30] -Arguments: [ss_hdemo_sk#27], [hd_demo_sk#30], Inner, BuildRight - -(52) CometProject -Input [4]: [ss_sold_time_sk#26, ss_hdemo_sk#27, ss_store_sk#28, hd_demo_sk#30] -Arguments: [ss_sold_time_sk#26, ss_store_sk#28], [ss_sold_time_sk#26, ss_store_sk#28] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Condition : ((((isnotnull(t_hour#32) AND isnotnull(t_minute#33)) AND (t_hour#32 = 9)) AND (t_minute#33 >= 30)) AND isnotnull(t_time_sk#31)) - -(55) CometProject -Input [3]: [t_time_sk#31, t_hour#32, t_minute#33] -Arguments: [t_time_sk#31], [t_time_sk#31] - -(56) CometBroadcastExchange -Input [1]: [t_time_sk#31] -Arguments: [t_time_sk#31] - -(57) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#26, ss_store_sk#28] -Right output [1]: [t_time_sk#31] -Arguments: [ss_sold_time_sk#26], [t_time_sk#31], Inner, BuildRight - -(58) CometProject -Input [3]: [ss_sold_time_sk#26, ss_store_sk#28, t_time_sk#31] -Arguments: [ss_store_sk#28], [ss_store_sk#28] - -(59) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#34] - -(60) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#28] -Right output [1]: [s_store_sk#34] -Arguments: [ss_store_sk#28], [s_store_sk#34], Inner, BuildRight - -(61) CometProject -Input [2]: [ss_store_sk#28, s_store_sk#34] - -(62) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(63) CometExchange -Input [1]: [count#35] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(64) CometHashAggregate -Input [1]: [count#35] -Keys: [] -Functions [1]: [count(1)] - -(65) CometColumnarToRow [codegen id : 2] -Input [1]: [h9_30_to_10#36] - -(66) BroadcastExchange -Input [1]: [h9_30_to_10#36] -Arguments: IdentityBroadcastMode, [plan_id=5] - -(67) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(69) CometFilter -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Condition : ((isnotnull(ss_hdemo_sk#38) AND isnotnull(ss_sold_time_sk#37)) AND isnotnull(ss_store_sk#39)) - -(70) CometProject -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Arguments: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39], [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] - -(71) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#41] - -(72) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] -Right output [1]: [hd_demo_sk#41] -Arguments: [ss_hdemo_sk#38], [hd_demo_sk#41], Inner, BuildRight - -(73) CometProject -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, hd_demo_sk#41] -Arguments: [ss_sold_time_sk#37, ss_store_sk#39], [ss_sold_time_sk#37, ss_store_sk#39] - -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(75) CometFilter -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Condition : ((((isnotnull(t_hour#43) AND isnotnull(t_minute#44)) AND (t_hour#43 = 10)) AND (t_minute#44 < 30)) AND isnotnull(t_time_sk#42)) - -(76) CometProject -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Arguments: [t_time_sk#42], [t_time_sk#42] - -(77) CometBroadcastExchange -Input [1]: [t_time_sk#42] -Arguments: [t_time_sk#42] - -(78) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#37, ss_store_sk#39] -Right output [1]: [t_time_sk#42] -Arguments: [ss_sold_time_sk#37], [t_time_sk#42], Inner, BuildRight - -(79) CometProject -Input [3]: [ss_sold_time_sk#37, ss_store_sk#39, t_time_sk#42] -Arguments: [ss_store_sk#39], [ss_store_sk#39] - -(80) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#45] - -(81) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#39] -Right output [1]: [s_store_sk#45] -Arguments: [ss_store_sk#39], [s_store_sk#45], Inner, BuildRight - -(82) CometProject -Input [2]: [ss_store_sk#39, s_store_sk#45] - -(83) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(84) CometExchange -Input [1]: [count#46] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(85) CometHashAggregate -Input [1]: [count#46] -Keys: [] -Functions [1]: [count(1)] - -(86) CometColumnarToRow [codegen id : 3] -Input [1]: [h10_to_10_30#47] - -(87) BroadcastExchange -Input [1]: [h10_to_10_30#47] -Arguments: IdentityBroadcastMode, [plan_id=7] - -(88) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(89) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(90) CometFilter -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Condition : ((isnotnull(ss_hdemo_sk#49) AND isnotnull(ss_sold_time_sk#48)) AND isnotnull(ss_store_sk#50)) - -(91) CometProject -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, ss_sold_date_sk#51] -Arguments: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50], [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] - -(92) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#52] - -(93) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50] -Right output [1]: [hd_demo_sk#52] -Arguments: [ss_hdemo_sk#49], [hd_demo_sk#52], Inner, BuildRight - -(94) CometProject -Input [4]: [ss_sold_time_sk#48, ss_hdemo_sk#49, ss_store_sk#50, hd_demo_sk#52] -Arguments: [ss_sold_time_sk#48, ss_store_sk#50], [ss_sold_time_sk#48, ss_store_sk#50] - -(95) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(96) CometFilter -Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Condition : ((((isnotnull(t_hour#54) AND isnotnull(t_minute#55)) AND (t_hour#54 = 10)) AND (t_minute#55 >= 30)) AND isnotnull(t_time_sk#53)) - -(97) CometProject -Input [3]: [t_time_sk#53, t_hour#54, t_minute#55] -Arguments: [t_time_sk#53], [t_time_sk#53] - -(98) CometBroadcastExchange -Input [1]: [t_time_sk#53] -Arguments: [t_time_sk#53] - -(99) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#48, ss_store_sk#50] -Right output [1]: [t_time_sk#53] -Arguments: [ss_sold_time_sk#48], [t_time_sk#53], Inner, BuildRight - -(100) CometProject -Input [3]: [ss_sold_time_sk#48, ss_store_sk#50, t_time_sk#53] -Arguments: [ss_store_sk#50], [ss_store_sk#50] - -(101) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#56] - -(102) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#50] -Right output [1]: [s_store_sk#56] -Arguments: [ss_store_sk#50], [s_store_sk#56], Inner, BuildRight - -(103) CometProject -Input [2]: [ss_store_sk#50, s_store_sk#56] - -(104) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(105) CometExchange -Input [1]: [count#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(106) CometHashAggregate -Input [1]: [count#57] -Keys: [] -Functions [1]: [count(1)] - -(107) CometColumnarToRow [codegen id : 4] -Input [1]: [h10_30_to_11#58] - -(108) BroadcastExchange -Input [1]: [h10_30_to_11#58] -Arguments: IdentityBroadcastMode, [plan_id=9] - -(109) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(110) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(111) CometFilter -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Condition : ((isnotnull(ss_hdemo_sk#60) AND isnotnull(ss_sold_time_sk#59)) AND isnotnull(ss_store_sk#61)) - -(112) CometProject -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, ss_sold_date_sk#62] -Arguments: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61], [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] - -(113) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#63] - -(114) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61] -Right output [1]: [hd_demo_sk#63] -Arguments: [ss_hdemo_sk#60], [hd_demo_sk#63], Inner, BuildRight - -(115) CometProject -Input [4]: [ss_sold_time_sk#59, ss_hdemo_sk#60, ss_store_sk#61, hd_demo_sk#63] -Arguments: [ss_sold_time_sk#59, ss_store_sk#61], [ss_sold_time_sk#59, ss_store_sk#61] - -(116) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(117) CometFilter -Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Condition : ((((isnotnull(t_hour#65) AND isnotnull(t_minute#66)) AND (t_hour#65 = 11)) AND (t_minute#66 < 30)) AND isnotnull(t_time_sk#64)) - -(118) CometProject -Input [3]: [t_time_sk#64, t_hour#65, t_minute#66] -Arguments: [t_time_sk#64], [t_time_sk#64] - -(119) CometBroadcastExchange -Input [1]: [t_time_sk#64] -Arguments: [t_time_sk#64] - -(120) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#59, ss_store_sk#61] -Right output [1]: [t_time_sk#64] -Arguments: [ss_sold_time_sk#59], [t_time_sk#64], Inner, BuildRight - -(121) CometProject -Input [3]: [ss_sold_time_sk#59, ss_store_sk#61, t_time_sk#64] -Arguments: [ss_store_sk#61], [ss_store_sk#61] - -(122) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#67] - -(123) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#61] -Right output [1]: [s_store_sk#67] -Arguments: [ss_store_sk#61], [s_store_sk#67], Inner, BuildRight - -(124) CometProject -Input [2]: [ss_store_sk#61, s_store_sk#67] - -(125) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(126) CometExchange -Input [1]: [count#68] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(127) CometHashAggregate -Input [1]: [count#68] -Keys: [] -Functions [1]: [count(1)] - -(128) CometColumnarToRow [codegen id : 5] -Input [1]: [h11_to_11_30#69] - -(129) BroadcastExchange -Input [1]: [h11_to_11_30#69] -Arguments: IdentityBroadcastMode, [plan_id=11] - -(130) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(131) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(132) CometFilter -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Condition : ((isnotnull(ss_hdemo_sk#71) AND isnotnull(ss_sold_time_sk#70)) AND isnotnull(ss_store_sk#72)) - -(133) CometProject -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, ss_sold_date_sk#73] -Arguments: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72], [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] - -(134) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#74] - -(135) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72] -Right output [1]: [hd_demo_sk#74] -Arguments: [ss_hdemo_sk#71], [hd_demo_sk#74], Inner, BuildRight - -(136) CometProject -Input [4]: [ss_sold_time_sk#70, ss_hdemo_sk#71, ss_store_sk#72, hd_demo_sk#74] -Arguments: [ss_sold_time_sk#70, ss_store_sk#72], [ss_sold_time_sk#70, ss_store_sk#72] - -(137) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(138) CometFilter -Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Condition : ((((isnotnull(t_hour#76) AND isnotnull(t_minute#77)) AND (t_hour#76 = 11)) AND (t_minute#77 >= 30)) AND isnotnull(t_time_sk#75)) - -(139) CometProject -Input [3]: [t_time_sk#75, t_hour#76, t_minute#77] -Arguments: [t_time_sk#75], [t_time_sk#75] - -(140) CometBroadcastExchange -Input [1]: [t_time_sk#75] -Arguments: [t_time_sk#75] - -(141) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#70, ss_store_sk#72] -Right output [1]: [t_time_sk#75] -Arguments: [ss_sold_time_sk#70], [t_time_sk#75], Inner, BuildRight - -(142) CometProject -Input [3]: [ss_sold_time_sk#70, ss_store_sk#72, t_time_sk#75] -Arguments: [ss_store_sk#72], [ss_store_sk#72] - -(143) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#78] - -(144) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#72] -Right output [1]: [s_store_sk#78] -Arguments: [ss_store_sk#72], [s_store_sk#78], Inner, BuildRight - -(145) CometProject -Input [2]: [ss_store_sk#72, s_store_sk#78] - -(146) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(147) CometExchange -Input [1]: [count#79] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(148) CometHashAggregate -Input [1]: [count#79] -Keys: [] -Functions [1]: [count(1)] - -(149) CometColumnarToRow [codegen id : 6] -Input [1]: [h11_30_to_12#80] - -(150) BroadcastExchange -Input [1]: [h11_30_to_12#80] -Arguments: IdentityBroadcastMode, [plan_id=13] - -(151) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - -(152) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(153) CometFilter -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Condition : ((isnotnull(ss_hdemo_sk#82) AND isnotnull(ss_sold_time_sk#81)) AND isnotnull(ss_store_sk#83)) - -(154) CometProject -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, ss_sold_date_sk#84] -Arguments: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83], [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] - -(155) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#85] - -(156) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83] -Right output [1]: [hd_demo_sk#85] -Arguments: [ss_hdemo_sk#82], [hd_demo_sk#85], Inner, BuildRight - -(157) CometProject -Input [4]: [ss_sold_time_sk#81, ss_hdemo_sk#82, ss_store_sk#83, hd_demo_sk#85] -Arguments: [ss_sold_time_sk#81, ss_store_sk#83], [ss_sold_time_sk#81, ss_store_sk#83] - -(158) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(159) CometFilter -Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Condition : ((((isnotnull(t_hour#87) AND isnotnull(t_minute#88)) AND (t_hour#87 = 12)) AND (t_minute#88 < 30)) AND isnotnull(t_time_sk#86)) - -(160) CometProject -Input [3]: [t_time_sk#86, t_hour#87, t_minute#88] -Arguments: [t_time_sk#86], [t_time_sk#86] - -(161) CometBroadcastExchange -Input [1]: [t_time_sk#86] -Arguments: [t_time_sk#86] - -(162) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#81, ss_store_sk#83] -Right output [1]: [t_time_sk#86] -Arguments: [ss_sold_time_sk#81], [t_time_sk#86], Inner, BuildRight - -(163) CometProject -Input [3]: [ss_sold_time_sk#81, ss_store_sk#83, t_time_sk#86] -Arguments: [ss_store_sk#83], [ss_store_sk#83] - -(164) ReusedExchange [Reuses operator id: 19] -Output [1]: [s_store_sk#89] - -(165) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#83] -Right output [1]: [s_store_sk#89] -Arguments: [ss_store_sk#83], [s_store_sk#89], Inner, BuildRight - -(166) CometProject -Input [2]: [ss_store_sk#83, s_store_sk#89] - -(167) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(168) CometExchange -Input [1]: [count#90] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(169) CometHashAggregate -Input [1]: [count#90] -Keys: [] -Functions [1]: [count(1)] - -(170) CometColumnarToRow [codegen id : 7] -Input [1]: [h12_to_12_30#91] - -(171) BroadcastExchange -Input [1]: [h12_to_12_30#91] -Arguments: IdentityBroadcastMode, [plan_id=15] - -(172) BroadcastNestedLoopJoin [codegen id : 8] -Join type: Inner -Join condition: None - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/extended.txt deleted file mode 100644 index 1e1247665c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/extended.txt +++ /dev/null @@ -1,216 +0,0 @@ -BroadcastNestedLoopJoin -:- BroadcastNestedLoopJoin -: :- BroadcastNestedLoopJoin -: : :- BroadcastNestedLoopJoin -: : : :- BroadcastNestedLoopJoin -: : : : :- BroadcastNestedLoopJoin -: : : : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] -: : : : : : :- CometColumnarToRow -: : : : : : : +- CometHashAggregate -: : : : : : : +- CometExchange -: : : : : : : +- CometHashAggregate -: : : : : : : +- CometProject -: : : : : : : +- CometBroadcastHashJoin -: : : : : : : :- CometProject -: : : : : : : : +- CometBroadcastHashJoin -: : : : : : : : :- CometProject -: : : : : : : : : +- CometBroadcastHashJoin -: : : : : : : : : :- CometProject -: : : : : : : : : : +- CometFilter -: : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : : : : : +- CometBroadcastExchange -: : : : : : : : : +- CometProject -: : : : : : : : : +- CometFilter -: : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : : : : : +- CometBroadcastExchange -: : : : : : : : +- CometProject -: : : : : : : : +- CometFilter -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : : : : : +- CometBroadcastExchange -: : : : : : : +- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : : : : +- BroadcastExchange -: : : : : : +- CometColumnarToRow -: : : : : : +- CometHashAggregate -: : : : : : +- CometExchange -: : : : : : +- CometHashAggregate -: : : : : : +- CometProject -: : : : : : +- CometBroadcastHashJoin -: : : : : : :- CometProject -: : : : : : : +- CometBroadcastHashJoin -: : : : : : : :- CometProject -: : : : : : : : +- CometBroadcastHashJoin -: : : : : : : : :- CometProject -: : : : : : : : : +- CometFilter -: : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : : : : +- CometBroadcastExchange -: : : : : : : : +- CometProject -: : : : : : : : +- CometFilter -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : : : : +- CometBroadcastExchange -: : : : : : : +- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : : : : +- CometBroadcastExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : : : +- BroadcastExchange -: : : : : +- CometColumnarToRow -: : : : : +- CometHashAggregate -: : : : : +- CometExchange -: : : : : +- CometHashAggregate -: : : : : +- CometProject -: : : : : +- CometBroadcastHashJoin -: : : : : :- CometProject -: : : : : : +- CometBroadcastHashJoin -: : : : : : :- CometProject -: : : : : : : +- CometBroadcastHashJoin -: : : : : : : :- CometProject -: : : : : : : : +- CometFilter -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : : : +- CometBroadcastExchange -: : : : : : : +- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : : : +- CometBroadcastExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : : : +- CometBroadcastExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : : +- BroadcastExchange -: : : : +- CometColumnarToRow -: : : : +- CometHashAggregate -: : : : +- CometExchange -: : : : +- CometHashAggregate -: : : : +- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometBroadcastHashJoin -: : : : : :- CometProject -: : : : : : +- CometBroadcastHashJoin -: : : : : : :- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : : +- CometBroadcastExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : : +- CometBroadcastExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : +- BroadcastExchange -: : : +- CometColumnarToRow -: : : +- CometHashAggregate -: : : +- CometExchange -: : : +- CometHashAggregate -: : : +- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometBroadcastHashJoin -: : : : : :- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : +- CometBroadcastExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : +- BroadcastExchange -: : +- CometColumnarToRow -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: +- BroadcastExchange -: +- CometColumnarToRow -: +- CometHashAggregate -: +- CometExchange -: +- CometHashAggregate -: +- CometProject -: +- CometBroadcastHashJoin -: :- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: +- CometBroadcastExchange -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -+- BroadcastExchange - +- CometColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 192 out of 206 eligible operators (93%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index e3923fb3ea..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/simplified.txt +++ /dev/null @@ -1,195 +0,0 @@ -WholeStageCodegen (8) - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h8_30_to_9,count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_to_9_30,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h9_30_to_10,count(1)] - CometExchange #9 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #10 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_to_10_30,count(1)] - CometExchange #12 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #13 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h10_30_to_11,count(1)] - CometExchange #15 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #16 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #17 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_to_11_30,count(1)] - CometExchange #18 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #19 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #20 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h11_30_to_12,count(1)] - CometExchange #21 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #22 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - ReusedExchange [s_store_sk] #4 - InputAdapter - BroadcastExchange #23 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [h12_to_12_30,count(1)] - CometExchange #24 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #25 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/explain.txt deleted file mode 100644 index 56583c6d26..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/explain.txt +++ /dev/null @@ -1,202 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * CometColumnarToRow (26) - +- CometSort (25) - +- CometColumnarExchange (24) - +- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometColumnarExchange (21) - +- * HashAggregate (20) - +- * Project (19) - +- * BroadcastHashJoin Inner BuildRight (18) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * CometColumnarToRow (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : +- BroadcastExchange (8) - : : +- * Filter (7) - : : +- * ColumnarToRow (6) - : : +- Scan parquet spark_catalog.default.store_sales (5) - : +- ReusedExchange (11) - +- BroadcastExchange (17) - +- * CometColumnarToRow (16) - +- CometFilter (15) - +- CometNativeScan parquet spark_catalog.default.store (14) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Books ,Electronics ,Sports ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (computers ,stereo ,football )) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Men ,Jewelry ,Women ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (shirts ,birdal ,dresses ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Arguments: [i_item_sk#1, i_brand#5, i_class#6, i_category#7], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) AS i_class#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) AS i_category#7] - -(4) CometColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7] - -(5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] - -(7) Filter [codegen id : 1] -Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Condition : (isnotnull(ss_item_sk#8) AND isnotnull(ss_store_sk#9)) - -(8) BroadcastExchange -Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#8] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Input [8]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7, ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] - -(11) ReusedExchange [Reuses operator id: 35] -Output [2]: [d_date_sk#13, d_moy#14] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#14] -Input [8]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11, d_date_sk#13, d_moy#14] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#15, s_store_name#16, s_company_name#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [s_store_sk#15, s_store_name#16, s_company_name#17] -Condition : isnotnull(s_store_sk#15) - -(16) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#15, s_store_name#16, s_company_name#17] - -(17) BroadcastExchange -Input [3]: [s_store_sk#15, s_store_name#16, s_company_name#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#9] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [7]: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#14, s_store_name#16, s_company_name#17] -Input [9]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#14, s_store_sk#15, s_store_name#16, s_company_name#17] - -(20) HashAggregate [codegen id : 4] -Input [7]: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#14, s_store_name#16, s_company_name#17] -Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#10))] -Aggregate Attributes [1]: [sum#18] -Results [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum#19] - -(21) CometColumnarExchange -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum#19] -Arguments: hashpartitioning(i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum#19] - -(23) HashAggregate [codegen id : 5] -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum#19] -Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14] -Functions [1]: [sum(UnscaledValue(ss_sales_price#10))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#10))#20] -Results [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#20,17,2) AS _w0#22] - -(24) CometColumnarExchange -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#16, s_company_name#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22] -Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#16 ASC NULLS FIRST, s_company_name#17 ASC NULLS FIRST] - -(26) CometColumnarToRow [codegen id : 6] -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22] - -(27) Window -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_category#7, i_brand#5, s_store_name#16, s_company_name#17, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#7, i_brand#5, s_store_name#16, s_company_name#17] - -(28) Filter [codegen id : 7] -Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22, avg_monthly_sales#23] -Condition : CASE WHEN NOT (avg_monthly_sales#23 = 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END - -(29) Project [codegen id : 7] -Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, avg_monthly_sales#23] -Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, _w0#22, avg_monthly_sales#23] - -(30) TakeOrderedAndProject -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, avg_monthly_sales#23] -Arguments: 100, [(sum_sales#21 - avg_monthly_sales#23) ASC NULLS FIRST, s_store_name#16 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#16, s_company_name#17, d_moy#14, sum_sales#21, avg_monthly_sales#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometNativeScan parquet spark_catalog.default.date_dim (31) - - -(31) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#24, d_moy#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [d_date_sk#13, d_year#24, d_moy#14] -Condition : ((isnotnull(d_year#24) AND (d_year#24 = 1999)) AND isnotnull(d_date_sk#13)) - -(33) CometProject -Input [3]: [d_date_sk#13, d_year#24, d_moy#14] -Arguments: [d_date_sk#13, d_moy#14], [d_date_sk#13, d_moy#14] - -(34) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_moy#14] - -(35) BroadcastExchange -Input [2]: [d_date_sk#13, d_moy#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/simplified.txt deleted file mode 100644 index f87ef33db4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/simplified.txt +++ /dev/null @@ -1,53 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (7) - Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometNativeScan 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/q89.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/explain.txt deleted file mode 100644 index 47ce0d4a91..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,198 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) - +- CometSort (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Books ,Electronics ,Sports ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (computers ,stereo ,football )) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Men ,Jewelry ,Women ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (shirts ,birdal ,dresses ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Arguments: [i_item_sk#1, i_brand#5, i_class#6, i_category#7], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) AS i_class#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) AS i_category#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Condition : (isnotnull(ss_item_sk#8) AND isnotnull(ss_store_sk#9)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Arguments: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] - -(7) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7] -Right output [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Arguments: [i_item_sk#1], [ss_item_sk#8], Inner, BuildRight - -(8) CometProject -Input [8]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7, ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Arguments: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11], [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 1999)) AND isnotnull(d_date_sk#13)) - -(11) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13, d_moy#15], [d_date_sk#13, d_moy#15] - -(12) CometBroadcastExchange -Input [2]: [d_date_sk#13, d_moy#15] -Arguments: [d_date_sk#13, d_moy#15] - -(13) CometBroadcastHashJoin -Left output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Right output [2]: [d_date_sk#13, d_moy#15] -Arguments: [ss_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(14) CometProject -Input [8]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11, d_date_sk#13, d_moy#15] -Arguments: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15], [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Condition : isnotnull(s_store_sk#16) - -(17) CometBroadcastExchange -Input [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Arguments: [s_store_sk#16, s_store_name#17, s_company_name#18] - -(18) CometBroadcastHashJoin -Left output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15] -Right output [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Arguments: [ss_store_sk#9], [s_store_sk#16], Inner, BuildRight - -(19) CometProject -Input [9]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15, s_store_sk#16, s_store_name#17, s_company_name#18] -Arguments: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#15, s_store_name#17, s_company_name#18], [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#15, s_store_name#17, s_company_name#18] - -(20) CometHashAggregate -Input [7]: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#15, s_store_name#17, s_company_name#18] -Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#10))] - -(21) CometExchange -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum#19] -Arguments: hashpartitioning(i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum#19] -Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15] -Functions [1]: [sum(UnscaledValue(ss_sales_price#10))] - -(23) CometExchange -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST, s_company_name#18 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 1] -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] - -(26) Window -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] - -(27) Filter [codegen id : 2] -Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] -Condition : CASE WHEN NOT (avg_monthly_sales#22 = 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END - -(28) Project [codegen id : 2] -Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] -Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] - -(29) TakeOrderedAndProject -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] -Arguments: 100, [(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 1999)) AND isnotnull(d_date_sk#13)) - -(32) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13, d_moy#15], [d_date_sk#13, d_moy#15] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_moy#15] - -(34) BroadcastExchange -Input [2]: [d_date_sk#13, d_moy#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/simplified.txt deleted file mode 100644 index 61bfd1d960..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (2) - Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] - CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - 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 [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,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] 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/q89/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt deleted file mode 100644 index 47ce0d4a91..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt +++ /dev/null @@ -1,198 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) - +- CometSort (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : +- CometBroadcastExchange (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Books ,Electronics ,Sports ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (computers ,stereo ,football )) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) IN (Men ,Jewelry ,Women ) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) IN (shirts ,birdal ,dresses ))) AND isnotnull(i_item_sk#1)) - -(3) CometProject -Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Arguments: [i_item_sk#1, i_brand#5, i_class#6, i_category#7], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#3, 50, true, false, true) AS i_class#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#4, 50, true, false, true) AS i_category#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Condition : (isnotnull(ss_item_sk#8) AND isnotnull(ss_store_sk#9)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Arguments: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] - -(7) CometBroadcastHashJoin -Left output [4]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7] -Right output [4]: [ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Arguments: [i_item_sk#1], [ss_item_sk#8], Inner, BuildRight - -(8) CometProject -Input [8]: [i_item_sk#1, i_brand#5, i_class#6, i_category#7, ss_item_sk#8, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Arguments: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11], [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 1999)) AND isnotnull(d_date_sk#13)) - -(11) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13, d_moy#15], [d_date_sk#13, d_moy#15] - -(12) CometBroadcastExchange -Input [2]: [d_date_sk#13, d_moy#15] -Arguments: [d_date_sk#13, d_moy#15] - -(13) CometBroadcastHashJoin -Left output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11] -Right output [2]: [d_date_sk#13, d_moy#15] -Arguments: [ss_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(14) CometProject -Input [8]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, ss_sold_date_sk#11, d_date_sk#13, d_moy#15] -Arguments: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15], [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Condition : isnotnull(s_store_sk#16) - -(17) CometBroadcastExchange -Input [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Arguments: [s_store_sk#16, s_store_name#17, s_company_name#18] - -(18) CometBroadcastHashJoin -Left output [6]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15] -Right output [3]: [s_store_sk#16, s_store_name#17, s_company_name#18] -Arguments: [ss_store_sk#9], [s_store_sk#16], Inner, BuildRight - -(19) CometProject -Input [9]: [i_brand#5, i_class#6, i_category#7, ss_store_sk#9, ss_sales_price#10, d_moy#15, s_store_sk#16, s_store_name#17, s_company_name#18] -Arguments: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#15, s_store_name#17, s_company_name#18], [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#15, s_store_name#17, s_company_name#18] - -(20) CometHashAggregate -Input [7]: [i_brand#5, i_class#6, i_category#7, ss_sales_price#10, d_moy#15, s_store_name#17, s_company_name#18] -Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#10))] - -(21) CometExchange -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum#19] -Arguments: hashpartitioning(i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum#19] -Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15] -Functions [1]: [sum(UnscaledValue(ss_sales_price#10))] - -(23) CometExchange -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(24) CometSort -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST, s_company_name#18 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 1] -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] - -(26) Window -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] - -(27) Filter [codegen id : 2] -Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] -Condition : CASE WHEN NOT (avg_monthly_sales#22 = 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END - -(28) Project [codegen id : 2] -Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] -Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] - -(29) TakeOrderedAndProject -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] -Arguments: 100, [(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (34) -+- * CometColumnarToRow (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 1999)) AND isnotnull(d_date_sk#13)) - -(32) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13, d_moy#15], [d_date_sk#13, d_moy#15] - -(33) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_moy#15] - -(34) BroadcastExchange -Input [2]: [d_date_sk#13, d_moy#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/extended.txt deleted file mode 100644 index 825b1ed81c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 61bfd1d960..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (2) - Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] - CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - 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 [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,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_datafusion/explain.txt deleted file mode 100644 index da194f2825..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_datafusion/explain.txt +++ /dev/null @@ -1,283 +0,0 @@ -== Physical Plan == -* Project (4) -+- * CometColumnarToRow (3) - +- CometFilter (2) - +- CometNativeScan parquet spark_catalog.default.reason (1) - - -(1) CometNativeScan parquet spark_catalog.default.reason -Output [1]: [r_reason_sk#1] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk), EqualTo(r_reason_sk,1)] -ReadSchema: struct - -(2) CometFilter -Input [1]: [r_reason_sk#1] -Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) - -(3) CometColumnarToRow [codegen id : 1] -Input [1]: [r_reason_sk#1] - -(4) Project [codegen id : 1] -Output [5]: [CASE WHEN (Subquery scalar-subquery#2, [id=#3].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket1#4, CASE WHEN (Subquery scalar-subquery#5, [id=#6].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_net_paid) END AS bucket2#7, CASE WHEN (Subquery scalar-subquery#8, [id=#9].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_net_paid) END AS bucket3#10, CASE WHEN (Subquery scalar-subquery#11, [id=#12].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_net_paid) END AS bucket4#13, CASE WHEN (Subquery scalar-subquery#14, [id=#15].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_net_paid) END AS bucket5#16] -Input [1]: [r_reason_sk#1] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] -* CometColumnarToRow (12) -+- CometProject (11) - +- CometHashAggregate (10) - +- CometExchange (9) - +- CometHashAggregate (8) - +- CometProject (7) - +- CometFilter (6) - +- CometNativeScan parquet spark_catalog.default.store_sales (5) - - -(5) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] -ReadSchema: struct - -(6) CometFilter -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Condition : ((isnotnull(ss_quantity#17) AND (ss_quantity#17 >= 1)) AND (ss_quantity#17 <= 20)) - -(7) CometProject -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Arguments: [ss_ext_discount_amt#18, ss_net_paid#19], [ss_ext_discount_amt#18, ss_net_paid#19] - -(8) CometHashAggregate -Input [2]: [ss_ext_discount_amt#18, ss_net_paid#19] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#18)), partial_avg(UnscaledValue(ss_net_paid#19))] - -(9) CometExchange -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometHashAggregate -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(UnscaledValue(ss_net_paid#19))] - -(11) CometProject -Input [3]: [count(1)#26, avg(ss_ext_discount_amt)#27, avg(ss_net_paid)#28] -Arguments: [mergedValue#29], [named_struct(count(1), count(1)#26, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#27, avg(ss_net_paid), avg(ss_net_paid)#28) AS mergedValue#29] - -(12) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#29] - -Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] - -Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] - -Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] -* CometColumnarToRow (20) -+- CometProject (19) - +- CometHashAggregate (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometProject (15) - +- CometFilter (14) - +- CometNativeScan parquet spark_catalog.default.store_sales (13) - - -(13) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Condition : ((isnotnull(ss_quantity#30) AND (ss_quantity#30 >= 21)) AND (ss_quantity#30 <= 40)) - -(15) CometProject -Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Arguments: [ss_ext_discount_amt#31, ss_net_paid#32], [ss_ext_discount_amt#31, ss_net_paid#32] - -(16) CometHashAggregate -Input [2]: [ss_ext_discount_amt#31, ss_net_paid#32] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#31)), partial_avg(UnscaledValue(ss_net_paid#32))] - -(17) CometExchange -Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(18) CometHashAggregate -Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#31)), avg(UnscaledValue(ss_net_paid#32))] - -(19) CometProject -Input [3]: [count(1)#39, avg(ss_ext_discount_amt)#40, avg(ss_net_paid)#41] -Arguments: [mergedValue#42], [named_struct(count(1), count(1)#39, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#40, avg(ss_net_paid), avg(ss_net_paid)#41) AS mergedValue#42] - -(20) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#42] - -Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] - -Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] - -Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* CometColumnarToRow (28) -+- CometProject (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometFilter (22) - +- CometNativeScan parquet spark_catalog.default.store_sales (21) - - -(21) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] -ReadSchema: struct - -(22) CometFilter -Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Condition : ((isnotnull(ss_quantity#43) AND (ss_quantity#43 >= 41)) AND (ss_quantity#43 <= 60)) - -(23) CometProject -Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Arguments: [ss_ext_discount_amt#44, ss_net_paid#45], [ss_ext_discount_amt#44, ss_net_paid#45] - -(24) CometHashAggregate -Input [2]: [ss_ext_discount_amt#44, ss_net_paid#45] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#44)), partial_avg(UnscaledValue(ss_net_paid#45))] - -(25) CometExchange -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(26) CometHashAggregate -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#44)), avg(UnscaledValue(ss_net_paid#45))] - -(27) CometProject -Input [3]: [count(1)#52, avg(ss_ext_discount_amt)#53, avg(ss_net_paid)#54] -Arguments: [mergedValue#55], [named_struct(count(1), count(1)#52, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#53, avg(ss_net_paid), avg(ss_net_paid)#54) AS mergedValue#55] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#55] - -Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] - -Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] - -Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* CometColumnarToRow (36) -+- CometProject (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometFilter (30) - +- CometNativeScan parquet spark_catalog.default.store_sales (29) - - -(29) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_quantity#56) AND (ss_quantity#56 >= 61)) AND (ss_quantity#56 <= 80)) - -(31) CometProject -Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Arguments: [ss_ext_discount_amt#57, ss_net_paid#58], [ss_ext_discount_amt#57, ss_net_paid#58] - -(32) CometHashAggregate -Input [2]: [ss_ext_discount_amt#57, ss_net_paid#58] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#57)), partial_avg(UnscaledValue(ss_net_paid#58))] - -(33) CometExchange -Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(34) CometHashAggregate -Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#57)), avg(UnscaledValue(ss_net_paid#58))] - -(35) CometProject -Input [3]: [count(1)#65, avg(ss_ext_discount_amt)#66, avg(ss_net_paid)#67] -Arguments: [mergedValue#68], [named_struct(count(1), count(1)#65, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#66, avg(ss_net_paid), avg(ss_net_paid)#67) AS mergedValue#68] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#68] - -Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] - -Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] - -Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* CometColumnarToRow (44) -+- CometProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometFilter (38) - +- CometNativeScan parquet spark_catalog.default.store_sales (37) - - -(37) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] -ReadSchema: struct - -(38) CometFilter -Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Condition : ((isnotnull(ss_quantity#69) AND (ss_quantity#69 >= 81)) AND (ss_quantity#69 <= 100)) - -(39) CometProject -Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Arguments: [ss_ext_discount_amt#70, ss_net_paid#71], [ss_ext_discount_amt#70, ss_net_paid#71] - -(40) CometHashAggregate -Input [2]: [ss_ext_discount_amt#70, ss_net_paid#71] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#70)), partial_avg(UnscaledValue(ss_net_paid#71))] - -(41) CometExchange -Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(42) CometHashAggregate -Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#70)), avg(UnscaledValue(ss_net_paid#71))] - -(43) CometProject -Input [3]: [count(1)#78, avg(ss_ext_discount_amt)#79, avg(ss_net_paid)#80] -Arguments: [mergedValue#81], [named_struct(count(1), count(1)#78, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#79, avg(ss_net_paid), avg(ss_net_paid)#80) AS mergedValue#81] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#81] - -Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] - -Subquery:15 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_datafusion/simplified.txt deleted file mode 100644 index 9593d6ebd1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_datafusion/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -WholeStageCodegen (1) - Project - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #1 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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 - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #2 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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 - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #3 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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 - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #4 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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 - Subquery #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #5 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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 - CometColumnarToRow - InputAdapter - CometFilter [r_reason_sk] - CometNativeScan parquet spark_catalog.default.reason [r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/explain.txt deleted file mode 100644 index b3f32555de..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,283 +0,0 @@ -== Physical Plan == -* Project (4) -+- * CometColumnarToRow (3) - +- CometFilter (2) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (1) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Output [1]: [r_reason_sk#1] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk), EqualTo(r_reason_sk,1)] -ReadSchema: struct - -(2) CometFilter -Input [1]: [r_reason_sk#1] -Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) - -(3) CometColumnarToRow [codegen id : 1] -Input [1]: [r_reason_sk#1] - -(4) Project [codegen id : 1] -Output [5]: [CASE WHEN (Subquery scalar-subquery#2, [id=#3].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket1#4, CASE WHEN (Subquery scalar-subquery#5, [id=#6].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_net_paid) END AS bucket2#7, CASE WHEN (Subquery scalar-subquery#8, [id=#9].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_net_paid) END AS bucket3#10, CASE WHEN (Subquery scalar-subquery#11, [id=#12].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_net_paid) END AS bucket4#13, CASE WHEN (Subquery scalar-subquery#14, [id=#15].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_net_paid) END AS bucket5#16] -Input [1]: [r_reason_sk#1] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] -* CometColumnarToRow (12) -+- CometProject (11) - +- CometHashAggregate (10) - +- CometExchange (9) - +- CometHashAggregate (8) - +- CometProject (7) - +- CometFilter (6) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] -ReadSchema: struct - -(6) CometFilter -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Condition : ((isnotnull(ss_quantity#17) AND (ss_quantity#17 >= 1)) AND (ss_quantity#17 <= 20)) - -(7) CometProject -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Arguments: [ss_ext_discount_amt#18, ss_net_paid#19], [ss_ext_discount_amt#18, ss_net_paid#19] - -(8) CometHashAggregate -Input [2]: [ss_ext_discount_amt#18, ss_net_paid#19] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#18)), partial_avg(UnscaledValue(ss_net_paid#19))] - -(9) CometExchange -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometHashAggregate -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(UnscaledValue(ss_net_paid#19))] - -(11) CometProject -Input [3]: [count(1)#26, avg(ss_ext_discount_amt)#27, avg(ss_net_paid)#28] -Arguments: [mergedValue#29], [named_struct(count(1), count(1)#26, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#27, avg(ss_net_paid), avg(ss_net_paid)#28) AS mergedValue#29] - -(12) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#29] - -Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] - -Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] - -Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] -* CometColumnarToRow (20) -+- CometProject (19) - +- CometHashAggregate (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometProject (15) - +- CometFilter (14) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Condition : ((isnotnull(ss_quantity#30) AND (ss_quantity#30 >= 21)) AND (ss_quantity#30 <= 40)) - -(15) CometProject -Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Arguments: [ss_ext_discount_amt#31, ss_net_paid#32], [ss_ext_discount_amt#31, ss_net_paid#32] - -(16) CometHashAggregate -Input [2]: [ss_ext_discount_amt#31, ss_net_paid#32] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#31)), partial_avg(UnscaledValue(ss_net_paid#32))] - -(17) CometExchange -Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(18) CometHashAggregate -Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#31)), avg(UnscaledValue(ss_net_paid#32))] - -(19) CometProject -Input [3]: [count(1)#39, avg(ss_ext_discount_amt)#40, avg(ss_net_paid)#41] -Arguments: [mergedValue#42], [named_struct(count(1), count(1)#39, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#40, avg(ss_net_paid), avg(ss_net_paid)#41) AS mergedValue#42] - -(20) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#42] - -Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] - -Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] - -Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* CometColumnarToRow (28) -+- CometProject (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] -ReadSchema: struct - -(22) CometFilter -Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Condition : ((isnotnull(ss_quantity#43) AND (ss_quantity#43 >= 41)) AND (ss_quantity#43 <= 60)) - -(23) CometProject -Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Arguments: [ss_ext_discount_amt#44, ss_net_paid#45], [ss_ext_discount_amt#44, ss_net_paid#45] - -(24) CometHashAggregate -Input [2]: [ss_ext_discount_amt#44, ss_net_paid#45] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#44)), partial_avg(UnscaledValue(ss_net_paid#45))] - -(25) CometExchange -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(26) CometHashAggregate -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#44)), avg(UnscaledValue(ss_net_paid#45))] - -(27) CometProject -Input [3]: [count(1)#52, avg(ss_ext_discount_amt)#53, avg(ss_net_paid)#54] -Arguments: [mergedValue#55], [named_struct(count(1), count(1)#52, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#53, avg(ss_net_paid), avg(ss_net_paid)#54) AS mergedValue#55] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#55] - -Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] - -Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] - -Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* CometColumnarToRow (36) -+- CometProject (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (29) - - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_quantity#56) AND (ss_quantity#56 >= 61)) AND (ss_quantity#56 <= 80)) - -(31) CometProject -Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Arguments: [ss_ext_discount_amt#57, ss_net_paid#58], [ss_ext_discount_amt#57, ss_net_paid#58] - -(32) CometHashAggregate -Input [2]: [ss_ext_discount_amt#57, ss_net_paid#58] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#57)), partial_avg(UnscaledValue(ss_net_paid#58))] - -(33) CometExchange -Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(34) CometHashAggregate -Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#57)), avg(UnscaledValue(ss_net_paid#58))] - -(35) CometProject -Input [3]: [count(1)#65, avg(ss_ext_discount_amt)#66, avg(ss_net_paid)#67] -Arguments: [mergedValue#68], [named_struct(count(1), count(1)#65, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#66, avg(ss_net_paid), avg(ss_net_paid)#67) AS mergedValue#68] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#68] - -Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] - -Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] - -Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* CometColumnarToRow (44) -+- CometProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (37) - - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] -ReadSchema: struct - -(38) CometFilter -Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Condition : ((isnotnull(ss_quantity#69) AND (ss_quantity#69 >= 81)) AND (ss_quantity#69 <= 100)) - -(39) CometProject -Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Arguments: [ss_ext_discount_amt#70, ss_net_paid#71], [ss_ext_discount_amt#70, ss_net_paid#71] - -(40) CometHashAggregate -Input [2]: [ss_ext_discount_amt#70, ss_net_paid#71] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#70)), partial_avg(UnscaledValue(ss_net_paid#71))] - -(41) CometExchange -Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(42) CometHashAggregate -Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#70)), avg(UnscaledValue(ss_net_paid#71))] - -(43) CometProject -Input [3]: [count(1)#78, avg(ss_ext_discount_amt)#79, avg(ss_net_paid)#80] -Arguments: [mergedValue#81], [named_struct(count(1), count(1)#78, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#79, avg(ss_net_paid), avg(ss_net_paid)#80) AS mergedValue#81] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#81] - -Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] - -Subquery:15 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/simplified.txt deleted file mode 100644 index 558f5f4b36..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -WholeStageCodegen (1) - Project - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #1 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #2 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #3 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #4 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #5 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometColumnarToRow - InputAdapter - CometFilter [r_reason_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.reason [r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt deleted file mode 100644 index b3f32555de..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt +++ /dev/null @@ -1,283 +0,0 @@ -== Physical Plan == -* Project (4) -+- * CometColumnarToRow (3) - +- CometFilter (2) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (1) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Output [1]: [r_reason_sk#1] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk), EqualTo(r_reason_sk,1)] -ReadSchema: struct - -(2) CometFilter -Input [1]: [r_reason_sk#1] -Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) - -(3) CometColumnarToRow [codegen id : 1] -Input [1]: [r_reason_sk#1] - -(4) Project [codegen id : 1] -Output [5]: [CASE WHEN (Subquery scalar-subquery#2, [id=#3].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket1#4, CASE WHEN (Subquery scalar-subquery#5, [id=#6].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_net_paid) END AS bucket2#7, CASE WHEN (Subquery scalar-subquery#8, [id=#9].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_net_paid) END AS bucket3#10, CASE WHEN (Subquery scalar-subquery#11, [id=#12].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_net_paid) END AS bucket4#13, CASE WHEN (Subquery scalar-subquery#14, [id=#15].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_net_paid) END AS bucket5#16] -Input [1]: [r_reason_sk#1] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] -* CometColumnarToRow (12) -+- CometProject (11) - +- CometHashAggregate (10) - +- CometExchange (9) - +- CometHashAggregate (8) - +- CometProject (7) - +- CometFilter (6) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] -ReadSchema: struct - -(6) CometFilter -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Condition : ((isnotnull(ss_quantity#17) AND (ss_quantity#17 >= 1)) AND (ss_quantity#17 <= 20)) - -(7) CometProject -Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] -Arguments: [ss_ext_discount_amt#18, ss_net_paid#19], [ss_ext_discount_amt#18, ss_net_paid#19] - -(8) CometHashAggregate -Input [2]: [ss_ext_discount_amt#18, ss_net_paid#19] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#18)), partial_avg(UnscaledValue(ss_net_paid#19))] - -(9) CometExchange -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometHashAggregate -Input [5]: [count#21, sum#22, count#23, sum#24, count#25] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(UnscaledValue(ss_net_paid#19))] - -(11) CometProject -Input [3]: [count(1)#26, avg(ss_ext_discount_amt)#27, avg(ss_net_paid)#28] -Arguments: [mergedValue#29], [named_struct(count(1), count(1)#26, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#27, avg(ss_net_paid), avg(ss_net_paid)#28) AS mergedValue#29] - -(12) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#29] - -Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] - -Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] - -Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] -* CometColumnarToRow (20) -+- CometProject (19) - +- CometHashAggregate (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometProject (15) - +- CometFilter (14) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Condition : ((isnotnull(ss_quantity#30) AND (ss_quantity#30 >= 21)) AND (ss_quantity#30 <= 40)) - -(15) CometProject -Input [4]: [ss_quantity#30, ss_ext_discount_amt#31, ss_net_paid#32, ss_sold_date_sk#33] -Arguments: [ss_ext_discount_amt#31, ss_net_paid#32], [ss_ext_discount_amt#31, ss_net_paid#32] - -(16) CometHashAggregate -Input [2]: [ss_ext_discount_amt#31, ss_net_paid#32] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#31)), partial_avg(UnscaledValue(ss_net_paid#32))] - -(17) CometExchange -Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(18) CometHashAggregate -Input [5]: [count#34, sum#35, count#36, sum#37, count#38] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#31)), avg(UnscaledValue(ss_net_paid#32))] - -(19) CometProject -Input [3]: [count(1)#39, avg(ss_ext_discount_amt)#40, avg(ss_net_paid)#41] -Arguments: [mergedValue#42], [named_struct(count(1), count(1)#39, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#40, avg(ss_net_paid), avg(ss_net_paid)#41) AS mergedValue#42] - -(20) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#42] - -Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] - -Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] - -Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] -* CometColumnarToRow (28) -+- CometProject (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometFilter (22) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] -ReadSchema: struct - -(22) CometFilter -Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Condition : ((isnotnull(ss_quantity#43) AND (ss_quantity#43 >= 41)) AND (ss_quantity#43 <= 60)) - -(23) CometProject -Input [4]: [ss_quantity#43, ss_ext_discount_amt#44, ss_net_paid#45, ss_sold_date_sk#46] -Arguments: [ss_ext_discount_amt#44, ss_net_paid#45], [ss_ext_discount_amt#44, ss_net_paid#45] - -(24) CometHashAggregate -Input [2]: [ss_ext_discount_amt#44, ss_net_paid#45] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#44)), partial_avg(UnscaledValue(ss_net_paid#45))] - -(25) CometExchange -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(26) CometHashAggregate -Input [5]: [count#47, sum#48, count#49, sum#50, count#51] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#44)), avg(UnscaledValue(ss_net_paid#45))] - -(27) CometProject -Input [3]: [count(1)#52, avg(ss_ext_discount_amt)#53, avg(ss_net_paid)#54] -Arguments: [mergedValue#55], [named_struct(count(1), count(1)#52, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#53, avg(ss_net_paid), avg(ss_net_paid)#54) AS mergedValue#55] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#55] - -Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] - -Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] - -Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] -* CometColumnarToRow (36) -+- CometProject (35) - +- CometHashAggregate (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (29) - - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] -ReadSchema: struct - -(30) CometFilter -Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_quantity#56) AND (ss_quantity#56 >= 61)) AND (ss_quantity#56 <= 80)) - -(31) CometProject -Input [4]: [ss_quantity#56, ss_ext_discount_amt#57, ss_net_paid#58, ss_sold_date_sk#59] -Arguments: [ss_ext_discount_amt#57, ss_net_paid#58], [ss_ext_discount_amt#57, ss_net_paid#58] - -(32) CometHashAggregate -Input [2]: [ss_ext_discount_amt#57, ss_net_paid#58] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#57)), partial_avg(UnscaledValue(ss_net_paid#58))] - -(33) CometExchange -Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(34) CometHashAggregate -Input [5]: [count#60, sum#61, count#62, sum#63, count#64] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#57)), avg(UnscaledValue(ss_net_paid#58))] - -(35) CometProject -Input [3]: [count(1)#65, avg(ss_ext_discount_amt)#66, avg(ss_net_paid)#67] -Arguments: [mergedValue#68], [named_struct(count(1), count(1)#65, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#66, avg(ss_net_paid), avg(ss_net_paid)#67) AS mergedValue#68] - -(36) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#68] - -Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] - -Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] - -Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] -* CometColumnarToRow (44) -+- CometProject (43) - +- CometHashAggregate (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (37) - - -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] -ReadSchema: struct - -(38) CometFilter -Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Condition : ((isnotnull(ss_quantity#69) AND (ss_quantity#69 >= 81)) AND (ss_quantity#69 <= 100)) - -(39) CometProject -Input [4]: [ss_quantity#69, ss_ext_discount_amt#70, ss_net_paid#71, ss_sold_date_sk#72] -Arguments: [ss_ext_discount_amt#70, ss_net_paid#71], [ss_ext_discount_amt#70, ss_net_paid#71] - -(40) CometHashAggregate -Input [2]: [ss_ext_discount_amt#70, ss_net_paid#71] -Keys: [] -Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#70)), partial_avg(UnscaledValue(ss_net_paid#71))] - -(41) CometExchange -Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(42) CometHashAggregate -Input [5]: [count#73, sum#74, count#75, sum#76, count#77] -Keys: [] -Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#70)), avg(UnscaledValue(ss_net_paid#71))] - -(43) CometProject -Input [3]: [count(1)#78, avg(ss_ext_discount_amt)#79, avg(ss_net_paid)#80] -Arguments: [mergedValue#81], [named_struct(count(1), count(1)#78, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#79, avg(ss_net_paid), avg(ss_net_paid)#80) AS mergedValue#81] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [mergedValue#81] - -Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] - -Subquery:15 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/extended.txt deleted file mode 100644 index ed71033b26..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/extended.txt +++ /dev/null @@ -1,61 +0,0 @@ - Project [COMET: ] -: :- Subquery -: : +- CometColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: +- ReusedSubquery -+- CometColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason - -Comet accelerated 37 out of 53 eligible operators (69%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 558f5f4b36..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -WholeStageCodegen (1) - Project - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #1 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #2 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #3 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #4 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - Subquery #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] [mergedValue] - CometHashAggregate [count,sum,count,sum,count] [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid))] - CometExchange #5 - CometHashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count] - CometProject [ss_ext_discount_amt,ss_net_paid] - CometFilter [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometColumnarToRow - InputAdapter - CometFilter [r_reason_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.reason [r_reason_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_datafusion/explain.txt deleted file mode 100644 index 73b54f439b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_datafusion/explain.txt +++ /dev/null @@ -1,260 +0,0 @@ -== Physical Plan == -* Project (47) -+- * BroadcastNestedLoopJoin Inner BuildRight (46) - :- * CometColumnarToRow (25) - : +- CometHashAggregate (24) - : +- CometExchange (23) - : +- CometHashAggregate (22) - : +- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometProject (15) - : : +- CometBroadcastHashJoin (14) - : : :- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (4) - : : +- CometBroadcastExchange (13) - : : +- CometProject (12) - : : +- CometFilter (11) - : : +- CometNativeScan parquet spark_catalog.default.time_dim (10) - : +- CometBroadcastExchange (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometNativeScan parquet spark_catalog.default.web_page (16) - +- BroadcastExchange (45) - +- * CometColumnarToRow (44) - +- CometHashAggregate (43) - +- CometExchange (42) - +- CometHashAggregate (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometNativeScan parquet spark_catalog.default.web_sales (26) - : : +- ReusedExchange (29) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometNativeScan parquet spark_catalog.default.time_dim (32) - +- ReusedExchange (38) - - -(1) CometNativeScan parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3)) - -(3) CometProject -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Arguments: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] - -(4) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#5, hd_dep_count#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,6), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 6)) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ws_ship_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, hd_demo_sk#5] -Arguments: [ws_sold_time_sk#1, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_web_page_sk#3] - -(10) CometNativeScan parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#7, t_hour#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,8), LessThanOrEqual(t_hour,9), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [t_time_sk#7, t_hour#8] -Condition : (((isnotnull(t_hour#8) AND (t_hour#8 >= 8)) AND (t_hour#8 <= 9)) AND isnotnull(t_time_sk#7)) - -(12) CometProject -Input [2]: [t_time_sk#7, t_hour#8] -Arguments: [t_time_sk#7], [t_time_sk#7] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#7] -Arguments: [t_time_sk#7] - -(14) CometBroadcastHashJoin -Left output [2]: [ws_sold_time_sk#1, ws_web_page_sk#3] -Right output [1]: [t_time_sk#7] -Arguments: [ws_sold_time_sk#1], [t_time_sk#7], Inner, BuildRight - -(15) CometProject -Input [3]: [ws_sold_time_sk#1, ws_web_page_sk#3, t_time_sk#7] -Arguments: [ws_web_page_sk#3], [ws_web_page_sk#3] - -(16) CometNativeScan parquet spark_catalog.default.web_page -Output [2]: [wp_web_page_sk#9, wp_char_count#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_char_count), GreaterThanOrEqual(wp_char_count,5000), LessThanOrEqual(wp_char_count,5200), IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [wp_web_page_sk#9, wp_char_count#10] -Condition : (((isnotnull(wp_char_count#10) AND (wp_char_count#10 >= 5000)) AND (wp_char_count#10 <= 5200)) AND isnotnull(wp_web_page_sk#9)) - -(18) CometProject -Input [2]: [wp_web_page_sk#9, wp_char_count#10] -Arguments: [wp_web_page_sk#9], [wp_web_page_sk#9] - -(19) CometBroadcastExchange -Input [1]: [wp_web_page_sk#9] -Arguments: [wp_web_page_sk#9] - -(20) CometBroadcastHashJoin -Left output [1]: [ws_web_page_sk#3] -Right output [1]: [wp_web_page_sk#9] -Arguments: [ws_web_page_sk#3], [wp_web_page_sk#9], Inner, BuildRight - -(21) CometProject -Input [2]: [ws_web_page_sk#3, wp_web_page_sk#9] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#11] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#11] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 2] -Input [1]: [amc#12] - -(26) CometNativeScan parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Condition : ((isnotnull(ws_ship_hdemo_sk#14) AND isnotnull(ws_sold_time_sk#13)) AND isnotnull(ws_web_page_sk#15)) - -(28) CometProject -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Arguments: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] - -(29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#17] - -(30) CometBroadcastHashJoin -Left output [3]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] -Right output [1]: [hd_demo_sk#17] -Arguments: [ws_ship_hdemo_sk#14], [hd_demo_sk#17], Inner, BuildRight - -(31) CometProject -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, hd_demo_sk#17] -Arguments: [ws_sold_time_sk#13, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_web_page_sk#15] - -(32) CometNativeScan parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#18, t_hour#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [t_time_sk#18, t_hour#19] -Condition : (((isnotnull(t_hour#19) AND (t_hour#19 >= 19)) AND (t_hour#19 <= 20)) AND isnotnull(t_time_sk#18)) - -(34) CometProject -Input [2]: [t_time_sk#18, t_hour#19] -Arguments: [t_time_sk#18], [t_time_sk#18] - -(35) CometBroadcastExchange -Input [1]: [t_time_sk#18] -Arguments: [t_time_sk#18] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_sold_time_sk#13, ws_web_page_sk#15] -Right output [1]: [t_time_sk#18] -Arguments: [ws_sold_time_sk#13], [t_time_sk#18], Inner, BuildRight - -(37) CometProject -Input [3]: [ws_sold_time_sk#13, ws_web_page_sk#15, t_time_sk#18] -Arguments: [ws_web_page_sk#15], [ws_web_page_sk#15] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [wp_web_page_sk#20] - -(39) CometBroadcastHashJoin -Left output [1]: [ws_web_page_sk#15] -Right output [1]: [wp_web_page_sk#20] -Arguments: [ws_web_page_sk#15], [wp_web_page_sk#20], Inner, BuildRight - -(40) CometProject -Input [2]: [ws_web_page_sk#15, wp_web_page_sk#20] - -(41) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(42) CometExchange -Input [1]: [count#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(43) CometHashAggregate -Input [1]: [count#21] -Keys: [] -Functions [1]: [count(1)] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [pmc#22] - -(45) BroadcastExchange -Input [1]: [pmc#22] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(46) BroadcastNestedLoopJoin [codegen id : 2] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 2] -Output [1]: [(cast(amc#12 as decimal(15,4)) / cast(pmc#22 as decimal(15,4))) AS am_pm_ratio#23] -Input [2]: [amc#12, pmc#22] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_datafusion/simplified.txt deleted file mode 100644 index a13072cd91..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_datafusion/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (2) - Project [amc,pmc] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [amc,count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_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_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange [wp_web_page_sk] #4 - CometProject [wp_web_page_sk] - CometFilter [wp_web_page_sk,wp_char_count] - CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [pmc,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_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_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometNativeScan 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 [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometNativeScan 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/q90.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_iceberg_compat/explain.txt deleted file mode 100644 index 08ae744b5a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,260 +0,0 @@ -== Physical Plan == -* Project (47) -+- * BroadcastNestedLoopJoin Inner BuildRight (46) - :- * CometColumnarToRow (25) - : +- CometHashAggregate (24) - : +- CometExchange (23) - : +- CometHashAggregate (22) - : +- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometProject (15) - : : +- CometBroadcastHashJoin (14) - : : :- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (4) - : : +- CometBroadcastExchange (13) - : : +- CometProject (12) - : : +- CometFilter (11) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (10) - : +- CometBroadcastExchange (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (16) - +- BroadcastExchange (45) - +- * CometColumnarToRow (44) - +- CometHashAggregate (43) - +- CometExchange (42) - +- CometHashAggregate (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (26) - : : +- ReusedExchange (29) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (32) - +- ReusedExchange (38) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3)) - -(3) CometProject -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Arguments: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#5, hd_dep_count#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,6), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 6)) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ws_ship_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, hd_demo_sk#5] -Arguments: [ws_sold_time_sk#1, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_web_page_sk#3] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#7, t_hour#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,8), LessThanOrEqual(t_hour,9), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [t_time_sk#7, t_hour#8] -Condition : (((isnotnull(t_hour#8) AND (t_hour#8 >= 8)) AND (t_hour#8 <= 9)) AND isnotnull(t_time_sk#7)) - -(12) CometProject -Input [2]: [t_time_sk#7, t_hour#8] -Arguments: [t_time_sk#7], [t_time_sk#7] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#7] -Arguments: [t_time_sk#7] - -(14) CometBroadcastHashJoin -Left output [2]: [ws_sold_time_sk#1, ws_web_page_sk#3] -Right output [1]: [t_time_sk#7] -Arguments: [ws_sold_time_sk#1], [t_time_sk#7], Inner, BuildRight - -(15) CometProject -Input [3]: [ws_sold_time_sk#1, ws_web_page_sk#3, t_time_sk#7] -Arguments: [ws_web_page_sk#3], [ws_web_page_sk#3] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [2]: [wp_web_page_sk#9, wp_char_count#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_char_count), GreaterThanOrEqual(wp_char_count,5000), LessThanOrEqual(wp_char_count,5200), IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [wp_web_page_sk#9, wp_char_count#10] -Condition : (((isnotnull(wp_char_count#10) AND (wp_char_count#10 >= 5000)) AND (wp_char_count#10 <= 5200)) AND isnotnull(wp_web_page_sk#9)) - -(18) CometProject -Input [2]: [wp_web_page_sk#9, wp_char_count#10] -Arguments: [wp_web_page_sk#9], [wp_web_page_sk#9] - -(19) CometBroadcastExchange -Input [1]: [wp_web_page_sk#9] -Arguments: [wp_web_page_sk#9] - -(20) CometBroadcastHashJoin -Left output [1]: [ws_web_page_sk#3] -Right output [1]: [wp_web_page_sk#9] -Arguments: [ws_web_page_sk#3], [wp_web_page_sk#9], Inner, BuildRight - -(21) CometProject -Input [2]: [ws_web_page_sk#3, wp_web_page_sk#9] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#11] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#11] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 2] -Input [1]: [amc#12] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Condition : ((isnotnull(ws_ship_hdemo_sk#14) AND isnotnull(ws_sold_time_sk#13)) AND isnotnull(ws_web_page_sk#15)) - -(28) CometProject -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Arguments: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] - -(29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#17] - -(30) CometBroadcastHashJoin -Left output [3]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] -Right output [1]: [hd_demo_sk#17] -Arguments: [ws_ship_hdemo_sk#14], [hd_demo_sk#17], Inner, BuildRight - -(31) CometProject -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, hd_demo_sk#17] -Arguments: [ws_sold_time_sk#13, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_web_page_sk#15] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#18, t_hour#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [t_time_sk#18, t_hour#19] -Condition : (((isnotnull(t_hour#19) AND (t_hour#19 >= 19)) AND (t_hour#19 <= 20)) AND isnotnull(t_time_sk#18)) - -(34) CometProject -Input [2]: [t_time_sk#18, t_hour#19] -Arguments: [t_time_sk#18], [t_time_sk#18] - -(35) CometBroadcastExchange -Input [1]: [t_time_sk#18] -Arguments: [t_time_sk#18] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_sold_time_sk#13, ws_web_page_sk#15] -Right output [1]: [t_time_sk#18] -Arguments: [ws_sold_time_sk#13], [t_time_sk#18], Inner, BuildRight - -(37) CometProject -Input [3]: [ws_sold_time_sk#13, ws_web_page_sk#15, t_time_sk#18] -Arguments: [ws_web_page_sk#15], [ws_web_page_sk#15] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [wp_web_page_sk#20] - -(39) CometBroadcastHashJoin -Left output [1]: [ws_web_page_sk#15] -Right output [1]: [wp_web_page_sk#20] -Arguments: [ws_web_page_sk#15], [wp_web_page_sk#20], Inner, BuildRight - -(40) CometProject -Input [2]: [ws_web_page_sk#15, wp_web_page_sk#20] - -(41) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(42) CometExchange -Input [1]: [count#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(43) CometHashAggregate -Input [1]: [count#21] -Keys: [] -Functions [1]: [count(1)] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [pmc#22] - -(45) BroadcastExchange -Input [1]: [pmc#22] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(46) BroadcastNestedLoopJoin [codegen id : 2] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 2] -Output [1]: [(cast(amc#12 as decimal(15,4)) / cast(pmc#22 as decimal(15,4))) AS am_pm_ratio#23] -Input [2]: [amc#12, pmc#22] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_iceberg_compat/simplified.txt deleted file mode 100644 index 0991e4e0dc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (2) - Project [amc,pmc] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [amc,count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_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_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange [wp_web_page_sk] #4 - CometProject [wp_web_page_sk] - CometFilter [wp_web_page_sk,wp_char_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [pmc,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_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_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometScan [native_iceberg_compat] 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/q90/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt deleted file mode 100644 index 08ae744b5a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt +++ /dev/null @@ -1,260 +0,0 @@ -== Physical Plan == -* Project (47) -+- * BroadcastNestedLoopJoin Inner BuildRight (46) - :- * CometColumnarToRow (25) - : +- CometHashAggregate (24) - : +- CometExchange (23) - : +- CometHashAggregate (22) - : +- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometProject (15) - : : +- CometBroadcastHashJoin (14) - : : :- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (4) - : : +- CometBroadcastExchange (13) - : : +- CometProject (12) - : : +- CometFilter (11) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (10) - : +- CometBroadcastExchange (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (16) - +- BroadcastExchange (45) - +- * CometColumnarToRow (44) - +- CometHashAggregate (43) - +- CometExchange (42) - +- CometHashAggregate (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (26) - : : +- ReusedExchange (29) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (32) - +- ReusedExchange (38) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3)) - -(3) CometProject -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Arguments: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#5, hd_dep_count#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,6), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 6)) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ws_ship_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, hd_demo_sk#5] -Arguments: [ws_sold_time_sk#1, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_web_page_sk#3] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#7, t_hour#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,8), LessThanOrEqual(t_hour,9), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [t_time_sk#7, t_hour#8] -Condition : (((isnotnull(t_hour#8) AND (t_hour#8 >= 8)) AND (t_hour#8 <= 9)) AND isnotnull(t_time_sk#7)) - -(12) CometProject -Input [2]: [t_time_sk#7, t_hour#8] -Arguments: [t_time_sk#7], [t_time_sk#7] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#7] -Arguments: [t_time_sk#7] - -(14) CometBroadcastHashJoin -Left output [2]: [ws_sold_time_sk#1, ws_web_page_sk#3] -Right output [1]: [t_time_sk#7] -Arguments: [ws_sold_time_sk#1], [t_time_sk#7], Inner, BuildRight - -(15) CometProject -Input [3]: [ws_sold_time_sk#1, ws_web_page_sk#3, t_time_sk#7] -Arguments: [ws_web_page_sk#3], [ws_web_page_sk#3] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [2]: [wp_web_page_sk#9, wp_char_count#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_char_count), GreaterThanOrEqual(wp_char_count,5000), LessThanOrEqual(wp_char_count,5200), IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [wp_web_page_sk#9, wp_char_count#10] -Condition : (((isnotnull(wp_char_count#10) AND (wp_char_count#10 >= 5000)) AND (wp_char_count#10 <= 5200)) AND isnotnull(wp_web_page_sk#9)) - -(18) CometProject -Input [2]: [wp_web_page_sk#9, wp_char_count#10] -Arguments: [wp_web_page_sk#9], [wp_web_page_sk#9] - -(19) CometBroadcastExchange -Input [1]: [wp_web_page_sk#9] -Arguments: [wp_web_page_sk#9] - -(20) CometBroadcastHashJoin -Left output [1]: [ws_web_page_sk#3] -Right output [1]: [wp_web_page_sk#9] -Arguments: [ws_web_page_sk#3], [wp_web_page_sk#9], Inner, BuildRight - -(21) CometProject -Input [2]: [ws_web_page_sk#3, wp_web_page_sk#9] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#11] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#11] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 2] -Input [1]: [amc#12] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Condition : ((isnotnull(ws_ship_hdemo_sk#14) AND isnotnull(ws_sold_time_sk#13)) AND isnotnull(ws_web_page_sk#15)) - -(28) CometProject -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, ws_sold_date_sk#16] -Arguments: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] - -(29) ReusedExchange [Reuses operator id: 7] -Output [1]: [hd_demo_sk#17] - -(30) CometBroadcastHashJoin -Left output [3]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15] -Right output [1]: [hd_demo_sk#17] -Arguments: [ws_ship_hdemo_sk#14], [hd_demo_sk#17], Inner, BuildRight - -(31) CometProject -Input [4]: [ws_sold_time_sk#13, ws_ship_hdemo_sk#14, ws_web_page_sk#15, hd_demo_sk#17] -Arguments: [ws_sold_time_sk#13, ws_web_page_sk#15], [ws_sold_time_sk#13, ws_web_page_sk#15] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [2]: [t_time_sk#18, t_hour#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [t_time_sk#18, t_hour#19] -Condition : (((isnotnull(t_hour#19) AND (t_hour#19 >= 19)) AND (t_hour#19 <= 20)) AND isnotnull(t_time_sk#18)) - -(34) CometProject -Input [2]: [t_time_sk#18, t_hour#19] -Arguments: [t_time_sk#18], [t_time_sk#18] - -(35) CometBroadcastExchange -Input [1]: [t_time_sk#18] -Arguments: [t_time_sk#18] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_sold_time_sk#13, ws_web_page_sk#15] -Right output [1]: [t_time_sk#18] -Arguments: [ws_sold_time_sk#13], [t_time_sk#18], Inner, BuildRight - -(37) CometProject -Input [3]: [ws_sold_time_sk#13, ws_web_page_sk#15, t_time_sk#18] -Arguments: [ws_web_page_sk#15], [ws_web_page_sk#15] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [wp_web_page_sk#20] - -(39) CometBroadcastHashJoin -Left output [1]: [ws_web_page_sk#15] -Right output [1]: [wp_web_page_sk#20] -Arguments: [ws_web_page_sk#15], [wp_web_page_sk#20], Inner, BuildRight - -(40) CometProject -Input [2]: [ws_web_page_sk#15, wp_web_page_sk#20] - -(41) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(42) CometExchange -Input [1]: [count#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(43) CometHashAggregate -Input [1]: [count#21] -Keys: [] -Functions [1]: [count(1)] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [pmc#22] - -(45) BroadcastExchange -Input [1]: [pmc#22] -Arguments: IdentityBroadcastMode, [plan_id=3] - -(46) BroadcastNestedLoopJoin [codegen id : 2] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 2] -Output [1]: [(cast(amc#12 as decimal(15,4)) / cast(pmc#22 as decimal(15,4))) AS am_pm_ratio#23] -Input [2]: [amc#12, pmc#22] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/extended.txt deleted file mode 100644 index aa6c577ed7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/extended.txt +++ /dev/null @@ -1,55 +0,0 @@ -Project -+- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- BroadcastExchange - +- CometColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - -Comet accelerated 48 out of 51 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 0991e4e0dc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (2) - Project [amc,pmc] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [amc,count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_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_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour] - CometBroadcastExchange [wp_web_page_sk] #4 - CometProject [wp_web_page_sk] - CometFilter [wp_web_page_sk,wp_char_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [pmc,count(1)] - CometExchange #6 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - CometProject [ws_web_page_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_web_page_sk,t_time_sk] - CometProject [ws_sold_time_sk,ws_web_page_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_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 [t_time_sk] #7 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/explain.txt deleted file mode 100644 index b52840adb9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/explain.txt +++ /dev/null @@ -1,299 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometSort (46) - +- CometColumnarExchange (45) - +- * HashAggregate (44) - +- * CometColumnarToRow (43) - +- CometColumnarExchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * BroadcastHashJoin Inner BuildRight (25) - : : :- * Project (19) - : : : +- * BroadcastHashJoin Inner BuildRight (18) - : : : :- * Project (13) - : : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : : :- * Project (10) - : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : :- * CometColumnarToRow (4) - : : : : : : +- CometProject (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometNativeScan parquet spark_catalog.default.call_center (1) - : : : : : +- BroadcastExchange (8) - : : : : : +- * Filter (7) - : : : : : +- * ColumnarToRow (6) - : : : : : +- Scan parquet spark_catalog.default.catalog_returns (5) - : : : : +- ReusedExchange (11) - : : : +- BroadcastExchange (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometFilter (15) - : : : +- CometNativeScan parquet spark_catalog.default.customer (14) - : : +- BroadcastExchange (24) - : : +- * CometColumnarToRow (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometNativeScan parquet spark_catalog.default.customer_address (20) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometNativeScan parquet spark_catalog.default.customer_demographics (27) - +- BroadcastExchange (38) - +- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometNativeScan parquet spark_catalog.default.household_demographics (34) - - -(1) CometNativeScan parquet spark_catalog.default.call_center -Output [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Condition : isnotnull(cc_call_center_sk#1) - -(3) CometProject -Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Arguments: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4], [cc_call_center_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cc_call_center_id#2, 16, true, false, true) AS cc_call_center_id#5, cc_name#3, cc_manager#4] - -(4) CometColumnarToRow [codegen id : 7] -Input [4]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4] - -(5) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#9), dynamicpruningexpression(cr_returned_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] - -(7) Filter [codegen id : 1] -Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Condition : (isnotnull(cr_call_center_sk#7) AND isnotnull(cr_returning_customer_sk#6)) - -(8) BroadcastExchange -Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cc_call_center_sk#1] -Right keys [1]: [cr_call_center_sk#7] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 7] -Output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9] -Input [8]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] - -(11) ReusedExchange [Reuses operator id: 52] -Output [1]: [d_date_sk#11] - -(12) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cr_returned_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 7] -Output [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8] -Input [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9, d_date_sk#11] - -(14) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#12, c_current_cdemo_sk#13, c_current_hdemo_sk#14, c_current_addr_sk#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [c_customer_sk#12, c_current_cdemo_sk#13, c_current_hdemo_sk#14, c_current_addr_sk#15] -Condition : (((isnotnull(c_customer_sk#12) AND isnotnull(c_current_addr_sk#15)) AND isnotnull(c_current_cdemo_sk#13)) AND isnotnull(c_current_hdemo_sk#14)) - -(16) CometColumnarToRow [codegen id : 3] -Input [4]: [c_customer_sk#12, c_current_cdemo_sk#13, c_current_hdemo_sk#14, c_current_addr_sk#15] - -(17) BroadcastExchange -Input [4]: [c_customer_sk#12, c_current_cdemo_sk#13, c_current_hdemo_sk#14, c_current_addr_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cr_returning_customer_sk#6] -Right keys [1]: [c_customer_sk#12] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 7] -Output [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#13, c_current_hdemo_sk#14, c_current_addr_sk#15] -Input [9]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, c_customer_sk#12, c_current_cdemo_sk#13, c_current_hdemo_sk#14, c_current_addr_sk#15] - -(20) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_gmt_offset#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-7.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [ca_address_sk#16, ca_gmt_offset#17] -Condition : ((isnotnull(ca_gmt_offset#17) AND (ca_gmt_offset#17 = -7.00)) AND isnotnull(ca_address_sk#16)) - -(22) CometProject -Input [2]: [ca_address_sk#16, ca_gmt_offset#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(23) CometColumnarToRow [codegen id : 4] -Input [1]: [ca_address_sk#16] - -(24) BroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(25) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#15] -Right keys [1]: [ca_address_sk#16] -Join type: Inner -Join condition: None - -(26) Project [codegen id : 7] -Output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#13, c_current_hdemo_sk#14] -Input [8]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#13, c_current_hdemo_sk#14, c_current_addr_sk#15, ca_address_sk#16] - -(27) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(28) CometFilter -Input [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#20, 20, true, false, true) = Unknown )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = W) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#20, 20, true, false, true) = Advanced Degree ))) AND isnotnull(cd_demo_sk#18)) - -(29) CometProject -Input [3]: [cd_demo_sk#18, cd_marital_status#19, cd_education_status#20] -Arguments: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22], [cd_demo_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) AS cd_marital_status#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#20, 20, true, false, true) AS cd_education_status#22] - -(30) CometColumnarToRow [codegen id : 5] -Input [3]: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] - -(31) BroadcastExchange -Input [3]: [cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#13] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 7] -Output [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#14, cd_marital_status#21, cd_education_status#22] -Input [9]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#13, c_current_hdemo_sk#14, cd_demo_sk#18, cd_marital_status#21, cd_education_status#22] - -(34) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#23, hd_buy_potential#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [hd_demo_sk#23, hd_buy_potential#24] -Condition : (StartsWith(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#24, 15, true, false, true), Unknown) AND isnotnull(hd_demo_sk#23)) - -(36) CometProject -Input [2]: [hd_demo_sk#23, hd_buy_potential#24] -Arguments: [hd_demo_sk#23], [hd_demo_sk#23] - -(37) CometColumnarToRow [codegen id : 6] -Input [1]: [hd_demo_sk#23] - -(38) BroadcastExchange -Input [1]: [hd_demo_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_hdemo_sk#14] -Right keys [1]: [hd_demo_sk#23] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 7] -Output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#21, cd_education_status#22] -Input [8]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#14, cd_marital_status#21, cd_education_status#22, hd_demo_sk#23] - -(41) HashAggregate [codegen id : 7] -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#21, cd_education_status#22] -Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#21, cd_education_status#22] -Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#8))] -Aggregate Attributes [1]: [sum#25] -Results [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#21, cd_education_status#22, sum#26] - -(42) CometColumnarExchange -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#21, cd_education_status#22, sum#26] -Arguments: hashpartitioning(cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#21, cd_education_status#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) CometColumnarToRow [codegen id : 8] -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#21, cd_education_status#22, sum#26] - -(44) HashAggregate [codegen id : 8] -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#21, cd_education_status#22, sum#26] -Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#21, cd_education_status#22] -Functions [1]: [sum(UnscaledValue(cr_net_loss#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#8))#27] -Results [4]: [cc_call_center_id#5 AS Call_Center#28, cc_name#3 AS Call_Center_Name#29, cc_manager#4 AS Manager#30, MakeDecimal(sum(UnscaledValue(cr_net_loss#8))#27,17,2) AS Returns_Loss#31] - -(45) CometColumnarExchange -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(46) CometSort -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31], [Returns_Loss#31 DESC NULLS LAST] - -(47) CometColumnarToRow [codegen id : 9] -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = cr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometNativeScan parquet spark_catalog.default.date_dim (48) - - -(48) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#32, d_moy#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#11, d_year#32, d_moy#33] -Condition : ((((isnotnull(d_year#32) AND isnotnull(d_moy#33)) AND (d_year#32 = 1998)) AND (d_moy#33 = 11)) AND isnotnull(d_date_sk#11)) - -(50) CometProject -Input [3]: [d_date_sk#11, d_year#32, d_moy#33] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(52) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/simplified.txt deleted file mode 100644 index 75fea0ec7f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/simplified.txt +++ /dev/null @@ -1,77 +0,0 @@ -WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] - CometColumnarExchange [Returns_Loss] #1 - WholeStageCodegen (8) - HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] [sum(UnscaledValue(cr_net_loss)),Call_Center,Call_Center_Name,Manager,Returns_Loss,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 - WholeStageCodegen (7) - HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] [sum,sum] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - BroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Project [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] - BroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] - CometColumnarToRow - InputAdapter - CometProject [cc_call_center_id] [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [cr_call_center_sk,cr_returning_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometNativeScan 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/q91.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/explain.txt deleted file mode 100644 index cf532d2b41..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,280 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (43) -+- CometSort (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (19) - : : : +- CometBroadcastHashJoin (18) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometProject (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (4) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : : +- CometBroadcastExchange (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (15) - : : +- CometBroadcastExchange (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Condition : isnotnull(cc_call_center_sk#1) - -(3) CometProject -Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Arguments: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4], [cc_call_center_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cc_call_center_id#2, 16, true, false, true) AS cc_call_center_id#5, cc_name#3, cc_manager#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#9), dynamicpruningexpression(cr_returned_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Condition : (isnotnull(cr_call_center_sk#7) AND isnotnull(cr_returning_customer_sk#6)) - -(6) CometBroadcastExchange -Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Arguments: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [4]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4] -Right output [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Arguments: [cc_call_center_sk#1], [cr_call_center_sk#7], Inner, BuildRight - -(8) CometProject -Input [8]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#13)) AND (d_year#12 = 1998)) AND (d_moy#13 = 11)) AND isnotnull(d_date_sk#11)) - -(11) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(13) CometBroadcastHashJoin -Left output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [cr_returned_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(14) CometProject -Input [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9, d_date_sk#11] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Condition : (((isnotnull(c_customer_sk#14) AND isnotnull(c_current_addr_sk#17)) AND isnotnull(c_current_cdemo_sk#15)) AND isnotnull(c_current_hdemo_sk#16)) - -(17) CometBroadcastExchange -Input [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Arguments: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] - -(18) CometBroadcastHashJoin -Left output [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8] -Right output [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Arguments: [cr_returning_customer_sk#6], [c_customer_sk#14], Inner, BuildRight - -(19) CometProject -Input [9]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_gmt_offset#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-7.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [ca_address_sk#18, ca_gmt_offset#19] -Condition : ((isnotnull(ca_gmt_offset#19) AND (ca_gmt_offset#19 = -7.00)) AND isnotnull(ca_address_sk#18)) - -(22) CometProject -Input [2]: [ca_address_sk#18, ca_gmt_offset#19] -Arguments: [ca_address_sk#18], [ca_address_sk#18] - -(23) CometBroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: [ca_address_sk#18] - -(24) CometBroadcastHashJoin -Left output [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Right output [1]: [ca_address_sk#18] -Arguments: [c_current_addr_sk#17], [ca_address_sk#18], Inner, BuildRight - -(25) CometProject -Input [8]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17, ca_address_sk#18] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#21, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#22, 20, true, false, true) = Unknown )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#21, 1, true, false, true) = W) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#22, 20, true, false, true) = Advanced Degree ))) AND isnotnull(cd_demo_sk#20)) - -(28) CometProject -Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Arguments: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24], [cd_demo_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#21, 1, true, false, true) AS cd_marital_status#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#22, 20, true, false, true) AS cd_education_status#24] - -(29) CometBroadcastExchange -Input [3]: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] -Arguments: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] - -(30) CometBroadcastHashJoin -Left output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16] -Right output [3]: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] -Arguments: [c_current_cdemo_sk#15], [cd_demo_sk#20], Inner, BuildRight - -(31) CometProject -Input [9]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#25, hd_buy_potential#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [hd_demo_sk#25, hd_buy_potential#26] -Condition : (StartsWith(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#26, 15, true, false, true), Unknown) AND isnotnull(hd_demo_sk#25)) - -(34) CometProject -Input [2]: [hd_demo_sk#25, hd_buy_potential#26] -Arguments: [hd_demo_sk#25], [hd_demo_sk#25] - -(35) CometBroadcastExchange -Input [1]: [hd_demo_sk#25] -Arguments: [hd_demo_sk#25] - -(36) CometBroadcastHashJoin -Left output [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24] -Right output [1]: [hd_demo_sk#25] -Arguments: [c_current_hdemo_sk#16], [hd_demo_sk#25], Inner, BuildRight - -(37) CometProject -Input [8]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24, hd_demo_sk#25] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#23, cd_education_status#24], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#23, cd_education_status#24] - -(38) CometHashAggregate -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#23, cd_education_status#24] -Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24] -Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#8))] - -(39) CometExchange -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, sum#27] -Arguments: hashpartitioning(cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(40) CometHashAggregate -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, sum#27] -Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24] -Functions [1]: [sum(UnscaledValue(cr_net_loss#8))] - -(41) CometExchange -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(42) CometSort -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31], [Returns_Loss#31 DESC NULLS LAST] - -(43) CometColumnarToRow [codegen id : 1] -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = cr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) - - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#13)) AND (d_year#12 = 1998)) AND (d_moy#13 = 11)) AND isnotnull(d_date_sk#11)) - -(46) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(48) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/simplified.txt deleted file mode 100644 index 3e9b8945da..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,53 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] - CometExchange [Returns_Loss] #1 - CometHashAggregate [cd_marital_status,cd_education_status,sum] [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,sum(UnscaledValue(cr_net_loss))] - CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 - CometHashAggregate [cr_net_loss] [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - 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 [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 [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 [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 [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,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] - CometProject [cc_call_center_id] [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk] #7 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [hd_demo_sk] #9 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan [native_iceberg_compat] 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/q91/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt deleted file mode 100644 index cf532d2b41..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt +++ /dev/null @@ -1,280 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (43) -+- CometSort (42) - +- CometExchange (41) - +- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometBroadcastHashJoin (36) - :- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (19) - : : : +- CometBroadcastHashJoin (18) - : : : :- CometProject (14) - : : : : +- CometBroadcastHashJoin (13) - : : : : :- CometProject (8) - : : : : : +- CometBroadcastHashJoin (7) - : : : : : :- CometProject (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (4) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : : +- CometBroadcastExchange (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (15) - : : +- CometBroadcastExchange (23) - : : +- CometProject (22) - : : +- CometFilter (21) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Condition : isnotnull(cc_call_center_sk#1) - -(3) CometProject -Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] -Arguments: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4], [cc_call_center_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cc_call_center_id#2, 16, true, false, true) AS cc_call_center_id#5, cc_name#3, cc_manager#4] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#9), dynamicpruningexpression(cr_returned_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Condition : (isnotnull(cr_call_center_sk#7) AND isnotnull(cr_returning_customer_sk#6)) - -(6) CometBroadcastExchange -Input [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Arguments: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [4]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4] -Right output [4]: [cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Arguments: [cc_call_center_sk#1], [cr_call_center_sk#7], Inner, BuildRight - -(8) CometProject -Input [8]: [cc_call_center_sk#1, cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_call_center_sk#7, cr_net_loss#8, cr_returned_date_sk#9] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#13)) AND (d_year#12 = 1998)) AND (d_moy#13 = 11)) AND isnotnull(d_date_sk#11)) - -(11) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(13) CometBroadcastHashJoin -Left output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [cr_returned_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(14) CometProject -Input [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, cr_returned_date_sk#9, d_date_sk#11] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Condition : (((isnotnull(c_customer_sk#14) AND isnotnull(c_current_addr_sk#17)) AND isnotnull(c_current_cdemo_sk#15)) AND isnotnull(c_current_hdemo_sk#16)) - -(17) CometBroadcastExchange -Input [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Arguments: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] - -(18) CometBroadcastHashJoin -Left output [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8] -Right output [4]: [c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Arguments: [cr_returning_customer_sk#6], [c_customer_sk#14], Inner, BuildRight - -(19) CometProject -Input [9]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_returning_customer_sk#6, cr_net_loss#8, c_customer_sk#14, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_gmt_offset#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-7.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [2]: [ca_address_sk#18, ca_gmt_offset#19] -Condition : ((isnotnull(ca_gmt_offset#19) AND (ca_gmt_offset#19 = -7.00)) AND isnotnull(ca_address_sk#18)) - -(22) CometProject -Input [2]: [ca_address_sk#18, ca_gmt_offset#19] -Arguments: [ca_address_sk#18], [ca_address_sk#18] - -(23) CometBroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: [ca_address_sk#18] - -(24) CometBroadcastHashJoin -Left output [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17] -Right output [1]: [ca_address_sk#18] -Arguments: [c_current_addr_sk#17], [ca_address_sk#18], Inner, BuildRight - -(25) CometProject -Input [8]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, c_current_addr_sk#17, ca_address_sk#18] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#21, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#22, 20, true, false, true) = Unknown )) OR ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#21, 1, true, false, true) = W) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#22, 20, true, false, true) = Advanced Degree ))) AND isnotnull(cd_demo_sk#20)) - -(28) CometProject -Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -Arguments: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24], [cd_demo_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#21, 1, true, false, true) AS cd_marital_status#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#22, 20, true, false, true) AS cd_education_status#24] - -(29) CometBroadcastExchange -Input [3]: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] -Arguments: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] - -(30) CometBroadcastHashJoin -Left output [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16] -Right output [3]: [cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] -Arguments: [c_current_cdemo_sk#15], [cd_demo_sk#20], Inner, BuildRight - -(31) CometProject -Input [9]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_cdemo_sk#15, c_current_hdemo_sk#16, cd_demo_sk#20, cd_marital_status#23, cd_education_status#24] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#25, hd_buy_potential#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [hd_demo_sk#25, hd_buy_potential#26] -Condition : (StartsWith(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#26, 15, true, false, true), Unknown) AND isnotnull(hd_demo_sk#25)) - -(34) CometProject -Input [2]: [hd_demo_sk#25, hd_buy_potential#26] -Arguments: [hd_demo_sk#25], [hd_demo_sk#25] - -(35) CometBroadcastExchange -Input [1]: [hd_demo_sk#25] -Arguments: [hd_demo_sk#25] - -(36) CometBroadcastHashJoin -Left output [7]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24] -Right output [1]: [hd_demo_sk#25] -Arguments: [c_current_hdemo_sk#16], [hd_demo_sk#25], Inner, BuildRight - -(37) CometProject -Input [8]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, c_current_hdemo_sk#16, cd_marital_status#23, cd_education_status#24, hd_demo_sk#25] -Arguments: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#23, cd_education_status#24], [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#23, cd_education_status#24] - -(38) CometHashAggregate -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cr_net_loss#8, cd_marital_status#23, cd_education_status#24] -Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24] -Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#8))] - -(39) CometExchange -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, sum#27] -Arguments: hashpartitioning(cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(40) CometHashAggregate -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, sum#27] -Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24] -Functions [1]: [sum(UnscaledValue(cr_net_loss#8))] - -(41) CometExchange -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(42) CometSort -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31], [Returns_Loss#31 DESC NULLS LAST] - -(43) CometColumnarToRow [codegen id : 1] -Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = cr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) - - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#13)) AND (d_year#12 = 1998)) AND (d_moy#13 = 11)) AND isnotnull(d_date_sk#11)) - -(46) CometProject -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(48) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/extended.txt deleted file mode 100644 index a1c6b73d37..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/extended.txt +++ /dev/null @@ -1,51 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 3e9b8945da..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt +++ /dev/null @@ -1,53 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] - CometExchange [Returns_Loss] #1 - CometHashAggregate [cd_marital_status,cd_education_status,sum] [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,sum(UnscaledValue(cr_net_loss))] - CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 - CometHashAggregate [cr_net_loss] [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - 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 [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 [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 [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 [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,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] - CometProject [cc_call_center_id] [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ca_address_sk] #7 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [hd_demo_sk] #9 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/explain.txt deleted file mode 100644 index 343037a34d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/explain.txt +++ /dev/null @@ -1,217 +0,0 @@ -== Physical Plan == -* HashAggregate (31) -+- * CometColumnarToRow (30) - +- CometColumnarExchange (29) - +- * HashAggregate (28) - +- * Project (27) - +- * BroadcastHashJoin Inner BuildRight (26) - :- * Project (24) - : +- * BroadcastHashJoin Inner BuildRight (23) - : :- * Project (10) - : : +- * BroadcastHashJoin Inner BuildRight (9) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : +- BroadcastExchange (8) - : : +- * CometColumnarToRow (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometNativeScan parquet spark_catalog.default.item (4) - : +- BroadcastExchange (22) - : +- * Filter (21) - : +- * HashAggregate (20) - : +- * CometColumnarToRow (19) - : +- CometColumnarExchange (18) - : +- * HashAggregate (17) - : +- * Project (16) - : +- * BroadcastHashJoin Inner BuildRight (15) - : :- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet spark_catalog.default.web_sales (11) - : +- ReusedExchange (14) - +- ReusedExchange (25) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_ext_discount_amt)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 6] -Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] - -(3) Filter [codegen id : 6] -Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Condition : (isnotnull(ws_item_sk#1) AND isnotnull(ws_ext_discount_amt#2)) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_manufact_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,350), IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [i_item_sk#5, i_manufact_id#6] -Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 350)) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [2]: [i_item_sk#5, i_manufact_id#6] -Arguments: [i_item_sk#5], [i_item_sk#5] - -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [i_item_sk#5] - -(8) BroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 6] -Output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] -Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] - -(11) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] - -(13) Filter [codegen id : 3] -Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Condition : isnotnull(ws_item_sk#7) - -(14) ReusedExchange [Reuses operator id: 36] -Output [1]: [d_date_sk#10] - -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(16) Project [codegen id : 3] -Output [2]: [ws_item_sk#7, ws_ext_discount_amt#8] -Input [4]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9, d_date_sk#10] - -(17) HashAggregate [codegen id : 3] -Input [2]: [ws_item_sk#7, ws_ext_discount_amt#8] -Keys [1]: [ws_item_sk#7] -Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#8))] -Aggregate Attributes [2]: [sum#11, count#12] -Results [3]: [ws_item_sk#7, sum#13, count#14] - -(18) CometColumnarExchange -Input [3]: [ws_item_sk#7, sum#13, count#14] -Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(19) CometColumnarToRow [codegen id : 4] -Input [3]: [ws_item_sk#7, sum#13, count#14] - -(20) HashAggregate [codegen id : 4] -Input [3]: [ws_item_sk#7, sum#13, count#14] -Keys [1]: [ws_item_sk#7] -Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))] -Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))#15] -Results [2]: [(1.3 * cast((avg(UnscaledValue(ws_ext_discount_amt#8))#15 / 100.0) as decimal(11,6))) AS (1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] - -(21) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] -Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#16) - -(22) BroadcastExchange -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] - -(23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [i_item_sk#5] -Right keys [1]: [ws_item_sk#7] -Join type: Inner -Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#16) - -(24) Project [codegen id : 6] -Output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] -Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7] - -(25) ReusedExchange [Reuses operator id: 36] -Output [1]: [d_date_sk#10] - -(26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 6] -Output [1]: [ws_ext_discount_amt#2] -Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#10] - -(28) HashAggregate [codegen id : 6] -Input [1]: [ws_ext_discount_amt#2] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#17] -Results [1]: [sum#18] - -(29) CometColumnarExchange -Input [1]: [sum#18] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 7] -Input [1]: [sum#18] - -(31) HashAggregate [codegen id : 7] -Input [1]: [sum#18] -Keys: [] -Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#19] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#19,17,2) AS Excess Discount Amount #20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.date_dim (32) - - -(32) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_date#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [d_date_sk#10, d_date#21] -Condition : (((isnotnull(d_date#21) AND (d_date#21 >= 2000-01-27)) AND (d_date#21 <= 2000-04-26)) AND isnotnull(d_date_sk#10)) - -(34) CometProject -Input [2]: [d_date_sk#10, d_date#21] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(35) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(36) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/simplified.txt deleted file mode 100644 index 17df728966..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/simplified.txt +++ /dev/null @@ -1,54 +0,0 @@ -WholeStageCodegen (7) - HashAggregate [sum] [sum(UnscaledValue(ws_ext_discount_amt)),Excess Discount Amount ,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #1 - WholeStageCodegen (6) - HashAggregate [ws_ext_discount_amt] [sum,sum] - Project [ws_ext_discount_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_discount_amt,ws_sold_date_sk] - BroadcastHashJoin [i_item_sk,ws_item_sk,ws_ext_discount_amt,(1.3 * avg(ws_ext_discount_amt))] - Project [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk,ws_ext_discount_amt] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Filter [(1.3 * avg(ws_ext_discount_amt))] - HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk] #5 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,ws_ext_discount_amt] [sum,count,sum,count] - Project [ws_item_sk,ws_ext_discount_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/explain.txt deleted file mode 100644 index c317a12697..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (30) -+- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : +- CometBroadcastExchange (21) - : +- CometFilter (20) - : +- CometHashAggregate (19) - : +- CometExchange (18) - : +- CometHashAggregate (17) - : +- CometProject (16) - : +- CometBroadcastHashJoin (15) - : :- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (9) - : +- CometBroadcastExchange (14) - : +- CometProject (13) - : +- CometFilter (12) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (11) - +- ReusedExchange (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_ext_discount_amt)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Condition : (isnotnull(ws_item_sk#1) AND isnotnull(ws_ext_discount_amt#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_manufact_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,350), IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_manufact_id#6] -Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 350)) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [2]: [i_item_sk#5, i_manufact_id#6] -Arguments: [i_item_sk#5], [i_item_sk#5] - -(6) CometBroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: [i_item_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Right output [1]: [i_item_sk#5] -Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] -Arguments: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5], [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Condition : isnotnull(ws_item_sk#7) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) - -(13) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(15) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [ws_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(16) CometProject -Input [4]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9, d_date_sk#11] -Arguments: [ws_item_sk#7, ws_ext_discount_amt#8], [ws_item_sk#7, ws_ext_discount_amt#8] - -(17) CometHashAggregate -Input [2]: [ws_item_sk#7, ws_ext_discount_amt#8] -Keys [1]: [ws_item_sk#7] -Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#8))] - -(18) CometExchange -Input [3]: [ws_item_sk#7, sum#13, count#14] -Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [3]: [ws_item_sk#7, sum#13, count#14] -Keys [1]: [ws_item_sk#7] -Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))] - -(20) CometFilter -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#15) - -(21) CometBroadcastExchange -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Arguments: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] - -(22) CometBroadcastHashJoin -Left output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] -Right output [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Arguments: [i_item_sk#5], [ws_item_sk#7], Inner, (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#15), BuildRight - -(23) CometProject -Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Arguments: [ws_ext_discount_amt#2, ws_sold_date_sk#3], [ws_ext_discount_amt#2, ws_sold_date_sk#3] - -(24) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#11] - -(25) CometBroadcastHashJoin -Left output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] -Right output [1]: [d_date_sk#11] -Arguments: [ws_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight - -(26) CometProject -Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#11] -Arguments: [ws_ext_discount_amt#2], [ws_ext_discount_amt#2] - -(27) CometHashAggregate -Input [1]: [ws_ext_discount_amt#2] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] - -(28) CometExchange -Input [1]: [sum#16] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [1]: [sum#16] -Keys: [] -Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [Excess Discount Amount #17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) - -(33) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(35) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 9 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/simplified.txt deleted file mode 100644 index 4ec5755df7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [Excess Discount Amount ,sum(UnscaledValue(ws_ext_discount_amt))] - CometExchange #1 - CometHashAggregate [ws_ext_discount_amt] [sum] - CometProject [ws_ext_discount_amt] - CometBroadcastHashJoin [ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] - CometProject [ws_ext_discount_amt,ws_sold_date_sk] - CometBroadcastHashJoin [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk,(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] - CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk] #3 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] #4 - CometFilter [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] - CometHashAggregate [sum,count] [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,avg(UnscaledValue(ws_ext_discount_amt))] - CometExchange [ws_item_sk] #5 - CometHashAggregate [ws_ext_discount_amt] [ws_item_sk,sum,count] - CometProject [ws_item_sk,ws_ext_discount_amt] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt deleted file mode 100644 index c317a12697..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt +++ /dev/null @@ -1,208 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (30) -+- CometHashAggregate (29) - +- CometExchange (28) - +- CometHashAggregate (27) - +- CometProject (26) - +- CometBroadcastHashJoin (25) - :- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : +- CometBroadcastExchange (21) - : +- CometFilter (20) - : +- CometHashAggregate (19) - : +- CometExchange (18) - : +- CometHashAggregate (17) - : +- CometProject (16) - : +- CometBroadcastHashJoin (15) - : :- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (9) - : +- CometBroadcastExchange (14) - : +- CometProject (13) - : +- CometFilter (12) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (11) - +- ReusedExchange (24) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_ext_discount_amt)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Condition : (isnotnull(ws_item_sk#1) AND isnotnull(ws_ext_discount_amt#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#5, i_manufact_id#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,350), IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [i_item_sk#5, i_manufact_id#6] -Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 350)) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [2]: [i_item_sk#5, i_manufact_id#6] -Arguments: [i_item_sk#5], [i_item_sk#5] - -(6) CometBroadcastExchange -Input [1]: [i_item_sk#5] -Arguments: [i_item_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Right output [1]: [i_item_sk#5] -Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] -Arguments: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5], [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Condition : isnotnull(ws_item_sk#7) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) - -(13) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(14) CometBroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: [d_date_sk#11] - -(15) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] -Right output [1]: [d_date_sk#11] -Arguments: [ws_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(16) CometProject -Input [4]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9, d_date_sk#11] -Arguments: [ws_item_sk#7, ws_ext_discount_amt#8], [ws_item_sk#7, ws_ext_discount_amt#8] - -(17) CometHashAggregate -Input [2]: [ws_item_sk#7, ws_ext_discount_amt#8] -Keys [1]: [ws_item_sk#7] -Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#8))] - -(18) CometExchange -Input [3]: [ws_item_sk#7, sum#13, count#14] -Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(19) CometHashAggregate -Input [3]: [ws_item_sk#7, sum#13, count#14] -Keys [1]: [ws_item_sk#7] -Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))] - -(20) CometFilter -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#15) - -(21) CometBroadcastExchange -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Arguments: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] - -(22) CometBroadcastHashJoin -Left output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] -Right output [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Arguments: [i_item_sk#5], [ws_item_sk#7], Inner, (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#15), BuildRight - -(23) CometProject -Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#7] -Arguments: [ws_ext_discount_amt#2, ws_sold_date_sk#3], [ws_ext_discount_amt#2, ws_sold_date_sk#3] - -(24) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#11] - -(25) CometBroadcastHashJoin -Left output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] -Right output [1]: [d_date_sk#11] -Arguments: [ws_sold_date_sk#3], [d_date_sk#11], Inner, BuildRight - -(26) CometProject -Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#11] -Arguments: [ws_ext_discount_amt#2], [ws_ext_discount_amt#2] - -(27) CometHashAggregate -Input [1]: [ws_ext_discount_amt#2] -Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] - -(28) CometExchange -Input [1]: [sum#16] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [1]: [sum#16] -Keys: [] -Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [Excess Discount Amount #17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#11, d_date#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) - -(33) CometProject -Input [2]: [d_date_sk#11, d_date#12] -Arguments: [d_date_sk#11], [d_date_sk#11] - -(34) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#11] - -(35) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 9 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/extended.txt deleted file mode 100644 index 20df9a8047..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 4ec5755df7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum] [Excess Discount Amount ,sum(UnscaledValue(ws_ext_discount_amt))] - CometExchange #1 - CometHashAggregate [ws_ext_discount_amt] [sum] - CometProject [ws_ext_discount_amt] - CometBroadcastHashJoin [ws_ext_discount_amt,ws_sold_date_sk,d_date_sk] - CometProject [ws_ext_discount_amt,ws_sold_date_sk] - CometBroadcastHashJoin [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk,(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] - CometProject [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk] #3 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] #4 - CometFilter [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk] - CometHashAggregate [sum,count] [(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,avg(UnscaledValue(ws_ext_discount_amt))] - CometExchange [ws_item_sk] #5 - CometHashAggregate [ws_ext_discount_amt] [ws_item_sk,sum,count] - CometProject [ws_item_sk,ws_ext_discount_amt] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_datafusion/explain.txt deleted file mode 100644 index 3f38adcd39..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_datafusion/explain.txt +++ /dev/null @@ -1,124 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (22) -+- CometTakeOrderedAndProject (21) - +- CometHashAggregate (20) - +- CometExchange (19) - +- CometHashAggregate (18) - +- CometProject (17) - +- CometBroadcastHashJoin (16) - :- CometProject (11) - : +- CometSortMergeJoin (10) - : :- CometSort (4) - : : +- CometExchange (3) - : : +- CometProject (2) - : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : +- CometSort (9) - : +- CometExchange (8) - : +- CometProject (7) - : +- CometFilter (6) - : +- CometNativeScan parquet spark_catalog.default.store_returns (5) - +- CometBroadcastExchange (15) - +- CometProject (14) - +- CometFilter (13) - +- CometNativeScan parquet spark_catalog.default.reason (12) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -ReadSchema: struct - -(2) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5], [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] - -(3) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#3 ASC NULLS FIRST] - -(5) CometNativeScan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_reason_sk)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Condition : ((isnotnull(sr_item_sk#7) AND isnotnull(sr_ticket_number#9)) AND isnotnull(sr_reason_sk#8)) - -(7) CometProject -Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] - -(8) CometExchange -Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: hashpartitioning(sr_item_sk#7, sr_ticket_number#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7 ASC NULLS FIRST, sr_ticket_number#9 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Right output [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [ss_item_sk#1, ss_ticket_number#3], [sr_item_sk#7, sr_ticket_number#9], Inner - -(11) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10], [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] - -(12) CometNativeScan parquet spark_catalog.default.reason -Output [2]: [r_reason_sk#12, r_reason_desc#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [r_reason_sk#12, r_reason_desc#13] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, r_reason_desc#13, 100, true, false, true) = reason 28 ) AND isnotnull(r_reason_sk#12)) - -(14) CometProject -Input [2]: [r_reason_sk#12, r_reason_desc#13] -Arguments: [r_reason_sk#12], [r_reason_sk#12] - -(15) CometBroadcastExchange -Input [1]: [r_reason_sk#12] -Arguments: [r_reason_sk#12] - -(16) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] -Right output [1]: [r_reason_sk#12] -Arguments: [sr_reason_sk#8], [r_reason_sk#12], Inner, BuildRight - -(17) CometProject -Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10, r_reason_sk#12] -Arguments: [ss_customer_sk#2, act_sales#14], [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#10) THEN (cast((ss_quantity#4 - sr_return_quantity#10) as decimal(10,0)) * ss_sales_price#5) ELSE (cast(ss_quantity#4 as decimal(10,0)) * ss_sales_price#5) END AS act_sales#14] - -(18) CometHashAggregate -Input [2]: [ss_customer_sk#2, act_sales#14] -Keys [1]: [ss_customer_sk#2] -Functions [1]: [partial_sum(act_sales#14)] - -(19) CometExchange -Input [3]: [ss_customer_sk#2, sum#15, isEmpty#16] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometHashAggregate -Input [3]: [ss_customer_sk#2, sum#15, isEmpty#16] -Keys [1]: [ss_customer_sk#2] -Functions [1]: [sum(act_sales#14)] - -(21) CometTakeOrderedAndProject -Input [2]: [ss_customer_sk#2, sumsales#17] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sumsales#17 ASC NULLS FIRST,ss_customer_sk#2 ASC NULLS FIRST], output=[ss_customer_sk#2,sumsales#17]), [ss_customer_sk#2, sumsales#17], 100, 0, [sumsales#17 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#17] - -(22) CometColumnarToRow [codegen id : 1] -Input [2]: [ss_customer_sk#2, sumsales#17] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_datafusion/simplified.txt deleted file mode 100644 index c0b74010df..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_datafusion/simplified.txt +++ /dev/null @@ -1,24 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ss_customer_sk,sumsales] - CometHashAggregate [sum,isEmpty] [ss_customer_sk,sumsales,sum(act_sales)] - CometExchange [ss_customer_sk] #1 - CometHashAggregate [act_sales] [ss_customer_sk,sum,isEmpty] - CometProject [sr_return_quantity,ss_quantity,ss_sales_price] [ss_customer_sk,act_sales] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity,r_reason_sk] - CometProject [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometExchange [ss_item_sk,ss_ticket_number] #2 - CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometNativeScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometSort [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometExchange [sr_item_sk,sr_ticket_number] #3 - CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometBroadcastExchange [r_reason_sk] #4 - CometProject [r_reason_sk] - CometFilter [r_reason_sk,r_reason_desc] - CometNativeScan 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/q93.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_iceberg_compat/explain.txt deleted file mode 100644 index 799d860c95..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,124 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (22) -+- CometTakeOrderedAndProject (21) - +- CometHashAggregate (20) - +- CometExchange (19) - +- CometHashAggregate (18) - +- CometProject (17) - +- CometBroadcastHashJoin (16) - :- CometProject (11) - : +- CometSortMergeJoin (10) - : :- CometSort (4) - : : +- CometExchange (3) - : : +- CometProject (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometSort (9) - : +- CometExchange (8) - : +- CometProject (7) - : +- CometFilter (6) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - +- CometBroadcastExchange (15) - +- CometProject (14) - +- CometFilter (13) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (12) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -ReadSchema: struct - -(2) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5], [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] - -(3) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#3 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_reason_sk)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Condition : ((isnotnull(sr_item_sk#7) AND isnotnull(sr_ticket_number#9)) AND isnotnull(sr_reason_sk#8)) - -(7) CometProject -Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] - -(8) CometExchange -Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: hashpartitioning(sr_item_sk#7, sr_ticket_number#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7 ASC NULLS FIRST, sr_ticket_number#9 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Right output [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [ss_item_sk#1, ss_ticket_number#3], [sr_item_sk#7, sr_ticket_number#9], Inner - -(11) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10], [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Output [2]: [r_reason_sk#12, r_reason_desc#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [r_reason_sk#12, r_reason_desc#13] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, r_reason_desc#13, 100, true, false, true) = reason 28 ) AND isnotnull(r_reason_sk#12)) - -(14) CometProject -Input [2]: [r_reason_sk#12, r_reason_desc#13] -Arguments: [r_reason_sk#12], [r_reason_sk#12] - -(15) CometBroadcastExchange -Input [1]: [r_reason_sk#12] -Arguments: [r_reason_sk#12] - -(16) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] -Right output [1]: [r_reason_sk#12] -Arguments: [sr_reason_sk#8], [r_reason_sk#12], Inner, BuildRight - -(17) CometProject -Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10, r_reason_sk#12] -Arguments: [ss_customer_sk#2, act_sales#14], [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#10) THEN (cast((ss_quantity#4 - sr_return_quantity#10) as decimal(10,0)) * ss_sales_price#5) ELSE (cast(ss_quantity#4 as decimal(10,0)) * ss_sales_price#5) END AS act_sales#14] - -(18) CometHashAggregate -Input [2]: [ss_customer_sk#2, act_sales#14] -Keys [1]: [ss_customer_sk#2] -Functions [1]: [partial_sum(act_sales#14)] - -(19) CometExchange -Input [3]: [ss_customer_sk#2, sum#15, isEmpty#16] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometHashAggregate -Input [3]: [ss_customer_sk#2, sum#15, isEmpty#16] -Keys [1]: [ss_customer_sk#2] -Functions [1]: [sum(act_sales#14)] - -(21) CometTakeOrderedAndProject -Input [2]: [ss_customer_sk#2, sumsales#17] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sumsales#17 ASC NULLS FIRST,ss_customer_sk#2 ASC NULLS FIRST], output=[ss_customer_sk#2,sumsales#17]), [ss_customer_sk#2, sumsales#17], 100, 0, [sumsales#17 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#17] - -(22) CometColumnarToRow [codegen id : 1] -Input [2]: [ss_customer_sk#2, sumsales#17] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_iceberg_compat/simplified.txt deleted file mode 100644 index d1de4f3475..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,24 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ss_customer_sk,sumsales] - CometHashAggregate [sum,isEmpty] [ss_customer_sk,sumsales,sum(act_sales)] - CometExchange [ss_customer_sk] #1 - CometHashAggregate [act_sales] [ss_customer_sk,sum,isEmpty] - CometProject [sr_return_quantity,ss_quantity,ss_sales_price] [ss_customer_sk,act_sales] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity,r_reason_sk] - CometProject [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometExchange [ss_item_sk,ss_ticket_number] #2 - CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometSort [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometExchange [sr_item_sk,sr_ticket_number] #3 - CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometBroadcastExchange [r_reason_sk] #4 - CometProject [r_reason_sk] - CometFilter [r_reason_sk,r_reason_desc] - CometScan [native_iceberg_compat] 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/q93/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt deleted file mode 100644 index 799d860c95..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt +++ /dev/null @@ -1,124 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (22) -+- CometTakeOrderedAndProject (21) - +- CometHashAggregate (20) - +- CometExchange (19) - +- CometHashAggregate (18) - +- CometProject (17) - +- CometBroadcastHashJoin (16) - :- CometProject (11) - : +- CometSortMergeJoin (10) - : :- CometSort (4) - : : +- CometExchange (3) - : : +- CometProject (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometSort (9) - : +- CometExchange (8) - : +- CometProject (7) - : +- CometFilter (6) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - +- CometBroadcastExchange (15) - +- CometProject (14) - +- CometFilter (13) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason (12) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -ReadSchema: struct - -(2) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5], [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] - -(3) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#3 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_reason_sk)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Condition : ((isnotnull(sr_item_sk#7) AND isnotnull(sr_ticket_number#9)) AND isnotnull(sr_reason_sk#8)) - -(7) CometProject -Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] - -(8) CometExchange -Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: hashpartitioning(sr_item_sk#7, sr_ticket_number#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7 ASC NULLS FIRST, sr_ticket_number#9 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] -Right output [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [ss_item_sk#1, ss_ticket_number#3], [sr_item_sk#7, sr_ticket_number#9], Inner - -(11) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] -Arguments: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10], [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.reason -Output [2]: [r_reason_sk#12, r_reason_desc#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [r_reason_sk#12, r_reason_desc#13] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, r_reason_desc#13, 100, true, false, true) = reason 28 ) AND isnotnull(r_reason_sk#12)) - -(14) CometProject -Input [2]: [r_reason_sk#12, r_reason_desc#13] -Arguments: [r_reason_sk#12], [r_reason_sk#12] - -(15) CometBroadcastExchange -Input [1]: [r_reason_sk#12] -Arguments: [r_reason_sk#12] - -(16) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] -Right output [1]: [r_reason_sk#12] -Arguments: [sr_reason_sk#8], [r_reason_sk#12], Inner, BuildRight - -(17) CometProject -Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10, r_reason_sk#12] -Arguments: [ss_customer_sk#2, act_sales#14], [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#10) THEN (cast((ss_quantity#4 - sr_return_quantity#10) as decimal(10,0)) * ss_sales_price#5) ELSE (cast(ss_quantity#4 as decimal(10,0)) * ss_sales_price#5) END AS act_sales#14] - -(18) CometHashAggregate -Input [2]: [ss_customer_sk#2, act_sales#14] -Keys [1]: [ss_customer_sk#2] -Functions [1]: [partial_sum(act_sales#14)] - -(19) CometExchange -Input [3]: [ss_customer_sk#2, sum#15, isEmpty#16] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometHashAggregate -Input [3]: [ss_customer_sk#2, sum#15, isEmpty#16] -Keys [1]: [ss_customer_sk#2] -Functions [1]: [sum(act_sales#14)] - -(21) CometTakeOrderedAndProject -Input [2]: [ss_customer_sk#2, sumsales#17] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sumsales#17 ASC NULLS FIRST,ss_customer_sk#2 ASC NULLS FIRST], output=[ss_customer_sk#2,sumsales#17]), [ss_customer_sk#2, sumsales#17], 100, 0, [sumsales#17 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#17] - -(22) CometColumnarToRow [codegen id : 1] -Input [2]: [ss_customer_sk#2, sumsales#17] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/extended.txt deleted file mode 100644 index 335f2765d7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/extended.txt +++ /dev/null @@ -1,24 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason - -Comet accelerated 21 out of 21 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d1de4f3475..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/simplified.txt +++ /dev/null @@ -1,24 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ss_customer_sk,sumsales] - CometHashAggregate [sum,isEmpty] [ss_customer_sk,sumsales,sum(act_sales)] - CometExchange [ss_customer_sk] #1 - CometHashAggregate [act_sales] [ss_customer_sk,sum,isEmpty] - CometProject [sr_return_quantity,ss_quantity,ss_sales_price] [ss_customer_sk,act_sales] - CometBroadcastHashJoin [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity,r_reason_sk] - CometProject [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometExchange [ss_item_sk,ss_ticket_number] #2 - CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometSort [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometExchange [sr_item_sk,sr_ticket_number] #3 - CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] - CometFilter [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] - CometBroadcastExchange [r_reason_sk] #4 - CometProject [r_reason_sk] - CometFilter [r_reason_sk,r_reason_desc] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_datafusion/explain.txt deleted file mode 100644 index 1caeb8ca76..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_datafusion/explain.txt +++ /dev/null @@ -1,235 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometColumnarExchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * CometColumnarToRow (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometNativeScan parquet spark_catalog.default.web_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometNativeScan parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometNativeScan parquet spark_catalog.default.web_site (29) - - -(1) CometNativeScan parquet spark_catalog.default.web_sales -Output [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) - -(3) CometProject -Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(4) CometExchange -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.web_sales -Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -ReadSchema: struct - -(7) CometProject -Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] -Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_warehouse_sk#9, ws_order_number#10] - -(8) CometExchange -Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_order_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_order_number#5], [ws_order_number#10], LeftSemi, NOT (ws_warehouse_sk#4 = ws_warehouse_sk#9) - -(11) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(12) CometNativeScan parquet spark_catalog.default.web_returns -Output [2]: [wr_order_number#12, wr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -ReadSchema: struct - -(13) CometProject -Input [2]: [wr_order_number#12, wr_returned_date_sk#13] -Arguments: [wr_order_number#12], [wr_order_number#12] - -(14) CometExchange -Input [1]: [wr_order_number#12] -Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(15) CometSort -Input [1]: [wr_order_number#12] -Arguments: [wr_order_number#12], [wr_order_number#12 ASC NULLS FIRST] - -(16) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [wr_order_number#12] -Arguments: [ws_order_number#5], [wr_order_number#12], LeftAnti - -(17) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-01)) AND (d_date#15 <= 1999-04-02)) AND isnotnull(d_date_sk#14)) - -(19) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(20) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(21) CometBroadcastHashJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [d_date_sk#14] -Arguments: [ws_ship_date_sk#1], [d_date_sk#14], Inner, BuildRight - -(22) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#14] -Arguments: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(23) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#17, 2, true, false, true) = IL) AND isnotnull(ca_address_sk#16)) - -(25) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(26) CometBroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: [ca_address_sk#16] - -(27) CometBroadcastHashJoin -Left output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [ca_address_sk#16] -Arguments: [ws_ship_addr_sk#2], [ca_address_sk#16], Inner, BuildRight - -(28) CometProject -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#16] -Arguments: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(29) CometNativeScan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#18, web_company_name#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [web_site_sk#18, web_company_name#19] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_company_name#19, 50, true, false, true) = pri ) AND isnotnull(web_site_sk#18)) - -(31) CometProject -Input [2]: [web_site_sk#18, web_company_name#19] -Arguments: [web_site_sk#18], [web_site_sk#18] - -(32) CometBroadcastExchange -Input [1]: [web_site_sk#18] -Arguments: [web_site_sk#18] - -(33) CometBroadcastHashJoin -Left output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [web_site_sk#18] -Arguments: [ws_web_site_sk#3], [web_site_sk#18], Inner, BuildRight - -(34) CometProject -Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#18] -Arguments: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(35) CometHashAggregate -Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Keys [1]: [ws_order_number#5] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] - -(36) CometColumnarToRow [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] - -(37) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] -Keys [1]: [ws_order_number#5] -Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23] -Results [3]: [ws_order_number#5, sum#20, sum#21] - -(38) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23, count(ws_order_number#5)#24] -Results [3]: [sum#20, sum#21, count#25] - -(39) CometColumnarExchange -Input [3]: [sum#20, sum#21, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(40) CometHashAggregate -Input [3]: [sum#20, sum#21, count#25] -Keys: [] -Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] - -(41) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #26, total shipping cost #27, total net profit #28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_datafusion/simplified.txt deleted file mode 100644 index feab73bcd6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_datafusion/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometSortMergeJoin [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_order_number,ws_warehouse_sk] - CometSort [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] - CometExchange [ws_order_number] #2 - 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,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometNativeScan 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] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometNativeScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometNativeScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometNativeScan 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/q94.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_iceberg_compat/explain.txt deleted file mode 100644 index 8164e345a6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,235 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometColumnarExchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * CometColumnarToRow (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (29) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) - -(3) CometProject -Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(4) CometExchange -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -ReadSchema: struct - -(7) CometProject -Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] -Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_warehouse_sk#9, ws_order_number#10] - -(8) CometExchange -Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_order_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_order_number#5], [ws_order_number#10], LeftSemi, NOT (ws_warehouse_sk#4 = ws_warehouse_sk#9) - -(11) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [2]: [wr_order_number#12, wr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -ReadSchema: struct - -(13) CometProject -Input [2]: [wr_order_number#12, wr_returned_date_sk#13] -Arguments: [wr_order_number#12], [wr_order_number#12] - -(14) CometExchange -Input [1]: [wr_order_number#12] -Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(15) CometSort -Input [1]: [wr_order_number#12] -Arguments: [wr_order_number#12], [wr_order_number#12 ASC NULLS FIRST] - -(16) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [wr_order_number#12] -Arguments: [ws_order_number#5], [wr_order_number#12], LeftAnti - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-01)) AND (d_date#15 <= 1999-04-02)) AND isnotnull(d_date_sk#14)) - -(19) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(20) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(21) CometBroadcastHashJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [d_date_sk#14] -Arguments: [ws_ship_date_sk#1], [d_date_sk#14], Inner, BuildRight - -(22) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#14] -Arguments: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#17, 2, true, false, true) = IL) AND isnotnull(ca_address_sk#16)) - -(25) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(26) CometBroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: [ca_address_sk#16] - -(27) CometBroadcastHashJoin -Left output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [ca_address_sk#16] -Arguments: [ws_ship_addr_sk#2], [ca_address_sk#16], Inner, BuildRight - -(28) CometProject -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#16] -Arguments: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#18, web_company_name#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [web_site_sk#18, web_company_name#19] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_company_name#19, 50, true, false, true) = pri ) AND isnotnull(web_site_sk#18)) - -(31) CometProject -Input [2]: [web_site_sk#18, web_company_name#19] -Arguments: [web_site_sk#18], [web_site_sk#18] - -(32) CometBroadcastExchange -Input [1]: [web_site_sk#18] -Arguments: [web_site_sk#18] - -(33) CometBroadcastHashJoin -Left output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [web_site_sk#18] -Arguments: [ws_web_site_sk#3], [web_site_sk#18], Inner, BuildRight - -(34) CometProject -Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#18] -Arguments: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(35) CometHashAggregate -Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Keys [1]: [ws_order_number#5] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] - -(36) CometColumnarToRow [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] - -(37) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] -Keys [1]: [ws_order_number#5] -Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23] -Results [3]: [ws_order_number#5, sum#20, sum#21] - -(38) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23, count(ws_order_number#5)#24] -Results [3]: [sum#20, sum#21, count#25] - -(39) CometColumnarExchange -Input [3]: [sum#20, sum#21, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(40) CometHashAggregate -Input [3]: [sum#20, sum#21, count#25] -Keys: [] -Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] - -(41) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #26, total shipping cost #27, total net profit #28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_iceberg_compat/simplified.txt deleted file mode 100644 index 629178d106..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometSortMergeJoin [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_order_number,ws_warehouse_sk] - CometSort [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] - CometExchange [ws_order_number] #2 - 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,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometScan [native_iceberg_compat] 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/q94/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt deleted file mode 100644 index 8164e345a6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ /dev/null @@ -1,235 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometColumnarExchange (39) - +- * HashAggregate (38) - +- * HashAggregate (37) - +- * CometColumnarToRow (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (28) - : +- CometBroadcastHashJoin (27) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometSortMergeJoin (16) - : : : :- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (6) - : : : +- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (12) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometFilter (18) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (17) - : +- CometBroadcastExchange (26) - : +- CometProject (25) - : +- CometFilter (24) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (23) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometFilter (30) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (29) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) - -(3) CometProject -Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(4) CometExchange -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -ReadSchema: struct - -(7) CometProject -Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] -Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_warehouse_sk#9, ws_order_number#10] - -(8) CometExchange -Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_order_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_order_number#5], [ws_order_number#10], LeftSemi, NOT (ws_warehouse_sk#4 = ws_warehouse_sk#9) - -(11) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [2]: [wr_order_number#12, wr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -ReadSchema: struct - -(13) CometProject -Input [2]: [wr_order_number#12, wr_returned_date_sk#13] -Arguments: [wr_order_number#12], [wr_order_number#12] - -(14) CometExchange -Input [1]: [wr_order_number#12] -Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(15) CometSort -Input [1]: [wr_order_number#12] -Arguments: [wr_order_number#12], [wr_order_number#12 ASC NULLS FIRST] - -(16) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [wr_order_number#12] -Arguments: [ws_order_number#5], [wr_order_number#12], LeftAnti - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(18) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-01)) AND (d_date#15 <= 1999-04-02)) AND isnotnull(d_date_sk#14)) - -(19) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(20) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(21) CometBroadcastHashJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [d_date_sk#14] -Arguments: [ws_ship_date_sk#1], [d_date_sk#14], Inner, BuildRight - -(22) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#14] -Arguments: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(24) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#17, 2, true, false, true) = IL) AND isnotnull(ca_address_sk#16)) - -(25) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(26) CometBroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: [ca_address_sk#16] - -(27) CometBroadcastHashJoin -Left output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [ca_address_sk#16] -Arguments: [ws_ship_addr_sk#2], [ca_address_sk#16], Inner, BuildRight - -(28) CometProject -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#16] -Arguments: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#18, web_company_name#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [web_site_sk#18, web_company_name#19] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_company_name#19, 50, true, false, true) = pri ) AND isnotnull(web_site_sk#18)) - -(31) CometProject -Input [2]: [web_site_sk#18, web_company_name#19] -Arguments: [web_site_sk#18], [web_site_sk#18] - -(32) CometBroadcastExchange -Input [1]: [web_site_sk#18] -Arguments: [web_site_sk#18] - -(33) CometBroadcastHashJoin -Left output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Right output [1]: [web_site_sk#18] -Arguments: [ws_web_site_sk#3], [web_site_sk#18], Inner, BuildRight - -(34) CometProject -Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#18] -Arguments: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] - -(35) CometHashAggregate -Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Keys [1]: [ws_order_number#5] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] - -(36) CometColumnarToRow [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] - -(37) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] -Keys [1]: [ws_order_number#5] -Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23] -Results [3]: [ws_order_number#5, sum#20, sum#21] - -(38) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#5, sum#20, sum#21] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23, count(ws_order_number#5)#24] -Results [3]: [sum#20, sum#21, count#25] - -(39) CometColumnarExchange -Input [3]: [sum#20, sum#21, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(40) CometHashAggregate -Input [3]: [sum#20, sum#21, count#25] -Keys: [] -Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] - -(41) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #26, total shipping cost #27, total net profit #28] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/extended.txt deleted file mode 100644 index eac4939621..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/extended.txt +++ /dev/null @@ -1,43 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - -Comet accelerated 37 out of 39 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 629178d106..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/simplified.txt +++ /dev/null @@ -1,45 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometProject [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometSortMergeJoin [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_order_number,ws_warehouse_sk] - CometSort [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] - CometExchange [ws_order_number] #2 - 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,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_datafusion/explain.txt deleted file mode 100644 index e49aacd1b5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_datafusion/explain.txt +++ /dev/null @@ -1,302 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (54) -+- CometHashAggregate (53) - +- CometColumnarExchange (52) - +- * HashAggregate (51) - +- * HashAggregate (50) - +- * CometColumnarToRow (49) - +- CometHashAggregate (48) - +- CometProject (47) - +- CometBroadcastHashJoin (46) - :- CometProject (41) - : +- CometBroadcastHashJoin (40) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometSortMergeJoin (29) - : : : :- CometSortMergeJoin (15) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales (1) - : : : : +- CometProject (14) - : : : : +- CometSortMergeJoin (13) - : : : : :- CometSort (10) - : : : : : +- CometExchange (9) - : : : : : +- CometProject (8) - : : : : : +- CometFilter (7) - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales (6) - : : : : +- CometSort (12) - : : : : +- ReusedExchange (11) - : : : +- CometProject (28) - : : : +- CometSortMergeJoin (27) - : : : :- CometSort (20) - : : : : +- CometExchange (19) - : : : : +- CometProject (18) - : : : : +- CometFilter (17) - : : : : +- CometNativeScan parquet spark_catalog.default.web_returns (16) - : : : +- CometProject (26) - : : : +- CometSortMergeJoin (25) - : : : :- CometSort (22) - : : : : +- ReusedExchange (21) - : : : +- CometSort (24) - : : : +- ReusedExchange (23) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (30) - : +- CometBroadcastExchange (39) - : +- CometProject (38) - : +- CometFilter (37) - : +- CometNativeScan parquet spark_catalog.default.customer_address (36) - +- CometBroadcastExchange (45) - +- CometProject (44) - +- CometFilter (43) - +- CometNativeScan parquet spark_catalog.default.web_site (42) - - -(1) CometNativeScan parquet spark_catalog.default.web_sales -Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) - -(3) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(4) CometExchange -Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.web_sales -Output [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Condition : (isnotnull(ws_order_number#9) AND isnotnull(ws_warehouse_sk#8)) - -(8) CometProject -Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_warehouse_sk#8, ws_order_number#9] - -(9) CometExchange -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: hashpartitioning(ws_order_number#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_order_number#9 ASC NULLS FIRST] - -(11) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#11, ws_order_number#12] - -(12) CometSort -Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_warehouse_sk#11, ws_order_number#12], [ws_order_number#12 ASC NULLS FIRST] - -(13) CometSortMergeJoin -Left output [2]: [ws_warehouse_sk#8, ws_order_number#9] -Right output [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#12], Inner, NOT (ws_warehouse_sk#8 = ws_warehouse_sk#11) - -(14) CometProject -Input [4]: [ws_warehouse_sk#8, ws_order_number#9, ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#9] - -(15) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [ws_order_number#9] -Arguments: [ws_order_number#4], [ws_order_number#9], LeftSemi - -(16) CometNativeScan parquet spark_catalog.default.web_returns -Output [2]: [wr_order_number#13, wr_returned_date_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [wr_order_number#13, wr_returned_date_sk#14] -Condition : isnotnull(wr_order_number#13) - -(18) CometProject -Input [2]: [wr_order_number#13, wr_returned_date_sk#14] -Arguments: [wr_order_number#13], [wr_order_number#13] - -(19) CometExchange -Input [1]: [wr_order_number#13] -Arguments: hashpartitioning(wr_order_number#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [1]: [wr_order_number#13] -Arguments: [wr_order_number#13], [wr_order_number#13 ASC NULLS FIRST] - -(21) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#8, ws_order_number#9] - -(22) CometSort -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_order_number#9 ASC NULLS FIRST] - -(23) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#11, ws_order_number#12] - -(24) CometSort -Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_warehouse_sk#11, ws_order_number#12], [ws_order_number#12 ASC NULLS FIRST] - -(25) CometSortMergeJoin -Left output [2]: [ws_warehouse_sk#8, ws_order_number#9] -Right output [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#12], Inner, NOT (ws_warehouse_sk#8 = ws_warehouse_sk#11) - -(26) CometProject -Input [4]: [ws_warehouse_sk#8, ws_order_number#9, ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#9] - -(27) CometSortMergeJoin -Left output [1]: [wr_order_number#13] -Right output [1]: [ws_order_number#9] -Arguments: [wr_order_number#13], [ws_order_number#9], Inner - -(28) CometProject -Input [2]: [wr_order_number#13, ws_order_number#9] -Arguments: [wr_order_number#13], [wr_order_number#13] - -(29) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [wr_order_number#13] -Arguments: [ws_order_number#4], [wr_order_number#13], LeftSemi - -(30) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#15, d_date#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [d_date_sk#15, d_date#16] -Condition : (((isnotnull(d_date#16) AND (d_date#16 >= 1999-02-01)) AND (d_date#16 <= 1999-04-02)) AND isnotnull(d_date_sk#15)) - -(32) CometProject -Input [2]: [d_date_sk#15, d_date#16] -Arguments: [d_date_sk#15], [d_date_sk#15] - -(33) CometBroadcastExchange -Input [1]: [d_date_sk#15] -Arguments: [d_date_sk#15] - -(34) CometBroadcastHashJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [d_date_sk#15] -Arguments: [ws_ship_date_sk#1], [d_date_sk#15], Inner, BuildRight - -(35) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#15] -Arguments: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(36) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#17, ca_state#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [ca_address_sk#17, ca_state#18] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#18, 2, true, false, true) = IL) AND isnotnull(ca_address_sk#17)) - -(38) CometProject -Input [2]: [ca_address_sk#17, ca_state#18] -Arguments: [ca_address_sk#17], [ca_address_sk#17] - -(39) CometBroadcastExchange -Input [1]: [ca_address_sk#17] -Arguments: [ca_address_sk#17] - -(40) CometBroadcastHashJoin -Left output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [ca_address_sk#17] -Arguments: [ws_ship_addr_sk#2], [ca_address_sk#17], Inner, BuildRight - -(41) CometProject -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#17] -Arguments: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(42) CometNativeScan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#19, web_company_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(43) CometFilter -Input [2]: [web_site_sk#19, web_company_name#20] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_company_name#20, 50, true, false, true) = pri ) AND isnotnull(web_site_sk#19)) - -(44) CometProject -Input [2]: [web_site_sk#19, web_company_name#20] -Arguments: [web_site_sk#19], [web_site_sk#19] - -(45) CometBroadcastExchange -Input [1]: [web_site_sk#19] -Arguments: [web_site_sk#19] - -(46) CometBroadcastHashJoin -Left output [4]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [web_site_sk#19] -Arguments: [ws_web_site_sk#3], [web_site_sk#19], Inner, BuildRight - -(47) CometProject -Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#19] -Arguments: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(48) CometHashAggregate -Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Keys [1]: [ws_order_number#4] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] - -(49) CometColumnarToRow [codegen id : 1] -Input [3]: [ws_order_number#4, sum#21, sum#22] - -(50) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#4, sum#21, sum#22] -Keys [1]: [ws_order_number#4] -Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#23, sum(UnscaledValue(ws_net_profit#6))#24] -Results [3]: [ws_order_number#4, sum#21, sum#22] - -(51) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#4, sum#21, sum#22] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#23, sum(UnscaledValue(ws_net_profit#6))#24, count(ws_order_number#4)#25] -Results [3]: [sum#21, sum#22, count#26] - -(52) CometColumnarExchange -Input [3]: [sum#21, sum#22, count#26] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(53) CometHashAggregate -Input [3]: [sum#21, sum#22, count#26] -Keys: [] -Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] - -(54) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #27, total shipping cost #28, total net profit #29] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_datafusion/simplified.txt deleted file mode 100644 index 15f29c507c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_datafusion/simplified.txt +++ /dev/null @@ -1,58 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number] - CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometExchange [ws_order_number] #2 - 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,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometNativeScan 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] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometProject [wr_order_number] - CometSortMergeJoin [wr_order_number,ws_order_number] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometFilter [wr_order_number,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometNativeScan 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.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_iceberg_compat/explain.txt deleted file mode 100644 index 5c319928ea..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,302 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (54) -+- CometHashAggregate (53) - +- CometColumnarExchange (52) - +- * HashAggregate (51) - +- * HashAggregate (50) - +- * CometColumnarToRow (49) - +- CometHashAggregate (48) - +- CometProject (47) - +- CometBroadcastHashJoin (46) - :- CometProject (41) - : +- CometBroadcastHashJoin (40) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometSortMergeJoin (29) - : : : :- CometSortMergeJoin (15) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometProject (14) - : : : : +- CometSortMergeJoin (13) - : : : : :- CometSort (10) - : : : : : +- CometExchange (9) - : : : : : +- CometProject (8) - : : : : : +- CometFilter (7) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (6) - : : : : +- CometSort (12) - : : : : +- ReusedExchange (11) - : : : +- CometProject (28) - : : : +- CometSortMergeJoin (27) - : : : :- CometSort (20) - : : : : +- CometExchange (19) - : : : : +- CometProject (18) - : : : : +- CometFilter (17) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (16) - : : : +- CometProject (26) - : : : +- CometSortMergeJoin (25) - : : : :- CometSort (22) - : : : : +- ReusedExchange (21) - : : : +- CometSort (24) - : : : +- ReusedExchange (23) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - : +- CometBroadcastExchange (39) - : +- CometProject (38) - : +- CometFilter (37) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (36) - +- CometBroadcastExchange (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (42) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) - -(3) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(4) CometExchange -Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Condition : (isnotnull(ws_order_number#9) AND isnotnull(ws_warehouse_sk#8)) - -(8) CometProject -Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_warehouse_sk#8, ws_order_number#9] - -(9) CometExchange -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: hashpartitioning(ws_order_number#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_order_number#9 ASC NULLS FIRST] - -(11) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#11, ws_order_number#12] - -(12) CometSort -Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_warehouse_sk#11, ws_order_number#12], [ws_order_number#12 ASC NULLS FIRST] - -(13) CometSortMergeJoin -Left output [2]: [ws_warehouse_sk#8, ws_order_number#9] -Right output [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#12], Inner, NOT (ws_warehouse_sk#8 = ws_warehouse_sk#11) - -(14) CometProject -Input [4]: [ws_warehouse_sk#8, ws_order_number#9, ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#9] - -(15) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [ws_order_number#9] -Arguments: [ws_order_number#4], [ws_order_number#9], LeftSemi - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [2]: [wr_order_number#13, wr_returned_date_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [wr_order_number#13, wr_returned_date_sk#14] -Condition : isnotnull(wr_order_number#13) - -(18) CometProject -Input [2]: [wr_order_number#13, wr_returned_date_sk#14] -Arguments: [wr_order_number#13], [wr_order_number#13] - -(19) CometExchange -Input [1]: [wr_order_number#13] -Arguments: hashpartitioning(wr_order_number#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [1]: [wr_order_number#13] -Arguments: [wr_order_number#13], [wr_order_number#13 ASC NULLS FIRST] - -(21) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#8, ws_order_number#9] - -(22) CometSort -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_order_number#9 ASC NULLS FIRST] - -(23) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#11, ws_order_number#12] - -(24) CometSort -Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_warehouse_sk#11, ws_order_number#12], [ws_order_number#12 ASC NULLS FIRST] - -(25) CometSortMergeJoin -Left output [2]: [ws_warehouse_sk#8, ws_order_number#9] -Right output [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#12], Inner, NOT (ws_warehouse_sk#8 = ws_warehouse_sk#11) - -(26) CometProject -Input [4]: [ws_warehouse_sk#8, ws_order_number#9, ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#9] - -(27) CometSortMergeJoin -Left output [1]: [wr_order_number#13] -Right output [1]: [ws_order_number#9] -Arguments: [wr_order_number#13], [ws_order_number#9], Inner - -(28) CometProject -Input [2]: [wr_order_number#13, ws_order_number#9] -Arguments: [wr_order_number#13], [wr_order_number#13] - -(29) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [wr_order_number#13] -Arguments: [ws_order_number#4], [wr_order_number#13], LeftSemi - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#15, d_date#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [d_date_sk#15, d_date#16] -Condition : (((isnotnull(d_date#16) AND (d_date#16 >= 1999-02-01)) AND (d_date#16 <= 1999-04-02)) AND isnotnull(d_date_sk#15)) - -(32) CometProject -Input [2]: [d_date_sk#15, d_date#16] -Arguments: [d_date_sk#15], [d_date_sk#15] - -(33) CometBroadcastExchange -Input [1]: [d_date_sk#15] -Arguments: [d_date_sk#15] - -(34) CometBroadcastHashJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [d_date_sk#15] -Arguments: [ws_ship_date_sk#1], [d_date_sk#15], Inner, BuildRight - -(35) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#15] -Arguments: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#17, ca_state#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [ca_address_sk#17, ca_state#18] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#18, 2, true, false, true) = IL) AND isnotnull(ca_address_sk#17)) - -(38) CometProject -Input [2]: [ca_address_sk#17, ca_state#18] -Arguments: [ca_address_sk#17], [ca_address_sk#17] - -(39) CometBroadcastExchange -Input [1]: [ca_address_sk#17] -Arguments: [ca_address_sk#17] - -(40) CometBroadcastHashJoin -Left output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [ca_address_sk#17] -Arguments: [ws_ship_addr_sk#2], [ca_address_sk#17], Inner, BuildRight - -(41) CometProject -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#17] -Arguments: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#19, web_company_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(43) CometFilter -Input [2]: [web_site_sk#19, web_company_name#20] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_company_name#20, 50, true, false, true) = pri ) AND isnotnull(web_site_sk#19)) - -(44) CometProject -Input [2]: [web_site_sk#19, web_company_name#20] -Arguments: [web_site_sk#19], [web_site_sk#19] - -(45) CometBroadcastExchange -Input [1]: [web_site_sk#19] -Arguments: [web_site_sk#19] - -(46) CometBroadcastHashJoin -Left output [4]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [web_site_sk#19] -Arguments: [ws_web_site_sk#3], [web_site_sk#19], Inner, BuildRight - -(47) CometProject -Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#19] -Arguments: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(48) CometHashAggregate -Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Keys [1]: [ws_order_number#4] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] - -(49) CometColumnarToRow [codegen id : 1] -Input [3]: [ws_order_number#4, sum#21, sum#22] - -(50) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#4, sum#21, sum#22] -Keys [1]: [ws_order_number#4] -Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#23, sum(UnscaledValue(ws_net_profit#6))#24] -Results [3]: [ws_order_number#4, sum#21, sum#22] - -(51) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#4, sum#21, sum#22] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#23, sum(UnscaledValue(ws_net_profit#6))#24, count(ws_order_number#4)#25] -Results [3]: [sum#21, sum#22, count#26] - -(52) CometColumnarExchange -Input [3]: [sum#21, sum#22, count#26] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(53) CometHashAggregate -Input [3]: [sum#21, sum#22, count#26] -Keys: [] -Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] - -(54) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #27, total shipping cost #28, total net profit #29] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_iceberg_compat/simplified.txt deleted file mode 100644 index 6c1b1bccfa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,58 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number] - CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometExchange [ws_order_number] #2 - 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,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometProject [wr_order_number] - CometSortMergeJoin [wr_order_number,ws_order_number] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometFilter [wr_order_number,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometScan [native_iceberg_compat] 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/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt deleted file mode 100644 index 5c319928ea..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/explain.txt +++ /dev/null @@ -1,302 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (54) -+- CometHashAggregate (53) - +- CometColumnarExchange (52) - +- * HashAggregate (51) - +- * HashAggregate (50) - +- * CometColumnarToRow (49) - +- CometHashAggregate (48) - +- CometProject (47) - +- CometBroadcastHashJoin (46) - :- CometProject (41) - : +- CometBroadcastHashJoin (40) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometSortMergeJoin (29) - : : : :- CometSortMergeJoin (15) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : : +- CometProject (14) - : : : : +- CometSortMergeJoin (13) - : : : : :- CometSort (10) - : : : : : +- CometExchange (9) - : : : : : +- CometProject (8) - : : : : : +- CometFilter (7) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (6) - : : : : +- CometSort (12) - : : : : +- ReusedExchange (11) - : : : +- CometProject (28) - : : : +- CometSortMergeJoin (27) - : : : :- CometSort (20) - : : : : +- CometExchange (19) - : : : : +- CometProject (18) - : : : : +- CometFilter (17) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (16) - : : : +- CometProject (26) - : : : +- CometSortMergeJoin (25) - : : : :- CometSort (22) - : : : : +- ReusedExchange (21) - : : : +- CometSort (24) - : : : +- ReusedExchange (23) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - : +- CometBroadcastExchange (39) - : +- CometProject (38) - : +- CometFilter (37) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (36) - +- CometBroadcastExchange (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (42) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) - -(3) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(4) CometExchange -Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Condition : (isnotnull(ws_order_number#9) AND isnotnull(ws_warehouse_sk#8)) - -(8) CometProject -Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_warehouse_sk#8, ws_order_number#9] - -(9) CometExchange -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: hashpartitioning(ws_order_number#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_order_number#9 ASC NULLS FIRST] - -(11) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#11, ws_order_number#12] - -(12) CometSort -Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_warehouse_sk#11, ws_order_number#12], [ws_order_number#12 ASC NULLS FIRST] - -(13) CometSortMergeJoin -Left output [2]: [ws_warehouse_sk#8, ws_order_number#9] -Right output [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#12], Inner, NOT (ws_warehouse_sk#8 = ws_warehouse_sk#11) - -(14) CometProject -Input [4]: [ws_warehouse_sk#8, ws_order_number#9, ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#9] - -(15) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [ws_order_number#9] -Arguments: [ws_order_number#4], [ws_order_number#9], LeftSemi - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [2]: [wr_order_number#13, wr_returned_date_sk#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [wr_order_number#13, wr_returned_date_sk#14] -Condition : isnotnull(wr_order_number#13) - -(18) CometProject -Input [2]: [wr_order_number#13, wr_returned_date_sk#14] -Arguments: [wr_order_number#13], [wr_order_number#13] - -(19) CometExchange -Input [1]: [wr_order_number#13] -Arguments: hashpartitioning(wr_order_number#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [1]: [wr_order_number#13] -Arguments: [wr_order_number#13], [wr_order_number#13 ASC NULLS FIRST] - -(21) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#8, ws_order_number#9] - -(22) CometSort -Input [2]: [ws_warehouse_sk#8, ws_order_number#9] -Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_order_number#9 ASC NULLS FIRST] - -(23) ReusedExchange [Reuses operator id: 9] -Output [2]: [ws_warehouse_sk#11, ws_order_number#12] - -(24) CometSort -Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_warehouse_sk#11, ws_order_number#12], [ws_order_number#12 ASC NULLS FIRST] - -(25) CometSortMergeJoin -Left output [2]: [ws_warehouse_sk#8, ws_order_number#9] -Right output [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#12], Inner, NOT (ws_warehouse_sk#8 = ws_warehouse_sk#11) - -(26) CometProject -Input [4]: [ws_warehouse_sk#8, ws_order_number#9, ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#9], [ws_order_number#9] - -(27) CometSortMergeJoin -Left output [1]: [wr_order_number#13] -Right output [1]: [ws_order_number#9] -Arguments: [wr_order_number#13], [ws_order_number#9], Inner - -(28) CometProject -Input [2]: [wr_order_number#13, ws_order_number#9] -Arguments: [wr_order_number#13], [wr_order_number#13] - -(29) CometSortMergeJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [wr_order_number#13] -Arguments: [ws_order_number#4], [wr_order_number#13], LeftSemi - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#15, d_date#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [d_date_sk#15, d_date#16] -Condition : (((isnotnull(d_date#16) AND (d_date#16 >= 1999-02-01)) AND (d_date#16 <= 1999-04-02)) AND isnotnull(d_date_sk#15)) - -(32) CometProject -Input [2]: [d_date_sk#15, d_date#16] -Arguments: [d_date_sk#15], [d_date_sk#15] - -(33) CometBroadcastExchange -Input [1]: [d_date_sk#15] -Arguments: [d_date_sk#15] - -(34) CometBroadcastHashJoin -Left output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [d_date_sk#15] -Arguments: [ws_ship_date_sk#1], [d_date_sk#15], Inner, BuildRight - -(35) CometProject -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#15] -Arguments: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#17, ca_state#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [ca_address_sk#17, ca_state#18] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#18, 2, true, false, true) = IL) AND isnotnull(ca_address_sk#17)) - -(38) CometProject -Input [2]: [ca_address_sk#17, ca_state#18] -Arguments: [ca_address_sk#17], [ca_address_sk#17] - -(39) CometBroadcastExchange -Input [1]: [ca_address_sk#17] -Arguments: [ca_address_sk#17] - -(40) CometBroadcastHashJoin -Left output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [ca_address_sk#17] -Arguments: [ws_ship_addr_sk#2], [ca_address_sk#17], Inner, BuildRight - -(41) CometProject -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#17] -Arguments: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#19, web_company_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(43) CometFilter -Input [2]: [web_site_sk#19, web_company_name#20] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_company_name#20, 50, true, false, true) = pri ) AND isnotnull(web_site_sk#19)) - -(44) CometProject -Input [2]: [web_site_sk#19, web_company_name#20] -Arguments: [web_site_sk#19], [web_site_sk#19] - -(45) CometBroadcastExchange -Input [1]: [web_site_sk#19] -Arguments: [web_site_sk#19] - -(46) CometBroadcastHashJoin -Left output [4]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Right output [1]: [web_site_sk#19] -Arguments: [ws_web_site_sk#3], [web_site_sk#19], Inner, BuildRight - -(47) CometProject -Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#19] -Arguments: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] - -(48) CometHashAggregate -Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] -Keys [1]: [ws_order_number#4] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] - -(49) CometColumnarToRow [codegen id : 1] -Input [3]: [ws_order_number#4, sum#21, sum#22] - -(50) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#4, sum#21, sum#22] -Keys [1]: [ws_order_number#4] -Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#23, sum(UnscaledValue(ws_net_profit#6))#24] -Results [3]: [ws_order_number#4, sum#21, sum#22] - -(51) HashAggregate [codegen id : 1] -Input [3]: [ws_order_number#4, sum#21, sum#22] -Keys: [] -Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#23, sum(UnscaledValue(ws_net_profit#6))#24, count(ws_order_number#4)#25] -Results [3]: [sum#21, sum#22, count#26] - -(52) CometColumnarExchange -Input [3]: [sum#21, sum#22, count#26] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(53) CometHashAggregate -Input [3]: [sum#21, sum#22, count#26] -Keys: [] -Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] - -(54) CometColumnarToRow [codegen id : 2] -Input [3]: [order count #27, total shipping cost #28, total net profit #29] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/extended.txt deleted file mode 100644 index 6ff8eba58f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] - +- CometColumnarToRow - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometProject - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- CometProject - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - -Comet accelerated 59 out of 61 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 6c1b1bccfa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/simplified.txt +++ /dev/null @@ -1,58 +0,0 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] - CometColumnarExchange #1 - WholeStageCodegen (1) - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] - HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometHashAggregate [ws_ext_ship_cost,ws_net_profit] [ws_order_number,sum,sum] - CometProject [ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,web_site_sk] - CometProject [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ca_address_sk] - CometProject [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometBroadcastHashJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,d_date_sk] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,wr_order_number] - CometSortMergeJoin [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_order_number] - CometSort [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] - CometExchange [ws_order_number] #2 - 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,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - CometExchange [ws_order_number] #3 - CometProject [ws_warehouse_sk,ws_order_number] - CometFilter [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometProject [wr_order_number] - CometSortMergeJoin [wr_order_number,ws_order_number] - CometSort [wr_order_number] - CometExchange [wr_order_number] #4 - CometProject [wr_order_number] - CometFilter [wr_order_number,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] - CometProject [ws_order_number] - CometSortMergeJoin [ws_warehouse_sk,ws_order_number,ws_warehouse_sk,ws_order_number] - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometSort [ws_warehouse_sk,ws_order_number] - ReusedExchange [ws_warehouse_sk,ws_order_number] #3 - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [web_site_sk] #7 - CometProject [web_site_sk] - CometFilter [web_site_sk,web_company_name] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_datafusion/explain.txt deleted file mode 100644 index db701aae34..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_datafusion/explain.txt +++ /dev/null @@ -1,143 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometNativeScan parquet spark_catalog.default.household_demographics (4) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometNativeScan parquet spark_catalog.default.time_dim (10) - +- CometBroadcastExchange (19) - +- CometProject (18) - +- CometFilter (17) - +- CometNativeScan parquet spark_catalog.default.store (16) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(3) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(4) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#5, hd_dep_count#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,7), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 7)) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] -Arguments: [ss_sold_time_sk#1, ss_store_sk#3], [ss_sold_time_sk#1, ss_store_sk#3] - -(10) CometNativeScan parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,20), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Condition : ((((isnotnull(t_hour#8) AND isnotnull(t_minute#9)) AND (t_hour#8 = 20)) AND (t_minute#9 >= 30)) AND isnotnull(t_time_sk#7)) - -(12) CometProject -Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Arguments: [t_time_sk#7], [t_time_sk#7] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#7] -Arguments: [t_time_sk#7] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Right output [1]: [t_time_sk#7] -Arguments: [ss_sold_time_sk#1], [t_time_sk#7], Inner, BuildRight - -(15) CometProject -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#7] -Arguments: [ss_store_sk#3], [ss_store_sk#3] - -(16) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_store_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [s_store_sk#10, s_store_name#11] -Condition : ((isnotnull(s_store_name#11) AND (s_store_name#11 = ese)) AND isnotnull(s_store_sk#10)) - -(18) CometProject -Input [2]: [s_store_sk#10, s_store_name#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(19) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(20) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#3] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(21) CometProject -Input [2]: [ss_store_sk#3, s_store_sk#10] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#12] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 1] -Input [1]: [count(1)#13] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_datafusion/simplified.txt deleted file mode 100644 index b40f41659f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_datafusion/simplified.txt +++ /dev/null @@ -1,27 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometNativeScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometNativeScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometNativeScan 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/q96.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_iceberg_compat/explain.txt deleted file mode 100644 index f90d8b12b9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,143 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (4) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (10) - +- CometBroadcastExchange (19) - +- CometProject (18) - +- CometFilter (17) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (16) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(3) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#5, hd_dep_count#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,7), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 7)) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] -Arguments: [ss_sold_time_sk#1, ss_store_sk#3], [ss_sold_time_sk#1, ss_store_sk#3] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,20), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Condition : ((((isnotnull(t_hour#8) AND isnotnull(t_minute#9)) AND (t_hour#8 = 20)) AND (t_minute#9 >= 30)) AND isnotnull(t_time_sk#7)) - -(12) CometProject -Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Arguments: [t_time_sk#7], [t_time_sk#7] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#7] -Arguments: [t_time_sk#7] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Right output [1]: [t_time_sk#7] -Arguments: [ss_sold_time_sk#1], [t_time_sk#7], Inner, BuildRight - -(15) CometProject -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#7] -Arguments: [ss_store_sk#3], [ss_store_sk#3] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_store_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [s_store_sk#10, s_store_name#11] -Condition : ((isnotnull(s_store_name#11) AND (s_store_name#11 = ese)) AND isnotnull(s_store_sk#10)) - -(18) CometProject -Input [2]: [s_store_sk#10, s_store_name#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(19) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(20) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#3] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(21) CometProject -Input [2]: [ss_store_sk#3, s_store_sk#10] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#12] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 1] -Input [1]: [count(1)#13] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_iceberg_compat/simplified.txt deleted file mode 100644 index bbbd07245e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,27 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometScan [native_iceberg_compat] 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/q96/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/explain.txt deleted file mode 100644 index f90d8b12b9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/explain.txt +++ /dev/null @@ -1,143 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (7) - : : +- CometProject (6) - : : +- CometFilter (5) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (4) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (10) - +- CometBroadcastExchange (19) - +- CometProject (18) - +- CometFilter (17) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (16) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) - -(3) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#5, hd_dep_count#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,7), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 7)) AND isnotnull(hd_demo_sk#5)) - -(6) CometProject -Input [2]: [hd_demo_sk#5, hd_dep_count#6] -Arguments: [hd_demo_sk#5], [hd_demo_sk#5] - -(7) CometBroadcastExchange -Input [1]: [hd_demo_sk#5] -Arguments: [hd_demo_sk#5] - -(8) CometBroadcastHashJoin -Left output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Right output [1]: [hd_demo_sk#5] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#5], Inner, BuildRight - -(9) CometProject -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] -Arguments: [ss_sold_time_sk#1, ss_store_sk#3], [ss_sold_time_sk#1, ss_store_sk#3] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Output [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,20), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Condition : ((((isnotnull(t_hour#8) AND isnotnull(t_minute#9)) AND (t_hour#8 = 20)) AND (t_minute#9 >= 30)) AND isnotnull(t_time_sk#7)) - -(12) CometProject -Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] -Arguments: [t_time_sk#7], [t_time_sk#7] - -(13) CometBroadcastExchange -Input [1]: [t_time_sk#7] -Arguments: [t_time_sk#7] - -(14) CometBroadcastHashJoin -Left output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Right output [1]: [t_time_sk#7] -Arguments: [ss_sold_time_sk#1], [t_time_sk#7], Inner, BuildRight - -(15) CometProject -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#7] -Arguments: [ss_store_sk#3], [ss_store_sk#3] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_store_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [s_store_sk#10, s_store_name#11] -Condition : ((isnotnull(s_store_name#11) AND (s_store_name#11 = ese)) AND isnotnull(s_store_sk#10)) - -(18) CometProject -Input [2]: [s_store_sk#10, s_store_name#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(19) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(20) CometBroadcastHashJoin -Left output [1]: [ss_store_sk#3] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(21) CometProject -Input [2]: [ss_store_sk#3, s_store_sk#10] - -(22) CometHashAggregate -Input: [] -Keys: [] -Functions [1]: [partial_count(1)] - -(23) CometExchange -Input [1]: [count#12] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(24) CometHashAggregate -Input [1]: [count#12] -Keys: [] -Functions [1]: [count(1)] - -(25) CometColumnarToRow [codegen id : 1] -Input [1]: [count(1)#13] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/extended.txt deleted file mode 100644 index 0f623c9021..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/extended.txt +++ /dev/null @@ -1,27 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 24 out of 24 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index bbbd07245e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/simplified.txt +++ /dev/null @@ -1,27 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [count] [count(1),count(1)] - CometExchange #1 - CometHashAggregate [count] - CometProject - CometBroadcastHashJoin [ss_store_sk,s_store_sk] - CometProject [ss_store_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_store_sk,t_time_sk] - CometProject [ss_sold_time_sk,ss_store_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_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] - CometBroadcastExchange [hd_demo_sk] #2 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] - CometBroadcastExchange [t_time_sk] #3 - CometProject [t_time_sk] - CometFilter [t_time_sk,t_hour,t_minute] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] - CometBroadcastExchange [s_store_sk] #4 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_store_name] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/explain.txt deleted file mode 100644 index a7f1cb451d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/explain.txt +++ /dev/null @@ -1,174 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (24) -+- CometHashAggregate (23) - +- CometExchange (22) - +- CometHashAggregate (21) - +- CometProject (20) - +- CometSortMergeJoin (19) - :- CometSort (9) - : +- CometHashAggregate (8) - : +- CometColumnarExchange (7) - : +- * HashAggregate (6) - : +- * Project (5) - : +- * BroadcastHashJoin Inner BuildRight (4) - : :- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.store_sales (1) - : +- ReusedExchange (3) - +- CometSort (18) - +- CometHashAggregate (17) - +- CometColumnarExchange (16) - +- * HashAggregate (15) - +- * Project (14) - +- * BroadcastHashJoin Inner BuildRight (13) - :- * ColumnarToRow (11) - : +- Scan parquet spark_catalog.default.catalog_sales (10) - +- ReusedExchange (12) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 2] -Input [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] - -(3) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#5] - -(4) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(5) Project [codegen id : 2] -Output [2]: [ss_item_sk#1, ss_customer_sk#2] -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#5] - -(6) HashAggregate [codegen id : 2] -Input [2]: [ss_item_sk#1, ss_customer_sk#2] -Keys [2]: [ss_customer_sk#2, ss_item_sk#1] -Functions: [] -Aggregate Attributes: [] -Results [2]: [ss_customer_sk#2, ss_item_sk#1] - -(7) CometColumnarExchange -Input [2]: [ss_customer_sk#2, ss_item_sk#1] -Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(8) CometHashAggregate -Input [2]: [ss_customer_sk#2, ss_item_sk#1] -Keys [2]: [ss_customer_sk#2, ss_item_sk#1] -Functions: [] - -(9) CometSort -Input [2]: [customer_sk#6, item_sk#7] -Arguments: [customer_sk#6, item_sk#7], [customer_sk#6 ASC NULLS FIRST, item_sk#7 ASC NULLS FIRST] - -(10) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#4)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 4] -Input [3]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10] - -(12) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#11] - -(13) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#10] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 4] -Output [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Input [4]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10, d_date_sk#11] - -(15) HashAggregate [codegen id : 4] -Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Keys [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Functions: [] -Aggregate Attributes: [] -Results [2]: [cs_bill_customer_sk#8, cs_item_sk#9] - -(16) CometColumnarExchange -Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Arguments: hashpartitioning(cs_bill_customer_sk#8, cs_item_sk#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(17) CometHashAggregate -Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Keys [2]: [cs_bill_customer_sk#8, cs_item_sk#9] -Functions: [] - -(18) CometSort -Input [2]: [customer_sk#12, item_sk#13] -Arguments: [customer_sk#12, item_sk#13], [customer_sk#12 ASC NULLS FIRST, item_sk#13 ASC NULLS FIRST] - -(19) CometSortMergeJoin -Left output [2]: [customer_sk#6, item_sk#7] -Right output [2]: [customer_sk#12, item_sk#13] -Arguments: [customer_sk#6, item_sk#7], [customer_sk#12, item_sk#13], FullOuter - -(20) CometProject -Input [4]: [customer_sk#6, item_sk#7, customer_sk#12, item_sk#13] -Arguments: [customer_sk#6, customer_sk#12], [customer_sk#6, customer_sk#12] - -(21) CometHashAggregate -Input [2]: [customer_sk#6, customer_sk#12] -Keys: [] -Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#12)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#12)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#12)) THEN 1 ELSE 0 END)] - -(22) CometExchange -Input [3]: [sum#14, sum#15, sum#16] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(23) CometHashAggregate -Input [3]: [sum#14, sum#15, sum#16] -Keys: [] -Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#12)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#12)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#12)) THEN 1 ELSE 0 END)] - -(24) CometColumnarToRow [codegen id : 5] -Input [3]: [store_only#17, catalog_only#18, store_and_catalog#19] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometNativeScan parquet spark_catalog.default.date_dim (25) - - -(25) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#20] -Condition : (((isnotnull(d_month_seq#20) AND (d_month_seq#20 >= 1200)) AND (d_month_seq#20 <= 1211)) AND isnotnull(d_date_sk#5)) - -(27) CometProject -Input [2]: [d_date_sk#5, d_month_seq#20] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(29) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/simplified.txt deleted file mode 100644 index dac440b122..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,sum] [store_only,catalog_only,store_and_catalog,sum(CASE WHEN (isnotnull(customer_sk) AND isnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnotnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END)] - CometExchange #1 - CometHashAggregate [customer_sk,customer_sk] [sum,sum,sum] - CometProject [customer_sk,customer_sk] - CometSortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] - CometSort [customer_sk,item_sk] - CometHashAggregate [customer_sk,item_sk,ss_customer_sk,ss_item_sk] - CometColumnarExchange [ss_customer_sk,ss_item_sk] #2 - WholeStageCodegen (2) - HashAggregate [ss_customer_sk,ss_item_sk] - Project [ss_item_sk,ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #3 - CometSort [customer_sk,item_sk] - CometHashAggregate [customer_sk,item_sk,cs_bill_customer_sk,cs_item_sk] - CometColumnarExchange [cs_bill_customer_sk,cs_item_sk] #4 - WholeStageCodegen (4) - HashAggregate [cs_bill_customer_sk,cs_item_sk] - Project [cs_bill_customer_sk,cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/explain.txt deleted file mode 100644 index 420f37fccc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,179 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometSortMergeJoin (20) - :- CometSort (11) - : +- CometHashAggregate (10) - : +- CometExchange (9) - : +- CometHashAggregate (8) - : +- CometProject (7) - : +- CometBroadcastHashJoin (6) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (5) - : +- CometProject (4) - : +- CometFilter (3) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (2) - +- CometSort (19) - +- CometHashAggregate (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometProject (15) - +- CometBroadcastHashJoin (14) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (12) - +- ReusedExchange (13) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -ReadSchema: struct - -(2) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(3) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(4) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(5) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(6) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(7) CometProject -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2], [ss_item_sk#1, ss_customer_sk#2] - -(8) CometHashAggregate -Input [2]: [ss_item_sk#1, ss_customer_sk#2] -Keys [2]: [ss_customer_sk#2, ss_item_sk#1] -Functions: [] - -(9) CometExchange -Input [2]: [ss_customer_sk#2, ss_item_sk#1] -Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometHashAggregate -Input [2]: [ss_customer_sk#2, ss_item_sk#1] -Keys [2]: [ss_customer_sk#2, ss_item_sk#1] -Functions: [] - -(11) CometSort -Input [2]: [customer_sk#7, item_sk#8] -Arguments: [customer_sk#7, item_sk#8], [customer_sk#7 ASC NULLS FIRST, item_sk#8 ASC NULLS FIRST] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#11), dynamicpruningexpression(cs_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(13) ReusedExchange [Reuses operator id: 5] -Output [1]: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [3]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11] -Right output [1]: [d_date_sk#13] -Arguments: [cs_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [4]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11, d_date_sk#13] -Arguments: [cs_bill_customer_sk#9, cs_item_sk#10], [cs_bill_customer_sk#9, cs_item_sk#10] - -(16) CometHashAggregate -Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Keys [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Functions: [] - -(17) CometExchange -Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Arguments: hashpartitioning(cs_bill_customer_sk#9, cs_item_sk#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(18) CometHashAggregate -Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Keys [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Functions: [] - -(19) CometSort -Input [2]: [customer_sk#14, item_sk#15] -Arguments: [customer_sk#14, item_sk#15], [customer_sk#14 ASC NULLS FIRST, item_sk#15 ASC NULLS FIRST] - -(20) CometSortMergeJoin -Left output [2]: [customer_sk#7, item_sk#8] -Right output [2]: [customer_sk#14, item_sk#15] -Arguments: [customer_sk#7, item_sk#8], [customer_sk#14, item_sk#15], FullOuter - -(21) CometProject -Input [4]: [customer_sk#7, item_sk#8, customer_sk#14, item_sk#15] -Arguments: [customer_sk#7, customer_sk#14], [customer_sk#7, customer_sk#14] - -(22) CometHashAggregate -Input [2]: [customer_sk#7, customer_sk#14] -Keys: [] -Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)] - -(23) CometExchange -Input [3]: [sum#16, sum#17, sum#18] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(24) CometHashAggregate -Input [3]: [sum#16, sum#17, sum#18] -Keys: [] -Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)] - -(25) CometColumnarToRow [codegen id : 1] -Input [3]: [store_only#19, catalog_only#20, store_and_catalog#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(28) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(30) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 12 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/simplified.txt deleted file mode 100644 index a7de047b25..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,sum] [store_only,catalog_only,store_and_catalog,sum(CASE WHEN (isnotnull(customer_sk) AND isnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnotnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END)] - CometExchange #1 - CometHashAggregate [customer_sk,customer_sk] [sum,sum,sum] - CometProject [customer_sk,customer_sk] - CometSortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] - CometSort [customer_sk,item_sk] - CometHashAggregate [customer_sk,item_sk,ss_customer_sk,ss_item_sk] - CometExchange [ss_customer_sk,ss_item_sk] #2 - CometHashAggregate [ss_customer_sk,ss_item_sk] - CometProject [ss_item_sk,ss_customer_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometSort [customer_sk,item_sk] - CometHashAggregate [customer_sk,item_sk,cs_bill_customer_sk,cs_item_sk] - CometExchange [cs_bill_customer_sk,cs_item_sk] #5 - CometHashAggregate [cs_bill_customer_sk,cs_item_sk] - CometProject [cs_bill_customer_sk,cs_item_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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/q97/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt deleted file mode 100644 index 420f37fccc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt +++ /dev/null @@ -1,179 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometSortMergeJoin (20) - :- CometSort (11) - : +- CometHashAggregate (10) - : +- CometExchange (9) - : +- CometHashAggregate (8) - : +- CometProject (7) - : +- CometBroadcastHashJoin (6) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (5) - : +- CometProject (4) - : +- CometFilter (3) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (2) - +- CometSort (19) - +- CometHashAggregate (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometProject (15) - +- CometBroadcastHashJoin (14) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (12) - +- ReusedExchange (13) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -ReadSchema: struct - -(2) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(3) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(4) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(5) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(6) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(7) CometProject -Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2], [ss_item_sk#1, ss_customer_sk#2] - -(8) CometHashAggregate -Input [2]: [ss_item_sk#1, ss_customer_sk#2] -Keys [2]: [ss_customer_sk#2, ss_item_sk#1] -Functions: [] - -(9) CometExchange -Input [2]: [ss_customer_sk#2, ss_item_sk#1] -Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometHashAggregate -Input [2]: [ss_customer_sk#2, ss_item_sk#1] -Keys [2]: [ss_customer_sk#2, ss_item_sk#1] -Functions: [] - -(11) CometSort -Input [2]: [customer_sk#7, item_sk#8] -Arguments: [customer_sk#7, item_sk#8], [customer_sk#7 ASC NULLS FIRST, item_sk#8 ASC NULLS FIRST] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#11), dynamicpruningexpression(cs_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(13) ReusedExchange [Reuses operator id: 5] -Output [1]: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [3]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11] -Right output [1]: [d_date_sk#13] -Arguments: [cs_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [4]: [cs_bill_customer_sk#9, cs_item_sk#10, cs_sold_date_sk#11, d_date_sk#13] -Arguments: [cs_bill_customer_sk#9, cs_item_sk#10], [cs_bill_customer_sk#9, cs_item_sk#10] - -(16) CometHashAggregate -Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Keys [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Functions: [] - -(17) CometExchange -Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Arguments: hashpartitioning(cs_bill_customer_sk#9, cs_item_sk#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(18) CometHashAggregate -Input [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Keys [2]: [cs_bill_customer_sk#9, cs_item_sk#10] -Functions: [] - -(19) CometSort -Input [2]: [customer_sk#14, item_sk#15] -Arguments: [customer_sk#14, item_sk#15], [customer_sk#14 ASC NULLS FIRST, item_sk#15 ASC NULLS FIRST] - -(20) CometSortMergeJoin -Left output [2]: [customer_sk#7, item_sk#8] -Right output [2]: [customer_sk#14, item_sk#15] -Arguments: [customer_sk#7, item_sk#8], [customer_sk#14, item_sk#15], FullOuter - -(21) CometProject -Input [4]: [customer_sk#7, item_sk#8, customer_sk#14, item_sk#15] -Arguments: [customer_sk#7, customer_sk#14], [customer_sk#7, customer_sk#14] - -(22) CometHashAggregate -Input [2]: [customer_sk#7, customer_sk#14] -Keys: [] -Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)] - -(23) CometExchange -Input [3]: [sum#16, sum#17, sum#18] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(24) CometHashAggregate -Input [3]: [sum#16, sum#17, sum#18] -Keys: [] -Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#7) AND isnull(customer_sk#14)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#7) AND isnotnull(customer_sk#14)) THEN 1 ELSE 0 END)] - -(25) CometColumnarToRow [codegen id : 1] -Input [3]: [store_only#19, catalog_only#20, store_and_catalog#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(28) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(30) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 12 Hosting Expression = cs_sold_date_sk#11 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/extended.txt deleted file mode 100644 index e1aefb8382..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -CometColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index a7de047b25..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/simplified.txt +++ /dev/null @@ -1,36 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum,sum] [store_only,catalog_only,store_and_catalog,sum(CASE WHEN (isnotnull(customer_sk) AND isnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnotnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END)] - CometExchange #1 - CometHashAggregate [customer_sk,customer_sk] [sum,sum,sum] - CometProject [customer_sk,customer_sk] - CometSortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] - CometSort [customer_sk,item_sk] - CometHashAggregate [customer_sk,item_sk,ss_customer_sk,ss_item_sk] - CometExchange [ss_customer_sk,ss_item_sk] #2 - CometHashAggregate [ss_customer_sk,ss_item_sk] - CometProject [ss_item_sk,ss_customer_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometSort [customer_sk,item_sk] - CometHashAggregate [customer_sk,item_sk,cs_bill_customer_sk,cs_item_sk] - CometExchange [cs_bill_customer_sk,cs_item_sk] #5 - CometHashAggregate [cs_bill_customer_sk,cs_item_sk] - CometProject [cs_bill_customer_sk,cs_item_sk] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/explain.txt deleted file mode 100644 index 8263680b2b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/explain.txt +++ /dev/null @@ -1,177 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometProject (25) - +- CometSort (24) - +- CometColumnarExchange (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.store_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.item (4) - +- ReusedExchange (11) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(7) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(8) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(11) ReusedExchange [Reuses operator id: 31] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] - -(14) HashAggregate [codegen id : 3] -Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(15) CometColumnarExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(17) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#17] -Results [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#17,17,2) AS _w0#19, i_item_id#11] - -(18) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(22) Project [codegen id : 6] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] - -(23) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(24) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST] - -(25) CometProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -(26) CometColumnarToRow [codegen id : 7] -Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometNativeScan parquet spark_catalog.default.date_dim (27) - - -(27) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(29) CometProject -Input [2]: [d_date_sk#14, d_date#22] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(31) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/simplified.txt deleted file mode 100644 index 3f64b57ffa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/simplified.txt +++ /dev/null @@ -1,48 +0,0 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (6) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/explain.txt deleted file mode 100644 index 4cc725ef2b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,178 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometProject (25) - +- CometSort (24) - +- CometColumnarExchange (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] - -(23) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] - -(25) CometProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -(26) CometColumnarToRow [codegen id : 3] -Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(29) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(31) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/simplified.txt deleted file mode 100644 index af05a7d8c6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,40 +0,0 @@ -WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #2 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt deleted file mode 100644 index 4cc725ef2b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt +++ /dev/null @@ -1,178 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (26) -+- CometProject (25) - +- CometSort (24) - +- CometColumnarExchange (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] - -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] - -(23) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] - -(25) CometProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -(26) CometColumnarToRow [codegen id : 3] -Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(29) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(31) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/extended.txt deleted file mode 100644 index 4c972848e7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/extended.txt +++ /dev/null @@ -1,34 +0,0 @@ -CometColumnarToRow -+- CometProject - +- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index af05a7d8c6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt +++ /dev/null @@ -1,40 +0,0 @@ -WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #2 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/explain.txt deleted file mode 100644 index 3447b6efa3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometNativeScan parquet spark_catalog.default.warehouse (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometNativeScan parquet spark_catalog.default.ship_mode (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.call_center (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometNativeScan parquet spark_catalog.default.date_dim (19) - - -(1) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_ship_mode_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Condition : (((isnotnull(cs_warehouse_sk#4) AND isnotnull(cs_ship_mode_sk#3)) AND isnotnull(cs_call_center_sk#2)) AND isnotnull(cs_ship_date_sk#1)) - -(3) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [cs_warehouse_sk#4], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7], [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] - -(8) CometNativeScan parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#8, sm_type#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Condition : isnotnull(sm_ship_mode_sk#8) - -(10) CometProject -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Arguments: [sm_ship_mode_sk#8, sm_type#10], [sm_ship_mode_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, sm_type#9, 30, true, false, true) AS sm_type#10] - -(11) CometBroadcastExchange -Input [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [sm_ship_mode_sk#8, sm_type#10] - -(12) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] -Right output [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [cs_ship_mode_sk#3], [sm_ship_mode_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#10] -Arguments: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10], [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10] - -(14) CometNativeScan parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#11, cc_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#11, cc_name#12] -Condition : isnotnull(cc_call_center_sk#11) - -(16) CometBroadcastExchange -Input [2]: [cc_call_center_sk#11, cc_name#12] -Arguments: [cc_call_center_sk#11, cc_name#12] - -(17) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10] -Right output [2]: [cc_call_center_sk#11, cc_name#12] -Arguments: [cs_call_center_sk#2], [cc_call_center_sk#11], Inner, BuildRight - -(18) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_call_center_sk#11, cc_name#12] -Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12], [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12] - -(19) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_month_seq#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#13, d_month_seq#14] -Condition : (((isnotnull(d_month_seq#14) AND (d_month_seq#14 >= 1200)) AND (d_month_seq#14 <= 1211)) AND isnotnull(d_date_sk#13)) - -(21) CometProject -Input [2]: [d_date_sk#13, d_month_seq#14] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(23) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12] -Right output [1]: [d_date_sk#13] -Arguments: [cs_ship_date_sk#1], [d_date_sk#13], Inner, BuildRight - -(24) CometProject -Input [6]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12, d_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, _groupingexpression#15], [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#15] - -(25) CometHashAggregate -Input [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, sm_type#10, cc_name#12] -Functions [5]: [partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [8]: [_groupingexpression#15, sm_type#10, cc_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Arguments: hashpartitioning(_groupingexpression#15, sm_type#10, cc_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [8]: [_groupingexpression#15, sm_type#10, cc_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Keys [3]: [_groupingexpression#15, sm_type#10, cc_name#12] -Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,cc_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,cc_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, cc_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - -(29) CometColumnarToRow [codegen id : 1] -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/simplified.txt deleted file mode 100644 index 98a215dbc6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 30) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 60) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 90) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [_groupingexpression,sm_type,cc_name] #1 - CometHashAggregate [cs_ship_date_sk,cs_sold_date_sk] [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum] - CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] - 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_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_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_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] - CometNativeScan 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 [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometNativeScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [cc_call_center_sk,cc_name] #4 - CometFilter [cc_call_center_sk,cc_name] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan 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/q99.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_iceberg_compat/explain.txt deleted file mode 100644 index 7e57135c10..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_ship_mode_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Condition : (((isnotnull(cs_warehouse_sk#4) AND isnotnull(cs_ship_mode_sk#3)) AND isnotnull(cs_call_center_sk#2)) AND isnotnull(cs_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [cs_warehouse_sk#4], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7], [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#8, sm_type#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Condition : isnotnull(sm_ship_mode_sk#8) - -(10) CometProject -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Arguments: [sm_ship_mode_sk#8, sm_type#10], [sm_ship_mode_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, sm_type#9, 30, true, false, true) AS sm_type#10] - -(11) CometBroadcastExchange -Input [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [sm_ship_mode_sk#8, sm_type#10] - -(12) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] -Right output [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [cs_ship_mode_sk#3], [sm_ship_mode_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#10] -Arguments: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10], [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#11, cc_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#11, cc_name#12] -Condition : isnotnull(cc_call_center_sk#11) - -(16) CometBroadcastExchange -Input [2]: [cc_call_center_sk#11, cc_name#12] -Arguments: [cc_call_center_sk#11, cc_name#12] - -(17) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10] -Right output [2]: [cc_call_center_sk#11, cc_name#12] -Arguments: [cs_call_center_sk#2], [cc_call_center_sk#11], Inner, BuildRight - -(18) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_call_center_sk#11, cc_name#12] -Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12], [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_month_seq#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#13, d_month_seq#14] -Condition : (((isnotnull(d_month_seq#14) AND (d_month_seq#14 >= 1200)) AND (d_month_seq#14 <= 1211)) AND isnotnull(d_date_sk#13)) - -(21) CometProject -Input [2]: [d_date_sk#13, d_month_seq#14] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(23) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12] -Right output [1]: [d_date_sk#13] -Arguments: [cs_ship_date_sk#1], [d_date_sk#13], Inner, BuildRight - -(24) CometProject -Input [6]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12, d_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, _groupingexpression#15], [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#15] - -(25) CometHashAggregate -Input [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, sm_type#10, cc_name#12] -Functions [5]: [partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [8]: [_groupingexpression#15, sm_type#10, cc_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Arguments: hashpartitioning(_groupingexpression#15, sm_type#10, cc_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [8]: [_groupingexpression#15, sm_type#10, cc_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Keys [3]: [_groupingexpression#15, sm_type#10, cc_name#12] -Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,cc_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,cc_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, cc_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - -(29) CometColumnarToRow [codegen id : 1] -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_iceberg_compat/simplified.txt deleted file mode 100644 index 22896ee566..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 30) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 60) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 90) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [_groupingexpression,sm_type,cc_name] #1 - CometHashAggregate [cs_ship_date_sk,cs_sold_date_sk] [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum] - CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] - 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_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_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_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 [native_iceberg_compat] 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 [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [cc_call_center_sk,cc_name] #4 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] 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/q99/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt deleted file mode 100644 index 7e57135c10..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt +++ /dev/null @@ -1,168 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (7) - : : : +- CometBroadcastHashJoin (6) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : +- CometBroadcastExchange (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (3) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode (8) - : +- CometBroadcastExchange (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - +- CometBroadcastExchange (22) - +- CometProject (21) - +- CometFilter (20) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_ship_mode_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Condition : (((isnotnull(cs_warehouse_sk#4) AND isnotnull(cs_ship_mode_sk#3)) AND isnotnull(cs_call_center_sk#2)) AND isnotnull(cs_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Condition : isnotnull(w_warehouse_sk#6) - -(5) CometBroadcastExchange -Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [w_warehouse_sk#6, w_warehouse_name#7] - -(6) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] -Right output [2]: [w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [cs_warehouse_sk#4], [w_warehouse_sk#6], Inner, BuildRight - -(7) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] -Arguments: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7], [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode -Output [2]: [sm_ship_mode_sk#8, sm_type#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/ship_mode] -PushedFilters: [IsNotNull(sm_ship_mode_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Condition : isnotnull(sm_ship_mode_sk#8) - -(10) CometProject -Input [2]: [sm_ship_mode_sk#8, sm_type#9] -Arguments: [sm_ship_mode_sk#8, sm_type#10], [sm_ship_mode_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, sm_type#9, 30, true, false, true) AS sm_type#10] - -(11) CometBroadcastExchange -Input [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [sm_ship_mode_sk#8, sm_type#10] - -(12) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] -Right output [2]: [sm_ship_mode_sk#8, sm_type#10] -Arguments: [cs_ship_mode_sk#3], [sm_ship_mode_sk#8], Inner, BuildRight - -(13) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#10] -Arguments: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10], [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#11, cc_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#11, cc_name#12] -Condition : isnotnull(cc_call_center_sk#11) - -(16) CometBroadcastExchange -Input [2]: [cc_call_center_sk#11, cc_name#12] -Arguments: [cc_call_center_sk#11, cc_name#12] - -(17) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10] -Right output [2]: [cc_call_center_sk#11, cc_name#12] -Arguments: [cs_call_center_sk#2], [cc_call_center_sk#11], Inner, BuildRight - -(18) CometProject -Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_call_center_sk#11, cc_name#12] -Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12], [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_month_seq#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [d_date_sk#13, d_month_seq#14] -Condition : (((isnotnull(d_month_seq#14) AND (d_month_seq#14 >= 1200)) AND (d_month_seq#14 <= 1211)) AND isnotnull(d_date_sk#13)) - -(21) CometProject -Input [2]: [d_date_sk#13, d_month_seq#14] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(22) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(23) CometBroadcastHashJoin -Left output [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12] -Right output [1]: [d_date_sk#13] -Arguments: [cs_ship_date_sk#1], [d_date_sk#13], Inner, BuildRight - -(24) CometProject -Input [6]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#10, cc_name#12, d_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, _groupingexpression#15], [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#15] - -(25) CometHashAggregate -Input [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#10, cc_name#12, _groupingexpression#15] -Keys [3]: [_groupingexpression#15, sm_type#10, cc_name#12] -Functions [5]: [partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(26) CometExchange -Input [8]: [_groupingexpression#15, sm_type#10, cc_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Arguments: hashpartitioning(_groupingexpression#15, sm_type#10, cc_name#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [8]: [_groupingexpression#15, sm_type#10, cc_name#12, sum#16, sum#17, sum#18, sum#19, sum#20] -Keys [3]: [_groupingexpression#15, sm_type#10, cc_name#12] -Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] - -(28) CometTakeOrderedAndProject -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST,sm_type#10 ASC NULLS FIRST,cc_name#12 ASC NULLS FIRST], output=[substr(w_warehouse_name, 1, 20)#21,sm_type#10,cc_name#12,30 days #22,31 - 60 days #23,61 - 90 days #24,91 - 120 days #25,>120 days #26]), [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26], 100, 0, [substr(w_warehouse_name, 1, 20)#21 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, cc_name#12 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - -(29) CometColumnarToRow [codegen id : 1] -Input [8]: [substr(w_warehouse_name, 1, 20)#21, sm_type#10, cc_name#12, 30 days #22, 31 - 60 days #23, 61 - 90 days #24, 91 - 120 days #25, >120 days #26] - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/extended.txt deleted file mode 100644 index 86cf2fc2f3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 28 out of 28 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 22896ee566..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/simplified.txt +++ /dev/null @@ -1,31 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] - CometHashAggregate [sum,sum,sum,sum,sum] [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,_groupingexpression,sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 30) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 60) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 90) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) > 120) THEN 1 ELSE 0 END)] - CometExchange [_groupingexpression,sm_type,cc_name] #1 - CometHashAggregate [cs_ship_date_sk,cs_sold_date_sk] [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum] - CometProject [w_warehouse_name] [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,_groupingexpression] - 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_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_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_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 [native_iceberg_compat] 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 [w_warehouse_sk,w_warehouse_name] #2 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [sm_ship_mode_sk,sm_type] #3 - CometProject [sm_type] [sm_ship_mode_sk,sm_type] - CometFilter [sm_ship_mode_sk,sm_type] - CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] - CometBroadcastExchange [cc_call_center_sk,cc_name] #4 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/explain.txt deleted file mode 100644 index 83b183d532..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/explain.txt +++ /dev/null @@ -1,281 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : +- BroadcastExchange (9) - : : : +- * Project (8) - : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : :- * ColumnarToRow (5) - : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (6) - : : +- BroadcastExchange (22) - : : +- Union (21) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * ColumnarToRow (17) - : : : +- Scan parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (18) - : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.customer_address (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.customer_demographics (32) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(4) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] - -(6) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#7] - -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#4] -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] - -(9) BroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#4] -Join type: LeftSemi -Join condition: None - -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#6)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] - -(13) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#10] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#8 AS customer_sk#11] -Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#10] - -(16) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#6)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] - -(18) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#14] - -(19) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#12 AS customer_sk#15] -Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#14] - -(21) Union - -(22) BroadcastExchange -Input [1]: [customer_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(23) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#11] -Join type: LeftSemi -Join condition: None - -(24) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(25) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_county#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_county, [Dona Ana County,Douglas County,Gaines County,Richland County,Walker County]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [ca_address_sk#16, ca_county#17] -Condition : (ca_county#17 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#16)) - -(27) CometProject -Input [2]: [ca_address_sk#16, ca_county#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(28) CometColumnarToRow [codegen id : 7] -Input [1]: [ca_address_sk#16] - -(29) BroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(30) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#16] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 9] -Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#16] - -(32) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(33) CometFilter -Input [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Condition : isnotnull(cd_demo_sk#18) - -(34) CometProject -Input [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#19, 1, true, false, true) AS cd_gender#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) AS cd_marital_status#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#21, 20, true, false, true) AS cd_education_status#29, cd_purchase_estimate#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#23, 10, true, false, true) AS cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(35) CometColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(36) BroadcastExchange -Input [9]: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 9] -Output [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(39) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#31] -Results [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] - -(40) CometColumnarExchange -Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] -Arguments: hashpartitioning(cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(41) CometColumnarToRow [codegen id : 10] -Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] - -(42) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] -Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#33] -Results [14]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, count(1)#33 AS cnt1#34, cd_purchase_estimate#22, count(1)#33 AS cnt2#35, cd_credit_rating#30, count(1)#33 AS cnt3#36, cd_dep_count#24, count(1)#33 AS cnt4#37, cd_dep_employed_count#25, count(1)#33 AS cnt5#38, cd_dep_college_count#26, count(1)#33 AS cnt6#39] - -(43) TakeOrderedAndProject -Input [14]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cnt1#34, cd_purchase_estimate#22, cnt2#35, cd_credit_rating#30, cnt3#36, cd_dep_count#24, cnt4#37, cd_dep_employed_count#25, cnt5#38, cd_dep_college_count#26, cnt6#39] -Arguments: 100, [cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_education_status#29 ASC NULLS FIRST, cd_purchase_estimate#22 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [cd_gender#27, cd_marital_status#28, cd_education_status#29, cnt1#34, cd_purchase_estimate#22, cnt2#35, cd_credit_rating#30, cnt3#36, cd_dep_count#24, cnt4#37, cd_dep_employed_count#25, cnt5#38, cd_dep_college_count#26, cnt6#39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometNativeScan parquet spark_catalog.default.date_dim (44) - - -(44) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#40, d_moy#41] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [3]: [d_date_sk#7, d_year#40, d_moy#41] -Condition : (((((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2002)) AND (d_moy#41 >= 4)) AND (d_moy#41 <= 7)) AND isnotnull(d_date_sk#7)) - -(46) CometProject -Input [3]: [d_date_sk#7, d_year#40, d_moy#41] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(48) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/simplified.txt deleted file mode 100644 index 11bfb6f6c3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (10) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - Union - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [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] - 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] - CometNativeScan 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/q10a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/explain.txt deleted file mode 100644 index b6c8291e3b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,267 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometHashAggregate (38) - +- CometExchange (37) - +- CometHashAggregate (36) - +- CometProject (35) - +- CometBroadcastHashJoin (34) - :- CometProject (29) - : +- CometBroadcastHashJoin (28) - : :- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometBroadcastHashJoin (11) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : +- CometBroadcastExchange (10) - : : : +- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : +- CometBroadcastExchange (21) - : : +- CometUnion (20) - : : :- CometProject (15) - : : : +- CometBroadcastHashJoin (14) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (12) - : : : +- ReusedExchange (13) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (17) - : +- CometBroadcastExchange (27) - : +- CometProject (26) - : +- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) - +- CometBroadcastExchange (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (30) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) - -(6) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#4], [ss_customer_sk#4] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: [ss_customer_sk#4] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ss_customer_sk#4] -Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(13) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] -Arguments: [customer_sk#14], [ws_bill_customer_sk#10 AS customer_sk#14] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] -ReadSchema: struct - -(17) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#18] - -(18) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Right output [1]: [d_date_sk#18] -Arguments: [cs_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight - -(19) CometProject -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#18] -Arguments: [customer_sk#19], [cs_ship_customer_sk#15 AS customer_sk#19] - -(20) CometUnion -Child 0 Input [1]: [customer_sk#14] -Child 1 Input [1]: [customer_sk#19] - -(21) CometBroadcastExchange -Input [1]: [customer_sk#14] -Arguments: [customer_sk#14] - -(22) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [customer_sk#14] -Arguments: [c_customer_sk#1], [customer_sk#14], LeftSemi, BuildRight - -(23) CometProject -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_county#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_county, [Dona Ana County,Douglas County,Gaines County,Richland County,Walker County]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [ca_address_sk#20, ca_county#21] -Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) - -(26) CometProject -Input [2]: [ca_address_sk#20, ca_county#21] -Arguments: [ca_address_sk#20], [ca_address_sk#20] - -(27) CometBroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: [ca_address_sk#20] - -(28) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ca_address_sk#20] -Arguments: [c_current_addr_sk#3], [ca_address_sk#20], Inner, BuildRight - -(29) CometProject -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] -Arguments: [c_current_cdemo_sk#2], [c_current_cdemo_sk#2] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Condition : isnotnull(cd_demo_sk#22) - -(32) CometProject -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_demo_sk#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#23, 1, true, false, true) AS cd_gender#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#24, 1, true, false, true) AS cd_marital_status#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#25, 20, true, false, true) AS cd_education_status#33, cd_purchase_estimate#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#27, 10, true, false, true) AS cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(33) CometBroadcastExchange -Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(34) CometBroadcastHashJoin -Left output [1]: [c_current_cdemo_sk#2] -Right output [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#22], Inner, BuildRight - -(35) CometProject -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(36) CometHashAggregate -Input [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [1]: [partial_count(1)] - -(37) CometExchange -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#35] -Arguments: hashpartitioning(cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(38) CometHashAggregate -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#35] -Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [1]: [count(1)] - -(39) CometTakeOrderedAndProject -Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#31 ASC NULLS FIRST,cd_marital_status#32 ASC NULLS FIRST,cd_education_status#33 ASC NULLS FIRST,cd_purchase_estimate#26 ASC NULLS FIRST,cd_credit_rating#34 ASC NULLS FIRST,cd_dep_count#28 ASC NULLS FIRST,cd_dep_employed_count#29 ASC NULLS FIRST,cd_dep_college_count#30 ASC NULLS FIRST], output=[cd_gender#31,cd_marital_status#32,cd_education_status#33,cnt1#36,cd_purchase_estimate#26,cnt2#37,cd_credit_rating#34,cnt3#38,cd_dep_count#28,cnt4#39,cd_dep_employed_count#29,cnt5#40,cd_dep_college_count#30,cnt6#41]), [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41], 100, 0, [cd_gender#31 ASC NULLS FIRST, cd_marital_status#32 ASC NULLS FIRST, cd_education_status#33 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#34 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] - -(40) CometColumnarToRow [codegen id : 1] -Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) - -(43) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(45) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 38f41d10a5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6] - CometHashAggregate [count] [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6,count(1)] - CometExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - 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,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_cdemo_sk,c_current_addr_sk,ca_address_sk] - CometProject [c_current_cdemo_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [customer_sk] #5 - CometUnion [customer_sk] - CometProject [ws_bill_customer_sk] [customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - 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 - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [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] - 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 [native_iceberg_compat] 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/q10a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/explain.txt deleted file mode 100644 index b6c8291e3b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/explain.txt +++ /dev/null @@ -1,267 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometHashAggregate (38) - +- CometExchange (37) - +- CometHashAggregate (36) - +- CometProject (35) - +- CometBroadcastHashJoin (34) - :- CometProject (29) - : +- CometBroadcastHashJoin (28) - : :- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometBroadcastHashJoin (11) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : +- CometBroadcastExchange (10) - : : : +- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : +- CometBroadcastExchange (21) - : : +- CometUnion (20) - : : :- CometProject (15) - : : : +- CometBroadcastHashJoin (14) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (12) - : : : +- ReusedExchange (13) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (17) - : +- CometBroadcastExchange (27) - : +- CometProject (26) - : +- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) - +- CometBroadcastExchange (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (30) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) - -(6) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#4], [ss_customer_sk#4] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: [ss_customer_sk#4] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ss_customer_sk#4] -Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(13) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] -Arguments: [customer_sk#14], [ws_bill_customer_sk#10 AS customer_sk#14] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] -ReadSchema: struct - -(17) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#18] - -(18) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Right output [1]: [d_date_sk#18] -Arguments: [cs_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight - -(19) CometProject -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#18] -Arguments: [customer_sk#19], [cs_ship_customer_sk#15 AS customer_sk#19] - -(20) CometUnion -Child 0 Input [1]: [customer_sk#14] -Child 1 Input [1]: [customer_sk#19] - -(21) CometBroadcastExchange -Input [1]: [customer_sk#14] -Arguments: [customer_sk#14] - -(22) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [customer_sk#14] -Arguments: [c_customer_sk#1], [customer_sk#14], LeftSemi, BuildRight - -(23) CometProject -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_county#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_county, [Dona Ana County,Douglas County,Gaines County,Richland County,Walker County]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [ca_address_sk#20, ca_county#21] -Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) - -(26) CometProject -Input [2]: [ca_address_sk#20, ca_county#21] -Arguments: [ca_address_sk#20], [ca_address_sk#20] - -(27) CometBroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: [ca_address_sk#20] - -(28) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ca_address_sk#20] -Arguments: [c_current_addr_sk#3], [ca_address_sk#20], Inner, BuildRight - -(29) CometProject -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] -Arguments: [c_current_cdemo_sk#2], [c_current_cdemo_sk#2] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Condition : isnotnull(cd_demo_sk#22) - -(32) CometProject -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_demo_sk#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#23, 1, true, false, true) AS cd_gender#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#24, 1, true, false, true) AS cd_marital_status#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#25, 20, true, false, true) AS cd_education_status#33, cd_purchase_estimate#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#27, 10, true, false, true) AS cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(33) CometBroadcastExchange -Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(34) CometBroadcastHashJoin -Left output [1]: [c_current_cdemo_sk#2] -Right output [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#22], Inner, BuildRight - -(35) CometProject -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(36) CometHashAggregate -Input [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [1]: [partial_count(1)] - -(37) CometExchange -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#35] -Arguments: hashpartitioning(cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(38) CometHashAggregate -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#35] -Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [1]: [count(1)] - -(39) CometTakeOrderedAndProject -Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#31 ASC NULLS FIRST,cd_marital_status#32 ASC NULLS FIRST,cd_education_status#33 ASC NULLS FIRST,cd_purchase_estimate#26 ASC NULLS FIRST,cd_credit_rating#34 ASC NULLS FIRST,cd_dep_count#28 ASC NULLS FIRST,cd_dep_employed_count#29 ASC NULLS FIRST,cd_dep_college_count#30 ASC NULLS FIRST], output=[cd_gender#31,cd_marital_status#32,cd_education_status#33,cnt1#36,cd_purchase_estimate#26,cnt2#37,cd_credit_rating#34,cnt3#38,cd_dep_count#28,cnt4#39,cd_dep_employed_count#29,cnt5#40,cd_dep_college_count#30,cnt6#41]), [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41], 100, 0, [cd_gender#31 ASC NULLS FIRST, cd_marital_status#32 ASC NULLS FIRST, cd_education_status#33 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#34 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] - -(40) CometColumnarToRow [codegen id : 1] -Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) - -(43) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(45) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/extended.txt deleted file mode 100644 index 2cdc75e15e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/extended.txt +++ /dev/null @@ -1,56 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 38f41d10a5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6] - CometHashAggregate [count] [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6,count(1)] - CometExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - 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,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_cdemo_sk,c_current_addr_sk,ca_address_sk] - CometProject [c_current_cdemo_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [customer_sk] #5 - CometUnion [customer_sk] - CometProject [ws_bill_customer_sk] [customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - 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 - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [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] - 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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/explain.txt deleted file mode 100644 index 207cd830ed..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/explain.txt +++ /dev/null @@ -1,513 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (79) -+- * Project (78) - +- * BroadcastHashJoin Inner BuildRight (77) - :- * Project (58) - : +- * BroadcastHashJoin Inner BuildRight (57) - : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- * HashAggregate (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * Project (28) - : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (26) - : : : +- * Filter (25) - : : : +- * ColumnarToRow (24) - : : : +- Scan parquet spark_catalog.default.store_sales (23) - : : +- ReusedExchange (29) - : +- BroadcastExchange (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- * CometColumnarToRow (53) - : +- CometColumnarExchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (41) - : : : +- CometProject (40) - : : : +- CometFilter (39) - : : : +- CometNativeScan parquet spark_catalog.default.customer (38) - : : +- BroadcastExchange (45) - : : +- * Filter (44) - : : +- * ColumnarToRow (43) - : : +- Scan parquet spark_catalog.default.web_sales (42) - : +- ReusedExchange (48) - +- BroadcastExchange (76) - +- * HashAggregate (75) - +- * CometColumnarToRow (74) - +- CometColumnarExchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * CometColumnarToRow (62) - : : +- CometProject (61) - : : +- CometFilter (60) - : : +- CometNativeScan parquet spark_catalog.default.customer (59) - : +- BroadcastExchange (66) - : +- * Filter (65) - : +- * ColumnarToRow (64) - : +- Scan parquet spark_catalog.default.web_sales (63) - +- ReusedExchange (69) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] - -(4) CometColumnarToRow [codegen id : 3] -Input [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] - -(5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(7) Filter [codegen id : 1] -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#15) - -(8) BroadcastExchange -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#15] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(11) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#20, d_year#21] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#18] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] - -(14) HashAggregate [codegen id : 3] -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] -Aggregate Attributes [1]: [sum#22] -Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] - -(15) CometColumnarExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 16] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] - -(17) HashAggregate [codegen id : 16] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24] -Results [2]: [c_customer_id#9 AS customer_id#25, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24,18,2) AS year_total#26] - -(18) Filter [codegen id : 16] -Input [2]: [customer_id#25, year_total#26] -Condition : (isnotnull(year_total#26) AND (year_total#26 > 0.00)) - -(19) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(20) CometFilter -Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] -Condition : (isnotnull(c_customer_sk#27) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#28, 16, true, false, true))) - -(21) CometProject -Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] -Arguments: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14], [c_customer_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#28, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#29, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#30, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#31, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#33, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#34, 50, true, false, true) AS c_email_address#14] - -(22) CometColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] - -(23) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_sold_date_sk#38 IN dynamicpruning#39)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] - -(25) Filter [codegen id : 4] -Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Condition : isnotnull(ss_customer_sk#35) - -(26) BroadcastExchange -Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#27] -Right keys [1]: [ss_customer_sk#35] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Input [12]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] - -(29) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#40, d_year#41] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38, d_date_sk#40, d_year#41] - -(32) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] -Aggregate Attributes [1]: [sum#42] -Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] - -(33) CometColumnarExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 7] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] - -(35) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))#24] -Results [5]: [c_customer_id#9 AS customer_id#44, c_first_name#10 AS customer_first_name#45, c_last_name#11 AS customer_last_name#46, c_email_address#14 AS customer_email_address#47, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))#24,18,2) AS year_total#48] - -(36) BroadcastExchange -Input [5]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(37) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#44] -Join type: Inner -Join condition: None - -(38) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(39) CometFilter -Input [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] -Condition : (isnotnull(c_customer_sk#49) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#50, 16, true, false, true))) - -(40) CometProject -Input [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] -Arguments: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62], [c_customer_sk#49, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#50, 16, true, false, true) AS c_customer_id#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#51, 20, true, false, true) AS c_first_name#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#52, 30, true, false, true) AS c_last_name#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#53, 1, true, false, true) AS c_preferred_cust_flag#60, c_birth_country#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#55, 13, true, false, true) AS c_login#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#56, 50, true, false, true) AS c_email_address#62] - -(41) CometColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62] - -(42) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#66), dynamicpruningexpression(ws_sold_date_sk#66 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] - -(44) Filter [codegen id : 8] -Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] -Condition : isnotnull(ws_bill_customer_sk#63) - -(45) BroadcastExchange -Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -(46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#49] -Right keys [1]: [ws_bill_customer_sk#63] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 10] -Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] -Input [12]: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] - -(48) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#67, d_year#68] - -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#66] -Right keys [1]: [d_date_sk#67] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 10] -Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, d_year#68] -Input [12]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66, d_date_sk#67, d_year#68] - -(51) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, d_year#68] -Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))] -Aggregate Attributes [1]: [sum#69] -Results [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68, sum#70] - -(52) CometColumnarExchange -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68, sum#70] -Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(53) CometColumnarToRow [codegen id : 11] -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68, sum#70] - -(54) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68, sum#70] -Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))#71] -Results [2]: [c_customer_id#57 AS customer_id#72, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))#71,18,2) AS year_total#73] - -(55) Filter [codegen id : 11] -Input [2]: [customer_id#72, year_total#73] -Condition : (isnotnull(year_total#73) AND (year_total#73 > 0.00)) - -(56) BroadcastExchange -Input [2]: [customer_id#72, year_total#73] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(57) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#72] -Join type: Inner -Join condition: None - -(58) Project [codegen id : 16] -Output [8]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, year_total#73] -Input [9]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, customer_id#72, year_total#73] - -(59) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(60) CometFilter -Input [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] -Condition : (isnotnull(c_customer_sk#74) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#75, 16, true, false, true))) - -(61) CometProject -Input [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] -Arguments: [c_customer_sk#74, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62], [c_customer_sk#74, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#75, 16, true, false, true) AS c_customer_id#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#76, 20, true, false, true) AS c_first_name#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#77, 30, true, false, true) AS c_last_name#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#78, 1, true, false, true) AS c_preferred_cust_flag#60, c_birth_country#79, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#80, 13, true, false, true) AS c_login#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#81, 50, true, false, true) AS c_email_address#62] - -(62) CometColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#74, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62] - -(63) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#85), dynamicpruningexpression(ws_sold_date_sk#85 IN dynamicpruning#39)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(64) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] - -(65) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] -Condition : isnotnull(ws_bill_customer_sk#82) - -(66) BroadcastExchange -Input [4]: [ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#74] -Right keys [1]: [ws_bill_customer_sk#82] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 14] -Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] -Input [12]: [c_customer_sk#74, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] - -(69) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#86, d_year#87] - -(70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#85] -Right keys [1]: [d_date_sk#86] -Join type: Inner -Join condition: None - -(71) Project [codegen id : 14] -Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, ws_ext_discount_amt#83, ws_ext_list_price#84, d_year#87] -Input [12]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85, d_date_sk#86, d_year#87] - -(72) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, ws_ext_discount_amt#83, ws_ext_list_price#84, d_year#87] -Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#84 - ws_ext_discount_amt#83)))] -Aggregate Attributes [1]: [sum#88] -Results [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87, sum#89] - -(73) CometColumnarExchange -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87, sum#89] -Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(74) CometColumnarToRow [codegen id : 15] -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87, sum#89] - -(75) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87, sum#89] -Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#84 - ws_ext_discount_amt#83)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#84 - ws_ext_discount_amt#83)))#71] -Results [2]: [c_customer_id#57 AS customer_id#90, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#84 - ws_ext_discount_amt#83)))#71,18,2) AS year_total#91] - -(76) BroadcastExchange -Input [2]: [customer_id#90, year_total#91] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] - -(77) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#90] -Join type: Inner -Join condition: (CASE WHEN (year_total#73 > 0.00) THEN (year_total#91 / year_total#73) ELSE 0E-20 END > CASE WHEN (year_total#26 > 0.00) THEN (year_total#48 / year_total#26) ELSE 0E-20 END) - -(78) Project [codegen id : 16] -Output [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] -Input [10]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, year_total#73, customer_id#90, year_total#91] - -(79) TakeOrderedAndProject -Input [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] -Arguments: 100, [customer_id#44 ASC NULLS FIRST, customer_first_name#45 ASC NULLS FIRST, customer_last_name#46 ASC NULLS FIRST, customer_email_address#47 ASC NULLS FIRST], [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometFilter (81) - +- CometNativeScan parquet spark_catalog.default.date_dim (80) - - -(80) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(81) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(82) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#20, d_year#21] - -(83) BroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 -BroadcastExchange (87) -+- * CometColumnarToRow (86) - +- CometFilter (85) - +- CometNativeScan parquet spark_catalog.default.date_dim (84) - - -(84) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#40, d_year#41] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(85) CometFilter -Input [2]: [d_date_sk#40, d_year#41] -Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#40)) - -(86) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#40, d_year#41] - -(87) BroadcastExchange -Input [2]: [d_date_sk#40, d_year#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] - -Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#66 IN dynamicpruning#19 - -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#85 IN dynamicpruning#39 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/simplified.txt deleted file mode 100644 index 00a3e659d4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/simplified.txt +++ /dev/null @@ -1,130 +0,0 @@ -TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - WholeStageCodegen (16) - Project [customer_id,customer_first_name,customer_last_name,customer_email_address] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - WholeStageCodegen (6) - HashAggregate [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] [sum,sum] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) - HashAggregate [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] [sum,sum] - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) - HashAggregate [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] [sum,sum] - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/explain.txt deleted file mode 100644 index 49c1c8e0d7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,470 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometProject (53) - : +- CometBroadcastHashJoin (52) - : :- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (51) - : +- CometFilter (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : +- CometBroadcastExchange (41) - : : +- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) - : +- ReusedExchange (44) - +- CometBroadcastExchange (68) - +- CometHashAggregate (67) - +- CometExchange (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometFilter (55) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (62) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#15) - -(6) CometBroadcastExchange -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(7) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_sk#1], [ss_customer_sk#15], Inner, BuildRight - -(8) CometProject -Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: [d_date_sk#20, d_year#21] - -(12) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Right output [2]: [d_date_sk#20, d_year#21] -Arguments: [ss_sold_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(13) CometProject -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] - -(14) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(15) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(17) CometFilter -Input [2]: [customer_id#23, year_total#24] -Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true))) - -(20) CometProject -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Arguments: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14], [c_customer_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#29, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#31, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#32, 50, true, false, true) AS c_email_address#14] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#36), dynamicpruningexpression(ss_sold_date_sk#36 IN dynamicpruning#37)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Condition : isnotnull(ss_customer_sk#33) - -(23) CometBroadcastExchange -Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] - -(24) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [c_customer_sk#25], [ss_customer_sk#33], Inner, BuildRight - -(25) CometProject -Input [12]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: [d_date_sk#38, d_year#39] - -(29) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Right output [2]: [d_date_sk#38, d_year#39] -Arguments: [ss_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight - -(30) CometProject -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36, d_date_sk#38, d_year#39] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] - -(31) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] - -(32) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] - -(34) CometBroadcastExchange -Input [5]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] -Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#23, year_total#24] -Right output [5]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] -Arguments: [customer_id#23], [customer_id#41], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] -Condition : (isnotnull(c_customer_sk#46) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#47, 16, true, false, true))) - -(38) CometProject -Input [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] -Arguments: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59], [c_customer_sk#46, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#47, 16, true, false, true) AS c_customer_id#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#48, 20, true, false, true) AS c_first_name#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#49, 30, true, false, true) AS c_last_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#50, 1, true, false, true) AS c_preferred_cust_flag#57, c_birth_country#51, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#52, 13, true, false, true) AS c_login#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#53, 50, true, false, true) AS c_email_address#59] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_bill_customer_sk#60) - -(41) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Arguments: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] - -(42) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59] -Right output [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Arguments: [c_customer_sk#46], [ws_bill_customer_sk#60], Inner, BuildRight - -(43) CometProject -Input [12]: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#65, d_year#66] - -(45) CometBroadcastHashJoin -Left output [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Right output [2]: [d_date_sk#65, d_year#66] -Arguments: [ws_sold_date_sk#63], [d_date_sk#65], Inner, BuildRight - -(46) CometProject -Input [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63, d_date_sk#65, d_year#66] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66] - -(47) CometHashAggregate -Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))] - -(48) CometExchange -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, sum#67] -Arguments: hashpartitioning(c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, sum#67] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))] - -(50) CometFilter -Input [2]: [customer_id#68, year_total#69] -Condition : (isnotnull(year_total#69) AND (year_total#69 > 0.00)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#68, year_total#69] -Arguments: [customer_id#68, year_total#69] - -(52) CometBroadcastHashJoin -Left output [7]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] -Right output [2]: [customer_id#68, year_total#69] -Arguments: [customer_id#23], [customer_id#68], Inner, BuildRight - -(53) CometProject -Input [9]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, customer_id#68, year_total#69] -Arguments: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69], [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] -Condition : (isnotnull(c_customer_sk#70) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#71, 16, true, false, true))) - -(56) CometProject -Input [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] -Arguments: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59], [c_customer_sk#70, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#71, 16, true, false, true) AS c_customer_id#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#72, 20, true, false, true) AS c_first_name#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#73, 30, true, false, true) AS c_last_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#74, 1, true, false, true) AS c_preferred_cust_flag#57, c_birth_country#75, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#76, 13, true, false, true) AS c_login#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#77, 50, true, false, true) AS c_email_address#59] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_bill_customer_sk#78) - -(59) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Arguments: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] - -(60) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59] -Right output [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Arguments: [c_customer_sk#70], [ws_bill_customer_sk#78], Inner, BuildRight - -(61) CometProject -Input [12]: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#83, d_year#84] - -(63) CometBroadcastHashJoin -Left output [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Right output [2]: [d_date_sk#83, d_year#84] -Arguments: [ws_sold_date_sk#81], [d_date_sk#83], Inner, BuildRight - -(64) CometProject -Input [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81, d_date_sk#83, d_year#84] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84] - -(65) CometHashAggregate -Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#80 - ws_ext_discount_amt#79)))] - -(66) CometExchange -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, sum#85] -Arguments: hashpartitioning(c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, sum#85] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#80 - ws_ext_discount_amt#79)))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#86, year_total#87] -Arguments: [customer_id#86, year_total#87] - -(69) CometBroadcastHashJoin -Left output [8]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69] -Right output [2]: [customer_id#86, year_total#87] -Arguments: [customer_id#23], [customer_id#86], Inner, (CASE WHEN (year_total#69 > 0.00) THEN (year_total#87 / year_total#69) ELSE 0E-20 END > CASE WHEN (year_total#24 > 0.00) THEN (year_total#45 / year_total#24) ELSE 0E-20 END), BuildRight - -(70) CometProject -Input [10]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69, customer_id#86, year_total#87] -Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44], [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] - -(71) CometTakeOrderedAndProject -Input [4]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#41 ASC NULLS FIRST,customer_first_name#42 ASC NULLS FIRST,customer_last_name#43 ASC NULLS FIRST,customer_email_address#44 ASC NULLS FIRST], output=[customer_id#41,customer_first_name#42,customer_last_name#43,customer_email_address#44]), [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44], 100, 0, [customer_id#41 ASC NULLS FIRST, customer_first_name#42 ASC NULLS FIRST, customer_last_name#43 ASC NULLS FIRST, customer_email_address#44 ASC NULLS FIRST], [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] - -(72) CometColumnarToRow [codegen id : 1] -Input [4]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(75) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#20, d_year#21] - -(76) BroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#36 IN dynamicpruning#37 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) - -(79) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#38, d_year#39] - -(80) BroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#19 - -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#37 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/simplified.txt deleted file mode 100644 index 157d1d587e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - CometProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] #5 - CometHashAggregate [d_year,c_preferred_cust_flag,c_birth_country,c_login,sum] [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_email_address,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/explain.txt deleted file mode 100644 index 49c1c8e0d7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/explain.txt +++ /dev/null @@ -1,470 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometProject (53) - : +- CometBroadcastHashJoin (52) - : :- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (51) - : +- CometFilter (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : +- CometBroadcastExchange (41) - : : +- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) - : +- ReusedExchange (44) - +- CometBroadcastExchange (68) - +- CometHashAggregate (67) - +- CometExchange (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometFilter (55) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (62) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#15) - -(6) CometBroadcastExchange -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(7) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_sk#1], [ss_customer_sk#15], Inner, BuildRight - -(8) CometProject -Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: [d_date_sk#20, d_year#21] - -(12) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Right output [2]: [d_date_sk#20, d_year#21] -Arguments: [ss_sold_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(13) CometProject -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] - -(14) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(15) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(17) CometFilter -Input [2]: [customer_id#23, year_total#24] -Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true))) - -(20) CometProject -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Arguments: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14], [c_customer_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#29, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#31, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#32, 50, true, false, true) AS c_email_address#14] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#36), dynamicpruningexpression(ss_sold_date_sk#36 IN dynamicpruning#37)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Condition : isnotnull(ss_customer_sk#33) - -(23) CometBroadcastExchange -Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] - -(24) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [c_customer_sk#25], [ss_customer_sk#33], Inner, BuildRight - -(25) CometProject -Input [12]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: [d_date_sk#38, d_year#39] - -(29) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Right output [2]: [d_date_sk#38, d_year#39] -Arguments: [ss_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight - -(30) CometProject -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36, d_date_sk#38, d_year#39] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] - -(31) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] - -(32) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] - -(34) CometBroadcastExchange -Input [5]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] -Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#23, year_total#24] -Right output [5]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] -Arguments: [customer_id#23], [customer_id#41], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] -Condition : (isnotnull(c_customer_sk#46) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#47, 16, true, false, true))) - -(38) CometProject -Input [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] -Arguments: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59], [c_customer_sk#46, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#47, 16, true, false, true) AS c_customer_id#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#48, 20, true, false, true) AS c_first_name#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#49, 30, true, false, true) AS c_last_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#50, 1, true, false, true) AS c_preferred_cust_flag#57, c_birth_country#51, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#52, 13, true, false, true) AS c_login#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#53, 50, true, false, true) AS c_email_address#59] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_bill_customer_sk#60) - -(41) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Arguments: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] - -(42) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59] -Right output [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Arguments: [c_customer_sk#46], [ws_bill_customer_sk#60], Inner, BuildRight - -(43) CometProject -Input [12]: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#65, d_year#66] - -(45) CometBroadcastHashJoin -Left output [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Right output [2]: [d_date_sk#65, d_year#66] -Arguments: [ws_sold_date_sk#63], [d_date_sk#65], Inner, BuildRight - -(46) CometProject -Input [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63, d_date_sk#65, d_year#66] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66] - -(47) CometHashAggregate -Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))] - -(48) CometExchange -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, sum#67] -Arguments: hashpartitioning(c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, sum#67] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))] - -(50) CometFilter -Input [2]: [customer_id#68, year_total#69] -Condition : (isnotnull(year_total#69) AND (year_total#69 > 0.00)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#68, year_total#69] -Arguments: [customer_id#68, year_total#69] - -(52) CometBroadcastHashJoin -Left output [7]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] -Right output [2]: [customer_id#68, year_total#69] -Arguments: [customer_id#23], [customer_id#68], Inner, BuildRight - -(53) CometProject -Input [9]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, customer_id#68, year_total#69] -Arguments: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69], [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] -Condition : (isnotnull(c_customer_sk#70) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#71, 16, true, false, true))) - -(56) CometProject -Input [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] -Arguments: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59], [c_customer_sk#70, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#71, 16, true, false, true) AS c_customer_id#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#72, 20, true, false, true) AS c_first_name#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#73, 30, true, false, true) AS c_last_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#74, 1, true, false, true) AS c_preferred_cust_flag#57, c_birth_country#75, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#76, 13, true, false, true) AS c_login#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#77, 50, true, false, true) AS c_email_address#59] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_bill_customer_sk#78) - -(59) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Arguments: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] - -(60) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59] -Right output [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Arguments: [c_customer_sk#70], [ws_bill_customer_sk#78], Inner, BuildRight - -(61) CometProject -Input [12]: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#83, d_year#84] - -(63) CometBroadcastHashJoin -Left output [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Right output [2]: [d_date_sk#83, d_year#84] -Arguments: [ws_sold_date_sk#81], [d_date_sk#83], Inner, BuildRight - -(64) CometProject -Input [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81, d_date_sk#83, d_year#84] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84] - -(65) CometHashAggregate -Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#80 - ws_ext_discount_amt#79)))] - -(66) CometExchange -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, sum#85] -Arguments: hashpartitioning(c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, sum#85] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#80 - ws_ext_discount_amt#79)))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#86, year_total#87] -Arguments: [customer_id#86, year_total#87] - -(69) CometBroadcastHashJoin -Left output [8]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69] -Right output [2]: [customer_id#86, year_total#87] -Arguments: [customer_id#23], [customer_id#86], Inner, (CASE WHEN (year_total#69 > 0.00) THEN (year_total#87 / year_total#69) ELSE 0E-20 END > CASE WHEN (year_total#24 > 0.00) THEN (year_total#45 / year_total#24) ELSE 0E-20 END), BuildRight - -(70) CometProject -Input [10]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69, customer_id#86, year_total#87] -Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44], [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] - -(71) CometTakeOrderedAndProject -Input [4]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#41 ASC NULLS FIRST,customer_first_name#42 ASC NULLS FIRST,customer_last_name#43 ASC NULLS FIRST,customer_email_address#44 ASC NULLS FIRST], output=[customer_id#41,customer_first_name#42,customer_last_name#43,customer_email_address#44]), [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44], 100, 0, [customer_id#41 ASC NULLS FIRST, customer_first_name#42 ASC NULLS FIRST, customer_last_name#43 ASC NULLS FIRST, customer_email_address#44 ASC NULLS FIRST], [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] - -(72) CometColumnarToRow [codegen id : 1] -Input [4]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(75) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#20, d_year#21] - -(76) BroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#36 IN dynamicpruning#37 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) - -(79) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#38, d_year#39] - -(80) BroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#19 - -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#37 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/extended.txt deleted file mode 100644 index 476c7be954..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/extended.txt +++ /dev/null @@ -1,90 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 157d1d587e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - CometProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] #5 - CometHashAggregate [d_year,c_preferred_cust_flag,c_birth_country,c_login,sum] [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_email_address,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/explain.txt deleted file mode 100644 index 2ebf9db593..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/explain.txt +++ /dev/null @@ -1,163 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.web_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.item (4) - +- ReusedExchange (11) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(7) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(8) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(11) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] - -(14) HashAggregate [codegen id : 3] -Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(15) CometColumnarExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(17) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#17] -Results [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#17,17,2) AS _w0#19] - -(18) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(22) Project [codegen id : 6] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] - -(23) TakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometNativeScan parquet spark_catalog.default.date_dim (24) - - -(24) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#22] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/simplified.txt deleted file mode 100644 index c129b42cdb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/explain.txt deleted file mode 100644 index 410aecff1b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ws_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/simplified.txt deleted file mode 100644 index 58ac81ba0d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/explain.txt deleted file mode 100644 index 410aecff1b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ws_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/extended.txt deleted file mode 100644 index 3f41c97ff5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 58ac81ba0d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/explain.txt deleted file mode 100644 index 59342eea7f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/explain.txt +++ /dev/null @@ -1,769 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (87) -+- * BroadcastHashJoin Inner BuildRight (86) - :- * Filter (68) - : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) - : +- CometColumnarExchange (65) - : +- * HashAggregate (64) - : +- * Project (63) - : +- * BroadcastHashJoin Inner BuildRight (62) - : :- * Project (60) - : : +- * BroadcastHashJoin Inner BuildRight (59) - : : :- * BroadcastHashJoin LeftSemi BuildRight (52) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * CometColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (48) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * CometColumnarToRow (36) - : : : : +- CometHashAggregate (35) - : : : : +- CometColumnarExchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * CometColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * CometColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * Filter (39) - : : : : : +- * ColumnarToRow (38) - : : : : : +- Scan parquet spark_catalog.default.web_sales (37) - : : : : +- ReusedExchange (40) - : : : +- ReusedExchange (43) - : : +- BroadcastExchange (58) - : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : :- * CometColumnarToRow (55) - : : : +- CometFilter (54) - : : : +- CometNativeScan parquet spark_catalog.default.item (53) - : : +- ReusedExchange (56) - : +- ReusedExchange (61) - +- BroadcastExchange (85) - +- * Filter (84) - +- * HashAggregate (83) - +- * CometColumnarToRow (82) - +- CometColumnarExchange (81) - +- * HashAggregate (80) - +- * Project (79) - +- * BroadcastHashJoin Inner BuildRight (78) - :- * Project (76) - : +- * BroadcastHashJoin Inner BuildRight (75) - : :- * BroadcastHashJoin LeftSemi BuildRight (73) - : : :- * Filter (71) - : : : +- * ColumnarToRow (70) - : : : +- Scan parquet spark_catalog.default.store_sales (69) - : : +- ReusedExchange (72) - : +- ReusedExchange (74) - +- ReusedExchange (77) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(6) CometColumnarToRow [codegen id : 11] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(10) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(11) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(12) CometColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(16) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) - -(18) CometColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(19) BroadcastExchange -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#19] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(22) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#23] - -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#23] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#23] - -(25) BroadcastExchange -Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] -Join type: LeftSemi -Join condition: None - -(27) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None - -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(30) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#24] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#25, i_class_id#15 AS class_id#26, i_category_id#16 AS category_id#27] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#24] - -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#25, class_id#26, category_id#27] - -(34) CometColumnarExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(35) CometHashAggregate -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] - -(36) CometColumnarToRow [codegen id : 10] -Input [3]: [brand_id#25, class_id#26, category_id#27] - -(37) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] - -(39) Filter [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Condition : isnotnull(ws_item_sk#28) - -(40) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#28] -Right keys [1]: [i_item_sk#30] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] -Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(43) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#34] - -(44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#29] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 9] -Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] - -(46) BroadcastExchange -Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] - -(47) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] -Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] -Join type: LeftSemi -Join condition: None - -(48) BroadcastExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] - -(49) BroadcastHashJoin [codegen id : 11] -Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#25, class_id#26, category_id#27] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#35] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] - -(51) BroadcastExchange -Input [1]: [ss_item_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(52) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(53) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(54) CometFilter -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Condition : (((isnotnull(i_item_sk#36) AND isnotnull(i_brand_id#37)) AND isnotnull(i_class_id#38)) AND isnotnull(i_category_id#39)) - -(55) CometColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(56) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#35] - -(57) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#36] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(58) BroadcastExchange -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(59) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#36] -Join type: Inner -Join condition: None - -(60) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(61) ReusedExchange [Reuses operator id: 112] -Output [1]: [d_date_sk#40] - -(62) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] - -(64) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] -Results [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(65) CometColumnarExchange -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Arguments: hashpartitioning(i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(66) CometColumnarToRow [codegen id : 52] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(67) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] -Results [6]: [store AS channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51] - -(68) Filter [codegen id : 52] -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51] -Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(69) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_sold_date_sk#57 IN dynamicpruning#58)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(70) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] - -(71) Filter [codegen id : 50] -Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -Condition : isnotnull(ss_item_sk#54) - -(72) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#59] - -(73) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [ss_item_sk#59] -Join type: LeftSemi -Join condition: None - -(74) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] - -(75) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [i_item_sk#60] -Join type: Inner -Join condition: None - -(76) Project [codegen id : 50] -Output [6]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] -Input [8]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] - -(77) ReusedExchange [Reuses operator id: 126] -Output [1]: [d_date_sk#64] - -(78) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#57] -Right keys [1]: [d_date_sk#64] -Join type: Inner -Join condition: None - -(79) Project [codegen id : 50] -Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] -Input [7]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] - -(80) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)] -Aggregate Attributes [3]: [sum#65, isEmpty#66, count#67] -Results [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] - -(81) CometColumnarExchange -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] -Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(82) CometColumnarToRow [codegen id : 51] -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] - -(83) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#71, count(1)#72] -Results [6]: [store AS channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#71 AS sales#74, count(1)#72 AS number_sales#75] - -(84) Filter [codegen id : 51] -Input [6]: [channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] -Condition : (isnotnull(sales#74) AND (cast(sales#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(85) BroadcastExchange -Input [6]: [channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] - -(86) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Right keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Join type: Inner -Join condition: None - -(87) TakeOrderedAndProject -Input [12]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51, channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] -Arguments: 100, [i_brand_id#37 ASC NULLS FIRST, i_class_id#38 ASC NULLS FIRST, i_category_id#39 ASC NULLS FIRST], [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51, channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* HashAggregate (107) -+- * CometColumnarToRow (106) - +- CometColumnarExchange (105) - +- * HashAggregate (104) - +- Union (103) - :- * Project (92) - : +- * BroadcastHashJoin Inner BuildRight (91) - : :- * ColumnarToRow (89) - : : +- Scan parquet spark_catalog.default.store_sales (88) - : +- ReusedExchange (90) - :- * Project (97) - : +- * BroadcastHashJoin Inner BuildRight (96) - : :- * ColumnarToRow (94) - : : +- Scan parquet spark_catalog.default.catalog_sales (93) - : +- ReusedExchange (95) - +- * Project (102) - +- * BroadcastHashJoin Inner BuildRight (101) - :- * ColumnarToRow (99) - : +- Scan parquet spark_catalog.default.web_sales (98) - +- ReusedExchange (100) - - -(88) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#12)] -ReadSchema: struct - -(89) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] - -(90) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#79] - -(91) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#78] -Right keys [1]: [d_date_sk#79] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 2] -Output [2]: [ss_quantity#76 AS quantity#80, ss_list_price#77 AS list_price#81] -Input [4]: [ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78, d_date_sk#79] - -(93) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#12)] -ReadSchema: struct - -(94) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] - -(95) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#85] - -(96) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#84] -Right keys [1]: [d_date_sk#85] -Join type: Inner -Join condition: None - -(97) Project [codegen id : 4] -Output [2]: [cs_quantity#82 AS quantity#86, cs_list_price#83 AS list_price#87] -Input [4]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84, d_date_sk#85] - -(98) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#90), dynamicpruningexpression(ws_sold_date_sk#90 IN dynamicpruning#12)] -ReadSchema: struct - -(99) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] - -(100) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#91] - -(101) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#90] -Right keys [1]: [d_date_sk#91] -Join type: Inner -Join condition: None - -(102) Project [codegen id : 6] -Output [2]: [ws_quantity#88 AS quantity#92, ws_list_price#89 AS list_price#93] -Input [4]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90, d_date_sk#91] - -(103) Union - -(104) HashAggregate [codegen id : 7] -Input [2]: [quantity#80, list_price#81] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] -Aggregate Attributes [2]: [sum#94, count#95] -Results [2]: [sum#96, count#97] - -(105) CometColumnarExchange -Input [2]: [sum#96, count#97] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(106) CometColumnarToRow [codegen id : 8] -Input [2]: [sum#96, count#97] - -(107) HashAggregate [codegen id : 8] -Input [2]: [sum#96, count#97] -Keys: [] -Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] -Aggregate Attributes [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))#98] -Results [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))#98 AS average_sales#99] - -Subquery:2 Hosting operator id = 88 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 93 Hosting Expression = cs_sold_date_sk#84 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 98 Hosting Expression = ws_sold_date_sk#90 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (112) -+- * CometColumnarToRow (111) - +- CometProject (110) - +- CometFilter (109) - +- CometNativeScan parquet spark_catalog.default.date_dim (108) - - -(108) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#40, d_week_seq#100] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(109) CometFilter -Input [2]: [d_date_sk#40, d_week_seq#100] -Condition : ((isnotnull(d_week_seq#100) AND (d_week_seq#100 = Subquery scalar-subquery#101, [id=#102])) AND isnotnull(d_date_sk#40)) - -(110) CometProject -Input [2]: [d_date_sk#40, d_week_seq#100] -Arguments: [d_date_sk#40], [d_date_sk#40] - -(111) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#40] - -(112) BroadcastExchange -Input [1]: [d_date_sk#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:6 Hosting operator id = 109 Hosting Expression = Subquery scalar-subquery#101, [id=#102] -* CometColumnarToRow (116) -+- CometProject (115) - +- CometFilter (114) - +- CometNativeScan parquet spark_catalog.default.date_dim (113) - - -(113) CometNativeScan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] -ReadSchema: struct - -(114) CometFilter -Input [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] -Condition : (((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND isnotnull(d_dom#106)) AND (d_year#104 = 1999)) AND (d_moy#105 = 12)) AND (d_dom#106 = 16)) - -(115) CometProject -Input [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] -Arguments: [d_week_seq#103], [d_week_seq#103] - -(116) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#103] - -Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometNativeScan parquet spark_catalog.default.date_dim (117) - - -(117) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#107] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#24, d_year#107] -Condition : (((isnotnull(d_year#107) AND (d_year#107 >= 1998)) AND (d_year#107 <= 2000)) AND isnotnull(d_date_sk#24)) - -(119) CometProject -Input [2]: [d_date_sk#24, d_year#107] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#24] - -(121) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:8 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 - -Subquery:10 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:11 Hosting operator id = 69 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 -BroadcastExchange (126) -+- * CometColumnarToRow (125) - +- CometProject (124) - +- CometFilter (123) - +- CometNativeScan parquet spark_catalog.default.date_dim (122) - - -(122) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#64, d_week_seq#108] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(123) CometFilter -Input [2]: [d_date_sk#64, d_week_seq#108] -Condition : ((isnotnull(d_week_seq#108) AND (d_week_seq#108 = Subquery scalar-subquery#109, [id=#110])) AND isnotnull(d_date_sk#64)) - -(124) CometProject -Input [2]: [d_date_sk#64, d_week_seq#108] -Arguments: [d_date_sk#64], [d_date_sk#64] - -(125) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#64] - -(126) BroadcastExchange -Input [1]: [d_date_sk#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] - -Subquery:12 Hosting operator id = 123 Hosting Expression = Subquery scalar-subquery#109, [id=#110] -* CometColumnarToRow (130) -+- CometProject (129) - +- CometFilter (128) - +- CometNativeScan parquet spark_catalog.default.date_dim (127) - - -(127) CometNativeScan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#111, d_year#112, d_moy#113, d_dom#114] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] -ReadSchema: struct - -(128) CometFilter -Input [4]: [d_week_seq#111, d_year#112, d_moy#113, d_dom#114] -Condition : (((((isnotnull(d_year#112) AND isnotnull(d_moy#113)) AND isnotnull(d_dom#114)) AND (d_year#112 = 1998)) AND (d_moy#113 = 12)) AND (d_dom#114 = 16)) - -(129) CometProject -Input [4]: [d_week_seq#111, d_year#112, d_moy#113, d_dom#114] -Arguments: [d_week_seq#111], [d_week_seq#111] - -(130) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#111] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/simplified.txt deleted file mode 100644 index b0eae963c3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/simplified.txt +++ /dev/null @@ -1,206 +0,0 @@ -TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (52) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Filter [sales] - Subquery #4 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #12 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (51) - Filter [sales] - ReusedSubquery [average_sales] #4 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (50) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #15 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/explain.txt deleted file mode 100644 index ea550c6c74..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,743 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (85) -+- CometTakeOrderedAndProject (84) - +- CometBroadcastHashJoin (83) - :- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometBroadcastHashJoin (47) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (46) - : : : +- CometProject (45) - : : : +- CometBroadcastHashJoin (44) - : : : :- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (43) - : : : +- CometBroadcastHashJoin (42) - : : : :- CometHashAggregate (32) - : : : : +- CometExchange (31) - : : : : +- CometHashAggregate (30) - : : : : +- CometProject (29) - : : : : +- CometBroadcastHashJoin (28) - : : : : :- CometProject (26) - : : : : : +- CometBroadcastHashJoin (25) - : : : : : :- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : +- CometBroadcastExchange (24) - : : : : : +- CometBroadcastHashJoin (23) - : : : : : :- CometFilter (8) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : +- CometBroadcastExchange (22) - : : : : : +- CometProject (21) - : : : : : +- CometBroadcastHashJoin (20) - : : : : : :- CometProject (15) - : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : :- CometFilter (10) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : +- CometBroadcastExchange (13) - : : : : : : +- CometFilter (12) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : +- CometBroadcastExchange (19) - : : : : : +- CometProject (18) - : : : : : +- CometFilter (17) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : +- ReusedExchange (27) - : : : +- CometBroadcastExchange (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (34) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometFilter (49) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : +- ReusedExchange (50) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometBroadcastExchange (82) - +- CometFilter (81) - +- CometHashAggregate (80) - +- CometExchange (79) - +- CometHashAggregate (78) - +- CometProject (77) - +- CometBroadcastHashJoin (76) - :- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometBroadcastHashJoin (68) - : : :- CometFilter (66) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (65) - : : +- ReusedExchange (67) - : +- ReusedExchange (69) - +- CometBroadcastExchange (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1998)) AND (d_year#25 <= 2000)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42)) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Condition : (isnotnull(sales#51) AND (cast(sales#51 as decimal(32,6)) > cast(Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Condition : isnotnull(ss_item_sk#55) - -(67) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#60] - -(68) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [1]: [ss_item_sk#60] -Arguments: [ss_item_sk#55], [ss_item_sk#60], LeftSemi, BuildRight - -(69) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] - -(70) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -Arguments: [ss_item_sk#55], [i_item_sk#61], Inner, BuildRight - -(71) CometProject -Input [8]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -Arguments: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#66] -Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = ReusedSubquery Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) - -(74) CometProject -Input [2]: [d_date_sk#65, d_week_seq#66] -Arguments: [d_date_sk#65], [d_date_sk#65] - -(75) CometBroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: [d_date_sk#65] - -(76) CometBroadcastHashJoin -Left output [6]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(77) CometProject -Input [7]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] -Arguments: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] - -(78) CometHashAggregate -Input [5]: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [partial_sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), partial_count(1)] - -(79) CometExchange -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(80) CometHashAggregate -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), count(1)] - -(81) CometFilter -Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(82) CometBroadcastExchange -Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -(83) CometBroadcastHashJoin -Left output [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Right output [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#62, i_class_id#63, i_category_id#64], Inner, BuildRight - -(84) CometTakeOrderedAndProject -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#72,i_brand_id#62,i_class_id#63,i_category_id#64,sales#73,number_sales#74]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74], 100, 0, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -(85) CometColumnarToRow [codegen id : 1] -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* CometColumnarToRow (102) -+- CometHashAggregate (101) - +- CometExchange (100) - +- CometHashAggregate (99) - +- CometUnion (98) - :- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (86) - : +- ReusedExchange (87) - :- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (90) - : +- ReusedExchange (91) - +- CometProject (97) - +- CometBroadcastHashJoin (96) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (94) - +- ReusedExchange (95) - - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#78)] -ReadSchema: struct - -(87) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#79] - -(88) CometBroadcastHashJoin -Left output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -Right output [1]: [d_date_sk#79] -Arguments: [ss_sold_date_sk#77], [d_date_sk#79], Inner, BuildRight - -(89) CometProject -Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#79] -Arguments: [quantity#80, list_price#81], [ss_quantity#75 AS quantity#80, ss_list_price#76 AS list_price#81] - -(90) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#85)] -ReadSchema: struct - -(91) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#86] - -(92) CometBroadcastHashJoin -Left output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] -Right output [1]: [d_date_sk#86] -Arguments: [cs_sold_date_sk#84], [d_date_sk#86], Inner, BuildRight - -(93) CometProject -Input [4]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84, d_date_sk#86] -Arguments: [quantity#87, list_price#88], [cs_quantity#82 AS quantity#87, cs_list_price#83 AS list_price#88] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#91), dynamicpruningexpression(ws_sold_date_sk#91 IN dynamicpruning#92)] -ReadSchema: struct - -(95) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#93] - -(96) CometBroadcastHashJoin -Left output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] -Right output [1]: [d_date_sk#93] -Arguments: [ws_sold_date_sk#91], [d_date_sk#93], Inner, BuildRight - -(97) CometProject -Input [4]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91, d_date_sk#93] -Arguments: [quantity#94, list_price#95], [ws_quantity#89 AS quantity#94, ws_list_price#90 AS list_price#95] - -(98) CometUnion -Child 0 Input [2]: [quantity#80, list_price#81] -Child 1 Input [2]: [quantity#87, list_price#88] -Child 2 Input [2]: [quantity#94, list_price#95] - -(99) CometHashAggregate -Input [2]: [quantity#80, list_price#81] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] - -(100) CometExchange -Input [2]: [sum#96, count#97] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(101) CometHashAggregate -Input [2]: [sum#96, count#97] -Keys: [] -Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] - -(102) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#98] - -Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#84 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (107) -+- * CometColumnarToRow (106) - +- CometProject (105) - +- CometFilter (104) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) - - -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(104) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(105) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(106) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(107) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) -+- CometProject (110) - +- CometFilter (109) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) - - -(108) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] -ReadSchema: struct - -(109) CometFilter -Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d_dom#102)) AND (d_year#100 = 1999)) AND (d_moy#101 = 12)) AND (d_dom#102 = 16)) - -(110) CometProject -Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Arguments: [d_week_seq#99], [d_week_seq#99] - -(111) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#99] - -Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (116) -+- * CometColumnarToRow (115) - +- CometProject (114) - +- CometFilter (113) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) - - -(112) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#103] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(113) CometFilter -Input [2]: [d_date_sk#26, d_year#103] -Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 <= 2000)) AND isnotnull(d_date_sk#26)) - -(114) CometProject -Input [2]: [d_date_sk#26, d_year#103] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(115) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(116) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:8 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:10 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] - -Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] - -Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#66] -Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) - -(119) CometProject -Input [2]: [d_date_sk#65, d_week_seq#66] -Arguments: [d_date_sk#65], [d_date_sk#65] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#65] - -(121) BroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* CometColumnarToRow (125) -+- CometProject (124) - +- CometFilter (123) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) - - -(122) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] -ReadSchema: struct - -(123) CometFilter -Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d_dom#107)) AND (d_year#105 = 1998)) AND (d_moy#106 = 12)) AND (d_dom#107 = 16)) - -(124) CometProject -Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Arguments: [d_week_seq#104], [d_week_seq#104] - -(125) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#104] - -Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/simplified.txt deleted file mode 100644 index fb9abae378..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,153 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #14 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #1 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [ss_item_sk] #3 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #4 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #5 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #10 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #11 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #3 - CometBroadcastExchange [d_date_sk] #13 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #4 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - ReusedExchange [ss_item_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastExchange [d_date_sk] #18 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/explain.txt deleted file mode 100644 index ea550c6c74..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/explain.txt +++ /dev/null @@ -1,743 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (85) -+- CometTakeOrderedAndProject (84) - +- CometBroadcastHashJoin (83) - :- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometBroadcastHashJoin (47) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (46) - : : : +- CometProject (45) - : : : +- CometBroadcastHashJoin (44) - : : : :- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (43) - : : : +- CometBroadcastHashJoin (42) - : : : :- CometHashAggregate (32) - : : : : +- CometExchange (31) - : : : : +- CometHashAggregate (30) - : : : : +- CometProject (29) - : : : : +- CometBroadcastHashJoin (28) - : : : : :- CometProject (26) - : : : : : +- CometBroadcastHashJoin (25) - : : : : : :- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : +- CometBroadcastExchange (24) - : : : : : +- CometBroadcastHashJoin (23) - : : : : : :- CometFilter (8) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : +- CometBroadcastExchange (22) - : : : : : +- CometProject (21) - : : : : : +- CometBroadcastHashJoin (20) - : : : : : :- CometProject (15) - : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : :- CometFilter (10) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : +- CometBroadcastExchange (13) - : : : : : : +- CometFilter (12) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : +- CometBroadcastExchange (19) - : : : : : +- CometProject (18) - : : : : : +- CometFilter (17) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : +- ReusedExchange (27) - : : : +- CometBroadcastExchange (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (34) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometFilter (49) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : +- ReusedExchange (50) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometBroadcastExchange (82) - +- CometFilter (81) - +- CometHashAggregate (80) - +- CometExchange (79) - +- CometHashAggregate (78) - +- CometProject (77) - +- CometBroadcastHashJoin (76) - :- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometBroadcastHashJoin (68) - : : :- CometFilter (66) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (65) - : : +- ReusedExchange (67) - : +- ReusedExchange (69) - +- CometBroadcastExchange (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1998)) AND (d_year#25 <= 2000)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42)) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Condition : (isnotnull(sales#51) AND (cast(sales#51 as decimal(32,6)) > cast(Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Condition : isnotnull(ss_item_sk#55) - -(67) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#60] - -(68) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [1]: [ss_item_sk#60] -Arguments: [ss_item_sk#55], [ss_item_sk#60], LeftSemi, BuildRight - -(69) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] - -(70) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -Arguments: [ss_item_sk#55], [i_item_sk#61], Inner, BuildRight - -(71) CometProject -Input [8]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -Arguments: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#66] -Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = ReusedSubquery Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) - -(74) CometProject -Input [2]: [d_date_sk#65, d_week_seq#66] -Arguments: [d_date_sk#65], [d_date_sk#65] - -(75) CometBroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: [d_date_sk#65] - -(76) CometBroadcastHashJoin -Left output [6]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(77) CometProject -Input [7]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] -Arguments: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] - -(78) CometHashAggregate -Input [5]: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [partial_sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), partial_count(1)] - -(79) CometExchange -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(80) CometHashAggregate -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), count(1)] - -(81) CometFilter -Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(82) CometBroadcastExchange -Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -(83) CometBroadcastHashJoin -Left output [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Right output [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#62, i_class_id#63, i_category_id#64], Inner, BuildRight - -(84) CometTakeOrderedAndProject -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#72,i_brand_id#62,i_class_id#63,i_category_id#64,sales#73,number_sales#74]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74], 100, 0, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -(85) CometColumnarToRow [codegen id : 1] -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* CometColumnarToRow (102) -+- CometHashAggregate (101) - +- CometExchange (100) - +- CometHashAggregate (99) - +- CometUnion (98) - :- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (86) - : +- ReusedExchange (87) - :- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (90) - : +- ReusedExchange (91) - +- CometProject (97) - +- CometBroadcastHashJoin (96) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (94) - +- ReusedExchange (95) - - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#78)] -ReadSchema: struct - -(87) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#79] - -(88) CometBroadcastHashJoin -Left output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -Right output [1]: [d_date_sk#79] -Arguments: [ss_sold_date_sk#77], [d_date_sk#79], Inner, BuildRight - -(89) CometProject -Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#79] -Arguments: [quantity#80, list_price#81], [ss_quantity#75 AS quantity#80, ss_list_price#76 AS list_price#81] - -(90) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#85)] -ReadSchema: struct - -(91) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#86] - -(92) CometBroadcastHashJoin -Left output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] -Right output [1]: [d_date_sk#86] -Arguments: [cs_sold_date_sk#84], [d_date_sk#86], Inner, BuildRight - -(93) CometProject -Input [4]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84, d_date_sk#86] -Arguments: [quantity#87, list_price#88], [cs_quantity#82 AS quantity#87, cs_list_price#83 AS list_price#88] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#91), dynamicpruningexpression(ws_sold_date_sk#91 IN dynamicpruning#92)] -ReadSchema: struct - -(95) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#93] - -(96) CometBroadcastHashJoin -Left output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] -Right output [1]: [d_date_sk#93] -Arguments: [ws_sold_date_sk#91], [d_date_sk#93], Inner, BuildRight - -(97) CometProject -Input [4]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91, d_date_sk#93] -Arguments: [quantity#94, list_price#95], [ws_quantity#89 AS quantity#94, ws_list_price#90 AS list_price#95] - -(98) CometUnion -Child 0 Input [2]: [quantity#80, list_price#81] -Child 1 Input [2]: [quantity#87, list_price#88] -Child 2 Input [2]: [quantity#94, list_price#95] - -(99) CometHashAggregate -Input [2]: [quantity#80, list_price#81] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] - -(100) CometExchange -Input [2]: [sum#96, count#97] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(101) CometHashAggregate -Input [2]: [sum#96, count#97] -Keys: [] -Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] - -(102) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#98] - -Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#84 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (107) -+- * CometColumnarToRow (106) - +- CometProject (105) - +- CometFilter (104) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) - - -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(104) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(105) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(106) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(107) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) -+- CometProject (110) - +- CometFilter (109) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) - - -(108) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] -ReadSchema: struct - -(109) CometFilter -Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d_dom#102)) AND (d_year#100 = 1999)) AND (d_moy#101 = 12)) AND (d_dom#102 = 16)) - -(110) CometProject -Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Arguments: [d_week_seq#99], [d_week_seq#99] - -(111) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#99] - -Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (116) -+- * CometColumnarToRow (115) - +- CometProject (114) - +- CometFilter (113) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) - - -(112) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#103] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(113) CometFilter -Input [2]: [d_date_sk#26, d_year#103] -Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 <= 2000)) AND isnotnull(d_date_sk#26)) - -(114) CometProject -Input [2]: [d_date_sk#26, d_year#103] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(115) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(116) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:8 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:10 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] - -Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] - -Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#66] -Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) - -(119) CometProject -Input [2]: [d_date_sk#65, d_week_seq#66] -Arguments: [d_date_sk#65], [d_date_sk#65] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#65] - -(121) BroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* CometColumnarToRow (125) -+- CometProject (124) - +- CometFilter (123) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) - - -(122) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] -ReadSchema: struct - -(123) CometFilter -Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d_dom#107)) AND (d_year#105 = 1998)) AND (d_moy#106 = 12)) AND (d_dom#107 = 16)) - -(124) CometProject -Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Arguments: [d_week_seq#104], [d_week_seq#104] - -(125) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#104] - -Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/extended.txt deleted file mode 100644 index f56d229b68..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/extended.txt +++ /dev/null @@ -1,339 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometBroadcastHashJoin - :- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 298 out of 327 eligible operators (91%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index fb9abae378..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14/simplified.txt +++ /dev/null @@ -1,153 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #14 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #1 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [ss_item_sk] #3 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #4 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #5 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #10 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #11 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #3 - CometBroadcastExchange [d_date_sk] #13 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #4 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - ReusedExchange [ss_item_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastExchange [d_date_sk] #18 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/explain.txt deleted file mode 100644 index f6335b03dd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/explain.txt +++ /dev/null @@ -1,1020 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (139) -+- CometTakeOrderedAndProject (138) - +- CometHashAggregate (137) - +- CometColumnarExchange (136) - +- * HashAggregate (135) - +- Union (134) - :- * HashAggregate (105) - : +- * CometColumnarToRow (104) - : +- CometColumnarExchange (103) - : +- * HashAggregate (102) - : +- Union (101) - : :- * Filter (68) - : : +- * HashAggregate (67) - : : +- * CometColumnarToRow (66) - : : +- CometColumnarExchange (65) - : : +- * HashAggregate (64) - : : +- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildRight (59) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (52) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (51) - : : : : +- * Project (50) - : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : :- * CometColumnarToRow (6) - : : : : : +- CometFilter (5) - : : : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : : : +- BroadcastExchange (48) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : : :- * CometColumnarToRow (36) - : : : : : +- CometHashAggregate (35) - : : : : : +- CometColumnarExchange (34) - : : : : : +- * HashAggregate (33) - : : : : : +- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Project (29) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : : +- BroadcastExchange (27) - : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : : :- * CometColumnarToRow (12) - : : : : : : : +- CometFilter (11) - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) - : : : : : : +- BroadcastExchange (25) - : : : : : : +- * Project (24) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Filter (15) - : : : : : : : : +- * ColumnarToRow (14) - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * CometColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) - : : : : : : +- ReusedExchange (22) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (46) - : : : : +- * Project (45) - : : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : : :- * Project (42) - : : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : : :- * Filter (39) - : : : : : : +- * ColumnarToRow (38) - : : : : : : +- Scan parquet spark_catalog.default.web_sales (37) - : : : : : +- ReusedExchange (40) - : : : : +- ReusedExchange (43) - : : : +- BroadcastExchange (58) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : : :- * CometColumnarToRow (55) - : : : : +- CometFilter (54) - : : : : +- CometNativeScan parquet spark_catalog.default.item (53) - : : : +- ReusedExchange (56) - : : +- ReusedExchange (61) - : :- * Filter (84) - : : +- * HashAggregate (83) - : : +- * CometColumnarToRow (82) - : : +- CometColumnarExchange (81) - : : +- * HashAggregate (80) - : : +- * Project (79) - : : +- * BroadcastHashJoin Inner BuildRight (78) - : : :- * Project (76) - : : : +- * BroadcastHashJoin Inner BuildRight (75) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (73) - : : : : :- * Filter (71) - : : : : : +- * ColumnarToRow (70) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (69) - : : : : +- ReusedExchange (72) - : : : +- ReusedExchange (74) - : : +- ReusedExchange (77) - : +- * Filter (100) - : +- * HashAggregate (99) - : +- * CometColumnarToRow (98) - : +- CometColumnarExchange (97) - : +- * HashAggregate (96) - : +- * Project (95) - : +- * BroadcastHashJoin Inner BuildRight (94) - : :- * Project (92) - : : +- * BroadcastHashJoin Inner BuildRight (91) - : : :- * BroadcastHashJoin LeftSemi BuildRight (89) - : : : :- * Filter (87) - : : : : +- * ColumnarToRow (86) - : : : : +- Scan parquet spark_catalog.default.web_sales (85) - : : : +- ReusedExchange (88) - : : +- ReusedExchange (90) - : +- ReusedExchange (93) - :- * HashAggregate (112) - : +- * CometColumnarToRow (111) - : +- CometColumnarExchange (110) - : +- * HashAggregate (109) - : +- * HashAggregate (108) - : +- * CometColumnarToRow (107) - : +- ReusedExchange (106) - :- * HashAggregate (119) - : +- * CometColumnarToRow (118) - : +- CometColumnarExchange (117) - : +- * HashAggregate (116) - : +- * HashAggregate (115) - : +- * CometColumnarToRow (114) - : +- ReusedExchange (113) - :- * HashAggregate (126) - : +- * CometColumnarToRow (125) - : +- CometColumnarExchange (124) - : +- * HashAggregate (123) - : +- * HashAggregate (122) - : +- * CometColumnarToRow (121) - : +- ReusedExchange (120) - +- * HashAggregate (133) - +- * CometColumnarToRow (132) - +- CometColumnarExchange (131) - +- * HashAggregate (130) - +- * HashAggregate (129) - +- * CometColumnarToRow (128) - +- ReusedExchange (127) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(6) CometColumnarToRow [codegen id : 11] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(10) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(11) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(12) CometColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(16) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) - -(18) CometColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(19) BroadcastExchange -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#19] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(22) ReusedExchange [Reuses operator id: 174] -Output [1]: [d_date_sk#23] - -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#23] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#23] - -(25) BroadcastExchange -Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] -Join type: LeftSemi -Join condition: None - -(27) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None - -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(30) ReusedExchange [Reuses operator id: 174] -Output [1]: [d_date_sk#24] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#25, i_class_id#15 AS class_id#26, i_category_id#16 AS category_id#27] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#24] - -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#25, class_id#26, category_id#27] - -(34) CometColumnarExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(35) CometHashAggregate -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] - -(36) CometColumnarToRow [codegen id : 10] -Input [3]: [brand_id#25, class_id#26, category_id#27] - -(37) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] - -(39) Filter [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Condition : isnotnull(ws_item_sk#28) - -(40) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#28] -Right keys [1]: [i_item_sk#30] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] -Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(43) ReusedExchange [Reuses operator id: 174] -Output [1]: [d_date_sk#34] - -(44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#29] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 9] -Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] - -(46) BroadcastExchange -Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] - -(47) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] -Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] -Join type: LeftSemi -Join condition: None - -(48) BroadcastExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] - -(49) BroadcastHashJoin [codegen id : 11] -Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#25, class_id#26, category_id#27] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#35] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] - -(51) BroadcastExchange -Input [1]: [ss_item_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(52) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(53) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(54) CometFilter -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Condition : isnotnull(i_item_sk#36) - -(55) CometColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(56) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#35] - -(57) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#36] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(58) BroadcastExchange -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(59) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#36] -Join type: Inner -Join condition: None - -(60) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(61) ReusedExchange [Reuses operator id: 169] -Output [1]: [d_date_sk#40] - -(62) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] - -(64) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] -Results [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(65) CometColumnarExchange -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Arguments: hashpartitioning(i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(66) CometColumnarToRow [codegen id : 26] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(67) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] -Results [6]: [store AS channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51] - -(68) Filter [codegen id : 26] -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51] -Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(69) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_sold_date_sk#57 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(70) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] - -(71) Filter [codegen id : 51] -Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Condition : isnotnull(cs_item_sk#54) - -(72) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#58] - -(73) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#54] -Right keys [1]: [ss_item_sk#58] -Join type: LeftSemi -Join condition: None - -(74) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] - -(75) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#54] -Right keys [1]: [i_item_sk#59] -Join type: Inner -Join condition: None - -(76) Project [codegen id : 51] -Output [6]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62] -Input [8]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] - -(77) ReusedExchange [Reuses operator id: 169] -Output [1]: [d_date_sk#63] - -(78) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#57] -Right keys [1]: [d_date_sk#63] -Join type: Inner -Join condition: None - -(79) Project [codegen id : 51] -Output [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] -Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62, d_date_sk#63] - -(80) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] -Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] -Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] - -(81) CometColumnarExchange -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] -Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(82) CometColumnarToRow [codegen id : 52] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] - -(83) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] -Aggregate Attributes [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#70, count(1)#71] -Results [6]: [catalog AS channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#70 AS sales#73, count(1)#71 AS number_sales#74] - -(84) Filter [codegen id : 52] -Input [6]: [channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] -Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(85) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#75, ws_quantity#76, ws_list_price#77, ws_sold_date_sk#78] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#78), dynamicpruningexpression(ws_sold_date_sk#78 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(86) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#75, ws_quantity#76, ws_list_price#77, ws_sold_date_sk#78] - -(87) Filter [codegen id : 77] -Input [4]: [ws_item_sk#75, ws_quantity#76, ws_list_price#77, ws_sold_date_sk#78] -Condition : isnotnull(ws_item_sk#75) - -(88) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#79] - -(89) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#75] -Right keys [1]: [ss_item_sk#79] -Join type: LeftSemi -Join condition: None - -(90) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] - -(91) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#75] -Right keys [1]: [i_item_sk#80] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 77] -Output [6]: [ws_quantity#76, ws_list_price#77, ws_sold_date_sk#78, i_brand_id#81, i_class_id#82, i_category_id#83] -Input [8]: [ws_item_sk#75, ws_quantity#76, ws_list_price#77, ws_sold_date_sk#78, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] - -(93) ReusedExchange [Reuses operator id: 169] -Output [1]: [d_date_sk#84] - -(94) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#78] -Right keys [1]: [d_date_sk#84] -Join type: Inner -Join condition: None - -(95) Project [codegen id : 77] -Output [5]: [ws_quantity#76, ws_list_price#77, i_brand_id#81, i_class_id#82, i_category_id#83] -Input [7]: [ws_quantity#76, ws_list_price#77, ws_sold_date_sk#78, i_brand_id#81, i_class_id#82, i_category_id#83, d_date_sk#84] - -(96) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#76, ws_list_price#77, i_brand_id#81, i_class_id#82, i_category_id#83] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [partial_sum((cast(ws_quantity#76 as decimal(10,0)) * ws_list_price#77)), partial_count(1)] -Aggregate Attributes [3]: [sum#85, isEmpty#86, count#87] -Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#88, isEmpty#89, count#90] - -(97) CometColumnarExchange -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#88, isEmpty#89, count#90] -Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(98) CometColumnarToRow [codegen id : 78] -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#88, isEmpty#89, count#90] - -(99) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#88, isEmpty#89, count#90] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [sum((cast(ws_quantity#76 as decimal(10,0)) * ws_list_price#77)), count(1)] -Aggregate Attributes [2]: [sum((cast(ws_quantity#76 as decimal(10,0)) * ws_list_price#77))#91, count(1)#92] -Results [6]: [web AS channel#93, i_brand_id#81, i_class_id#82, i_category_id#83, sum((cast(ws_quantity#76 as decimal(10,0)) * ws_list_price#77))#91 AS sales#94, count(1)#92 AS number_sales#95] - -(100) Filter [codegen id : 78] -Input [6]: [channel#93, i_brand_id#81, i_class_id#82, i_category_id#83, sales#94, number_sales#95] -Condition : (isnotnull(sales#94) AND (cast(sales#94 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(101) Union - -(102) HashAggregate [codegen id : 79] -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51] -Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] -Aggregate Attributes [3]: [sum#96, isEmpty#97, sum#98] -Results [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#99, isEmpty#100, sum#101] - -(103) CometColumnarExchange -Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#99, isEmpty#100, sum#101] -Arguments: hashpartitioning(channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(104) CometColumnarToRow [codegen id : 80] -Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#99, isEmpty#100, sum#101] - -(105) HashAggregate [codegen id : 80] -Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#99, isEmpty#100, sum#101] -Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [sum(sales#50), sum(number_sales#51)] -Aggregate Attributes [2]: [sum(sales#50)#102, sum(number_sales#51)#103] -Results [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum(sales#50)#102 AS sum_sales#104, sum(number_sales#51)#103 AS number_sales#105] - -(106) ReusedExchange [Reuses operator id: 103] -Output [7]: [channel#49, i_brand_id#106, i_class_id#107, i_category_id#108, sum#99, isEmpty#100, sum#101] - -(107) CometColumnarToRow [codegen id : 160] -Input [7]: [channel#49, i_brand_id#106, i_class_id#107, i_category_id#108, sum#99, isEmpty#100, sum#101] - -(108) HashAggregate [codegen id : 160] -Input [7]: [channel#49, i_brand_id#106, i_class_id#107, i_category_id#108, sum#99, isEmpty#100, sum#101] -Keys [4]: [channel#49, i_brand_id#106, i_class_id#107, i_category_id#108] -Functions [2]: [sum(sales#50), sum(number_sales#51)] -Aggregate Attributes [2]: [sum(sales#50)#102, sum(number_sales#51)#103] -Results [5]: [channel#49, i_brand_id#106, i_class_id#107, sum(sales#50)#102 AS sum_sales#109, sum(number_sales#51)#103 AS number_sales#110] - -(109) HashAggregate [codegen id : 160] -Input [5]: [channel#49, i_brand_id#106, i_class_id#107, sum_sales#109, number_sales#110] -Keys [3]: [channel#49, i_brand_id#106, i_class_id#107] -Functions [2]: [partial_sum(sum_sales#109), partial_sum(number_sales#110)] -Aggregate Attributes [3]: [sum#111, isEmpty#112, sum#113] -Results [6]: [channel#49, i_brand_id#106, i_class_id#107, sum#114, isEmpty#115, sum#116] - -(110) CometColumnarExchange -Input [6]: [channel#49, i_brand_id#106, i_class_id#107, sum#114, isEmpty#115, sum#116] -Arguments: hashpartitioning(channel#49, i_brand_id#106, i_class_id#107, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(111) CometColumnarToRow [codegen id : 161] -Input [6]: [channel#49, i_brand_id#106, i_class_id#107, sum#114, isEmpty#115, sum#116] - -(112) HashAggregate [codegen id : 161] -Input [6]: [channel#49, i_brand_id#106, i_class_id#107, sum#114, isEmpty#115, sum#116] -Keys [3]: [channel#49, i_brand_id#106, i_class_id#107] -Functions [2]: [sum(sum_sales#109), sum(number_sales#110)] -Aggregate Attributes [2]: [sum(sum_sales#109)#117, sum(number_sales#110)#118] -Results [6]: [channel#49, i_brand_id#106, i_class_id#107, null AS i_category_id#119, sum(sum_sales#109)#117 AS sum(sum_sales)#120, sum(number_sales#110)#118 AS sum(number_sales)#121] - -(113) ReusedExchange [Reuses operator id: 103] -Output [7]: [channel#49, i_brand_id#122, i_class_id#123, i_category_id#124, sum#99, isEmpty#100, sum#101] - -(114) CometColumnarToRow [codegen id : 241] -Input [7]: [channel#49, i_brand_id#122, i_class_id#123, i_category_id#124, sum#99, isEmpty#100, sum#101] - -(115) HashAggregate [codegen id : 241] -Input [7]: [channel#49, i_brand_id#122, i_class_id#123, i_category_id#124, sum#99, isEmpty#100, sum#101] -Keys [4]: [channel#49, i_brand_id#122, i_class_id#123, i_category_id#124] -Functions [2]: [sum(sales#50), sum(number_sales#51)] -Aggregate Attributes [2]: [sum(sales#50)#102, sum(number_sales#51)#103] -Results [4]: [channel#49, i_brand_id#122, sum(sales#50)#102 AS sum_sales#125, sum(number_sales#51)#103 AS number_sales#126] - -(116) HashAggregate [codegen id : 241] -Input [4]: [channel#49, i_brand_id#122, sum_sales#125, number_sales#126] -Keys [2]: [channel#49, i_brand_id#122] -Functions [2]: [partial_sum(sum_sales#125), partial_sum(number_sales#126)] -Aggregate Attributes [3]: [sum#127, isEmpty#128, sum#129] -Results [5]: [channel#49, i_brand_id#122, sum#130, isEmpty#131, sum#132] - -(117) CometColumnarExchange -Input [5]: [channel#49, i_brand_id#122, sum#130, isEmpty#131, sum#132] -Arguments: hashpartitioning(channel#49, i_brand_id#122, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(118) CometColumnarToRow [codegen id : 242] -Input [5]: [channel#49, i_brand_id#122, sum#130, isEmpty#131, sum#132] - -(119) HashAggregate [codegen id : 242] -Input [5]: [channel#49, i_brand_id#122, sum#130, isEmpty#131, sum#132] -Keys [2]: [channel#49, i_brand_id#122] -Functions [2]: [sum(sum_sales#125), sum(number_sales#126)] -Aggregate Attributes [2]: [sum(sum_sales#125)#133, sum(number_sales#126)#134] -Results [6]: [channel#49, i_brand_id#122, null AS i_class_id#135, null AS i_category_id#136, sum(sum_sales#125)#133 AS sum(sum_sales)#137, sum(number_sales#126)#134 AS sum(number_sales)#138] - -(120) ReusedExchange [Reuses operator id: 103] -Output [7]: [channel#49, i_brand_id#139, i_class_id#140, i_category_id#141, sum#99, isEmpty#100, sum#101] - -(121) CometColumnarToRow [codegen id : 322] -Input [7]: [channel#49, i_brand_id#139, i_class_id#140, i_category_id#141, sum#99, isEmpty#100, sum#101] - -(122) HashAggregate [codegen id : 322] -Input [7]: [channel#49, i_brand_id#139, i_class_id#140, i_category_id#141, sum#99, isEmpty#100, sum#101] -Keys [4]: [channel#49, i_brand_id#139, i_class_id#140, i_category_id#141] -Functions [2]: [sum(sales#50), sum(number_sales#51)] -Aggregate Attributes [2]: [sum(sales#50)#102, sum(number_sales#51)#103] -Results [3]: [channel#49, sum(sales#50)#102 AS sum_sales#142, sum(number_sales#51)#103 AS number_sales#143] - -(123) HashAggregate [codegen id : 322] -Input [3]: [channel#49, sum_sales#142, number_sales#143] -Keys [1]: [channel#49] -Functions [2]: [partial_sum(sum_sales#142), partial_sum(number_sales#143)] -Aggregate Attributes [3]: [sum#144, isEmpty#145, sum#146] -Results [4]: [channel#49, sum#147, isEmpty#148, sum#149] - -(124) CometColumnarExchange -Input [4]: [channel#49, sum#147, isEmpty#148, sum#149] -Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] - -(125) CometColumnarToRow [codegen id : 323] -Input [4]: [channel#49, sum#147, isEmpty#148, sum#149] - -(126) HashAggregate [codegen id : 323] -Input [4]: [channel#49, sum#147, isEmpty#148, sum#149] -Keys [1]: [channel#49] -Functions [2]: [sum(sum_sales#142), sum(number_sales#143)] -Aggregate Attributes [2]: [sum(sum_sales#142)#150, sum(number_sales#143)#151] -Results [6]: [channel#49, null AS i_brand_id#152, null AS i_class_id#153, null AS i_category_id#154, sum(sum_sales#142)#150 AS sum(sum_sales)#155, sum(number_sales#143)#151 AS sum(number_sales)#156] - -(127) ReusedExchange [Reuses operator id: 103] -Output [7]: [channel#49, i_brand_id#157, i_class_id#158, i_category_id#159, sum#99, isEmpty#100, sum#101] - -(128) CometColumnarToRow [codegen id : 403] -Input [7]: [channel#49, i_brand_id#157, i_class_id#158, i_category_id#159, sum#99, isEmpty#100, sum#101] - -(129) HashAggregate [codegen id : 403] -Input [7]: [channel#49, i_brand_id#157, i_class_id#158, i_category_id#159, sum#99, isEmpty#100, sum#101] -Keys [4]: [channel#49, i_brand_id#157, i_class_id#158, i_category_id#159] -Functions [2]: [sum(sales#50), sum(number_sales#51)] -Aggregate Attributes [2]: [sum(sales#50)#102, sum(number_sales#51)#103] -Results [2]: [sum(sales#50)#102 AS sum_sales#160, sum(number_sales#51)#103 AS number_sales#161] - -(130) HashAggregate [codegen id : 403] -Input [2]: [sum_sales#160, number_sales#161] -Keys: [] -Functions [2]: [partial_sum(sum_sales#160), partial_sum(number_sales#161)] -Aggregate Attributes [3]: [sum#162, isEmpty#163, sum#164] -Results [3]: [sum#165, isEmpty#166, sum#167] - -(131) CometColumnarExchange -Input [3]: [sum#165, isEmpty#166, sum#167] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(132) CometColumnarToRow [codegen id : 404] -Input [3]: [sum#165, isEmpty#166, sum#167] - -(133) HashAggregate [codegen id : 404] -Input [3]: [sum#165, isEmpty#166, sum#167] -Keys: [] -Functions [2]: [sum(sum_sales#160), sum(number_sales#161)] -Aggregate Attributes [2]: [sum(sum_sales#160)#168, sum(number_sales#161)#169] -Results [6]: [null AS channel#170, null AS i_brand_id#171, null AS i_class_id#172, null AS i_category_id#173, sum(sum_sales#160)#168 AS sum(sum_sales)#174, sum(number_sales#161)#169 AS sum(number_sales)#175] - -(134) Union - -(135) HashAggregate [codegen id : 405] -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] -Keys [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] -Functions: [] -Aggregate Attributes: [] -Results [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] - -(136) CometColumnarExchange -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] -Arguments: hashpartitioning(channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] - -(137) CometHashAggregate -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] -Keys [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] -Functions: [] - -(138) CometTakeOrderedAndProject -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#49 ASC NULLS FIRST,i_brand_id#37 ASC NULLS FIRST,i_class_id#38 ASC NULLS FIRST,i_category_id#39 ASC NULLS FIRST], output=[channel#49,i_brand_id#37,i_class_id#38,i_category_id#39,sum_sales#104,number_sales#105]), [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105], 100, 0, [channel#49 ASC NULLS FIRST, i_brand_id#37 ASC NULLS FIRST, i_class_id#38 ASC NULLS FIRST, i_category_id#39 ASC NULLS FIRST], [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] - -(139) CometColumnarToRow [codegen id : 406] -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* HashAggregate (159) -+- * CometColumnarToRow (158) - +- CometColumnarExchange (157) - +- * HashAggregate (156) - +- Union (155) - :- * Project (144) - : +- * BroadcastHashJoin Inner BuildRight (143) - : :- * ColumnarToRow (141) - : : +- Scan parquet spark_catalog.default.store_sales (140) - : +- ReusedExchange (142) - :- * Project (149) - : +- * BroadcastHashJoin Inner BuildRight (148) - : :- * ColumnarToRow (146) - : : +- Scan parquet spark_catalog.default.catalog_sales (145) - : +- ReusedExchange (147) - +- * Project (154) - +- * BroadcastHashJoin Inner BuildRight (153) - :- * ColumnarToRow (151) - : +- Scan parquet spark_catalog.default.web_sales (150) - +- ReusedExchange (152) - - -(140) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#176, ss_list_price#177, ss_sold_date_sk#178] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#178), dynamicpruningexpression(ss_sold_date_sk#178 IN dynamicpruning#12)] -ReadSchema: struct - -(141) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#176, ss_list_price#177, ss_sold_date_sk#178] - -(142) ReusedExchange [Reuses operator id: 174] -Output [1]: [d_date_sk#179] - -(143) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#178] -Right keys [1]: [d_date_sk#179] -Join type: Inner -Join condition: None - -(144) Project [codegen id : 2] -Output [2]: [ss_quantity#176 AS quantity#180, ss_list_price#177 AS list_price#181] -Input [4]: [ss_quantity#176, ss_list_price#177, ss_sold_date_sk#178, d_date_sk#179] - -(145) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#182, cs_list_price#183, cs_sold_date_sk#184] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#184), dynamicpruningexpression(cs_sold_date_sk#184 IN dynamicpruning#185)] -ReadSchema: struct - -(146) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#182, cs_list_price#183, cs_sold_date_sk#184] - -(147) ReusedExchange [Reuses operator id: 164] -Output [1]: [d_date_sk#186] - -(148) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#184] -Right keys [1]: [d_date_sk#186] -Join type: Inner -Join condition: None - -(149) Project [codegen id : 4] -Output [2]: [cs_quantity#182 AS quantity#187, cs_list_price#183 AS list_price#188] -Input [4]: [cs_quantity#182, cs_list_price#183, cs_sold_date_sk#184, d_date_sk#186] - -(150) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#189, ws_list_price#190, ws_sold_date_sk#191] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#191), dynamicpruningexpression(ws_sold_date_sk#191 IN dynamicpruning#185)] -ReadSchema: struct - -(151) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#189, ws_list_price#190, ws_sold_date_sk#191] - -(152) ReusedExchange [Reuses operator id: 164] -Output [1]: [d_date_sk#192] - -(153) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#191] -Right keys [1]: [d_date_sk#192] -Join type: Inner -Join condition: None - -(154) Project [codegen id : 6] -Output [2]: [ws_quantity#189 AS quantity#193, ws_list_price#190 AS list_price#194] -Input [4]: [ws_quantity#189, ws_list_price#190, ws_sold_date_sk#191, d_date_sk#192] - -(155) Union - -(156) HashAggregate [codegen id : 7] -Input [2]: [quantity#180, list_price#181] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#180 as decimal(10,0)) * list_price#181))] -Aggregate Attributes [2]: [sum#195, count#196] -Results [2]: [sum#197, count#198] - -(157) CometColumnarExchange -Input [2]: [sum#197, count#198] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] - -(158) CometColumnarToRow [codegen id : 8] -Input [2]: [sum#197, count#198] - -(159) HashAggregate [codegen id : 8] -Input [2]: [sum#197, count#198] -Keys: [] -Functions [1]: [avg((cast(quantity#180 as decimal(10,0)) * list_price#181))] -Aggregate Attributes [1]: [avg((cast(quantity#180 as decimal(10,0)) * list_price#181))#199] -Results [1]: [avg((cast(quantity#180 as decimal(10,0)) * list_price#181))#199 AS average_sales#200] - -Subquery:2 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#178 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 145 Hosting Expression = cs_sold_date_sk#184 IN dynamicpruning#185 -BroadcastExchange (164) -+- * CometColumnarToRow (163) - +- CometProject (162) - +- CometFilter (161) - +- CometNativeScan parquet spark_catalog.default.date_dim (160) - - -(160) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#186, d_year#201] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(161) CometFilter -Input [2]: [d_date_sk#186, d_year#201] -Condition : (((isnotnull(d_year#201) AND (d_year#201 >= 1998)) AND (d_year#201 <= 2000)) AND isnotnull(d_date_sk#186)) - -(162) CometProject -Input [2]: [d_date_sk#186, d_year#201] -Arguments: [d_date_sk#186], [d_date_sk#186] - -(163) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#186] - -(164) BroadcastExchange -Input [1]: [d_date_sk#186] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] - -Subquery:4 Hosting operator id = 150 Hosting Expression = ws_sold_date_sk#191 IN dynamicpruning#185 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (169) -+- * CometColumnarToRow (168) - +- CometProject (167) - +- CometFilter (166) - +- CometNativeScan parquet spark_catalog.default.date_dim (165) - - -(165) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#40, d_year#202, d_moy#203] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(166) CometFilter -Input [3]: [d_date_sk#40, d_year#202, d_moy#203] -Condition : ((((isnotnull(d_year#202) AND isnotnull(d_moy#203)) AND (d_year#202 = 2000)) AND (d_moy#203 = 11)) AND isnotnull(d_date_sk#40)) - -(167) CometProject -Input [3]: [d_date_sk#40, d_year#202, d_moy#203] -Arguments: [d_date_sk#40], [d_date_sk#40] - -(168) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#40] - -(169) BroadcastExchange -Input [1]: [d_date_sk#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] - -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (174) -+- * CometColumnarToRow (173) - +- CometProject (172) - +- CometFilter (171) - +- CometNativeScan parquet spark_catalog.default.date_dim (170) - - -(170) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#204] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(171) CometFilter -Input [2]: [d_date_sk#24, d_year#204] -Condition : (((isnotnull(d_year#204) AND (d_year#204 >= 1999)) AND (d_year#204 <= 2001)) AND isnotnull(d_date_sk#24)) - -(172) CometProject -Input [2]: [d_date_sk#24, d_year#204] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(173) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#24] - -(174) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21] - -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:8 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:10 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 - -Subquery:11 Hosting operator id = 100 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:12 Hosting operator id = 85 Hosting Expression = ws_sold_date_sk#78 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/simplified.txt deleted file mode 100644 index 4458723393..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/simplified.txt +++ /dev/null @@ -1,276 +0,0 @@ -WholeStageCodegen (406) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - WholeStageCodegen (405) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - InputAdapter - Union - WholeStageCodegen (80) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (79) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (26) - Filter [sales] - Subquery #3 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #14 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #8 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #15 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #15 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #15 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (52) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (51) - HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (78) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (77) - HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (161) - HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id,i_class_id] #18 - WholeStageCodegen (160) - HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (242) - HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id] #19 - WholeStageCodegen (241) - HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (323) - HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #20 - WholeStageCodegen (322) - HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (404) - HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #21 - WholeStageCodegen (403) - HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/explain.txt deleted file mode 100644 index a727c5e743..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,914 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (122) -+- CometTakeOrderedAndProject (121) - +- CometHashAggregate (120) - +- CometExchange (119) - +- CometHashAggregate (118) - +- CometUnion (117) - :- CometHashAggregate (96) - : +- CometExchange (95) - : +- CometHashAggregate (94) - : +- CometUnion (93) - : :- CometFilter (64) - : : +- CometHashAggregate (63) - : : +- CometExchange (62) - : : +- CometHashAggregate (61) - : : +- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometBroadcastHashJoin (47) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (46) - : : : : +- CometProject (45) - : : : : +- CometBroadcastHashJoin (44) - : : : : :- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : : +- CometBroadcastExchange (43) - : : : : +- CometBroadcastHashJoin (42) - : : : : :- CometHashAggregate (32) - : : : : : +- CometExchange (31) - : : : : : +- CometHashAggregate (30) - : : : : : +- CometProject (29) - : : : : : +- CometBroadcastHashJoin (28) - : : : : : :- CometProject (26) - : : : : : : +- CometBroadcastHashJoin (25) - : : : : : : :- CometFilter (6) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : : +- CometBroadcastExchange (24) - : : : : : : +- CometBroadcastHashJoin (23) - : : : : : : :- CometFilter (8) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : : +- CometBroadcastExchange (22) - : : : : : : +- CometProject (21) - : : : : : : +- CometBroadcastHashJoin (20) - : : : : : : :- CometProject (15) - : : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : : :- CometFilter (10) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : : +- CometBroadcastExchange (13) - : : : : : : : +- CometFilter (12) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : : +- CometBroadcastExchange (19) - : : : : : : +- CometProject (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : : +- ReusedExchange (27) - : : : : +- CometBroadcastExchange (41) - : : : : +- CometProject (40) - : : : : +- CometBroadcastHashJoin (39) - : : : : :- CometProject (37) - : : : : : +- CometBroadcastHashJoin (36) - : : : : : :- CometFilter (34) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : : +- ReusedExchange (35) - : : : : +- ReusedExchange (38) - : : : +- CometBroadcastExchange (52) - : : : +- CometBroadcastHashJoin (51) - : : : :- CometFilter (49) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : : +- ReusedExchange (50) - : : +- CometBroadcastExchange (58) - : : +- CometProject (57) - : : +- CometFilter (56) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - : :- CometFilter (78) - : : +- CometHashAggregate (77) - : : +- CometExchange (76) - : : +- CometHashAggregate (75) - : : +- CometProject (74) - : : +- CometBroadcastHashJoin (73) - : : :- CometProject (71) - : : : +- CometBroadcastHashJoin (70) - : : : :- CometBroadcastHashJoin (68) - : : : : :- CometFilter (66) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (65) - : : : : +- ReusedExchange (67) - : : : +- ReusedExchange (69) - : : +- ReusedExchange (72) - : +- CometFilter (92) - : +- CometHashAggregate (91) - : +- CometExchange (90) - : +- CometHashAggregate (89) - : +- CometProject (88) - : +- CometBroadcastHashJoin (87) - : :- CometProject (85) - : : +- CometBroadcastHashJoin (84) - : : :- CometBroadcastHashJoin (82) - : : : :- CometFilter (80) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (79) - : : : +- ReusedExchange (81) - : : +- ReusedExchange (83) - : +- ReusedExchange (86) - :- CometHashAggregate (101) - : +- CometExchange (100) - : +- CometHashAggregate (99) - : +- CometHashAggregate (98) - : +- ReusedExchange (97) - :- CometHashAggregate (106) - : +- CometExchange (105) - : +- CometHashAggregate (104) - : +- CometHashAggregate (103) - : +- ReusedExchange (102) - :- CometHashAggregate (111) - : +- CometExchange (110) - : +- CometHashAggregate (109) - : +- CometHashAggregate (108) - : +- ReusedExchange (107) - +- CometHashAggregate (116) - +- CometExchange (115) - +- CometHashAggregate (114) - +- CometHashAggregate (113) - +- ReusedExchange (112) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : isnotnull(i_item_sk#39) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_sold_date_sk#57 IN dynamicpruning#58)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Condition : isnotnull(cs_item_sk#54) - -(67) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#59] - -(68) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Right output [1]: [ss_item_sk#59] -Arguments: [cs_item_sk#54], [ss_item_sk#59], LeftSemi, BuildRight - -(69) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] - -(70) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Right output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -Arguments: [cs_item_sk#54], [i_item_sk#60], Inner, BuildRight - -(71) CometProject -Input [8]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -Arguments: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63], [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] - -(72) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#64] - -(73) CometBroadcastHashJoin -Left output [6]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] -Right output [1]: [d_date_sk#64] -Arguments: [cs_sold_date_sk#57], [d_date_sk#64], Inner, BuildRight - -(74) CometProject -Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] -Arguments: [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63], [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] - -(75) CometHashAggregate -Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] - -(76) CometExchange -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#65, isEmpty#66, count#67] -Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(77) CometHashAggregate -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#65, isEmpty#66, count#67] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] - -(78) CometFilter -Input [6]: [channel#68, i_brand_id#61, i_class_id#62, i_category_id#63, sales#69, number_sales#70] -Condition : (isnotnull(sales#69) AND (cast(sales#69 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(80) CometFilter -Input [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] -Condition : isnotnull(ws_item_sk#71) - -(81) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#76] - -(82) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] -Right output [1]: [ss_item_sk#76] -Arguments: [ws_item_sk#71], [ss_item_sk#76], LeftSemi, BuildRight - -(83) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] - -(84) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] -Right output [4]: [i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] -Arguments: [ws_item_sk#71], [i_item_sk#77], Inner, BuildRight - -(85) CometProject -Input [8]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] -Arguments: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80], [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80] - -(86) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#81] - -(87) CometBroadcastHashJoin -Left output [6]: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80] -Right output [1]: [d_date_sk#81] -Arguments: [ws_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight - -(88) CometProject -Input [7]: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80, d_date_sk#81] -Arguments: [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80], [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80] - -(89) CometHashAggregate -Input [5]: [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80] -Keys [3]: [i_brand_id#78, i_class_id#79, i_category_id#80] -Functions [2]: [partial_sum((cast(ws_quantity#72 as decimal(10,0)) * ws_list_price#73)), partial_count(1)] - -(90) CometExchange -Input [6]: [i_brand_id#78, i_class_id#79, i_category_id#80, sum#82, isEmpty#83, count#84] -Arguments: hashpartitioning(i_brand_id#78, i_class_id#79, i_category_id#80, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(91) CometHashAggregate -Input [6]: [i_brand_id#78, i_class_id#79, i_category_id#80, sum#82, isEmpty#83, count#84] -Keys [3]: [i_brand_id#78, i_class_id#79, i_category_id#80] -Functions [2]: [sum((cast(ws_quantity#72 as decimal(10,0)) * ws_list_price#73)), count(1)] - -(92) CometFilter -Input [6]: [channel#85, i_brand_id#78, i_class_id#79, i_category_id#80, sales#86, number_sales#87] -Condition : (isnotnull(sales#86) AND (cast(sales#86 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(93) CometUnion -Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Child 1 Input [6]: [channel#68, i_brand_id#61, i_class_id#62, i_category_id#63, sales#69, number_sales#70] -Child 2 Input [6]: [channel#85, i_brand_id#78, i_class_id#79, i_category_id#80, sales#86, number_sales#87] - -(94) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] - -(95) CometExchange -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#88, isEmpty#89, sum#90] -Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(96) CometHashAggregate -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#88, isEmpty#89, sum#90] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(97) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#93, sum#88, isEmpty#89, sum#90] - -(98) CometHashAggregate -Input [7]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#93, sum#88, isEmpty#89, sum#90] -Keys [4]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#93] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(99) CometHashAggregate -Input [5]: [channel#49, i_brand_id#91, i_class_id#92, sum_sales#94, number_sales#95] -Keys [3]: [channel#49, i_brand_id#91, i_class_id#92] -Functions [2]: [partial_sum(sum_sales#94), partial_sum(number_sales#95)] - -(100) CometExchange -Input [6]: [channel#49, i_brand_id#91, i_class_id#92, sum#96, isEmpty#97, sum#98] -Arguments: hashpartitioning(channel#49, i_brand_id#91, i_class_id#92, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(101) CometHashAggregate -Input [6]: [channel#49, i_brand_id#91, i_class_id#92, sum#96, isEmpty#97, sum#98] -Keys [3]: [channel#49, i_brand_id#91, i_class_id#92] -Functions [2]: [sum(sum_sales#94), sum(number_sales#95)] - -(102) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#99, i_class_id#100, i_category_id#101, sum#88, isEmpty#89, sum#90] - -(103) CometHashAggregate -Input [7]: [channel#49, i_brand_id#99, i_class_id#100, i_category_id#101, sum#88, isEmpty#89, sum#90] -Keys [4]: [channel#49, i_brand_id#99, i_class_id#100, i_category_id#101] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(104) CometHashAggregate -Input [4]: [channel#49, i_brand_id#99, sum_sales#102, number_sales#103] -Keys [2]: [channel#49, i_brand_id#99] -Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] - -(105) CometExchange -Input [5]: [channel#49, i_brand_id#99, sum#104, isEmpty#105, sum#106] -Arguments: hashpartitioning(channel#49, i_brand_id#99, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(106) CometHashAggregate -Input [5]: [channel#49, i_brand_id#99, sum#104, isEmpty#105, sum#106] -Keys [2]: [channel#49, i_brand_id#99] -Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] - -(107) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#107, i_class_id#108, i_category_id#109, sum#88, isEmpty#89, sum#90] - -(108) CometHashAggregate -Input [7]: [channel#49, i_brand_id#107, i_class_id#108, i_category_id#109, sum#88, isEmpty#89, sum#90] -Keys [4]: [channel#49, i_brand_id#107, i_class_id#108, i_category_id#109] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(109) CometHashAggregate -Input [3]: [channel#49, sum_sales#110, number_sales#111] -Keys [1]: [channel#49] -Functions [2]: [partial_sum(sum_sales#110), partial_sum(number_sales#111)] - -(110) CometExchange -Input [4]: [channel#49, sum#112, isEmpty#113, sum#114] -Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(111) CometHashAggregate -Input [4]: [channel#49, sum#112, isEmpty#113, sum#114] -Keys [1]: [channel#49] -Functions [2]: [sum(sum_sales#110), sum(number_sales#111)] - -(112) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#115, i_class_id#116, i_category_id#117, sum#88, isEmpty#89, sum#90] - -(113) CometHashAggregate -Input [7]: [channel#49, i_brand_id#115, i_class_id#116, i_category_id#117, sum#88, isEmpty#89, sum#90] -Keys [4]: [channel#49, i_brand_id#115, i_class_id#116, i_category_id#117] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(114) CometHashAggregate -Input [2]: [sum_sales#118, number_sales#119] -Keys: [] -Functions [2]: [partial_sum(sum_sales#118), partial_sum(number_sales#119)] - -(115) CometExchange -Input [3]: [sum#120, isEmpty#121, sum#122] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(116) CometHashAggregate -Input [3]: [sum#120, isEmpty#121, sum#122] -Keys: [] -Functions [2]: [sum(sum_sales#118), sum(number_sales#119)] - -(117) CometUnion -Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Child 1 Input [6]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#125, sum(sum_sales)#126, sum(number_sales)#127] -Child 2 Input [6]: [channel#49, i_brand_id#99, i_class_id#128, i_category_id#129, sum(sum_sales)#130, sum(number_sales)#131] -Child 3 Input [6]: [channel#49, i_brand_id#132, i_class_id#133, i_category_id#134, sum(sum_sales)#135, sum(number_sales)#136] -Child 4 Input [6]: [channel#137, i_brand_id#138, i_class_id#139, i_category_id#140, sum(sum_sales)#141, sum(number_sales)#142] - -(118) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Functions: [] - -(119) CometExchange -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(120) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Functions: [] - -(121) CometTakeOrderedAndProject -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#49 ASC NULLS FIRST,i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#49,i_brand_id#40,i_class_id#41,i_category_id#42,sum_sales#123,number_sales#124]), [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124], 100, 0, [channel#49 ASC NULLS FIRST, i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] - -(122) CometColumnarToRow [codegen id : 1] -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* CometColumnarToRow (142) -+- CometHashAggregate (141) - +- CometExchange (140) - +- CometHashAggregate (139) - +- CometUnion (138) - :- CometProject (126) - : +- CometBroadcastHashJoin (125) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (123) - : +- ReusedExchange (124) - :- CometProject (133) - : +- CometBroadcastHashJoin (132) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (127) - : +- CometBroadcastExchange (131) - : +- CometProject (130) - : +- CometFilter (129) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (128) - +- CometProject (137) - +- CometBroadcastHashJoin (136) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (134) - +- ReusedExchange (135) - - -(123) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#143, ss_list_price#144, ss_sold_date_sk#145] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#145), dynamicpruningexpression(ss_sold_date_sk#145 IN dynamicpruning#146)] -ReadSchema: struct - -(124) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#147] - -(125) CometBroadcastHashJoin -Left output [3]: [ss_quantity#143, ss_list_price#144, ss_sold_date_sk#145] -Right output [1]: [d_date_sk#147] -Arguments: [ss_sold_date_sk#145], [d_date_sk#147], Inner, BuildRight - -(126) CometProject -Input [4]: [ss_quantity#143, ss_list_price#144, ss_sold_date_sk#145, d_date_sk#147] -Arguments: [quantity#148, list_price#149], [ss_quantity#143 AS quantity#148, ss_list_price#144 AS list_price#149] - -(127) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#150, cs_list_price#151, cs_sold_date_sk#152] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#152), dynamicpruningexpression(cs_sold_date_sk#152 IN dynamicpruning#153)] -ReadSchema: struct - -(128) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#154, d_year#155] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(129) CometFilter -Input [2]: [d_date_sk#154, d_year#155] -Condition : (((isnotnull(d_year#155) AND (d_year#155 >= 1998)) AND (d_year#155 <= 2000)) AND isnotnull(d_date_sk#154)) - -(130) CometProject -Input [2]: [d_date_sk#154, d_year#155] -Arguments: [d_date_sk#154], [d_date_sk#154] - -(131) CometBroadcastExchange -Input [1]: [d_date_sk#154] -Arguments: [d_date_sk#154] - -(132) CometBroadcastHashJoin -Left output [3]: [cs_quantity#150, cs_list_price#151, cs_sold_date_sk#152] -Right output [1]: [d_date_sk#154] -Arguments: [cs_sold_date_sk#152], [d_date_sk#154], Inner, BuildRight - -(133) CometProject -Input [4]: [cs_quantity#150, cs_list_price#151, cs_sold_date_sk#152, d_date_sk#154] -Arguments: [quantity#156, list_price#157], [cs_quantity#150 AS quantity#156, cs_list_price#151 AS list_price#157] - -(134) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#158, ws_list_price#159, ws_sold_date_sk#160] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#160), dynamicpruningexpression(ws_sold_date_sk#160 IN dynamicpruning#161)] -ReadSchema: struct - -(135) ReusedExchange [Reuses operator id: 131] -Output [1]: [d_date_sk#162] - -(136) CometBroadcastHashJoin -Left output [3]: [ws_quantity#158, ws_list_price#159, ws_sold_date_sk#160] -Right output [1]: [d_date_sk#162] -Arguments: [ws_sold_date_sk#160], [d_date_sk#162], Inner, BuildRight - -(137) CometProject -Input [4]: [ws_quantity#158, ws_list_price#159, ws_sold_date_sk#160, d_date_sk#162] -Arguments: [quantity#163, list_price#164], [ws_quantity#158 AS quantity#163, ws_list_price#159 AS list_price#164] - -(138) CometUnion -Child 0 Input [2]: [quantity#148, list_price#149] -Child 1 Input [2]: [quantity#156, list_price#157] -Child 2 Input [2]: [quantity#163, list_price#164] - -(139) CometHashAggregate -Input [2]: [quantity#148, list_price#149] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#148 as decimal(10,0)) * list_price#149))] - -(140) CometExchange -Input [2]: [sum#165, count#166] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(141) CometHashAggregate -Input [2]: [sum#165, count#166] -Keys: [] -Functions [1]: [avg((cast(quantity#148 as decimal(10,0)) * list_price#149))] - -(142) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#167] - -Subquery:2 Hosting operator id = 123 Hosting Expression = ss_sold_date_sk#145 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 127 Hosting Expression = cs_sold_date_sk#152 IN dynamicpruning#153 -BroadcastExchange (147) -+- * CometColumnarToRow (146) - +- CometProject (145) - +- CometFilter (144) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (143) - - -(143) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#154, d_year#155] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(144) CometFilter -Input [2]: [d_date_sk#154, d_year#155] -Condition : (((isnotnull(d_year#155) AND (d_year#155 >= 1998)) AND (d_year#155 <= 2000)) AND isnotnull(d_date_sk#154)) - -(145) CometProject -Input [2]: [d_date_sk#154, d_year#155] -Arguments: [d_date_sk#154], [d_date_sk#154] - -(146) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#154] - -(147) BroadcastExchange -Input [1]: [d_date_sk#154] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#160 IN dynamicpruning#153 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (152) -+- * CometColumnarToRow (151) - +- CometProject (150) - +- CometFilter (149) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (148) - - -(148) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(149) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(150) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(151) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(152) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (157) -+- * CometColumnarToRow (156) - +- CometProject (155) - +- CometFilter (154) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (153) - - -(153) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#168] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(154) CometFilter -Input [2]: [d_date_sk#26, d_year#168] -Condition : (((isnotnull(d_year#168) AND (d_year#168 >= 1999)) AND (d_year#168 <= 2001)) AND isnotnull(d_date_sk#26)) - -(155) CometProject -Input [2]: [d_date_sk#26, d_year#168] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(156) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(157) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 78 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:10 Hosting operator id = 65 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 - -Subquery:11 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:12 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 758e917335..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,179 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometUnion [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - CometExchange [channel,i_brand_id,i_class_id,i_category_id] #2 - CometHashAggregate [sales,number_sales] [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] - CometUnion [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #16 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #18 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 - ReusedExchange [d_date_sk] #18 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #3 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk] #5 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #6 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #7 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #12 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #12 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #13 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #12 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #5 - CometBroadcastExchange [d_date_sk] #15 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #19 - CometHashAggregate [cs_quantity,cs_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk] - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #5 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - ReusedExchange [d_date_sk] #15 - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #20 - CometHashAggregate [ws_quantity,ws_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk] - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #5 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - ReusedExchange [d_date_sk] #15 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel,i_brand_id,i_class_id] #21 - CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,i_class_id,sum,isEmpty,sum] - CometHashAggregate [i_category_id,sum,isEmpty,sum] [channel,i_brand_id,i_class_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel,i_brand_id] #22 - CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,sum,isEmpty,sum] - CometHashAggregate [i_class_id,i_category_id,sum,isEmpty,sum] [channel,i_brand_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel] #23 - CometHashAggregate [sum_sales,number_sales] [channel,sum,isEmpty,sum] - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [channel,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange #24 - CometHashAggregate [sum_sales,number_sales] [sum,isEmpty,sum] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt deleted file mode 100644 index a727c5e743..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/explain.txt +++ /dev/null @@ -1,914 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (122) -+- CometTakeOrderedAndProject (121) - +- CometHashAggregate (120) - +- CometExchange (119) - +- CometHashAggregate (118) - +- CometUnion (117) - :- CometHashAggregate (96) - : +- CometExchange (95) - : +- CometHashAggregate (94) - : +- CometUnion (93) - : :- CometFilter (64) - : : +- CometHashAggregate (63) - : : +- CometExchange (62) - : : +- CometHashAggregate (61) - : : +- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometBroadcastHashJoin (47) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (46) - : : : : +- CometProject (45) - : : : : +- CometBroadcastHashJoin (44) - : : : : :- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : : +- CometBroadcastExchange (43) - : : : : +- CometBroadcastHashJoin (42) - : : : : :- CometHashAggregate (32) - : : : : : +- CometExchange (31) - : : : : : +- CometHashAggregate (30) - : : : : : +- CometProject (29) - : : : : : +- CometBroadcastHashJoin (28) - : : : : : :- CometProject (26) - : : : : : : +- CometBroadcastHashJoin (25) - : : : : : : :- CometFilter (6) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : : +- CometBroadcastExchange (24) - : : : : : : +- CometBroadcastHashJoin (23) - : : : : : : :- CometFilter (8) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : : +- CometBroadcastExchange (22) - : : : : : : +- CometProject (21) - : : : : : : +- CometBroadcastHashJoin (20) - : : : : : : :- CometProject (15) - : : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : : :- CometFilter (10) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : : +- CometBroadcastExchange (13) - : : : : : : : +- CometFilter (12) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : : +- CometBroadcastExchange (19) - : : : : : : +- CometProject (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : : +- ReusedExchange (27) - : : : : +- CometBroadcastExchange (41) - : : : : +- CometProject (40) - : : : : +- CometBroadcastHashJoin (39) - : : : : :- CometProject (37) - : : : : : +- CometBroadcastHashJoin (36) - : : : : : :- CometFilter (34) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : : +- ReusedExchange (35) - : : : : +- ReusedExchange (38) - : : : +- CometBroadcastExchange (52) - : : : +- CometBroadcastHashJoin (51) - : : : :- CometFilter (49) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : : +- ReusedExchange (50) - : : +- CometBroadcastExchange (58) - : : +- CometProject (57) - : : +- CometFilter (56) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - : :- CometFilter (78) - : : +- CometHashAggregate (77) - : : +- CometExchange (76) - : : +- CometHashAggregate (75) - : : +- CometProject (74) - : : +- CometBroadcastHashJoin (73) - : : :- CometProject (71) - : : : +- CometBroadcastHashJoin (70) - : : : :- CometBroadcastHashJoin (68) - : : : : :- CometFilter (66) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (65) - : : : : +- ReusedExchange (67) - : : : +- ReusedExchange (69) - : : +- ReusedExchange (72) - : +- CometFilter (92) - : +- CometHashAggregate (91) - : +- CometExchange (90) - : +- CometHashAggregate (89) - : +- CometProject (88) - : +- CometBroadcastHashJoin (87) - : :- CometProject (85) - : : +- CometBroadcastHashJoin (84) - : : :- CometBroadcastHashJoin (82) - : : : :- CometFilter (80) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (79) - : : : +- ReusedExchange (81) - : : +- ReusedExchange (83) - : +- ReusedExchange (86) - :- CometHashAggregate (101) - : +- CometExchange (100) - : +- CometHashAggregate (99) - : +- CometHashAggregate (98) - : +- ReusedExchange (97) - :- CometHashAggregate (106) - : +- CometExchange (105) - : +- CometHashAggregate (104) - : +- CometHashAggregate (103) - : +- ReusedExchange (102) - :- CometHashAggregate (111) - : +- CometExchange (110) - : +- CometHashAggregate (109) - : +- CometHashAggregate (108) - : +- ReusedExchange (107) - +- CometHashAggregate (116) - +- CometExchange (115) - +- CometHashAggregate (114) - +- CometHashAggregate (113) - +- ReusedExchange (112) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : isnotnull(i_item_sk#39) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_sold_date_sk#57 IN dynamicpruning#58)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Condition : isnotnull(cs_item_sk#54) - -(67) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#59] - -(68) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Right output [1]: [ss_item_sk#59] -Arguments: [cs_item_sk#54], [ss_item_sk#59], LeftSemi, BuildRight - -(69) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] - -(70) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Right output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -Arguments: [cs_item_sk#54], [i_item_sk#60], Inner, BuildRight - -(71) CometProject -Input [8]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -Arguments: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63], [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] - -(72) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#64] - -(73) CometBroadcastHashJoin -Left output [6]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] -Right output [1]: [d_date_sk#64] -Arguments: [cs_sold_date_sk#57], [d_date_sk#64], Inner, BuildRight - -(74) CometProject -Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] -Arguments: [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63], [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] - -(75) CometHashAggregate -Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] - -(76) CometExchange -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#65, isEmpty#66, count#67] -Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(77) CometHashAggregate -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#65, isEmpty#66, count#67] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] - -(78) CometFilter -Input [6]: [channel#68, i_brand_id#61, i_class_id#62, i_category_id#63, sales#69, number_sales#70] -Condition : (isnotnull(sales#69) AND (cast(sales#69 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(80) CometFilter -Input [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] -Condition : isnotnull(ws_item_sk#71) - -(81) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#76] - -(82) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] -Right output [1]: [ss_item_sk#76] -Arguments: [ws_item_sk#71], [ss_item_sk#76], LeftSemi, BuildRight - -(83) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] - -(84) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] -Right output [4]: [i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] -Arguments: [ws_item_sk#71], [i_item_sk#77], Inner, BuildRight - -(85) CometProject -Input [8]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] -Arguments: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80], [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80] - -(86) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#81] - -(87) CometBroadcastHashJoin -Left output [6]: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80] -Right output [1]: [d_date_sk#81] -Arguments: [ws_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight - -(88) CometProject -Input [7]: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80, d_date_sk#81] -Arguments: [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80], [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80] - -(89) CometHashAggregate -Input [5]: [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80] -Keys [3]: [i_brand_id#78, i_class_id#79, i_category_id#80] -Functions [2]: [partial_sum((cast(ws_quantity#72 as decimal(10,0)) * ws_list_price#73)), partial_count(1)] - -(90) CometExchange -Input [6]: [i_brand_id#78, i_class_id#79, i_category_id#80, sum#82, isEmpty#83, count#84] -Arguments: hashpartitioning(i_brand_id#78, i_class_id#79, i_category_id#80, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(91) CometHashAggregate -Input [6]: [i_brand_id#78, i_class_id#79, i_category_id#80, sum#82, isEmpty#83, count#84] -Keys [3]: [i_brand_id#78, i_class_id#79, i_category_id#80] -Functions [2]: [sum((cast(ws_quantity#72 as decimal(10,0)) * ws_list_price#73)), count(1)] - -(92) CometFilter -Input [6]: [channel#85, i_brand_id#78, i_class_id#79, i_category_id#80, sales#86, number_sales#87] -Condition : (isnotnull(sales#86) AND (cast(sales#86 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(93) CometUnion -Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Child 1 Input [6]: [channel#68, i_brand_id#61, i_class_id#62, i_category_id#63, sales#69, number_sales#70] -Child 2 Input [6]: [channel#85, i_brand_id#78, i_class_id#79, i_category_id#80, sales#86, number_sales#87] - -(94) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] - -(95) CometExchange -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#88, isEmpty#89, sum#90] -Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(96) CometHashAggregate -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#88, isEmpty#89, sum#90] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(97) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#93, sum#88, isEmpty#89, sum#90] - -(98) CometHashAggregate -Input [7]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#93, sum#88, isEmpty#89, sum#90] -Keys [4]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#93] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(99) CometHashAggregate -Input [5]: [channel#49, i_brand_id#91, i_class_id#92, sum_sales#94, number_sales#95] -Keys [3]: [channel#49, i_brand_id#91, i_class_id#92] -Functions [2]: [partial_sum(sum_sales#94), partial_sum(number_sales#95)] - -(100) CometExchange -Input [6]: [channel#49, i_brand_id#91, i_class_id#92, sum#96, isEmpty#97, sum#98] -Arguments: hashpartitioning(channel#49, i_brand_id#91, i_class_id#92, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(101) CometHashAggregate -Input [6]: [channel#49, i_brand_id#91, i_class_id#92, sum#96, isEmpty#97, sum#98] -Keys [3]: [channel#49, i_brand_id#91, i_class_id#92] -Functions [2]: [sum(sum_sales#94), sum(number_sales#95)] - -(102) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#99, i_class_id#100, i_category_id#101, sum#88, isEmpty#89, sum#90] - -(103) CometHashAggregate -Input [7]: [channel#49, i_brand_id#99, i_class_id#100, i_category_id#101, sum#88, isEmpty#89, sum#90] -Keys [4]: [channel#49, i_brand_id#99, i_class_id#100, i_category_id#101] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(104) CometHashAggregate -Input [4]: [channel#49, i_brand_id#99, sum_sales#102, number_sales#103] -Keys [2]: [channel#49, i_brand_id#99] -Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] - -(105) CometExchange -Input [5]: [channel#49, i_brand_id#99, sum#104, isEmpty#105, sum#106] -Arguments: hashpartitioning(channel#49, i_brand_id#99, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(106) CometHashAggregate -Input [5]: [channel#49, i_brand_id#99, sum#104, isEmpty#105, sum#106] -Keys [2]: [channel#49, i_brand_id#99] -Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] - -(107) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#107, i_class_id#108, i_category_id#109, sum#88, isEmpty#89, sum#90] - -(108) CometHashAggregate -Input [7]: [channel#49, i_brand_id#107, i_class_id#108, i_category_id#109, sum#88, isEmpty#89, sum#90] -Keys [4]: [channel#49, i_brand_id#107, i_class_id#108, i_category_id#109] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(109) CometHashAggregate -Input [3]: [channel#49, sum_sales#110, number_sales#111] -Keys [1]: [channel#49] -Functions [2]: [partial_sum(sum_sales#110), partial_sum(number_sales#111)] - -(110) CometExchange -Input [4]: [channel#49, sum#112, isEmpty#113, sum#114] -Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(111) CometHashAggregate -Input [4]: [channel#49, sum#112, isEmpty#113, sum#114] -Keys [1]: [channel#49] -Functions [2]: [sum(sum_sales#110), sum(number_sales#111)] - -(112) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#115, i_class_id#116, i_category_id#117, sum#88, isEmpty#89, sum#90] - -(113) CometHashAggregate -Input [7]: [channel#49, i_brand_id#115, i_class_id#116, i_category_id#117, sum#88, isEmpty#89, sum#90] -Keys [4]: [channel#49, i_brand_id#115, i_class_id#116, i_category_id#117] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(114) CometHashAggregate -Input [2]: [sum_sales#118, number_sales#119] -Keys: [] -Functions [2]: [partial_sum(sum_sales#118), partial_sum(number_sales#119)] - -(115) CometExchange -Input [3]: [sum#120, isEmpty#121, sum#122] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(116) CometHashAggregate -Input [3]: [sum#120, isEmpty#121, sum#122] -Keys: [] -Functions [2]: [sum(sum_sales#118), sum(number_sales#119)] - -(117) CometUnion -Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Child 1 Input [6]: [channel#49, i_brand_id#91, i_class_id#92, i_category_id#125, sum(sum_sales)#126, sum(number_sales)#127] -Child 2 Input [6]: [channel#49, i_brand_id#99, i_class_id#128, i_category_id#129, sum(sum_sales)#130, sum(number_sales)#131] -Child 3 Input [6]: [channel#49, i_brand_id#132, i_class_id#133, i_category_id#134, sum(sum_sales)#135, sum(number_sales)#136] -Child 4 Input [6]: [channel#137, i_brand_id#138, i_class_id#139, i_category_id#140, sum(sum_sales)#141, sum(number_sales)#142] - -(118) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Functions: [] - -(119) CometExchange -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(120) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Functions: [] - -(121) CometTakeOrderedAndProject -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#49 ASC NULLS FIRST,i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#49,i_brand_id#40,i_class_id#41,i_category_id#42,sum_sales#123,number_sales#124]), [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124], 100, 0, [channel#49 ASC NULLS FIRST, i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] - -(122) CometColumnarToRow [codegen id : 1] -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#123, number_sales#124] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* CometColumnarToRow (142) -+- CometHashAggregate (141) - +- CometExchange (140) - +- CometHashAggregate (139) - +- CometUnion (138) - :- CometProject (126) - : +- CometBroadcastHashJoin (125) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (123) - : +- ReusedExchange (124) - :- CometProject (133) - : +- CometBroadcastHashJoin (132) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (127) - : +- CometBroadcastExchange (131) - : +- CometProject (130) - : +- CometFilter (129) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (128) - +- CometProject (137) - +- CometBroadcastHashJoin (136) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (134) - +- ReusedExchange (135) - - -(123) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#143, ss_list_price#144, ss_sold_date_sk#145] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#145), dynamicpruningexpression(ss_sold_date_sk#145 IN dynamicpruning#146)] -ReadSchema: struct - -(124) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#147] - -(125) CometBroadcastHashJoin -Left output [3]: [ss_quantity#143, ss_list_price#144, ss_sold_date_sk#145] -Right output [1]: [d_date_sk#147] -Arguments: [ss_sold_date_sk#145], [d_date_sk#147], Inner, BuildRight - -(126) CometProject -Input [4]: [ss_quantity#143, ss_list_price#144, ss_sold_date_sk#145, d_date_sk#147] -Arguments: [quantity#148, list_price#149], [ss_quantity#143 AS quantity#148, ss_list_price#144 AS list_price#149] - -(127) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#150, cs_list_price#151, cs_sold_date_sk#152] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#152), dynamicpruningexpression(cs_sold_date_sk#152 IN dynamicpruning#153)] -ReadSchema: struct - -(128) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#154, d_year#155] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(129) CometFilter -Input [2]: [d_date_sk#154, d_year#155] -Condition : (((isnotnull(d_year#155) AND (d_year#155 >= 1998)) AND (d_year#155 <= 2000)) AND isnotnull(d_date_sk#154)) - -(130) CometProject -Input [2]: [d_date_sk#154, d_year#155] -Arguments: [d_date_sk#154], [d_date_sk#154] - -(131) CometBroadcastExchange -Input [1]: [d_date_sk#154] -Arguments: [d_date_sk#154] - -(132) CometBroadcastHashJoin -Left output [3]: [cs_quantity#150, cs_list_price#151, cs_sold_date_sk#152] -Right output [1]: [d_date_sk#154] -Arguments: [cs_sold_date_sk#152], [d_date_sk#154], Inner, BuildRight - -(133) CometProject -Input [4]: [cs_quantity#150, cs_list_price#151, cs_sold_date_sk#152, d_date_sk#154] -Arguments: [quantity#156, list_price#157], [cs_quantity#150 AS quantity#156, cs_list_price#151 AS list_price#157] - -(134) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#158, ws_list_price#159, ws_sold_date_sk#160] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#160), dynamicpruningexpression(ws_sold_date_sk#160 IN dynamicpruning#161)] -ReadSchema: struct - -(135) ReusedExchange [Reuses operator id: 131] -Output [1]: [d_date_sk#162] - -(136) CometBroadcastHashJoin -Left output [3]: [ws_quantity#158, ws_list_price#159, ws_sold_date_sk#160] -Right output [1]: [d_date_sk#162] -Arguments: [ws_sold_date_sk#160], [d_date_sk#162], Inner, BuildRight - -(137) CometProject -Input [4]: [ws_quantity#158, ws_list_price#159, ws_sold_date_sk#160, d_date_sk#162] -Arguments: [quantity#163, list_price#164], [ws_quantity#158 AS quantity#163, ws_list_price#159 AS list_price#164] - -(138) CometUnion -Child 0 Input [2]: [quantity#148, list_price#149] -Child 1 Input [2]: [quantity#156, list_price#157] -Child 2 Input [2]: [quantity#163, list_price#164] - -(139) CometHashAggregate -Input [2]: [quantity#148, list_price#149] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#148 as decimal(10,0)) * list_price#149))] - -(140) CometExchange -Input [2]: [sum#165, count#166] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(141) CometHashAggregate -Input [2]: [sum#165, count#166] -Keys: [] -Functions [1]: [avg((cast(quantity#148 as decimal(10,0)) * list_price#149))] - -(142) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#167] - -Subquery:2 Hosting operator id = 123 Hosting Expression = ss_sold_date_sk#145 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 127 Hosting Expression = cs_sold_date_sk#152 IN dynamicpruning#153 -BroadcastExchange (147) -+- * CometColumnarToRow (146) - +- CometProject (145) - +- CometFilter (144) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (143) - - -(143) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#154, d_year#155] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(144) CometFilter -Input [2]: [d_date_sk#154, d_year#155] -Condition : (((isnotnull(d_year#155) AND (d_year#155 >= 1998)) AND (d_year#155 <= 2000)) AND isnotnull(d_date_sk#154)) - -(145) CometProject -Input [2]: [d_date_sk#154, d_year#155] -Arguments: [d_date_sk#154], [d_date_sk#154] - -(146) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#154] - -(147) BroadcastExchange -Input [1]: [d_date_sk#154] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#160 IN dynamicpruning#153 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (152) -+- * CometColumnarToRow (151) - +- CometProject (150) - +- CometFilter (149) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (148) - - -(148) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(149) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(150) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(151) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(152) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (157) -+- * CometColumnarToRow (156) - +- CometProject (155) - +- CometFilter (154) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (153) - - -(153) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#168] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(154) CometFilter -Input [2]: [d_date_sk#26, d_year#168] -Condition : (((isnotnull(d_year#168) AND (d_year#168 >= 1999)) AND (d_year#168 <= 2001)) AND isnotnull(d_date_sk#26)) - -(155) CometProject -Input [2]: [d_date_sk#26, d_year#168] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(156) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(157) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 78 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:10 Hosting operator id = 65 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 - -Subquery:11 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:12 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/extended.txt deleted file mode 100644 index 83deb45432..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/extended.txt +++ /dev/null @@ -1,2350 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 2127 out of 2302 eligible operators (92%). Final plan contains 46 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 758e917335..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a/simplified.txt +++ /dev/null @@ -1,179 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometUnion [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - CometExchange [channel,i_brand_id,i_class_id,i_category_id] #2 - CometHashAggregate [sales,number_sales] [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] - CometUnion [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #16 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #18 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 - ReusedExchange [d_date_sk] #18 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #3 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk] #5 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #6 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #7 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #12 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #12 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #13 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #12 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #5 - CometBroadcastExchange [d_date_sk] #15 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #19 - CometHashAggregate [cs_quantity,cs_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk] - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #5 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - ReusedExchange [d_date_sk] #15 - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #20 - CometHashAggregate [ws_quantity,ws_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk] - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #5 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - ReusedExchange [d_date_sk] #15 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel,i_brand_id,i_class_id] #21 - CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,i_class_id,sum,isEmpty,sum] - CometHashAggregate [i_category_id,sum,isEmpty,sum] [channel,i_brand_id,i_class_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel,i_brand_id] #22 - CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,sum,isEmpty,sum] - CometHashAggregate [i_class_id,i_category_id,sum,isEmpty,sum] [channel,i_brand_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel] #23 - CometHashAggregate [sum_sales,number_sales] [channel,sum,isEmpty,sum] - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [channel,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange #24 - CometHashAggregate [sum_sales,number_sales] [sum,isEmpty,sum] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/explain.txt deleted file mode 100644 index db9dbcc51a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/explain.txt +++ /dev/null @@ -1,944 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (161) -+- Union (160) - :- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- CometColumnarExchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (33) - : : +- * BroadcastHashJoin Inner BuildRight (32) - : : :- * Project (30) - : : : +- * BroadcastHashJoin Inner BuildRight (29) - : : : :- * Project (23) - : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : :- * Project (17) - : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * CometColumnarToRow (7) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) - : : : : : +- BroadcastExchange (15) - : : : : : +- * CometColumnarToRow (14) - : : : : : +- CometProject (13) - : : : : : +- CometFilter (12) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (11) - : : : : +- BroadcastExchange (21) - : : : : +- * CometColumnarToRow (20) - : : : : +- CometFilter (19) - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (18) - : : : +- BroadcastExchange (28) - : : : +- * CometColumnarToRow (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (24) - : : +- ReusedExchange (31) - : +- BroadcastExchange (38) - : +- * CometColumnarToRow (37) - : +- CometProject (36) - : +- CometFilter (35) - : +- CometNativeScan parquet spark_catalog.default.item (34) - :- * HashAggregate (73) - : +- * CometColumnarToRow (72) - : +- CometColumnarExchange (71) - : +- * HashAggregate (70) - : +- * Project (69) - : +- * BroadcastHashJoin Inner BuildRight (68) - : :- * Project (66) - : : +- * BroadcastHashJoin Inner BuildRight (65) - : : :- * Project (63) - : : : +- * BroadcastHashJoin Inner BuildRight (62) - : : : :- * Project (56) - : : : : +- * BroadcastHashJoin Inner BuildRight (55) - : : : : :- * Project (53) - : : : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : : : :- * Project (50) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : : : :- * Filter (47) - : : : : : : : +- * ColumnarToRow (46) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (45) - : : : : : : +- ReusedExchange (48) - : : : : : +- ReusedExchange (51) - : : : : +- ReusedExchange (54) - : : : +- BroadcastExchange (61) - : : : +- * CometColumnarToRow (60) - : : : +- CometProject (59) - : : : +- CometFilter (58) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (57) - : : +- ReusedExchange (64) - : +- ReusedExchange (67) - :- * HashAggregate (102) - : +- * CometColumnarToRow (101) - : +- CometColumnarExchange (100) - : +- * HashAggregate (99) - : +- * Project (98) - : +- * BroadcastHashJoin Inner BuildRight (97) - : :- * Project (95) - : : +- * BroadcastHashJoin Inner BuildRight (94) - : : :- * Project (92) - : : : +- * BroadcastHashJoin Inner BuildRight (91) - : : : :- * Project (85) - : : : : +- * BroadcastHashJoin Inner BuildRight (84) - : : : : :- * Project (82) - : : : : : +- * BroadcastHashJoin Inner BuildRight (81) - : : : : : :- * Project (79) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) - : : : : : : :- * Filter (76) - : : : : : : : +- * ColumnarToRow (75) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (74) - : : : : : : +- ReusedExchange (77) - : : : : : +- ReusedExchange (80) - : : : : +- ReusedExchange (83) - : : : +- BroadcastExchange (90) - : : : +- * CometColumnarToRow (89) - : : : +- CometProject (88) - : : : +- CometFilter (87) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (86) - : : +- ReusedExchange (93) - : +- ReusedExchange (96) - :- * HashAggregate (131) - : +- * CometColumnarToRow (130) - : +- CometColumnarExchange (129) - : +- * HashAggregate (128) - : +- * Project (127) - : +- * BroadcastHashJoin Inner BuildRight (126) - : :- * Project (124) - : : +- * BroadcastHashJoin Inner BuildRight (123) - : : :- * Project (121) - : : : +- * BroadcastHashJoin Inner BuildRight (120) - : : : :- * Project (114) - : : : : +- * BroadcastHashJoin Inner BuildRight (113) - : : : : :- * Project (111) - : : : : : +- * BroadcastHashJoin Inner BuildRight (110) - : : : : : :- * Project (108) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (107) - : : : : : : :- * Filter (105) - : : : : : : : +- * ColumnarToRow (104) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (103) - : : : : : : +- ReusedExchange (106) - : : : : : +- ReusedExchange (109) - : : : : +- ReusedExchange (112) - : : : +- BroadcastExchange (119) - : : : +- * CometColumnarToRow (118) - : : : +- CometProject (117) - : : : +- CometFilter (116) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (115) - : : +- ReusedExchange (122) - : +- ReusedExchange (125) - +- * HashAggregate (159) - +- * CometColumnarToRow (158) - +- CometColumnarExchange (157) - +- * HashAggregate (156) - +- * Project (155) - +- * BroadcastHashJoin Inner BuildRight (154) - :- * Project (149) - : +- * BroadcastHashJoin Inner BuildRight (148) - : :- * Project (146) - : : +- * BroadcastHashJoin Inner BuildRight (145) - : : :- * Project (143) - : : : +- * BroadcastHashJoin Inner BuildRight (142) - : : : :- * Project (140) - : : : : +- * BroadcastHashJoin Inner BuildRight (139) - : : : : :- * Project (137) - : : : : : +- * BroadcastHashJoin Inner BuildRight (136) - : : : : : :- * Filter (134) - : : : : : : +- * ColumnarToRow (133) - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (132) - : : : : : +- ReusedExchange (135) - : : : : +- ReusedExchange (138) - : : : +- ReusedExchange (141) - : : +- ReusedExchange (144) - : +- ReusedExchange (147) - +- BroadcastExchange (153) - +- * CometColumnarToRow (152) - +- CometFilter (151) - +- CometNativeScan parquet spark_catalog.default.item (150) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 7] -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] - -(3) Filter [codegen id : 7] -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) - -(4) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#12, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#11)) - -(6) CometProject -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] - -(7) CometColumnarToRow [codegen id : 1] -Input [2]: [cd_demo_sk#11, cd_dep_count#14] - -(8) BroadcastExchange -Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 7] -Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] - -(11) CometNativeScan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [In(c_birth_month, [1,10,12,4,5,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(12) CometFilter -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Condition : (((c_birth_month#18 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) - -(13) CometProject -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(14) CometColumnarToRow [codegen id : 2] -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(15) BroadcastExchange -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(18) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(19) CometFilter -Input [1]: [cd_demo_sk#20] -Condition : isnotnull(cd_demo_sk#20) - -(20) CometColumnarToRow [codegen id : 3] -Input [1]: [cd_demo_sk#20] - -(21) BroadcastExchange -Input [1]: [cd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#16] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None - -(23) Project [codegen id : 7] -Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] - -(24) CometNativeScan parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) - -(26) CometProject -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) AS ca_state#25, ca_country#24] - -(27) CometColumnarToRow [codegen id : 4] -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(28) BroadcastExchange -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(29) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#17] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None - -(30) Project [codegen id : 7] -Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(31) ReusedExchange [Reuses operator id: 166] -Output [1]: [d_date_sk#26] - -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#26] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] - -(34) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#27, i_item_id#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [i_item_sk#27, i_item_id#28] -Condition : isnotnull(i_item_sk#27) - -(36) CometProject -Input [2]: [i_item_sk#27, i_item_id#28] -Arguments: [i_item_sk#27, i_item_id#29], [i_item_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#28, 16, true, false, true) AS i_item_id#29] - -(37) CometColumnarToRow [codegen id : 6] -Input [2]: [i_item_sk#27, i_item_id#29] - -(38) BroadcastExchange -Input [2]: [i_item_sk#27, i_item_id#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#27] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 7] -Output [11]: [i_item_id#29 AS i_item_id#30, ca_country#24 AS ca_country#31, ca_state#25 AS ca_state#32, ca_county#22 AS ca_county#33, cast(cs_quantity#4 as decimal(12,2)) AS agg1#34, cast(cs_list_price#5 as decimal(12,2)) AS agg2#35, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#36, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#37, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#38, cast(c_birth_year#19 as decimal(12,2)) AS agg6#39, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#40] -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#27, i_item_id#29] - -(41) HashAggregate [codegen id : 7] -Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#34, agg2#35, agg3#36, agg4#37, agg5#38, agg6#39, agg7#40] -Keys [4]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33] -Functions [7]: [partial_avg(agg1#34), partial_avg(agg2#35), partial_avg(agg3#36), partial_avg(agg4#37), partial_avg(agg5#38), partial_avg(agg6#39), partial_avg(agg7#40)] -Aggregate Attributes [14]: [sum#41, count#42, sum#43, count#44, sum#45, count#46, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54] -Results [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] - -(42) CometColumnarExchange -Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] -Arguments: hashpartitioning(i_item_id#30, ca_country#31, ca_state#32, ca_county#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) CometColumnarToRow [codegen id : 8] -Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] - -(44) HashAggregate [codegen id : 8] -Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] -Keys [4]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33] -Functions [7]: [avg(agg1#34), avg(agg2#35), avg(agg3#36), avg(agg4#37), avg(agg5#38), avg(agg6#39), avg(agg7#40)] -Aggregate Attributes [7]: [avg(agg1#34)#69, avg(agg2#35)#70, avg(agg3#36)#71, avg(agg4#37)#72, avg(agg5#38)#73, avg(agg6#39)#74, avg(agg7#40)#75] -Results [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, avg(agg1#34)#69 AS agg1#76, avg(agg2#35)#70 AS agg2#77, avg(agg3#36)#71 AS agg3#78, avg(agg4#37)#72 AS agg4#79, avg(agg5#38)#73 AS agg5#80, avg(agg6#39)#74 AS agg6#81, avg(agg7#40)#75 AS agg7#82] - -(45) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#91), dynamicpruningexpression(cs_sold_date_sk#91 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 15] -Input [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] - -(47) Filter [codegen id : 15] -Input [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] -Condition : ((isnotnull(cs_bill_cdemo_sk#84) AND isnotnull(cs_bill_customer_sk#83)) AND isnotnull(cs_item_sk#85)) - -(48) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#92, cd_dep_count#93] - -(49) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_cdemo_sk#84] -Right keys [1]: [cd_demo_sk#92] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 15] -Output [9]: [cs_bill_customer_sk#83, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93] -Input [11]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_demo_sk#92, cd_dep_count#93] - -(51) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#94, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97] - -(52) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_customer_sk#83] -Right keys [1]: [c_customer_sk#94] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 15] -Output [11]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97] -Input [13]: [cs_bill_customer_sk#83, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_customer_sk#94, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97] - -(54) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#98] - -(55) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_cdemo_sk#95] -Right keys [1]: [cd_demo_sk#98] -Join type: Inner -Join condition: None - -(56) Project [codegen id : 15] -Output [10]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_addr_sk#96, c_birth_year#97] -Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97, cd_demo_sk#98] - -(57) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#99, ca_state#100, ca_country#101] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [ca_address_sk#99, ca_state#100, ca_country#101] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#100, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#99)) - -(59) CometProject -Input [3]: [ca_address_sk#99, ca_state#100, ca_country#101] -Arguments: [ca_address_sk#99, ca_state#25, ca_country#101], [ca_address_sk#99, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#100, 2, true, false, true) AS ca_state#25, ca_country#101] - -(60) CometColumnarToRow [codegen id : 12] -Input [3]: [ca_address_sk#99, ca_state#25, ca_country#101] - -(61) BroadcastExchange -Input [3]: [ca_address_sk#99, ca_state#25, ca_country#101] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(62) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_addr_sk#96] -Right keys [1]: [ca_address_sk#99] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 15] -Output [11]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_birth_year#97, ca_state#25, ca_country#101] -Input [13]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_addr_sk#96, c_birth_year#97, ca_address_sk#99, ca_state#25, ca_country#101] - -(64) ReusedExchange [Reuses operator id: 166] -Output [1]: [d_date_sk#102] - -(65) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_sold_date_sk#91] -Right keys [1]: [d_date_sk#102] -Join type: Inner -Join condition: None - -(66) Project [codegen id : 15] -Output [10]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#93, c_birth_year#97, ca_state#25, ca_country#101] -Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_birth_year#97, ca_state#25, ca_country#101, d_date_sk#102] - -(67) ReusedExchange [Reuses operator id: 38] -Output [2]: [i_item_sk#103, i_item_id#29] - -(68) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_item_sk#85] -Right keys [1]: [i_item_sk#103] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 15] -Output [10]: [i_item_id#29, ca_country#101, ca_state#25, cast(cs_quantity#86 as decimal(12,2)) AS agg1#104, cast(cs_list_price#87 as decimal(12,2)) AS agg2#105, cast(cs_coupon_amt#89 as decimal(12,2)) AS agg3#106, cast(cs_sales_price#88 as decimal(12,2)) AS agg4#107, cast(cs_net_profit#90 as decimal(12,2)) AS agg5#108, cast(c_birth_year#97 as decimal(12,2)) AS agg6#109, cast(cd_dep_count#93 as decimal(12,2)) AS agg7#110] -Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#93, c_birth_year#97, ca_state#25, ca_country#101, i_item_sk#103, i_item_id#29] - -(70) HashAggregate [codegen id : 15] -Input [10]: [i_item_id#29, ca_country#101, ca_state#25, agg1#104, agg2#105, agg3#106, agg4#107, agg5#108, agg6#109, agg7#110] -Keys [3]: [i_item_id#29, ca_country#101, ca_state#25] -Functions [7]: [partial_avg(agg1#104), partial_avg(agg2#105), partial_avg(agg3#106), partial_avg(agg4#107), partial_avg(agg5#108), partial_avg(agg6#109), partial_avg(agg7#110)] -Aggregate Attributes [14]: [sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] -Results [17]: [i_item_id#29, ca_country#101, ca_state#25, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134, sum#135, count#136, sum#137, count#138] - -(71) CometColumnarExchange -Input [17]: [i_item_id#29, ca_country#101, ca_state#25, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134, sum#135, count#136, sum#137, count#138] -Arguments: hashpartitioning(i_item_id#29, ca_country#101, ca_state#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(72) CometColumnarToRow [codegen id : 16] -Input [17]: [i_item_id#29, ca_country#101, ca_state#25, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134, sum#135, count#136, sum#137, count#138] - -(73) HashAggregate [codegen id : 16] -Input [17]: [i_item_id#29, ca_country#101, ca_state#25, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134, sum#135, count#136, sum#137, count#138] -Keys [3]: [i_item_id#29, ca_country#101, ca_state#25] -Functions [7]: [avg(agg1#104), avg(agg2#105), avg(agg3#106), avg(agg4#107), avg(agg5#108), avg(agg6#109), avg(agg7#110)] -Aggregate Attributes [7]: [avg(agg1#104)#139, avg(agg2#105)#140, avg(agg3#106)#141, avg(agg4#107)#142, avg(agg5#108)#143, avg(agg6#109)#144, avg(agg7#110)#145] -Results [11]: [i_item_id#29, ca_country#101, ca_state#25, null AS county#146, avg(agg1#104)#139 AS agg1#147, avg(agg2#105)#140 AS agg2#148, avg(agg3#106)#141 AS agg3#149, avg(agg4#107)#142 AS agg4#150, avg(agg5#108)#143 AS agg5#151, avg(agg6#109)#144 AS agg6#152, avg(agg7#110)#145 AS agg7#153] - -(74) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#154, cs_bill_cdemo_sk#155, cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#162), dynamicpruningexpression(cs_sold_date_sk#162 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(75) ColumnarToRow [codegen id : 23] -Input [9]: [cs_bill_customer_sk#154, cs_bill_cdemo_sk#155, cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162] - -(76) Filter [codegen id : 23] -Input [9]: [cs_bill_customer_sk#154, cs_bill_cdemo_sk#155, cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162] -Condition : ((isnotnull(cs_bill_cdemo_sk#155) AND isnotnull(cs_bill_customer_sk#154)) AND isnotnull(cs_item_sk#156)) - -(77) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#163, cd_dep_count#164] - -(78) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_cdemo_sk#155] -Right keys [1]: [cd_demo_sk#163] -Join type: Inner -Join condition: None - -(79) Project [codegen id : 23] -Output [9]: [cs_bill_customer_sk#154, cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162, cd_dep_count#164] -Input [11]: [cs_bill_customer_sk#154, cs_bill_cdemo_sk#155, cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162, cd_demo_sk#163, cd_dep_count#164] - -(80) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#165, c_current_cdemo_sk#166, c_current_addr_sk#167, c_birth_year#168] - -(81) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_customer_sk#154] -Right keys [1]: [c_customer_sk#165] -Join type: Inner -Join condition: None - -(82) Project [codegen id : 23] -Output [11]: [cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162, cd_dep_count#164, c_current_cdemo_sk#166, c_current_addr_sk#167, c_birth_year#168] -Input [13]: [cs_bill_customer_sk#154, cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162, cd_dep_count#164, c_customer_sk#165, c_current_cdemo_sk#166, c_current_addr_sk#167, c_birth_year#168] - -(83) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#169] - -(84) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_cdemo_sk#166] -Right keys [1]: [cd_demo_sk#169] -Join type: Inner -Join condition: None - -(85) Project [codegen id : 23] -Output [10]: [cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162, cd_dep_count#164, c_current_addr_sk#167, c_birth_year#168] -Input [12]: [cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162, cd_dep_count#164, c_current_cdemo_sk#166, c_current_addr_sk#167, c_birth_year#168, cd_demo_sk#169] - -(86) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#170, ca_state#171, ca_country#172] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(87) CometFilter -Input [3]: [ca_address_sk#170, ca_state#171, ca_country#172] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#171, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#170)) - -(88) CometProject -Input [3]: [ca_address_sk#170, ca_state#171, ca_country#172] -Arguments: [ca_address_sk#170, ca_country#172], [ca_address_sk#170, ca_country#172] - -(89) CometColumnarToRow [codegen id : 20] -Input [2]: [ca_address_sk#170, ca_country#172] - -(90) BroadcastExchange -Input [2]: [ca_address_sk#170, ca_country#172] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -(91) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_addr_sk#167] -Right keys [1]: [ca_address_sk#170] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 23] -Output [10]: [cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162, cd_dep_count#164, c_birth_year#168, ca_country#172] -Input [12]: [cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162, cd_dep_count#164, c_current_addr_sk#167, c_birth_year#168, ca_address_sk#170, ca_country#172] - -(93) ReusedExchange [Reuses operator id: 166] -Output [1]: [d_date_sk#173] - -(94) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_sold_date_sk#162] -Right keys [1]: [d_date_sk#173] -Join type: Inner -Join condition: None - -(95) Project [codegen id : 23] -Output [9]: [cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cd_dep_count#164, c_birth_year#168, ca_country#172] -Input [11]: [cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cs_sold_date_sk#162, cd_dep_count#164, c_birth_year#168, ca_country#172, d_date_sk#173] - -(96) ReusedExchange [Reuses operator id: 38] -Output [2]: [i_item_sk#174, i_item_id#29] - -(97) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_item_sk#156] -Right keys [1]: [i_item_sk#174] -Join type: Inner -Join condition: None - -(98) Project [codegen id : 23] -Output [9]: [i_item_id#29, ca_country#172, cast(cs_quantity#157 as decimal(12,2)) AS agg1#175, cast(cs_list_price#158 as decimal(12,2)) AS agg2#176, cast(cs_coupon_amt#160 as decimal(12,2)) AS agg3#177, cast(cs_sales_price#159 as decimal(12,2)) AS agg4#178, cast(cs_net_profit#161 as decimal(12,2)) AS agg5#179, cast(c_birth_year#168 as decimal(12,2)) AS agg6#180, cast(cd_dep_count#164 as decimal(12,2)) AS agg7#181] -Input [11]: [cs_item_sk#156, cs_quantity#157, cs_list_price#158, cs_sales_price#159, cs_coupon_amt#160, cs_net_profit#161, cd_dep_count#164, c_birth_year#168, ca_country#172, i_item_sk#174, i_item_id#29] - -(99) HashAggregate [codegen id : 23] -Input [9]: [i_item_id#29, ca_country#172, agg1#175, agg2#176, agg3#177, agg4#178, agg5#179, agg6#180, agg7#181] -Keys [2]: [i_item_id#29, ca_country#172] -Functions [7]: [partial_avg(agg1#175), partial_avg(agg2#176), partial_avg(agg3#177), partial_avg(agg4#178), partial_avg(agg5#179), partial_avg(agg6#180), partial_avg(agg7#181)] -Aggregate Attributes [14]: [sum#182, count#183, sum#184, count#185, sum#186, count#187, sum#188, count#189, sum#190, count#191, sum#192, count#193, sum#194, count#195] -Results [16]: [i_item_id#29, ca_country#172, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209] - -(100) CometColumnarExchange -Input [16]: [i_item_id#29, ca_country#172, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209] -Arguments: hashpartitioning(i_item_id#29, ca_country#172, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(101) CometColumnarToRow [codegen id : 24] -Input [16]: [i_item_id#29, ca_country#172, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209] - -(102) HashAggregate [codegen id : 24] -Input [16]: [i_item_id#29, ca_country#172, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209] -Keys [2]: [i_item_id#29, ca_country#172] -Functions [7]: [avg(agg1#175), avg(agg2#176), avg(agg3#177), avg(agg4#178), avg(agg5#179), avg(agg6#180), avg(agg7#181)] -Aggregate Attributes [7]: [avg(agg1#175)#210, avg(agg2#176)#211, avg(agg3#177)#212, avg(agg4#178)#213, avg(agg5#179)#214, avg(agg6#180)#215, avg(agg7#181)#216] -Results [11]: [i_item_id#29, ca_country#172, null AS ca_state#217, null AS county#218, avg(agg1#175)#210 AS agg1#219, avg(agg2#176)#211 AS agg2#220, avg(agg3#177)#212 AS agg3#221, avg(agg4#178)#213 AS agg4#222, avg(agg5#179)#214 AS agg5#223, avg(agg6#180)#215 AS agg6#224, avg(agg7#181)#216 AS agg7#225] - -(103) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#226, cs_bill_cdemo_sk#227, cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#234), dynamicpruningexpression(cs_sold_date_sk#234 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(104) ColumnarToRow [codegen id : 31] -Input [9]: [cs_bill_customer_sk#226, cs_bill_cdemo_sk#227, cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234] - -(105) Filter [codegen id : 31] -Input [9]: [cs_bill_customer_sk#226, cs_bill_cdemo_sk#227, cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234] -Condition : ((isnotnull(cs_bill_cdemo_sk#227) AND isnotnull(cs_bill_customer_sk#226)) AND isnotnull(cs_item_sk#228)) - -(106) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#235, cd_dep_count#236] - -(107) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_cdemo_sk#227] -Right keys [1]: [cd_demo_sk#235] -Join type: Inner -Join condition: None - -(108) Project [codegen id : 31] -Output [9]: [cs_bill_customer_sk#226, cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234, cd_dep_count#236] -Input [11]: [cs_bill_customer_sk#226, cs_bill_cdemo_sk#227, cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234, cd_demo_sk#235, cd_dep_count#236] - -(109) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#237, c_current_cdemo_sk#238, c_current_addr_sk#239, c_birth_year#240] - -(110) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_customer_sk#226] -Right keys [1]: [c_customer_sk#237] -Join type: Inner -Join condition: None - -(111) Project [codegen id : 31] -Output [11]: [cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234, cd_dep_count#236, c_current_cdemo_sk#238, c_current_addr_sk#239, c_birth_year#240] -Input [13]: [cs_bill_customer_sk#226, cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234, cd_dep_count#236, c_customer_sk#237, c_current_cdemo_sk#238, c_current_addr_sk#239, c_birth_year#240] - -(112) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#241] - -(113) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_cdemo_sk#238] -Right keys [1]: [cd_demo_sk#241] -Join type: Inner -Join condition: None - -(114) Project [codegen id : 31] -Output [10]: [cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234, cd_dep_count#236, c_current_addr_sk#239, c_birth_year#240] -Input [12]: [cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234, cd_dep_count#236, c_current_cdemo_sk#238, c_current_addr_sk#239, c_birth_year#240, cd_demo_sk#241] - -(115) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#242, ca_state#243] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(116) CometFilter -Input [2]: [ca_address_sk#242, ca_state#243] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#243, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#242)) - -(117) CometProject -Input [2]: [ca_address_sk#242, ca_state#243] -Arguments: [ca_address_sk#242], [ca_address_sk#242] - -(118) CometColumnarToRow [codegen id : 28] -Input [1]: [ca_address_sk#242] - -(119) BroadcastExchange -Input [1]: [ca_address_sk#242] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -(120) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_addr_sk#239] -Right keys [1]: [ca_address_sk#242] -Join type: Inner -Join condition: None - -(121) Project [codegen id : 31] -Output [9]: [cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234, cd_dep_count#236, c_birth_year#240] -Input [11]: [cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234, cd_dep_count#236, c_current_addr_sk#239, c_birth_year#240, ca_address_sk#242] - -(122) ReusedExchange [Reuses operator id: 166] -Output [1]: [d_date_sk#244] - -(123) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_sold_date_sk#234] -Right keys [1]: [d_date_sk#244] -Join type: Inner -Join condition: None - -(124) Project [codegen id : 31] -Output [8]: [cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cd_dep_count#236, c_birth_year#240] -Input [10]: [cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cs_sold_date_sk#234, cd_dep_count#236, c_birth_year#240, d_date_sk#244] - -(125) ReusedExchange [Reuses operator id: 38] -Output [2]: [i_item_sk#245, i_item_id#29] - -(126) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_item_sk#228] -Right keys [1]: [i_item_sk#245] -Join type: Inner -Join condition: None - -(127) Project [codegen id : 31] -Output [8]: [i_item_id#29, cast(cs_quantity#229 as decimal(12,2)) AS agg1#246, cast(cs_list_price#230 as decimal(12,2)) AS agg2#247, cast(cs_coupon_amt#232 as decimal(12,2)) AS agg3#248, cast(cs_sales_price#231 as decimal(12,2)) AS agg4#249, cast(cs_net_profit#233 as decimal(12,2)) AS agg5#250, cast(c_birth_year#240 as decimal(12,2)) AS agg6#251, cast(cd_dep_count#236 as decimal(12,2)) AS agg7#252] -Input [10]: [cs_item_sk#228, cs_quantity#229, cs_list_price#230, cs_sales_price#231, cs_coupon_amt#232, cs_net_profit#233, cd_dep_count#236, c_birth_year#240, i_item_sk#245, i_item_id#29] - -(128) HashAggregate [codegen id : 31] -Input [8]: [i_item_id#29, agg1#246, agg2#247, agg3#248, agg4#249, agg5#250, agg6#251, agg7#252] -Keys [1]: [i_item_id#29] -Functions [7]: [partial_avg(agg1#246), partial_avg(agg2#247), partial_avg(agg3#248), partial_avg(agg4#249), partial_avg(agg5#250), partial_avg(agg6#251), partial_avg(agg7#252)] -Aggregate Attributes [14]: [sum#253, count#254, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266] -Results [15]: [i_item_id#29, sum#267, count#268, sum#269, count#270, sum#271, count#272, sum#273, count#274, sum#275, count#276, sum#277, count#278, sum#279, count#280] - -(129) CometColumnarExchange -Input [15]: [i_item_id#29, sum#267, count#268, sum#269, count#270, sum#271, count#272, sum#273, count#274, sum#275, count#276, sum#277, count#278, sum#279, count#280] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(130) CometColumnarToRow [codegen id : 32] -Input [15]: [i_item_id#29, sum#267, count#268, sum#269, count#270, sum#271, count#272, sum#273, count#274, sum#275, count#276, sum#277, count#278, sum#279, count#280] - -(131) HashAggregate [codegen id : 32] -Input [15]: [i_item_id#29, sum#267, count#268, sum#269, count#270, sum#271, count#272, sum#273, count#274, sum#275, count#276, sum#277, count#278, sum#279, count#280] -Keys [1]: [i_item_id#29] -Functions [7]: [avg(agg1#246), avg(agg2#247), avg(agg3#248), avg(agg4#249), avg(agg5#250), avg(agg6#251), avg(agg7#252)] -Aggregate Attributes [7]: [avg(agg1#246)#281, avg(agg2#247)#282, avg(agg3#248)#283, avg(agg4#249)#284, avg(agg5#250)#285, avg(agg6#251)#286, avg(agg7#252)#287] -Results [11]: [i_item_id#29, null AS ca_country#288, null AS ca_state#289, null AS county#290, avg(agg1#246)#281 AS agg1#291, avg(agg2#247)#282 AS agg2#292, avg(agg3#248)#283 AS agg3#293, avg(agg4#249)#284 AS agg4#294, avg(agg5#250)#285 AS agg5#295, avg(agg6#251)#286 AS agg6#296, avg(agg7#252)#287 AS agg7#297] - -(132) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#298, cs_bill_cdemo_sk#299, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#306), dynamicpruningexpression(cs_sold_date_sk#306 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(133) ColumnarToRow [codegen id : 39] -Input [9]: [cs_bill_customer_sk#298, cs_bill_cdemo_sk#299, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306] - -(134) Filter [codegen id : 39] -Input [9]: [cs_bill_customer_sk#298, cs_bill_cdemo_sk#299, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306] -Condition : ((isnotnull(cs_bill_cdemo_sk#299) AND isnotnull(cs_bill_customer_sk#298)) AND isnotnull(cs_item_sk#300)) - -(135) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#307, cd_dep_count#308] - -(136) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_cdemo_sk#299] -Right keys [1]: [cd_demo_sk#307] -Join type: Inner -Join condition: None - -(137) Project [codegen id : 39] -Output [9]: [cs_bill_customer_sk#298, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#308] -Input [11]: [cs_bill_customer_sk#298, cs_bill_cdemo_sk#299, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_demo_sk#307, cd_dep_count#308] - -(138) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#309, c_current_cdemo_sk#310, c_current_addr_sk#311, c_birth_year#312] - -(139) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_customer_sk#298] -Right keys [1]: [c_customer_sk#309] -Join type: Inner -Join condition: None - -(140) Project [codegen id : 39] -Output [11]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#308, c_current_cdemo_sk#310, c_current_addr_sk#311, c_birth_year#312] -Input [13]: [cs_bill_customer_sk#298, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#308, c_customer_sk#309, c_current_cdemo_sk#310, c_current_addr_sk#311, c_birth_year#312] - -(141) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#313] - -(142) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_cdemo_sk#310] -Right keys [1]: [cd_demo_sk#313] -Join type: Inner -Join condition: None - -(143) Project [codegen id : 39] -Output [10]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#308, c_current_addr_sk#311, c_birth_year#312] -Input [12]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#308, c_current_cdemo_sk#310, c_current_addr_sk#311, c_birth_year#312, cd_demo_sk#313] - -(144) ReusedExchange [Reuses operator id: 119] -Output [1]: [ca_address_sk#314] - -(145) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_addr_sk#311] -Right keys [1]: [ca_address_sk#314] -Join type: Inner -Join condition: None - -(146) Project [codegen id : 39] -Output [9]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#308, c_birth_year#312] -Input [11]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#308, c_current_addr_sk#311, c_birth_year#312, ca_address_sk#314] - -(147) ReusedExchange [Reuses operator id: 166] -Output [1]: [d_date_sk#315] - -(148) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_sold_date_sk#306] -Right keys [1]: [d_date_sk#315] -Join type: Inner -Join condition: None - -(149) Project [codegen id : 39] -Output [8]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cd_dep_count#308, c_birth_year#312] -Input [10]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#308, c_birth_year#312, d_date_sk#315] - -(150) CometNativeScan parquet spark_catalog.default.item -Output [1]: [i_item_sk#316] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(151) CometFilter -Input [1]: [i_item_sk#316] -Condition : isnotnull(i_item_sk#316) - -(152) CometColumnarToRow [codegen id : 38] -Input [1]: [i_item_sk#316] - -(153) BroadcastExchange -Input [1]: [i_item_sk#316] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] - -(154) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_item_sk#300] -Right keys [1]: [i_item_sk#316] -Join type: Inner -Join condition: None - -(155) Project [codegen id : 39] -Output [7]: [cast(cs_quantity#301 as decimal(12,2)) AS agg1#317, cast(cs_list_price#302 as decimal(12,2)) AS agg2#318, cast(cs_coupon_amt#304 as decimal(12,2)) AS agg3#319, cast(cs_sales_price#303 as decimal(12,2)) AS agg4#320, cast(cs_net_profit#305 as decimal(12,2)) AS agg5#321, cast(c_birth_year#312 as decimal(12,2)) AS agg6#322, cast(cd_dep_count#308 as decimal(12,2)) AS agg7#323] -Input [9]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cd_dep_count#308, c_birth_year#312, i_item_sk#316] - -(156) HashAggregate [codegen id : 39] -Input [7]: [agg1#317, agg2#318, agg3#319, agg4#320, agg5#321, agg6#322, agg7#323] -Keys: [] -Functions [7]: [partial_avg(agg1#317), partial_avg(agg2#318), partial_avg(agg3#319), partial_avg(agg4#320), partial_avg(agg5#321), partial_avg(agg6#322), partial_avg(agg7#323)] -Aggregate Attributes [14]: [sum#324, count#325, sum#326, count#327, sum#328, count#329, sum#330, count#331, sum#332, count#333, sum#334, count#335, sum#336, count#337] -Results [14]: [sum#338, count#339, sum#340, count#341, sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351] - -(157) CometColumnarExchange -Input [14]: [sum#338, count#339, sum#340, count#341, sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(158) CometColumnarToRow [codegen id : 40] -Input [14]: [sum#338, count#339, sum#340, count#341, sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351] - -(159) HashAggregate [codegen id : 40] -Input [14]: [sum#338, count#339, sum#340, count#341, sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351] -Keys: [] -Functions [7]: [avg(agg1#317), avg(agg2#318), avg(agg3#319), avg(agg4#320), avg(agg5#321), avg(agg6#322), avg(agg7#323)] -Aggregate Attributes [7]: [avg(agg1#317)#352, avg(agg2#318)#353, avg(agg3#319)#354, avg(agg4#320)#355, avg(agg5#321)#356, avg(agg6#322)#357, avg(agg7#323)#358] -Results [11]: [null AS i_item_id#359, null AS ca_country#360, null AS ca_state#361, null AS county#362, avg(agg1#317)#352 AS agg1#363, avg(agg2#318)#353 AS agg2#364, avg(agg3#319)#354 AS agg3#365, avg(agg4#320)#355 AS agg4#366, avg(agg5#321)#356 AS agg5#367, avg(agg6#322)#357 AS agg6#368, avg(agg7#323)#358 AS agg7#369] - -(160) Union - -(161) TakeOrderedAndProject -Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] -Arguments: 100, [ca_country#31 ASC NULLS FIRST, ca_state#32 ASC NULLS FIRST, ca_county#33 ASC NULLS FIRST, i_item_id#30 ASC NULLS FIRST], [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (166) -+- * CometColumnarToRow (165) - +- CometProject (164) - +- CometFilter (163) - +- CometNativeScan parquet spark_catalog.default.date_dim (162) - - -(162) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#370] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(163) CometFilter -Input [2]: [d_date_sk#26, d_year#370] -Condition : ((isnotnull(d_year#370) AND (d_year#370 = 2001)) AND isnotnull(d_date_sk#26)) - -(164) CometProject -Input [2]: [d_date_sk#26, d_year#370] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(165) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(166) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] - -Subquery:2 Hosting operator id = 45 Hosting Expression = cs_sold_date_sk#91 IN dynamicpruning#10 - -Subquery:3 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#162 IN dynamicpruning#10 - -Subquery:4 Hosting operator id = 103 Hosting Expression = cs_sold_date_sk#234 IN dynamicpruning#10 - -Subquery:5 Hosting operator id = 132 Hosting Expression = cs_sold_date_sk#306 IN dynamicpruning#10 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/simplified.txt deleted file mode 100644 index 71745258b5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/simplified.txt +++ /dev/null @@ -1,241 +0,0 @@ -TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - Union - WholeStageCodegen (8) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,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),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county] #1 - WholeStageCodegen (7) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [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] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - WholeStageCodegen (16) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,ca_country,ca_state] #8 - WholeStageCodegen (15) - HashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,ca_state,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [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] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (24) - HashAggregate [i_item_id,ca_country,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),ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,ca_country] #10 - WholeStageCodegen (23) - HashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [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] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (32) - HashAggregate [i_item_id,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),ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #12 - WholeStageCodegen (31) - HashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [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] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (28) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (40) - HashAggregate [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),i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #14 - WholeStageCodegen (39) - HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [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] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - ReusedExchange [ca_address_sk] #13 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (38) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/explain.txt deleted file mode 100644 index 72e9bd1b4a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,846 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (146) -+- CometTakeOrderedAndProject (145) - +- CometUnion (144) - :- CometHashAggregate (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (19) - : : : : +- CometBroadcastHashJoin (18) - : : : : :- CometProject (14) - : : : : : +- CometBroadcastHashJoin (13) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : +- CometBroadcastExchange (6) - : : : : : : +- CometProject (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : : : : +- CometBroadcastExchange (12) - : : : : : +- CometProject (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : : : : +- CometBroadcastExchange (17) - : : : : +- CometFilter (16) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (15) - : : : +- CometBroadcastExchange (23) - : : : +- CometProject (22) - : : : +- CometFilter (21) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - : : +- CometBroadcastExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) - :- CometHashAggregate (66) - : +- CometExchange (65) - : +- CometHashAggregate (64) - : +- CometProject (63) - : +- CometBroadcastHashJoin (62) - : :- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (57) - : : : +- CometBroadcastHashJoin (56) - : : : :- CometProject (51) - : : : : +- CometBroadcastHashJoin (50) - : : : : :- CometProject (48) - : : : : : +- CometBroadcastHashJoin (47) - : : : : : :- CometProject (45) - : : : : : : +- CometBroadcastHashJoin (44) - : : : : : : :- CometFilter (42) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (41) - : : : : : : +- ReusedExchange (43) - : : : : : +- ReusedExchange (46) - : : : : +- ReusedExchange (49) - : : : +- CometBroadcastExchange (55) - : : : +- CometProject (54) - : : : +- CometFilter (53) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (52) - : : +- ReusedExchange (58) - : +- ReusedExchange (61) - :- CometHashAggregate (92) - : +- CometExchange (91) - : +- CometHashAggregate (90) - : +- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometProject (86) - : : +- CometBroadcastHashJoin (85) - : : :- CometProject (83) - : : : +- CometBroadcastHashJoin (82) - : : : :- CometProject (77) - : : : : +- CometBroadcastHashJoin (76) - : : : : :- CometProject (74) - : : : : : +- CometBroadcastHashJoin (73) - : : : : : :- CometProject (71) - : : : : : : +- CometBroadcastHashJoin (70) - : : : : : : :- CometFilter (68) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) - : : : : : : +- ReusedExchange (69) - : : : : : +- ReusedExchange (72) - : : : : +- ReusedExchange (75) - : : : +- CometBroadcastExchange (81) - : : : +- CometProject (80) - : : : +- CometFilter (79) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (78) - : : +- ReusedExchange (84) - : +- ReusedExchange (87) - :- CometHashAggregate (118) - : +- CometExchange (117) - : +- CometHashAggregate (116) - : +- CometProject (115) - : +- CometBroadcastHashJoin (114) - : :- CometProject (112) - : : +- CometBroadcastHashJoin (111) - : : :- CometProject (109) - : : : +- CometBroadcastHashJoin (108) - : : : :- CometProject (103) - : : : : +- CometBroadcastHashJoin (102) - : : : : :- CometProject (100) - : : : : : +- CometBroadcastHashJoin (99) - : : : : : :- CometProject (97) - : : : : : : +- CometBroadcastHashJoin (96) - : : : : : : :- CometFilter (94) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (93) - : : : : : : +- ReusedExchange (95) - : : : : : +- ReusedExchange (98) - : : : : +- ReusedExchange (101) - : : : +- CometBroadcastExchange (107) - : : : +- CometProject (106) - : : : +- CometFilter (105) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (104) - : : +- ReusedExchange (110) - : +- ReusedExchange (113) - +- CometHashAggregate (143) - +- CometExchange (142) - +- CometHashAggregate (141) - +- CometProject (140) - +- CometBroadcastHashJoin (139) - :- CometProject (135) - : +- CometBroadcastHashJoin (134) - : :- CometProject (132) - : : +- CometBroadcastHashJoin (131) - : : :- CometProject (129) - : : : +- CometBroadcastHashJoin (128) - : : : :- CometProject (126) - : : : : +- CometBroadcastHashJoin (125) - : : : : :- CometProject (123) - : : : : : +- CometBroadcastHashJoin (122) - : : : : : :- CometFilter (120) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (119) - : : : : : +- ReusedExchange (121) - : : : : +- ReusedExchange (124) - : : : +- ReusedExchange (127) - : : +- ReusedExchange (130) - : +- ReusedExchange (133) - +- CometBroadcastExchange (138) - +- CometFilter (137) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (136) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#12, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#11)) - -(5) CometProject -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] - -(6) CometBroadcastExchange -Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14] - -(7) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Right output [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight - -(8) CometProject -Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [In(c_birth_month, [1,10,12,4,5,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Condition : (((c_birth_month#18 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) - -(11) CometProject -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(12) CometBroadcastExchange -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(13) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight - -(14) CometProject -Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [cd_demo_sk#20] -Condition : isnotnull(cd_demo_sk#20) - -(17) CometBroadcastExchange -Input [1]: [cd_demo_sk#20] -Arguments: [cd_demo_sk#20] - -(18) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Right output [1]: [cd_demo_sk#20] -Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight - -(19) CometProject -Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) - -(22) CometProject -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) AS ca_state#25, ca_country#24] - -(23) CometBroadcastExchange -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(24) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -Right output [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight - -(25) CometProject -Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#26)) - -(28) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(29) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] - -(30) CometBroadcastHashJoin -Left output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [1]: [d_date_sk#26] -Arguments: [cs_sold_date_sk#9], [d_date_sk#26], Inner, BuildRight - -(31) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#28, i_item_id#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [i_item_sk#28, i_item_id#29] -Condition : isnotnull(i_item_sk#28) - -(34) CometProject -Input [2]: [i_item_sk#28, i_item_id#29] -Arguments: [i_item_sk#28, i_item_id#30], [i_item_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#29, 16, true, false, true) AS i_item_id#30] - -(35) CometBroadcastExchange -Input [2]: [i_item_sk#28, i_item_id#30] -Arguments: [i_item_sk#28, i_item_id#30] - -(36) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [2]: [i_item_sk#28, i_item_id#30] -Arguments: [cs_item_sk#3], [i_item_sk#28], Inner, BuildRight - -(37) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#28, i_item_id#30] -Arguments: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41], [i_item_id#30 AS i_item_id#31, ca_country#24 AS ca_country#32, ca_state#25 AS ca_state#33, ca_county#22 AS ca_county#34, cast(cs_quantity#4 as decimal(12,2)) AS agg1#35, cast(cs_list_price#5 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#39, cast(c_birth_year#19 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] - -(38) CometHashAggregate -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [4]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34] -Functions [7]: [partial_avg(agg1#35), partial_avg(agg2#36), partial_avg(agg3#37), partial_avg(agg4#38), partial_avg(agg5#39), partial_avg(agg6#40), partial_avg(agg7#41)] - -(39) CometExchange -Input [18]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] -Arguments: hashpartitioning(i_item_id#31, ca_country#32, ca_state#33, ca_county#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(40) CometHashAggregate -Input [18]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] -Keys [4]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34] -Functions [7]: [avg(agg1#35), avg(agg2#36), avg(agg3#37), avg(agg4#38), avg(agg5#39), avg(agg6#40), avg(agg7#41)] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#64), dynamicpruningexpression(cs_sold_date_sk#64 IN dynamicpruning#65)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(42) CometFilter -Input [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] -Condition : ((isnotnull(cs_bill_cdemo_sk#57) AND isnotnull(cs_bill_customer_sk#56)) AND isnotnull(cs_item_sk#58)) - -(43) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#66, cd_dep_count#67] - -(44) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] -Right output [2]: [cd_demo_sk#66, cd_dep_count#67] -Arguments: [cs_bill_cdemo_sk#57], [cd_demo_sk#66], Inner, BuildRight - -(45) CometProject -Input [11]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_demo_sk#66, cd_dep_count#67] -Arguments: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67], [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67] - -(46) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] - -(47) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67] -Right output [4]: [c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Arguments: [cs_bill_customer_sk#56], [c_customer_sk#68], Inner, BuildRight - -(48) CometProject -Input [13]: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] - -(49) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#72] - -(50) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Right output [1]: [cd_demo_sk#72] -Arguments: [c_current_cdemo_sk#69], [cd_demo_sk#72], Inner, BuildRight - -(51) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71, cd_demo_sk#72] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71] - -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#73, ca_state#74, ca_country#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [ca_address_sk#73, ca_state#74, ca_country#75] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#74, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#73)) - -(54) CometProject -Input [3]: [ca_address_sk#73, ca_state#74, ca_country#75] -Arguments: [ca_address_sk#73, ca_state#25, ca_country#75], [ca_address_sk#73, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#74, 2, true, false, true) AS ca_state#25, ca_country#75] - -(55) CometBroadcastExchange -Input [3]: [ca_address_sk#73, ca_state#25, ca_country#75] -Arguments: [ca_address_sk#73, ca_state#25, ca_country#75] - -(56) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71] -Right output [3]: [ca_address_sk#73, ca_state#25, ca_country#75] -Arguments: [c_current_addr_sk#70], [ca_address_sk#73], Inner, BuildRight - -(57) CometProject -Input [13]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71, ca_address_sk#73, ca_state#25, ca_country#75] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] - -(58) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#76] - -(59) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] -Right output [1]: [d_date_sk#76] -Arguments: [cs_sold_date_sk#64], [d_date_sk#76], Inner, BuildRight - -(60) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75, d_date_sk#76] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] - -(61) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#77, i_item_id#30] - -(62) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] -Right output [2]: [i_item_sk#77, i_item_id#30] -Arguments: [cs_item_sk#58], [i_item_sk#77], Inner, BuildRight - -(63) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75, i_item_sk#77, i_item_id#30] -Arguments: [i_item_id#30, ca_country#75, ca_state#25, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84], [i_item_id#30, ca_country#75, ca_state#25, cast(cs_quantity#59 as decimal(12,2)) AS agg1#78, cast(cs_list_price#60 as decimal(12,2)) AS agg2#79, cast(cs_coupon_amt#62 as decimal(12,2)) AS agg3#80, cast(cs_sales_price#61 as decimal(12,2)) AS agg4#81, cast(cs_net_profit#63 as decimal(12,2)) AS agg5#82, cast(c_birth_year#71 as decimal(12,2)) AS agg6#83, cast(cd_dep_count#67 as decimal(12,2)) AS agg7#84] - -(64) CometHashAggregate -Input [10]: [i_item_id#30, ca_country#75, ca_state#25, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84] -Keys [3]: [i_item_id#30, ca_country#75, ca_state#25] -Functions [7]: [partial_avg(agg1#78), partial_avg(agg2#79), partial_avg(agg3#80), partial_avg(agg4#81), partial_avg(agg5#82), partial_avg(agg6#83), partial_avg(agg7#84)] - -(65) CometExchange -Input [17]: [i_item_id#30, ca_country#75, ca_state#25, sum#85, count#86, sum#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] -Arguments: hashpartitioning(i_item_id#30, ca_country#75, ca_state#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(66) CometHashAggregate -Input [17]: [i_item_id#30, ca_country#75, ca_state#25, sum#85, count#86, sum#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] -Keys [3]: [i_item_id#30, ca_country#75, ca_state#25] -Functions [7]: [avg(agg1#78), avg(agg2#79), avg(agg3#80), avg(agg4#81), avg(agg5#82), avg(agg6#83), avg(agg7#84)] - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#107), dynamicpruningexpression(cs_sold_date_sk#107 IN dynamicpruning#108)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(68) CometFilter -Input [9]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107] -Condition : ((isnotnull(cs_bill_cdemo_sk#100) AND isnotnull(cs_bill_customer_sk#99)) AND isnotnull(cs_item_sk#101)) - -(69) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#109, cd_dep_count#110] - -(70) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107] -Right output [2]: [cd_demo_sk#109, cd_dep_count#110] -Arguments: [cs_bill_cdemo_sk#100], [cd_demo_sk#109], Inner, BuildRight - -(71) CometProject -Input [11]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_demo_sk#109, cd_dep_count#110] -Arguments: [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110], [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110] - -(72) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#111, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] - -(73) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110] -Right output [4]: [c_customer_sk#111, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] -Arguments: [cs_bill_customer_sk#99], [c_customer_sk#111], Inner, BuildRight - -(74) CometProject -Input [13]: [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_customer_sk#111, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] -Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] - -(75) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#115] - -(76) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] -Right output [1]: [cd_demo_sk#115] -Arguments: [c_current_cdemo_sk#112], [cd_demo_sk#115], Inner, BuildRight - -(77) CometProject -Input [12]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114, cd_demo_sk#115] -Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114] - -(78) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#116, ca_state#117, ca_country#118] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(79) CometFilter -Input [3]: [ca_address_sk#116, ca_state#117, ca_country#118] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#117, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#116)) - -(80) CometProject -Input [3]: [ca_address_sk#116, ca_state#117, ca_country#118] -Arguments: [ca_address_sk#116, ca_country#118], [ca_address_sk#116, ca_country#118] - -(81) CometBroadcastExchange -Input [2]: [ca_address_sk#116, ca_country#118] -Arguments: [ca_address_sk#116, ca_country#118] - -(82) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114] -Right output [2]: [ca_address_sk#116, ca_country#118] -Arguments: [c_current_addr_sk#113], [ca_address_sk#116], Inner, BuildRight - -(83) CometProject -Input [12]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114, ca_address_sk#116, ca_country#118] -Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118] - -(84) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#119] - -(85) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118] -Right output [1]: [d_date_sk#119] -Arguments: [cs_sold_date_sk#107], [d_date_sk#119], Inner, BuildRight - -(86) CometProject -Input [11]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118, d_date_sk#119] -Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118] - -(87) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#120, i_item_id#30] - -(88) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118] -Right output [2]: [i_item_sk#120, i_item_id#30] -Arguments: [cs_item_sk#101], [i_item_sk#120], Inner, BuildRight - -(89) CometProject -Input [11]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118, i_item_sk#120, i_item_id#30] -Arguments: [i_item_id#30, ca_country#118, agg1#121, agg2#122, agg3#123, agg4#124, agg5#125, agg6#126, agg7#127], [i_item_id#30, ca_country#118, cast(cs_quantity#102 as decimal(12,2)) AS agg1#121, cast(cs_list_price#103 as decimal(12,2)) AS agg2#122, cast(cs_coupon_amt#105 as decimal(12,2)) AS agg3#123, cast(cs_sales_price#104 as decimal(12,2)) AS agg4#124, cast(cs_net_profit#106 as decimal(12,2)) AS agg5#125, cast(c_birth_year#114 as decimal(12,2)) AS agg6#126, cast(cd_dep_count#110 as decimal(12,2)) AS agg7#127] - -(90) CometHashAggregate -Input [9]: [i_item_id#30, ca_country#118, agg1#121, agg2#122, agg3#123, agg4#124, agg5#125, agg6#126, agg7#127] -Keys [2]: [i_item_id#30, ca_country#118] -Functions [7]: [partial_avg(agg1#121), partial_avg(agg2#122), partial_avg(agg3#123), partial_avg(agg4#124), partial_avg(agg5#125), partial_avg(agg6#126), partial_avg(agg7#127)] - -(91) CometExchange -Input [16]: [i_item_id#30, ca_country#118, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139, sum#140, count#141] -Arguments: hashpartitioning(i_item_id#30, ca_country#118, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(92) CometHashAggregate -Input [16]: [i_item_id#30, ca_country#118, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139, sum#140, count#141] -Keys [2]: [i_item_id#30, ca_country#118] -Functions [7]: [avg(agg1#121), avg(agg2#122), avg(agg3#123), avg(agg4#124), avg(agg5#125), avg(agg6#126), avg(agg7#127)] - -(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#150), dynamicpruningexpression(cs_sold_date_sk#150 IN dynamicpruning#151)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(94) CometFilter -Input [9]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150] -Condition : ((isnotnull(cs_bill_cdemo_sk#143) AND isnotnull(cs_bill_customer_sk#142)) AND isnotnull(cs_item_sk#144)) - -(95) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#152, cd_dep_count#153] - -(96) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150] -Right output [2]: [cd_demo_sk#152, cd_dep_count#153] -Arguments: [cs_bill_cdemo_sk#143], [cd_demo_sk#152], Inner, BuildRight - -(97) CometProject -Input [11]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_demo_sk#152, cd_dep_count#153] -Arguments: [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153], [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153] - -(98) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#154, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] - -(99) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153] -Right output [4]: [c_customer_sk#154, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] -Arguments: [cs_bill_customer_sk#142], [c_customer_sk#154], Inner, BuildRight - -(100) CometProject -Input [13]: [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_customer_sk#154, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] -Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] - -(101) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#158] - -(102) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] -Right output [1]: [cd_demo_sk#158] -Arguments: [c_current_cdemo_sk#155], [cd_demo_sk#158], Inner, BuildRight - -(103) CometProject -Input [12]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157, cd_demo_sk#158] -Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#159, ca_state#160] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [ca_address_sk#159, ca_state#160] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#160, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#159)) - -(106) CometProject -Input [2]: [ca_address_sk#159, ca_state#160] -Arguments: [ca_address_sk#159], [ca_address_sk#159] - -(107) CometBroadcastExchange -Input [1]: [ca_address_sk#159] -Arguments: [ca_address_sk#159] - -(108) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157] -Right output [1]: [ca_address_sk#159] -Arguments: [c_current_addr_sk#156], [ca_address_sk#159], Inner, BuildRight - -(109) CometProject -Input [11]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157, ca_address_sk#159] -Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157] - -(110) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#161] - -(111) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157] -Right output [1]: [d_date_sk#161] -Arguments: [cs_sold_date_sk#150], [d_date_sk#161], Inner, BuildRight - -(112) CometProject -Input [10]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157, d_date_sk#161] -Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157] - -(113) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#162, i_item_id#30] - -(114) CometBroadcastHashJoin -Left output [8]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157] -Right output [2]: [i_item_sk#162, i_item_id#30] -Arguments: [cs_item_sk#144], [i_item_sk#162], Inner, BuildRight - -(115) CometProject -Input [10]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157, i_item_sk#162, i_item_id#30] -Arguments: [i_item_id#30, agg1#163, agg2#164, agg3#165, agg4#166, agg5#167, agg6#168, agg7#169], [i_item_id#30, cast(cs_quantity#145 as decimal(12,2)) AS agg1#163, cast(cs_list_price#146 as decimal(12,2)) AS agg2#164, cast(cs_coupon_amt#148 as decimal(12,2)) AS agg3#165, cast(cs_sales_price#147 as decimal(12,2)) AS agg4#166, cast(cs_net_profit#149 as decimal(12,2)) AS agg5#167, cast(c_birth_year#157 as decimal(12,2)) AS agg6#168, cast(cd_dep_count#153 as decimal(12,2)) AS agg7#169] - -(116) CometHashAggregate -Input [8]: [i_item_id#30, agg1#163, agg2#164, agg3#165, agg4#166, agg5#167, agg6#168, agg7#169] -Keys [1]: [i_item_id#30] -Functions [7]: [partial_avg(agg1#163), partial_avg(agg2#164), partial_avg(agg3#165), partial_avg(agg4#166), partial_avg(agg5#167), partial_avg(agg6#168), partial_avg(agg7#169)] - -(117) CometExchange -Input [15]: [i_item_id#30, sum#170, count#171, sum#172, count#173, sum#174, count#175, sum#176, count#177, sum#178, count#179, sum#180, count#181, sum#182, count#183] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(118) CometHashAggregate -Input [15]: [i_item_id#30, sum#170, count#171, sum#172, count#173, sum#174, count#175, sum#176, count#177, sum#178, count#179, sum#180, count#181, sum#182, count#183] -Keys [1]: [i_item_id#30] -Functions [7]: [avg(agg1#163), avg(agg2#164), avg(agg3#165), avg(agg4#166), avg(agg5#167), avg(agg6#168), avg(agg7#169)] - -(119) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#192), dynamicpruningexpression(cs_sold_date_sk#192 IN dynamicpruning#193)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(120) CometFilter -Input [9]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192] -Condition : ((isnotnull(cs_bill_cdemo_sk#185) AND isnotnull(cs_bill_customer_sk#184)) AND isnotnull(cs_item_sk#186)) - -(121) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#194, cd_dep_count#195] - -(122) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192] -Right output [2]: [cd_demo_sk#194, cd_dep_count#195] -Arguments: [cs_bill_cdemo_sk#185], [cd_demo_sk#194], Inner, BuildRight - -(123) CometProject -Input [11]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_demo_sk#194, cd_dep_count#195] -Arguments: [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195], [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195] - -(124) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#196, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] - -(125) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195] -Right output [4]: [c_customer_sk#196, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] -Arguments: [cs_bill_customer_sk#184], [c_customer_sk#196], Inner, BuildRight - -(126) CometProject -Input [13]: [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_customer_sk#196, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] -Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] - -(127) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#200] - -(128) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] -Right output [1]: [cd_demo_sk#200] -Arguments: [c_current_cdemo_sk#197], [cd_demo_sk#200], Inner, BuildRight - -(129) CometProject -Input [12]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199, cd_demo_sk#200] -Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199] - -(130) ReusedExchange [Reuses operator id: 107] -Output [1]: [ca_address_sk#201] - -(131) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199] -Right output [1]: [ca_address_sk#201] -Arguments: [c_current_addr_sk#198], [ca_address_sk#201], Inner, BuildRight - -(132) CometProject -Input [11]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199, ca_address_sk#201] -Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199] - -(133) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#202] - -(134) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199] -Right output [1]: [d_date_sk#202] -Arguments: [cs_sold_date_sk#192], [d_date_sk#202], Inner, BuildRight - -(135) CometProject -Input [10]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199, d_date_sk#202] -Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199] - -(136) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#203] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(137) CometFilter -Input [1]: [i_item_sk#203] -Condition : isnotnull(i_item_sk#203) - -(138) CometBroadcastExchange -Input [1]: [i_item_sk#203] -Arguments: [i_item_sk#203] - -(139) CometBroadcastHashJoin -Left output [8]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199] -Right output [1]: [i_item_sk#203] -Arguments: [cs_item_sk#186], [i_item_sk#203], Inner, BuildRight - -(140) CometProject -Input [9]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199, i_item_sk#203] -Arguments: [agg1#204, agg2#205, agg3#206, agg4#207, agg5#208, agg6#209, agg7#210], [cast(cs_quantity#187 as decimal(12,2)) AS agg1#204, cast(cs_list_price#188 as decimal(12,2)) AS agg2#205, cast(cs_coupon_amt#190 as decimal(12,2)) AS agg3#206, cast(cs_sales_price#189 as decimal(12,2)) AS agg4#207, cast(cs_net_profit#191 as decimal(12,2)) AS agg5#208, cast(c_birth_year#199 as decimal(12,2)) AS agg6#209, cast(cd_dep_count#195 as decimal(12,2)) AS agg7#210] - -(141) CometHashAggregate -Input [7]: [agg1#204, agg2#205, agg3#206, agg4#207, agg5#208, agg6#209, agg7#210] -Keys: [] -Functions [7]: [partial_avg(agg1#204), partial_avg(agg2#205), partial_avg(agg3#206), partial_avg(agg4#207), partial_avg(agg5#208), partial_avg(agg6#209), partial_avg(agg7#210)] - -(142) CometExchange -Input [14]: [sum#211, count#212, sum#213, count#214, sum#215, count#216, sum#217, count#218, sum#219, count#220, sum#221, count#222, sum#223, count#224] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(143) CometHashAggregate -Input [14]: [sum#211, count#212, sum#213, count#214, sum#215, count#216, sum#217, count#218, sum#219, count#220, sum#221, count#222, sum#223, count#224] -Keys: [] -Functions [7]: [avg(agg1#204), avg(agg2#205), avg(agg3#206), avg(agg4#207), avg(agg5#208), avg(agg6#209), avg(agg7#210)] - -(144) CometUnion -Child 0 Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] -Child 1 Input [11]: [i_item_id#30, ca_country#75, ca_state#25, county#232, agg1#233, agg2#234, agg3#235, agg4#236, agg5#237, agg6#238, agg7#239] -Child 2 Input [11]: [i_item_id#30, ca_country#118, ca_state#240, county#241, agg1#242, agg2#243, agg3#244, agg4#245, agg5#246, agg6#247, agg7#248] -Child 3 Input [11]: [i_item_id#30, ca_country#249, ca_state#250, county#251, agg1#252, agg2#253, agg3#254, agg4#255, agg5#256, agg6#257, agg7#258] -Child 4 Input [11]: [i_item_id#259, ca_country#260, ca_state#261, county#262, agg1#263, agg2#264, agg3#265, agg4#266, agg5#267, agg6#268, agg7#269] - -(145) CometTakeOrderedAndProject -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#32 ASC NULLS FIRST,ca_state#33 ASC NULLS FIRST,ca_county#34 ASC NULLS FIRST,i_item_id#31 ASC NULLS FIRST], output=[i_item_id#31,ca_country#32,ca_state#33,ca_county#34,agg1#225,agg2#226,agg3#227,agg4#228,agg5#229,agg6#230,agg7#231]), [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231], 100, 0, [ca_country#32 ASC NULLS FIRST, ca_state#33 ASC NULLS FIRST, ca_county#34 ASC NULLS FIRST, i_item_id#31 ASC NULLS FIRST], [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] - -(146) CometColumnarToRow [codegen id : 1] -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (151) -+- * CometColumnarToRow (150) - +- CometProject (149) - +- CometFilter (148) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (147) - - -(147) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(148) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#26)) - -(149) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(150) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(151) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#64 IN dynamicpruning#10 - -Subquery:3 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#107 IN dynamicpruning#10 - -Subquery:4 Hosting operator id = 93 Hosting Expression = cs_sold_date_sk#150 IN dynamicpruning#10 - -Subquery:5 Hosting operator id = 119 Hosting Expression = cs_sold_date_sk#192 IN dynamicpruning#10 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 1572a2a240..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,160 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometUnion [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country,ca_state,ca_county] #1 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometProject [i_item_id,ca_country,ca_state,ca_county,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,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_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 [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 [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,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_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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - 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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange [cd_demo_sk] #5 - CometFilter [cd_demo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #8 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country,ca_state] #9 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 [ca_address_sk,ca_state,ca_country] #10 - CometProject [ca_state] [ca_address_sk,ca_state,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country] #11 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 [ca_address_sk,ca_country] #12 - CometProject [ca_address_sk,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id] #13 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 [ca_address_sk] #14 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - ReusedExchange [d_date_sk] #7 - ReusedExchange [i_item_sk,i_item_id] #8 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange #15 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 - ReusedExchange [ca_address_sk] #14 - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [i_item_sk] #16 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt deleted file mode 100644 index 72e9bd1b4a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/explain.txt +++ /dev/null @@ -1,846 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (146) -+- CometTakeOrderedAndProject (145) - +- CometUnion (144) - :- CometHashAggregate (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (19) - : : : : +- CometBroadcastHashJoin (18) - : : : : :- CometProject (14) - : : : : : +- CometBroadcastHashJoin (13) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : +- CometBroadcastExchange (6) - : : : : : : +- CometProject (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : : : : +- CometBroadcastExchange (12) - : : : : : +- CometProject (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : : : : +- CometBroadcastExchange (17) - : : : : +- CometFilter (16) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (15) - : : : +- CometBroadcastExchange (23) - : : : +- CometProject (22) - : : : +- CometFilter (21) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - : : +- CometBroadcastExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) - :- CometHashAggregate (66) - : +- CometExchange (65) - : +- CometHashAggregate (64) - : +- CometProject (63) - : +- CometBroadcastHashJoin (62) - : :- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (57) - : : : +- CometBroadcastHashJoin (56) - : : : :- CometProject (51) - : : : : +- CometBroadcastHashJoin (50) - : : : : :- CometProject (48) - : : : : : +- CometBroadcastHashJoin (47) - : : : : : :- CometProject (45) - : : : : : : +- CometBroadcastHashJoin (44) - : : : : : : :- CometFilter (42) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (41) - : : : : : : +- ReusedExchange (43) - : : : : : +- ReusedExchange (46) - : : : : +- ReusedExchange (49) - : : : +- CometBroadcastExchange (55) - : : : +- CometProject (54) - : : : +- CometFilter (53) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (52) - : : +- ReusedExchange (58) - : +- ReusedExchange (61) - :- CometHashAggregate (92) - : +- CometExchange (91) - : +- CometHashAggregate (90) - : +- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometProject (86) - : : +- CometBroadcastHashJoin (85) - : : :- CometProject (83) - : : : +- CometBroadcastHashJoin (82) - : : : :- CometProject (77) - : : : : +- CometBroadcastHashJoin (76) - : : : : :- CometProject (74) - : : : : : +- CometBroadcastHashJoin (73) - : : : : : :- CometProject (71) - : : : : : : +- CometBroadcastHashJoin (70) - : : : : : : :- CometFilter (68) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) - : : : : : : +- ReusedExchange (69) - : : : : : +- ReusedExchange (72) - : : : : +- ReusedExchange (75) - : : : +- CometBroadcastExchange (81) - : : : +- CometProject (80) - : : : +- CometFilter (79) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (78) - : : +- ReusedExchange (84) - : +- ReusedExchange (87) - :- CometHashAggregate (118) - : +- CometExchange (117) - : +- CometHashAggregate (116) - : +- CometProject (115) - : +- CometBroadcastHashJoin (114) - : :- CometProject (112) - : : +- CometBroadcastHashJoin (111) - : : :- CometProject (109) - : : : +- CometBroadcastHashJoin (108) - : : : :- CometProject (103) - : : : : +- CometBroadcastHashJoin (102) - : : : : :- CometProject (100) - : : : : : +- CometBroadcastHashJoin (99) - : : : : : :- CometProject (97) - : : : : : : +- CometBroadcastHashJoin (96) - : : : : : : :- CometFilter (94) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (93) - : : : : : : +- ReusedExchange (95) - : : : : : +- ReusedExchange (98) - : : : : +- ReusedExchange (101) - : : : +- CometBroadcastExchange (107) - : : : +- CometProject (106) - : : : +- CometFilter (105) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (104) - : : +- ReusedExchange (110) - : +- ReusedExchange (113) - +- CometHashAggregate (143) - +- CometExchange (142) - +- CometHashAggregate (141) - +- CometProject (140) - +- CometBroadcastHashJoin (139) - :- CometProject (135) - : +- CometBroadcastHashJoin (134) - : :- CometProject (132) - : : +- CometBroadcastHashJoin (131) - : : :- CometProject (129) - : : : +- CometBroadcastHashJoin (128) - : : : :- CometProject (126) - : : : : +- CometBroadcastHashJoin (125) - : : : : :- CometProject (123) - : : : : : +- CometBroadcastHashJoin (122) - : : : : : :- CometFilter (120) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (119) - : : : : : +- ReusedExchange (121) - : : : : +- ReusedExchange (124) - : : : +- ReusedExchange (127) - : : +- ReusedExchange (130) - : +- ReusedExchange (133) - +- CometBroadcastExchange (138) - +- CometFilter (137) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (136) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#12, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#11)) - -(5) CometProject -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] - -(6) CometBroadcastExchange -Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14] - -(7) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Right output [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight - -(8) CometProject -Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [In(c_birth_month, [1,10,12,4,5,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Condition : (((c_birth_month#18 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) - -(11) CometProject -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(12) CometBroadcastExchange -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(13) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight - -(14) CometProject -Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [cd_demo_sk#20] -Condition : isnotnull(cd_demo_sk#20) - -(17) CometBroadcastExchange -Input [1]: [cd_demo_sk#20] -Arguments: [cd_demo_sk#20] - -(18) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Right output [1]: [cd_demo_sk#20] -Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight - -(19) CometProject -Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) - -(22) CometProject -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) AS ca_state#25, ca_country#24] - -(23) CometBroadcastExchange -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(24) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -Right output [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight - -(25) CometProject -Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#26)) - -(28) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(29) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] - -(30) CometBroadcastHashJoin -Left output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [1]: [d_date_sk#26] -Arguments: [cs_sold_date_sk#9], [d_date_sk#26], Inner, BuildRight - -(31) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#28, i_item_id#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [i_item_sk#28, i_item_id#29] -Condition : isnotnull(i_item_sk#28) - -(34) CometProject -Input [2]: [i_item_sk#28, i_item_id#29] -Arguments: [i_item_sk#28, i_item_id#30], [i_item_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#29, 16, true, false, true) AS i_item_id#30] - -(35) CometBroadcastExchange -Input [2]: [i_item_sk#28, i_item_id#30] -Arguments: [i_item_sk#28, i_item_id#30] - -(36) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [2]: [i_item_sk#28, i_item_id#30] -Arguments: [cs_item_sk#3], [i_item_sk#28], Inner, BuildRight - -(37) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#28, i_item_id#30] -Arguments: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41], [i_item_id#30 AS i_item_id#31, ca_country#24 AS ca_country#32, ca_state#25 AS ca_state#33, ca_county#22 AS ca_county#34, cast(cs_quantity#4 as decimal(12,2)) AS agg1#35, cast(cs_list_price#5 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#39, cast(c_birth_year#19 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] - -(38) CometHashAggregate -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [4]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34] -Functions [7]: [partial_avg(agg1#35), partial_avg(agg2#36), partial_avg(agg3#37), partial_avg(agg4#38), partial_avg(agg5#39), partial_avg(agg6#40), partial_avg(agg7#41)] - -(39) CometExchange -Input [18]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] -Arguments: hashpartitioning(i_item_id#31, ca_country#32, ca_state#33, ca_county#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(40) CometHashAggregate -Input [18]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] -Keys [4]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34] -Functions [7]: [avg(agg1#35), avg(agg2#36), avg(agg3#37), avg(agg4#38), avg(agg5#39), avg(agg6#40), avg(agg7#41)] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#64), dynamicpruningexpression(cs_sold_date_sk#64 IN dynamicpruning#65)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(42) CometFilter -Input [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] -Condition : ((isnotnull(cs_bill_cdemo_sk#57) AND isnotnull(cs_bill_customer_sk#56)) AND isnotnull(cs_item_sk#58)) - -(43) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#66, cd_dep_count#67] - -(44) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] -Right output [2]: [cd_demo_sk#66, cd_dep_count#67] -Arguments: [cs_bill_cdemo_sk#57], [cd_demo_sk#66], Inner, BuildRight - -(45) CometProject -Input [11]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_demo_sk#66, cd_dep_count#67] -Arguments: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67], [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67] - -(46) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] - -(47) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67] -Right output [4]: [c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Arguments: [cs_bill_customer_sk#56], [c_customer_sk#68], Inner, BuildRight - -(48) CometProject -Input [13]: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] - -(49) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#72] - -(50) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Right output [1]: [cd_demo_sk#72] -Arguments: [c_current_cdemo_sk#69], [cd_demo_sk#72], Inner, BuildRight - -(51) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71, cd_demo_sk#72] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71] - -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#73, ca_state#74, ca_country#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [ca_address_sk#73, ca_state#74, ca_country#75] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#74, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#73)) - -(54) CometProject -Input [3]: [ca_address_sk#73, ca_state#74, ca_country#75] -Arguments: [ca_address_sk#73, ca_state#25, ca_country#75], [ca_address_sk#73, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#74, 2, true, false, true) AS ca_state#25, ca_country#75] - -(55) CometBroadcastExchange -Input [3]: [ca_address_sk#73, ca_state#25, ca_country#75] -Arguments: [ca_address_sk#73, ca_state#25, ca_country#75] - -(56) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71] -Right output [3]: [ca_address_sk#73, ca_state#25, ca_country#75] -Arguments: [c_current_addr_sk#70], [ca_address_sk#73], Inner, BuildRight - -(57) CometProject -Input [13]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71, ca_address_sk#73, ca_state#25, ca_country#75] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] - -(58) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#76] - -(59) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] -Right output [1]: [d_date_sk#76] -Arguments: [cs_sold_date_sk#64], [d_date_sk#76], Inner, BuildRight - -(60) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75, d_date_sk#76] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] - -(61) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#77, i_item_id#30] - -(62) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] -Right output [2]: [i_item_sk#77, i_item_id#30] -Arguments: [cs_item_sk#58], [i_item_sk#77], Inner, BuildRight - -(63) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75, i_item_sk#77, i_item_id#30] -Arguments: [i_item_id#30, ca_country#75, ca_state#25, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84], [i_item_id#30, ca_country#75, ca_state#25, cast(cs_quantity#59 as decimal(12,2)) AS agg1#78, cast(cs_list_price#60 as decimal(12,2)) AS agg2#79, cast(cs_coupon_amt#62 as decimal(12,2)) AS agg3#80, cast(cs_sales_price#61 as decimal(12,2)) AS agg4#81, cast(cs_net_profit#63 as decimal(12,2)) AS agg5#82, cast(c_birth_year#71 as decimal(12,2)) AS agg6#83, cast(cd_dep_count#67 as decimal(12,2)) AS agg7#84] - -(64) CometHashAggregate -Input [10]: [i_item_id#30, ca_country#75, ca_state#25, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84] -Keys [3]: [i_item_id#30, ca_country#75, ca_state#25] -Functions [7]: [partial_avg(agg1#78), partial_avg(agg2#79), partial_avg(agg3#80), partial_avg(agg4#81), partial_avg(agg5#82), partial_avg(agg6#83), partial_avg(agg7#84)] - -(65) CometExchange -Input [17]: [i_item_id#30, ca_country#75, ca_state#25, sum#85, count#86, sum#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] -Arguments: hashpartitioning(i_item_id#30, ca_country#75, ca_state#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(66) CometHashAggregate -Input [17]: [i_item_id#30, ca_country#75, ca_state#25, sum#85, count#86, sum#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] -Keys [3]: [i_item_id#30, ca_country#75, ca_state#25] -Functions [7]: [avg(agg1#78), avg(agg2#79), avg(agg3#80), avg(agg4#81), avg(agg5#82), avg(agg6#83), avg(agg7#84)] - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#107), dynamicpruningexpression(cs_sold_date_sk#107 IN dynamicpruning#108)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(68) CometFilter -Input [9]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107] -Condition : ((isnotnull(cs_bill_cdemo_sk#100) AND isnotnull(cs_bill_customer_sk#99)) AND isnotnull(cs_item_sk#101)) - -(69) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#109, cd_dep_count#110] - -(70) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107] -Right output [2]: [cd_demo_sk#109, cd_dep_count#110] -Arguments: [cs_bill_cdemo_sk#100], [cd_demo_sk#109], Inner, BuildRight - -(71) CometProject -Input [11]: [cs_bill_customer_sk#99, cs_bill_cdemo_sk#100, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_demo_sk#109, cd_dep_count#110] -Arguments: [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110], [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110] - -(72) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#111, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] - -(73) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110] -Right output [4]: [c_customer_sk#111, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] -Arguments: [cs_bill_customer_sk#99], [c_customer_sk#111], Inner, BuildRight - -(74) CometProject -Input [13]: [cs_bill_customer_sk#99, cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_customer_sk#111, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] -Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] - -(75) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#115] - -(76) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114] -Right output [1]: [cd_demo_sk#115] -Arguments: [c_current_cdemo_sk#112], [cd_demo_sk#115], Inner, BuildRight - -(77) CometProject -Input [12]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_cdemo_sk#112, c_current_addr_sk#113, c_birth_year#114, cd_demo_sk#115] -Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114] - -(78) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#116, ca_state#117, ca_country#118] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(79) CometFilter -Input [3]: [ca_address_sk#116, ca_state#117, ca_country#118] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#117, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#116)) - -(80) CometProject -Input [3]: [ca_address_sk#116, ca_state#117, ca_country#118] -Arguments: [ca_address_sk#116, ca_country#118], [ca_address_sk#116, ca_country#118] - -(81) CometBroadcastExchange -Input [2]: [ca_address_sk#116, ca_country#118] -Arguments: [ca_address_sk#116, ca_country#118] - -(82) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114] -Right output [2]: [ca_address_sk#116, ca_country#118] -Arguments: [c_current_addr_sk#113], [ca_address_sk#116], Inner, BuildRight - -(83) CometProject -Input [12]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_current_addr_sk#113, c_birth_year#114, ca_address_sk#116, ca_country#118] -Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118] - -(84) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#119] - -(85) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118] -Right output [1]: [d_date_sk#119] -Arguments: [cs_sold_date_sk#107], [d_date_sk#119], Inner, BuildRight - -(86) CometProject -Input [11]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cs_sold_date_sk#107, cd_dep_count#110, c_birth_year#114, ca_country#118, d_date_sk#119] -Arguments: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118], [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118] - -(87) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#120, i_item_id#30] - -(88) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118] -Right output [2]: [i_item_sk#120, i_item_id#30] -Arguments: [cs_item_sk#101], [i_item_sk#120], Inner, BuildRight - -(89) CometProject -Input [11]: [cs_item_sk#101, cs_quantity#102, cs_list_price#103, cs_sales_price#104, cs_coupon_amt#105, cs_net_profit#106, cd_dep_count#110, c_birth_year#114, ca_country#118, i_item_sk#120, i_item_id#30] -Arguments: [i_item_id#30, ca_country#118, agg1#121, agg2#122, agg3#123, agg4#124, agg5#125, agg6#126, agg7#127], [i_item_id#30, ca_country#118, cast(cs_quantity#102 as decimal(12,2)) AS agg1#121, cast(cs_list_price#103 as decimal(12,2)) AS agg2#122, cast(cs_coupon_amt#105 as decimal(12,2)) AS agg3#123, cast(cs_sales_price#104 as decimal(12,2)) AS agg4#124, cast(cs_net_profit#106 as decimal(12,2)) AS agg5#125, cast(c_birth_year#114 as decimal(12,2)) AS agg6#126, cast(cd_dep_count#110 as decimal(12,2)) AS agg7#127] - -(90) CometHashAggregate -Input [9]: [i_item_id#30, ca_country#118, agg1#121, agg2#122, agg3#123, agg4#124, agg5#125, agg6#126, agg7#127] -Keys [2]: [i_item_id#30, ca_country#118] -Functions [7]: [partial_avg(agg1#121), partial_avg(agg2#122), partial_avg(agg3#123), partial_avg(agg4#124), partial_avg(agg5#125), partial_avg(agg6#126), partial_avg(agg7#127)] - -(91) CometExchange -Input [16]: [i_item_id#30, ca_country#118, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139, sum#140, count#141] -Arguments: hashpartitioning(i_item_id#30, ca_country#118, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(92) CometHashAggregate -Input [16]: [i_item_id#30, ca_country#118, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135, sum#136, count#137, sum#138, count#139, sum#140, count#141] -Keys [2]: [i_item_id#30, ca_country#118] -Functions [7]: [avg(agg1#121), avg(agg2#122), avg(agg3#123), avg(agg4#124), avg(agg5#125), avg(agg6#126), avg(agg7#127)] - -(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#150), dynamicpruningexpression(cs_sold_date_sk#150 IN dynamicpruning#151)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(94) CometFilter -Input [9]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150] -Condition : ((isnotnull(cs_bill_cdemo_sk#143) AND isnotnull(cs_bill_customer_sk#142)) AND isnotnull(cs_item_sk#144)) - -(95) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#152, cd_dep_count#153] - -(96) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150] -Right output [2]: [cd_demo_sk#152, cd_dep_count#153] -Arguments: [cs_bill_cdemo_sk#143], [cd_demo_sk#152], Inner, BuildRight - -(97) CometProject -Input [11]: [cs_bill_customer_sk#142, cs_bill_cdemo_sk#143, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_demo_sk#152, cd_dep_count#153] -Arguments: [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153], [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153] - -(98) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#154, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] - -(99) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153] -Right output [4]: [c_customer_sk#154, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] -Arguments: [cs_bill_customer_sk#142], [c_customer_sk#154], Inner, BuildRight - -(100) CometProject -Input [13]: [cs_bill_customer_sk#142, cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_customer_sk#154, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] -Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] - -(101) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#158] - -(102) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157] -Right output [1]: [cd_demo_sk#158] -Arguments: [c_current_cdemo_sk#155], [cd_demo_sk#158], Inner, BuildRight - -(103) CometProject -Input [12]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_cdemo_sk#155, c_current_addr_sk#156, c_birth_year#157, cd_demo_sk#158] -Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#159, ca_state#160] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [ca_address_sk#159, ca_state#160] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#160, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#159)) - -(106) CometProject -Input [2]: [ca_address_sk#159, ca_state#160] -Arguments: [ca_address_sk#159], [ca_address_sk#159] - -(107) CometBroadcastExchange -Input [1]: [ca_address_sk#159] -Arguments: [ca_address_sk#159] - -(108) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157] -Right output [1]: [ca_address_sk#159] -Arguments: [c_current_addr_sk#156], [ca_address_sk#159], Inner, BuildRight - -(109) CometProject -Input [11]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_current_addr_sk#156, c_birth_year#157, ca_address_sk#159] -Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157] - -(110) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#161] - -(111) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157] -Right output [1]: [d_date_sk#161] -Arguments: [cs_sold_date_sk#150], [d_date_sk#161], Inner, BuildRight - -(112) CometProject -Input [10]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cs_sold_date_sk#150, cd_dep_count#153, c_birth_year#157, d_date_sk#161] -Arguments: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157], [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157] - -(113) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#162, i_item_id#30] - -(114) CometBroadcastHashJoin -Left output [8]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157] -Right output [2]: [i_item_sk#162, i_item_id#30] -Arguments: [cs_item_sk#144], [i_item_sk#162], Inner, BuildRight - -(115) CometProject -Input [10]: [cs_item_sk#144, cs_quantity#145, cs_list_price#146, cs_sales_price#147, cs_coupon_amt#148, cs_net_profit#149, cd_dep_count#153, c_birth_year#157, i_item_sk#162, i_item_id#30] -Arguments: [i_item_id#30, agg1#163, agg2#164, agg3#165, agg4#166, agg5#167, agg6#168, agg7#169], [i_item_id#30, cast(cs_quantity#145 as decimal(12,2)) AS agg1#163, cast(cs_list_price#146 as decimal(12,2)) AS agg2#164, cast(cs_coupon_amt#148 as decimal(12,2)) AS agg3#165, cast(cs_sales_price#147 as decimal(12,2)) AS agg4#166, cast(cs_net_profit#149 as decimal(12,2)) AS agg5#167, cast(c_birth_year#157 as decimal(12,2)) AS agg6#168, cast(cd_dep_count#153 as decimal(12,2)) AS agg7#169] - -(116) CometHashAggregate -Input [8]: [i_item_id#30, agg1#163, agg2#164, agg3#165, agg4#166, agg5#167, agg6#168, agg7#169] -Keys [1]: [i_item_id#30] -Functions [7]: [partial_avg(agg1#163), partial_avg(agg2#164), partial_avg(agg3#165), partial_avg(agg4#166), partial_avg(agg5#167), partial_avg(agg6#168), partial_avg(agg7#169)] - -(117) CometExchange -Input [15]: [i_item_id#30, sum#170, count#171, sum#172, count#173, sum#174, count#175, sum#176, count#177, sum#178, count#179, sum#180, count#181, sum#182, count#183] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(118) CometHashAggregate -Input [15]: [i_item_id#30, sum#170, count#171, sum#172, count#173, sum#174, count#175, sum#176, count#177, sum#178, count#179, sum#180, count#181, sum#182, count#183] -Keys [1]: [i_item_id#30] -Functions [7]: [avg(agg1#163), avg(agg2#164), avg(agg3#165), avg(agg4#166), avg(agg5#167), avg(agg6#168), avg(agg7#169)] - -(119) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#192), dynamicpruningexpression(cs_sold_date_sk#192 IN dynamicpruning#193)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(120) CometFilter -Input [9]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192] -Condition : ((isnotnull(cs_bill_cdemo_sk#185) AND isnotnull(cs_bill_customer_sk#184)) AND isnotnull(cs_item_sk#186)) - -(121) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#194, cd_dep_count#195] - -(122) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192] -Right output [2]: [cd_demo_sk#194, cd_dep_count#195] -Arguments: [cs_bill_cdemo_sk#185], [cd_demo_sk#194], Inner, BuildRight - -(123) CometProject -Input [11]: [cs_bill_customer_sk#184, cs_bill_cdemo_sk#185, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_demo_sk#194, cd_dep_count#195] -Arguments: [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195], [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195] - -(124) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#196, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] - -(125) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195] -Right output [4]: [c_customer_sk#196, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] -Arguments: [cs_bill_customer_sk#184], [c_customer_sk#196], Inner, BuildRight - -(126) CometProject -Input [13]: [cs_bill_customer_sk#184, cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_customer_sk#196, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] -Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] - -(127) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#200] - -(128) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199] -Right output [1]: [cd_demo_sk#200] -Arguments: [c_current_cdemo_sk#197], [cd_demo_sk#200], Inner, BuildRight - -(129) CometProject -Input [12]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_cdemo_sk#197, c_current_addr_sk#198, c_birth_year#199, cd_demo_sk#200] -Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199] - -(130) ReusedExchange [Reuses operator id: 107] -Output [1]: [ca_address_sk#201] - -(131) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199] -Right output [1]: [ca_address_sk#201] -Arguments: [c_current_addr_sk#198], [ca_address_sk#201], Inner, BuildRight - -(132) CometProject -Input [11]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_current_addr_sk#198, c_birth_year#199, ca_address_sk#201] -Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199] - -(133) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#202] - -(134) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199] -Right output [1]: [d_date_sk#202] -Arguments: [cs_sold_date_sk#192], [d_date_sk#202], Inner, BuildRight - -(135) CometProject -Input [10]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cs_sold_date_sk#192, cd_dep_count#195, c_birth_year#199, d_date_sk#202] -Arguments: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199], [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199] - -(136) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#203] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(137) CometFilter -Input [1]: [i_item_sk#203] -Condition : isnotnull(i_item_sk#203) - -(138) CometBroadcastExchange -Input [1]: [i_item_sk#203] -Arguments: [i_item_sk#203] - -(139) CometBroadcastHashJoin -Left output [8]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199] -Right output [1]: [i_item_sk#203] -Arguments: [cs_item_sk#186], [i_item_sk#203], Inner, BuildRight - -(140) CometProject -Input [9]: [cs_item_sk#186, cs_quantity#187, cs_list_price#188, cs_sales_price#189, cs_coupon_amt#190, cs_net_profit#191, cd_dep_count#195, c_birth_year#199, i_item_sk#203] -Arguments: [agg1#204, agg2#205, agg3#206, agg4#207, agg5#208, agg6#209, agg7#210], [cast(cs_quantity#187 as decimal(12,2)) AS agg1#204, cast(cs_list_price#188 as decimal(12,2)) AS agg2#205, cast(cs_coupon_amt#190 as decimal(12,2)) AS agg3#206, cast(cs_sales_price#189 as decimal(12,2)) AS agg4#207, cast(cs_net_profit#191 as decimal(12,2)) AS agg5#208, cast(c_birth_year#199 as decimal(12,2)) AS agg6#209, cast(cd_dep_count#195 as decimal(12,2)) AS agg7#210] - -(141) CometHashAggregate -Input [7]: [agg1#204, agg2#205, agg3#206, agg4#207, agg5#208, agg6#209, agg7#210] -Keys: [] -Functions [7]: [partial_avg(agg1#204), partial_avg(agg2#205), partial_avg(agg3#206), partial_avg(agg4#207), partial_avg(agg5#208), partial_avg(agg6#209), partial_avg(agg7#210)] - -(142) CometExchange -Input [14]: [sum#211, count#212, sum#213, count#214, sum#215, count#216, sum#217, count#218, sum#219, count#220, sum#221, count#222, sum#223, count#224] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(143) CometHashAggregate -Input [14]: [sum#211, count#212, sum#213, count#214, sum#215, count#216, sum#217, count#218, sum#219, count#220, sum#221, count#222, sum#223, count#224] -Keys: [] -Functions [7]: [avg(agg1#204), avg(agg2#205), avg(agg3#206), avg(agg4#207), avg(agg5#208), avg(agg6#209), avg(agg7#210)] - -(144) CometUnion -Child 0 Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] -Child 1 Input [11]: [i_item_id#30, ca_country#75, ca_state#25, county#232, agg1#233, agg2#234, agg3#235, agg4#236, agg5#237, agg6#238, agg7#239] -Child 2 Input [11]: [i_item_id#30, ca_country#118, ca_state#240, county#241, agg1#242, agg2#243, agg3#244, agg4#245, agg5#246, agg6#247, agg7#248] -Child 3 Input [11]: [i_item_id#30, ca_country#249, ca_state#250, county#251, agg1#252, agg2#253, agg3#254, agg4#255, agg5#256, agg6#257, agg7#258] -Child 4 Input [11]: [i_item_id#259, ca_country#260, ca_state#261, county#262, agg1#263, agg2#264, agg3#265, agg4#266, agg5#267, agg6#268, agg7#269] - -(145) CometTakeOrderedAndProject -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#32 ASC NULLS FIRST,ca_state#33 ASC NULLS FIRST,ca_county#34 ASC NULLS FIRST,i_item_id#31 ASC NULLS FIRST], output=[i_item_id#31,ca_country#32,ca_state#33,ca_county#34,agg1#225,agg2#226,agg3#227,agg4#228,agg5#229,agg6#230,agg7#231]), [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231], 100, 0, [ca_country#32 ASC NULLS FIRST, ca_state#33 ASC NULLS FIRST, ca_county#34 ASC NULLS FIRST, i_item_id#31 ASC NULLS FIRST], [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] - -(146) CometColumnarToRow [codegen id : 1] -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#225, agg2#226, agg3#227, agg4#228, agg5#229, agg6#230, agg7#231] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (151) -+- * CometColumnarToRow (150) - +- CometProject (149) - +- CometFilter (148) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (147) - - -(147) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(148) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#26)) - -(149) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(150) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(151) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#64 IN dynamicpruning#10 - -Subquery:3 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#107 IN dynamicpruning#10 - -Subquery:4 Hosting operator id = 93 Hosting Expression = cs_sold_date_sk#150 IN dynamicpruning#10 - -Subquery:5 Hosting operator id = 119 Hosting Expression = cs_sold_date_sk#192 IN dynamicpruning#10 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/extended.txt deleted file mode 100644 index b18a444bb9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/extended.txt +++ /dev/null @@ -1,214 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 204 out of 210 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 1572a2a240..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a/simplified.txt +++ /dev/null @@ -1,160 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometUnion [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country,ca_state,ca_county] #1 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometProject [i_item_id,ca_country,ca_state,ca_county,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,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_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 [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 [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,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_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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - 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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange [cd_demo_sk] #5 - CometFilter [cd_demo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #8 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country,ca_state] #9 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 [ca_address_sk,ca_state,ca_country] #10 - CometProject [ca_state] [ca_address_sk,ca_state,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country] #11 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 [ca_address_sk,ca_country] #12 - CometProject [ca_address_sk,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id] #13 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 [ca_address_sk] #14 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - ReusedExchange [d_date_sk] #7 - ReusedExchange [i_item_sk,i_item_id] #8 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange #15 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 - ReusedExchange [ca_address_sk] #14 - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [i_item_sk] #16 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/explain.txt deleted file mode 100644 index 5a32d4019b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/explain.txt +++ /dev/null @@ -1,163 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.item (4) - +- ReusedExchange (11) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(7) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(8) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(11) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] - -(14) HashAggregate [codegen id : 3] -Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(15) CometColumnarExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(17) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#17] -Results [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#17,17,2) AS _w0#19] - -(18) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(22) Project [codegen id : 6] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] - -(23) TakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometNativeScan parquet spark_catalog.default.date_dim (24) - - -(24) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#22] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/simplified.txt deleted file mode 100644 index 7bc0779e53..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/explain.txt deleted file mode 100644 index 322bbe63b3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [cs_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/simplified.txt deleted file mode 100644 index cf18e68a3d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/explain.txt deleted file mode 100644 index 322bbe63b3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [cs_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/extended.txt deleted file mode 100644 index cd52b2cd12..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index cf18e68a3d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/explain.txt deleted file mode 100644 index aa06ae2c5f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/explain.txt +++ /dev/null @@ -1,170 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (24) -+- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometColumnarExchange (21) - +- * HashAggregate (20) - +- * Expand (19) - +- * Project (18) - +- * BroadcastNestedLoopJoin Inner BuildRight (17) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.inventory (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) - : +- CometProject (9) - : +- CometFilter (8) - : +- CometNativeScan parquet spark_catalog.default.item (7) - +- BroadcastExchange (16) - +- * CometColumnarToRow (15) - +- CometNativeScan parquet spark_catalog.default.warehouse (14) - - -(1) Scan parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#3), dynamicpruningexpression(inv_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(inv_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] - -(3) Filter [codegen id : 4] -Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Condition : isnotnull(inv_item_sk#1) - -(4) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#5] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] -Input [4]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3, d_date_sk#5] - -(7) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(8) CometFilter -Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] -Condition : isnotnull(i_item_sk#6) - -(9) CometProject -Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] -Arguments: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14], [i_item_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#7, 50, true, false, true) AS i_brand#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#8, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#10, 50, true, false, true) AS i_product_name#14] - -(10) CometColumnarToRow [codegen id : 2] -Input [5]: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] - -(11) BroadcastExchange -Input [5]: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#2, i_brand#11, i_class#12, i_category#13, i_product_name#14] -Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] - -(14) CometNativeScan parquet spark_catalog.default.warehouse -Output: [] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -ReadSchema: struct<> - -(15) CometColumnarToRow [codegen id : 3] -Input: [] - -(16) BroadcastExchange -Input: [] -Arguments: IdentityBroadcastMode, [plan_id=2] - -(17) BroadcastNestedLoopJoin [codegen id : 4] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13] -Input [5]: [inv_quantity_on_hand#2, i_brand#11, i_class#12, i_category#13, i_product_name#14] - -(19) Expand [codegen id : 4] -Input [5]: [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13] -Arguments: [[inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13, 0], [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, null, 1], [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, null, null, 3], [inv_quantity_on_hand#2, i_product_name#14, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] - -(20) HashAggregate [codegen id : 4] -Input [6]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] -Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] -Functions [1]: [partial_avg(inv_quantity_on_hand#2)] -Aggregate Attributes [2]: [sum#20, count#21] -Results [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] - -(21) CometColumnarExchange -Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] -Arguments: hashpartitioning(i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] - -(23) HashAggregate [codegen id : 5] -Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] -Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] -Functions [1]: [avg(inv_quantity_on_hand#2)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#24] -Results [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, avg(inv_quantity_on_hand#2)#24 AS qoh#25] - -(24) TakeOrderedAndProject -Input [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25] -Arguments: 100, [qoh#25 ASC NULLS FIRST, i_product_name#15 ASC NULLS FIRST, i_brand#16 ASC NULLS FIRST, i_class#17 ASC NULLS FIRST, i_category#18 ASC NULLS FIRST], [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometNativeScan parquet spark_catalog.default.date_dim (25) - - -(25) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#26] -Condition : (((isnotnull(d_month_seq#26) AND (d_month_seq#26 >= 1200)) AND (d_month_seq#26 <= 1211)) AND isnotnull(d_date_sk#5)) - -(27) CometProject -Input [2]: [d_date_sk#5, d_month_seq#26] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(29) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/simplified.txt deleted file mode 100644 index 457dff46e2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (5) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastNestedLoopJoin - Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Filter [inv_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometNativeScan parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/explain.txt deleted file mode 100644 index f85c898208..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (26) -+- * HashAggregate (25) - +- * CometColumnarToRow (24) - +- CometColumnarExchange (23) - +- * HashAggregate (22) - +- * Expand (21) - +- * Project (20) - +- * BroadcastNestedLoopJoin Inner BuildRight (19) - :- * CometColumnarToRow (15) - : +- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - +- BroadcastExchange (18) - +- * CometColumnarToRow (17) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (16) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#3), dynamicpruningexpression(inv_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(inv_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Condition : isnotnull(inv_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [inv_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3, d_date_sk#5] -Arguments: [inv_item_sk#1, inv_quantity_on_hand#2], [inv_item_sk#1, inv_quantity_on_hand#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Condition : isnotnull(i_item_sk#7) - -(11) CometProject -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#8, 50, true, false, true) AS i_brand#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#11, 50, true, false, true) AS i_product_name#15] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(13) CometBroadcastHashJoin -Left output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] -Right output [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15], [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(15) CometColumnarToRow [codegen id : 2] -Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output: [] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -ReadSchema: struct<> - -(17) CometColumnarToRow [codegen id : 1] -Input: [] - -(18) BroadcastExchange -Input: [] -Arguments: IdentityBroadcastMode, [plan_id=1] - -(19) BroadcastNestedLoopJoin [codegen id : 2] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 2] -Output [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] -Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(21) Expand [codegen id : 2] -Input [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] -Arguments: [[inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14, 0], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, null, 1], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, null, null, 3], [inv_quantity_on_hand#2, i_product_name#15, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] - -(22) HashAggregate [codegen id : 2] -Input [6]: [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Functions [1]: [partial_avg(inv_quantity_on_hand#2)] -Aggregate Attributes [2]: [sum#21, count#22] -Results [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] - -(23) CometColumnarExchange -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -Arguments: hashpartitioning(i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(24) CometColumnarToRow [codegen id : 3] -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] - -(25) HashAggregate [codegen id : 3] -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Functions [1]: [avg(inv_quantity_on_hand#2)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#25] -Results [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, avg(inv_quantity_on_hand#2)#25 AS qoh#26] - -(26) TakeOrderedAndProject -Input [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#26] -Arguments: 100, [qoh#26 ASC NULLS FIRST, i_product_name#16 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_category#19 ASC NULLS FIRST], [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(29) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(31) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/simplified.txt deleted file mode 100644 index 221c6063ce..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (3) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (2) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - 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_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/explain.txt deleted file mode 100644 index f85c898208..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (26) -+- * HashAggregate (25) - +- * CometColumnarToRow (24) - +- CometColumnarExchange (23) - +- * HashAggregate (22) - +- * Expand (21) - +- * Project (20) - +- * BroadcastNestedLoopJoin Inner BuildRight (19) - :- * CometColumnarToRow (15) - : +- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - +- BroadcastExchange (18) - +- * CometColumnarToRow (17) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (16) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#3), dynamicpruningexpression(inv_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(inv_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Condition : isnotnull(inv_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [inv_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3, d_date_sk#5] -Arguments: [inv_item_sk#1, inv_quantity_on_hand#2], [inv_item_sk#1, inv_quantity_on_hand#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Condition : isnotnull(i_item_sk#7) - -(11) CometProject -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#8, 50, true, false, true) AS i_brand#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#11, 50, true, false, true) AS i_product_name#15] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(13) CometBroadcastHashJoin -Left output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] -Right output [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15], [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(15) CometColumnarToRow [codegen id : 2] -Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output: [] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -ReadSchema: struct<> - -(17) CometColumnarToRow [codegen id : 1] -Input: [] - -(18) BroadcastExchange -Input: [] -Arguments: IdentityBroadcastMode, [plan_id=1] - -(19) BroadcastNestedLoopJoin [codegen id : 2] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 2] -Output [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] -Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(21) Expand [codegen id : 2] -Input [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] -Arguments: [[inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14, 0], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, null, 1], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, null, null, 3], [inv_quantity_on_hand#2, i_product_name#15, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] - -(22) HashAggregate [codegen id : 2] -Input [6]: [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Functions [1]: [partial_avg(inv_quantity_on_hand#2)] -Aggregate Attributes [2]: [sum#21, count#22] -Results [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] - -(23) CometColumnarExchange -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -Arguments: hashpartitioning(i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(24) CometColumnarToRow [codegen id : 3] -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] - -(25) HashAggregate [codegen id : 3] -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Functions [1]: [avg(inv_quantity_on_hand#2)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#25] -Results [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, avg(inv_quantity_on_hand#2)#25 AS qoh#26] - -(26) TakeOrderedAndProject -Input [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#26] -Arguments: 100, [qoh#26 ASC NULLS FIRST, i_product_name#16 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_category#19 ASC NULLS FIRST], [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(29) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(31) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/extended.txt deleted file mode 100644 index 6cb89465d2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/extended.txt +++ /dev/null @@ -1,34 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - -Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 221c6063ce..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (3) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (2) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - 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_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/explain.txt deleted file mode 100644 index 0998977eb2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/explain.txt +++ /dev/null @@ -1,356 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (55) -+- Union (54) - :- * HashAggregate (25) - : +- * HashAggregate (24) - : +- * HashAggregate (23) - : +- * CometColumnarToRow (22) - : +- CometColumnarExchange (21) - : +- * HashAggregate (20) - : +- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.inventory (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.item (7) - : +- BroadcastExchange (17) - : +- * CometColumnarToRow (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.warehouse (14) - :- * HashAggregate (32) - : +- * CometColumnarToRow (31) - : +- CometColumnarExchange (30) - : +- * HashAggregate (29) - : +- * HashAggregate (28) - : +- * CometColumnarToRow (27) - : +- ReusedExchange (26) - :- * HashAggregate (39) - : +- * CometColumnarToRow (38) - : +- CometColumnarExchange (37) - : +- * HashAggregate (36) - : +- * HashAggregate (35) - : +- * CometColumnarToRow (34) - : +- ReusedExchange (33) - :- * HashAggregate (46) - : +- * CometColumnarToRow (45) - : +- CometColumnarExchange (44) - : +- * HashAggregate (43) - : +- * HashAggregate (42) - : +- * CometColumnarToRow (41) - : +- ReusedExchange (40) - +- * HashAggregate (53) - +- * CometColumnarToRow (52) - +- CometColumnarExchange (51) - +- * HashAggregate (50) - +- * HashAggregate (49) - +- * CometColumnarToRow (48) - +- ReusedExchange (47) - - -(1) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(4) ReusedExchange [Reuses operator id: 60] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(8) CometFilter -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Condition : isnotnull(i_item_sk#7) - -(9) CometProject -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#8, 50, true, false, true) AS i_brand#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#11, 50, true, false, true) AS i_product_name#15] - -(10) CometColumnarToRow [codegen id : 2] -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(11) BroadcastExchange -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(14) CometNativeScan parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(15) CometFilter -Input [1]: [w_warehouse_sk#16] -Condition : isnotnull(w_warehouse_sk#16) - -(16) CometColumnarToRow [codegen id : 3] -Input [1]: [w_warehouse_sk#16] - -(17) BroadcastExchange -Input [1]: [w_warehouse_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#16] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15, w_warehouse_sk#16] - -(20) HashAggregate [codegen id : 4] -Input [5]: [inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [sum#17, count#18] -Results [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] - -(21) CometColumnarExchange -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] -Arguments: hashpartitioning(i_product_name#15, i_brand#12, i_class#13, i_category#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] - -(23) HashAggregate [codegen id : 5] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#21] -Results [5]: [i_product_name#15 AS i_product_name#22, i_brand#12 AS i_brand#23, i_class#13 AS i_class#24, i_category#14 AS i_category#25, avg(inv_quantity_on_hand#3)#21 AS qoh#26] - -(24) HashAggregate [codegen id : 5] -Input [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#26] -Keys [4]: [i_product_name#22, i_brand#23, i_class#24, i_category#25] -Functions [1]: [partial_avg(qoh#26)] -Aggregate Attributes [2]: [sum#27, count#28] -Results [6]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, sum#29, count#30] - -(25) HashAggregate [codegen id : 5] -Input [6]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, sum#29, count#30] -Keys [4]: [i_product_name#22, i_brand#23, i_class#24, i_category#25] -Functions [1]: [avg(qoh#26)] -Aggregate Attributes [1]: [avg(qoh#26)#31] -Results [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, avg(qoh#26)#31 AS qoh#32] - -(26) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] - -(27) CometColumnarToRow [codegen id : 10] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] - -(28) HashAggregate [codegen id : 10] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [avg(inv_quantity_on_hand#35)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#35)#21] -Results [4]: [i_product_name#15, i_brand#12, i_class#13, avg(inv_quantity_on_hand#35)#21 AS qoh#36] - -(29) HashAggregate [codegen id : 10] -Input [4]: [i_product_name#15, i_brand#12, i_class#13, qoh#36] -Keys [3]: [i_product_name#15, i_brand#12, i_class#13] -Functions [1]: [partial_avg(qoh#36)] -Aggregate Attributes [2]: [sum#37, count#38] -Results [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] - -(30) CometColumnarExchange -Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] -Arguments: hashpartitioning(i_product_name#15, i_brand#12, i_class#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(31) CometColumnarToRow [codegen id : 11] -Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] - -(32) HashAggregate [codegen id : 11] -Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#39, count#40] -Keys [3]: [i_product_name#15, i_brand#12, i_class#13] -Functions [1]: [avg(qoh#36)] -Aggregate Attributes [1]: [avg(qoh#36)#41] -Results [5]: [i_product_name#15, i_brand#12, i_class#13, null AS i_category#42, avg(qoh#36)#41 AS qoh#43] - -(33) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#44, count#45] - -(34) CometColumnarToRow [codegen id : 16] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#44, count#45] - -(35) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#44, count#45] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [avg(inv_quantity_on_hand#46)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#46)#21] -Results [3]: [i_product_name#15, i_brand#12, avg(inv_quantity_on_hand#46)#21 AS qoh#47] - -(36) HashAggregate [codegen id : 16] -Input [3]: [i_product_name#15, i_brand#12, qoh#47] -Keys [2]: [i_product_name#15, i_brand#12] -Functions [1]: [partial_avg(qoh#47)] -Aggregate Attributes [2]: [sum#48, count#49] -Results [4]: [i_product_name#15, i_brand#12, sum#50, count#51] - -(37) CometColumnarExchange -Input [4]: [i_product_name#15, i_brand#12, sum#50, count#51] -Arguments: hashpartitioning(i_product_name#15, i_brand#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(38) CometColumnarToRow [codegen id : 17] -Input [4]: [i_product_name#15, i_brand#12, sum#50, count#51] - -(39) HashAggregate [codegen id : 17] -Input [4]: [i_product_name#15, i_brand#12, sum#50, count#51] -Keys [2]: [i_product_name#15, i_brand#12] -Functions [1]: [avg(qoh#47)] -Aggregate Attributes [1]: [avg(qoh#47)#52] -Results [5]: [i_product_name#15, i_brand#12, null AS i_class#53, null AS i_category#54, avg(qoh#47)#52 AS qoh#55] - -(40) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#56, count#57] - -(41) CometColumnarToRow [codegen id : 22] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#56, count#57] - -(42) HashAggregate [codegen id : 22] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#56, count#57] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [avg(inv_quantity_on_hand#58)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#58)#21] -Results [2]: [i_product_name#15, avg(inv_quantity_on_hand#58)#21 AS qoh#59] - -(43) HashAggregate [codegen id : 22] -Input [2]: [i_product_name#15, qoh#59] -Keys [1]: [i_product_name#15] -Functions [1]: [partial_avg(qoh#59)] -Aggregate Attributes [2]: [sum#60, count#61] -Results [3]: [i_product_name#15, sum#62, count#63] - -(44) CometColumnarExchange -Input [3]: [i_product_name#15, sum#62, count#63] -Arguments: hashpartitioning(i_product_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(45) CometColumnarToRow [codegen id : 23] -Input [3]: [i_product_name#15, sum#62, count#63] - -(46) HashAggregate [codegen id : 23] -Input [3]: [i_product_name#15, sum#62, count#63] -Keys [1]: [i_product_name#15] -Functions [1]: [avg(qoh#59)] -Aggregate Attributes [1]: [avg(qoh#59)#64] -Results [5]: [i_product_name#15, null AS i_brand#65, null AS i_class#66, null AS i_category#67, avg(qoh#59)#64 AS qoh#68] - -(47) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#69, count#70] - -(48) CometColumnarToRow [codegen id : 28] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#69, count#70] - -(49) HashAggregate [codegen id : 28] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#69, count#70] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [avg(inv_quantity_on_hand#71)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#71)#21] -Results [1]: [avg(inv_quantity_on_hand#71)#21 AS qoh#72] - -(50) HashAggregate [codegen id : 28] -Input [1]: [qoh#72] -Keys: [] -Functions [1]: [partial_avg(qoh#72)] -Aggregate Attributes [2]: [sum#73, count#74] -Results [2]: [sum#75, count#76] - -(51) CometColumnarExchange -Input [2]: [sum#75, count#76] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(52) CometColumnarToRow [codegen id : 29] -Input [2]: [sum#75, count#76] - -(53) HashAggregate [codegen id : 29] -Input [2]: [sum#75, count#76] -Keys: [] -Functions [1]: [avg(qoh#72)] -Aggregate Attributes [1]: [avg(qoh#72)#77] -Results [5]: [null AS i_product_name#78, null AS i_brand#79, null AS i_class#80, null AS i_category#81, avg(qoh#72)#77 AS qoh#82] - -(54) Union - -(55) TakeOrderedAndProject -Input [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32] -Arguments: 100, [qoh#32 ASC NULLS FIRST, i_product_name#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, i_class#24 ASC NULLS FIRST, i_category#25 ASC NULLS FIRST], [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (60) -+- * CometColumnarToRow (59) - +- CometProject (58) - +- CometFilter (57) - +- CometNativeScan parquet spark_catalog.default.date_dim (56) - - -(56) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#83] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(57) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#83] -Condition : (((isnotnull(d_month_seq#83) AND (d_month_seq#83 >= 1212)) AND (d_month_seq#83 <= 1223)) AND isnotnull(d_date_sk#6)) - -(58) CometProject -Input [2]: [d_date_sk#6, d_month_seq#83] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(59) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(60) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/simplified.txt deleted file mode 100644 index a55feaac8c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - Union - WholeStageCodegen (5) - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),i_product_name,i_brand,i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category] #1 - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] - Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk] - WholeStageCodegen (11) - HashAggregate [i_product_name,i_brand,i_class,sum,count] [avg(qoh),i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class] #5 - WholeStageCodegen (10) - HashAggregate [i_product_name,i_brand,i_class,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (17) - HashAggregate [i_product_name,i_brand,sum,count] [avg(qoh),i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand] #6 - WholeStageCodegen (16) - HashAggregate [i_product_name,i_brand,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (23) - HashAggregate [i_product_name,sum,count] [avg(qoh),i_brand,i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name] #7 - WholeStageCodegen (22) - HashAggregate [i_product_name,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (29) - HashAggregate [sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (28) - HashAggregate [qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/explain.txt deleted file mode 100644 index bf16cc4d21..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,301 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometTakeOrderedAndProject (46) - +- CometUnion (45) - :- CometHashAggregate (24) - : +- CometHashAggregate (23) - : +- CometHashAggregate (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - : +- CometBroadcastExchange (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (15) - :- CometHashAggregate (29) - : +- CometExchange (28) - : +- CometHashAggregate (27) - : +- CometHashAggregate (26) - : +- ReusedExchange (25) - :- CometHashAggregate (34) - : +- CometExchange (33) - : +- CometHashAggregate (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - :- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometHashAggregate (36) - : +- ReusedExchange (35) - +- CometHashAggregate (44) - +- CometExchange (43) - +- CometHashAggregate (42) - +- CometHashAggregate (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [inv_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] -Arguments: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3], [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Condition : isnotnull(i_item_sk#8) - -(11) CometProject -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#12, 50, true, false, true) AS i_product_name#16] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(13) CometBroadcastHashJoin -Left output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -Right output [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight - -(14) CometProject -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [w_warehouse_sk#17] -Condition : isnotnull(w_warehouse_sk#17) - -(17) CometBroadcastExchange -Input [1]: [w_warehouse_sk#17] -Arguments: [w_warehouse_sk#17] - -(18) CometBroadcastHashJoin -Left output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Right output [1]: [w_warehouse_sk#17] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#17], Inner, BuildRight - -(19) CometProject -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16, w_warehouse_sk#17] -Arguments: [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(20) CometHashAggregate -Input [5]: [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [partial_avg(inv_quantity_on_hand#3)] - -(21) CometExchange -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#18, count#19] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, i_category#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#18, count#19] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#3)] - -(23) CometHashAggregate -Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#24] -Keys [4]: [i_product_name#20, i_brand#21, i_class#22, i_category#23] -Functions [1]: [partial_avg(qoh#24)] - -(24) CometHashAggregate -Input [6]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, sum#25, count#26] -Keys [4]: [i_product_name#20, i_brand#21, i_class#22, i_category#23] -Functions [1]: [avg(qoh#24)] - -(25) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#27, count#28] - -(26) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#27, count#28] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#29)] - -(27) CometHashAggregate -Input [4]: [i_product_name#16, i_brand#13, i_class#14, qoh#30] -Keys [3]: [i_product_name#16, i_brand#13, i_class#14] -Functions [1]: [partial_avg(qoh#30)] - -(28) CometExchange -Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#31, count#32] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#31, count#32] -Keys [3]: [i_product_name#16, i_brand#13, i_class#14] -Functions [1]: [avg(qoh#30)] - -(30) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#33, count#34] - -(31) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#33, count#34] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#35)] - -(32) CometHashAggregate -Input [3]: [i_product_name#16, i_brand#13, qoh#36] -Keys [2]: [i_product_name#16, i_brand#13] -Functions [1]: [partial_avg(qoh#36)] - -(33) CometExchange -Input [4]: [i_product_name#16, i_brand#13, sum#37, count#38] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(34) CometHashAggregate -Input [4]: [i_product_name#16, i_brand#13, sum#37, count#38] -Keys [2]: [i_product_name#16, i_brand#13] -Functions [1]: [avg(qoh#36)] - -(35) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#39, count#40] - -(36) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#39, count#40] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#41)] - -(37) CometHashAggregate -Input [2]: [i_product_name#16, qoh#42] -Keys [1]: [i_product_name#16] -Functions [1]: [partial_avg(qoh#42)] - -(38) CometExchange -Input [3]: [i_product_name#16, sum#43, count#44] -Arguments: hashpartitioning(i_product_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(39) CometHashAggregate -Input [3]: [i_product_name#16, sum#43, count#44] -Keys [1]: [i_product_name#16] -Functions [1]: [avg(qoh#42)] - -(40) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#45, count#46] - -(41) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#45, count#46] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#47)] - -(42) CometHashAggregate -Input [1]: [qoh#48] -Keys: [] -Functions [1]: [partial_avg(qoh#48)] - -(43) CometExchange -Input [2]: [sum#49, count#50] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(44) CometHashAggregate -Input [2]: [sum#49, count#50] -Keys: [] -Functions [1]: [avg(qoh#48)] - -(45) CometUnion -Child 0 Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] -Child 1 Input [5]: [i_product_name#16, i_brand#13, i_class#14, i_category#52, qoh#53] -Child 2 Input [5]: [i_product_name#16, i_brand#13, i_class#54, i_category#55, qoh#56] -Child 3 Input [5]: [i_product_name#16, i_brand#57, i_class#58, i_category#59, qoh#60] -Child 4 Input [5]: [i_product_name#61, i_brand#62, i_class#63, i_category#64, qoh#65] - -(46) CometTakeOrderedAndProject -Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#51 ASC NULLS FIRST,i_product_name#20 ASC NULLS FIRST,i_brand#21 ASC NULLS FIRST,i_class#22 ASC NULLS FIRST,i_category#23 ASC NULLS FIRST], output=[i_product_name#20,i_brand#21,i_class#22,i_category#23,qoh#51]), [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51], 100, 0, [qoh#51 ASC NULLS FIRST, i_product_name#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, i_class#22 ASC NULLS FIRST, i_category#23 ASC NULLS FIRST], [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] - -(47) CometColumnarToRow [codegen id : 1] -Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) - -(50) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(52) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/simplified.txt deleted file mode 100644 index b59605103e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,57 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] - CometUnion [i_product_name,i_brand,i_class,i_category,qoh] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometHashAggregate [qoh] [i_product_name,i_brand,i_class,i_category,sum,count] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,i_product_name,i_brand,i_class,i_category,avg(inv_quantity_on_hand)] - CometExchange [i_product_name,i_brand,i_class,i_category] #1 - CometHashAggregate [inv_quantity_on_hand] [i_product_name,i_brand,i_class,i_category,sum,count] - CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - 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,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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange [w_warehouse_sk] #5 - CometFilter [w_warehouse_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name,i_brand,i_class] #6 - CometHashAggregate [qoh] [i_product_name,i_brand,i_class,sum,count] - CometHashAggregate [i_category,sum,count] [i_product_name,i_brand,i_class,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name,i_brand] #7 - CometHashAggregate [qoh] [i_product_name,i_brand,sum,count] - CometHashAggregate [i_class,i_category,sum,count] [i_product_name,i_brand,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name] #8 - CometHashAggregate [qoh] [i_product_name,sum,count] - CometHashAggregate [i_brand,i_class,i_category,sum,count] [i_product_name,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange #9 - CometHashAggregate [qoh] [sum,count] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/explain.txt deleted file mode 100644 index bf16cc4d21..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/explain.txt +++ /dev/null @@ -1,301 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometTakeOrderedAndProject (46) - +- CometUnion (45) - :- CometHashAggregate (24) - : +- CometHashAggregate (23) - : +- CometHashAggregate (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - : +- CometBroadcastExchange (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (15) - :- CometHashAggregate (29) - : +- CometExchange (28) - : +- CometHashAggregate (27) - : +- CometHashAggregate (26) - : +- ReusedExchange (25) - :- CometHashAggregate (34) - : +- CometExchange (33) - : +- CometHashAggregate (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - :- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometHashAggregate (36) - : +- ReusedExchange (35) - +- CometHashAggregate (44) - +- CometExchange (43) - +- CometHashAggregate (42) - +- CometHashAggregate (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [inv_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] -Arguments: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3], [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Condition : isnotnull(i_item_sk#8) - -(11) CometProject -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#12, 50, true, false, true) AS i_product_name#16] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(13) CometBroadcastHashJoin -Left output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -Right output [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight - -(14) CometProject -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [w_warehouse_sk#17] -Condition : isnotnull(w_warehouse_sk#17) - -(17) CometBroadcastExchange -Input [1]: [w_warehouse_sk#17] -Arguments: [w_warehouse_sk#17] - -(18) CometBroadcastHashJoin -Left output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Right output [1]: [w_warehouse_sk#17] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#17], Inner, BuildRight - -(19) CometProject -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16, w_warehouse_sk#17] -Arguments: [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(20) CometHashAggregate -Input [5]: [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [partial_avg(inv_quantity_on_hand#3)] - -(21) CometExchange -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#18, count#19] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, i_category#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#18, count#19] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#3)] - -(23) CometHashAggregate -Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#24] -Keys [4]: [i_product_name#20, i_brand#21, i_class#22, i_category#23] -Functions [1]: [partial_avg(qoh#24)] - -(24) CometHashAggregate -Input [6]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, sum#25, count#26] -Keys [4]: [i_product_name#20, i_brand#21, i_class#22, i_category#23] -Functions [1]: [avg(qoh#24)] - -(25) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#27, count#28] - -(26) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#27, count#28] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#29)] - -(27) CometHashAggregate -Input [4]: [i_product_name#16, i_brand#13, i_class#14, qoh#30] -Keys [3]: [i_product_name#16, i_brand#13, i_class#14] -Functions [1]: [partial_avg(qoh#30)] - -(28) CometExchange -Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#31, count#32] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#31, count#32] -Keys [3]: [i_product_name#16, i_brand#13, i_class#14] -Functions [1]: [avg(qoh#30)] - -(30) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#33, count#34] - -(31) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#33, count#34] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#35)] - -(32) CometHashAggregate -Input [3]: [i_product_name#16, i_brand#13, qoh#36] -Keys [2]: [i_product_name#16, i_brand#13] -Functions [1]: [partial_avg(qoh#36)] - -(33) CometExchange -Input [4]: [i_product_name#16, i_brand#13, sum#37, count#38] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(34) CometHashAggregate -Input [4]: [i_product_name#16, i_brand#13, sum#37, count#38] -Keys [2]: [i_product_name#16, i_brand#13] -Functions [1]: [avg(qoh#36)] - -(35) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#39, count#40] - -(36) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#39, count#40] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#41)] - -(37) CometHashAggregate -Input [2]: [i_product_name#16, qoh#42] -Keys [1]: [i_product_name#16] -Functions [1]: [partial_avg(qoh#42)] - -(38) CometExchange -Input [3]: [i_product_name#16, sum#43, count#44] -Arguments: hashpartitioning(i_product_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(39) CometHashAggregate -Input [3]: [i_product_name#16, sum#43, count#44] -Keys [1]: [i_product_name#16] -Functions [1]: [avg(qoh#42)] - -(40) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#45, count#46] - -(41) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#45, count#46] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#47)] - -(42) CometHashAggregate -Input [1]: [qoh#48] -Keys: [] -Functions [1]: [partial_avg(qoh#48)] - -(43) CometExchange -Input [2]: [sum#49, count#50] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(44) CometHashAggregate -Input [2]: [sum#49, count#50] -Keys: [] -Functions [1]: [avg(qoh#48)] - -(45) CometUnion -Child 0 Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] -Child 1 Input [5]: [i_product_name#16, i_brand#13, i_class#14, i_category#52, qoh#53] -Child 2 Input [5]: [i_product_name#16, i_brand#13, i_class#54, i_category#55, qoh#56] -Child 3 Input [5]: [i_product_name#16, i_brand#57, i_class#58, i_category#59, qoh#60] -Child 4 Input [5]: [i_product_name#61, i_brand#62, i_class#63, i_category#64, qoh#65] - -(46) CometTakeOrderedAndProject -Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#51 ASC NULLS FIRST,i_product_name#20 ASC NULLS FIRST,i_brand#21 ASC NULLS FIRST,i_class#22 ASC NULLS FIRST,i_category#23 ASC NULLS FIRST], output=[i_product_name#20,i_brand#21,i_class#22,i_category#23,qoh#51]), [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51], 100, 0, [qoh#51 ASC NULLS FIRST, i_product_name#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, i_class#22 ASC NULLS FIRST, i_category#23 ASC NULLS FIRST], [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] - -(47) CometColumnarToRow [codegen id : 1] -Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#51] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) - -(50) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(52) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/extended.txt deleted file mode 100644 index c018673888..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/extended.txt +++ /dev/null @@ -1,159 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - -Comet accelerated 141 out of 151 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index b59605103e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a/simplified.txt +++ /dev/null @@ -1,57 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] - CometUnion [i_product_name,i_brand,i_class,i_category,qoh] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometHashAggregate [qoh] [i_product_name,i_brand,i_class,i_category,sum,count] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,i_product_name,i_brand,i_class,i_category,avg(inv_quantity_on_hand)] - CometExchange [i_product_name,i_brand,i_class,i_category] #1 - CometHashAggregate [inv_quantity_on_hand] [i_product_name,i_brand,i_class,i_category,sum,count] - CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - 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,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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange [w_warehouse_sk] #5 - CometFilter [w_warehouse_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name,i_brand,i_class] #6 - CometHashAggregate [qoh] [i_product_name,i_brand,i_class,sum,count] - CometHashAggregate [i_category,sum,count] [i_product_name,i_brand,i_class,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name,i_brand] #7 - CometHashAggregate [qoh] [i_product_name,i_brand,sum,count] - CometHashAggregate [i_class,i_category,sum,count] [i_product_name,i_brand,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name] #8 - CometHashAggregate [qoh] [i_product_name,sum,count] - CometHashAggregate [i_brand,i_class,i_category,sum,count] [i_product_name,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange #9 - CometHashAggregate [qoh] [sum,count] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_datafusion/explain.txt deleted file mode 100644 index 696a4579e6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_datafusion/explain.txt +++ /dev/null @@ -1,453 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (50) -+- CometSort (49) - +- CometColumnarExchange (48) - +- * Filter (47) - +- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometNativeScan parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometNativeScan parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.customer_address (32) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometNativeScan parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) = pale ) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#19, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#21, 10, true, false, true) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometNativeScan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Condition : ((isnotnull(c_customer_sk#26) AND isnotnull(c_current_addr_sk#27)) AND isnotnull(c_birth_country#30)) - -(27) CometProject -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [c_customer_sk#26, c_current_addr_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#28, 20, true, false, true) AS c_first_name#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#29, 30, true, false, true) AS c_last_name#32, c_birth_country#30] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(31) CometColumnarToRow [codegen id : 2] -Input [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(32) CometNativeScan parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Condition : ((isnotnull(ca_address_sk#33) AND isnotnull(ca_country#36)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true))) - -(34) CometProject -Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Arguments: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36], [ca_address_sk#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#34, 2, true, false, true) AS ca_state#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true) AS ca_zip#38, ca_country#36] - -(35) CometColumnarToRow [codegen id : 1] -Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] - -(36) BroadcastExchange -Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [3]: [c_current_addr_sk#27, c_birth_country#30, s_zip#16] -Right keys [3]: [ca_address_sk#33, upper(ca_country#36), ca_zip#38] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] -Input [17]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30, ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#39] -Results [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] - -(40) CometColumnarExchange -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#41] -Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#41,17,2) AS netpaid#42] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, netpaid#42] -Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] -Functions [1]: [partial_sum(netpaid#42)] -Aggregate Attributes [2]: [sum#43, isEmpty#44] -Results [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] - -(44) CometColumnarExchange -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] -Functions [1]: [sum(netpaid#42)] -Aggregate Attributes [1]: [sum(netpaid#42)#47] -Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, sum(netpaid#42)#47 AS paid#48] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) - -(48) CometColumnarExchange -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(49) CometSort -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Arguments: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48], [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] - -(50) CometColumnarToRow [codegen id : 5] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#49, [id=#50] -* HashAggregate (80) -+- * CometColumnarToRow (79) - +- CometColumnarExchange (78) - +- * HashAggregate (77) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- CometColumnarExchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * CometColumnarToRow (69) - : +- CometProject (68) - : +- CometBroadcastHashJoin (67) - : :- CometProject (65) - : : +- CometBroadcastHashJoin (64) - : : :- CometProject (59) - : : : +- CometBroadcastHashJoin (58) - : : : :- CometProject (56) - : : : : +- CometSortMergeJoin (55) - : : : : :- CometSort (52) - : : : : : +- ReusedExchange (51) - : : : : +- CometSort (54) - : : : : +- ReusedExchange (53) - : : : +- ReusedExchange (57) - : : +- CometBroadcastExchange (63) - : : +- CometProject (62) - : : +- CometFilter (61) - : : +- CometNativeScan parquet spark_catalog.default.item (60) - : +- ReusedExchange (66) - +- ReusedExchange (70) - - -(51) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] - -(52) CometSort -Input [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55], [ss_ticket_number#54 ASC NULLS FIRST, ss_item_sk#51 ASC NULLS FIRST] - -(53) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#56, sr_ticket_number#57] - -(54) CometSort -Input [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [sr_item_sk#56, sr_ticket_number#57], [sr_ticket_number#57 ASC NULLS FIRST, sr_item_sk#56 ASC NULLS FIRST] - -(55) CometSortMergeJoin -Left output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -Right output [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [ss_ticket_number#54, ss_item_sk#51], [sr_ticket_number#57, sr_item_sk#56], Inner - -(56) CometProject -Input [7]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55, sr_item_sk#56, sr_ticket_number#57] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55], [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] - -(57) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] - -(58) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] -Right output [4]: [s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] -Arguments: [ss_store_sk#53], [s_store_sk#58], Inner, BuildRight - -(59) CometProject -Input [8]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55, s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16], [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] - -(60) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(61) CometFilter -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Condition : isnotnull(i_item_sk#60) - -(62) CometProject -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65], [i_item_sk#60, i_current_price#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#62, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#63, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#64, 10, true, false, true) AS i_units#25, i_manager_id#65] - -(63) CometBroadcastExchange -Input [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] - -(64) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [ss_item_sk#51], [i_item_sk#60], Inner, BuildRight - -(65) CometProject -Input [12]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65], [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] - -(66) ReusedExchange [Reuses operator id: 28] -Output [5]: [c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] - -(67) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Right output [5]: [c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -Arguments: [ss_customer_sk#52], [c_customer_sk#66], Inner, BuildRight - -(68) CometProject -Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -Arguments: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68], [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] - -(69) CometColumnarToRow [codegen id : 2] -Input [13]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] - -(70) ReusedExchange [Reuses operator id: 36] -Output [4]: [ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] - -(71) BroadcastHashJoin [codegen id : 2] -Left keys [3]: [c_current_addr_sk#67, c_birth_country#68, s_zip#16] -Right keys [3]: [ca_address_sk#69, upper(ca_country#70), ca_zip#38] -Join type: Inner -Join condition: None - -(72) Project [codegen id : 2] -Output [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] -Input [17]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68, ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] - -(73) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [sum#71] -Results [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] - -(74) CometColumnarExchange -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(75) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] - -(76) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#41] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#41,17,2) AS netpaid#73] - -(77) HashAggregate [codegen id : 3] -Input [1]: [netpaid#73] -Keys: [] -Functions [1]: [partial_avg(netpaid#73)] -Aggregate Attributes [2]: [sum#74, count#75] -Results [2]: [sum#76, count#77] - -(78) CometColumnarExchange -Input [2]: [sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(79) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#76, count#77] - -(80) HashAggregate [codegen id : 4] -Input [2]: [sum#76, count#77] -Keys: [] -Functions [1]: [avg(netpaid#73)] -Aggregate Attributes [1]: [avg(netpaid#73)#78] -Results [1]: [(0.05 * avg(netpaid#73)#78) AS (0.05 * avg(netpaid))#79] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_datafusion/simplified.txt deleted file mode 100644 index b24e51723a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_datafusion/simplified.txt +++ /dev/null @@ -1,99 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,s_store_name,paid] - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #5 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #12 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - InputAdapter - ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #4 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #5 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #7 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometNativeScan 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 (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] - CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] - CometNativeScan 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/q24.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/explain.txt deleted file mode 100644 index 17ef709df5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,453 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (50) -+- CometSort (49) - +- CometColumnarExchange (48) - +- * Filter (47) - +- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) = pale ) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#19, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#21, 10, true, false, true) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Condition : ((isnotnull(c_customer_sk#26) AND isnotnull(c_current_addr_sk#27)) AND isnotnull(c_birth_country#30)) - -(27) CometProject -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [c_customer_sk#26, c_current_addr_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#28, 20, true, false, true) AS c_first_name#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#29, 30, true, false, true) AS c_last_name#32, c_birth_country#30] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(31) CometColumnarToRow [codegen id : 2] -Input [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Condition : ((isnotnull(ca_address_sk#33) AND isnotnull(ca_country#36)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true))) - -(34) CometProject -Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Arguments: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36], [ca_address_sk#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#34, 2, true, false, true) AS ca_state#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true) AS ca_zip#38, ca_country#36] - -(35) CometColumnarToRow [codegen id : 1] -Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] - -(36) BroadcastExchange -Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [3]: [c_current_addr_sk#27, c_birth_country#30, s_zip#16] -Right keys [3]: [ca_address_sk#33, upper(ca_country#36), ca_zip#38] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] -Input [17]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30, ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#39] -Results [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] - -(40) CometColumnarExchange -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#41] -Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#41,17,2) AS netpaid#42] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, netpaid#42] -Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] -Functions [1]: [partial_sum(netpaid#42)] -Aggregate Attributes [2]: [sum#43, isEmpty#44] -Results [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] - -(44) CometColumnarExchange -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] -Functions [1]: [sum(netpaid#42)] -Aggregate Attributes [1]: [sum(netpaid#42)#47] -Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, sum(netpaid#42)#47 AS paid#48] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) - -(48) CometColumnarExchange -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(49) CometSort -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Arguments: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48], [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] - -(50) CometColumnarToRow [codegen id : 5] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#49, [id=#50] -* HashAggregate (80) -+- * CometColumnarToRow (79) - +- CometColumnarExchange (78) - +- * HashAggregate (77) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- CometColumnarExchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * CometColumnarToRow (69) - : +- CometProject (68) - : +- CometBroadcastHashJoin (67) - : :- CometProject (65) - : : +- CometBroadcastHashJoin (64) - : : :- CometProject (59) - : : : +- CometBroadcastHashJoin (58) - : : : :- CometProject (56) - : : : : +- CometSortMergeJoin (55) - : : : : :- CometSort (52) - : : : : : +- ReusedExchange (51) - : : : : +- CometSort (54) - : : : : +- ReusedExchange (53) - : : : +- ReusedExchange (57) - : : +- CometBroadcastExchange (63) - : : +- CometProject (62) - : : +- CometFilter (61) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (60) - : +- ReusedExchange (66) - +- ReusedExchange (70) - - -(51) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] - -(52) CometSort -Input [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55], [ss_ticket_number#54 ASC NULLS FIRST, ss_item_sk#51 ASC NULLS FIRST] - -(53) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#56, sr_ticket_number#57] - -(54) CometSort -Input [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [sr_item_sk#56, sr_ticket_number#57], [sr_ticket_number#57 ASC NULLS FIRST, sr_item_sk#56 ASC NULLS FIRST] - -(55) CometSortMergeJoin -Left output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -Right output [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [ss_ticket_number#54, ss_item_sk#51], [sr_ticket_number#57, sr_item_sk#56], Inner - -(56) CometProject -Input [7]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55, sr_item_sk#56, sr_ticket_number#57] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55], [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] - -(57) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] - -(58) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] -Right output [4]: [s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] -Arguments: [ss_store_sk#53], [s_store_sk#58], Inner, BuildRight - -(59) CometProject -Input [8]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55, s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16], [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] - -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(61) CometFilter -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Condition : isnotnull(i_item_sk#60) - -(62) CometProject -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65], [i_item_sk#60, i_current_price#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#62, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#63, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#64, 10, true, false, true) AS i_units#25, i_manager_id#65] - -(63) CometBroadcastExchange -Input [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] - -(64) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [ss_item_sk#51], [i_item_sk#60], Inner, BuildRight - -(65) CometProject -Input [12]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65], [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] - -(66) ReusedExchange [Reuses operator id: 28] -Output [5]: [c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] - -(67) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Right output [5]: [c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -Arguments: [ss_customer_sk#52], [c_customer_sk#66], Inner, BuildRight - -(68) CometProject -Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -Arguments: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68], [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] - -(69) CometColumnarToRow [codegen id : 2] -Input [13]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] - -(70) ReusedExchange [Reuses operator id: 36] -Output [4]: [ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] - -(71) BroadcastHashJoin [codegen id : 2] -Left keys [3]: [c_current_addr_sk#67, c_birth_country#68, s_zip#16] -Right keys [3]: [ca_address_sk#69, upper(ca_country#70), ca_zip#38] -Join type: Inner -Join condition: None - -(72) Project [codegen id : 2] -Output [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] -Input [17]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68, ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] - -(73) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [sum#71] -Results [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] - -(74) CometColumnarExchange -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(75) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] - -(76) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#41] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#41,17,2) AS netpaid#73] - -(77) HashAggregate [codegen id : 3] -Input [1]: [netpaid#73] -Keys: [] -Functions [1]: [partial_avg(netpaid#73)] -Aggregate Attributes [2]: [sum#74, count#75] -Results [2]: [sum#76, count#77] - -(78) CometColumnarExchange -Input [2]: [sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(79) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#76, count#77] - -(80) HashAggregate [codegen id : 4] -Input [2]: [sum#76, count#77] -Keys: [] -Functions [1]: [avg(netpaid#73)] -Aggregate Attributes [1]: [avg(netpaid#73)#78] -Results [1]: [(0.05 * avg(netpaid#73)#78) AS (0.05 * avg(netpaid))#79] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/simplified.txt deleted file mode 100644 index 62f492f632..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,99 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,s_store_name,paid] - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #5 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #12 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - InputAdapter - ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #4 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #5 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #7 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometScan [native_iceberg_compat] 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 (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] - CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] 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/q24/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/explain.txt deleted file mode 100644 index 17ef709df5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/explain.txt +++ /dev/null @@ -1,453 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (50) -+- CometSort (49) - +- CometColumnarExchange (48) - +- * Filter (47) - +- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) = pale ) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#19, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#21, 10, true, false, true) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Condition : ((isnotnull(c_customer_sk#26) AND isnotnull(c_current_addr_sk#27)) AND isnotnull(c_birth_country#30)) - -(27) CometProject -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [c_customer_sk#26, c_current_addr_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#28, 20, true, false, true) AS c_first_name#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#29, 30, true, false, true) AS c_last_name#32, c_birth_country#30] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(31) CometColumnarToRow [codegen id : 2] -Input [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Condition : ((isnotnull(ca_address_sk#33) AND isnotnull(ca_country#36)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true))) - -(34) CometProject -Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Arguments: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36], [ca_address_sk#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#34, 2, true, false, true) AS ca_state#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true) AS ca_zip#38, ca_country#36] - -(35) CometColumnarToRow [codegen id : 1] -Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] - -(36) BroadcastExchange -Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [3]: [c_current_addr_sk#27, c_birth_country#30, s_zip#16] -Right keys [3]: [ca_address_sk#33, upper(ca_country#36), ca_zip#38] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] -Input [17]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30, ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#39] -Results [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] - -(40) CometColumnarExchange -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#41] -Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#41,17,2) AS netpaid#42] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, netpaid#42] -Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] -Functions [1]: [partial_sum(netpaid#42)] -Aggregate Attributes [2]: [sum#43, isEmpty#44] -Results [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] - -(44) CometColumnarExchange -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] -Functions [1]: [sum(netpaid#42)] -Aggregate Attributes [1]: [sum(netpaid#42)#47] -Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, sum(netpaid#42)#47 AS paid#48] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) - -(48) CometColumnarExchange -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(49) CometSort -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Arguments: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48], [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] - -(50) CometColumnarToRow [codegen id : 5] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#49, [id=#50] -* HashAggregate (80) -+- * CometColumnarToRow (79) - +- CometColumnarExchange (78) - +- * HashAggregate (77) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- CometColumnarExchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * CometColumnarToRow (69) - : +- CometProject (68) - : +- CometBroadcastHashJoin (67) - : :- CometProject (65) - : : +- CometBroadcastHashJoin (64) - : : :- CometProject (59) - : : : +- CometBroadcastHashJoin (58) - : : : :- CometProject (56) - : : : : +- CometSortMergeJoin (55) - : : : : :- CometSort (52) - : : : : : +- ReusedExchange (51) - : : : : +- CometSort (54) - : : : : +- ReusedExchange (53) - : : : +- ReusedExchange (57) - : : +- CometBroadcastExchange (63) - : : +- CometProject (62) - : : +- CometFilter (61) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (60) - : +- ReusedExchange (66) - +- ReusedExchange (70) - - -(51) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] - -(52) CometSort -Input [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55], [ss_ticket_number#54 ASC NULLS FIRST, ss_item_sk#51 ASC NULLS FIRST] - -(53) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#56, sr_ticket_number#57] - -(54) CometSort -Input [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [sr_item_sk#56, sr_ticket_number#57], [sr_ticket_number#57 ASC NULLS FIRST, sr_item_sk#56 ASC NULLS FIRST] - -(55) CometSortMergeJoin -Left output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -Right output [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [ss_ticket_number#54, ss_item_sk#51], [sr_ticket_number#57, sr_item_sk#56], Inner - -(56) CometProject -Input [7]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55, sr_item_sk#56, sr_ticket_number#57] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55], [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] - -(57) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] - -(58) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] -Right output [4]: [s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] -Arguments: [ss_store_sk#53], [s_store_sk#58], Inner, BuildRight - -(59) CometProject -Input [8]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55, s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16], [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] - -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(61) CometFilter -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Condition : isnotnull(i_item_sk#60) - -(62) CometProject -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65], [i_item_sk#60, i_current_price#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#62, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#63, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#64, 10, true, false, true) AS i_units#25, i_manager_id#65] - -(63) CometBroadcastExchange -Input [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] - -(64) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [ss_item_sk#51], [i_item_sk#60], Inner, BuildRight - -(65) CometProject -Input [12]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65], [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] - -(66) ReusedExchange [Reuses operator id: 28] -Output [5]: [c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] - -(67) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Right output [5]: [c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -Arguments: [ss_customer_sk#52], [c_customer_sk#66], Inner, BuildRight - -(68) CometProject -Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -Arguments: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68], [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] - -(69) CometColumnarToRow [codegen id : 2] -Input [13]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] - -(70) ReusedExchange [Reuses operator id: 36] -Output [4]: [ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] - -(71) BroadcastHashJoin [codegen id : 2] -Left keys [3]: [c_current_addr_sk#67, c_birth_country#68, s_zip#16] -Right keys [3]: [ca_address_sk#69, upper(ca_country#70), ca_zip#38] -Join type: Inner -Join condition: None - -(72) Project [codegen id : 2] -Output [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] -Input [17]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68, ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] - -(73) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [sum#71] -Results [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] - -(74) CometColumnarExchange -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(75) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] - -(76) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#41] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#41,17,2) AS netpaid#73] - -(77) HashAggregate [codegen id : 3] -Input [1]: [netpaid#73] -Keys: [] -Functions [1]: [partial_avg(netpaid#73)] -Aggregate Attributes [2]: [sum#74, count#75] -Results [2]: [sum#76, count#77] - -(78) CometColumnarExchange -Input [2]: [sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(79) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#76, count#77] - -(80) HashAggregate [codegen id : 4] -Input [2]: [sum#76, count#77] -Keys: [] -Functions [1]: [avg(netpaid#73)] -Aggregate Attributes [1]: [avg(netpaid#73)#78] -Results [1]: [(0.05 * avg(netpaid#73)#78) AS (0.05 * avg(netpaid))#79] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/extended.txt deleted file mode 100644 index b9384ca04f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/extended.txt +++ /dev/null @@ -1,99 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometColumnarExchange - +- Filter - : +- Subquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 72 out of 88 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 62f492f632..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24/simplified.txt +++ /dev/null @@ -1,99 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,s_store_name,paid] - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #5 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #12 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - InputAdapter - ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #4 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #5 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #7 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometScan [native_iceberg_compat] 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 (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] - CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/explain.txt deleted file mode 100644 index c7e92e4c27..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/explain.txt +++ /dev/null @@ -1,479 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (78) -+- Union (77) - :- * HashAggregate (31) - : +- * CometColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- * HashAggregate (28) - : +- * Project (27) - : +- * BroadcastHashJoin Inner BuildRight (26) - : :- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (18) - : : +- * CometColumnarToRow (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.store (14) - : +- BroadcastExchange (25) - : +- * CometColumnarToRow (24) - : +- CometProject (23) - : +- CometFilter (22) - : +- CometNativeScan parquet spark_catalog.default.item (21) - :- * HashAggregate (54) - : +- * CometColumnarToRow (53) - : +- CometColumnarExchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * Project (40) - : : : +- * BroadcastHashJoin Inner BuildRight (39) - : : : :- * Project (37) - : : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : : :- * Filter (34) - : : : : : +- * ColumnarToRow (33) - : : : : : +- Scan parquet spark_catalog.default.store_sales (32) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- BroadcastExchange (45) - : : +- * CometColumnarToRow (44) - : : +- CometProject (43) - : : +- CometFilter (42) - : : +- CometNativeScan parquet spark_catalog.default.store (41) - : +- ReusedExchange (48) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- CometColumnarExchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * Project (66) - : +- * BroadcastHashJoin Inner BuildRight (65) - : :- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildRight (59) - : : : :- * Filter (57) - : : : : +- * ColumnarToRow (56) - : : : : +- Scan parquet spark_catalog.default.store_sales (55) - : : : +- ReusedExchange (58) - : : +- ReusedExchange (61) - : +- ReusedExchange (64) - +- BroadcastExchange (70) - +- * CometColumnarToRow (69) - +- CometFilter (68) - +- CometNativeScan parquet spark_catalog.default.item (67) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(3) Filter [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) - -(4) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = F) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = W)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = Primary )) AND isnotnull(cd_demo_sk#10)) - -(6) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [cd_demo_sk#10] - -(8) BroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 5] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] - -(11) ReusedExchange [Reuses operator id: 83] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#15, s_state#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#15, s_state#16] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#16, 2, true, false, true) = TN) AND isnotnull(s_store_sk#15)) - -(16) CometProject -Input [2]: [s_store_sk#15, s_state#16] -Arguments: [s_store_sk#15, s_state#17], [s_store_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#16, 2, true, false, true) AS s_state#17] - -(17) CometColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#15, s_state#17] - -(18) BroadcastExchange -Input [2]: [s_store_sk#15, s_state#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15, s_state#17] - -(21) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#18, i_item_id#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_item_sk#18, i_item_id#19] -Condition : isnotnull(i_item_sk#18) - -(23) CometProject -Input [2]: [i_item_sk#18, i_item_id#19] -Arguments: [i_item_sk#18, i_item_id#20], [i_item_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#19, 16, true, false, true) AS i_item_id#20] - -(24) CometColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#18, i_item_id#20] - -(25) BroadcastExchange -Input [2]: [i_item_sk#18, i_item_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#18] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 5] -Output [6]: [i_item_id#20 AS i_item_id#21, s_state#17 AS s_state#22, ss_quantity#4 AS agg1#23, ss_list_price#5 AS agg2#24, ss_coupon_amt#7 AS agg3#25, ss_sales_price#6 AS agg4#26] -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17, i_item_sk#18, i_item_id#20] - -(28) HashAggregate [codegen id : 5] -Input [6]: [i_item_id#21, s_state#22, agg1#23, agg2#24, agg3#25, agg4#26] -Keys [2]: [i_item_id#21, s_state#22] -Functions [4]: [partial_avg(agg1#23), partial_avg(UnscaledValue(agg2#24)), partial_avg(UnscaledValue(agg3#25)), partial_avg(UnscaledValue(agg4#26))] -Aggregate Attributes [8]: [sum#27, count#28, sum#29, count#30, sum#31, count#32, sum#33, count#34] -Results [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] - -(29) CometColumnarExchange -Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] -Arguments: hashpartitioning(i_item_id#21, s_state#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 6] -Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] - -(31) HashAggregate [codegen id : 6] -Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] -Keys [2]: [i_item_id#21, s_state#22] -Functions [4]: [avg(agg1#23), avg(UnscaledValue(agg2#24)), avg(UnscaledValue(agg3#25)), avg(UnscaledValue(agg4#26))] -Aggregate Attributes [4]: [avg(agg1#23)#43, avg(UnscaledValue(agg2#24))#44, avg(UnscaledValue(agg3#25))#45, avg(UnscaledValue(agg4#26))#46] -Results [7]: [i_item_id#21, s_state#22, 0 AS g_state#47, avg(agg1#23)#43 AS agg1#48, cast((avg(UnscaledValue(agg2#24))#44 / 100.0) as decimal(11,6)) AS agg2#49, cast((avg(UnscaledValue(agg3#25))#45 / 100.0) as decimal(11,6)) AS agg3#50, cast((avg(UnscaledValue(agg4#26))#46 / 100.0) as decimal(11,6)) AS agg4#51] - -(32) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(33) ColumnarToRow [codegen id : 11] -Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] - -(34) Filter [codegen id : 11] -Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_cdemo_sk#53) AND isnotnull(ss_store_sk#54)) AND isnotnull(ss_item_sk#52)) - -(35) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#60] - -(36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#53] -Right keys [1]: [cd_demo_sk#60] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 11] -Output [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -Input [9]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, cd_demo_sk#60] - -(38) ReusedExchange [Reuses operator id: 83] -Output [1]: [d_date_sk#61] - -(39) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#59] -Right keys [1]: [d_date_sk#61] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 11] -Output [6]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] -Input [8]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, d_date_sk#61] - -(41) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#62, s_state#63] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [s_store_sk#62, s_state#63] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#63, 2, true, false, true) = TN) AND isnotnull(s_store_sk#62)) - -(43) CometProject -Input [2]: [s_store_sk#62, s_state#63] -Arguments: [s_store_sk#62], [s_store_sk#62] - -(44) CometColumnarToRow [codegen id : 9] -Input [1]: [s_store_sk#62] - -(45) BroadcastExchange -Input [1]: [s_store_sk#62] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#54] -Right keys [1]: [s_store_sk#62] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 11] -Output [5]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] -Input [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, s_store_sk#62] - -(48) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#64, i_item_id#20] - -(49) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#52] -Right keys [1]: [i_item_sk#64] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 11] -Output [5]: [i_item_id#20, ss_quantity#55 AS agg1#65, ss_list_price#56 AS agg2#66, ss_coupon_amt#58 AS agg3#67, ss_sales_price#57 AS agg4#68] -Input [7]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, i_item_sk#64, i_item_id#20] - -(51) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#20, agg1#65, agg2#66, agg3#67, agg4#68] -Keys [1]: [i_item_id#20] -Functions [4]: [partial_avg(agg1#65), partial_avg(UnscaledValue(agg2#66)), partial_avg(UnscaledValue(agg3#67)), partial_avg(UnscaledValue(agg4#68))] -Aggregate Attributes [8]: [sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76] -Results [9]: [i_item_id#20, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] - -(52) CometColumnarExchange -Input [9]: [i_item_id#20, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] -Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(53) CometColumnarToRow [codegen id : 12] -Input [9]: [i_item_id#20, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] - -(54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#20, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] -Keys [1]: [i_item_id#20] -Functions [4]: [avg(agg1#65), avg(UnscaledValue(agg2#66)), avg(UnscaledValue(agg3#67)), avg(UnscaledValue(agg4#68))] -Aggregate Attributes [4]: [avg(agg1#65)#85, avg(UnscaledValue(agg2#66))#86, avg(UnscaledValue(agg3#67))#87, avg(UnscaledValue(agg4#68))#88] -Results [7]: [i_item_id#20, null AS s_state#89, 1 AS g_state#90, avg(agg1#65)#85 AS agg1#91, cast((avg(UnscaledValue(agg2#66))#86 / 100.0) as decimal(11,6)) AS agg2#92, cast((avg(UnscaledValue(agg3#67))#87 / 100.0) as decimal(11,6)) AS agg3#93, cast((avg(UnscaledValue(agg4#68))#88 / 100.0) as decimal(11,6)) AS agg4#94] - -(55) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#95, ss_cdemo_sk#96, ss_store_sk#97, ss_quantity#98, ss_list_price#99, ss_sales_price#100, ss_coupon_amt#101, ss_sold_date_sk#102] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#102), dynamicpruningexpression(ss_sold_date_sk#102 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(56) ColumnarToRow [codegen id : 17] -Input [8]: [ss_item_sk#95, ss_cdemo_sk#96, ss_store_sk#97, ss_quantity#98, ss_list_price#99, ss_sales_price#100, ss_coupon_amt#101, ss_sold_date_sk#102] - -(57) Filter [codegen id : 17] -Input [8]: [ss_item_sk#95, ss_cdemo_sk#96, ss_store_sk#97, ss_quantity#98, ss_list_price#99, ss_sales_price#100, ss_coupon_amt#101, ss_sold_date_sk#102] -Condition : ((isnotnull(ss_cdemo_sk#96) AND isnotnull(ss_store_sk#97)) AND isnotnull(ss_item_sk#95)) - -(58) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#103] - -(59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#96] -Right keys [1]: [cd_demo_sk#103] -Join type: Inner -Join condition: None - -(60) Project [codegen id : 17] -Output [7]: [ss_item_sk#95, ss_store_sk#97, ss_quantity#98, ss_list_price#99, ss_sales_price#100, ss_coupon_amt#101, ss_sold_date_sk#102] -Input [9]: [ss_item_sk#95, ss_cdemo_sk#96, ss_store_sk#97, ss_quantity#98, ss_list_price#99, ss_sales_price#100, ss_coupon_amt#101, ss_sold_date_sk#102, cd_demo_sk#103] - -(61) ReusedExchange [Reuses operator id: 83] -Output [1]: [d_date_sk#104] - -(62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#102] -Right keys [1]: [d_date_sk#104] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 17] -Output [6]: [ss_item_sk#95, ss_store_sk#97, ss_quantity#98, ss_list_price#99, ss_sales_price#100, ss_coupon_amt#101] -Input [8]: [ss_item_sk#95, ss_store_sk#97, ss_quantity#98, ss_list_price#99, ss_sales_price#100, ss_coupon_amt#101, ss_sold_date_sk#102, d_date_sk#104] - -(64) ReusedExchange [Reuses operator id: 45] -Output [1]: [s_store_sk#105] - -(65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#97] -Right keys [1]: [s_store_sk#105] -Join type: Inner -Join condition: None - -(66) Project [codegen id : 17] -Output [5]: [ss_item_sk#95, ss_quantity#98, ss_list_price#99, ss_sales_price#100, ss_coupon_amt#101] -Input [7]: [ss_item_sk#95, ss_store_sk#97, ss_quantity#98, ss_list_price#99, ss_sales_price#100, ss_coupon_amt#101, s_store_sk#105] - -(67) CometNativeScan parquet spark_catalog.default.item -Output [1]: [i_item_sk#106] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(68) CometFilter -Input [1]: [i_item_sk#106] -Condition : isnotnull(i_item_sk#106) - -(69) CometColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#106] - -(70) BroadcastExchange -Input [1]: [i_item_sk#106] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -(71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#95] -Right keys [1]: [i_item_sk#106] -Join type: Inner -Join condition: None - -(72) Project [codegen id : 17] -Output [4]: [ss_quantity#98 AS agg1#107, ss_list_price#99 AS agg2#108, ss_coupon_amt#101 AS agg3#109, ss_sales_price#100 AS agg4#110] -Input [6]: [ss_item_sk#95, ss_quantity#98, ss_list_price#99, ss_sales_price#100, ss_coupon_amt#101, i_item_sk#106] - -(73) HashAggregate [codegen id : 17] -Input [4]: [agg1#107, agg2#108, agg3#109, agg4#110] -Keys: [] -Functions [4]: [partial_avg(agg1#107), partial_avg(UnscaledValue(agg2#108)), partial_avg(UnscaledValue(agg3#109)), partial_avg(UnscaledValue(agg4#110))] -Aggregate Attributes [8]: [sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118] -Results [8]: [sum#119, count#120, sum#121, count#122, sum#123, count#124, sum#125, count#126] - -(74) CometColumnarExchange -Input [8]: [sum#119, count#120, sum#121, count#122, sum#123, count#124, sum#125, count#126] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(75) CometColumnarToRow [codegen id : 18] -Input [8]: [sum#119, count#120, sum#121, count#122, sum#123, count#124, sum#125, count#126] - -(76) HashAggregate [codegen id : 18] -Input [8]: [sum#119, count#120, sum#121, count#122, sum#123, count#124, sum#125, count#126] -Keys: [] -Functions [4]: [avg(agg1#107), avg(UnscaledValue(agg2#108)), avg(UnscaledValue(agg3#109)), avg(UnscaledValue(agg4#110))] -Aggregate Attributes [4]: [avg(agg1#107)#127, avg(UnscaledValue(agg2#108))#128, avg(UnscaledValue(agg3#109))#129, avg(UnscaledValue(agg4#110))#130] -Results [7]: [null AS i_item_id#131, null AS s_state#132, 1 AS g_state#133, avg(agg1#107)#127 AS agg1#134, cast((avg(UnscaledValue(agg2#108))#128 / 100.0) as decimal(11,6)) AS agg2#135, cast((avg(UnscaledValue(agg3#109))#129 / 100.0) as decimal(11,6)) AS agg3#136, cast((avg(UnscaledValue(agg4#110))#130 / 100.0) as decimal(11,6)) AS agg4#137] - -(77) Union - -(78) TakeOrderedAndProject -Input [7]: [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] -Arguments: 100, [i_item_id#21 ASC NULLS FIRST, s_state#22 ASC NULLS FIRST], [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometNativeScan parquet spark_catalog.default.date_dim (79) - - -(79) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#138] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(80) CometFilter -Input [2]: [d_date_sk#14, d_year#138] -Condition : ((isnotnull(d_year#138) AND (d_year#138 = 1998)) AND isnotnull(d_date_sk#14)) - -(81) CometProject -Input [2]: [d_date_sk#14, d_year#138] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(82) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(83) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#9 - -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#102 IN dynamicpruning#9 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/simplified.txt deleted file mode 100644 index c7093301af..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/simplified.txt +++ /dev/null @@ -1,122 +0,0 @@ -TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - Union - WholeStageCodegen (6) - HashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,s_state] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Filter [ss_cdemo_sk,ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - WholeStageCodegen (12) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #6 - WholeStageCodegen (11) - HashAggregate [i_item_id,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Filter [ss_cdemo_sk,ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [cd_demo_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) - HashAggregate [sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (17) - HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Filter [ss_cdemo_sk,ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [cd_demo_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [s_store_sk] #7 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/explain.txt deleted file mode 100644 index 397a3eb10c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,437 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (71) -+- CometTakeOrderedAndProject (70) - +- CometUnion (69) - :- CometHashAggregate (29) - : +- CometExchange (28) - : +- CometHashAggregate (27) - : +- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometProject (20) - : : +- CometBroadcastHashJoin (19) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (18) - : : +- CometProject (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - : +- CometBroadcastExchange (24) - : +- CometProject (23) - : +- CometFilter (22) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) - :- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (37) - : : : +- CometBroadcastHashJoin (36) - : : : :- CometProject (34) - : : : : +- CometBroadcastHashJoin (33) - : : : : :- CometFilter (31) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (30) - : : : : +- ReusedExchange (32) - : : : +- ReusedExchange (35) - : : +- CometBroadcastExchange (41) - : : +- CometProject (40) - : : +- CometFilter (39) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (38) - : +- ReusedExchange (44) - +- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometProject (65) - +- CometBroadcastHashJoin (64) - :- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (57) - : : +- CometBroadcastHashJoin (56) - : : :- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometFilter (51) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : : : +- ReusedExchange (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (58) - +- CometBroadcastExchange (63) - +- CometFilter (62) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (61) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = F) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = W)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = Primary )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#16, s_state#17] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) = TN) AND isnotnull(s_store_sk#16)) - -(17) CometProject -Input [2]: [s_store_sk#16, s_state#17] -Arguments: [s_store_sk#16, s_state#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) AS s_state#18] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_state#18] -Arguments: [s_store_sk#16, s_state#18] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Right output [2]: [s_store_sk#16, s_state#18] -Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16, s_state#18] -Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_item_id#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_item_sk#19, i_item_id#20] -Condition : isnotnull(i_item_sk#19) - -(23) CometProject -Input [2]: [i_item_sk#19, i_item_id#20] -Arguments: [i_item_sk#19, i_item_id#21], [i_item_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#20, 16, true, false, true) AS i_item_id#21] - -(24) CometBroadcastExchange -Input [2]: [i_item_sk#19, i_item_id#21] -Arguments: [i_item_sk#19, i_item_id#21] - -(25) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] -Right output [2]: [i_item_sk#19, i_item_id#21] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(26) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18, i_item_sk#19, i_item_id#21] -Arguments: [i_item_id#22, s_state#23, agg1#24, agg2#25, agg3#26, agg4#27], [i_item_id#21 AS i_item_id#22, s_state#18 AS s_state#23, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] - -(27) CometHashAggregate -Input [6]: [i_item_id#22, s_state#23, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [2]: [i_item_id#22, s_state#23] -Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] - -(28) CometExchange -Input [10]: [i_item_id#22, s_state#23, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Arguments: hashpartitioning(i_item_id#22, s_state#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [10]: [i_item_id#22, s_state#23, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Keys [2]: [i_item_id#22, s_state#23] -Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#43), dynamicpruningexpression(ss_sold_date_sk#43 IN dynamicpruning#44)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(31) CometFilter -Input [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Condition : ((isnotnull(ss_cdemo_sk#37) AND isnotnull(ss_store_sk#38)) AND isnotnull(ss_item_sk#36)) - -(32) ReusedExchange [Reuses operator id: 6] -Output [1]: [cd_demo_sk#45] - -(33) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Right output [1]: [cd_demo_sk#45] -Arguments: [ss_cdemo_sk#37], [cd_demo_sk#45], Inner, BuildRight - -(34) CometProject -Input [9]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43, cd_demo_sk#45] -Arguments: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43], [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] - -(35) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#46] - -(36) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Right output [1]: [d_date_sk#46] -Arguments: [ss_sold_date_sk#43], [d_date_sk#46], Inner, BuildRight - -(37) CometProject -Input [8]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43, d_date_sk#46] -Arguments: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42], [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#47, s_state#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(39) CometFilter -Input [2]: [s_store_sk#47, s_state#48] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#48, 2, true, false, true) = TN) AND isnotnull(s_store_sk#47)) - -(40) CometProject -Input [2]: [s_store_sk#47, s_state#48] -Arguments: [s_store_sk#47], [s_store_sk#47] - -(41) CometBroadcastExchange -Input [1]: [s_store_sk#47] -Arguments: [s_store_sk#47] - -(42) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] -Right output [1]: [s_store_sk#47] -Arguments: [ss_store_sk#38], [s_store_sk#47], Inner, BuildRight - -(43) CometProject -Input [7]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, s_store_sk#47] -Arguments: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42], [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] - -(44) ReusedExchange [Reuses operator id: 24] -Output [2]: [i_item_sk#49, i_item_id#21] - -(45) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] -Right output [2]: [i_item_sk#49, i_item_id#21] -Arguments: [ss_item_sk#36], [i_item_sk#49], Inner, BuildRight - -(46) CometProject -Input [7]: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, i_item_sk#49, i_item_id#21] -Arguments: [i_item_id#21, agg1#50, agg2#51, agg3#52, agg4#53], [i_item_id#21, ss_quantity#39 AS agg1#50, ss_list_price#40 AS agg2#51, ss_coupon_amt#42 AS agg3#52, ss_sales_price#41 AS agg4#53] - -(47) CometHashAggregate -Input [5]: [i_item_id#21, agg1#50, agg2#51, agg3#52, agg4#53] -Keys [1]: [i_item_id#21] -Functions [4]: [partial_avg(agg1#50), partial_avg(UnscaledValue(agg2#51)), partial_avg(UnscaledValue(agg3#52)), partial_avg(UnscaledValue(agg4#53))] - -(48) CometExchange -Input [9]: [i_item_id#21, sum#54, count#55, sum#56, count#57, sum#58, count#59, sum#60, count#61] -Arguments: hashpartitioning(i_item_id#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(49) CometHashAggregate -Input [9]: [i_item_id#21, sum#54, count#55, sum#56, count#57, sum#58, count#59, sum#60, count#61] -Keys [1]: [i_item_id#21] -Functions [4]: [avg(agg1#50), avg(UnscaledValue(agg2#51)), avg(UnscaledValue(agg3#52)), avg(UnscaledValue(agg4#53))] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#69), dynamicpruningexpression(ss_sold_date_sk#69 IN dynamicpruning#70)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [8]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] -Condition : ((isnotnull(ss_cdemo_sk#63) AND isnotnull(ss_store_sk#64)) AND isnotnull(ss_item_sk#62)) - -(52) ReusedExchange [Reuses operator id: 6] -Output [1]: [cd_demo_sk#71] - -(53) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] -Right output [1]: [cd_demo_sk#71] -Arguments: [ss_cdemo_sk#63], [cd_demo_sk#71], Inner, BuildRight - -(54) CometProject -Input [9]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69, cd_demo_sk#71] -Arguments: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69], [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] - -(55) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#72] - -(56) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] -Right output [1]: [d_date_sk#72] -Arguments: [ss_sold_date_sk#69], [d_date_sk#72], Inner, BuildRight - -(57) CometProject -Input [8]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69, d_date_sk#72] -Arguments: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68], [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] - -(58) ReusedExchange [Reuses operator id: 41] -Output [1]: [s_store_sk#73] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] -Right output [1]: [s_store_sk#73] -Arguments: [ss_store_sk#64], [s_store_sk#73], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, s_store_sk#73] -Arguments: [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68], [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] - -(61) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#74] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(62) CometFilter -Input [1]: [i_item_sk#74] -Condition : isnotnull(i_item_sk#74) - -(63) CometBroadcastExchange -Input [1]: [i_item_sk#74] -Arguments: [i_item_sk#74] - -(64) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] -Right output [1]: [i_item_sk#74] -Arguments: [ss_item_sk#62], [i_item_sk#74], Inner, BuildRight - -(65) CometProject -Input [6]: [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, i_item_sk#74] -Arguments: [agg1#75, agg2#76, agg3#77, agg4#78], [ss_quantity#65 AS agg1#75, ss_list_price#66 AS agg2#76, ss_coupon_amt#68 AS agg3#77, ss_sales_price#67 AS agg4#78] - -(66) CometHashAggregate -Input [4]: [agg1#75, agg2#76, agg3#77, agg4#78] -Keys: [] -Functions [4]: [partial_avg(agg1#75), partial_avg(UnscaledValue(agg2#76)), partial_avg(UnscaledValue(agg3#77)), partial_avg(UnscaledValue(agg4#78))] - -(67) CometExchange -Input [8]: [sum#79, count#80, sum#81, count#82, sum#83, count#84, sum#85, count#86] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(68) CometHashAggregate -Input [8]: [sum#79, count#80, sum#81, count#82, sum#83, count#84, sum#85, count#86] -Keys: [] -Functions [4]: [avg(agg1#75), avg(UnscaledValue(agg2#76)), avg(UnscaledValue(agg3#77)), avg(UnscaledValue(agg4#78))] - -(69) CometUnion -Child 0 Input [7]: [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] -Child 1 Input [7]: [i_item_id#21, s_state#92, g_state#93, agg1#94, agg2#95, agg3#96, agg4#97] -Child 2 Input [7]: [i_item_id#98, s_state#99, g_state#100, agg1#101, agg2#102, agg3#103, agg4#104] - -(70) CometTakeOrderedAndProject -Input [7]: [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#22 ASC NULLS FIRST,s_state#23 ASC NULLS FIRST], output=[i_item_id#22,s_state#23,g_state#87,agg1#88,agg2#89,agg3#90,agg4#91]), [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91], 100, 0, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] - -(71) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) - -(74) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(75) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(76) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#43 IN dynamicpruning#9 - -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#69 IN dynamicpruning#9 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 0d18ca5626..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,83 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometUnion [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange [i_item_id,s_state] #1 - CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] - CometProject [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] - 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_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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_state] #5 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange [i_item_id] #7 - CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,sum,count,sum,count,sum,count,sum,count] - CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] - 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_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_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_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 [native_iceberg_compat] 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 [s_store_sk] #8 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange #9 - CometHashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count] - CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] - 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_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_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_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 [native_iceberg_compat] 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 [i_item_sk] #10 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/explain.txt deleted file mode 100644 index 397a3eb10c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/explain.txt +++ /dev/null @@ -1,437 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (71) -+- CometTakeOrderedAndProject (70) - +- CometUnion (69) - :- CometHashAggregate (29) - : +- CometExchange (28) - : +- CometHashAggregate (27) - : +- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometProject (20) - : : +- CometBroadcastHashJoin (19) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (18) - : : +- CometProject (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - : +- CometBroadcastExchange (24) - : +- CometProject (23) - : +- CometFilter (22) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) - :- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (37) - : : : +- CometBroadcastHashJoin (36) - : : : :- CometProject (34) - : : : : +- CometBroadcastHashJoin (33) - : : : : :- CometFilter (31) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (30) - : : : : +- ReusedExchange (32) - : : : +- ReusedExchange (35) - : : +- CometBroadcastExchange (41) - : : +- CometProject (40) - : : +- CometFilter (39) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (38) - : +- ReusedExchange (44) - +- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometProject (65) - +- CometBroadcastHashJoin (64) - :- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (57) - : : +- CometBroadcastHashJoin (56) - : : :- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometFilter (51) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : : : +- ReusedExchange (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (58) - +- CometBroadcastExchange (63) - +- CometFilter (62) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (61) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = F) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = W)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = Primary )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#16, s_state#17] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) = TN) AND isnotnull(s_store_sk#16)) - -(17) CometProject -Input [2]: [s_store_sk#16, s_state#17] -Arguments: [s_store_sk#16, s_state#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) AS s_state#18] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_state#18] -Arguments: [s_store_sk#16, s_state#18] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Right output [2]: [s_store_sk#16, s_state#18] -Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16, s_state#18] -Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_item_id#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_item_sk#19, i_item_id#20] -Condition : isnotnull(i_item_sk#19) - -(23) CometProject -Input [2]: [i_item_sk#19, i_item_id#20] -Arguments: [i_item_sk#19, i_item_id#21], [i_item_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#20, 16, true, false, true) AS i_item_id#21] - -(24) CometBroadcastExchange -Input [2]: [i_item_sk#19, i_item_id#21] -Arguments: [i_item_sk#19, i_item_id#21] - -(25) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] -Right output [2]: [i_item_sk#19, i_item_id#21] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(26) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18, i_item_sk#19, i_item_id#21] -Arguments: [i_item_id#22, s_state#23, agg1#24, agg2#25, agg3#26, agg4#27], [i_item_id#21 AS i_item_id#22, s_state#18 AS s_state#23, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] - -(27) CometHashAggregate -Input [6]: [i_item_id#22, s_state#23, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [2]: [i_item_id#22, s_state#23] -Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] - -(28) CometExchange -Input [10]: [i_item_id#22, s_state#23, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Arguments: hashpartitioning(i_item_id#22, s_state#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [10]: [i_item_id#22, s_state#23, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Keys [2]: [i_item_id#22, s_state#23] -Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#43), dynamicpruningexpression(ss_sold_date_sk#43 IN dynamicpruning#44)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(31) CometFilter -Input [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Condition : ((isnotnull(ss_cdemo_sk#37) AND isnotnull(ss_store_sk#38)) AND isnotnull(ss_item_sk#36)) - -(32) ReusedExchange [Reuses operator id: 6] -Output [1]: [cd_demo_sk#45] - -(33) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Right output [1]: [cd_demo_sk#45] -Arguments: [ss_cdemo_sk#37], [cd_demo_sk#45], Inner, BuildRight - -(34) CometProject -Input [9]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43, cd_demo_sk#45] -Arguments: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43], [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] - -(35) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#46] - -(36) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Right output [1]: [d_date_sk#46] -Arguments: [ss_sold_date_sk#43], [d_date_sk#46], Inner, BuildRight - -(37) CometProject -Input [8]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43, d_date_sk#46] -Arguments: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42], [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#47, s_state#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(39) CometFilter -Input [2]: [s_store_sk#47, s_state#48] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#48, 2, true, false, true) = TN) AND isnotnull(s_store_sk#47)) - -(40) CometProject -Input [2]: [s_store_sk#47, s_state#48] -Arguments: [s_store_sk#47], [s_store_sk#47] - -(41) CometBroadcastExchange -Input [1]: [s_store_sk#47] -Arguments: [s_store_sk#47] - -(42) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] -Right output [1]: [s_store_sk#47] -Arguments: [ss_store_sk#38], [s_store_sk#47], Inner, BuildRight - -(43) CometProject -Input [7]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, s_store_sk#47] -Arguments: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42], [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] - -(44) ReusedExchange [Reuses operator id: 24] -Output [2]: [i_item_sk#49, i_item_id#21] - -(45) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] -Right output [2]: [i_item_sk#49, i_item_id#21] -Arguments: [ss_item_sk#36], [i_item_sk#49], Inner, BuildRight - -(46) CometProject -Input [7]: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, i_item_sk#49, i_item_id#21] -Arguments: [i_item_id#21, agg1#50, agg2#51, agg3#52, agg4#53], [i_item_id#21, ss_quantity#39 AS agg1#50, ss_list_price#40 AS agg2#51, ss_coupon_amt#42 AS agg3#52, ss_sales_price#41 AS agg4#53] - -(47) CometHashAggregate -Input [5]: [i_item_id#21, agg1#50, agg2#51, agg3#52, agg4#53] -Keys [1]: [i_item_id#21] -Functions [4]: [partial_avg(agg1#50), partial_avg(UnscaledValue(agg2#51)), partial_avg(UnscaledValue(agg3#52)), partial_avg(UnscaledValue(agg4#53))] - -(48) CometExchange -Input [9]: [i_item_id#21, sum#54, count#55, sum#56, count#57, sum#58, count#59, sum#60, count#61] -Arguments: hashpartitioning(i_item_id#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(49) CometHashAggregate -Input [9]: [i_item_id#21, sum#54, count#55, sum#56, count#57, sum#58, count#59, sum#60, count#61] -Keys [1]: [i_item_id#21] -Functions [4]: [avg(agg1#50), avg(UnscaledValue(agg2#51)), avg(UnscaledValue(agg3#52)), avg(UnscaledValue(agg4#53))] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#69), dynamicpruningexpression(ss_sold_date_sk#69 IN dynamicpruning#70)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [8]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] -Condition : ((isnotnull(ss_cdemo_sk#63) AND isnotnull(ss_store_sk#64)) AND isnotnull(ss_item_sk#62)) - -(52) ReusedExchange [Reuses operator id: 6] -Output [1]: [cd_demo_sk#71] - -(53) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] -Right output [1]: [cd_demo_sk#71] -Arguments: [ss_cdemo_sk#63], [cd_demo_sk#71], Inner, BuildRight - -(54) CometProject -Input [9]: [ss_item_sk#62, ss_cdemo_sk#63, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69, cd_demo_sk#71] -Arguments: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69], [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] - -(55) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#72] - -(56) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69] -Right output [1]: [d_date_sk#72] -Arguments: [ss_sold_date_sk#69], [d_date_sk#72], Inner, BuildRight - -(57) CometProject -Input [8]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, ss_sold_date_sk#69, d_date_sk#72] -Arguments: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68], [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] - -(58) ReusedExchange [Reuses operator id: 41] -Output [1]: [s_store_sk#73] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] -Right output [1]: [s_store_sk#73] -Arguments: [ss_store_sk#64], [s_store_sk#73], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_item_sk#62, ss_store_sk#64, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, s_store_sk#73] -Arguments: [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68], [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] - -(61) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#74] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(62) CometFilter -Input [1]: [i_item_sk#74] -Condition : isnotnull(i_item_sk#74) - -(63) CometBroadcastExchange -Input [1]: [i_item_sk#74] -Arguments: [i_item_sk#74] - -(64) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68] -Right output [1]: [i_item_sk#74] -Arguments: [ss_item_sk#62], [i_item_sk#74], Inner, BuildRight - -(65) CometProject -Input [6]: [ss_item_sk#62, ss_quantity#65, ss_list_price#66, ss_sales_price#67, ss_coupon_amt#68, i_item_sk#74] -Arguments: [agg1#75, agg2#76, agg3#77, agg4#78], [ss_quantity#65 AS agg1#75, ss_list_price#66 AS agg2#76, ss_coupon_amt#68 AS agg3#77, ss_sales_price#67 AS agg4#78] - -(66) CometHashAggregate -Input [4]: [agg1#75, agg2#76, agg3#77, agg4#78] -Keys: [] -Functions [4]: [partial_avg(agg1#75), partial_avg(UnscaledValue(agg2#76)), partial_avg(UnscaledValue(agg3#77)), partial_avg(UnscaledValue(agg4#78))] - -(67) CometExchange -Input [8]: [sum#79, count#80, sum#81, count#82, sum#83, count#84, sum#85, count#86] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(68) CometHashAggregate -Input [8]: [sum#79, count#80, sum#81, count#82, sum#83, count#84, sum#85, count#86] -Keys: [] -Functions [4]: [avg(agg1#75), avg(UnscaledValue(agg2#76)), avg(UnscaledValue(agg3#77)), avg(UnscaledValue(agg4#78))] - -(69) CometUnion -Child 0 Input [7]: [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] -Child 1 Input [7]: [i_item_id#21, s_state#92, g_state#93, agg1#94, agg2#95, agg3#96, agg4#97] -Child 2 Input [7]: [i_item_id#98, s_state#99, g_state#100, agg1#101, agg2#102, agg3#103, agg4#104] - -(70) CometTakeOrderedAndProject -Input [7]: [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#22 ASC NULLS FIRST,s_state#23 ASC NULLS FIRST], output=[i_item_id#22,s_state#23,g_state#87,agg1#88,agg2#89,agg3#90,agg4#91]), [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91], 100, 0, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] - -(71) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#22, s_state#23, g_state#87, agg1#88, agg2#89, agg3#90, agg4#91] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) - -(74) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(75) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(76) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#43 IN dynamicpruning#9 - -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#69 IN dynamicpruning#9 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/extended.txt deleted file mode 100644 index affbc9ef37..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/extended.txt +++ /dev/null @@ -1,99 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 91 out of 95 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 0d18ca5626..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a/simplified.txt +++ /dev/null @@ -1,83 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometUnion [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange [i_item_id,s_state] #1 - CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] - CometProject [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] - 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_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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_state] #5 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange [i_item_id] #7 - CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,sum,count,sum,count,sum,count,sum,count] - CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] - 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_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_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_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 [native_iceberg_compat] 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 [s_store_sk] #8 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange #9 - CometHashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count] - CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] - 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_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_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_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 [native_iceberg_compat] 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 [i_item_sk] #10 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/explain.txt deleted file mode 100644 index 587f232ea8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/explain.txt +++ /dev/null @@ -1,231 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (35) -+- CometSort (34) - +- CometColumnarExchange (33) - +- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * Filter (25) - : +- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (30) - +- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometNativeScan parquet spark_catalog.default.customer (26) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 40] -Output [1]: [d_date_sk#7] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_county#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [s_store_sk#8, s_county#9] -Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND isnotnull(s_store_sk#8)) - -(9) CometProject -Input [2]: [s_store_sk#8, s_county#9] -Arguments: [s_store_sk#8], [s_store_sk#8] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#8] - -(11) BroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] - -(14) CometNativeScan parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : ((((isnotnull(hd_vehicle_count#13) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#11, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#11, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#12 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#13 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#10)) - -(16) CometProject -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Arguments: [hd_demo_sk#10], [hd_demo_sk#10] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#10] - -(18) BroadcastExchange -Input [1]: [hd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#10] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [2]: [ss_customer_sk#1, ss_ticket_number#4] -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] - -(21) HashAggregate [codegen id : 4] -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#14] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(22) CometColumnarExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(24) HashAggregate [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#16] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] - -(25) Filter [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) - -(26) CometNativeScan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(27) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(28) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] - -(29) CometColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(30) BroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#18] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 6] -Output [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(33) CometColumnarExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(34) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST] - -(35) CometColumnarToRow [codegen id : 7] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (40) -+- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometNativeScan parquet spark_catalog.default.date_dim (36) - - -(36) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#27, d_dom#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [d_date_sk#7, d_year#27, d_dom#28] -Condition : (((((d_dom#28 >= 1) AND (d_dom#28 <= 3)) OR ((d_dom#28 >= 25) AND (d_dom#28 <= 28))) AND d_year#27 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(38) CometProject -Input [3]: [d_date_sk#7, d_year#27, d_dom#28] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(39) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(40) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/simplified.txt deleted file mode 100644 index 63c6ac3645..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/simplified.txt +++ /dev/null @@ -1,59 +0,0 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometNativeScan 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/q34.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/explain.txt deleted file mode 100644 index 50af5f4f75..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometSort (32) - +- CometExchange (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (24) - : +- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_county#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#10, s_county#11] -Condition : ((isnotnull(s_county#11) AND (s_county#11 = Williamson County)) AND isnotnull(s_store_sk#10)) - -(11) CometProject -Input [2]: [s_store_sk#10, s_county#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : ((((isnotnull(hd_vehicle_count#15) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#14 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#15 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#12)) - -(17) CometProject -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#12], [hd_demo_sk#12] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#12] -Arguments: [hd_demo_sk#12] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Right output [1]: [hd_demo_sk#12] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight - -(20) CometProject -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] -Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] - -(21) CometHashAggregate -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] - -(22) CometExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] - -(24) CometFilter -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(27) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(29) CometBroadcastHashJoin -Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Right output [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -(31) CometExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(32) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST] - -(33) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(36) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(37) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(38) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/simplified.txt deleted file mode 100644 index d57afd4fcf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 - CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] - CometExchange [ss_ticket_number,ss_customer_sk] #2 - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - 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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/explain.txt deleted file mode 100644 index 50af5f4f75..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometSort (32) - +- CometExchange (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (24) - : +- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_county#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#10, s_county#11] -Condition : ((isnotnull(s_county#11) AND (s_county#11 = Williamson County)) AND isnotnull(s_store_sk#10)) - -(11) CometProject -Input [2]: [s_store_sk#10, s_county#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : ((((isnotnull(hd_vehicle_count#15) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#14 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#15 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#12)) - -(17) CometProject -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#12], [hd_demo_sk#12] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#12] -Arguments: [hd_demo_sk#12] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Right output [1]: [hd_demo_sk#12] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight - -(20) CometProject -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] -Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] - -(21) CometHashAggregate -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] - -(22) CometExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] - -(24) CometFilter -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(27) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(29) CometBroadcastHashJoin -Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Right output [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -(31) CometExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(32) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST] - -(33) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(36) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(37) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(38) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/extended.txt deleted file mode 100644 index 3d9d23d2cc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/extended.txt +++ /dev/null @@ -1,41 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d57afd4fcf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 - CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] - CometExchange [ss_ticket_number,ss_customer_sk] #2 - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - 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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/explain.txt deleted file mode 100644 index 0b89e61f7b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/explain.txt +++ /dev/null @@ -1,295 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (45) -+- * HashAggregate (44) - +- * CometColumnarToRow (43) - +- CometColumnarExchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometNativeScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (38) - +- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometNativeScan parquet spark_catalog.default.customer_demographics (34) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(4) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] - -(6) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#9] - -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None - -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#6] -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] - -(9) BroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#6] -Join type: LeftSemi -Join condition: None - -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] - -(13) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#12] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#10] -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] - -(16) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(17) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#10] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] - -(20) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#15] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#13] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] - -(23) BroadcastExchange -Input [1]: [cs_ship_customer_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#13] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(25) Filter [codegen id : 9] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(26) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(27) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : isnotnull(ca_address_sk#16) - -(29) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16, ca_state#18], [ca_address_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#17, 2, true, false, true) AS ca_state#18] - -(30) CometColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#16, ca_state#18] - -(31) BroadcastExchange -Input [2]: [ca_address_sk#16, ca_state#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#16] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#18] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#16, ca_state#18] - -(34) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(35) CometFilter -Input [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Condition : isnotnull(cd_demo_sk#19) - -(36) CometProject -Input [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Arguments: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24], [cd_demo_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#20, 1, true, false, true) AS cd_gender#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#21, 1, true, false, true) AS cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] - -(37) CometColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] - -(38) BroadcastExchange -Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#19] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 9] -Output [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Input [8]: [c_current_cdemo_sk#4, ca_state#18, cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] - -(41) HashAggregate [codegen id : 9] -Input [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#22), partial_max(cd_dep_count#22), partial_sum(cd_dep_count#22), partial_avg(cd_dep_employed_count#23), partial_max(cd_dep_employed_count#23), partial_sum(cd_dep_employed_count#23), partial_avg(cd_dep_college_count#24), partial_max(cd_dep_college_count#24), partial_sum(cd_dep_college_count#24)] -Aggregate Attributes [13]: [count#27, sum#28, count#29, max#30, sum#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39] -Results [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] - -(42) CometColumnarExchange -Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] -Arguments: hashpartitioning(ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) CometColumnarToRow [codegen id : 10] -Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] - -(44) HashAggregate [codegen id : 10] -Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] -Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Functions [10]: [count(1), avg(cd_dep_count#22), max(cd_dep_count#22), sum(cd_dep_count#22), avg(cd_dep_employed_count#23), max(cd_dep_employed_count#23), sum(cd_dep_employed_count#23), avg(cd_dep_college_count#24), max(cd_dep_college_count#24), sum(cd_dep_college_count#24)] -Aggregate Attributes [10]: [count(1)#53, avg(cd_dep_count#22)#54, max(cd_dep_count#22)#55, sum(cd_dep_count#22)#56, avg(cd_dep_employed_count#23)#57, max(cd_dep_employed_count#23)#58, sum(cd_dep_employed_count#23)#59, avg(cd_dep_college_count#24)#60, max(cd_dep_college_count#24)#61, sum(cd_dep_college_count#24)#62] -Results [18]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, count(1)#53 AS cnt1#63, avg(cd_dep_count#22)#54 AS avg(cd_dep_count)#64, max(cd_dep_count#22)#55 AS max(cd_dep_count)#65, sum(cd_dep_count#22)#56 AS sum(cd_dep_count)#66, cd_dep_employed_count#23, count(1)#53 AS cnt2#67, avg(cd_dep_employed_count#23)#57 AS avg(cd_dep_employed_count)#68, max(cd_dep_employed_count#23)#58 AS max(cd_dep_employed_count)#69, sum(cd_dep_employed_count#23)#59 AS sum(cd_dep_employed_count)#70, cd_dep_college_count#24, count(1)#53 AS cnt3#71, avg(cd_dep_college_count#24)#60 AS avg(cd_dep_college_count)#72, max(cd_dep_college_count#24)#61 AS max(cd_dep_college_count)#73, sum(cd_dep_college_count#24)#62 AS sum(cd_dep_college_count)#74] - -(45) TakeOrderedAndProject -Input [18]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cnt1#63, avg(cd_dep_count)#64, max(cd_dep_count)#65, sum(cd_dep_count)#66, cd_dep_employed_count#23, cnt2#67, avg(cd_dep_employed_count)#68, max(cd_dep_employed_count)#69, sum(cd_dep_employed_count)#70, cd_dep_college_count#24, cnt3#71, avg(cd_dep_college_count)#72, max(cd_dep_college_count)#73, sum(cd_dep_college_count)#74] -Arguments: 100, [ca_state#18 ASC NULLS FIRST, cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_dep_count#22 ASC NULLS FIRST, cd_dep_employed_count#23 ASC NULLS FIRST, cd_dep_college_count#24 ASC NULLS FIRST], [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cnt1#63, avg(cd_dep_count)#64, max(cd_dep_count)#65, sum(cd_dep_count)#66, cd_dep_employed_count#23, cnt2#67, avg(cd_dep_employed_count)#68, max(cd_dep_employed_count)#69, sum(cd_dep_employed_count)#70, cd_dep_college_count#24, cnt3#71, avg(cd_dep_college_count)#72, max(cd_dep_college_count)#73, sum(cd_dep_college_count)#74] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (50) -+- * CometColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometNativeScan parquet spark_catalog.default.date_dim (46) - - -(46) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#75, d_qoy#76] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_date_sk#9, d_year#75, d_qoy#76] -Condition : ((((isnotnull(d_year#75) AND isnotnull(d_qoy#76)) AND (d_year#75 = 2002)) AND (d_qoy#76 < 4)) AND isnotnull(d_date_sk#9)) - -(48) CometProject -Input [3]: [d_date_sk#9, d_year#75, d_qoy#76] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(49) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(50) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/simplified.txt deleted file mode 100644 index 0fb95102c3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/simplified.txt +++ /dev/null @@ -1,77 +0,0 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (10) - HashAggregate [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] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [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,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan 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/q35.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/explain.txt deleted file mode 100644 index b43ed9b192..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * Filter (27) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(6) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: [d_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#9] -Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -Arguments: [ss_customer_sk#6], [ss_customer_sk#6] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: [ss_customer_sk#6] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Right output [1]: [ss_customer_sk#6] -Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#15] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Right output [1]: [d_date_sk#15] -Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] -Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#12] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#12] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#19] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#16] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#16] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(27) Filter [codegen id : 5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(28) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : isnotnull(ca_address_sk#20) - -(31) CometProject -Input [2]: [ca_address_sk#20, ca_state#21] -Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#21, 2, true, false, true) AS ca_state#22] - -(32) CometColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#20, ca_state#22] - -(33) BroadcastExchange -Input [2]: [ca_address_sk#20, ca_state#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(34) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, ca_state#22] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#22] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(37) CometFilter -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#23) - -(38) CometProject -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#24, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(39) CometColumnarToRow [codegen id : 4] -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(40) BroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(41) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#23] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 5] -Output [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [8]: [c_current_cdemo_sk#4, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(43) HashAggregate [codegen id : 5] -Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] -Aggregate Attributes [13]: [count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] -Results [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] - -(44) CometColumnarExchange -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 6] -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] - -(46) HashAggregate [codegen id : 6] -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] -Aggregate Attributes [10]: [count(1)#57, avg(cd_dep_count#26)#58, max(cd_dep_count#26)#59, sum(cd_dep_count#26)#60, avg(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, sum(cd_dep_employed_count#27)#63, avg(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, sum(cd_dep_college_count#28)#66] -Results [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, count(1)#57 AS cnt1#67, avg(cd_dep_count#26)#58 AS avg(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, sum(cd_dep_count#26)#60 AS sum(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, avg(cd_dep_employed_count#27)#61 AS avg(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, sum(cd_dep_employed_count#27)#63 AS sum(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, avg(cd_dep_college_count#28)#64 AS avg(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, sum(cd_dep_college_count#28)#66 AS sum(cd_dep_college_count)#78] - -(47) TakeOrderedAndProject -Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] -Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(52) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/simplified.txt deleted file mode 100644 index fea6a32741..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (6) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] 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/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/explain.txt deleted file mode 100644 index b43ed9b192..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * Filter (27) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(6) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: [d_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#9] -Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -Arguments: [ss_customer_sk#6], [ss_customer_sk#6] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: [ss_customer_sk#6] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Right output [1]: [ss_customer_sk#6] -Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#15] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Right output [1]: [d_date_sk#15] -Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] -Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#12] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#12] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#19] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#16] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#16] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(27) Filter [codegen id : 5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(28) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : isnotnull(ca_address_sk#20) - -(31) CometProject -Input [2]: [ca_address_sk#20, ca_state#21] -Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#21, 2, true, false, true) AS ca_state#22] - -(32) CometColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#20, ca_state#22] - -(33) BroadcastExchange -Input [2]: [ca_address_sk#20, ca_state#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(34) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, ca_state#22] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#22] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(37) CometFilter -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#23) - -(38) CometProject -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#24, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(39) CometColumnarToRow [codegen id : 4] -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(40) BroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(41) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#23] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 5] -Output [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [8]: [c_current_cdemo_sk#4, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(43) HashAggregate [codegen id : 5] -Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] -Aggregate Attributes [13]: [count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] -Results [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] - -(44) CometColumnarExchange -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 6] -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] - -(46) HashAggregate [codegen id : 6] -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] -Aggregate Attributes [10]: [count(1)#57, avg(cd_dep_count#26)#58, max(cd_dep_count#26)#59, sum(cd_dep_count#26)#60, avg(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, sum(cd_dep_employed_count#27)#63, avg(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, sum(cd_dep_college_count#28)#66] -Results [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, count(1)#57 AS cnt1#67, avg(cd_dep_count#26)#58 AS avg(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, sum(cd_dep_count#26)#60 AS sum(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, avg(cd_dep_employed_count#27)#61 AS avg(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, sum(cd_dep_employed_count#27)#63 AS sum(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, avg(cd_dep_college_count#28)#64 AS avg(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, sum(cd_dep_college_count#28)#66 AS sum(cd_dep_college_count)#78] - -(47) TakeOrderedAndProject -Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] -Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(52) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/extended.txt deleted file mode 100644 index a6f33d6f7e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/extended.txt +++ /dev/null @@ -1,63 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index fea6a32741..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (6) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/explain.txt deleted file mode 100644 index 4ff78399c8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/explain.txt +++ /dev/null @@ -1,281 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : +- BroadcastExchange (9) - : : : +- * Project (8) - : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : :- * ColumnarToRow (5) - : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (6) - : : +- BroadcastExchange (22) - : : +- Union (21) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * ColumnarToRow (17) - : : : +- Scan parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (18) - : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.customer_address (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.customer_demographics (32) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(4) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] - -(6) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#7] - -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#4] -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] - -(9) BroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#4] -Join type: LeftSemi -Join condition: None - -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#6)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] - -(13) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#10] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#8 AS customsk#11] -Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#10] - -(16) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#6)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] - -(18) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#14] - -(19) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#12 AS customsk#15] -Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#14] - -(21) Union - -(22) BroadcastExchange -Input [1]: [customsk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(23) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customsk#11] -Join type: LeftSemi -Join condition: None - -(24) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(25) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : isnotnull(ca_address_sk#16) - -(27) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16, ca_state#18], [ca_address_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#17, 2, true, false, true) AS ca_state#18] - -(28) CometColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#16, ca_state#18] - -(29) BroadcastExchange -Input [2]: [ca_address_sk#16, ca_state#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(30) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#16] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, ca_state#18] -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#16, ca_state#18] - -(32) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(33) CometFilter -Input [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Condition : isnotnull(cd_demo_sk#19) - -(34) CometProject -Input [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Arguments: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24], [cd_demo_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#20, 1, true, false, true) AS cd_gender#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#21, 1, true, false, true) AS cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] - -(35) CometColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] - -(36) BroadcastExchange -Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#19] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 9] -Output [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Input [8]: [c_current_cdemo_sk#2, ca_state#18, cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] - -(39) HashAggregate [codegen id : 9] -Input [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#22), partial_max(cd_dep_count#22), partial_sum(cd_dep_count#22), partial_avg(cd_dep_employed_count#23), partial_max(cd_dep_employed_count#23), partial_sum(cd_dep_employed_count#23), partial_avg(cd_dep_college_count#24), partial_max(cd_dep_college_count#24), partial_sum(cd_dep_college_count#24)] -Aggregate Attributes [13]: [count#27, sum#28, count#29, max#30, sum#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39] -Results [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] - -(40) CometColumnarExchange -Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] -Arguments: hashpartitioning(ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(41) CometColumnarToRow [codegen id : 10] -Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] - -(42) HashAggregate [codegen id : 10] -Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] -Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Functions [10]: [count(1), avg(cd_dep_count#22), max(cd_dep_count#22), sum(cd_dep_count#22), avg(cd_dep_employed_count#23), max(cd_dep_employed_count#23), sum(cd_dep_employed_count#23), avg(cd_dep_college_count#24), max(cd_dep_college_count#24), sum(cd_dep_college_count#24)] -Aggregate Attributes [10]: [count(1)#53, avg(cd_dep_count#22)#54, max(cd_dep_count#22)#55, sum(cd_dep_count#22)#56, avg(cd_dep_employed_count#23)#57, max(cd_dep_employed_count#23)#58, sum(cd_dep_employed_count#23)#59, avg(cd_dep_college_count#24)#60, max(cd_dep_college_count#24)#61, sum(cd_dep_college_count#24)#62] -Results [18]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, count(1)#53 AS cnt1#63, avg(cd_dep_count#22)#54 AS avg(cd_dep_count)#64, max(cd_dep_count#22)#55 AS max(cd_dep_count)#65, sum(cd_dep_count#22)#56 AS sum(cd_dep_count)#66, cd_dep_employed_count#23, count(1)#53 AS cnt2#67, avg(cd_dep_employed_count#23)#57 AS avg(cd_dep_employed_count)#68, max(cd_dep_employed_count#23)#58 AS max(cd_dep_employed_count)#69, sum(cd_dep_employed_count#23)#59 AS sum(cd_dep_employed_count)#70, cd_dep_college_count#24, count(1)#53 AS cnt3#71, avg(cd_dep_college_count#24)#60 AS avg(cd_dep_college_count)#72, max(cd_dep_college_count#24)#61 AS max(cd_dep_college_count)#73, sum(cd_dep_college_count#24)#62 AS sum(cd_dep_college_count)#74] - -(43) TakeOrderedAndProject -Input [18]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cnt1#63, avg(cd_dep_count)#64, max(cd_dep_count)#65, sum(cd_dep_count)#66, cd_dep_employed_count#23, cnt2#67, avg(cd_dep_employed_count)#68, max(cd_dep_employed_count)#69, sum(cd_dep_employed_count)#70, cd_dep_college_count#24, cnt3#71, avg(cd_dep_college_count)#72, max(cd_dep_college_count)#73, sum(cd_dep_college_count)#74] -Arguments: 100, [ca_state#18 ASC NULLS FIRST, cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_dep_count#22 ASC NULLS FIRST, cd_dep_employed_count#23 ASC NULLS FIRST, cd_dep_college_count#24 ASC NULLS FIRST], [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cnt1#63, avg(cd_dep_count)#64, max(cd_dep_count)#65, sum(cd_dep_count)#66, cd_dep_employed_count#23, cnt2#67, avg(cd_dep_employed_count)#68, max(cd_dep_employed_count)#69, sum(cd_dep_employed_count)#70, cd_dep_college_count#24, cnt3#71, avg(cd_dep_college_count)#72, max(cd_dep_college_count)#73, sum(cd_dep_college_count)#74] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometNativeScan parquet spark_catalog.default.date_dim (44) - - -(44) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#75, d_qoy#76] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [3]: [d_date_sk#7, d_year#75, d_qoy#76] -Condition : ((((isnotnull(d_year#75) AND isnotnull(d_qoy#76)) AND (d_year#75 = 1999)) AND (d_qoy#76 < 4)) AND isnotnull(d_date_sk#7)) - -(46) CometProject -Input [3]: [d_date_sk#7, d_year#75, d_qoy#76] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(48) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/simplified.txt deleted file mode 100644 index bed891a0a3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (10) - HashAggregate [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] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [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,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,customsk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - Union - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan 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.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/explain.txt deleted file mode 100644 index e22a5c492c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,267 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometHashAggregate (38) - +- CometExchange (37) - +- CometHashAggregate (36) - +- CometProject (35) - +- CometBroadcastHashJoin (34) - :- CometProject (29) - : +- CometBroadcastHashJoin (28) - : :- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometBroadcastHashJoin (11) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : +- CometBroadcastExchange (10) - : : : +- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : +- CometBroadcastExchange (21) - : : +- CometUnion (20) - : : :- CometProject (15) - : : : +- CometBroadcastHashJoin (14) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (12) - : : : +- ReusedExchange (13) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (17) - : +- CometBroadcastExchange (27) - : +- CometProject (26) - : +- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) - +- CometBroadcastExchange (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (30) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) - -(6) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#4], [ss_customer_sk#4] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: [ss_customer_sk#4] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ss_customer_sk#4] -Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(13) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] -Arguments: [customsk#14], [ws_bill_customer_sk#10 AS customsk#14] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] -ReadSchema: struct - -(17) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#18] - -(18) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Right output [1]: [d_date_sk#18] -Arguments: [cs_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight - -(19) CometProject -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#18] -Arguments: [customsk#19], [cs_ship_customer_sk#15 AS customsk#19] - -(20) CometUnion -Child 0 Input [1]: [customsk#14] -Child 1 Input [1]: [customsk#19] - -(21) CometBroadcastExchange -Input [1]: [customsk#14] -Arguments: [customsk#14] - -(22) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [customsk#14] -Arguments: [c_customer_sk#1], [customsk#14], LeftSemi, BuildRight - -(23) CometProject -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : isnotnull(ca_address_sk#20) - -(26) CometProject -Input [2]: [ca_address_sk#20, ca_state#21] -Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#21, 2, true, false, true) AS ca_state#22] - -(27) CometBroadcastExchange -Input [2]: [ca_address_sk#20, ca_state#22] -Arguments: [ca_address_sk#20, ca_state#22] - -(28) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [2]: [ca_address_sk#20, ca_state#22] -Arguments: [c_current_addr_sk#3], [ca_address_sk#20], Inner, BuildRight - -(29) CometProject -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20, ca_state#22] -Arguments: [c_current_cdemo_sk#2, ca_state#22], [c_current_cdemo_sk#2, ca_state#22] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#23) - -(32) CometProject -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#24, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(33) CometBroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(34) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, ca_state#22] -Right output [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#23], Inner, BuildRight - -(35) CometProject -Input [8]: [c_current_cdemo_sk#2, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(36) CometHashAggregate -Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] - -(37) CometExchange -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] -Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(38) CometHashAggregate -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] - -(39) CometTakeOrderedAndProject -Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#22 ASC NULLS FIRST,cd_gender#29 ASC NULLS FIRST,cd_marital_status#30 ASC NULLS FIRST,cd_dep_count#26 ASC NULLS FIRST,cd_dep_employed_count#27 ASC NULLS FIRST,cd_dep_college_count#28 ASC NULLS FIRST], output=[ca_state#22,cd_gender#29,cd_marital_status#30,cd_dep_count#26,cnt1#44,avg(cd_dep_count)#45,max(cd_dep_count)#46,sum(cd_dep_count)#47,cd_dep_employed_count#27,cnt2#48,avg(cd_dep_employed_count)#49,max(cd_dep_employed_count)#50,sum(cd_dep_employed_count)#51,cd_dep_college_count#28,cnt3#52,avg(cd_dep_college_count)#53,max(cd_dep_college_count)#54,sum(cd_dep_college_count)#55]), [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55], 100, 0, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] - -(40) CometColumnarToRow [codegen id : 1] -Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) - -(43) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(45) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 69e8d4868c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - CometHashAggregate [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - CometExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - 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,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_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] - CometProject [c_current_cdemo_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [customsk] #5 - CometUnion [customsk] - CometProject [ws_bill_customer_sk] [customsk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 [ca_address_sk,ca_state] #6 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] 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/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/explain.txt deleted file mode 100644 index e22a5c492c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/explain.txt +++ /dev/null @@ -1,267 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometHashAggregate (38) - +- CometExchange (37) - +- CometHashAggregate (36) - +- CometProject (35) - +- CometBroadcastHashJoin (34) - :- CometProject (29) - : +- CometBroadcastHashJoin (28) - : :- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometBroadcastHashJoin (11) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : +- CometBroadcastExchange (10) - : : : +- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : +- CometBroadcastExchange (21) - : : +- CometUnion (20) - : : :- CometProject (15) - : : : +- CometBroadcastHashJoin (14) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (12) - : : : +- ReusedExchange (13) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (17) - : +- CometBroadcastExchange (27) - : +- CometProject (26) - : +- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) - +- CometBroadcastExchange (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (30) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) - -(6) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#4], [ss_customer_sk#4] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: [ss_customer_sk#4] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ss_customer_sk#4] -Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(13) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] -Arguments: [customsk#14], [ws_bill_customer_sk#10 AS customsk#14] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] -ReadSchema: struct - -(17) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#18] - -(18) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Right output [1]: [d_date_sk#18] -Arguments: [cs_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight - -(19) CometProject -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#18] -Arguments: [customsk#19], [cs_ship_customer_sk#15 AS customsk#19] - -(20) CometUnion -Child 0 Input [1]: [customsk#14] -Child 1 Input [1]: [customsk#19] - -(21) CometBroadcastExchange -Input [1]: [customsk#14] -Arguments: [customsk#14] - -(22) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [customsk#14] -Arguments: [c_customer_sk#1], [customsk#14], LeftSemi, BuildRight - -(23) CometProject -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : isnotnull(ca_address_sk#20) - -(26) CometProject -Input [2]: [ca_address_sk#20, ca_state#21] -Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#21, 2, true, false, true) AS ca_state#22] - -(27) CometBroadcastExchange -Input [2]: [ca_address_sk#20, ca_state#22] -Arguments: [ca_address_sk#20, ca_state#22] - -(28) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [2]: [ca_address_sk#20, ca_state#22] -Arguments: [c_current_addr_sk#3], [ca_address_sk#20], Inner, BuildRight - -(29) CometProject -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20, ca_state#22] -Arguments: [c_current_cdemo_sk#2, ca_state#22], [c_current_cdemo_sk#2, ca_state#22] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#23) - -(32) CometProject -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#24, 1, true, false, true) AS cd_gender#29, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#25, 1, true, false, true) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(33) CometBroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(34) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, ca_state#22] -Right output [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#23], Inner, BuildRight - -(35) CometProject -Input [8]: [c_current_cdemo_sk#2, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(36) CometHashAggregate -Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] - -(37) CometExchange -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] -Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(38) CometHashAggregate -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] - -(39) CometTakeOrderedAndProject -Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#22 ASC NULLS FIRST,cd_gender#29 ASC NULLS FIRST,cd_marital_status#30 ASC NULLS FIRST,cd_dep_count#26 ASC NULLS FIRST,cd_dep_employed_count#27 ASC NULLS FIRST,cd_dep_college_count#28 ASC NULLS FIRST], output=[ca_state#22,cd_gender#29,cd_marital_status#30,cd_dep_count#26,cnt1#44,avg(cd_dep_count)#45,max(cd_dep_count)#46,sum(cd_dep_count)#47,cd_dep_employed_count#27,cnt2#48,avg(cd_dep_employed_count)#49,max(cd_dep_employed_count)#50,sum(cd_dep_employed_count)#51,cd_dep_college_count#28,cnt3#52,avg(cd_dep_college_count)#53,max(cd_dep_college_count)#54,sum(cd_dep_college_count)#55]), [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55], 100, 0, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] - -(40) CometColumnarToRow [codegen id : 1] -Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) - -(43) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(45) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/extended.txt deleted file mode 100644 index 2cdc75e15e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/extended.txt +++ /dev/null @@ -1,56 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 69e8d4868c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - CometHashAggregate [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - CometExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - 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,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_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] - CometProject [c_current_cdemo_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [customsk] #5 - CometUnion [customsk] - CometProject [ws_bill_customer_sk] [customsk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 [ca_address_sk,ca_state] #6 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/explain.txt deleted file mode 100644 index 7911d900e1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- CometExchange (43) - +- CometHashAggregate (42) - +- CometColumnarExchange (41) - +- * HashAggregate (40) - +- Union (39) - :- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.item (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.store (14) - :- * HashAggregate (31) - : +- * CometColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- * HashAggregate (28) - : +- * HashAggregate (27) - : +- * CometColumnarToRow (26) - : +- ReusedExchange (25) - +- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * HashAggregate (34) - +- * CometColumnarToRow (33) - +- ReusedExchange (32) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) - -(4) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#7] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] - -(7) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Condition : isnotnull(i_item_sk#8) - -(9) CometProject -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Arguments: [i_item_sk#8, i_class#11, i_category#12], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#12] - -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#8, i_class#11, i_category#12] - -(11) BroadcastExchange -Input [3]: [i_item_sk#8, i_class#11, i_category#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#8] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#8, i_class#11, i_category#12] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#13, s_state#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#13, s_state#14] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#14, 2, true, false, true) = TN) AND isnotnull(s_store_sk#13)) - -(16) CometProject -Input [2]: [s_store_sk#13, s_state#14] -Arguments: [s_store_sk#13], [s_store_sk#13] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#13] - -(18) BroadcastExchange -Input [1]: [s_store_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#13] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12, s_store_sk#13] - -(21) HashAggregate [codegen id : 4] -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] -Keys [2]: [i_category#12, i_class#11] -Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum#15, sum#16] -Results [4]: [i_category#12, i_class#11, sum#17, sum#18] - -(22) CometColumnarExchange -Input [4]: [i_category#12, i_class#11, sum#17, sum#18] -Arguments: hashpartitioning(i_category#12, i_class#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 5] -Input [4]: [i_category#12, i_class#11, sum#17, sum#18] - -(24) HashAggregate [codegen id : 5] -Input [4]: [i_category#12, i_class#11, sum#17, sum#18] -Keys [2]: [i_category#12, i_class#11] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#19, sum(UnscaledValue(ss_ext_sales_price#3))#20] -Results [6]: [cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#19,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#20,17,2)) as decimal(38,20)) AS gross_margin#21, i_category#12 AS i_category#22, i_class#11 AS i_class#23, 0 AS t_category#24, 0 AS t_class#25, 0 AS lochierarchy#26] - -(25) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#12, i_class#11, sum#27, sum#28] - -(26) CometColumnarToRow [codegen id : 10] -Input [4]: [i_category#12, i_class#11, sum#27, sum#28] - -(27) HashAggregate [codegen id : 10] -Input [4]: [i_category#12, i_class#11, sum#27, sum#28] -Keys [2]: [i_category#12, i_class#11] -Functions [2]: [sum(UnscaledValue(ss_net_profit#29)), sum(UnscaledValue(ss_ext_sales_price#30))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#29))#31, sum(UnscaledValue(ss_ext_sales_price#30))#32] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#29))#31,17,2) AS ss_net_profit#33, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#30))#32,17,2) AS ss_ext_sales_price#34, i_category#12] - -(28) HashAggregate [codegen id : 10] -Input [3]: [ss_net_profit#33, ss_ext_sales_price#34, i_category#12] -Keys [1]: [i_category#12] -Functions [2]: [partial_sum(ss_net_profit#33), partial_sum(ss_ext_sales_price#34)] -Aggregate Attributes [4]: [sum#35, isEmpty#36, sum#37, isEmpty#38] -Results [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] - -(29) CometColumnarExchange -Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] -Arguments: hashpartitioning(i_category#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 11] -Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] - -(31) HashAggregate [codegen id : 11] -Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] -Keys [1]: [i_category#12] -Functions [2]: [sum(ss_net_profit#33), sum(ss_ext_sales_price#34)] -Aggregate Attributes [2]: [sum(ss_net_profit#33)#43, sum(ss_ext_sales_price#34)#44] -Results [6]: [cast((sum(ss_net_profit#33)#43 / sum(ss_ext_sales_price#34)#44) as decimal(38,20)) AS gross_margin#45, i_category#12, null AS i_class#46, 0 AS t_category#47, 1 AS t_class#48, 1 AS lochierarchy#49] - -(32) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#12, i_class#11, sum#50, sum#51] - -(33) CometColumnarToRow [codegen id : 16] -Input [4]: [i_category#12, i_class#11, sum#50, sum#51] - -(34) HashAggregate [codegen id : 16] -Input [4]: [i_category#12, i_class#11, sum#50, sum#51] -Keys [2]: [i_category#12, i_class#11] -Functions [2]: [sum(UnscaledValue(ss_net_profit#52)), sum(UnscaledValue(ss_ext_sales_price#53))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#52))#31, sum(UnscaledValue(ss_ext_sales_price#53))#32] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#52))#31,17,2) AS ss_net_profit#54, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#53))#32,17,2) AS ss_ext_sales_price#55] - -(35) HashAggregate [codegen id : 16] -Input [2]: [ss_net_profit#54, ss_ext_sales_price#55] -Keys: [] -Functions [2]: [partial_sum(ss_net_profit#54), partial_sum(ss_ext_sales_price#55)] -Aggregate Attributes [4]: [sum#56, isEmpty#57, sum#58, isEmpty#59] -Results [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] - -(36) CometColumnarExchange -Input [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(37) CometColumnarToRow [codegen id : 17] -Input [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] - -(38) HashAggregate [codegen id : 17] -Input [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] -Keys: [] -Functions [2]: [sum(ss_net_profit#54), sum(ss_ext_sales_price#55)] -Aggregate Attributes [2]: [sum(ss_net_profit#54)#64, sum(ss_ext_sales_price#55)#65] -Results [6]: [cast((sum(ss_net_profit#54)#64 / sum(ss_ext_sales_price#55)#65) as decimal(38,20)) AS gross_margin#66, null AS i_category#67, null AS i_class#68, 1 AS t_category#69, 1 AS t_class#70, 2 AS lochierarchy#71] - -(39) Union - -(40) HashAggregate [codegen id : 18] -Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Keys [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Functions: [] -Aggregate Attributes: [] -Results [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] - -(41) CometColumnarExchange -Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Arguments: hashpartitioning(gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(42) CometHashAggregate -Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Keys [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Functions: [] - -(43) CometExchange -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] -Arguments: hashpartitioning(lochierarchy#26, _w0#72, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(44) CometSort -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] -Arguments: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72], [lochierarchy#26 ASC NULLS FIRST, _w0#72 ASC NULLS FIRST, gross_margin#21 ASC NULLS FIRST] - -(45) CometColumnarToRow [codegen id : 19] -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] - -(46) Window -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72] -Arguments: [rank(gross_margin#21) windowspecdefinition(lochierarchy#26, _w0#72, gross_margin#21 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#73], [lochierarchy#26, _w0#72], [gross_margin#21 ASC NULLS FIRST] - -(47) Project [codegen id : 20] -Output [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] -Input [6]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#72, rank_within_parent#73] - -(48) TakeOrderedAndProject -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] -Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN i_category#22 END ASC NULLS FIRST, rank_within_parent#73 ASC NULLS FIRST], [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#73] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometNativeScan parquet spark_catalog.default.date_dim (49) - - -(49) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#74] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [2]: [d_date_sk#7, d_year#74] -Condition : ((isnotnull(d_year#74) AND (d_year#74 = 2001)) AND isnotnull(d_date_sk#7)) - -(51) CometProject -Input [2]: [d_date_sk#7, d_year#74] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(53) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/simplified.txt deleted file mode 100644 index 65728fc6f2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/simplified.txt +++ /dev/null @@ -1,81 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (20) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] - CometColumnarExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - WholeStageCodegen (18) - HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #3 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - WholeStageCodegen (11) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #7 - WholeStageCodegen (10) - HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (16) - HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/explain.txt deleted file mode 100644 index 2562693a01..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,275 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- Window (41) - +- * CometColumnarToRow (40) - +- CometSort (39) - +- CometExchange (38) - +- CometHashAggregate (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometUnion (34) - :- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometHashAggregate (25) - : +- ReusedExchange (24) - +- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometHashAggregate (30) - +- ReusedExchange (29) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4], [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#9, i_class#10, i_category#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Condition : isnotnull(i_item_sk#9) - -(11) CometProject -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Arguments: [i_item_sk#9, i_class#12, i_category#13], [i_item_sk#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#13] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [i_item_sk#9, i_class#12, i_category#13] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Right output [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#9], Inner, BuildRight - -(14) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) = TN) AND isnotnull(s_store_sk#14)) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14], [s_store_sk#14] - -(18) CometBroadcastExchange -Input [1]: [s_store_sk#14] -Arguments: [s_store_sk#14] - -(19) CometBroadcastHashJoin -Left output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -Right output [1]: [s_store_sk#14] -Arguments: [ss_store_sk#2], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13, s_store_sk#14] -Arguments: [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] - -(21) CometHashAggregate -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(22) CometExchange -Input [4]: [i_category#13, i_class#12, sum#16, sum#17] -Arguments: hashpartitioning(i_category#13, i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [4]: [i_category#13, i_class#12, sum#16, sum#17] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] - -(24) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#13, i_class#12, sum#18, sum#19] - -(25) CometHashAggregate -Input [4]: [i_category#13, i_class#12, sum#18, sum#19] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [sum(UnscaledValue(ss_net_profit#20)), sum(UnscaledValue(ss_ext_sales_price#21))] - -(26) CometHashAggregate -Input [3]: [ss_net_profit#22, ss_ext_sales_price#23, i_category#13] -Keys [1]: [i_category#13] -Functions [2]: [partial_sum(ss_net_profit#22), partial_sum(ss_ext_sales_price#23)] - -(27) CometExchange -Input [5]: [i_category#13, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(i_category#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [5]: [i_category#13, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [i_category#13] -Functions [2]: [sum(ss_net_profit#22), sum(ss_ext_sales_price#23)] - -(29) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#13, i_class#12, sum#28, sum#29] - -(30) CometHashAggregate -Input [4]: [i_category#13, i_class#12, sum#28, sum#29] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [sum(UnscaledValue(ss_net_profit#30)), sum(UnscaledValue(ss_ext_sales_price#31))] - -(31) CometHashAggregate -Input [2]: [ss_net_profit#32, ss_ext_sales_price#33] -Keys: [] -Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] - -(32) CometExchange -Input [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometHashAggregate -Input [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] -Keys: [] -Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] - -(34) CometUnion -Child 0 Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] -Child 1 Input [6]: [gross_margin#44, i_category#13, i_class#45, t_category#46, t_class#47, lochierarchy#48] -Child 2 Input [6]: [gross_margin#49, i_category#50, i_class#51, t_category#52, t_class#53, lochierarchy#54] - -(35) CometHashAggregate -Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] -Keys [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] -Functions: [] - -(36) CometExchange -Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] -Arguments: hashpartitioning(gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(37) CometHashAggregate -Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] -Keys [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] -Functions: [] - -(38) CometExchange -Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] -Arguments: hashpartitioning(lochierarchy#43, _w0#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(39) CometSort -Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] -Arguments: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55], [lochierarchy#43 ASC NULLS FIRST, _w0#55 ASC NULLS FIRST, gross_margin#38 ASC NULLS FIRST] - -(40) CometColumnarToRow [codegen id : 1] -Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] - -(41) Window -Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] -Arguments: [rank(gross_margin#38) windowspecdefinition(lochierarchy#43, _w0#55, gross_margin#38 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#56], [lochierarchy#43, _w0#55], [gross_margin#38 ASC NULLS FIRST] - -(42) Project [codegen id : 2] -Output [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, rank_within_parent#56] -Input [6]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55, rank_within_parent#56] - -(43) TakeOrderedAndProject -Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, rank_within_parent#56] -Arguments: 100, [lochierarchy#43 DESC NULLS LAST, CASE WHEN (lochierarchy#43 = 0) THEN i_category#39 END ASC NULLS FIRST, rank_within_parent#56 ASC NULLS FIRST], [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, rank_within_parent#56] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) - - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(46) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(48) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/simplified.txt deleted file mode 100644 index e91b278c14..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,55 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (2) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] - CometExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometUnion [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - 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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange [i_category] #8 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,i_category,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange #9 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_category,i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/explain.txt deleted file mode 100644 index 2562693a01..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/explain.txt +++ /dev/null @@ -1,275 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- Window (41) - +- * CometColumnarToRow (40) - +- CometSort (39) - +- CometExchange (38) - +- CometHashAggregate (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometUnion (34) - :- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometHashAggregate (25) - : +- ReusedExchange (24) - +- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometHashAggregate (30) - +- ReusedExchange (29) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4], [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#9, i_class#10, i_category#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Condition : isnotnull(i_item_sk#9) - -(11) CometProject -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Arguments: [i_item_sk#9, i_class#12, i_category#13], [i_item_sk#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#13] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [i_item_sk#9, i_class#12, i_category#13] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Right output [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#9], Inner, BuildRight - -(14) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) = TN) AND isnotnull(s_store_sk#14)) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14], [s_store_sk#14] - -(18) CometBroadcastExchange -Input [1]: [s_store_sk#14] -Arguments: [s_store_sk#14] - -(19) CometBroadcastHashJoin -Left output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -Right output [1]: [s_store_sk#14] -Arguments: [ss_store_sk#2], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13, s_store_sk#14] -Arguments: [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] - -(21) CometHashAggregate -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(22) CometExchange -Input [4]: [i_category#13, i_class#12, sum#16, sum#17] -Arguments: hashpartitioning(i_category#13, i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [4]: [i_category#13, i_class#12, sum#16, sum#17] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] - -(24) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#13, i_class#12, sum#18, sum#19] - -(25) CometHashAggregate -Input [4]: [i_category#13, i_class#12, sum#18, sum#19] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [sum(UnscaledValue(ss_net_profit#20)), sum(UnscaledValue(ss_ext_sales_price#21))] - -(26) CometHashAggregate -Input [3]: [ss_net_profit#22, ss_ext_sales_price#23, i_category#13] -Keys [1]: [i_category#13] -Functions [2]: [partial_sum(ss_net_profit#22), partial_sum(ss_ext_sales_price#23)] - -(27) CometExchange -Input [5]: [i_category#13, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(i_category#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [5]: [i_category#13, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [i_category#13] -Functions [2]: [sum(ss_net_profit#22), sum(ss_ext_sales_price#23)] - -(29) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#13, i_class#12, sum#28, sum#29] - -(30) CometHashAggregate -Input [4]: [i_category#13, i_class#12, sum#28, sum#29] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [sum(UnscaledValue(ss_net_profit#30)), sum(UnscaledValue(ss_ext_sales_price#31))] - -(31) CometHashAggregate -Input [2]: [ss_net_profit#32, ss_ext_sales_price#33] -Keys: [] -Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] - -(32) CometExchange -Input [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometHashAggregate -Input [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] -Keys: [] -Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] - -(34) CometUnion -Child 0 Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] -Child 1 Input [6]: [gross_margin#44, i_category#13, i_class#45, t_category#46, t_class#47, lochierarchy#48] -Child 2 Input [6]: [gross_margin#49, i_category#50, i_class#51, t_category#52, t_class#53, lochierarchy#54] - -(35) CometHashAggregate -Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] -Keys [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] -Functions: [] - -(36) CometExchange -Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] -Arguments: hashpartitioning(gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(37) CometHashAggregate -Input [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] -Keys [6]: [gross_margin#38, i_category#39, i_class#40, t_category#41, t_class#42, lochierarchy#43] -Functions: [] - -(38) CometExchange -Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] -Arguments: hashpartitioning(lochierarchy#43, _w0#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(39) CometSort -Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] -Arguments: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55], [lochierarchy#43 ASC NULLS FIRST, _w0#55 ASC NULLS FIRST, gross_margin#38 ASC NULLS FIRST] - -(40) CometColumnarToRow [codegen id : 1] -Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] - -(41) Window -Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55] -Arguments: [rank(gross_margin#38) windowspecdefinition(lochierarchy#43, _w0#55, gross_margin#38 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#56], [lochierarchy#43, _w0#55], [gross_margin#38 ASC NULLS FIRST] - -(42) Project [codegen id : 2] -Output [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, rank_within_parent#56] -Input [6]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, _w0#55, rank_within_parent#56] - -(43) TakeOrderedAndProject -Input [5]: [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, rank_within_parent#56] -Arguments: 100, [lochierarchy#43 DESC NULLS LAST, CASE WHEN (lochierarchy#43 = 0) THEN i_category#39 END ASC NULLS FIRST, rank_within_parent#56 ASC NULLS FIRST], [gross_margin#38, i_category#39, i_class#40, lochierarchy#43, rank_within_parent#56] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) - - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(46) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(48) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/extended.txt deleted file mode 100644 index 1f6c984b22..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index e91b278c14..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/simplified.txt +++ /dev/null @@ -1,55 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (2) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] - CometExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometUnion [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - 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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange [i_category] #8 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,i_category,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange #9 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_category,i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/explain.txt deleted file mode 100644 index 9613d03c41..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/explain.txt +++ /dev/null @@ -1,304 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (51) -+- * Project (50) - +- * BroadcastHashJoin Inner BuildRight (49) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * CometColumnarToRow (26) - : : +- CometSort (25) - : : +- CometColumnarExchange (24) - : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- * HashAggregate (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.store (14) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- Window (38) - : +- * CometColumnarToRow (37) - : +- CometSort (36) - : +- CometColumnarExchange (35) - : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) - : +- ReusedExchange (32) - +- BroadcastExchange (48) - +- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- ReusedExchange (43) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#4, i_category#5] - -(5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(7) Filter [codegen id : 1] -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) - -(8) BroadcastExchange -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#6] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(11) ReusedExchange [Reuses operator id: 55] -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] -Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) - -(16) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] - -(17) BroadcastExchange -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#7] -Right keys [1]: [s_store_sk#14] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] - -(20) HashAggregate [codegen id : 4] -Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] -Aggregate Attributes [1]: [sum#17] -Results [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] - -(21) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] - -(23) HashAggregate [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#8))#19] -Results [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#19,17,2) AS sum_sales#20, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#19,17,2) AS _w0#21] - -(24) CometColumnarExchange -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) CometColumnarToRow [codegen id : 6] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] - -(27) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(28) Filter [codegen id : 7] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(29) Window -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] -Arguments: [avg(_w0#21) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(30) Filter [codegen id : 22] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] -Condition : ((isnotnull(avg_monthly_sales#23) AND (avg_monthly_sales#23 > 0.000000)) AND CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END) - -(31) Project [codegen id : 22] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] - -(32) ReusedExchange [Reuses operator id: 21] -Output [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] - -(33) CometColumnarToRow [codegen id : 12] -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] - -(34) HashAggregate [codegen id : 12] -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -Keys [6]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] -Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#19] -Results [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#19,17,2) AS sum_sales#20] - -(35) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(36) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] -Arguments: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] - -(37) CometColumnarToRow [codegen id : 13] -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] - -(38) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] -Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#5, i_brand#4, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] - -(39) Project [codegen id : 14] -Output [6]: [i_category#5 AS i_category#31, i_brand#4 AS i_brand#32, s_store_name#24, s_company_name#25, sum_sales#20 AS sum_sales#33, rn#30] -Input [8]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20, rn#30] - -(40) BroadcastExchange -Input [6]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] - -(41) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] -Right keys [5]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, (rn#30 + 1)] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 22] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33] -Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] - -(43) ReusedExchange [Reuses operator id: 35] -Output [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] - -(44) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] -Arguments: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] - -(45) CometColumnarToRow [codegen id : 20] -Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] - -(46) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] -Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#5, i_brand#4, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] - -(47) Project [codegen id : 21] -Output [6]: [i_category#5 AS i_category#39, i_brand#4 AS i_brand#40, s_store_name#34, s_company_name#35, sum_sales#20 AS sum_sales#41, rn#38] -Input [8]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20, rn#38] - -(48) BroadcastExchange -Input [6]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] - -(49) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] -Right keys [5]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, (rn#38 - 1)] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 22] -Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, sum_sales#33 AS psum#42, sum_sales#41 AS nsum#43] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33, i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] - -(51) TakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] -Arguments: 100, [(sum_sales#20 - avg_monthly_sales#23) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometFilter (53) - +- CometNativeScan parquet spark_catalog.default.date_dim (52) - - -(52) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(54) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(55) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/simplified.txt deleted file mode 100644 index ee8e4dd40c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/simplified.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (22) - Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (7) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/explain.txt deleted file mode 100644 index 7b1f860f32..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_item_sk#1], [ss_item_sk#6], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) - -(16) CometBroadcastExchange -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [s_store_sk#14, s_store_name#15, s_company_name#16] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] -Right output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [ss_store_sk#7], [s_store_sk#14], Inner, BuildRight - -(18) CometProject -Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16], [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] - -(19) CometHashAggregate -Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] - -(20) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] - -(22) CometExchange -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] - -(25) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(28) Filter [codegen id : 7] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] - -(30) ReusedExchange [Reuses operator id: 20] -Output [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] - -(31) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] -Keys [6]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(ss_sales_price#27))] - -(32) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] - -(35) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [6]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] -Input [8]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28] - -(37) BroadcastExchange -Input [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] - -(40) ReusedExchange [Reuses operator id: 32] -Output [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(41) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(43) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#5, i_brand#4, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [6]: [i_category#5 AS i_category#37, i_brand#4 AS i_brand#38, s_store_name#32, s_company_name#33, sum_sales#18 AS sum_sales#39, rn#36] -Input [8]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18, rn#36] - -(45) BroadcastExchange -Input [6]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, (rn#36 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] - -(48) TakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/simplified.txt deleted file mode 100644 index a62c33ecc1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (7) - Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - 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 [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,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/explain.txt deleted file mode 100644 index 7b1f860f32..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_item_sk#1], [ss_item_sk#6], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) - -(16) CometBroadcastExchange -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [s_store_sk#14, s_store_name#15, s_company_name#16] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] -Right output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [ss_store_sk#7], [s_store_sk#14], Inner, BuildRight - -(18) CometProject -Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16], [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] - -(19) CometHashAggregate -Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] - -(20) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] - -(22) CometExchange -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] - -(25) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(28) Filter [codegen id : 7] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] - -(30) ReusedExchange [Reuses operator id: 20] -Output [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] - -(31) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] -Keys [6]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(ss_sales_price#27))] - -(32) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] - -(35) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [6]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] -Input [8]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28] - -(37) BroadcastExchange -Input [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] - -(40) ReusedExchange [Reuses operator id: 32] -Output [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(41) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(43) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#5, i_brand#4, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [6]: [i_category#5 AS i_category#37, i_brand#4 AS i_brand#38, s_store_name#32, s_company_name#33, sum_sales#18 AS sum_sales#39, rn#36] -Input [8]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18, rn#36] - -(45) BroadcastExchange -Input [6]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, (rn#36 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] - -(48) TakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/extended.txt deleted file mode 100644 index b50b570b4b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index a62c33ecc1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (7) - Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - 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 [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,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/explain.txt deleted file mode 100644 index 070113d8ea..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/explain.txt +++ /dev/null @@ -1,497 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (84) -+- CometTakeOrderedAndProject (83) - +- CometHashAggregate (82) - +- CometColumnarExchange (81) - +- * HashAggregate (80) - +- Union (79) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometColumnarExchange (19) - : +- * HashAggregate (18) - : +- * CometColumnarToRow (17) - : +- CometColumnarExchange (16) - : +- * HashAggregate (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildLeft (10) - : : :- BroadcastExchange (5) - : : : +- * Project (4) - : : : +- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : +- * CometColumnarToRow (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometNativeScan parquet spark_catalog.default.web_returns (6) - : +- ReusedExchange (12) - :- * Project (52) - : +- * Filter (51) - : +- Window (50) - : +- * Sort (49) - : +- Window (48) - : +- * CometColumnarToRow (47) - : +- CometSort (46) - : +- CometColumnarExchange (45) - : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- CometColumnarExchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildLeft (36) - : : :- BroadcastExchange (31) - : : : +- * Project (30) - : : : +- * Filter (29) - : : : +- * ColumnarToRow (28) - : : : +- Scan parquet spark_catalog.default.catalog_sales (27) - : : +- * CometColumnarToRow (35) - : : +- CometProject (34) - : : +- CometFilter (33) - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (32) - : +- ReusedExchange (38) - +- * Project (78) - +- * Filter (77) - +- Window (76) - +- * Sort (75) - +- Window (74) - +- * CometColumnarToRow (73) - +- CometSort (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * CometColumnarToRow (69) - +- CometColumnarExchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * BroadcastHashJoin Inner BuildRight (65) - :- * Project (63) - : +- * BroadcastHashJoin Inner BuildLeft (62) - : :- BroadcastExchange (57) - : : +- * Project (56) - : : +- * Filter (55) - : : +- * ColumnarToRow (54) - : : +- Scan parquet spark_catalog.default.store_sales (53) - : +- * CometColumnarToRow (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometNativeScan parquet spark_catalog.default.store_returns (58) - +- ReusedExchange (64) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] - -(3) Filter [codegen id : 1] -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND isnotnull(ws_quantity#3)) AND (ws_net_profit#5 > 1.00)) AND (ws_net_paid#4 > 0.00)) AND (ws_quantity#3 > 0)) AND isnotnull(ws_order_number#2)) AND isnotnull(ws_item_sk#1)) - -(4) Project [codegen id : 1] -Output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] - -(5) BroadcastExchange -Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=1] - -(6) CometNativeScan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) - -(8) CometProject -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(9) CometColumnarToRow -Input [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(10) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [ws_order_number#2, ws_item_sk#1] -Right keys [2]: [wr_order_number#9, wr_item_sk#8] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 3] -Output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(12) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#13] - -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#6] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 3] -Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] - -(15) HashAggregate [codegen id : 3] -Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Keys [1]: [ws_item_sk#1] -Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#14, sum#15, sum#16, isEmpty#17, sum#18, isEmpty#19] -Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] - -(16) CometColumnarExchange -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(17) CometColumnarToRow [codegen id : 4] -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] - -(18) HashAggregate [codegen id : 4] -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Keys [1]: [ws_item_sk#1] -Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] -Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] - -(19) CometColumnarExchange -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(20) CometSort -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [item#30, return_ratio#31, currency_ratio#32], [return_ratio#31 ASC NULLS FIRST] - -(21) CometColumnarToRow [codegen id : 5] -Input [3]: [item#30, return_ratio#31, currency_ratio#32] - -(22) Window -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] - -(23) Sort [codegen id : 6] -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 - -(24) Window -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] - -(25) Filter [codegen id : 7] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] -Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) - -(26) Project [codegen id : 7] -Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] - -(27) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(28) ColumnarToRow [codegen id : 8] -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] - -(29) Filter [codegen id : 8] -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) - -(30) Project [codegen id : 8] -Output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] - -(31) BroadcastExchange -Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=4] - -(32) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] -Condition : (((isnotnull(cr_return_amount#45) AND (cr_return_amount#45 > 10000.00)) AND isnotnull(cr_order_number#43)) AND isnotnull(cr_item_sk#42)) - -(34) CometProject -Input [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] -Arguments: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45], [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] - -(35) CometColumnarToRow -Input [4]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] - -(36) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [cs_order_number#37, cs_item_sk#36] -Right keys [2]: [cr_order_number#43, cr_item_sk#42] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 10] -Output [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45] -Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] - -(38) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#47] - -(39) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#41] -Right keys [1]: [d_date_sk#47] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 10] -Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] -Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45, d_date_sk#47] - -(41) HashAggregate [codegen id : 10] -Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] -Keys [1]: [cs_item_sk#36] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#44, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#48, sum#49, sum#50, isEmpty#51, sum#52, isEmpty#53] -Results [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] - -(42) CometColumnarExchange -Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(43) CometColumnarToRow [codegen id : 11] -Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] - -(44) HashAggregate [codegen id : 11] -Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Keys [1]: [cs_item_sk#36] -Functions [4]: [sum(coalesce(cr_return_quantity#44, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#44, 0))#60, sum(coalesce(cs_quantity#38, 0))#61, sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#62, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63] -Results [3]: [cs_item_sk#36 AS item#64, (cast(sum(coalesce(cr_return_quantity#44, 0))#60 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#61 as decimal(15,4))) AS return_ratio#65, (cast(sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#62 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63 as decimal(15,4))) AS currency_ratio#66] - -(45) CometColumnarExchange -Input [3]: [item#64, return_ratio#65, currency_ratio#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(46) CometSort -Input [3]: [item#64, return_ratio#65, currency_ratio#66] -Arguments: [item#64, return_ratio#65, currency_ratio#66], [return_ratio#65 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 12] -Input [3]: [item#64, return_ratio#65, currency_ratio#66] - -(48) Window -Input [3]: [item#64, return_ratio#65, currency_ratio#66] -Arguments: [rank(return_ratio#65) windowspecdefinition(return_ratio#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#67], [return_ratio#65 ASC NULLS FIRST] - -(49) Sort [codegen id : 13] -Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] -Arguments: [currency_ratio#66 ASC NULLS FIRST], false, 0 - -(50) Window -Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] -Arguments: [rank(currency_ratio#66) windowspecdefinition(currency_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#68], [currency_ratio#66 ASC NULLS FIRST] - -(51) Filter [codegen id : 14] -Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] -Condition : ((return_rank#67 <= 10) OR (currency_rank#68 <= 10)) - -(52) Project [codegen id : 14] -Output [5]: [catalog AS channel#69, item#64, return_ratio#65, return_rank#67, currency_rank#68] -Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] - -(53) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#75), dynamicpruningexpression(ss_sold_date_sk#75 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(54) ColumnarToRow [codegen id : 15] -Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] - -(55) Filter [codegen id : 15] -Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] -Condition : (((((((isnotnull(ss_net_profit#74) AND isnotnull(ss_net_paid#73)) AND isnotnull(ss_quantity#72)) AND (ss_net_profit#74 > 1.00)) AND (ss_net_paid#73 > 0.00)) AND (ss_quantity#72 > 0)) AND isnotnull(ss_ticket_number#71)) AND isnotnull(ss_item_sk#70)) - -(56) Project [codegen id : 15] -Output [5]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75] -Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] - -(57) BroadcastExchange -Input [5]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=7] - -(58) CometNativeScan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(59) CometFilter -Input [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] -Condition : (((isnotnull(sr_return_amt#79) AND (sr_return_amt#79 > 10000.00)) AND isnotnull(sr_ticket_number#77)) AND isnotnull(sr_item_sk#76)) - -(60) CometProject -Input [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] -Arguments: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79], [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] - -(61) CometColumnarToRow -Input [4]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] - -(62) BroadcastHashJoin [codegen id : 17] -Left keys [2]: [ss_ticket_number#71, ss_item_sk#70] -Right keys [2]: [sr_ticket_number#77, sr_item_sk#76] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 17] -Output [6]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79] -Input [9]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] - -(64) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#81] - -(65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#75] -Right keys [1]: [d_date_sk#81] -Join type: Inner -Join condition: None - -(66) Project [codegen id : 17] -Output [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] -Input [7]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79, d_date_sk#81] - -(67) HashAggregate [codegen id : 17] -Input [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] -Keys [1]: [ss_item_sk#70] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#78, 0)), partial_sum(coalesce(ss_quantity#72, 0)), partial_sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#82, sum#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Results [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] - -(68) CometColumnarExchange -Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] -Arguments: hashpartitioning(ss_item_sk#70, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(69) CometColumnarToRow [codegen id : 18] -Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] - -(70) HashAggregate [codegen id : 18] -Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] -Keys [1]: [ss_item_sk#70] -Functions [4]: [sum(coalesce(sr_return_quantity#78, 0)), sum(coalesce(ss_quantity#72, 0)), sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#78, 0))#94, sum(coalesce(ss_quantity#72, 0))#95, sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00))#96, sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97] -Results [3]: [ss_item_sk#70 AS item#98, (cast(sum(coalesce(sr_return_quantity#78, 0))#94 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#72, 0))#95 as decimal(15,4))) AS return_ratio#99, (cast(sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00))#96 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97 as decimal(15,4))) AS currency_ratio#100] - -(71) CometColumnarExchange -Input [3]: [item#98, return_ratio#99, currency_ratio#100] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(72) CometSort -Input [3]: [item#98, return_ratio#99, currency_ratio#100] -Arguments: [item#98, return_ratio#99, currency_ratio#100], [return_ratio#99 ASC NULLS FIRST] - -(73) CometColumnarToRow [codegen id : 19] -Input [3]: [item#98, return_ratio#99, currency_ratio#100] - -(74) Window -Input [3]: [item#98, return_ratio#99, currency_ratio#100] -Arguments: [rank(return_ratio#99) windowspecdefinition(return_ratio#99 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#101], [return_ratio#99 ASC NULLS FIRST] - -(75) Sort [codegen id : 20] -Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] -Arguments: [currency_ratio#100 ASC NULLS FIRST], false, 0 - -(76) Window -Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] -Arguments: [rank(currency_ratio#100) windowspecdefinition(currency_ratio#100 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#102], [currency_ratio#100 ASC NULLS FIRST] - -(77) Filter [codegen id : 21] -Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] -Condition : ((return_rank#101 <= 10) OR (currency_rank#102 <= 10)) - -(78) Project [codegen id : 21] -Output [5]: [store AS channel#103, item#98, return_ratio#99, return_rank#101, currency_rank#102] -Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] - -(79) Union - -(80) HashAggregate [codegen id : 22] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] - -(81) CometColumnarExchange -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(82) CometHashAggregate -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Functions: [] - -(83) CometTakeOrderedAndProject -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,return_rank#33 ASC NULLS FIRST,currency_rank#34 ASC NULLS FIRST,item#30 ASC NULLS FIRST], output=[channel#35,item#30,return_ratio#31,return_rank#33,currency_rank#34]), [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34], 100, 0, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST, item#30 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] - -(84) CometColumnarToRow [codegen id : 23] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (89) -+- * CometColumnarToRow (88) - +- CometProject (87) - +- CometFilter (86) - +- CometNativeScan parquet spark_catalog.default.date_dim (85) - - -(85) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#104, d_moy#105] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(86) CometFilter -Input [3]: [d_date_sk#13, d_year#104, d_moy#105] -Condition : ((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND (d_year#104 = 2001)) AND (d_moy#105 = 12)) AND isnotnull(d_date_sk#13)) - -(87) CometProject -Input [3]: [d_date_sk#13, d_year#104, d_moy#105] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(88) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(89) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 - -Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#75 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/simplified.txt deleted file mode 100644 index 047269434d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/simplified.txt +++ /dev/null @@ -1,140 +0,0 @@ -WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (22) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (7) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (6) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #2 - WholeStageCodegen (4) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (14) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (13) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #6 - WholeStageCodegen (11) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_item_sk] #7 - WholeStageCodegen (10) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (21) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (20) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #9 - WholeStageCodegen (18) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk] #10 - WholeStageCodegen (17) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/explain.txt deleted file mode 100644 index 74702d596d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,462 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- * Project (49) - : +- * Filter (48) - : +- Window (47) - : +- * Sort (46) - : +- Window (45) - : +- * CometColumnarToRow (44) - : +- CometSort (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometBroadcastExchange (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometBroadcastHashJoin (57) - : :- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (59) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND isnotnull(ws_quantity#3)) AND (ws_net_profit#5 > 1.00)) AND (ws_net_paid#4 > 0.00)) AND (ws_quantity#3 > 0)) AND isnotnull(ws_order_number#2)) AND isnotnull(ws_item_sk#1)) - -(3) CometProject -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6], [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(4) CometBroadcastExchange -Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) - -(7) CometProject -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(8) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner, BuildLeft - -(9) CometProject -Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(12) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] - -(16) CometHashAggregate -Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Keys [1]: [ws_item_sk#1] -Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(17) CometExchange -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Keys [1]: [ws_item_sk#1] -Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(19) CometExchange -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] - -(21) CometColumnarToRow [codegen id : 1] -Input [3]: [item#22, return_ratio#23, currency_ratio#24] - -(22) Window -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] - -(23) Sort [codegen id : 2] -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 - -(24) Window -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] - -(25) Filter [codegen id : 3] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] -Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) - -(26) Project [codegen id : 3] -Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_sold_date_sk#33 IN dynamicpruning#34)] -PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(28) CometFilter -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) - -(29) CometProject -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(30) CometBroadcastExchange -Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(32) CometFilter -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) - -(33) CometProject -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] - -(34) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft - -(35) CometProject -Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] - -(36) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#40] - -(37) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -Right output [1]: [d_date_sk#40] -Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight - -(38) CometProject -Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] - -(39) CometHashAggregate -Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -Keys [1]: [cs_item_sk#28] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(40) CometExchange -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(41) CometHashAggregate -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Keys [1]: [cs_item_sk#28] -Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(42) CometExchange -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(43) CometSort -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] - -(44) CometColumnarToRow [codegen id : 4] -Input [3]: [item#47, return_ratio#48, currency_ratio#49] - -(45) Window -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] - -(46) Sort [codegen id : 5] -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 - -(47) Window -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] - -(48) Filter [codegen id : 6] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] -Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) - -(49) Project [codegen id : 6] -Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) - -(52) CometProject -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(53) CometBroadcastExchange -Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(55) CometFilter -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) - -(56) CometProject -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] - -(57) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft - -(58) CometProject -Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] - -(59) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#65] - -(60) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(61) CometProject -Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] - -(62) CometHashAggregate -Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -Keys [1]: [ss_item_sk#53] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(63) CometExchange -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(64) CometHashAggregate -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Keys [1]: [ss_item_sk#53] -Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(65) CometExchange -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(66) CometSort -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] - -(67) CometColumnarToRow [codegen id : 7] -Input [3]: [item#72, return_ratio#73, currency_ratio#74] - -(68) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] - -(69) Sort [codegen id : 8] -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 - -(70) Window -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] - -(71) Filter [codegen id : 9] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] -Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) - -(72) Project [codegen id : 9] -Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] - -(73) Union - -(74) HashAggregate [codegen id : 10] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(75) CometColumnarExchange -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometHashAggregate -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] - -(77) CometTakeOrderedAndProject -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST,item#22 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST, item#22 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(78) CometColumnarToRow [codegen id : 11] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) - - -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(80) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(81) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(82) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(83) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 - -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/simplified.txt deleted file mode 100644 index ca80833ee5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,110 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (10) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (3) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (2) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - 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_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_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - 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_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_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 (9) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (8) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - 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_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_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt deleted file mode 100644 index 74702d596d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/explain.txt +++ /dev/null @@ -1,462 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- * Project (49) - : +- * Filter (48) - : +- Window (47) - : +- * Sort (46) - : +- Window (45) - : +- * CometColumnarToRow (44) - : +- CometSort (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometBroadcastExchange (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometBroadcastHashJoin (57) - : :- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (59) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND isnotnull(ws_quantity#3)) AND (ws_net_profit#5 > 1.00)) AND (ws_net_paid#4 > 0.00)) AND (ws_quantity#3 > 0)) AND isnotnull(ws_order_number#2)) AND isnotnull(ws_item_sk#1)) - -(3) CometProject -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6], [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(4) CometBroadcastExchange -Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) - -(7) CometProject -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(8) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner, BuildLeft - -(9) CometProject -Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(12) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] - -(16) CometHashAggregate -Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Keys [1]: [ws_item_sk#1] -Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(17) CometExchange -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Keys [1]: [ws_item_sk#1] -Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(19) CometExchange -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] - -(21) CometColumnarToRow [codegen id : 1] -Input [3]: [item#22, return_ratio#23, currency_ratio#24] - -(22) Window -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] - -(23) Sort [codegen id : 2] -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 - -(24) Window -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] - -(25) Filter [codegen id : 3] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] -Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) - -(26) Project [codegen id : 3] -Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_sold_date_sk#33 IN dynamicpruning#34)] -PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(28) CometFilter -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) - -(29) CometProject -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(30) CometBroadcastExchange -Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(32) CometFilter -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) - -(33) CometProject -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] - -(34) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft - -(35) CometProject -Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] - -(36) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#40] - -(37) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -Right output [1]: [d_date_sk#40] -Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight - -(38) CometProject -Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] - -(39) CometHashAggregate -Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -Keys [1]: [cs_item_sk#28] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(40) CometExchange -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(41) CometHashAggregate -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Keys [1]: [cs_item_sk#28] -Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(42) CometExchange -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(43) CometSort -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] - -(44) CometColumnarToRow [codegen id : 4] -Input [3]: [item#47, return_ratio#48, currency_ratio#49] - -(45) Window -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] - -(46) Sort [codegen id : 5] -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 - -(47) Window -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] - -(48) Filter [codegen id : 6] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] -Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) - -(49) Project [codegen id : 6] -Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) - -(52) CometProject -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(53) CometBroadcastExchange -Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(55) CometFilter -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) - -(56) CometProject -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] - -(57) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft - -(58) CometProject -Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] - -(59) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#65] - -(60) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(61) CometProject -Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] - -(62) CometHashAggregate -Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -Keys [1]: [ss_item_sk#53] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(63) CometExchange -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(64) CometHashAggregate -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Keys [1]: [ss_item_sk#53] -Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(65) CometExchange -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(66) CometSort -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] - -(67) CometColumnarToRow [codegen id : 7] -Input [3]: [item#72, return_ratio#73, currency_ratio#74] - -(68) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] - -(69) Sort [codegen id : 8] -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 - -(70) Window -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] - -(71) Filter [codegen id : 9] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] -Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) - -(72) Project [codegen id : 9] -Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] - -(73) Union - -(74) HashAggregate [codegen id : 10] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(75) CometColumnarExchange -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometHashAggregate -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] - -(77) CometTakeOrderedAndProject -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST,item#22 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST, item#22 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(78) CometColumnarToRow [codegen id : 11] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) - - -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(80) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(81) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(82) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(83) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 - -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/extended.txt deleted file mode 100644 index 75684a9669..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/extended.txt +++ /dev/null @@ -1,94 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index ca80833ee5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/simplified.txt +++ /dev/null @@ -1,110 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (10) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (3) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (2) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - 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_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_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - 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_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_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 (9) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (8) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - 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_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_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/explain.txt deleted file mode 100644 index 23adfe692b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/explain.txt +++ /dev/null @@ -1,455 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (77) -+- * Filter (76) - +- * HashAggregate (75) - +- * HashAggregate (74) - +- * Project (73) - +- * BroadcastHashJoin Inner BuildRight (72) - :- Window (65) - : +- * CometColumnarToRow (64) - : +- CometSort (63) - : +- CometExchange (62) - : +- CometProject (61) - : +- CometFilter (60) - : +- CometSortMergeJoin (59) - : :- CometSort (29) - : : +- CometColumnarExchange (28) - : : +- * HashAggregate (27) - : : +- * CometColumnarToRow (26) - : : +- CometColumnarExchange (25) - : : +- * HashAggregate (24) - : : +- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * Project (15) - : : : +- Window (14) - : : : +- * CometColumnarToRow (13) - : : : +- CometSort (12) - : : : +- CometColumnarExchange (11) - : : : +- * HashAggregate (10) - : : : +- * CometColumnarToRow (9) - : : : +- CometColumnarExchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (21) - : : +- * Project (20) - : : +- Window (19) - : : +- * CometColumnarToRow (18) - : : +- CometSort (17) - : : +- ReusedExchange (16) - : +- CometSort (58) - : +- CometColumnarExchange (57) - : +- * HashAggregate (56) - : +- * CometColumnarToRow (55) - : +- CometColumnarExchange (54) - : +- * HashAggregate (53) - : +- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Project (44) - : : +- Window (43) - : : +- * CometColumnarToRow (42) - : : +- CometSort (41) - : : +- CometColumnarExchange (40) - : : +- * HashAggregate (39) - : : +- * CometColumnarToRow (38) - : : +- CometColumnarExchange (37) - : : +- * HashAggregate (36) - : : +- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * Filter (32) - : : : +- * ColumnarToRow (31) - : : : +- Scan parquet spark_catalog.default.store_sales (30) - : : +- ReusedExchange (33) - : +- BroadcastExchange (50) - : +- * Project (49) - : +- Window (48) - : +- * CometColumnarToRow (47) - : +- CometSort (46) - : +- ReusedExchange (45) - +- BroadcastExchange (71) - +- * Project (70) - +- Window (69) - +- * CometColumnarToRow (68) - +- CometSort (67) - +- ReusedExchange (66) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] - -(3) Filter [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(4) ReusedExchange [Reuses operator id: 82] -Output [2]: [d_date_sk#5, d_date#6] - -(5) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 2] -Output [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] - -(7) HashAggregate [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum#7] -Results [3]: [ws_item_sk#1, d_date#6, sum#8] - -(8) CometColumnarExchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(9) CometColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, d_date#6, sum#8] - -(10) HashAggregate [codegen id : 3] -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#9] -Results [4]: [ws_item_sk#1 AS item_sk#10, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#9,17,2) AS sumws#11, ws_item_sk#1] - -(11) CometColumnarExchange -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(12) CometSort -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(13) CometColumnarToRow [codegen id : 4] -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] - -(14) Window -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(15) Project [codegen id : 10] -Output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] -Input [5]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12] - -(16) ReusedExchange [Reuses operator id: 11] -Output [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] - -(17) CometSort -Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -Arguments: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14], [ws_item_sk#14 ASC NULLS FIRST, d_date#13 ASC NULLS FIRST] - -(18) CometColumnarToRow [codegen id : 8] -Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] - -(19) Window -Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -Arguments: [row_number() windowspecdefinition(ws_item_sk#14, d_date#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#15], [ws_item_sk#14], [d_date#13 ASC NULLS FIRST] - -(20) Project [codegen id : 9] -Output [3]: [item_sk#10 AS item_sk#16, sumws#11 AS sumws#17, rk#15] -Input [5]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14, rk#15] - -(21) BroadcastExchange -Input [3]: [item_sk#16, sumws#17, rk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(22) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [item_sk#10] -Right keys [1]: [item_sk#16] -Join type: Inner -Join condition: (rk#12 >= rk#15) - -(23) Project [codegen id : 10] -Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] -Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#16, sumws#17, rk#15] - -(24) HashAggregate [codegen id : 10] -Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] -Keys [3]: [item_sk#10, d_date#6, sumws#11] -Functions [1]: [partial_sum(sumws#17)] -Aggregate Attributes [2]: [sum#18, isEmpty#19] -Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] - -(25) CometColumnarExchange -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(26) CometColumnarToRow [codegen id : 11] -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] - -(27) HashAggregate [codegen id : 11] -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -Keys [3]: [item_sk#10, d_date#6, sumws#11] -Functions [1]: [sum(sumws#17)] -Aggregate Attributes [1]: [sum(sumws#17)#22] -Results [3]: [item_sk#10, d_date#6, sum(sumws#17)#22 AS cume_sales#23] - -(28) CometColumnarExchange -Input [3]: [item_sk#10, d_date#6, cume_sales#23] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(29) CometSort -Input [3]: [item_sk#10, d_date#6, cume_sales#23] -Arguments: [item_sk#10, d_date#6, cume_sales#23], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(30) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 13] -Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] - -(32) Filter [codegen id : 13] -Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#24) - -(33) ReusedExchange [Reuses operator id: 82] -Output [2]: [d_date_sk#27, d_date#28] - -(34) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 13] -Output [3]: [ss_item_sk#24, ss_sales_price#25, d_date#28] -Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#27, d_date#28] - -(36) HashAggregate [codegen id : 13] -Input [3]: [ss_item_sk#24, ss_sales_price#25, d_date#28] -Keys [2]: [ss_item_sk#24, d_date#28] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum#29] -Results [3]: [ss_item_sk#24, d_date#28, sum#30] - -(37) CometColumnarExchange -Input [3]: [ss_item_sk#24, d_date#28, sum#30] -Arguments: hashpartitioning(ss_item_sk#24, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(38) CometColumnarToRow [codegen id : 14] -Input [3]: [ss_item_sk#24, d_date#28, sum#30] - -(39) HashAggregate [codegen id : 14] -Input [3]: [ss_item_sk#24, d_date#28, sum#30] -Keys [2]: [ss_item_sk#24, d_date#28] -Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#31] -Results [4]: [ss_item_sk#24 AS item_sk#32, d_date#28, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#31,17,2) AS sumss#33, ss_item_sk#24] - -(40) CometColumnarExchange -Input [4]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24] -Arguments: hashpartitioning(ss_item_sk#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(41) CometSort -Input [4]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24] -Arguments: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24], [ss_item_sk#24 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 15] -Input [4]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24] - -(43) Window -Input [4]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24] -Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#34], [ss_item_sk#24], [d_date#28 ASC NULLS FIRST] - -(44) Project [codegen id : 21] -Output [4]: [item_sk#32, d_date#28, sumss#33, rk#34] -Input [5]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24, rk#34] - -(45) ReusedExchange [Reuses operator id: 40] -Output [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] - -(46) CometSort -Input [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] -Arguments: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36], [ss_item_sk#36 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 19] -Input [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] - -(48) Window -Input [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] -Arguments: [row_number() windowspecdefinition(ss_item_sk#36, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#37], [ss_item_sk#36], [d_date#35 ASC NULLS FIRST] - -(49) Project [codegen id : 20] -Output [3]: [item_sk#32 AS item_sk#38, sumss#33 AS sumss#39, rk#37] -Input [5]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36, rk#37] - -(50) BroadcastExchange -Input [3]: [item_sk#38, sumss#39, rk#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -(51) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [item_sk#32] -Right keys [1]: [item_sk#38] -Join type: Inner -Join condition: (rk#34 >= rk#37) - -(52) Project [codegen id : 21] -Output [4]: [item_sk#32, d_date#28, sumss#33, sumss#39] -Input [7]: [item_sk#32, d_date#28, sumss#33, rk#34, item_sk#38, sumss#39, rk#37] - -(53) HashAggregate [codegen id : 21] -Input [4]: [item_sk#32, d_date#28, sumss#33, sumss#39] -Keys [3]: [item_sk#32, d_date#28, sumss#33] -Functions [1]: [partial_sum(sumss#39)] -Aggregate Attributes [2]: [sum#40, isEmpty#41] -Results [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] - -(54) CometColumnarExchange -Input [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] -Arguments: hashpartitioning(item_sk#32, d_date#28, sumss#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(55) CometColumnarToRow [codegen id : 22] -Input [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] - -(56) HashAggregate [codegen id : 22] -Input [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] -Keys [3]: [item_sk#32, d_date#28, sumss#33] -Functions [1]: [sum(sumss#39)] -Aggregate Attributes [1]: [sum(sumss#39)#44] -Results [3]: [item_sk#32, d_date#28, sum(sumss#39)#44 AS cume_sales#45] - -(57) CometColumnarExchange -Input [3]: [item_sk#32, d_date#28, cume_sales#45] -Arguments: hashpartitioning(item_sk#32, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(58) CometSort -Input [3]: [item_sk#32, d_date#28, cume_sales#45] -Arguments: [item_sk#32, d_date#28, cume_sales#45], [item_sk#32 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] - -(59) CometSortMergeJoin -Left output [3]: [item_sk#10, d_date#6, cume_sales#23] -Right output [3]: [item_sk#32, d_date#28, cume_sales#45] -Arguments: [item_sk#10, d_date#6], [item_sk#32, d_date#28], FullOuter - -(60) CometFilter -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#32, d_date#28, cume_sales#45] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#32 END) - -(61) CometProject -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#32, d_date#28, cume_sales#45] -Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#32 END AS item_sk#46, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#28 END AS d_date#47, cume_sales#23 AS web_sales#48, cume_sales#45 AS store_sales#49] - -(62) CometExchange -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: hashpartitioning(item_sk#46, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(63) CometSort -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST] - -(64) CometColumnarToRow [codegen id : 23] -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] - -(65) Window -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: [row_number() windowspecdefinition(item_sk#46, d_date#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#50], [item_sk#46], [d_date#47 ASC NULLS FIRST] - -(66) ReusedExchange [Reuses operator id: 62] -Output [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] - -(67) CometSort -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST] - -(68) CometColumnarToRow [codegen id : 46] -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] - -(69) Window -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: [row_number() windowspecdefinition(item_sk#46, d_date#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#51], [item_sk#46], [d_date#47 ASC NULLS FIRST] - -(70) Project [codegen id : 47] -Output [4]: [item_sk#46 AS item_sk#52, web_sales#48 AS web_sales#53, store_sales#49 AS store_sales#54, rk#51] -Input [5]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#51] - -(71) BroadcastExchange -Input [4]: [item_sk#52, web_sales#53, store_sales#54, rk#51] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -(72) BroadcastHashJoin [codegen id : 48] -Left keys [1]: [item_sk#46] -Right keys [1]: [item_sk#52] -Join type: Inner -Join condition: (rk#50 >= rk#51) - -(73) Project [codegen id : 48] -Output [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] -Input [9]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#50, item_sk#52, web_sales#53, store_sales#54, rk#51] - -(74) HashAggregate [codegen id : 48] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] -Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Functions [2]: [partial_max(web_sales#53), partial_max(store_sales#54)] -Aggregate Attributes [2]: [max#55, max#56] -Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#57, max#58] - -(75) HashAggregate [codegen id : 48] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#57, max#58] -Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Functions [2]: [max(web_sales#53), max(store_sales#54)] -Aggregate Attributes [2]: [max(web_sales#53)#59, max(store_sales#54)#60] -Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max(web_sales#53)#59 AS web_cumulative#61, max(store_sales#54)#60 AS store_cumulative#62] - -(76) Filter [codegen id : 48] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#61, store_cumulative#62] -Condition : ((isnotnull(web_cumulative#61) AND isnotnull(store_cumulative#62)) AND (web_cumulative#61 > store_cumulative#62)) - -(77) TakeOrderedAndProject -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#61, store_cumulative#62] -Arguments: 100, [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#61, store_cumulative#62] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (82) -+- * CometColumnarToRow (81) - +- CometProject (80) - +- CometFilter (79) - +- CometNativeScan parquet spark_catalog.default.date_dim (78) - - -(78) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#63] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(79) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#63] -Condition : (((isnotnull(d_month_seq#63) AND (d_month_seq#63 >= 1212)) AND (d_month_seq#63 <= 1223)) AND isnotnull(d_date_sk#5)) - -(80) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#63] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(81) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] - -(82) BroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/simplified.txt deleted file mode 100644 index 00cd0a509c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/simplified.txt +++ /dev/null @@ -1,127 +0,0 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (48) - Filter [web_cumulative,store_cumulative] - HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] - HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] - Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (11) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (10) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometColumnarExchange [ws_item_sk] #4 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk,d_date] #5 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - Project [item_sk,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #8 - WholeStageCodegen (22) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumss] #9 - WholeStageCodegen (21) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (15) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometColumnarExchange [ss_item_sk] #10 - WholeStageCodegen (14) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk,d_date] #11 - WholeStageCodegen (13) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (20) - Project [item_sk,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (47) - Project [item_sk,web_sales,store_sales,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (46) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/explain.txt deleted file mode 100644 index f8a95a6db8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,448 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (76) -+- * Filter (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- Window (64) - : +- * CometColumnarToRow (63) - : +- CometSort (62) - : +- CometExchange (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometSortMergeJoin (58) - : :- CometSort (30) - : : +- CometColumnarExchange (29) - : : +- * HashAggregate (28) - : : +- * CometColumnarToRow (27) - : : +- CometColumnarExchange (26) - : : +- * HashAggregate (25) - : : +- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (16) - : : : +- Window (15) - : : : +- * CometColumnarToRow (14) - : : : +- CometSort (13) - : : : +- CometExchange (12) - : : : +- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- BroadcastExchange (22) - : : +- * Project (21) - : : +- Window (20) - : : +- * CometColumnarToRow (19) - : : +- CometSort (18) - : : +- ReusedExchange (17) - : +- CometSort (57) - : +- CometColumnarExchange (56) - : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) - : +- CometColumnarExchange (53) - : +- * HashAggregate (52) - : +- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Project (43) - : : +- Window (42) - : : +- * CometColumnarToRow (41) - : : +- CometSort (40) - : : +- CometExchange (39) - : : +- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometFilter (32) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (31) - : : +- ReusedExchange (33) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- Window (47) - : +- * CometColumnarToRow (46) - : +- CometSort (45) - : +- ReusedExchange (44) - +- BroadcastExchange (70) - +- * Project (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- ReusedExchange (65) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: [d_date_sk#5, d_date#6] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Right output [2]: [d_date_sk#5, d_date#6] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] -Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] - -(9) CometHashAggregate -Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] - -(10) CometExchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] - -(12) CometExchange -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(13) CometSort -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(14) CometColumnarToRow [codegen id : 1] -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] - -(15) Window -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(16) Project [codegen id : 4] -Output [4]: [item_sk#9, d_date#6, sumws#10, rk#11] -Input [5]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1, rk#11] - -(17) ReusedExchange [Reuses operator id: 12] -Output [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] - -(18) CometSort -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -Arguments: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13], [ws_item_sk#13 ASC NULLS FIRST, d_date#12 ASC NULLS FIRST] - -(19) CometColumnarToRow [codegen id : 2] -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] - -(20) Window -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -Arguments: [row_number() windowspecdefinition(ws_item_sk#13, d_date#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#14], [ws_item_sk#13], [d_date#12 ASC NULLS FIRST] - -(21) Project [codegen id : 3] -Output [3]: [item_sk#9 AS item_sk#15, sumws#10 AS sumws#16, rk#14] -Input [5]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13, rk#14] - -(22) BroadcastExchange -Input [3]: [item_sk#15, sumws#16, rk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [item_sk#9] -Right keys [1]: [item_sk#15] -Join type: Inner -Join condition: (rk#11 >= rk#14) - -(24) Project [codegen id : 4] -Output [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] -Input [7]: [item_sk#9, d_date#6, sumws#10, rk#11, item_sk#15, sumws#16, rk#14] - -(25) HashAggregate [codegen id : 4] -Input [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] -Keys [3]: [item_sk#9, d_date#6, sumws#10] -Functions [1]: [partial_sum(sumws#16)] -Aggregate Attributes [2]: [sum#17, isEmpty#18] -Results [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] - -(26) CometColumnarExchange -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] -Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometColumnarToRow [codegen id : 5] -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] - -(28) HashAggregate [codegen id : 5] -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] -Keys [3]: [item_sk#9, d_date#6, sumws#10] -Functions [1]: [sum(sumws#16)] -Aggregate Attributes [1]: [sum(sumws#16)#21] -Results [3]: [item_sk#9, d_date#6, sum(sumws#16)#21 AS cume_sales#22] - -(29) CometColumnarExchange -Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(30) CometSort -Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: [item_sk#9, d_date#6, cume_sales#22], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_item_sk#23) - -(33) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#27, d_date#28] - -(34) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Right output [2]: [d_date_sk#27, d_date#28] -Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight - -(35) CometProject -Input [5]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_date#28] -Arguments: [ss_item_sk#23, ss_sales_price#24, d_date#28], [ss_item_sk#23, ss_sales_price#24, d_date#28] - -(36) CometHashAggregate -Input [3]: [ss_item_sk#23, ss_sales_price#24, d_date#28] -Keys [2]: [ss_item_sk#23, d_date#28] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#24))] - -(37) CometExchange -Input [3]: [ss_item_sk#23, d_date#28, sum#29] -Arguments: hashpartitioning(ss_item_sk#23, d_date#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(38) CometHashAggregate -Input [3]: [ss_item_sk#23, d_date#28, sum#29] -Keys [2]: [ss_item_sk#23, d_date#28] -Functions [1]: [sum(UnscaledValue(ss_sales_price#24))] - -(39) CometExchange -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: hashpartitioning(ss_item_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(40) CometSort -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23], [ss_item_sk#23 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] - -(41) CometColumnarToRow [codegen id : 6] -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] - -(42) Window -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: [row_number() windowspecdefinition(ss_item_sk#23, d_date#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [ss_item_sk#23], [d_date#28 ASC NULLS FIRST] - -(43) Project [codegen id : 9] -Output [4]: [item_sk#30, d_date#28, sumss#31, rk#32] -Input [5]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23, rk#32] - -(44) ReusedExchange [Reuses operator id: 39] -Output [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] - -(45) CometSort -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -Arguments: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34], [ss_item_sk#34 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST] - -(46) CometColumnarToRow [codegen id : 7] -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] - -(47) Window -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -Arguments: [row_number() windowspecdefinition(ss_item_sk#34, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#34], [d_date#33 ASC NULLS FIRST] - -(48) Project [codegen id : 8] -Output [3]: [item_sk#30 AS item_sk#36, sumss#31 AS sumss#37, rk#35] -Input [5]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34, rk#35] - -(49) BroadcastExchange -Input [3]: [item_sk#36, sumss#37, rk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -(50) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [item_sk#30] -Right keys [1]: [item_sk#36] -Join type: Inner -Join condition: (rk#32 >= rk#35) - -(51) Project [codegen id : 9] -Output [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] -Input [7]: [item_sk#30, d_date#28, sumss#31, rk#32, item_sk#36, sumss#37, rk#35] - -(52) HashAggregate [codegen id : 9] -Input [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] -Keys [3]: [item_sk#30, d_date#28, sumss#31] -Functions [1]: [partial_sum(sumss#37)] -Aggregate Attributes [2]: [sum#38, isEmpty#39] -Results [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] - -(53) CometColumnarExchange -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -Arguments: hashpartitioning(item_sk#30, d_date#28, sumss#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(54) CometColumnarToRow [codegen id : 10] -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] - -(55) HashAggregate [codegen id : 10] -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -Keys [3]: [item_sk#30, d_date#28, sumss#31] -Functions [1]: [sum(sumss#37)] -Aggregate Attributes [1]: [sum(sumss#37)#42] -Results [3]: [item_sk#30, d_date#28, sum(sumss#37)#42 AS cume_sales#43] - -(56) CometColumnarExchange -Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: hashpartitioning(item_sk#30, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(57) CometSort -Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#30, d_date#28, cume_sales#43], [item_sk#30 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [3]: [item_sk#9, d_date#6, cume_sales#22] -Right output [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#9, d_date#6], [item_sk#30, d_date#28], FullOuter - -(59) CometFilter -Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END) - -(60) CometProject -Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END AS item_sk#44, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#28 END AS d_date#45, cume_sales#22 AS web_sales#46, cume_sales#43 AS store_sales#47] - -(61) CometExchange -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(62) CometSort -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] - -(63) CometColumnarToRow [codegen id : 11] -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] - -(64) Window -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [item_sk#44], [d_date#45 ASC NULLS FIRST] - -(65) ReusedExchange [Reuses operator id: 61] -Output [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] - -(66) CometSort -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] - -(67) CometColumnarToRow [codegen id : 22] -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] - -(68) Window -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [item_sk#44], [d_date#45 ASC NULLS FIRST] - -(69) Project [codegen id : 23] -Output [4]: [item_sk#44 AS item_sk#50, web_sales#46 AS web_sales#51, store_sales#47 AS store_sales#52, rk#49] -Input [5]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#49] - -(70) BroadcastExchange -Input [4]: [item_sk#50, web_sales#51, store_sales#52, rk#49] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -(71) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [item_sk#44] -Right keys [1]: [item_sk#50] -Join type: Inner -Join condition: (rk#48 >= rk#49) - -(72) Project [codegen id : 24] -Output [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] -Input [9]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#48, item_sk#50, web_sales#51, store_sales#52, rk#49] - -(73) HashAggregate [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] -Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Functions [2]: [partial_max(web_sales#51), partial_max(store_sales#52)] -Aggregate Attributes [2]: [max#53, max#54] -Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] - -(74) HashAggregate [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] -Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Functions [2]: [max(web_sales#51), max(store_sales#52)] -Aggregate Attributes [2]: [max(web_sales#51)#57, max(store_sales#52)#58] -Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max(web_sales#51)#57 AS web_cumulative#59, max(store_sales#52)#58 AS store_cumulative#60] - -(75) Filter [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] -Condition : ((isnotnull(web_cumulative#59) AND isnotnull(store_cumulative#60)) AND (web_cumulative#59 > store_cumulative#60)) - -(76) TakeOrderedAndProject -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] -Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (81) -+- * CometColumnarToRow (80) - +- CometProject (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) - -(79) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(80) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] - -(81) BroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:2 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/simplified.txt deleted file mode 100644 index b3013059b0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,116 +0,0 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (24) - Filter [web_cumulative,store_cumulative] - HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] - HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] - Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (5) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (4) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometExchange [ws_item_sk] #4 - CometHashAggregate [sum] [item_sk,d_date,sumws,ws_item_sk,sum(UnscaledValue(ws_sales_price))] - CometExchange [ws_item_sk,d_date] #5 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [item_sk,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #9 - WholeStageCodegen (10) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumss] #10 - WholeStageCodegen (9) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sum] [item_sk,d_date,sumss,ss_item_sk,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_item_sk,d_date] #12 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - 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 [native_iceberg_compat] 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 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - Project [item_sk,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (23) - Project [item_sk,web_sales,store_sales,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (22) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt deleted file mode 100644 index f8a95a6db8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/explain.txt +++ /dev/null @@ -1,448 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (76) -+- * Filter (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- Window (64) - : +- * CometColumnarToRow (63) - : +- CometSort (62) - : +- CometExchange (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometSortMergeJoin (58) - : :- CometSort (30) - : : +- CometColumnarExchange (29) - : : +- * HashAggregate (28) - : : +- * CometColumnarToRow (27) - : : +- CometColumnarExchange (26) - : : +- * HashAggregate (25) - : : +- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (16) - : : : +- Window (15) - : : : +- * CometColumnarToRow (14) - : : : +- CometSort (13) - : : : +- CometExchange (12) - : : : +- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- BroadcastExchange (22) - : : +- * Project (21) - : : +- Window (20) - : : +- * CometColumnarToRow (19) - : : +- CometSort (18) - : : +- ReusedExchange (17) - : +- CometSort (57) - : +- CometColumnarExchange (56) - : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) - : +- CometColumnarExchange (53) - : +- * HashAggregate (52) - : +- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Project (43) - : : +- Window (42) - : : +- * CometColumnarToRow (41) - : : +- CometSort (40) - : : +- CometExchange (39) - : : +- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometFilter (32) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (31) - : : +- ReusedExchange (33) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- Window (47) - : +- * CometColumnarToRow (46) - : +- CometSort (45) - : +- ReusedExchange (44) - +- BroadcastExchange (70) - +- * Project (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- ReusedExchange (65) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: [d_date_sk#5, d_date#6] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Right output [2]: [d_date_sk#5, d_date#6] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] -Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] - -(9) CometHashAggregate -Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] - -(10) CometExchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] - -(12) CometExchange -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(13) CometSort -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(14) CometColumnarToRow [codegen id : 1] -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] - -(15) Window -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(16) Project [codegen id : 4] -Output [4]: [item_sk#9, d_date#6, sumws#10, rk#11] -Input [5]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1, rk#11] - -(17) ReusedExchange [Reuses operator id: 12] -Output [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] - -(18) CometSort -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -Arguments: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13], [ws_item_sk#13 ASC NULLS FIRST, d_date#12 ASC NULLS FIRST] - -(19) CometColumnarToRow [codegen id : 2] -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] - -(20) Window -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -Arguments: [row_number() windowspecdefinition(ws_item_sk#13, d_date#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#14], [ws_item_sk#13], [d_date#12 ASC NULLS FIRST] - -(21) Project [codegen id : 3] -Output [3]: [item_sk#9 AS item_sk#15, sumws#10 AS sumws#16, rk#14] -Input [5]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13, rk#14] - -(22) BroadcastExchange -Input [3]: [item_sk#15, sumws#16, rk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [item_sk#9] -Right keys [1]: [item_sk#15] -Join type: Inner -Join condition: (rk#11 >= rk#14) - -(24) Project [codegen id : 4] -Output [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] -Input [7]: [item_sk#9, d_date#6, sumws#10, rk#11, item_sk#15, sumws#16, rk#14] - -(25) HashAggregate [codegen id : 4] -Input [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] -Keys [3]: [item_sk#9, d_date#6, sumws#10] -Functions [1]: [partial_sum(sumws#16)] -Aggregate Attributes [2]: [sum#17, isEmpty#18] -Results [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] - -(26) CometColumnarExchange -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] -Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometColumnarToRow [codegen id : 5] -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] - -(28) HashAggregate [codegen id : 5] -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] -Keys [3]: [item_sk#9, d_date#6, sumws#10] -Functions [1]: [sum(sumws#16)] -Aggregate Attributes [1]: [sum(sumws#16)#21] -Results [3]: [item_sk#9, d_date#6, sum(sumws#16)#21 AS cume_sales#22] - -(29) CometColumnarExchange -Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(30) CometSort -Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: [item_sk#9, d_date#6, cume_sales#22], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_item_sk#23) - -(33) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#27, d_date#28] - -(34) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Right output [2]: [d_date_sk#27, d_date#28] -Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight - -(35) CometProject -Input [5]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_date#28] -Arguments: [ss_item_sk#23, ss_sales_price#24, d_date#28], [ss_item_sk#23, ss_sales_price#24, d_date#28] - -(36) CometHashAggregate -Input [3]: [ss_item_sk#23, ss_sales_price#24, d_date#28] -Keys [2]: [ss_item_sk#23, d_date#28] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#24))] - -(37) CometExchange -Input [3]: [ss_item_sk#23, d_date#28, sum#29] -Arguments: hashpartitioning(ss_item_sk#23, d_date#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(38) CometHashAggregate -Input [3]: [ss_item_sk#23, d_date#28, sum#29] -Keys [2]: [ss_item_sk#23, d_date#28] -Functions [1]: [sum(UnscaledValue(ss_sales_price#24))] - -(39) CometExchange -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: hashpartitioning(ss_item_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(40) CometSort -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23], [ss_item_sk#23 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] - -(41) CometColumnarToRow [codegen id : 6] -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] - -(42) Window -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: [row_number() windowspecdefinition(ss_item_sk#23, d_date#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [ss_item_sk#23], [d_date#28 ASC NULLS FIRST] - -(43) Project [codegen id : 9] -Output [4]: [item_sk#30, d_date#28, sumss#31, rk#32] -Input [5]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23, rk#32] - -(44) ReusedExchange [Reuses operator id: 39] -Output [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] - -(45) CometSort -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -Arguments: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34], [ss_item_sk#34 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST] - -(46) CometColumnarToRow [codegen id : 7] -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] - -(47) Window -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -Arguments: [row_number() windowspecdefinition(ss_item_sk#34, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#34], [d_date#33 ASC NULLS FIRST] - -(48) Project [codegen id : 8] -Output [3]: [item_sk#30 AS item_sk#36, sumss#31 AS sumss#37, rk#35] -Input [5]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34, rk#35] - -(49) BroadcastExchange -Input [3]: [item_sk#36, sumss#37, rk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -(50) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [item_sk#30] -Right keys [1]: [item_sk#36] -Join type: Inner -Join condition: (rk#32 >= rk#35) - -(51) Project [codegen id : 9] -Output [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] -Input [7]: [item_sk#30, d_date#28, sumss#31, rk#32, item_sk#36, sumss#37, rk#35] - -(52) HashAggregate [codegen id : 9] -Input [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] -Keys [3]: [item_sk#30, d_date#28, sumss#31] -Functions [1]: [partial_sum(sumss#37)] -Aggregate Attributes [2]: [sum#38, isEmpty#39] -Results [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] - -(53) CometColumnarExchange -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -Arguments: hashpartitioning(item_sk#30, d_date#28, sumss#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(54) CometColumnarToRow [codegen id : 10] -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] - -(55) HashAggregate [codegen id : 10] -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -Keys [3]: [item_sk#30, d_date#28, sumss#31] -Functions [1]: [sum(sumss#37)] -Aggregate Attributes [1]: [sum(sumss#37)#42] -Results [3]: [item_sk#30, d_date#28, sum(sumss#37)#42 AS cume_sales#43] - -(56) CometColumnarExchange -Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: hashpartitioning(item_sk#30, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(57) CometSort -Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#30, d_date#28, cume_sales#43], [item_sk#30 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [3]: [item_sk#9, d_date#6, cume_sales#22] -Right output [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#9, d_date#6], [item_sk#30, d_date#28], FullOuter - -(59) CometFilter -Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END) - -(60) CometProject -Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END AS item_sk#44, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#28 END AS d_date#45, cume_sales#22 AS web_sales#46, cume_sales#43 AS store_sales#47] - -(61) CometExchange -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(62) CometSort -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] - -(63) CometColumnarToRow [codegen id : 11] -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] - -(64) Window -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [item_sk#44], [d_date#45 ASC NULLS FIRST] - -(65) ReusedExchange [Reuses operator id: 61] -Output [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] - -(66) CometSort -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] - -(67) CometColumnarToRow [codegen id : 22] -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] - -(68) Window -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [item_sk#44], [d_date#45 ASC NULLS FIRST] - -(69) Project [codegen id : 23] -Output [4]: [item_sk#44 AS item_sk#50, web_sales#46 AS web_sales#51, store_sales#47 AS store_sales#52, rk#49] -Input [5]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#49] - -(70) BroadcastExchange -Input [4]: [item_sk#50, web_sales#51, store_sales#52, rk#49] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -(71) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [item_sk#44] -Right keys [1]: [item_sk#50] -Join type: Inner -Join condition: (rk#48 >= rk#49) - -(72) Project [codegen id : 24] -Output [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] -Input [9]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#48, item_sk#50, web_sales#51, store_sales#52, rk#49] - -(73) HashAggregate [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] -Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Functions [2]: [partial_max(web_sales#51), partial_max(store_sales#52)] -Aggregate Attributes [2]: [max#53, max#54] -Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] - -(74) HashAggregate [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] -Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Functions [2]: [max(web_sales#51), max(store_sales#52)] -Aggregate Attributes [2]: [max(web_sales#51)#57, max(store_sales#52)#58] -Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max(web_sales#51)#57 AS web_cumulative#59, max(store_sales#52)#58 AS store_cumulative#60] - -(75) Filter [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] -Condition : ((isnotnull(web_cumulative#59) AND isnotnull(store_cumulative#60)) AND (web_cumulative#59 > store_cumulative#60)) - -(76) TakeOrderedAndProject -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] -Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (81) -+- * CometColumnarToRow (80) - +- CometProject (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) - -(79) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(80) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] - -(81) BroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:2 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/extended.txt deleted file mode 100644 index f7d3371108..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/extended.txt +++ /dev/null @@ -1,216 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : : +- CometColumnarToRow - : : : +- CometSort - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometSortMergeJoin - :- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 138 out of 196 eligible operators (70%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index b3013059b0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/simplified.txt +++ /dev/null @@ -1,116 +0,0 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (24) - Filter [web_cumulative,store_cumulative] - HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] - HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] - Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (5) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (4) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometExchange [ws_item_sk] #4 - CometHashAggregate [sum] [item_sk,d_date,sumws,ws_item_sk,sum(UnscaledValue(ws_sales_price))] - CometExchange [ws_item_sk,d_date] #5 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [item_sk,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #9 - WholeStageCodegen (10) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumss] #10 - WholeStageCodegen (9) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sum] [item_sk,d_date,sumss,ss_item_sk,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_item_sk,d_date] #12 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - 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 [native_iceberg_compat] 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 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - Project [item_sk,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (23) - Project [item_sk,web_sales,store_sales,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (22) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/explain.txt deleted file mode 100644 index be90722d2e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/explain.txt +++ /dev/null @@ -1,304 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (51) -+- * Project (50) - +- * BroadcastHashJoin Inner BuildRight (49) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * CometColumnarToRow (26) - : : +- CometSort (25) - : : +- CometColumnarExchange (24) - : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- * HashAggregate (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- Window (38) - : +- * CometColumnarToRow (37) - : +- CometSort (36) - : +- CometColumnarExchange (35) - : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) - : +- ReusedExchange (32) - +- BroadcastExchange (48) - +- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- ReusedExchange (43) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#4, i_category#5] - -(5) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(7) Filter [codegen id : 1] -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) - -(8) BroadcastExchange -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#7] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(11) ReusedExchange [Reuses operator id: 55] -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] - -(14) CometNativeScan parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#14, cc_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#14, cc_name#15] -Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) - -(16) CometColumnarToRow [codegen id : 3] -Input [2]: [cc_call_center_sk#14, cc_name#15] - -(17) BroadcastExchange -Input [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_call_center_sk#6] -Right keys [1]: [cc_call_center_sk#14] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] - -(20) HashAggregate [codegen id : 4] -Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] -Aggregate Attributes [1]: [sum#16] -Results [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] - -(21) CometColumnarExchange -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] - -(23) HashAggregate [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#8))#18] -Results [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#18,17,2) AS sum_sales#19, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#18,17,2) AS _w0#20] - -(24) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) CometColumnarToRow [codegen id : 6] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] - -(27) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(28) Filter [codegen id : 7] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(29) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] -Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(30) Filter [codegen id : 22] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] -Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) - -(31) Project [codegen id : 22] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] - -(32) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] - -(33) CometColumnarToRow [codegen id : 12] -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] - -(34) HashAggregate [codegen id : 12] -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] -Keys [5]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#27))#18] -Results [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#18,17,2) AS sum_sales#19] - -(35) CometColumnarExchange -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(36) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] -Arguments: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(37) CometColumnarToRow [codegen id : 13] -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] - -(38) Window -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(39) Project [codegen id : 14] -Output [5]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, cc_name#23, sum_sales#19 AS sum_sales#31, rn#28] -Input [7]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19, rn#28] - -(40) BroadcastExchange -Input [5]: [i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] - -(41) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] -Right keys [4]: [i_category#29, i_brand#30, cc_name#23, (rn#28 + 1)] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 22] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] - -(43) ReusedExchange [Reuses operator id: 35] -Output [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] - -(44) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] -Arguments: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] - -(45) CometColumnarToRow [codegen id : 20] -Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] - -(46) Window -Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] -Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#5, i_brand#4, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#5, i_brand#4, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] - -(47) Project [codegen id : 21] -Output [5]: [i_category#5 AS i_category#36, i_brand#4 AS i_brand#37, cc_name#32, sum_sales#19 AS sum_sales#38, rn#35] -Input [7]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19, rn#35] - -(48) BroadcastExchange -Input [5]: [i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] - -(49) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] -Right keys [4]: [i_category#36, i_brand#37, cc_name#32, (rn#35 - 1)] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 22] -Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#39, sum_sales#38 AS nsum#40] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] - -(51) TakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] -Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometFilter (53) - +- CometNativeScan parquet spark_catalog.default.date_dim (52) - - -(52) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(54) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(55) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/simplified.txt deleted file mode 100644 index e603f05322..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/simplified.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (22) - Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (7) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [cs_item_sk,cs_call_center_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk,cc_name] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/explain.txt deleted file mode 100644 index d1daee98c1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) - -(6) CometBroadcastExchange -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_item_sk#1], [cs_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#14, cc_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#14, cc_name#15] -Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) - -(16) CometBroadcastExchange -Input [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cc_call_center_sk#14, cc_name#15] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] -Right output [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cs_call_center_sk#6], [cc_call_center_sk#14], Inner, BuildRight - -(18) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] -Arguments: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15], [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] - -(19) CometHashAggregate -Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] - -(20) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] - -(22) CometExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] - -(25) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(28) Filter [codegen id : 7] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] - -(30) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] - -(31) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] -Keys [5]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23] -Functions [1]: [sum(UnscaledValue(cs_sales_price#25))] - -(32) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] - -(35) Window -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [5]: [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] -Input [7]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26] - -(37) BroadcastExchange -Input [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] - -(40) ReusedExchange [Reuses operator id: 32] -Output [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(41) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(43) Window -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#5, i_brand#4, cc_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#5, i_brand#4, cc_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [5]: [i_category#5 AS i_category#34, i_brand#4 AS i_brand#35, cc_name#30, sum_sales#17 AS sum_sales#36, rn#33] -Input [7]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17, rn#33] - -(45) BroadcastExchange -Input [5]: [i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#34, i_brand#35, cc_name#30, (rn#33 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] - -(48) TakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/simplified.txt deleted file mode 100644 index d655789fe6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - 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 [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,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt deleted file mode 100644 index d1daee98c1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) - -(6) CometBroadcastExchange -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_item_sk#1], [cs_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#14, cc_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#14, cc_name#15] -Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) - -(16) CometBroadcastExchange -Input [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cc_call_center_sk#14, cc_name#15] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] -Right output [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cs_call_center_sk#6], [cc_call_center_sk#14], Inner, BuildRight - -(18) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] -Arguments: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15], [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] - -(19) CometHashAggregate -Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] - -(20) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] - -(22) CometExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] - -(25) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(28) Filter [codegen id : 7] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] - -(30) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] - -(31) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] -Keys [5]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23] -Functions [1]: [sum(UnscaledValue(cs_sales_price#25))] - -(32) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] - -(35) Window -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [5]: [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] -Input [7]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26] - -(37) BroadcastExchange -Input [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] - -(40) ReusedExchange [Reuses operator id: 32] -Output [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(41) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(43) Window -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#5, i_brand#4, cc_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#5, i_brand#4, cc_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [5]: [i_category#5 AS i_category#34, i_brand#4 AS i_brand#35, cc_name#30, sum_sales#17 AS sum_sales#36, rn#33] -Input [7]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17, rn#33] - -(45) BroadcastExchange -Input [5]: [i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#34, i_brand#35, cc_name#30, (rn#33 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] - -(48) TakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/extended.txt deleted file mode 100644 index de7d26cd06..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d655789fe6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - 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 [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,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/explain.txt deleted file mode 100644 index 62a1484a64..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/explain.txt +++ /dev/null @@ -1,602 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (99) -+- CometTakeOrderedAndProject (98) - +- CometHashAggregate (97) - +- CometColumnarExchange (96) - +- * HashAggregate (95) - +- Union (94) - :- * HashAggregate (79) - : +- * CometColumnarToRow (78) - : +- CometColumnarExchange (77) - : +- * HashAggregate (76) - : +- Union (75) - : :- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- * HashAggregate (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- Union (9) - : : : : :- * Project (4) - : : : : : +- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- * Project (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_returns (5) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometProject (15) - : : +- CometFilter (14) - : : +- CometNativeScan parquet spark_catalog.default.store (13) - : :- * HashAggregate (46) - : : +- * CometColumnarToRow (45) - : : +- CometColumnarExchange (44) - : : +- * HashAggregate (43) - : : +- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- Union (32) - : : : : :- * Project (27) - : : : : : +- * Filter (26) - : : : : : +- * ColumnarToRow (25) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (24) - : : : : +- * Project (31) - : : : : +- * Filter (30) - : : : : +- * ColumnarToRow (29) - : : : : +- Scan parquet spark_catalog.default.catalog_returns (28) - : : : +- ReusedExchange (33) - : : +- BroadcastExchange (40) - : : +- * CometColumnarToRow (39) - : : +- CometProject (38) - : : +- CometFilter (37) - : : +- CometNativeScan parquet spark_catalog.default.catalog_page (36) - : +- * HashAggregate (74) - : +- * CometColumnarToRow (73) - : +- CometColumnarExchange (72) - : +- * HashAggregate (71) - : +- * Project (70) - : +- * BroadcastHashJoin Inner BuildRight (69) - : :- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- Union (60) - : : : :- * Project (50) - : : : : +- * Filter (49) - : : : : +- * ColumnarToRow (48) - : : : : +- Scan parquet spark_catalog.default.web_sales (47) - : : : +- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildLeft (58) - : : : :- BroadcastExchange (53) - : : : : +- * ColumnarToRow (52) - : : : : +- Scan parquet spark_catalog.default.web_returns (51) - : : : +- * CometColumnarToRow (57) - : : : +- CometProject (56) - : : : +- CometFilter (55) - : : : +- CometNativeScan parquet spark_catalog.default.web_sales (54) - : : +- ReusedExchange (61) - : +- BroadcastExchange (68) - : +- * CometColumnarToRow (67) - : +- CometProject (66) - : +- CometFilter (65) - : +- CometNativeScan parquet spark_catalog.default.web_site (64) - :- * HashAggregate (86) - : +- * CometColumnarToRow (85) - : +- CometColumnarExchange (84) - : +- * HashAggregate (83) - : +- * HashAggregate (82) - : +- * CometColumnarToRow (81) - : +- ReusedExchange (80) - +- * HashAggregate (93) - +- * CometColumnarToRow (92) - +- CometColumnarExchange (91) - +- * HashAggregate (90) - +- * HashAggregate (89) - +- * CometColumnarToRow (88) - +- ReusedExchange (87) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 1] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(4) Project [codegen id : 1] -Output [6]: [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] - -(5) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 2] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] - -(7) Filter [codegen id : 2] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#12) - -(8) Project [codegen id : 2] -Output [6]: [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] - -(9) Union - -(10) ReusedExchange [Reuses operator id: 104] -Output [1]: [d_date_sk#22] - -(11) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [date_sk#7] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None - -(12) Project [codegen id : 5] -Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] - -(13) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#23, s_store_id#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [s_store_sk#23, s_store_id#24] -Condition : isnotnull(s_store_sk#23) - -(15) CometProject -Input [2]: [s_store_sk#23, s_store_id#24] -Arguments: [s_store_sk#23, s_store_id#25], [s_store_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#24, 16, true, false, true) AS s_store_id#25] - -(16) CometColumnarToRow [codegen id : 4] -Input [2]: [s_store_sk#23, s_store_id#25] - -(17) BroadcastExchange -Input [2]: [s_store_sk#23, s_store_id#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(18) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [store_sk#6] -Right keys [1]: [s_store_sk#23] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 5] -Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#25] - -(20) HashAggregate [codegen id : 5] -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -Keys [1]: [s_store_id#25] -Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum#26, sum#27, sum#28, sum#29] -Results [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] - -(21) CometColumnarExchange -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(22) CometColumnarToRow [codegen id : 6] -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] - -(23) HashAggregate [codegen id : 6] -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -Keys [1]: [s_store_id#25] -Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#34, sum(UnscaledValue(return_amt#10))#35, sum(UnscaledValue(profit#9))#36, sum(UnscaledValue(net_loss#11))#37] -Results [5]: [store channel AS channel#38, concat(store, s_store_id#25) AS id#39, MakeDecimal(sum(UnscaledValue(sales_price#8))#34,17,2) AS sales#40, MakeDecimal(sum(UnscaledValue(return_amt#10))#35,17,2) AS returns#41, (MakeDecimal(sum(UnscaledValue(profit#9))#36,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#37,17,2)) AS profit#42] - -(24) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_sold_date_sk#46 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cs_catalog_page_sk)] -ReadSchema: struct - -(25) ColumnarToRow [codegen id : 7] -Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] - -(26) Filter [codegen id : 7] -Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Condition : isnotnull(cs_catalog_page_sk#43) - -(27) Project [codegen id : 7] -Output [6]: [cs_catalog_page_sk#43 AS page_sk#47, cs_sold_date_sk#46 AS date_sk#48, cs_ext_sales_price#44 AS sales_price#49, cs_net_profit#45 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] -Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] - -(28) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cr_catalog_page_sk)] -ReadSchema: struct - -(29) ColumnarToRow [codegen id : 8] -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] - -(30) Filter [codegen id : 8] -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -Condition : isnotnull(cr_catalog_page_sk#53) - -(31) Project [codegen id : 8] -Output [6]: [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] - -(32) Union - -(33) ReusedExchange [Reuses operator id: 104] -Output [1]: [d_date_sk#63] - -(34) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [date_sk#48] -Right keys [1]: [d_date_sk#63] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 11] -Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] -Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] - -(36) CometNativeScan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Condition : isnotnull(cp_catalog_page_sk#64) - -(38) CometProject -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Arguments: [cp_catalog_page_sk#64, cp_catalog_page_id#66], [cp_catalog_page_sk#64, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#65, 16, true, false, true) AS cp_catalog_page_id#66] - -(39) CometColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] - -(40) BroadcastExchange -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(41) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [page_sk#47] -Right keys [1]: [cp_catalog_page_sk#64] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 11] -Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] -Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#66] - -(43) HashAggregate [codegen id : 11] -Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] -Keys [1]: [cp_catalog_page_id#66] -Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] -Aggregate Attributes [4]: [sum#67, sum#68, sum#69, sum#70] -Results [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] - -(44) CometColumnarExchange -Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] -Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(45) CometColumnarToRow [codegen id : 12] -Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] - -(46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] -Keys [1]: [cp_catalog_page_id#66] -Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#75, sum(UnscaledValue(return_amt#51))#76, sum(UnscaledValue(profit#50))#77, sum(UnscaledValue(net_loss#52))#78] -Results [5]: [catalog channel AS channel#79, concat(catalog_page, cp_catalog_page_id#66) AS id#80, MakeDecimal(sum(UnscaledValue(sales_price#49))#75,17,2) AS sales#81, MakeDecimal(sum(UnscaledValue(return_amt#51))#76,17,2) AS returns#82, (MakeDecimal(sum(UnscaledValue(profit#50))#77,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#52))#78,17,2)) AS profit#83] - -(47) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_sold_date_sk#87 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] - -(49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] -Condition : isnotnull(ws_web_site_sk#84) - -(50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#84 AS wsr_web_site_sk#88, ws_sold_date_sk#87 AS date_sk#89, ws_ext_sales_price#85 AS sales_price#90, ws_net_profit#86 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] -Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] - -(51) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#5)] -ReadSchema: struct - -(52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] - -(53) BroadcastExchange -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [plan_id=5] - -(54) CometNativeScan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(55) CometFilter -Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) - -(56) CometProject -Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] - -(57) CometColumnarToRow -Input [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] - -(58) BroadcastHashJoin [codegen id : 15] -Left keys [2]: [wr_item_sk#94, wr_order_number#95] -Right keys [2]: [ws_item_sk#99, ws_order_number#101] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 15] -Output [6]: [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] -Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] - -(60) Union - -(61) ReusedExchange [Reuses operator id: 104] -Output [1]: [d_date_sk#109] - -(62) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [date_sk#89] -Right keys [1]: [d_date_sk#109] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 18] -Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] -Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] - -(64) CometNativeScan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#110, web_site_id#111] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(65) CometFilter -Input [2]: [web_site_sk#110, web_site_id#111] -Condition : isnotnull(web_site_sk#110) - -(66) CometProject -Input [2]: [web_site_sk#110, web_site_id#111] -Arguments: [web_site_sk#110, web_site_id#112], [web_site_sk#110, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#111, 16, true, false, true) AS web_site_id#112] - -(67) CometColumnarToRow [codegen id : 17] -Input [2]: [web_site_sk#110, web_site_id#112] - -(68) BroadcastExchange -Input [2]: [web_site_sk#110, web_site_id#112] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(69) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [wsr_web_site_sk#88] -Right keys [1]: [web_site_sk#110] -Join type: Inner -Join condition: None - -(70) Project [codegen id : 18] -Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] -Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#112] - -(71) HashAggregate [codegen id : 18] -Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] -Keys [1]: [web_site_id#112] -Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum#113, sum#114, sum#115, sum#116] -Results [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] - -(72) CometColumnarExchange -Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] -Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(73) CometColumnarToRow [codegen id : 19] -Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] - -(74) HashAggregate [codegen id : 19] -Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] -Keys [1]: [web_site_id#112] -Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#121, sum(UnscaledValue(return_amt#92))#122, sum(UnscaledValue(profit#91))#123, sum(UnscaledValue(net_loss#93))#124] -Results [5]: [web channel AS channel#125, concat(web_site, web_site_id#112) AS id#126, MakeDecimal(sum(UnscaledValue(sales_price#90))#121,17,2) AS sales#127, MakeDecimal(sum(UnscaledValue(return_amt#92))#122,17,2) AS returns#128, (MakeDecimal(sum(UnscaledValue(profit#91))#123,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#124,17,2)) AS profit#129] - -(75) Union - -(76) HashAggregate [codegen id : 20] -Input [5]: [channel#38, id#39, sales#40, returns#41, profit#42] -Keys [2]: [channel#38, id#39] -Functions [3]: [partial_sum(sales#40), partial_sum(returns#41), partial_sum(profit#42)] -Aggregate Attributes [6]: [sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] -Results [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] - -(77) CometColumnarExchange -Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -Arguments: hashpartitioning(channel#38, id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(78) CometColumnarToRow [codegen id : 21] -Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] - -(79) HashAggregate [codegen id : 21] -Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -Keys [2]: [channel#38, id#39] -Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] -Aggregate Attributes [3]: [sum(sales#40)#142, sum(returns#41)#143, sum(profit#42)#144] -Results [5]: [channel#38, id#39, cast(sum(sales#40)#142 as decimal(37,2)) AS sales#145, cast(sum(returns#41)#143 as decimal(37,2)) AS returns#146, cast(sum(profit#42)#144 as decimal(38,2)) AS profit#147] - -(80) ReusedExchange [Reuses operator id: 77] -Output [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] - -(81) CometColumnarToRow [codegen id : 42] -Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] - -(82) HashAggregate [codegen id : 42] -Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -Keys [2]: [channel#38, id#39] -Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] -Aggregate Attributes [3]: [sum(sales#40)#142, sum(returns#41)#143, sum(profit#42)#144] -Results [4]: [channel#38, sum(sales#40)#142 AS sales#148, sum(returns#41)#143 AS returns#149, sum(profit#42)#144 AS profit#150] - -(83) HashAggregate [codegen id : 42] -Input [4]: [channel#38, sales#148, returns#149, profit#150] -Keys [1]: [channel#38] -Functions [3]: [partial_sum(sales#148), partial_sum(returns#149), partial_sum(profit#150)] -Aggregate Attributes [6]: [sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156] -Results [7]: [channel#38, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] - -(84) CometColumnarExchange -Input [7]: [channel#38, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] -Arguments: hashpartitioning(channel#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(85) CometColumnarToRow [codegen id : 43] -Input [7]: [channel#38, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] - -(86) HashAggregate [codegen id : 43] -Input [7]: [channel#38, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] -Keys [1]: [channel#38] -Functions [3]: [sum(sales#148), sum(returns#149), sum(profit#150)] -Aggregate Attributes [3]: [sum(sales#148)#163, sum(returns#149)#164, sum(profit#150)#165] -Results [5]: [channel#38, null AS id#166, sum(sales#148)#163 AS sum(sales)#167, sum(returns#149)#164 AS sum(returns)#168, sum(profit#150)#165 AS sum(profit)#169] - -(87) ReusedExchange [Reuses operator id: 77] -Output [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] - -(88) CometColumnarToRow [codegen id : 64] -Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] - -(89) HashAggregate [codegen id : 64] -Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -Keys [2]: [channel#38, id#39] -Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] -Aggregate Attributes [3]: [sum(sales#40)#142, sum(returns#41)#143, sum(profit#42)#144] -Results [3]: [sum(sales#40)#142 AS sales#170, sum(returns#41)#143 AS returns#171, sum(profit#42)#144 AS profit#172] - -(90) HashAggregate [codegen id : 64] -Input [3]: [sales#170, returns#171, profit#172] -Keys: [] -Functions [3]: [partial_sum(sales#170), partial_sum(returns#171), partial_sum(profit#172)] -Aggregate Attributes [6]: [sum#173, isEmpty#174, sum#175, isEmpty#176, sum#177, isEmpty#178] -Results [6]: [sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184] - -(91) CometColumnarExchange -Input [6]: [sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(92) CometColumnarToRow [codegen id : 65] -Input [6]: [sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184] - -(93) HashAggregate [codegen id : 65] -Input [6]: [sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184] -Keys: [] -Functions [3]: [sum(sales#170), sum(returns#171), sum(profit#172)] -Aggregate Attributes [3]: [sum(sales#170)#185, sum(returns#171)#186, sum(profit#172)#187] -Results [5]: [null AS channel#188, null AS id#189, sum(sales#170)#185 AS sum(sales)#190, sum(returns#171)#186 AS sum(returns)#191, sum(profit#172)#187 AS sum(profit)#192] - -(94) Union - -(95) HashAggregate [codegen id : 66] -Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] -Keys [5]: [channel#38, id#39, sales#145, returns#146, profit#147] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#38, id#39, sales#145, returns#146, profit#147] - -(96) CometColumnarExchange -Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] -Arguments: hashpartitioning(channel#38, id#39, sales#145, returns#146, profit#147, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(97) CometHashAggregate -Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] -Keys [5]: [channel#38, id#39, sales#145, returns#146, profit#147] -Functions: [] - -(98) CometTakeOrderedAndProject -Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#38 ASC NULLS FIRST,id#39 ASC NULLS FIRST], output=[channel#38,id#39,sales#145,returns#146,profit#147]), [channel#38, id#39, sales#145, returns#146, profit#147], 100, 0, [channel#38 ASC NULLS FIRST, id#39 ASC NULLS FIRST], [channel#38, id#39, sales#145, returns#146, profit#147] - -(99) CometColumnarToRow [codegen id : 67] -Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (104) -+- * CometColumnarToRow (103) - +- CometProject (102) - +- CometFilter (101) - +- CometNativeScan parquet spark_catalog.default.date_dim (100) - - -(100) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#193] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] -ReadSchema: struct - -(101) CometFilter -Input [2]: [d_date_sk#22, d_date#193] -Condition : (((isnotnull(d_date#193) AND (d_date#193 >= 1998-08-04)) AND (d_date#193 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) - -(102) CometProject -Input [2]: [d_date_sk#22, d_date#193] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(103) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(104) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 28 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#87 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/simplified.txt deleted file mode 100644 index dc9c9b1ce7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/simplified.txt +++ /dev/null @@ -1,167 +0,0 @@ -WholeStageCodegen (67) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (66) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (21) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (20) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [s_store_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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_id] #3 - WholeStageCodegen (5) - HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,s_store_id] - BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (1) - Project [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - WholeStageCodegen (2) - Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] - Filter [sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (12) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cp_catalog_page_id] #6 - WholeStageCodegen (11) - HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (7) - Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] - Filter [cs_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (8) - Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] - Filter [cr_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (19) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [web_site_id] #8 - WholeStageCodegen (18) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (13) - Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] - Filter [ws_web_site_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (15) - Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (14) - ColumnarToRow - InputAdapter - Scan 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 - CometColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (17) - CometColumnarToRow - InputAdapter - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - WholeStageCodegen (43) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #11 - WholeStageCodegen (42) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (65) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #12 - WholeStageCodegen (64) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/explain.txt deleted file mode 100644 index 33c3a5de2e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,524 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (84) -+- CometTakeOrderedAndProject (83) - +- CometHashAggregate (82) - +- CometExchange (81) - +- CometHashAggregate (80) - +- CometUnion (79) - :- CometHashAggregate (68) - : +- CometExchange (67) - : +- CometHashAggregate (66) - : +- CometUnion (65) - : :- CometHashAggregate (22) - : : +- CometExchange (21) - : : +- CometHashAggregate (20) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometUnion (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : :- CometHashAggregate (41) - : : +- CometExchange (40) - : : +- CometHashAggregate (39) - : : +- CometProject (38) - : : +- CometBroadcastHashJoin (37) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometUnion (29) - : : : : :- CometProject (25) - : : : : : +- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) - : : : : +- CometProject (28) - : : : : +- CometFilter (27) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) - : : : +- ReusedExchange (30) - : : +- CometBroadcastExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) - : +- CometHashAggregate (64) - : +- CometExchange (63) - : +- CometHashAggregate (62) - : +- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (55) - : : +- CometBroadcastHashJoin (54) - : : :- CometUnion (52) - : : : :- CometProject (44) - : : : : +- CometFilter (43) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : : : +- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometBroadcastExchange (46) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) - : : : +- CometProject (49) - : : : +- CometFilter (48) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : : +- ReusedExchange (53) - : +- CometBroadcastExchange (59) - : +- CometProject (58) - : +- CometFilter (57) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) - :- CometHashAggregate (73) - : +- CometExchange (72) - : +- CometHashAggregate (71) - : +- CometHashAggregate (70) - : +- ReusedExchange (69) - +- CometHashAggregate (78) - +- CometExchange (77) - +- CometHashAggregate (76) - +- CometHashAggregate (75) - +- ReusedExchange (74) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Arguments: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11], [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#12) - -(6) CometProject -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Arguments: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21], [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] - -(7) CometUnion -Child 0 Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) - -(10) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(11) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(12) CometBroadcastHashJoin -Left output [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [1]: [d_date_sk#22] -Arguments: [date_sk#7], [d_date_sk#22], Inner, BuildRight - -(13) CometProject -Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] -Arguments: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11], [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_store_id#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#24, s_store_id#25] -Condition : isnotnull(s_store_sk#24) - -(16) CometProject -Input [2]: [s_store_sk#24, s_store_id#25] -Arguments: [s_store_sk#24, s_store_id#26], [s_store_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#25, 16, true, false, true) AS s_store_id#26] - -(17) CometBroadcastExchange -Input [2]: [s_store_sk#24, s_store_id#26] -Arguments: [s_store_sk#24, s_store_id#26] - -(18) CometBroadcastHashJoin -Left output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [2]: [s_store_sk#24, s_store_id#26] -Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight - -(19) CometProject -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#26] -Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] - -(20) CometHashAggregate -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] -Keys [1]: [s_store_id#26] -Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] - -(21) CometExchange -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Keys [1]: [s_store_id#26] -Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk)] -ReadSchema: struct - -(24) CometFilter -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : isnotnull(cs_catalog_page_sk#31) - -(25) CometProject -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cr_catalog_page_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Condition : isnotnull(cr_catalog_page_sk#42) - -(28) CometProject -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] - -(29) CometUnion -Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] - -(30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#52] - -(31) CometBroadcastHashJoin -Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [1]: [d_date_sk#52] -Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight - -(32) CometProject -Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] -Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Condition : isnotnull(cp_catalog_page_sk#53) - -(35) CometProject -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#54, 16, true, false, true) AS cp_catalog_page_id#55] - -(36) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] - -(37) CometBroadcastHashJoin -Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight - -(38) CometProject -Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] - -(39) CometHashAggregate -Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] - -(40) CometExchange -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(41) CometHashAggregate -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] -PushedFilters: [IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(43) CometFilter -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_web_site_sk#60) - -(44) CometProject -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] -ReadSchema: struct - -(46) CometBroadcastExchange -Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(48) CometFilter -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) - -(49) CometProject -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] - -(50) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft - -(51) CometProject -Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] - -(52) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] - -(53) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#86] - -(54) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [1]: [d_date_sk#86] -Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight - -(55) CometProject -Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] -Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#87, web_site_id#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(57) CometFilter -Input [2]: [web_site_sk#87, web_site_id#88] -Condition : isnotnull(web_site_sk#87) - -(58) CometProject -Input [2]: [web_site_sk#87, web_site_id#88] -Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#88, 16, true, false, true) AS web_site_id#89] - -(59) CometBroadcastExchange -Input [2]: [web_site_sk#87, web_site_id#89] -Arguments: [web_site_sk#87, web_site_id#89] - -(60) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [2]: [web_site_sk#87, web_site_id#89] -Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight - -(61) CometProject -Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] -Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] - -(62) CometHashAggregate -Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] -Keys [1]: [web_site_id#89] -Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] - -(63) CometExchange -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(64) CometHashAggregate -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Keys [1]: [web_site_id#89] -Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] - -(65) CometUnion -Child 0 Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] -Child 1 Input [5]: [channel#99, id#100, sales#101, returns#102, profit#103] -Child 2 Input [5]: [channel#104, id#105, sales#106, returns#107, profit#108] - -(66) CometHashAggregate -Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] -Keys [2]: [channel#94, id#95] -Functions [3]: [partial_sum(sales#96), partial_sum(returns#97), partial_sum(profit#98)] - -(67) CometExchange -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: hashpartitioning(channel#94, id#95, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(68) CometHashAggregate -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#94, id#95] -Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] - -(69) ReusedExchange [Reuses operator id: 67] -Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(70) CometHashAggregate -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#94, id#95] -Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] - -(71) CometHashAggregate -Input [4]: [channel#94, sales#115, returns#116, profit#117] -Keys [1]: [channel#94] -Functions [3]: [partial_sum(sales#115), partial_sum(returns#116), partial_sum(profit#117)] - -(72) CometExchange -Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] -Arguments: hashpartitioning(channel#94, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(73) CometHashAggregate -Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] -Keys [1]: [channel#94] -Functions [3]: [sum(sales#115), sum(returns#116), sum(profit#117)] - -(74) ReusedExchange [Reuses operator id: 67] -Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(75) CometHashAggregate -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#94, id#95] -Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] - -(76) CometHashAggregate -Input [3]: [sales#124, returns#125, profit#126] -Keys: [] -Functions [3]: [partial_sum(sales#124), partial_sum(returns#125), partial_sum(profit#126)] - -(77) CometExchange -Input [6]: [sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(78) CometHashAggregate -Input [6]: [sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys: [] -Functions [3]: [sum(sales#124), sum(returns#125), sum(profit#126)] - -(79) CometUnion -Child 0 Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Child 1 Input [5]: [channel#94, id#136, sum(sales)#137, sum(returns)#138, sum(profit)#139] -Child 2 Input [5]: [channel#140, id#141, sum(sales)#142, sum(returns)#143, sum(profit)#144] - -(80) CometHashAggregate -Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Keys [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Functions: [] - -(81) CometExchange -Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Arguments: hashpartitioning(channel#94, id#95, sales#133, returns#134, profit#135, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(82) CometHashAggregate -Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Keys [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Functions: [] - -(83) CometTakeOrderedAndProject -Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#94 ASC NULLS FIRST,id#95 ASC NULLS FIRST], output=[channel#94,id#95,sales#133,returns#134,profit#135]), [channel#94, id#95, sales#133, returns#134, profit#135], 100, 0, [channel#94 ASC NULLS FIRST, id#95 ASC NULLS FIRST], [channel#94, id#95, sales#133, returns#134, profit#135] - -(84) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (89) -+- * CometColumnarToRow (88) - +- CometProject (87) - +- CometFilter (86) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) - - -(85) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] -ReadSchema: struct - -(86) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) - -(87) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(88) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(89) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/simplified.txt deleted file mode 100644 index c0b236a57a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,99 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id] #2 - CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [s_store_id] #3 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - 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 [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,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #6 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [cp_catalog_page_id] #7 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - 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 [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,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [cp_catalog_page_sk,cp_catalog_page_id] #8 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [web_site_id] #9 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - 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 [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,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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,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 [native_iceberg_compat] 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_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 [web_site_sk,web_site_id] #11 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] - CometExchange [channel] #12 - CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] - CometExchange #13 - CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt deleted file mode 100644 index 33c3a5de2e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt +++ /dev/null @@ -1,524 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (84) -+- CometTakeOrderedAndProject (83) - +- CometHashAggregate (82) - +- CometExchange (81) - +- CometHashAggregate (80) - +- CometUnion (79) - :- CometHashAggregate (68) - : +- CometExchange (67) - : +- CometHashAggregate (66) - : +- CometUnion (65) - : :- CometHashAggregate (22) - : : +- CometExchange (21) - : : +- CometHashAggregate (20) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometUnion (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : :- CometHashAggregate (41) - : : +- CometExchange (40) - : : +- CometHashAggregate (39) - : : +- CometProject (38) - : : +- CometBroadcastHashJoin (37) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometUnion (29) - : : : : :- CometProject (25) - : : : : : +- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) - : : : : +- CometProject (28) - : : : : +- CometFilter (27) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) - : : : +- ReusedExchange (30) - : : +- CometBroadcastExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) - : +- CometHashAggregate (64) - : +- CometExchange (63) - : +- CometHashAggregate (62) - : +- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (55) - : : +- CometBroadcastHashJoin (54) - : : :- CometUnion (52) - : : : :- CometProject (44) - : : : : +- CometFilter (43) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : : : +- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometBroadcastExchange (46) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) - : : : +- CometProject (49) - : : : +- CometFilter (48) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : : +- ReusedExchange (53) - : +- CometBroadcastExchange (59) - : +- CometProject (58) - : +- CometFilter (57) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) - :- CometHashAggregate (73) - : +- CometExchange (72) - : +- CometHashAggregate (71) - : +- CometHashAggregate (70) - : +- ReusedExchange (69) - +- CometHashAggregate (78) - +- CometExchange (77) - +- CometHashAggregate (76) - +- CometHashAggregate (75) - +- ReusedExchange (74) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Arguments: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11], [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#12) - -(6) CometProject -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Arguments: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21], [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] - -(7) CometUnion -Child 0 Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) - -(10) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(11) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(12) CometBroadcastHashJoin -Left output [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [1]: [d_date_sk#22] -Arguments: [date_sk#7], [d_date_sk#22], Inner, BuildRight - -(13) CometProject -Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] -Arguments: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11], [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_store_id#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#24, s_store_id#25] -Condition : isnotnull(s_store_sk#24) - -(16) CometProject -Input [2]: [s_store_sk#24, s_store_id#25] -Arguments: [s_store_sk#24, s_store_id#26], [s_store_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#25, 16, true, false, true) AS s_store_id#26] - -(17) CometBroadcastExchange -Input [2]: [s_store_sk#24, s_store_id#26] -Arguments: [s_store_sk#24, s_store_id#26] - -(18) CometBroadcastHashJoin -Left output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [2]: [s_store_sk#24, s_store_id#26] -Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight - -(19) CometProject -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#26] -Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] - -(20) CometHashAggregate -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] -Keys [1]: [s_store_id#26] -Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] - -(21) CometExchange -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Keys [1]: [s_store_id#26] -Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk)] -ReadSchema: struct - -(24) CometFilter -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : isnotnull(cs_catalog_page_sk#31) - -(25) CometProject -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cr_catalog_page_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Condition : isnotnull(cr_catalog_page_sk#42) - -(28) CometProject -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] - -(29) CometUnion -Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] - -(30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#52] - -(31) CometBroadcastHashJoin -Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [1]: [d_date_sk#52] -Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight - -(32) CometProject -Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] -Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Condition : isnotnull(cp_catalog_page_sk#53) - -(35) CometProject -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#54, 16, true, false, true) AS cp_catalog_page_id#55] - -(36) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] - -(37) CometBroadcastHashJoin -Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight - -(38) CometProject -Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] - -(39) CometHashAggregate -Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] - -(40) CometExchange -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(41) CometHashAggregate -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] -PushedFilters: [IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(43) CometFilter -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_web_site_sk#60) - -(44) CometProject -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] -ReadSchema: struct - -(46) CometBroadcastExchange -Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(48) CometFilter -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) - -(49) CometProject -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] - -(50) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft - -(51) CometProject -Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] - -(52) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] - -(53) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#86] - -(54) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [1]: [d_date_sk#86] -Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight - -(55) CometProject -Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] -Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#87, web_site_id#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(57) CometFilter -Input [2]: [web_site_sk#87, web_site_id#88] -Condition : isnotnull(web_site_sk#87) - -(58) CometProject -Input [2]: [web_site_sk#87, web_site_id#88] -Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#88, 16, true, false, true) AS web_site_id#89] - -(59) CometBroadcastExchange -Input [2]: [web_site_sk#87, web_site_id#89] -Arguments: [web_site_sk#87, web_site_id#89] - -(60) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [2]: [web_site_sk#87, web_site_id#89] -Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight - -(61) CometProject -Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] -Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] - -(62) CometHashAggregate -Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] -Keys [1]: [web_site_id#89] -Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] - -(63) CometExchange -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(64) CometHashAggregate -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Keys [1]: [web_site_id#89] -Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] - -(65) CometUnion -Child 0 Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] -Child 1 Input [5]: [channel#99, id#100, sales#101, returns#102, profit#103] -Child 2 Input [5]: [channel#104, id#105, sales#106, returns#107, profit#108] - -(66) CometHashAggregate -Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] -Keys [2]: [channel#94, id#95] -Functions [3]: [partial_sum(sales#96), partial_sum(returns#97), partial_sum(profit#98)] - -(67) CometExchange -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: hashpartitioning(channel#94, id#95, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(68) CometHashAggregate -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#94, id#95] -Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] - -(69) ReusedExchange [Reuses operator id: 67] -Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(70) CometHashAggregate -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#94, id#95] -Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] - -(71) CometHashAggregate -Input [4]: [channel#94, sales#115, returns#116, profit#117] -Keys [1]: [channel#94] -Functions [3]: [partial_sum(sales#115), partial_sum(returns#116), partial_sum(profit#117)] - -(72) CometExchange -Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] -Arguments: hashpartitioning(channel#94, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(73) CometHashAggregate -Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] -Keys [1]: [channel#94] -Functions [3]: [sum(sales#115), sum(returns#116), sum(profit#117)] - -(74) ReusedExchange [Reuses operator id: 67] -Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(75) CometHashAggregate -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#94, id#95] -Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] - -(76) CometHashAggregate -Input [3]: [sales#124, returns#125, profit#126] -Keys: [] -Functions [3]: [partial_sum(sales#124), partial_sum(returns#125), partial_sum(profit#126)] - -(77) CometExchange -Input [6]: [sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(78) CometHashAggregate -Input [6]: [sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys: [] -Functions [3]: [sum(sales#124), sum(returns#125), sum(profit#126)] - -(79) CometUnion -Child 0 Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Child 1 Input [5]: [channel#94, id#136, sum(sales)#137, sum(returns)#138, sum(profit)#139] -Child 2 Input [5]: [channel#140, id#141, sum(sales)#142, sum(returns)#143, sum(profit)#144] - -(80) CometHashAggregate -Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Keys [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Functions: [] - -(81) CometExchange -Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Arguments: hashpartitioning(channel#94, id#95, sales#133, returns#134, profit#135, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(82) CometHashAggregate -Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Keys [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Functions: [] - -(83) CometTakeOrderedAndProject -Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#94 ASC NULLS FIRST,id#95 ASC NULLS FIRST], output=[channel#94,id#95,sales#133,returns#134,profit#135]), [channel#94, id#95, sales#133, returns#134, profit#135], 100, 0, [channel#94 ASC NULLS FIRST, id#95 ASC NULLS FIRST], [channel#94, id#95, sales#133, returns#134, profit#135] - -(84) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (89) -+- * CometColumnarToRow (88) - +- CometProject (87) - +- CometFilter (86) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) - - -(85) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] -ReadSchema: struct - -(86) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) - -(87) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(88) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(89) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/extended.txt deleted file mode 100644 index 35b86dbff5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/extended.txt +++ /dev/null @@ -1,269 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastExchange - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastExchange - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - -Comet accelerated 242 out of 263 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index c0b236a57a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt +++ /dev/null @@ -1,99 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id] #2 - CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [s_store_id] #3 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - 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 [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,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #6 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [cp_catalog_page_id] #7 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - 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 [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,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [cp_catalog_page_sk,cp_catalog_page_id] #8 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [web_site_id] #9 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - 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 [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,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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,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 [native_iceberg_compat] 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_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 [web_site_sk,web_site_id] #11 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] - CometExchange [channel] #12 - CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] - CometExchange #13 - CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/explain.txt deleted file mode 100644 index a57e1f5e30..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/explain.txt +++ /dev/null @@ -1,304 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (40) -+- * Filter (39) - +- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * CometColumnarToRow (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.customer (4) - : : +- BroadcastExchange (13) - : : +- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet spark_catalog.default.store_sales (10) - : +- ReusedExchange (16) - +- BroadcastExchange (32) - +- * CometColumnarToRow (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (20) - : +- CometNativeScan parquet spark_catalog.default.item (19) - +- CometBroadcastExchange (28) - +- CometFilter (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometFilter (22) - +- CometNativeScan parquet spark_catalog.default.item (21) - - -(1) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#1, ca_state#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ca_address_sk#1, ca_state#2] -Condition : isnotnull(ca_address_sk#1) - -(3) CometProject -Input [2]: [ca_address_sk#1, ca_state#2] -Arguments: [ca_address_sk#1, ca_state#3], [ca_address_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#2, 2, true, false, true) AS ca_state#3] - -(4) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_customer_sk#4)) - -(6) CometBroadcastExchange -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [c_customer_sk#4, c_current_addr_sk#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#1, ca_state#3] -Right output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] - -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ca_state#3, c_customer_sk#4] - -(10) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 1] -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(12) Filter [codegen id : 1] -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) - -(13) BroadcastExchange -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [c_customer_sk#4] -Right keys [1]: [ss_customer_sk#7] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(16) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#10] - -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [2]: [ca_state#3, ss_item_sk#6] -Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] - -(19) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#11, i_current_price#12, i_category#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [i_item_sk#11, i_current_price#12, i_category#13] -Condition : (isnotnull(i_current_price#12) AND isnotnull(i_item_sk#11)) - -(21) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_current_price#14, i_category#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_current_price#14, i_category#15] -Condition : isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#15, 50, true, false, true)) - -(23) CometProject -Input [2]: [i_current_price#14, i_category#15] -Arguments: [i_current_price#14, i_category#16], [i_current_price#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#15, 50, true, false, true) AS i_category#16] - -(24) CometHashAggregate -Input [2]: [i_current_price#14, i_category#16] -Keys [1]: [i_category#16] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#14))] - -(25) CometExchange -Input [3]: [i_category#16, sum#17, count#18] -Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(26) CometHashAggregate -Input [3]: [i_category#16, sum#17, count#18] -Keys [1]: [i_category#16] -Functions [1]: [avg(UnscaledValue(i_current_price#14))] - -(27) CometFilter -Input [2]: [avg(i_current_price)#19, i_category#16] -Condition : isnotnull(avg(i_current_price)#19) - -(28) CometBroadcastExchange -Input [2]: [avg(i_current_price)#19, i_category#16] -Arguments: [avg(i_current_price)#19, i_category#16] - -(29) CometBroadcastHashJoin -Left output [3]: [i_item_sk#11, i_current_price#12, i_category#13] -Right output [2]: [avg(i_current_price)#19, i_category#16] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#13, 50, true, false, true)], [i_category#16], Inner, (cast(i_current_price#12 as decimal(14,7)) > (1.2 * avg(i_current_price)#19)), BuildRight - -(30) CometProject -Input [5]: [i_item_sk#11, i_current_price#12, i_category#13, avg(i_current_price)#19, i_category#16] -Arguments: [i_item_sk#11], [i_item_sk#11] - -(31) CometColumnarToRow [codegen id : 3] -Input [1]: [i_item_sk#11] - -(32) BroadcastExchange -Input [1]: [i_item_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(33) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#6] -Right keys [1]: [i_item_sk#11] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 4] -Output [1]: [ca_state#3] -Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#11] - -(35) HashAggregate [codegen id : 4] -Input [1]: [ca_state#3] -Keys [1]: [ca_state#3] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#20] -Results [2]: [ca_state#3, count#21] - -(36) CometColumnarExchange -Input [2]: [ca_state#3, count#21] -Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(37) CometColumnarToRow [codegen id : 5] -Input [2]: [ca_state#3, count#21] - -(38) HashAggregate [codegen id : 5] -Input [2]: [ca_state#3, count#21] -Keys [1]: [ca_state#3] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#22] -Results [3]: [ca_state#3 AS state#23, count(1)#22 AS cnt#24, ca_state#3] - -(39) Filter [codegen id : 5] -Input [3]: [state#23, cnt#24, ca_state#3] -Condition : (cnt#24 >= 10) - -(40) TakeOrderedAndProject -Input [3]: [state#23, cnt#24, ca_state#3] -Arguments: 100, [cnt#24 ASC NULLS FIRST, ca_state#3 ASC NULLS FIRST], [state#23, cnt#24] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometNativeScan parquet spark_catalog.default.date_dim (41) - - -(41) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#25] -Condition : ((isnotnull(d_month_seq#25) AND (d_month_seq#25 = Subquery scalar-subquery#26, [id=#27])) AND isnotnull(d_date_sk#10)) - -(43) CometProject -Input [2]: [d_date_sk#10, d_month_seq#25] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(45) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#26, [id=#27] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometNativeScan parquet spark_catalog.default.date_dim (46) - - -(46) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_month_seq#28, d_year#29, d_moy#30] -Condition : (((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2000)) AND (d_moy#30 = 1)) - -(48) CometProject -Input [3]: [d_month_seq#28, d_year#29, d_moy#30] -Arguments: [d_month_seq#28], [d_month_seq#28] - -(49) CometHashAggregate -Input [1]: [d_month_seq#28] -Keys [1]: [d_month_seq#28] -Functions: [] - -(50) CometExchange -Input [1]: [d_month_seq#28] -Arguments: hashpartitioning(d_month_seq#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(51) CometHashAggregate -Input [1]: [d_month_seq#28] -Keys [1]: [d_month_seq#28] -Functions: [] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#28] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/simplified.txt deleted file mode 100644 index 76dd43f9c7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/simplified.txt +++ /dev/null @@ -1,69 +0,0 @@ -TakeOrderedAndProject [cnt,ca_state,state] - WholeStageCodegen (5) - Filter [cnt] - HashAggregate [ca_state,count] [count(1),state,cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state] #1 - WholeStageCodegen (4) - HashAggregate [ca_state] [count,count] - Project [ca_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ca_state,ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ca_state,ss_item_sk,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_customer_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #7 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #8 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_current_price,i_category] - CometFilter [i_current_price,i_category] - CometNativeScan 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/q6.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/explain.txt deleted file mode 100644 index 73dd1d4955..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometFilter (38) - +- CometHashAggregate (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (4) - : : +- CometBroadcastExchange (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (9) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- CometBroadcastExchange (29) - +- CometFilter (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (22) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#1, ca_state#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ca_address_sk#1, ca_state#2] -Condition : isnotnull(ca_address_sk#1) - -(3) CometProject -Input [2]: [ca_address_sk#1, ca_state#2] -Arguments: [ca_address_sk#1, ca_state#3], [ca_address_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#2, 2, true, false, true) AS ca_state#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_customer_sk#4)) - -(6) CometBroadcastExchange -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [c_customer_sk#4, c_current_addr_sk#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#1, ca_state#3] -Right output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) - -(11) CometBroadcastExchange -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(12) CometBroadcastHashJoin -Left output [2]: [ca_state#3, c_customer_sk#4] -Right output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [c_customer_sk#4], [ss_customer_sk#7], Inner, BuildRight - -(13) CometProject -Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8], [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(16) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(18) CometBroadcastHashJoin -Left output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(19) CometProject -Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ca_state#3, ss_item_sk#6], [ca_state#3, ss_item_sk#6] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Condition : (isnotnull(i_current_price#15) AND isnotnull(i_item_sk#14)) - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_current_price#17, i_category#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(23) CometFilter -Input [2]: [i_current_price#17, i_category#18] -Condition : isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true)) - -(24) CometProject -Input [2]: [i_current_price#17, i_category#18] -Arguments: [i_current_price#17, i_category#19], [i_current_price#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true) AS i_category#19] - -(25) CometHashAggregate -Input [2]: [i_current_price#17, i_category#19] -Keys [1]: [i_category#19] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#17))] - -(26) CometExchange -Input [3]: [i_category#19, sum#20, count#21] -Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [3]: [i_category#19, sum#20, count#21] -Keys [1]: [i_category#19] -Functions [1]: [avg(UnscaledValue(i_current_price#17))] - -(28) CometFilter -Input [2]: [avg(i_current_price)#22, i_category#19] -Condition : isnotnull(avg(i_current_price)#22) - -(29) CometBroadcastExchange -Input [2]: [avg(i_current_price)#22, i_category#19] -Arguments: [avg(i_current_price)#22, i_category#19] - -(30) CometBroadcastHashJoin -Left output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Right output [2]: [avg(i_current_price)#22, i_category#19] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true)], [i_category#19], Inner, (cast(i_current_price#15 as decimal(14,7)) > (1.2 * avg(i_current_price)#22)), BuildRight - -(31) CometProject -Input [5]: [i_item_sk#14, i_current_price#15, i_category#16, avg(i_current_price)#22, i_category#19] -Arguments: [i_item_sk#14], [i_item_sk#14] - -(32) CometBroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: [i_item_sk#14] - -(33) CometBroadcastHashJoin -Left output [2]: [ca_state#3, ss_item_sk#6] -Right output [1]: [i_item_sk#14] -Arguments: [ss_item_sk#6], [i_item_sk#14], Inner, BuildRight - -(34) CometProject -Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#14] -Arguments: [ca_state#3], [ca_state#3] - -(35) CometHashAggregate -Input [1]: [ca_state#3] -Keys [1]: [ca_state#3] -Functions [1]: [partial_count(1)] - -(36) CometExchange -Input [2]: [ca_state#3, count#23] -Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(37) CometHashAggregate -Input [2]: [ca_state#3, count#23] -Keys [1]: [ca_state#3] -Functions [1]: [count(1)] - -(38) CometFilter -Input [3]: [state#24, cnt#25, ca_state#3] -Condition : (cnt#25 >= 10) - -(39) CometTakeOrderedAndProject -Input [3]: [state#24, cnt#25, ca_state#3] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#25 ASC NULLS FIRST,ca_state#3 ASC NULLS FIRST], output=[state#24,cnt#25]), [state#24, cnt#25], 100, 0, [cnt#25 ASC NULLS FIRST, ca_state#3 ASC NULLS FIRST], [state#24, cnt#25] - -(40) CometColumnarToRow [codegen id : 1] -Input [2]: [state#24, cnt#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(43) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(45) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) - - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#26, d_year#27, d_moy#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Condition : (((isnotnull(d_year#27) AND isnotnull(d_moy#28)) AND (d_year#27 = 2000)) AND (d_moy#28 = 1)) - -(48) CometProject -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Arguments: [d_month_seq#26], [d_month_seq#26] - -(49) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] -Functions: [] - -(50) CometExchange -Input [1]: [d_month_seq#26] -Arguments: hashpartitioning(d_month_seq#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(51) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] -Functions: [] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#26] - -Subquery:3 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/simplified.txt deleted file mode 100644 index 2978e30c1c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_state] [state,cnt] - CometFilter [state,cnt,ca_state] - CometHashAggregate [count] [state,cnt,ca_state,count(1)] - CometExchange [ca_state] #1 - CometHashAggregate [ca_state,count] - CometProject [ca_state] - CometBroadcastHashJoin [ca_state,ss_item_sk,i_item_sk] - CometProject [ca_state,ss_item_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 [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,ca_state,c_customer_sk,c_current_addr_sk] - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk] #7 - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #8 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #9 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_current_price,i_category] - CometFilter [i_current_price,i_category] - CometScan [native_iceberg_compat] 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/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/explain.txt deleted file mode 100644 index 73dd1d4955..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometFilter (38) - +- CometHashAggregate (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (4) - : : +- CometBroadcastExchange (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (9) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- CometBroadcastExchange (29) - +- CometFilter (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (22) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#1, ca_state#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ca_address_sk#1, ca_state#2] -Condition : isnotnull(ca_address_sk#1) - -(3) CometProject -Input [2]: [ca_address_sk#1, ca_state#2] -Arguments: [ca_address_sk#1, ca_state#3], [ca_address_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#2, 2, true, false, true) AS ca_state#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_customer_sk#4)) - -(6) CometBroadcastExchange -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [c_customer_sk#4, c_current_addr_sk#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#1, ca_state#3] -Right output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) - -(11) CometBroadcastExchange -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(12) CometBroadcastHashJoin -Left output [2]: [ca_state#3, c_customer_sk#4] -Right output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [c_customer_sk#4], [ss_customer_sk#7], Inner, BuildRight - -(13) CometProject -Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8], [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(16) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(18) CometBroadcastHashJoin -Left output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(19) CometProject -Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ca_state#3, ss_item_sk#6], [ca_state#3, ss_item_sk#6] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Condition : (isnotnull(i_current_price#15) AND isnotnull(i_item_sk#14)) - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_current_price#17, i_category#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(23) CometFilter -Input [2]: [i_current_price#17, i_category#18] -Condition : isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true)) - -(24) CometProject -Input [2]: [i_current_price#17, i_category#18] -Arguments: [i_current_price#17, i_category#19], [i_current_price#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true) AS i_category#19] - -(25) CometHashAggregate -Input [2]: [i_current_price#17, i_category#19] -Keys [1]: [i_category#19] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#17))] - -(26) CometExchange -Input [3]: [i_category#19, sum#20, count#21] -Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [3]: [i_category#19, sum#20, count#21] -Keys [1]: [i_category#19] -Functions [1]: [avg(UnscaledValue(i_current_price#17))] - -(28) CometFilter -Input [2]: [avg(i_current_price)#22, i_category#19] -Condition : isnotnull(avg(i_current_price)#22) - -(29) CometBroadcastExchange -Input [2]: [avg(i_current_price)#22, i_category#19] -Arguments: [avg(i_current_price)#22, i_category#19] - -(30) CometBroadcastHashJoin -Left output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Right output [2]: [avg(i_current_price)#22, i_category#19] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true)], [i_category#19], Inner, (cast(i_current_price#15 as decimal(14,7)) > (1.2 * avg(i_current_price)#22)), BuildRight - -(31) CometProject -Input [5]: [i_item_sk#14, i_current_price#15, i_category#16, avg(i_current_price)#22, i_category#19] -Arguments: [i_item_sk#14], [i_item_sk#14] - -(32) CometBroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: [i_item_sk#14] - -(33) CometBroadcastHashJoin -Left output [2]: [ca_state#3, ss_item_sk#6] -Right output [1]: [i_item_sk#14] -Arguments: [ss_item_sk#6], [i_item_sk#14], Inner, BuildRight - -(34) CometProject -Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#14] -Arguments: [ca_state#3], [ca_state#3] - -(35) CometHashAggregate -Input [1]: [ca_state#3] -Keys [1]: [ca_state#3] -Functions [1]: [partial_count(1)] - -(36) CometExchange -Input [2]: [ca_state#3, count#23] -Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(37) CometHashAggregate -Input [2]: [ca_state#3, count#23] -Keys [1]: [ca_state#3] -Functions [1]: [count(1)] - -(38) CometFilter -Input [3]: [state#24, cnt#25, ca_state#3] -Condition : (cnt#25 >= 10) - -(39) CometTakeOrderedAndProject -Input [3]: [state#24, cnt#25, ca_state#3] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#25 ASC NULLS FIRST,ca_state#3 ASC NULLS FIRST], output=[state#24,cnt#25]), [state#24, cnt#25], 100, 0, [cnt#25 ASC NULLS FIRST, ca_state#3 ASC NULLS FIRST], [state#24, cnt#25] - -(40) CometColumnarToRow [codegen id : 1] -Input [2]: [state#24, cnt#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(43) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(45) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) - - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#26, d_year#27, d_moy#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Condition : (((isnotnull(d_year#27) AND isnotnull(d_moy#28)) AND (d_year#27 = 2000)) AND (d_moy#28 = 1)) - -(48) CometProject -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Arguments: [d_month_seq#26], [d_month_seq#26] - -(49) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] -Functions: [] - -(50) CometExchange -Input [1]: [d_month_seq#26] -Arguments: hashpartitioning(d_month_seq#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(51) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] -Functions: [] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#26] - -Subquery:3 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/extended.txt deleted file mode 100644 index f5b69fc6cf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/extended.txt +++ /dev/null @@ -1,57 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 2978e30c1c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_state] [state,cnt] - CometFilter [state,cnt,ca_state] - CometHashAggregate [count] [state,cnt,ca_state,count(1)] - CometExchange [ca_state] #1 - CometHashAggregate [ca_state,count] - CometProject [ca_state] - CometBroadcastHashJoin [ca_state,ss_item_sk,i_item_sk] - CometProject [ca_state,ss_item_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 [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,ca_state,c_customer_sk,c_current_addr_sk] - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk] #7 - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #8 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #9 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_current_price,i_category] - CometFilter [i_current_price,i_category] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/explain.txt deleted file mode 100644 index 135549db3d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/explain.txt +++ /dev/null @@ -1,1029 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (180) -+- CometSort (179) - +- CometExchange (178) - +- CometProject (177) - +- CometSortMergeJoin (176) - :- CometSort (105) - : +- CometExchange (104) - : +- CometHashAggregate (103) - : +- CometHashAggregate (102) - : +- CometProject (101) - : +- CometBroadcastHashJoin (100) - : :- CometProject (95) - : : +- CometBroadcastHashJoin (94) - : : :- CometProject (92) - : : : +- CometBroadcastHashJoin (91) - : : : :- CometProject (87) - : : : : +- CometBroadcastHashJoin (86) - : : : : :- CometProject (84) - : : : : : +- CometBroadcastHashJoin (83) - : : : : : :- CometProject (78) - : : : : : : +- CometBroadcastHashJoin (77) - : : : : : : :- CometProject (75) - : : : : : : : +- CometBroadcastHashJoin (74) - : : : : : : : :- CometProject (70) - : : : : : : : : +- CometBroadcastHashJoin (69) - : : : : : : : : :- CometProject (65) - : : : : : : : : : +- CometBroadcastHashJoin (64) - : : : : : : : : : :- CometProject (62) - : : : : : : : : : : +- CometBroadcastHashJoin (61) - : : : : : : : : : : :- CometProject (56) - : : : : : : : : : : : +- CometBroadcastHashJoin (55) - : : : : : : : : : : : :- CometProject (53) - : : : : : : : : : : : : +- CometBroadcastHashJoin (52) - : : : : : : : : : : : : :- CometProject (48) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (47) - : : : : : : : : : : : : : :- CometProject (43) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (42) - : : : : : : : : : : : : : : :- CometProject (37) - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (36) - : : : : : : : : : : : : : : : :- CometProject (32) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (31) - : : : : : : : : : : : : : : : : :- CometSort (12) - : : : : : : : : : : : : : : : : : +- CometColumnarExchange (11) - : : : : : : : : : : : : : : : : : +- * Project (10) - : : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : : : : : : : : : : : : : : :- BroadcastExchange (4) - : : : : : : : : : : : : : : : : : : +- * Filter (3) - : : : : : : : : : : : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (8) - : : : : : : : : : : : : : : : : : +- CometProject (7) - : : : : : : : : : : : : : : : : : +- CometFilter (6) - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (5) - : : : : : : : : : : : : : : : : +- CometSort (30) - : : : : : : : : : : : : : : : : +- CometProject (29) - : : : : : : : : : : : : : : : : +- CometFilter (28) - : : : : : : : : : : : : : : : : +- CometHashAggregate (27) - : : : : : : : : : : : : : : : : +- CometExchange (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometProject (24) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (23) - : : : : : : : : : : : : : : : : :- CometSort (17) - : : : : : : : : : : : : : : : : : +- CometExchange (16) - : : : : : : : : : : : : : : : : : +- CometProject (15) - : : : : : : : : : : : : : : : : : +- CometFilter (14) - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : : : : : : : : : : : +- CometSort (22) - : : : : : : : : : : : : : : : : +- CometExchange (21) - : : : : : : : : : : : : : : : : +- CometProject (20) - : : : : : : : : : : : : : : : : +- CometFilter (19) - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (18) - : : : : : : : : : : : : : : : +- CometBroadcastExchange (35) - : : : : : : : : : : : : : : : +- CometFilter (34) - : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (33) - : : : : : : : : : : : : : : +- CometBroadcastExchange (41) - : : : : : : : : : : : : : : +- CometProject (40) - : : : : : : : : : : : : : : +- CometFilter (39) - : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store (38) - : : : : : : : : : : : : : +- CometBroadcastExchange (46) - : : : : : : : : : : : : : +- CometFilter (45) - : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer (44) - : : : : : : : : : : : : +- CometBroadcastExchange (51) - : : : : : : : : : : : : +- CometFilter (50) - : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (49) - : : : : : : : : : : : +- ReusedExchange (54) - : : : : : : : : : : +- CometBroadcastExchange (60) - : : : : : : : : : : +- CometProject (59) - : : : : : : : : : : +- CometFilter (58) - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (57) - : : : : : : : : : +- ReusedExchange (63) - : : : : : : : : +- CometBroadcastExchange (68) - : : : : : : : : +- CometFilter (67) - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.promotion (66) - : : : : : : : +- CometBroadcastExchange (73) - : : : : : : : +- CometFilter (72) - : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (71) - : : : : : : +- ReusedExchange (76) - : : : : : +- CometBroadcastExchange (82) - : : : : : +- CometProject (81) - : : : : : +- CometFilter (80) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address (79) - : : : : +- ReusedExchange (85) - : : : +- CometBroadcastExchange (90) - : : : +- CometFilter (89) - : : : +- CometNativeScan parquet spark_catalog.default.income_band (88) - : : +- ReusedExchange (93) - : +- CometBroadcastExchange (99) - : +- CometProject (98) - : +- CometFilter (97) - : +- CometNativeScan parquet spark_catalog.default.item (96) - +- CometSort (175) - +- CometExchange (174) - +- CometHashAggregate (173) - +- CometHashAggregate (172) - +- CometProject (171) - +- CometBroadcastHashJoin (170) - :- CometProject (168) - : +- CometBroadcastHashJoin (167) - : :- CometProject (165) - : : +- CometBroadcastHashJoin (164) - : : :- CometProject (162) - : : : +- CometBroadcastHashJoin (161) - : : : :- CometProject (159) - : : : : +- CometBroadcastHashJoin (158) - : : : : :- CometProject (156) - : : : : : +- CometBroadcastHashJoin (155) - : : : : : :- CometProject (153) - : : : : : : +- CometBroadcastHashJoin (152) - : : : : : : :- CometProject (150) - : : : : : : : +- CometBroadcastHashJoin (149) - : : : : : : : :- CometProject (147) - : : : : : : : : +- CometBroadcastHashJoin (146) - : : : : : : : : :- CometProject (144) - : : : : : : : : : +- CometBroadcastHashJoin (143) - : : : : : : : : : :- CometProject (141) - : : : : : : : : : : +- CometBroadcastHashJoin (140) - : : : : : : : : : : :- CometProject (138) - : : : : : : : : : : : +- CometBroadcastHashJoin (137) - : : : : : : : : : : : :- CometProject (135) - : : : : : : : : : : : : +- CometBroadcastHashJoin (134) - : : : : : : : : : : : : :- CometProject (132) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (131) - : : : : : : : : : : : : : :- CometProject (129) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (128) - : : : : : : : : : : : : : : :- CometProject (124) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (123) - : : : : : : : : : : : : : : : :- CometSort (117) - : : : : : : : : : : : : : : : : +- CometColumnarExchange (116) - : : : : : : : : : : : : : : : : +- * Project (115) - : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (114) - : : : : : : : : : : : : : : : : :- BroadcastExchange (109) - : : : : : : : : : : : : : : : : : +- * Filter (108) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (107) - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (106) - : : : : : : : : : : : : : : : : +- * CometColumnarToRow (113) - : : : : : : : : : : : : : : : : +- CometProject (112) - : : : : : : : : : : : : : : : : +- CometFilter (111) - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (110) - : : : : : : : : : : : : : : : +- CometSort (122) - : : : : : : : : : : : : : : : +- CometProject (121) - : : : : : : : : : : : : : : : +- CometFilter (120) - : : : : : : : : : : : : : : : +- CometHashAggregate (119) - : : : : : : : : : : : : : : : +- ReusedExchange (118) - : : : : : : : : : : : : : : +- CometBroadcastExchange (127) - : : : : : : : : : : : : : : +- CometFilter (126) - : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (125) - : : : : : : : : : : : : : +- ReusedExchange (130) - : : : : : : : : : : : : +- ReusedExchange (133) - : : : : : : : : : : : +- ReusedExchange (136) - : : : : : : : : : : +- ReusedExchange (139) - : : : : : : : : : +- ReusedExchange (142) - : : : : : : : : +- ReusedExchange (145) - : : : : : : : +- ReusedExchange (148) - : : : : : : +- ReusedExchange (151) - : : : : : +- ReusedExchange (154) - : : : : +- ReusedExchange (157) - : : : +- ReusedExchange (160) - : : +- ReusedExchange (163) - : +- ReusedExchange (166) - +- ReusedExchange (169) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(3) Filter [codegen id : 1] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) - -(4) BroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=1] - -(5) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) - -(7) CometProject -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] - -(8) CometColumnarToRow -Input [2]: [sr_item_sk#14, sr_ticket_number#15] - -(9) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [ss_item_sk#1, ss_ticket_number#8] -Right keys [2]: [sr_item_sk#14, sr_ticket_number#15] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 2] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] - -(11) CometColumnarExchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(12) CometSort -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] - -(13) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) - -(15) CometProject -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] - -(16) CometExchange -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(17) CometSort -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] - -(18) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(19) CometFilter -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) - -(20) CometProject -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(21) CometExchange -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(22) CometSort -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] - -(23) CometSortMergeJoin -Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner - -(24) CometProject -Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(25) CometHashAggregate -Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Keys [1]: [cs_item_sk#17] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(26) CometExchange -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(27) CometHashAggregate -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Keys [1]: [cs_item_sk#17] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(28) CometFilter -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(29) CometProject -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Arguments: [cs_item_sk#17], [cs_item_sk#17] - -(30) CometSort -Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] - -(31) CometSortMergeJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [1]: [cs_item_sk#17] -Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner - -(32) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(33) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(35) CometBroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: [d_date_sk#32, d_year#33] - -(36) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight - -(37) CometProject -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] - -(38) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name)] -ReadSchema: struct - -(39) CometFilter -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true))) - -(40) CometProject -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37], [s_store_sk#34, s_store_name#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true) AS s_zip#37] - -(41) CometBroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37] - -(42) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight - -(43) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] - -(44) CometNativeScan parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(45) CometFilter -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Condition : (((((isnotnull(c_customer_sk#38) AND isnotnull(c_first_sales_date_sk#43)) AND isnotnull(c_first_shipto_date_sk#42)) AND isnotnull(c_current_cdemo_sk#39)) AND isnotnull(c_current_hdemo_sk#40)) AND isnotnull(c_current_addr_sk#41)) - -(46) CometBroadcastExchange -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(47) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -Right output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_customer_sk#2], [c_customer_sk#38], Inner, BuildRight - -(48) CometProject -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(49) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : isnotnull(d_date_sk#44) - -(51) CometBroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] - -(52) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [c_first_sales_date_sk#43], [d_date_sk#44], Inner, BuildRight - -(53) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43, d_date_sk#44, d_year#45] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] - -(54) ReusedExchange [Reuses operator id: 51] -Output [2]: [d_date_sk#46, d_year#47] - -(55) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [c_first_shipto_date_sk#42], [d_date_sk#46], Inner, BuildRight - -(56) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45, d_date_sk#46, d_year#47] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(57) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#48, cd_marital_status#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(58) CometFilter -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true))) - -(59) CometProject -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Arguments: [cd_demo_sk#48, cd_marital_status#50], [cd_demo_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true) AS cd_marital_status#50] - -(60) CometBroadcastExchange -Input [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [cd_demo_sk#48, cd_marital_status#50] - -(61) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_cdemo_sk#3], [cd_demo_sk#48], Inner, BuildRight - -(62) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] - -(63) ReusedExchange [Reuses operator id: 60] -Output [2]: [cd_demo_sk#51, cd_marital_status#52] - -(64) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -Right output [2]: [cd_demo_sk#51, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#39], [cd_demo_sk#51], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(65) CometProject -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50, cd_demo_sk#51, cd_marital_status#52] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(66) CometNativeScan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(67) CometFilter -Input [1]: [p_promo_sk#53] -Condition : isnotnull(p_promo_sk#53) - -(68) CometBroadcastExchange -Input [1]: [p_promo_sk#53] -Arguments: [p_promo_sk#53] - -(69) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [1]: [p_promo_sk#53] -Arguments: [ss_promo_sk#7], [p_promo_sk#53], Inner, BuildRight - -(70) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, p_promo_sk#53] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(71) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(72) CometFilter -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Condition : (isnotnull(hd_demo_sk#54) AND isnotnull(hd_income_band_sk#55)) - -(73) CometBroadcastExchange -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [hd_demo_sk#54, hd_income_band_sk#55] - -(74) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_hdemo_sk#4], [hd_demo_sk#54], Inner, BuildRight - -(75) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] - -(76) ReusedExchange [Reuses operator id: 73] -Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] - -(77) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -Right output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [c_current_hdemo_sk#40], [hd_demo_sk#56], Inner, BuildRight - -(78) CometProject -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] - -(79) CometNativeScan parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(80) CometFilter -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Condition : isnotnull(ca_address_sk#58) - -(81) CometProject -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ca_address_sk#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#59, 10, true, false, true) AS ca_street_number#63, ca_street_name#60, ca_city#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#62, 10, true, false, true) AS ca_zip#64] - -(82) CometBroadcastExchange -Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(83) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -Right output [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_addr_sk#5], [ca_address_sk#58], Inner, BuildRight - -(84) CometProject -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(85) ReusedExchange [Reuses operator id: 82] -Output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(86) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Right output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [c_current_addr_sk#41], [ca_address_sk#65], Inner, BuildRight - -(87) CometProject -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(88) CometNativeScan parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(89) CometFilter -Input [1]: [ib_income_band_sk#70] -Condition : isnotnull(ib_income_band_sk#70) - -(90) CometBroadcastExchange -Input [1]: [ib_income_band_sk#70] -Arguments: [ib_income_band_sk#70] - -(91) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#70] -Arguments: [hd_income_band_sk#55], [ib_income_band_sk#70], Inner, BuildRight - -(92) CometProject -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#70] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(93) ReusedExchange [Reuses operator id: 90] -Output [1]: [ib_income_band_sk#71] - -(94) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#71] -Arguments: [hd_income_band_sk#57], [ib_income_band_sk#71], Inner, BuildRight - -(95) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#71] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(96) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(97) CometFilter -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Condition : ((((((isnotnull(i_current_price#73) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#74, 20, true, false, true) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#73 >= 64.00)) AND (i_current_price#73 <= 74.00)) AND (i_current_price#73 >= 65.00)) AND (i_current_price#73 <= 79.00)) AND isnotnull(i_item_sk#72)) - -(98) CometProject -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Arguments: [i_item_sk#72, i_product_name#76], [i_item_sk#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#75, 50, true, false, true) AS i_product_name#76] - -(99) CometBroadcastExchange -Input [2]: [i_item_sk#72, i_product_name#76] -Arguments: [i_item_sk#72, i_product_name#76] - -(100) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [2]: [i_item_sk#72, i_product_name#76] -Arguments: [ss_item_sk#1], [i_item_sk#72], Inner, BuildRight - -(101) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] - -(102) CometHashAggregate -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] - -(103) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#77, sum#78, sum#79, sum#80] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] - -(104) CometExchange -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: hashpartitioning(item_sk#82, store_name#83, store_zip#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(105) CometSort -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97], [item_sk#82 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, store_zip#84 ASC NULLS FIRST] - -(106) Scan parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#109), dynamicpruningexpression(ss_sold_date_sk#109 IN dynamicpruning#110)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(107) ColumnarToRow [codegen id : 3] -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(108) Filter [codegen id : 3] -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Condition : (((((((isnotnull(ss_item_sk#98) AND isnotnull(ss_ticket_number#105)) AND isnotnull(ss_store_sk#103)) AND isnotnull(ss_customer_sk#99)) AND isnotnull(ss_cdemo_sk#100)) AND isnotnull(ss_promo_sk#104)) AND isnotnull(ss_hdemo_sk#101)) AND isnotnull(ss_addr_sk#102)) - -(109) BroadcastExchange -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=7] - -(110) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(111) CometFilter -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) - -(112) CometProject -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] - -(113) CometColumnarToRow -Input [2]: [sr_item_sk#111, sr_ticket_number#112] - -(114) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [ss_item_sk#98, ss_ticket_number#105] -Right keys [2]: [sr_item_sk#111, sr_ticket_number#112] -Join type: Inner -Join condition: None - -(115) Project [codegen id : 4] -Output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, sr_item_sk#111, sr_ticket_number#112] - -(116) CometColumnarExchange -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: hashpartitioning(ss_item_sk#98, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(117) CometSort -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98 ASC NULLS FIRST] - -(118) ReusedExchange [Reuses operator id: 26] -Output [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] - -(119) CometHashAggregate -Input [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] -Keys [1]: [cs_item_sk#114] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#118)), sum(((cr_refunded_cash#119 + cr_reversed_charge#120) + cr_store_credit#121))] - -(120) CometFilter -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(121) CometProject -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Arguments: [cs_item_sk#114], [cs_item_sk#114] - -(122) CometSort -Input [1]: [cs_item_sk#114] -Arguments: [cs_item_sk#114], [cs_item_sk#114 ASC NULLS FIRST] - -(123) CometSortMergeJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [1]: [cs_item_sk#114] -Arguments: [ss_item_sk#98], [cs_item_sk#114], Inner - -(124) CometProject -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, cs_item_sk#114] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(125) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(127) CometBroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: [d_date_sk#122, d_year#123] - -(128) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [d_date_sk#122, d_year#123] -Arguments: [ss_sold_date_sk#109], [d_date_sk#122], Inner, BuildRight - -(129) CometProject -Input [13]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, d_date_sk#122, d_year#123] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] - -(130) ReusedExchange [Reuses operator id: 41] -Output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] - -(131) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] -Right output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_store_sk#103], [s_store_sk#124], Inner, BuildRight - -(132) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] - -(133) ReusedExchange [Reuses operator id: 46] -Output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(134) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] -Right output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_customer_sk#99], [c_customer_sk#126], Inner, BuildRight - -(135) CometProject -Input [18]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(136) ReusedExchange [Reuses operator id: 51] -Output [2]: [d_date_sk#132, d_year#133] - -(137) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Right output [2]: [d_date_sk#132, d_year#133] -Arguments: [c_first_sales_date_sk#131], [d_date_sk#132], Inner, BuildRight - -(138) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131, d_date_sk#132, d_year#133] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] - -(139) ReusedExchange [Reuses operator id: 51] -Output [2]: [d_date_sk#134, d_year#135] - -(140) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] -Right output [2]: [d_date_sk#134, d_year#135] -Arguments: [c_first_shipto_date_sk#130], [d_date_sk#134], Inner, BuildRight - -(141) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133, d_date_sk#134, d_year#135] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(142) ReusedExchange [Reuses operator id: 60] -Output [2]: [cd_demo_sk#136, cd_marital_status#50] - -(143) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_cdemo_sk#100], [cd_demo_sk#136], Inner, BuildRight - -(144) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] - -(145) ReusedExchange [Reuses operator id: 60] -Output [2]: [cd_demo_sk#137, cd_marital_status#52] - -(146) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] -Right output [2]: [cd_demo_sk#137, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#127], [cd_demo_sk#137], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(147) CometProject -Input [18]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50, cd_demo_sk#137, cd_marital_status#52] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(148) ReusedExchange [Reuses operator id: 68] -Output [1]: [p_promo_sk#138] - -(149) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [1]: [p_promo_sk#138] -Arguments: [ss_promo_sk#104], [p_promo_sk#138], Inner, BuildRight - -(150) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, p_promo_sk#138] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(151) ReusedExchange [Reuses operator id: 73] -Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] - -(152) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_hdemo_sk#101], [hd_demo_sk#139], Inner, BuildRight - -(153) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] - -(154) ReusedExchange [Reuses operator id: 73] -Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] - -(155) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] -Right output [2]: [hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [c_current_hdemo_sk#128], [hd_demo_sk#141], Inner, BuildRight - -(156) CometProject -Input [15]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] - -(157) ReusedExchange [Reuses operator id: 82] -Output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(158) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] -Right output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_addr_sk#102], [ca_address_sk#143], Inner, BuildRight - -(159) CometProject -Input [18]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(160) ReusedExchange [Reuses operator id: 82] -Output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(161) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Right output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [c_current_addr_sk#129], [ca_address_sk#146], Inner, BuildRight - -(162) CometProject -Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(163) ReusedExchange [Reuses operator id: 90] -Output [1]: [ib_income_band_sk#149] - -(164) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#149] -Arguments: [hd_income_band_sk#140], [ib_income_band_sk#149], Inner, BuildRight - -(165) CometProject -Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#149] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(166) ReusedExchange [Reuses operator id: 90] -Output [1]: [ib_income_band_sk#150] - -(167) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#150] -Arguments: [hd_income_band_sk#142], [ib_income_band_sk#150], Inner, BuildRight - -(168) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#150] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(169) ReusedExchange [Reuses operator id: 99] -Output [2]: [i_item_sk#151, i_product_name#76] - -(170) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [2]: [i_item_sk#151, i_product_name#76] -Arguments: [ss_item_sk#98], [i_item_sk#151], Inner, BuildRight - -(171) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] - -(172) CometHashAggregate -Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#106)), partial_sum(UnscaledValue(ss_list_price#107)), partial_sum(UnscaledValue(ss_coupon_amt#108))] - -(173) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135, count#77, sum#152, sum#153, sum#154] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#106)), sum(UnscaledValue(ss_list_price#107)), sum(UnscaledValue(ss_coupon_amt#108))] - -(174) CometExchange -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: hashpartitioning(item_sk#155, store_name#156, store_zip#157, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(175) CometSort -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162], [item_sk#155 ASC NULLS FIRST, store_name#156 ASC NULLS FIRST, store_zip#157 ASC NULLS FIRST] - -(176) CometSortMergeJoin -Left output [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Right output [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#156, store_zip#157], Inner, (cnt#159 <= cnt#94) - -(177) CometProject -Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -(178) CometExchange -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(179) CometSort -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST] - -(180) CometColumnarToRow [codegen id : 5] -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (184) -+- * CometColumnarToRow (183) - +- CometFilter (182) - +- CometNativeScan parquet spark_catalog.default.date_dim (181) - - -(181) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(182) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(183) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#33] - -(184) BroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] - -Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 -BroadcastExchange (188) -+- * CometColumnarToRow (187) - +- CometFilter (186) - +- CometNativeScan parquet spark_catalog.default.date_dim (185) - - -(185) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(186) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(187) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#122, d_year#123] - -(188) BroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/simplified.txt deleted file mode 100644 index d933995ffc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/simplified.txt +++ /dev/null @@ -1,206 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometExchange [product_name,store_name,cnt,s1,s1] #1 - CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometColumnarExchange [ss_item_sk] #3 - WholeStageCodegen (2) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometNativeScan 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] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [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] #11 - 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] - CometNativeScan 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] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk] - ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometColumnarExchange [ss_item_sk] #20 - WholeStageCodegen (4) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #21 - WholeStageCodegen (3) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet 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 #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - CometBroadcastExchange [d_date_sk,d_year] #23 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/explain.txt deleted file mode 100644 index 077daeca3a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,1011 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (176) -+- CometSort (175) - +- CometExchange (174) - +- CometProject (173) - +- CometSortMergeJoin (172) - :- CometSort (103) - : +- CometExchange (102) - : +- CometHashAggregate (101) - : +- CometHashAggregate (100) - : +- CometProject (99) - : +- CometBroadcastHashJoin (98) - : :- CometProject (93) - : : +- CometBroadcastHashJoin (92) - : : :- CometProject (90) - : : : +- CometBroadcastHashJoin (89) - : : : :- CometProject (85) - : : : : +- CometBroadcastHashJoin (84) - : : : : :- CometProject (82) - : : : : : +- CometBroadcastHashJoin (81) - : : : : : :- CometProject (76) - : : : : : : +- CometBroadcastHashJoin (75) - : : : : : : :- CometProject (73) - : : : : : : : +- CometBroadcastHashJoin (72) - : : : : : : : :- CometProject (68) - : : : : : : : : +- CometBroadcastHashJoin (67) - : : : : : : : : :- CometProject (63) - : : : : : : : : : +- CometBroadcastHashJoin (62) - : : : : : : : : : :- CometProject (60) - : : : : : : : : : : +- CometBroadcastHashJoin (59) - : : : : : : : : : : :- CometProject (54) - : : : : : : : : : : : +- CometBroadcastHashJoin (53) - : : : : : : : : : : : :- CometProject (51) - : : : : : : : : : : : : +- CometBroadcastHashJoin (50) - : : : : : : : : : : : : :- CometProject (46) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (45) - : : : : : : : : : : : : : :- CometProject (41) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (40) - : : : : : : : : : : : : : : :- CometProject (35) - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (34) - : : : : : : : : : : : : : : : :- CometProject (30) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) - : : : : : : : : : : : : : : : : :- CometSort (10) - : : : : : : : : : : : : : : : : : +- CometExchange (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- CometSort (28) - : : : : : : : : : : : : : : : : +- CometProject (27) - : : : : : : : : : : : : : : : : +- CometFilter (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometExchange (24) - : : : : : : : : : : : : : : : : +- CometHashAggregate (23) - : : : : : : : : : : : : : : : : +- CometProject (22) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) - : : : : : : : : : : : : : : : : :- CometSort (15) - : : : : : : : : : : : : : : : : : +- CometExchange (14) - : : : : : : : : : : : : : : : : : +- CometProject (13) - : : : : : : : : : : : : : : : : : +- CometFilter (12) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (11) - : : : : : : : : : : : : : : : : +- CometSort (20) - : : : : : : : : : : : : : : : : +- CometExchange (19) - : : : : : : : : : : : : : : : : +- CometProject (18) - : : : : : : : : : : : : : : : : +- CometFilter (17) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : : : : : : : : : : : : : : : +- CometBroadcastExchange (33) - : : : : : : : : : : : : : : : +- CometFilter (32) - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - : : : : : : : : : : : : : : +- CometBroadcastExchange (39) - : : : : : : : : : : : : : : +- CometProject (38) - : : : : : : : : : : : : : : +- CometFilter (37) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (36) - : : : : : : : : : : : : : +- CometBroadcastExchange (44) - : : : : : : : : : : : : : +- CometFilter (43) - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (42) - : : : : : : : : : : : : +- CometBroadcastExchange (49) - : : : : : : : : : : : : +- CometFilter (48) - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) - : : : : : : : : : : : +- ReusedExchange (52) - : : : : : : : : : : +- CometBroadcastExchange (58) - : : : : : : : : : : +- CometProject (57) - : : : : : : : : : : +- CometFilter (56) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (55) - : : : : : : : : : +- ReusedExchange (61) - : : : : : : : : +- CometBroadcastExchange (66) - : : : : : : : : +- CometFilter (65) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (64) - : : : : : : : +- CometBroadcastExchange (71) - : : : : : : : +- CometFilter (70) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (69) - : : : : : : +- ReusedExchange (74) - : : : : : +- CometBroadcastExchange (80) - : : : : : +- CometProject (79) - : : : : : +- CometFilter (78) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (77) - : : : : +- ReusedExchange (83) - : : : +- CometBroadcastExchange (88) - : : : +- CometFilter (87) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (86) - : : +- ReusedExchange (91) - : +- CometBroadcastExchange (97) - : +- CometProject (96) - : +- CometFilter (95) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (94) - +- CometSort (171) - +- CometExchange (170) - +- CometHashAggregate (169) - +- CometHashAggregate (168) - +- CometProject (167) - +- CometBroadcastHashJoin (166) - :- CometProject (164) - : +- CometBroadcastHashJoin (163) - : :- CometProject (161) - : : +- CometBroadcastHashJoin (160) - : : :- CometProject (158) - : : : +- CometBroadcastHashJoin (157) - : : : :- CometProject (155) - : : : : +- CometBroadcastHashJoin (154) - : : : : :- CometProject (152) - : : : : : +- CometBroadcastHashJoin (151) - : : : : : :- CometProject (149) - : : : : : : +- CometBroadcastHashJoin (148) - : : : : : : :- CometProject (146) - : : : : : : : +- CometBroadcastHashJoin (145) - : : : : : : : :- CometProject (143) - : : : : : : : : +- CometBroadcastHashJoin (142) - : : : : : : : : :- CometProject (140) - : : : : : : : : : +- CometBroadcastHashJoin (139) - : : : : : : : : : :- CometProject (137) - : : : : : : : : : : +- CometBroadcastHashJoin (136) - : : : : : : : : : : :- CometProject (134) - : : : : : : : : : : : +- CometBroadcastHashJoin (133) - : : : : : : : : : : : :- CometProject (131) - : : : : : : : : : : : : +- CometBroadcastHashJoin (130) - : : : : : : : : : : : : :- CometProject (128) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (127) - : : : : : : : : : : : : : :- CometProject (125) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (124) - : : : : : : : : : : : : : : :- CometProject (120) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (119) - : : : : : : : : : : : : : : : :- CometSort (113) - : : : : : : : : : : : : : : : : +- CometExchange (112) - : : : : : : : : : : : : : : : : +- CometProject (111) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (110) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (106) - : : : : : : : : : : : : : : : : : +- CometFilter (105) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (104) - : : : : : : : : : : : : : : : : +- CometProject (109) - : : : : : : : : : : : : : : : : +- CometFilter (108) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (107) - : : : : : : : : : : : : : : : +- CometSort (118) - : : : : : : : : : : : : : : : +- CometProject (117) - : : : : : : : : : : : : : : : +- CometFilter (116) - : : : : : : : : : : : : : : : +- CometHashAggregate (115) - : : : : : : : : : : : : : : : +- ReusedExchange (114) - : : : : : : : : : : : : : : +- CometBroadcastExchange (123) - : : : : : : : : : : : : : : +- CometFilter (122) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (121) - : : : : : : : : : : : : : +- ReusedExchange (126) - : : : : : : : : : : : : +- ReusedExchange (129) - : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : +- ReusedExchange (141) - : : : : : : : +- ReusedExchange (144) - : : : : : : +- ReusedExchange (147) - : : : : : +- ReusedExchange (150) - : : : : +- ReusedExchange (153) - : : : +- ReusedExchange (156) - : : +- ReusedExchange (159) - : +- ReusedExchange (162) - +- ReusedExchange (165) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) CometFilter -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) - -(3) CometBroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) - -(6) CometProject -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] - -(7) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner, BuildLeft - -(8) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(9) CometExchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometSort -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) - -(13) CometProject -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] - -(14) CometExchange -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(15) CometSort -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(17) CometFilter -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) - -(18) CometProject -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(19) CometExchange -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner - -(22) CometProject -Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(23) CometHashAggregate -Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Keys [1]: [cs_item_sk#17] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(24) CometExchange -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometHashAggregate -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Keys [1]: [cs_item_sk#17] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(26) CometFilter -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(27) CometProject -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Arguments: [cs_item_sk#17], [cs_item_sk#17] - -(28) CometSort -Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] - -(29) CometSortMergeJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [1]: [cs_item_sk#17] -Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner - -(30) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(33) CometBroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: [d_date_sk#32, d_year#33] - -(34) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight - -(35) CometProject -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true))) - -(38) CometProject -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37], [s_store_sk#34, s_store_name#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true) AS s_zip#37] - -(39) CometBroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37] - -(40) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight - -(41) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(43) CometFilter -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Condition : (((((isnotnull(c_customer_sk#38) AND isnotnull(c_first_sales_date_sk#43)) AND isnotnull(c_first_shipto_date_sk#42)) AND isnotnull(c_current_cdemo_sk#39)) AND isnotnull(c_current_hdemo_sk#40)) AND isnotnull(c_current_addr_sk#41)) - -(44) CometBroadcastExchange -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(45) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -Right output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_customer_sk#2], [c_customer_sk#38], Inner, BuildRight - -(46) CometProject -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(48) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : isnotnull(d_date_sk#44) - -(49) CometBroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] - -(50) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [c_first_sales_date_sk#43], [d_date_sk#44], Inner, BuildRight - -(51) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43, d_date_sk#44, d_year#45] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] - -(52) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#46, d_year#47] - -(53) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [c_first_shipto_date_sk#42], [d_date_sk#46], Inner, BuildRight - -(54) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45, d_date_sk#46, d_year#47] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#48, cd_marital_status#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true))) - -(57) CometProject -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Arguments: [cd_demo_sk#48, cd_marital_status#50], [cd_demo_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true) AS cd_marital_status#50] - -(58) CometBroadcastExchange -Input [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [cd_demo_sk#48, cd_marital_status#50] - -(59) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_cdemo_sk#3], [cd_demo_sk#48], Inner, BuildRight - -(60) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] - -(61) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#51, cd_marital_status#52] - -(62) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -Right output [2]: [cd_demo_sk#51, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#39], [cd_demo_sk#51], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(63) CometProject -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50, cd_demo_sk#51, cd_marital_status#52] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(65) CometFilter -Input [1]: [p_promo_sk#53] -Condition : isnotnull(p_promo_sk#53) - -(66) CometBroadcastExchange -Input [1]: [p_promo_sk#53] -Arguments: [p_promo_sk#53] - -(67) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [1]: [p_promo_sk#53] -Arguments: [ss_promo_sk#7], [p_promo_sk#53], Inner, BuildRight - -(68) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, p_promo_sk#53] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(70) CometFilter -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Condition : (isnotnull(hd_demo_sk#54) AND isnotnull(hd_income_band_sk#55)) - -(71) CometBroadcastExchange -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [hd_demo_sk#54, hd_income_band_sk#55] - -(72) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_hdemo_sk#4], [hd_demo_sk#54], Inner, BuildRight - -(73) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] - -(74) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] - -(75) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -Right output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [c_current_hdemo_sk#40], [hd_demo_sk#56], Inner, BuildRight - -(76) CometProject -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(78) CometFilter -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Condition : isnotnull(ca_address_sk#58) - -(79) CometProject -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ca_address_sk#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#59, 10, true, false, true) AS ca_street_number#63, ca_street_name#60, ca_city#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#62, 10, true, false, true) AS ca_zip#64] - -(80) CometBroadcastExchange -Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(81) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -Right output [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_addr_sk#5], [ca_address_sk#58], Inner, BuildRight - -(82) CometProject -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(83) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(84) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Right output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [c_current_addr_sk#41], [ca_address_sk#65], Inner, BuildRight - -(85) CometProject -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(87) CometFilter -Input [1]: [ib_income_band_sk#70] -Condition : isnotnull(ib_income_band_sk#70) - -(88) CometBroadcastExchange -Input [1]: [ib_income_band_sk#70] -Arguments: [ib_income_band_sk#70] - -(89) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#70] -Arguments: [hd_income_band_sk#55], [ib_income_band_sk#70], Inner, BuildRight - -(90) CometProject -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#70] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(91) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#71] - -(92) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#71] -Arguments: [hd_income_band_sk#57], [ib_income_band_sk#71], Inner, BuildRight - -(93) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#71] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(95) CometFilter -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Condition : ((((((isnotnull(i_current_price#73) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#74, 20, true, false, true) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#73 >= 64.00)) AND (i_current_price#73 <= 74.00)) AND (i_current_price#73 >= 65.00)) AND (i_current_price#73 <= 79.00)) AND isnotnull(i_item_sk#72)) - -(96) CometProject -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Arguments: [i_item_sk#72, i_product_name#76], [i_item_sk#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#75, 50, true, false, true) AS i_product_name#76] - -(97) CometBroadcastExchange -Input [2]: [i_item_sk#72, i_product_name#76] -Arguments: [i_item_sk#72, i_product_name#76] - -(98) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [2]: [i_item_sk#72, i_product_name#76] -Arguments: [ss_item_sk#1], [i_item_sk#72], Inner, BuildRight - -(99) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] - -(100) CometHashAggregate -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] - -(101) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#77, sum#78, sum#79, sum#80] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] - -(102) CometExchange -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: hashpartitioning(item_sk#82, store_name#83, store_zip#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(103) CometSort -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97], [item_sk#82 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, store_zip#84 ASC NULLS FIRST] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#109), dynamicpruningexpression(ss_sold_date_sk#109 IN dynamicpruning#110)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(105) CometFilter -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Condition : (((((((isnotnull(ss_item_sk#98) AND isnotnull(ss_ticket_number#105)) AND isnotnull(ss_store_sk#103)) AND isnotnull(ss_customer_sk#99)) AND isnotnull(ss_cdemo_sk#100)) AND isnotnull(ss_promo_sk#104)) AND isnotnull(ss_hdemo_sk#101)) AND isnotnull(ss_addr_sk#102)) - -(106) CometBroadcastExchange -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(108) CometFilter -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) - -(109) CometProject -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] - -(110) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_ticket_number#105], [sr_item_sk#111, sr_ticket_number#112], Inner, BuildLeft - -(111) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(112) CometExchange -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: hashpartitioning(ss_item_sk#98, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(113) CometSort -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98 ASC NULLS FIRST] - -(114) ReusedExchange [Reuses operator id: 24] -Output [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] - -(115) CometHashAggregate -Input [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] -Keys [1]: [cs_item_sk#114] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#118)), sum(((cr_refunded_cash#119 + cr_reversed_charge#120) + cr_store_credit#121))] - -(116) CometFilter -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(117) CometProject -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Arguments: [cs_item_sk#114], [cs_item_sk#114] - -(118) CometSort -Input [1]: [cs_item_sk#114] -Arguments: [cs_item_sk#114], [cs_item_sk#114 ASC NULLS FIRST] - -(119) CometSortMergeJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [1]: [cs_item_sk#114] -Arguments: [ss_item_sk#98], [cs_item_sk#114], Inner - -(120) CometProject -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, cs_item_sk#114] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(121) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(122) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(123) CometBroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: [d_date_sk#122, d_year#123] - -(124) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [d_date_sk#122, d_year#123] -Arguments: [ss_sold_date_sk#109], [d_date_sk#122], Inner, BuildRight - -(125) CometProject -Input [13]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, d_date_sk#122, d_year#123] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] - -(126) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] - -(127) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] -Right output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_store_sk#103], [s_store_sk#124], Inner, BuildRight - -(128) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] - -(129) ReusedExchange [Reuses operator id: 44] -Output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(130) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] -Right output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_customer_sk#99], [c_customer_sk#126], Inner, BuildRight - -(131) CometProject -Input [18]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(132) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#132, d_year#133] - -(133) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Right output [2]: [d_date_sk#132, d_year#133] -Arguments: [c_first_sales_date_sk#131], [d_date_sk#132], Inner, BuildRight - -(134) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131, d_date_sk#132, d_year#133] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] - -(135) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#134, d_year#135] - -(136) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] -Right output [2]: [d_date_sk#134, d_year#135] -Arguments: [c_first_shipto_date_sk#130], [d_date_sk#134], Inner, BuildRight - -(137) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133, d_date_sk#134, d_year#135] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(138) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#136, cd_marital_status#50] - -(139) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_cdemo_sk#100], [cd_demo_sk#136], Inner, BuildRight - -(140) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] - -(141) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#137, cd_marital_status#52] - -(142) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] -Right output [2]: [cd_demo_sk#137, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#127], [cd_demo_sk#137], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(143) CometProject -Input [18]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50, cd_demo_sk#137, cd_marital_status#52] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(144) ReusedExchange [Reuses operator id: 66] -Output [1]: [p_promo_sk#138] - -(145) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [1]: [p_promo_sk#138] -Arguments: [ss_promo_sk#104], [p_promo_sk#138], Inner, BuildRight - -(146) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, p_promo_sk#138] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(147) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] - -(148) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_hdemo_sk#101], [hd_demo_sk#139], Inner, BuildRight - -(149) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] - -(150) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] - -(151) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] -Right output [2]: [hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [c_current_hdemo_sk#128], [hd_demo_sk#141], Inner, BuildRight - -(152) CometProject -Input [15]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] - -(153) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(154) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] -Right output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_addr_sk#102], [ca_address_sk#143], Inner, BuildRight - -(155) CometProject -Input [18]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(156) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(157) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Right output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [c_current_addr_sk#129], [ca_address_sk#146], Inner, BuildRight - -(158) CometProject -Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(159) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#149] - -(160) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#149] -Arguments: [hd_income_band_sk#140], [ib_income_band_sk#149], Inner, BuildRight - -(161) CometProject -Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#149] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(162) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#150] - -(163) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#150] -Arguments: [hd_income_band_sk#142], [ib_income_band_sk#150], Inner, BuildRight - -(164) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#150] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(165) ReusedExchange [Reuses operator id: 97] -Output [2]: [i_item_sk#151, i_product_name#76] - -(166) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [2]: [i_item_sk#151, i_product_name#76] -Arguments: [ss_item_sk#98], [i_item_sk#151], Inner, BuildRight - -(167) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] - -(168) CometHashAggregate -Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#106)), partial_sum(UnscaledValue(ss_list_price#107)), partial_sum(UnscaledValue(ss_coupon_amt#108))] - -(169) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135, count#77, sum#152, sum#153, sum#154] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#106)), sum(UnscaledValue(ss_list_price#107)), sum(UnscaledValue(ss_coupon_amt#108))] - -(170) CometExchange -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: hashpartitioning(item_sk#155, store_name#156, store_zip#157, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(171) CometSort -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162], [item_sk#155 ASC NULLS FIRST, store_name#156 ASC NULLS FIRST, store_zip#157 ASC NULLS FIRST] - -(172) CometSortMergeJoin -Left output [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Right output [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#156, store_zip#157], Inner, (cnt#159 <= cnt#94) - -(173) CometProject -Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -(174) CometExchange -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(175) CometSort -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST] - -(176) CometColumnarToRow [codegen id : 1] -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (180) -+- * CometColumnarToRow (179) - +- CometFilter (178) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) - - -(177) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(178) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(179) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#33] - -(180) BroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 -BroadcastExchange (184) -+- * CometColumnarToRow (183) - +- CometFilter (182) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) - - -(181) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(182) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(183) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#122, d_year#123] - -(184) BroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/simplified.txt deleted file mode 100644 index 46fe063430..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,192 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometExchange [product_name,store_name,cnt,s1,s1] #1 - CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #3 - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [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] #11 - 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 [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk] - ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #20 - 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_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] #21 - 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 [native_iceberg_compat] 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 #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - CometBroadcastExchange [d_date_sk,d_year] #23 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt deleted file mode 100644 index 077daeca3a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt +++ /dev/null @@ -1,1011 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (176) -+- CometSort (175) - +- CometExchange (174) - +- CometProject (173) - +- CometSortMergeJoin (172) - :- CometSort (103) - : +- CometExchange (102) - : +- CometHashAggregate (101) - : +- CometHashAggregate (100) - : +- CometProject (99) - : +- CometBroadcastHashJoin (98) - : :- CometProject (93) - : : +- CometBroadcastHashJoin (92) - : : :- CometProject (90) - : : : +- CometBroadcastHashJoin (89) - : : : :- CometProject (85) - : : : : +- CometBroadcastHashJoin (84) - : : : : :- CometProject (82) - : : : : : +- CometBroadcastHashJoin (81) - : : : : : :- CometProject (76) - : : : : : : +- CometBroadcastHashJoin (75) - : : : : : : :- CometProject (73) - : : : : : : : +- CometBroadcastHashJoin (72) - : : : : : : : :- CometProject (68) - : : : : : : : : +- CometBroadcastHashJoin (67) - : : : : : : : : :- CometProject (63) - : : : : : : : : : +- CometBroadcastHashJoin (62) - : : : : : : : : : :- CometProject (60) - : : : : : : : : : : +- CometBroadcastHashJoin (59) - : : : : : : : : : : :- CometProject (54) - : : : : : : : : : : : +- CometBroadcastHashJoin (53) - : : : : : : : : : : : :- CometProject (51) - : : : : : : : : : : : : +- CometBroadcastHashJoin (50) - : : : : : : : : : : : : :- CometProject (46) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (45) - : : : : : : : : : : : : : :- CometProject (41) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (40) - : : : : : : : : : : : : : : :- CometProject (35) - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (34) - : : : : : : : : : : : : : : : :- CometProject (30) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) - : : : : : : : : : : : : : : : : :- CometSort (10) - : : : : : : : : : : : : : : : : : +- CometExchange (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- CometSort (28) - : : : : : : : : : : : : : : : : +- CometProject (27) - : : : : : : : : : : : : : : : : +- CometFilter (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometExchange (24) - : : : : : : : : : : : : : : : : +- CometHashAggregate (23) - : : : : : : : : : : : : : : : : +- CometProject (22) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) - : : : : : : : : : : : : : : : : :- CometSort (15) - : : : : : : : : : : : : : : : : : +- CometExchange (14) - : : : : : : : : : : : : : : : : : +- CometProject (13) - : : : : : : : : : : : : : : : : : +- CometFilter (12) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (11) - : : : : : : : : : : : : : : : : +- CometSort (20) - : : : : : : : : : : : : : : : : +- CometExchange (19) - : : : : : : : : : : : : : : : : +- CometProject (18) - : : : : : : : : : : : : : : : : +- CometFilter (17) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : : : : : : : : : : : : : : : +- CometBroadcastExchange (33) - : : : : : : : : : : : : : : : +- CometFilter (32) - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - : : : : : : : : : : : : : : +- CometBroadcastExchange (39) - : : : : : : : : : : : : : : +- CometProject (38) - : : : : : : : : : : : : : : +- CometFilter (37) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (36) - : : : : : : : : : : : : : +- CometBroadcastExchange (44) - : : : : : : : : : : : : : +- CometFilter (43) - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (42) - : : : : : : : : : : : : +- CometBroadcastExchange (49) - : : : : : : : : : : : : +- CometFilter (48) - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) - : : : : : : : : : : : +- ReusedExchange (52) - : : : : : : : : : : +- CometBroadcastExchange (58) - : : : : : : : : : : +- CometProject (57) - : : : : : : : : : : +- CometFilter (56) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (55) - : : : : : : : : : +- ReusedExchange (61) - : : : : : : : : +- CometBroadcastExchange (66) - : : : : : : : : +- CometFilter (65) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (64) - : : : : : : : +- CometBroadcastExchange (71) - : : : : : : : +- CometFilter (70) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (69) - : : : : : : +- ReusedExchange (74) - : : : : : +- CometBroadcastExchange (80) - : : : : : +- CometProject (79) - : : : : : +- CometFilter (78) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (77) - : : : : +- ReusedExchange (83) - : : : +- CometBroadcastExchange (88) - : : : +- CometFilter (87) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (86) - : : +- ReusedExchange (91) - : +- CometBroadcastExchange (97) - : +- CometProject (96) - : +- CometFilter (95) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (94) - +- CometSort (171) - +- CometExchange (170) - +- CometHashAggregate (169) - +- CometHashAggregate (168) - +- CometProject (167) - +- CometBroadcastHashJoin (166) - :- CometProject (164) - : +- CometBroadcastHashJoin (163) - : :- CometProject (161) - : : +- CometBroadcastHashJoin (160) - : : :- CometProject (158) - : : : +- CometBroadcastHashJoin (157) - : : : :- CometProject (155) - : : : : +- CometBroadcastHashJoin (154) - : : : : :- CometProject (152) - : : : : : +- CometBroadcastHashJoin (151) - : : : : : :- CometProject (149) - : : : : : : +- CometBroadcastHashJoin (148) - : : : : : : :- CometProject (146) - : : : : : : : +- CometBroadcastHashJoin (145) - : : : : : : : :- CometProject (143) - : : : : : : : : +- CometBroadcastHashJoin (142) - : : : : : : : : :- CometProject (140) - : : : : : : : : : +- CometBroadcastHashJoin (139) - : : : : : : : : : :- CometProject (137) - : : : : : : : : : : +- CometBroadcastHashJoin (136) - : : : : : : : : : : :- CometProject (134) - : : : : : : : : : : : +- CometBroadcastHashJoin (133) - : : : : : : : : : : : :- CometProject (131) - : : : : : : : : : : : : +- CometBroadcastHashJoin (130) - : : : : : : : : : : : : :- CometProject (128) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (127) - : : : : : : : : : : : : : :- CometProject (125) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (124) - : : : : : : : : : : : : : : :- CometProject (120) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (119) - : : : : : : : : : : : : : : : :- CometSort (113) - : : : : : : : : : : : : : : : : +- CometExchange (112) - : : : : : : : : : : : : : : : : +- CometProject (111) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (110) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (106) - : : : : : : : : : : : : : : : : : +- CometFilter (105) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (104) - : : : : : : : : : : : : : : : : +- CometProject (109) - : : : : : : : : : : : : : : : : +- CometFilter (108) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (107) - : : : : : : : : : : : : : : : +- CometSort (118) - : : : : : : : : : : : : : : : +- CometProject (117) - : : : : : : : : : : : : : : : +- CometFilter (116) - : : : : : : : : : : : : : : : +- CometHashAggregate (115) - : : : : : : : : : : : : : : : +- ReusedExchange (114) - : : : : : : : : : : : : : : +- CometBroadcastExchange (123) - : : : : : : : : : : : : : : +- CometFilter (122) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (121) - : : : : : : : : : : : : : +- ReusedExchange (126) - : : : : : : : : : : : : +- ReusedExchange (129) - : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : +- ReusedExchange (141) - : : : : : : : +- ReusedExchange (144) - : : : : : : +- ReusedExchange (147) - : : : : : +- ReusedExchange (150) - : : : : +- ReusedExchange (153) - : : : +- ReusedExchange (156) - : : +- ReusedExchange (159) - : +- ReusedExchange (162) - +- ReusedExchange (165) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) CometFilter -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) - -(3) CometBroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) - -(6) CometProject -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] - -(7) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner, BuildLeft - -(8) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(9) CometExchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometSort -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) - -(13) CometProject -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] - -(14) CometExchange -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(15) CometSort -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(17) CometFilter -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) - -(18) CometProject -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(19) CometExchange -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner - -(22) CometProject -Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(23) CometHashAggregate -Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Keys [1]: [cs_item_sk#17] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(24) CometExchange -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometHashAggregate -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Keys [1]: [cs_item_sk#17] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(26) CometFilter -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(27) CometProject -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Arguments: [cs_item_sk#17], [cs_item_sk#17] - -(28) CometSort -Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] - -(29) CometSortMergeJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [1]: [cs_item_sk#17] -Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner - -(30) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(33) CometBroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: [d_date_sk#32, d_year#33] - -(34) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight - -(35) CometProject -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true))) - -(38) CometProject -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37], [s_store_sk#34, s_store_name#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true) AS s_zip#37] - -(39) CometBroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37] - -(40) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight - -(41) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(43) CometFilter -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Condition : (((((isnotnull(c_customer_sk#38) AND isnotnull(c_first_sales_date_sk#43)) AND isnotnull(c_first_shipto_date_sk#42)) AND isnotnull(c_current_cdemo_sk#39)) AND isnotnull(c_current_hdemo_sk#40)) AND isnotnull(c_current_addr_sk#41)) - -(44) CometBroadcastExchange -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(45) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -Right output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_customer_sk#2], [c_customer_sk#38], Inner, BuildRight - -(46) CometProject -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(48) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : isnotnull(d_date_sk#44) - -(49) CometBroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] - -(50) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [c_first_sales_date_sk#43], [d_date_sk#44], Inner, BuildRight - -(51) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43, d_date_sk#44, d_year#45] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] - -(52) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#46, d_year#47] - -(53) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [c_first_shipto_date_sk#42], [d_date_sk#46], Inner, BuildRight - -(54) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45, d_date_sk#46, d_year#47] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#48, cd_marital_status#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true))) - -(57) CometProject -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Arguments: [cd_demo_sk#48, cd_marital_status#50], [cd_demo_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true) AS cd_marital_status#50] - -(58) CometBroadcastExchange -Input [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [cd_demo_sk#48, cd_marital_status#50] - -(59) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_cdemo_sk#3], [cd_demo_sk#48], Inner, BuildRight - -(60) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] - -(61) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#51, cd_marital_status#52] - -(62) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -Right output [2]: [cd_demo_sk#51, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#39], [cd_demo_sk#51], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(63) CometProject -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50, cd_demo_sk#51, cd_marital_status#52] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(65) CometFilter -Input [1]: [p_promo_sk#53] -Condition : isnotnull(p_promo_sk#53) - -(66) CometBroadcastExchange -Input [1]: [p_promo_sk#53] -Arguments: [p_promo_sk#53] - -(67) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [1]: [p_promo_sk#53] -Arguments: [ss_promo_sk#7], [p_promo_sk#53], Inner, BuildRight - -(68) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, p_promo_sk#53] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(70) CometFilter -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Condition : (isnotnull(hd_demo_sk#54) AND isnotnull(hd_income_band_sk#55)) - -(71) CometBroadcastExchange -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [hd_demo_sk#54, hd_income_band_sk#55] - -(72) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_hdemo_sk#4], [hd_demo_sk#54], Inner, BuildRight - -(73) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] - -(74) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] - -(75) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -Right output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [c_current_hdemo_sk#40], [hd_demo_sk#56], Inner, BuildRight - -(76) CometProject -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(78) CometFilter -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Condition : isnotnull(ca_address_sk#58) - -(79) CometProject -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ca_address_sk#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#59, 10, true, false, true) AS ca_street_number#63, ca_street_name#60, ca_city#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#62, 10, true, false, true) AS ca_zip#64] - -(80) CometBroadcastExchange -Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(81) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -Right output [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_addr_sk#5], [ca_address_sk#58], Inner, BuildRight - -(82) CometProject -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(83) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(84) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Right output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [c_current_addr_sk#41], [ca_address_sk#65], Inner, BuildRight - -(85) CometProject -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(87) CometFilter -Input [1]: [ib_income_band_sk#70] -Condition : isnotnull(ib_income_band_sk#70) - -(88) CometBroadcastExchange -Input [1]: [ib_income_band_sk#70] -Arguments: [ib_income_band_sk#70] - -(89) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#70] -Arguments: [hd_income_band_sk#55], [ib_income_band_sk#70], Inner, BuildRight - -(90) CometProject -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#70] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(91) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#71] - -(92) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#71] -Arguments: [hd_income_band_sk#57], [ib_income_band_sk#71], Inner, BuildRight - -(93) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#71] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(95) CometFilter -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Condition : ((((((isnotnull(i_current_price#73) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#74, 20, true, false, true) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#73 >= 64.00)) AND (i_current_price#73 <= 74.00)) AND (i_current_price#73 >= 65.00)) AND (i_current_price#73 <= 79.00)) AND isnotnull(i_item_sk#72)) - -(96) CometProject -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Arguments: [i_item_sk#72, i_product_name#76], [i_item_sk#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#75, 50, true, false, true) AS i_product_name#76] - -(97) CometBroadcastExchange -Input [2]: [i_item_sk#72, i_product_name#76] -Arguments: [i_item_sk#72, i_product_name#76] - -(98) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [2]: [i_item_sk#72, i_product_name#76] -Arguments: [ss_item_sk#1], [i_item_sk#72], Inner, BuildRight - -(99) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] - -(100) CometHashAggregate -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] - -(101) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#77, sum#78, sum#79, sum#80] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] - -(102) CometExchange -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: hashpartitioning(item_sk#82, store_name#83, store_zip#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(103) CometSort -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97], [item_sk#82 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, store_zip#84 ASC NULLS FIRST] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#109), dynamicpruningexpression(ss_sold_date_sk#109 IN dynamicpruning#110)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(105) CometFilter -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Condition : (((((((isnotnull(ss_item_sk#98) AND isnotnull(ss_ticket_number#105)) AND isnotnull(ss_store_sk#103)) AND isnotnull(ss_customer_sk#99)) AND isnotnull(ss_cdemo_sk#100)) AND isnotnull(ss_promo_sk#104)) AND isnotnull(ss_hdemo_sk#101)) AND isnotnull(ss_addr_sk#102)) - -(106) CometBroadcastExchange -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(108) CometFilter -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) - -(109) CometProject -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] - -(110) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_ticket_number#105], [sr_item_sk#111, sr_ticket_number#112], Inner, BuildLeft - -(111) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(112) CometExchange -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: hashpartitioning(ss_item_sk#98, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(113) CometSort -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98 ASC NULLS FIRST] - -(114) ReusedExchange [Reuses operator id: 24] -Output [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] - -(115) CometHashAggregate -Input [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] -Keys [1]: [cs_item_sk#114] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#118)), sum(((cr_refunded_cash#119 + cr_reversed_charge#120) + cr_store_credit#121))] - -(116) CometFilter -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(117) CometProject -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Arguments: [cs_item_sk#114], [cs_item_sk#114] - -(118) CometSort -Input [1]: [cs_item_sk#114] -Arguments: [cs_item_sk#114], [cs_item_sk#114 ASC NULLS FIRST] - -(119) CometSortMergeJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [1]: [cs_item_sk#114] -Arguments: [ss_item_sk#98], [cs_item_sk#114], Inner - -(120) CometProject -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, cs_item_sk#114] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(121) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(122) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(123) CometBroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: [d_date_sk#122, d_year#123] - -(124) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [d_date_sk#122, d_year#123] -Arguments: [ss_sold_date_sk#109], [d_date_sk#122], Inner, BuildRight - -(125) CometProject -Input [13]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, d_date_sk#122, d_year#123] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] - -(126) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] - -(127) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] -Right output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_store_sk#103], [s_store_sk#124], Inner, BuildRight - -(128) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] - -(129) ReusedExchange [Reuses operator id: 44] -Output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(130) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] -Right output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_customer_sk#99], [c_customer_sk#126], Inner, BuildRight - -(131) CometProject -Input [18]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(132) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#132, d_year#133] - -(133) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Right output [2]: [d_date_sk#132, d_year#133] -Arguments: [c_first_sales_date_sk#131], [d_date_sk#132], Inner, BuildRight - -(134) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131, d_date_sk#132, d_year#133] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] - -(135) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#134, d_year#135] - -(136) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] -Right output [2]: [d_date_sk#134, d_year#135] -Arguments: [c_first_shipto_date_sk#130], [d_date_sk#134], Inner, BuildRight - -(137) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133, d_date_sk#134, d_year#135] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(138) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#136, cd_marital_status#50] - -(139) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_cdemo_sk#100], [cd_demo_sk#136], Inner, BuildRight - -(140) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] - -(141) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#137, cd_marital_status#52] - -(142) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] -Right output [2]: [cd_demo_sk#137, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#127], [cd_demo_sk#137], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(143) CometProject -Input [18]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50, cd_demo_sk#137, cd_marital_status#52] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(144) ReusedExchange [Reuses operator id: 66] -Output [1]: [p_promo_sk#138] - -(145) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [1]: [p_promo_sk#138] -Arguments: [ss_promo_sk#104], [p_promo_sk#138], Inner, BuildRight - -(146) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, p_promo_sk#138] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(147) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] - -(148) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_hdemo_sk#101], [hd_demo_sk#139], Inner, BuildRight - -(149) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] - -(150) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] - -(151) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] -Right output [2]: [hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [c_current_hdemo_sk#128], [hd_demo_sk#141], Inner, BuildRight - -(152) CometProject -Input [15]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] - -(153) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(154) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] -Right output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_addr_sk#102], [ca_address_sk#143], Inner, BuildRight - -(155) CometProject -Input [18]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(156) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(157) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Right output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [c_current_addr_sk#129], [ca_address_sk#146], Inner, BuildRight - -(158) CometProject -Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(159) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#149] - -(160) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#149] -Arguments: [hd_income_band_sk#140], [ib_income_band_sk#149], Inner, BuildRight - -(161) CometProject -Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#149] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(162) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#150] - -(163) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#150] -Arguments: [hd_income_band_sk#142], [ib_income_band_sk#150], Inner, BuildRight - -(164) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#150] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(165) ReusedExchange [Reuses operator id: 97] -Output [2]: [i_item_sk#151, i_product_name#76] - -(166) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [2]: [i_item_sk#151, i_product_name#76] -Arguments: [ss_item_sk#98], [i_item_sk#151], Inner, BuildRight - -(167) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] - -(168) CometHashAggregate -Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#106)), partial_sum(UnscaledValue(ss_list_price#107)), partial_sum(UnscaledValue(ss_coupon_amt#108))] - -(169) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135, count#77, sum#152, sum#153, sum#154] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#106)), sum(UnscaledValue(ss_list_price#107)), sum(UnscaledValue(ss_coupon_amt#108))] - -(170) CometExchange -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: hashpartitioning(item_sk#155, store_name#156, store_zip#157, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(171) CometSort -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162], [item_sk#155 ASC NULLS FIRST, store_name#156 ASC NULLS FIRST, store_zip#157 ASC NULLS FIRST] - -(172) CometSortMergeJoin -Left output [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Right output [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#156, store_zip#157], Inner, (cnt#159 <= cnt#94) - -(173) CometProject -Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -(174) CometExchange -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(175) CometSort -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST] - -(176) CometColumnarToRow [codegen id : 1] -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (180) -+- * CometColumnarToRow (179) - +- CometFilter (178) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) - - -(177) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(178) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(179) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#33] - -(180) BroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 -BroadcastExchange (184) -+- * CometColumnarToRow (183) - +- CometFilter (182) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) - - -(181) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(182) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(183) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#122, d_year#123] - -(184) BroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/extended.txt deleted file mode 100644 index 059acab385..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/extended.txt +++ /dev/null @@ -1,247 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 46fe063430..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt +++ /dev/null @@ -1,192 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometExchange [product_name,store_name,cnt,s1,s1] #1 - CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #3 - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [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] #11 - 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 [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk] - ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #20 - 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_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] #21 - 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 [native_iceberg_compat] 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 #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - CometBroadcastExchange [d_date_sk,d_year] #23 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/explain.txt deleted file mode 100644 index 1436e02064..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/explain.txt +++ /dev/null @@ -1,548 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (90) -+- * Filter (89) - +- Window (88) - +- WindowGroupLimit (87) - +- * CometColumnarToRow (86) - +- CometSort (85) - +- CometColumnarExchange (84) - +- WindowGroupLimit (83) - +- * Sort (82) - +- Union (81) - :- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.item (14) - :- * HashAggregate (31) - : +- * CometColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- * HashAggregate (28) - : +- * HashAggregate (27) - : +- * CometColumnarToRow (26) - : +- ReusedExchange (25) - :- * HashAggregate (38) - : +- * CometColumnarToRow (37) - : +- CometColumnarExchange (36) - : +- * HashAggregate (35) - : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) - : +- ReusedExchange (32) - :- * HashAggregate (45) - : +- * CometColumnarToRow (44) - : +- CometColumnarExchange (43) - : +- * HashAggregate (42) - : +- * HashAggregate (41) - : +- * CometColumnarToRow (40) - : +- ReusedExchange (39) - :- * HashAggregate (52) - : +- * CometColumnarToRow (51) - : +- CometColumnarExchange (50) - : +- * HashAggregate (49) - : +- * HashAggregate (48) - : +- * CometColumnarToRow (47) - : +- ReusedExchange (46) - :- * HashAggregate (59) - : +- * CometColumnarToRow (58) - : +- CometColumnarExchange (57) - : +- * HashAggregate (56) - : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) - : +- ReusedExchange (53) - :- * HashAggregate (66) - : +- * CometColumnarToRow (65) - : +- CometColumnarExchange (64) - : +- * HashAggregate (63) - : +- * HashAggregate (62) - : +- * CometColumnarToRow (61) - : +- ReusedExchange (60) - :- * HashAggregate (73) - : +- * CometColumnarToRow (72) - : +- CometColumnarExchange (71) - : +- * HashAggregate (70) - : +- * HashAggregate (69) - : +- * CometColumnarToRow (68) - : +- ReusedExchange (67) - +- * HashAggregate (80) - +- * CometColumnarToRow (79) - +- CometColumnarExchange (78) - +- * HashAggregate (77) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- ReusedExchange (74) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(4) ReusedExchange [Reuses operator id: 95] -Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_store_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [s_store_sk#11, s_store_id#12] -Condition : isnotnull(s_store_sk#11) - -(9) CometProject -Input [2]: [s_store_sk#11, s_store_id#12] -Arguments: [s_store_sk#11, s_store_id#13], [s_store_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#12, 16, true, false, true) AS s_store_id#13] - -(10) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#11, s_store_id#13] - -(11) BroadcastExchange -Input [2]: [s_store_sk#11, s_store_id#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#13] - -(14) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -Condition : isnotnull(i_item_sk#14) - -(16) CometProject -Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -Arguments: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22], [i_item_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#15, 50, true, false, true) AS i_brand#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#16, 50, true, false, true) AS i_class#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#17, 50, true, false, true) AS i_category#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#18, 50, true, false, true) AS i_product_name#22] - -(17) CometColumnarToRow [codegen id : 3] -Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] - -(18) BroadcastExchange -Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#14] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_brand#19, i_class#20, i_category#21, i_product_name#22] -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] - -(21) HashAggregate [codegen id : 4] -Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_brand#19, i_class#20, i_category#21, i_product_name#22] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] -Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [2]: [sum#23, isEmpty#24] -Results [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] - -(22) CometColumnarExchange -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 5] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] - -(24) HashAggregate [codegen id : 5] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#27] -Results [9]: [i_category#21 AS i_category#28, i_class#20 AS i_class#29, i_brand#19 AS i_brand#30, i_product_name#22 AS i_product_name#31, d_year#8 AS d_year#32, d_qoy#10 AS d_qoy#33, d_moy#9 AS d_moy#34, s_store_id#13 AS s_store_id#35, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#27 as decimal(38,2)) AS sumsales#36] - -(25) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] - -(26) CometColumnarToRow [codegen id : 10] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] - -(27) HashAggregate [codegen id : 10] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#27] -Results [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#27 AS sumsales#44] - -(28) HashAggregate [codegen id : 10] -Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sumsales#44] -Keys [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39] -Functions [1]: [partial_sum(sumsales#44)] -Aggregate Attributes [2]: [sum#45, isEmpty#46] -Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] - -(29) CometColumnarExchange -Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 11] -Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] - -(31) HashAggregate [codegen id : 11] -Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] -Keys [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39] -Functions [1]: [sum(sumsales#44)] -Aggregate Attributes [1]: [sum(sumsales#44)#49] -Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, null AS s_store_id#50, sum(sumsales#44)#49 AS sumsales#51] - -(32) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] - -(33) CometColumnarToRow [codegen id : 16] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] - -(34) HashAggregate [codegen id : 16] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))#27] -Results [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))#27 AS sumsales#59] - -(35) HashAggregate [codegen id : 16] -Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sumsales#59] -Keys [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53] -Functions [1]: [partial_sum(sumsales#59)] -Aggregate Attributes [2]: [sum#60, isEmpty#61] -Results [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] - -(36) CometColumnarExchange -Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(37) CometColumnarToRow [codegen id : 17] -Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] - -(38) HashAggregate [codegen id : 17] -Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#62, isEmpty#63] -Keys [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53] -Functions [1]: [sum(sumsales#59)] -Aggregate Attributes [1]: [sum(sumsales#59)#64] -Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, null AS d_moy#65, null AS s_store_id#66, sum(sumsales#59)#64 AS sumsales#67] - -(39) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13, sum#71, isEmpty#72] - -(40) CometColumnarToRow [codegen id : 22] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13, sum#71, isEmpty#72] - -(41) HashAggregate [codegen id : 22] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13, sum#71, isEmpty#72] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, d_qoy#69, d_moy#70, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#73 * cast(ss_quantity#74 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#73 * cast(ss_quantity#74 as decimal(10,0))), 0.00))#27] -Results [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum(coalesce((ss_sales_price#73 * cast(ss_quantity#74 as decimal(10,0))), 0.00))#27 AS sumsales#75] - -(42) HashAggregate [codegen id : 22] -Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sumsales#75] -Keys [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68] -Functions [1]: [partial_sum(sumsales#75)] -Aggregate Attributes [2]: [sum#76, isEmpty#77] -Results [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] - -(43) CometColumnarExchange -Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(44) CometColumnarToRow [codegen id : 23] -Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] - -(45) HashAggregate [codegen id : 23] -Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, sum#78, isEmpty#79] -Keys [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68] -Functions [1]: [sum(sumsales#75)] -Aggregate Attributes [1]: [sum(sumsales#75)#80] -Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#68, null AS d_qoy#81, null AS d_moy#82, null AS s_store_id#83, sum(sumsales#75)#80 AS sumsales#84] - -(46) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13, sum#88, isEmpty#89] - -(47) CometColumnarToRow [codegen id : 28] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13, sum#88, isEmpty#89] - -(48) HashAggregate [codegen id : 28] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13, sum#88, isEmpty#89] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#85, d_qoy#86, d_moy#87, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#90 * cast(ss_quantity#91 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#90 * cast(ss_quantity#91 as decimal(10,0))), 0.00))#27] -Results [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum(coalesce((ss_sales_price#90 * cast(ss_quantity#91 as decimal(10,0))), 0.00))#27 AS sumsales#92] - -(49) HashAggregate [codegen id : 28] -Input [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sumsales#92] -Keys [4]: [i_category#21, i_class#20, i_brand#19, i_product_name#22] -Functions [1]: [partial_sum(sumsales#92)] -Aggregate Attributes [2]: [sum#93, isEmpty#94] -Results [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] - -(50) CometColumnarExchange -Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(51) CometColumnarToRow [codegen id : 29] -Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] - -(52) HashAggregate [codegen id : 29] -Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#95, isEmpty#96] -Keys [4]: [i_category#21, i_class#20, i_brand#19, i_product_name#22] -Functions [1]: [sum(sumsales#92)] -Aggregate Attributes [1]: [sum(sumsales#92)#97] -Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, null AS d_year#98, null AS d_qoy#99, null AS d_moy#100, null AS s_store_id#101, sum(sumsales#92)#97 AS sumsales#102] - -(53) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13, sum#106, isEmpty#107] - -(54) CometColumnarToRow [codegen id : 34] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13, sum#106, isEmpty#107] - -(55) HashAggregate [codegen id : 34] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13, sum#106, isEmpty#107] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#103, d_qoy#104, d_moy#105, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#108 * cast(ss_quantity#109 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#108 * cast(ss_quantity#109 as decimal(10,0))), 0.00))#27] -Results [4]: [i_category#21, i_class#20, i_brand#19, sum(coalesce((ss_sales_price#108 * cast(ss_quantity#109 as decimal(10,0))), 0.00))#27 AS sumsales#110] - -(56) HashAggregate [codegen id : 34] -Input [4]: [i_category#21, i_class#20, i_brand#19, sumsales#110] -Keys [3]: [i_category#21, i_class#20, i_brand#19] -Functions [1]: [partial_sum(sumsales#110)] -Aggregate Attributes [2]: [sum#111, isEmpty#112] -Results [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] - -(57) CometColumnarExchange -Input [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(58) CometColumnarToRow [codegen id : 35] -Input [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] - -(59) HashAggregate [codegen id : 35] -Input [5]: [i_category#21, i_class#20, i_brand#19, sum#113, isEmpty#114] -Keys [3]: [i_category#21, i_class#20, i_brand#19] -Functions [1]: [sum(sumsales#110)] -Aggregate Attributes [1]: [sum(sumsales#110)#115] -Results [9]: [i_category#21, i_class#20, i_brand#19, null AS i_product_name#116, null AS d_year#117, null AS d_qoy#118, null AS d_moy#119, null AS s_store_id#120, sum(sumsales#110)#115 AS sumsales#121] - -(60) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13, sum#125, isEmpty#126] - -(61) CometColumnarToRow [codegen id : 40] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13, sum#125, isEmpty#126] - -(62) HashAggregate [codegen id : 40] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13, sum#125, isEmpty#126] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#122, d_qoy#123, d_moy#124, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#127 * cast(ss_quantity#128 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#127 * cast(ss_quantity#128 as decimal(10,0))), 0.00))#27] -Results [3]: [i_category#21, i_class#20, sum(coalesce((ss_sales_price#127 * cast(ss_quantity#128 as decimal(10,0))), 0.00))#27 AS sumsales#129] - -(63) HashAggregate [codegen id : 40] -Input [3]: [i_category#21, i_class#20, sumsales#129] -Keys [2]: [i_category#21, i_class#20] -Functions [1]: [partial_sum(sumsales#129)] -Aggregate Attributes [2]: [sum#130, isEmpty#131] -Results [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] - -(64) CometColumnarExchange -Input [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] -Arguments: hashpartitioning(i_category#21, i_class#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(65) CometColumnarToRow [codegen id : 41] -Input [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] - -(66) HashAggregate [codegen id : 41] -Input [4]: [i_category#21, i_class#20, sum#132, isEmpty#133] -Keys [2]: [i_category#21, i_class#20] -Functions [1]: [sum(sumsales#129)] -Aggregate Attributes [1]: [sum(sumsales#129)#134] -Results [9]: [i_category#21, i_class#20, null AS i_brand#135, null AS i_product_name#136, null AS d_year#137, null AS d_qoy#138, null AS d_moy#139, null AS s_store_id#140, sum(sumsales#129)#134 AS sumsales#141] - -(67) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13, sum#145, isEmpty#146] - -(68) CometColumnarToRow [codegen id : 46] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13, sum#145, isEmpty#146] - -(69) HashAggregate [codegen id : 46] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13, sum#145, isEmpty#146] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#142, d_qoy#143, d_moy#144, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#147 * cast(ss_quantity#148 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#147 * cast(ss_quantity#148 as decimal(10,0))), 0.00))#27] -Results [2]: [i_category#21, sum(coalesce((ss_sales_price#147 * cast(ss_quantity#148 as decimal(10,0))), 0.00))#27 AS sumsales#149] - -(70) HashAggregate [codegen id : 46] -Input [2]: [i_category#21, sumsales#149] -Keys [1]: [i_category#21] -Functions [1]: [partial_sum(sumsales#149)] -Aggregate Attributes [2]: [sum#150, isEmpty#151] -Results [3]: [i_category#21, sum#152, isEmpty#153] - -(71) CometColumnarExchange -Input [3]: [i_category#21, sum#152, isEmpty#153] -Arguments: hashpartitioning(i_category#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(72) CometColumnarToRow [codegen id : 47] -Input [3]: [i_category#21, sum#152, isEmpty#153] - -(73) HashAggregate [codegen id : 47] -Input [3]: [i_category#21, sum#152, isEmpty#153] -Keys [1]: [i_category#21] -Functions [1]: [sum(sumsales#149)] -Aggregate Attributes [1]: [sum(sumsales#149)#154] -Results [9]: [i_category#21, null AS i_class#155, null AS i_brand#156, null AS i_product_name#157, null AS d_year#158, null AS d_qoy#159, null AS d_moy#160, null AS s_store_id#161, sum(sumsales#149)#154 AS sumsales#162] - -(74) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13, sum#166, isEmpty#167] - -(75) CometColumnarToRow [codegen id : 52] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13, sum#166, isEmpty#167] - -(76) HashAggregate [codegen id : 52] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13, sum#166, isEmpty#167] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#163, d_qoy#164, d_moy#165, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#168 * cast(ss_quantity#169 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#168 * cast(ss_quantity#169 as decimal(10,0))), 0.00))#27] -Results [1]: [sum(coalesce((ss_sales_price#168 * cast(ss_quantity#169 as decimal(10,0))), 0.00))#27 AS sumsales#170] - -(77) HashAggregate [codegen id : 52] -Input [1]: [sumsales#170] -Keys: [] -Functions [1]: [partial_sum(sumsales#170)] -Aggregate Attributes [2]: [sum#171, isEmpty#172] -Results [2]: [sum#173, isEmpty#174] - -(78) CometColumnarExchange -Input [2]: [sum#173, isEmpty#174] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(79) CometColumnarToRow [codegen id : 53] -Input [2]: [sum#173, isEmpty#174] - -(80) HashAggregate [codegen id : 53] -Input [2]: [sum#173, isEmpty#174] -Keys: [] -Functions [1]: [sum(sumsales#170)] -Aggregate Attributes [1]: [sum(sumsales#170)#175] -Results [9]: [null AS i_category#176, null AS i_class#177, null AS i_brand#178, null AS i_product_name#179, null AS d_year#180, null AS d_qoy#181, null AS d_moy#182, null AS s_store_id#183, sum(sumsales#170)#175 AS sumsales#184] - -(81) Union - -(82) Sort [codegen id : 54] -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: [i_category#28 ASC NULLS FIRST, sumsales#36 DESC NULLS LAST], false, 0 - -(83) WindowGroupLimit -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: [i_category#28], [sumsales#36 DESC NULLS LAST], rank(sumsales#36), 100, Partial - -(84) CometColumnarExchange -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: hashpartitioning(i_category#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(85) CometSort -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36], [i_category#28 ASC NULLS FIRST, sumsales#36 DESC NULLS LAST] - -(86) CometColumnarToRow [codegen id : 55] -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] - -(87) WindowGroupLimit -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: [i_category#28], [sumsales#36 DESC NULLS LAST], rank(sumsales#36), 100, Final - -(88) Window -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: [rank(sumsales#36) windowspecdefinition(i_category#28, sumsales#36 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#185], [i_category#28], [sumsales#36 DESC NULLS LAST] - -(89) Filter [codegen id : 56] -Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] -Condition : (rk#185 <= 100) - -(90) TakeOrderedAndProject -Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] -Arguments: 100, [i_category#28 ASC NULLS FIRST, i_class#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, i_product_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_qoy#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, s_store_id#35 ASC NULLS FIRST, sumsales#36 ASC NULLS FIRST, rk#185 ASC NULLS FIRST], [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#185] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (95) -+- * CometColumnarToRow (94) - +- CometProject (93) - +- CometFilter (92) - +- CometNativeScan parquet spark_catalog.default.date_dim (91) - - -(91) CometNativeScan parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#186, d_year#8, d_moy#9, d_qoy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(92) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#186, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#186) AND (d_month_seq#186 >= 1212)) AND (d_month_seq#186 <= 1223)) AND isnotnull(d_date_sk#7)) - -(93) CometProject -Input [5]: [d_date_sk#7, d_month_seq#186, d_year#8, d_moy#9, d_qoy#10] -Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(94) CometColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(95) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/simplified.txt deleted file mode 100644 index 38fa0be072..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/simplified.txt +++ /dev/null @@ -1,147 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (56) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (55) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (54) - Sort [i_category,sumsales] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - WholeStageCodegen (11) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 - WholeStageCodegen (10) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (17) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 - WholeStageCodegen (16) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (23) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year] #8 - WholeStageCodegen (22) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (29) - HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name] #9 - WholeStageCodegen (28) - HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (35) - HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand] #10 - WholeStageCodegen (34) - HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (41) - HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #11 - WholeStageCodegen (40) - HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (47) - HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #12 - WholeStageCodegen (46) - HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (53) - HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #13 - WholeStageCodegen (52) - HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/explain.txt deleted file mode 100644 index 46f670a0bd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,445 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (74) -+- * Filter (73) - +- Window (72) - +- WindowGroupLimit (71) - +- * CometColumnarToRow (70) - +- CometSort (69) - +- CometColumnarExchange (68) - +- WindowGroupLimit (67) - +- * CometColumnarToRow (66) - +- CometSort (65) - +- CometUnion (64) - :- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometHashAggregate (25) - : +- ReusedExchange (24) - :- CometHashAggregate (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- CometHashAggregate (30) - : +- ReusedExchange (29) - :- CometHashAggregate (38) - : +- CometExchange (37) - : +- CometHashAggregate (36) - : +- CometHashAggregate (35) - : +- ReusedExchange (34) - :- CometHashAggregate (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometHashAggregate (40) - : +- ReusedExchange (39) - :- CometHashAggregate (48) - : +- CometExchange (47) - : +- CometHashAggregate (46) - : +- CometHashAggregate (45) - : +- ReusedExchange (44) - :- CometHashAggregate (53) - : +- CometExchange (52) - : +- CometHashAggregate (51) - : +- CometHashAggregate (50) - : +- ReusedExchange (49) - :- CometHashAggregate (58) - : +- CometExchange (57) - : +- CometHashAggregate (56) - : +- CometHashAggregate (55) - : +- ReusedExchange (54) - +- CometHashAggregate (63) - +- CometExchange (62) - +- CometHashAggregate (61) - +- CometHashAggregate (60) - +- ReusedExchange (59) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(6) CometBroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Right output [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11], [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_store_id#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#12, s_store_id#13] -Condition : isnotnull(s_store_sk#12) - -(11) CometProject -Input [2]: [s_store_sk#12, s_store_id#13] -Arguments: [s_store_sk#12, s_store_id#14], [s_store_sk#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#13, 16, true, false, true) AS s_store_id#14] - -(12) CometBroadcastExchange -Input [2]: [s_store_sk#12, s_store_id#14] -Arguments: [s_store_sk#12, s_store_id#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] -Right output [2]: [s_store_sk#12, s_store_id#14] -Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_sk#12, s_store_id#14] -Arguments: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14], [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Condition : isnotnull(i_item_sk#15) - -(17) CometProject -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#16, 50, true, false, true) AS i_brand#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#17, 50, true, false, true) AS i_class#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true) AS i_category#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#19, 50, true, false, true) AS i_product_name#23] - -(18) CometBroadcastExchange -Input [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] -Right output [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight - -(20) CometProject -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23], [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23] - -(21) CometHashAggregate -Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(22) CometExchange -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#24, isEmpty#25] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#24, isEmpty#25] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(24) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14, sum#29, isEmpty#30] - -(25) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14, sum#29, isEmpty#30] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#31 * cast(ss_quantity#32 as decimal(10,0))), 0.00))] - -(26) CometHashAggregate -Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sumsales#33] -Keys [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28] -Functions [1]: [partial_sum(sumsales#33)] - -(27) CometExchange -Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sum#34, isEmpty#35] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sum#34, isEmpty#35] -Keys [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28] -Functions [1]: [sum(sumsales#33)] - -(29) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14, sum#39, isEmpty#40] - -(30) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14, sum#39, isEmpty#40] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#41 * cast(ss_quantity#42 as decimal(10,0))), 0.00))] - -(31) CometHashAggregate -Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sumsales#43] -Keys [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37] -Functions [1]: [partial_sum(sumsales#43)] - -(32) CometExchange -Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometHashAggregate -Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sum#44, isEmpty#45] -Keys [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37] -Functions [1]: [sum(sumsales#43)] - -(34) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#47, d_moy#48, s_store_id#14, sum#49, isEmpty#50] - -(35) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#47, d_moy#48, s_store_id#14, sum#49, isEmpty#50] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#47, d_moy#48, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#51 * cast(ss_quantity#52 as decimal(10,0))), 0.00))] - -(36) CometHashAggregate -Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, sumsales#53] -Keys [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46] -Functions [1]: [partial_sum(sumsales#53)] - -(37) CometExchange -Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, sum#54, isEmpty#55] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(38) CometHashAggregate -Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, sum#54, isEmpty#55] -Keys [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46] -Functions [1]: [sum(sumsales#53)] - -(39) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#56, d_qoy#57, d_moy#58, s_store_id#14, sum#59, isEmpty#60] - -(40) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#56, d_qoy#57, d_moy#58, s_store_id#14, sum#59, isEmpty#60] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#56, d_qoy#57, d_moy#58, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#61 * cast(ss_quantity#62 as decimal(10,0))), 0.00))] - -(41) CometHashAggregate -Input [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sumsales#63] -Keys [4]: [i_category#22, i_class#21, i_brand#20, i_product_name#23] -Functions [1]: [partial_sum(sumsales#63)] - -(42) CometExchange -Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sum#64, isEmpty#65] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(43) CometHashAggregate -Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sum#64, isEmpty#65] -Keys [4]: [i_category#22, i_class#21, i_brand#20, i_product_name#23] -Functions [1]: [sum(sumsales#63)] - -(44) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#66, d_qoy#67, d_moy#68, s_store_id#14, sum#69, isEmpty#70] - -(45) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#66, d_qoy#67, d_moy#68, s_store_id#14, sum#69, isEmpty#70] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#66, d_qoy#67, d_moy#68, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#71 * cast(ss_quantity#72 as decimal(10,0))), 0.00))] - -(46) CometHashAggregate -Input [4]: [i_category#22, i_class#21, i_brand#20, sumsales#73] -Keys [3]: [i_category#22, i_class#21, i_brand#20] -Functions [1]: [partial_sum(sumsales#73)] - -(47) CometExchange -Input [5]: [i_category#22, i_class#21, i_brand#20, sum#74, isEmpty#75] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(48) CometHashAggregate -Input [5]: [i_category#22, i_class#21, i_brand#20, sum#74, isEmpty#75] -Keys [3]: [i_category#22, i_class#21, i_brand#20] -Functions [1]: [sum(sumsales#73)] - -(49) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#76, d_qoy#77, d_moy#78, s_store_id#14, sum#79, isEmpty#80] - -(50) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#76, d_qoy#77, d_moy#78, s_store_id#14, sum#79, isEmpty#80] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#76, d_qoy#77, d_moy#78, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#81 * cast(ss_quantity#82 as decimal(10,0))), 0.00))] - -(51) CometHashAggregate -Input [3]: [i_category#22, i_class#21, sumsales#83] -Keys [2]: [i_category#22, i_class#21] -Functions [1]: [partial_sum(sumsales#83)] - -(52) CometExchange -Input [4]: [i_category#22, i_class#21, sum#84, isEmpty#85] -Arguments: hashpartitioning(i_category#22, i_class#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(53) CometHashAggregate -Input [4]: [i_category#22, i_class#21, sum#84, isEmpty#85] -Keys [2]: [i_category#22, i_class#21] -Functions [1]: [sum(sumsales#83)] - -(54) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#86, d_qoy#87, d_moy#88, s_store_id#14, sum#89, isEmpty#90] - -(55) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#86, d_qoy#87, d_moy#88, s_store_id#14, sum#89, isEmpty#90] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#86, d_qoy#87, d_moy#88, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#91 * cast(ss_quantity#92 as decimal(10,0))), 0.00))] - -(56) CometHashAggregate -Input [2]: [i_category#22, sumsales#93] -Keys [1]: [i_category#22] -Functions [1]: [partial_sum(sumsales#93)] - -(57) CometExchange -Input [3]: [i_category#22, sum#94, isEmpty#95] -Arguments: hashpartitioning(i_category#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(58) CometHashAggregate -Input [3]: [i_category#22, sum#94, isEmpty#95] -Keys [1]: [i_category#22] -Functions [1]: [sum(sumsales#93)] - -(59) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#96, d_qoy#97, d_moy#98, s_store_id#14, sum#99, isEmpty#100] - -(60) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#96, d_qoy#97, d_moy#98, s_store_id#14, sum#99, isEmpty#100] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#96, d_qoy#97, d_moy#98, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#101 * cast(ss_quantity#102 as decimal(10,0))), 0.00))] - -(61) CometHashAggregate -Input [1]: [sumsales#103] -Keys: [] -Functions [1]: [partial_sum(sumsales#103)] - -(62) CometExchange -Input [2]: [sum#104, isEmpty#105] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(63) CometHashAggregate -Input [2]: [sum#104, isEmpty#105] -Keys: [] -Functions [1]: [sum(sumsales#103)] - -(64) CometUnion -Child 0 Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Child 1 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#115, sumsales#116] -Child 2 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#117, s_store_id#118, sumsales#119] -Child 3 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#120, d_moy#121, s_store_id#122, sumsales#123] -Child 4 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#124, d_qoy#125, d_moy#126, s_store_id#127, sumsales#128] -Child 5 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#129, d_year#130, d_qoy#131, d_moy#132, s_store_id#133, sumsales#134] -Child 6 Input [9]: [i_category#22, i_class#21, i_brand#135, i_product_name#136, d_year#137, d_qoy#138, d_moy#139, s_store_id#140, sumsales#141] -Child 7 Input [9]: [i_category#22, i_class#142, i_brand#143, i_product_name#144, d_year#145, d_qoy#146, d_moy#147, s_store_id#148, sumsales#149] -Child 8 Input [9]: [i_category#150, i_class#151, i_brand#152, i_product_name#153, d_year#154, d_qoy#155, d_moy#156, s_store_id#157, sumsales#158] - -(65) CometSort -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114], [i_category#106 ASC NULLS FIRST, sumsales#114 DESC NULLS LAST] - -(66) CometColumnarToRow [codegen id : 1] -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] - -(67) WindowGroupLimit -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: [i_category#106], [sumsales#114 DESC NULLS LAST], rank(sumsales#114), 100, Partial - -(68) CometColumnarExchange -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: hashpartitioning(i_category#106, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(69) CometSort -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114], [i_category#106 ASC NULLS FIRST, sumsales#114 DESC NULLS LAST] - -(70) CometColumnarToRow [codegen id : 2] -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] - -(71) WindowGroupLimit -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: [i_category#106], [sumsales#114 DESC NULLS LAST], rank(sumsales#114), 100, Final - -(72) Window -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: [rank(sumsales#114) windowspecdefinition(i_category#106, sumsales#114 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#159], [i_category#106], [sumsales#114 DESC NULLS LAST] - -(73) Filter [codegen id : 3] -Input [10]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114, rk#159] -Condition : (rk#159 <= 100) - -(74) TakeOrderedAndProject -Input [10]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114, rk#159] -Arguments: 100, [i_category#106 ASC NULLS FIRST, i_class#107 ASC NULLS FIRST, i_brand#108 ASC NULLS FIRST, i_product_name#109 ASC NULLS FIRST, d_year#110 ASC NULLS FIRST, d_qoy#111 ASC NULLS FIRST, d_moy#112 ASC NULLS FIRST, s_store_id#113 ASC NULLS FIRST, sumsales#114 ASC NULLS FIRST, rk#159 ASC NULLS FIRST], [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114, rk#159] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (79) -+- * CometColumnarToRow (78) - +- CometProject (77) - +- CometFilter (76) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (75) - - -(75) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(76) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) - -(77) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(78) CometColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(79) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 6096ee1c25..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,88 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (3) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometUnion [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] - 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,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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] - CometHashAggregate [s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] - CometHashAggregate [d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year] #9 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] - CometHashAggregate [d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name] #10 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,sum,isEmpty] - CometHashAggregate [d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand] #11 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,sum,isEmpty] - CometHashAggregate [i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class] #12 - CometHashAggregate [sumsales] [i_category,i_class,sum,isEmpty] - CometHashAggregate [i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category] #13 - CometHashAggregate [sumsales] [i_category,sum,isEmpty] - CometHashAggregate [i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange #14 - CometHashAggregate [sumsales] [sum,isEmpty] - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt deleted file mode 100644 index 46f670a0bd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/explain.txt +++ /dev/null @@ -1,445 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (74) -+- * Filter (73) - +- Window (72) - +- WindowGroupLimit (71) - +- * CometColumnarToRow (70) - +- CometSort (69) - +- CometColumnarExchange (68) - +- WindowGroupLimit (67) - +- * CometColumnarToRow (66) - +- CometSort (65) - +- CometUnion (64) - :- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometHashAggregate (25) - : +- ReusedExchange (24) - :- CometHashAggregate (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- CometHashAggregate (30) - : +- ReusedExchange (29) - :- CometHashAggregate (38) - : +- CometExchange (37) - : +- CometHashAggregate (36) - : +- CometHashAggregate (35) - : +- ReusedExchange (34) - :- CometHashAggregate (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometHashAggregate (40) - : +- ReusedExchange (39) - :- CometHashAggregate (48) - : +- CometExchange (47) - : +- CometHashAggregate (46) - : +- CometHashAggregate (45) - : +- ReusedExchange (44) - :- CometHashAggregate (53) - : +- CometExchange (52) - : +- CometHashAggregate (51) - : +- CometHashAggregate (50) - : +- ReusedExchange (49) - :- CometHashAggregate (58) - : +- CometExchange (57) - : +- CometHashAggregate (56) - : +- CometHashAggregate (55) - : +- ReusedExchange (54) - +- CometHashAggregate (63) - +- CometExchange (62) - +- CometHashAggregate (61) - +- CometHashAggregate (60) - +- ReusedExchange (59) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(6) CometBroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Right output [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11], [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_store_id#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#12, s_store_id#13] -Condition : isnotnull(s_store_sk#12) - -(11) CometProject -Input [2]: [s_store_sk#12, s_store_id#13] -Arguments: [s_store_sk#12, s_store_id#14], [s_store_sk#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#13, 16, true, false, true) AS s_store_id#14] - -(12) CometBroadcastExchange -Input [2]: [s_store_sk#12, s_store_id#14] -Arguments: [s_store_sk#12, s_store_id#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] -Right output [2]: [s_store_sk#12, s_store_id#14] -Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_sk#12, s_store_id#14] -Arguments: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14], [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Condition : isnotnull(i_item_sk#15) - -(17) CometProject -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#16, 50, true, false, true) AS i_brand#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#17, 50, true, false, true) AS i_class#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true) AS i_category#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#19, 50, true, false, true) AS i_product_name#23] - -(18) CometBroadcastExchange -Input [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] -Right output [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight - -(20) CometProject -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23], [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23] - -(21) CometHashAggregate -Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(22) CometExchange -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#24, isEmpty#25] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#24, isEmpty#25] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(24) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14, sum#29, isEmpty#30] - -(25) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14, sum#29, isEmpty#30] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#31 * cast(ss_quantity#32 as decimal(10,0))), 0.00))] - -(26) CometHashAggregate -Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sumsales#33] -Keys [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28] -Functions [1]: [partial_sum(sumsales#33)] - -(27) CometExchange -Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sum#34, isEmpty#35] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sum#34, isEmpty#35] -Keys [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28] -Functions [1]: [sum(sumsales#33)] - -(29) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14, sum#39, isEmpty#40] - -(30) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14, sum#39, isEmpty#40] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#41 * cast(ss_quantity#42 as decimal(10,0))), 0.00))] - -(31) CometHashAggregate -Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sumsales#43] -Keys [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37] -Functions [1]: [partial_sum(sumsales#43)] - -(32) CometExchange -Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometHashAggregate -Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sum#44, isEmpty#45] -Keys [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37] -Functions [1]: [sum(sumsales#43)] - -(34) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#47, d_moy#48, s_store_id#14, sum#49, isEmpty#50] - -(35) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#47, d_moy#48, s_store_id#14, sum#49, isEmpty#50] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#47, d_moy#48, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#51 * cast(ss_quantity#52 as decimal(10,0))), 0.00))] - -(36) CometHashAggregate -Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, sumsales#53] -Keys [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46] -Functions [1]: [partial_sum(sumsales#53)] - -(37) CometExchange -Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, sum#54, isEmpty#55] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(38) CometHashAggregate -Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, sum#54, isEmpty#55] -Keys [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46] -Functions [1]: [sum(sumsales#53)] - -(39) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#56, d_qoy#57, d_moy#58, s_store_id#14, sum#59, isEmpty#60] - -(40) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#56, d_qoy#57, d_moy#58, s_store_id#14, sum#59, isEmpty#60] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#56, d_qoy#57, d_moy#58, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#61 * cast(ss_quantity#62 as decimal(10,0))), 0.00))] - -(41) CometHashAggregate -Input [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sumsales#63] -Keys [4]: [i_category#22, i_class#21, i_brand#20, i_product_name#23] -Functions [1]: [partial_sum(sumsales#63)] - -(42) CometExchange -Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sum#64, isEmpty#65] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(43) CometHashAggregate -Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sum#64, isEmpty#65] -Keys [4]: [i_category#22, i_class#21, i_brand#20, i_product_name#23] -Functions [1]: [sum(sumsales#63)] - -(44) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#66, d_qoy#67, d_moy#68, s_store_id#14, sum#69, isEmpty#70] - -(45) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#66, d_qoy#67, d_moy#68, s_store_id#14, sum#69, isEmpty#70] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#66, d_qoy#67, d_moy#68, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#71 * cast(ss_quantity#72 as decimal(10,0))), 0.00))] - -(46) CometHashAggregate -Input [4]: [i_category#22, i_class#21, i_brand#20, sumsales#73] -Keys [3]: [i_category#22, i_class#21, i_brand#20] -Functions [1]: [partial_sum(sumsales#73)] - -(47) CometExchange -Input [5]: [i_category#22, i_class#21, i_brand#20, sum#74, isEmpty#75] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(48) CometHashAggregate -Input [5]: [i_category#22, i_class#21, i_brand#20, sum#74, isEmpty#75] -Keys [3]: [i_category#22, i_class#21, i_brand#20] -Functions [1]: [sum(sumsales#73)] - -(49) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#76, d_qoy#77, d_moy#78, s_store_id#14, sum#79, isEmpty#80] - -(50) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#76, d_qoy#77, d_moy#78, s_store_id#14, sum#79, isEmpty#80] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#76, d_qoy#77, d_moy#78, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#81 * cast(ss_quantity#82 as decimal(10,0))), 0.00))] - -(51) CometHashAggregate -Input [3]: [i_category#22, i_class#21, sumsales#83] -Keys [2]: [i_category#22, i_class#21] -Functions [1]: [partial_sum(sumsales#83)] - -(52) CometExchange -Input [4]: [i_category#22, i_class#21, sum#84, isEmpty#85] -Arguments: hashpartitioning(i_category#22, i_class#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(53) CometHashAggregate -Input [4]: [i_category#22, i_class#21, sum#84, isEmpty#85] -Keys [2]: [i_category#22, i_class#21] -Functions [1]: [sum(sumsales#83)] - -(54) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#86, d_qoy#87, d_moy#88, s_store_id#14, sum#89, isEmpty#90] - -(55) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#86, d_qoy#87, d_moy#88, s_store_id#14, sum#89, isEmpty#90] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#86, d_qoy#87, d_moy#88, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#91 * cast(ss_quantity#92 as decimal(10,0))), 0.00))] - -(56) CometHashAggregate -Input [2]: [i_category#22, sumsales#93] -Keys [1]: [i_category#22] -Functions [1]: [partial_sum(sumsales#93)] - -(57) CometExchange -Input [3]: [i_category#22, sum#94, isEmpty#95] -Arguments: hashpartitioning(i_category#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(58) CometHashAggregate -Input [3]: [i_category#22, sum#94, isEmpty#95] -Keys [1]: [i_category#22] -Functions [1]: [sum(sumsales#93)] - -(59) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#96, d_qoy#97, d_moy#98, s_store_id#14, sum#99, isEmpty#100] - -(60) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#96, d_qoy#97, d_moy#98, s_store_id#14, sum#99, isEmpty#100] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#96, d_qoy#97, d_moy#98, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#101 * cast(ss_quantity#102 as decimal(10,0))), 0.00))] - -(61) CometHashAggregate -Input [1]: [sumsales#103] -Keys: [] -Functions [1]: [partial_sum(sumsales#103)] - -(62) CometExchange -Input [2]: [sum#104, isEmpty#105] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(63) CometHashAggregate -Input [2]: [sum#104, isEmpty#105] -Keys: [] -Functions [1]: [sum(sumsales#103)] - -(64) CometUnion -Child 0 Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Child 1 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#115, sumsales#116] -Child 2 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#117, s_store_id#118, sumsales#119] -Child 3 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#46, d_qoy#120, d_moy#121, s_store_id#122, sumsales#123] -Child 4 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#124, d_qoy#125, d_moy#126, s_store_id#127, sumsales#128] -Child 5 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#129, d_year#130, d_qoy#131, d_moy#132, s_store_id#133, sumsales#134] -Child 6 Input [9]: [i_category#22, i_class#21, i_brand#135, i_product_name#136, d_year#137, d_qoy#138, d_moy#139, s_store_id#140, sumsales#141] -Child 7 Input [9]: [i_category#22, i_class#142, i_brand#143, i_product_name#144, d_year#145, d_qoy#146, d_moy#147, s_store_id#148, sumsales#149] -Child 8 Input [9]: [i_category#150, i_class#151, i_brand#152, i_product_name#153, d_year#154, d_qoy#155, d_moy#156, s_store_id#157, sumsales#158] - -(65) CometSort -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114], [i_category#106 ASC NULLS FIRST, sumsales#114 DESC NULLS LAST] - -(66) CometColumnarToRow [codegen id : 1] -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] - -(67) WindowGroupLimit -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: [i_category#106], [sumsales#114 DESC NULLS LAST], rank(sumsales#114), 100, Partial - -(68) CometColumnarExchange -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: hashpartitioning(i_category#106, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(69) CometSort -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114], [i_category#106 ASC NULLS FIRST, sumsales#114 DESC NULLS LAST] - -(70) CometColumnarToRow [codegen id : 2] -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] - -(71) WindowGroupLimit -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: [i_category#106], [sumsales#114 DESC NULLS LAST], rank(sumsales#114), 100, Final - -(72) Window -Input [9]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114] -Arguments: [rank(sumsales#114) windowspecdefinition(i_category#106, sumsales#114 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#159], [i_category#106], [sumsales#114 DESC NULLS LAST] - -(73) Filter [codegen id : 3] -Input [10]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114, rk#159] -Condition : (rk#159 <= 100) - -(74) TakeOrderedAndProject -Input [10]: [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114, rk#159] -Arguments: 100, [i_category#106 ASC NULLS FIRST, i_class#107 ASC NULLS FIRST, i_brand#108 ASC NULLS FIRST, i_product_name#109 ASC NULLS FIRST, d_year#110 ASC NULLS FIRST, d_qoy#111 ASC NULLS FIRST, d_moy#112 ASC NULLS FIRST, s_store_id#113 ASC NULLS FIRST, sumsales#114 ASC NULLS FIRST, rk#159 ASC NULLS FIRST], [i_category#106, i_class#107, i_brand#108, i_product_name#109, d_year#110, d_qoy#111, d_moy#112, s_store_id#113, sumsales#114, rk#159] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (79) -+- * CometColumnarToRow (78) - +- CometProject (77) - +- CometFilter (76) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (75) - - -(75) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(76) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) - -(77) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(78) CometColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(79) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/extended.txt deleted file mode 100644 index b96ce25ae1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/extended.txt +++ /dev/null @@ -1,298 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 262 out of 285 eligible operators (91%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 6096ee1c25..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/simplified.txt +++ /dev/null @@ -1,88 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (3) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometUnion [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] - 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,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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] - CometHashAggregate [s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] - CometHashAggregate [d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year] #9 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] - CometHashAggregate [d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name] #10 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,sum,isEmpty] - CometHashAggregate [d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand] #11 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,sum,isEmpty] - CometHashAggregate [i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class] #12 - CometHashAggregate [sumsales] [i_category,i_class,sum,isEmpty] - CometHashAggregate [i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category] #13 - CometHashAggregate [sumsales] [i_category,sum,isEmpty] - CometHashAggregate [i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange #14 - CometHashAggregate [sumsales] [sum,isEmpty] - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/explain.txt deleted file mode 100644 index 603308d760..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/explain.txt +++ /dev/null @@ -1,404 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (65) -+- * Project (64) - +- Window (63) - +- * CometColumnarToRow (62) - +- CometSort (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometColumnarExchange (58) - +- * HashAggregate (57) - +- Union (56) - :- * HashAggregate (41) - : +- * CometColumnarToRow (40) - : +- CometColumnarExchange (39) - : +- * HashAggregate (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- * BroadcastHashJoin LeftSemi BuildRight (33) - : :- * CometColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (32) - : +- * Project (31) - : +- * Filter (30) - : +- Window (29) - : +- WindowGroupLimit (28) - : +- * Sort (27) - : +- * HashAggregate (26) - : +- * CometColumnarToRow (25) - : +- CometColumnarExchange (24) - : +- * HashAggregate (23) - : +- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet spark_catalog.default.store_sales (10) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometProject (15) - : : +- CometFilter (14) - : : +- CometNativeScan parquet spark_catalog.default.store (13) - : +- ReusedExchange (20) - :- * HashAggregate (48) - : +- * CometColumnarToRow (47) - : +- CometColumnarExchange (46) - : +- * HashAggregate (45) - : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- ReusedExchange (42) - +- * HashAggregate (55) - +- * CometColumnarToRow (54) - +- CometColumnarExchange (53) - +- * HashAggregate (52) - +- * HashAggregate (51) - +- * CometColumnarToRow (50) - +- ReusedExchange (49) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 8] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] - -(3) Filter [codegen id : 8] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(4) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#5] - -(5) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 8] -Output [2]: [ss_store_sk#1, ss_net_profit#2] -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#6, s_county#7, s_state#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Condition : isnotnull(s_store_sk#6) - -(9) CometColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#6, s_county#7, s_state#8] - -(10) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] - -(12) Filter [codegen id : 4] -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_store_sk#9) - -(13) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_state#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [s_store_sk#12, s_state#13] -Condition : isnotnull(s_store_sk#12) - -(15) CometProject -Input [2]: [s_store_sk#12, s_state#13] -Arguments: [s_store_sk#12, s_state#14], [s_store_sk#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#14] - -(16) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#12, s_state#14] - -(17) BroadcastExchange -Input [2]: [s_store_sk#12, s_state#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#9] -Right keys [1]: [s_store_sk#12] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14] -Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#12, s_state#14] - -(20) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#15] - -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 4] -Output [2]: [ss_net_profit#10, s_state#14] -Input [4]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14, d_date_sk#15] - -(23) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#10, s_state#14] -Keys [1]: [s_state#14] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum#16] -Results [2]: [s_state#14, sum#17] - -(24) CometColumnarExchange -Input [2]: [s_state#14, sum#17] -Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(25) CometColumnarToRow [codegen id : 5] -Input [2]: [s_state#14, sum#17] - -(26) HashAggregate [codegen id : 5] -Input [2]: [s_state#14, sum#17] -Keys [1]: [s_state#14] -Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#18] -Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#18,17,2) AS _w0#19, s_state#14] - -(27) Sort [codegen id : 5] -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 - -(28) WindowGroupLimit -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [s_state#14], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final - -(29) Window -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#14, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#14], [_w0#19 DESC NULLS LAST] - -(30) Filter [codegen id : 6] -Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] -Condition : (ranking#20 <= 5) - -(31) Project [codegen id : 6] -Output [1]: [s_state#14] -Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] - -(32) BroadcastExchange -Input [1]: [s_state#14] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(33) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true)] -Right keys [1]: [s_state#14] -Join type: LeftSemi -Join condition: None - -(34) Project [codegen id : 7] -Output [3]: [s_store_sk#6, s_county#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true) AS s_state#21] -Input [3]: [s_store_sk#6, s_county#7, s_state#8] - -(35) BroadcastExchange -Input [3]: [s_store_sk#6, s_county#7, s_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(36) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#6] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 8] -Output [3]: [ss_net_profit#2, s_county#7, s_state#21] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#21] - -(38) HashAggregate [codegen id : 8] -Input [3]: [ss_net_profit#2, s_county#7, s_state#21] -Keys [2]: [s_state#21, s_county#7] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [s_state#21, s_county#7, sum#23] - -(39) CometColumnarExchange -Input [3]: [s_state#21, s_county#7, sum#23] -Arguments: hashpartitioning(s_state#21, s_county#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(40) CometColumnarToRow [codegen id : 9] -Input [3]: [s_state#21, s_county#7, sum#23] - -(41) HashAggregate [codegen id : 9] -Input [3]: [s_state#21, s_county#7, sum#23] -Keys [2]: [s_state#21, s_county#7] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) as decimal(27,2)) AS total_sum#25, s_state#21 AS s_state#26, s_county#7 AS s_county#27, 0 AS g_state#28, 0 AS g_county#29, 0 AS lochierarchy#30] - -(42) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_state#21, s_county#31, sum#32] - -(43) CometColumnarToRow [codegen id : 18] -Input [3]: [s_state#21, s_county#31, sum#32] - -(44) HashAggregate [codegen id : 18] -Input [3]: [s_state#21, s_county#31, sum#32] -Keys [2]: [s_state#21, s_county#31] -Functions [1]: [sum(UnscaledValue(ss_net_profit#33))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#33))#24] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#33))#24,17,2) AS total_sum#34, s_state#21] - -(45) HashAggregate [codegen id : 18] -Input [2]: [total_sum#34, s_state#21] -Keys [1]: [s_state#21] -Functions [1]: [partial_sum(total_sum#34)] -Aggregate Attributes [2]: [sum#35, isEmpty#36] -Results [3]: [s_state#21, sum#37, isEmpty#38] - -(46) CometColumnarExchange -Input [3]: [s_state#21, sum#37, isEmpty#38] -Arguments: hashpartitioning(s_state#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(47) CometColumnarToRow [codegen id : 19] -Input [3]: [s_state#21, sum#37, isEmpty#38] - -(48) HashAggregate [codegen id : 19] -Input [3]: [s_state#21, sum#37, isEmpty#38] -Keys [1]: [s_state#21] -Functions [1]: [sum(total_sum#34)] -Aggregate Attributes [1]: [sum(total_sum#34)#39] -Results [6]: [sum(total_sum#34)#39 AS total_sum#40, s_state#21, null AS s_county#41, 0 AS g_state#42, 1 AS g_county#43, 1 AS lochierarchy#44] - -(49) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_state#21, s_county#45, sum#46] - -(50) CometColumnarToRow [codegen id : 28] -Input [3]: [s_state#21, s_county#45, sum#46] - -(51) HashAggregate [codegen id : 28] -Input [3]: [s_state#21, s_county#45, sum#46] -Keys [2]: [s_state#21, s_county#45] -Functions [1]: [sum(UnscaledValue(ss_net_profit#47))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#47))#24] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#47))#24,17,2) AS total_sum#48] - -(52) HashAggregate [codegen id : 28] -Input [1]: [total_sum#48] -Keys: [] -Functions [1]: [partial_sum(total_sum#48)] -Aggregate Attributes [2]: [sum#49, isEmpty#50] -Results [2]: [sum#51, isEmpty#52] - -(53) CometColumnarExchange -Input [2]: [sum#51, isEmpty#52] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(54) CometColumnarToRow [codegen id : 29] -Input [2]: [sum#51, isEmpty#52] - -(55) HashAggregate [codegen id : 29] -Input [2]: [sum#51, isEmpty#52] -Keys: [] -Functions [1]: [sum(total_sum#48)] -Aggregate Attributes [1]: [sum(total_sum#48)#53] -Results [6]: [sum(total_sum#48)#53 AS total_sum#54, null AS s_state#55, null AS s_county#56, 1 AS g_state#57, 1 AS g_county#58, 2 AS lochierarchy#59] - -(56) Union - -(57) HashAggregate [codegen id : 30] -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Functions: [] -Aggregate Attributes: [] -Results [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] - -(58) CometColumnarExchange -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Arguments: hashpartitioning(total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(59) CometHashAggregate -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Functions: [] - -(60) CometExchange -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] -Arguments: hashpartitioning(lochierarchy#30, _w0#60, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(61) CometSort -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] -Arguments: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60], [lochierarchy#30 ASC NULLS FIRST, _w0#60 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] - -(62) CometColumnarToRow [codegen id : 31] -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] - -(63) Window -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] -Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#30, _w0#60, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#61], [lochierarchy#30, _w0#60], [total_sum#25 DESC NULLS LAST] - -(64) Project [codegen id : 32] -Output [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#61] -Input [6]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60, rank_within_parent#61] - -(65) TakeOrderedAndProject -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#61] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#61 ASC NULLS FIRST], [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#61] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (70) -+- * CometColumnarToRow (69) - +- CometProject (68) - +- CometFilter (67) - +- CometNativeScan parquet spark_catalog.default.date_dim (66) - - -(66) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(67) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#62] -Condition : (((isnotnull(d_month_seq#62) AND (d_month_seq#62 >= 1212)) AND (d_month_seq#62 <= 1223)) AND isnotnull(d_date_sk#5)) - -(68) CometProject -Input [2]: [d_date_sk#5, d_month_seq#62] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(69) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(70) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/simplified.txt deleted file mode 100644 index ba40c1d591..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/simplified.txt +++ /dev/null @@ -1,107 +0,0 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (32) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (31) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] - CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (30) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (9) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - WholeStageCodegen (8) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (5) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #7 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (19) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #9 - WholeStageCodegen (18) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (29) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (28) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/explain.txt deleted file mode 100644 index af71da5f87..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,408 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (66) -+- * Project (65) - +- Window (64) - +- * CometColumnarToRow (63) - +- CometSort (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometColumnarExchange (59) - +- * HashAggregate (58) - +- Union (57) - :- * HashAggregate (42) - : +- * CometColumnarToRow (41) - : +- CometColumnarExchange (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * CometColumnarToRow (9) - : : +- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- BroadcastExchange (36) - : +- * Project (35) - : +- * BroadcastHashJoin LeftSemi BuildRight (34) - : :- * CometColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) - : +- BroadcastExchange (33) - : +- * Project (32) - : +- * Filter (31) - : +- Window (30) - : +- WindowGroupLimit (29) - : +- * CometColumnarToRow (28) - : +- CometSort (27) - : +- CometHashAggregate (26) - : +- CometExchange (25) - : +- CometHashAggregate (24) - : +- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (20) - : : +- CometBroadcastHashJoin (19) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - : : +- CometBroadcastExchange (18) - : : +- CometProject (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - : +- ReusedExchange (21) - :- * HashAggregate (49) - : +- * CometColumnarToRow (48) - : +- CometColumnarExchange (47) - : +- * HashAggregate (46) - : +- * HashAggregate (45) - : +- * CometColumnarToRow (44) - : +- ReusedExchange (43) - +- * HashAggregate (56) - +- * CometColumnarToRow (55) - +- CometColumnarExchange (54) - +- * HashAggregate (53) - +- * HashAggregate (52) - +- * CometColumnarToRow (51) - +- ReusedExchange (50) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] - -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ss_store_sk#1, ss_net_profit#2] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#7, s_county#8, s_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Condition : isnotnull(s_store_sk#7) - -(12) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_store_sk#10) - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : isnotnull(s_store_sk#14) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14, s_state#16], [s_store_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) AS s_state#16] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#14, s_state#16] -Arguments: [s_store_sk#14, s_state#16] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Right output [2]: [s_store_sk#14, s_state#16] -Arguments: [ss_store_sk#10], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#16] -Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16], [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#17] - -(22) CometBroadcastHashJoin -Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] -Right output [1]: [d_date_sk#17] -Arguments: [ss_sold_date_sk#12], [d_date_sk#17], Inner, BuildRight - -(23) CometProject -Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16, d_date_sk#17] -Arguments: [ss_net_profit#11, s_state#16], [ss_net_profit#11, s_state#16] - -(24) CometHashAggregate -Input [2]: [ss_net_profit#11, s_state#16] -Keys [1]: [s_state#16] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] - -(25) CometExchange -Input [2]: [s_state#16, sum#18] -Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(26) CometHashAggregate -Input [2]: [s_state#16, sum#18] -Keys [1]: [s_state#16] -Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] - -(27) CometSort -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16, _w0#19, s_state#16], [s_state#16 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] - -(28) CometColumnarToRow [codegen id : 1] -Input [3]: [s_state#16, _w0#19, s_state#16] - -(29) WindowGroupLimit -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final - -(30) Window -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#16, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#16], [_w0#19 DESC NULLS LAST] - -(31) Filter [codegen id : 2] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] -Condition : (ranking#20 <= 5) - -(32) Project [codegen id : 2] -Output [1]: [s_state#16] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] - -(33) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] - -(34) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)] -Right keys [1]: [s_state#16] -Join type: LeftSemi -Join condition: None - -(35) Project [codegen id : 3] -Output [3]: [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#21] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(36) BroadcastExchange -Input [3]: [s_store_sk#7, s_county#8, s_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_county#8, s_state#21] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#21] - -(39) HashAggregate [codegen id : 4] -Input [3]: [ss_net_profit#2, s_county#8, s_state#21] -Keys [2]: [s_state#21, s_county#8] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [s_state#21, s_county#8, sum#23] - -(40) CometColumnarExchange -Input [3]: [s_state#21, s_county#8, sum#23] -Arguments: hashpartitioning(s_state#21, s_county#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 5] -Input [3]: [s_state#21, s_county#8, sum#23] - -(42) HashAggregate [codegen id : 5] -Input [3]: [s_state#21, s_county#8, sum#23] -Keys [2]: [s_state#21, s_county#8] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) as decimal(27,2)) AS total_sum#25, s_state#21 AS s_state#26, s_county#8 AS s_county#27, 0 AS g_state#28, 0 AS g_county#29, 0 AS lochierarchy#30] - -(43) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_state#21, s_county#31, sum#32] - -(44) CometColumnarToRow [codegen id : 10] -Input [3]: [s_state#21, s_county#31, sum#32] - -(45) HashAggregate [codegen id : 10] -Input [3]: [s_state#21, s_county#31, sum#32] -Keys [2]: [s_state#21, s_county#31] -Functions [1]: [sum(UnscaledValue(ss_net_profit#33))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#33))#24] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#33))#24,17,2) AS total_sum#34, s_state#21] - -(46) HashAggregate [codegen id : 10] -Input [2]: [total_sum#34, s_state#21] -Keys [1]: [s_state#21] -Functions [1]: [partial_sum(total_sum#34)] -Aggregate Attributes [2]: [sum#35, isEmpty#36] -Results [3]: [s_state#21, sum#37, isEmpty#38] - -(47) CometColumnarExchange -Input [3]: [s_state#21, sum#37, isEmpty#38] -Arguments: hashpartitioning(s_state#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(48) CometColumnarToRow [codegen id : 11] -Input [3]: [s_state#21, sum#37, isEmpty#38] - -(49) HashAggregate [codegen id : 11] -Input [3]: [s_state#21, sum#37, isEmpty#38] -Keys [1]: [s_state#21] -Functions [1]: [sum(total_sum#34)] -Aggregate Attributes [1]: [sum(total_sum#34)#39] -Results [6]: [sum(total_sum#34)#39 AS total_sum#40, s_state#21, null AS s_county#41, 0 AS g_state#42, 1 AS g_county#43, 1 AS lochierarchy#44] - -(50) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_state#21, s_county#45, sum#46] - -(51) CometColumnarToRow [codegen id : 16] -Input [3]: [s_state#21, s_county#45, sum#46] - -(52) HashAggregate [codegen id : 16] -Input [3]: [s_state#21, s_county#45, sum#46] -Keys [2]: [s_state#21, s_county#45] -Functions [1]: [sum(UnscaledValue(ss_net_profit#47))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#47))#24] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#47))#24,17,2) AS total_sum#48] - -(53) HashAggregate [codegen id : 16] -Input [1]: [total_sum#48] -Keys: [] -Functions [1]: [partial_sum(total_sum#48)] -Aggregate Attributes [2]: [sum#49, isEmpty#50] -Results [2]: [sum#51, isEmpty#52] - -(54) CometColumnarExchange -Input [2]: [sum#51, isEmpty#52] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(55) CometColumnarToRow [codegen id : 17] -Input [2]: [sum#51, isEmpty#52] - -(56) HashAggregate [codegen id : 17] -Input [2]: [sum#51, isEmpty#52] -Keys: [] -Functions [1]: [sum(total_sum#48)] -Aggregate Attributes [1]: [sum(total_sum#48)#53] -Results [6]: [sum(total_sum#48)#53 AS total_sum#54, null AS s_state#55, null AS s_county#56, 1 AS g_state#57, 1 AS g_county#58, 2 AS lochierarchy#59] - -(57) Union - -(58) HashAggregate [codegen id : 18] -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Functions: [] -Aggregate Attributes: [] -Results [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] - -(59) CometColumnarExchange -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Arguments: hashpartitioning(total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(60) CometHashAggregate -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Functions: [] - -(61) CometExchange -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] -Arguments: hashpartitioning(lochierarchy#30, _w0#60, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(62) CometSort -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] -Arguments: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60], [lochierarchy#30 ASC NULLS FIRST, _w0#60 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] - -(63) CometColumnarToRow [codegen id : 19] -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] - -(64) Window -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] -Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#30, _w0#60, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#61], [lochierarchy#30, _w0#60], [total_sum#25 DESC NULLS LAST] - -(65) Project [codegen id : 20] -Output [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#61] -Input [6]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60, rank_within_parent#61] - -(66) TakeOrderedAndProject -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#61] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#61 ASC NULLS FIRST], [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#61] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometProject (69) - +- CometFilter (68) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (67) - - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(68) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(69) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(70) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(71) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 2832c5a348..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,101 +0,0 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (20) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] - CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (18) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [s_state,_w0] - CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] - CometExchange [s_state] #8 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - 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,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #9 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (11) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #10 - WholeStageCodegen (10) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #11 - WholeStageCodegen (16) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt deleted file mode 100644 index af71da5f87..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/explain.txt +++ /dev/null @@ -1,408 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (66) -+- * Project (65) - +- Window (64) - +- * CometColumnarToRow (63) - +- CometSort (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometColumnarExchange (59) - +- * HashAggregate (58) - +- Union (57) - :- * HashAggregate (42) - : +- * CometColumnarToRow (41) - : +- CometColumnarExchange (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * CometColumnarToRow (9) - : : +- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- BroadcastExchange (36) - : +- * Project (35) - : +- * BroadcastHashJoin LeftSemi BuildRight (34) - : :- * CometColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) - : +- BroadcastExchange (33) - : +- * Project (32) - : +- * Filter (31) - : +- Window (30) - : +- WindowGroupLimit (29) - : +- * CometColumnarToRow (28) - : +- CometSort (27) - : +- CometHashAggregate (26) - : +- CometExchange (25) - : +- CometHashAggregate (24) - : +- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (20) - : : +- CometBroadcastHashJoin (19) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - : : +- CometBroadcastExchange (18) - : : +- CometProject (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - : +- ReusedExchange (21) - :- * HashAggregate (49) - : +- * CometColumnarToRow (48) - : +- CometColumnarExchange (47) - : +- * HashAggregate (46) - : +- * HashAggregate (45) - : +- * CometColumnarToRow (44) - : +- ReusedExchange (43) - +- * HashAggregate (56) - +- * CometColumnarToRow (55) - +- CometColumnarExchange (54) - +- * HashAggregate (53) - +- * HashAggregate (52) - +- * CometColumnarToRow (51) - +- ReusedExchange (50) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] - -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ss_store_sk#1, ss_net_profit#2] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#7, s_county#8, s_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Condition : isnotnull(s_store_sk#7) - -(12) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_store_sk#10) - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : isnotnull(s_store_sk#14) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14, s_state#16], [s_store_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) AS s_state#16] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#14, s_state#16] -Arguments: [s_store_sk#14, s_state#16] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Right output [2]: [s_store_sk#14, s_state#16] -Arguments: [ss_store_sk#10], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#16] -Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16], [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#17] - -(22) CometBroadcastHashJoin -Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] -Right output [1]: [d_date_sk#17] -Arguments: [ss_sold_date_sk#12], [d_date_sk#17], Inner, BuildRight - -(23) CometProject -Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16, d_date_sk#17] -Arguments: [ss_net_profit#11, s_state#16], [ss_net_profit#11, s_state#16] - -(24) CometHashAggregate -Input [2]: [ss_net_profit#11, s_state#16] -Keys [1]: [s_state#16] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] - -(25) CometExchange -Input [2]: [s_state#16, sum#18] -Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(26) CometHashAggregate -Input [2]: [s_state#16, sum#18] -Keys [1]: [s_state#16] -Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] - -(27) CometSort -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16, _w0#19, s_state#16], [s_state#16 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] - -(28) CometColumnarToRow [codegen id : 1] -Input [3]: [s_state#16, _w0#19, s_state#16] - -(29) WindowGroupLimit -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final - -(30) Window -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#16, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#16], [_w0#19 DESC NULLS LAST] - -(31) Filter [codegen id : 2] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] -Condition : (ranking#20 <= 5) - -(32) Project [codegen id : 2] -Output [1]: [s_state#16] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] - -(33) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] - -(34) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)] -Right keys [1]: [s_state#16] -Join type: LeftSemi -Join condition: None - -(35) Project [codegen id : 3] -Output [3]: [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#21] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(36) BroadcastExchange -Input [3]: [s_store_sk#7, s_county#8, s_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_county#8, s_state#21] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#21] - -(39) HashAggregate [codegen id : 4] -Input [3]: [ss_net_profit#2, s_county#8, s_state#21] -Keys [2]: [s_state#21, s_county#8] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [s_state#21, s_county#8, sum#23] - -(40) CometColumnarExchange -Input [3]: [s_state#21, s_county#8, sum#23] -Arguments: hashpartitioning(s_state#21, s_county#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 5] -Input [3]: [s_state#21, s_county#8, sum#23] - -(42) HashAggregate [codegen id : 5] -Input [3]: [s_state#21, s_county#8, sum#23] -Keys [2]: [s_state#21, s_county#8] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) as decimal(27,2)) AS total_sum#25, s_state#21 AS s_state#26, s_county#8 AS s_county#27, 0 AS g_state#28, 0 AS g_county#29, 0 AS lochierarchy#30] - -(43) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_state#21, s_county#31, sum#32] - -(44) CometColumnarToRow [codegen id : 10] -Input [3]: [s_state#21, s_county#31, sum#32] - -(45) HashAggregate [codegen id : 10] -Input [3]: [s_state#21, s_county#31, sum#32] -Keys [2]: [s_state#21, s_county#31] -Functions [1]: [sum(UnscaledValue(ss_net_profit#33))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#33))#24] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#33))#24,17,2) AS total_sum#34, s_state#21] - -(46) HashAggregate [codegen id : 10] -Input [2]: [total_sum#34, s_state#21] -Keys [1]: [s_state#21] -Functions [1]: [partial_sum(total_sum#34)] -Aggregate Attributes [2]: [sum#35, isEmpty#36] -Results [3]: [s_state#21, sum#37, isEmpty#38] - -(47) CometColumnarExchange -Input [3]: [s_state#21, sum#37, isEmpty#38] -Arguments: hashpartitioning(s_state#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(48) CometColumnarToRow [codegen id : 11] -Input [3]: [s_state#21, sum#37, isEmpty#38] - -(49) HashAggregate [codegen id : 11] -Input [3]: [s_state#21, sum#37, isEmpty#38] -Keys [1]: [s_state#21] -Functions [1]: [sum(total_sum#34)] -Aggregate Attributes [1]: [sum(total_sum#34)#39] -Results [6]: [sum(total_sum#34)#39 AS total_sum#40, s_state#21, null AS s_county#41, 0 AS g_state#42, 1 AS g_county#43, 1 AS lochierarchy#44] - -(50) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_state#21, s_county#45, sum#46] - -(51) CometColumnarToRow [codegen id : 16] -Input [3]: [s_state#21, s_county#45, sum#46] - -(52) HashAggregate [codegen id : 16] -Input [3]: [s_state#21, s_county#45, sum#46] -Keys [2]: [s_state#21, s_county#45] -Functions [1]: [sum(UnscaledValue(ss_net_profit#47))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#47))#24] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#47))#24,17,2) AS total_sum#48] - -(53) HashAggregate [codegen id : 16] -Input [1]: [total_sum#48] -Keys: [] -Functions [1]: [partial_sum(total_sum#48)] -Aggregate Attributes [2]: [sum#49, isEmpty#50] -Results [2]: [sum#51, isEmpty#52] - -(54) CometColumnarExchange -Input [2]: [sum#51, isEmpty#52] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(55) CometColumnarToRow [codegen id : 17] -Input [2]: [sum#51, isEmpty#52] - -(56) HashAggregate [codegen id : 17] -Input [2]: [sum#51, isEmpty#52] -Keys: [] -Functions [1]: [sum(total_sum#48)] -Aggregate Attributes [1]: [sum(total_sum#48)#53] -Results [6]: [sum(total_sum#48)#53 AS total_sum#54, null AS s_state#55, null AS s_county#56, 1 AS g_state#57, 1 AS g_county#58, 2 AS lochierarchy#59] - -(57) Union - -(58) HashAggregate [codegen id : 18] -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Functions: [] -Aggregate Attributes: [] -Results [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] - -(59) CometColumnarExchange -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Arguments: hashpartitioning(total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(60) CometHashAggregate -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Functions: [] - -(61) CometExchange -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] -Arguments: hashpartitioning(lochierarchy#30, _w0#60, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(62) CometSort -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] -Arguments: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60], [lochierarchy#30 ASC NULLS FIRST, _w0#60 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] - -(63) CometColumnarToRow [codegen id : 19] -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] - -(64) Window -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60] -Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#30, _w0#60, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#61], [lochierarchy#30, _w0#60], [total_sum#25 DESC NULLS LAST] - -(65) Project [codegen id : 20] -Output [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#61] -Input [6]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#60, rank_within_parent#61] - -(66) TakeOrderedAndProject -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#61] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#61 ASC NULLS FIRST], [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#61] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometProject (69) - +- CometFilter (68) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (67) - - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(68) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(69) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(70) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(71) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/extended.txt deleted file mode 100644 index 191d0ef18a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/extended.txt +++ /dev/null @@ -1,176 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window - : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window - : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 102 out of 156 eligible operators (65%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 2832c5a348..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/simplified.txt +++ /dev/null @@ -1,101 +0,0 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (20) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] - CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (18) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [s_state,_w0] - CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] - CometExchange [s_state] #8 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - 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,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #9 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (11) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #10 - WholeStageCodegen (10) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #11 - WholeStageCodegen (16) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/explain.txt deleted file mode 100644 index 44a85d557d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/explain.txt +++ /dev/null @@ -1,428 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (70) -+- CometTakeOrderedAndProject (69) - +- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometProject (65) - +- CometSortMergeJoin (64) - :- CometSort (58) - : +- CometColumnarExchange (57) - : +- * Project (56) - : +- * BroadcastHashJoin LeftOuter BuildRight (55) - : :- * Project (50) - : : +- * BroadcastHashJoin Inner BuildRight (49) - : : :- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (28) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Project (15) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : : :- * Project (9) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : : : : :- * Filter (3) - : : : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * CometColumnarToRow (6) - : : : : : : : : : +- CometFilter (5) - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory (4) - : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * CometColumnarToRow (12) - : : : : : : : : +- CometFilter (11) - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * CometColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) - : : : : : : +- BroadcastExchange (26) - : : : : : : +- * CometColumnarToRow (25) - : : : : : : +- CometProject (24) - : : : : : : +- CometFilter (23) - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (22) - : : : : : +- BroadcastExchange (33) - : : : : : +- * CometColumnarToRow (32) - : : : : : +- CometProject (31) - : : : : : +- CometFilter (30) - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (29) - : : : : +- ReusedExchange (36) - : : : +- BroadcastExchange (42) - : : : +- * CometColumnarToRow (41) - : : : +- CometFilter (40) - : : : +- CometNativeScan parquet spark_catalog.default.date_dim (39) - : : +- BroadcastExchange (48) - : : +- * CometColumnarToRow (47) - : : +- CometFilter (46) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (45) - : +- BroadcastExchange (54) - : +- * CometColumnarToRow (53) - : +- CometFilter (52) - : +- CometNativeScan parquet spark_catalog.default.promotion (51) - +- CometSort (63) - +- CometExchange (62) - +- CometProject (61) - +- CometFilter (60) - +- CometNativeScan parquet spark_catalog.default.catalog_returns (59) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 10] -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] - -(3) Filter [codegen id : 10] -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) - -(4) CometNativeScan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(inv_date_sk#13)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) - -(6) CometColumnarToRow [codegen id : 1] -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(7) BroadcastExchange -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [inv_item_sk#10] -Join type: Inner -Join condition: (inv_quantity_on_hand#12 < cs_quantity#7) - -(9) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(10) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Condition : isnotnull(w_warehouse_sk#14) - -(12) CometColumnarToRow [codegen id : 2] -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] - -(13) BroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_warehouse_sk#11] -Right keys [1]: [w_warehouse_sk#14] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] - -(16) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_desc#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [i_item_sk#16, i_item_desc#17] -Condition : isnotnull(i_item_sk#16) - -(18) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#16, i_item_desc#17] - -(19) BroadcastExchange -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(20) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#16] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 10] -Output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] - -(22) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#18, cd_marital_status#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(23) CometFilter -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = M) AND isnotnull(cd_demo_sk#18)) - -(24) CometProject -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Arguments: [cd_demo_sk#18], [cd_demo_sk#18] - -(25) CometColumnarToRow [codegen id : 4] -Input [1]: [cd_demo_sk#18] - -(26) BroadcastExchange -Input [1]: [cd_demo_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(27) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] - -(29) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_buy_potential#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#21, 15, true, false, true) = 1001-5000 ) AND isnotnull(hd_demo_sk#20)) - -(31) CometProject -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Arguments: [hd_demo_sk#20], [hd_demo_sk#20] - -(32) CometColumnarToRow [codegen id : 5] -Input [1]: [hd_demo_sk#20] - -(33) BroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] - -(36) ReusedExchange [Reuses operator id: 75] -Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#8] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] - -(39) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_week_seq#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(40) CometFilter -Input [2]: [d_date_sk#25, d_week_seq#26] -Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) - -(41) CometColumnarToRow [codegen id : 7] -Input [2]: [d_date_sk#25, d_week_seq#26] - -(42) BroadcastExchange -Input [2]: [d_date_sk#25, d_week_seq#26] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] - -(43) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [d_week_seq#24, inv_date_sk#13] -Right keys [2]: [d_week_seq#26, d_date_sk#25] -Join type: Inner -Join condition: None - -(44) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#25, d_week_seq#26] - -(45) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#27, d_date#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [d_date_sk#27, d_date#28] -Condition : (isnotnull(d_date#28) AND isnotnull(d_date_sk#27)) - -(47) CometColumnarToRow [codegen id : 8] -Input [2]: [d_date_sk#27, d_date#28] - -(48) BroadcastExchange -Input [2]: [d_date_sk#27, d_date#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: (d_date#28 > date_add(d_date#23, 5)) - -(50) Project [codegen id : 10] -Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#27, d_date#28] - -(51) CometNativeScan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(52) CometFilter -Input [1]: [p_promo_sk#29] -Condition : isnotnull(p_promo_sk#29) - -(53) CometColumnarToRow [codegen id : 9] -Input [1]: [p_promo_sk#29] - -(54) BroadcastExchange -Input [1]: [p_promo_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(55) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_promo_sk#5] -Right keys [1]: [p_promo_sk#29] -Join type: LeftOuter -Join condition: None - -(56) Project [codegen id : 10] -Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] - -(57) CometColumnarExchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(58) CometSort -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] - -(59) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(60) CometFilter -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Condition : (isnotnull(cr_item_sk#30) AND isnotnull(cr_order_number#31)) - -(61) CometProject -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30, cr_order_number#31] - -(62) CometExchange -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(63) CometSort -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST] - -(64) CometSortMergeJoin -Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#30, cr_order_number#31], LeftOuter - -(65) CometProject -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#30, cr_order_number#31] -Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(66) CometHashAggregate -Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [partial_count(1)] - -(67) CometExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(68) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [count(1)] - -(69) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#36 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#34,promo#35,total_cnt#36]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36], 100, 0, [total_cnt#36 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] - -(70) CometColumnarToRow [codegen id : 11] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (75) -+- * CometColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometNativeScan parquet spark_catalog.default.date_dim (71) - - -(71) CometNativeScan parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(72) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] -Condition : ((((isnotnull(d_year#37) AND (d_year#37 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(73) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(74) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(75) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/simplified.txt deleted file mode 100644 index 025217a607..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/simplified.txt +++ /dev/null @@ -1,107 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] - CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] - CometProject [w_warehouse_name,i_item_desc,d_week_seq] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometColumnarExchange [cs_item_sk,cs_order_number] #2 - WholeStageCodegen (10) - Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [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] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometNativeScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometNativeScan 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/q72.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/explain.txt deleted file mode 100644 index ee3242f2b1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,402 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (64) -+- CometTakeOrderedAndProject (63) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometSortMergeJoin (58) - :- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (45) - : : +- CometBroadcastHashJoin (44) - : : :- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (35) - : : : : +- CometBroadcastHashJoin (34) - : : : : :- CometProject (29) - : : : : : +- CometBroadcastHashJoin (28) - : : : : : :- CometProject (23) - : : : : : : +- CometBroadcastHashJoin (22) - : : : : : : :- CometProject (17) - : : : : : : : +- CometBroadcastHashJoin (16) - : : : : : : : :- CometProject (12) - : : : : : : : : +- CometBroadcastHashJoin (11) - : : : : : : : : :- CometProject (7) - : : : : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : : : : :- CometFilter (2) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- CometBroadcastExchange (5) - : : : : : : : : : +- CometFilter (4) - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (3) - : : : : : : : : +- CometBroadcastExchange (10) - : : : : : : : : +- CometFilter (9) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : : : : : : : +- CometBroadcastExchange (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (13) - : : : : : : +- CometBroadcastExchange (21) - : : : : : : +- CometProject (20) - : : : : : : +- CometFilter (19) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (18) - : : : : : +- CometBroadcastExchange (27) - : : : : : +- CometProject (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (24) - : : : : +- CometBroadcastExchange (33) - : : : : +- CometProject (32) - : : : : +- CometFilter (31) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - : : : +- CometBroadcastExchange (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (36) - : : +- CometBroadcastExchange (43) - : : +- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - : +- CometBroadcastExchange (48) - : +- CometFilter (47) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (46) - +- CometSort (57) - +- CometExchange (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (53) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#13)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) - -(5) CometBroadcastExchange -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(6) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Right output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_item_sk#4], [inv_item_sk#10], Inner, (inv_quantity_on_hand#12 < cs_quantity#7), BuildRight - -(7) CometProject -Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Condition : isnotnull(w_warehouse_sk#14) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [w_warehouse_sk#14, w_warehouse_name#15] - -(11) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -Right output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [inv_warehouse_sk#11], [w_warehouse_sk#14], Inner, BuildRight - -(12) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_desc#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [i_item_sk#16, i_item_desc#17] -Condition : isnotnull(i_item_sk#16) - -(15) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [i_item_sk#16, i_item_desc#17] - -(16) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -Right output [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [cs_item_sk#4], [i_item_sk#16], Inner, BuildRight - -(17) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#18, cd_marital_status#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = M) AND isnotnull(cd_demo_sk#18)) - -(20) CometProject -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Arguments: [cd_demo_sk#18], [cd_demo_sk#18] - -(21) CometBroadcastExchange -Input [1]: [cd_demo_sk#18] -Arguments: [cd_demo_sk#18] - -(22) CometBroadcastHashJoin -Left output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [cd_demo_sk#18] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#18], Inner, BuildRight - -(23) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] -Arguments: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_buy_potential#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#21, 15, true, false, true) = 1001-5000 ) AND isnotnull(hd_demo_sk#20)) - -(26) CometProject -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Arguments: [hd_demo_sk#20], [hd_demo_sk#20] - -(27) CometBroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: [hd_demo_sk#20] - -(28) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [hd_demo_sk#20] -Arguments: [cs_bill_hdemo_sk#3], [hd_demo_sk#20], Inner, BuildRight - -(29) CometProject -Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(31) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(32) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(33) CometBroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24] - -(34) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_sold_date_sk#8], [d_date_sk#22], Inner, BuildRight - -(35) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_week_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [d_date_sk#26, d_week_seq#27] -Condition : (isnotnull(d_week_seq#27) AND isnotnull(d_date_sk#26)) - -(38) CometBroadcastExchange -Input [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_date_sk#26, d_week_seq#27] - -(39) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_week_seq#24, inv_date_sk#13], [d_week_seq#27, d_date_sk#26], Inner, BuildRight - -(40) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#26, d_week_seq#27] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_date#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#28, d_date#29] -Condition : (isnotnull(d_date#29) AND isnotnull(d_date_sk#28)) - -(43) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_date#29] -Arguments: [d_date_sk#28, d_date#29] - -(44) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#28, d_date#29] -Arguments: [cs_ship_date_sk#1], [d_date_sk#28], Inner, (d_date#29 > date_add(d_date#23, 5)), BuildRight - -(45) CometProject -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#28, d_date#29] -Arguments: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(47) CometFilter -Input [1]: [p_promo_sk#30] -Condition : isnotnull(p_promo_sk#30) - -(48) CometBroadcastExchange -Input [1]: [p_promo_sk#30] -Arguments: [p_promo_sk#30] - -(49) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [1]: [p_promo_sk#30] -Arguments: [cs_promo_sk#5], [p_promo_sk#30], LeftOuter, BuildRight - -(50) CometProject -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#30] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(51) CometExchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(52) CometSort -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Condition : (isnotnull(cr_item_sk#31) AND isnotnull(cr_order_number#32)) - -(55) CometProject -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31, cr_order_number#32] - -(56) CometExchange -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(57) CometSort -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#31, cr_order_number#32], LeftOuter - -(59) CometProject -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#31, cr_order_number#32] -Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(60) CometHashAggregate -Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [partial_count(1)] - -(61) CometExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(62) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [count(1)] - -(63) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, 0, [total_cnt#37 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -(64) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (69) -+- * CometColumnarToRow (68) - +- CometProject (67) - +- CometFilter (66) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) - - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(67) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(68) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(69) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/simplified.txt deleted file mode 100644 index 678f8c67df..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] - CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] - CometProject [w_warehouse_name,i_item_desc,d_week_seq] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometExchange [cs_item_sk,cs_order_number] #2 - CometProject [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,p_promo_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_date] - 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 [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_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_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_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_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 [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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_desc] #6 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange [cd_demo_sk] #7 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange [hd_demo_sk] #8 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [d_date_sk,d_week_seq] #10 - CometFilter [d_date_sk,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [d_date_sk,d_date] #11 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [p_promo_sk] #12 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/explain.txt deleted file mode 100644 index ee3242f2b1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/explain.txt +++ /dev/null @@ -1,402 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (64) -+- CometTakeOrderedAndProject (63) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometSortMergeJoin (58) - :- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (45) - : : +- CometBroadcastHashJoin (44) - : : :- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (35) - : : : : +- CometBroadcastHashJoin (34) - : : : : :- CometProject (29) - : : : : : +- CometBroadcastHashJoin (28) - : : : : : :- CometProject (23) - : : : : : : +- CometBroadcastHashJoin (22) - : : : : : : :- CometProject (17) - : : : : : : : +- CometBroadcastHashJoin (16) - : : : : : : : :- CometProject (12) - : : : : : : : : +- CometBroadcastHashJoin (11) - : : : : : : : : :- CometProject (7) - : : : : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : : : : :- CometFilter (2) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- CometBroadcastExchange (5) - : : : : : : : : : +- CometFilter (4) - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (3) - : : : : : : : : +- CometBroadcastExchange (10) - : : : : : : : : +- CometFilter (9) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : : : : : : : +- CometBroadcastExchange (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (13) - : : : : : : +- CometBroadcastExchange (21) - : : : : : : +- CometProject (20) - : : : : : : +- CometFilter (19) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (18) - : : : : : +- CometBroadcastExchange (27) - : : : : : +- CometProject (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (24) - : : : : +- CometBroadcastExchange (33) - : : : : +- CometProject (32) - : : : : +- CometFilter (31) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - : : : +- CometBroadcastExchange (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (36) - : : +- CometBroadcastExchange (43) - : : +- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - : +- CometBroadcastExchange (48) - : +- CometFilter (47) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (46) - +- CometSort (57) - +- CometExchange (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (53) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#13)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) - -(5) CometBroadcastExchange -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(6) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Right output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_item_sk#4], [inv_item_sk#10], Inner, (inv_quantity_on_hand#12 < cs_quantity#7), BuildRight - -(7) CometProject -Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Condition : isnotnull(w_warehouse_sk#14) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [w_warehouse_sk#14, w_warehouse_name#15] - -(11) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -Right output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [inv_warehouse_sk#11], [w_warehouse_sk#14], Inner, BuildRight - -(12) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_desc#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [i_item_sk#16, i_item_desc#17] -Condition : isnotnull(i_item_sk#16) - -(15) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [i_item_sk#16, i_item_desc#17] - -(16) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -Right output [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [cs_item_sk#4], [i_item_sk#16], Inner, BuildRight - -(17) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#18, cd_marital_status#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = M) AND isnotnull(cd_demo_sk#18)) - -(20) CometProject -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Arguments: [cd_demo_sk#18], [cd_demo_sk#18] - -(21) CometBroadcastExchange -Input [1]: [cd_demo_sk#18] -Arguments: [cd_demo_sk#18] - -(22) CometBroadcastHashJoin -Left output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [cd_demo_sk#18] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#18], Inner, BuildRight - -(23) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] -Arguments: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_buy_potential#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#21, 15, true, false, true) = 1001-5000 ) AND isnotnull(hd_demo_sk#20)) - -(26) CometProject -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Arguments: [hd_demo_sk#20], [hd_demo_sk#20] - -(27) CometBroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: [hd_demo_sk#20] - -(28) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [hd_demo_sk#20] -Arguments: [cs_bill_hdemo_sk#3], [hd_demo_sk#20], Inner, BuildRight - -(29) CometProject -Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(31) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(32) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(33) CometBroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24] - -(34) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_sold_date_sk#8], [d_date_sk#22], Inner, BuildRight - -(35) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_week_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [d_date_sk#26, d_week_seq#27] -Condition : (isnotnull(d_week_seq#27) AND isnotnull(d_date_sk#26)) - -(38) CometBroadcastExchange -Input [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_date_sk#26, d_week_seq#27] - -(39) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_week_seq#24, inv_date_sk#13], [d_week_seq#27, d_date_sk#26], Inner, BuildRight - -(40) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#26, d_week_seq#27] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_date#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#28, d_date#29] -Condition : (isnotnull(d_date#29) AND isnotnull(d_date_sk#28)) - -(43) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_date#29] -Arguments: [d_date_sk#28, d_date#29] - -(44) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#28, d_date#29] -Arguments: [cs_ship_date_sk#1], [d_date_sk#28], Inner, (d_date#29 > date_add(d_date#23, 5)), BuildRight - -(45) CometProject -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#28, d_date#29] -Arguments: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(47) CometFilter -Input [1]: [p_promo_sk#30] -Condition : isnotnull(p_promo_sk#30) - -(48) CometBroadcastExchange -Input [1]: [p_promo_sk#30] -Arguments: [p_promo_sk#30] - -(49) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [1]: [p_promo_sk#30] -Arguments: [cs_promo_sk#5], [p_promo_sk#30], LeftOuter, BuildRight - -(50) CometProject -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#30] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(51) CometExchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(52) CometSort -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Condition : (isnotnull(cr_item_sk#31) AND isnotnull(cr_order_number#32)) - -(55) CometProject -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31, cr_order_number#32] - -(56) CometExchange -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(57) CometSort -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#31, cr_order_number#32], LeftOuter - -(59) CometProject -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#31, cr_order_number#32] -Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(60) CometHashAggregate -Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [partial_count(1)] - -(61) CometExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(62) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [count(1)] - -(63) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, 0, [total_cnt#37 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -(64) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (69) -+- * CometColumnarToRow (68) - +- CometProject (67) - +- CometFilter (66) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) - - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(67) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(68) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(69) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/extended.txt deleted file mode 100644 index 9000d27d1b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/extended.txt +++ /dev/null @@ -1,72 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 678f8c67df..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] - CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] - CometProject [w_warehouse_name,i_item_desc,d_week_seq] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometExchange [cs_item_sk,cs_order_number] #2 - CometProject [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,p_promo_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_date] - 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 [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_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_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_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_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 [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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_desc] #6 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange [cd_demo_sk] #7 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange [hd_demo_sk] #8 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [d_date_sk,d_week_seq] #10 - CometFilter [d_date_sk,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [d_date_sk,d_date] #11 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [p_promo_sk] #12 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/explain.txt deleted file mode 100644 index 5d2bcfdc5c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/explain.txt +++ /dev/null @@ -1,513 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (79) -+- * Project (78) - +- * BroadcastHashJoin Inner BuildRight (77) - :- * Project (58) - : +- * BroadcastHashJoin Inner BuildRight (57) - : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- * HashAggregate (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * Project (28) - : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (26) - : : : +- * Filter (25) - : : : +- * ColumnarToRow (24) - : : : +- Scan parquet spark_catalog.default.store_sales (23) - : : +- ReusedExchange (29) - : +- BroadcastExchange (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- * CometColumnarToRow (53) - : +- CometColumnarExchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (41) - : : : +- CometProject (40) - : : : +- CometFilter (39) - : : : +- CometNativeScan parquet spark_catalog.default.customer (38) - : : +- BroadcastExchange (45) - : : +- * Filter (44) - : : +- * ColumnarToRow (43) - : : +- Scan parquet spark_catalog.default.web_sales (42) - : +- ReusedExchange (48) - +- BroadcastExchange (76) - +- * HashAggregate (75) - +- * CometColumnarToRow (74) - +- CometColumnarExchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * CometColumnarToRow (62) - : : +- CometProject (61) - : : +- CometFilter (60) - : : +- CometNativeScan parquet spark_catalog.default.customer (59) - : +- BroadcastExchange (66) - : +- * Filter (65) - : +- * ColumnarToRow (64) - : +- Scan parquet spark_catalog.default.web_sales (63) - +- ReusedExchange (69) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#7] - -(4) CometColumnarToRow [codegen id : 3] -Input [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] - -(5) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(7) Filter [codegen id : 1] -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Condition : isnotnull(ss_customer_sk#8) - -(8) BroadcastExchange -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#8] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(11) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#12, d_year#13] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#10] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] - -(14) HashAggregate [codegen id : 3] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] -Aggregate Attributes [1]: [sum#14] -Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] - -(15) CometColumnarExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 16] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] - -(17) HashAggregate [codegen id : 16] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#9))#16] -Results [2]: [c_customer_id#5 AS customer_id#17, MakeDecimal(sum(UnscaledValue(ss_net_paid#9))#16,17,2) AS year_total#18] - -(18) Filter [codegen id : 16] -Input [2]: [customer_id#17, year_total#18] -Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) - -(19) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(20) CometFilter -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Condition : (isnotnull(c_customer_sk#19) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#20, 16, true, false, true))) - -(21) CometProject -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Arguments: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#20, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#21, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#22, 30, true, false, true) AS c_last_name#7] - -(22) CometColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7] - -(23) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] - -(25) Filter [codegen id : 4] -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_customer_sk#23) - -(26) BroadcastExchange -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#19] -Right keys [1]: [ss_customer_sk#23] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25] -Input [7]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] - -(29) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#27, d_year#28] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#25] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25, d_date_sk#27, d_year#28] - -(32) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#24))] -Aggregate Attributes [1]: [sum#29] -Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] - -(33) CometColumnarExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 7] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] - -(35) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] -Functions [1]: [sum(UnscaledValue(ss_net_paid#24))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#24))#16] -Results [4]: [c_customer_id#5 AS customer_id#31, c_first_name#6 AS customer_first_name#32, c_last_name#7 AS customer_last_name#33, MakeDecimal(sum(UnscaledValue(ss_net_paid#24))#16,17,2) AS year_total#34] - -(36) BroadcastExchange -Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(37) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#31] -Join type: Inner -Join condition: None - -(38) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(39) CometFilter -Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Condition : (isnotnull(c_customer_sk#35) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#36, 16, true, false, true))) - -(40) CometProject -Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Arguments: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#36, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#37, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#38, 30, true, false, true) AS c_last_name#41] - -(41) CometColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41] - -(42) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] - -(44) Filter [codegen id : 8] -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_bill_customer_sk#42) - -(45) BroadcastExchange -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -(46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#35] -Right keys [1]: [ws_bill_customer_sk#42] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 10] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] -Input [7]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] - -(48) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#45, d_year#46] - -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#45] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 10] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#46] -Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#45, d_year#46] - -(51) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#46] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] -Aggregate Attributes [1]: [sum#47] -Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] - -(52) CometColumnarExchange -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] -Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(53) CometColumnarToRow [codegen id : 11] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] - -(54) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46] -Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#49] -Results [2]: [c_customer_id#39 AS customer_id#50, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#49,17,2) AS year_total#51] - -(55) Filter [codegen id : 11] -Input [2]: [customer_id#50, year_total#51] -Condition : (isnotnull(year_total#51) AND (year_total#51 > 0.00)) - -(56) BroadcastExchange -Input [2]: [customer_id#50, year_total#51] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(57) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#50] -Join type: Inner -Join condition: None - -(58) Project [codegen id : 16] -Output [7]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#51] -Input [8]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, customer_id#50, year_total#51] - -(59) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(60) CometFilter -Input [4]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55] -Condition : (isnotnull(c_customer_sk#52) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#53, 16, true, false, true))) - -(61) CometProject -Input [4]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55] -Arguments: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#52, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#53, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#54, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#55, 30, true, false, true) AS c_last_name#41] - -(62) CometColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41] - -(63) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#26)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(64) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] - -(65) Filter [codegen id : 12] -Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] -Condition : isnotnull(ws_bill_customer_sk#56) - -(66) BroadcastExchange -Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#52] -Right keys [1]: [ws_bill_customer_sk#56] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 14] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, ws_sold_date_sk#58] -Input [7]: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] - -(69) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#59, d_year#60] - -(70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#58] -Right keys [1]: [d_date_sk#59] -Join type: Inner -Join condition: None - -(71) Project [codegen id : 14] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, d_year#60] -Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, ws_sold_date_sk#58, d_date_sk#59, d_year#60] - -(72) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, d_year#60] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#57))] -Aggregate Attributes [1]: [sum#61] -Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] - -(73) CometColumnarExchange -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] -Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(74) CometColumnarToRow [codegen id : 15] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] - -(75) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60] -Functions [1]: [sum(UnscaledValue(ws_net_paid#57))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#57))#49] -Results [2]: [c_customer_id#39 AS customer_id#63, MakeDecimal(sum(UnscaledValue(ws_net_paid#57))#49,17,2) AS year_total#64] - -(76) BroadcastExchange -Input [2]: [customer_id#63, year_total#64] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] - -(77) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#63] -Join type: Inner -Join condition: (CASE WHEN (year_total#51 > 0.00) THEN (year_total#64 / year_total#51) END > CASE WHEN (year_total#18 > 0.00) THEN (year_total#34 / year_total#18) END) - -(78) Project [codegen id : 16] -Output [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] -Input [9]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#51, customer_id#63, year_total#64] - -(79) TakeOrderedAndProject -Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] -Arguments: 100, [customer_first_name#32 ASC NULLS FIRST, customer_id#31 ASC NULLS FIRST, customer_last_name#33 ASC NULLS FIRST], [customer_id#31, customer_first_name#32, customer_last_name#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometFilter (81) - +- CometNativeScan parquet spark_catalog.default.date_dim (80) - - -(80) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(81) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(82) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(83) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 -BroadcastExchange (87) -+- * CometColumnarToRow (86) - +- CometFilter (85) - +- CometNativeScan parquet spark_catalog.default.date_dim (84) - - -(84) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#27, d_year#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(85) CometFilter -Input [2]: [d_date_sk#27, d_year#28] -Condition : (((isnotnull(d_year#28) AND (d_year#28 = 2002)) AND d_year#28 IN (2001,2002)) AND isnotnull(d_date_sk#27)) - -(86) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#28] - -(87) BroadcastExchange -Input [2]: [d_date_sk#27, d_year#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] - -Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#11 - -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#26 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/simplified.txt deleted file mode 100644 index 9a8a029e1b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/simplified.txt +++ /dev/null @@ -1,130 +0,0 @@ -TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] - WholeStageCodegen (16) - Project [customer_id,customer_first_name,customer_last_name] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/explain.txt deleted file mode 100644 index b0ae8bb695..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,470 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometProject (53) - : +- CometBroadcastHashJoin (52) - : :- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (51) - : +- CometFilter (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : +- CometBroadcastExchange (41) - : : +- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) - : +- ReusedExchange (44) - +- CometBroadcastExchange (68) - +- CometHashAggregate (67) - +- CometExchange (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometFilter (55) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (62) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Condition : isnotnull(ss_customer_sk#8) - -(6) CometBroadcastExchange -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_sk#1], [ss_customer_sk#8], Inner, BuildRight - -(8) CometProject -Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(12) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#10], [d_date_sk#12], Inner, BuildRight - -(13) CometProject -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] - -(14) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] - -(15) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] - -(17) CometFilter -Input [2]: [customer_id#15, year_total#16] -Condition : (isnotnull(year_total#16) AND (year_total#16 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true))) - -(20) CometProject -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Arguments: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#19, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#20, 30, true, false, true) AS c_last_name#7] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_customer_sk#21) - -(23) CometBroadcastExchange -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] - -(24) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [c_customer_sk#17], [ss_customer_sk#21], Inner, BuildRight - -(25) CometProject -Input [7]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: [d_date_sk#25, d_year#26] - -(29) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] -Right output [2]: [d_date_sk#25, d_year#26] -Arguments: [ss_sold_date_sk#23], [d_date_sk#25], Inner, BuildRight - -(30) CometProject -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23, d_date_sk#25, d_year#26] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] - -(31) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] - -(32) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] -Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] - -(34) CometBroadcastExchange -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#15, year_total#16] -Right output [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#15], [customer_id#28], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true))) - -(38) CometProject -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Arguments: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#34, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#35, 30, true, false, true) AS c_last_name#38] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(ws_sold_date_sk#41 IN dynamicpruning#42)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Condition : isnotnull(ws_bill_customer_sk#39) - -(41) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] - -(42) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38] -Right output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [c_customer_sk#32], [ws_bill_customer_sk#39], Inner, BuildRight - -(43) CometProject -Input [7]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#43, d_year#44] - -(45) CometBroadcastHashJoin -Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] -Right output [2]: [d_date_sk#43, d_year#44] -Arguments: [ws_sold_date_sk#41], [d_date_sk#43], Inner, BuildRight - -(46) CometProject -Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41, d_date_sk#43, d_year#44] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] - -(47) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#40))] - -(48) CometExchange -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] -Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] - -(50) CometFilter -Input [2]: [customer_id#46, year_total#47] -Condition : (isnotnull(year_total#47) AND (year_total#47 > 0.00)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#46, year_total#47] -Arguments: [customer_id#46, year_total#47] - -(52) CometBroadcastHashJoin -Left output [6]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Right output [2]: [customer_id#46, year_total#47] -Arguments: [customer_id#15], [customer_id#46], Inner, BuildRight - -(53) CometProject -Input [8]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#46, year_total#47] -Arguments: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47], [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Condition : (isnotnull(c_customer_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true))) - -(56) CometProject -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Arguments: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#50, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#51, 30, true, false, true) AS c_last_name#38] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#55)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Condition : isnotnull(ws_bill_customer_sk#52) - -(59) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] - -(60) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38] -Right output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [c_customer_sk#48], [ws_bill_customer_sk#52], Inner, BuildRight - -(61) CometProject -Input [7]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#56, d_year#57] - -(63) CometBroadcastHashJoin -Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] -Right output [2]: [d_date_sk#56, d_year#57] -Arguments: [ws_sold_date_sk#54], [d_date_sk#56], Inner, BuildRight - -(64) CometProject -Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54, d_date_sk#56, d_year#57] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] - -(65) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#53))] - -(66) CometExchange -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] -Functions [1]: [sum(UnscaledValue(ws_net_paid#53))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#59, year_total#60] -Arguments: [customer_id#59, year_total#60] - -(69) CometBroadcastHashJoin -Left output [7]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] -Right output [2]: [customer_id#59, year_total#60] -Arguments: [customer_id#15], [customer_id#59], Inner, (CASE WHEN (year_total#47 > 0.00) THEN (year_total#60 / year_total#47) END > CASE WHEN (year_total#16 > 0.00) THEN (year_total#31 / year_total#16) END), BuildRight - -(70) CometProject -Input [9]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47, customer_id#59, year_total#60] -Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30], [customer_id#28, customer_first_name#29, customer_last_name#30] - -(71) CometTakeOrderedAndProject -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_first_name#29 ASC NULLS FIRST,customer_id#28 ASC NULLS FIRST,customer_last_name#30 ASC NULLS FIRST], output=[customer_id#28,customer_first_name#29,customer_last_name#30]), [customer_id#28, customer_first_name#29, customer_last_name#30], 100, 0, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_last_name#30 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] - -(72) CometColumnarToRow [codegen id : 1] -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(75) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(76) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) - -(79) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#25, d_year#26] - -(80) BroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#41 IN dynamicpruning#11 - -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#24 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/simplified.txt deleted file mode 100644 index 72e39422e0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - CometProject [customer_id,customer_first_name,customer_last_name] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 - CometHashAggregate [d_year,sum] [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/explain.txt deleted file mode 100644 index b0ae8bb695..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/explain.txt +++ /dev/null @@ -1,470 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometProject (53) - : +- CometBroadcastHashJoin (52) - : :- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (51) - : +- CometFilter (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : +- CometBroadcastExchange (41) - : : +- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) - : +- ReusedExchange (44) - +- CometBroadcastExchange (68) - +- CometHashAggregate (67) - +- CometExchange (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometFilter (55) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (62) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Condition : isnotnull(ss_customer_sk#8) - -(6) CometBroadcastExchange -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_sk#1], [ss_customer_sk#8], Inner, BuildRight - -(8) CometProject -Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(12) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#10], [d_date_sk#12], Inner, BuildRight - -(13) CometProject -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] - -(14) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] - -(15) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] - -(17) CometFilter -Input [2]: [customer_id#15, year_total#16] -Condition : (isnotnull(year_total#16) AND (year_total#16 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true))) - -(20) CometProject -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Arguments: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#19, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#20, 30, true, false, true) AS c_last_name#7] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_customer_sk#21) - -(23) CometBroadcastExchange -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] - -(24) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [c_customer_sk#17], [ss_customer_sk#21], Inner, BuildRight - -(25) CometProject -Input [7]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: [d_date_sk#25, d_year#26] - -(29) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] -Right output [2]: [d_date_sk#25, d_year#26] -Arguments: [ss_sold_date_sk#23], [d_date_sk#25], Inner, BuildRight - -(30) CometProject -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23, d_date_sk#25, d_year#26] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] - -(31) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] - -(32) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] -Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] - -(34) CometBroadcastExchange -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#15, year_total#16] -Right output [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#15], [customer_id#28], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true))) - -(38) CometProject -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Arguments: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#34, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#35, 30, true, false, true) AS c_last_name#38] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(ws_sold_date_sk#41 IN dynamicpruning#42)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Condition : isnotnull(ws_bill_customer_sk#39) - -(41) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] - -(42) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38] -Right output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [c_customer_sk#32], [ws_bill_customer_sk#39], Inner, BuildRight - -(43) CometProject -Input [7]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#43, d_year#44] - -(45) CometBroadcastHashJoin -Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] -Right output [2]: [d_date_sk#43, d_year#44] -Arguments: [ws_sold_date_sk#41], [d_date_sk#43], Inner, BuildRight - -(46) CometProject -Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41, d_date_sk#43, d_year#44] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] - -(47) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#40))] - -(48) CometExchange -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] -Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] - -(50) CometFilter -Input [2]: [customer_id#46, year_total#47] -Condition : (isnotnull(year_total#47) AND (year_total#47 > 0.00)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#46, year_total#47] -Arguments: [customer_id#46, year_total#47] - -(52) CometBroadcastHashJoin -Left output [6]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Right output [2]: [customer_id#46, year_total#47] -Arguments: [customer_id#15], [customer_id#46], Inner, BuildRight - -(53) CometProject -Input [8]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#46, year_total#47] -Arguments: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47], [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Condition : (isnotnull(c_customer_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true))) - -(56) CometProject -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Arguments: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#50, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#51, 30, true, false, true) AS c_last_name#38] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#55)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Condition : isnotnull(ws_bill_customer_sk#52) - -(59) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] - -(60) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38] -Right output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [c_customer_sk#48], [ws_bill_customer_sk#52], Inner, BuildRight - -(61) CometProject -Input [7]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#56, d_year#57] - -(63) CometBroadcastHashJoin -Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] -Right output [2]: [d_date_sk#56, d_year#57] -Arguments: [ws_sold_date_sk#54], [d_date_sk#56], Inner, BuildRight - -(64) CometProject -Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54, d_date_sk#56, d_year#57] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] - -(65) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#53))] - -(66) CometExchange -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] -Functions [1]: [sum(UnscaledValue(ws_net_paid#53))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#59, year_total#60] -Arguments: [customer_id#59, year_total#60] - -(69) CometBroadcastHashJoin -Left output [7]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] -Right output [2]: [customer_id#59, year_total#60] -Arguments: [customer_id#15], [customer_id#59], Inner, (CASE WHEN (year_total#47 > 0.00) THEN (year_total#60 / year_total#47) END > CASE WHEN (year_total#16 > 0.00) THEN (year_total#31 / year_total#16) END), BuildRight - -(70) CometProject -Input [9]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47, customer_id#59, year_total#60] -Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30], [customer_id#28, customer_first_name#29, customer_last_name#30] - -(71) CometTakeOrderedAndProject -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_first_name#29 ASC NULLS FIRST,customer_id#28 ASC NULLS FIRST,customer_last_name#30 ASC NULLS FIRST], output=[customer_id#28,customer_first_name#29,customer_last_name#30]), [customer_id#28, customer_first_name#29, customer_last_name#30], 100, 0, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_last_name#30 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] - -(72) CometColumnarToRow [codegen id : 1] -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(75) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(76) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) - -(79) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#25, d_year#26] - -(80) BroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#41 IN dynamicpruning#11 - -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#24 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/extended.txt deleted file mode 100644 index 476c7be954..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/extended.txt +++ /dev/null @@ -1,90 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 72e39422e0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - CometProject [customer_id,customer_first_name,customer_last_name] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 - CometHashAggregate [d_year,sum] [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/explain.txt deleted file mode 100644 index eddee4d8e5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/explain.txt +++ /dev/null @@ -1,754 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (127) -+- CometTakeOrderedAndProject (126) - +- CometProject (125) - +- CometSortMergeJoin (124) - :- CometSort (68) - : +- CometExchange (67) - : +- CometFilter (66) - : +- CometHashAggregate (65) - : +- CometExchange (64) - : +- CometHashAggregate (63) - : +- CometHashAggregate (62) - : +- CometExchange (61) - : +- CometHashAggregate (60) - : +- CometUnion (59) - : :- CometProject (22) - : : +- CometSortMergeJoin (21) - : : :- CometSort (15) - : : : +- CometColumnarExchange (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : : +- ReusedExchange (11) - : : +- CometSort (20) - : : +- CometExchange (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (40) - : : +- CometSortMergeJoin (39) - : : :- CometSort (33) - : : : +- CometColumnarExchange (32) - : : : +- * Project (31) - : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : :- * Project (28) - : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : :- * Filter (25) - : : : : : +- * ColumnarToRow (24) - : : : : : +- Scan parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (26) - : : : +- ReusedExchange (29) - : : +- CometSort (38) - : : +- CometExchange (37) - : : +- CometProject (36) - : : +- CometFilter (35) - : : +- CometNativeScan parquet spark_catalog.default.store_returns (34) - : +- CometProject (58) - : +- CometSortMergeJoin (57) - : :- CometSort (51) - : : +- CometColumnarExchange (50) - : : +- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * Project (46) - : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : :- * Filter (43) - : : : : +- * ColumnarToRow (42) - : : : : +- Scan parquet spark_catalog.default.web_sales (41) - : : : +- ReusedExchange (44) - : : +- ReusedExchange (47) - : +- CometSort (56) - : +- CometExchange (55) - : +- CometProject (54) - : +- CometFilter (53) - : +- CometNativeScan parquet spark_catalog.default.web_returns (52) - +- CometSort (123) - +- CometExchange (122) - +- CometFilter (121) - +- CometHashAggregate (120) - +- CometExchange (119) - +- CometHashAggregate (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometUnion (114) - :- CometProject (83) - : +- CometSortMergeJoin (82) - : :- CometSort (79) - : : +- CometColumnarExchange (78) - : : +- * Project (77) - : : +- * BroadcastHashJoin Inner BuildRight (76) - : : :- * Project (74) - : : : +- * BroadcastHashJoin Inner BuildRight (73) - : : : :- * Filter (71) - : : : : +- * ColumnarToRow (70) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (69) - : : : +- ReusedExchange (72) - : : +- ReusedExchange (75) - : +- CometSort (81) - : +- ReusedExchange (80) - :- CometProject (98) - : +- CometSortMergeJoin (97) - : :- CometSort (94) - : : +- CometColumnarExchange (93) - : : +- * Project (92) - : : +- * BroadcastHashJoin Inner BuildRight (91) - : : :- * Project (89) - : : : +- * BroadcastHashJoin Inner BuildRight (88) - : : : :- * Filter (86) - : : : : +- * ColumnarToRow (85) - : : : : +- Scan parquet spark_catalog.default.store_sales (84) - : : : +- ReusedExchange (87) - : : +- ReusedExchange (90) - : +- CometSort (96) - : +- ReusedExchange (95) - +- CometProject (113) - +- CometSortMergeJoin (112) - :- CometSort (109) - : +- CometColumnarExchange (108) - : +- * Project (107) - : +- * BroadcastHashJoin Inner BuildRight (106) - : :- * Project (104) - : : +- * BroadcastHashJoin Inner BuildRight (103) - : : :- * Filter (101) - : : : +- * ColumnarToRow (100) - : : : +- Scan parquet spark_catalog.default.web_sales (99) - : : +- ReusedExchange (102) - : +- ReusedExchange (105) - +- CometSort (111) - +- ReusedExchange (110) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] - -(3) Filter [codegen id : 3] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) = Books ) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) - -(6) CometProject -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(7) CometColumnarToRow [codegen id : 1] -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(8) BroadcastExchange -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(11) ReusedExchange [Reuses operator id: 131] -Output [2]: [d_date_sk#13, d_year#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] - -(14) CometColumnarExchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(15) CometSort -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] - -(16) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) - -(18) CometProject -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(19) CometExchange -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter - -(22) CometProject -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21], [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] - -(23) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 6] -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] - -(25) Filter [codegen id : 6] -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#22) - -(26) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#22] -Right keys [1]: [i_item_sk#27] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] -Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] - -(29) ReusedExchange [Reuses operator id: 131] -Output [2]: [d_date_sk#32, d_year#33] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#32] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_date_sk#32, d_year#33] - -(32) CometColumnarExchange -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(33) CometSort -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33], [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] - -(34) CometNativeScan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37, sr_returned_date_sk#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [5]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37, sr_returned_date_sk#38] -Condition : (isnotnull(sr_ticket_number#35) AND isnotnull(sr_item_sk#34)) - -(36) CometProject -Input [5]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37, sr_returned_date_sk#38] -Arguments: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37], [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] - -(37) CometExchange -Input [4]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: hashpartitioning(sr_ticket_number#35, sr_item_sk#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(38) CometSort -Input [4]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37], [sr_ticket_number#35 ASC NULLS FIRST, sr_item_sk#34 ASC NULLS FIRST] - -(39) CometSortMergeJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Right output [4]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#35, sr_item_sk#34], LeftOuter - -(40) CometProject -Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33, sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: [d_year#33, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, sales_cnt#39, sales_amt#40], [d_year#33, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, (ss_quantity#24 - coalesce(sr_return_quantity#36, 0)) AS sales_cnt#39, (ss_ext_sales_price#25 - coalesce(sr_return_amt#37, 0.00)) AS sales_amt#40] - -(41) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(42) ColumnarToRow [codegen id : 9] -Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] - -(43) Filter [codegen id : 9] -Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -Condition : isnotnull(ws_item_sk#41) - -(44) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] - -(45) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#41] -Right keys [1]: [i_item_sk#46] -Join type: Inner -Join condition: None - -(46) Project [codegen id : 9] -Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] -Input [10]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] - -(47) ReusedExchange [Reuses operator id: 131] -Output [2]: [d_date_sk#51, d_year#52] - -(48) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#45] -Right keys [1]: [d_date_sk#51] -Join type: Inner -Join condition: None - -(49) Project [codegen id : 9] -Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Input [11]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_date_sk#51, d_year#52] - -(50) CometColumnarExchange -Input [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Arguments: hashpartitioning(ws_order_number#42, ws_item_sk#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(51) CometSort -Input [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Arguments: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52], [ws_order_number#42 ASC NULLS FIRST, ws_item_sk#41 ASC NULLS FIRST] - -(52) CometNativeScan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56, wr_returned_date_sk#57] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(53) CometFilter -Input [5]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56, wr_returned_date_sk#57] -Condition : (isnotnull(wr_order_number#54) AND isnotnull(wr_item_sk#53)) - -(54) CometProject -Input [5]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56, wr_returned_date_sk#57] -Arguments: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56], [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] - -(55) CometExchange -Input [4]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: hashpartitioning(wr_order_number#54, wr_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(56) CometSort -Input [4]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56], [wr_order_number#54 ASC NULLS FIRST, wr_item_sk#53 ASC NULLS FIRST] - -(57) CometSortMergeJoin -Left output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Right output [4]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: [ws_order_number#42, ws_item_sk#41], [wr_order_number#54, wr_item_sk#53], LeftOuter - -(58) CometProject -Input [13]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52, wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: [d_year#52, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, sales_cnt#58, sales_amt#59], [d_year#52, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, (ws_quantity#43 - coalesce(wr_return_quantity#55, 0)) AS sales_cnt#58, (ws_ext_sales_price#44 - coalesce(wr_return_amt#56, 0.00)) AS sales_amt#59] - -(59) CometUnion -Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#33, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, sales_cnt#39, sales_amt#40] -Child 2 Input [7]: [d_year#52, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, sales_cnt#58, sales_amt#59] - -(60) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(61) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(62) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(63) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(64) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#60, sum#61] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(65) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#60, sum#61] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(66) CometFilter -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Condition : isnotnull(sales_cnt#62) - -(67) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(68) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] - -(69) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#68), dynamicpruningexpression(cs_sold_date_sk#68 IN dynamicpruning#69)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(70) ColumnarToRow [codegen id : 12] -Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] - -(71) Filter [codegen id : 12] -Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] -Condition : isnotnull(cs_item_sk#64) - -(72) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] - -(73) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cs_item_sk#64] -Right keys [1]: [i_item_sk#70] -Join type: Inner -Join condition: None - -(74) Project [codegen id : 12] -Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -Input [10]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] - -(75) ReusedExchange [Reuses operator id: 135] -Output [2]: [d_date_sk#75, d_year#76] - -(76) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cs_sold_date_sk#68] -Right keys [1]: [d_date_sk#75] -Join type: Inner -Join condition: None - -(77) Project [codegen id : 12] -Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Input [11]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_date_sk#75, d_year#76] - -(78) CometColumnarExchange -Input [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Arguments: hashpartitioning(cs_order_number#65, cs_item_sk#64, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(79) CometSort -Input [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Arguments: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76], [cs_order_number#65 ASC NULLS FIRST, cs_item_sk#64 ASC NULLS FIRST] - -(80) ReusedExchange [Reuses operator id: 19] -Output [4]: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] - -(81) CometSort -Input [4]: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] -Arguments: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80], [cr_order_number#78 ASC NULLS FIRST, cr_item_sk#77 ASC NULLS FIRST] - -(82) CometSortMergeJoin -Left output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Right output [4]: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] -Arguments: [cs_order_number#65, cs_item_sk#64], [cr_order_number#78, cr_item_sk#77], LeftOuter - -(83) CometProject -Input [13]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76, cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] -Arguments: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21], [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, (cs_quantity#66 - coalesce(cr_return_quantity#79, 0)) AS sales_cnt#20, (cs_ext_sales_price#67 - coalesce(cr_return_amount#80, 0.00)) AS sales_amt#21] - -(84) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#85), dynamicpruningexpression(ss_sold_date_sk#85 IN dynamicpruning#69)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(85) ColumnarToRow [codegen id : 15] -Input [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85] - -(86) Filter [codegen id : 15] -Input [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85] -Condition : isnotnull(ss_item_sk#81) - -(87) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] - -(88) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ss_item_sk#81] -Right keys [1]: [i_item_sk#86] -Join type: Inner -Join condition: None - -(89) Project [codegen id : 15] -Output [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] -Input [10]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] - -(90) ReusedExchange [Reuses operator id: 135] -Output [2]: [d_date_sk#91, d_year#92] - -(91) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ss_sold_date_sk#85] -Right keys [1]: [d_date_sk#91] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 15] -Output [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92] -Input [11]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_date_sk#91, d_year#92] - -(93) CometColumnarExchange -Input [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92] -Arguments: hashpartitioning(ss_ticket_number#82, ss_item_sk#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(94) CometSort -Input [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92] -Arguments: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92], [ss_ticket_number#82 ASC NULLS FIRST, ss_item_sk#81 ASC NULLS FIRST] - -(95) ReusedExchange [Reuses operator id: 37] -Output [4]: [sr_item_sk#93, sr_ticket_number#94, sr_return_quantity#95, sr_return_amt#96] - -(96) CometSort -Input [4]: [sr_item_sk#93, sr_ticket_number#94, sr_return_quantity#95, sr_return_amt#96] -Arguments: [sr_item_sk#93, sr_ticket_number#94, sr_return_quantity#95, sr_return_amt#96], [sr_ticket_number#94 ASC NULLS FIRST, sr_item_sk#93 ASC NULLS FIRST] - -(97) CometSortMergeJoin -Left output [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92] -Right output [4]: [sr_item_sk#93, sr_ticket_number#94, sr_return_quantity#95, sr_return_amt#96] -Arguments: [ss_ticket_number#82, ss_item_sk#81], [sr_ticket_number#94, sr_item_sk#93], LeftOuter - -(98) CometProject -Input [13]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92, sr_item_sk#93, sr_ticket_number#94, sr_return_quantity#95, sr_return_amt#96] -Arguments: [d_year#92, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, sales_cnt#39, sales_amt#40], [d_year#92, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, (ss_quantity#83 - coalesce(sr_return_quantity#95, 0)) AS sales_cnt#39, (ss_ext_sales_price#84 - coalesce(sr_return_amt#96, 0.00)) AS sales_amt#40] - -(99) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#101), dynamicpruningexpression(ws_sold_date_sk#101 IN dynamicpruning#69)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(100) ColumnarToRow [codegen id : 18] -Input [5]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101] - -(101) Filter [codegen id : 18] -Input [5]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101] -Condition : isnotnull(ws_item_sk#97) - -(102) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#102, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] - -(103) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_item_sk#97] -Right keys [1]: [i_item_sk#102] -Join type: Inner -Join condition: None - -(104) Project [codegen id : 18] -Output [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] -Input [10]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_item_sk#102, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] - -(105) ReusedExchange [Reuses operator id: 135] -Output [2]: [d_date_sk#107, d_year#108] - -(106) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#101] -Right keys [1]: [d_date_sk#107] -Join type: Inner -Join condition: None - -(107) Project [codegen id : 18] -Output [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108] -Input [11]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_date_sk#107, d_year#108] - -(108) CometColumnarExchange -Input [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108] -Arguments: hashpartitioning(ws_order_number#98, ws_item_sk#97, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(109) CometSort -Input [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108] -Arguments: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108], [ws_order_number#98 ASC NULLS FIRST, ws_item_sk#97 ASC NULLS FIRST] - -(110) ReusedExchange [Reuses operator id: 55] -Output [4]: [wr_item_sk#109, wr_order_number#110, wr_return_quantity#111, wr_return_amt#112] - -(111) CometSort -Input [4]: [wr_item_sk#109, wr_order_number#110, wr_return_quantity#111, wr_return_amt#112] -Arguments: [wr_item_sk#109, wr_order_number#110, wr_return_quantity#111, wr_return_amt#112], [wr_order_number#110 ASC NULLS FIRST, wr_item_sk#109 ASC NULLS FIRST] - -(112) CometSortMergeJoin -Left output [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108] -Right output [4]: [wr_item_sk#109, wr_order_number#110, wr_return_quantity#111, wr_return_amt#112] -Arguments: [ws_order_number#98, ws_item_sk#97], [wr_order_number#110, wr_item_sk#109], LeftOuter - -(113) CometProject -Input [13]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108, wr_item_sk#109, wr_order_number#110, wr_return_quantity#111, wr_return_amt#112] -Arguments: [d_year#108, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, sales_cnt#58, sales_amt#59], [d_year#108, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, (ws_quantity#99 - coalesce(wr_return_quantity#111, 0)) AS sales_cnt#58, (ws_ext_sales_price#100 - coalesce(wr_return_amt#112, 0.00)) AS sales_amt#59] - -(114) CometUnion -Child 0 Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#92, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, sales_cnt#39, sales_amt#40] -Child 2 Input [7]: [d_year#108, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, sales_cnt#58, sales_amt#59] - -(115) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Functions: [] - -(116) CometExchange -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(117) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Functions: [] - -(118) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(119) CometExchange -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sum#60, sum#113] -Arguments: hashpartitioning(d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(120) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sum#60, sum#113] -Keys [5]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(121) CometFilter -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115] -Condition : isnotnull(sales_cnt#114) - -(122) CometExchange -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115] -Arguments: hashpartitioning(i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=16] - -(123) CometSort -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115] -Arguments: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115], [i_brand_id#71 ASC NULLS FIRST, i_class_id#72 ASC NULLS FIRST, i_category_id#73 ASC NULLS FIRST, i_manufact_id#74 ASC NULLS FIRST] - -(124) CometSortMergeJoin -Left output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Right output [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115] -Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74], Inner, ((cast(sales_cnt#62 as decimal(17,2)) / cast(sales_cnt#114 as decimal(17,2))) < 0.90000000000000000000) - -(125) CometProject -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115] -Arguments: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121], [d_year#76 AS prev_year#116, d_year#14 AS year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#114 AS prev_yr_cnt#118, sales_cnt#62 AS curr_yr_cnt#119, (sales_cnt#62 - sales_cnt#114) AS sales_cnt_diff#120, (sales_amt#63 - sales_amt#115) AS sales_amt_diff#121] - -(126) CometTakeOrderedAndProject -Input [10]: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#120 ASC NULLS FIRST,sales_amt_diff#121 ASC NULLS FIRST], output=[prev_year#116,year#117,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#118,curr_yr_cnt#119,sales_cnt_diff#120,sales_amt_diff#121]), [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121], 100, 0, [sales_cnt_diff#120 ASC NULLS FIRST, sales_amt_diff#121 ASC NULLS FIRST], [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] - -(127) CometColumnarToRow [codegen id : 19] -Input [10]: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (131) -+- * CometColumnarToRow (130) - +- CometFilter (129) - +- CometNativeScan parquet spark_catalog.default.date_dim (128) - - -(128) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(129) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(130) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_year#14] - -(131) BroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#6 - -Subquery:4 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#68 IN dynamicpruning#69 -BroadcastExchange (135) -+- * CometColumnarToRow (134) - +- CometFilter (133) - +- CometNativeScan parquet spark_catalog.default.date_dim (132) - - -(132) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#75, d_year#76] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(133) CometFilter -Input [2]: [d_date_sk#75, d_year#76] -Condition : ((isnotnull(d_year#76) AND (d_year#76 = 2001)) AND isnotnull(d_date_sk#75)) - -(134) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#75, d_year#76] - -(135) BroadcastExchange -Input [2]: [d_date_sk#75, d_year#76] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] - -Subquery:5 Hosting operator id = 84 Hosting Expression = ss_sold_date_sk#85 IN dynamicpruning#69 - -Subquery:6 Hosting operator id = 99 Hosting Expression = ws_sold_date_sk#101 IN dynamicpruning#69 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/simplified.txt deleted file mode 100644 index aaf98f4b09..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/simplified.txt +++ /dev/null @@ -1,173 +0,0 @@ -WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometColumnarExchange [cs_order_number,cs_item_sk] #4 - WholeStageCodegen (3) - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [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] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #7 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 - WholeStageCodegen (6) - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #9 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometColumnarExchange [ws_order_number,ws_item_sk] #10 - WholeStageCodegen (9) - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #11 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometColumnarExchange [cs_order_number,cs_item_sk] #15 - WholeStageCodegen (12) - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 #16 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 - WholeStageCodegen (15) - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometColumnarExchange [ws_order_number,ws_item_sk] #18 - WholeStageCodegen (18) - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/explain.txt deleted file mode 100644 index 6925eb301e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,742 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (124) -+- CometTakeOrderedAndProject (123) - +- CometProject (122) - +- CometSortMergeJoin (121) - :- CometSort (66) - : +- CometExchange (65) - : +- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometHashAggregate (60) - : +- CometExchange (59) - : +- CometHashAggregate (58) - : +- CometUnion (57) - : :- CometProject (22) - : : +- CometSortMergeJoin (21) - : : :- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometSort (20) - : : +- CometExchange (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (39) - : : +- CometSortMergeJoin (38) - : : :- CometSort (32) - : : : +- CometExchange (31) - : : : +- CometProject (30) - : : : +- CometBroadcastHashJoin (29) - : : : :- CometProject (27) - : : : : +- CometBroadcastHashJoin (26) - : : : : :- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (25) - : : : +- ReusedExchange (28) - : : +- CometSort (37) - : : +- CometExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (33) - : +- CometProject (56) - : +- CometSortMergeJoin (55) - : :- CometSort (49) - : : +- CometExchange (48) - : : +- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometProject (44) - : : : +- CometBroadcastHashJoin (43) - : : : :- CometFilter (41) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) - : : : +- ReusedExchange (42) - : : +- ReusedExchange (45) - : +- CometSort (54) - : +- CometExchange (53) - : +- CometProject (52) - : +- CometFilter (51) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (50) - +- CometSort (120) - +- CometExchange (119) - +- CometFilter (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometHashAggregate (114) - +- CometExchange (113) - +- CometHashAggregate (112) - +- CometUnion (111) - :- CometProject (82) - : +- CometSortMergeJoin (81) - : :- CometSort (78) - : : +- CometExchange (77) - : : +- CometProject (76) - : : +- CometBroadcastHashJoin (75) - : : :- CometProject (71) - : : : +- CometBroadcastHashJoin (70) - : : : :- CometFilter (68) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) - : : : +- ReusedExchange (69) - : : +- CometBroadcastExchange (74) - : : +- CometFilter (73) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - : +- CometSort (80) - : +- ReusedExchange (79) - :- CometProject (96) - : +- CometSortMergeJoin (95) - : :- CometSort (92) - : : +- CometExchange (91) - : : +- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (87) - : : : +- CometBroadcastHashJoin (86) - : : : :- CometFilter (84) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (83) - : : : +- ReusedExchange (85) - : : +- ReusedExchange (88) - : +- CometSort (94) - : +- ReusedExchange (93) - +- CometProject (110) - +- CometSortMergeJoin (109) - :- CometSort (106) - : +- CometExchange (105) - : +- CometProject (104) - : +- CometBroadcastHashJoin (103) - : :- CometProject (101) - : : +- CometBroadcastHashJoin (100) - : : :- CometFilter (98) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (97) - : : +- ReusedExchange (99) - : +- ReusedExchange (102) - +- CometSort (108) - +- ReusedExchange (107) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) = Books ) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) - -(5) CometProject -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(6) CometBroadcastExchange -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(7) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Right output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: [d_date_sk#13, d_year#14] - -(12) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Right output [2]: [d_date_sk#13, d_year#14] -Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight - -(13) CometProject -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] - -(14) CometExchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(15) CometSort -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) - -(18) CometProject -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(19) CometExchange -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter - -(22) CometProject -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21], [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(24) CometFilter -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#22) - -(25) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(26) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Right output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22], [i_item_sk#28], Inner, BuildRight - -(27) CometProject -Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(28) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#33, d_year#34] - -(29) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Right output [2]: [d_date_sk#33, d_year#34] -Arguments: [ss_sold_date_sk#26], [d_date_sk#33], Inner, BuildRight - -(30) CometProject -Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_date_sk#33, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] - -(31) CometExchange -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometSort -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) - -(35) CometProject -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] - -(36) CometExchange -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(37) CometSort -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] - -(38) CometSortMergeJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter - -(39) CometProject -Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#47)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(41) CometFilter -Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Condition : isnotnull(ws_item_sk#42) - -(42) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(43) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Right output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42], [i_item_sk#48], Inner, BuildRight - -(44) CometProject -Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(45) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#53, d_year#54] - -(46) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [ws_sold_date_sk#46], [d_date_sk#53], Inner, BuildRight - -(47) CometProject -Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] - -(48) CometExchange -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(49) CometSort -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) - -(52) CometProject -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] - -(53) CometExchange -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(54) CometSort -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] - -(55) CometSortMergeJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter - -(56) CometProject -Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] - -(57) CometUnion -Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] - -(58) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(59) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(60) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(61) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(62) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(63) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(64) CometFilter -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Condition : isnotnull(sales_cnt#64) - -(65) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(66) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(68) CometFilter -Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Condition : isnotnull(cs_item_sk#66) - -(69) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(70) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Right output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66], [i_item_sk#72], Inner, BuildRight - -(71) CometProject -Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(74) CometBroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: [d_date_sk#77, d_year#78] - -(75) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Right output [2]: [d_date_sk#77, d_year#78] -Arguments: [cs_sold_date_sk#70], [d_date_sk#77], Inner, BuildRight - -(76) CometProject -Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] - -(77) CometExchange -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(78) CometSort -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] - -(79) ReusedExchange [Reuses operator id: 19] -Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] - -(80) CometSort -Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] - -(81) CometSortMergeJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter - -(82) CometProject -Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#20, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#21] - -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(84) CometFilter -Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Condition : isnotnull(ss_item_sk#83) - -(85) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] - -(86) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Right output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Arguments: [ss_item_sk#83], [i_item_sk#89], Inner, BuildRight - -(87) CometProject -Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] - -(88) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#94, d_year#95] - -(89) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Right output [2]: [d_date_sk#94, d_year#95] -Arguments: [ss_sold_date_sk#87], [d_date_sk#94], Inner, BuildRight - -(90) CometProject -Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_date_sk#94, d_year#95] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] - -(91) CometExchange -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(92) CometSort -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_ticket_number#84 ASC NULLS FIRST, ss_item_sk#83 ASC NULLS FIRST] - -(93) ReusedExchange [Reuses operator id: 36] -Output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] - -(94) CometSort -Input [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99], [sr_ticket_number#97 ASC NULLS FIRST, sr_item_sk#96 ASC NULLS FIRST] - -(95) CometSortMergeJoin -Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Right output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [ss_ticket_number#84, ss_item_sk#83], [sr_ticket_number#97, sr_item_sk#96], LeftOuter - -(96) CometProject -Input [13]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95, sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41], [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, (ss_quantity#85 - coalesce(sr_return_quantity#98, 0)) AS sales_cnt#40, (ss_ext_sales_price#86 - coalesce(sr_return_amt#99, 0.00)) AS sales_amt#41] - -(97) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_sold_date_sk#104 IN dynamicpruning#105)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(98) CometFilter -Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Condition : isnotnull(ws_item_sk#100) - -(99) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] - -(100) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Right output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Arguments: [ws_item_sk#100], [i_item_sk#106], Inner, BuildRight - -(101) CometProject -Input [10]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] - -(102) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#111, d_year#112] - -(103) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Right output [2]: [d_date_sk#111, d_year#112] -Arguments: [ws_sold_date_sk#104], [d_date_sk#111], Inner, BuildRight - -(104) CometProject -Input [11]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] - -(105) CometExchange -Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(106) CometSort -Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_order_number#101 ASC NULLS FIRST, ws_item_sk#100 ASC NULLS FIRST] - -(107) ReusedExchange [Reuses operator id: 53] -Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] - -(108) CometSort -Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116], [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST] - -(109) CometSortMergeJoin -Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Right output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [ws_order_number#101, ws_item_sk#100], [wr_order_number#114, wr_item_sk#113], LeftOuter - -(110) CometProject -Input [13]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61], [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#102 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#60, (ws_ext_sales_price#103 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#61] - -(111) CometUnion -Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61] - -(112) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Functions: [] - -(113) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] - -(114) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Functions: [] - -(115) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(116) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(117) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(118) CometFilter -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Condition : isnotnull(sales_cnt#118) - -(119) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(120) CometSort -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] - -(121) CometSortMergeJoin -Left output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Right output [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], Inner, ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#118 as decimal(17,2))) < 0.90000000000000000000) - -(122) CometProject -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], [d_year#78 AS prev_year#120, d_year#14 AS year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#118 AS prev_yr_cnt#122, sales_cnt#64 AS curr_yr_cnt#123, (sales_cnt#64 - sales_cnt#118) AS sales_cnt_diff#124, (sales_amt#65 - sales_amt#119) AS sales_amt_diff#125] - -(123) CometTakeOrderedAndProject -Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#124 ASC NULLS FIRST,sales_amt_diff#125 ASC NULLS FIRST], output=[prev_year#120,year#121,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#122,curr_yr_cnt#123,sales_cnt_diff#124,sales_amt_diff#125]), [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], 100, 0, [sales_cnt_diff#124 ASC NULLS FIRST, sales_amt_diff#125 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] - -(124) CometColumnarToRow [codegen id : 1] -Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (128) -+- * CometColumnarToRow (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) - - -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(127) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_year#14] - -(128) BroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 - -Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 -BroadcastExchange (132) -+- * CometColumnarToRow (131) - +- CometFilter (130) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) - - -(129) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(130) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(131) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#77, d_year#78] - -(132) BroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] - -Subquery:5 Hosting operator id = 83 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 - -Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/simplified.txt deleted file mode 100644 index d0520c8b63..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,144 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #4 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange [d_date_sk,d_year] #7 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #8 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #9 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #10 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #11 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #12 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #16 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_year] #18 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #19 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #20 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/explain.txt deleted file mode 100644 index 6925eb301e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/explain.txt +++ /dev/null @@ -1,742 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (124) -+- CometTakeOrderedAndProject (123) - +- CometProject (122) - +- CometSortMergeJoin (121) - :- CometSort (66) - : +- CometExchange (65) - : +- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometHashAggregate (60) - : +- CometExchange (59) - : +- CometHashAggregate (58) - : +- CometUnion (57) - : :- CometProject (22) - : : +- CometSortMergeJoin (21) - : : :- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometSort (20) - : : +- CometExchange (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (39) - : : +- CometSortMergeJoin (38) - : : :- CometSort (32) - : : : +- CometExchange (31) - : : : +- CometProject (30) - : : : +- CometBroadcastHashJoin (29) - : : : :- CometProject (27) - : : : : +- CometBroadcastHashJoin (26) - : : : : :- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (25) - : : : +- ReusedExchange (28) - : : +- CometSort (37) - : : +- CometExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (33) - : +- CometProject (56) - : +- CometSortMergeJoin (55) - : :- CometSort (49) - : : +- CometExchange (48) - : : +- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometProject (44) - : : : +- CometBroadcastHashJoin (43) - : : : :- CometFilter (41) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) - : : : +- ReusedExchange (42) - : : +- ReusedExchange (45) - : +- CometSort (54) - : +- CometExchange (53) - : +- CometProject (52) - : +- CometFilter (51) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (50) - +- CometSort (120) - +- CometExchange (119) - +- CometFilter (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometHashAggregate (114) - +- CometExchange (113) - +- CometHashAggregate (112) - +- CometUnion (111) - :- CometProject (82) - : +- CometSortMergeJoin (81) - : :- CometSort (78) - : : +- CometExchange (77) - : : +- CometProject (76) - : : +- CometBroadcastHashJoin (75) - : : :- CometProject (71) - : : : +- CometBroadcastHashJoin (70) - : : : :- CometFilter (68) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) - : : : +- ReusedExchange (69) - : : +- CometBroadcastExchange (74) - : : +- CometFilter (73) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - : +- CometSort (80) - : +- ReusedExchange (79) - :- CometProject (96) - : +- CometSortMergeJoin (95) - : :- CometSort (92) - : : +- CometExchange (91) - : : +- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (87) - : : : +- CometBroadcastHashJoin (86) - : : : :- CometFilter (84) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (83) - : : : +- ReusedExchange (85) - : : +- ReusedExchange (88) - : +- CometSort (94) - : +- ReusedExchange (93) - +- CometProject (110) - +- CometSortMergeJoin (109) - :- CometSort (106) - : +- CometExchange (105) - : +- CometProject (104) - : +- CometBroadcastHashJoin (103) - : :- CometProject (101) - : : +- CometBroadcastHashJoin (100) - : : :- CometFilter (98) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (97) - : : +- ReusedExchange (99) - : +- ReusedExchange (102) - +- CometSort (108) - +- ReusedExchange (107) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) = Books ) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) - -(5) CometProject -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(6) CometBroadcastExchange -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(7) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Right output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: [d_date_sk#13, d_year#14] - -(12) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Right output [2]: [d_date_sk#13, d_year#14] -Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight - -(13) CometProject -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] - -(14) CometExchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(15) CometSort -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) - -(18) CometProject -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(19) CometExchange -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter - -(22) CometProject -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21], [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(24) CometFilter -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#22) - -(25) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(26) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Right output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22], [i_item_sk#28], Inner, BuildRight - -(27) CometProject -Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(28) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#33, d_year#34] - -(29) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Right output [2]: [d_date_sk#33, d_year#34] -Arguments: [ss_sold_date_sk#26], [d_date_sk#33], Inner, BuildRight - -(30) CometProject -Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_date_sk#33, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] - -(31) CometExchange -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometSort -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) - -(35) CometProject -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] - -(36) CometExchange -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(37) CometSort -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] - -(38) CometSortMergeJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter - -(39) CometProject -Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#47)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(41) CometFilter -Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Condition : isnotnull(ws_item_sk#42) - -(42) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(43) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Right output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42], [i_item_sk#48], Inner, BuildRight - -(44) CometProject -Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(45) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#53, d_year#54] - -(46) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [ws_sold_date_sk#46], [d_date_sk#53], Inner, BuildRight - -(47) CometProject -Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] - -(48) CometExchange -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(49) CometSort -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) - -(52) CometProject -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] - -(53) CometExchange -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(54) CometSort -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] - -(55) CometSortMergeJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter - -(56) CometProject -Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] - -(57) CometUnion -Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] - -(58) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(59) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(60) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(61) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(62) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(63) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(64) CometFilter -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Condition : isnotnull(sales_cnt#64) - -(65) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(66) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(68) CometFilter -Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Condition : isnotnull(cs_item_sk#66) - -(69) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(70) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Right output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66], [i_item_sk#72], Inner, BuildRight - -(71) CometProject -Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(74) CometBroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: [d_date_sk#77, d_year#78] - -(75) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Right output [2]: [d_date_sk#77, d_year#78] -Arguments: [cs_sold_date_sk#70], [d_date_sk#77], Inner, BuildRight - -(76) CometProject -Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] - -(77) CometExchange -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(78) CometSort -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] - -(79) ReusedExchange [Reuses operator id: 19] -Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] - -(80) CometSort -Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] - -(81) CometSortMergeJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter - -(82) CometProject -Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#20, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#21] - -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(84) CometFilter -Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Condition : isnotnull(ss_item_sk#83) - -(85) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] - -(86) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Right output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Arguments: [ss_item_sk#83], [i_item_sk#89], Inner, BuildRight - -(87) CometProject -Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] - -(88) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#94, d_year#95] - -(89) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Right output [2]: [d_date_sk#94, d_year#95] -Arguments: [ss_sold_date_sk#87], [d_date_sk#94], Inner, BuildRight - -(90) CometProject -Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_date_sk#94, d_year#95] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] - -(91) CometExchange -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(92) CometSort -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_ticket_number#84 ASC NULLS FIRST, ss_item_sk#83 ASC NULLS FIRST] - -(93) ReusedExchange [Reuses operator id: 36] -Output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] - -(94) CometSort -Input [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99], [sr_ticket_number#97 ASC NULLS FIRST, sr_item_sk#96 ASC NULLS FIRST] - -(95) CometSortMergeJoin -Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Right output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [ss_ticket_number#84, ss_item_sk#83], [sr_ticket_number#97, sr_item_sk#96], LeftOuter - -(96) CometProject -Input [13]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95, sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41], [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, (ss_quantity#85 - coalesce(sr_return_quantity#98, 0)) AS sales_cnt#40, (ss_ext_sales_price#86 - coalesce(sr_return_amt#99, 0.00)) AS sales_amt#41] - -(97) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_sold_date_sk#104 IN dynamicpruning#105)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(98) CometFilter -Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Condition : isnotnull(ws_item_sk#100) - -(99) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] - -(100) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Right output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Arguments: [ws_item_sk#100], [i_item_sk#106], Inner, BuildRight - -(101) CometProject -Input [10]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] - -(102) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#111, d_year#112] - -(103) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Right output [2]: [d_date_sk#111, d_year#112] -Arguments: [ws_sold_date_sk#104], [d_date_sk#111], Inner, BuildRight - -(104) CometProject -Input [11]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] - -(105) CometExchange -Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(106) CometSort -Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_order_number#101 ASC NULLS FIRST, ws_item_sk#100 ASC NULLS FIRST] - -(107) ReusedExchange [Reuses operator id: 53] -Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] - -(108) CometSort -Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116], [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST] - -(109) CometSortMergeJoin -Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Right output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [ws_order_number#101, ws_item_sk#100], [wr_order_number#114, wr_item_sk#113], LeftOuter - -(110) CometProject -Input [13]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61], [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#102 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#60, (ws_ext_sales_price#103 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#61] - -(111) CometUnion -Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61] - -(112) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Functions: [] - -(113) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] - -(114) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Functions: [] - -(115) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(116) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(117) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(118) CometFilter -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Condition : isnotnull(sales_cnt#118) - -(119) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(120) CometSort -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] - -(121) CometSortMergeJoin -Left output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Right output [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], Inner, ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#118 as decimal(17,2))) < 0.90000000000000000000) - -(122) CometProject -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], [d_year#78 AS prev_year#120, d_year#14 AS year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#118 AS prev_yr_cnt#122, sales_cnt#64 AS curr_yr_cnt#123, (sales_cnt#64 - sales_cnt#118) AS sales_cnt_diff#124, (sales_amt#65 - sales_amt#119) AS sales_amt_diff#125] - -(123) CometTakeOrderedAndProject -Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#124 ASC NULLS FIRST,sales_amt_diff#125 ASC NULLS FIRST], output=[prev_year#120,year#121,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#122,curr_yr_cnt#123,sales_cnt_diff#124,sales_amt_diff#125]), [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], 100, 0, [sales_cnt_diff#124 ASC NULLS FIRST, sales_amt_diff#125 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] - -(124) CometColumnarToRow [codegen id : 1] -Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (128) -+- * CometColumnarToRow (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) - - -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(127) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_year#14] - -(128) BroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 - -Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 -BroadcastExchange (132) -+- * CometColumnarToRow (131) - +- CometFilter (130) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) - - -(129) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(130) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(131) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#77, d_year#78] - -(132) BroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] - -Subquery:5 Hosting operator id = 83 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 - -Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/extended.txt deleted file mode 100644 index 22ccc17641..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/extended.txt +++ /dev/null @@ -1,172 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : +- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - +- CometSort - +- CometExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - -Comet accelerated 159 out of 167 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d0520c8b63..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75/simplified.txt +++ /dev/null @@ -1,144 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #4 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange [d_date_sk,d_year] #7 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #8 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #9 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #10 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #11 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #12 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #16 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_year] #18 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #19 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #20 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/explain.txt deleted file mode 100644 index 744f3a986d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/explain.txt +++ /dev/null @@ -1,678 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (110) -+- CometTakeOrderedAndProject (109) - +- CometHashAggregate (108) - +- CometColumnarExchange (107) - +- * HashAggregate (106) - +- Union (105) - :- * HashAggregate (90) - : +- * CometColumnarToRow (89) - : +- CometColumnarExchange (88) - : +- * HashAggregate (87) - : +- Union (86) - : :- * Project (32) - : : +- * BroadcastHashJoin LeftOuter BuildRight (31) - : : :- * HashAggregate (16) - : : : +- * CometColumnarToRow (15) - : : : +- CometColumnarExchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * CometColumnarToRow (9) - : : : +- CometFilter (8) - : : : +- CometNativeScan parquet spark_catalog.default.store (7) - : : +- BroadcastExchange (30) - : : +- * HashAggregate (29) - : : +- * CometColumnarToRow (28) - : : +- CometColumnarExchange (27) - : : +- * HashAggregate (26) - : : +- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * Filter (19) - : : : : +- * ColumnarToRow (18) - : : : : +- Scan parquet spark_catalog.default.store_returns (17) - : : : +- ReusedExchange (20) - : : +- ReusedExchange (23) - : :- * Project (53) - : : +- * BroadcastNestedLoopJoin Inner BuildLeft (52) - : : :- BroadcastExchange (42) - : : : +- * HashAggregate (41) - : : : +- * CometColumnarToRow (40) - : : : +- CometColumnarExchange (39) - : : : +- * HashAggregate (38) - : : : +- * Project (37) - : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : :- * ColumnarToRow (34) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (33) - : : : +- ReusedExchange (35) - : : +- * HashAggregate (51) - : : +- * CometColumnarToRow (50) - : : +- CometColumnarExchange (49) - : : +- * HashAggregate (48) - : : +- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * ColumnarToRow (44) - : : : +- Scan parquet spark_catalog.default.catalog_returns (43) - : : +- ReusedExchange (45) - : +- * Project (85) - : +- * BroadcastHashJoin LeftOuter BuildRight (84) - : :- * HashAggregate (69) - : : +- * CometColumnarToRow (68) - : : +- CometColumnarExchange (67) - : : +- * HashAggregate (66) - : : +- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildRight (58) - : : : :- * Filter (56) - : : : : +- * ColumnarToRow (55) - : : : : +- Scan parquet spark_catalog.default.web_sales (54) - : : : +- ReusedExchange (57) - : : +- BroadcastExchange (63) - : : +- * CometColumnarToRow (62) - : : +- CometFilter (61) - : : +- CometNativeScan parquet spark_catalog.default.web_page (60) - : +- BroadcastExchange (83) - : +- * HashAggregate (82) - : +- * CometColumnarToRow (81) - : +- CometColumnarExchange (80) - : +- * HashAggregate (79) - : +- * Project (78) - : +- * BroadcastHashJoin Inner BuildRight (77) - : :- * Project (75) - : : +- * BroadcastHashJoin Inner BuildRight (74) - : : :- * Filter (72) - : : : +- * ColumnarToRow (71) - : : : +- Scan parquet spark_catalog.default.web_returns (70) - : : +- ReusedExchange (73) - : +- ReusedExchange (76) - :- * HashAggregate (97) - : +- * CometColumnarToRow (96) - : +- CometColumnarExchange (95) - : +- * HashAggregate (94) - : +- * HashAggregate (93) - : +- * CometColumnarToRow (92) - : +- ReusedExchange (91) - +- * HashAggregate (104) - +- * CometColumnarToRow (103) - +- CometColumnarExchange (102) - +- * HashAggregate (101) - +- * HashAggregate (100) - +- * CometColumnarToRow (99) - +- ReusedExchange (98) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 3] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(4) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [1]: [s_store_sk#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [1]: [s_store_sk#7] -Condition : isnotnull(s_store_sk#7) - -(9) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#7] - -(10) BroadcastExchange -Input [1]: [s_store_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None - -(12) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] - -(13) HashAggregate [codegen id : 3] -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Keys [1]: [s_store_sk#7] -Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum#8, sum#9] -Results [3]: [s_store_sk#7, sum#10, sum#11] - -(14) CometColumnarExchange -Input [3]: [s_store_sk#7, sum#10, sum#11] -Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(15) CometColumnarToRow [codegen id : 8] -Input [3]: [s_store_sk#7, sum#10, sum#11] - -(16) HashAggregate [codegen id : 8] -Input [3]: [s_store_sk#7, sum#10, sum#11] -Keys [1]: [s_store_sk#7] -Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#12, sum(UnscaledValue(ss_net_profit#3))#13] -Results [3]: [s_store_sk#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS sales#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#13,17,2) AS profit#15] - -(17) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 6] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] - -(19) Filter [codegen id : 6] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_store_sk#16) - -(20) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#20] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_returned_date_sk#19] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] -Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#20] - -(23) ReusedExchange [Reuses operator id: 10] -Output [1]: [s_store_sk#21] - -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_store_sk#16] -Right keys [1]: [s_store_sk#21] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#21] - -(26) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] -Keys [1]: [s_store_sk#21] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum#22, sum#23] -Results [3]: [s_store_sk#21, sum#24, sum#25] - -(27) CometColumnarExchange -Input [3]: [s_store_sk#21, sum#24, sum#25] -Arguments: hashpartitioning(s_store_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#21, sum#24, sum#25] - -(29) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#21, sum#24, sum#25] -Keys [1]: [s_store_sk#21] -Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#17))#26, sum(UnscaledValue(sr_net_loss#18))#27] -Results [3]: [s_store_sk#21, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#26,17,2) AS returns#28, MakeDecimal(sum(UnscaledValue(sr_net_loss#18))#27,17,2) AS profit_loss#29] - -(30) BroadcastExchange -Input [3]: [s_store_sk#21, returns#28, profit_loss#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [s_store_sk#7] -Right keys [1]: [s_store_sk#21] -Join type: LeftOuter -Join condition: None - -(32) Project [codegen id : 8] -Output [5]: [store channel AS channel#30, s_store_sk#7 AS id#31, sales#14, coalesce(returns#28, 0.00) AS returns#32, (profit#15 - coalesce(profit_loss#29, 0.00)) AS profit#33] -Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#21, returns#28, profit_loss#29] - -(33) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#5)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37] - -(35) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#38] - -(36) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#37] -Right keys [1]: [d_date_sk#38] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] -Input [5]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37, d_date_sk#38] - -(38) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] -Keys [1]: [cs_call_center_sk#34] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#35)), partial_sum(UnscaledValue(cs_net_profit#36))] -Aggregate Attributes [2]: [sum#39, sum#40] -Results [3]: [cs_call_center_sk#34, sum#41, sum#42] - -(39) CometColumnarExchange -Input [3]: [cs_call_center_sk#34, sum#41, sum#42] -Arguments: hashpartitioning(cs_call_center_sk#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(40) CometColumnarToRow [codegen id : 11] -Input [3]: [cs_call_center_sk#34, sum#41, sum#42] - -(41) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#34, sum#41, sum#42] -Keys [1]: [cs_call_center_sk#34] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#35)), sum(UnscaledValue(cs_net_profit#36))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#35))#43, sum(UnscaledValue(cs_net_profit#36))#44] -Results [3]: [cs_call_center_sk#34, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#35))#43,17,2) AS sales#45, MakeDecimal(sum(UnscaledValue(cs_net_profit#36))#44,17,2) AS profit#46] - -(42) BroadcastExchange -Input [3]: [cs_call_center_sk#34, sales#45, profit#46] -Arguments: IdentityBroadcastMode, [plan_id=6] - -(43) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#49), dynamicpruningexpression(cr_returned_date_sk#49 IN dynamicpruning#5)] -ReadSchema: struct - -(44) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49] - -(45) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#50] - -(46) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#49] -Right keys [1]: [d_date_sk#50] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 13] -Output [2]: [cr_return_amount#47, cr_net_loss#48] -Input [4]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49, d_date_sk#50] - -(48) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#47, cr_net_loss#48] -Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#47)), partial_sum(UnscaledValue(cr_net_loss#48))] -Aggregate Attributes [2]: [sum#51, sum#52] -Results [2]: [sum#53, sum#54] - -(49) CometColumnarExchange -Input [2]: [sum#53, sum#54] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(50) CometColumnarToRow -Input [2]: [sum#53, sum#54] - -(51) HashAggregate -Input [2]: [sum#53, sum#54] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#47)), sum(UnscaledValue(cr_net_loss#48))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#47))#55, sum(UnscaledValue(cr_net_loss#48))#56] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#47))#55,17,2) AS returns#57, MakeDecimal(sum(UnscaledValue(cr_net_loss#48))#56,17,2) AS profit_loss#58] - -(52) BroadcastNestedLoopJoin [codegen id : 14] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 14] -Output [5]: [catalog channel AS channel#59, cs_call_center_sk#34 AS id#60, sales#45, returns#57, (profit#46 - profit_loss#58) AS profit#61] -Input [5]: [cs_call_center_sk#34, sales#45, profit#46, returns#57, profit_loss#58] - -(54) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] - -(56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] -Condition : isnotnull(ws_web_page_sk#62) - -(57) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#66] - -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#65] -Right keys [1]: [d_date_sk#66] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64] -Input [5]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65, d_date_sk#66] - -(60) CometNativeScan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(61) CometFilter -Input [1]: [wp_web_page_sk#67] -Condition : isnotnull(wp_web_page_sk#67) - -(62) CometColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#67] - -(63) BroadcastExchange -Input [1]: [wp_web_page_sk#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#62] -Right keys [1]: [wp_web_page_sk#67] -Join type: Inner -Join condition: None - -(65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] -Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] - -(66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] -Keys [1]: [wp_web_page_sk#67] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#63)), partial_sum(UnscaledValue(ws_net_profit#64))] -Aggregate Attributes [2]: [sum#68, sum#69] -Results [3]: [wp_web_page_sk#67, sum#70, sum#71] - -(67) CometColumnarExchange -Input [3]: [wp_web_page_sk#67, sum#70, sum#71] -Arguments: hashpartitioning(wp_web_page_sk#67, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(68) CometColumnarToRow [codegen id : 22] -Input [3]: [wp_web_page_sk#67, sum#70, sum#71] - -(69) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#67, sum#70, sum#71] -Keys [1]: [wp_web_page_sk#67] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#63)), sum(UnscaledValue(ws_net_profit#64))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#63))#72, sum(UnscaledValue(ws_net_profit#64))#73] -Results [3]: [wp_web_page_sk#67, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#63))#72,17,2) AS sales#74, MakeDecimal(sum(UnscaledValue(ws_net_profit#64))#73,17,2) AS profit#75] - -(70) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#79), dynamicpruningexpression(wr_returned_date_sk#79 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_web_page_sk)] -ReadSchema: struct - -(71) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] - -(72) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] -Condition : isnotnull(wr_web_page_sk#76) - -(73) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#80] - -(74) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#79] -Right keys [1]: [d_date_sk#80] -Join type: Inner -Join condition: None - -(75) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78] -Input [5]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79, d_date_sk#80] - -(76) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#81] - -(77) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#76] -Right keys [1]: [wp_web_page_sk#81] -Join type: Inner -Join condition: None - -(78) Project [codegen id : 20] -Output [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] -Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] - -(79) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] -Keys [1]: [wp_web_page_sk#81] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#77)), partial_sum(UnscaledValue(wr_net_loss#78))] -Aggregate Attributes [2]: [sum#82, sum#83] -Results [3]: [wp_web_page_sk#81, sum#84, sum#85] - -(80) CometColumnarExchange -Input [3]: [wp_web_page_sk#81, sum#84, sum#85] -Arguments: hashpartitioning(wp_web_page_sk#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(81) CometColumnarToRow [codegen id : 21] -Input [3]: [wp_web_page_sk#81, sum#84, sum#85] - -(82) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#81, sum#84, sum#85] -Keys [1]: [wp_web_page_sk#81] -Functions [2]: [sum(UnscaledValue(wr_return_amt#77)), sum(UnscaledValue(wr_net_loss#78))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#77))#86, sum(UnscaledValue(wr_net_loss#78))#87] -Results [3]: [wp_web_page_sk#81, MakeDecimal(sum(UnscaledValue(wr_return_amt#77))#86,17,2) AS returns#88, MakeDecimal(sum(UnscaledValue(wr_net_loss#78))#87,17,2) AS profit_loss#89] - -(83) BroadcastExchange -Input [3]: [wp_web_page_sk#81, returns#88, profit_loss#89] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -(84) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#67] -Right keys [1]: [wp_web_page_sk#81] -Join type: LeftOuter -Join condition: None - -(85) Project [codegen id : 22] -Output [5]: [web channel AS channel#90, wp_web_page_sk#67 AS id#91, sales#74, coalesce(returns#88, 0.00) AS returns#92, (profit#75 - coalesce(profit_loss#89, 0.00)) AS profit#93] -Input [6]: [wp_web_page_sk#67, sales#74, profit#75, wp_web_page_sk#81, returns#88, profit_loss#89] - -(86) Union - -(87) HashAggregate [codegen id : 23] -Input [5]: [channel#30, id#31, sales#14, returns#32, profit#33] -Keys [2]: [channel#30, id#31] -Functions [3]: [partial_sum(sales#14), partial_sum(returns#32), partial_sum(profit#33)] -Aggregate Attributes [6]: [sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] -Results [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] - -(88) CometColumnarExchange -Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Arguments: hashpartitioning(channel#30, id#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(89) CometColumnarToRow [codegen id : 24] -Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] - -(90) HashAggregate [codegen id : 24] -Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Keys [2]: [channel#30, id#31] -Functions [3]: [sum(sales#14), sum(returns#32), sum(profit#33)] -Aggregate Attributes [3]: [sum(sales#14)#106, sum(returns#32)#107, sum(profit#33)#108] -Results [5]: [channel#30, id#31, cast(sum(sales#14)#106 as decimal(37,2)) AS sales#109, cast(sum(returns#32)#107 as decimal(37,2)) AS returns#110, cast(sum(profit#33)#108 as decimal(38,2)) AS profit#111] - -(91) ReusedExchange [Reuses operator id: 88] -Output [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] - -(92) CometColumnarToRow [codegen id : 48] -Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] - -(93) HashAggregate [codegen id : 48] -Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Keys [2]: [channel#30, id#31] -Functions [3]: [sum(sales#14), sum(returns#32), sum(profit#33)] -Aggregate Attributes [3]: [sum(sales#14)#106, sum(returns#32)#107, sum(profit#33)#108] -Results [4]: [channel#30, sum(sales#14)#106 AS sales#112, sum(returns#32)#107 AS returns#113, sum(profit#33)#108 AS profit#114] - -(94) HashAggregate [codegen id : 48] -Input [4]: [channel#30, sales#112, returns#113, profit#114] -Keys [1]: [channel#30] -Functions [3]: [partial_sum(sales#112), partial_sum(returns#113), partial_sum(profit#114)] -Aggregate Attributes [6]: [sum#115, isEmpty#116, sum#117, isEmpty#118, sum#119, isEmpty#120] -Results [7]: [channel#30, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] - -(95) CometColumnarExchange -Input [7]: [channel#30, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] -Arguments: hashpartitioning(channel#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(96) CometColumnarToRow [codegen id : 49] -Input [7]: [channel#30, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] - -(97) HashAggregate [codegen id : 49] -Input [7]: [channel#30, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] -Keys [1]: [channel#30] -Functions [3]: [sum(sales#112), sum(returns#113), sum(profit#114)] -Aggregate Attributes [3]: [sum(sales#112)#127, sum(returns#113)#128, sum(profit#114)#129] -Results [5]: [channel#30, null AS id#130, sum(sales#112)#127 AS sales#131, sum(returns#113)#128 AS returns#132, sum(profit#114)#129 AS profit#133] - -(98) ReusedExchange [Reuses operator id: 88] -Output [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] - -(99) CometColumnarToRow [codegen id : 73] -Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] - -(100) HashAggregate [codegen id : 73] -Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Keys [2]: [channel#30, id#31] -Functions [3]: [sum(sales#14), sum(returns#32), sum(profit#33)] -Aggregate Attributes [3]: [sum(sales#14)#106, sum(returns#32)#107, sum(profit#33)#108] -Results [3]: [sum(sales#14)#106 AS sales#134, sum(returns#32)#107 AS returns#135, sum(profit#33)#108 AS profit#136] - -(101) HashAggregate [codegen id : 73] -Input [3]: [sales#134, returns#135, profit#136] -Keys: [] -Functions [3]: [partial_sum(sales#134), partial_sum(returns#135), partial_sum(profit#136)] -Aggregate Attributes [6]: [sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] -Results [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] - -(102) CometColumnarExchange -Input [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(103) CometColumnarToRow [codegen id : 74] -Input [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] - -(104) HashAggregate [codegen id : 74] -Input [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] -Keys: [] -Functions [3]: [sum(sales#134), sum(returns#135), sum(profit#136)] -Aggregate Attributes [3]: [sum(sales#134)#149, sum(returns#135)#150, sum(profit#136)#151] -Results [5]: [null AS channel#152, null AS id#153, sum(sales#134)#149 AS sales#154, sum(returns#135)#150 AS returns#155, sum(profit#136)#151 AS profit#156] - -(105) Union - -(106) HashAggregate [codegen id : 75] -Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] -Keys [5]: [channel#30, id#31, sales#109, returns#110, profit#111] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#30, id#31, sales#109, returns#110, profit#111] - -(107) CometColumnarExchange -Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] -Arguments: hashpartitioning(channel#30, id#31, sales#109, returns#110, profit#111, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] - -(108) CometHashAggregate -Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] -Keys [5]: [channel#30, id#31, sales#109, returns#110, profit#111] -Functions: [] - -(109) CometTakeOrderedAndProject -Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#30 ASC NULLS FIRST,id#31 ASC NULLS FIRST], output=[channel#30,id#31,sales#109,returns#110,profit#111]), [channel#30, id#31, sales#109, returns#110, profit#111], 100, 0, [channel#30 ASC NULLS FIRST, id#31 ASC NULLS FIRST], [channel#30, id#31, sales#109, returns#110, profit#111] - -(110) CometColumnarToRow [codegen id : 76] -Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (115) -+- * CometColumnarToRow (114) - +- CometProject (113) - +- CometFilter (112) - +- CometNativeScan parquet spark_catalog.default.date_dim (111) - - -(111) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#157] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(112) CometFilter -Input [2]: [d_date_sk#6, d_date#157] -Condition : (((isnotnull(d_date#157) AND (d_date#157 >= 1998-08-04)) AND (d_date#157 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) - -(113) CometProject -Input [2]: [d_date_sk#6, d_date#157] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(114) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(115) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] - -Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#49 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 70 Hosting Expression = wr_returned_date_sk#79 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/simplified.txt deleted file mode 100644 index 6f40fcdddf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/simplified.txt +++ /dev/null @@ -1,180 +0,0 @@ -WholeStageCodegen (76) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (75) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (24) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (23) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (8) - Project [s_store_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [s_store_sk,s_store_sk] - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #3 - WholeStageCodegen (3) - HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,s_store_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #7 - WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - Project [sr_return_amt,sr_net_loss,s_store_sk] - BroadcastHashJoin [sr_store_sk,s_store_sk] - Project [sr_store_sk,sr_return_amt,sr_net_loss] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [s_store_sk] #5 - WholeStageCodegen (14) - Project [cs_call_center_sk,sales,returns,profit,profit_loss] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_call_center_sk] #9 - WholeStageCodegen (10) - HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #4 - HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (13) - HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - Project [cr_return_amount,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (22) - Project [wp_web_page_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wp_web_page_sk] #11 - WholeStageCodegen (17) - HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_web_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (21) - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wp_web_page_sk] #14 - WholeStageCodegen (20) - HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - Project [wr_return_amt,wr_net_loss,wp_web_page_sk] - BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] - Project [wr_web_page_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Filter [wr_web_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [wp_web_page_sk] #12 - WholeStageCodegen (49) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #15 - WholeStageCodegen (48) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (74) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #16 - WholeStageCodegen (73) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/explain.txt deleted file mode 100644 index 4b5efc84e3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,621 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (103) -+- CometTakeOrderedAndProject (102) - +- CometHashAggregate (101) - +- CometColumnarExchange (100) - +- * HashAggregate (99) - +- Union (98) - :- * HashAggregate (83) - : +- * CometColumnarToRow (82) - : +- CometColumnarExchange (81) - : +- * HashAggregate (80) - : +- Union (79) - : :- * CometColumnarToRow (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : : +- CometBroadcastExchange (28) - : : +- CometHashAggregate (27) - : : +- CometExchange (26) - : : +- CometHashAggregate (25) - : : +- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (21) - : : : +- CometBroadcastHashJoin (20) - : : : :- CometFilter (18) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (17) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : :- * Project (50) - : : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) - : : :- BroadcastExchange (40) - : : : +- * CometColumnarToRow (39) - : : : +- CometHashAggregate (38) - : : : +- CometExchange (37) - : : : +- CometHashAggregate (36) - : : : +- CometProject (35) - : : : +- CometBroadcastHashJoin (34) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (32) - : : : +- ReusedExchange (33) - : : +- * CometColumnarToRow (48) - : : +- CometHashAggregate (47) - : : +- CometExchange (46) - : : +- CometHashAggregate (45) - : : +- CometProject (44) - : : +- CometBroadcastHashJoin (43) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (41) - : : +- ReusedExchange (42) - : +- * CometColumnarToRow (78) - : +- CometProject (77) - : +- CometBroadcastHashJoin (76) - : :- CometHashAggregate (63) - : : +- CometExchange (62) - : : +- CometHashAggregate (61) - : : +- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (55) - : : : +- CometBroadcastHashJoin (54) - : : : :- CometFilter (52) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (51) - : : : +- ReusedExchange (53) - : : +- CometBroadcastExchange (58) - : : +- CometFilter (57) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (56) - : +- CometBroadcastExchange (75) - : +- CometHashAggregate (74) - : +- CometExchange (73) - : +- CometHashAggregate (72) - : +- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometProject (68) - : : +- CometBroadcastHashJoin (67) - : : :- CometFilter (65) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (64) - : : +- ReusedExchange (66) - : +- ReusedExchange (69) - :- * HashAggregate (90) - : +- * CometColumnarToRow (89) - : +- CometColumnarExchange (88) - : +- * HashAggregate (87) - : +- * HashAggregate (86) - : +- * CometColumnarToRow (85) - : +- ReusedExchange (84) - +- * HashAggregate (97) - +- * CometColumnarToRow (96) - +- CometColumnarExchange (95) - +- * HashAggregate (94) - +- * HashAggregate (93) - +- * CometColumnarToRow (92) - +- ReusedExchange (91) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3], [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [1]: [s_store_sk#8] -Condition : isnotnull(s_store_sk#8) - -(11) CometBroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: [s_store_sk#8] - -(12) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -Right output [1]: [s_store_sk#8] -Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight - -(13) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] - -(14) CometHashAggregate -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Keys [1]: [s_store_sk#8] -Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] - -(15) CometExchange -Input [3]: [s_store_sk#8, sum#9, sum#10] -Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [3]: [s_store_sk#8, sum#9, sum#10] -Keys [1]: [s_store_sk#8] -Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(18) CometFilter -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Condition : isnotnull(sr_store_sk#11) - -(19) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#16] - -(20) CometBroadcastHashJoin -Left output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Right output [1]: [d_date_sk#16] -Arguments: [sr_returned_date_sk#14], [d_date_sk#16], Inner, BuildRight - -(21) CometProject -Input [5]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14, d_date_sk#16] -Arguments: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13], [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] - -(22) ReusedExchange [Reuses operator id: 11] -Output [1]: [s_store_sk#17] - -(23) CometBroadcastHashJoin -Left output [3]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] -Right output [1]: [s_store_sk#17] -Arguments: [sr_store_sk#11], [s_store_sk#17], Inner, BuildRight - -(24) CometProject -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Arguments: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17], [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] - -(25) CometHashAggregate -Input [3]: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Keys [1]: [s_store_sk#17] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#12)), partial_sum(UnscaledValue(sr_net_loss#13))] - -(26) CometExchange -Input [3]: [s_store_sk#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(27) CometHashAggregate -Input [3]: [s_store_sk#17, sum#18, sum#19] -Keys [1]: [s_store_sk#17] -Functions [2]: [sum(UnscaledValue(sr_return_amt#12)), sum(UnscaledValue(sr_net_loss#13))] - -(28) CometBroadcastExchange -Input [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#17, returns#20, profit_loss#21] - -(29) CometBroadcastHashJoin -Left output [3]: [s_store_sk#8, sales#22, profit#23] -Right output [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight - -(30) CometProject -Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] -Arguments: [channel#24, id#25, sales#22, returns#26, profit#27], [store channel AS channel#24, s_store_sk#8 AS id#25, sales#22, coalesce(returns#20, 0.00) AS returns#26, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#27] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] -ReadSchema: struct - -(33) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(34) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [cs_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(35) CometProject -Input [5]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31, d_date_sk#33] -Arguments: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30], [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] - -(36) CometHashAggregate -Input [3]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#29)), partial_sum(UnscaledValue(cs_net_profit#30))] - -(37) CometExchange -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Arguments: hashpartitioning(cs_call_center_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] - -(39) CometColumnarToRow [codegen id : 2] -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] - -(40) BroadcastExchange -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] -Arguments: IdentityBroadcastMode, [plan_id=4] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] -ReadSchema: struct - -(42) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#42] - -(43) CometBroadcastHashJoin -Left output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Right output [1]: [d_date_sk#42] -Arguments: [cr_returned_date_sk#40], [d_date_sk#42], Inner, BuildRight - -(44) CometProject -Input [4]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40, d_date_sk#42] -Arguments: [cr_return_amount#38, cr_net_loss#39], [cr_return_amount#38, cr_net_loss#39] - -(45) CometHashAggregate -Input [2]: [cr_return_amount#38, cr_net_loss#39] -Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] - -(46) CometExchange -Input [2]: [sum#43, sum#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometHashAggregate -Input [2]: [sum#43, sum#44] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] - -(48) CometColumnarToRow -Input [2]: [returns#45, profit_loss#46] - -(49) BroadcastNestedLoopJoin [codegen id : 3] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 3] -Output [5]: [catalog channel AS channel#47, cs_call_center_sk#28 AS id#48, sales#36, returns#45, (profit#37 - profit_loss#46) AS profit#49] -Input [5]: [cs_call_center_sk#28, sales#36, profit#37, returns#45, profit_loss#46] - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] -PushedFilters: [IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(52) CometFilter -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Condition : isnotnull(ws_web_page_sk#50) - -(53) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#55] - -(54) CometBroadcastHashJoin -Left output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Right output [1]: [d_date_sk#55] -Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight - -(55) CometProject -Input [5]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53, d_date_sk#55] -Arguments: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52], [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(57) CometFilter -Input [1]: [wp_web_page_sk#56] -Condition : isnotnull(wp_web_page_sk#56) - -(58) CometBroadcastExchange -Input [1]: [wp_web_page_sk#56] -Arguments: [wp_web_page_sk#56] - -(59) CometBroadcastHashJoin -Left output [3]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] -Right output [1]: [wp_web_page_sk#56] -Arguments: [ws_web_page_sk#50], [wp_web_page_sk#56], Inner, BuildRight - -(60) CometProject -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Arguments: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56], [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] - -(61) CometHashAggregate -Input [3]: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#51)), partial_sum(UnscaledValue(ws_net_profit#52))] - -(62) CometExchange -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Arguments: hashpartitioning(wp_web_page_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(63) CometHashAggregate -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#51)), sum(UnscaledValue(ws_net_profit#52))] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] -PushedFilters: [IsNotNull(wr_web_page_sk)] -ReadSchema: struct - -(65) CometFilter -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Condition : isnotnull(wr_web_page_sk#59) - -(66) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#64] - -(67) CometBroadcastHashJoin -Left output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Right output [1]: [d_date_sk#64] -Arguments: [wr_returned_date_sk#62], [d_date_sk#64], Inner, BuildRight - -(68) CometProject -Input [5]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62, d_date_sk#64] -Arguments: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61], [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] - -(69) ReusedExchange [Reuses operator id: 58] -Output [1]: [wp_web_page_sk#65] - -(70) CometBroadcastHashJoin -Left output [3]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] -Right output [1]: [wp_web_page_sk#65] -Arguments: [wr_web_page_sk#59], [wp_web_page_sk#65], Inner, BuildRight - -(71) CometProject -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Arguments: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65], [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] - -(72) CometHashAggregate -Input [3]: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#60)), partial_sum(UnscaledValue(wr_net_loss#61))] - -(73) CometExchange -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Arguments: hashpartitioning(wp_web_page_sk#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(74) CometHashAggregate -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [sum(UnscaledValue(wr_return_amt#60)), sum(UnscaledValue(wr_net_loss#61))] - -(75) CometBroadcastExchange -Input [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#65, returns#68, profit_loss#69] - -(76) CometBroadcastHashJoin -Left output [3]: [wp_web_page_sk#56, sales#70, profit#71] -Right output [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight - -(77) CometProject -Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [channel#72, id#73, sales#70, returns#74, profit#75], [web channel AS channel#72, wp_web_page_sk#56 AS id#73, sales#70, coalesce(returns#68, 0.00) AS returns#74, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#75] - -(78) CometColumnarToRow [codegen id : 4] -Input [5]: [channel#72, id#73, sales#70, returns#74, profit#75] - -(79) Union - -(80) HashAggregate [codegen id : 5] -Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] -Keys [2]: [channel#24, id#25] -Functions [3]: [partial_sum(sales#22), partial_sum(returns#26), partial_sum(profit#27)] -Aggregate Attributes [6]: [sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81] -Results [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(81) CometColumnarExchange -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Arguments: hashpartitioning(channel#24, id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(82) CometColumnarToRow [codegen id : 6] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(83) HashAggregate [codegen id : 6] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Keys [2]: [channel#24, id#25] -Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] -Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] -Results [5]: [channel#24, id#25, cast(sum(sales#22)#88 as decimal(37,2)) AS sales#91, cast(sum(returns#26)#89 as decimal(37,2)) AS returns#92, cast(sum(profit#27)#90 as decimal(38,2)) AS profit#93] - -(84) ReusedExchange [Reuses operator id: 81] -Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(85) CometColumnarToRow [codegen id : 12] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(86) HashAggregate [codegen id : 12] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Keys [2]: [channel#24, id#25] -Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] -Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] -Results [4]: [channel#24, sum(sales#22)#88 AS sales#94, sum(returns#26)#89 AS returns#95, sum(profit#27)#90 AS profit#96] - -(87) HashAggregate [codegen id : 12] -Input [4]: [channel#24, sales#94, returns#95, profit#96] -Keys [1]: [channel#24] -Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] -Aggregate Attributes [6]: [sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Results [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] - -(88) CometColumnarExchange -Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Arguments: hashpartitioning(channel#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(89) CometColumnarToRow [codegen id : 13] -Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] - -(90) HashAggregate [codegen id : 13] -Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Keys [1]: [channel#24] -Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] -Aggregate Attributes [3]: [sum(sales#94)#109, sum(returns#95)#110, sum(profit#96)#111] -Results [5]: [channel#24, null AS id#112, sum(sales#94)#109 AS sales#113, sum(returns#95)#110 AS returns#114, sum(profit#96)#111 AS profit#115] - -(91) ReusedExchange [Reuses operator id: 81] -Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(92) CometColumnarToRow [codegen id : 19] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(93) HashAggregate [codegen id : 19] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Keys [2]: [channel#24, id#25] -Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] -Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] -Results [3]: [sum(sales#22)#88 AS sales#116, sum(returns#26)#89 AS returns#117, sum(profit#27)#90 AS profit#118] - -(94) HashAggregate [codegen id : 19] -Input [3]: [sales#116, returns#117, profit#118] -Keys: [] -Functions [3]: [partial_sum(sales#116), partial_sum(returns#117), partial_sum(profit#118)] -Aggregate Attributes [6]: [sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] -Results [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] - -(95) CometColumnarExchange -Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(96) CometColumnarToRow [codegen id : 20] -Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] - -(97) HashAggregate [codegen id : 20] -Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] -Keys: [] -Functions [3]: [sum(sales#116), sum(returns#117), sum(profit#118)] -Aggregate Attributes [3]: [sum(sales#116)#131, sum(returns#117)#132, sum(profit#118)#133] -Results [5]: [null AS channel#134, null AS id#135, sum(sales#116)#131 AS sales#136, sum(returns#117)#132 AS returns#137, sum(profit#118)#133 AS profit#138] - -(98) Union - -(99) HashAggregate [codegen id : 21] -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#24, id#25, sales#91, returns#92, profit#93] - -(100) CometColumnarExchange -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Arguments: hashpartitioning(channel#24, id#25, sales#91, returns#92, profit#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(101) CometHashAggregate -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Functions: [] - -(102) CometTakeOrderedAndProject -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#24 ASC NULLS FIRST,id#25 ASC NULLS FIRST], output=[channel#24,id#25,sales#91,returns#92,profit#93]), [channel#24, id#25, sales#91, returns#92, profit#93], 100, 0, [channel#24 ASC NULLS FIRST, id#25 ASC NULLS FIRST], [channel#24, id#25, sales#91, returns#92, profit#93] - -(103) CometColumnarToRow [codegen id : 22] -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (108) -+- * CometColumnarToRow (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) - - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) - -(106) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(107) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(108) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#14 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 64 Hosting Expression = wr_returned_date_sk#62 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/simplified.txt deleted file mode 100644 index ac3d312ee8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,141 +0,0 @@ -WholeStageCodegen (22) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (21) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (5) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] - CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [s_store_sk,sales,profit,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] - CometExchange [s_store_sk] #3 - CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [s_store_sk,returns,profit_loss] #7 - CometHashAggregate [sum,sum] [s_store_sk,returns,profit_loss,sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss))] - CometExchange [s_store_sk] #8 - CometHashAggregate [sr_return_amt,sr_net_loss] [s_store_sk,sum,sum] - CometProject [sr_return_amt,sr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [s_store_sk] #6 - WholeStageCodegen (3) - Project [cs_call_center_sk,sales,returns,profit,profit_loss] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [cs_call_center_sk,sales,profit,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] - CometExchange [cs_call_center_sk] #10 - CometHashAggregate [cs_ext_sales_price,cs_net_profit] [cs_call_center_sk,sum,sum] - CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [returns,profit_loss,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] - CometExchange #11 - CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] - CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [wp_web_page_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] - CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [wp_web_page_sk,sales,profit,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] - CometExchange [wp_web_page_sk] #12 - CometHashAggregate [ws_ext_sales_price,ws_net_profit] [wp_web_page_sk,sum,sum] - CometProject [ws_ext_sales_price,ws_net_profit,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_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 [native_iceberg_compat] 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 [wp_web_page_sk] #13 - CometFilter [wp_web_page_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [wp_web_page_sk,returns,profit_loss] #14 - CometHashAggregate [sum,sum] [wp_web_page_sk,returns,profit_loss,sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss))] - CometExchange [wp_web_page_sk] #15 - CometHashAggregate [wr_return_amt,wr_net_loss] [wp_web_page_sk,sum,sum] - CometProject [wr_return_amt,wr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [wp_web_page_sk] #13 - WholeStageCodegen (13) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #16 - WholeStageCodegen (12) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (20) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #17 - WholeStageCodegen (19) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/explain.txt deleted file mode 100644 index 4b5efc84e3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/explain.txt +++ /dev/null @@ -1,621 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (103) -+- CometTakeOrderedAndProject (102) - +- CometHashAggregate (101) - +- CometColumnarExchange (100) - +- * HashAggregate (99) - +- Union (98) - :- * HashAggregate (83) - : +- * CometColumnarToRow (82) - : +- CometColumnarExchange (81) - : +- * HashAggregate (80) - : +- Union (79) - : :- * CometColumnarToRow (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : : +- CometBroadcastExchange (28) - : : +- CometHashAggregate (27) - : : +- CometExchange (26) - : : +- CometHashAggregate (25) - : : +- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (21) - : : : +- CometBroadcastHashJoin (20) - : : : :- CometFilter (18) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (17) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : :- * Project (50) - : : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) - : : :- BroadcastExchange (40) - : : : +- * CometColumnarToRow (39) - : : : +- CometHashAggregate (38) - : : : +- CometExchange (37) - : : : +- CometHashAggregate (36) - : : : +- CometProject (35) - : : : +- CometBroadcastHashJoin (34) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (32) - : : : +- ReusedExchange (33) - : : +- * CometColumnarToRow (48) - : : +- CometHashAggregate (47) - : : +- CometExchange (46) - : : +- CometHashAggregate (45) - : : +- CometProject (44) - : : +- CometBroadcastHashJoin (43) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (41) - : : +- ReusedExchange (42) - : +- * CometColumnarToRow (78) - : +- CometProject (77) - : +- CometBroadcastHashJoin (76) - : :- CometHashAggregate (63) - : : +- CometExchange (62) - : : +- CometHashAggregate (61) - : : +- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (55) - : : : +- CometBroadcastHashJoin (54) - : : : :- CometFilter (52) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (51) - : : : +- ReusedExchange (53) - : : +- CometBroadcastExchange (58) - : : +- CometFilter (57) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (56) - : +- CometBroadcastExchange (75) - : +- CometHashAggregate (74) - : +- CometExchange (73) - : +- CometHashAggregate (72) - : +- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometProject (68) - : : +- CometBroadcastHashJoin (67) - : : :- CometFilter (65) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (64) - : : +- ReusedExchange (66) - : +- ReusedExchange (69) - :- * HashAggregate (90) - : +- * CometColumnarToRow (89) - : +- CometColumnarExchange (88) - : +- * HashAggregate (87) - : +- * HashAggregate (86) - : +- * CometColumnarToRow (85) - : +- ReusedExchange (84) - +- * HashAggregate (97) - +- * CometColumnarToRow (96) - +- CometColumnarExchange (95) - +- * HashAggregate (94) - +- * HashAggregate (93) - +- * CometColumnarToRow (92) - +- ReusedExchange (91) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3], [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [1]: [s_store_sk#8] -Condition : isnotnull(s_store_sk#8) - -(11) CometBroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: [s_store_sk#8] - -(12) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -Right output [1]: [s_store_sk#8] -Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight - -(13) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] - -(14) CometHashAggregate -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Keys [1]: [s_store_sk#8] -Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] - -(15) CometExchange -Input [3]: [s_store_sk#8, sum#9, sum#10] -Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [3]: [s_store_sk#8, sum#9, sum#10] -Keys [1]: [s_store_sk#8] -Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(18) CometFilter -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Condition : isnotnull(sr_store_sk#11) - -(19) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#16] - -(20) CometBroadcastHashJoin -Left output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Right output [1]: [d_date_sk#16] -Arguments: [sr_returned_date_sk#14], [d_date_sk#16], Inner, BuildRight - -(21) CometProject -Input [5]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14, d_date_sk#16] -Arguments: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13], [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] - -(22) ReusedExchange [Reuses operator id: 11] -Output [1]: [s_store_sk#17] - -(23) CometBroadcastHashJoin -Left output [3]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] -Right output [1]: [s_store_sk#17] -Arguments: [sr_store_sk#11], [s_store_sk#17], Inner, BuildRight - -(24) CometProject -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Arguments: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17], [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] - -(25) CometHashAggregate -Input [3]: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Keys [1]: [s_store_sk#17] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#12)), partial_sum(UnscaledValue(sr_net_loss#13))] - -(26) CometExchange -Input [3]: [s_store_sk#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(27) CometHashAggregate -Input [3]: [s_store_sk#17, sum#18, sum#19] -Keys [1]: [s_store_sk#17] -Functions [2]: [sum(UnscaledValue(sr_return_amt#12)), sum(UnscaledValue(sr_net_loss#13))] - -(28) CometBroadcastExchange -Input [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#17, returns#20, profit_loss#21] - -(29) CometBroadcastHashJoin -Left output [3]: [s_store_sk#8, sales#22, profit#23] -Right output [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight - -(30) CometProject -Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] -Arguments: [channel#24, id#25, sales#22, returns#26, profit#27], [store channel AS channel#24, s_store_sk#8 AS id#25, sales#22, coalesce(returns#20, 0.00) AS returns#26, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#27] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] -ReadSchema: struct - -(33) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(34) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [cs_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(35) CometProject -Input [5]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31, d_date_sk#33] -Arguments: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30], [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] - -(36) CometHashAggregate -Input [3]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#29)), partial_sum(UnscaledValue(cs_net_profit#30))] - -(37) CometExchange -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Arguments: hashpartitioning(cs_call_center_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] - -(39) CometColumnarToRow [codegen id : 2] -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] - -(40) BroadcastExchange -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] -Arguments: IdentityBroadcastMode, [plan_id=4] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] -ReadSchema: struct - -(42) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#42] - -(43) CometBroadcastHashJoin -Left output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Right output [1]: [d_date_sk#42] -Arguments: [cr_returned_date_sk#40], [d_date_sk#42], Inner, BuildRight - -(44) CometProject -Input [4]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40, d_date_sk#42] -Arguments: [cr_return_amount#38, cr_net_loss#39], [cr_return_amount#38, cr_net_loss#39] - -(45) CometHashAggregate -Input [2]: [cr_return_amount#38, cr_net_loss#39] -Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] - -(46) CometExchange -Input [2]: [sum#43, sum#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometHashAggregate -Input [2]: [sum#43, sum#44] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] - -(48) CometColumnarToRow -Input [2]: [returns#45, profit_loss#46] - -(49) BroadcastNestedLoopJoin [codegen id : 3] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 3] -Output [5]: [catalog channel AS channel#47, cs_call_center_sk#28 AS id#48, sales#36, returns#45, (profit#37 - profit_loss#46) AS profit#49] -Input [5]: [cs_call_center_sk#28, sales#36, profit#37, returns#45, profit_loss#46] - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] -PushedFilters: [IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(52) CometFilter -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Condition : isnotnull(ws_web_page_sk#50) - -(53) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#55] - -(54) CometBroadcastHashJoin -Left output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Right output [1]: [d_date_sk#55] -Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight - -(55) CometProject -Input [5]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53, d_date_sk#55] -Arguments: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52], [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(57) CometFilter -Input [1]: [wp_web_page_sk#56] -Condition : isnotnull(wp_web_page_sk#56) - -(58) CometBroadcastExchange -Input [1]: [wp_web_page_sk#56] -Arguments: [wp_web_page_sk#56] - -(59) CometBroadcastHashJoin -Left output [3]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] -Right output [1]: [wp_web_page_sk#56] -Arguments: [ws_web_page_sk#50], [wp_web_page_sk#56], Inner, BuildRight - -(60) CometProject -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Arguments: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56], [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] - -(61) CometHashAggregate -Input [3]: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#51)), partial_sum(UnscaledValue(ws_net_profit#52))] - -(62) CometExchange -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Arguments: hashpartitioning(wp_web_page_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(63) CometHashAggregate -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#51)), sum(UnscaledValue(ws_net_profit#52))] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] -PushedFilters: [IsNotNull(wr_web_page_sk)] -ReadSchema: struct - -(65) CometFilter -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Condition : isnotnull(wr_web_page_sk#59) - -(66) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#64] - -(67) CometBroadcastHashJoin -Left output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Right output [1]: [d_date_sk#64] -Arguments: [wr_returned_date_sk#62], [d_date_sk#64], Inner, BuildRight - -(68) CometProject -Input [5]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62, d_date_sk#64] -Arguments: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61], [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] - -(69) ReusedExchange [Reuses operator id: 58] -Output [1]: [wp_web_page_sk#65] - -(70) CometBroadcastHashJoin -Left output [3]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] -Right output [1]: [wp_web_page_sk#65] -Arguments: [wr_web_page_sk#59], [wp_web_page_sk#65], Inner, BuildRight - -(71) CometProject -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Arguments: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65], [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] - -(72) CometHashAggregate -Input [3]: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#60)), partial_sum(UnscaledValue(wr_net_loss#61))] - -(73) CometExchange -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Arguments: hashpartitioning(wp_web_page_sk#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(74) CometHashAggregate -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [sum(UnscaledValue(wr_return_amt#60)), sum(UnscaledValue(wr_net_loss#61))] - -(75) CometBroadcastExchange -Input [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#65, returns#68, profit_loss#69] - -(76) CometBroadcastHashJoin -Left output [3]: [wp_web_page_sk#56, sales#70, profit#71] -Right output [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight - -(77) CometProject -Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [channel#72, id#73, sales#70, returns#74, profit#75], [web channel AS channel#72, wp_web_page_sk#56 AS id#73, sales#70, coalesce(returns#68, 0.00) AS returns#74, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#75] - -(78) CometColumnarToRow [codegen id : 4] -Input [5]: [channel#72, id#73, sales#70, returns#74, profit#75] - -(79) Union - -(80) HashAggregate [codegen id : 5] -Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] -Keys [2]: [channel#24, id#25] -Functions [3]: [partial_sum(sales#22), partial_sum(returns#26), partial_sum(profit#27)] -Aggregate Attributes [6]: [sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81] -Results [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(81) CometColumnarExchange -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Arguments: hashpartitioning(channel#24, id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(82) CometColumnarToRow [codegen id : 6] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(83) HashAggregate [codegen id : 6] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Keys [2]: [channel#24, id#25] -Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] -Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] -Results [5]: [channel#24, id#25, cast(sum(sales#22)#88 as decimal(37,2)) AS sales#91, cast(sum(returns#26)#89 as decimal(37,2)) AS returns#92, cast(sum(profit#27)#90 as decimal(38,2)) AS profit#93] - -(84) ReusedExchange [Reuses operator id: 81] -Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(85) CometColumnarToRow [codegen id : 12] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(86) HashAggregate [codegen id : 12] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Keys [2]: [channel#24, id#25] -Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] -Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] -Results [4]: [channel#24, sum(sales#22)#88 AS sales#94, sum(returns#26)#89 AS returns#95, sum(profit#27)#90 AS profit#96] - -(87) HashAggregate [codegen id : 12] -Input [4]: [channel#24, sales#94, returns#95, profit#96] -Keys [1]: [channel#24] -Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] -Aggregate Attributes [6]: [sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Results [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] - -(88) CometColumnarExchange -Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Arguments: hashpartitioning(channel#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(89) CometColumnarToRow [codegen id : 13] -Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] - -(90) HashAggregate [codegen id : 13] -Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Keys [1]: [channel#24] -Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] -Aggregate Attributes [3]: [sum(sales#94)#109, sum(returns#95)#110, sum(profit#96)#111] -Results [5]: [channel#24, null AS id#112, sum(sales#94)#109 AS sales#113, sum(returns#95)#110 AS returns#114, sum(profit#96)#111 AS profit#115] - -(91) ReusedExchange [Reuses operator id: 81] -Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(92) CometColumnarToRow [codegen id : 19] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(93) HashAggregate [codegen id : 19] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Keys [2]: [channel#24, id#25] -Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] -Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] -Results [3]: [sum(sales#22)#88 AS sales#116, sum(returns#26)#89 AS returns#117, sum(profit#27)#90 AS profit#118] - -(94) HashAggregate [codegen id : 19] -Input [3]: [sales#116, returns#117, profit#118] -Keys: [] -Functions [3]: [partial_sum(sales#116), partial_sum(returns#117), partial_sum(profit#118)] -Aggregate Attributes [6]: [sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] -Results [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] - -(95) CometColumnarExchange -Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(96) CometColumnarToRow [codegen id : 20] -Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] - -(97) HashAggregate [codegen id : 20] -Input [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] -Keys: [] -Functions [3]: [sum(sales#116), sum(returns#117), sum(profit#118)] -Aggregate Attributes [3]: [sum(sales#116)#131, sum(returns#117)#132, sum(profit#118)#133] -Results [5]: [null AS channel#134, null AS id#135, sum(sales#116)#131 AS sales#136, sum(returns#117)#132 AS returns#137, sum(profit#118)#133 AS profit#138] - -(98) Union - -(99) HashAggregate [codegen id : 21] -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#24, id#25, sales#91, returns#92, profit#93] - -(100) CometColumnarExchange -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Arguments: hashpartitioning(channel#24, id#25, sales#91, returns#92, profit#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(101) CometHashAggregate -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Functions: [] - -(102) CometTakeOrderedAndProject -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#24 ASC NULLS FIRST,id#25 ASC NULLS FIRST], output=[channel#24,id#25,sales#91,returns#92,profit#93]), [channel#24, id#25, sales#91, returns#92, profit#93], 100, 0, [channel#24 ASC NULLS FIRST, id#25 ASC NULLS FIRST], [channel#24, id#25, sales#91, returns#92, profit#93] - -(103) CometColumnarToRow [codegen id : 22] -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (108) -+- * CometColumnarToRow (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) - - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) - -(106) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(107) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(108) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#14 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 64 Hosting Expression = wr_returned_date_sk#62 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/extended.txt deleted file mode 100644 index b4318d03e2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/extended.txt +++ /dev/null @@ -1,355 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : : :- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : : :- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : :- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - -Comet accelerated 287 out of 332 eligible operators (86%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index ac3d312ee8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/simplified.txt +++ /dev/null @@ -1,141 +0,0 @@ -WholeStageCodegen (22) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (21) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (5) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] - CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [s_store_sk,sales,profit,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] - CometExchange [s_store_sk] #3 - CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [s_store_sk,returns,profit_loss] #7 - CometHashAggregate [sum,sum] [s_store_sk,returns,profit_loss,sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss))] - CometExchange [s_store_sk] #8 - CometHashAggregate [sr_return_amt,sr_net_loss] [s_store_sk,sum,sum] - CometProject [sr_return_amt,sr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [s_store_sk] #6 - WholeStageCodegen (3) - Project [cs_call_center_sk,sales,returns,profit,profit_loss] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [cs_call_center_sk,sales,profit,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] - CometExchange [cs_call_center_sk] #10 - CometHashAggregate [cs_ext_sales_price,cs_net_profit] [cs_call_center_sk,sum,sum] - CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [returns,profit_loss,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] - CometExchange #11 - CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] - CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [wp_web_page_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] - CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [wp_web_page_sk,sales,profit,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] - CometExchange [wp_web_page_sk] #12 - CometHashAggregate [ws_ext_sales_price,ws_net_profit] [wp_web_page_sk,sum,sum] - CometProject [ws_ext_sales_price,ws_net_profit,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_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 [native_iceberg_compat] 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 [wp_web_page_sk] #13 - CometFilter [wp_web_page_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [wp_web_page_sk,returns,profit_loss] #14 - CometHashAggregate [sum,sum] [wp_web_page_sk,returns,profit_loss,sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss))] - CometExchange [wp_web_page_sk] #15 - CometHashAggregate [wr_return_amt,wr_net_loss] [wp_web_page_sk,sum,sum] - CometProject [wr_return_amt,wr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [wp_web_page_sk] #13 - WholeStageCodegen (13) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #16 - WholeStageCodegen (12) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (20) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #17 - WholeStageCodegen (19) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/explain.txt deleted file mode 100644 index dbd053cfac..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/explain.txt +++ /dev/null @@ -1,417 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (70) -+- * Project (69) - +- * CometColumnarToRow (68) - +- CometSortMergeJoin (67) - :- CometProject (45) - : +- CometSortMergeJoin (44) - : :- CometSort (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometFilter (12) - : : : +- CometSortMergeJoin (11) - : : : :- CometSort (5) - : : : : +- CometColumnarExchange (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (10) - : : : +- CometExchange (9) - : : : +- CometProject (8) - : : : +- CometFilter (7) - : : : +- CometNativeScan parquet spark_catalog.default.store_returns (6) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (14) - : +- CometSort (43) - : +- CometFilter (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometFilter (34) - : : +- CometSortMergeJoin (33) - : : :- CometSort (27) - : : : +- CometColumnarExchange (26) - : : : +- * Filter (25) - : : : +- * ColumnarToRow (24) - : : : +- Scan parquet spark_catalog.default.web_sales (23) - : : +- CometSort (32) - : : +- CometExchange (31) - : : +- CometProject (30) - : : +- CometFilter (29) - : : +- CometNativeScan parquet spark_catalog.default.web_returns (28) - : +- ReusedExchange (36) - +- CometSort (66) - +- CometFilter (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometFilter (57) - : +- CometSortMergeJoin (56) - : :- CometSort (50) - : : +- CometColumnarExchange (49) - : : +- * Filter (48) - : : +- * ColumnarToRow (47) - : : +- Scan parquet spark_catalog.default.catalog_sales (46) - : +- CometSort (55) - : +- CometExchange (54) - : +- CometProject (53) - : +- CometFilter (52) - : +- CometNativeScan parquet spark_catalog.default.catalog_returns (51) - +- ReusedExchange (59) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(3) Filter [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_customer_sk#2)) - -(4) CometColumnarExchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_ticket_number#3, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_ticket_number#3 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) - -(8) CometProject -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] - -(9) CometExchange -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter - -(12) CometFilter -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Condition : isnull(sr_ticket_number#10) - -(13) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(14) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(17) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] - -(19) CometHashAggregate -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] - -(20) CometExchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(21) CometHashAggregate -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] - -(22) CometSort -Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] - -(23) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 2] -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(25) Filter [codegen id : 2] -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) - -(26) CometColumnarExchange -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometSort -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] - -(28) CometNativeScan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#28, wr_order_number#29, wr_returned_date_sk#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(29) CometFilter -Input [3]: [wr_item_sk#28, wr_order_number#29, wr_returned_date_sk#30] -Condition : (isnotnull(wr_order_number#29) AND isnotnull(wr_item_sk#28)) - -(30) CometProject -Input [3]: [wr_item_sk#28, wr_order_number#29, wr_returned_date_sk#30] -Arguments: [wr_item_sk#28, wr_order_number#29], [wr_item_sk#28, wr_order_number#29] - -(31) CometExchange -Input [2]: [wr_item_sk#28, wr_order_number#29] -Arguments: hashpartitioning(wr_order_number#29, wr_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(32) CometSort -Input [2]: [wr_item_sk#28, wr_order_number#29] -Arguments: [wr_item_sk#28, wr_order_number#29], [wr_order_number#29 ASC NULLS FIRST, wr_item_sk#28 ASC NULLS FIRST] - -(33) CometSortMergeJoin -Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [wr_item_sk#28, wr_order_number#29] -Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#29, wr_item_sk#28], LeftOuter - -(34) CometFilter -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#28, wr_order_number#29] -Condition : isnull(wr_order_number#29) - -(35) CometProject -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#28, wr_order_number#29] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(36) ReusedExchange [Reuses operator id: 16] -Output [2]: [d_date_sk#31, d_year#32] - -(37) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [d_date_sk#31, d_year#32] -Arguments: [ws_sold_date_sk#27], [d_date_sk#31], Inner, BuildRight - -(38) CometProject -Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#31, d_year#32] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#32], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#32] - -(39) CometHashAggregate -Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#32] -Keys [3]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] - -(40) CometExchange -Input [6]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22, sum#33, sum#34, sum#35] -Arguments: hashpartitioning(d_year#32, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(41) CometHashAggregate -Input [6]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22, sum#33, sum#34, sum#35] -Keys [3]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] - -(42) CometFilter -Input [6]: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Condition : (coalesce(ws_qty#38, 0) > 0) - -(43) CometSort -Input [6]: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Arguments: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40], [ws_sold_year#36 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#37 ASC NULLS FIRST] - -(44) CometSortMergeJoin -Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Right output [6]: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37], Inner - -(45) CometProject -Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40] - -(46) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#47), dynamicpruningexpression(cs_sold_date_sk#47 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(47) ColumnarToRow [codegen id : 3] -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] - -(48) Filter [codegen id : 3] -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Condition : (isnotnull(cs_item_sk#42) AND isnotnull(cs_bill_customer_sk#41)) - -(49) CometColumnarExchange -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Arguments: hashpartitioning(cs_order_number#43, cs_item_sk#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(50) CometSort -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Arguments: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47], [cs_order_number#43 ASC NULLS FIRST, cs_item_sk#42 ASC NULLS FIRST] - -(51) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#48, cr_order_number#49, cr_returned_date_sk#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(52) CometFilter -Input [3]: [cr_item_sk#48, cr_order_number#49, cr_returned_date_sk#50] -Condition : (isnotnull(cr_order_number#49) AND isnotnull(cr_item_sk#48)) - -(53) CometProject -Input [3]: [cr_item_sk#48, cr_order_number#49, cr_returned_date_sk#50] -Arguments: [cr_item_sk#48, cr_order_number#49], [cr_item_sk#48, cr_order_number#49] - -(54) CometExchange -Input [2]: [cr_item_sk#48, cr_order_number#49] -Arguments: hashpartitioning(cr_order_number#49, cr_item_sk#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(55) CometSort -Input [2]: [cr_item_sk#48, cr_order_number#49] -Arguments: [cr_item_sk#48, cr_order_number#49], [cr_order_number#49 ASC NULLS FIRST, cr_item_sk#48 ASC NULLS FIRST] - -(56) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Right output [2]: [cr_item_sk#48, cr_order_number#49] -Arguments: [cs_order_number#43, cs_item_sk#42], [cr_order_number#49, cr_item_sk#48], LeftOuter - -(57) CometFilter -Input [9]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47, cr_item_sk#48, cr_order_number#49] -Condition : isnull(cr_order_number#49) - -(58) CometProject -Input [9]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47, cr_item_sk#48, cr_order_number#49] -Arguments: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47], [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] - -(59) ReusedExchange [Reuses operator id: 16] -Output [2]: [d_date_sk#51, d_year#52] - -(60) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Right output [2]: [d_date_sk#51, d_year#52] -Arguments: [cs_sold_date_sk#47], [d_date_sk#51], Inner, BuildRight - -(61) CometProject -Input [8]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47, d_date_sk#51, d_year#52] -Arguments: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, d_year#52], [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, d_year#52] - -(62) CometHashAggregate -Input [6]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, d_year#52] -Keys [3]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41] -Functions [3]: [partial_sum(cs_quantity#44), partial_sum(UnscaledValue(cs_wholesale_cost#45)), partial_sum(UnscaledValue(cs_sales_price#46))] - -(63) CometExchange -Input [6]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41, sum#53, sum#54, sum#55] -Arguments: hashpartitioning(d_year#52, cs_item_sk#42, cs_bill_customer_sk#41, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(64) CometHashAggregate -Input [6]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41, sum#53, sum#54, sum#55] -Keys [3]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41] -Functions [3]: [sum(cs_quantity#44), sum(UnscaledValue(cs_wholesale_cost#45)), sum(UnscaledValue(cs_sales_price#46))] - -(65) CometFilter -Input [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] -Condition : (coalesce(cs_qty#58, 0) > 0) - -(66) CometSort -Input [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] -Arguments: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60], [cs_sold_year#56 ASC NULLS FIRST, cs_item_sk#42 ASC NULLS FIRST, cs_customer_sk#57 ASC NULLS FIRST] - -(67) CometSortMergeJoin -Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40] -Right output [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57], Inner - -(68) CometColumnarToRow [codegen id : 4] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40, cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] - -(69) Project [codegen id : 4] -Output [13]: [round((cast(ss_qty#18 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#38 + cs_qty#58), 1) as double)))), 2) AS ratio#61, ss_qty#18 AS store_qty#62, ss_wc#19 AS store_wholesale_cost#63, ss_sp#20 AS store_sales_price#64, (coalesce(ws_qty#38, 0) + coalesce(cs_qty#58, 0)) AS other_chan_qty#65, (coalesce(ws_wc#39, 0.00) + coalesce(cs_wc#59, 0.00)) AS other_chan_wholesale_cost#66, (coalesce(ws_sp#40, 0.00) + coalesce(cs_sp#60, 0.00)) AS other_chan_sales_price#67, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40, cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] - -(70) TakeOrderedAndProject -Input [13]: [ratio#61, store_qty#62, store_wholesale_cost#63, store_sales_price#64, other_chan_qty#65, other_chan_wholesale_cost#66, other_chan_sales_price#67, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#65 ASC NULLS FIRST, other_chan_wholesale_cost#66 ASC NULLS FIRST, other_chan_sales_price#67 ASC NULLS FIRST, ratio#61 ASC NULLS FIRST], [ratio#61, store_qty#62, store_wholesale_cost#63, store_sales_price#64, other_chan_qty#65, other_chan_wholesale_cost#66, other_chan_sales_price#67] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (74) -+- * CometColumnarToRow (73) - +- CometFilter (72) - +- CometNativeScan parquet spark_catalog.default.date_dim (71) - - -(71) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(72) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(73) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(74) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#47 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/simplified.txt deleted file mode 100644 index 4711669981..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/simplified.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (4) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] - CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #2 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #5 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] - CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometColumnarExchange [ws_order_number,ws_item_sk] #7 - WholeStageCodegen (2) - Filter [ws_item_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_order_number,wr_item_sk] #8 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] - CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometColumnarExchange [cs_order_number,cs_item_sk] #10 - WholeStageCodegen (3) - Filter [cs_item_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_order_number,cr_item_sk] #11 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/explain.txt deleted file mode 100644 index 82a5357f89..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,405 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (67) -+- * Project (66) - +- * CometColumnarToRow (65) - +- CometSortMergeJoin (64) - :- CometProject (43) - : +- CometSortMergeJoin (42) - : :- CometSort (21) - : : +- CometHashAggregate (20) - : : +- CometExchange (19) - : : +- CometHashAggregate (18) - : : +- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : +- CometBroadcastExchange (15) - : : +- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : +- CometSort (41) - : +- CometFilter (40) - : +- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (33) - : : +- CometFilter (32) - : : +- CometSortMergeJoin (31) - : : :- CometSort (25) - : : : +- CometExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (22) - : : +- CometSort (30) - : : +- CometExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (26) - : +- ReusedExchange (34) - +- CometSort (63) - +- CometFilter (62) - +- CometHashAggregate (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometProject (55) - : +- CometFilter (54) - : +- CometSortMergeJoin (53) - : :- CometSort (47) - : : +- CometExchange (46) - : : +- CometFilter (45) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (44) - : +- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometFilter (49) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (48) - +- ReusedExchange (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_customer_sk#2)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_ticket_number#3, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_ticket_number#3 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) - -(7) CometProject -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] - -(8) CometExchange -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter - -(11) CometFilter -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Condition : isnull(sr_ticket_number#10) - -(12) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(15) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(16) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight - -(17) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] - -(18) CometHashAggregate -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] - -(19) CometExchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometHashAggregate -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] - -(21) CometSort -Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(23) CometFilter -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) - -(24) CometExchange -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometSort -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Condition : (isnotnull(wr_order_number#30) AND isnotnull(wr_item_sk#29)) - -(28) CometProject -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_item_sk#29, wr_order_number#30] - -(29) CometExchange -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: hashpartitioning(wr_order_number#30, wr_item_sk#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(30) CometSort -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_order_number#30 ASC NULLS FIRST, wr_item_sk#29 ASC NULLS FIRST] - -(31) CometSortMergeJoin -Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#30, wr_item_sk#29], LeftOuter - -(32) CometFilter -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Condition : isnull(wr_order_number#30) - -(33) CometProject -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(34) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#32, d_year#33] - -(35) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ws_sold_date_sk#27], [d_date_sk#32], Inner, BuildRight - -(36) CometProject -Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#32, d_year#33] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] - -(37) CometHashAggregate -Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] - -(38) CometExchange -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Arguments: hashpartitioning(d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(39) CometHashAggregate -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] - -(40) CometFilter -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Condition : (coalesce(ws_qty#39, 0) > 0) - -(41) CometSort -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41], [ws_sold_year#37 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#38 ASC NULLS FIRST] - -(42) CometSortMergeJoin -Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Right output [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38], Inner - -(43) CometProject -Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#48), dynamicpruningexpression(cs_sold_date_sk#48 IN dynamicpruning#49)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(45) CometFilter -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Condition : (isnotnull(cs_item_sk#43) AND isnotnull(cs_bill_customer_sk#42)) - -(46) CometExchange -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(47) CometSort -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST] - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Condition : (isnotnull(cr_order_number#51) AND isnotnull(cr_item_sk#50)) - -(50) CometProject -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_item_sk#50, cr_order_number#51] - -(51) CometExchange -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(52) CometSort -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST] - -(53) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cs_order_number#44, cs_item_sk#43], [cr_order_number#51, cr_item_sk#50], LeftOuter - -(54) CometFilter -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Condition : isnull(cr_order_number#51) - -(55) CometProject -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] - -(56) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#53, d_year#54] - -(57) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [cs_sold_date_sk#48], [d_date_sk#53], Inner, BuildRight - -(58) CometProject -Input [8]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, d_date_sk#53, d_year#54] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] - -(59) CometHashAggregate -Input [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [partial_sum(cs_quantity#45), partial_sum(UnscaledValue(cs_wholesale_cost#46)), partial_sum(UnscaledValue(cs_sales_price#47))] - -(60) CometExchange -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Arguments: hashpartitioning(d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(61) CometHashAggregate -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [sum(cs_quantity#45), sum(UnscaledValue(cs_wholesale_cost#46)), sum(UnscaledValue(cs_sales_price#47))] - -(62) CometFilter -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Condition : (coalesce(cs_qty#60, 0) > 0) - -(63) CometSort -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62], [cs_sold_year#58 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST, cs_customer_sk#59 ASC NULLS FIRST] - -(64) CometSortMergeJoin -Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] -Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner - -(65) CometColumnarToRow [codegen id : 1] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(66) Project [codegen id : 1] -Output [13]: [round((cast(ss_qty#18 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)))), 2) AS ratio#63, ss_qty#18 AS store_qty#64, ss_wc#19 AS store_wholesale_cost#65, ss_sp#20 AS store_sales_price#66, (coalesce(ws_qty#39, 0) + coalesce(cs_qty#60, 0)) AS other_chan_qty#67, (coalesce(ws_wc#40, 0.00) + coalesce(cs_wc#61, 0.00)) AS other_chan_wholesale_cost#68, (coalesce(ws_sp#41, 0.00) + coalesce(cs_sp#62, 0.00)) AS other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(67) TakeOrderedAndProject -Input [13]: [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#67 ASC NULLS FIRST, other_chan_wholesale_cost#68 ASC NULLS FIRST, other_chan_sales_price#69 ASC NULLS FIRST, ratio#63 ASC NULLS FIRST], [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) - - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(70) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(71) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/simplified.txt deleted file mode 100644 index 273db28e77..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,78 +0,0 @@ -TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (1) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] - CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #5 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] - CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometExchange [ws_order_number,ws_item_sk] #7 - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_order_number,wr_item_sk] #8 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] - CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometExchange [cs_order_number,cs_item_sk] #10 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_order_number,cr_item_sk] #11 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt deleted file mode 100644 index 82a5357f89..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/explain.txt +++ /dev/null @@ -1,405 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (67) -+- * Project (66) - +- * CometColumnarToRow (65) - +- CometSortMergeJoin (64) - :- CometProject (43) - : +- CometSortMergeJoin (42) - : :- CometSort (21) - : : +- CometHashAggregate (20) - : : +- CometExchange (19) - : : +- CometHashAggregate (18) - : : +- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : +- CometBroadcastExchange (15) - : : +- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : +- CometSort (41) - : +- CometFilter (40) - : +- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (33) - : : +- CometFilter (32) - : : +- CometSortMergeJoin (31) - : : :- CometSort (25) - : : : +- CometExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (22) - : : +- CometSort (30) - : : +- CometExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (26) - : +- ReusedExchange (34) - +- CometSort (63) - +- CometFilter (62) - +- CometHashAggregate (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometProject (55) - : +- CometFilter (54) - : +- CometSortMergeJoin (53) - : :- CometSort (47) - : : +- CometExchange (46) - : : +- CometFilter (45) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (44) - : +- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometFilter (49) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (48) - +- ReusedExchange (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_customer_sk#2)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_ticket_number#3, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_ticket_number#3 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) - -(7) CometProject -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] - -(8) CometExchange -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter - -(11) CometFilter -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Condition : isnull(sr_ticket_number#10) - -(12) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(15) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(16) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight - -(17) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] - -(18) CometHashAggregate -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] - -(19) CometExchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometHashAggregate -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] - -(21) CometSort -Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(23) CometFilter -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) - -(24) CometExchange -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometSort -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Condition : (isnotnull(wr_order_number#30) AND isnotnull(wr_item_sk#29)) - -(28) CometProject -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_item_sk#29, wr_order_number#30] - -(29) CometExchange -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: hashpartitioning(wr_order_number#30, wr_item_sk#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(30) CometSort -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_order_number#30 ASC NULLS FIRST, wr_item_sk#29 ASC NULLS FIRST] - -(31) CometSortMergeJoin -Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#30, wr_item_sk#29], LeftOuter - -(32) CometFilter -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Condition : isnull(wr_order_number#30) - -(33) CometProject -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(34) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#32, d_year#33] - -(35) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ws_sold_date_sk#27], [d_date_sk#32], Inner, BuildRight - -(36) CometProject -Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#32, d_year#33] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] - -(37) CometHashAggregate -Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] - -(38) CometExchange -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Arguments: hashpartitioning(d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(39) CometHashAggregate -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] - -(40) CometFilter -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Condition : (coalesce(ws_qty#39, 0) > 0) - -(41) CometSort -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41], [ws_sold_year#37 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#38 ASC NULLS FIRST] - -(42) CometSortMergeJoin -Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Right output [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38], Inner - -(43) CometProject -Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#48), dynamicpruningexpression(cs_sold_date_sk#48 IN dynamicpruning#49)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(45) CometFilter -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Condition : (isnotnull(cs_item_sk#43) AND isnotnull(cs_bill_customer_sk#42)) - -(46) CometExchange -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(47) CometSort -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST] - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Condition : (isnotnull(cr_order_number#51) AND isnotnull(cr_item_sk#50)) - -(50) CometProject -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_item_sk#50, cr_order_number#51] - -(51) CometExchange -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(52) CometSort -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST] - -(53) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cs_order_number#44, cs_item_sk#43], [cr_order_number#51, cr_item_sk#50], LeftOuter - -(54) CometFilter -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Condition : isnull(cr_order_number#51) - -(55) CometProject -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] - -(56) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#53, d_year#54] - -(57) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [cs_sold_date_sk#48], [d_date_sk#53], Inner, BuildRight - -(58) CometProject -Input [8]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, d_date_sk#53, d_year#54] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] - -(59) CometHashAggregate -Input [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [partial_sum(cs_quantity#45), partial_sum(UnscaledValue(cs_wholesale_cost#46)), partial_sum(UnscaledValue(cs_sales_price#47))] - -(60) CometExchange -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Arguments: hashpartitioning(d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(61) CometHashAggregate -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [sum(cs_quantity#45), sum(UnscaledValue(cs_wholesale_cost#46)), sum(UnscaledValue(cs_sales_price#47))] - -(62) CometFilter -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Condition : (coalesce(cs_qty#60, 0) > 0) - -(63) CometSort -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62], [cs_sold_year#58 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST, cs_customer_sk#59 ASC NULLS FIRST] - -(64) CometSortMergeJoin -Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] -Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner - -(65) CometColumnarToRow [codegen id : 1] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(66) Project [codegen id : 1] -Output [13]: [round((cast(ss_qty#18 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)))), 2) AS ratio#63, ss_qty#18 AS store_qty#64, ss_wc#19 AS store_wholesale_cost#65, ss_sp#20 AS store_sales_price#66, (coalesce(ws_qty#39, 0) + coalesce(cs_qty#60, 0)) AS other_chan_qty#67, (coalesce(ws_wc#40, 0.00) + coalesce(cs_wc#61, 0.00)) AS other_chan_wholesale_cost#68, (coalesce(ws_sp#41, 0.00) + coalesce(cs_sp#62, 0.00)) AS other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(67) TakeOrderedAndProject -Input [13]: [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#67 ASC NULLS FIRST, other_chan_wholesale_cost#68 ASC NULLS FIRST, other_chan_sales_price#69 ASC NULLS FIRST, ratio#63 ASC NULLS FIRST], [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) - - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(70) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(71) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/extended.txt deleted file mode 100644 index 9d5ba53be1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/extended.txt +++ /dev/null @@ -1,80 +0,0 @@ -TakeOrderedAndProject -+- Project [COMET: Comet does not support Spark's BigDecimal rounding] - +- CometColumnarToRow - +- CometSortMergeJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 273db28e77..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78/simplified.txt +++ /dev/null @@ -1,78 +0,0 @@ -TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (1) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] - CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #5 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] - CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometExchange [ws_order_number,ws_item_sk] #7 - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_order_number,wr_item_sk] #8 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] - CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometExchange [cs_order_number,cs_item_sk] #10 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_order_number,cr_item_sk] #11 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/explain.txt deleted file mode 100644 index ae7a6e67aa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/explain.txt +++ /dev/null @@ -1,695 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (119) -+- CometTakeOrderedAndProject (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometUnion (114) - :- CometHashAggregate (103) - : +- CometExchange (102) - : +- CometHashAggregate (101) - : +- CometUnion (100) - : :- CometHashAggregate (39) - : : +- CometExchange (38) - : : +- CometHashAggregate (37) - : : +- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (30) - : : : +- CometBroadcastHashJoin (29) - : : : :- CometProject (24) - : : : : +- CometBroadcastHashJoin (23) - : : : : :- CometProject (18) - : : : : : +- CometBroadcastHashJoin (17) - : : : : : :- CometProject (12) - : : : : : : +- CometSortMergeJoin (11) - : : : : : : :- CometSort (5) - : : : : : : : +- CometColumnarExchange (4) - : : : : : : : +- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (10) - : : : : : : +- CometExchange (9) - : : : : : : +- CometProject (8) - : : : : : : +- CometFilter (7) - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (6) - : : : : : +- CometBroadcastExchange (16) - : : : : : +- CometProject (15) - : : : : : +- CometFilter (14) - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (13) - : : : : +- CometBroadcastExchange (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometNativeScan parquet spark_catalog.default.store (19) - : : : +- CometBroadcastExchange (28) - : : : +- CometProject (27) - : : : +- CometFilter (26) - : : : +- CometNativeScan parquet spark_catalog.default.item (25) - : : +- CometBroadcastExchange (34) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometNativeScan parquet spark_catalog.default.promotion (31) - : :- CometHashAggregate (69) - : : +- CometExchange (68) - : : +- CometHashAggregate (67) - : : +- CometProject (66) - : : +- CometBroadcastHashJoin (65) - : : :- CometProject (63) - : : : +- CometBroadcastHashJoin (62) - : : : :- CometProject (60) - : : : : +- CometBroadcastHashJoin (59) - : : : : :- CometProject (54) - : : : : : +- CometBroadcastHashJoin (53) - : : : : : :- CometProject (51) - : : : : : : +- CometSortMergeJoin (50) - : : : : : : :- CometSort (44) - : : : : : : : +- CometColumnarExchange (43) - : : : : : : : +- * Filter (42) - : : : : : : : +- * ColumnarToRow (41) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (40) - : : : : : : +- CometSort (49) - : : : : : : +- CometExchange (48) - : : : : : : +- CometProject (47) - : : : : : : +- CometFilter (46) - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (45) - : : : : : +- ReusedExchange (52) - : : : : +- CometBroadcastExchange (58) - : : : : +- CometProject (57) - : : : : +- CometFilter (56) - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_page (55) - : : : +- ReusedExchange (61) - : : +- ReusedExchange (64) - : +- CometHashAggregate (99) - : +- CometExchange (98) - : +- CometHashAggregate (97) - : +- CometProject (96) - : +- CometBroadcastHashJoin (95) - : :- CometProject (93) - : : +- CometBroadcastHashJoin (92) - : : :- CometProject (90) - : : : +- CometBroadcastHashJoin (89) - : : : :- CometProject (84) - : : : : +- CometBroadcastHashJoin (83) - : : : : :- CometProject (81) - : : : : : +- CometSortMergeJoin (80) - : : : : : :- CometSort (74) - : : : : : : +- CometColumnarExchange (73) - : : : : : : +- * Filter (72) - : : : : : : +- * ColumnarToRow (71) - : : : : : : +- Scan parquet spark_catalog.default.web_sales (70) - : : : : : +- CometSort (79) - : : : : : +- CometExchange (78) - : : : : : +- CometProject (77) - : : : : : +- CometFilter (76) - : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns (75) - : : : : +- ReusedExchange (82) - : : : +- CometBroadcastExchange (88) - : : : +- CometProject (87) - : : : +- CometFilter (86) - : : : +- CometNativeScan parquet spark_catalog.default.web_site (85) - : : +- ReusedExchange (91) - : +- ReusedExchange (94) - :- CometHashAggregate (108) - : +- CometExchange (107) - : +- CometHashAggregate (106) - : +- CometHashAggregate (105) - : +- ReusedExchange (104) - +- CometHashAggregate (113) - +- CometExchange (112) - +- CometHashAggregate (111) - +- CometHashAggregate (110) - +- ReusedExchange (109) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] - -(3) Filter [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(4) CometColumnarExchange -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(7) CometFilter -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) - -(8) CometProject -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(9) CometExchange -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter - -(12) CometProject -Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] - -(13) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) - -(15) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] - -(19) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_store_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [s_store_sk#16, s_store_id#17] -Condition : isnotnull(s_store_sk#16) - -(21) CometProject -Input [2]: [s_store_sk#16, s_store_id#17] -Arguments: [s_store_sk#16, s_store_id#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#17, 16, true, false, true) AS s_store_id#18] - -(22) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_store_id#18] -Arguments: [s_store_sk#16, s_store_id#18] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -Right output [2]: [s_store_sk#16, s_store_id#18] -Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight - -(24) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#18] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(25) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_current_price#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [i_item_sk#19, i_current_price#20] -Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) - -(27) CometProject -Input [2]: [i_item_sk#19, i_current_price#20] -Arguments: [i_item_sk#19], [i_item_sk#19] - -(28) CometBroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: [i_item_sk#19] - -(29) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [i_item_sk#19] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, i_item_sk#19] -Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(31) CometNativeScan parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#21, p_channel_tv#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#22, 1, true, false, true) = N) AND isnotnull(p_promo_sk#21)) - -(33) CometProject -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Arguments: [p_promo_sk#21], [p_promo_sk#21] - -(34) CometBroadcastExchange -Input [1]: [p_promo_sk#21] -Arguments: [p_promo_sk#21] - -(35) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [p_promo_sk#21] -Arguments: [ss_promo_sk#3], [p_promo_sk#21], Inner, BuildRight - -(36) CometProject -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, p_promo_sk#21] -Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(37) CometHashAggregate -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Keys [1]: [s_store_id#18] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(38) CometExchange -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(39) CometHashAggregate -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [s_store_id#18] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(40) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(41) ColumnarToRow [codegen id : 2] -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] - -(42) Filter [codegen id : 2] -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) - -(43) CometColumnarExchange -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(44) CometSort -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] - -(45) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38, cr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(46) CometFilter -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38, cr_returned_date_sk#39] -Condition : (isnotnull(cr_item_sk#35) AND isnotnull(cr_order_number#36)) - -(47) CometProject -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38, cr_returned_date_sk#39] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38], [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] - -(48) CometExchange -Input [4]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: hashpartitioning(cr_item_sk#35, cr_order_number#36, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(49) CometSort -Input [4]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38], [cr_item_sk#35 ASC NULLS FIRST, cr_order_number#36 ASC NULLS FIRST] - -(50) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#35, cr_order_number#36], LeftOuter - -(51) CometProject -Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38] - -(52) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#40] - -(53) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38] -Right output [1]: [d_date_sk#40] -Arguments: [cs_sold_date_sk#34], [d_date_sk#40], Inner, BuildRight - -(54) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38, d_date_sk#40] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38] - -(55) CometNativeScan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#42] -Condition : isnotnull(cp_catalog_page_sk#41) - -(57) CometProject -Input [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#42] -Arguments: [cp_catalog_page_sk#41, cp_catalog_page_id#43], [cp_catalog_page_sk#41, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#42, 16, true, false, true) AS cp_catalog_page_id#43] - -(58) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#43] -Arguments: [cp_catalog_page_sk#41, cp_catalog_page_id#43] - -(59) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38] -Right output [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#43] -Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#41], Inner, BuildRight - -(60) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_sk#41, cp_catalog_page_id#43] -Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] - -(61) ReusedExchange [Reuses operator id: 28] -Output [1]: [i_item_sk#44] - -(62) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] -Right output [1]: [i_item_sk#44] -Arguments: [cs_item_sk#29], [i_item_sk#44], Inner, BuildRight - -(63) CometProject -Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43, i_item_sk#44] -Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] - -(64) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#45] - -(65) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] -Right output [1]: [p_promo_sk#45] -Arguments: [cs_promo_sk#30], [p_promo_sk#45], Inner, BuildRight - -(66) CometProject -Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43, p_promo_sk#45] -Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] - -(67) CometHashAggregate -Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] -Keys [1]: [cp_catalog_page_id#43] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#37 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#38 as decimal(12,2)), 0.00)))] - -(68) CometExchange -Input [6]: [cp_catalog_page_id#43, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] -Arguments: hashpartitioning(cp_catalog_page_id#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(69) CometHashAggregate -Input [6]: [cp_catalog_page_id#43, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] -Keys [1]: [cp_catalog_page_id#43] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#37 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#38 as decimal(12,2)), 0.00)))] - -(70) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#57), dynamicpruningexpression(ws_sold_date_sk#57 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] -ReadSchema: struct - -(71) ColumnarToRow [codegen id : 3] -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] - -(72) Filter [codegen id : 3] -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Condition : ((isnotnull(ws_web_site_sk#52) AND isnotnull(ws_item_sk#51)) AND isnotnull(ws_promo_sk#53)) - -(73) CometColumnarExchange -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Arguments: hashpartitioning(ws_item_sk#51, ws_order_number#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(74) CometSort -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Arguments: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57], [ws_item_sk#51 ASC NULLS FIRST, ws_order_number#54 ASC NULLS FIRST] - -(75) CometNativeScan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] -ReadSchema: struct - -(76) CometFilter -Input [5]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Condition : (isnotnull(wr_item_sk#58) AND isnotnull(wr_order_number#59)) - -(77) CometProject -Input [5]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Arguments: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61], [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] - -(78) CometExchange -Input [4]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: hashpartitioning(wr_item_sk#58, wr_order_number#59, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(79) CometSort -Input [4]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61], [wr_item_sk#58 ASC NULLS FIRST, wr_order_number#59 ASC NULLS FIRST] - -(80) CometSortMergeJoin -Left output [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Right output [4]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: [ws_item_sk#51, ws_order_number#54], [wr_item_sk#58, wr_order_number#59], LeftOuter - -(81) CometProject -Input [11]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61], [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61] - -(82) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#63] - -(83) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61] -Right output [1]: [d_date_sk#63] -Arguments: [ws_sold_date_sk#57], [d_date_sk#63], Inner, BuildRight - -(84) CometProject -Input [9]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61, d_date_sk#63] -Arguments: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61], [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61] - -(85) CometNativeScan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#64, web_site_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(86) CometFilter -Input [2]: [web_site_sk#64, web_site_id#65] -Condition : isnotnull(web_site_sk#64) - -(87) CometProject -Input [2]: [web_site_sk#64, web_site_id#65] -Arguments: [web_site_sk#64, web_site_id#66], [web_site_sk#64, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#65, 16, true, false, true) AS web_site_id#66] - -(88) CometBroadcastExchange -Input [2]: [web_site_sk#64, web_site_id#66] -Arguments: [web_site_sk#64, web_site_id#66] - -(89) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61] -Right output [2]: [web_site_sk#64, web_site_id#66] -Arguments: [ws_web_site_sk#52], [web_site_sk#64], Inner, BuildRight - -(90) CometProject -Input [9]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_sk#64, web_site_id#66] -Arguments: [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66], [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] - -(91) ReusedExchange [Reuses operator id: 28] -Output [1]: [i_item_sk#67] - -(92) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] -Right output [1]: [i_item_sk#67] -Arguments: [ws_item_sk#51], [i_item_sk#67], Inner, BuildRight - -(93) CometProject -Input [8]: [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66, i_item_sk#67] -Arguments: [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66], [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] - -(94) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#68] - -(95) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] -Right output [1]: [p_promo_sk#68] -Arguments: [ws_promo_sk#53], [p_promo_sk#68], Inner, BuildRight - -(96) CometProject -Input [7]: [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66, p_promo_sk#68] -Arguments: [ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66], [ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] - -(97) CometHashAggregate -Input [5]: [ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] -Keys [1]: [web_site_id#66] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#55)), partial_sum(coalesce(cast(wr_return_amt#60 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#56 - coalesce(cast(wr_net_loss#61 as decimal(12,2)), 0.00)))] - -(98) CometExchange -Input [6]: [web_site_id#66, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Arguments: hashpartitioning(web_site_id#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(99) CometHashAggregate -Input [6]: [web_site_id#66, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Keys [1]: [web_site_id#66] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#55)), sum(coalesce(cast(wr_return_amt#60 as decimal(12,2)), 0.00)), sum((ws_net_profit#56 - coalesce(cast(wr_net_loss#61 as decimal(12,2)), 0.00)))] - -(100) CometUnion -Child 0 Input [5]: [channel#74, id#75, sales#76, returns#77, profit#78] -Child 1 Input [5]: [channel#79, id#80, sales#81, returns#82, profit#83] -Child 2 Input [5]: [channel#84, id#85, sales#86, returns#87, profit#88] - -(101) CometHashAggregate -Input [5]: [channel#74, id#75, sales#76, returns#77, profit#78] -Keys [2]: [channel#74, id#75] -Functions [3]: [partial_sum(sales#76), partial_sum(returns#77), partial_sum(profit#78)] - -(102) CometExchange -Input [8]: [channel#74, id#75, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94] -Arguments: hashpartitioning(channel#74, id#75, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(103) CometHashAggregate -Input [8]: [channel#74, id#75, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94] -Keys [2]: [channel#74, id#75] -Functions [3]: [sum(sales#76), sum(returns#77), sum(profit#78)] - -(104) ReusedExchange [Reuses operator id: 102] -Output [8]: [channel#74, id#75, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94] - -(105) CometHashAggregate -Input [8]: [channel#74, id#75, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94] -Keys [2]: [channel#74, id#75] -Functions [3]: [sum(sales#76), sum(returns#77), sum(profit#78)] - -(106) CometHashAggregate -Input [4]: [channel#74, sales#95, returns#96, profit#97] -Keys [1]: [channel#74] -Functions [3]: [partial_sum(sales#95), partial_sum(returns#96), partial_sum(profit#97)] - -(107) CometExchange -Input [7]: [channel#74, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103] -Arguments: hashpartitioning(channel#74, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(108) CometHashAggregate -Input [7]: [channel#74, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103] -Keys [1]: [channel#74] -Functions [3]: [sum(sales#95), sum(returns#96), sum(profit#97)] - -(109) ReusedExchange [Reuses operator id: 102] -Output [8]: [channel#74, id#75, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94] - -(110) CometHashAggregate -Input [8]: [channel#74, id#75, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94] -Keys [2]: [channel#74, id#75] -Functions [3]: [sum(sales#76), sum(returns#77), sum(profit#78)] - -(111) CometHashAggregate -Input [3]: [sales#104, returns#105, profit#106] -Keys: [] -Functions [3]: [partial_sum(sales#104), partial_sum(returns#105), partial_sum(profit#106)] - -(112) CometExchange -Input [6]: [sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(113) CometHashAggregate -Input [6]: [sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112] -Keys: [] -Functions [3]: [sum(sales#104), sum(returns#105), sum(profit#106)] - -(114) CometUnion -Child 0 Input [5]: [channel#74, id#75, sales#113, returns#114, profit#115] -Child 1 Input [5]: [channel#74, id#116, sales#117, returns#118, profit#119] -Child 2 Input [5]: [channel#120, id#121, sales#122, returns#123, profit#124] - -(115) CometHashAggregate -Input [5]: [channel#74, id#75, sales#113, returns#114, profit#115] -Keys [5]: [channel#74, id#75, sales#113, returns#114, profit#115] -Functions: [] - -(116) CometExchange -Input [5]: [channel#74, id#75, sales#113, returns#114, profit#115] -Arguments: hashpartitioning(channel#74, id#75, sales#113, returns#114, profit#115, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] - -(117) CometHashAggregate -Input [5]: [channel#74, id#75, sales#113, returns#114, profit#115] -Keys [5]: [channel#74, id#75, sales#113, returns#114, profit#115] -Functions: [] - -(118) CometTakeOrderedAndProject -Input [5]: [channel#74, id#75, sales#113, returns#114, profit#115] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#74 ASC NULLS FIRST,id#75 ASC NULLS FIRST], output=[channel#74,id#75,sales#113,returns#114,profit#115]), [channel#74, id#75, sales#113, returns#114, profit#115], 100, 0, [channel#74 ASC NULLS FIRST, id#75 ASC NULLS FIRST], [channel#74, id#75, sales#113, returns#114, profit#115] - -(119) CometColumnarToRow [codegen id : 10] -Input [5]: [channel#74, id#75, sales#113, returns#114, profit#115] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (124) -+- * CometColumnarToRow (123) - +- CometProject (122) - +- CometFilter (121) - +- CometNativeScan parquet spark_catalog.default.date_dim (120) - - -(120) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(121) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) - -(122) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(123) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(124) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#57 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/simplified.txt deleted file mode 100644 index cfce9e20a1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/simplified.txt +++ /dev/null @@ -1,137 +0,0 @@ -WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id] #2 - CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [s_store_id] #3 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk,ss_ticket_number] #4 - WholeStageCodegen (1) - Filter [ss_store_sk,ss_item_sk,ss_promo_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #6 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #8 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #9 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #10 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,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)))] - CometExchange [cp_catalog_page_id] #11 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometColumnarExchange [cs_item_sk,cs_order_number] #12 - WholeStageCodegen (2) - Filter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [web_site_id] #15 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometColumnarExchange [ws_item_sk,ws_order_number] #16 - WholeStageCodegen (3) - Filter [ws_web_site_sk,ws_item_sk,ws_promo_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #17 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [web_site_sk,web_site_id] #18 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel] #19 - CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange #20 - CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/explain.txt deleted file mode 100644 index f9901f13d7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,683 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (116) -+- CometTakeOrderedAndProject (115) - +- CometHashAggregate (114) - +- CometExchange (113) - +- CometHashAggregate (112) - +- CometUnion (111) - :- CometHashAggregate (100) - : +- CometExchange (99) - : +- CometHashAggregate (98) - : +- CometUnion (97) - : :- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometProject (29) - : : : +- CometBroadcastHashJoin (28) - : : : :- CometProject (23) - : : : : +- CometBroadcastHashJoin (22) - : : : : :- CometProject (17) - : : : : : +- CometBroadcastHashJoin (16) - : : : : : :- CometProject (11) - : : : : : : +- CometSortMergeJoin (10) - : : : : : : :- CometSort (4) - : : : : : : : +- CometExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (9) - : : : : : : +- CometExchange (8) - : : : : : : +- CometProject (7) - : : : : : : +- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : : +- CometBroadcastExchange (15) - : : : : : +- CometProject (14) - : : : : : +- CometFilter (13) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) - : : : : +- CometBroadcastExchange (21) - : : : : +- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) - : : : +- CometBroadcastExchange (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - : :- CometHashAggregate (67) - : : +- CometExchange (66) - : : +- CometHashAggregate (65) - : : +- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometProject (61) - : : : +- CometBroadcastHashJoin (60) - : : : :- CometProject (58) - : : : : +- CometBroadcastHashJoin (57) - : : : : :- CometProject (52) - : : : : : +- CometBroadcastHashJoin (51) - : : : : : :- CometProject (49) - : : : : : : +- CometSortMergeJoin (48) - : : : : : : :- CometSort (42) - : : : : : : : +- CometExchange (41) - : : : : : : : +- CometFilter (40) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) - : : : : : : +- CometSort (47) - : : : : : : +- CometExchange (46) - : : : : : : +- CometProject (45) - : : : : : : +- CometFilter (44) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) - : : : : : +- ReusedExchange (50) - : : : : +- CometBroadcastExchange (56) - : : : : +- CometProject (55) - : : : : +- CometFilter (54) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) - : : : +- ReusedExchange (59) - : : +- ReusedExchange (62) - : +- CometHashAggregate (96) - : +- CometExchange (95) - : +- CometHashAggregate (94) - : +- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (87) - : : : +- CometBroadcastHashJoin (86) - : : : :- CometProject (81) - : : : : +- CometBroadcastHashJoin (80) - : : : : :- CometProject (78) - : : : : : +- CometSortMergeJoin (77) - : : : : : :- CometSort (71) - : : : : : : +- CometExchange (70) - : : : : : : +- CometFilter (69) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) - : : : : : +- CometSort (76) - : : : : : +- CometExchange (75) - : : : : : +- CometProject (74) - : : : : : +- CometFilter (73) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) - : : : : +- ReusedExchange (79) - : : : +- CometBroadcastExchange (85) - : : : +- CometProject (84) - : : : +- CometFilter (83) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) - : : +- ReusedExchange (88) - : +- ReusedExchange (91) - :- CometHashAggregate (105) - : +- CometExchange (104) - : +- CometHashAggregate (103) - : +- CometHashAggregate (102) - : +- ReusedExchange (101) - +- CometHashAggregate (110) - +- CometExchange (109) - +- CometHashAggregate (108) - +- CometHashAggregate (107) - +- ReusedExchange (106) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) - -(7) CometProject -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(8) CometExchange -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter - -(11) CometProject -Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) - -(14) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(15) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(16) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight - -(17) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_store_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [s_store_sk#16, s_store_id#17] -Condition : isnotnull(s_store_sk#16) - -(20) CometProject -Input [2]: [s_store_sk#16, s_store_id#17] -Arguments: [s_store_sk#16, s_store_id#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#17, 16, true, false, true) AS s_store_id#18] - -(21) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_store_id#18] -Arguments: [s_store_sk#16, s_store_id#18] - -(22) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -Right output [2]: [s_store_sk#16, s_store_id#18] -Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight - -(23) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#18] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_current_price#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [i_item_sk#19, i_current_price#20] -Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) - -(26) CometProject -Input [2]: [i_item_sk#19, i_current_price#20] -Arguments: [i_item_sk#19], [i_item_sk#19] - -(27) CometBroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: [i_item_sk#19] - -(28) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [i_item_sk#19] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(29) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, i_item_sk#19] -Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#21, p_channel_tv#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#22, 1, true, false, true) = N) AND isnotnull(p_promo_sk#21)) - -(32) CometProject -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Arguments: [p_promo_sk#21], [p_promo_sk#21] - -(33) CometBroadcastExchange -Input [1]: [p_promo_sk#21] -Arguments: [p_promo_sk#21] - -(34) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [p_promo_sk#21] -Arguments: [ss_promo_sk#3], [p_promo_sk#21], Inner, BuildRight - -(35) CometProject -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, p_promo_sk#21] -Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(36) CometHashAggregate -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Keys [1]: [s_store_id#18] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(37) CometExchange -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [s_store_id#18] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(40) CometFilter -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) - -(41) CometExchange -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(42) CometSort -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(44) CometFilter -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) - -(45) CometProject -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] - -(46) CometExchange -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometSort -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] - -(48) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter - -(49) CometProject -Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] - -(50) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#41] - -(51) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -Right output [1]: [d_date_sk#41] -Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight - -(52) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(54) CometFilter -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Condition : isnotnull(cp_catalog_page_sk#42) - -(55) CometProject -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#43, 16, true, false, true) AS cp_catalog_page_id#44] - -(56) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] - -(57) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight - -(58) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(59) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#45] - -(60) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [i_item_sk#45] -Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight - -(61) CometProject -Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] -Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(62) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#46] - -(63) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [p_promo_sk#46] -Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight - -(64) CometProject -Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] -Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(65) CometHashAggregate -Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(66) CometExchange -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(67) CometHashAggregate -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] -ReadSchema: struct - -(69) CometFilter -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) - -(70) CometExchange -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(71) CometSort -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] -ReadSchema: struct - -(73) CometFilter -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) - -(74) CometProject -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] - -(75) CometExchange -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(76) CometSort -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] - -(77) CometSortMergeJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter - -(78) CometProject -Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] - -(79) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#65] - -(80) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -Right output [1]: [d_date_sk#65] -Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(81) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] - -(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#66, web_site_id#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(83) CometFilter -Input [2]: [web_site_sk#66, web_site_id#67] -Condition : isnotnull(web_site_sk#66) - -(84) CometProject -Input [2]: [web_site_sk#66, web_site_id#67] -Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#67, 16, true, false, true) AS web_site_id#68] - -(85) CometBroadcastExchange -Input [2]: [web_site_sk#66, web_site_id#68] -Arguments: [web_site_sk#66, web_site_id#68] - -(86) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -Right output [2]: [web_site_sk#66, web_site_id#68] -Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight - -(87) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] -Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(88) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#69] - -(89) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [i_item_sk#69] -Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight - -(90) CometProject -Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] -Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(91) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#70] - -(92) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [p_promo_sk#70] -Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight - -(93) CometProject -Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] -Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(94) CometHashAggregate -Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Keys [1]: [web_site_id#68] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(95) CometExchange -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(96) CometHashAggregate -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Keys [1]: [web_site_id#68] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(97) CometUnion -Child 0 Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] -Child 1 Input [5]: [channel#81, id#82, sales#83, returns#84, profit#85] -Child 2 Input [5]: [channel#86, id#87, sales#88, returns#89, profit#90] - -(98) CometHashAggregate -Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] -Keys [2]: [channel#76, id#77] -Functions [3]: [partial_sum(sales#78), partial_sum(returns#79), partial_sum(profit#80)] - -(99) CometExchange -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Arguments: hashpartitioning(channel#76, id#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(100) CometHashAggregate -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Keys [2]: [channel#76, id#77] -Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] - -(101) ReusedExchange [Reuses operator id: 99] -Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] - -(102) CometHashAggregate -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Keys [2]: [channel#76, id#77] -Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] - -(103) CometHashAggregate -Input [4]: [channel#76, sales#97, returns#98, profit#99] -Keys [1]: [channel#76] -Functions [3]: [partial_sum(sales#97), partial_sum(returns#98), partial_sum(profit#99)] - -(104) CometExchange -Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Arguments: hashpartitioning(channel#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(105) CometHashAggregate -Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Keys [1]: [channel#76] -Functions [3]: [sum(sales#97), sum(returns#98), sum(profit#99)] - -(106) ReusedExchange [Reuses operator id: 99] -Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] - -(107) CometHashAggregate -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Keys [2]: [channel#76, id#77] -Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] - -(108) CometHashAggregate -Input [3]: [sales#106, returns#107, profit#108] -Keys: [] -Functions [3]: [partial_sum(sales#106), partial_sum(returns#107), partial_sum(profit#108)] - -(109) CometExchange -Input [6]: [sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(110) CometHashAggregate -Input [6]: [sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys: [] -Functions [3]: [sum(sales#106), sum(returns#107), sum(profit#108)] - -(111) CometUnion -Child 0 Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Child 1 Input [5]: [channel#76, id#118, sales#119, returns#120, profit#121] -Child 2 Input [5]: [channel#122, id#123, sales#124, returns#125, profit#126] - -(112) CometHashAggregate -Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Keys [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Functions: [] - -(113) CometExchange -Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Arguments: hashpartitioning(channel#76, id#77, sales#115, returns#116, profit#117, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] - -(114) CometHashAggregate -Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Keys [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Functions: [] - -(115) CometTakeOrderedAndProject -Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#76 ASC NULLS FIRST,id#77 ASC NULLS FIRST], output=[channel#76,id#77,sales#115,returns#116,profit#117]), [channel#76, id#77, sales#115, returns#116, profit#117], 100, 0, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#115, returns#116, profit#117] - -(116) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) - -(119) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(121) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/simplified.txt deleted file mode 100644 index d27ab827e1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,128 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id] #2 - CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [s_store_id] #3 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #4 - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #6 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #8 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #9 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #10 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,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)))] - CometExchange [cp_catalog_page_id] #11 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #12 - 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 [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [web_site_id] #15 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #16 - 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 [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #17 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [web_site_sk,web_site_id] #18 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel] #19 - CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange #20 - CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt deleted file mode 100644 index f9901f13d7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt +++ /dev/null @@ -1,683 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (116) -+- CometTakeOrderedAndProject (115) - +- CometHashAggregate (114) - +- CometExchange (113) - +- CometHashAggregate (112) - +- CometUnion (111) - :- CometHashAggregate (100) - : +- CometExchange (99) - : +- CometHashAggregate (98) - : +- CometUnion (97) - : :- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometProject (29) - : : : +- CometBroadcastHashJoin (28) - : : : :- CometProject (23) - : : : : +- CometBroadcastHashJoin (22) - : : : : :- CometProject (17) - : : : : : +- CometBroadcastHashJoin (16) - : : : : : :- CometProject (11) - : : : : : : +- CometSortMergeJoin (10) - : : : : : : :- CometSort (4) - : : : : : : : +- CometExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (9) - : : : : : : +- CometExchange (8) - : : : : : : +- CometProject (7) - : : : : : : +- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : : +- CometBroadcastExchange (15) - : : : : : +- CometProject (14) - : : : : : +- CometFilter (13) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) - : : : : +- CometBroadcastExchange (21) - : : : : +- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) - : : : +- CometBroadcastExchange (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - : :- CometHashAggregate (67) - : : +- CometExchange (66) - : : +- CometHashAggregate (65) - : : +- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometProject (61) - : : : +- CometBroadcastHashJoin (60) - : : : :- CometProject (58) - : : : : +- CometBroadcastHashJoin (57) - : : : : :- CometProject (52) - : : : : : +- CometBroadcastHashJoin (51) - : : : : : :- CometProject (49) - : : : : : : +- CometSortMergeJoin (48) - : : : : : : :- CometSort (42) - : : : : : : : +- CometExchange (41) - : : : : : : : +- CometFilter (40) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) - : : : : : : +- CometSort (47) - : : : : : : +- CometExchange (46) - : : : : : : +- CometProject (45) - : : : : : : +- CometFilter (44) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) - : : : : : +- ReusedExchange (50) - : : : : +- CometBroadcastExchange (56) - : : : : +- CometProject (55) - : : : : +- CometFilter (54) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) - : : : +- ReusedExchange (59) - : : +- ReusedExchange (62) - : +- CometHashAggregate (96) - : +- CometExchange (95) - : +- CometHashAggregate (94) - : +- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (87) - : : : +- CometBroadcastHashJoin (86) - : : : :- CometProject (81) - : : : : +- CometBroadcastHashJoin (80) - : : : : :- CometProject (78) - : : : : : +- CometSortMergeJoin (77) - : : : : : :- CometSort (71) - : : : : : : +- CometExchange (70) - : : : : : : +- CometFilter (69) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) - : : : : : +- CometSort (76) - : : : : : +- CometExchange (75) - : : : : : +- CometProject (74) - : : : : : +- CometFilter (73) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) - : : : : +- ReusedExchange (79) - : : : +- CometBroadcastExchange (85) - : : : +- CometProject (84) - : : : +- CometFilter (83) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) - : : +- ReusedExchange (88) - : +- ReusedExchange (91) - :- CometHashAggregate (105) - : +- CometExchange (104) - : +- CometHashAggregate (103) - : +- CometHashAggregate (102) - : +- ReusedExchange (101) - +- CometHashAggregate (110) - +- CometExchange (109) - +- CometHashAggregate (108) - +- CometHashAggregate (107) - +- ReusedExchange (106) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) - -(7) CometProject -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(8) CometExchange -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter - -(11) CometProject -Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) - -(14) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(15) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(16) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight - -(17) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_store_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [s_store_sk#16, s_store_id#17] -Condition : isnotnull(s_store_sk#16) - -(20) CometProject -Input [2]: [s_store_sk#16, s_store_id#17] -Arguments: [s_store_sk#16, s_store_id#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#17, 16, true, false, true) AS s_store_id#18] - -(21) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_store_id#18] -Arguments: [s_store_sk#16, s_store_id#18] - -(22) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -Right output [2]: [s_store_sk#16, s_store_id#18] -Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight - -(23) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#18] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_current_price#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [i_item_sk#19, i_current_price#20] -Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) - -(26) CometProject -Input [2]: [i_item_sk#19, i_current_price#20] -Arguments: [i_item_sk#19], [i_item_sk#19] - -(27) CometBroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: [i_item_sk#19] - -(28) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [i_item_sk#19] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(29) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, i_item_sk#19] -Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#21, p_channel_tv#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#22, 1, true, false, true) = N) AND isnotnull(p_promo_sk#21)) - -(32) CometProject -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Arguments: [p_promo_sk#21], [p_promo_sk#21] - -(33) CometBroadcastExchange -Input [1]: [p_promo_sk#21] -Arguments: [p_promo_sk#21] - -(34) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [p_promo_sk#21] -Arguments: [ss_promo_sk#3], [p_promo_sk#21], Inner, BuildRight - -(35) CometProject -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, p_promo_sk#21] -Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(36) CometHashAggregate -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Keys [1]: [s_store_id#18] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(37) CometExchange -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [s_store_id#18] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(40) CometFilter -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) - -(41) CometExchange -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(42) CometSort -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(44) CometFilter -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) - -(45) CometProject -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] - -(46) CometExchange -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometSort -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] - -(48) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter - -(49) CometProject -Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] - -(50) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#41] - -(51) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -Right output [1]: [d_date_sk#41] -Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight - -(52) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(54) CometFilter -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Condition : isnotnull(cp_catalog_page_sk#42) - -(55) CometProject -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#43, 16, true, false, true) AS cp_catalog_page_id#44] - -(56) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] - -(57) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight - -(58) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(59) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#45] - -(60) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [i_item_sk#45] -Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight - -(61) CometProject -Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] -Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(62) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#46] - -(63) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [p_promo_sk#46] -Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight - -(64) CometProject -Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] -Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(65) CometHashAggregate -Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(66) CometExchange -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(67) CometHashAggregate -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] -ReadSchema: struct - -(69) CometFilter -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) - -(70) CometExchange -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(71) CometSort -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] -ReadSchema: struct - -(73) CometFilter -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) - -(74) CometProject -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] - -(75) CometExchange -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(76) CometSort -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] - -(77) CometSortMergeJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter - -(78) CometProject -Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] - -(79) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#65] - -(80) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -Right output [1]: [d_date_sk#65] -Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(81) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] - -(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#66, web_site_id#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(83) CometFilter -Input [2]: [web_site_sk#66, web_site_id#67] -Condition : isnotnull(web_site_sk#66) - -(84) CometProject -Input [2]: [web_site_sk#66, web_site_id#67] -Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#67, 16, true, false, true) AS web_site_id#68] - -(85) CometBroadcastExchange -Input [2]: [web_site_sk#66, web_site_id#68] -Arguments: [web_site_sk#66, web_site_id#68] - -(86) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -Right output [2]: [web_site_sk#66, web_site_id#68] -Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight - -(87) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] -Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(88) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#69] - -(89) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [i_item_sk#69] -Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight - -(90) CometProject -Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] -Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(91) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#70] - -(92) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [p_promo_sk#70] -Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight - -(93) CometProject -Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] -Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(94) CometHashAggregate -Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Keys [1]: [web_site_id#68] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(95) CometExchange -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(96) CometHashAggregate -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Keys [1]: [web_site_id#68] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(97) CometUnion -Child 0 Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] -Child 1 Input [5]: [channel#81, id#82, sales#83, returns#84, profit#85] -Child 2 Input [5]: [channel#86, id#87, sales#88, returns#89, profit#90] - -(98) CometHashAggregate -Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] -Keys [2]: [channel#76, id#77] -Functions [3]: [partial_sum(sales#78), partial_sum(returns#79), partial_sum(profit#80)] - -(99) CometExchange -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Arguments: hashpartitioning(channel#76, id#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(100) CometHashAggregate -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Keys [2]: [channel#76, id#77] -Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] - -(101) ReusedExchange [Reuses operator id: 99] -Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] - -(102) CometHashAggregate -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Keys [2]: [channel#76, id#77] -Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] - -(103) CometHashAggregate -Input [4]: [channel#76, sales#97, returns#98, profit#99] -Keys [1]: [channel#76] -Functions [3]: [partial_sum(sales#97), partial_sum(returns#98), partial_sum(profit#99)] - -(104) CometExchange -Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Arguments: hashpartitioning(channel#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(105) CometHashAggregate -Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Keys [1]: [channel#76] -Functions [3]: [sum(sales#97), sum(returns#98), sum(profit#99)] - -(106) ReusedExchange [Reuses operator id: 99] -Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] - -(107) CometHashAggregate -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Keys [2]: [channel#76, id#77] -Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] - -(108) CometHashAggregate -Input [3]: [sales#106, returns#107, profit#108] -Keys: [] -Functions [3]: [partial_sum(sales#106), partial_sum(returns#107), partial_sum(profit#108)] - -(109) CometExchange -Input [6]: [sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(110) CometHashAggregate -Input [6]: [sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys: [] -Functions [3]: [sum(sales#106), sum(returns#107), sum(profit#108)] - -(111) CometUnion -Child 0 Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Child 1 Input [5]: [channel#76, id#118, sales#119, returns#120, profit#121] -Child 2 Input [5]: [channel#122, id#123, sales#124, returns#125, profit#126] - -(112) CometHashAggregate -Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Keys [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Functions: [] - -(113) CometExchange -Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Arguments: hashpartitioning(channel#76, id#77, sales#115, returns#116, profit#117, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] - -(114) CometHashAggregate -Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Keys [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Functions: [] - -(115) CometTakeOrderedAndProject -Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#76 ASC NULLS FIRST,id#77 ASC NULLS FIRST], output=[channel#76,id#77,sales#115,returns#116,profit#117]), [channel#76, id#77, sales#115, returns#116, profit#117], 100, 0, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#115, returns#116, profit#117] - -(116) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) - -(119) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(121) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/extended.txt deleted file mode 100644 index 0d6844d93c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/extended.txt +++ /dev/null @@ -1,392 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - -Comet accelerated 374 out of 386 eligible operators (96%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d27ab827e1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt +++ /dev/null @@ -1,128 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id] #2 - CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [s_store_id] #3 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #4 - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #6 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #8 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #9 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #10 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,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)))] - CometExchange [cp_catalog_page_id] #11 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #12 - 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 [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [web_site_id] #15 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #16 - 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 [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #17 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [web_site_sk,web_site_id] #18 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel] #19 - CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange #20 - CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/explain.txt deleted file mode 100644 index 03194c1609..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/explain.txt +++ /dev/null @@ -1,267 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (41) -+- * Project (40) - +- Window (39) - +- * CometColumnarToRow (38) - +- CometSort (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometColumnarExchange (34) - +- * HashAggregate (33) - +- Union (32) - :- * HashAggregate (17) - : +- * CometColumnarToRow (16) - : +- CometColumnarExchange (15) - : +- * HashAggregate (14) - : +- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) - : +- CometProject (9) - : +- CometFilter (8) - : +- CometNativeScan parquet spark_catalog.default.item (7) - :- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * HashAggregate (20) - : +- * CometColumnarToRow (19) - : +- ReusedExchange (18) - +- * HashAggregate (31) - +- * CometColumnarToRow (30) - +- CometColumnarExchange (29) - +- * HashAggregate (28) - +- * HashAggregate (27) - +- * CometColumnarToRow (26) - +- ReusedExchange (25) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(4) ReusedExchange [Reuses operator id: 46] -Output [1]: [d_date_sk#5] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [2]: [ws_item_sk#1, ws_net_paid#2] -Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] - -(7) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#6, i_class#7, i_category#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Condition : isnotnull(i_item_sk#6) - -(9) CometProject -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Arguments: [i_item_sk#6, i_class#9, i_category#10], [i_item_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#7, 50, true, false, true) AS i_class#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#8, 50, true, false, true) AS i_category#10] - -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#6, i_class#9, i_category#10] - -(11) BroadcastExchange -Input [3]: [i_item_sk#6, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [3]: [ws_net_paid#2, i_class#9, i_category#10] -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#6, i_class#9, i_category#10] - -(14) HashAggregate [codegen id : 3] -Input [3]: [ws_net_paid#2, i_class#9, i_category#10] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum#11] -Results [3]: [i_category#10, i_class#9, sum#12] - -(15) CometColumnarExchange -Input [3]: [i_category#10, i_class#9, sum#12] -Arguments: hashpartitioning(i_category#10, i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 4] -Input [3]: [i_category#10, i_class#9, sum#12] - -(17) HashAggregate [codegen id : 4] -Input [3]: [i_category#10, i_class#9, sum#12] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#13] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#13,17,2) as decimal(27,2)) AS total_sum#14, i_category#10 AS i_category#15, i_class#9 AS i_class#16, 0 AS g_category#17, 0 AS g_class#18, 0 AS lochierarchy#19] - -(18) ReusedExchange [Reuses operator id: 15] -Output [3]: [i_category#10, i_class#9, sum#20] - -(19) CometColumnarToRow [codegen id : 8] -Input [3]: [i_category#10, i_class#9, sum#20] - -(20) HashAggregate [codegen id : 8] -Input [3]: [i_category#10, i_class#9, sum#20] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#21))#13] -Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#21))#13,17,2) AS total_sum#22, i_category#10] - -(21) HashAggregate [codegen id : 8] -Input [2]: [total_sum#22, i_category#10] -Keys [1]: [i_category#10] -Functions [1]: [partial_sum(total_sum#22)] -Aggregate Attributes [2]: [sum#23, isEmpty#24] -Results [3]: [i_category#10, sum#25, isEmpty#26] - -(22) CometColumnarExchange -Input [3]: [i_category#10, sum#25, isEmpty#26] -Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 9] -Input [3]: [i_category#10, sum#25, isEmpty#26] - -(24) HashAggregate [codegen id : 9] -Input [3]: [i_category#10, sum#25, isEmpty#26] -Keys [1]: [i_category#10] -Functions [1]: [sum(total_sum#22)] -Aggregate Attributes [1]: [sum(total_sum#22)#27] -Results [6]: [sum(total_sum#22)#27 AS total_sum#28, i_category#10, null AS i_class#29, 0 AS g_category#30, 1 AS g_class#31, 1 AS lochierarchy#32] - -(25) ReusedExchange [Reuses operator id: 15] -Output [3]: [i_category#10, i_class#9, sum#33] - -(26) CometColumnarToRow [codegen id : 13] -Input [3]: [i_category#10, i_class#9, sum#33] - -(27) HashAggregate [codegen id : 13] -Input [3]: [i_category#10, i_class#9, sum#33] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#13] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#13,17,2) AS total_sum#35] - -(28) HashAggregate [codegen id : 13] -Input [1]: [total_sum#35] -Keys: [] -Functions [1]: [partial_sum(total_sum#35)] -Aggregate Attributes [2]: [sum#36, isEmpty#37] -Results [2]: [sum#38, isEmpty#39] - -(29) CometColumnarExchange -Input [2]: [sum#38, isEmpty#39] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 14] -Input [2]: [sum#38, isEmpty#39] - -(31) HashAggregate [codegen id : 14] -Input [2]: [sum#38, isEmpty#39] -Keys: [] -Functions [1]: [sum(total_sum#35)] -Aggregate Attributes [1]: [sum(total_sum#35)#40] -Results [6]: [sum(total_sum#35)#40 AS total_sum#41, null AS i_category#42, null AS i_class#43, 1 AS g_category#44, 1 AS g_class#45, 2 AS lochierarchy#46] - -(32) Union - -(33) HashAggregate [codegen id : 15] -Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Keys [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Functions: [] -Aggregate Attributes: [] -Results [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] - -(34) CometColumnarExchange -Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Arguments: hashpartitioning(total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(35) CometHashAggregate -Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Keys [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Functions: [] - -(36) CometExchange -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] -Arguments: hashpartitioning(lochierarchy#19, _w0#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(37) CometSort -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] -Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47], [lochierarchy#19 ASC NULLS FIRST, _w0#47 ASC NULLS FIRST, total_sum#14 DESC NULLS LAST] - -(38) CometColumnarToRow [codegen id : 16] -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] - -(39) Window -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47] -Arguments: [rank(total_sum#14) windowspecdefinition(lochierarchy#19, _w0#47, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#48], [lochierarchy#19, _w0#47], [total_sum#14 DESC NULLS LAST] - -(40) Project [codegen id : 17] -Output [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] -Input [6]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#47, rank_within_parent#48] - -(41) TakeOrderedAndProject -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] -Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#48 ASC NULLS FIRST], [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#48] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometNativeScan parquet spark_catalog.default.date_dim (42) - - -(42) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#49] -Condition : (((isnotnull(d_month_seq#49) AND (d_month_seq#49 >= 1212)) AND (d_month_seq#49 <= 1223)) AND isnotnull(d_date_sk#5)) - -(44) CometProject -Input [2]: [d_date_sk#5, d_month_seq#49] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(45) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(46) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/simplified.txt deleted file mode 100644 index 2ccc8c0c39..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (17) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] - CometColumnarExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - WholeStageCodegen (15) - HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (4) - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #3 - WholeStageCodegen (3) - HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] - Project [ws_net_paid,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - WholeStageCodegen (9) - HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #6 - WholeStageCodegen (8) - HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 - WholeStageCodegen (14) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #7 - WholeStageCodegen (13) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/explain.txt deleted file mode 100644 index b623de85d0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,241 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (37) -+- * Project (36) - +- Window (35) - +- * CometColumnarToRow (34) - +- CometSort (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometExchange (30) - +- CometHashAggregate (29) - +- CometUnion (28) - :- CometHashAggregate (17) - : +- CometExchange (16) - : +- CometHashAggregate (15) - : +- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - :- CometHashAggregate (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometHashAggregate (19) - : +- ReusedExchange (18) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometHashAggregate (24) - +- ReusedExchange (23) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] -Arguments: [ws_item_sk#1, ws_net_paid#2], [ws_item_sk#1, ws_net_paid#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#7, i_class#8, i_category#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Condition : isnotnull(i_item_sk#7) - -(11) CometProject -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Arguments: [i_item_sk#7, i_class#10, i_category#11], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#8, 50, true, false, true) AS i_class#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [i_item_sk#7, i_class#10, i_category#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#1, ws_net_paid#2] -Right output [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_net_paid#2, i_class#10, i_category#11], [ws_net_paid#2, i_class#10, i_category#11] - -(15) CometHashAggregate -Input [3]: [ws_net_paid#2, i_class#10, i_category#11] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] - -(16) CometExchange -Input [3]: [i_category#11, i_class#10, sum#12] -Arguments: hashpartitioning(i_category#11, i_class#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [i_category#11, i_class#10, sum#12] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] - -(18) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#11, i_class#10, sum#13] - -(19) CometHashAggregate -Input [3]: [i_category#11, i_class#10, sum#13] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [sum(UnscaledValue(ws_net_paid#14))] - -(20) CometHashAggregate -Input [2]: [total_sum#15, i_category#11] -Keys [1]: [i_category#11] -Functions [1]: [partial_sum(total_sum#15)] - -(21) CometExchange -Input [3]: [i_category#11, sum#16, isEmpty#17] -Arguments: hashpartitioning(i_category#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(22) CometHashAggregate -Input [3]: [i_category#11, sum#16, isEmpty#17] -Keys [1]: [i_category#11] -Functions [1]: [sum(total_sum#15)] - -(23) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#11, i_class#10, sum#18] - -(24) CometHashAggregate -Input [3]: [i_category#11, i_class#10, sum#18] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [sum(UnscaledValue(ws_net_paid#19))] - -(25) CometHashAggregate -Input [1]: [total_sum#20] -Keys: [] -Functions [1]: [partial_sum(total_sum#20)] - -(26) CometExchange -Input [2]: [sum#21, isEmpty#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(27) CometHashAggregate -Input [2]: [sum#21, isEmpty#22] -Keys: [] -Functions [1]: [sum(total_sum#20)] - -(28) CometUnion -Child 0 Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] -Child 1 Input [6]: [total_sum#29, i_category#11, i_class#30, g_category#31, g_class#32, lochierarchy#33] -Child 2 Input [6]: [total_sum#34, i_category#35, i_class#36, g_category#37, g_class#38, lochierarchy#39] - -(29) CometHashAggregate -Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] -Keys [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] -Functions: [] - -(30) CometExchange -Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] -Arguments: hashpartitioning(total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(31) CometHashAggregate -Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] -Keys [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] -Functions: [] - -(32) CometExchange -Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] -Arguments: hashpartitioning(lochierarchy#28, _w0#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(33) CometSort -Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] -Arguments: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40], [lochierarchy#28 ASC NULLS FIRST, _w0#40 ASC NULLS FIRST, total_sum#23 DESC NULLS LAST] - -(34) CometColumnarToRow [codegen id : 1] -Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] - -(35) Window -Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] -Arguments: [rank(total_sum#23) windowspecdefinition(lochierarchy#28, _w0#40, total_sum#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#41], [lochierarchy#28, _w0#40], [total_sum#23 DESC NULLS LAST] - -(36) Project [codegen id : 2] -Output [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, rank_within_parent#41] -Input [6]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40, rank_within_parent#41] - -(37) TakeOrderedAndProject -Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, rank_within_parent#41] -Arguments: 100, [lochierarchy#28 DESC NULLS LAST, CASE WHEN (lochierarchy#28 = 0) THEN i_category#24 END ASC NULLS FIRST, rank_within_parent#41 ASC NULLS FIRST], [total_sum#23, i_category#24, i_class#25, lochierarchy#28, rank_within_parent#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (42) -+- * CometColumnarToRow (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) - - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(39) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(40) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(41) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(42) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 11e3c03d80..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (2) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] - CometExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometUnion [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometHashAggregate [sum] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(UnscaledValue(ws_net_paid)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ws_net_paid] [i_category,i_class,sum] - CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange [i_category] #7 - CometHashAggregate [total_sum] [i_category,sum,isEmpty] - CometHashAggregate [i_class,sum] [total_sum,i_category,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange #8 - CometHashAggregate [total_sum] [sum,isEmpty] - CometHashAggregate [i_category,i_class,sum] [total_sum,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt deleted file mode 100644 index b623de85d0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/explain.txt +++ /dev/null @@ -1,241 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (37) -+- * Project (36) - +- Window (35) - +- * CometColumnarToRow (34) - +- CometSort (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometExchange (30) - +- CometHashAggregate (29) - +- CometUnion (28) - :- CometHashAggregate (17) - : +- CometExchange (16) - : +- CometHashAggregate (15) - : +- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - :- CometHashAggregate (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometHashAggregate (19) - : +- ReusedExchange (18) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometHashAggregate (24) - +- ReusedExchange (23) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] -Arguments: [ws_item_sk#1, ws_net_paid#2], [ws_item_sk#1, ws_net_paid#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#7, i_class#8, i_category#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Condition : isnotnull(i_item_sk#7) - -(11) CometProject -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Arguments: [i_item_sk#7, i_class#10, i_category#11], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#8, 50, true, false, true) AS i_class#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [i_item_sk#7, i_class#10, i_category#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#1, ws_net_paid#2] -Right output [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_net_paid#2, i_class#10, i_category#11], [ws_net_paid#2, i_class#10, i_category#11] - -(15) CometHashAggregate -Input [3]: [ws_net_paid#2, i_class#10, i_category#11] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] - -(16) CometExchange -Input [3]: [i_category#11, i_class#10, sum#12] -Arguments: hashpartitioning(i_category#11, i_class#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [i_category#11, i_class#10, sum#12] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] - -(18) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#11, i_class#10, sum#13] - -(19) CometHashAggregate -Input [3]: [i_category#11, i_class#10, sum#13] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [sum(UnscaledValue(ws_net_paid#14))] - -(20) CometHashAggregate -Input [2]: [total_sum#15, i_category#11] -Keys [1]: [i_category#11] -Functions [1]: [partial_sum(total_sum#15)] - -(21) CometExchange -Input [3]: [i_category#11, sum#16, isEmpty#17] -Arguments: hashpartitioning(i_category#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(22) CometHashAggregate -Input [3]: [i_category#11, sum#16, isEmpty#17] -Keys [1]: [i_category#11] -Functions [1]: [sum(total_sum#15)] - -(23) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#11, i_class#10, sum#18] - -(24) CometHashAggregate -Input [3]: [i_category#11, i_class#10, sum#18] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [sum(UnscaledValue(ws_net_paid#19))] - -(25) CometHashAggregate -Input [1]: [total_sum#20] -Keys: [] -Functions [1]: [partial_sum(total_sum#20)] - -(26) CometExchange -Input [2]: [sum#21, isEmpty#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(27) CometHashAggregate -Input [2]: [sum#21, isEmpty#22] -Keys: [] -Functions [1]: [sum(total_sum#20)] - -(28) CometUnion -Child 0 Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] -Child 1 Input [6]: [total_sum#29, i_category#11, i_class#30, g_category#31, g_class#32, lochierarchy#33] -Child 2 Input [6]: [total_sum#34, i_category#35, i_class#36, g_category#37, g_class#38, lochierarchy#39] - -(29) CometHashAggregate -Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] -Keys [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] -Functions: [] - -(30) CometExchange -Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] -Arguments: hashpartitioning(total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(31) CometHashAggregate -Input [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] -Keys [6]: [total_sum#23, i_category#24, i_class#25, g_category#26, g_class#27, lochierarchy#28] -Functions: [] - -(32) CometExchange -Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] -Arguments: hashpartitioning(lochierarchy#28, _w0#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(33) CometSort -Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] -Arguments: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40], [lochierarchy#28 ASC NULLS FIRST, _w0#40 ASC NULLS FIRST, total_sum#23 DESC NULLS LAST] - -(34) CometColumnarToRow [codegen id : 1] -Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] - -(35) Window -Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40] -Arguments: [rank(total_sum#23) windowspecdefinition(lochierarchy#28, _w0#40, total_sum#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#41], [lochierarchy#28, _w0#40], [total_sum#23 DESC NULLS LAST] - -(36) Project [codegen id : 2] -Output [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, rank_within_parent#41] -Input [6]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, _w0#40, rank_within_parent#41] - -(37) TakeOrderedAndProject -Input [5]: [total_sum#23, i_category#24, i_class#25, lochierarchy#28, rank_within_parent#41] -Arguments: 100, [lochierarchy#28 DESC NULLS LAST, CASE WHEN (lochierarchy#28 = 0) THEN i_category#24 END ASC NULLS FIRST, rank_within_parent#41 ASC NULLS FIRST], [total_sum#23, i_category#24, i_class#25, lochierarchy#28, rank_within_parent#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (42) -+- * CometColumnarToRow (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) - - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(39) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(40) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(41) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(42) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/extended.txt deleted file mode 100644 index 46c47555a8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/extended.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 11e3c03d80..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (2) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] - CometExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometUnion [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometHashAggregate [sum] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(UnscaledValue(ws_net_paid)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ws_net_paid] [i_category,i_class,sum] - CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange [i_category] #7 - CometHashAggregate [total_sum] [i_category,sum,isEmpty] - CometHashAggregate [i_class,sum] [total_sum,i_category,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange #8 - CometHashAggregate [total_sum] [sum,isEmpty] - CometHashAggregate [i_category,i_class,sum] [total_sum,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/explain.txt deleted file mode 100644 index 646b85d12e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/explain.txt +++ /dev/null @@ -1,172 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometSort (24) - +- CometColumnarExchange (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.store_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.item (4) - +- ReusedExchange (11) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(7) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(8) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(11) ReusedExchange [Reuses operator id: 30] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] - -(14) HashAggregate [codegen id : 3] -Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(15) CometColumnarExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(17) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#17] -Results [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#17,17,2) AS _w0#19] - -(18) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(22) Project [codegen id : 6] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] - -(23) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(24) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 7] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometNativeScan parquet spark_catalog.default.date_dim (26) - - -(26) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(28) CometProject -Input [2]: [d_date_sk#14, d_date#22] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(30) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/simplified.txt deleted file mode 100644 index c1ec019e57..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/simplified.txt +++ /dev/null @@ -1,47 +0,0 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (6) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/explain.txt deleted file mode 100644 index 96a0ec392e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,173 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometSort (24) - +- CometColumnarExchange (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] - -(23) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 3] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(28) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(30) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/simplified.txt deleted file mode 100644 index 46191f59cd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,39 +0,0 @@ -WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #2 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/explain.txt deleted file mode 100644 index 96a0ec392e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/explain.txt +++ /dev/null @@ -1,173 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometSort (24) - +- CometColumnarExchange (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] - -(23) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 3] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(28) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(30) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/extended.txt deleted file mode 100644 index 3dbaf2e346..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/extended.txt +++ /dev/null @@ -1,33 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 46191f59cd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/simplified.txt +++ /dev/null @@ -1,39 +0,0 @@ -WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #2 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/explain.txt deleted file mode 100644 index 7cfe036ae9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/explain.txt +++ /dev/null @@ -1,281 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : +- BroadcastExchange (9) - : : : +- * Project (8) - : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : :- * ColumnarToRow (5) - : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (6) - : : +- BroadcastExchange (22) - : : +- Union (21) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * ColumnarToRow (17) - : : : +- Scan parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (18) - : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.customer_address (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.customer_demographics (32) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(4) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] - -(6) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#7] - -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#4] -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] - -(9) BroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#4] -Join type: LeftSemi -Join condition: None - -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#6)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] - -(13) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#10] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#8 AS customer_sk#11] -Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#10] - -(16) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#6)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] - -(18) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#14] - -(19) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#12 AS customer_sk#15] -Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#14] - -(21) Union - -(22) BroadcastExchange -Input [1]: [customer_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(23) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#11] -Join type: LeftSemi -Join condition: None - -(24) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(25) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_county#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_county, [Dona Ana County,Douglas County,Gaines County,Richland County,Walker County]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [ca_address_sk#16, ca_county#17] -Condition : (ca_county#17 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#16)) - -(27) CometProject -Input [2]: [ca_address_sk#16, ca_county#17] -Arguments: [ca_address_sk#16], [ca_address_sk#16] - -(28) CometColumnarToRow [codegen id : 7] -Input [1]: [ca_address_sk#16] - -(29) BroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(30) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#16] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 9] -Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#16] - -(32) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(33) CometFilter -Input [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Condition : isnotnull(cd_demo_sk#18) - -(34) CometProject -Input [9]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#19, 1)) AS cd_gender#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#20, 1)) AS cd_marital_status#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#21, 20)) AS cd_education_status#29, cd_purchase_estimate#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_credit_rating#23, 10)) AS cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(35) CometColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(36) BroadcastExchange -Input [9]: [cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 9] -Output [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#18, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(39) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#31] -Results [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] - -(40) CometColumnarExchange -Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] -Arguments: hashpartitioning(cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(41) CometColumnarToRow [codegen id : 10] -Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] - -(42) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#32] -Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#22, cd_credit_rating#30, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#33] -Results [14]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, count(1)#33 AS cnt1#34, cd_purchase_estimate#22, count(1)#33 AS cnt2#35, cd_credit_rating#30, count(1)#33 AS cnt3#36, cd_dep_count#24, count(1)#33 AS cnt4#37, cd_dep_employed_count#25, count(1)#33 AS cnt5#38, cd_dep_college_count#26, count(1)#33 AS cnt6#39] - -(43) TakeOrderedAndProject -Input [14]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cnt1#34, cd_purchase_estimate#22, cnt2#35, cd_credit_rating#30, cnt3#36, cd_dep_count#24, cnt4#37, cd_dep_employed_count#25, cnt5#38, cd_dep_college_count#26, cnt6#39] -Arguments: 100, [cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_education_status#29 ASC NULLS FIRST, cd_purchase_estimate#22 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [cd_gender#27, cd_marital_status#28, cd_education_status#29, cnt1#34, cd_purchase_estimate#22, cnt2#35, cd_credit_rating#30, cnt3#36, cd_dep_count#24, cnt4#37, cd_dep_employed_count#25, cnt5#38, cd_dep_college_count#26, cnt6#39] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometNativeScan parquet spark_catalog.default.date_dim (44) - - -(44) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#40, d_moy#41] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [3]: [d_date_sk#7, d_year#40, d_moy#41] -Condition : (((((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2002)) AND (d_moy#41 >= 4)) AND (d_moy#41 <= 7)) AND isnotnull(d_date_sk#7)) - -(46) CometProject -Input [3]: [d_date_sk#7, d_year#40, d_moy#41] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(48) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/simplified.txt deleted file mode 100644 index 11bfb6f6c3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (10) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - Union - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [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] - 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] - CometNativeScan 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/q10a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/explain.txt deleted file mode 100644 index 7e11c91157..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,267 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometHashAggregate (38) - +- CometExchange (37) - +- CometHashAggregate (36) - +- CometProject (35) - +- CometBroadcastHashJoin (34) - :- CometProject (29) - : +- CometBroadcastHashJoin (28) - : :- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometBroadcastHashJoin (11) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : +- CometBroadcastExchange (10) - : : : +- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : +- CometBroadcastExchange (21) - : : +- CometUnion (20) - : : :- CometProject (15) - : : : +- CometBroadcastHashJoin (14) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (12) - : : : +- ReusedExchange (13) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (17) - : +- CometBroadcastExchange (27) - : +- CometProject (26) - : +- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) - +- CometBroadcastExchange (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (30) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) - -(6) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#4], [ss_customer_sk#4] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: [ss_customer_sk#4] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ss_customer_sk#4] -Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(13) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] -Arguments: [customer_sk#14], [ws_bill_customer_sk#10 AS customer_sk#14] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] -ReadSchema: struct - -(17) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#18] - -(18) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Right output [1]: [d_date_sk#18] -Arguments: [cs_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight - -(19) CometProject -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#18] -Arguments: [customer_sk#19], [cs_ship_customer_sk#15 AS customer_sk#19] - -(20) CometUnion -Child 0 Input [1]: [customer_sk#14] -Child 1 Input [1]: [customer_sk#19] - -(21) CometBroadcastExchange -Input [1]: [customer_sk#14] -Arguments: [customer_sk#14] - -(22) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [customer_sk#14] -Arguments: [c_customer_sk#1], [customer_sk#14], LeftSemi, BuildRight - -(23) CometProject -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_county#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_county, [Dona Ana County,Douglas County,Gaines County,Richland County,Walker County]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [ca_address_sk#20, ca_county#21] -Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) - -(26) CometProject -Input [2]: [ca_address_sk#20, ca_county#21] -Arguments: [ca_address_sk#20], [ca_address_sk#20] - -(27) CometBroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: [ca_address_sk#20] - -(28) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ca_address_sk#20] -Arguments: [c_current_addr_sk#3], [ca_address_sk#20], Inner, BuildRight - -(29) CometProject -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] -Arguments: [c_current_cdemo_sk#2], [c_current_cdemo_sk#2] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Condition : isnotnull(cd_demo_sk#22) - -(32) CometProject -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_demo_sk#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#23, 1)) AS cd_gender#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#24, 1)) AS cd_marital_status#32, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#25, 20)) AS cd_education_status#33, cd_purchase_estimate#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_credit_rating#27, 10)) AS cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(33) CometBroadcastExchange -Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(34) CometBroadcastHashJoin -Left output [1]: [c_current_cdemo_sk#2] -Right output [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#22], Inner, BuildRight - -(35) CometProject -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(36) CometHashAggregate -Input [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [1]: [partial_count(1)] - -(37) CometExchange -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#35] -Arguments: hashpartitioning(cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(38) CometHashAggregate -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#35] -Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [1]: [count(1)] - -(39) CometTakeOrderedAndProject -Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#31 ASC NULLS FIRST,cd_marital_status#32 ASC NULLS FIRST,cd_education_status#33 ASC NULLS FIRST,cd_purchase_estimate#26 ASC NULLS FIRST,cd_credit_rating#34 ASC NULLS FIRST,cd_dep_count#28 ASC NULLS FIRST,cd_dep_employed_count#29 ASC NULLS FIRST,cd_dep_college_count#30 ASC NULLS FIRST], output=[cd_gender#31,cd_marital_status#32,cd_education_status#33,cnt1#36,cd_purchase_estimate#26,cnt2#37,cd_credit_rating#34,cnt3#38,cd_dep_count#28,cnt4#39,cd_dep_employed_count#29,cnt5#40,cd_dep_college_count#30,cnt6#41]), [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41], 100, 0, [cd_gender#31 ASC NULLS FIRST, cd_marital_status#32 ASC NULLS FIRST, cd_education_status#33 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#34 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] - -(40) CometColumnarToRow [codegen id : 1] -Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) - -(43) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(45) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 38f41d10a5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6] - CometHashAggregate [count] [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6,count(1)] - CometExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - 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,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_cdemo_sk,c_current_addr_sk,ca_address_sk] - CometProject [c_current_cdemo_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [customer_sk] #5 - CometUnion [customer_sk] - CometProject [ws_bill_customer_sk] [customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - 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 - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [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] - 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 [native_iceberg_compat] 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/q10a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/explain.txt deleted file mode 100644 index 7e11c91157..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/explain.txt +++ /dev/null @@ -1,267 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometHashAggregate (38) - +- CometExchange (37) - +- CometHashAggregate (36) - +- CometProject (35) - +- CometBroadcastHashJoin (34) - :- CometProject (29) - : +- CometBroadcastHashJoin (28) - : :- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometBroadcastHashJoin (11) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : +- CometBroadcastExchange (10) - : : : +- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : +- CometBroadcastExchange (21) - : : +- CometUnion (20) - : : :- CometProject (15) - : : : +- CometBroadcastHashJoin (14) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (12) - : : : +- ReusedExchange (13) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (17) - : +- CometBroadcastExchange (27) - : +- CometProject (26) - : +- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) - +- CometBroadcastExchange (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (30) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) - -(6) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#4], [ss_customer_sk#4] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: [ss_customer_sk#4] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ss_customer_sk#4] -Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(13) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] -Arguments: [customer_sk#14], [ws_bill_customer_sk#10 AS customer_sk#14] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] -ReadSchema: struct - -(17) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#18] - -(18) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Right output [1]: [d_date_sk#18] -Arguments: [cs_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight - -(19) CometProject -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#18] -Arguments: [customer_sk#19], [cs_ship_customer_sk#15 AS customer_sk#19] - -(20) CometUnion -Child 0 Input [1]: [customer_sk#14] -Child 1 Input [1]: [customer_sk#19] - -(21) CometBroadcastExchange -Input [1]: [customer_sk#14] -Arguments: [customer_sk#14] - -(22) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [customer_sk#14] -Arguments: [c_customer_sk#1], [customer_sk#14], LeftSemi, BuildRight - -(23) CometProject -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_county#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_county, [Dona Ana County,Douglas County,Gaines County,Richland County,Walker County]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [ca_address_sk#20, ca_county#21] -Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) - -(26) CometProject -Input [2]: [ca_address_sk#20, ca_county#21] -Arguments: [ca_address_sk#20], [ca_address_sk#20] - -(27) CometBroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: [ca_address_sk#20] - -(28) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ca_address_sk#20] -Arguments: [c_current_addr_sk#3], [ca_address_sk#20], Inner, BuildRight - -(29) CometProject -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] -Arguments: [c_current_cdemo_sk#2], [c_current_cdemo_sk#2] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Condition : isnotnull(cd_demo_sk#22) - -(32) CometProject -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_demo_sk#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#23, 1)) AS cd_gender#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#24, 1)) AS cd_marital_status#32, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#25, 20)) AS cd_education_status#33, cd_purchase_estimate#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_credit_rating#27, 10)) AS cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(33) CometBroadcastExchange -Input [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(34) CometBroadcastHashJoin -Left output [1]: [c_current_cdemo_sk#2] -Right output [9]: [cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#22], Inner, BuildRight - -(35) CometProject -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#22, cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30], [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] - -(36) CometHashAggregate -Input [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [1]: [partial_count(1)] - -(37) CometExchange -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#35] -Arguments: hashpartitioning(cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(38) CometHashAggregate -Input [9]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#35] -Keys [8]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cd_purchase_estimate#26, cd_credit_rating#34, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [1]: [count(1)] - -(39) CometTakeOrderedAndProject -Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#31 ASC NULLS FIRST,cd_marital_status#32 ASC NULLS FIRST,cd_education_status#33 ASC NULLS FIRST,cd_purchase_estimate#26 ASC NULLS FIRST,cd_credit_rating#34 ASC NULLS FIRST,cd_dep_count#28 ASC NULLS FIRST,cd_dep_employed_count#29 ASC NULLS FIRST,cd_dep_college_count#30 ASC NULLS FIRST], output=[cd_gender#31,cd_marital_status#32,cd_education_status#33,cnt1#36,cd_purchase_estimate#26,cnt2#37,cd_credit_rating#34,cnt3#38,cd_dep_count#28,cnt4#39,cd_dep_employed_count#29,cnt5#40,cd_dep_college_count#30,cnt6#41]), [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41], 100, 0, [cd_gender#31 ASC NULLS FIRST, cd_marital_status#32 ASC NULLS FIRST, cd_education_status#33 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#34 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] - -(40) CometColumnarToRow [codegen id : 1] -Input [14]: [cd_gender#31, cd_marital_status#32, cd_education_status#33, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#34, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) - -(43) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(45) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/extended.txt deleted file mode 100644 index 2cdc75e15e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/extended.txt +++ /dev/null @@ -1,56 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 38f41d10a5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6] - CometHashAggregate [count] [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6,count(1)] - CometExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - 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,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_cdemo_sk,c_current_addr_sk,ca_address_sk] - CometProject [c_current_cdemo_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [customer_sk] #5 - CometUnion [customer_sk] - CometProject [ws_bill_customer_sk] [customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - 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 - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [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] - 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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/explain.txt deleted file mode 100644 index 700f90222d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/explain.txt +++ /dev/null @@ -1,513 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (79) -+- * Project (78) - +- * BroadcastHashJoin Inner BuildRight (77) - :- * Project (58) - : +- * BroadcastHashJoin Inner BuildRight (57) - : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- * HashAggregate (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * Project (28) - : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (26) - : : : +- * Filter (25) - : : : +- * ColumnarToRow (24) - : : : +- Scan parquet spark_catalog.default.store_sales (23) - : : +- ReusedExchange (29) - : +- BroadcastExchange (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- * CometColumnarToRow (53) - : +- CometColumnarExchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (41) - : : : +- CometProject (40) - : : : +- CometFilter (39) - : : : +- CometNativeScan parquet spark_catalog.default.customer (38) - : : +- BroadcastExchange (45) - : : +- * Filter (44) - : : +- * ColumnarToRow (43) - : : +- Scan parquet spark_catalog.default.web_sales (42) - : +- ReusedExchange (48) - +- BroadcastExchange (76) - +- * HashAggregate (75) - +- * CometColumnarToRow (74) - +- CometColumnarExchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * CometColumnarToRow (62) - : : +- CometProject (61) - : : +- CometFilter (60) - : : +- CometNativeScan parquet spark_catalog.default.customer (59) - : +- BroadcastExchange (66) - : +- * Filter (65) - : +- * ColumnarToRow (64) - : +- Scan parquet spark_catalog.default.web_sales (63) - +- ReusedExchange (69) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)) AS c_customer_id#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#3, 20)) AS c_first_name#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#4, 30)) AS c_last_name#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#5, 1)) AS c_preferred_cust_flag#12, c_birth_country#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#7, 13)) AS c_login#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#8, 50)) AS c_email_address#14] - -(4) CometColumnarToRow [codegen id : 3] -Input [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] - -(5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(7) Filter [codegen id : 1] -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#15) - -(8) BroadcastExchange -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#15] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(11) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#20, d_year#21] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#18] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] - -(14) HashAggregate [codegen id : 3] -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] -Aggregate Attributes [1]: [sum#22] -Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] - -(15) CometColumnarExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 16] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] - -(17) HashAggregate [codegen id : 16] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24] -Results [2]: [c_customer_id#9 AS customer_id#25, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24,18,2) AS year_total#26] - -(18) Filter [codegen id : 16] -Input [2]: [customer_id#25, year_total#26] -Condition : (isnotnull(year_total#26) AND (year_total#26 > 0.00)) - -(19) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(20) CometFilter -Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] -Condition : (isnotnull(c_customer_sk#27) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#28, 16)))) - -(21) CometProject -Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] -Arguments: [c_customer_sk#27, c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40], [c_customer_sk#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#28, 16)) AS c_customer_id#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#29, 20)) AS c_first_name#36, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#30, 30)) AS c_last_name#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#31, 1)) AS c_preferred_cust_flag#38, c_birth_country#32, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#33, 13)) AS c_login#39, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#34, 50)) AS c_email_address#40] - -(22) CometColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#27, c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40] - -(23) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#41, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#44), dynamicpruningexpression(ss_sold_date_sk#44 IN dynamicpruning#45)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#41, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44] - -(25) Filter [codegen id : 4] -Input [4]: [ss_customer_sk#41, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44] -Condition : isnotnull(ss_customer_sk#41) - -(26) BroadcastExchange -Input [4]: [ss_customer_sk#41, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#27] -Right keys [1]: [ss_customer_sk#41] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [10]: [c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44] -Input [12]: [c_customer_sk#27, c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, ss_customer_sk#41, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44] - -(29) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#46, d_year#47] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#44] -Right keys [1]: [d_date_sk#46] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [10]: [c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, ss_ext_discount_amt#42, ss_ext_list_price#43, d_year#47] -Input [12]: [c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, ss_ext_discount_amt#42, ss_ext_list_price#43, ss_sold_date_sk#44, d_date_sk#46, d_year#47] - -(32) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, ss_ext_discount_amt#42, ss_ext_list_price#43, d_year#47] -Keys [8]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#43 - ss_ext_discount_amt#42)))] -Aggregate Attributes [1]: [sum#48] -Results [9]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, sum#49] - -(33) CometColumnarExchange -Input [9]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, sum#49] -Arguments: hashpartitioning(c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 7] -Input [9]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, sum#49] - -(35) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40, sum#49] -Keys [8]: [c_customer_id#35, c_first_name#36, c_last_name#37, d_year#47, c_preferred_cust_flag#38, c_birth_country#32, c_login#39, c_email_address#40] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#43 - ss_ext_discount_amt#42)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#43 - ss_ext_discount_amt#42)))#24] -Results [5]: [c_customer_id#35 AS customer_id#50, c_first_name#36 AS customer_first_name#51, c_last_name#37 AS customer_last_name#52, c_email_address#40 AS customer_email_address#53, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#43 - ss_ext_discount_amt#42)))#24,18,2) AS year_total#54] - -(36) BroadcastExchange -Input [5]: [customer_id#50, customer_first_name#51, customer_last_name#52, customer_email_address#53, year_total#54] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(37) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#50] -Join type: Inner -Join condition: None - -(38) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(39) CometFilter -Input [8]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62] -Condition : (isnotnull(c_customer_sk#55) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#56, 16)))) - -(40) CometProject -Input [8]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58, c_preferred_cust_flag#59, c_birth_country#60, c_login#61, c_email_address#62] -Arguments: [c_customer_sk#55, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68], [c_customer_sk#55, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#56, 16)) AS c_customer_id#63, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#57, 20)) AS c_first_name#64, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#58, 30)) AS c_last_name#65, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#59, 1)) AS c_preferred_cust_flag#66, c_birth_country#60, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#61, 13)) AS c_login#67, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#62, 50)) AS c_email_address#68] - -(41) CometColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#55, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68] - -(42) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] - -(44) Filter [codegen id : 8] -Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] -Condition : isnotnull(ws_bill_customer_sk#69) - -(45) BroadcastExchange -Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -(46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#55] -Right keys [1]: [ws_bill_customer_sk#69] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 10] -Output [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] -Input [12]: [c_customer_sk#55, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] - -(48) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#73, d_year#74] - -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#72] -Right keys [1]: [d_date_sk#73] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 10] -Output [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, d_year#74] -Input [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72, d_date_sk#73, d_year#74] - -(51) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, d_year#74] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#74] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))] -Aggregate Attributes [1]: [sum#75] -Results [9]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#74, sum#76] - -(52) CometColumnarExchange -Input [9]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#74, sum#76] -Arguments: hashpartitioning(c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#74, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(53) CometColumnarToRow [codegen id : 11] -Input [9]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#74, sum#76] - -(54) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#74, sum#76] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#60, c_login#67, c_email_address#68, d_year#74] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))#77] -Results [2]: [c_customer_id#63 AS customer_id#78, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))#77,18,2) AS year_total#79] - -(55) Filter [codegen id : 11] -Input [2]: [customer_id#78, year_total#79] -Condition : (isnotnull(year_total#79) AND (year_total#79 > 0.00)) - -(56) BroadcastExchange -Input [2]: [customer_id#78, year_total#79] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(57) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#78] -Join type: Inner -Join condition: None - -(58) Project [codegen id : 16] -Output [8]: [customer_id#25, year_total#26, customer_id#50, customer_first_name#51, customer_last_name#52, customer_email_address#53, year_total#54, year_total#79] -Input [9]: [customer_id#25, year_total#26, customer_id#50, customer_first_name#51, customer_last_name#52, customer_email_address#53, year_total#54, customer_id#78, year_total#79] - -(59) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#80, c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(60) CometFilter -Input [8]: [c_customer_sk#80, c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87] -Condition : (isnotnull(c_customer_sk#80) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#81, 16)))) - -(61) CometProject -Input [8]: [c_customer_sk#80, c_customer_id#81, c_first_name#82, c_last_name#83, c_preferred_cust_flag#84, c_birth_country#85, c_login#86, c_email_address#87] -Arguments: [c_customer_sk#80, c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93], [c_customer_sk#80, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#81, 16)) AS c_customer_id#88, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#82, 20)) AS c_first_name#89, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#83, 30)) AS c_last_name#90, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#84, 1)) AS c_preferred_cust_flag#91, c_birth_country#85, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#86, 13)) AS c_login#92, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#87, 50)) AS c_email_address#93] - -(62) CometColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#80, c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93] - -(63) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#94, ws_ext_discount_amt#95, ws_ext_list_price#96, ws_sold_date_sk#97] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#97), dynamicpruningexpression(ws_sold_date_sk#97 IN dynamicpruning#45)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(64) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#94, ws_ext_discount_amt#95, ws_ext_list_price#96, ws_sold_date_sk#97] - -(65) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#94, ws_ext_discount_amt#95, ws_ext_list_price#96, ws_sold_date_sk#97] -Condition : isnotnull(ws_bill_customer_sk#94) - -(66) BroadcastExchange -Input [4]: [ws_bill_customer_sk#94, ws_ext_discount_amt#95, ws_ext_list_price#96, ws_sold_date_sk#97] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#80] -Right keys [1]: [ws_bill_customer_sk#94] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 14] -Output [10]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, ws_ext_discount_amt#95, ws_ext_list_price#96, ws_sold_date_sk#97] -Input [12]: [c_customer_sk#80, c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, ws_bill_customer_sk#94, ws_ext_discount_amt#95, ws_ext_list_price#96, ws_sold_date_sk#97] - -(69) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#98, d_year#99] - -(70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#97] -Right keys [1]: [d_date_sk#98] -Join type: Inner -Join condition: None - -(71) Project [codegen id : 14] -Output [10]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, ws_ext_discount_amt#95, ws_ext_list_price#96, d_year#99] -Input [12]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, ws_ext_discount_amt#95, ws_ext_list_price#96, ws_sold_date_sk#97, d_date_sk#98, d_year#99] - -(72) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, ws_ext_discount_amt#95, ws_ext_list_price#96, d_year#99] -Keys [8]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, d_year#99] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#96 - ws_ext_discount_amt#95)))] -Aggregate Attributes [1]: [sum#100] -Results [9]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, d_year#99, sum#101] - -(73) CometColumnarExchange -Input [9]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, d_year#99, sum#101] -Arguments: hashpartitioning(c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, d_year#99, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(74) CometColumnarToRow [codegen id : 15] -Input [9]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, d_year#99, sum#101] - -(75) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, d_year#99, sum#101] -Keys [8]: [c_customer_id#88, c_first_name#89, c_last_name#90, c_preferred_cust_flag#91, c_birth_country#85, c_login#92, c_email_address#93, d_year#99] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#96 - ws_ext_discount_amt#95)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#96 - ws_ext_discount_amt#95)))#77] -Results [2]: [c_customer_id#88 AS customer_id#102, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#96 - ws_ext_discount_amt#95)))#77,18,2) AS year_total#103] - -(76) BroadcastExchange -Input [2]: [customer_id#102, year_total#103] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] - -(77) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#102] -Join type: Inner -Join condition: (CASE WHEN (year_total#79 > 0.00) THEN (year_total#103 / year_total#79) ELSE 0E-20 END > CASE WHEN (year_total#26 > 0.00) THEN (year_total#54 / year_total#26) ELSE 0E-20 END) - -(78) Project [codegen id : 16] -Output [4]: [customer_id#50, customer_first_name#51, customer_last_name#52, customer_email_address#53] -Input [10]: [customer_id#25, year_total#26, customer_id#50, customer_first_name#51, customer_last_name#52, customer_email_address#53, year_total#54, year_total#79, customer_id#102, year_total#103] - -(79) TakeOrderedAndProject -Input [4]: [customer_id#50, customer_first_name#51, customer_last_name#52, customer_email_address#53] -Arguments: 100, [customer_id#50 ASC NULLS FIRST, customer_first_name#51 ASC NULLS FIRST, customer_last_name#52 ASC NULLS FIRST, customer_email_address#53 ASC NULLS FIRST], [customer_id#50, customer_first_name#51, customer_last_name#52, customer_email_address#53] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometFilter (81) - +- CometNativeScan parquet spark_catalog.default.date_dim (80) - - -(80) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(81) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(82) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#20, d_year#21] - -(83) BroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#44 IN dynamicpruning#45 -BroadcastExchange (87) -+- * CometColumnarToRow (86) - +- CometFilter (85) - +- CometNativeScan parquet spark_catalog.default.date_dim (84) - - -(84) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#46, d_year#47] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(85) CometFilter -Input [2]: [d_date_sk#46, d_year#47] -Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2002)) AND isnotnull(d_date_sk#46)) - -(86) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#46, d_year#47] - -(87) BroadcastExchange -Input [2]: [d_date_sk#46, d_year#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] - -Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#19 - -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#97 IN dynamicpruning#45 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/simplified.txt deleted file mode 100644 index 00a3e659d4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/simplified.txt +++ /dev/null @@ -1,130 +0,0 @@ -TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - WholeStageCodegen (16) - Project [customer_id,customer_first_name,customer_last_name,customer_email_address] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - WholeStageCodegen (6) - HashAggregate [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] [sum,sum] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) - HashAggregate [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] [sum,sum] - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) - HashAggregate [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] [sum,sum] - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/explain.txt deleted file mode 100644 index 676d488824..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,470 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometProject (53) - : +- CometBroadcastHashJoin (52) - : :- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (51) - : +- CometFilter (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : +- CometBroadcastExchange (41) - : : +- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) - : +- ReusedExchange (44) - +- CometBroadcastExchange (68) - +- CometHashAggregate (67) - +- CometExchange (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometFilter (55) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (62) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)) AS c_customer_id#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#3, 20)) AS c_first_name#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#4, 30)) AS c_last_name#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#5, 1)) AS c_preferred_cust_flag#12, c_birth_country#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#7, 13)) AS c_login#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#8, 50)) AS c_email_address#14] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#15) - -(6) CometBroadcastExchange -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(7) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_sk#1], [ss_customer_sk#15], Inner, BuildRight - -(8) CometProject -Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: [d_date_sk#20, d_year#21] - -(12) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Right output [2]: [d_date_sk#20, d_year#21] -Arguments: [ss_sold_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(13) CometProject -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] - -(14) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(15) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(17) CometFilter -Input [2]: [customer_id#23, year_total#24] -Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#26, 16)))) - -(20) CometProject -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Arguments: [c_customer_sk#25, c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38], [c_customer_sk#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#26, 16)) AS c_customer_id#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#27, 20)) AS c_first_name#34, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#28, 30)) AS c_last_name#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#29, 1)) AS c_preferred_cust_flag#36, c_birth_country#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#31, 13)) AS c_login#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#32, 50)) AS c_email_address#38] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#42), dynamicpruningexpression(ss_sold_date_sk#42 IN dynamicpruning#43)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Condition : isnotnull(ss_customer_sk#39) - -(23) CometBroadcastExchange -Input [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Arguments: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] - -(24) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#25, c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] -Right output [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Arguments: [c_customer_sk#25], [ss_customer_sk#39], Inner, BuildRight - -(25) CometProject -Input [12]: [c_customer_sk#25, c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Arguments: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42], [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] - -(29) CometBroadcastHashJoin -Left output [10]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [ss_sold_date_sk#42], [d_date_sk#44], Inner, BuildRight - -(30) CometProject -Input [12]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42, d_date_sk#44, d_year#45] -Arguments: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45], [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45] - -(31) CometHashAggregate -Input [10]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45] -Keys [8]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#41 - ss_ext_discount_amt#40)))] - -(32) CometExchange -Input [9]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, sum#46] -Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [9]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, sum#46] -Keys [8]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#41 - ss_ext_discount_amt#40)))] - -(34) CometBroadcastExchange -Input [5]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] -Arguments: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#23, year_total#24] -Right output [5]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] -Arguments: [customer_id#23], [customer_id#47], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55, c_preferred_cust_flag#56, c_birth_country#57, c_login#58, c_email_address#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [8]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55, c_preferred_cust_flag#56, c_birth_country#57, c_login#58, c_email_address#59] -Condition : (isnotnull(c_customer_sk#52) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#53, 16)))) - -(38) CometProject -Input [8]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55, c_preferred_cust_flag#56, c_birth_country#57, c_login#58, c_email_address#59] -Arguments: [c_customer_sk#52, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65], [c_customer_sk#52, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#53, 16)) AS c_customer_id#60, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#54, 20)) AS c_first_name#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#55, 30)) AS c_last_name#62, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#56, 1)) AS c_preferred_cust_flag#63, c_birth_country#57, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#58, 13)) AS c_login#64, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#59, 50)) AS c_email_address#65] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#69), dynamicpruningexpression(ws_sold_date_sk#69 IN dynamicpruning#70)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] -Condition : isnotnull(ws_bill_customer_sk#66) - -(41) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] -Arguments: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] - -(42) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#52, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65] -Right output [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] -Arguments: [c_customer_sk#52], [ws_bill_customer_sk#66], Inner, BuildRight - -(43) CometProject -Input [12]: [c_customer_sk#52, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] -Arguments: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69], [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#71, d_year#72] - -(45) CometBroadcastHashJoin -Left output [10]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] -Right output [2]: [d_date_sk#71, d_year#72] -Arguments: [ws_sold_date_sk#69], [d_date_sk#71], Inner, BuildRight - -(46) CometProject -Input [12]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69, d_date_sk#71, d_year#72] -Arguments: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, d_year#72], [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, d_year#72] - -(47) CometHashAggregate -Input [10]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, d_year#72] -Keys [8]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, d_year#72] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#68 - ws_ext_discount_amt#67)))] - -(48) CometExchange -Input [9]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, d_year#72, sum#73] -Arguments: hashpartitioning(c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, d_year#72, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [9]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, d_year#72, sum#73] -Keys [8]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, d_year#72] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#68 - ws_ext_discount_amt#67)))] - -(50) CometFilter -Input [2]: [customer_id#74, year_total#75] -Condition : (isnotnull(year_total#75) AND (year_total#75 > 0.00)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#74, year_total#75] -Arguments: [customer_id#74, year_total#75] - -(52) CometBroadcastHashJoin -Left output [7]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] -Right output [2]: [customer_id#74, year_total#75] -Arguments: [customer_id#23], [customer_id#74], Inner, BuildRight - -(53) CometProject -Input [9]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, customer_id#74, year_total#75] -Arguments: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75], [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] -Condition : (isnotnull(c_customer_sk#76) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#77, 16)))) - -(56) CometProject -Input [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] -Arguments: [c_customer_sk#76, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89], [c_customer_sk#76, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#77, 16)) AS c_customer_id#84, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#78, 20)) AS c_first_name#85, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#79, 30)) AS c_last_name#86, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#80, 1)) AS c_preferred_cust_flag#87, c_birth_country#81, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#82, 13)) AS c_login#88, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#83, 50)) AS c_email_address#89] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#93), dynamicpruningexpression(ws_sold_date_sk#93 IN dynamicpruning#94)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] -Condition : isnotnull(ws_bill_customer_sk#90) - -(59) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] -Arguments: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] - -(60) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#76, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89] -Right output [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] -Arguments: [c_customer_sk#76], [ws_bill_customer_sk#90], Inner, BuildRight - -(61) CometProject -Input [12]: [c_customer_sk#76, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] -Arguments: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93], [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#95, d_year#96] - -(63) CometBroadcastHashJoin -Left output [10]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] -Right output [2]: [d_date_sk#95, d_year#96] -Arguments: [ws_sold_date_sk#93], [d_date_sk#95], Inner, BuildRight - -(64) CometProject -Input [12]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93, d_date_sk#95, d_year#96] -Arguments: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, d_year#96], [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, d_year#96] - -(65) CometHashAggregate -Input [10]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, d_year#96] -Keys [8]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#92 - ws_ext_discount_amt#91)))] - -(66) CometExchange -Input [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96, sum#97] -Arguments: hashpartitioning(c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96, sum#97] -Keys [8]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#92 - ws_ext_discount_amt#91)))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#98, year_total#99] -Arguments: [customer_id#98, year_total#99] - -(69) CometBroadcastHashJoin -Left output [8]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75] -Right output [2]: [customer_id#98, year_total#99] -Arguments: [customer_id#23], [customer_id#98], Inner, (CASE WHEN (year_total#75 > 0.00) THEN (year_total#99 / year_total#75) ELSE 0E-20 END > CASE WHEN (year_total#24 > 0.00) THEN (year_total#51 / year_total#24) ELSE 0E-20 END), BuildRight - -(70) CometProject -Input [10]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75, customer_id#98, year_total#99] -Arguments: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50], [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] - -(71) CometTakeOrderedAndProject -Input [4]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#47 ASC NULLS FIRST,customer_first_name#48 ASC NULLS FIRST,customer_last_name#49 ASC NULLS FIRST,customer_email_address#50 ASC NULLS FIRST], output=[customer_id#47,customer_first_name#48,customer_last_name#49,customer_email_address#50]), [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50], 100, 0, [customer_id#47 ASC NULLS FIRST, customer_first_name#48 ASC NULLS FIRST, customer_last_name#49 ASC NULLS FIRST, customer_email_address#50 ASC NULLS FIRST], [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] - -(72) CometColumnarToRow [codegen id : 1] -Input [4]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(75) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#20, d_year#21] - -(76) BroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#42 IN dynamicpruning#43 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) - -(79) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#44, d_year#45] - -(80) BroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#69 IN dynamicpruning#19 - -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#93 IN dynamicpruning#43 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/simplified.txt deleted file mode 100644 index 157d1d587e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - CometProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] #5 - CometHashAggregate [d_year,c_preferred_cust_flag,c_birth_country,c_login,sum] [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_email_address,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/explain.txt deleted file mode 100644 index 676d488824..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/explain.txt +++ /dev/null @@ -1,470 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometProject (53) - : +- CometBroadcastHashJoin (52) - : :- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (51) - : +- CometFilter (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : +- CometBroadcastExchange (41) - : : +- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) - : +- ReusedExchange (44) - +- CometBroadcastExchange (68) - +- CometHashAggregate (67) - +- CometExchange (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometFilter (55) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (62) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)) AS c_customer_id#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#3, 20)) AS c_first_name#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#4, 30)) AS c_last_name#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#5, 1)) AS c_preferred_cust_flag#12, c_birth_country#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#7, 13)) AS c_login#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#8, 50)) AS c_email_address#14] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#15) - -(6) CometBroadcastExchange -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(7) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_sk#1], [ss_customer_sk#15], Inner, BuildRight - -(8) CometProject -Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: [d_date_sk#20, d_year#21] - -(12) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Right output [2]: [d_date_sk#20, d_year#21] -Arguments: [ss_sold_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(13) CometProject -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] - -(14) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(15) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(17) CometFilter -Input [2]: [customer_id#23, year_total#24] -Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#26, 16)))) - -(20) CometProject -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Arguments: [c_customer_sk#25, c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38], [c_customer_sk#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#26, 16)) AS c_customer_id#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#27, 20)) AS c_first_name#34, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#28, 30)) AS c_last_name#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#29, 1)) AS c_preferred_cust_flag#36, c_birth_country#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#31, 13)) AS c_login#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#32, 50)) AS c_email_address#38] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#42), dynamicpruningexpression(ss_sold_date_sk#42 IN dynamicpruning#43)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Condition : isnotnull(ss_customer_sk#39) - -(23) CometBroadcastExchange -Input [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Arguments: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] - -(24) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#25, c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] -Right output [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Arguments: [c_customer_sk#25], [ss_customer_sk#39], Inner, BuildRight - -(25) CometProject -Input [12]: [c_customer_sk#25, c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Arguments: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42], [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] - -(29) CometBroadcastHashJoin -Left output [10]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [ss_sold_date_sk#42], [d_date_sk#44], Inner, BuildRight - -(30) CometProject -Input [12]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42, d_date_sk#44, d_year#45] -Arguments: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45], [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45] - -(31) CometHashAggregate -Input [10]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45] -Keys [8]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#41 - ss_ext_discount_amt#40)))] - -(32) CometExchange -Input [9]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, sum#46] -Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [9]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, sum#46] -Keys [8]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#41 - ss_ext_discount_amt#40)))] - -(34) CometBroadcastExchange -Input [5]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] -Arguments: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#23, year_total#24] -Right output [5]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] -Arguments: [customer_id#23], [customer_id#47], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55, c_preferred_cust_flag#56, c_birth_country#57, c_login#58, c_email_address#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [8]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55, c_preferred_cust_flag#56, c_birth_country#57, c_login#58, c_email_address#59] -Condition : (isnotnull(c_customer_sk#52) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#53, 16)))) - -(38) CometProject -Input [8]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55, c_preferred_cust_flag#56, c_birth_country#57, c_login#58, c_email_address#59] -Arguments: [c_customer_sk#52, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65], [c_customer_sk#52, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#53, 16)) AS c_customer_id#60, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#54, 20)) AS c_first_name#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#55, 30)) AS c_last_name#62, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#56, 1)) AS c_preferred_cust_flag#63, c_birth_country#57, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#58, 13)) AS c_login#64, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#59, 50)) AS c_email_address#65] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#69), dynamicpruningexpression(ws_sold_date_sk#69 IN dynamicpruning#70)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] -Condition : isnotnull(ws_bill_customer_sk#66) - -(41) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] -Arguments: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] - -(42) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#52, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65] -Right output [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] -Arguments: [c_customer_sk#52], [ws_bill_customer_sk#66], Inner, BuildRight - -(43) CometProject -Input [12]: [c_customer_sk#52, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] -Arguments: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69], [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#71, d_year#72] - -(45) CometBroadcastHashJoin -Left output [10]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] -Right output [2]: [d_date_sk#71, d_year#72] -Arguments: [ws_sold_date_sk#69], [d_date_sk#71], Inner, BuildRight - -(46) CometProject -Input [12]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69, d_date_sk#71, d_year#72] -Arguments: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, d_year#72], [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, d_year#72] - -(47) CometHashAggregate -Input [10]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, d_year#72] -Keys [8]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, d_year#72] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#68 - ws_ext_discount_amt#67)))] - -(48) CometExchange -Input [9]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, d_year#72, sum#73] -Arguments: hashpartitioning(c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, d_year#72, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [9]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, d_year#72, sum#73] -Keys [8]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, d_year#72] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#68 - ws_ext_discount_amt#67)))] - -(50) CometFilter -Input [2]: [customer_id#74, year_total#75] -Condition : (isnotnull(year_total#75) AND (year_total#75 > 0.00)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#74, year_total#75] -Arguments: [customer_id#74, year_total#75] - -(52) CometBroadcastHashJoin -Left output [7]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] -Right output [2]: [customer_id#74, year_total#75] -Arguments: [customer_id#23], [customer_id#74], Inner, BuildRight - -(53) CometProject -Input [9]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, customer_id#74, year_total#75] -Arguments: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75], [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] -Condition : (isnotnull(c_customer_sk#76) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#77, 16)))) - -(56) CometProject -Input [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] -Arguments: [c_customer_sk#76, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89], [c_customer_sk#76, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#77, 16)) AS c_customer_id#84, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#78, 20)) AS c_first_name#85, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#79, 30)) AS c_last_name#86, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#80, 1)) AS c_preferred_cust_flag#87, c_birth_country#81, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#82, 13)) AS c_login#88, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#83, 50)) AS c_email_address#89] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#93), dynamicpruningexpression(ws_sold_date_sk#93 IN dynamicpruning#94)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] -Condition : isnotnull(ws_bill_customer_sk#90) - -(59) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] -Arguments: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] - -(60) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#76, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89] -Right output [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] -Arguments: [c_customer_sk#76], [ws_bill_customer_sk#90], Inner, BuildRight - -(61) CometProject -Input [12]: [c_customer_sk#76, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] -Arguments: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93], [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#95, d_year#96] - -(63) CometBroadcastHashJoin -Left output [10]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] -Right output [2]: [d_date_sk#95, d_year#96] -Arguments: [ws_sold_date_sk#93], [d_date_sk#95], Inner, BuildRight - -(64) CometProject -Input [12]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93, d_date_sk#95, d_year#96] -Arguments: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, d_year#96], [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, d_year#96] - -(65) CometHashAggregate -Input [10]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, d_year#96] -Keys [8]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#92 - ws_ext_discount_amt#91)))] - -(66) CometExchange -Input [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96, sum#97] -Arguments: hashpartitioning(c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96, sum#97] -Keys [8]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#92 - ws_ext_discount_amt#91)))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#98, year_total#99] -Arguments: [customer_id#98, year_total#99] - -(69) CometBroadcastHashJoin -Left output [8]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75] -Right output [2]: [customer_id#98, year_total#99] -Arguments: [customer_id#23], [customer_id#98], Inner, (CASE WHEN (year_total#75 > 0.00) THEN (year_total#99 / year_total#75) ELSE 0E-20 END > CASE WHEN (year_total#24 > 0.00) THEN (year_total#51 / year_total#24) ELSE 0E-20 END), BuildRight - -(70) CometProject -Input [10]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75, customer_id#98, year_total#99] -Arguments: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50], [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] - -(71) CometTakeOrderedAndProject -Input [4]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#47 ASC NULLS FIRST,customer_first_name#48 ASC NULLS FIRST,customer_last_name#49 ASC NULLS FIRST,customer_email_address#50 ASC NULLS FIRST], output=[customer_id#47,customer_first_name#48,customer_last_name#49,customer_email_address#50]), [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50], 100, 0, [customer_id#47 ASC NULLS FIRST, customer_first_name#48 ASC NULLS FIRST, customer_last_name#49 ASC NULLS FIRST, customer_email_address#50 ASC NULLS FIRST], [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] - -(72) CometColumnarToRow [codegen id : 1] -Input [4]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(75) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#20, d_year#21] - -(76) BroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#42 IN dynamicpruning#43 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) - -(79) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#44, d_year#45] - -(80) BroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#69 IN dynamicpruning#19 - -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#93 IN dynamicpruning#43 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/extended.txt deleted file mode 100644 index 476c7be954..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/extended.txt +++ /dev/null @@ -1,90 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 157d1d587e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - CometProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] #5 - CometHashAggregate [d_year,c_preferred_cust_flag,c_birth_country,c_login,sum] [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_email_address,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/explain.txt deleted file mode 100644 index 51f95a9e5b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/explain.txt +++ /dev/null @@ -1,163 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.web_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.item (4) - +- ReusedExchange (11) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] - -(7) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(8) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(11) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] - -(14) HashAggregate [codegen id : 3] -Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(15) CometColumnarExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(17) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#17] -Results [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#17,17,2) AS _w0#19] - -(18) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(22) Project [codegen id : 6] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] - -(23) TakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometNativeScan parquet spark_catalog.default.date_dim (24) - - -(24) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#22] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/simplified.txt deleted file mode 100644 index c129b42cdb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/explain.txt deleted file mode 100644 index affae68344..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ws_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/simplified.txt deleted file mode 100644 index 58ac81ba0d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt deleted file mode 100644 index affae68344..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ws_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/extended.txt deleted file mode 100644 index 3f41c97ff5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 58ac81ba0d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/explain.txt deleted file mode 100644 index 5b9627c41c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/explain.txt +++ /dev/null @@ -1,773 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (87) -+- * BroadcastHashJoin Inner BuildRight (86) - :- * Filter (68) - : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) - : +- CometColumnarExchange (65) - : +- * HashAggregate (64) - : +- * Project (63) - : +- * BroadcastHashJoin Inner BuildRight (62) - : :- * Project (60) - : : +- * BroadcastHashJoin Inner BuildRight (59) - : : :- * BroadcastHashJoin LeftSemi BuildRight (52) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * CometColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (48) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * CometColumnarToRow (36) - : : : : +- CometHashAggregate (35) - : : : : +- CometColumnarExchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * CometColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * CometColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * Filter (39) - : : : : : +- * ColumnarToRow (38) - : : : : : +- Scan parquet spark_catalog.default.web_sales (37) - : : : : +- ReusedExchange (40) - : : : +- ReusedExchange (43) - : : +- BroadcastExchange (58) - : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : :- * CometColumnarToRow (55) - : : : +- CometFilter (54) - : : : +- CometNativeScan parquet spark_catalog.default.item (53) - : : +- ReusedExchange (56) - : +- ReusedExchange (61) - +- BroadcastExchange (85) - +- * Filter (84) - +- * HashAggregate (83) - +- * CometColumnarToRow (82) - +- CometColumnarExchange (81) - +- * HashAggregate (80) - +- * Project (79) - +- * BroadcastHashJoin Inner BuildRight (78) - :- * Project (76) - : +- * BroadcastHashJoin Inner BuildRight (75) - : :- * BroadcastHashJoin LeftSemi BuildRight (73) - : : :- * Filter (71) - : : : +- * ColumnarToRow (70) - : : : +- Scan parquet spark_catalog.default.store_sales (69) - : : +- ReusedExchange (72) - : +- ReusedExchange (74) - +- ReusedExchange (77) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(6) CometColumnarToRow [codegen id : 11] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(10) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(11) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(12) CometColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(16) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) - -(18) CometColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(19) BroadcastExchange -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#19] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(22) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#23] - -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#23] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#23] - -(25) BroadcastExchange -Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] -Join type: LeftSemi -Join condition: None - -(27) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None - -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(30) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#24] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#25, i_class_id#15 AS class_id#26, i_category_id#16 AS category_id#27] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#24] - -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#25, class_id#26, category_id#27] - -(34) CometColumnarExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(35) CometHashAggregate -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] - -(36) CometColumnarToRow [codegen id : 10] -Input [3]: [brand_id#25, class_id#26, category_id#27] - -(37) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] - -(39) Filter [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Condition : isnotnull(ws_item_sk#28) - -(40) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#28] -Right keys [1]: [i_item_sk#30] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] -Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(43) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#34] - -(44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#29] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 9] -Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] - -(46) BroadcastExchange -Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] - -(47) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] -Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] -Join type: LeftSemi -Join condition: None - -(48) BroadcastExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] - -(49) BroadcastHashJoin [codegen id : 11] -Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#25, class_id#26, category_id#27] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#35] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] - -(51) BroadcastExchange -Input [1]: [ss_item_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(52) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(53) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(54) CometFilter -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Condition : (((isnotnull(i_item_sk#36) AND isnotnull(i_brand_id#37)) AND isnotnull(i_class_id#38)) AND isnotnull(i_category_id#39)) - -(55) CometColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(56) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#35] - -(57) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#36] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(58) BroadcastExchange -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(59) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#36] -Join type: Inner -Join condition: None - -(60) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(61) ReusedExchange [Reuses operator id: 112] -Output [1]: [d_date_sk#40] - -(62) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] - -(64) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] -Results [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(65) CometColumnarExchange -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Arguments: hashpartitioning(i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(66) CometColumnarToRow [codegen id : 52] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(67) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] -Results [6]: [store AS channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51] - -(68) Filter [codegen id : 52] -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51] -Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(69) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_sold_date_sk#57 IN dynamicpruning#58)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(70) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] - -(71) Filter [codegen id : 50] -Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -Condition : isnotnull(ss_item_sk#54) - -(72) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#59] - -(73) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [ss_item_sk#59] -Join type: LeftSemi -Join condition: None - -(74) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] - -(75) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [i_item_sk#60] -Join type: Inner -Join condition: None - -(76) Project [codegen id : 50] -Output [6]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] -Input [8]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] - -(77) ReusedExchange [Reuses operator id: 126] -Output [1]: [d_date_sk#64] - -(78) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#57] -Right keys [1]: [d_date_sk#64] -Join type: Inner -Join condition: None - -(79) Project [codegen id : 50] -Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] -Input [7]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] - -(80) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)] -Aggregate Attributes [3]: [sum#65, isEmpty#66, count#67] -Results [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] - -(81) CometColumnarExchange -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] -Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(82) CometColumnarToRow [codegen id : 51] -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] - -(83) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#71, count(1)#72] -Results [6]: [store AS channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#71 AS sales#74, count(1)#72 AS number_sales#75] - -(84) Filter [codegen id : 51] -Input [6]: [channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] -Condition : (isnotnull(sales#74) AND (cast(sales#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(85) BroadcastExchange -Input [6]: [channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] - -(86) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Right keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Join type: Inner -Join condition: None - -(87) TakeOrderedAndProject -Input [12]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51, channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] -Arguments: 100, [i_brand_id#37 ASC NULLS FIRST, i_class_id#38 ASC NULLS FIRST, i_category_id#39 ASC NULLS FIRST], [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51, channel#73, i_brand_id#61, i_class_id#62, i_category_id#63, sales#74, number_sales#75] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* HashAggregate (107) -+- * CometColumnarToRow (106) - +- CometColumnarExchange (105) - +- * HashAggregate (104) - +- Union (103) - :- * Project (92) - : +- * BroadcastHashJoin Inner BuildRight (91) - : :- * ColumnarToRow (89) - : : +- Scan parquet spark_catalog.default.store_sales (88) - : +- ReusedExchange (90) - :- * Project (97) - : +- * BroadcastHashJoin Inner BuildRight (96) - : :- * ColumnarToRow (94) - : : +- Scan parquet spark_catalog.default.catalog_sales (93) - : +- ReusedExchange (95) - +- * Project (102) - +- * BroadcastHashJoin Inner BuildRight (101) - :- * ColumnarToRow (99) - : +- Scan parquet spark_catalog.default.web_sales (98) - +- ReusedExchange (100) - - -(88) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#12)] -ReadSchema: struct - -(89) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] - -(90) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#79] - -(91) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#78] -Right keys [1]: [d_date_sk#79] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 2] -Output [2]: [ss_quantity#76 AS quantity#80, ss_list_price#77 AS list_price#81] -Input [4]: [ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78, d_date_sk#79] - -(93) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#12)] -ReadSchema: struct - -(94) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] - -(95) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#85] - -(96) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#84] -Right keys [1]: [d_date_sk#85] -Join type: Inner -Join condition: None - -(97) Project [codegen id : 4] -Output [2]: [cs_quantity#82 AS quantity#86, cs_list_price#83 AS list_price#87] -Input [4]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84, d_date_sk#85] - -(98) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#90), dynamicpruningexpression(ws_sold_date_sk#90 IN dynamicpruning#12)] -ReadSchema: struct - -(99) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] - -(100) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#91] - -(101) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#90] -Right keys [1]: [d_date_sk#91] -Join type: Inner -Join condition: None - -(102) Project [codegen id : 6] -Output [2]: [ws_quantity#88 AS quantity#92, ws_list_price#89 AS list_price#93] -Input [4]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90, d_date_sk#91] - -(103) Union - -(104) HashAggregate [codegen id : 7] -Input [2]: [quantity#80, list_price#81] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] -Aggregate Attributes [2]: [sum#94, count#95] -Results [2]: [sum#96, count#97] - -(105) CometColumnarExchange -Input [2]: [sum#96, count#97] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(106) CometColumnarToRow [codegen id : 8] -Input [2]: [sum#96, count#97] - -(107) HashAggregate [codegen id : 8] -Input [2]: [sum#96, count#97] -Keys: [] -Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] -Aggregate Attributes [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))#98] -Results [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))#98 AS average_sales#99] - -Subquery:2 Hosting operator id = 88 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 93 Hosting Expression = cs_sold_date_sk#84 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 98 Hosting Expression = ws_sold_date_sk#90 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (112) -+- * CometColumnarToRow (111) - +- CometProject (110) - +- CometFilter (109) - +- CometNativeScan parquet spark_catalog.default.date_dim (108) - - -(108) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#40, d_week_seq#100] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#101), IsNotNull(d_date_sk)] -ReadSchema: struct - -(109) CometFilter -Input [2]: [d_date_sk#40, d_week_seq#100] -Condition : ((isnotnull(d_week_seq#100) AND (d_week_seq#100 = ReusedSubquery Subquery scalar-subquery#101, [id=#102])) AND isnotnull(d_date_sk#40)) - -(110) CometProject -Input [2]: [d_date_sk#40, d_week_seq#100] -Arguments: [d_date_sk#40], [d_date_sk#40] - -(111) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#40] - -(112) BroadcastExchange -Input [1]: [d_date_sk#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:6 Hosting operator id = 109 Hosting Expression = ReusedSubquery Subquery scalar-subquery#101, [id=#102] - -Subquery:7 Hosting operator id = 108 Hosting Expression = Subquery scalar-subquery#101, [id=#102] -* CometColumnarToRow (116) -+- CometProject (115) - +- CometFilter (114) - +- CometNativeScan parquet spark_catalog.default.date_dim (113) - - -(113) CometNativeScan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] -ReadSchema: struct - -(114) CometFilter -Input [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] -Condition : (((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND isnotnull(d_dom#106)) AND (d_year#104 = 1999)) AND (d_moy#105 = 12)) AND (d_dom#106 = 16)) - -(115) CometProject -Input [4]: [d_week_seq#103, d_year#104, d_moy#105, d_dom#106] -Arguments: [d_week_seq#103], [d_week_seq#103] - -(116) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#103] - -Subquery:8 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometNativeScan parquet spark_catalog.default.date_dim (117) - - -(117) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#107] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#24, d_year#107] -Condition : (((isnotnull(d_year#107) AND (d_year#107 >= 1998)) AND (d_year#107 <= 2000)) AND isnotnull(d_date_sk#24)) - -(119) CometProject -Input [2]: [d_date_sk#24, d_year#107] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#24] - -(121) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:9 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:10 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 - -Subquery:11 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:12 Hosting operator id = 69 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 -BroadcastExchange (126) -+- * CometColumnarToRow (125) - +- CometProject (124) - +- CometFilter (123) - +- CometNativeScan parquet spark_catalog.default.date_dim (122) - - -(122) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#64, d_week_seq#108] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#109), IsNotNull(d_date_sk)] -ReadSchema: struct - -(123) CometFilter -Input [2]: [d_date_sk#64, d_week_seq#108] -Condition : ((isnotnull(d_week_seq#108) AND (d_week_seq#108 = ReusedSubquery Subquery scalar-subquery#109, [id=#110])) AND isnotnull(d_date_sk#64)) - -(124) CometProject -Input [2]: [d_date_sk#64, d_week_seq#108] -Arguments: [d_date_sk#64], [d_date_sk#64] - -(125) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#64] - -(126) BroadcastExchange -Input [1]: [d_date_sk#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] - -Subquery:13 Hosting operator id = 123 Hosting Expression = ReusedSubquery Subquery scalar-subquery#109, [id=#110] - -Subquery:14 Hosting operator id = 122 Hosting Expression = Subquery scalar-subquery#109, [id=#110] -* CometColumnarToRow (130) -+- CometProject (129) - +- CometFilter (128) - +- CometNativeScan parquet spark_catalog.default.date_dim (127) - - -(127) CometNativeScan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#111, d_year#112, d_moy#113, d_dom#114] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] -ReadSchema: struct - -(128) CometFilter -Input [4]: [d_week_seq#111, d_year#112, d_moy#113, d_dom#114] -Condition : (((((isnotnull(d_year#112) AND isnotnull(d_moy#113)) AND isnotnull(d_dom#114)) AND (d_year#112 = 1998)) AND (d_moy#113 = 12)) AND (d_dom#114 = 16)) - -(129) CometProject -Input [4]: [d_week_seq#111, d_year#112, d_moy#113, d_dom#114] -Arguments: [d_week_seq#111], [d_week_seq#111] - -(130) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#111] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/simplified.txt deleted file mode 100644 index cc02a716ea..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/simplified.txt +++ /dev/null @@ -1,208 +0,0 @@ -TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (52) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Filter [sales] - Subquery #4 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #12 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (51) - Filter [sales] - ReusedSubquery [average_sales] #4 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (50) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #15 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/explain.txt deleted file mode 100644 index 31cec72dc1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,751 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (85) -+- CometTakeOrderedAndProject (84) - +- CometBroadcastHashJoin (83) - :- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometBroadcastHashJoin (47) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (46) - : : : +- CometProject (45) - : : : +- CometBroadcastHashJoin (44) - : : : :- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (43) - : : : +- CometBroadcastHashJoin (42) - : : : :- CometHashAggregate (32) - : : : : +- CometExchange (31) - : : : : +- CometHashAggregate (30) - : : : : +- CometProject (29) - : : : : +- CometBroadcastHashJoin (28) - : : : : :- CometProject (26) - : : : : : +- CometBroadcastHashJoin (25) - : : : : : :- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : +- CometBroadcastExchange (24) - : : : : : +- CometBroadcastHashJoin (23) - : : : : : :- CometFilter (8) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : +- CometBroadcastExchange (22) - : : : : : +- CometProject (21) - : : : : : +- CometBroadcastHashJoin (20) - : : : : : :- CometProject (15) - : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : :- CometFilter (10) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : +- CometBroadcastExchange (13) - : : : : : : +- CometFilter (12) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : +- CometBroadcastExchange (19) - : : : : : +- CometProject (18) - : : : : : +- CometFilter (17) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : +- ReusedExchange (27) - : : : +- CometBroadcastExchange (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (34) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometFilter (49) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : +- ReusedExchange (50) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometBroadcastExchange (82) - +- CometFilter (81) - +- CometHashAggregate (80) - +- CometExchange (79) - +- CometHashAggregate (78) - +- CometProject (77) - +- CometBroadcastHashJoin (76) - :- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometBroadcastHashJoin (68) - : : :- CometFilter (66) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (65) - : : +- ReusedExchange (67) - : +- ReusedExchange (69) - +- CometBroadcastExchange (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1998)) AND (d_year#25 <= 2000)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42)) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#45), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Condition : (isnotnull(sales#51) AND (cast(sales#51 as decimal(32,6)) > cast(Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Condition : isnotnull(ss_item_sk#55) - -(67) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#60] - -(68) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [1]: [ss_item_sk#60] -Arguments: [ss_item_sk#55], [ss_item_sk#60], LeftSemi, BuildRight - -(69) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] - -(70) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -Arguments: [ss_item_sk#55], [i_item_sk#61], Inner, BuildRight - -(71) CometProject -Input [8]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -Arguments: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#67), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#66] -Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = ReusedSubquery Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) - -(74) CometProject -Input [2]: [d_date_sk#65, d_week_seq#66] -Arguments: [d_date_sk#65], [d_date_sk#65] - -(75) CometBroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: [d_date_sk#65] - -(76) CometBroadcastHashJoin -Left output [6]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(77) CometProject -Input [7]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] -Arguments: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] - -(78) CometHashAggregate -Input [5]: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [partial_sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), partial_count(1)] - -(79) CometExchange -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(80) CometHashAggregate -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), count(1)] - -(81) CometFilter -Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(82) CometBroadcastExchange -Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -(83) CometBroadcastHashJoin -Left output [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Right output [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#62, i_class_id#63, i_category_id#64], Inner, BuildRight - -(84) CometTakeOrderedAndProject -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#72,i_brand_id#62,i_class_id#63,i_category_id#64,sales#73,number_sales#74]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74], 100, 0, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -(85) CometColumnarToRow [codegen id : 1] -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* CometColumnarToRow (102) -+- CometHashAggregate (101) - +- CometExchange (100) - +- CometHashAggregate (99) - +- CometUnion (98) - :- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (86) - : +- ReusedExchange (87) - :- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (90) - : +- ReusedExchange (91) - +- CometProject (97) - +- CometBroadcastHashJoin (96) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (94) - +- ReusedExchange (95) - - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#78)] -ReadSchema: struct - -(87) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#79] - -(88) CometBroadcastHashJoin -Left output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -Right output [1]: [d_date_sk#79] -Arguments: [ss_sold_date_sk#77], [d_date_sk#79], Inner, BuildRight - -(89) CometProject -Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#79] -Arguments: [quantity#80, list_price#81], [ss_quantity#75 AS quantity#80, ss_list_price#76 AS list_price#81] - -(90) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#85)] -ReadSchema: struct - -(91) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#86] - -(92) CometBroadcastHashJoin -Left output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] -Right output [1]: [d_date_sk#86] -Arguments: [cs_sold_date_sk#84], [d_date_sk#86], Inner, BuildRight - -(93) CometProject -Input [4]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84, d_date_sk#86] -Arguments: [quantity#87, list_price#88], [cs_quantity#82 AS quantity#87, cs_list_price#83 AS list_price#88] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#91), dynamicpruningexpression(ws_sold_date_sk#91 IN dynamicpruning#92)] -ReadSchema: struct - -(95) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#93] - -(96) CometBroadcastHashJoin -Left output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] -Right output [1]: [d_date_sk#93] -Arguments: [ws_sold_date_sk#91], [d_date_sk#93], Inner, BuildRight - -(97) CometProject -Input [4]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91, d_date_sk#93] -Arguments: [quantity#94, list_price#95], [ws_quantity#89 AS quantity#94, ws_list_price#90 AS list_price#95] - -(98) CometUnion -Child 0 Input [2]: [quantity#80, list_price#81] -Child 1 Input [2]: [quantity#87, list_price#88] -Child 2 Input [2]: [quantity#94, list_price#95] - -(99) CometHashAggregate -Input [2]: [quantity#80, list_price#81] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] - -(100) CometExchange -Input [2]: [sum#96, count#97] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(101) CometHashAggregate -Input [2]: [sum#96, count#97] -Keys: [] -Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] - -(102) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#98] - -Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#84 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (107) -+- * CometColumnarToRow (106) - +- CometProject (105) - +- CometFilter (104) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) - - -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#45), IsNotNull(d_date_sk)] -ReadSchema: struct - -(104) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(105) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(106) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(107) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:6 Hosting operator id = 104 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] - -Subquery:7 Hosting operator id = 103 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) -+- CometProject (110) - +- CometFilter (109) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) - - -(108) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] -ReadSchema: struct - -(109) CometFilter -Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d_dom#102)) AND (d_year#100 = 1999)) AND (d_moy#101 = 12)) AND (d_dom#102 = 16)) - -(110) CometProject -Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Arguments: [d_week_seq#99], [d_week_seq#99] - -(111) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#99] - -Subquery:8 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (116) -+- * CometColumnarToRow (115) - +- CometProject (114) - +- CometFilter (113) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) - - -(112) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#103] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(113) CometFilter -Input [2]: [d_date_sk#26, d_year#103] -Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 <= 2000)) AND isnotnull(d_date_sk#26)) - -(114) CometProject -Input [2]: [d_date_sk#26, d_year#103] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(115) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(116) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:9 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:10 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:11 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] - -Subquery:12 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] - -Subquery:13 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] - -Subquery:14 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#67), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#66] -Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = ReusedSubquery Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) - -(119) CometProject -Input [2]: [d_date_sk#65, d_week_seq#66] -Arguments: [d_date_sk#65], [d_date_sk#65] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#65] - -(121) BroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:15 Hosting operator id = 118 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] - -Subquery:16 Hosting operator id = 117 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* CometColumnarToRow (125) -+- CometProject (124) - +- CometFilter (123) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) - - -(122) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] -ReadSchema: struct - -(123) CometFilter -Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d_dom#107)) AND (d_year#105 = 1998)) AND (d_moy#106 = 12)) AND (d_dom#107 = 16)) - -(124) CometProject -Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Arguments: [d_week_seq#104], [d_week_seq#104] - -(125) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#104] - -Subquery:17 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] - -Subquery:18 Hosting operator id = 72 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/simplified.txt deleted file mode 100644 index 6b5264b76d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,157 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #14 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #1 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometBroadcastExchange [ss_item_sk] #3 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #4 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #5 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #10 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #11 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #3 - CometBroadcastExchange [d_date_sk] #13 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #4 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - ReusedExchange [ss_item_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastExchange [d_date_sk] #18 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/explain.txt deleted file mode 100644 index 31cec72dc1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/explain.txt +++ /dev/null @@ -1,751 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (85) -+- CometTakeOrderedAndProject (84) - +- CometBroadcastHashJoin (83) - :- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometBroadcastHashJoin (47) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (46) - : : : +- CometProject (45) - : : : +- CometBroadcastHashJoin (44) - : : : :- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (43) - : : : +- CometBroadcastHashJoin (42) - : : : :- CometHashAggregate (32) - : : : : +- CometExchange (31) - : : : : +- CometHashAggregate (30) - : : : : +- CometProject (29) - : : : : +- CometBroadcastHashJoin (28) - : : : : :- CometProject (26) - : : : : : +- CometBroadcastHashJoin (25) - : : : : : :- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : +- CometBroadcastExchange (24) - : : : : : +- CometBroadcastHashJoin (23) - : : : : : :- CometFilter (8) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : +- CometBroadcastExchange (22) - : : : : : +- CometProject (21) - : : : : : +- CometBroadcastHashJoin (20) - : : : : : :- CometProject (15) - : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : :- CometFilter (10) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : +- CometBroadcastExchange (13) - : : : : : : +- CometFilter (12) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : +- CometBroadcastExchange (19) - : : : : : +- CometProject (18) - : : : : : +- CometFilter (17) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : +- ReusedExchange (27) - : : : +- CometBroadcastExchange (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (34) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometFilter (49) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : +- ReusedExchange (50) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometBroadcastExchange (82) - +- CometFilter (81) - +- CometHashAggregate (80) - +- CometExchange (79) - +- CometHashAggregate (78) - +- CometProject (77) - +- CometBroadcastHashJoin (76) - :- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometBroadcastHashJoin (68) - : : :- CometFilter (66) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (65) - : : +- ReusedExchange (67) - : +- ReusedExchange (69) - +- CometBroadcastExchange (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1998)) AND (d_year#25 <= 2000)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42)) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#45), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Condition : (isnotnull(sales#51) AND (cast(sales#51 as decimal(32,6)) > cast(Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Condition : isnotnull(ss_item_sk#55) - -(67) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#60] - -(68) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [1]: [ss_item_sk#60] -Arguments: [ss_item_sk#55], [ss_item_sk#60], LeftSemi, BuildRight - -(69) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] - -(70) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [4]: [i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -Arguments: [ss_item_sk#55], [i_item_sk#61], Inner, BuildRight - -(71) CometProject -Input [8]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_item_sk#61, i_brand_id#62, i_class_id#63, i_category_id#64] -Arguments: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#67), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#66] -Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = ReusedSubquery Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) - -(74) CometProject -Input [2]: [d_date_sk#65, d_week_seq#66] -Arguments: [d_date_sk#65], [d_date_sk#65] - -(75) CometBroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: [d_date_sk#65] - -(76) CometBroadcastHashJoin -Left output [6]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(77) CometProject -Input [7]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#62, i_class_id#63, i_category_id#64, d_date_sk#65] -Arguments: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64], [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] - -(78) CometHashAggregate -Input [5]: [ss_quantity#56, ss_list_price#57, i_brand_id#62, i_class_id#63, i_category_id#64] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [partial_sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), partial_count(1)] - -(79) CometExchange -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Arguments: hashpartitioning(i_brand_id#62, i_class_id#63, i_category_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(80) CometHashAggregate -Input [6]: [i_brand_id#62, i_class_id#63, i_category_id#64, sum#69, isEmpty#70, count#71] -Keys [3]: [i_brand_id#62, i_class_id#63, i_category_id#64] -Functions [2]: [sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), count(1)] - -(81) CometFilter -Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(82) CometBroadcastExchange -Input [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -(83) CometBroadcastHashJoin -Left output [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Right output [6]: [channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#62, i_class_id#63, i_category_id#64], Inner, BuildRight - -(84) CometTakeOrderedAndProject -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#72,i_brand_id#62,i_class_id#63,i_category_id#64,sales#73,number_sales#74]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74], 100, 0, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -(85) CometColumnarToRow [codegen id : 1] -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#72, i_brand_id#62, i_class_id#63, i_category_id#64, sales#73, number_sales#74] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* CometColumnarToRow (102) -+- CometHashAggregate (101) - +- CometExchange (100) - +- CometHashAggregate (99) - +- CometUnion (98) - :- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (86) - : +- ReusedExchange (87) - :- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (90) - : +- ReusedExchange (91) - +- CometProject (97) - +- CometBroadcastHashJoin (96) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (94) - +- ReusedExchange (95) - - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#78)] -ReadSchema: struct - -(87) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#79] - -(88) CometBroadcastHashJoin -Left output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -Right output [1]: [d_date_sk#79] -Arguments: [ss_sold_date_sk#77], [d_date_sk#79], Inner, BuildRight - -(89) CometProject -Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#79] -Arguments: [quantity#80, list_price#81], [ss_quantity#75 AS quantity#80, ss_list_price#76 AS list_price#81] - -(90) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#84), dynamicpruningexpression(cs_sold_date_sk#84 IN dynamicpruning#85)] -ReadSchema: struct - -(91) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#86] - -(92) CometBroadcastHashJoin -Left output [3]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84] -Right output [1]: [d_date_sk#86] -Arguments: [cs_sold_date_sk#84], [d_date_sk#86], Inner, BuildRight - -(93) CometProject -Input [4]: [cs_quantity#82, cs_list_price#83, cs_sold_date_sk#84, d_date_sk#86] -Arguments: [quantity#87, list_price#88], [cs_quantity#82 AS quantity#87, cs_list_price#83 AS list_price#88] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#91), dynamicpruningexpression(ws_sold_date_sk#91 IN dynamicpruning#92)] -ReadSchema: struct - -(95) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#93] - -(96) CometBroadcastHashJoin -Left output [3]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91] -Right output [1]: [d_date_sk#93] -Arguments: [ws_sold_date_sk#91], [d_date_sk#93], Inner, BuildRight - -(97) CometProject -Input [4]: [ws_quantity#89, ws_list_price#90, ws_sold_date_sk#91, d_date_sk#93] -Arguments: [quantity#94, list_price#95], [ws_quantity#89 AS quantity#94, ws_list_price#90 AS list_price#95] - -(98) CometUnion -Child 0 Input [2]: [quantity#80, list_price#81] -Child 1 Input [2]: [quantity#87, list_price#88] -Child 2 Input [2]: [quantity#94, list_price#95] - -(99) CometHashAggregate -Input [2]: [quantity#80, list_price#81] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] - -(100) CometExchange -Input [2]: [sum#96, count#97] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(101) CometHashAggregate -Input [2]: [sum#96, count#97] -Keys: [] -Functions [1]: [avg((cast(quantity#80 as decimal(10,0)) * list_price#81))] - -(102) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#98] - -Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#84 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#91 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (107) -+- * CometColumnarToRow (106) - +- CometProject (105) - +- CometFilter (104) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) - - -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#45), IsNotNull(d_date_sk)] -ReadSchema: struct - -(104) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(105) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(106) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(107) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:6 Hosting operator id = 104 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] - -Subquery:7 Hosting operator id = 103 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) -+- CometProject (110) - +- CometFilter (109) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) - - -(108) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] -ReadSchema: struct - -(109) CometFilter -Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Condition : (((((isnotnull(d_year#100) AND isnotnull(d_moy#101)) AND isnotnull(d_dom#102)) AND (d_year#100 = 1999)) AND (d_moy#101 = 12)) AND (d_dom#102 = 16)) - -(110) CometProject -Input [4]: [d_week_seq#99, d_year#100, d_moy#101, d_dom#102] -Arguments: [d_week_seq#99], [d_week_seq#99] - -(111) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#99] - -Subquery:8 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (116) -+- * CometColumnarToRow (115) - +- CometProject (114) - +- CometFilter (113) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) - - -(112) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#103] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(113) CometFilter -Input [2]: [d_date_sk#26, d_year#103] -Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 <= 2000)) AND isnotnull(d_date_sk#26)) - -(114) CometProject -Input [2]: [d_date_sk#26, d_year#103] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(115) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(116) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:9 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:10 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:11 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] - -Subquery:12 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] - -Subquery:13 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] - -Subquery:14 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#65, d_week_seq#66] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#67), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#65, d_week_seq#66] -Condition : ((isnotnull(d_week_seq#66) AND (d_week_seq#66 = ReusedSubquery Subquery scalar-subquery#67, [id=#68])) AND isnotnull(d_date_sk#65)) - -(119) CometProject -Input [2]: [d_date_sk#65, d_week_seq#66] -Arguments: [d_date_sk#65], [d_date_sk#65] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#65] - -(121) BroadcastExchange -Input [1]: [d_date_sk#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:15 Hosting operator id = 118 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] - -Subquery:16 Hosting operator id = 117 Hosting Expression = Subquery scalar-subquery#67, [id=#68] -* CometColumnarToRow (125) -+- CometProject (124) - +- CometFilter (123) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) - - -(122) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] -ReadSchema: struct - -(123) CometFilter -Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Condition : (((((isnotnull(d_year#105) AND isnotnull(d_moy#106)) AND isnotnull(d_dom#107)) AND (d_year#105 = 1998)) AND (d_moy#106 = 12)) AND (d_dom#107 = 16)) - -(124) CometProject -Input [4]: [d_week_seq#104, d_year#105, d_moy#106, d_dom#107] -Arguments: [d_week_seq#104], [d_week_seq#104] - -(125) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#104] - -Subquery:17 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] - -Subquery:18 Hosting operator id = 72 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/extended.txt deleted file mode 100644 index b32dd95954..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/extended.txt +++ /dev/null @@ -1,343 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometBroadcastHashJoin - :- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- ReusedSubquery - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- ReusedSubquery - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- ReusedSubquery - -Comet accelerated 298 out of 331 eligible operators (90%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 6b5264b76d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt +++ /dev/null @@ -1,157 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #14 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #1 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometBroadcastExchange [ss_item_sk] #3 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #4 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #5 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #10 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #11 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #3 - CometBroadcastExchange [d_date_sk] #13 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #4 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - ReusedExchange [ss_item_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastExchange [d_date_sk] #18 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/explain.txt deleted file mode 100644 index d17a5a745a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/explain.txt +++ /dev/null @@ -1,1020 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (139) -+- CometTakeOrderedAndProject (138) - +- CometHashAggregate (137) - +- CometColumnarExchange (136) - +- * HashAggregate (135) - +- Union (134) - :- * HashAggregate (105) - : +- * CometColumnarToRow (104) - : +- CometColumnarExchange (103) - : +- * HashAggregate (102) - : +- Union (101) - : :- * Filter (68) - : : +- * HashAggregate (67) - : : +- * CometColumnarToRow (66) - : : +- CometColumnarExchange (65) - : : +- * HashAggregate (64) - : : +- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildRight (59) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (52) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (51) - : : : : +- * Project (50) - : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : :- * CometColumnarToRow (6) - : : : : : +- CometFilter (5) - : : : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : : : +- BroadcastExchange (48) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : : :- * CometColumnarToRow (36) - : : : : : +- CometHashAggregate (35) - : : : : : +- CometColumnarExchange (34) - : : : : : +- * HashAggregate (33) - : : : : : +- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Project (29) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : : +- BroadcastExchange (27) - : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : : :- * CometColumnarToRow (12) - : : : : : : : +- CometFilter (11) - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) - : : : : : : +- BroadcastExchange (25) - : : : : : : +- * Project (24) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Filter (15) - : : : : : : : : +- * ColumnarToRow (14) - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * CometColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) - : : : : : : +- ReusedExchange (22) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (46) - : : : : +- * Project (45) - : : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : : :- * Project (42) - : : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : : :- * Filter (39) - : : : : : : +- * ColumnarToRow (38) - : : : : : : +- Scan parquet spark_catalog.default.web_sales (37) - : : : : : +- ReusedExchange (40) - : : : : +- ReusedExchange (43) - : : : +- BroadcastExchange (58) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : : :- * CometColumnarToRow (55) - : : : : +- CometFilter (54) - : : : : +- CometNativeScan parquet spark_catalog.default.item (53) - : : : +- ReusedExchange (56) - : : +- ReusedExchange (61) - : :- * Filter (84) - : : +- * HashAggregate (83) - : : +- * CometColumnarToRow (82) - : : +- CometColumnarExchange (81) - : : +- * HashAggregate (80) - : : +- * Project (79) - : : +- * BroadcastHashJoin Inner BuildRight (78) - : : :- * Project (76) - : : : +- * BroadcastHashJoin Inner BuildRight (75) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (73) - : : : : :- * Filter (71) - : : : : : +- * ColumnarToRow (70) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (69) - : : : : +- ReusedExchange (72) - : : : +- ReusedExchange (74) - : : +- ReusedExchange (77) - : +- * Filter (100) - : +- * HashAggregate (99) - : +- * CometColumnarToRow (98) - : +- CometColumnarExchange (97) - : +- * HashAggregate (96) - : +- * Project (95) - : +- * BroadcastHashJoin Inner BuildRight (94) - : :- * Project (92) - : : +- * BroadcastHashJoin Inner BuildRight (91) - : : :- * BroadcastHashJoin LeftSemi BuildRight (89) - : : : :- * Filter (87) - : : : : +- * ColumnarToRow (86) - : : : : +- Scan parquet spark_catalog.default.web_sales (85) - : : : +- ReusedExchange (88) - : : +- ReusedExchange (90) - : +- ReusedExchange (93) - :- * HashAggregate (112) - : +- * CometColumnarToRow (111) - : +- CometColumnarExchange (110) - : +- * HashAggregate (109) - : +- * HashAggregate (108) - : +- * CometColumnarToRow (107) - : +- ReusedExchange (106) - :- * HashAggregate (119) - : +- * CometColumnarToRow (118) - : +- CometColumnarExchange (117) - : +- * HashAggregate (116) - : +- * HashAggregate (115) - : +- * CometColumnarToRow (114) - : +- ReusedExchange (113) - :- * HashAggregate (126) - : +- * CometColumnarToRow (125) - : +- CometColumnarExchange (124) - : +- * HashAggregate (123) - : +- * HashAggregate (122) - : +- * CometColumnarToRow (121) - : +- ReusedExchange (120) - +- * HashAggregate (133) - +- * CometColumnarToRow (132) - +- CometColumnarExchange (131) - +- * HashAggregate (130) - +- * HashAggregate (129) - +- * CometColumnarToRow (128) - +- ReusedExchange (127) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(6) CometColumnarToRow [codegen id : 11] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(10) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(11) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(12) CometColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(16) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) - -(18) CometColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(19) BroadcastExchange -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#19] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(22) ReusedExchange [Reuses operator id: 174] -Output [1]: [d_date_sk#23] - -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#23] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#23] - -(25) BroadcastExchange -Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] -Join type: LeftSemi -Join condition: None - -(27) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None - -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(30) ReusedExchange [Reuses operator id: 174] -Output [1]: [d_date_sk#24] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#25, i_class_id#15 AS class_id#26, i_category_id#16 AS category_id#27] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#24] - -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#25, class_id#26, category_id#27] - -(34) CometColumnarExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(35) CometHashAggregate -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] - -(36) CometColumnarToRow [codegen id : 10] -Input [3]: [brand_id#25, class_id#26, category_id#27] - -(37) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] - -(39) Filter [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Condition : isnotnull(ws_item_sk#28) - -(40) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#28] -Right keys [1]: [i_item_sk#30] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] -Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(43) ReusedExchange [Reuses operator id: 174] -Output [1]: [d_date_sk#34] - -(44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#29] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 9] -Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] - -(46) BroadcastExchange -Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] - -(47) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] -Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] -Join type: LeftSemi -Join condition: None - -(48) BroadcastExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] - -(49) BroadcastHashJoin [codegen id : 11] -Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#25, class_id#26, category_id#27] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#35] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] - -(51) BroadcastExchange -Input [1]: [ss_item_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(52) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(53) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(54) CometFilter -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Condition : isnotnull(i_item_sk#36) - -(55) CometColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(56) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#35] - -(57) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#36] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(58) BroadcastExchange -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(59) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#36] -Join type: Inner -Join condition: None - -(60) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(61) ReusedExchange [Reuses operator id: 169] -Output [1]: [d_date_sk#40] - -(62) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] - -(64) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] -Results [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(65) CometColumnarExchange -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Arguments: hashpartitioning(i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(66) CometColumnarToRow [codegen id : 26] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(67) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] -Results [6]: [store AS channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51] - -(68) Filter [codegen id : 26] -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51] -Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(69) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_sold_date_sk#57 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(70) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] - -(71) Filter [codegen id : 51] -Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Condition : isnotnull(cs_item_sk#54) - -(72) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#58] - -(73) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#54] -Right keys [1]: [ss_item_sk#58] -Join type: LeftSemi -Join condition: None - -(74) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] - -(75) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#54] -Right keys [1]: [i_item_sk#59] -Join type: Inner -Join condition: None - -(76) Project [codegen id : 51] -Output [6]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62] -Input [8]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] - -(77) ReusedExchange [Reuses operator id: 169] -Output [1]: [d_date_sk#63] - -(78) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#57] -Right keys [1]: [d_date_sk#63] -Join type: Inner -Join condition: None - -(79) Project [codegen id : 51] -Output [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] -Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62, d_date_sk#63] - -(80) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] -Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] -Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] - -(81) CometColumnarExchange -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] -Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(82) CometColumnarToRow [codegen id : 52] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] - -(83) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] -Aggregate Attributes [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#70, count(1)#71] -Results [6]: [catalog AS channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#70 AS sales#73, count(1)#71 AS number_sales#74] - -(84) Filter [codegen id : 52] -Input [6]: [channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] -Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(85) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#75, ws_quantity#76, ws_list_price#77, ws_sold_date_sk#78] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#78), dynamicpruningexpression(ws_sold_date_sk#78 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(86) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#75, ws_quantity#76, ws_list_price#77, ws_sold_date_sk#78] - -(87) Filter [codegen id : 77] -Input [4]: [ws_item_sk#75, ws_quantity#76, ws_list_price#77, ws_sold_date_sk#78] -Condition : isnotnull(ws_item_sk#75) - -(88) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#79] - -(89) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#75] -Right keys [1]: [ss_item_sk#79] -Join type: LeftSemi -Join condition: None - -(90) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] - -(91) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#75] -Right keys [1]: [i_item_sk#80] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 77] -Output [6]: [ws_quantity#76, ws_list_price#77, ws_sold_date_sk#78, i_brand_id#81, i_class_id#82, i_category_id#83] -Input [8]: [ws_item_sk#75, ws_quantity#76, ws_list_price#77, ws_sold_date_sk#78, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] - -(93) ReusedExchange [Reuses operator id: 169] -Output [1]: [d_date_sk#84] - -(94) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#78] -Right keys [1]: [d_date_sk#84] -Join type: Inner -Join condition: None - -(95) Project [codegen id : 77] -Output [5]: [ws_quantity#76, ws_list_price#77, i_brand_id#81, i_class_id#82, i_category_id#83] -Input [7]: [ws_quantity#76, ws_list_price#77, ws_sold_date_sk#78, i_brand_id#81, i_class_id#82, i_category_id#83, d_date_sk#84] - -(96) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#76, ws_list_price#77, i_brand_id#81, i_class_id#82, i_category_id#83] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [partial_sum((cast(ws_quantity#76 as decimal(10,0)) * ws_list_price#77)), partial_count(1)] -Aggregate Attributes [3]: [sum#85, isEmpty#86, count#87] -Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#88, isEmpty#89, count#90] - -(97) CometColumnarExchange -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#88, isEmpty#89, count#90] -Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(98) CometColumnarToRow [codegen id : 78] -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#88, isEmpty#89, count#90] - -(99) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#88, isEmpty#89, count#90] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [sum((cast(ws_quantity#76 as decimal(10,0)) * ws_list_price#77)), count(1)] -Aggregate Attributes [2]: [sum((cast(ws_quantity#76 as decimal(10,0)) * ws_list_price#77))#91, count(1)#92] -Results [6]: [web AS channel#93, i_brand_id#81, i_class_id#82, i_category_id#83, sum((cast(ws_quantity#76 as decimal(10,0)) * ws_list_price#77))#91 AS sales#94, count(1)#92 AS number_sales#95] - -(100) Filter [codegen id : 78] -Input [6]: [channel#93, i_brand_id#81, i_class_id#82, i_category_id#83, sales#94, number_sales#95] -Condition : (isnotnull(sales#94) AND (cast(sales#94 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(101) Union - -(102) HashAggregate [codegen id : 79] -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51] -Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] -Aggregate Attributes [3]: [sum#96, isEmpty#97, sum#98] -Results [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#99, isEmpty#100, sum#101] - -(103) CometColumnarExchange -Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#99, isEmpty#100, sum#101] -Arguments: hashpartitioning(channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(104) CometColumnarToRow [codegen id : 80] -Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#99, isEmpty#100, sum#101] - -(105) HashAggregate [codegen id : 80] -Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#99, isEmpty#100, sum#101] -Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [sum(sales#50), sum(number_sales#51)] -Aggregate Attributes [2]: [sum(sales#50)#102, sum(number_sales#51)#103] -Results [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum(sales#50)#102 AS sum_sales#104, sum(number_sales#51)#103 AS number_sales#105] - -(106) ReusedExchange [Reuses operator id: 103] -Output [7]: [channel#106, i_brand_id#107, i_class_id#108, i_category_id#109, sum#110, isEmpty#111, sum#112] - -(107) CometColumnarToRow [codegen id : 160] -Input [7]: [channel#106, i_brand_id#107, i_class_id#108, i_category_id#109, sum#110, isEmpty#111, sum#112] - -(108) HashAggregate [codegen id : 160] -Input [7]: [channel#106, i_brand_id#107, i_class_id#108, i_category_id#109, sum#110, isEmpty#111, sum#112] -Keys [4]: [channel#106, i_brand_id#107, i_class_id#108, i_category_id#109] -Functions [2]: [sum(sales#113), sum(number_sales#114)] -Aggregate Attributes [2]: [sum(sales#113)#102, sum(number_sales#114)#103] -Results [5]: [channel#106, i_brand_id#107, i_class_id#108, sum(sales#113)#102 AS sum_sales#115, sum(number_sales#114)#103 AS number_sales#116] - -(109) HashAggregate [codegen id : 160] -Input [5]: [channel#106, i_brand_id#107, i_class_id#108, sum_sales#115, number_sales#116] -Keys [3]: [channel#106, i_brand_id#107, i_class_id#108] -Functions [2]: [partial_sum(sum_sales#115), partial_sum(number_sales#116)] -Aggregate Attributes [3]: [sum#117, isEmpty#118, sum#119] -Results [6]: [channel#106, i_brand_id#107, i_class_id#108, sum#120, isEmpty#121, sum#122] - -(110) CometColumnarExchange -Input [6]: [channel#106, i_brand_id#107, i_class_id#108, sum#120, isEmpty#121, sum#122] -Arguments: hashpartitioning(channel#106, i_brand_id#107, i_class_id#108, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(111) CometColumnarToRow [codegen id : 161] -Input [6]: [channel#106, i_brand_id#107, i_class_id#108, sum#120, isEmpty#121, sum#122] - -(112) HashAggregate [codegen id : 161] -Input [6]: [channel#106, i_brand_id#107, i_class_id#108, sum#120, isEmpty#121, sum#122] -Keys [3]: [channel#106, i_brand_id#107, i_class_id#108] -Functions [2]: [sum(sum_sales#115), sum(number_sales#116)] -Aggregate Attributes [2]: [sum(sum_sales#115)#123, sum(number_sales#116)#124] -Results [6]: [channel#106, i_brand_id#107, i_class_id#108, null AS i_category_id#125, sum(sum_sales#115)#123 AS sum(sum_sales)#126, sum(number_sales#116)#124 AS sum(number_sales)#127] - -(113) ReusedExchange [Reuses operator id: 103] -Output [7]: [channel#128, i_brand_id#129, i_class_id#130, i_category_id#131, sum#132, isEmpty#133, sum#134] - -(114) CometColumnarToRow [codegen id : 241] -Input [7]: [channel#128, i_brand_id#129, i_class_id#130, i_category_id#131, sum#132, isEmpty#133, sum#134] - -(115) HashAggregate [codegen id : 241] -Input [7]: [channel#128, i_brand_id#129, i_class_id#130, i_category_id#131, sum#132, isEmpty#133, sum#134] -Keys [4]: [channel#128, i_brand_id#129, i_class_id#130, i_category_id#131] -Functions [2]: [sum(sales#135), sum(number_sales#136)] -Aggregate Attributes [2]: [sum(sales#135)#102, sum(number_sales#136)#103] -Results [4]: [channel#128, i_brand_id#129, sum(sales#135)#102 AS sum_sales#137, sum(number_sales#136)#103 AS number_sales#138] - -(116) HashAggregate [codegen id : 241] -Input [4]: [channel#128, i_brand_id#129, sum_sales#137, number_sales#138] -Keys [2]: [channel#128, i_brand_id#129] -Functions [2]: [partial_sum(sum_sales#137), partial_sum(number_sales#138)] -Aggregate Attributes [3]: [sum#139, isEmpty#140, sum#141] -Results [5]: [channel#128, i_brand_id#129, sum#142, isEmpty#143, sum#144] - -(117) CometColumnarExchange -Input [5]: [channel#128, i_brand_id#129, sum#142, isEmpty#143, sum#144] -Arguments: hashpartitioning(channel#128, i_brand_id#129, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(118) CometColumnarToRow [codegen id : 242] -Input [5]: [channel#128, i_brand_id#129, sum#142, isEmpty#143, sum#144] - -(119) HashAggregate [codegen id : 242] -Input [5]: [channel#128, i_brand_id#129, sum#142, isEmpty#143, sum#144] -Keys [2]: [channel#128, i_brand_id#129] -Functions [2]: [sum(sum_sales#137), sum(number_sales#138)] -Aggregate Attributes [2]: [sum(sum_sales#137)#145, sum(number_sales#138)#146] -Results [6]: [channel#128, i_brand_id#129, null AS i_class_id#147, null AS i_category_id#148, sum(sum_sales#137)#145 AS sum(sum_sales)#149, sum(number_sales#138)#146 AS sum(number_sales)#150] - -(120) ReusedExchange [Reuses operator id: 103] -Output [7]: [channel#151, i_brand_id#152, i_class_id#153, i_category_id#154, sum#155, isEmpty#156, sum#157] - -(121) CometColumnarToRow [codegen id : 322] -Input [7]: [channel#151, i_brand_id#152, i_class_id#153, i_category_id#154, sum#155, isEmpty#156, sum#157] - -(122) HashAggregate [codegen id : 322] -Input [7]: [channel#151, i_brand_id#152, i_class_id#153, i_category_id#154, sum#155, isEmpty#156, sum#157] -Keys [4]: [channel#151, i_brand_id#152, i_class_id#153, i_category_id#154] -Functions [2]: [sum(sales#158), sum(number_sales#159)] -Aggregate Attributes [2]: [sum(sales#158)#102, sum(number_sales#159)#103] -Results [3]: [channel#151, sum(sales#158)#102 AS sum_sales#160, sum(number_sales#159)#103 AS number_sales#161] - -(123) HashAggregate [codegen id : 322] -Input [3]: [channel#151, sum_sales#160, number_sales#161] -Keys [1]: [channel#151] -Functions [2]: [partial_sum(sum_sales#160), partial_sum(number_sales#161)] -Aggregate Attributes [3]: [sum#162, isEmpty#163, sum#164] -Results [4]: [channel#151, sum#165, isEmpty#166, sum#167] - -(124) CometColumnarExchange -Input [4]: [channel#151, sum#165, isEmpty#166, sum#167] -Arguments: hashpartitioning(channel#151, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] - -(125) CometColumnarToRow [codegen id : 323] -Input [4]: [channel#151, sum#165, isEmpty#166, sum#167] - -(126) HashAggregate [codegen id : 323] -Input [4]: [channel#151, sum#165, isEmpty#166, sum#167] -Keys [1]: [channel#151] -Functions [2]: [sum(sum_sales#160), sum(number_sales#161)] -Aggregate Attributes [2]: [sum(sum_sales#160)#168, sum(number_sales#161)#169] -Results [6]: [channel#151, null AS i_brand_id#170, null AS i_class_id#171, null AS i_category_id#172, sum(sum_sales#160)#168 AS sum(sum_sales)#173, sum(number_sales#161)#169 AS sum(number_sales)#174] - -(127) ReusedExchange [Reuses operator id: 103] -Output [7]: [channel#175, i_brand_id#176, i_class_id#177, i_category_id#178, sum#179, isEmpty#180, sum#181] - -(128) CometColumnarToRow [codegen id : 403] -Input [7]: [channel#175, i_brand_id#176, i_class_id#177, i_category_id#178, sum#179, isEmpty#180, sum#181] - -(129) HashAggregate [codegen id : 403] -Input [7]: [channel#175, i_brand_id#176, i_class_id#177, i_category_id#178, sum#179, isEmpty#180, sum#181] -Keys [4]: [channel#175, i_brand_id#176, i_class_id#177, i_category_id#178] -Functions [2]: [sum(sales#182), sum(number_sales#183)] -Aggregate Attributes [2]: [sum(sales#182)#102, sum(number_sales#183)#103] -Results [2]: [sum(sales#182)#102 AS sum_sales#184, sum(number_sales#183)#103 AS number_sales#185] - -(130) HashAggregate [codegen id : 403] -Input [2]: [sum_sales#184, number_sales#185] -Keys: [] -Functions [2]: [partial_sum(sum_sales#184), partial_sum(number_sales#185)] -Aggregate Attributes [3]: [sum#186, isEmpty#187, sum#188] -Results [3]: [sum#189, isEmpty#190, sum#191] - -(131) CometColumnarExchange -Input [3]: [sum#189, isEmpty#190, sum#191] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(132) CometColumnarToRow [codegen id : 404] -Input [3]: [sum#189, isEmpty#190, sum#191] - -(133) HashAggregate [codegen id : 404] -Input [3]: [sum#189, isEmpty#190, sum#191] -Keys: [] -Functions [2]: [sum(sum_sales#184), sum(number_sales#185)] -Aggregate Attributes [2]: [sum(sum_sales#184)#192, sum(number_sales#185)#193] -Results [6]: [null AS channel#194, null AS i_brand_id#195, null AS i_class_id#196, null AS i_category_id#197, sum(sum_sales#184)#192 AS sum(sum_sales)#198, sum(number_sales#185)#193 AS sum(number_sales)#199] - -(134) Union - -(135) HashAggregate [codegen id : 405] -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] -Keys [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] -Functions: [] -Aggregate Attributes: [] -Results [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] - -(136) CometColumnarExchange -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] -Arguments: hashpartitioning(channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] - -(137) CometHashAggregate -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] -Keys [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] -Functions: [] - -(138) CometTakeOrderedAndProject -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#49 ASC NULLS FIRST,i_brand_id#37 ASC NULLS FIRST,i_class_id#38 ASC NULLS FIRST,i_category_id#39 ASC NULLS FIRST], output=[channel#49,i_brand_id#37,i_class_id#38,i_category_id#39,sum_sales#104,number_sales#105]), [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105], 100, 0, [channel#49 ASC NULLS FIRST, i_brand_id#37 ASC NULLS FIRST, i_class_id#38 ASC NULLS FIRST, i_category_id#39 ASC NULLS FIRST], [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] - -(139) CometColumnarToRow [codegen id : 406] -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#104, number_sales#105] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* HashAggregate (159) -+- * CometColumnarToRow (158) - +- CometColumnarExchange (157) - +- * HashAggregate (156) - +- Union (155) - :- * Project (144) - : +- * BroadcastHashJoin Inner BuildRight (143) - : :- * ColumnarToRow (141) - : : +- Scan parquet spark_catalog.default.store_sales (140) - : +- ReusedExchange (142) - :- * Project (149) - : +- * BroadcastHashJoin Inner BuildRight (148) - : :- * ColumnarToRow (146) - : : +- Scan parquet spark_catalog.default.catalog_sales (145) - : +- ReusedExchange (147) - +- * Project (154) - +- * BroadcastHashJoin Inner BuildRight (153) - :- * ColumnarToRow (151) - : +- Scan parquet spark_catalog.default.web_sales (150) - +- ReusedExchange (152) - - -(140) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#200, ss_list_price#201, ss_sold_date_sk#202] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#202), dynamicpruningexpression(ss_sold_date_sk#202 IN dynamicpruning#12)] -ReadSchema: struct - -(141) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#200, ss_list_price#201, ss_sold_date_sk#202] - -(142) ReusedExchange [Reuses operator id: 174] -Output [1]: [d_date_sk#203] - -(143) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#202] -Right keys [1]: [d_date_sk#203] -Join type: Inner -Join condition: None - -(144) Project [codegen id : 2] -Output [2]: [ss_quantity#200 AS quantity#204, ss_list_price#201 AS list_price#205] -Input [4]: [ss_quantity#200, ss_list_price#201, ss_sold_date_sk#202, d_date_sk#203] - -(145) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#206, cs_list_price#207, cs_sold_date_sk#208] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#208), dynamicpruningexpression(cs_sold_date_sk#208 IN dynamicpruning#209)] -ReadSchema: struct - -(146) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#206, cs_list_price#207, cs_sold_date_sk#208] - -(147) ReusedExchange [Reuses operator id: 164] -Output [1]: [d_date_sk#210] - -(148) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#208] -Right keys [1]: [d_date_sk#210] -Join type: Inner -Join condition: None - -(149) Project [codegen id : 4] -Output [2]: [cs_quantity#206 AS quantity#211, cs_list_price#207 AS list_price#212] -Input [4]: [cs_quantity#206, cs_list_price#207, cs_sold_date_sk#208, d_date_sk#210] - -(150) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#213, ws_list_price#214, ws_sold_date_sk#215] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#215), dynamicpruningexpression(ws_sold_date_sk#215 IN dynamicpruning#209)] -ReadSchema: struct - -(151) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#213, ws_list_price#214, ws_sold_date_sk#215] - -(152) ReusedExchange [Reuses operator id: 164] -Output [1]: [d_date_sk#216] - -(153) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#215] -Right keys [1]: [d_date_sk#216] -Join type: Inner -Join condition: None - -(154) Project [codegen id : 6] -Output [2]: [ws_quantity#213 AS quantity#217, ws_list_price#214 AS list_price#218] -Input [4]: [ws_quantity#213, ws_list_price#214, ws_sold_date_sk#215, d_date_sk#216] - -(155) Union - -(156) HashAggregate [codegen id : 7] -Input [2]: [quantity#204, list_price#205] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#204 as decimal(10,0)) * list_price#205))] -Aggregate Attributes [2]: [sum#219, count#220] -Results [2]: [sum#221, count#222] - -(157) CometColumnarExchange -Input [2]: [sum#221, count#222] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] - -(158) CometColumnarToRow [codegen id : 8] -Input [2]: [sum#221, count#222] - -(159) HashAggregate [codegen id : 8] -Input [2]: [sum#221, count#222] -Keys: [] -Functions [1]: [avg((cast(quantity#204 as decimal(10,0)) * list_price#205))] -Aggregate Attributes [1]: [avg((cast(quantity#204 as decimal(10,0)) * list_price#205))#223] -Results [1]: [avg((cast(quantity#204 as decimal(10,0)) * list_price#205))#223 AS average_sales#224] - -Subquery:2 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#202 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 145 Hosting Expression = cs_sold_date_sk#208 IN dynamicpruning#209 -BroadcastExchange (164) -+- * CometColumnarToRow (163) - +- CometProject (162) - +- CometFilter (161) - +- CometNativeScan parquet spark_catalog.default.date_dim (160) - - -(160) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#210, d_year#225] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(161) CometFilter -Input [2]: [d_date_sk#210, d_year#225] -Condition : (((isnotnull(d_year#225) AND (d_year#225 >= 1998)) AND (d_year#225 <= 2000)) AND isnotnull(d_date_sk#210)) - -(162) CometProject -Input [2]: [d_date_sk#210, d_year#225] -Arguments: [d_date_sk#210], [d_date_sk#210] - -(163) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#210] - -(164) BroadcastExchange -Input [1]: [d_date_sk#210] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] - -Subquery:4 Hosting operator id = 150 Hosting Expression = ws_sold_date_sk#215 IN dynamicpruning#209 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (169) -+- * CometColumnarToRow (168) - +- CometProject (167) - +- CometFilter (166) - +- CometNativeScan parquet spark_catalog.default.date_dim (165) - - -(165) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#40, d_year#226, d_moy#227] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(166) CometFilter -Input [3]: [d_date_sk#40, d_year#226, d_moy#227] -Condition : ((((isnotnull(d_year#226) AND isnotnull(d_moy#227)) AND (d_year#226 = 2000)) AND (d_moy#227 = 11)) AND isnotnull(d_date_sk#40)) - -(167) CometProject -Input [3]: [d_date_sk#40, d_year#226, d_moy#227] -Arguments: [d_date_sk#40], [d_date_sk#40] - -(168) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#40] - -(169) BroadcastExchange -Input [1]: [d_date_sk#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] - -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (174) -+- * CometColumnarToRow (173) - +- CometProject (172) - +- CometFilter (171) - +- CometNativeScan parquet spark_catalog.default.date_dim (170) - - -(170) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#228] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(171) CometFilter -Input [2]: [d_date_sk#24, d_year#228] -Condition : (((isnotnull(d_year#228) AND (d_year#228 >= 1999)) AND (d_year#228 <= 2001)) AND isnotnull(d_date_sk#24)) - -(172) CometProject -Input [2]: [d_date_sk#24, d_year#228] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(173) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#24] - -(174) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21] - -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:8 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:10 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 - -Subquery:11 Hosting operator id = 100 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:12 Hosting operator id = 85 Hosting Expression = ws_sold_date_sk#78 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/simplified.txt deleted file mode 100644 index 4458723393..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/simplified.txt +++ /dev/null @@ -1,276 +0,0 @@ -WholeStageCodegen (406) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - WholeStageCodegen (405) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - InputAdapter - Union - WholeStageCodegen (80) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (79) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (26) - Filter [sales] - Subquery #3 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #14 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #8 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #15 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #15 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #15 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (52) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (51) - HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (78) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (77) - HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (161) - HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id,i_class_id] #18 - WholeStageCodegen (160) - HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (242) - HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id] #19 - WholeStageCodegen (241) - HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (323) - HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #20 - WholeStageCodegen (322) - HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (404) - HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #21 - WholeStageCodegen (403) - HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/explain.txt deleted file mode 100644 index 22106459a6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,914 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (122) -+- CometTakeOrderedAndProject (121) - +- CometHashAggregate (120) - +- CometExchange (119) - +- CometHashAggregate (118) - +- CometUnion (117) - :- CometHashAggregate (96) - : +- CometExchange (95) - : +- CometHashAggregate (94) - : +- CometUnion (93) - : :- CometFilter (64) - : : +- CometHashAggregate (63) - : : +- CometExchange (62) - : : +- CometHashAggregate (61) - : : +- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometBroadcastHashJoin (47) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (46) - : : : : +- CometProject (45) - : : : : +- CometBroadcastHashJoin (44) - : : : : :- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : : +- CometBroadcastExchange (43) - : : : : +- CometBroadcastHashJoin (42) - : : : : :- CometHashAggregate (32) - : : : : : +- CometExchange (31) - : : : : : +- CometHashAggregate (30) - : : : : : +- CometProject (29) - : : : : : +- CometBroadcastHashJoin (28) - : : : : : :- CometProject (26) - : : : : : : +- CometBroadcastHashJoin (25) - : : : : : : :- CometFilter (6) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : : +- CometBroadcastExchange (24) - : : : : : : +- CometBroadcastHashJoin (23) - : : : : : : :- CometFilter (8) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : : +- CometBroadcastExchange (22) - : : : : : : +- CometProject (21) - : : : : : : +- CometBroadcastHashJoin (20) - : : : : : : :- CometProject (15) - : : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : : :- CometFilter (10) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : : +- CometBroadcastExchange (13) - : : : : : : : +- CometFilter (12) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : : +- CometBroadcastExchange (19) - : : : : : : +- CometProject (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : : +- ReusedExchange (27) - : : : : +- CometBroadcastExchange (41) - : : : : +- CometProject (40) - : : : : +- CometBroadcastHashJoin (39) - : : : : :- CometProject (37) - : : : : : +- CometBroadcastHashJoin (36) - : : : : : :- CometFilter (34) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : : +- ReusedExchange (35) - : : : : +- ReusedExchange (38) - : : : +- CometBroadcastExchange (52) - : : : +- CometBroadcastHashJoin (51) - : : : :- CometFilter (49) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : : +- ReusedExchange (50) - : : +- CometBroadcastExchange (58) - : : +- CometProject (57) - : : +- CometFilter (56) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - : :- CometFilter (78) - : : +- CometHashAggregate (77) - : : +- CometExchange (76) - : : +- CometHashAggregate (75) - : : +- CometProject (74) - : : +- CometBroadcastHashJoin (73) - : : :- CometProject (71) - : : : +- CometBroadcastHashJoin (70) - : : : :- CometBroadcastHashJoin (68) - : : : : :- CometFilter (66) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (65) - : : : : +- ReusedExchange (67) - : : : +- ReusedExchange (69) - : : +- ReusedExchange (72) - : +- CometFilter (92) - : +- CometHashAggregate (91) - : +- CometExchange (90) - : +- CometHashAggregate (89) - : +- CometProject (88) - : +- CometBroadcastHashJoin (87) - : :- CometProject (85) - : : +- CometBroadcastHashJoin (84) - : : :- CometBroadcastHashJoin (82) - : : : :- CometFilter (80) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (79) - : : : +- ReusedExchange (81) - : : +- ReusedExchange (83) - : +- ReusedExchange (86) - :- CometHashAggregate (101) - : +- CometExchange (100) - : +- CometHashAggregate (99) - : +- CometHashAggregate (98) - : +- ReusedExchange (97) - :- CometHashAggregate (106) - : +- CometExchange (105) - : +- CometHashAggregate (104) - : +- CometHashAggregate (103) - : +- ReusedExchange (102) - :- CometHashAggregate (111) - : +- CometExchange (110) - : +- CometHashAggregate (109) - : +- CometHashAggregate (108) - : +- ReusedExchange (107) - +- CometHashAggregate (116) - +- CometExchange (115) - +- CometHashAggregate (114) - +- CometHashAggregate (113) - +- ReusedExchange (112) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : isnotnull(i_item_sk#39) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_sold_date_sk#57 IN dynamicpruning#58)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Condition : isnotnull(cs_item_sk#54) - -(67) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#59] - -(68) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Right output [1]: [ss_item_sk#59] -Arguments: [cs_item_sk#54], [ss_item_sk#59], LeftSemi, BuildRight - -(69) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] - -(70) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Right output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -Arguments: [cs_item_sk#54], [i_item_sk#60], Inner, BuildRight - -(71) CometProject -Input [8]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -Arguments: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63], [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] - -(72) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#64] - -(73) CometBroadcastHashJoin -Left output [6]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] -Right output [1]: [d_date_sk#64] -Arguments: [cs_sold_date_sk#57], [d_date_sk#64], Inner, BuildRight - -(74) CometProject -Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] -Arguments: [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63], [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] - -(75) CometHashAggregate -Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] - -(76) CometExchange -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#65, isEmpty#66, count#67] -Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(77) CometHashAggregate -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#65, isEmpty#66, count#67] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] - -(78) CometFilter -Input [6]: [channel#68, i_brand_id#61, i_class_id#62, i_category_id#63, sales#69, number_sales#70] -Condition : (isnotnull(sales#69) AND (cast(sales#69 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(80) CometFilter -Input [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] -Condition : isnotnull(ws_item_sk#71) - -(81) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#76] - -(82) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] -Right output [1]: [ss_item_sk#76] -Arguments: [ws_item_sk#71], [ss_item_sk#76], LeftSemi, BuildRight - -(83) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] - -(84) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] -Right output [4]: [i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] -Arguments: [ws_item_sk#71], [i_item_sk#77], Inner, BuildRight - -(85) CometProject -Input [8]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] -Arguments: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80], [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80] - -(86) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#81] - -(87) CometBroadcastHashJoin -Left output [6]: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80] -Right output [1]: [d_date_sk#81] -Arguments: [ws_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight - -(88) CometProject -Input [7]: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80, d_date_sk#81] -Arguments: [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80], [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80] - -(89) CometHashAggregate -Input [5]: [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80] -Keys [3]: [i_brand_id#78, i_class_id#79, i_category_id#80] -Functions [2]: [partial_sum((cast(ws_quantity#72 as decimal(10,0)) * ws_list_price#73)), partial_count(1)] - -(90) CometExchange -Input [6]: [i_brand_id#78, i_class_id#79, i_category_id#80, sum#82, isEmpty#83, count#84] -Arguments: hashpartitioning(i_brand_id#78, i_class_id#79, i_category_id#80, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(91) CometHashAggregate -Input [6]: [i_brand_id#78, i_class_id#79, i_category_id#80, sum#82, isEmpty#83, count#84] -Keys [3]: [i_brand_id#78, i_class_id#79, i_category_id#80] -Functions [2]: [sum((cast(ws_quantity#72 as decimal(10,0)) * ws_list_price#73)), count(1)] - -(92) CometFilter -Input [6]: [channel#85, i_brand_id#78, i_class_id#79, i_category_id#80, sales#86, number_sales#87] -Condition : (isnotnull(sales#86) AND (cast(sales#86 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(93) CometUnion -Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Child 1 Input [6]: [channel#68, i_brand_id#61, i_class_id#62, i_category_id#63, sales#69, number_sales#70] -Child 2 Input [6]: [channel#85, i_brand_id#78, i_class_id#79, i_category_id#80, sales#86, number_sales#87] - -(94) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] - -(95) CometExchange -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#88, isEmpty#89, sum#90] -Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(96) CometHashAggregate -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#88, isEmpty#89, sum#90] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(97) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum#95, isEmpty#96, sum#97] - -(98) CometHashAggregate -Input [7]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum#95, isEmpty#96, sum#97] -Keys [4]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94] -Functions [2]: [sum(sales#98), sum(number_sales#99)] - -(99) CometHashAggregate -Input [5]: [channel#91, i_brand_id#92, i_class_id#93, sum_sales#100, number_sales#101] -Keys [3]: [channel#91, i_brand_id#92, i_class_id#93] -Functions [2]: [partial_sum(sum_sales#100), partial_sum(number_sales#101)] - -(100) CometExchange -Input [6]: [channel#91, i_brand_id#92, i_class_id#93, sum#102, isEmpty#103, sum#104] -Arguments: hashpartitioning(channel#91, i_brand_id#92, i_class_id#93, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(101) CometHashAggregate -Input [6]: [channel#91, i_brand_id#92, i_class_id#93, sum#102, isEmpty#103, sum#104] -Keys [3]: [channel#91, i_brand_id#92, i_class_id#93] -Functions [2]: [sum(sum_sales#100), sum(number_sales#101)] - -(102) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, sum#109, isEmpty#110, sum#111] - -(103) CometHashAggregate -Input [7]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, sum#109, isEmpty#110, sum#111] -Keys [4]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108] -Functions [2]: [sum(sales#112), sum(number_sales#113)] - -(104) CometHashAggregate -Input [4]: [channel#105, i_brand_id#106, sum_sales#114, number_sales#115] -Keys [2]: [channel#105, i_brand_id#106] -Functions [2]: [partial_sum(sum_sales#114), partial_sum(number_sales#115)] - -(105) CometExchange -Input [5]: [channel#105, i_brand_id#106, sum#116, isEmpty#117, sum#118] -Arguments: hashpartitioning(channel#105, i_brand_id#106, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(106) CometHashAggregate -Input [5]: [channel#105, i_brand_id#106, sum#116, isEmpty#117, sum#118] -Keys [2]: [channel#105, i_brand_id#106] -Functions [2]: [sum(sum_sales#114), sum(number_sales#115)] - -(107) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#119, i_brand_id#120, i_class_id#121, i_category_id#122, sum#123, isEmpty#124, sum#125] - -(108) CometHashAggregate -Input [7]: [channel#119, i_brand_id#120, i_class_id#121, i_category_id#122, sum#123, isEmpty#124, sum#125] -Keys [4]: [channel#119, i_brand_id#120, i_class_id#121, i_category_id#122] -Functions [2]: [sum(sales#126), sum(number_sales#127)] - -(109) CometHashAggregate -Input [3]: [channel#119, sum_sales#128, number_sales#129] -Keys [1]: [channel#119] -Functions [2]: [partial_sum(sum_sales#128), partial_sum(number_sales#129)] - -(110) CometExchange -Input [4]: [channel#119, sum#130, isEmpty#131, sum#132] -Arguments: hashpartitioning(channel#119, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(111) CometHashAggregate -Input [4]: [channel#119, sum#130, isEmpty#131, sum#132] -Keys [1]: [channel#119] -Functions [2]: [sum(sum_sales#128), sum(number_sales#129)] - -(112) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#133, i_brand_id#134, i_class_id#135, i_category_id#136, sum#137, isEmpty#138, sum#139] - -(113) CometHashAggregate -Input [7]: [channel#133, i_brand_id#134, i_class_id#135, i_category_id#136, sum#137, isEmpty#138, sum#139] -Keys [4]: [channel#133, i_brand_id#134, i_class_id#135, i_category_id#136] -Functions [2]: [sum(sales#140), sum(number_sales#141)] - -(114) CometHashAggregate -Input [2]: [sum_sales#142, number_sales#143] -Keys: [] -Functions [2]: [partial_sum(sum_sales#142), partial_sum(number_sales#143)] - -(115) CometExchange -Input [3]: [sum#144, isEmpty#145, sum#146] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(116) CometHashAggregate -Input [3]: [sum#144, isEmpty#145, sum#146] -Keys: [] -Functions [2]: [sum(sum_sales#142), sum(number_sales#143)] - -(117) CometUnion -Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#147, number_sales#148] -Child 1 Input [6]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#149, sum(sum_sales)#150, sum(number_sales)#151] -Child 2 Input [6]: [channel#105, i_brand_id#106, i_class_id#152, i_category_id#153, sum(sum_sales)#154, sum(number_sales)#155] -Child 3 Input [6]: [channel#119, i_brand_id#156, i_class_id#157, i_category_id#158, sum(sum_sales)#159, sum(number_sales)#160] -Child 4 Input [6]: [channel#161, i_brand_id#162, i_class_id#163, i_category_id#164, sum(sum_sales)#165, sum(number_sales)#166] - -(118) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#147, number_sales#148] -Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#147, number_sales#148] -Functions: [] - -(119) CometExchange -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#147, number_sales#148] -Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#147, number_sales#148, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(120) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#147, number_sales#148] -Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#147, number_sales#148] -Functions: [] - -(121) CometTakeOrderedAndProject -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#147, number_sales#148] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#49 ASC NULLS FIRST,i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#49,i_brand_id#40,i_class_id#41,i_category_id#42,sum_sales#147,number_sales#148]), [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#147, number_sales#148], 100, 0, [channel#49 ASC NULLS FIRST, i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#147, number_sales#148] - -(122) CometColumnarToRow [codegen id : 1] -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#147, number_sales#148] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* CometColumnarToRow (142) -+- CometHashAggregate (141) - +- CometExchange (140) - +- CometHashAggregate (139) - +- CometUnion (138) - :- CometProject (126) - : +- CometBroadcastHashJoin (125) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (123) - : +- ReusedExchange (124) - :- CometProject (133) - : +- CometBroadcastHashJoin (132) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (127) - : +- CometBroadcastExchange (131) - : +- CometProject (130) - : +- CometFilter (129) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (128) - +- CometProject (137) - +- CometBroadcastHashJoin (136) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (134) - +- ReusedExchange (135) - - -(123) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#167, ss_list_price#168, ss_sold_date_sk#169] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#169), dynamicpruningexpression(ss_sold_date_sk#169 IN dynamicpruning#170)] -ReadSchema: struct - -(124) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#171] - -(125) CometBroadcastHashJoin -Left output [3]: [ss_quantity#167, ss_list_price#168, ss_sold_date_sk#169] -Right output [1]: [d_date_sk#171] -Arguments: [ss_sold_date_sk#169], [d_date_sk#171], Inner, BuildRight - -(126) CometProject -Input [4]: [ss_quantity#167, ss_list_price#168, ss_sold_date_sk#169, d_date_sk#171] -Arguments: [quantity#172, list_price#173], [ss_quantity#167 AS quantity#172, ss_list_price#168 AS list_price#173] - -(127) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#174, cs_list_price#175, cs_sold_date_sk#176] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#176), dynamicpruningexpression(cs_sold_date_sk#176 IN dynamicpruning#177)] -ReadSchema: struct - -(128) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#178, d_year#179] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(129) CometFilter -Input [2]: [d_date_sk#178, d_year#179] -Condition : (((isnotnull(d_year#179) AND (d_year#179 >= 1998)) AND (d_year#179 <= 2000)) AND isnotnull(d_date_sk#178)) - -(130) CometProject -Input [2]: [d_date_sk#178, d_year#179] -Arguments: [d_date_sk#178], [d_date_sk#178] - -(131) CometBroadcastExchange -Input [1]: [d_date_sk#178] -Arguments: [d_date_sk#178] - -(132) CometBroadcastHashJoin -Left output [3]: [cs_quantity#174, cs_list_price#175, cs_sold_date_sk#176] -Right output [1]: [d_date_sk#178] -Arguments: [cs_sold_date_sk#176], [d_date_sk#178], Inner, BuildRight - -(133) CometProject -Input [4]: [cs_quantity#174, cs_list_price#175, cs_sold_date_sk#176, d_date_sk#178] -Arguments: [quantity#180, list_price#181], [cs_quantity#174 AS quantity#180, cs_list_price#175 AS list_price#181] - -(134) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#182, ws_list_price#183, ws_sold_date_sk#184] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#184), dynamicpruningexpression(ws_sold_date_sk#184 IN dynamicpruning#185)] -ReadSchema: struct - -(135) ReusedExchange [Reuses operator id: 131] -Output [1]: [d_date_sk#186] - -(136) CometBroadcastHashJoin -Left output [3]: [ws_quantity#182, ws_list_price#183, ws_sold_date_sk#184] -Right output [1]: [d_date_sk#186] -Arguments: [ws_sold_date_sk#184], [d_date_sk#186], Inner, BuildRight - -(137) CometProject -Input [4]: [ws_quantity#182, ws_list_price#183, ws_sold_date_sk#184, d_date_sk#186] -Arguments: [quantity#187, list_price#188], [ws_quantity#182 AS quantity#187, ws_list_price#183 AS list_price#188] - -(138) CometUnion -Child 0 Input [2]: [quantity#172, list_price#173] -Child 1 Input [2]: [quantity#180, list_price#181] -Child 2 Input [2]: [quantity#187, list_price#188] - -(139) CometHashAggregate -Input [2]: [quantity#172, list_price#173] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#172 as decimal(10,0)) * list_price#173))] - -(140) CometExchange -Input [2]: [sum#189, count#190] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(141) CometHashAggregate -Input [2]: [sum#189, count#190] -Keys: [] -Functions [1]: [avg((cast(quantity#172 as decimal(10,0)) * list_price#173))] - -(142) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#191] - -Subquery:2 Hosting operator id = 123 Hosting Expression = ss_sold_date_sk#169 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 127 Hosting Expression = cs_sold_date_sk#176 IN dynamicpruning#177 -BroadcastExchange (147) -+- * CometColumnarToRow (146) - +- CometProject (145) - +- CometFilter (144) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (143) - - -(143) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#178, d_year#179] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(144) CometFilter -Input [2]: [d_date_sk#178, d_year#179] -Condition : (((isnotnull(d_year#179) AND (d_year#179 >= 1998)) AND (d_year#179 <= 2000)) AND isnotnull(d_date_sk#178)) - -(145) CometProject -Input [2]: [d_date_sk#178, d_year#179] -Arguments: [d_date_sk#178], [d_date_sk#178] - -(146) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#178] - -(147) BroadcastExchange -Input [1]: [d_date_sk#178] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#184 IN dynamicpruning#177 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (152) -+- * CometColumnarToRow (151) - +- CometProject (150) - +- CometFilter (149) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (148) - - -(148) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(149) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(150) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(151) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(152) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (157) -+- * CometColumnarToRow (156) - +- CometProject (155) - +- CometFilter (154) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (153) - - -(153) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#192] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(154) CometFilter -Input [2]: [d_date_sk#26, d_year#192] -Condition : (((isnotnull(d_year#192) AND (d_year#192 >= 1999)) AND (d_year#192 <= 2001)) AND isnotnull(d_date_sk#26)) - -(155) CometProject -Input [2]: [d_date_sk#26, d_year#192] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(156) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(157) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 78 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:10 Hosting operator id = 65 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 - -Subquery:11 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:12 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 758e917335..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,179 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometUnion [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - CometExchange [channel,i_brand_id,i_class_id,i_category_id] #2 - CometHashAggregate [sales,number_sales] [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] - CometUnion [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #16 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #18 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 - ReusedExchange [d_date_sk] #18 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #3 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk] #5 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #6 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #7 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #12 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #12 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #13 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #12 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #5 - CometBroadcastExchange [d_date_sk] #15 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #19 - CometHashAggregate [cs_quantity,cs_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk] - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #5 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - ReusedExchange [d_date_sk] #15 - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #20 - CometHashAggregate [ws_quantity,ws_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk] - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #5 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - ReusedExchange [d_date_sk] #15 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel,i_brand_id,i_class_id] #21 - CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,i_class_id,sum,isEmpty,sum] - CometHashAggregate [i_category_id,sum,isEmpty,sum] [channel,i_brand_id,i_class_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel,i_brand_id] #22 - CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,sum,isEmpty,sum] - CometHashAggregate [i_class_id,i_category_id,sum,isEmpty,sum] [channel,i_brand_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel] #23 - CometHashAggregate [sum_sales,number_sales] [channel,sum,isEmpty,sum] - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [channel,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange #24 - CometHashAggregate [sum_sales,number_sales] [sum,isEmpty,sum] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/explain.txt deleted file mode 100644 index 22106459a6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/explain.txt +++ /dev/null @@ -1,914 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (122) -+- CometTakeOrderedAndProject (121) - +- CometHashAggregate (120) - +- CometExchange (119) - +- CometHashAggregate (118) - +- CometUnion (117) - :- CometHashAggregate (96) - : +- CometExchange (95) - : +- CometHashAggregate (94) - : +- CometUnion (93) - : :- CometFilter (64) - : : +- CometHashAggregate (63) - : : +- CometExchange (62) - : : +- CometHashAggregate (61) - : : +- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometBroadcastHashJoin (47) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (46) - : : : : +- CometProject (45) - : : : : +- CometBroadcastHashJoin (44) - : : : : :- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : : +- CometBroadcastExchange (43) - : : : : +- CometBroadcastHashJoin (42) - : : : : :- CometHashAggregate (32) - : : : : : +- CometExchange (31) - : : : : : +- CometHashAggregate (30) - : : : : : +- CometProject (29) - : : : : : +- CometBroadcastHashJoin (28) - : : : : : :- CometProject (26) - : : : : : : +- CometBroadcastHashJoin (25) - : : : : : : :- CometFilter (6) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : : +- CometBroadcastExchange (24) - : : : : : : +- CometBroadcastHashJoin (23) - : : : : : : :- CometFilter (8) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : : +- CometBroadcastExchange (22) - : : : : : : +- CometProject (21) - : : : : : : +- CometBroadcastHashJoin (20) - : : : : : : :- CometProject (15) - : : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : : :- CometFilter (10) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : : +- CometBroadcastExchange (13) - : : : : : : : +- CometFilter (12) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : : +- CometBroadcastExchange (19) - : : : : : : +- CometProject (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : : +- ReusedExchange (27) - : : : : +- CometBroadcastExchange (41) - : : : : +- CometProject (40) - : : : : +- CometBroadcastHashJoin (39) - : : : : :- CometProject (37) - : : : : : +- CometBroadcastHashJoin (36) - : : : : : :- CometFilter (34) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : : +- ReusedExchange (35) - : : : : +- ReusedExchange (38) - : : : +- CometBroadcastExchange (52) - : : : +- CometBroadcastHashJoin (51) - : : : :- CometFilter (49) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : : +- ReusedExchange (50) - : : +- CometBroadcastExchange (58) - : : +- CometProject (57) - : : +- CometFilter (56) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - : :- CometFilter (78) - : : +- CometHashAggregate (77) - : : +- CometExchange (76) - : : +- CometHashAggregate (75) - : : +- CometProject (74) - : : +- CometBroadcastHashJoin (73) - : : :- CometProject (71) - : : : +- CometBroadcastHashJoin (70) - : : : :- CometBroadcastHashJoin (68) - : : : : :- CometFilter (66) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (65) - : : : : +- ReusedExchange (67) - : : : +- ReusedExchange (69) - : : +- ReusedExchange (72) - : +- CometFilter (92) - : +- CometHashAggregate (91) - : +- CometExchange (90) - : +- CometHashAggregate (89) - : +- CometProject (88) - : +- CometBroadcastHashJoin (87) - : :- CometProject (85) - : : +- CometBroadcastHashJoin (84) - : : :- CometBroadcastHashJoin (82) - : : : :- CometFilter (80) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (79) - : : : +- ReusedExchange (81) - : : +- ReusedExchange (83) - : +- ReusedExchange (86) - :- CometHashAggregate (101) - : +- CometExchange (100) - : +- CometHashAggregate (99) - : +- CometHashAggregate (98) - : +- ReusedExchange (97) - :- CometHashAggregate (106) - : +- CometExchange (105) - : +- CometHashAggregate (104) - : +- CometHashAggregate (103) - : +- ReusedExchange (102) - :- CometHashAggregate (111) - : +- CometExchange (110) - : +- CometHashAggregate (109) - : +- CometHashAggregate (108) - : +- ReusedExchange (107) - +- CometHashAggregate (116) - +- CometExchange (115) - +- CometHashAggregate (114) - +- CometHashAggregate (113) - +- ReusedExchange (112) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : isnotnull(i_item_sk#39) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_sold_date_sk#57 IN dynamicpruning#58)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Condition : isnotnull(cs_item_sk#54) - -(67) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#59] - -(68) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Right output [1]: [ss_item_sk#59] -Arguments: [cs_item_sk#54], [ss_item_sk#59], LeftSemi, BuildRight - -(69) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] - -(70) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Right output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -Arguments: [cs_item_sk#54], [i_item_sk#60], Inner, BuildRight - -(71) CometProject -Input [8]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -Arguments: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63], [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] - -(72) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#64] - -(73) CometBroadcastHashJoin -Left output [6]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63] -Right output [1]: [d_date_sk#64] -Arguments: [cs_sold_date_sk#57], [d_date_sk#64], Inner, BuildRight - -(74) CometProject -Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] -Arguments: [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63], [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] - -(75) CometHashAggregate -Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] - -(76) CometExchange -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#65, isEmpty#66, count#67] -Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(77) CometHashAggregate -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#65, isEmpty#66, count#67] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] - -(78) CometFilter -Input [6]: [channel#68, i_brand_id#61, i_class_id#62, i_category_id#63, sales#69, number_sales#70] -Condition : (isnotnull(sales#69) AND (cast(sales#69 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(80) CometFilter -Input [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] -Condition : isnotnull(ws_item_sk#71) - -(81) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#76] - -(82) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] -Right output [1]: [ss_item_sk#76] -Arguments: [ws_item_sk#71], [ss_item_sk#76], LeftSemi, BuildRight - -(83) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] - -(84) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74] -Right output [4]: [i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] -Arguments: [ws_item_sk#71], [i_item_sk#77], Inner, BuildRight - -(85) CometProject -Input [8]: [ws_item_sk#71, ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_item_sk#77, i_brand_id#78, i_class_id#79, i_category_id#80] -Arguments: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80], [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80] - -(86) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#81] - -(87) CometBroadcastHashJoin -Left output [6]: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80] -Right output [1]: [d_date_sk#81] -Arguments: [ws_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight - -(88) CometProject -Input [7]: [ws_quantity#72, ws_list_price#73, ws_sold_date_sk#74, i_brand_id#78, i_class_id#79, i_category_id#80, d_date_sk#81] -Arguments: [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80], [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80] - -(89) CometHashAggregate -Input [5]: [ws_quantity#72, ws_list_price#73, i_brand_id#78, i_class_id#79, i_category_id#80] -Keys [3]: [i_brand_id#78, i_class_id#79, i_category_id#80] -Functions [2]: [partial_sum((cast(ws_quantity#72 as decimal(10,0)) * ws_list_price#73)), partial_count(1)] - -(90) CometExchange -Input [6]: [i_brand_id#78, i_class_id#79, i_category_id#80, sum#82, isEmpty#83, count#84] -Arguments: hashpartitioning(i_brand_id#78, i_class_id#79, i_category_id#80, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(91) CometHashAggregate -Input [6]: [i_brand_id#78, i_class_id#79, i_category_id#80, sum#82, isEmpty#83, count#84] -Keys [3]: [i_brand_id#78, i_class_id#79, i_category_id#80] -Functions [2]: [sum((cast(ws_quantity#72 as decimal(10,0)) * ws_list_price#73)), count(1)] - -(92) CometFilter -Input [6]: [channel#85, i_brand_id#78, i_class_id#79, i_category_id#80, sales#86, number_sales#87] -Condition : (isnotnull(sales#86) AND (cast(sales#86 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(93) CometUnion -Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Child 1 Input [6]: [channel#68, i_brand_id#61, i_class_id#62, i_category_id#63, sales#69, number_sales#70] -Child 2 Input [6]: [channel#85, i_brand_id#78, i_class_id#79, i_category_id#80, sales#86, number_sales#87] - -(94) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] - -(95) CometExchange -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#88, isEmpty#89, sum#90] -Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(96) CometHashAggregate -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#88, isEmpty#89, sum#90] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(97) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum#95, isEmpty#96, sum#97] - -(98) CometHashAggregate -Input [7]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94, sum#95, isEmpty#96, sum#97] -Keys [4]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#94] -Functions [2]: [sum(sales#98), sum(number_sales#99)] - -(99) CometHashAggregate -Input [5]: [channel#91, i_brand_id#92, i_class_id#93, sum_sales#100, number_sales#101] -Keys [3]: [channel#91, i_brand_id#92, i_class_id#93] -Functions [2]: [partial_sum(sum_sales#100), partial_sum(number_sales#101)] - -(100) CometExchange -Input [6]: [channel#91, i_brand_id#92, i_class_id#93, sum#102, isEmpty#103, sum#104] -Arguments: hashpartitioning(channel#91, i_brand_id#92, i_class_id#93, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(101) CometHashAggregate -Input [6]: [channel#91, i_brand_id#92, i_class_id#93, sum#102, isEmpty#103, sum#104] -Keys [3]: [channel#91, i_brand_id#92, i_class_id#93] -Functions [2]: [sum(sum_sales#100), sum(number_sales#101)] - -(102) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, sum#109, isEmpty#110, sum#111] - -(103) CometHashAggregate -Input [7]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, sum#109, isEmpty#110, sum#111] -Keys [4]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108] -Functions [2]: [sum(sales#112), sum(number_sales#113)] - -(104) CometHashAggregate -Input [4]: [channel#105, i_brand_id#106, sum_sales#114, number_sales#115] -Keys [2]: [channel#105, i_brand_id#106] -Functions [2]: [partial_sum(sum_sales#114), partial_sum(number_sales#115)] - -(105) CometExchange -Input [5]: [channel#105, i_brand_id#106, sum#116, isEmpty#117, sum#118] -Arguments: hashpartitioning(channel#105, i_brand_id#106, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(106) CometHashAggregate -Input [5]: [channel#105, i_brand_id#106, sum#116, isEmpty#117, sum#118] -Keys [2]: [channel#105, i_brand_id#106] -Functions [2]: [sum(sum_sales#114), sum(number_sales#115)] - -(107) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#119, i_brand_id#120, i_class_id#121, i_category_id#122, sum#123, isEmpty#124, sum#125] - -(108) CometHashAggregate -Input [7]: [channel#119, i_brand_id#120, i_class_id#121, i_category_id#122, sum#123, isEmpty#124, sum#125] -Keys [4]: [channel#119, i_brand_id#120, i_class_id#121, i_category_id#122] -Functions [2]: [sum(sales#126), sum(number_sales#127)] - -(109) CometHashAggregate -Input [3]: [channel#119, sum_sales#128, number_sales#129] -Keys [1]: [channel#119] -Functions [2]: [partial_sum(sum_sales#128), partial_sum(number_sales#129)] - -(110) CometExchange -Input [4]: [channel#119, sum#130, isEmpty#131, sum#132] -Arguments: hashpartitioning(channel#119, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(111) CometHashAggregate -Input [4]: [channel#119, sum#130, isEmpty#131, sum#132] -Keys [1]: [channel#119] -Functions [2]: [sum(sum_sales#128), sum(number_sales#129)] - -(112) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#133, i_brand_id#134, i_class_id#135, i_category_id#136, sum#137, isEmpty#138, sum#139] - -(113) CometHashAggregate -Input [7]: [channel#133, i_brand_id#134, i_class_id#135, i_category_id#136, sum#137, isEmpty#138, sum#139] -Keys [4]: [channel#133, i_brand_id#134, i_class_id#135, i_category_id#136] -Functions [2]: [sum(sales#140), sum(number_sales#141)] - -(114) CometHashAggregate -Input [2]: [sum_sales#142, number_sales#143] -Keys: [] -Functions [2]: [partial_sum(sum_sales#142), partial_sum(number_sales#143)] - -(115) CometExchange -Input [3]: [sum#144, isEmpty#145, sum#146] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(116) CometHashAggregate -Input [3]: [sum#144, isEmpty#145, sum#146] -Keys: [] -Functions [2]: [sum(sum_sales#142), sum(number_sales#143)] - -(117) CometUnion -Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#147, number_sales#148] -Child 1 Input [6]: [channel#91, i_brand_id#92, i_class_id#93, i_category_id#149, sum(sum_sales)#150, sum(number_sales)#151] -Child 2 Input [6]: [channel#105, i_brand_id#106, i_class_id#152, i_category_id#153, sum(sum_sales)#154, sum(number_sales)#155] -Child 3 Input [6]: [channel#119, i_brand_id#156, i_class_id#157, i_category_id#158, sum(sum_sales)#159, sum(number_sales)#160] -Child 4 Input [6]: [channel#161, i_brand_id#162, i_class_id#163, i_category_id#164, sum(sum_sales)#165, sum(number_sales)#166] - -(118) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#147, number_sales#148] -Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#147, number_sales#148] -Functions: [] - -(119) CometExchange -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#147, number_sales#148] -Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#147, number_sales#148, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(120) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#147, number_sales#148] -Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#147, number_sales#148] -Functions: [] - -(121) CometTakeOrderedAndProject -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#147, number_sales#148] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#49 ASC NULLS FIRST,i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#49,i_brand_id#40,i_class_id#41,i_category_id#42,sum_sales#147,number_sales#148]), [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#147, number_sales#148], 100, 0, [channel#49 ASC NULLS FIRST, i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#147, number_sales#148] - -(122) CometColumnarToRow [codegen id : 1] -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#147, number_sales#148] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* CometColumnarToRow (142) -+- CometHashAggregate (141) - +- CometExchange (140) - +- CometHashAggregate (139) - +- CometUnion (138) - :- CometProject (126) - : +- CometBroadcastHashJoin (125) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (123) - : +- ReusedExchange (124) - :- CometProject (133) - : +- CometBroadcastHashJoin (132) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (127) - : +- CometBroadcastExchange (131) - : +- CometProject (130) - : +- CometFilter (129) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (128) - +- CometProject (137) - +- CometBroadcastHashJoin (136) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (134) - +- ReusedExchange (135) - - -(123) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#167, ss_list_price#168, ss_sold_date_sk#169] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#169), dynamicpruningexpression(ss_sold_date_sk#169 IN dynamicpruning#170)] -ReadSchema: struct - -(124) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#171] - -(125) CometBroadcastHashJoin -Left output [3]: [ss_quantity#167, ss_list_price#168, ss_sold_date_sk#169] -Right output [1]: [d_date_sk#171] -Arguments: [ss_sold_date_sk#169], [d_date_sk#171], Inner, BuildRight - -(126) CometProject -Input [4]: [ss_quantity#167, ss_list_price#168, ss_sold_date_sk#169, d_date_sk#171] -Arguments: [quantity#172, list_price#173], [ss_quantity#167 AS quantity#172, ss_list_price#168 AS list_price#173] - -(127) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#174, cs_list_price#175, cs_sold_date_sk#176] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#176), dynamicpruningexpression(cs_sold_date_sk#176 IN dynamicpruning#177)] -ReadSchema: struct - -(128) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#178, d_year#179] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(129) CometFilter -Input [2]: [d_date_sk#178, d_year#179] -Condition : (((isnotnull(d_year#179) AND (d_year#179 >= 1998)) AND (d_year#179 <= 2000)) AND isnotnull(d_date_sk#178)) - -(130) CometProject -Input [2]: [d_date_sk#178, d_year#179] -Arguments: [d_date_sk#178], [d_date_sk#178] - -(131) CometBroadcastExchange -Input [1]: [d_date_sk#178] -Arguments: [d_date_sk#178] - -(132) CometBroadcastHashJoin -Left output [3]: [cs_quantity#174, cs_list_price#175, cs_sold_date_sk#176] -Right output [1]: [d_date_sk#178] -Arguments: [cs_sold_date_sk#176], [d_date_sk#178], Inner, BuildRight - -(133) CometProject -Input [4]: [cs_quantity#174, cs_list_price#175, cs_sold_date_sk#176, d_date_sk#178] -Arguments: [quantity#180, list_price#181], [cs_quantity#174 AS quantity#180, cs_list_price#175 AS list_price#181] - -(134) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#182, ws_list_price#183, ws_sold_date_sk#184] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#184), dynamicpruningexpression(ws_sold_date_sk#184 IN dynamicpruning#185)] -ReadSchema: struct - -(135) ReusedExchange [Reuses operator id: 131] -Output [1]: [d_date_sk#186] - -(136) CometBroadcastHashJoin -Left output [3]: [ws_quantity#182, ws_list_price#183, ws_sold_date_sk#184] -Right output [1]: [d_date_sk#186] -Arguments: [ws_sold_date_sk#184], [d_date_sk#186], Inner, BuildRight - -(137) CometProject -Input [4]: [ws_quantity#182, ws_list_price#183, ws_sold_date_sk#184, d_date_sk#186] -Arguments: [quantity#187, list_price#188], [ws_quantity#182 AS quantity#187, ws_list_price#183 AS list_price#188] - -(138) CometUnion -Child 0 Input [2]: [quantity#172, list_price#173] -Child 1 Input [2]: [quantity#180, list_price#181] -Child 2 Input [2]: [quantity#187, list_price#188] - -(139) CometHashAggregate -Input [2]: [quantity#172, list_price#173] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#172 as decimal(10,0)) * list_price#173))] - -(140) CometExchange -Input [2]: [sum#189, count#190] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(141) CometHashAggregate -Input [2]: [sum#189, count#190] -Keys: [] -Functions [1]: [avg((cast(quantity#172 as decimal(10,0)) * list_price#173))] - -(142) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#191] - -Subquery:2 Hosting operator id = 123 Hosting Expression = ss_sold_date_sk#169 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 127 Hosting Expression = cs_sold_date_sk#176 IN dynamicpruning#177 -BroadcastExchange (147) -+- * CometColumnarToRow (146) - +- CometProject (145) - +- CometFilter (144) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (143) - - -(143) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#178, d_year#179] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(144) CometFilter -Input [2]: [d_date_sk#178, d_year#179] -Condition : (((isnotnull(d_year#179) AND (d_year#179 >= 1998)) AND (d_year#179 <= 2000)) AND isnotnull(d_date_sk#178)) - -(145) CometProject -Input [2]: [d_date_sk#178, d_year#179] -Arguments: [d_date_sk#178], [d_date_sk#178] - -(146) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#178] - -(147) BroadcastExchange -Input [1]: [d_date_sk#178] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#184 IN dynamicpruning#177 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (152) -+- * CometColumnarToRow (151) - +- CometProject (150) - +- CometFilter (149) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (148) - - -(148) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(149) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(150) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(151) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(152) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (157) -+- * CometColumnarToRow (156) - +- CometProject (155) - +- CometFilter (154) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (153) - - -(153) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#192] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(154) CometFilter -Input [2]: [d_date_sk#26, d_year#192] -Condition : (((isnotnull(d_year#192) AND (d_year#192 >= 1999)) AND (d_year#192 <= 2001)) AND isnotnull(d_date_sk#26)) - -(155) CometProject -Input [2]: [d_date_sk#26, d_year#192] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(156) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(157) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 78 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:10 Hosting operator id = 65 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 - -Subquery:11 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:12 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/extended.txt deleted file mode 100644 index 83deb45432..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/extended.txt +++ /dev/null @@ -1,2350 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 2127 out of 2302 eligible operators (92%). Final plan contains 46 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 758e917335..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt +++ /dev/null @@ -1,179 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometUnion [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - CometExchange [channel,i_brand_id,i_class_id,i_category_id] #2 - CometHashAggregate [sales,number_sales] [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] - CometUnion [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #16 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #18 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 - ReusedExchange [d_date_sk] #18 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #3 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk] #5 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #6 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #7 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #12 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #12 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #13 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #12 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #5 - CometBroadcastExchange [d_date_sk] #15 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #19 - CometHashAggregate [cs_quantity,cs_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk] - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #5 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - ReusedExchange [d_date_sk] #15 - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #20 - CometHashAggregate [ws_quantity,ws_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk] - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #5 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - ReusedExchange [d_date_sk] #15 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel,i_brand_id,i_class_id] #21 - CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,i_class_id,sum,isEmpty,sum] - CometHashAggregate [i_category_id,sum,isEmpty,sum] [channel,i_brand_id,i_class_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel,i_brand_id] #22 - CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,sum,isEmpty,sum] - CometHashAggregate [i_class_id,i_category_id,sum,isEmpty,sum] [channel,i_brand_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel] #23 - CometHashAggregate [sum_sales,number_sales] [channel,sum,isEmpty,sum] - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [channel,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange #24 - CometHashAggregate [sum_sales,number_sales] [sum,isEmpty,sum] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/explain.txt deleted file mode 100644 index 765cd78ddb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/explain.txt +++ /dev/null @@ -1,944 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (161) -+- Union (160) - :- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- CometColumnarExchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (33) - : : +- * BroadcastHashJoin Inner BuildRight (32) - : : :- * Project (30) - : : : +- * BroadcastHashJoin Inner BuildRight (29) - : : : :- * Project (23) - : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : :- * Project (17) - : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * CometColumnarToRow (7) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) - : : : : : +- BroadcastExchange (15) - : : : : : +- * CometColumnarToRow (14) - : : : : : +- CometProject (13) - : : : : : +- CometFilter (12) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (11) - : : : : +- BroadcastExchange (21) - : : : : +- * CometColumnarToRow (20) - : : : : +- CometFilter (19) - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (18) - : : : +- BroadcastExchange (28) - : : : +- * CometColumnarToRow (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (24) - : : +- ReusedExchange (31) - : +- BroadcastExchange (38) - : +- * CometColumnarToRow (37) - : +- CometProject (36) - : +- CometFilter (35) - : +- CometNativeScan parquet spark_catalog.default.item (34) - :- * HashAggregate (73) - : +- * CometColumnarToRow (72) - : +- CometColumnarExchange (71) - : +- * HashAggregate (70) - : +- * Project (69) - : +- * BroadcastHashJoin Inner BuildRight (68) - : :- * Project (66) - : : +- * BroadcastHashJoin Inner BuildRight (65) - : : :- * Project (63) - : : : +- * BroadcastHashJoin Inner BuildRight (62) - : : : :- * Project (56) - : : : : +- * BroadcastHashJoin Inner BuildRight (55) - : : : : :- * Project (53) - : : : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : : : :- * Project (50) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : : : :- * Filter (47) - : : : : : : : +- * ColumnarToRow (46) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (45) - : : : : : : +- ReusedExchange (48) - : : : : : +- ReusedExchange (51) - : : : : +- ReusedExchange (54) - : : : +- BroadcastExchange (61) - : : : +- * CometColumnarToRow (60) - : : : +- CometProject (59) - : : : +- CometFilter (58) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (57) - : : +- ReusedExchange (64) - : +- ReusedExchange (67) - :- * HashAggregate (102) - : +- * CometColumnarToRow (101) - : +- CometColumnarExchange (100) - : +- * HashAggregate (99) - : +- * Project (98) - : +- * BroadcastHashJoin Inner BuildRight (97) - : :- * Project (95) - : : +- * BroadcastHashJoin Inner BuildRight (94) - : : :- * Project (92) - : : : +- * BroadcastHashJoin Inner BuildRight (91) - : : : :- * Project (85) - : : : : +- * BroadcastHashJoin Inner BuildRight (84) - : : : : :- * Project (82) - : : : : : +- * BroadcastHashJoin Inner BuildRight (81) - : : : : : :- * Project (79) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) - : : : : : : :- * Filter (76) - : : : : : : : +- * ColumnarToRow (75) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (74) - : : : : : : +- ReusedExchange (77) - : : : : : +- ReusedExchange (80) - : : : : +- ReusedExchange (83) - : : : +- BroadcastExchange (90) - : : : +- * CometColumnarToRow (89) - : : : +- CometProject (88) - : : : +- CometFilter (87) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (86) - : : +- ReusedExchange (93) - : +- ReusedExchange (96) - :- * HashAggregate (131) - : +- * CometColumnarToRow (130) - : +- CometColumnarExchange (129) - : +- * HashAggregate (128) - : +- * Project (127) - : +- * BroadcastHashJoin Inner BuildRight (126) - : :- * Project (124) - : : +- * BroadcastHashJoin Inner BuildRight (123) - : : :- * Project (121) - : : : +- * BroadcastHashJoin Inner BuildRight (120) - : : : :- * Project (114) - : : : : +- * BroadcastHashJoin Inner BuildRight (113) - : : : : :- * Project (111) - : : : : : +- * BroadcastHashJoin Inner BuildRight (110) - : : : : : :- * Project (108) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (107) - : : : : : : :- * Filter (105) - : : : : : : : +- * ColumnarToRow (104) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (103) - : : : : : : +- ReusedExchange (106) - : : : : : +- ReusedExchange (109) - : : : : +- ReusedExchange (112) - : : : +- BroadcastExchange (119) - : : : +- * CometColumnarToRow (118) - : : : +- CometProject (117) - : : : +- CometFilter (116) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (115) - : : +- ReusedExchange (122) - : +- ReusedExchange (125) - +- * HashAggregate (159) - +- * CometColumnarToRow (158) - +- CometColumnarExchange (157) - +- * HashAggregate (156) - +- * Project (155) - +- * BroadcastHashJoin Inner BuildRight (154) - :- * Project (149) - : +- * BroadcastHashJoin Inner BuildRight (148) - : :- * Project (146) - : : +- * BroadcastHashJoin Inner BuildRight (145) - : : :- * Project (143) - : : : +- * BroadcastHashJoin Inner BuildRight (142) - : : : :- * Project (140) - : : : : +- * BroadcastHashJoin Inner BuildRight (139) - : : : : :- * Project (137) - : : : : : +- * BroadcastHashJoin Inner BuildRight (136) - : : : : : :- * Filter (134) - : : : : : : +- * ColumnarToRow (133) - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (132) - : : : : : +- ReusedExchange (135) - : : : : +- ReusedExchange (138) - : : : +- ReusedExchange (141) - : : +- ReusedExchange (144) - : +- ReusedExchange (147) - +- BroadcastExchange (153) - +- * CometColumnarToRow (152) - +- CometFilter (151) - +- CometNativeScan parquet spark_catalog.default.item (150) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 7] -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] - -(3) Filter [codegen id : 7] -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) - -(4) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#12, 1)) = M)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#13, 20)) = College )) AND isnotnull(cd_demo_sk#11)) - -(6) CometProject -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] - -(7) CometColumnarToRow [codegen id : 1] -Input [2]: [cd_demo_sk#11, cd_dep_count#14] - -(8) BroadcastExchange -Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 7] -Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] - -(11) CometNativeScan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [In(c_birth_month, [1,10,12,4,5,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(12) CometFilter -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Condition : (((c_birth_month#18 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) - -(13) CometProject -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(14) CometColumnarToRow [codegen id : 2] -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(15) BroadcastExchange -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(18) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(19) CometFilter -Input [1]: [cd_demo_sk#20] -Condition : isnotnull(cd_demo_sk#20) - -(20) CometColumnarToRow [codegen id : 3] -Input [1]: [cd_demo_sk#20] - -(21) BroadcastExchange -Input [1]: [cd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#16] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None - -(23) Project [codegen id : 7] -Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] - -(24) CometNativeScan parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#23, 2)) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) - -(26) CometProject -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#23, 2)) AS ca_state#25, ca_country#24] - -(27) CometColumnarToRow [codegen id : 4] -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(28) BroadcastExchange -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(29) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#17] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None - -(30) Project [codegen id : 7] -Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(31) ReusedExchange [Reuses operator id: 166] -Output [1]: [d_date_sk#26] - -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#26] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] - -(34) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#27, i_item_id#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [i_item_sk#27, i_item_id#28] -Condition : isnotnull(i_item_sk#27) - -(36) CometProject -Input [2]: [i_item_sk#27, i_item_id#28] -Arguments: [i_item_sk#27, i_item_id#29], [i_item_sk#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#28, 16)) AS i_item_id#29] - -(37) CometColumnarToRow [codegen id : 6] -Input [2]: [i_item_sk#27, i_item_id#29] - -(38) BroadcastExchange -Input [2]: [i_item_sk#27, i_item_id#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#27] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 7] -Output [11]: [i_item_id#29 AS i_item_id#30, ca_country#24 AS ca_country#31, ca_state#25 AS ca_state#32, ca_county#22 AS ca_county#33, cast(cs_quantity#4 as decimal(12,2)) AS agg1#34, cast(cs_list_price#5 as decimal(12,2)) AS agg2#35, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#36, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#37, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#38, cast(c_birth_year#19 as decimal(12,2)) AS agg6#39, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#40] -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#27, i_item_id#29] - -(41) HashAggregate [codegen id : 7] -Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#34, agg2#35, agg3#36, agg4#37, agg5#38, agg6#39, agg7#40] -Keys [4]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33] -Functions [7]: [partial_avg(agg1#34), partial_avg(agg2#35), partial_avg(agg3#36), partial_avg(agg4#37), partial_avg(agg5#38), partial_avg(agg6#39), partial_avg(agg7#40)] -Aggregate Attributes [14]: [sum#41, count#42, sum#43, count#44, sum#45, count#46, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54] -Results [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] - -(42) CometColumnarExchange -Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] -Arguments: hashpartitioning(i_item_id#30, ca_country#31, ca_state#32, ca_county#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) CometColumnarToRow [codegen id : 8] -Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] - -(44) HashAggregate [codegen id : 8] -Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] -Keys [4]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33] -Functions [7]: [avg(agg1#34), avg(agg2#35), avg(agg3#36), avg(agg4#37), avg(agg5#38), avg(agg6#39), avg(agg7#40)] -Aggregate Attributes [7]: [avg(agg1#34)#69, avg(agg2#35)#70, avg(agg3#36)#71, avg(agg4#37)#72, avg(agg5#38)#73, avg(agg6#39)#74, avg(agg7#40)#75] -Results [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, avg(agg1#34)#69 AS agg1#76, avg(agg2#35)#70 AS agg2#77, avg(agg3#36)#71 AS agg3#78, avg(agg4#37)#72 AS agg4#79, avg(agg5#38)#73 AS agg5#80, avg(agg6#39)#74 AS agg6#81, avg(agg7#40)#75 AS agg7#82] - -(45) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#91), dynamicpruningexpression(cs_sold_date_sk#91 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 15] -Input [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] - -(47) Filter [codegen id : 15] -Input [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] -Condition : ((isnotnull(cs_bill_cdemo_sk#84) AND isnotnull(cs_bill_customer_sk#83)) AND isnotnull(cs_item_sk#85)) - -(48) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#92, cd_dep_count#93] - -(49) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_cdemo_sk#84] -Right keys [1]: [cd_demo_sk#92] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 15] -Output [9]: [cs_bill_customer_sk#83, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93] -Input [11]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_demo_sk#92, cd_dep_count#93] - -(51) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#94, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97] - -(52) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_customer_sk#83] -Right keys [1]: [c_customer_sk#94] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 15] -Output [11]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97] -Input [13]: [cs_bill_customer_sk#83, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_customer_sk#94, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97] - -(54) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#98] - -(55) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_cdemo_sk#95] -Right keys [1]: [cd_demo_sk#98] -Join type: Inner -Join condition: None - -(56) Project [codegen id : 15] -Output [10]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_addr_sk#96, c_birth_year#97] -Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97, cd_demo_sk#98] - -(57) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#99, ca_state#100, ca_country#101] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [ca_address_sk#99, ca_state#100, ca_country#101] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#100, 2)) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#99)) - -(59) CometProject -Input [3]: [ca_address_sk#99, ca_state#100, ca_country#101] -Arguments: [ca_address_sk#99, ca_state#102, ca_country#101], [ca_address_sk#99, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#100, 2)) AS ca_state#102, ca_country#101] - -(60) CometColumnarToRow [codegen id : 12] -Input [3]: [ca_address_sk#99, ca_state#102, ca_country#101] - -(61) BroadcastExchange -Input [3]: [ca_address_sk#99, ca_state#102, ca_country#101] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(62) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_addr_sk#96] -Right keys [1]: [ca_address_sk#99] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 15] -Output [11]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_birth_year#97, ca_state#102, ca_country#101] -Input [13]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_addr_sk#96, c_birth_year#97, ca_address_sk#99, ca_state#102, ca_country#101] - -(64) ReusedExchange [Reuses operator id: 166] -Output [1]: [d_date_sk#103] - -(65) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_sold_date_sk#91] -Right keys [1]: [d_date_sk#103] -Join type: Inner -Join condition: None - -(66) Project [codegen id : 15] -Output [10]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#93, c_birth_year#97, ca_state#102, ca_country#101] -Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_birth_year#97, ca_state#102, ca_country#101, d_date_sk#103] - -(67) ReusedExchange [Reuses operator id: 38] -Output [2]: [i_item_sk#104, i_item_id#105] - -(68) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_item_sk#85] -Right keys [1]: [i_item_sk#104] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 15] -Output [10]: [i_item_id#105, ca_country#101, ca_state#102, cast(cs_quantity#86 as decimal(12,2)) AS agg1#106, cast(cs_list_price#87 as decimal(12,2)) AS agg2#107, cast(cs_coupon_amt#89 as decimal(12,2)) AS agg3#108, cast(cs_sales_price#88 as decimal(12,2)) AS agg4#109, cast(cs_net_profit#90 as decimal(12,2)) AS agg5#110, cast(c_birth_year#97 as decimal(12,2)) AS agg6#111, cast(cd_dep_count#93 as decimal(12,2)) AS agg7#112] -Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#93, c_birth_year#97, ca_state#102, ca_country#101, i_item_sk#104, i_item_id#105] - -(70) HashAggregate [codegen id : 15] -Input [10]: [i_item_id#105, ca_country#101, ca_state#102, agg1#106, agg2#107, agg3#108, agg4#109, agg5#110, agg6#111, agg7#112] -Keys [3]: [i_item_id#105, ca_country#101, ca_state#102] -Functions [7]: [partial_avg(agg1#106), partial_avg(agg2#107), partial_avg(agg3#108), partial_avg(agg4#109), partial_avg(agg5#110), partial_avg(agg6#111), partial_avg(agg7#112)] -Aggregate Attributes [14]: [sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124, sum#125, count#126] -Results [17]: [i_item_id#105, ca_country#101, ca_state#102, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134, sum#135, count#136, sum#137, count#138, sum#139, count#140] - -(71) CometColumnarExchange -Input [17]: [i_item_id#105, ca_country#101, ca_state#102, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134, sum#135, count#136, sum#137, count#138, sum#139, count#140] -Arguments: hashpartitioning(i_item_id#105, ca_country#101, ca_state#102, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(72) CometColumnarToRow [codegen id : 16] -Input [17]: [i_item_id#105, ca_country#101, ca_state#102, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134, sum#135, count#136, sum#137, count#138, sum#139, count#140] - -(73) HashAggregate [codegen id : 16] -Input [17]: [i_item_id#105, ca_country#101, ca_state#102, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134, sum#135, count#136, sum#137, count#138, sum#139, count#140] -Keys [3]: [i_item_id#105, ca_country#101, ca_state#102] -Functions [7]: [avg(agg1#106), avg(agg2#107), avg(agg3#108), avg(agg4#109), avg(agg5#110), avg(agg6#111), avg(agg7#112)] -Aggregate Attributes [7]: [avg(agg1#106)#141, avg(agg2#107)#142, avg(agg3#108)#143, avg(agg4#109)#144, avg(agg5#110)#145, avg(agg6#111)#146, avg(agg7#112)#147] -Results [11]: [i_item_id#105, ca_country#101, ca_state#102, null AS county#148, avg(agg1#106)#141 AS agg1#149, avg(agg2#107)#142 AS agg2#150, avg(agg3#108)#143 AS agg3#151, avg(agg4#109)#144 AS agg4#152, avg(agg5#110)#145 AS agg5#153, avg(agg6#111)#146 AS agg6#154, avg(agg7#112)#147 AS agg7#155] - -(74) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#156, cs_bill_cdemo_sk#157, cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#164), dynamicpruningexpression(cs_sold_date_sk#164 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(75) ColumnarToRow [codegen id : 23] -Input [9]: [cs_bill_customer_sk#156, cs_bill_cdemo_sk#157, cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164] - -(76) Filter [codegen id : 23] -Input [9]: [cs_bill_customer_sk#156, cs_bill_cdemo_sk#157, cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164] -Condition : ((isnotnull(cs_bill_cdemo_sk#157) AND isnotnull(cs_bill_customer_sk#156)) AND isnotnull(cs_item_sk#158)) - -(77) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#165, cd_dep_count#166] - -(78) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_cdemo_sk#157] -Right keys [1]: [cd_demo_sk#165] -Join type: Inner -Join condition: None - -(79) Project [codegen id : 23] -Output [9]: [cs_bill_customer_sk#156, cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164, cd_dep_count#166] -Input [11]: [cs_bill_customer_sk#156, cs_bill_cdemo_sk#157, cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164, cd_demo_sk#165, cd_dep_count#166] - -(80) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#167, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170] - -(81) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_customer_sk#156] -Right keys [1]: [c_customer_sk#167] -Join type: Inner -Join condition: None - -(82) Project [codegen id : 23] -Output [11]: [cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164, cd_dep_count#166, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170] -Input [13]: [cs_bill_customer_sk#156, cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164, cd_dep_count#166, c_customer_sk#167, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170] - -(83) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#171] - -(84) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_cdemo_sk#168] -Right keys [1]: [cd_demo_sk#171] -Join type: Inner -Join condition: None - -(85) Project [codegen id : 23] -Output [10]: [cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164, cd_dep_count#166, c_current_addr_sk#169, c_birth_year#170] -Input [12]: [cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164, cd_dep_count#166, c_current_cdemo_sk#168, c_current_addr_sk#169, c_birth_year#170, cd_demo_sk#171] - -(86) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#172, ca_state#173, ca_country#174] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(87) CometFilter -Input [3]: [ca_address_sk#172, ca_state#173, ca_country#174] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#173, 2)) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#172)) - -(88) CometProject -Input [3]: [ca_address_sk#172, ca_state#173, ca_country#174] -Arguments: [ca_address_sk#172, ca_country#174], [ca_address_sk#172, ca_country#174] - -(89) CometColumnarToRow [codegen id : 20] -Input [2]: [ca_address_sk#172, ca_country#174] - -(90) BroadcastExchange -Input [2]: [ca_address_sk#172, ca_country#174] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -(91) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_addr_sk#169] -Right keys [1]: [ca_address_sk#172] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 23] -Output [10]: [cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164, cd_dep_count#166, c_birth_year#170, ca_country#174] -Input [12]: [cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164, cd_dep_count#166, c_current_addr_sk#169, c_birth_year#170, ca_address_sk#172, ca_country#174] - -(93) ReusedExchange [Reuses operator id: 166] -Output [1]: [d_date_sk#175] - -(94) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_sold_date_sk#164] -Right keys [1]: [d_date_sk#175] -Join type: Inner -Join condition: None - -(95) Project [codegen id : 23] -Output [9]: [cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cd_dep_count#166, c_birth_year#170, ca_country#174] -Input [11]: [cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cs_sold_date_sk#164, cd_dep_count#166, c_birth_year#170, ca_country#174, d_date_sk#175] - -(96) ReusedExchange [Reuses operator id: 38] -Output [2]: [i_item_sk#176, i_item_id#177] - -(97) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_item_sk#158] -Right keys [1]: [i_item_sk#176] -Join type: Inner -Join condition: None - -(98) Project [codegen id : 23] -Output [9]: [i_item_id#177, ca_country#174, cast(cs_quantity#159 as decimal(12,2)) AS agg1#178, cast(cs_list_price#160 as decimal(12,2)) AS agg2#179, cast(cs_coupon_amt#162 as decimal(12,2)) AS agg3#180, cast(cs_sales_price#161 as decimal(12,2)) AS agg4#181, cast(cs_net_profit#163 as decimal(12,2)) AS agg5#182, cast(c_birth_year#170 as decimal(12,2)) AS agg6#183, cast(cd_dep_count#166 as decimal(12,2)) AS agg7#184] -Input [11]: [cs_item_sk#158, cs_quantity#159, cs_list_price#160, cs_sales_price#161, cs_coupon_amt#162, cs_net_profit#163, cd_dep_count#166, c_birth_year#170, ca_country#174, i_item_sk#176, i_item_id#177] - -(99) HashAggregate [codegen id : 23] -Input [9]: [i_item_id#177, ca_country#174, agg1#178, agg2#179, agg3#180, agg4#181, agg5#182, agg6#183, agg7#184] -Keys [2]: [i_item_id#177, ca_country#174] -Functions [7]: [partial_avg(agg1#178), partial_avg(agg2#179), partial_avg(agg3#180), partial_avg(agg4#181), partial_avg(agg5#182), partial_avg(agg6#183), partial_avg(agg7#184)] -Aggregate Attributes [14]: [sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] -Results [16]: [i_item_id#177, ca_country#174, sum#199, count#200, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208, sum#209, count#210, sum#211, count#212] - -(100) CometColumnarExchange -Input [16]: [i_item_id#177, ca_country#174, sum#199, count#200, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208, sum#209, count#210, sum#211, count#212] -Arguments: hashpartitioning(i_item_id#177, ca_country#174, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(101) CometColumnarToRow [codegen id : 24] -Input [16]: [i_item_id#177, ca_country#174, sum#199, count#200, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208, sum#209, count#210, sum#211, count#212] - -(102) HashAggregate [codegen id : 24] -Input [16]: [i_item_id#177, ca_country#174, sum#199, count#200, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208, sum#209, count#210, sum#211, count#212] -Keys [2]: [i_item_id#177, ca_country#174] -Functions [7]: [avg(agg1#178), avg(agg2#179), avg(agg3#180), avg(agg4#181), avg(agg5#182), avg(agg6#183), avg(agg7#184)] -Aggregate Attributes [7]: [avg(agg1#178)#213, avg(agg2#179)#214, avg(agg3#180)#215, avg(agg4#181)#216, avg(agg5#182)#217, avg(agg6#183)#218, avg(agg7#184)#219] -Results [11]: [i_item_id#177, ca_country#174, null AS ca_state#220, null AS county#221, avg(agg1#178)#213 AS agg1#222, avg(agg2#179)#214 AS agg2#223, avg(agg3#180)#215 AS agg3#224, avg(agg4#181)#216 AS agg4#225, avg(agg5#182)#217 AS agg5#226, avg(agg6#183)#218 AS agg6#227, avg(agg7#184)#219 AS agg7#228] - -(103) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#229, cs_bill_cdemo_sk#230, cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#237), dynamicpruningexpression(cs_sold_date_sk#237 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(104) ColumnarToRow [codegen id : 31] -Input [9]: [cs_bill_customer_sk#229, cs_bill_cdemo_sk#230, cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237] - -(105) Filter [codegen id : 31] -Input [9]: [cs_bill_customer_sk#229, cs_bill_cdemo_sk#230, cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237] -Condition : ((isnotnull(cs_bill_cdemo_sk#230) AND isnotnull(cs_bill_customer_sk#229)) AND isnotnull(cs_item_sk#231)) - -(106) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#238, cd_dep_count#239] - -(107) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_cdemo_sk#230] -Right keys [1]: [cd_demo_sk#238] -Join type: Inner -Join condition: None - -(108) Project [codegen id : 31] -Output [9]: [cs_bill_customer_sk#229, cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237, cd_dep_count#239] -Input [11]: [cs_bill_customer_sk#229, cs_bill_cdemo_sk#230, cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237, cd_demo_sk#238, cd_dep_count#239] - -(109) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#240, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243] - -(110) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_customer_sk#229] -Right keys [1]: [c_customer_sk#240] -Join type: Inner -Join condition: None - -(111) Project [codegen id : 31] -Output [11]: [cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237, cd_dep_count#239, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243] -Input [13]: [cs_bill_customer_sk#229, cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237, cd_dep_count#239, c_customer_sk#240, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243] - -(112) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#244] - -(113) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_cdemo_sk#241] -Right keys [1]: [cd_demo_sk#244] -Join type: Inner -Join condition: None - -(114) Project [codegen id : 31] -Output [10]: [cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237, cd_dep_count#239, c_current_addr_sk#242, c_birth_year#243] -Input [12]: [cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237, cd_dep_count#239, c_current_cdemo_sk#241, c_current_addr_sk#242, c_birth_year#243, cd_demo_sk#244] - -(115) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#245, ca_state#246] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(116) CometFilter -Input [2]: [ca_address_sk#245, ca_state#246] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#246, 2)) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#245)) - -(117) CometProject -Input [2]: [ca_address_sk#245, ca_state#246] -Arguments: [ca_address_sk#245], [ca_address_sk#245] - -(118) CometColumnarToRow [codegen id : 28] -Input [1]: [ca_address_sk#245] - -(119) BroadcastExchange -Input [1]: [ca_address_sk#245] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -(120) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_addr_sk#242] -Right keys [1]: [ca_address_sk#245] -Join type: Inner -Join condition: None - -(121) Project [codegen id : 31] -Output [9]: [cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237, cd_dep_count#239, c_birth_year#243] -Input [11]: [cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237, cd_dep_count#239, c_current_addr_sk#242, c_birth_year#243, ca_address_sk#245] - -(122) ReusedExchange [Reuses operator id: 166] -Output [1]: [d_date_sk#247] - -(123) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_sold_date_sk#237] -Right keys [1]: [d_date_sk#247] -Join type: Inner -Join condition: None - -(124) Project [codegen id : 31] -Output [8]: [cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cd_dep_count#239, c_birth_year#243] -Input [10]: [cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cs_sold_date_sk#237, cd_dep_count#239, c_birth_year#243, d_date_sk#247] - -(125) ReusedExchange [Reuses operator id: 38] -Output [2]: [i_item_sk#248, i_item_id#249] - -(126) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_item_sk#231] -Right keys [1]: [i_item_sk#248] -Join type: Inner -Join condition: None - -(127) Project [codegen id : 31] -Output [8]: [i_item_id#249, cast(cs_quantity#232 as decimal(12,2)) AS agg1#250, cast(cs_list_price#233 as decimal(12,2)) AS agg2#251, cast(cs_coupon_amt#235 as decimal(12,2)) AS agg3#252, cast(cs_sales_price#234 as decimal(12,2)) AS agg4#253, cast(cs_net_profit#236 as decimal(12,2)) AS agg5#254, cast(c_birth_year#243 as decimal(12,2)) AS agg6#255, cast(cd_dep_count#239 as decimal(12,2)) AS agg7#256] -Input [10]: [cs_item_sk#231, cs_quantity#232, cs_list_price#233, cs_sales_price#234, cs_coupon_amt#235, cs_net_profit#236, cd_dep_count#239, c_birth_year#243, i_item_sk#248, i_item_id#249] - -(128) HashAggregate [codegen id : 31] -Input [8]: [i_item_id#249, agg1#250, agg2#251, agg3#252, agg4#253, agg5#254, agg6#255, agg7#256] -Keys [1]: [i_item_id#249] -Functions [7]: [partial_avg(agg1#250), partial_avg(agg2#251), partial_avg(agg3#252), partial_avg(agg4#253), partial_avg(agg5#254), partial_avg(agg6#255), partial_avg(agg7#256)] -Aggregate Attributes [14]: [sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268, sum#269, count#270] -Results [15]: [i_item_id#249, sum#271, count#272, sum#273, count#274, sum#275, count#276, sum#277, count#278, sum#279, count#280, sum#281, count#282, sum#283, count#284] - -(129) CometColumnarExchange -Input [15]: [i_item_id#249, sum#271, count#272, sum#273, count#274, sum#275, count#276, sum#277, count#278, sum#279, count#280, sum#281, count#282, sum#283, count#284] -Arguments: hashpartitioning(i_item_id#249, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(130) CometColumnarToRow [codegen id : 32] -Input [15]: [i_item_id#249, sum#271, count#272, sum#273, count#274, sum#275, count#276, sum#277, count#278, sum#279, count#280, sum#281, count#282, sum#283, count#284] - -(131) HashAggregate [codegen id : 32] -Input [15]: [i_item_id#249, sum#271, count#272, sum#273, count#274, sum#275, count#276, sum#277, count#278, sum#279, count#280, sum#281, count#282, sum#283, count#284] -Keys [1]: [i_item_id#249] -Functions [7]: [avg(agg1#250), avg(agg2#251), avg(agg3#252), avg(agg4#253), avg(agg5#254), avg(agg6#255), avg(agg7#256)] -Aggregate Attributes [7]: [avg(agg1#250)#285, avg(agg2#251)#286, avg(agg3#252)#287, avg(agg4#253)#288, avg(agg5#254)#289, avg(agg6#255)#290, avg(agg7#256)#291] -Results [11]: [i_item_id#249, null AS ca_country#292, null AS ca_state#293, null AS county#294, avg(agg1#250)#285 AS agg1#295, avg(agg2#251)#286 AS agg2#296, avg(agg3#252)#287 AS agg3#297, avg(agg4#253)#288 AS agg4#298, avg(agg5#254)#289 AS agg5#299, avg(agg6#255)#290 AS agg6#300, avg(agg7#256)#291 AS agg7#301] - -(132) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#302, cs_bill_cdemo_sk#303, cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#310), dynamicpruningexpression(cs_sold_date_sk#310 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(133) ColumnarToRow [codegen id : 39] -Input [9]: [cs_bill_customer_sk#302, cs_bill_cdemo_sk#303, cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310] - -(134) Filter [codegen id : 39] -Input [9]: [cs_bill_customer_sk#302, cs_bill_cdemo_sk#303, cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310] -Condition : ((isnotnull(cs_bill_cdemo_sk#303) AND isnotnull(cs_bill_customer_sk#302)) AND isnotnull(cs_item_sk#304)) - -(135) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#311, cd_dep_count#312] - -(136) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_cdemo_sk#303] -Right keys [1]: [cd_demo_sk#311] -Join type: Inner -Join condition: None - -(137) Project [codegen id : 39] -Output [9]: [cs_bill_customer_sk#302, cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310, cd_dep_count#312] -Input [11]: [cs_bill_customer_sk#302, cs_bill_cdemo_sk#303, cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310, cd_demo_sk#311, cd_dep_count#312] - -(138) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#313, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316] - -(139) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_customer_sk#302] -Right keys [1]: [c_customer_sk#313] -Join type: Inner -Join condition: None - -(140) Project [codegen id : 39] -Output [11]: [cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310, cd_dep_count#312, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316] -Input [13]: [cs_bill_customer_sk#302, cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310, cd_dep_count#312, c_customer_sk#313, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316] - -(141) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#317] - -(142) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_cdemo_sk#314] -Right keys [1]: [cd_demo_sk#317] -Join type: Inner -Join condition: None - -(143) Project [codegen id : 39] -Output [10]: [cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310, cd_dep_count#312, c_current_addr_sk#315, c_birth_year#316] -Input [12]: [cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310, cd_dep_count#312, c_current_cdemo_sk#314, c_current_addr_sk#315, c_birth_year#316, cd_demo_sk#317] - -(144) ReusedExchange [Reuses operator id: 119] -Output [1]: [ca_address_sk#318] - -(145) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_addr_sk#315] -Right keys [1]: [ca_address_sk#318] -Join type: Inner -Join condition: None - -(146) Project [codegen id : 39] -Output [9]: [cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310, cd_dep_count#312, c_birth_year#316] -Input [11]: [cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310, cd_dep_count#312, c_current_addr_sk#315, c_birth_year#316, ca_address_sk#318] - -(147) ReusedExchange [Reuses operator id: 166] -Output [1]: [d_date_sk#319] - -(148) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_sold_date_sk#310] -Right keys [1]: [d_date_sk#319] -Join type: Inner -Join condition: None - -(149) Project [codegen id : 39] -Output [8]: [cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cd_dep_count#312, c_birth_year#316] -Input [10]: [cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cs_sold_date_sk#310, cd_dep_count#312, c_birth_year#316, d_date_sk#319] - -(150) CometNativeScan parquet spark_catalog.default.item -Output [1]: [i_item_sk#320] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(151) CometFilter -Input [1]: [i_item_sk#320] -Condition : isnotnull(i_item_sk#320) - -(152) CometColumnarToRow [codegen id : 38] -Input [1]: [i_item_sk#320] - -(153) BroadcastExchange -Input [1]: [i_item_sk#320] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] - -(154) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_item_sk#304] -Right keys [1]: [i_item_sk#320] -Join type: Inner -Join condition: None - -(155) Project [codegen id : 39] -Output [7]: [cast(cs_quantity#305 as decimal(12,2)) AS agg1#321, cast(cs_list_price#306 as decimal(12,2)) AS agg2#322, cast(cs_coupon_amt#308 as decimal(12,2)) AS agg3#323, cast(cs_sales_price#307 as decimal(12,2)) AS agg4#324, cast(cs_net_profit#309 as decimal(12,2)) AS agg5#325, cast(c_birth_year#316 as decimal(12,2)) AS agg6#326, cast(cd_dep_count#312 as decimal(12,2)) AS agg7#327] -Input [9]: [cs_item_sk#304, cs_quantity#305, cs_list_price#306, cs_sales_price#307, cs_coupon_amt#308, cs_net_profit#309, cd_dep_count#312, c_birth_year#316, i_item_sk#320] - -(156) HashAggregate [codegen id : 39] -Input [7]: [agg1#321, agg2#322, agg3#323, agg4#324, agg5#325, agg6#326, agg7#327] -Keys: [] -Functions [7]: [partial_avg(agg1#321), partial_avg(agg2#322), partial_avg(agg3#323), partial_avg(agg4#324), partial_avg(agg5#325), partial_avg(agg6#326), partial_avg(agg7#327)] -Aggregate Attributes [14]: [sum#328, count#329, sum#330, count#331, sum#332, count#333, sum#334, count#335, sum#336, count#337, sum#338, count#339, sum#340, count#341] -Results [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] - -(157) CometColumnarExchange -Input [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(158) CometColumnarToRow [codegen id : 40] -Input [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] - -(159) HashAggregate [codegen id : 40] -Input [14]: [sum#342, count#343, sum#344, count#345, sum#346, count#347, sum#348, count#349, sum#350, count#351, sum#352, count#353, sum#354, count#355] -Keys: [] -Functions [7]: [avg(agg1#321), avg(agg2#322), avg(agg3#323), avg(agg4#324), avg(agg5#325), avg(agg6#326), avg(agg7#327)] -Aggregate Attributes [7]: [avg(agg1#321)#356, avg(agg2#322)#357, avg(agg3#323)#358, avg(agg4#324)#359, avg(agg5#325)#360, avg(agg6#326)#361, avg(agg7#327)#362] -Results [11]: [null AS i_item_id#363, null AS ca_country#364, null AS ca_state#365, null AS county#366, avg(agg1#321)#356 AS agg1#367, avg(agg2#322)#357 AS agg2#368, avg(agg3#323)#358 AS agg3#369, avg(agg4#324)#359 AS agg4#370, avg(agg5#325)#360 AS agg5#371, avg(agg6#326)#361 AS agg6#372, avg(agg7#327)#362 AS agg7#373] - -(160) Union - -(161) TakeOrderedAndProject -Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] -Arguments: 100, [ca_country#31 ASC NULLS FIRST, ca_state#32 ASC NULLS FIRST, ca_county#33 ASC NULLS FIRST, i_item_id#30 ASC NULLS FIRST], [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (166) -+- * CometColumnarToRow (165) - +- CometProject (164) - +- CometFilter (163) - +- CometNativeScan parquet spark_catalog.default.date_dim (162) - - -(162) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#374] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(163) CometFilter -Input [2]: [d_date_sk#26, d_year#374] -Condition : ((isnotnull(d_year#374) AND (d_year#374 = 2001)) AND isnotnull(d_date_sk#26)) - -(164) CometProject -Input [2]: [d_date_sk#26, d_year#374] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(165) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(166) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] - -Subquery:2 Hosting operator id = 45 Hosting Expression = cs_sold_date_sk#91 IN dynamicpruning#10 - -Subquery:3 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#164 IN dynamicpruning#10 - -Subquery:4 Hosting operator id = 103 Hosting Expression = cs_sold_date_sk#237 IN dynamicpruning#10 - -Subquery:5 Hosting operator id = 132 Hosting Expression = cs_sold_date_sk#310 IN dynamicpruning#10 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/simplified.txt deleted file mode 100644 index 71745258b5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/simplified.txt +++ /dev/null @@ -1,241 +0,0 @@ -TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - Union - WholeStageCodegen (8) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,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),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county] #1 - WholeStageCodegen (7) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [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] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - WholeStageCodegen (16) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,ca_country,ca_state] #8 - WholeStageCodegen (15) - HashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,ca_state,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [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] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (24) - HashAggregate [i_item_id,ca_country,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),ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,ca_country] #10 - WholeStageCodegen (23) - HashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [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] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (32) - HashAggregate [i_item_id,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),ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #12 - WholeStageCodegen (31) - HashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [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] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (28) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (40) - HashAggregate [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),i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #14 - WholeStageCodegen (39) - HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [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] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - ReusedExchange [ca_address_sk] #13 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (38) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/explain.txt deleted file mode 100644 index 40d7f1ecde..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,846 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (146) -+- CometTakeOrderedAndProject (145) - +- CometUnion (144) - :- CometHashAggregate (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (19) - : : : : +- CometBroadcastHashJoin (18) - : : : : :- CometProject (14) - : : : : : +- CometBroadcastHashJoin (13) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : +- CometBroadcastExchange (6) - : : : : : : +- CometProject (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : : : : +- CometBroadcastExchange (12) - : : : : : +- CometProject (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : : : : +- CometBroadcastExchange (17) - : : : : +- CometFilter (16) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (15) - : : : +- CometBroadcastExchange (23) - : : : +- CometProject (22) - : : : +- CometFilter (21) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - : : +- CometBroadcastExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) - :- CometHashAggregate (66) - : +- CometExchange (65) - : +- CometHashAggregate (64) - : +- CometProject (63) - : +- CometBroadcastHashJoin (62) - : :- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (57) - : : : +- CometBroadcastHashJoin (56) - : : : :- CometProject (51) - : : : : +- CometBroadcastHashJoin (50) - : : : : :- CometProject (48) - : : : : : +- CometBroadcastHashJoin (47) - : : : : : :- CometProject (45) - : : : : : : +- CometBroadcastHashJoin (44) - : : : : : : :- CometFilter (42) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (41) - : : : : : : +- ReusedExchange (43) - : : : : : +- ReusedExchange (46) - : : : : +- ReusedExchange (49) - : : : +- CometBroadcastExchange (55) - : : : +- CometProject (54) - : : : +- CometFilter (53) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (52) - : : +- ReusedExchange (58) - : +- ReusedExchange (61) - :- CometHashAggregate (92) - : +- CometExchange (91) - : +- CometHashAggregate (90) - : +- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometProject (86) - : : +- CometBroadcastHashJoin (85) - : : :- CometProject (83) - : : : +- CometBroadcastHashJoin (82) - : : : :- CometProject (77) - : : : : +- CometBroadcastHashJoin (76) - : : : : :- CometProject (74) - : : : : : +- CometBroadcastHashJoin (73) - : : : : : :- CometProject (71) - : : : : : : +- CometBroadcastHashJoin (70) - : : : : : : :- CometFilter (68) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) - : : : : : : +- ReusedExchange (69) - : : : : : +- ReusedExchange (72) - : : : : +- ReusedExchange (75) - : : : +- CometBroadcastExchange (81) - : : : +- CometProject (80) - : : : +- CometFilter (79) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (78) - : : +- ReusedExchange (84) - : +- ReusedExchange (87) - :- CometHashAggregate (118) - : +- CometExchange (117) - : +- CometHashAggregate (116) - : +- CometProject (115) - : +- CometBroadcastHashJoin (114) - : :- CometProject (112) - : : +- CometBroadcastHashJoin (111) - : : :- CometProject (109) - : : : +- CometBroadcastHashJoin (108) - : : : :- CometProject (103) - : : : : +- CometBroadcastHashJoin (102) - : : : : :- CometProject (100) - : : : : : +- CometBroadcastHashJoin (99) - : : : : : :- CometProject (97) - : : : : : : +- CometBroadcastHashJoin (96) - : : : : : : :- CometFilter (94) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (93) - : : : : : : +- ReusedExchange (95) - : : : : : +- ReusedExchange (98) - : : : : +- ReusedExchange (101) - : : : +- CometBroadcastExchange (107) - : : : +- CometProject (106) - : : : +- CometFilter (105) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (104) - : : +- ReusedExchange (110) - : +- ReusedExchange (113) - +- CometHashAggregate (143) - +- CometExchange (142) - +- CometHashAggregate (141) - +- CometProject (140) - +- CometBroadcastHashJoin (139) - :- CometProject (135) - : +- CometBroadcastHashJoin (134) - : :- CometProject (132) - : : +- CometBroadcastHashJoin (131) - : : :- CometProject (129) - : : : +- CometBroadcastHashJoin (128) - : : : :- CometProject (126) - : : : : +- CometBroadcastHashJoin (125) - : : : : :- CometProject (123) - : : : : : +- CometBroadcastHashJoin (122) - : : : : : :- CometFilter (120) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (119) - : : : : : +- ReusedExchange (121) - : : : : +- ReusedExchange (124) - : : : +- ReusedExchange (127) - : : +- ReusedExchange (130) - : +- ReusedExchange (133) - +- CometBroadcastExchange (138) - +- CometFilter (137) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (136) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#12, 1)) = M)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#13, 20)) = College )) AND isnotnull(cd_demo_sk#11)) - -(5) CometProject -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] - -(6) CometBroadcastExchange -Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14] - -(7) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Right output [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight - -(8) CometProject -Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [In(c_birth_month, [1,10,12,4,5,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Condition : (((c_birth_month#18 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) - -(11) CometProject -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(12) CometBroadcastExchange -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(13) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight - -(14) CometProject -Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [cd_demo_sk#20] -Condition : isnotnull(cd_demo_sk#20) - -(17) CometBroadcastExchange -Input [1]: [cd_demo_sk#20] -Arguments: [cd_demo_sk#20] - -(18) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Right output [1]: [cd_demo_sk#20] -Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight - -(19) CometProject -Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#23, 2)) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) - -(22) CometProject -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#23, 2)) AS ca_state#25, ca_country#24] - -(23) CometBroadcastExchange -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(24) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -Right output [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight - -(25) CometProject -Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#26)) - -(28) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(29) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] - -(30) CometBroadcastHashJoin -Left output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [1]: [d_date_sk#26] -Arguments: [cs_sold_date_sk#9], [d_date_sk#26], Inner, BuildRight - -(31) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#28, i_item_id#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [i_item_sk#28, i_item_id#29] -Condition : isnotnull(i_item_sk#28) - -(34) CometProject -Input [2]: [i_item_sk#28, i_item_id#29] -Arguments: [i_item_sk#28, i_item_id#30], [i_item_sk#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#29, 16)) AS i_item_id#30] - -(35) CometBroadcastExchange -Input [2]: [i_item_sk#28, i_item_id#30] -Arguments: [i_item_sk#28, i_item_id#30] - -(36) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [2]: [i_item_sk#28, i_item_id#30] -Arguments: [cs_item_sk#3], [i_item_sk#28], Inner, BuildRight - -(37) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#28, i_item_id#30] -Arguments: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41], [i_item_id#30 AS i_item_id#31, ca_country#24 AS ca_country#32, ca_state#25 AS ca_state#33, ca_county#22 AS ca_county#34, cast(cs_quantity#4 as decimal(12,2)) AS agg1#35, cast(cs_list_price#5 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#39, cast(c_birth_year#19 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] - -(38) CometHashAggregate -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [4]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34] -Functions [7]: [partial_avg(agg1#35), partial_avg(agg2#36), partial_avg(agg3#37), partial_avg(agg4#38), partial_avg(agg5#39), partial_avg(agg6#40), partial_avg(agg7#41)] - -(39) CometExchange -Input [18]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] -Arguments: hashpartitioning(i_item_id#31, ca_country#32, ca_state#33, ca_county#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(40) CometHashAggregate -Input [18]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] -Keys [4]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34] -Functions [7]: [avg(agg1#35), avg(agg2#36), avg(agg3#37), avg(agg4#38), avg(agg5#39), avg(agg6#40), avg(agg7#41)] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#64), dynamicpruningexpression(cs_sold_date_sk#64 IN dynamicpruning#65)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(42) CometFilter -Input [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] -Condition : ((isnotnull(cs_bill_cdemo_sk#57) AND isnotnull(cs_bill_customer_sk#56)) AND isnotnull(cs_item_sk#58)) - -(43) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#66, cd_dep_count#67] - -(44) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] -Right output [2]: [cd_demo_sk#66, cd_dep_count#67] -Arguments: [cs_bill_cdemo_sk#57], [cd_demo_sk#66], Inner, BuildRight - -(45) CometProject -Input [11]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_demo_sk#66, cd_dep_count#67] -Arguments: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67], [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67] - -(46) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] - -(47) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67] -Right output [4]: [c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Arguments: [cs_bill_customer_sk#56], [c_customer_sk#68], Inner, BuildRight - -(48) CometProject -Input [13]: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] - -(49) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#72] - -(50) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Right output [1]: [cd_demo_sk#72] -Arguments: [c_current_cdemo_sk#69], [cd_demo_sk#72], Inner, BuildRight - -(51) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71, cd_demo_sk#72] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71] - -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#73, ca_state#74, ca_country#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [ca_address_sk#73, ca_state#74, ca_country#75] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#74, 2)) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#73)) - -(54) CometProject -Input [3]: [ca_address_sk#73, ca_state#74, ca_country#75] -Arguments: [ca_address_sk#73, ca_state#76, ca_country#75], [ca_address_sk#73, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#74, 2)) AS ca_state#76, ca_country#75] - -(55) CometBroadcastExchange -Input [3]: [ca_address_sk#73, ca_state#76, ca_country#75] -Arguments: [ca_address_sk#73, ca_state#76, ca_country#75] - -(56) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71] -Right output [3]: [ca_address_sk#73, ca_state#76, ca_country#75] -Arguments: [c_current_addr_sk#70], [ca_address_sk#73], Inner, BuildRight - -(57) CometProject -Input [13]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71, ca_address_sk#73, ca_state#76, ca_country#75] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#76, ca_country#75], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#76, ca_country#75] - -(58) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#77] - -(59) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#76, ca_country#75] -Right output [1]: [d_date_sk#77] -Arguments: [cs_sold_date_sk#64], [d_date_sk#77], Inner, BuildRight - -(60) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#76, ca_country#75, d_date_sk#77] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#76, ca_country#75], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#76, ca_country#75] - -(61) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#78, i_item_id#79] - -(62) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#76, ca_country#75] -Right output [2]: [i_item_sk#78, i_item_id#79] -Arguments: [cs_item_sk#58], [i_item_sk#78], Inner, BuildRight - -(63) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#76, ca_country#75, i_item_sk#78, i_item_id#79] -Arguments: [i_item_id#79, ca_country#75, ca_state#76, agg1#80, agg2#81, agg3#82, agg4#83, agg5#84, agg6#85, agg7#86], [i_item_id#79, ca_country#75, ca_state#76, cast(cs_quantity#59 as decimal(12,2)) AS agg1#80, cast(cs_list_price#60 as decimal(12,2)) AS agg2#81, cast(cs_coupon_amt#62 as decimal(12,2)) AS agg3#82, cast(cs_sales_price#61 as decimal(12,2)) AS agg4#83, cast(cs_net_profit#63 as decimal(12,2)) AS agg5#84, cast(c_birth_year#71 as decimal(12,2)) AS agg6#85, cast(cd_dep_count#67 as decimal(12,2)) AS agg7#86] - -(64) CometHashAggregate -Input [10]: [i_item_id#79, ca_country#75, ca_state#76, agg1#80, agg2#81, agg3#82, agg4#83, agg5#84, agg6#85, agg7#86] -Keys [3]: [i_item_id#79, ca_country#75, ca_state#76] -Functions [7]: [partial_avg(agg1#80), partial_avg(agg2#81), partial_avg(agg3#82), partial_avg(agg4#83), partial_avg(agg5#84), partial_avg(agg6#85), partial_avg(agg7#86)] - -(65) CometExchange -Input [17]: [i_item_id#79, ca_country#75, ca_state#76, sum#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98, sum#99, count#100] -Arguments: hashpartitioning(i_item_id#79, ca_country#75, ca_state#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(66) CometHashAggregate -Input [17]: [i_item_id#79, ca_country#75, ca_state#76, sum#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98, sum#99, count#100] -Keys [3]: [i_item_id#79, ca_country#75, ca_state#76] -Functions [7]: [avg(agg1#80), avg(agg2#81), avg(agg3#82), avg(agg4#83), avg(agg5#84), avg(agg6#85), avg(agg7#86)] - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#101, cs_bill_cdemo_sk#102, cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#109), dynamicpruningexpression(cs_sold_date_sk#109 IN dynamicpruning#110)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(68) CometFilter -Input [9]: [cs_bill_customer_sk#101, cs_bill_cdemo_sk#102, cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109] -Condition : ((isnotnull(cs_bill_cdemo_sk#102) AND isnotnull(cs_bill_customer_sk#101)) AND isnotnull(cs_item_sk#103)) - -(69) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#111, cd_dep_count#112] - -(70) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#101, cs_bill_cdemo_sk#102, cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109] -Right output [2]: [cd_demo_sk#111, cd_dep_count#112] -Arguments: [cs_bill_cdemo_sk#102], [cd_demo_sk#111], Inner, BuildRight - -(71) CometProject -Input [11]: [cs_bill_customer_sk#101, cs_bill_cdemo_sk#102, cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_demo_sk#111, cd_dep_count#112] -Arguments: [cs_bill_customer_sk#101, cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112], [cs_bill_customer_sk#101, cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112] - -(72) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#113, c_current_cdemo_sk#114, c_current_addr_sk#115, c_birth_year#116] - -(73) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#101, cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112] -Right output [4]: [c_customer_sk#113, c_current_cdemo_sk#114, c_current_addr_sk#115, c_birth_year#116] -Arguments: [cs_bill_customer_sk#101], [c_customer_sk#113], Inner, BuildRight - -(74) CometProject -Input [13]: [cs_bill_customer_sk#101, cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_customer_sk#113, c_current_cdemo_sk#114, c_current_addr_sk#115, c_birth_year#116] -Arguments: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_current_cdemo_sk#114, c_current_addr_sk#115, c_birth_year#116], [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_current_cdemo_sk#114, c_current_addr_sk#115, c_birth_year#116] - -(75) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#117] - -(76) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_current_cdemo_sk#114, c_current_addr_sk#115, c_birth_year#116] -Right output [1]: [cd_demo_sk#117] -Arguments: [c_current_cdemo_sk#114], [cd_demo_sk#117], Inner, BuildRight - -(77) CometProject -Input [12]: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_current_cdemo_sk#114, c_current_addr_sk#115, c_birth_year#116, cd_demo_sk#117] -Arguments: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_current_addr_sk#115, c_birth_year#116], [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_current_addr_sk#115, c_birth_year#116] - -(78) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#118, ca_state#119, ca_country#120] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(79) CometFilter -Input [3]: [ca_address_sk#118, ca_state#119, ca_country#120] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#119, 2)) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#118)) - -(80) CometProject -Input [3]: [ca_address_sk#118, ca_state#119, ca_country#120] -Arguments: [ca_address_sk#118, ca_country#120], [ca_address_sk#118, ca_country#120] - -(81) CometBroadcastExchange -Input [2]: [ca_address_sk#118, ca_country#120] -Arguments: [ca_address_sk#118, ca_country#120] - -(82) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_current_addr_sk#115, c_birth_year#116] -Right output [2]: [ca_address_sk#118, ca_country#120] -Arguments: [c_current_addr_sk#115], [ca_address_sk#118], Inner, BuildRight - -(83) CometProject -Input [12]: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_current_addr_sk#115, c_birth_year#116, ca_address_sk#118, ca_country#120] -Arguments: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_birth_year#116, ca_country#120], [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_birth_year#116, ca_country#120] - -(84) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#121] - -(85) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_birth_year#116, ca_country#120] -Right output [1]: [d_date_sk#121] -Arguments: [cs_sold_date_sk#109], [d_date_sk#121], Inner, BuildRight - -(86) CometProject -Input [11]: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_birth_year#116, ca_country#120, d_date_sk#121] -Arguments: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cd_dep_count#112, c_birth_year#116, ca_country#120], [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cd_dep_count#112, c_birth_year#116, ca_country#120] - -(87) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#122, i_item_id#123] - -(88) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cd_dep_count#112, c_birth_year#116, ca_country#120] -Right output [2]: [i_item_sk#122, i_item_id#123] -Arguments: [cs_item_sk#103], [i_item_sk#122], Inner, BuildRight - -(89) CometProject -Input [11]: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cd_dep_count#112, c_birth_year#116, ca_country#120, i_item_sk#122, i_item_id#123] -Arguments: [i_item_id#123, ca_country#120, agg1#124, agg2#125, agg3#126, agg4#127, agg5#128, agg6#129, agg7#130], [i_item_id#123, ca_country#120, cast(cs_quantity#104 as decimal(12,2)) AS agg1#124, cast(cs_list_price#105 as decimal(12,2)) AS agg2#125, cast(cs_coupon_amt#107 as decimal(12,2)) AS agg3#126, cast(cs_sales_price#106 as decimal(12,2)) AS agg4#127, cast(cs_net_profit#108 as decimal(12,2)) AS agg5#128, cast(c_birth_year#116 as decimal(12,2)) AS agg6#129, cast(cd_dep_count#112 as decimal(12,2)) AS agg7#130] - -(90) CometHashAggregate -Input [9]: [i_item_id#123, ca_country#120, agg1#124, agg2#125, agg3#126, agg4#127, agg5#128, agg6#129, agg7#130] -Keys [2]: [i_item_id#123, ca_country#120] -Functions [7]: [partial_avg(agg1#124), partial_avg(agg2#125), partial_avg(agg3#126), partial_avg(agg4#127), partial_avg(agg5#128), partial_avg(agg6#129), partial_avg(agg7#130)] - -(91) CometExchange -Input [16]: [i_item_id#123, ca_country#120, sum#131, count#132, sum#133, count#134, sum#135, count#136, sum#137, count#138, sum#139, count#140, sum#141, count#142, sum#143, count#144] -Arguments: hashpartitioning(i_item_id#123, ca_country#120, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(92) CometHashAggregate -Input [16]: [i_item_id#123, ca_country#120, sum#131, count#132, sum#133, count#134, sum#135, count#136, sum#137, count#138, sum#139, count#140, sum#141, count#142, sum#143, count#144] -Keys [2]: [i_item_id#123, ca_country#120] -Functions [7]: [avg(agg1#124), avg(agg2#125), avg(agg3#126), avg(agg4#127), avg(agg5#128), avg(agg6#129), avg(agg7#130)] - -(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#145, cs_bill_cdemo_sk#146, cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#153), dynamicpruningexpression(cs_sold_date_sk#153 IN dynamicpruning#154)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(94) CometFilter -Input [9]: [cs_bill_customer_sk#145, cs_bill_cdemo_sk#146, cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153] -Condition : ((isnotnull(cs_bill_cdemo_sk#146) AND isnotnull(cs_bill_customer_sk#145)) AND isnotnull(cs_item_sk#147)) - -(95) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#155, cd_dep_count#156] - -(96) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#145, cs_bill_cdemo_sk#146, cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153] -Right output [2]: [cd_demo_sk#155, cd_dep_count#156] -Arguments: [cs_bill_cdemo_sk#146], [cd_demo_sk#155], Inner, BuildRight - -(97) CometProject -Input [11]: [cs_bill_customer_sk#145, cs_bill_cdemo_sk#146, cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_demo_sk#155, cd_dep_count#156] -Arguments: [cs_bill_customer_sk#145, cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156], [cs_bill_customer_sk#145, cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156] - -(98) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#157, c_current_cdemo_sk#158, c_current_addr_sk#159, c_birth_year#160] - -(99) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#145, cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156] -Right output [4]: [c_customer_sk#157, c_current_cdemo_sk#158, c_current_addr_sk#159, c_birth_year#160] -Arguments: [cs_bill_customer_sk#145], [c_customer_sk#157], Inner, BuildRight - -(100) CometProject -Input [13]: [cs_bill_customer_sk#145, cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_customer_sk#157, c_current_cdemo_sk#158, c_current_addr_sk#159, c_birth_year#160] -Arguments: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_current_cdemo_sk#158, c_current_addr_sk#159, c_birth_year#160], [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_current_cdemo_sk#158, c_current_addr_sk#159, c_birth_year#160] - -(101) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#161] - -(102) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_current_cdemo_sk#158, c_current_addr_sk#159, c_birth_year#160] -Right output [1]: [cd_demo_sk#161] -Arguments: [c_current_cdemo_sk#158], [cd_demo_sk#161], Inner, BuildRight - -(103) CometProject -Input [12]: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_current_cdemo_sk#158, c_current_addr_sk#159, c_birth_year#160, cd_demo_sk#161] -Arguments: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_current_addr_sk#159, c_birth_year#160], [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_current_addr_sk#159, c_birth_year#160] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#162, ca_state#163] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [ca_address_sk#162, ca_state#163] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#163, 2)) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#162)) - -(106) CometProject -Input [2]: [ca_address_sk#162, ca_state#163] -Arguments: [ca_address_sk#162], [ca_address_sk#162] - -(107) CometBroadcastExchange -Input [1]: [ca_address_sk#162] -Arguments: [ca_address_sk#162] - -(108) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_current_addr_sk#159, c_birth_year#160] -Right output [1]: [ca_address_sk#162] -Arguments: [c_current_addr_sk#159], [ca_address_sk#162], Inner, BuildRight - -(109) CometProject -Input [11]: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_current_addr_sk#159, c_birth_year#160, ca_address_sk#162] -Arguments: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_birth_year#160], [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_birth_year#160] - -(110) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#164] - -(111) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_birth_year#160] -Right output [1]: [d_date_sk#164] -Arguments: [cs_sold_date_sk#153], [d_date_sk#164], Inner, BuildRight - -(112) CometProject -Input [10]: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_birth_year#160, d_date_sk#164] -Arguments: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cd_dep_count#156, c_birth_year#160], [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cd_dep_count#156, c_birth_year#160] - -(113) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#165, i_item_id#166] - -(114) CometBroadcastHashJoin -Left output [8]: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cd_dep_count#156, c_birth_year#160] -Right output [2]: [i_item_sk#165, i_item_id#166] -Arguments: [cs_item_sk#147], [i_item_sk#165], Inner, BuildRight - -(115) CometProject -Input [10]: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cd_dep_count#156, c_birth_year#160, i_item_sk#165, i_item_id#166] -Arguments: [i_item_id#166, agg1#167, agg2#168, agg3#169, agg4#170, agg5#171, agg6#172, agg7#173], [i_item_id#166, cast(cs_quantity#148 as decimal(12,2)) AS agg1#167, cast(cs_list_price#149 as decimal(12,2)) AS agg2#168, cast(cs_coupon_amt#151 as decimal(12,2)) AS agg3#169, cast(cs_sales_price#150 as decimal(12,2)) AS agg4#170, cast(cs_net_profit#152 as decimal(12,2)) AS agg5#171, cast(c_birth_year#160 as decimal(12,2)) AS agg6#172, cast(cd_dep_count#156 as decimal(12,2)) AS agg7#173] - -(116) CometHashAggregate -Input [8]: [i_item_id#166, agg1#167, agg2#168, agg3#169, agg4#170, agg5#171, agg6#172, agg7#173] -Keys [1]: [i_item_id#166] -Functions [7]: [partial_avg(agg1#167), partial_avg(agg2#168), partial_avg(agg3#169), partial_avg(agg4#170), partial_avg(agg5#171), partial_avg(agg6#172), partial_avg(agg7#173)] - -(117) CometExchange -Input [15]: [i_item_id#166, sum#174, count#175, sum#176, count#177, sum#178, count#179, sum#180, count#181, sum#182, count#183, sum#184, count#185, sum#186, count#187] -Arguments: hashpartitioning(i_item_id#166, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(118) CometHashAggregate -Input [15]: [i_item_id#166, sum#174, count#175, sum#176, count#177, sum#178, count#179, sum#180, count#181, sum#182, count#183, sum#184, count#185, sum#186, count#187] -Keys [1]: [i_item_id#166] -Functions [7]: [avg(agg1#167), avg(agg2#168), avg(agg3#169), avg(agg4#170), avg(agg5#171), avg(agg6#172), avg(agg7#173)] - -(119) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#188, cs_bill_cdemo_sk#189, cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#196), dynamicpruningexpression(cs_sold_date_sk#196 IN dynamicpruning#197)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(120) CometFilter -Input [9]: [cs_bill_customer_sk#188, cs_bill_cdemo_sk#189, cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196] -Condition : ((isnotnull(cs_bill_cdemo_sk#189) AND isnotnull(cs_bill_customer_sk#188)) AND isnotnull(cs_item_sk#190)) - -(121) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#198, cd_dep_count#199] - -(122) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#188, cs_bill_cdemo_sk#189, cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196] -Right output [2]: [cd_demo_sk#198, cd_dep_count#199] -Arguments: [cs_bill_cdemo_sk#189], [cd_demo_sk#198], Inner, BuildRight - -(123) CometProject -Input [11]: [cs_bill_customer_sk#188, cs_bill_cdemo_sk#189, cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_demo_sk#198, cd_dep_count#199] -Arguments: [cs_bill_customer_sk#188, cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199], [cs_bill_customer_sk#188, cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199] - -(124) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#200, c_current_cdemo_sk#201, c_current_addr_sk#202, c_birth_year#203] - -(125) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#188, cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199] -Right output [4]: [c_customer_sk#200, c_current_cdemo_sk#201, c_current_addr_sk#202, c_birth_year#203] -Arguments: [cs_bill_customer_sk#188], [c_customer_sk#200], Inner, BuildRight - -(126) CometProject -Input [13]: [cs_bill_customer_sk#188, cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_customer_sk#200, c_current_cdemo_sk#201, c_current_addr_sk#202, c_birth_year#203] -Arguments: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_current_cdemo_sk#201, c_current_addr_sk#202, c_birth_year#203], [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_current_cdemo_sk#201, c_current_addr_sk#202, c_birth_year#203] - -(127) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#204] - -(128) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_current_cdemo_sk#201, c_current_addr_sk#202, c_birth_year#203] -Right output [1]: [cd_demo_sk#204] -Arguments: [c_current_cdemo_sk#201], [cd_demo_sk#204], Inner, BuildRight - -(129) CometProject -Input [12]: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_current_cdemo_sk#201, c_current_addr_sk#202, c_birth_year#203, cd_demo_sk#204] -Arguments: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_current_addr_sk#202, c_birth_year#203], [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_current_addr_sk#202, c_birth_year#203] - -(130) ReusedExchange [Reuses operator id: 107] -Output [1]: [ca_address_sk#205] - -(131) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_current_addr_sk#202, c_birth_year#203] -Right output [1]: [ca_address_sk#205] -Arguments: [c_current_addr_sk#202], [ca_address_sk#205], Inner, BuildRight - -(132) CometProject -Input [11]: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_current_addr_sk#202, c_birth_year#203, ca_address_sk#205] -Arguments: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_birth_year#203], [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_birth_year#203] - -(133) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#206] - -(134) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_birth_year#203] -Right output [1]: [d_date_sk#206] -Arguments: [cs_sold_date_sk#196], [d_date_sk#206], Inner, BuildRight - -(135) CometProject -Input [10]: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_birth_year#203, d_date_sk#206] -Arguments: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cd_dep_count#199, c_birth_year#203], [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cd_dep_count#199, c_birth_year#203] - -(136) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#207] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(137) CometFilter -Input [1]: [i_item_sk#207] -Condition : isnotnull(i_item_sk#207) - -(138) CometBroadcastExchange -Input [1]: [i_item_sk#207] -Arguments: [i_item_sk#207] - -(139) CometBroadcastHashJoin -Left output [8]: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cd_dep_count#199, c_birth_year#203] -Right output [1]: [i_item_sk#207] -Arguments: [cs_item_sk#190], [i_item_sk#207], Inner, BuildRight - -(140) CometProject -Input [9]: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cd_dep_count#199, c_birth_year#203, i_item_sk#207] -Arguments: [agg1#208, agg2#209, agg3#210, agg4#211, agg5#212, agg6#213, agg7#214], [cast(cs_quantity#191 as decimal(12,2)) AS agg1#208, cast(cs_list_price#192 as decimal(12,2)) AS agg2#209, cast(cs_coupon_amt#194 as decimal(12,2)) AS agg3#210, cast(cs_sales_price#193 as decimal(12,2)) AS agg4#211, cast(cs_net_profit#195 as decimal(12,2)) AS agg5#212, cast(c_birth_year#203 as decimal(12,2)) AS agg6#213, cast(cd_dep_count#199 as decimal(12,2)) AS agg7#214] - -(141) CometHashAggregate -Input [7]: [agg1#208, agg2#209, agg3#210, agg4#211, agg5#212, agg6#213, agg7#214] -Keys: [] -Functions [7]: [partial_avg(agg1#208), partial_avg(agg2#209), partial_avg(agg3#210), partial_avg(agg4#211), partial_avg(agg5#212), partial_avg(agg6#213), partial_avg(agg7#214)] - -(142) CometExchange -Input [14]: [sum#215, count#216, sum#217, count#218, sum#219, count#220, sum#221, count#222, sum#223, count#224, sum#225, count#226, sum#227, count#228] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(143) CometHashAggregate -Input [14]: [sum#215, count#216, sum#217, count#218, sum#219, count#220, sum#221, count#222, sum#223, count#224, sum#225, count#226, sum#227, count#228] -Keys: [] -Functions [7]: [avg(agg1#208), avg(agg2#209), avg(agg3#210), avg(agg4#211), avg(agg5#212), avg(agg6#213), avg(agg7#214)] - -(144) CometUnion -Child 0 Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#229, agg2#230, agg3#231, agg4#232, agg5#233, agg6#234, agg7#235] -Child 1 Input [11]: [i_item_id#79, ca_country#75, ca_state#76, county#236, agg1#237, agg2#238, agg3#239, agg4#240, agg5#241, agg6#242, agg7#243] -Child 2 Input [11]: [i_item_id#123, ca_country#120, ca_state#244, county#245, agg1#246, agg2#247, agg3#248, agg4#249, agg5#250, agg6#251, agg7#252] -Child 3 Input [11]: [i_item_id#166, ca_country#253, ca_state#254, county#255, agg1#256, agg2#257, agg3#258, agg4#259, agg5#260, agg6#261, agg7#262] -Child 4 Input [11]: [i_item_id#263, ca_country#264, ca_state#265, county#266, agg1#267, agg2#268, agg3#269, agg4#270, agg5#271, agg6#272, agg7#273] - -(145) CometTakeOrderedAndProject -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#229, agg2#230, agg3#231, agg4#232, agg5#233, agg6#234, agg7#235] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#32 ASC NULLS FIRST,ca_state#33 ASC NULLS FIRST,ca_county#34 ASC NULLS FIRST,i_item_id#31 ASC NULLS FIRST], output=[i_item_id#31,ca_country#32,ca_state#33,ca_county#34,agg1#229,agg2#230,agg3#231,agg4#232,agg5#233,agg6#234,agg7#235]), [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#229, agg2#230, agg3#231, agg4#232, agg5#233, agg6#234, agg7#235], 100, 0, [ca_country#32 ASC NULLS FIRST, ca_state#33 ASC NULLS FIRST, ca_county#34 ASC NULLS FIRST, i_item_id#31 ASC NULLS FIRST], [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#229, agg2#230, agg3#231, agg4#232, agg5#233, agg6#234, agg7#235] - -(146) CometColumnarToRow [codegen id : 1] -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#229, agg2#230, agg3#231, agg4#232, agg5#233, agg6#234, agg7#235] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (151) -+- * CometColumnarToRow (150) - +- CometProject (149) - +- CometFilter (148) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (147) - - -(147) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(148) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#26)) - -(149) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(150) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(151) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#64 IN dynamicpruning#10 - -Subquery:3 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#109 IN dynamicpruning#10 - -Subquery:4 Hosting operator id = 93 Hosting Expression = cs_sold_date_sk#153 IN dynamicpruning#10 - -Subquery:5 Hosting operator id = 119 Hosting Expression = cs_sold_date_sk#196 IN dynamicpruning#10 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 1572a2a240..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,160 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometUnion [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country,ca_state,ca_county] #1 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometProject [i_item_id,ca_country,ca_state,ca_county,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,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_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 [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 [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,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_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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - 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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange [cd_demo_sk] #5 - CometFilter [cd_demo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #8 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country,ca_state] #9 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 [ca_address_sk,ca_state,ca_country] #10 - CometProject [ca_state] [ca_address_sk,ca_state,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country] #11 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 [ca_address_sk,ca_country] #12 - CometProject [ca_address_sk,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id] #13 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 [ca_address_sk] #14 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - ReusedExchange [d_date_sk] #7 - ReusedExchange [i_item_sk,i_item_id] #8 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange #15 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 - ReusedExchange [ca_address_sk] #14 - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [i_item_sk] #16 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/explain.txt deleted file mode 100644 index 40d7f1ecde..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/explain.txt +++ /dev/null @@ -1,846 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (146) -+- CometTakeOrderedAndProject (145) - +- CometUnion (144) - :- CometHashAggregate (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (19) - : : : : +- CometBroadcastHashJoin (18) - : : : : :- CometProject (14) - : : : : : +- CometBroadcastHashJoin (13) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : +- CometBroadcastExchange (6) - : : : : : : +- CometProject (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : : : : +- CometBroadcastExchange (12) - : : : : : +- CometProject (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : : : : +- CometBroadcastExchange (17) - : : : : +- CometFilter (16) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (15) - : : : +- CometBroadcastExchange (23) - : : : +- CometProject (22) - : : : +- CometFilter (21) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - : : +- CometBroadcastExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) - :- CometHashAggregate (66) - : +- CometExchange (65) - : +- CometHashAggregate (64) - : +- CometProject (63) - : +- CometBroadcastHashJoin (62) - : :- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (57) - : : : +- CometBroadcastHashJoin (56) - : : : :- CometProject (51) - : : : : +- CometBroadcastHashJoin (50) - : : : : :- CometProject (48) - : : : : : +- CometBroadcastHashJoin (47) - : : : : : :- CometProject (45) - : : : : : : +- CometBroadcastHashJoin (44) - : : : : : : :- CometFilter (42) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (41) - : : : : : : +- ReusedExchange (43) - : : : : : +- ReusedExchange (46) - : : : : +- ReusedExchange (49) - : : : +- CometBroadcastExchange (55) - : : : +- CometProject (54) - : : : +- CometFilter (53) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (52) - : : +- ReusedExchange (58) - : +- ReusedExchange (61) - :- CometHashAggregate (92) - : +- CometExchange (91) - : +- CometHashAggregate (90) - : +- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometProject (86) - : : +- CometBroadcastHashJoin (85) - : : :- CometProject (83) - : : : +- CometBroadcastHashJoin (82) - : : : :- CometProject (77) - : : : : +- CometBroadcastHashJoin (76) - : : : : :- CometProject (74) - : : : : : +- CometBroadcastHashJoin (73) - : : : : : :- CometProject (71) - : : : : : : +- CometBroadcastHashJoin (70) - : : : : : : :- CometFilter (68) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) - : : : : : : +- ReusedExchange (69) - : : : : : +- ReusedExchange (72) - : : : : +- ReusedExchange (75) - : : : +- CometBroadcastExchange (81) - : : : +- CometProject (80) - : : : +- CometFilter (79) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (78) - : : +- ReusedExchange (84) - : +- ReusedExchange (87) - :- CometHashAggregate (118) - : +- CometExchange (117) - : +- CometHashAggregate (116) - : +- CometProject (115) - : +- CometBroadcastHashJoin (114) - : :- CometProject (112) - : : +- CometBroadcastHashJoin (111) - : : :- CometProject (109) - : : : +- CometBroadcastHashJoin (108) - : : : :- CometProject (103) - : : : : +- CometBroadcastHashJoin (102) - : : : : :- CometProject (100) - : : : : : +- CometBroadcastHashJoin (99) - : : : : : :- CometProject (97) - : : : : : : +- CometBroadcastHashJoin (96) - : : : : : : :- CometFilter (94) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (93) - : : : : : : +- ReusedExchange (95) - : : : : : +- ReusedExchange (98) - : : : : +- ReusedExchange (101) - : : : +- CometBroadcastExchange (107) - : : : +- CometProject (106) - : : : +- CometFilter (105) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (104) - : : +- ReusedExchange (110) - : +- ReusedExchange (113) - +- CometHashAggregate (143) - +- CometExchange (142) - +- CometHashAggregate (141) - +- CometProject (140) - +- CometBroadcastHashJoin (139) - :- CometProject (135) - : +- CometBroadcastHashJoin (134) - : :- CometProject (132) - : : +- CometBroadcastHashJoin (131) - : : :- CometProject (129) - : : : +- CometBroadcastHashJoin (128) - : : : :- CometProject (126) - : : : : +- CometBroadcastHashJoin (125) - : : : : :- CometProject (123) - : : : : : +- CometBroadcastHashJoin (122) - : : : : : :- CometFilter (120) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (119) - : : : : : +- ReusedExchange (121) - : : : : +- ReusedExchange (124) - : : : +- ReusedExchange (127) - : : +- ReusedExchange (130) - : +- ReusedExchange (133) - +- CometBroadcastExchange (138) - +- CometFilter (137) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (136) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#12, 1)) = M)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#13, 20)) = College )) AND isnotnull(cd_demo_sk#11)) - -(5) CometProject -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] - -(6) CometBroadcastExchange -Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14] - -(7) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Right output [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight - -(8) CometProject -Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [In(c_birth_month, [1,10,12,4,5,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Condition : (((c_birth_month#18 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) - -(11) CometProject -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(12) CometBroadcastExchange -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(13) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight - -(14) CometProject -Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [cd_demo_sk#20] -Condition : isnotnull(cd_demo_sk#20) - -(17) CometBroadcastExchange -Input [1]: [cd_demo_sk#20] -Arguments: [cd_demo_sk#20] - -(18) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Right output [1]: [cd_demo_sk#20] -Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight - -(19) CometProject -Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#23, 2)) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) - -(22) CometProject -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#23, 2)) AS ca_state#25, ca_country#24] - -(23) CometBroadcastExchange -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(24) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -Right output [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight - -(25) CometProject -Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#26)) - -(28) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(29) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] - -(30) CometBroadcastHashJoin -Left output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [1]: [d_date_sk#26] -Arguments: [cs_sold_date_sk#9], [d_date_sk#26], Inner, BuildRight - -(31) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#28, i_item_id#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [i_item_sk#28, i_item_id#29] -Condition : isnotnull(i_item_sk#28) - -(34) CometProject -Input [2]: [i_item_sk#28, i_item_id#29] -Arguments: [i_item_sk#28, i_item_id#30], [i_item_sk#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#29, 16)) AS i_item_id#30] - -(35) CometBroadcastExchange -Input [2]: [i_item_sk#28, i_item_id#30] -Arguments: [i_item_sk#28, i_item_id#30] - -(36) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [2]: [i_item_sk#28, i_item_id#30] -Arguments: [cs_item_sk#3], [i_item_sk#28], Inner, BuildRight - -(37) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#28, i_item_id#30] -Arguments: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41], [i_item_id#30 AS i_item_id#31, ca_country#24 AS ca_country#32, ca_state#25 AS ca_state#33, ca_county#22 AS ca_county#34, cast(cs_quantity#4 as decimal(12,2)) AS agg1#35, cast(cs_list_price#5 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#39, cast(c_birth_year#19 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] - -(38) CometHashAggregate -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [4]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34] -Functions [7]: [partial_avg(agg1#35), partial_avg(agg2#36), partial_avg(agg3#37), partial_avg(agg4#38), partial_avg(agg5#39), partial_avg(agg6#40), partial_avg(agg7#41)] - -(39) CometExchange -Input [18]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] -Arguments: hashpartitioning(i_item_id#31, ca_country#32, ca_state#33, ca_county#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(40) CometHashAggregate -Input [18]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] -Keys [4]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34] -Functions [7]: [avg(agg1#35), avg(agg2#36), avg(agg3#37), avg(agg4#38), avg(agg5#39), avg(agg6#40), avg(agg7#41)] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#64), dynamicpruningexpression(cs_sold_date_sk#64 IN dynamicpruning#65)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(42) CometFilter -Input [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] -Condition : ((isnotnull(cs_bill_cdemo_sk#57) AND isnotnull(cs_bill_customer_sk#56)) AND isnotnull(cs_item_sk#58)) - -(43) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#66, cd_dep_count#67] - -(44) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] -Right output [2]: [cd_demo_sk#66, cd_dep_count#67] -Arguments: [cs_bill_cdemo_sk#57], [cd_demo_sk#66], Inner, BuildRight - -(45) CometProject -Input [11]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_demo_sk#66, cd_dep_count#67] -Arguments: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67], [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67] - -(46) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] - -(47) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67] -Right output [4]: [c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Arguments: [cs_bill_customer_sk#56], [c_customer_sk#68], Inner, BuildRight - -(48) CometProject -Input [13]: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] - -(49) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#72] - -(50) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Right output [1]: [cd_demo_sk#72] -Arguments: [c_current_cdemo_sk#69], [cd_demo_sk#72], Inner, BuildRight - -(51) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71, cd_demo_sk#72] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71] - -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#73, ca_state#74, ca_country#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [ca_address_sk#73, ca_state#74, ca_country#75] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#74, 2)) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#73)) - -(54) CometProject -Input [3]: [ca_address_sk#73, ca_state#74, ca_country#75] -Arguments: [ca_address_sk#73, ca_state#76, ca_country#75], [ca_address_sk#73, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#74, 2)) AS ca_state#76, ca_country#75] - -(55) CometBroadcastExchange -Input [3]: [ca_address_sk#73, ca_state#76, ca_country#75] -Arguments: [ca_address_sk#73, ca_state#76, ca_country#75] - -(56) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71] -Right output [3]: [ca_address_sk#73, ca_state#76, ca_country#75] -Arguments: [c_current_addr_sk#70], [ca_address_sk#73], Inner, BuildRight - -(57) CometProject -Input [13]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71, ca_address_sk#73, ca_state#76, ca_country#75] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#76, ca_country#75], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#76, ca_country#75] - -(58) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#77] - -(59) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#76, ca_country#75] -Right output [1]: [d_date_sk#77] -Arguments: [cs_sold_date_sk#64], [d_date_sk#77], Inner, BuildRight - -(60) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#76, ca_country#75, d_date_sk#77] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#76, ca_country#75], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#76, ca_country#75] - -(61) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#78, i_item_id#79] - -(62) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#76, ca_country#75] -Right output [2]: [i_item_sk#78, i_item_id#79] -Arguments: [cs_item_sk#58], [i_item_sk#78], Inner, BuildRight - -(63) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#76, ca_country#75, i_item_sk#78, i_item_id#79] -Arguments: [i_item_id#79, ca_country#75, ca_state#76, agg1#80, agg2#81, agg3#82, agg4#83, agg5#84, agg6#85, agg7#86], [i_item_id#79, ca_country#75, ca_state#76, cast(cs_quantity#59 as decimal(12,2)) AS agg1#80, cast(cs_list_price#60 as decimal(12,2)) AS agg2#81, cast(cs_coupon_amt#62 as decimal(12,2)) AS agg3#82, cast(cs_sales_price#61 as decimal(12,2)) AS agg4#83, cast(cs_net_profit#63 as decimal(12,2)) AS agg5#84, cast(c_birth_year#71 as decimal(12,2)) AS agg6#85, cast(cd_dep_count#67 as decimal(12,2)) AS agg7#86] - -(64) CometHashAggregate -Input [10]: [i_item_id#79, ca_country#75, ca_state#76, agg1#80, agg2#81, agg3#82, agg4#83, agg5#84, agg6#85, agg7#86] -Keys [3]: [i_item_id#79, ca_country#75, ca_state#76] -Functions [7]: [partial_avg(agg1#80), partial_avg(agg2#81), partial_avg(agg3#82), partial_avg(agg4#83), partial_avg(agg5#84), partial_avg(agg6#85), partial_avg(agg7#86)] - -(65) CometExchange -Input [17]: [i_item_id#79, ca_country#75, ca_state#76, sum#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98, sum#99, count#100] -Arguments: hashpartitioning(i_item_id#79, ca_country#75, ca_state#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(66) CometHashAggregate -Input [17]: [i_item_id#79, ca_country#75, ca_state#76, sum#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98, sum#99, count#100] -Keys [3]: [i_item_id#79, ca_country#75, ca_state#76] -Functions [7]: [avg(agg1#80), avg(agg2#81), avg(agg3#82), avg(agg4#83), avg(agg5#84), avg(agg6#85), avg(agg7#86)] - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#101, cs_bill_cdemo_sk#102, cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#109), dynamicpruningexpression(cs_sold_date_sk#109 IN dynamicpruning#110)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(68) CometFilter -Input [9]: [cs_bill_customer_sk#101, cs_bill_cdemo_sk#102, cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109] -Condition : ((isnotnull(cs_bill_cdemo_sk#102) AND isnotnull(cs_bill_customer_sk#101)) AND isnotnull(cs_item_sk#103)) - -(69) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#111, cd_dep_count#112] - -(70) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#101, cs_bill_cdemo_sk#102, cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109] -Right output [2]: [cd_demo_sk#111, cd_dep_count#112] -Arguments: [cs_bill_cdemo_sk#102], [cd_demo_sk#111], Inner, BuildRight - -(71) CometProject -Input [11]: [cs_bill_customer_sk#101, cs_bill_cdemo_sk#102, cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_demo_sk#111, cd_dep_count#112] -Arguments: [cs_bill_customer_sk#101, cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112], [cs_bill_customer_sk#101, cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112] - -(72) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#113, c_current_cdemo_sk#114, c_current_addr_sk#115, c_birth_year#116] - -(73) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#101, cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112] -Right output [4]: [c_customer_sk#113, c_current_cdemo_sk#114, c_current_addr_sk#115, c_birth_year#116] -Arguments: [cs_bill_customer_sk#101], [c_customer_sk#113], Inner, BuildRight - -(74) CometProject -Input [13]: [cs_bill_customer_sk#101, cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_customer_sk#113, c_current_cdemo_sk#114, c_current_addr_sk#115, c_birth_year#116] -Arguments: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_current_cdemo_sk#114, c_current_addr_sk#115, c_birth_year#116], [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_current_cdemo_sk#114, c_current_addr_sk#115, c_birth_year#116] - -(75) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#117] - -(76) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_current_cdemo_sk#114, c_current_addr_sk#115, c_birth_year#116] -Right output [1]: [cd_demo_sk#117] -Arguments: [c_current_cdemo_sk#114], [cd_demo_sk#117], Inner, BuildRight - -(77) CometProject -Input [12]: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_current_cdemo_sk#114, c_current_addr_sk#115, c_birth_year#116, cd_demo_sk#117] -Arguments: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_current_addr_sk#115, c_birth_year#116], [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_current_addr_sk#115, c_birth_year#116] - -(78) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#118, ca_state#119, ca_country#120] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(79) CometFilter -Input [3]: [ca_address_sk#118, ca_state#119, ca_country#120] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#119, 2)) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#118)) - -(80) CometProject -Input [3]: [ca_address_sk#118, ca_state#119, ca_country#120] -Arguments: [ca_address_sk#118, ca_country#120], [ca_address_sk#118, ca_country#120] - -(81) CometBroadcastExchange -Input [2]: [ca_address_sk#118, ca_country#120] -Arguments: [ca_address_sk#118, ca_country#120] - -(82) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_current_addr_sk#115, c_birth_year#116] -Right output [2]: [ca_address_sk#118, ca_country#120] -Arguments: [c_current_addr_sk#115], [ca_address_sk#118], Inner, BuildRight - -(83) CometProject -Input [12]: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_current_addr_sk#115, c_birth_year#116, ca_address_sk#118, ca_country#120] -Arguments: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_birth_year#116, ca_country#120], [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_birth_year#116, ca_country#120] - -(84) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#121] - -(85) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_birth_year#116, ca_country#120] -Right output [1]: [d_date_sk#121] -Arguments: [cs_sold_date_sk#109], [d_date_sk#121], Inner, BuildRight - -(86) CometProject -Input [11]: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cs_sold_date_sk#109, cd_dep_count#112, c_birth_year#116, ca_country#120, d_date_sk#121] -Arguments: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cd_dep_count#112, c_birth_year#116, ca_country#120], [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cd_dep_count#112, c_birth_year#116, ca_country#120] - -(87) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#122, i_item_id#123] - -(88) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cd_dep_count#112, c_birth_year#116, ca_country#120] -Right output [2]: [i_item_sk#122, i_item_id#123] -Arguments: [cs_item_sk#103], [i_item_sk#122], Inner, BuildRight - -(89) CometProject -Input [11]: [cs_item_sk#103, cs_quantity#104, cs_list_price#105, cs_sales_price#106, cs_coupon_amt#107, cs_net_profit#108, cd_dep_count#112, c_birth_year#116, ca_country#120, i_item_sk#122, i_item_id#123] -Arguments: [i_item_id#123, ca_country#120, agg1#124, agg2#125, agg3#126, agg4#127, agg5#128, agg6#129, agg7#130], [i_item_id#123, ca_country#120, cast(cs_quantity#104 as decimal(12,2)) AS agg1#124, cast(cs_list_price#105 as decimal(12,2)) AS agg2#125, cast(cs_coupon_amt#107 as decimal(12,2)) AS agg3#126, cast(cs_sales_price#106 as decimal(12,2)) AS agg4#127, cast(cs_net_profit#108 as decimal(12,2)) AS agg5#128, cast(c_birth_year#116 as decimal(12,2)) AS agg6#129, cast(cd_dep_count#112 as decimal(12,2)) AS agg7#130] - -(90) CometHashAggregate -Input [9]: [i_item_id#123, ca_country#120, agg1#124, agg2#125, agg3#126, agg4#127, agg5#128, agg6#129, agg7#130] -Keys [2]: [i_item_id#123, ca_country#120] -Functions [7]: [partial_avg(agg1#124), partial_avg(agg2#125), partial_avg(agg3#126), partial_avg(agg4#127), partial_avg(agg5#128), partial_avg(agg6#129), partial_avg(agg7#130)] - -(91) CometExchange -Input [16]: [i_item_id#123, ca_country#120, sum#131, count#132, sum#133, count#134, sum#135, count#136, sum#137, count#138, sum#139, count#140, sum#141, count#142, sum#143, count#144] -Arguments: hashpartitioning(i_item_id#123, ca_country#120, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(92) CometHashAggregate -Input [16]: [i_item_id#123, ca_country#120, sum#131, count#132, sum#133, count#134, sum#135, count#136, sum#137, count#138, sum#139, count#140, sum#141, count#142, sum#143, count#144] -Keys [2]: [i_item_id#123, ca_country#120] -Functions [7]: [avg(agg1#124), avg(agg2#125), avg(agg3#126), avg(agg4#127), avg(agg5#128), avg(agg6#129), avg(agg7#130)] - -(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#145, cs_bill_cdemo_sk#146, cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#153), dynamicpruningexpression(cs_sold_date_sk#153 IN dynamicpruning#154)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(94) CometFilter -Input [9]: [cs_bill_customer_sk#145, cs_bill_cdemo_sk#146, cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153] -Condition : ((isnotnull(cs_bill_cdemo_sk#146) AND isnotnull(cs_bill_customer_sk#145)) AND isnotnull(cs_item_sk#147)) - -(95) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#155, cd_dep_count#156] - -(96) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#145, cs_bill_cdemo_sk#146, cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153] -Right output [2]: [cd_demo_sk#155, cd_dep_count#156] -Arguments: [cs_bill_cdemo_sk#146], [cd_demo_sk#155], Inner, BuildRight - -(97) CometProject -Input [11]: [cs_bill_customer_sk#145, cs_bill_cdemo_sk#146, cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_demo_sk#155, cd_dep_count#156] -Arguments: [cs_bill_customer_sk#145, cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156], [cs_bill_customer_sk#145, cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156] - -(98) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#157, c_current_cdemo_sk#158, c_current_addr_sk#159, c_birth_year#160] - -(99) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#145, cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156] -Right output [4]: [c_customer_sk#157, c_current_cdemo_sk#158, c_current_addr_sk#159, c_birth_year#160] -Arguments: [cs_bill_customer_sk#145], [c_customer_sk#157], Inner, BuildRight - -(100) CometProject -Input [13]: [cs_bill_customer_sk#145, cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_customer_sk#157, c_current_cdemo_sk#158, c_current_addr_sk#159, c_birth_year#160] -Arguments: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_current_cdemo_sk#158, c_current_addr_sk#159, c_birth_year#160], [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_current_cdemo_sk#158, c_current_addr_sk#159, c_birth_year#160] - -(101) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#161] - -(102) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_current_cdemo_sk#158, c_current_addr_sk#159, c_birth_year#160] -Right output [1]: [cd_demo_sk#161] -Arguments: [c_current_cdemo_sk#158], [cd_demo_sk#161], Inner, BuildRight - -(103) CometProject -Input [12]: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_current_cdemo_sk#158, c_current_addr_sk#159, c_birth_year#160, cd_demo_sk#161] -Arguments: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_current_addr_sk#159, c_birth_year#160], [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_current_addr_sk#159, c_birth_year#160] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#162, ca_state#163] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [ca_address_sk#162, ca_state#163] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#163, 2)) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#162)) - -(106) CometProject -Input [2]: [ca_address_sk#162, ca_state#163] -Arguments: [ca_address_sk#162], [ca_address_sk#162] - -(107) CometBroadcastExchange -Input [1]: [ca_address_sk#162] -Arguments: [ca_address_sk#162] - -(108) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_current_addr_sk#159, c_birth_year#160] -Right output [1]: [ca_address_sk#162] -Arguments: [c_current_addr_sk#159], [ca_address_sk#162], Inner, BuildRight - -(109) CometProject -Input [11]: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_current_addr_sk#159, c_birth_year#160, ca_address_sk#162] -Arguments: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_birth_year#160], [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_birth_year#160] - -(110) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#164] - -(111) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_birth_year#160] -Right output [1]: [d_date_sk#164] -Arguments: [cs_sold_date_sk#153], [d_date_sk#164], Inner, BuildRight - -(112) CometProject -Input [10]: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cs_sold_date_sk#153, cd_dep_count#156, c_birth_year#160, d_date_sk#164] -Arguments: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cd_dep_count#156, c_birth_year#160], [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cd_dep_count#156, c_birth_year#160] - -(113) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#165, i_item_id#166] - -(114) CometBroadcastHashJoin -Left output [8]: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cd_dep_count#156, c_birth_year#160] -Right output [2]: [i_item_sk#165, i_item_id#166] -Arguments: [cs_item_sk#147], [i_item_sk#165], Inner, BuildRight - -(115) CometProject -Input [10]: [cs_item_sk#147, cs_quantity#148, cs_list_price#149, cs_sales_price#150, cs_coupon_amt#151, cs_net_profit#152, cd_dep_count#156, c_birth_year#160, i_item_sk#165, i_item_id#166] -Arguments: [i_item_id#166, agg1#167, agg2#168, agg3#169, agg4#170, agg5#171, agg6#172, agg7#173], [i_item_id#166, cast(cs_quantity#148 as decimal(12,2)) AS agg1#167, cast(cs_list_price#149 as decimal(12,2)) AS agg2#168, cast(cs_coupon_amt#151 as decimal(12,2)) AS agg3#169, cast(cs_sales_price#150 as decimal(12,2)) AS agg4#170, cast(cs_net_profit#152 as decimal(12,2)) AS agg5#171, cast(c_birth_year#160 as decimal(12,2)) AS agg6#172, cast(cd_dep_count#156 as decimal(12,2)) AS agg7#173] - -(116) CometHashAggregate -Input [8]: [i_item_id#166, agg1#167, agg2#168, agg3#169, agg4#170, agg5#171, agg6#172, agg7#173] -Keys [1]: [i_item_id#166] -Functions [7]: [partial_avg(agg1#167), partial_avg(agg2#168), partial_avg(agg3#169), partial_avg(agg4#170), partial_avg(agg5#171), partial_avg(agg6#172), partial_avg(agg7#173)] - -(117) CometExchange -Input [15]: [i_item_id#166, sum#174, count#175, sum#176, count#177, sum#178, count#179, sum#180, count#181, sum#182, count#183, sum#184, count#185, sum#186, count#187] -Arguments: hashpartitioning(i_item_id#166, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(118) CometHashAggregate -Input [15]: [i_item_id#166, sum#174, count#175, sum#176, count#177, sum#178, count#179, sum#180, count#181, sum#182, count#183, sum#184, count#185, sum#186, count#187] -Keys [1]: [i_item_id#166] -Functions [7]: [avg(agg1#167), avg(agg2#168), avg(agg3#169), avg(agg4#170), avg(agg5#171), avg(agg6#172), avg(agg7#173)] - -(119) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#188, cs_bill_cdemo_sk#189, cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#196), dynamicpruningexpression(cs_sold_date_sk#196 IN dynamicpruning#197)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(120) CometFilter -Input [9]: [cs_bill_customer_sk#188, cs_bill_cdemo_sk#189, cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196] -Condition : ((isnotnull(cs_bill_cdemo_sk#189) AND isnotnull(cs_bill_customer_sk#188)) AND isnotnull(cs_item_sk#190)) - -(121) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#198, cd_dep_count#199] - -(122) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#188, cs_bill_cdemo_sk#189, cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196] -Right output [2]: [cd_demo_sk#198, cd_dep_count#199] -Arguments: [cs_bill_cdemo_sk#189], [cd_demo_sk#198], Inner, BuildRight - -(123) CometProject -Input [11]: [cs_bill_customer_sk#188, cs_bill_cdemo_sk#189, cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_demo_sk#198, cd_dep_count#199] -Arguments: [cs_bill_customer_sk#188, cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199], [cs_bill_customer_sk#188, cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199] - -(124) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#200, c_current_cdemo_sk#201, c_current_addr_sk#202, c_birth_year#203] - -(125) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#188, cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199] -Right output [4]: [c_customer_sk#200, c_current_cdemo_sk#201, c_current_addr_sk#202, c_birth_year#203] -Arguments: [cs_bill_customer_sk#188], [c_customer_sk#200], Inner, BuildRight - -(126) CometProject -Input [13]: [cs_bill_customer_sk#188, cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_customer_sk#200, c_current_cdemo_sk#201, c_current_addr_sk#202, c_birth_year#203] -Arguments: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_current_cdemo_sk#201, c_current_addr_sk#202, c_birth_year#203], [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_current_cdemo_sk#201, c_current_addr_sk#202, c_birth_year#203] - -(127) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#204] - -(128) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_current_cdemo_sk#201, c_current_addr_sk#202, c_birth_year#203] -Right output [1]: [cd_demo_sk#204] -Arguments: [c_current_cdemo_sk#201], [cd_demo_sk#204], Inner, BuildRight - -(129) CometProject -Input [12]: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_current_cdemo_sk#201, c_current_addr_sk#202, c_birth_year#203, cd_demo_sk#204] -Arguments: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_current_addr_sk#202, c_birth_year#203], [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_current_addr_sk#202, c_birth_year#203] - -(130) ReusedExchange [Reuses operator id: 107] -Output [1]: [ca_address_sk#205] - -(131) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_current_addr_sk#202, c_birth_year#203] -Right output [1]: [ca_address_sk#205] -Arguments: [c_current_addr_sk#202], [ca_address_sk#205], Inner, BuildRight - -(132) CometProject -Input [11]: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_current_addr_sk#202, c_birth_year#203, ca_address_sk#205] -Arguments: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_birth_year#203], [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_birth_year#203] - -(133) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#206] - -(134) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_birth_year#203] -Right output [1]: [d_date_sk#206] -Arguments: [cs_sold_date_sk#196], [d_date_sk#206], Inner, BuildRight - -(135) CometProject -Input [10]: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cs_sold_date_sk#196, cd_dep_count#199, c_birth_year#203, d_date_sk#206] -Arguments: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cd_dep_count#199, c_birth_year#203], [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cd_dep_count#199, c_birth_year#203] - -(136) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#207] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(137) CometFilter -Input [1]: [i_item_sk#207] -Condition : isnotnull(i_item_sk#207) - -(138) CometBroadcastExchange -Input [1]: [i_item_sk#207] -Arguments: [i_item_sk#207] - -(139) CometBroadcastHashJoin -Left output [8]: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cd_dep_count#199, c_birth_year#203] -Right output [1]: [i_item_sk#207] -Arguments: [cs_item_sk#190], [i_item_sk#207], Inner, BuildRight - -(140) CometProject -Input [9]: [cs_item_sk#190, cs_quantity#191, cs_list_price#192, cs_sales_price#193, cs_coupon_amt#194, cs_net_profit#195, cd_dep_count#199, c_birth_year#203, i_item_sk#207] -Arguments: [agg1#208, agg2#209, agg3#210, agg4#211, agg5#212, agg6#213, agg7#214], [cast(cs_quantity#191 as decimal(12,2)) AS agg1#208, cast(cs_list_price#192 as decimal(12,2)) AS agg2#209, cast(cs_coupon_amt#194 as decimal(12,2)) AS agg3#210, cast(cs_sales_price#193 as decimal(12,2)) AS agg4#211, cast(cs_net_profit#195 as decimal(12,2)) AS agg5#212, cast(c_birth_year#203 as decimal(12,2)) AS agg6#213, cast(cd_dep_count#199 as decimal(12,2)) AS agg7#214] - -(141) CometHashAggregate -Input [7]: [agg1#208, agg2#209, agg3#210, agg4#211, agg5#212, agg6#213, agg7#214] -Keys: [] -Functions [7]: [partial_avg(agg1#208), partial_avg(agg2#209), partial_avg(agg3#210), partial_avg(agg4#211), partial_avg(agg5#212), partial_avg(agg6#213), partial_avg(agg7#214)] - -(142) CometExchange -Input [14]: [sum#215, count#216, sum#217, count#218, sum#219, count#220, sum#221, count#222, sum#223, count#224, sum#225, count#226, sum#227, count#228] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(143) CometHashAggregate -Input [14]: [sum#215, count#216, sum#217, count#218, sum#219, count#220, sum#221, count#222, sum#223, count#224, sum#225, count#226, sum#227, count#228] -Keys: [] -Functions [7]: [avg(agg1#208), avg(agg2#209), avg(agg3#210), avg(agg4#211), avg(agg5#212), avg(agg6#213), avg(agg7#214)] - -(144) CometUnion -Child 0 Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#229, agg2#230, agg3#231, agg4#232, agg5#233, agg6#234, agg7#235] -Child 1 Input [11]: [i_item_id#79, ca_country#75, ca_state#76, county#236, agg1#237, agg2#238, agg3#239, agg4#240, agg5#241, agg6#242, agg7#243] -Child 2 Input [11]: [i_item_id#123, ca_country#120, ca_state#244, county#245, agg1#246, agg2#247, agg3#248, agg4#249, agg5#250, agg6#251, agg7#252] -Child 3 Input [11]: [i_item_id#166, ca_country#253, ca_state#254, county#255, agg1#256, agg2#257, agg3#258, agg4#259, agg5#260, agg6#261, agg7#262] -Child 4 Input [11]: [i_item_id#263, ca_country#264, ca_state#265, county#266, agg1#267, agg2#268, agg3#269, agg4#270, agg5#271, agg6#272, agg7#273] - -(145) CometTakeOrderedAndProject -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#229, agg2#230, agg3#231, agg4#232, agg5#233, agg6#234, agg7#235] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#32 ASC NULLS FIRST,ca_state#33 ASC NULLS FIRST,ca_county#34 ASC NULLS FIRST,i_item_id#31 ASC NULLS FIRST], output=[i_item_id#31,ca_country#32,ca_state#33,ca_county#34,agg1#229,agg2#230,agg3#231,agg4#232,agg5#233,agg6#234,agg7#235]), [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#229, agg2#230, agg3#231, agg4#232, agg5#233, agg6#234, agg7#235], 100, 0, [ca_country#32 ASC NULLS FIRST, ca_state#33 ASC NULLS FIRST, ca_county#34 ASC NULLS FIRST, i_item_id#31 ASC NULLS FIRST], [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#229, agg2#230, agg3#231, agg4#232, agg5#233, agg6#234, agg7#235] - -(146) CometColumnarToRow [codegen id : 1] -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#229, agg2#230, agg3#231, agg4#232, agg5#233, agg6#234, agg7#235] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (151) -+- * CometColumnarToRow (150) - +- CometProject (149) - +- CometFilter (148) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (147) - - -(147) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(148) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#26)) - -(149) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(150) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(151) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#64 IN dynamicpruning#10 - -Subquery:3 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#109 IN dynamicpruning#10 - -Subquery:4 Hosting operator id = 93 Hosting Expression = cs_sold_date_sk#153 IN dynamicpruning#10 - -Subquery:5 Hosting operator id = 119 Hosting Expression = cs_sold_date_sk#196 IN dynamicpruning#10 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/extended.txt deleted file mode 100644 index b18a444bb9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/extended.txt +++ /dev/null @@ -1,214 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 204 out of 210 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 1572a2a240..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt +++ /dev/null @@ -1,160 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometUnion [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country,ca_state,ca_county] #1 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometProject [i_item_id,ca_country,ca_state,ca_county,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,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_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 [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 [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,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_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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - 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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange [cd_demo_sk] #5 - CometFilter [cd_demo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #8 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country,ca_state] #9 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 [ca_address_sk,ca_state,ca_country] #10 - CometProject [ca_state] [ca_address_sk,ca_state,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country] #11 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 [ca_address_sk,ca_country] #12 - CometProject [ca_address_sk,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id] #13 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 [ca_address_sk] #14 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - ReusedExchange [d_date_sk] #7 - ReusedExchange [i_item_sk,i_item_id] #8 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange #15 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 - ReusedExchange [ca_address_sk] #14 - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [i_item_sk] #16 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/explain.txt deleted file mode 100644 index 4da3e0a758..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/explain.txt +++ /dev/null @@ -1,163 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.item (4) - +- ReusedExchange (11) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] - -(7) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(8) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(11) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] - -(14) HashAggregate [codegen id : 3] -Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(15) CometColumnarExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(17) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#17] -Results [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#17,17,2) AS _w0#19] - -(18) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(22) Project [codegen id : 6] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] - -(23) TakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometNativeScan parquet spark_catalog.default.date_dim (24) - - -(24) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#22] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/simplified.txt deleted file mode 100644 index 7bc0779e53..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/explain.txt deleted file mode 100644 index 2e7abc07bb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [cs_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/simplified.txt deleted file mode 100644 index cf18e68a3d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt deleted file mode 100644 index 2e7abc07bb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [cs_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/extended.txt deleted file mode 100644 index cd52b2cd12..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index cf18e68a3d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/explain.txt deleted file mode 100644 index 0a6d891afc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/explain.txt +++ /dev/null @@ -1,170 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (24) -+- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometColumnarExchange (21) - +- * HashAggregate (20) - +- * Expand (19) - +- * Project (18) - +- * BroadcastNestedLoopJoin Inner BuildRight (17) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.inventory (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) - : +- CometProject (9) - : +- CometFilter (8) - : +- CometNativeScan parquet spark_catalog.default.item (7) - +- BroadcastExchange (16) - +- * CometColumnarToRow (15) - +- CometNativeScan parquet spark_catalog.default.warehouse (14) - - -(1) Scan parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#3), dynamicpruningexpression(inv_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(inv_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] - -(3) Filter [codegen id : 4] -Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Condition : isnotnull(inv_item_sk#1) - -(4) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#5] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] -Input [4]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3, d_date_sk#5] - -(7) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(8) CometFilter -Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] -Condition : isnotnull(i_item_sk#6) - -(9) CometProject -Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] -Arguments: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14], [i_item_sk#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#7, 50)) AS i_brand#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#8, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#9, 50)) AS i_category#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#10, 50)) AS i_product_name#14] - -(10) CometColumnarToRow [codegen id : 2] -Input [5]: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] - -(11) BroadcastExchange -Input [5]: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#2, i_brand#11, i_class#12, i_category#13, i_product_name#14] -Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] - -(14) CometNativeScan parquet spark_catalog.default.warehouse -Output: [] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -ReadSchema: struct<> - -(15) CometColumnarToRow [codegen id : 3] -Input: [] - -(16) BroadcastExchange -Input: [] -Arguments: IdentityBroadcastMode, [plan_id=2] - -(17) BroadcastNestedLoopJoin [codegen id : 4] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13] -Input [5]: [inv_quantity_on_hand#2, i_brand#11, i_class#12, i_category#13, i_product_name#14] - -(19) Expand [codegen id : 4] -Input [5]: [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13] -Arguments: [[inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13, 0], [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, null, 1], [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, null, null, 3], [inv_quantity_on_hand#2, i_product_name#14, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] - -(20) HashAggregate [codegen id : 4] -Input [6]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] -Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] -Functions [1]: [partial_avg(inv_quantity_on_hand#2)] -Aggregate Attributes [2]: [sum#20, count#21] -Results [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] - -(21) CometColumnarExchange -Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] -Arguments: hashpartitioning(i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] - -(23) HashAggregate [codegen id : 5] -Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] -Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] -Functions [1]: [avg(inv_quantity_on_hand#2)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#24] -Results [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, avg(inv_quantity_on_hand#2)#24 AS qoh#25] - -(24) TakeOrderedAndProject -Input [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25] -Arguments: 100, [qoh#25 ASC NULLS FIRST, i_product_name#15 ASC NULLS FIRST, i_brand#16 ASC NULLS FIRST, i_class#17 ASC NULLS FIRST, i_category#18 ASC NULLS FIRST], [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometNativeScan parquet spark_catalog.default.date_dim (25) - - -(25) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#26] -Condition : (((isnotnull(d_month_seq#26) AND (d_month_seq#26 >= 1200)) AND (d_month_seq#26 <= 1211)) AND isnotnull(d_date_sk#5)) - -(27) CometProject -Input [2]: [d_date_sk#5, d_month_seq#26] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(29) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/simplified.txt deleted file mode 100644 index 457dff46e2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (5) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastNestedLoopJoin - Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Filter [inv_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometNativeScan parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/explain.txt deleted file mode 100644 index 1f6f727512..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (26) -+- * HashAggregate (25) - +- * CometColumnarToRow (24) - +- CometColumnarExchange (23) - +- * HashAggregate (22) - +- * Expand (21) - +- * Project (20) - +- * BroadcastNestedLoopJoin Inner BuildRight (19) - :- * CometColumnarToRow (15) - : +- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - +- BroadcastExchange (18) - +- * CometColumnarToRow (17) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (16) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#3), dynamicpruningexpression(inv_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(inv_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Condition : isnotnull(inv_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [inv_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3, d_date_sk#5] -Arguments: [inv_item_sk#1, inv_quantity_on_hand#2], [inv_item_sk#1, inv_quantity_on_hand#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Condition : isnotnull(i_item_sk#7) - -(11) CometProject -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#8, 50)) AS i_brand#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#11, 50)) AS i_product_name#15] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(13) CometBroadcastHashJoin -Left output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] -Right output [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15], [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(15) CometColumnarToRow [codegen id : 2] -Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output: [] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -ReadSchema: struct<> - -(17) CometColumnarToRow [codegen id : 1] -Input: [] - -(18) BroadcastExchange -Input: [] -Arguments: IdentityBroadcastMode, [plan_id=1] - -(19) BroadcastNestedLoopJoin [codegen id : 2] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 2] -Output [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] -Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(21) Expand [codegen id : 2] -Input [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] -Arguments: [[inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14, 0], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, null, 1], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, null, null, 3], [inv_quantity_on_hand#2, i_product_name#15, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] - -(22) HashAggregate [codegen id : 2] -Input [6]: [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Functions [1]: [partial_avg(inv_quantity_on_hand#2)] -Aggregate Attributes [2]: [sum#21, count#22] -Results [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] - -(23) CometColumnarExchange -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -Arguments: hashpartitioning(i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(24) CometColumnarToRow [codegen id : 3] -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] - -(25) HashAggregate [codegen id : 3] -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Functions [1]: [avg(inv_quantity_on_hand#2)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#25] -Results [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, avg(inv_quantity_on_hand#2)#25 AS qoh#26] - -(26) TakeOrderedAndProject -Input [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#26] -Arguments: 100, [qoh#26 ASC NULLS FIRST, i_product_name#16 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_category#19 ASC NULLS FIRST], [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(29) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(31) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/simplified.txt deleted file mode 100644 index 221c6063ce..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (3) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (2) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - 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_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/explain.txt deleted file mode 100644 index 1f6f727512..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (26) -+- * HashAggregate (25) - +- * CometColumnarToRow (24) - +- CometColumnarExchange (23) - +- * HashAggregate (22) - +- * Expand (21) - +- * Project (20) - +- * BroadcastNestedLoopJoin Inner BuildRight (19) - :- * CometColumnarToRow (15) - : +- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - +- BroadcastExchange (18) - +- * CometColumnarToRow (17) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (16) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#3), dynamicpruningexpression(inv_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(inv_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Condition : isnotnull(inv_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [inv_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3, d_date_sk#5] -Arguments: [inv_item_sk#1, inv_quantity_on_hand#2], [inv_item_sk#1, inv_quantity_on_hand#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Condition : isnotnull(i_item_sk#7) - -(11) CometProject -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#8, 50)) AS i_brand#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#11, 50)) AS i_product_name#15] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(13) CometBroadcastHashJoin -Left output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] -Right output [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15], [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(15) CometColumnarToRow [codegen id : 2] -Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output: [] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -ReadSchema: struct<> - -(17) CometColumnarToRow [codegen id : 1] -Input: [] - -(18) BroadcastExchange -Input: [] -Arguments: IdentityBroadcastMode, [plan_id=1] - -(19) BroadcastNestedLoopJoin [codegen id : 2] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 2] -Output [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] -Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(21) Expand [codegen id : 2] -Input [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] -Arguments: [[inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14, 0], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, null, 1], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, null, null, 3], [inv_quantity_on_hand#2, i_product_name#15, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] - -(22) HashAggregate [codegen id : 2] -Input [6]: [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Functions [1]: [partial_avg(inv_quantity_on_hand#2)] -Aggregate Attributes [2]: [sum#21, count#22] -Results [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] - -(23) CometColumnarExchange -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -Arguments: hashpartitioning(i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(24) CometColumnarToRow [codegen id : 3] -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] - -(25) HashAggregate [codegen id : 3] -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Functions [1]: [avg(inv_quantity_on_hand#2)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#25] -Results [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, avg(inv_quantity_on_hand#2)#25 AS qoh#26] - -(26) TakeOrderedAndProject -Input [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#26] -Arguments: 100, [qoh#26 ASC NULLS FIRST, i_product_name#16 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_category#19 ASC NULLS FIRST], [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(29) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(31) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/extended.txt deleted file mode 100644 index 6cb89465d2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/extended.txt +++ /dev/null @@ -1,34 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - -Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 221c6063ce..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (3) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (2) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - 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_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/explain.txt deleted file mode 100644 index 9842213949..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/explain.txt +++ /dev/null @@ -1,356 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (55) -+- Union (54) - :- * HashAggregate (25) - : +- * HashAggregate (24) - : +- * HashAggregate (23) - : +- * CometColumnarToRow (22) - : +- CometColumnarExchange (21) - : +- * HashAggregate (20) - : +- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.inventory (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.item (7) - : +- BroadcastExchange (17) - : +- * CometColumnarToRow (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.warehouse (14) - :- * HashAggregate (32) - : +- * CometColumnarToRow (31) - : +- CometColumnarExchange (30) - : +- * HashAggregate (29) - : +- * HashAggregate (28) - : +- * CometColumnarToRow (27) - : +- ReusedExchange (26) - :- * HashAggregate (39) - : +- * CometColumnarToRow (38) - : +- CometColumnarExchange (37) - : +- * HashAggregate (36) - : +- * HashAggregate (35) - : +- * CometColumnarToRow (34) - : +- ReusedExchange (33) - :- * HashAggregate (46) - : +- * CometColumnarToRow (45) - : +- CometColumnarExchange (44) - : +- * HashAggregate (43) - : +- * HashAggregate (42) - : +- * CometColumnarToRow (41) - : +- ReusedExchange (40) - +- * HashAggregate (53) - +- * CometColumnarToRow (52) - +- CometColumnarExchange (51) - +- * HashAggregate (50) - +- * HashAggregate (49) - +- * CometColumnarToRow (48) - +- ReusedExchange (47) - - -(1) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(4) ReusedExchange [Reuses operator id: 60] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(8) CometFilter -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Condition : isnotnull(i_item_sk#7) - -(9) CometProject -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#8, 50)) AS i_brand#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#11, 50)) AS i_product_name#15] - -(10) CometColumnarToRow [codegen id : 2] -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(11) BroadcastExchange -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(14) CometNativeScan parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(15) CometFilter -Input [1]: [w_warehouse_sk#16] -Condition : isnotnull(w_warehouse_sk#16) - -(16) CometColumnarToRow [codegen id : 3] -Input [1]: [w_warehouse_sk#16] - -(17) BroadcastExchange -Input [1]: [w_warehouse_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#16] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15, w_warehouse_sk#16] - -(20) HashAggregate [codegen id : 4] -Input [5]: [inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [sum#17, count#18] -Results [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] - -(21) CometColumnarExchange -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] -Arguments: hashpartitioning(i_product_name#15, i_brand#12, i_class#13, i_category#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] - -(23) HashAggregate [codegen id : 5] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#21] -Results [5]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, avg(inv_quantity_on_hand#3)#21 AS qoh#22] - -(24) HashAggregate [codegen id : 5] -Input [5]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, qoh#22] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [partial_avg(qoh#22)] -Aggregate Attributes [2]: [sum#23, count#24] -Results [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#25, count#26] - -(25) HashAggregate [codegen id : 5] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#25, count#26] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [avg(qoh#22)] -Aggregate Attributes [1]: [avg(qoh#22)#27] -Results [5]: [i_product_name#15 AS i_product_name#28, i_brand#12 AS i_brand#29, i_class#13 AS i_class#30, i_category#14 AS i_category#31, avg(qoh#22)#27 AS qoh#32] - -(26) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#33, i_brand#34, i_class#35, i_category#36, sum#37, count#38] - -(27) CometColumnarToRow [codegen id : 10] -Input [6]: [i_product_name#33, i_brand#34, i_class#35, i_category#36, sum#37, count#38] - -(28) HashAggregate [codegen id : 10] -Input [6]: [i_product_name#33, i_brand#34, i_class#35, i_category#36, sum#37, count#38] -Keys [4]: [i_product_name#33, i_brand#34, i_class#35, i_category#36] -Functions [1]: [avg(inv_quantity_on_hand#39)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#39)#21] -Results [4]: [i_product_name#33, i_brand#34, i_class#35, avg(inv_quantity_on_hand#39)#21 AS qoh#40] - -(29) HashAggregate [codegen id : 10] -Input [4]: [i_product_name#33, i_brand#34, i_class#35, qoh#40] -Keys [3]: [i_product_name#33, i_brand#34, i_class#35] -Functions [1]: [partial_avg(qoh#40)] -Aggregate Attributes [2]: [sum#41, count#42] -Results [5]: [i_product_name#33, i_brand#34, i_class#35, sum#43, count#44] - -(30) CometColumnarExchange -Input [5]: [i_product_name#33, i_brand#34, i_class#35, sum#43, count#44] -Arguments: hashpartitioning(i_product_name#33, i_brand#34, i_class#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(31) CometColumnarToRow [codegen id : 11] -Input [5]: [i_product_name#33, i_brand#34, i_class#35, sum#43, count#44] - -(32) HashAggregate [codegen id : 11] -Input [5]: [i_product_name#33, i_brand#34, i_class#35, sum#43, count#44] -Keys [3]: [i_product_name#33, i_brand#34, i_class#35] -Functions [1]: [avg(qoh#40)] -Aggregate Attributes [1]: [avg(qoh#40)#45] -Results [5]: [i_product_name#33, i_brand#34, i_class#35, null AS i_category#46, avg(qoh#40)#45 AS qoh#47] - -(33) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#48, i_brand#49, i_class#50, i_category#51, sum#52, count#53] - -(34) CometColumnarToRow [codegen id : 16] -Input [6]: [i_product_name#48, i_brand#49, i_class#50, i_category#51, sum#52, count#53] - -(35) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#48, i_brand#49, i_class#50, i_category#51, sum#52, count#53] -Keys [4]: [i_product_name#48, i_brand#49, i_class#50, i_category#51] -Functions [1]: [avg(inv_quantity_on_hand#54)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#54)#21] -Results [3]: [i_product_name#48, i_brand#49, avg(inv_quantity_on_hand#54)#21 AS qoh#55] - -(36) HashAggregate [codegen id : 16] -Input [3]: [i_product_name#48, i_brand#49, qoh#55] -Keys [2]: [i_product_name#48, i_brand#49] -Functions [1]: [partial_avg(qoh#55)] -Aggregate Attributes [2]: [sum#56, count#57] -Results [4]: [i_product_name#48, i_brand#49, sum#58, count#59] - -(37) CometColumnarExchange -Input [4]: [i_product_name#48, i_brand#49, sum#58, count#59] -Arguments: hashpartitioning(i_product_name#48, i_brand#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(38) CometColumnarToRow [codegen id : 17] -Input [4]: [i_product_name#48, i_brand#49, sum#58, count#59] - -(39) HashAggregate [codegen id : 17] -Input [4]: [i_product_name#48, i_brand#49, sum#58, count#59] -Keys [2]: [i_product_name#48, i_brand#49] -Functions [1]: [avg(qoh#55)] -Aggregate Attributes [1]: [avg(qoh#55)#60] -Results [5]: [i_product_name#48, i_brand#49, null AS i_class#61, null AS i_category#62, avg(qoh#55)#60 AS qoh#63] - -(40) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#64, i_brand#65, i_class#66, i_category#67, sum#68, count#69] - -(41) CometColumnarToRow [codegen id : 22] -Input [6]: [i_product_name#64, i_brand#65, i_class#66, i_category#67, sum#68, count#69] - -(42) HashAggregate [codegen id : 22] -Input [6]: [i_product_name#64, i_brand#65, i_class#66, i_category#67, sum#68, count#69] -Keys [4]: [i_product_name#64, i_brand#65, i_class#66, i_category#67] -Functions [1]: [avg(inv_quantity_on_hand#70)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#70)#21] -Results [2]: [i_product_name#64, avg(inv_quantity_on_hand#70)#21 AS qoh#71] - -(43) HashAggregate [codegen id : 22] -Input [2]: [i_product_name#64, qoh#71] -Keys [1]: [i_product_name#64] -Functions [1]: [partial_avg(qoh#71)] -Aggregate Attributes [2]: [sum#72, count#73] -Results [3]: [i_product_name#64, sum#74, count#75] - -(44) CometColumnarExchange -Input [3]: [i_product_name#64, sum#74, count#75] -Arguments: hashpartitioning(i_product_name#64, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(45) CometColumnarToRow [codegen id : 23] -Input [3]: [i_product_name#64, sum#74, count#75] - -(46) HashAggregate [codegen id : 23] -Input [3]: [i_product_name#64, sum#74, count#75] -Keys [1]: [i_product_name#64] -Functions [1]: [avg(qoh#71)] -Aggregate Attributes [1]: [avg(qoh#71)#76] -Results [5]: [i_product_name#64, null AS i_brand#77, null AS i_class#78, null AS i_category#79, avg(qoh#71)#76 AS qoh#80] - -(47) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#81, i_brand#82, i_class#83, i_category#84, sum#85, count#86] - -(48) CometColumnarToRow [codegen id : 28] -Input [6]: [i_product_name#81, i_brand#82, i_class#83, i_category#84, sum#85, count#86] - -(49) HashAggregate [codegen id : 28] -Input [6]: [i_product_name#81, i_brand#82, i_class#83, i_category#84, sum#85, count#86] -Keys [4]: [i_product_name#81, i_brand#82, i_class#83, i_category#84] -Functions [1]: [avg(inv_quantity_on_hand#87)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#87)#21] -Results [1]: [avg(inv_quantity_on_hand#87)#21 AS qoh#88] - -(50) HashAggregate [codegen id : 28] -Input [1]: [qoh#88] -Keys: [] -Functions [1]: [partial_avg(qoh#88)] -Aggregate Attributes [2]: [sum#89, count#90] -Results [2]: [sum#91, count#92] - -(51) CometColumnarExchange -Input [2]: [sum#91, count#92] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(52) CometColumnarToRow [codegen id : 29] -Input [2]: [sum#91, count#92] - -(53) HashAggregate [codegen id : 29] -Input [2]: [sum#91, count#92] -Keys: [] -Functions [1]: [avg(qoh#88)] -Aggregate Attributes [1]: [avg(qoh#88)#93] -Results [5]: [null AS i_product_name#94, null AS i_brand#95, null AS i_class#96, null AS i_category#97, avg(qoh#88)#93 AS qoh#98] - -(54) Union - -(55) TakeOrderedAndProject -Input [5]: [i_product_name#28, i_brand#29, i_class#30, i_category#31, qoh#32] -Arguments: 100, [qoh#32 ASC NULLS FIRST, i_product_name#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, i_class#30 ASC NULLS FIRST, i_category#31 ASC NULLS FIRST], [i_product_name#28, i_brand#29, i_class#30, i_category#31, qoh#32] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (60) -+- * CometColumnarToRow (59) - +- CometProject (58) - +- CometFilter (57) - +- CometNativeScan parquet spark_catalog.default.date_dim (56) - - -(56) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#99] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(57) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#99] -Condition : (((isnotnull(d_month_seq#99) AND (d_month_seq#99 >= 1212)) AND (d_month_seq#99 <= 1223)) AND isnotnull(d_date_sk#6)) - -(58) CometProject -Input [2]: [d_date_sk#6, d_month_seq#99] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(59) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(60) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/simplified.txt deleted file mode 100644 index 239424b0a0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - Union - WholeStageCodegen (5) - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category] #1 - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] - Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk] - WholeStageCodegen (11) - HashAggregate [i_product_name,i_brand,i_class,sum,count] [avg(qoh),i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class] #5 - WholeStageCodegen (10) - HashAggregate [i_product_name,i_brand,i_class,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (17) - HashAggregate [i_product_name,i_brand,sum,count] [avg(qoh),i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand] #6 - WholeStageCodegen (16) - HashAggregate [i_product_name,i_brand,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (23) - HashAggregate [i_product_name,sum,count] [avg(qoh),i_brand,i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name] #7 - WholeStageCodegen (22) - HashAggregate [i_product_name,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (29) - HashAggregate [sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (28) - HashAggregate [qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/explain.txt deleted file mode 100644 index b1912021ee..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,301 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometTakeOrderedAndProject (46) - +- CometUnion (45) - :- CometHashAggregate (24) - : +- CometHashAggregate (23) - : +- CometHashAggregate (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - : +- CometBroadcastExchange (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (15) - :- CometHashAggregate (29) - : +- CometExchange (28) - : +- CometHashAggregate (27) - : +- CometHashAggregate (26) - : +- ReusedExchange (25) - :- CometHashAggregate (34) - : +- CometExchange (33) - : +- CometHashAggregate (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - :- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometHashAggregate (36) - : +- ReusedExchange (35) - +- CometHashAggregate (44) - +- CometExchange (43) - +- CometHashAggregate (42) - +- CometHashAggregate (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [inv_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] -Arguments: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3], [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Condition : isnotnull(i_item_sk#8) - -(11) CometProject -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16], [i_item_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#9, 50)) AS i_brand#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#10, 50)) AS i_class#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#11, 50)) AS i_category#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#12, 50)) AS i_product_name#16] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(13) CometBroadcastHashJoin -Left output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -Right output [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight - -(14) CometProject -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [w_warehouse_sk#17] -Condition : isnotnull(w_warehouse_sk#17) - -(17) CometBroadcastExchange -Input [1]: [w_warehouse_sk#17] -Arguments: [w_warehouse_sk#17] - -(18) CometBroadcastHashJoin -Left output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Right output [1]: [w_warehouse_sk#17] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#17], Inner, BuildRight - -(19) CometProject -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16, w_warehouse_sk#17] -Arguments: [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(20) CometHashAggregate -Input [5]: [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [partial_avg(inv_quantity_on_hand#3)] - -(21) CometExchange -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#18, count#19] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, i_category#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#18, count#19] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#3)] - -(23) CometHashAggregate -Input [5]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, qoh#20] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [partial_avg(qoh#20)] - -(24) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#21, count#22] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(qoh#20)] - -(25) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#23, i_brand#24, i_class#25, i_category#26, sum#27, count#28] - -(26) CometHashAggregate -Input [6]: [i_product_name#23, i_brand#24, i_class#25, i_category#26, sum#27, count#28] -Keys [4]: [i_product_name#23, i_brand#24, i_class#25, i_category#26] -Functions [1]: [avg(inv_quantity_on_hand#29)] - -(27) CometHashAggregate -Input [4]: [i_product_name#23, i_brand#24, i_class#25, qoh#30] -Keys [3]: [i_product_name#23, i_brand#24, i_class#25] -Functions [1]: [partial_avg(qoh#30)] - -(28) CometExchange -Input [5]: [i_product_name#23, i_brand#24, i_class#25, sum#31, count#32] -Arguments: hashpartitioning(i_product_name#23, i_brand#24, i_class#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [5]: [i_product_name#23, i_brand#24, i_class#25, sum#31, count#32] -Keys [3]: [i_product_name#23, i_brand#24, i_class#25] -Functions [1]: [avg(qoh#30)] - -(30) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#33, i_brand#34, i_class#35, i_category#36, sum#37, count#38] - -(31) CometHashAggregate -Input [6]: [i_product_name#33, i_brand#34, i_class#35, i_category#36, sum#37, count#38] -Keys [4]: [i_product_name#33, i_brand#34, i_class#35, i_category#36] -Functions [1]: [avg(inv_quantity_on_hand#39)] - -(32) CometHashAggregate -Input [3]: [i_product_name#33, i_brand#34, qoh#40] -Keys [2]: [i_product_name#33, i_brand#34] -Functions [1]: [partial_avg(qoh#40)] - -(33) CometExchange -Input [4]: [i_product_name#33, i_brand#34, sum#41, count#42] -Arguments: hashpartitioning(i_product_name#33, i_brand#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(34) CometHashAggregate -Input [4]: [i_product_name#33, i_brand#34, sum#41, count#42] -Keys [2]: [i_product_name#33, i_brand#34] -Functions [1]: [avg(qoh#40)] - -(35) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#43, i_brand#44, i_class#45, i_category#46, sum#47, count#48] - -(36) CometHashAggregate -Input [6]: [i_product_name#43, i_brand#44, i_class#45, i_category#46, sum#47, count#48] -Keys [4]: [i_product_name#43, i_brand#44, i_class#45, i_category#46] -Functions [1]: [avg(inv_quantity_on_hand#49)] - -(37) CometHashAggregate -Input [2]: [i_product_name#43, qoh#50] -Keys [1]: [i_product_name#43] -Functions [1]: [partial_avg(qoh#50)] - -(38) CometExchange -Input [3]: [i_product_name#43, sum#51, count#52] -Arguments: hashpartitioning(i_product_name#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(39) CometHashAggregate -Input [3]: [i_product_name#43, sum#51, count#52] -Keys [1]: [i_product_name#43] -Functions [1]: [avg(qoh#50)] - -(40) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#53, i_brand#54, i_class#55, i_category#56, sum#57, count#58] - -(41) CometHashAggregate -Input [6]: [i_product_name#53, i_brand#54, i_class#55, i_category#56, sum#57, count#58] -Keys [4]: [i_product_name#53, i_brand#54, i_class#55, i_category#56] -Functions [1]: [avg(inv_quantity_on_hand#59)] - -(42) CometHashAggregate -Input [1]: [qoh#60] -Keys: [] -Functions [1]: [partial_avg(qoh#60)] - -(43) CometExchange -Input [2]: [sum#61, count#62] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(44) CometHashAggregate -Input [2]: [sum#61, count#62] -Keys: [] -Functions [1]: [avg(qoh#60)] - -(45) CometUnion -Child 0 Input [5]: [i_product_name#63, i_brand#64, i_class#65, i_category#66, qoh#67] -Child 1 Input [5]: [i_product_name#23, i_brand#24, i_class#25, i_category#68, qoh#69] -Child 2 Input [5]: [i_product_name#33, i_brand#34, i_class#70, i_category#71, qoh#72] -Child 3 Input [5]: [i_product_name#43, i_brand#73, i_class#74, i_category#75, qoh#76] -Child 4 Input [5]: [i_product_name#77, i_brand#78, i_class#79, i_category#80, qoh#81] - -(46) CometTakeOrderedAndProject -Input [5]: [i_product_name#63, i_brand#64, i_class#65, i_category#66, qoh#67] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#67 ASC NULLS FIRST,i_product_name#63 ASC NULLS FIRST,i_brand#64 ASC NULLS FIRST,i_class#65 ASC NULLS FIRST,i_category#66 ASC NULLS FIRST], output=[i_product_name#63,i_brand#64,i_class#65,i_category#66,qoh#67]), [i_product_name#63, i_brand#64, i_class#65, i_category#66, qoh#67], 100, 0, [qoh#67 ASC NULLS FIRST, i_product_name#63 ASC NULLS FIRST, i_brand#64 ASC NULLS FIRST, i_class#65 ASC NULLS FIRST, i_category#66 ASC NULLS FIRST], [i_product_name#63, i_brand#64, i_class#65, i_category#66, qoh#67] - -(47) CometColumnarToRow [codegen id : 1] -Input [5]: [i_product_name#63, i_brand#64, i_class#65, i_category#66, qoh#67] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) - -(50) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(52) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/simplified.txt deleted file mode 100644 index d68adf3747..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,57 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] - CometUnion [i_product_name,i_brand,i_class,i_category,qoh] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,i_product_name,i_brand,i_class,i_category,avg(qoh)] - CometHashAggregate [qoh] [i_product_name,i_brand,i_class,i_category,sum,count] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(inv_quantity_on_hand)] - CometExchange [i_product_name,i_brand,i_class,i_category] #1 - CometHashAggregate [inv_quantity_on_hand] [i_product_name,i_brand,i_class,i_category,sum,count] - CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - 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,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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange [w_warehouse_sk] #5 - CometFilter [w_warehouse_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name,i_brand,i_class] #6 - CometHashAggregate [qoh] [i_product_name,i_brand,i_class,sum,count] - CometHashAggregate [i_category,sum,count] [i_product_name,i_brand,i_class,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name,i_brand] #7 - CometHashAggregate [qoh] [i_product_name,i_brand,sum,count] - CometHashAggregate [i_class,i_category,sum,count] [i_product_name,i_brand,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name] #8 - CometHashAggregate [qoh] [i_product_name,sum,count] - CometHashAggregate [i_brand,i_class,i_category,sum,count] [i_product_name,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange #9 - CometHashAggregate [qoh] [sum,count] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/explain.txt deleted file mode 100644 index b1912021ee..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/explain.txt +++ /dev/null @@ -1,301 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometTakeOrderedAndProject (46) - +- CometUnion (45) - :- CometHashAggregate (24) - : +- CometHashAggregate (23) - : +- CometHashAggregate (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - : +- CometBroadcastExchange (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (15) - :- CometHashAggregate (29) - : +- CometExchange (28) - : +- CometHashAggregate (27) - : +- CometHashAggregate (26) - : +- ReusedExchange (25) - :- CometHashAggregate (34) - : +- CometExchange (33) - : +- CometHashAggregate (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - :- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometHashAggregate (36) - : +- ReusedExchange (35) - +- CometHashAggregate (44) - +- CometExchange (43) - +- CometHashAggregate (42) - +- CometHashAggregate (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [inv_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] -Arguments: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3], [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Condition : isnotnull(i_item_sk#8) - -(11) CometProject -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16], [i_item_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#9, 50)) AS i_brand#13, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#10, 50)) AS i_class#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#11, 50)) AS i_category#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#12, 50)) AS i_product_name#16] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(13) CometBroadcastHashJoin -Left output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -Right output [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight - -(14) CometProject -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [w_warehouse_sk#17] -Condition : isnotnull(w_warehouse_sk#17) - -(17) CometBroadcastExchange -Input [1]: [w_warehouse_sk#17] -Arguments: [w_warehouse_sk#17] - -(18) CometBroadcastHashJoin -Left output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Right output [1]: [w_warehouse_sk#17] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#17], Inner, BuildRight - -(19) CometProject -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16, w_warehouse_sk#17] -Arguments: [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(20) CometHashAggregate -Input [5]: [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [partial_avg(inv_quantity_on_hand#3)] - -(21) CometExchange -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#18, count#19] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, i_category#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#18, count#19] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#3)] - -(23) CometHashAggregate -Input [5]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, qoh#20] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [partial_avg(qoh#20)] - -(24) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#21, count#22] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(qoh#20)] - -(25) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#23, i_brand#24, i_class#25, i_category#26, sum#27, count#28] - -(26) CometHashAggregate -Input [6]: [i_product_name#23, i_brand#24, i_class#25, i_category#26, sum#27, count#28] -Keys [4]: [i_product_name#23, i_brand#24, i_class#25, i_category#26] -Functions [1]: [avg(inv_quantity_on_hand#29)] - -(27) CometHashAggregate -Input [4]: [i_product_name#23, i_brand#24, i_class#25, qoh#30] -Keys [3]: [i_product_name#23, i_brand#24, i_class#25] -Functions [1]: [partial_avg(qoh#30)] - -(28) CometExchange -Input [5]: [i_product_name#23, i_brand#24, i_class#25, sum#31, count#32] -Arguments: hashpartitioning(i_product_name#23, i_brand#24, i_class#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [5]: [i_product_name#23, i_brand#24, i_class#25, sum#31, count#32] -Keys [3]: [i_product_name#23, i_brand#24, i_class#25] -Functions [1]: [avg(qoh#30)] - -(30) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#33, i_brand#34, i_class#35, i_category#36, sum#37, count#38] - -(31) CometHashAggregate -Input [6]: [i_product_name#33, i_brand#34, i_class#35, i_category#36, sum#37, count#38] -Keys [4]: [i_product_name#33, i_brand#34, i_class#35, i_category#36] -Functions [1]: [avg(inv_quantity_on_hand#39)] - -(32) CometHashAggregate -Input [3]: [i_product_name#33, i_brand#34, qoh#40] -Keys [2]: [i_product_name#33, i_brand#34] -Functions [1]: [partial_avg(qoh#40)] - -(33) CometExchange -Input [4]: [i_product_name#33, i_brand#34, sum#41, count#42] -Arguments: hashpartitioning(i_product_name#33, i_brand#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(34) CometHashAggregate -Input [4]: [i_product_name#33, i_brand#34, sum#41, count#42] -Keys [2]: [i_product_name#33, i_brand#34] -Functions [1]: [avg(qoh#40)] - -(35) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#43, i_brand#44, i_class#45, i_category#46, sum#47, count#48] - -(36) CometHashAggregate -Input [6]: [i_product_name#43, i_brand#44, i_class#45, i_category#46, sum#47, count#48] -Keys [4]: [i_product_name#43, i_brand#44, i_class#45, i_category#46] -Functions [1]: [avg(inv_quantity_on_hand#49)] - -(37) CometHashAggregate -Input [2]: [i_product_name#43, qoh#50] -Keys [1]: [i_product_name#43] -Functions [1]: [partial_avg(qoh#50)] - -(38) CometExchange -Input [3]: [i_product_name#43, sum#51, count#52] -Arguments: hashpartitioning(i_product_name#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(39) CometHashAggregate -Input [3]: [i_product_name#43, sum#51, count#52] -Keys [1]: [i_product_name#43] -Functions [1]: [avg(qoh#50)] - -(40) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#53, i_brand#54, i_class#55, i_category#56, sum#57, count#58] - -(41) CometHashAggregate -Input [6]: [i_product_name#53, i_brand#54, i_class#55, i_category#56, sum#57, count#58] -Keys [4]: [i_product_name#53, i_brand#54, i_class#55, i_category#56] -Functions [1]: [avg(inv_quantity_on_hand#59)] - -(42) CometHashAggregate -Input [1]: [qoh#60] -Keys: [] -Functions [1]: [partial_avg(qoh#60)] - -(43) CometExchange -Input [2]: [sum#61, count#62] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(44) CometHashAggregate -Input [2]: [sum#61, count#62] -Keys: [] -Functions [1]: [avg(qoh#60)] - -(45) CometUnion -Child 0 Input [5]: [i_product_name#63, i_brand#64, i_class#65, i_category#66, qoh#67] -Child 1 Input [5]: [i_product_name#23, i_brand#24, i_class#25, i_category#68, qoh#69] -Child 2 Input [5]: [i_product_name#33, i_brand#34, i_class#70, i_category#71, qoh#72] -Child 3 Input [5]: [i_product_name#43, i_brand#73, i_class#74, i_category#75, qoh#76] -Child 4 Input [5]: [i_product_name#77, i_brand#78, i_class#79, i_category#80, qoh#81] - -(46) CometTakeOrderedAndProject -Input [5]: [i_product_name#63, i_brand#64, i_class#65, i_category#66, qoh#67] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#67 ASC NULLS FIRST,i_product_name#63 ASC NULLS FIRST,i_brand#64 ASC NULLS FIRST,i_class#65 ASC NULLS FIRST,i_category#66 ASC NULLS FIRST], output=[i_product_name#63,i_brand#64,i_class#65,i_category#66,qoh#67]), [i_product_name#63, i_brand#64, i_class#65, i_category#66, qoh#67], 100, 0, [qoh#67 ASC NULLS FIRST, i_product_name#63 ASC NULLS FIRST, i_brand#64 ASC NULLS FIRST, i_class#65 ASC NULLS FIRST, i_category#66 ASC NULLS FIRST], [i_product_name#63, i_brand#64, i_class#65, i_category#66, qoh#67] - -(47) CometColumnarToRow [codegen id : 1] -Input [5]: [i_product_name#63, i_brand#64, i_class#65, i_category#66, qoh#67] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) - -(50) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(52) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/extended.txt deleted file mode 100644 index c018673888..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/extended.txt +++ /dev/null @@ -1,159 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - -Comet accelerated 141 out of 151 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d68adf3747..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt +++ /dev/null @@ -1,57 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] - CometUnion [i_product_name,i_brand,i_class,i_category,qoh] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,i_product_name,i_brand,i_class,i_category,avg(qoh)] - CometHashAggregate [qoh] [i_product_name,i_brand,i_class,i_category,sum,count] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(inv_quantity_on_hand)] - CometExchange [i_product_name,i_brand,i_class,i_category] #1 - CometHashAggregate [inv_quantity_on_hand] [i_product_name,i_brand,i_class,i_category,sum,count] - CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - 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,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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange [w_warehouse_sk] #5 - CometFilter [w_warehouse_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name,i_brand,i_class] #6 - CometHashAggregate [qoh] [i_product_name,i_brand,i_class,sum,count] - CometHashAggregate [i_category,sum,count] [i_product_name,i_brand,i_class,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name,i_brand] #7 - CometHashAggregate [qoh] [i_product_name,i_brand,sum,count] - CometHashAggregate [i_class,i_category,sum,count] [i_product_name,i_brand,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name] #8 - CometHashAggregate [qoh] [i_product_name,sum,count] - CometHashAggregate [i_brand,i_class,i_category,sum,count] [i_product_name,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange #9 - CometHashAggregate [qoh] [sum,count] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_datafusion/explain.txt deleted file mode 100644 index 6ca8ff436b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_datafusion/explain.txt +++ /dev/null @@ -1,453 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (50) -+- CometSort (49) - +- CometColumnarExchange (48) - +- * Filter (47) - +- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometNativeScan parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometNativeScan parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.customer_address (32) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometNativeScan parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#14, 10)))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#13, 2)) AS s_state#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#14, 10)) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((isnotnull(i_color#20) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#20, 20)) = pale )) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#19, 20)) AS i_size#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#20, 20)) AS i_color#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#21, 10)) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometNativeScan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Condition : ((isnotnull(c_customer_sk#26) AND isnotnull(c_current_addr_sk#27)) AND isnotnull(c_birth_country#30)) - -(27) CometProject -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [c_customer_sk#26, c_current_addr_sk#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#28, 20)) AS c_first_name#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#29, 30)) AS c_last_name#32, c_birth_country#30] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(31) CometColumnarToRow [codegen id : 2] -Input [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(32) CometNativeScan parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Condition : ((isnotnull(ca_address_sk#33) AND isnotnull(ca_country#36)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#35, 10)))) - -(34) CometProject -Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Arguments: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36], [ca_address_sk#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#34, 2)) AS ca_state#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#35, 10)) AS ca_zip#38, ca_country#36] - -(35) CometColumnarToRow [codegen id : 1] -Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] - -(36) BroadcastExchange -Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [3]: [c_current_addr_sk#27, c_birth_country#30, s_zip#16] -Right keys [3]: [ca_address_sk#33, upper(ca_country#36), ca_zip#38] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] -Input [17]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30, ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#39] -Results [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] - -(40) CometColumnarExchange -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#41] -Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#41,17,2) AS netpaid#42] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, netpaid#42] -Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] -Functions [1]: [partial_sum(netpaid#42)] -Aggregate Attributes [2]: [sum#43, isEmpty#44] -Results [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] - -(44) CometColumnarExchange -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] -Functions [1]: [sum(netpaid#42)] -Aggregate Attributes [1]: [sum(netpaid#42)#47] -Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, sum(netpaid#42)#47 AS paid#48] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) - -(48) CometColumnarExchange -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(49) CometSort -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Arguments: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48], [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] - -(50) CometColumnarToRow [codegen id : 5] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#49, [id=#50] -* HashAggregate (80) -+- * CometColumnarToRow (79) - +- CometColumnarExchange (78) - +- * HashAggregate (77) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- CometColumnarExchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * CometColumnarToRow (69) - : +- CometProject (68) - : +- CometBroadcastHashJoin (67) - : :- CometProject (65) - : : +- CometBroadcastHashJoin (64) - : : :- CometProject (59) - : : : +- CometBroadcastHashJoin (58) - : : : :- CometProject (56) - : : : : +- CometSortMergeJoin (55) - : : : : :- CometSort (52) - : : : : : +- ReusedExchange (51) - : : : : +- CometSort (54) - : : : : +- ReusedExchange (53) - : : : +- ReusedExchange (57) - : : +- CometBroadcastExchange (63) - : : +- CometProject (62) - : : +- CometFilter (61) - : : +- CometNativeScan parquet spark_catalog.default.item (60) - : +- ReusedExchange (66) - +- ReusedExchange (70) - - -(51) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] - -(52) CometSort -Input [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55], [ss_ticket_number#54 ASC NULLS FIRST, ss_item_sk#51 ASC NULLS FIRST] - -(53) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#56, sr_ticket_number#57] - -(54) CometSort -Input [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [sr_item_sk#56, sr_ticket_number#57], [sr_ticket_number#57 ASC NULLS FIRST, sr_item_sk#56 ASC NULLS FIRST] - -(55) CometSortMergeJoin -Left output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -Right output [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [ss_ticket_number#54, ss_item_sk#51], [sr_ticket_number#57, sr_item_sk#56], Inner - -(56) CometProject -Input [7]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55, sr_item_sk#56, sr_ticket_number#57] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55], [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] - -(57) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#58, s_store_name#59, s_state#60, s_zip#61] - -(58) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] -Right output [4]: [s_store_sk#58, s_store_name#59, s_state#60, s_zip#61] -Arguments: [ss_store_sk#53], [s_store_sk#58], Inner, BuildRight - -(59) CometProject -Input [8]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55, s_store_sk#58, s_store_name#59, s_state#60, s_zip#61] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61], [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61] - -(60) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(61) CometFilter -Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Condition : isnotnull(i_item_sk#62) - -(62) CometProject -Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Arguments: [i_item_sk#62, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67], [i_item_sk#62, i_current_price#63, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#64, 20)) AS i_size#68, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#65, 20)) AS i_color#69, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#66, 10)) AS i_units#70, i_manager_id#67] - -(63) CometBroadcastExchange -Input [6]: [i_item_sk#62, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67] -Arguments: [i_item_sk#62, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67] - -(64) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61] -Right output [6]: [i_item_sk#62, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67] -Arguments: [ss_item_sk#51], [i_item_sk#62], Inner, BuildRight - -(65) CometProject -Input [12]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_item_sk#62, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67] -Arguments: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67], [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67] - -(66) ReusedExchange [Reuses operator id: 28] -Output [5]: [c_customer_sk#71, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] - -(67) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67] -Right output [5]: [c_customer_sk#71, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] -Arguments: [ss_customer_sk#52], [c_customer_sk#71], Inner, BuildRight - -(68) CometProject -Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_customer_sk#71, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] -Arguments: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75], [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] - -(69) CometColumnarToRow [codegen id : 2] -Input [13]: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] - -(70) ReusedExchange [Reuses operator id: 36] -Output [4]: [ca_address_sk#76, ca_state#77, ca_zip#78, ca_country#79] - -(71) BroadcastHashJoin [codegen id : 2] -Left keys [3]: [c_current_addr_sk#72, c_birth_country#75, s_zip#61] -Right keys [3]: [ca_address_sk#76, upper(ca_country#79), ca_zip#78] -Join type: Inner -Join condition: None - -(72) Project [codegen id : 2] -Output [11]: [ss_net_paid#55, s_store_name#59, s_state#60, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_first_name#73, c_last_name#74, ca_state#77] -Input [17]: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75, ca_address_sk#76, ca_state#77, ca_zip#78, ca_country#79] - -(73) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#55, s_store_name#59, s_state#60, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_first_name#73, c_last_name#74, ca_state#77] -Keys [10]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [sum#80] -Results [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, sum#81] - -(74) CometColumnarExchange -Input [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, sum#81] -Arguments: hashpartitioning(c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(75) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, sum#81] - -(76) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, sum#81] -Keys [10]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68] -Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#41] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#41,17,2) AS netpaid#82] - -(77) HashAggregate [codegen id : 3] -Input [1]: [netpaid#82] -Keys: [] -Functions [1]: [partial_avg(netpaid#82)] -Aggregate Attributes [2]: [sum#83, count#84] -Results [2]: [sum#85, count#86] - -(78) CometColumnarExchange -Input [2]: [sum#85, count#86] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(79) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#85, count#86] - -(80) HashAggregate [codegen id : 4] -Input [2]: [sum#85, count#86] -Keys: [] -Functions [1]: [avg(netpaid#82)] -Aggregate Attributes [1]: [avg(netpaid#82)#87] -Results [1]: [(0.05 * avg(netpaid#82)#87) AS (0.05 * avg(netpaid))#88] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_datafusion/simplified.txt deleted file mode 100644 index b24e51723a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_datafusion/simplified.txt +++ /dev/null @@ -1,99 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,s_store_name,paid] - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #5 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #12 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - InputAdapter - ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #4 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #5 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #7 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometNativeScan 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 (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] - CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] - CometNativeScan 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/q24.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/explain.txt deleted file mode 100644 index 54ebedf7d3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,453 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (50) -+- CometSort (49) - +- CometColumnarExchange (48) - +- * Filter (47) - +- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#14, 10)))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#13, 2)) AS s_state#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#14, 10)) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((isnotnull(i_color#20) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#20, 20)) = pale )) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#19, 20)) AS i_size#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#20, 20)) AS i_color#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#21, 10)) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Condition : ((isnotnull(c_customer_sk#26) AND isnotnull(c_current_addr_sk#27)) AND isnotnull(c_birth_country#30)) - -(27) CometProject -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [c_customer_sk#26, c_current_addr_sk#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#28, 20)) AS c_first_name#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#29, 30)) AS c_last_name#32, c_birth_country#30] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(31) CometColumnarToRow [codegen id : 2] -Input [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Condition : ((isnotnull(ca_address_sk#33) AND isnotnull(ca_country#36)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#35, 10)))) - -(34) CometProject -Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Arguments: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36], [ca_address_sk#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#34, 2)) AS ca_state#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#35, 10)) AS ca_zip#38, ca_country#36] - -(35) CometColumnarToRow [codegen id : 1] -Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] - -(36) BroadcastExchange -Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [3]: [c_current_addr_sk#27, c_birth_country#30, s_zip#16] -Right keys [3]: [ca_address_sk#33, upper(ca_country#36), ca_zip#38] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] -Input [17]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30, ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#39] -Results [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] - -(40) CometColumnarExchange -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#41] -Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#41,17,2) AS netpaid#42] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, netpaid#42] -Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] -Functions [1]: [partial_sum(netpaid#42)] -Aggregate Attributes [2]: [sum#43, isEmpty#44] -Results [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] - -(44) CometColumnarExchange -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] -Functions [1]: [sum(netpaid#42)] -Aggregate Attributes [1]: [sum(netpaid#42)#47] -Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, sum(netpaid#42)#47 AS paid#48] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) - -(48) CometColumnarExchange -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(49) CometSort -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Arguments: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48], [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] - -(50) CometColumnarToRow [codegen id : 5] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#49, [id=#50] -* HashAggregate (80) -+- * CometColumnarToRow (79) - +- CometColumnarExchange (78) - +- * HashAggregate (77) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- CometColumnarExchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * CometColumnarToRow (69) - : +- CometProject (68) - : +- CometBroadcastHashJoin (67) - : :- CometProject (65) - : : +- CometBroadcastHashJoin (64) - : : :- CometProject (59) - : : : +- CometBroadcastHashJoin (58) - : : : :- CometProject (56) - : : : : +- CometSortMergeJoin (55) - : : : : :- CometSort (52) - : : : : : +- ReusedExchange (51) - : : : : +- CometSort (54) - : : : : +- ReusedExchange (53) - : : : +- ReusedExchange (57) - : : +- CometBroadcastExchange (63) - : : +- CometProject (62) - : : +- CometFilter (61) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (60) - : +- ReusedExchange (66) - +- ReusedExchange (70) - - -(51) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] - -(52) CometSort -Input [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55], [ss_ticket_number#54 ASC NULLS FIRST, ss_item_sk#51 ASC NULLS FIRST] - -(53) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#56, sr_ticket_number#57] - -(54) CometSort -Input [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [sr_item_sk#56, sr_ticket_number#57], [sr_ticket_number#57 ASC NULLS FIRST, sr_item_sk#56 ASC NULLS FIRST] - -(55) CometSortMergeJoin -Left output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -Right output [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [ss_ticket_number#54, ss_item_sk#51], [sr_ticket_number#57, sr_item_sk#56], Inner - -(56) CometProject -Input [7]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55, sr_item_sk#56, sr_ticket_number#57] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55], [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] - -(57) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#58, s_store_name#59, s_state#60, s_zip#61] - -(58) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] -Right output [4]: [s_store_sk#58, s_store_name#59, s_state#60, s_zip#61] -Arguments: [ss_store_sk#53], [s_store_sk#58], Inner, BuildRight - -(59) CometProject -Input [8]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55, s_store_sk#58, s_store_name#59, s_state#60, s_zip#61] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61], [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61] - -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(61) CometFilter -Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Condition : isnotnull(i_item_sk#62) - -(62) CometProject -Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Arguments: [i_item_sk#62, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67], [i_item_sk#62, i_current_price#63, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#64, 20)) AS i_size#68, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#65, 20)) AS i_color#69, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#66, 10)) AS i_units#70, i_manager_id#67] - -(63) CometBroadcastExchange -Input [6]: [i_item_sk#62, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67] -Arguments: [i_item_sk#62, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67] - -(64) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61] -Right output [6]: [i_item_sk#62, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67] -Arguments: [ss_item_sk#51], [i_item_sk#62], Inner, BuildRight - -(65) CometProject -Input [12]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_item_sk#62, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67] -Arguments: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67], [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67] - -(66) ReusedExchange [Reuses operator id: 28] -Output [5]: [c_customer_sk#71, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] - -(67) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67] -Right output [5]: [c_customer_sk#71, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] -Arguments: [ss_customer_sk#52], [c_customer_sk#71], Inner, BuildRight - -(68) CometProject -Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_customer_sk#71, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] -Arguments: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75], [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] - -(69) CometColumnarToRow [codegen id : 2] -Input [13]: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] - -(70) ReusedExchange [Reuses operator id: 36] -Output [4]: [ca_address_sk#76, ca_state#77, ca_zip#78, ca_country#79] - -(71) BroadcastHashJoin [codegen id : 2] -Left keys [3]: [c_current_addr_sk#72, c_birth_country#75, s_zip#61] -Right keys [3]: [ca_address_sk#76, upper(ca_country#79), ca_zip#78] -Join type: Inner -Join condition: None - -(72) Project [codegen id : 2] -Output [11]: [ss_net_paid#55, s_store_name#59, s_state#60, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_first_name#73, c_last_name#74, ca_state#77] -Input [17]: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75, ca_address_sk#76, ca_state#77, ca_zip#78, ca_country#79] - -(73) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#55, s_store_name#59, s_state#60, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_first_name#73, c_last_name#74, ca_state#77] -Keys [10]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [sum#80] -Results [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, sum#81] - -(74) CometColumnarExchange -Input [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, sum#81] -Arguments: hashpartitioning(c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(75) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, sum#81] - -(76) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, sum#81] -Keys [10]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68] -Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#41] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#41,17,2) AS netpaid#82] - -(77) HashAggregate [codegen id : 3] -Input [1]: [netpaid#82] -Keys: [] -Functions [1]: [partial_avg(netpaid#82)] -Aggregate Attributes [2]: [sum#83, count#84] -Results [2]: [sum#85, count#86] - -(78) CometColumnarExchange -Input [2]: [sum#85, count#86] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(79) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#85, count#86] - -(80) HashAggregate [codegen id : 4] -Input [2]: [sum#85, count#86] -Keys: [] -Functions [1]: [avg(netpaid#82)] -Aggregate Attributes [1]: [avg(netpaid#82)#87] -Results [1]: [(0.05 * avg(netpaid#82)#87) AS (0.05 * avg(netpaid))#88] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/simplified.txt deleted file mode 100644 index 62f492f632..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,99 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,s_store_name,paid] - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #5 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #12 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - InputAdapter - ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #4 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #5 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #7 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometScan [native_iceberg_compat] 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 (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] - CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] 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/q24/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/explain.txt deleted file mode 100644 index 54ebedf7d3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/explain.txt +++ /dev/null @@ -1,453 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (50) -+- CometSort (49) - +- CometColumnarExchange (48) - +- * Filter (47) - +- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#14, 10)))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#13, 2)) AS s_state#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#14, 10)) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((isnotnull(i_color#20) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#20, 20)) = pale )) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#19, 20)) AS i_size#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#20, 20)) AS i_color#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#21, 10)) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Condition : ((isnotnull(c_customer_sk#26) AND isnotnull(c_current_addr_sk#27)) AND isnotnull(c_birth_country#30)) - -(27) CometProject -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [c_customer_sk#26, c_current_addr_sk#27, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#28, 20)) AS c_first_name#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#29, 30)) AS c_last_name#32, c_birth_country#30] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(31) CometColumnarToRow [codegen id : 2] -Input [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Condition : ((isnotnull(ca_address_sk#33) AND isnotnull(ca_country#36)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#35, 10)))) - -(34) CometProject -Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Arguments: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36], [ca_address_sk#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#34, 2)) AS ca_state#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#35, 10)) AS ca_zip#38, ca_country#36] - -(35) CometColumnarToRow [codegen id : 1] -Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] - -(36) BroadcastExchange -Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [3]: [c_current_addr_sk#27, c_birth_country#30, s_zip#16] -Right keys [3]: [ca_address_sk#33, upper(ca_country#36), ca_zip#38] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] -Input [17]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30, ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#39] -Results [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] - -(40) CometColumnarExchange -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#41] -Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#41,17,2) AS netpaid#42] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, netpaid#42] -Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] -Functions [1]: [partial_sum(netpaid#42)] -Aggregate Attributes [2]: [sum#43, isEmpty#44] -Results [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] - -(44) CometColumnarExchange -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] -Functions [1]: [sum(netpaid#42)] -Aggregate Attributes [1]: [sum(netpaid#42)#47] -Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, sum(netpaid#42)#47 AS paid#48] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) - -(48) CometColumnarExchange -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(49) CometSort -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Arguments: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48], [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] - -(50) CometColumnarToRow [codegen id : 5] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#49, [id=#50] -* HashAggregate (80) -+- * CometColumnarToRow (79) - +- CometColumnarExchange (78) - +- * HashAggregate (77) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- CometColumnarExchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * CometColumnarToRow (69) - : +- CometProject (68) - : +- CometBroadcastHashJoin (67) - : :- CometProject (65) - : : +- CometBroadcastHashJoin (64) - : : :- CometProject (59) - : : : +- CometBroadcastHashJoin (58) - : : : :- CometProject (56) - : : : : +- CometSortMergeJoin (55) - : : : : :- CometSort (52) - : : : : : +- ReusedExchange (51) - : : : : +- CometSort (54) - : : : : +- ReusedExchange (53) - : : : +- ReusedExchange (57) - : : +- CometBroadcastExchange (63) - : : +- CometProject (62) - : : +- CometFilter (61) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (60) - : +- ReusedExchange (66) - +- ReusedExchange (70) - - -(51) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] - -(52) CometSort -Input [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55], [ss_ticket_number#54 ASC NULLS FIRST, ss_item_sk#51 ASC NULLS FIRST] - -(53) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#56, sr_ticket_number#57] - -(54) CometSort -Input [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [sr_item_sk#56, sr_ticket_number#57], [sr_ticket_number#57 ASC NULLS FIRST, sr_item_sk#56 ASC NULLS FIRST] - -(55) CometSortMergeJoin -Left output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -Right output [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [ss_ticket_number#54, ss_item_sk#51], [sr_ticket_number#57, sr_item_sk#56], Inner - -(56) CometProject -Input [7]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55, sr_item_sk#56, sr_ticket_number#57] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55], [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] - -(57) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#58, s_store_name#59, s_state#60, s_zip#61] - -(58) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] -Right output [4]: [s_store_sk#58, s_store_name#59, s_state#60, s_zip#61] -Arguments: [ss_store_sk#53], [s_store_sk#58], Inner, BuildRight - -(59) CometProject -Input [8]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55, s_store_sk#58, s_store_name#59, s_state#60, s_zip#61] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61], [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61] - -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(61) CometFilter -Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Condition : isnotnull(i_item_sk#62) - -(62) CometProject -Input [6]: [i_item_sk#62, i_current_price#63, i_size#64, i_color#65, i_units#66, i_manager_id#67] -Arguments: [i_item_sk#62, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67], [i_item_sk#62, i_current_price#63, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_size#64, 20)) AS i_size#68, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#65, 20)) AS i_color#69, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_units#66, 10)) AS i_units#70, i_manager_id#67] - -(63) CometBroadcastExchange -Input [6]: [i_item_sk#62, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67] -Arguments: [i_item_sk#62, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67] - -(64) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61] -Right output [6]: [i_item_sk#62, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67] -Arguments: [ss_item_sk#51], [i_item_sk#62], Inner, BuildRight - -(65) CometProject -Input [12]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_item_sk#62, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67] -Arguments: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67], [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67] - -(66) ReusedExchange [Reuses operator id: 28] -Output [5]: [c_customer_sk#71, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] - -(67) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67] -Right output [5]: [c_customer_sk#71, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] -Arguments: [ss_customer_sk#52], [c_customer_sk#71], Inner, BuildRight - -(68) CometProject -Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_customer_sk#71, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] -Arguments: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75], [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] - -(69) CometColumnarToRow [codegen id : 2] -Input [13]: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75] - -(70) ReusedExchange [Reuses operator id: 36] -Output [4]: [ca_address_sk#76, ca_state#77, ca_zip#78, ca_country#79] - -(71) BroadcastHashJoin [codegen id : 2] -Left keys [3]: [c_current_addr_sk#72, c_birth_country#75, s_zip#61] -Right keys [3]: [ca_address_sk#76, upper(ca_country#79), ca_zip#78] -Join type: Inner -Join condition: None - -(72) Project [codegen id : 2] -Output [11]: [ss_net_paid#55, s_store_name#59, s_state#60, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_first_name#73, c_last_name#74, ca_state#77] -Input [17]: [ss_net_paid#55, s_store_name#59, s_state#60, s_zip#61, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_current_addr_sk#72, c_first_name#73, c_last_name#74, c_birth_country#75, ca_address_sk#76, ca_state#77, ca_zip#78, ca_country#79] - -(73) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#55, s_store_name#59, s_state#60, i_current_price#63, i_size#68, i_color#69, i_units#70, i_manager_id#67, c_first_name#73, c_last_name#74, ca_state#77] -Keys [10]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [sum#80] -Results [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, sum#81] - -(74) CometColumnarExchange -Input [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, sum#81] -Arguments: hashpartitioning(c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(75) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, sum#81] - -(76) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68, sum#81] -Keys [10]: [c_last_name#74, c_first_name#73, s_store_name#59, ca_state#77, s_state#60, i_color#69, i_current_price#63, i_manager_id#67, i_units#70, i_size#68] -Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#41] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#41,17,2) AS netpaid#82] - -(77) HashAggregate [codegen id : 3] -Input [1]: [netpaid#82] -Keys: [] -Functions [1]: [partial_avg(netpaid#82)] -Aggregate Attributes [2]: [sum#83, count#84] -Results [2]: [sum#85, count#86] - -(78) CometColumnarExchange -Input [2]: [sum#85, count#86] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(79) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#85, count#86] - -(80) HashAggregate [codegen id : 4] -Input [2]: [sum#85, count#86] -Keys: [] -Functions [1]: [avg(netpaid#82)] -Aggregate Attributes [1]: [avg(netpaid#82)#87] -Results [1]: [(0.05 * avg(netpaid#82)#87) AS (0.05 * avg(netpaid))#88] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/extended.txt deleted file mode 100644 index b9384ca04f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/extended.txt +++ /dev/null @@ -1,99 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometColumnarExchange - +- Filter - : +- Subquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 72 out of 88 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 62f492f632..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt +++ /dev/null @@ -1,99 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,s_store_name,paid] - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #5 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #12 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - InputAdapter - ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #4 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #5 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #7 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometScan [native_iceberg_compat] 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 (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] - CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/explain.txt deleted file mode 100644 index f025c0c7fd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/explain.txt +++ /dev/null @@ -1,479 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (78) -+- Union (77) - :- * HashAggregate (31) - : +- * CometColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- * HashAggregate (28) - : +- * Project (27) - : +- * BroadcastHashJoin Inner BuildRight (26) - : :- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (18) - : : +- * CometColumnarToRow (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.store (14) - : +- BroadcastExchange (25) - : +- * CometColumnarToRow (24) - : +- CometProject (23) - : +- CometFilter (22) - : +- CometNativeScan parquet spark_catalog.default.item (21) - :- * HashAggregate (54) - : +- * CometColumnarToRow (53) - : +- CometColumnarExchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * Project (40) - : : : +- * BroadcastHashJoin Inner BuildRight (39) - : : : :- * Project (37) - : : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : : :- * Filter (34) - : : : : : +- * ColumnarToRow (33) - : : : : : +- Scan parquet spark_catalog.default.store_sales (32) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- BroadcastExchange (45) - : : +- * CometColumnarToRow (44) - : : +- CometProject (43) - : : +- CometFilter (42) - : : +- CometNativeScan parquet spark_catalog.default.store (41) - : +- ReusedExchange (48) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- CometColumnarExchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * Project (66) - : +- * BroadcastHashJoin Inner BuildRight (65) - : :- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildRight (59) - : : : :- * Filter (57) - : : : : +- * ColumnarToRow (56) - : : : : +- Scan parquet spark_catalog.default.store_sales (55) - : : : +- ReusedExchange (58) - : : +- ReusedExchange (61) - : +- ReusedExchange (64) - +- BroadcastExchange (70) - +- * CometColumnarToRow (69) - +- CometFilter (68) - +- CometNativeScan parquet spark_catalog.default.item (67) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(3) Filter [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) - -(4) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#11, 1)) = F)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#12, 1)) = W)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#13, 20)) = Primary )) AND isnotnull(cd_demo_sk#10)) - -(6) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [cd_demo_sk#10] - -(8) BroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 5] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] - -(11) ReusedExchange [Reuses operator id: 83] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#15, s_state#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#15, s_state#16] -Condition : ((isnotnull(s_state#16) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#16, 2)) = TN)) AND isnotnull(s_store_sk#15)) - -(16) CometProject -Input [2]: [s_store_sk#15, s_state#16] -Arguments: [s_store_sk#15, s_state#17], [s_store_sk#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#16, 2)) AS s_state#17] - -(17) CometColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#15, s_state#17] - -(18) BroadcastExchange -Input [2]: [s_store_sk#15, s_state#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15, s_state#17] - -(21) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#18, i_item_id#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_item_sk#18, i_item_id#19] -Condition : isnotnull(i_item_sk#18) - -(23) CometProject -Input [2]: [i_item_sk#18, i_item_id#19] -Arguments: [i_item_sk#18, i_item_id#20], [i_item_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#19, 16)) AS i_item_id#20] - -(24) CometColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#18, i_item_id#20] - -(25) BroadcastExchange -Input [2]: [i_item_sk#18, i_item_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#18] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 5] -Output [6]: [i_item_id#20 AS i_item_id#21, s_state#17 AS s_state#22, ss_quantity#4 AS agg1#23, ss_list_price#5 AS agg2#24, ss_coupon_amt#7 AS agg3#25, ss_sales_price#6 AS agg4#26] -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17, i_item_sk#18, i_item_id#20] - -(28) HashAggregate [codegen id : 5] -Input [6]: [i_item_id#21, s_state#22, agg1#23, agg2#24, agg3#25, agg4#26] -Keys [2]: [i_item_id#21, s_state#22] -Functions [4]: [partial_avg(agg1#23), partial_avg(UnscaledValue(agg2#24)), partial_avg(UnscaledValue(agg3#25)), partial_avg(UnscaledValue(agg4#26))] -Aggregate Attributes [8]: [sum#27, count#28, sum#29, count#30, sum#31, count#32, sum#33, count#34] -Results [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] - -(29) CometColumnarExchange -Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] -Arguments: hashpartitioning(i_item_id#21, s_state#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 6] -Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] - -(31) HashAggregate [codegen id : 6] -Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] -Keys [2]: [i_item_id#21, s_state#22] -Functions [4]: [avg(agg1#23), avg(UnscaledValue(agg2#24)), avg(UnscaledValue(agg3#25)), avg(UnscaledValue(agg4#26))] -Aggregate Attributes [4]: [avg(agg1#23)#43, avg(UnscaledValue(agg2#24))#44, avg(UnscaledValue(agg3#25))#45, avg(UnscaledValue(agg4#26))#46] -Results [7]: [i_item_id#21, s_state#22, 0 AS g_state#47, avg(agg1#23)#43 AS agg1#48, cast((avg(UnscaledValue(agg2#24))#44 / 100.0) as decimal(11,6)) AS agg2#49, cast((avg(UnscaledValue(agg3#25))#45 / 100.0) as decimal(11,6)) AS agg3#50, cast((avg(UnscaledValue(agg4#26))#46 / 100.0) as decimal(11,6)) AS agg4#51] - -(32) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(33) ColumnarToRow [codegen id : 11] -Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] - -(34) Filter [codegen id : 11] -Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_cdemo_sk#53) AND isnotnull(ss_store_sk#54)) AND isnotnull(ss_item_sk#52)) - -(35) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#60] - -(36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#53] -Right keys [1]: [cd_demo_sk#60] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 11] -Output [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -Input [9]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, cd_demo_sk#60] - -(38) ReusedExchange [Reuses operator id: 83] -Output [1]: [d_date_sk#61] - -(39) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#59] -Right keys [1]: [d_date_sk#61] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 11] -Output [6]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] -Input [8]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, d_date_sk#61] - -(41) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#62, s_state#63] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [s_store_sk#62, s_state#63] -Condition : ((isnotnull(s_state#63) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#63, 2)) = TN)) AND isnotnull(s_store_sk#62)) - -(43) CometProject -Input [2]: [s_store_sk#62, s_state#63] -Arguments: [s_store_sk#62], [s_store_sk#62] - -(44) CometColumnarToRow [codegen id : 9] -Input [1]: [s_store_sk#62] - -(45) BroadcastExchange -Input [1]: [s_store_sk#62] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#54] -Right keys [1]: [s_store_sk#62] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 11] -Output [5]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] -Input [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, s_store_sk#62] - -(48) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#64, i_item_id#65] - -(49) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#52] -Right keys [1]: [i_item_sk#64] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 11] -Output [5]: [i_item_id#65, ss_quantity#55 AS agg1#66, ss_list_price#56 AS agg2#67, ss_coupon_amt#58 AS agg3#68, ss_sales_price#57 AS agg4#69] -Input [7]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, i_item_sk#64, i_item_id#65] - -(51) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#65, agg1#66, agg2#67, agg3#68, agg4#69] -Keys [1]: [i_item_id#65] -Functions [4]: [partial_avg(agg1#66), partial_avg(UnscaledValue(agg2#67)), partial_avg(UnscaledValue(agg3#68)), partial_avg(UnscaledValue(agg4#69))] -Aggregate Attributes [8]: [sum#70, count#71, sum#72, count#73, sum#74, count#75, sum#76, count#77] -Results [9]: [i_item_id#65, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] - -(52) CometColumnarExchange -Input [9]: [i_item_id#65, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] -Arguments: hashpartitioning(i_item_id#65, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(53) CometColumnarToRow [codegen id : 12] -Input [9]: [i_item_id#65, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] - -(54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#65, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85] -Keys [1]: [i_item_id#65] -Functions [4]: [avg(agg1#66), avg(UnscaledValue(agg2#67)), avg(UnscaledValue(agg3#68)), avg(UnscaledValue(agg4#69))] -Aggregate Attributes [4]: [avg(agg1#66)#86, avg(UnscaledValue(agg2#67))#87, avg(UnscaledValue(agg3#68))#88, avg(UnscaledValue(agg4#69))#89] -Results [7]: [i_item_id#65, null AS s_state#90, 1 AS g_state#91, avg(agg1#66)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#67))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#68))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#69))#89 / 100.0) as decimal(11,6)) AS agg4#95] - -(55) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#103), dynamicpruningexpression(ss_sold_date_sk#103 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(56) ColumnarToRow [codegen id : 17] -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] - -(57) Filter [codegen id : 17] -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) - -(58) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#104] - -(59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#97] -Right keys [1]: [cd_demo_sk#104] -Join type: Inner -Join condition: None - -(60) Project [codegen id : 17] -Output [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, cd_demo_sk#104] - -(61) ReusedExchange [Reuses operator id: 83] -Output [1]: [d_date_sk#105] - -(62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#103] -Right keys [1]: [d_date_sk#105] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 17] -Output [6]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] -Input [8]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#105] - -(64) ReusedExchange [Reuses operator id: 45] -Output [1]: [s_store_sk#106] - -(65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#98] -Right keys [1]: [s_store_sk#106] -Join type: Inner -Join condition: None - -(66) Project [codegen id : 17] -Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] -Input [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, s_store_sk#106] - -(67) CometNativeScan parquet spark_catalog.default.item -Output [1]: [i_item_sk#107] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(68) CometFilter -Input [1]: [i_item_sk#107] -Condition : isnotnull(i_item_sk#107) - -(69) CometColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#107] - -(70) BroadcastExchange -Input [1]: [i_item_sk#107] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -(71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#96] -Right keys [1]: [i_item_sk#107] -Join type: Inner -Join condition: None - -(72) Project [codegen id : 17] -Output [4]: [ss_quantity#99 AS agg1#108, ss_list_price#100 AS agg2#109, ss_coupon_amt#102 AS agg3#110, ss_sales_price#101 AS agg4#111] -Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#107] - -(73) HashAggregate [codegen id : 17] -Input [4]: [agg1#108, agg2#109, agg3#110, agg4#111] -Keys: [] -Functions [4]: [partial_avg(agg1#108), partial_avg(UnscaledValue(agg2#109)), partial_avg(UnscaledValue(agg3#110)), partial_avg(UnscaledValue(agg4#111))] -Aggregate Attributes [8]: [sum#112, count#113, sum#114, count#115, sum#116, count#117, sum#118, count#119] -Results [8]: [sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#126, count#127] - -(74) CometColumnarExchange -Input [8]: [sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#126, count#127] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(75) CometColumnarToRow [codegen id : 18] -Input [8]: [sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#126, count#127] - -(76) HashAggregate [codegen id : 18] -Input [8]: [sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#126, count#127] -Keys: [] -Functions [4]: [avg(agg1#108), avg(UnscaledValue(agg2#109)), avg(UnscaledValue(agg3#110)), avg(UnscaledValue(agg4#111))] -Aggregate Attributes [4]: [avg(agg1#108)#128, avg(UnscaledValue(agg2#109))#129, avg(UnscaledValue(agg3#110))#130, avg(UnscaledValue(agg4#111))#131] -Results [7]: [null AS i_item_id#132, null AS s_state#133, 1 AS g_state#134, avg(agg1#108)#128 AS agg1#135, cast((avg(UnscaledValue(agg2#109))#129 / 100.0) as decimal(11,6)) AS agg2#136, cast((avg(UnscaledValue(agg3#110))#130 / 100.0) as decimal(11,6)) AS agg3#137, cast((avg(UnscaledValue(agg4#111))#131 / 100.0) as decimal(11,6)) AS agg4#138] - -(77) Union - -(78) TakeOrderedAndProject -Input [7]: [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] -Arguments: 100, [i_item_id#21 ASC NULLS FIRST, s_state#22 ASC NULLS FIRST], [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometNativeScan parquet spark_catalog.default.date_dim (79) - - -(79) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#139] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(80) CometFilter -Input [2]: [d_date_sk#14, d_year#139] -Condition : ((isnotnull(d_year#139) AND (d_year#139 = 1998)) AND isnotnull(d_date_sk#14)) - -(81) CometProject -Input [2]: [d_date_sk#14, d_year#139] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(82) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(83) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#9 - -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#9 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/simplified.txt deleted file mode 100644 index c7093301af..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/simplified.txt +++ /dev/null @@ -1,122 +0,0 @@ -TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - Union - WholeStageCodegen (6) - HashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,s_state] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Filter [ss_cdemo_sk,ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - WholeStageCodegen (12) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #6 - WholeStageCodegen (11) - HashAggregate [i_item_id,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Filter [ss_cdemo_sk,ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [cd_demo_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) - HashAggregate [sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (17) - HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Filter [ss_cdemo_sk,ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [cd_demo_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [s_store_sk] #7 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/explain.txt deleted file mode 100644 index bcba275572..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,437 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (71) -+- CometTakeOrderedAndProject (70) - +- CometUnion (69) - :- CometHashAggregate (29) - : +- CometExchange (28) - : +- CometHashAggregate (27) - : +- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometProject (20) - : : +- CometBroadcastHashJoin (19) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (18) - : : +- CometProject (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - : +- CometBroadcastExchange (24) - : +- CometProject (23) - : +- CometFilter (22) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) - :- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (37) - : : : +- CometBroadcastHashJoin (36) - : : : :- CometProject (34) - : : : : +- CometBroadcastHashJoin (33) - : : : : :- CometFilter (31) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (30) - : : : : +- ReusedExchange (32) - : : : +- ReusedExchange (35) - : : +- CometBroadcastExchange (41) - : : +- CometProject (40) - : : +- CometFilter (39) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (38) - : +- ReusedExchange (44) - +- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometProject (65) - +- CometBroadcastHashJoin (64) - :- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (57) - : : +- CometBroadcastHashJoin (56) - : : :- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometFilter (51) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : : : +- ReusedExchange (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (58) - +- CometBroadcastExchange (63) - +- CometFilter (62) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (61) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#11, 1)) = F)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#12, 1)) = W)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#13, 20)) = Primary )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#16, s_state#17] -Condition : ((isnotnull(s_state#17) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#17, 2)) = TN)) AND isnotnull(s_store_sk#16)) - -(17) CometProject -Input [2]: [s_store_sk#16, s_state#17] -Arguments: [s_store_sk#16, s_state#18], [s_store_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#17, 2)) AS s_state#18] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_state#18] -Arguments: [s_store_sk#16, s_state#18] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Right output [2]: [s_store_sk#16, s_state#18] -Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16, s_state#18] -Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_item_id#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_item_sk#19, i_item_id#20] -Condition : isnotnull(i_item_sk#19) - -(23) CometProject -Input [2]: [i_item_sk#19, i_item_id#20] -Arguments: [i_item_sk#19, i_item_id#21], [i_item_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#20, 16)) AS i_item_id#21] - -(24) CometBroadcastExchange -Input [2]: [i_item_sk#19, i_item_id#21] -Arguments: [i_item_sk#19, i_item_id#21] - -(25) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] -Right output [2]: [i_item_sk#19, i_item_id#21] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(26) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18, i_item_sk#19, i_item_id#21] -Arguments: [i_item_id#22, s_state#23, agg1#24, agg2#25, agg3#26, agg4#27], [i_item_id#21 AS i_item_id#22, s_state#18 AS s_state#23, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] - -(27) CometHashAggregate -Input [6]: [i_item_id#22, s_state#23, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [2]: [i_item_id#22, s_state#23] -Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] - -(28) CometExchange -Input [10]: [i_item_id#22, s_state#23, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Arguments: hashpartitioning(i_item_id#22, s_state#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [10]: [i_item_id#22, s_state#23, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Keys [2]: [i_item_id#22, s_state#23] -Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#43), dynamicpruningexpression(ss_sold_date_sk#43 IN dynamicpruning#44)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(31) CometFilter -Input [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Condition : ((isnotnull(ss_cdemo_sk#37) AND isnotnull(ss_store_sk#38)) AND isnotnull(ss_item_sk#36)) - -(32) ReusedExchange [Reuses operator id: 6] -Output [1]: [cd_demo_sk#45] - -(33) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Right output [1]: [cd_demo_sk#45] -Arguments: [ss_cdemo_sk#37], [cd_demo_sk#45], Inner, BuildRight - -(34) CometProject -Input [9]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43, cd_demo_sk#45] -Arguments: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43], [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] - -(35) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#46] - -(36) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Right output [1]: [d_date_sk#46] -Arguments: [ss_sold_date_sk#43], [d_date_sk#46], Inner, BuildRight - -(37) CometProject -Input [8]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43, d_date_sk#46] -Arguments: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42], [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#47, s_state#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] -ReadSchema: struct - -(39) CometFilter -Input [2]: [s_store_sk#47, s_state#48] -Condition : ((isnotnull(s_state#48) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#48, 2)) = TN)) AND isnotnull(s_store_sk#47)) - -(40) CometProject -Input [2]: [s_store_sk#47, s_state#48] -Arguments: [s_store_sk#47], [s_store_sk#47] - -(41) CometBroadcastExchange -Input [1]: [s_store_sk#47] -Arguments: [s_store_sk#47] - -(42) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] -Right output [1]: [s_store_sk#47] -Arguments: [ss_store_sk#38], [s_store_sk#47], Inner, BuildRight - -(43) CometProject -Input [7]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, s_store_sk#47] -Arguments: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42], [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] - -(44) ReusedExchange [Reuses operator id: 24] -Output [2]: [i_item_sk#49, i_item_id#50] - -(45) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] -Right output [2]: [i_item_sk#49, i_item_id#50] -Arguments: [ss_item_sk#36], [i_item_sk#49], Inner, BuildRight - -(46) CometProject -Input [7]: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, i_item_sk#49, i_item_id#50] -Arguments: [i_item_id#50, agg1#51, agg2#52, agg3#53, agg4#54], [i_item_id#50, ss_quantity#39 AS agg1#51, ss_list_price#40 AS agg2#52, ss_coupon_amt#42 AS agg3#53, ss_sales_price#41 AS agg4#54] - -(47) CometHashAggregate -Input [5]: [i_item_id#50, agg1#51, agg2#52, agg3#53, agg4#54] -Keys [1]: [i_item_id#50] -Functions [4]: [partial_avg(agg1#51), partial_avg(UnscaledValue(agg2#52)), partial_avg(UnscaledValue(agg3#53)), partial_avg(UnscaledValue(agg4#54))] - -(48) CometExchange -Input [9]: [i_item_id#50, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Arguments: hashpartitioning(i_item_id#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(49) CometHashAggregate -Input [9]: [i_item_id#50, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Keys [1]: [i_item_id#50] -Functions [4]: [avg(agg1#51), avg(UnscaledValue(agg2#52)), avg(UnscaledValue(agg3#53)), avg(UnscaledValue(agg4#54))] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#63, ss_cdemo_sk#64, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69, ss_sold_date_sk#70] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#70), dynamicpruningexpression(ss_sold_date_sk#70 IN dynamicpruning#71)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [8]: [ss_item_sk#63, ss_cdemo_sk#64, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69, ss_sold_date_sk#70] -Condition : ((isnotnull(ss_cdemo_sk#64) AND isnotnull(ss_store_sk#65)) AND isnotnull(ss_item_sk#63)) - -(52) ReusedExchange [Reuses operator id: 6] -Output [1]: [cd_demo_sk#72] - -(53) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#63, ss_cdemo_sk#64, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69, ss_sold_date_sk#70] -Right output [1]: [cd_demo_sk#72] -Arguments: [ss_cdemo_sk#64], [cd_demo_sk#72], Inner, BuildRight - -(54) CometProject -Input [9]: [ss_item_sk#63, ss_cdemo_sk#64, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69, ss_sold_date_sk#70, cd_demo_sk#72] -Arguments: [ss_item_sk#63, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69, ss_sold_date_sk#70], [ss_item_sk#63, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69, ss_sold_date_sk#70] - -(55) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#73] - -(56) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#63, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69, ss_sold_date_sk#70] -Right output [1]: [d_date_sk#73] -Arguments: [ss_sold_date_sk#70], [d_date_sk#73], Inner, BuildRight - -(57) CometProject -Input [8]: [ss_item_sk#63, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69, ss_sold_date_sk#70, d_date_sk#73] -Arguments: [ss_item_sk#63, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69], [ss_item_sk#63, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69] - -(58) ReusedExchange [Reuses operator id: 41] -Output [1]: [s_store_sk#74] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#63, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69] -Right output [1]: [s_store_sk#74] -Arguments: [ss_store_sk#65], [s_store_sk#74], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_item_sk#63, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69, s_store_sk#74] -Arguments: [ss_item_sk#63, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69], [ss_item_sk#63, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69] - -(61) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(62) CometFilter -Input [1]: [i_item_sk#75] -Condition : isnotnull(i_item_sk#75) - -(63) CometBroadcastExchange -Input [1]: [i_item_sk#75] -Arguments: [i_item_sk#75] - -(64) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#63, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69] -Right output [1]: [i_item_sk#75] -Arguments: [ss_item_sk#63], [i_item_sk#75], Inner, BuildRight - -(65) CometProject -Input [6]: [ss_item_sk#63, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69, i_item_sk#75] -Arguments: [agg1#76, agg2#77, agg3#78, agg4#79], [ss_quantity#66 AS agg1#76, ss_list_price#67 AS agg2#77, ss_coupon_amt#69 AS agg3#78, ss_sales_price#68 AS agg4#79] - -(66) CometHashAggregate -Input [4]: [agg1#76, agg2#77, agg3#78, agg4#79] -Keys: [] -Functions [4]: [partial_avg(agg1#76), partial_avg(UnscaledValue(agg2#77)), partial_avg(UnscaledValue(agg3#78)), partial_avg(UnscaledValue(agg4#79))] - -(67) CometExchange -Input [8]: [sum#80, count#81, sum#82, count#83, sum#84, count#85, sum#86, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(68) CometHashAggregate -Input [8]: [sum#80, count#81, sum#82, count#83, sum#84, count#85, sum#86, count#87] -Keys: [] -Functions [4]: [avg(agg1#76), avg(UnscaledValue(agg2#77)), avg(UnscaledValue(agg3#78)), avg(UnscaledValue(agg4#79))] - -(69) CometUnion -Child 0 Input [7]: [i_item_id#22, s_state#23, g_state#88, agg1#89, agg2#90, agg3#91, agg4#92] -Child 1 Input [7]: [i_item_id#50, s_state#93, g_state#94, agg1#95, agg2#96, agg3#97, agg4#98] -Child 2 Input [7]: [i_item_id#99, s_state#100, g_state#101, agg1#102, agg2#103, agg3#104, agg4#105] - -(70) CometTakeOrderedAndProject -Input [7]: [i_item_id#22, s_state#23, g_state#88, agg1#89, agg2#90, agg3#91, agg4#92] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#22 ASC NULLS FIRST,s_state#23 ASC NULLS FIRST], output=[i_item_id#22,s_state#23,g_state#88,agg1#89,agg2#90,agg3#91,agg4#92]), [i_item_id#22, s_state#23, g_state#88, agg1#89, agg2#90, agg3#91, agg4#92], 100, 0, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#88, agg1#89, agg2#90, agg3#91, agg4#92] - -(71) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#22, s_state#23, g_state#88, agg1#89, agg2#90, agg3#91, agg4#92] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) - -(74) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(75) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(76) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#43 IN dynamicpruning#9 - -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#70 IN dynamicpruning#9 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 0d18ca5626..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,83 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometUnion [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange [i_item_id,s_state] #1 - CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] - CometProject [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] - 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_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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_state] #5 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange [i_item_id] #7 - CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,sum,count,sum,count,sum,count,sum,count] - CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] - 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_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_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_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 [native_iceberg_compat] 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 [s_store_sk] #8 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange #9 - CometHashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count] - CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] - 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_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_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_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 [native_iceberg_compat] 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 [i_item_sk] #10 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/explain.txt deleted file mode 100644 index bcba275572..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/explain.txt +++ /dev/null @@ -1,437 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (71) -+- CometTakeOrderedAndProject (70) - +- CometUnion (69) - :- CometHashAggregate (29) - : +- CometExchange (28) - : +- CometHashAggregate (27) - : +- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometProject (20) - : : +- CometBroadcastHashJoin (19) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (18) - : : +- CometProject (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - : +- CometBroadcastExchange (24) - : +- CometProject (23) - : +- CometFilter (22) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) - :- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (37) - : : : +- CometBroadcastHashJoin (36) - : : : :- CometProject (34) - : : : : +- CometBroadcastHashJoin (33) - : : : : :- CometFilter (31) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (30) - : : : : +- ReusedExchange (32) - : : : +- ReusedExchange (35) - : : +- CometBroadcastExchange (41) - : : +- CometProject (40) - : : +- CometFilter (39) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (38) - : +- ReusedExchange (44) - +- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometProject (65) - +- CometBroadcastHashJoin (64) - :- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (57) - : : +- CometBroadcastHashJoin (56) - : : :- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometFilter (51) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : : : +- ReusedExchange (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (58) - +- CometBroadcastExchange (63) - +- CometFilter (62) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (61) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#11, 1)) = F)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#12, 1)) = W)) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_education_status#13, 20)) = Primary )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#16, s_state#17] -Condition : ((isnotnull(s_state#17) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#17, 2)) = TN)) AND isnotnull(s_store_sk#16)) - -(17) CometProject -Input [2]: [s_store_sk#16, s_state#17] -Arguments: [s_store_sk#16, s_state#18], [s_store_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#17, 2)) AS s_state#18] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_state#18] -Arguments: [s_store_sk#16, s_state#18] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Right output [2]: [s_store_sk#16, s_state#18] -Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16, s_state#18] -Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_item_id#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_item_sk#19, i_item_id#20] -Condition : isnotnull(i_item_sk#19) - -(23) CometProject -Input [2]: [i_item_sk#19, i_item_id#20] -Arguments: [i_item_sk#19, i_item_id#21], [i_item_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#20, 16)) AS i_item_id#21] - -(24) CometBroadcastExchange -Input [2]: [i_item_sk#19, i_item_id#21] -Arguments: [i_item_sk#19, i_item_id#21] - -(25) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] -Right output [2]: [i_item_sk#19, i_item_id#21] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(26) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18, i_item_sk#19, i_item_id#21] -Arguments: [i_item_id#22, s_state#23, agg1#24, agg2#25, agg3#26, agg4#27], [i_item_id#21 AS i_item_id#22, s_state#18 AS s_state#23, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] - -(27) CometHashAggregate -Input [6]: [i_item_id#22, s_state#23, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [2]: [i_item_id#22, s_state#23] -Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] - -(28) CometExchange -Input [10]: [i_item_id#22, s_state#23, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Arguments: hashpartitioning(i_item_id#22, s_state#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [10]: [i_item_id#22, s_state#23, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Keys [2]: [i_item_id#22, s_state#23] -Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#43), dynamicpruningexpression(ss_sold_date_sk#43 IN dynamicpruning#44)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(31) CometFilter -Input [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Condition : ((isnotnull(ss_cdemo_sk#37) AND isnotnull(ss_store_sk#38)) AND isnotnull(ss_item_sk#36)) - -(32) ReusedExchange [Reuses operator id: 6] -Output [1]: [cd_demo_sk#45] - -(33) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Right output [1]: [cd_demo_sk#45] -Arguments: [ss_cdemo_sk#37], [cd_demo_sk#45], Inner, BuildRight - -(34) CometProject -Input [9]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43, cd_demo_sk#45] -Arguments: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43], [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] - -(35) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#46] - -(36) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Right output [1]: [d_date_sk#46] -Arguments: [ss_sold_date_sk#43], [d_date_sk#46], Inner, BuildRight - -(37) CometProject -Input [8]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43, d_date_sk#46] -Arguments: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42], [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#47, s_state#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] -ReadSchema: struct - -(39) CometFilter -Input [2]: [s_store_sk#47, s_state#48] -Condition : ((isnotnull(s_state#48) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#48, 2)) = TN)) AND isnotnull(s_store_sk#47)) - -(40) CometProject -Input [2]: [s_store_sk#47, s_state#48] -Arguments: [s_store_sk#47], [s_store_sk#47] - -(41) CometBroadcastExchange -Input [1]: [s_store_sk#47] -Arguments: [s_store_sk#47] - -(42) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] -Right output [1]: [s_store_sk#47] -Arguments: [ss_store_sk#38], [s_store_sk#47], Inner, BuildRight - -(43) CometProject -Input [7]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, s_store_sk#47] -Arguments: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42], [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] - -(44) ReusedExchange [Reuses operator id: 24] -Output [2]: [i_item_sk#49, i_item_id#50] - -(45) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] -Right output [2]: [i_item_sk#49, i_item_id#50] -Arguments: [ss_item_sk#36], [i_item_sk#49], Inner, BuildRight - -(46) CometProject -Input [7]: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, i_item_sk#49, i_item_id#50] -Arguments: [i_item_id#50, agg1#51, agg2#52, agg3#53, agg4#54], [i_item_id#50, ss_quantity#39 AS agg1#51, ss_list_price#40 AS agg2#52, ss_coupon_amt#42 AS agg3#53, ss_sales_price#41 AS agg4#54] - -(47) CometHashAggregate -Input [5]: [i_item_id#50, agg1#51, agg2#52, agg3#53, agg4#54] -Keys [1]: [i_item_id#50] -Functions [4]: [partial_avg(agg1#51), partial_avg(UnscaledValue(agg2#52)), partial_avg(UnscaledValue(agg3#53)), partial_avg(UnscaledValue(agg4#54))] - -(48) CometExchange -Input [9]: [i_item_id#50, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Arguments: hashpartitioning(i_item_id#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(49) CometHashAggregate -Input [9]: [i_item_id#50, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Keys [1]: [i_item_id#50] -Functions [4]: [avg(agg1#51), avg(UnscaledValue(agg2#52)), avg(UnscaledValue(agg3#53)), avg(UnscaledValue(agg4#54))] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#63, ss_cdemo_sk#64, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69, ss_sold_date_sk#70] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#70), dynamicpruningexpression(ss_sold_date_sk#70 IN dynamicpruning#71)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [8]: [ss_item_sk#63, ss_cdemo_sk#64, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69, ss_sold_date_sk#70] -Condition : ((isnotnull(ss_cdemo_sk#64) AND isnotnull(ss_store_sk#65)) AND isnotnull(ss_item_sk#63)) - -(52) ReusedExchange [Reuses operator id: 6] -Output [1]: [cd_demo_sk#72] - -(53) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#63, ss_cdemo_sk#64, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69, ss_sold_date_sk#70] -Right output [1]: [cd_demo_sk#72] -Arguments: [ss_cdemo_sk#64], [cd_demo_sk#72], Inner, BuildRight - -(54) CometProject -Input [9]: [ss_item_sk#63, ss_cdemo_sk#64, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69, ss_sold_date_sk#70, cd_demo_sk#72] -Arguments: [ss_item_sk#63, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69, ss_sold_date_sk#70], [ss_item_sk#63, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69, ss_sold_date_sk#70] - -(55) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#73] - -(56) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#63, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69, ss_sold_date_sk#70] -Right output [1]: [d_date_sk#73] -Arguments: [ss_sold_date_sk#70], [d_date_sk#73], Inner, BuildRight - -(57) CometProject -Input [8]: [ss_item_sk#63, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69, ss_sold_date_sk#70, d_date_sk#73] -Arguments: [ss_item_sk#63, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69], [ss_item_sk#63, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69] - -(58) ReusedExchange [Reuses operator id: 41] -Output [1]: [s_store_sk#74] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#63, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69] -Right output [1]: [s_store_sk#74] -Arguments: [ss_store_sk#65], [s_store_sk#74], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_item_sk#63, ss_store_sk#65, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69, s_store_sk#74] -Arguments: [ss_item_sk#63, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69], [ss_item_sk#63, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69] - -(61) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(62) CometFilter -Input [1]: [i_item_sk#75] -Condition : isnotnull(i_item_sk#75) - -(63) CometBroadcastExchange -Input [1]: [i_item_sk#75] -Arguments: [i_item_sk#75] - -(64) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#63, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69] -Right output [1]: [i_item_sk#75] -Arguments: [ss_item_sk#63], [i_item_sk#75], Inner, BuildRight - -(65) CometProject -Input [6]: [ss_item_sk#63, ss_quantity#66, ss_list_price#67, ss_sales_price#68, ss_coupon_amt#69, i_item_sk#75] -Arguments: [agg1#76, agg2#77, agg3#78, agg4#79], [ss_quantity#66 AS agg1#76, ss_list_price#67 AS agg2#77, ss_coupon_amt#69 AS agg3#78, ss_sales_price#68 AS agg4#79] - -(66) CometHashAggregate -Input [4]: [agg1#76, agg2#77, agg3#78, agg4#79] -Keys: [] -Functions [4]: [partial_avg(agg1#76), partial_avg(UnscaledValue(agg2#77)), partial_avg(UnscaledValue(agg3#78)), partial_avg(UnscaledValue(agg4#79))] - -(67) CometExchange -Input [8]: [sum#80, count#81, sum#82, count#83, sum#84, count#85, sum#86, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(68) CometHashAggregate -Input [8]: [sum#80, count#81, sum#82, count#83, sum#84, count#85, sum#86, count#87] -Keys: [] -Functions [4]: [avg(agg1#76), avg(UnscaledValue(agg2#77)), avg(UnscaledValue(agg3#78)), avg(UnscaledValue(agg4#79))] - -(69) CometUnion -Child 0 Input [7]: [i_item_id#22, s_state#23, g_state#88, agg1#89, agg2#90, agg3#91, agg4#92] -Child 1 Input [7]: [i_item_id#50, s_state#93, g_state#94, agg1#95, agg2#96, agg3#97, agg4#98] -Child 2 Input [7]: [i_item_id#99, s_state#100, g_state#101, agg1#102, agg2#103, agg3#104, agg4#105] - -(70) CometTakeOrderedAndProject -Input [7]: [i_item_id#22, s_state#23, g_state#88, agg1#89, agg2#90, agg3#91, agg4#92] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#22 ASC NULLS FIRST,s_state#23 ASC NULLS FIRST], output=[i_item_id#22,s_state#23,g_state#88,agg1#89,agg2#90,agg3#91,agg4#92]), [i_item_id#22, s_state#23, g_state#88, agg1#89, agg2#90, agg3#91, agg4#92], 100, 0, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#88, agg1#89, agg2#90, agg3#91, agg4#92] - -(71) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#22, s_state#23, g_state#88, agg1#89, agg2#90, agg3#91, agg4#92] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) - -(74) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(75) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(76) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#43 IN dynamicpruning#9 - -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#70 IN dynamicpruning#9 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/extended.txt deleted file mode 100644 index affbc9ef37..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/extended.txt +++ /dev/null @@ -1,99 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 91 out of 95 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 0d18ca5626..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt +++ /dev/null @@ -1,83 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometUnion [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange [i_item_id,s_state] #1 - CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] - CometProject [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] - 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_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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_state] #5 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange [i_item_id] #7 - CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,sum,count,sum,count,sum,count,sum,count] - CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] - 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_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_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_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 [native_iceberg_compat] 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 [s_store_sk] #8 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange #9 - CometHashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count] - CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] - 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_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_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_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 [native_iceberg_compat] 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 [i_item_sk] #10 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/explain.txt deleted file mode 100644 index 243b950bc1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/explain.txt +++ /dev/null @@ -1,231 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (35) -+- CometSort (34) - +- CometColumnarExchange (33) - +- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * Filter (25) - : +- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (30) - +- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometNativeScan parquet spark_catalog.default.customer (26) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 40] -Output [1]: [d_date_sk#7] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_county#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [s_store_sk#8, s_county#9] -Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND isnotnull(s_store_sk#8)) - -(9) CometProject -Input [2]: [s_store_sk#8, s_county#9] -Arguments: [s_store_sk#8], [s_store_sk#8] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#8] - -(11) BroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] - -(14) CometNativeScan parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : ((((isnotnull(hd_vehicle_count#13) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#11, 15)) = >10000 ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#11, 15)) = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#12 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#13 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#10)) - -(16) CometProject -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Arguments: [hd_demo_sk#10], [hd_demo_sk#10] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#10] - -(18) BroadcastExchange -Input [1]: [hd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#10] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [2]: [ss_customer_sk#1, ss_ticket_number#4] -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] - -(21) HashAggregate [codegen id : 4] -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#14] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(22) CometColumnarExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(24) HashAggregate [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#16] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] - -(25) Filter [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) - -(26) CometNativeScan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(27) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(28) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#19, 10)) AS c_salutation#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#20, 20)) AS c_first_name#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#21, 30)) AS c_last_name#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#22, 1)) AS c_preferred_cust_flag#26] - -(29) CometColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(30) BroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#18] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 6] -Output [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(33) CometColumnarExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(34) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST] - -(35) CometColumnarToRow [codegen id : 7] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (40) -+- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometNativeScan parquet spark_catalog.default.date_dim (36) - - -(36) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#27, d_dom#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [d_date_sk#7, d_year#27, d_dom#28] -Condition : (((((d_dom#28 >= 1) AND (d_dom#28 <= 3)) OR ((d_dom#28 >= 25) AND (d_dom#28 <= 28))) AND d_year#27 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(38) CometProject -Input [3]: [d_date_sk#7, d_year#27, d_dom#28] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(39) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(40) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/simplified.txt deleted file mode 100644 index 63c6ac3645..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/simplified.txt +++ /dev/null @@ -1,59 +0,0 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometNativeScan 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/q34.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/explain.txt deleted file mode 100644 index 94be39468a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometSort (32) - +- CometExchange (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (24) - : +- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_county#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#10, s_county#11] -Condition : ((isnotnull(s_county#11) AND (s_county#11 = Williamson County)) AND isnotnull(s_store_sk#10)) - -(11) CometProject -Input [2]: [s_store_sk#10, s_county#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : ((((isnotnull(hd_vehicle_count#15) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#13, 15)) = >10000 ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#13, 15)) = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#14 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#15 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#12)) - -(17) CometProject -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#12], [hd_demo_sk#12] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#12] -Arguments: [hd_demo_sk#12] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Right output [1]: [hd_demo_sk#12] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight - -(20) CometProject -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] -Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] - -(21) CometHashAggregate -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] - -(22) CometExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] - -(24) CometFilter -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(27) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#19, 10)) AS c_salutation#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#20, 20)) AS c_first_name#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#21, 30)) AS c_last_name#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#22, 1)) AS c_preferred_cust_flag#26] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(29) CometBroadcastHashJoin -Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Right output [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -(31) CometExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(32) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST] - -(33) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(36) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(37) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(38) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/simplified.txt deleted file mode 100644 index d57afd4fcf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 - CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] - CometExchange [ss_ticket_number,ss_customer_sk] #2 - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - 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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/explain.txt deleted file mode 100644 index 94be39468a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometSort (32) - +- CometExchange (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (24) - : +- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_county#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#10, s_county#11] -Condition : ((isnotnull(s_county#11) AND (s_county#11 = Williamson County)) AND isnotnull(s_store_sk#10)) - -(11) CometProject -Input [2]: [s_store_sk#10, s_county#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : ((((isnotnull(hd_vehicle_count#15) AND ((static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#13, 15)) = >10000 ) OR (static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#13, 15)) = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#14 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#15 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#12)) - -(17) CometProject -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#12], [hd_demo_sk#12] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#12] -Arguments: [hd_demo_sk#12] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Right output [1]: [hd_demo_sk#12] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight - -(20) CometProject -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] -Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] - -(21) CometHashAggregate -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] - -(22) CometExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] - -(24) CometFilter -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(27) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#19, 10)) AS c_salutation#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#20, 20)) AS c_first_name#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#21, 30)) AS c_last_name#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#22, 1)) AS c_preferred_cust_flag#26] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(29) CometBroadcastHashJoin -Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Right output [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -(31) CometExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(32) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST] - -(33) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(36) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(37) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(38) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/extended.txt deleted file mode 100644 index 3d9d23d2cc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/extended.txt +++ /dev/null @@ -1,41 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d57afd4fcf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 - CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] - CometExchange [ss_ticket_number,ss_customer_sk] #2 - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - 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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/explain.txt deleted file mode 100644 index 7e28922e9f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/explain.txt +++ /dev/null @@ -1,295 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (45) -+- * HashAggregate (44) - +- * CometColumnarToRow (43) - +- CometColumnarExchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometNativeScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (38) - +- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometNativeScan parquet spark_catalog.default.customer_demographics (34) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(4) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] - -(6) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#9] - -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None - -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#6] -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] - -(9) BroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#6] -Join type: LeftSemi -Join condition: None - -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] - -(13) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#12] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#10] -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#12] - -(16) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(17) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#10] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#8)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] - -(20) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#15] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#13] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] - -(23) BroadcastExchange -Input [1]: [cs_ship_customer_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#13] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(25) Filter [codegen id : 9] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(26) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(27) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : isnotnull(ca_address_sk#16) - -(29) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16, ca_state#18], [ca_address_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#17, 2)) AS ca_state#18] - -(30) CometColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#16, ca_state#18] - -(31) BroadcastExchange -Input [2]: [ca_address_sk#16, ca_state#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#16] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#18] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#16, ca_state#18] - -(34) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(35) CometFilter -Input [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Condition : isnotnull(cd_demo_sk#19) - -(36) CometProject -Input [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Arguments: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24], [cd_demo_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#20, 1)) AS cd_gender#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#21, 1)) AS cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] - -(37) CometColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] - -(38) BroadcastExchange -Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#19] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 9] -Output [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Input [8]: [c_current_cdemo_sk#4, ca_state#18, cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] - -(41) HashAggregate [codegen id : 9] -Input [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#22), partial_max(cd_dep_count#22), partial_sum(cd_dep_count#22), partial_avg(cd_dep_employed_count#23), partial_max(cd_dep_employed_count#23), partial_sum(cd_dep_employed_count#23), partial_avg(cd_dep_college_count#24), partial_max(cd_dep_college_count#24), partial_sum(cd_dep_college_count#24)] -Aggregate Attributes [13]: [count#27, sum#28, count#29, max#30, sum#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39] -Results [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] - -(42) CometColumnarExchange -Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] -Arguments: hashpartitioning(ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) CometColumnarToRow [codegen id : 10] -Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] - -(44) HashAggregate [codegen id : 10] -Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] -Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Functions [10]: [count(1), avg(cd_dep_count#22), max(cd_dep_count#22), sum(cd_dep_count#22), avg(cd_dep_employed_count#23), max(cd_dep_employed_count#23), sum(cd_dep_employed_count#23), avg(cd_dep_college_count#24), max(cd_dep_college_count#24), sum(cd_dep_college_count#24)] -Aggregate Attributes [10]: [count(1)#53, avg(cd_dep_count#22)#54, max(cd_dep_count#22)#55, sum(cd_dep_count#22)#56, avg(cd_dep_employed_count#23)#57, max(cd_dep_employed_count#23)#58, sum(cd_dep_employed_count#23)#59, avg(cd_dep_college_count#24)#60, max(cd_dep_college_count#24)#61, sum(cd_dep_college_count#24)#62] -Results [18]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, count(1)#53 AS cnt1#63, avg(cd_dep_count#22)#54 AS avg(cd_dep_count)#64, max(cd_dep_count#22)#55 AS max(cd_dep_count)#65, sum(cd_dep_count#22)#56 AS sum(cd_dep_count)#66, cd_dep_employed_count#23, count(1)#53 AS cnt2#67, avg(cd_dep_employed_count#23)#57 AS avg(cd_dep_employed_count)#68, max(cd_dep_employed_count#23)#58 AS max(cd_dep_employed_count)#69, sum(cd_dep_employed_count#23)#59 AS sum(cd_dep_employed_count)#70, cd_dep_college_count#24, count(1)#53 AS cnt3#71, avg(cd_dep_college_count#24)#60 AS avg(cd_dep_college_count)#72, max(cd_dep_college_count#24)#61 AS max(cd_dep_college_count)#73, sum(cd_dep_college_count#24)#62 AS sum(cd_dep_college_count)#74] - -(45) TakeOrderedAndProject -Input [18]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cnt1#63, avg(cd_dep_count)#64, max(cd_dep_count)#65, sum(cd_dep_count)#66, cd_dep_employed_count#23, cnt2#67, avg(cd_dep_employed_count)#68, max(cd_dep_employed_count)#69, sum(cd_dep_employed_count)#70, cd_dep_college_count#24, cnt3#71, avg(cd_dep_college_count)#72, max(cd_dep_college_count)#73, sum(cd_dep_college_count)#74] -Arguments: 100, [ca_state#18 ASC NULLS FIRST, cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_dep_count#22 ASC NULLS FIRST, cd_dep_employed_count#23 ASC NULLS FIRST, cd_dep_college_count#24 ASC NULLS FIRST], [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cnt1#63, avg(cd_dep_count)#64, max(cd_dep_count)#65, sum(cd_dep_count)#66, cd_dep_employed_count#23, cnt2#67, avg(cd_dep_employed_count)#68, max(cd_dep_employed_count)#69, sum(cd_dep_employed_count)#70, cd_dep_college_count#24, cnt3#71, avg(cd_dep_college_count)#72, max(cd_dep_college_count)#73, sum(cd_dep_college_count)#74] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (50) -+- * CometColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometNativeScan parquet spark_catalog.default.date_dim (46) - - -(46) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#75, d_qoy#76] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_date_sk#9, d_year#75, d_qoy#76] -Condition : ((((isnotnull(d_year#75) AND isnotnull(d_qoy#76)) AND (d_year#75 = 2002)) AND (d_qoy#76 < 4)) AND isnotnull(d_date_sk#9)) - -(48) CometProject -Input [3]: [d_date_sk#9, d_year#75, d_qoy#76] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(49) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(50) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/simplified.txt deleted file mode 100644 index 0fb95102c3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/simplified.txt +++ /dev/null @@ -1,77 +0,0 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (10) - HashAggregate [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] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [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,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan 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/q35.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/explain.txt deleted file mode 100644 index c8f5d42144..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * Filter (27) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(6) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: [d_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#9] -Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -Arguments: [ss_customer_sk#6], [ss_customer_sk#6] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: [ss_customer_sk#6] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Right output [1]: [ss_customer_sk#6] -Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#15] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Right output [1]: [d_date_sk#15] -Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] -Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#12] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#12] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#19] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#16] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#16] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(27) Filter [codegen id : 5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(28) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : isnotnull(ca_address_sk#20) - -(31) CometProject -Input [2]: [ca_address_sk#20, ca_state#21] -Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#21, 2)) AS ca_state#22] - -(32) CometColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#20, ca_state#22] - -(33) BroadcastExchange -Input [2]: [ca_address_sk#20, ca_state#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(34) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, ca_state#22] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#22] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(37) CometFilter -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#23) - -(38) CometProject -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#24, 1)) AS cd_gender#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#25, 1)) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(39) CometColumnarToRow [codegen id : 4] -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(40) BroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(41) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#23] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 5] -Output [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [8]: [c_current_cdemo_sk#4, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(43) HashAggregate [codegen id : 5] -Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] -Aggregate Attributes [13]: [count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] -Results [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] - -(44) CometColumnarExchange -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 6] -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] - -(46) HashAggregate [codegen id : 6] -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] -Aggregate Attributes [10]: [count(1)#57, avg(cd_dep_count#26)#58, max(cd_dep_count#26)#59, sum(cd_dep_count#26)#60, avg(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, sum(cd_dep_employed_count#27)#63, avg(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, sum(cd_dep_college_count#28)#66] -Results [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, count(1)#57 AS cnt1#67, avg(cd_dep_count#26)#58 AS avg(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, sum(cd_dep_count#26)#60 AS sum(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, avg(cd_dep_employed_count#27)#61 AS avg(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, sum(cd_dep_employed_count#27)#63 AS sum(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, avg(cd_dep_college_count#28)#64 AS avg(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, sum(cd_dep_college_count#28)#66 AS sum(cd_dep_college_count)#78] - -(47) TakeOrderedAndProject -Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] -Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(52) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/simplified.txt deleted file mode 100644 index fea6a32741..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (6) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] 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/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/explain.txt deleted file mode 100644 index c8f5d42144..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * Filter (27) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(6) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: [d_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#9] -Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -Arguments: [ss_customer_sk#6], [ss_customer_sk#6] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: [ss_customer_sk#6] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Right output [1]: [ss_customer_sk#6] -Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#15] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Right output [1]: [d_date_sk#15] -Arguments: [ws_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#15] -Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#12] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#12] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#19] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] -Right output [1]: [d_date_sk#19] -Arguments: [cs_sold_date_sk#17], [d_date_sk#19], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#19] -Arguments: [cs_ship_customer_sk#16], [cs_ship_customer_sk#16] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#16] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#16] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(27) Filter [codegen id : 5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(28) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : isnotnull(ca_address_sk#20) - -(31) CometProject -Input [2]: [ca_address_sk#20, ca_state#21] -Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#21, 2)) AS ca_state#22] - -(32) CometColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#20, ca_state#22] - -(33) BroadcastExchange -Input [2]: [ca_address_sk#20, ca_state#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(34) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, ca_state#22] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#22] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(37) CometFilter -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#23) - -(38) CometProject -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#24, 1)) AS cd_gender#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#25, 1)) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(39) CometColumnarToRow [codegen id : 4] -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(40) BroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(41) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#23] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 5] -Output [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [8]: [c_current_cdemo_sk#4, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(43) HashAggregate [codegen id : 5] -Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] -Aggregate Attributes [13]: [count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] -Results [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] - -(44) CometColumnarExchange -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 6] -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] - -(46) HashAggregate [codegen id : 6] -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] -Aggregate Attributes [10]: [count(1)#57, avg(cd_dep_count#26)#58, max(cd_dep_count#26)#59, sum(cd_dep_count#26)#60, avg(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, sum(cd_dep_employed_count#27)#63, avg(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, sum(cd_dep_college_count#28)#66] -Results [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, count(1)#57 AS cnt1#67, avg(cd_dep_count#26)#58 AS avg(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, sum(cd_dep_count#26)#60 AS sum(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, avg(cd_dep_employed_count#27)#61 AS avg(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, sum(cd_dep_employed_count#27)#63 AS sum(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, avg(cd_dep_college_count#28)#64 AS avg(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, sum(cd_dep_college_count#28)#66 AS sum(cd_dep_college_count)#78] - -(47) TakeOrderedAndProject -Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] -Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(52) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/extended.txt deleted file mode 100644 index a6f33d6f7e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/extended.txt +++ /dev/null @@ -1,63 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index fea6a32741..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (6) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/explain.txt deleted file mode 100644 index bfbcea798a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/explain.txt +++ /dev/null @@ -1,281 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : +- BroadcastExchange (9) - : : : +- * Project (8) - : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : :- * ColumnarToRow (5) - : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (6) - : : +- BroadcastExchange (22) - : : +- Union (21) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * ColumnarToRow (17) - : : : +- Scan parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (18) - : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.customer_address (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.customer_demographics (32) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(4) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] - -(6) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#7] - -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#4] -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] - -(9) BroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#4] -Join type: LeftSemi -Join condition: None - -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#6)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] - -(13) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#10] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#8 AS customsk#11] -Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#10] - -(16) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#6)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] - -(18) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#14] - -(19) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#12 AS customsk#15] -Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#14] - -(21) Union - -(22) BroadcastExchange -Input [1]: [customsk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(23) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customsk#11] -Join type: LeftSemi -Join condition: None - -(24) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(25) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : isnotnull(ca_address_sk#16) - -(27) CometProject -Input [2]: [ca_address_sk#16, ca_state#17] -Arguments: [ca_address_sk#16, ca_state#18], [ca_address_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#17, 2)) AS ca_state#18] - -(28) CometColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#16, ca_state#18] - -(29) BroadcastExchange -Input [2]: [ca_address_sk#16, ca_state#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(30) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#16] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, ca_state#18] -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#16, ca_state#18] - -(32) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(33) CometFilter -Input [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Condition : isnotnull(cd_demo_sk#19) - -(34) CometProject -Input [6]: [cd_demo_sk#19, cd_gender#20, cd_marital_status#21, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Arguments: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24], [cd_demo_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#20, 1)) AS cd_gender#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#21, 1)) AS cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] - -(35) CometColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] - -(36) BroadcastExchange -Input [6]: [cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#19] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 9] -Output [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Input [8]: [c_current_cdemo_sk#2, ca_state#18, cd_demo_sk#19, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] - -(39) HashAggregate [codegen id : 9] -Input [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#22), partial_max(cd_dep_count#22), partial_sum(cd_dep_count#22), partial_avg(cd_dep_employed_count#23), partial_max(cd_dep_employed_count#23), partial_sum(cd_dep_employed_count#23), partial_avg(cd_dep_college_count#24), partial_max(cd_dep_college_count#24), partial_sum(cd_dep_college_count#24)] -Aggregate Attributes [13]: [count#27, sum#28, count#29, max#30, sum#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39] -Results [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] - -(40) CometColumnarExchange -Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] -Arguments: hashpartitioning(ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(41) CometColumnarToRow [codegen id : 10] -Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] - -(42) HashAggregate [codegen id : 10] -Input [19]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24, count#40, sum#41, count#42, max#43, sum#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52] -Keys [6]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cd_dep_employed_count#23, cd_dep_college_count#24] -Functions [10]: [count(1), avg(cd_dep_count#22), max(cd_dep_count#22), sum(cd_dep_count#22), avg(cd_dep_employed_count#23), max(cd_dep_employed_count#23), sum(cd_dep_employed_count#23), avg(cd_dep_college_count#24), max(cd_dep_college_count#24), sum(cd_dep_college_count#24)] -Aggregate Attributes [10]: [count(1)#53, avg(cd_dep_count#22)#54, max(cd_dep_count#22)#55, sum(cd_dep_count#22)#56, avg(cd_dep_employed_count#23)#57, max(cd_dep_employed_count#23)#58, sum(cd_dep_employed_count#23)#59, avg(cd_dep_college_count#24)#60, max(cd_dep_college_count#24)#61, sum(cd_dep_college_count#24)#62] -Results [18]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, count(1)#53 AS cnt1#63, avg(cd_dep_count#22)#54 AS avg(cd_dep_count)#64, max(cd_dep_count#22)#55 AS max(cd_dep_count)#65, sum(cd_dep_count#22)#56 AS sum(cd_dep_count)#66, cd_dep_employed_count#23, count(1)#53 AS cnt2#67, avg(cd_dep_employed_count#23)#57 AS avg(cd_dep_employed_count)#68, max(cd_dep_employed_count#23)#58 AS max(cd_dep_employed_count)#69, sum(cd_dep_employed_count#23)#59 AS sum(cd_dep_employed_count)#70, cd_dep_college_count#24, count(1)#53 AS cnt3#71, avg(cd_dep_college_count#24)#60 AS avg(cd_dep_college_count)#72, max(cd_dep_college_count#24)#61 AS max(cd_dep_college_count)#73, sum(cd_dep_college_count#24)#62 AS sum(cd_dep_college_count)#74] - -(43) TakeOrderedAndProject -Input [18]: [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cnt1#63, avg(cd_dep_count)#64, max(cd_dep_count)#65, sum(cd_dep_count)#66, cd_dep_employed_count#23, cnt2#67, avg(cd_dep_employed_count)#68, max(cd_dep_employed_count)#69, sum(cd_dep_employed_count)#70, cd_dep_college_count#24, cnt3#71, avg(cd_dep_college_count)#72, max(cd_dep_college_count)#73, sum(cd_dep_college_count)#74] -Arguments: 100, [ca_state#18 ASC NULLS FIRST, cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_dep_count#22 ASC NULLS FIRST, cd_dep_employed_count#23 ASC NULLS FIRST, cd_dep_college_count#24 ASC NULLS FIRST], [ca_state#18, cd_gender#25, cd_marital_status#26, cd_dep_count#22, cnt1#63, avg(cd_dep_count)#64, max(cd_dep_count)#65, sum(cd_dep_count)#66, cd_dep_employed_count#23, cnt2#67, avg(cd_dep_employed_count)#68, max(cd_dep_employed_count)#69, sum(cd_dep_employed_count)#70, cd_dep_college_count#24, cnt3#71, avg(cd_dep_college_count)#72, max(cd_dep_college_count)#73, sum(cd_dep_college_count)#74] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometNativeScan parquet spark_catalog.default.date_dim (44) - - -(44) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#75, d_qoy#76] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [3]: [d_date_sk#7, d_year#75, d_qoy#76] -Condition : ((((isnotnull(d_year#75) AND isnotnull(d_qoy#76)) AND (d_year#75 = 1999)) AND (d_qoy#76 < 4)) AND isnotnull(d_date_sk#7)) - -(46) CometProject -Input [3]: [d_date_sk#7, d_year#75, d_qoy#76] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(48) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/simplified.txt deleted file mode 100644 index bed891a0a3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (10) - HashAggregate [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] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [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,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,customsk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - Union - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan 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.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/explain.txt deleted file mode 100644 index e4756635dc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,267 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometHashAggregate (38) - +- CometExchange (37) - +- CometHashAggregate (36) - +- CometProject (35) - +- CometBroadcastHashJoin (34) - :- CometProject (29) - : +- CometBroadcastHashJoin (28) - : :- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometBroadcastHashJoin (11) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : +- CometBroadcastExchange (10) - : : : +- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : +- CometBroadcastExchange (21) - : : +- CometUnion (20) - : : :- CometProject (15) - : : : +- CometBroadcastHashJoin (14) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (12) - : : : +- ReusedExchange (13) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (17) - : +- CometBroadcastExchange (27) - : +- CometProject (26) - : +- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) - +- CometBroadcastExchange (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (30) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) - -(6) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#4], [ss_customer_sk#4] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: [ss_customer_sk#4] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ss_customer_sk#4] -Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(13) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] -Arguments: [customsk#14], [ws_bill_customer_sk#10 AS customsk#14] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] -ReadSchema: struct - -(17) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#18] - -(18) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Right output [1]: [d_date_sk#18] -Arguments: [cs_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight - -(19) CometProject -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#18] -Arguments: [customsk#19], [cs_ship_customer_sk#15 AS customsk#19] - -(20) CometUnion -Child 0 Input [1]: [customsk#14] -Child 1 Input [1]: [customsk#19] - -(21) CometBroadcastExchange -Input [1]: [customsk#14] -Arguments: [customsk#14] - -(22) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [customsk#14] -Arguments: [c_customer_sk#1], [customsk#14], LeftSemi, BuildRight - -(23) CometProject -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : isnotnull(ca_address_sk#20) - -(26) CometProject -Input [2]: [ca_address_sk#20, ca_state#21] -Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#21, 2)) AS ca_state#22] - -(27) CometBroadcastExchange -Input [2]: [ca_address_sk#20, ca_state#22] -Arguments: [ca_address_sk#20, ca_state#22] - -(28) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [2]: [ca_address_sk#20, ca_state#22] -Arguments: [c_current_addr_sk#3], [ca_address_sk#20], Inner, BuildRight - -(29) CometProject -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20, ca_state#22] -Arguments: [c_current_cdemo_sk#2, ca_state#22], [c_current_cdemo_sk#2, ca_state#22] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#23) - -(32) CometProject -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#24, 1)) AS cd_gender#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#25, 1)) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(33) CometBroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(34) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, ca_state#22] -Right output [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#23], Inner, BuildRight - -(35) CometProject -Input [8]: [c_current_cdemo_sk#2, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(36) CometHashAggregate -Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] - -(37) CometExchange -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] -Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(38) CometHashAggregate -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] - -(39) CometTakeOrderedAndProject -Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#22 ASC NULLS FIRST,cd_gender#29 ASC NULLS FIRST,cd_marital_status#30 ASC NULLS FIRST,cd_dep_count#26 ASC NULLS FIRST,cd_dep_employed_count#27 ASC NULLS FIRST,cd_dep_college_count#28 ASC NULLS FIRST], output=[ca_state#22,cd_gender#29,cd_marital_status#30,cd_dep_count#26,cnt1#44,avg(cd_dep_count)#45,max(cd_dep_count)#46,sum(cd_dep_count)#47,cd_dep_employed_count#27,cnt2#48,avg(cd_dep_employed_count)#49,max(cd_dep_employed_count)#50,sum(cd_dep_employed_count)#51,cd_dep_college_count#28,cnt3#52,avg(cd_dep_college_count)#53,max(cd_dep_college_count)#54,sum(cd_dep_college_count)#55]), [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55], 100, 0, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] - -(40) CometColumnarToRow [codegen id : 1] -Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) - -(43) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(45) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 69e8d4868c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - CometHashAggregate [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - CometExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - 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,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_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] - CometProject [c_current_cdemo_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [customsk] #5 - CometUnion [customsk] - CometProject [ws_bill_customer_sk] [customsk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 [ca_address_sk,ca_state] #6 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] 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/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/explain.txt deleted file mode 100644 index e4756635dc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/explain.txt +++ /dev/null @@ -1,267 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometHashAggregate (38) - +- CometExchange (37) - +- CometHashAggregate (36) - +- CometProject (35) - +- CometBroadcastHashJoin (34) - :- CometProject (29) - : +- CometBroadcastHashJoin (28) - : :- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometBroadcastHashJoin (11) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : +- CometBroadcastExchange (10) - : : : +- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : +- CometBroadcastExchange (21) - : : +- CometUnion (20) - : : :- CometProject (15) - : : : +- CometBroadcastHashJoin (14) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (12) - : : : +- ReusedExchange (13) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (17) - : +- CometBroadcastExchange (27) - : +- CometProject (26) - : +- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) - +- CometBroadcastExchange (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (30) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) - -(6) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#4], [ss_customer_sk#4] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: [ss_customer_sk#4] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ss_customer_sk#4] -Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(13) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#11], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] -Arguments: [customsk#14], [ws_bill_customer_sk#10 AS customsk#14] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] -ReadSchema: struct - -(17) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#18] - -(18) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Right output [1]: [d_date_sk#18] -Arguments: [cs_sold_date_sk#16], [d_date_sk#18], Inner, BuildRight - -(19) CometProject -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#18] -Arguments: [customsk#19], [cs_ship_customer_sk#15 AS customsk#19] - -(20) CometUnion -Child 0 Input [1]: [customsk#14] -Child 1 Input [1]: [customsk#19] - -(21) CometBroadcastExchange -Input [1]: [customsk#14] -Arguments: [customsk#14] - -(22) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [customsk#14] -Arguments: [c_customer_sk#1], [customsk#14], LeftSemi, BuildRight - -(23) CometProject -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : isnotnull(ca_address_sk#20) - -(26) CometProject -Input [2]: [ca_address_sk#20, ca_state#21] -Arguments: [ca_address_sk#20, ca_state#22], [ca_address_sk#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#21, 2)) AS ca_state#22] - -(27) CometBroadcastExchange -Input [2]: [ca_address_sk#20, ca_state#22] -Arguments: [ca_address_sk#20, ca_state#22] - -(28) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [2]: [ca_address_sk#20, ca_state#22] -Arguments: [c_current_addr_sk#3], [ca_address_sk#20], Inner, BuildRight - -(29) CometProject -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20, ca_state#22] -Arguments: [c_current_cdemo_sk#2, ca_state#22], [c_current_cdemo_sk#2, ca_state#22] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#23) - -(32) CometProject -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [cd_demo_sk#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_gender#24, 1)) AS cd_gender#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#25, 1)) AS cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(33) CometBroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(34) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, ca_state#22] -Right output [6]: [cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#23], Inner, BuildRight - -(35) CometProject -Input [8]: [c_current_cdemo_sk#2, ca_state#22, cd_demo_sk#23, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] - -(36) CometHashAggregate -Input [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] - -(37) CometExchange -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] -Arguments: hashpartitioning(ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(38) CometHashAggregate -Input [19]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] -Keys [6]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] - -(39) CometTakeOrderedAndProject -Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#22 ASC NULLS FIRST,cd_gender#29 ASC NULLS FIRST,cd_marital_status#30 ASC NULLS FIRST,cd_dep_count#26 ASC NULLS FIRST,cd_dep_employed_count#27 ASC NULLS FIRST,cd_dep_college_count#28 ASC NULLS FIRST], output=[ca_state#22,cd_gender#29,cd_marital_status#30,cd_dep_count#26,cnt1#44,avg(cd_dep_count)#45,max(cd_dep_count)#46,sum(cd_dep_count)#47,cd_dep_employed_count#27,cnt2#48,avg(cd_dep_employed_count)#49,max(cd_dep_employed_count)#50,sum(cd_dep_employed_count)#51,cd_dep_college_count#28,cnt3#52,avg(cd_dep_college_count)#53,max(cd_dep_college_count)#54,sum(cd_dep_college_count)#55]), [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55], 100, 0, [ca_state#22 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] - -(40) CometColumnarToRow [codegen id : 1] -Input [18]: [ca_state#22, cd_gender#29, cd_marital_status#30, cd_dep_count#26, cnt1#44, avg(cd_dep_count)#45, max(cd_dep_count)#46, sum(cd_dep_count)#47, cd_dep_employed_count#27, cnt2#48, avg(cd_dep_employed_count)#49, max(cd_dep_employed_count)#50, sum(cd_dep_employed_count)#51, cd_dep_college_count#28, cnt3#52, avg(cd_dep_college_count)#53, max(cd_dep_college_count)#54, sum(cd_dep_college_count)#55] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) - -(43) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(45) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/extended.txt deleted file mode 100644 index 2cdc75e15e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/extended.txt +++ /dev/null @@ -1,56 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 69e8d4868c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - CometHashAggregate [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - CometExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - 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,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_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] - CometProject [c_current_cdemo_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [customsk] #5 - CometUnion [customsk] - CometProject [ws_bill_customer_sk] [customsk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 [ca_address_sk,ca_state] #6 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/explain.txt deleted file mode 100644 index 19961d7852..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- CometExchange (43) - +- CometHashAggregate (42) - +- CometColumnarExchange (41) - +- * HashAggregate (40) - +- Union (39) - :- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.item (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.store (14) - :- * HashAggregate (31) - : +- * CometColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- * HashAggregate (28) - : +- * HashAggregate (27) - : +- * CometColumnarToRow (26) - : +- ReusedExchange (25) - +- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * HashAggregate (34) - +- * CometColumnarToRow (33) - +- ReusedExchange (32) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) - -(4) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#7] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] - -(7) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Condition : isnotnull(i_item_sk#8) - -(9) CometProject -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Arguments: [i_item_sk#8, i_class#11, i_category#12], [i_item_sk#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#12] - -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#8, i_class#11, i_category#12] - -(11) BroadcastExchange -Input [3]: [i_item_sk#8, i_class#11, i_category#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#8] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#8, i_class#11, i_category#12] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#13, s_state#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#13, s_state#14] -Condition : ((isnotnull(s_state#14) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#14, 2)) = TN)) AND isnotnull(s_store_sk#13)) - -(16) CometProject -Input [2]: [s_store_sk#13, s_state#14] -Arguments: [s_store_sk#13], [s_store_sk#13] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#13] - -(18) BroadcastExchange -Input [1]: [s_store_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#13] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12, s_store_sk#13] - -(21) HashAggregate [codegen id : 4] -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] -Keys [2]: [i_category#12, i_class#11] -Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum#15, sum#16] -Results [4]: [i_category#12, i_class#11, sum#17, sum#18] - -(22) CometColumnarExchange -Input [4]: [i_category#12, i_class#11, sum#17, sum#18] -Arguments: hashpartitioning(i_category#12, i_class#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 5] -Input [4]: [i_category#12, i_class#11, sum#17, sum#18] - -(24) HashAggregate [codegen id : 5] -Input [4]: [i_category#12, i_class#11, sum#17, sum#18] -Keys [2]: [i_category#12, i_class#11] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#19, sum(UnscaledValue(ss_ext_sales_price#3))#20] -Results [6]: [cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#19,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#20,17,2)) as decimal(38,11)) AS gross_margin#21, i_category#12 AS i_category#22, i_class#11 AS i_class#23, 0 AS t_category#24, 0 AS t_class#25, 0 AS lochierarchy#26] - -(25) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#27, i_class#28, sum#29, sum#30] - -(26) CometColumnarToRow [codegen id : 10] -Input [4]: [i_category#27, i_class#28, sum#29, sum#30] - -(27) HashAggregate [codegen id : 10] -Input [4]: [i_category#27, i_class#28, sum#29, sum#30] -Keys [2]: [i_category#27, i_class#28] -Functions [2]: [sum(UnscaledValue(ss_net_profit#31)), sum(UnscaledValue(ss_ext_sales_price#32))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#31))#33, sum(UnscaledValue(ss_ext_sales_price#32))#34] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#31))#33,17,2) AS ss_net_profit#35, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#32))#34,17,2) AS ss_ext_sales_price#36, i_category#27] - -(28) HashAggregate [codegen id : 10] -Input [3]: [ss_net_profit#35, ss_ext_sales_price#36, i_category#27] -Keys [1]: [i_category#27] -Functions [2]: [partial_sum(ss_net_profit#35), partial_sum(ss_ext_sales_price#36)] -Aggregate Attributes [4]: [sum#37, isEmpty#38, sum#39, isEmpty#40] -Results [5]: [i_category#27, sum#41, isEmpty#42, sum#43, isEmpty#44] - -(29) CometColumnarExchange -Input [5]: [i_category#27, sum#41, isEmpty#42, sum#43, isEmpty#44] -Arguments: hashpartitioning(i_category#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 11] -Input [5]: [i_category#27, sum#41, isEmpty#42, sum#43, isEmpty#44] - -(31) HashAggregate [codegen id : 11] -Input [5]: [i_category#27, sum#41, isEmpty#42, sum#43, isEmpty#44] -Keys [1]: [i_category#27] -Functions [2]: [sum(ss_net_profit#35), sum(ss_ext_sales_price#36)] -Aggregate Attributes [2]: [sum(ss_net_profit#35)#45, sum(ss_ext_sales_price#36)#46] -Results [6]: [(sum(ss_net_profit#35)#45 / sum(ss_ext_sales_price#36)#46) AS gross_margin#47, i_category#27, null AS i_class#48, 0 AS t_category#49, 1 AS t_class#50, 1 AS lochierarchy#51] - -(32) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#52, i_class#53, sum#54, sum#55] - -(33) CometColumnarToRow [codegen id : 16] -Input [4]: [i_category#52, i_class#53, sum#54, sum#55] - -(34) HashAggregate [codegen id : 16] -Input [4]: [i_category#52, i_class#53, sum#54, sum#55] -Keys [2]: [i_category#52, i_class#53] -Functions [2]: [sum(UnscaledValue(ss_net_profit#56)), sum(UnscaledValue(ss_ext_sales_price#57))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#56))#33, sum(UnscaledValue(ss_ext_sales_price#57))#34] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#56))#33,17,2) AS ss_net_profit#58, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#57))#34,17,2) AS ss_ext_sales_price#59] - -(35) HashAggregate [codegen id : 16] -Input [2]: [ss_net_profit#58, ss_ext_sales_price#59] -Keys: [] -Functions [2]: [partial_sum(ss_net_profit#58), partial_sum(ss_ext_sales_price#59)] -Aggregate Attributes [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] -Results [4]: [sum#64, isEmpty#65, sum#66, isEmpty#67] - -(36) CometColumnarExchange -Input [4]: [sum#64, isEmpty#65, sum#66, isEmpty#67] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(37) CometColumnarToRow [codegen id : 17] -Input [4]: [sum#64, isEmpty#65, sum#66, isEmpty#67] - -(38) HashAggregate [codegen id : 17] -Input [4]: [sum#64, isEmpty#65, sum#66, isEmpty#67] -Keys: [] -Functions [2]: [sum(ss_net_profit#58), sum(ss_ext_sales_price#59)] -Aggregate Attributes [2]: [sum(ss_net_profit#58)#68, sum(ss_ext_sales_price#59)#69] -Results [6]: [(sum(ss_net_profit#58)#68 / sum(ss_ext_sales_price#59)#69) AS gross_margin#70, null AS i_category#71, null AS i_class#72, 1 AS t_category#73, 1 AS t_class#74, 2 AS lochierarchy#75] - -(39) Union - -(40) HashAggregate [codegen id : 18] -Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Keys [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Functions: [] -Aggregate Attributes: [] -Results [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] - -(41) CometColumnarExchange -Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Arguments: hashpartitioning(gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(42) CometHashAggregate -Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Keys [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Functions: [] - -(43) CometExchange -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#76] -Arguments: hashpartitioning(lochierarchy#26, _w0#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(44) CometSort -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#76] -Arguments: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#76], [lochierarchy#26 ASC NULLS FIRST, _w0#76 ASC NULLS FIRST, gross_margin#21 ASC NULLS FIRST] - -(45) CometColumnarToRow [codegen id : 19] -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#76] - -(46) Window -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#76] -Arguments: [rank(gross_margin#21) windowspecdefinition(lochierarchy#26, _w0#76, gross_margin#21 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#77], [lochierarchy#26, _w0#76], [gross_margin#21 ASC NULLS FIRST] - -(47) Project [codegen id : 20] -Output [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#77] -Input [6]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#76, rank_within_parent#77] - -(48) TakeOrderedAndProject -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#77] -Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN i_category#22 END ASC NULLS FIRST, rank_within_parent#77 ASC NULLS FIRST], [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#77] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometNativeScan parquet spark_catalog.default.date_dim (49) - - -(49) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [2]: [d_date_sk#7, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#7)) - -(51) CometProject -Input [2]: [d_date_sk#7, d_year#78] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(53) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/simplified.txt deleted file mode 100644 index 65728fc6f2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/simplified.txt +++ /dev/null @@ -1,81 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (20) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] - CometColumnarExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - WholeStageCodegen (18) - HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #3 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - WholeStageCodegen (11) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #7 - WholeStageCodegen (10) - HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (16) - HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/explain.txt deleted file mode 100644 index 7233391f5e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,275 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- Window (41) - +- * CometColumnarToRow (40) - +- CometSort (39) - +- CometExchange (38) - +- CometHashAggregate (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometUnion (34) - :- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometHashAggregate (25) - : +- ReusedExchange (24) - +- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometHashAggregate (30) - +- ReusedExchange (29) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4], [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#9, i_class#10, i_category#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Condition : isnotnull(i_item_sk#9) - -(11) CometProject -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Arguments: [i_item_sk#9, i_class#12, i_category#13], [i_item_sk#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#10, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#11, 50)) AS i_category#13] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [i_item_sk#9, i_class#12, i_category#13] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Right output [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#9], Inner, BuildRight - -(14) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : ((isnotnull(s_state#15) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#15, 2)) = TN)) AND isnotnull(s_store_sk#14)) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14], [s_store_sk#14] - -(18) CometBroadcastExchange -Input [1]: [s_store_sk#14] -Arguments: [s_store_sk#14] - -(19) CometBroadcastHashJoin -Left output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -Right output [1]: [s_store_sk#14] -Arguments: [ss_store_sk#2], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13, s_store_sk#14] -Arguments: [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] - -(21) CometHashAggregate -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(22) CometExchange -Input [4]: [i_category#13, i_class#12, sum#16, sum#17] -Arguments: hashpartitioning(i_category#13, i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [4]: [i_category#13, i_class#12, sum#16, sum#17] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] - -(24) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#18, i_class#19, sum#20, sum#21] - -(25) CometHashAggregate -Input [4]: [i_category#18, i_class#19, sum#20, sum#21] -Keys [2]: [i_category#18, i_class#19] -Functions [2]: [sum(UnscaledValue(ss_net_profit#22)), sum(UnscaledValue(ss_ext_sales_price#23))] - -(26) CometHashAggregate -Input [3]: [ss_net_profit#24, ss_ext_sales_price#25, i_category#18] -Keys [1]: [i_category#18] -Functions [2]: [partial_sum(ss_net_profit#24), partial_sum(ss_ext_sales_price#25)] - -(27) CometExchange -Input [5]: [i_category#18, sum#26, isEmpty#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [5]: [i_category#18, sum#26, isEmpty#27, sum#28, isEmpty#29] -Keys [1]: [i_category#18] -Functions [2]: [sum(ss_net_profit#24), sum(ss_ext_sales_price#25)] - -(29) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#30, i_class#31, sum#32, sum#33] - -(30) CometHashAggregate -Input [4]: [i_category#30, i_class#31, sum#32, sum#33] -Keys [2]: [i_category#30, i_class#31] -Functions [2]: [sum(UnscaledValue(ss_net_profit#34)), sum(UnscaledValue(ss_ext_sales_price#35))] - -(31) CometHashAggregate -Input [2]: [ss_net_profit#36, ss_ext_sales_price#37] -Keys: [] -Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] - -(32) CometExchange -Input [4]: [sum#38, isEmpty#39, sum#40, isEmpty#41] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometHashAggregate -Input [4]: [sum#38, isEmpty#39, sum#40, isEmpty#41] -Keys: [] -Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] - -(34) CometUnion -Child 0 Input [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] -Child 1 Input [6]: [gross_margin#48, i_category#18, i_class#49, t_category#50, t_class#51, lochierarchy#52] -Child 2 Input [6]: [gross_margin#53, i_category#54, i_class#55, t_category#56, t_class#57, lochierarchy#58] - -(35) CometHashAggregate -Input [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] -Keys [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] -Functions: [] - -(36) CometExchange -Input [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] -Arguments: hashpartitioning(gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(37) CometHashAggregate -Input [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] -Keys [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] -Functions: [] - -(38) CometExchange -Input [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59] -Arguments: hashpartitioning(lochierarchy#47, _w0#59, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(39) CometSort -Input [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59] -Arguments: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59], [lochierarchy#47 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, gross_margin#42 ASC NULLS FIRST] - -(40) CometColumnarToRow [codegen id : 1] -Input [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59] - -(41) Window -Input [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59] -Arguments: [rank(gross_margin#42) windowspecdefinition(lochierarchy#47, _w0#59, gross_margin#42 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#60], [lochierarchy#47, _w0#59], [gross_margin#42 ASC NULLS FIRST] - -(42) Project [codegen id : 2] -Output [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, rank_within_parent#60] -Input [6]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59, rank_within_parent#60] - -(43) TakeOrderedAndProject -Input [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, rank_within_parent#60] -Arguments: 100, [lochierarchy#47 DESC NULLS LAST, CASE WHEN (lochierarchy#47 = 0) THEN i_category#43 END ASC NULLS FIRST, rank_within_parent#60 ASC NULLS FIRST], [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, rank_within_parent#60] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) - - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(46) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(48) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/simplified.txt deleted file mode 100644 index e91b278c14..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,55 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (2) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] - CometExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometUnion [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - 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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange [i_category] #8 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,i_category,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange #9 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_category,i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt deleted file mode 100644 index 7233391f5e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt +++ /dev/null @@ -1,275 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- Window (41) - +- * CometColumnarToRow (40) - +- CometSort (39) - +- CometExchange (38) - +- CometHashAggregate (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometUnion (34) - :- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometHashAggregate (25) - : +- ReusedExchange (24) - +- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometHashAggregate (30) - +- ReusedExchange (29) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4], [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#9, i_class#10, i_category#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Condition : isnotnull(i_item_sk#9) - -(11) CometProject -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Arguments: [i_item_sk#9, i_class#12, i_category#13], [i_item_sk#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#10, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#11, 50)) AS i_category#13] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [i_item_sk#9, i_class#12, i_category#13] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Right output [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#9], Inner, BuildRight - -(14) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : ((isnotnull(s_state#15) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#15, 2)) = TN)) AND isnotnull(s_store_sk#14)) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14], [s_store_sk#14] - -(18) CometBroadcastExchange -Input [1]: [s_store_sk#14] -Arguments: [s_store_sk#14] - -(19) CometBroadcastHashJoin -Left output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -Right output [1]: [s_store_sk#14] -Arguments: [ss_store_sk#2], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13, s_store_sk#14] -Arguments: [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] - -(21) CometHashAggregate -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(22) CometExchange -Input [4]: [i_category#13, i_class#12, sum#16, sum#17] -Arguments: hashpartitioning(i_category#13, i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [4]: [i_category#13, i_class#12, sum#16, sum#17] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] - -(24) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#18, i_class#19, sum#20, sum#21] - -(25) CometHashAggregate -Input [4]: [i_category#18, i_class#19, sum#20, sum#21] -Keys [2]: [i_category#18, i_class#19] -Functions [2]: [sum(UnscaledValue(ss_net_profit#22)), sum(UnscaledValue(ss_ext_sales_price#23))] - -(26) CometHashAggregate -Input [3]: [ss_net_profit#24, ss_ext_sales_price#25, i_category#18] -Keys [1]: [i_category#18] -Functions [2]: [partial_sum(ss_net_profit#24), partial_sum(ss_ext_sales_price#25)] - -(27) CometExchange -Input [5]: [i_category#18, sum#26, isEmpty#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [5]: [i_category#18, sum#26, isEmpty#27, sum#28, isEmpty#29] -Keys [1]: [i_category#18] -Functions [2]: [sum(ss_net_profit#24), sum(ss_ext_sales_price#25)] - -(29) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#30, i_class#31, sum#32, sum#33] - -(30) CometHashAggregate -Input [4]: [i_category#30, i_class#31, sum#32, sum#33] -Keys [2]: [i_category#30, i_class#31] -Functions [2]: [sum(UnscaledValue(ss_net_profit#34)), sum(UnscaledValue(ss_ext_sales_price#35))] - -(31) CometHashAggregate -Input [2]: [ss_net_profit#36, ss_ext_sales_price#37] -Keys: [] -Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] - -(32) CometExchange -Input [4]: [sum#38, isEmpty#39, sum#40, isEmpty#41] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometHashAggregate -Input [4]: [sum#38, isEmpty#39, sum#40, isEmpty#41] -Keys: [] -Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] - -(34) CometUnion -Child 0 Input [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] -Child 1 Input [6]: [gross_margin#48, i_category#18, i_class#49, t_category#50, t_class#51, lochierarchy#52] -Child 2 Input [6]: [gross_margin#53, i_category#54, i_class#55, t_category#56, t_class#57, lochierarchy#58] - -(35) CometHashAggregate -Input [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] -Keys [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] -Functions: [] - -(36) CometExchange -Input [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] -Arguments: hashpartitioning(gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(37) CometHashAggregate -Input [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] -Keys [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] -Functions: [] - -(38) CometExchange -Input [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59] -Arguments: hashpartitioning(lochierarchy#47, _w0#59, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(39) CometSort -Input [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59] -Arguments: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59], [lochierarchy#47 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, gross_margin#42 ASC NULLS FIRST] - -(40) CometColumnarToRow [codegen id : 1] -Input [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59] - -(41) Window -Input [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59] -Arguments: [rank(gross_margin#42) windowspecdefinition(lochierarchy#47, _w0#59, gross_margin#42 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#60], [lochierarchy#47, _w0#59], [gross_margin#42 ASC NULLS FIRST] - -(42) Project [codegen id : 2] -Output [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, rank_within_parent#60] -Input [6]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59, rank_within_parent#60] - -(43) TakeOrderedAndProject -Input [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, rank_within_parent#60] -Arguments: 100, [lochierarchy#47 DESC NULLS LAST, CASE WHEN (lochierarchy#47 = 0) THEN i_category#43 END ASC NULLS FIRST, rank_within_parent#60 ASC NULLS FIRST], [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, rank_within_parent#60] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) - - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(46) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(48) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/extended.txt deleted file mode 100644 index 1f6c984b22..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index e91b278c14..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt +++ /dev/null @@ -1,55 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (2) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] - CometExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometUnion [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - 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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange [i_category] #8 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,i_category,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange #9 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_category,i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/explain.txt deleted file mode 100644 index 490593f3bf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/explain.txt +++ /dev/null @@ -1,304 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (51) -+- * Project (50) - +- * BroadcastHashJoin Inner BuildRight (49) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * CometColumnarToRow (26) - : : +- CometSort (25) - : : +- CometColumnarExchange (24) - : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- * HashAggregate (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.store (14) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- Window (38) - : +- * CometColumnarToRow (37) - : +- CometSort (36) - : +- CometColumnarExchange (35) - : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) - : +- ReusedExchange (32) - +- BroadcastExchange (48) - +- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- ReusedExchange (43) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)))) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) AS i_brand#4, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)) AS i_category#5] - -(4) CometColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#4, i_category#5] - -(5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(7) Filter [codegen id : 1] -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) - -(8) BroadcastExchange -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#6] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(11) ReusedExchange [Reuses operator id: 55] -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] -Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) - -(16) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] - -(17) BroadcastExchange -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#7] -Right keys [1]: [s_store_sk#14] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] - -(20) HashAggregate [codegen id : 4] -Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] -Aggregate Attributes [1]: [sum#17] -Results [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] - -(21) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] - -(23) HashAggregate [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#8))#19] -Results [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#19,17,2) AS sum_sales#20, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#19,17,2) AS _w0#21] - -(24) CometColumnarExchange -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) CometColumnarToRow [codegen id : 6] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] - -(27) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(28) Filter [codegen id : 7] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(29) Window -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] -Arguments: [avg(_w0#21) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(30) Filter [codegen id : 22] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] -Condition : ((isnotnull(avg_monthly_sales#23) AND (avg_monthly_sales#23 > 0.000000)) AND CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END) - -(31) Project [codegen id : 22] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] - -(32) ReusedExchange [Reuses operator id: 21] -Output [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum#30] - -(33) CometColumnarToRow [codegen id : 12] -Input [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum#30] - -(34) HashAggregate [codegen id : 12] -Input [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum#30] -Keys [6]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29] -Functions [1]: [sum(UnscaledValue(ss_sales_price#31))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#31))#19] -Results [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, MakeDecimal(sum(UnscaledValue(ss_sales_price#31))#19,17,2) AS sum_sales#32] - -(35) CometColumnarExchange -Input [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum_sales#32] -Arguments: hashpartitioning(i_category#24, i_brand#25, s_store_name#26, s_company_name#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(36) CometSort -Input [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum_sales#32] -Arguments: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum_sales#32], [i_category#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, s_store_name#26 ASC NULLS FIRST, s_company_name#27 ASC NULLS FIRST, d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST] - -(37) CometColumnarToRow [codegen id : 13] -Input [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum_sales#32] - -(38) Window -Input [7]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum_sales#32] -Arguments: [rank(d_year#28, d_moy#29) windowspecdefinition(i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#24, i_brand#25, s_store_name#26, s_company_name#27], [d_year#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST] - -(39) Project [codegen id : 14] -Output [6]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, sum_sales#32, rn#33] -Input [8]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, d_year#28, d_moy#29, sum_sales#32, rn#33] - -(40) BroadcastExchange -Input [6]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, sum_sales#32, rn#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] - -(41) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] -Right keys [5]: [i_category#24, i_brand#25, s_store_name#26, s_company_name#27, (rn#33 + 1)] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 22] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#32] -Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, i_category#24, i_brand#25, s_store_name#26, s_company_name#27, sum_sales#32, rn#33] - -(43) ReusedExchange [Reuses operator id: 35] -Output [7]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38, d_moy#39, sum_sales#40] - -(44) CometSort -Input [7]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38, d_moy#39, sum_sales#40] -Arguments: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38, d_moy#39, sum_sales#40], [i_category#34 ASC NULLS FIRST, i_brand#35 ASC NULLS FIRST, s_store_name#36 ASC NULLS FIRST, s_company_name#37 ASC NULLS FIRST, d_year#38 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST] - -(45) CometColumnarToRow [codegen id : 20] -Input [7]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38, d_moy#39, sum_sales#40] - -(46) Window -Input [7]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38, d_moy#39, sum_sales#40] -Arguments: [rank(d_year#38, d_moy#39) windowspecdefinition(i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#34, i_brand#35, s_store_name#36, s_company_name#37], [d_year#38 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST] - -(47) Project [codegen id : 21] -Output [6]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, sum_sales#40, rn#41] -Input [8]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, d_year#38, d_moy#39, sum_sales#40, rn#41] - -(48) BroadcastExchange -Input [6]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, sum_sales#40, rn#41] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] - -(49) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] -Right keys [5]: [i_category#34, i_brand#35, s_store_name#36, s_company_name#37, (rn#41 - 1)] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 22] -Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, sum_sales#32 AS psum#42, sum_sales#40 AS nsum#43] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#32, i_category#34, i_brand#35, s_store_name#36, s_company_name#37, sum_sales#40, rn#41] - -(51) TakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] -Arguments: 100, [(sum_sales#20 - avg_monthly_sales#23) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometFilter (53) - +- CometNativeScan parquet spark_catalog.default.date_dim (52) - - -(52) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(54) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(55) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/simplified.txt deleted file mode 100644 index ee8e4dd40c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/simplified.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (22) - Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (7) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/explain.txt deleted file mode 100644 index bf8cb9a216..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)))) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) AS i_brand#4, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_item_sk#1], [ss_item_sk#6], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) - -(16) CometBroadcastExchange -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [s_store_sk#14, s_store_name#15, s_company_name#16] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] -Right output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [ss_store_sk#7], [s_store_sk#14], Inner, BuildRight - -(18) CometProject -Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16], [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] - -(19) CometHashAggregate -Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] - -(20) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] - -(22) CometExchange -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] - -(25) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(28) Filter [codegen id : 7] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] - -(30) ReusedExchange [Reuses operator id: 20] -Output [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] - -(31) CometHashAggregate -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] -Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] - -(32) CometExchange -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] -Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] -Arguments: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30], [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] - -(35) Window -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] -Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#31], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] -Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30, rn#31] - -(37) BroadcastExchange -Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#31 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30] -Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] - -(40) ReusedExchange [Reuses operator id: 32] -Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] - -(41) CometSort -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] -Arguments: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38], [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] - -(43) Window -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] -Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] -Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38, rn#39] - -(45) BroadcastExchange -Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#39 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#30 AS psum#40, sum_sales#38 AS nsum#41] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] - -(48) TakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/simplified.txt deleted file mode 100644 index a62c33ecc1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (7) - Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - 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 [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,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt deleted file mode 100644 index bf8cb9a216..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)))) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) AS i_brand#4, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_item_sk#1], [ss_item_sk#6], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) - -(16) CometBroadcastExchange -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [s_store_sk#14, s_store_name#15, s_company_name#16] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] -Right output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [ss_store_sk#7], [s_store_sk#14], Inner, BuildRight - -(18) CometProject -Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16], [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] - -(19) CometHashAggregate -Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] - -(20) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] - -(22) CometExchange -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] - -(25) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(28) Filter [codegen id : 7] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] - -(30) ReusedExchange [Reuses operator id: 20] -Output [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] - -(31) CometHashAggregate -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] -Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] - -(32) CometExchange -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] -Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] -Arguments: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30], [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] - -(35) Window -Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] -Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#31], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] -Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30, rn#31] - -(37) BroadcastExchange -Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#31 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30] -Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] - -(40) ReusedExchange [Reuses operator id: 32] -Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] - -(41) CometSort -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] -Arguments: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38], [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] - -(43) Window -Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] -Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] -Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38, rn#39] - -(45) BroadcastExchange -Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#39 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#30 AS psum#40, sum_sales#38 AS nsum#41] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] - -(48) TakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/extended.txt deleted file mode 100644 index b50b570b4b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index a62c33ecc1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (7) - Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - 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 [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,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/explain.txt deleted file mode 100644 index 070113d8ea..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/explain.txt +++ /dev/null @@ -1,497 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (84) -+- CometTakeOrderedAndProject (83) - +- CometHashAggregate (82) - +- CometColumnarExchange (81) - +- * HashAggregate (80) - +- Union (79) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometColumnarExchange (19) - : +- * HashAggregate (18) - : +- * CometColumnarToRow (17) - : +- CometColumnarExchange (16) - : +- * HashAggregate (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildLeft (10) - : : :- BroadcastExchange (5) - : : : +- * Project (4) - : : : +- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : +- * CometColumnarToRow (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometNativeScan parquet spark_catalog.default.web_returns (6) - : +- ReusedExchange (12) - :- * Project (52) - : +- * Filter (51) - : +- Window (50) - : +- * Sort (49) - : +- Window (48) - : +- * CometColumnarToRow (47) - : +- CometSort (46) - : +- CometColumnarExchange (45) - : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- CometColumnarExchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildLeft (36) - : : :- BroadcastExchange (31) - : : : +- * Project (30) - : : : +- * Filter (29) - : : : +- * ColumnarToRow (28) - : : : +- Scan parquet spark_catalog.default.catalog_sales (27) - : : +- * CometColumnarToRow (35) - : : +- CometProject (34) - : : +- CometFilter (33) - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (32) - : +- ReusedExchange (38) - +- * Project (78) - +- * Filter (77) - +- Window (76) - +- * Sort (75) - +- Window (74) - +- * CometColumnarToRow (73) - +- CometSort (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * CometColumnarToRow (69) - +- CometColumnarExchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * BroadcastHashJoin Inner BuildRight (65) - :- * Project (63) - : +- * BroadcastHashJoin Inner BuildLeft (62) - : :- BroadcastExchange (57) - : : +- * Project (56) - : : +- * Filter (55) - : : +- * ColumnarToRow (54) - : : +- Scan parquet spark_catalog.default.store_sales (53) - : +- * CometColumnarToRow (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometNativeScan parquet spark_catalog.default.store_returns (58) - +- ReusedExchange (64) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] - -(3) Filter [codegen id : 1] -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND isnotnull(ws_quantity#3)) AND (ws_net_profit#5 > 1.00)) AND (ws_net_paid#4 > 0.00)) AND (ws_quantity#3 > 0)) AND isnotnull(ws_order_number#2)) AND isnotnull(ws_item_sk#1)) - -(4) Project [codegen id : 1] -Output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] - -(5) BroadcastExchange -Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=1] - -(6) CometNativeScan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) - -(8) CometProject -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(9) CometColumnarToRow -Input [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(10) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [ws_order_number#2, ws_item_sk#1] -Right keys [2]: [wr_order_number#9, wr_item_sk#8] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 3] -Output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(12) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#13] - -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#6] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 3] -Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] - -(15) HashAggregate [codegen id : 3] -Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Keys [1]: [ws_item_sk#1] -Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#14, sum#15, sum#16, isEmpty#17, sum#18, isEmpty#19] -Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] - -(16) CometColumnarExchange -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(17) CometColumnarToRow [codegen id : 4] -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] - -(18) HashAggregate [codegen id : 4] -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Keys [1]: [ws_item_sk#1] -Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] -Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] - -(19) CometColumnarExchange -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(20) CometSort -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [item#30, return_ratio#31, currency_ratio#32], [return_ratio#31 ASC NULLS FIRST] - -(21) CometColumnarToRow [codegen id : 5] -Input [3]: [item#30, return_ratio#31, currency_ratio#32] - -(22) Window -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] - -(23) Sort [codegen id : 6] -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 - -(24) Window -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] - -(25) Filter [codegen id : 7] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] -Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) - -(26) Project [codegen id : 7] -Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] - -(27) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(28) ColumnarToRow [codegen id : 8] -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] - -(29) Filter [codegen id : 8] -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) - -(30) Project [codegen id : 8] -Output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] - -(31) BroadcastExchange -Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=4] - -(32) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] -Condition : (((isnotnull(cr_return_amount#45) AND (cr_return_amount#45 > 10000.00)) AND isnotnull(cr_order_number#43)) AND isnotnull(cr_item_sk#42)) - -(34) CometProject -Input [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] -Arguments: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45], [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] - -(35) CometColumnarToRow -Input [4]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] - -(36) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [cs_order_number#37, cs_item_sk#36] -Right keys [2]: [cr_order_number#43, cr_item_sk#42] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 10] -Output [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45] -Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] - -(38) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#47] - -(39) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#41] -Right keys [1]: [d_date_sk#47] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 10] -Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] -Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45, d_date_sk#47] - -(41) HashAggregate [codegen id : 10] -Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] -Keys [1]: [cs_item_sk#36] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#44, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#48, sum#49, sum#50, isEmpty#51, sum#52, isEmpty#53] -Results [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] - -(42) CometColumnarExchange -Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(43) CometColumnarToRow [codegen id : 11] -Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] - -(44) HashAggregate [codegen id : 11] -Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Keys [1]: [cs_item_sk#36] -Functions [4]: [sum(coalesce(cr_return_quantity#44, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#44, 0))#60, sum(coalesce(cs_quantity#38, 0))#61, sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#62, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63] -Results [3]: [cs_item_sk#36 AS item#64, (cast(sum(coalesce(cr_return_quantity#44, 0))#60 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#61 as decimal(15,4))) AS return_ratio#65, (cast(sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#62 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63 as decimal(15,4))) AS currency_ratio#66] - -(45) CometColumnarExchange -Input [3]: [item#64, return_ratio#65, currency_ratio#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(46) CometSort -Input [3]: [item#64, return_ratio#65, currency_ratio#66] -Arguments: [item#64, return_ratio#65, currency_ratio#66], [return_ratio#65 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 12] -Input [3]: [item#64, return_ratio#65, currency_ratio#66] - -(48) Window -Input [3]: [item#64, return_ratio#65, currency_ratio#66] -Arguments: [rank(return_ratio#65) windowspecdefinition(return_ratio#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#67], [return_ratio#65 ASC NULLS FIRST] - -(49) Sort [codegen id : 13] -Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] -Arguments: [currency_ratio#66 ASC NULLS FIRST], false, 0 - -(50) Window -Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] -Arguments: [rank(currency_ratio#66) windowspecdefinition(currency_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#68], [currency_ratio#66 ASC NULLS FIRST] - -(51) Filter [codegen id : 14] -Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] -Condition : ((return_rank#67 <= 10) OR (currency_rank#68 <= 10)) - -(52) Project [codegen id : 14] -Output [5]: [catalog AS channel#69, item#64, return_ratio#65, return_rank#67, currency_rank#68] -Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] - -(53) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#75), dynamicpruningexpression(ss_sold_date_sk#75 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(54) ColumnarToRow [codegen id : 15] -Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] - -(55) Filter [codegen id : 15] -Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] -Condition : (((((((isnotnull(ss_net_profit#74) AND isnotnull(ss_net_paid#73)) AND isnotnull(ss_quantity#72)) AND (ss_net_profit#74 > 1.00)) AND (ss_net_paid#73 > 0.00)) AND (ss_quantity#72 > 0)) AND isnotnull(ss_ticket_number#71)) AND isnotnull(ss_item_sk#70)) - -(56) Project [codegen id : 15] -Output [5]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75] -Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] - -(57) BroadcastExchange -Input [5]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=7] - -(58) CometNativeScan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(59) CometFilter -Input [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] -Condition : (((isnotnull(sr_return_amt#79) AND (sr_return_amt#79 > 10000.00)) AND isnotnull(sr_ticket_number#77)) AND isnotnull(sr_item_sk#76)) - -(60) CometProject -Input [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] -Arguments: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79], [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] - -(61) CometColumnarToRow -Input [4]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] - -(62) BroadcastHashJoin [codegen id : 17] -Left keys [2]: [ss_ticket_number#71, ss_item_sk#70] -Right keys [2]: [sr_ticket_number#77, sr_item_sk#76] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 17] -Output [6]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79] -Input [9]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] - -(64) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#81] - -(65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#75] -Right keys [1]: [d_date_sk#81] -Join type: Inner -Join condition: None - -(66) Project [codegen id : 17] -Output [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] -Input [7]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79, d_date_sk#81] - -(67) HashAggregate [codegen id : 17] -Input [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] -Keys [1]: [ss_item_sk#70] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#78, 0)), partial_sum(coalesce(ss_quantity#72, 0)), partial_sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#82, sum#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Results [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] - -(68) CometColumnarExchange -Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] -Arguments: hashpartitioning(ss_item_sk#70, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(69) CometColumnarToRow [codegen id : 18] -Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] - -(70) HashAggregate [codegen id : 18] -Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] -Keys [1]: [ss_item_sk#70] -Functions [4]: [sum(coalesce(sr_return_quantity#78, 0)), sum(coalesce(ss_quantity#72, 0)), sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#78, 0))#94, sum(coalesce(ss_quantity#72, 0))#95, sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00))#96, sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97] -Results [3]: [ss_item_sk#70 AS item#98, (cast(sum(coalesce(sr_return_quantity#78, 0))#94 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#72, 0))#95 as decimal(15,4))) AS return_ratio#99, (cast(sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00))#96 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97 as decimal(15,4))) AS currency_ratio#100] - -(71) CometColumnarExchange -Input [3]: [item#98, return_ratio#99, currency_ratio#100] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(72) CometSort -Input [3]: [item#98, return_ratio#99, currency_ratio#100] -Arguments: [item#98, return_ratio#99, currency_ratio#100], [return_ratio#99 ASC NULLS FIRST] - -(73) CometColumnarToRow [codegen id : 19] -Input [3]: [item#98, return_ratio#99, currency_ratio#100] - -(74) Window -Input [3]: [item#98, return_ratio#99, currency_ratio#100] -Arguments: [rank(return_ratio#99) windowspecdefinition(return_ratio#99 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#101], [return_ratio#99 ASC NULLS FIRST] - -(75) Sort [codegen id : 20] -Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] -Arguments: [currency_ratio#100 ASC NULLS FIRST], false, 0 - -(76) Window -Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] -Arguments: [rank(currency_ratio#100) windowspecdefinition(currency_ratio#100 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#102], [currency_ratio#100 ASC NULLS FIRST] - -(77) Filter [codegen id : 21] -Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] -Condition : ((return_rank#101 <= 10) OR (currency_rank#102 <= 10)) - -(78) Project [codegen id : 21] -Output [5]: [store AS channel#103, item#98, return_ratio#99, return_rank#101, currency_rank#102] -Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] - -(79) Union - -(80) HashAggregate [codegen id : 22] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] - -(81) CometColumnarExchange -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(82) CometHashAggregate -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Functions: [] - -(83) CometTakeOrderedAndProject -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,return_rank#33 ASC NULLS FIRST,currency_rank#34 ASC NULLS FIRST,item#30 ASC NULLS FIRST], output=[channel#35,item#30,return_ratio#31,return_rank#33,currency_rank#34]), [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34], 100, 0, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST, item#30 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] - -(84) CometColumnarToRow [codegen id : 23] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (89) -+- * CometColumnarToRow (88) - +- CometProject (87) - +- CometFilter (86) - +- CometNativeScan parquet spark_catalog.default.date_dim (85) - - -(85) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#104, d_moy#105] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(86) CometFilter -Input [3]: [d_date_sk#13, d_year#104, d_moy#105] -Condition : ((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND (d_year#104 = 2001)) AND (d_moy#105 = 12)) AND isnotnull(d_date_sk#13)) - -(87) CometProject -Input [3]: [d_date_sk#13, d_year#104, d_moy#105] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(88) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(89) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 - -Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#75 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/simplified.txt deleted file mode 100644 index 047269434d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/simplified.txt +++ /dev/null @@ -1,140 +0,0 @@ -WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (22) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (7) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (6) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #2 - WholeStageCodegen (4) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (14) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (13) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #6 - WholeStageCodegen (11) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_item_sk] #7 - WholeStageCodegen (10) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (21) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (20) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #9 - WholeStageCodegen (18) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk] #10 - WholeStageCodegen (17) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/explain.txt deleted file mode 100644 index 74702d596d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,462 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- * Project (49) - : +- * Filter (48) - : +- Window (47) - : +- * Sort (46) - : +- Window (45) - : +- * CometColumnarToRow (44) - : +- CometSort (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometBroadcastExchange (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometBroadcastHashJoin (57) - : :- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (59) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND isnotnull(ws_quantity#3)) AND (ws_net_profit#5 > 1.00)) AND (ws_net_paid#4 > 0.00)) AND (ws_quantity#3 > 0)) AND isnotnull(ws_order_number#2)) AND isnotnull(ws_item_sk#1)) - -(3) CometProject -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6], [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(4) CometBroadcastExchange -Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) - -(7) CometProject -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(8) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner, BuildLeft - -(9) CometProject -Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(12) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] - -(16) CometHashAggregate -Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Keys [1]: [ws_item_sk#1] -Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(17) CometExchange -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Keys [1]: [ws_item_sk#1] -Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(19) CometExchange -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] - -(21) CometColumnarToRow [codegen id : 1] -Input [3]: [item#22, return_ratio#23, currency_ratio#24] - -(22) Window -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] - -(23) Sort [codegen id : 2] -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 - -(24) Window -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] - -(25) Filter [codegen id : 3] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] -Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) - -(26) Project [codegen id : 3] -Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_sold_date_sk#33 IN dynamicpruning#34)] -PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(28) CometFilter -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) - -(29) CometProject -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(30) CometBroadcastExchange -Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(32) CometFilter -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) - -(33) CometProject -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] - -(34) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft - -(35) CometProject -Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] - -(36) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#40] - -(37) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -Right output [1]: [d_date_sk#40] -Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight - -(38) CometProject -Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] - -(39) CometHashAggregate -Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -Keys [1]: [cs_item_sk#28] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(40) CometExchange -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(41) CometHashAggregate -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Keys [1]: [cs_item_sk#28] -Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(42) CometExchange -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(43) CometSort -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] - -(44) CometColumnarToRow [codegen id : 4] -Input [3]: [item#47, return_ratio#48, currency_ratio#49] - -(45) Window -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] - -(46) Sort [codegen id : 5] -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 - -(47) Window -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] - -(48) Filter [codegen id : 6] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] -Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) - -(49) Project [codegen id : 6] -Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) - -(52) CometProject -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(53) CometBroadcastExchange -Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(55) CometFilter -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) - -(56) CometProject -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] - -(57) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft - -(58) CometProject -Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] - -(59) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#65] - -(60) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(61) CometProject -Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] - -(62) CometHashAggregate -Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -Keys [1]: [ss_item_sk#53] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(63) CometExchange -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(64) CometHashAggregate -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Keys [1]: [ss_item_sk#53] -Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(65) CometExchange -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(66) CometSort -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] - -(67) CometColumnarToRow [codegen id : 7] -Input [3]: [item#72, return_ratio#73, currency_ratio#74] - -(68) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] - -(69) Sort [codegen id : 8] -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 - -(70) Window -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] - -(71) Filter [codegen id : 9] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] -Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) - -(72) Project [codegen id : 9] -Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] - -(73) Union - -(74) HashAggregate [codegen id : 10] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(75) CometColumnarExchange -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometHashAggregate -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] - -(77) CometTakeOrderedAndProject -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST,item#22 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST, item#22 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(78) CometColumnarToRow [codegen id : 11] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) - - -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(80) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(81) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(82) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(83) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 - -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/simplified.txt deleted file mode 100644 index ca80833ee5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,110 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (10) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (3) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (2) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - 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_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_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - 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_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_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 (9) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (8) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - 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_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_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt deleted file mode 100644 index 74702d596d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt +++ /dev/null @@ -1,462 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- * Project (49) - : +- * Filter (48) - : +- Window (47) - : +- * Sort (46) - : +- Window (45) - : +- * CometColumnarToRow (44) - : +- CometSort (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometBroadcastExchange (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometBroadcastHashJoin (57) - : :- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (59) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND isnotnull(ws_quantity#3)) AND (ws_net_profit#5 > 1.00)) AND (ws_net_paid#4 > 0.00)) AND (ws_quantity#3 > 0)) AND isnotnull(ws_order_number#2)) AND isnotnull(ws_item_sk#1)) - -(3) CometProject -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6], [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(4) CometBroadcastExchange -Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) - -(7) CometProject -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(8) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner, BuildLeft - -(9) CometProject -Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(12) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] - -(16) CometHashAggregate -Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Keys [1]: [ws_item_sk#1] -Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(17) CometExchange -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Keys [1]: [ws_item_sk#1] -Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(19) CometExchange -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] - -(21) CometColumnarToRow [codegen id : 1] -Input [3]: [item#22, return_ratio#23, currency_ratio#24] - -(22) Window -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] - -(23) Sort [codegen id : 2] -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 - -(24) Window -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] - -(25) Filter [codegen id : 3] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] -Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) - -(26) Project [codegen id : 3] -Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_sold_date_sk#33 IN dynamicpruning#34)] -PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(28) CometFilter -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) - -(29) CometProject -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(30) CometBroadcastExchange -Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(32) CometFilter -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) - -(33) CometProject -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] - -(34) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft - -(35) CometProject -Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] - -(36) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#40] - -(37) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -Right output [1]: [d_date_sk#40] -Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight - -(38) CometProject -Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] - -(39) CometHashAggregate -Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -Keys [1]: [cs_item_sk#28] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(40) CometExchange -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(41) CometHashAggregate -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Keys [1]: [cs_item_sk#28] -Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(42) CometExchange -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(43) CometSort -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] - -(44) CometColumnarToRow [codegen id : 4] -Input [3]: [item#47, return_ratio#48, currency_ratio#49] - -(45) Window -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] - -(46) Sort [codegen id : 5] -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 - -(47) Window -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] - -(48) Filter [codegen id : 6] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] -Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) - -(49) Project [codegen id : 6] -Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) - -(52) CometProject -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(53) CometBroadcastExchange -Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(55) CometFilter -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) - -(56) CometProject -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] - -(57) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft - -(58) CometProject -Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] - -(59) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#65] - -(60) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(61) CometProject -Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] - -(62) CometHashAggregate -Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -Keys [1]: [ss_item_sk#53] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(63) CometExchange -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(64) CometHashAggregate -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Keys [1]: [ss_item_sk#53] -Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(65) CometExchange -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(66) CometSort -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] - -(67) CometColumnarToRow [codegen id : 7] -Input [3]: [item#72, return_ratio#73, currency_ratio#74] - -(68) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] - -(69) Sort [codegen id : 8] -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 - -(70) Window -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] - -(71) Filter [codegen id : 9] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] -Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) - -(72) Project [codegen id : 9] -Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] - -(73) Union - -(74) HashAggregate [codegen id : 10] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(75) CometColumnarExchange -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometHashAggregate -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] - -(77) CometTakeOrderedAndProject -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST,item#22 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST, item#22 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(78) CometColumnarToRow [codegen id : 11] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) - - -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(80) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(81) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(82) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(83) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 - -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/extended.txt deleted file mode 100644 index 75684a9669..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/extended.txt +++ /dev/null @@ -1,94 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index ca80833ee5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt +++ /dev/null @@ -1,110 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (10) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (3) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (2) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - 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_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_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - 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_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_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 (9) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (8) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - 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_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_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/explain.txt deleted file mode 100644 index 35eddeb2fd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/explain.txt +++ /dev/null @@ -1,455 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (77) -+- * Filter (76) - +- * HashAggregate (75) - +- * HashAggregate (74) - +- * Project (73) - +- * BroadcastHashJoin Inner BuildRight (72) - :- Window (65) - : +- * CometColumnarToRow (64) - : +- CometSort (63) - : +- CometExchange (62) - : +- CometProject (61) - : +- CometFilter (60) - : +- CometSortMergeJoin (59) - : :- CometSort (29) - : : +- CometColumnarExchange (28) - : : +- * HashAggregate (27) - : : +- * CometColumnarToRow (26) - : : +- CometColumnarExchange (25) - : : +- * HashAggregate (24) - : : +- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * Project (15) - : : : +- Window (14) - : : : +- * CometColumnarToRow (13) - : : : +- CometSort (12) - : : : +- CometColumnarExchange (11) - : : : +- * HashAggregate (10) - : : : +- * CometColumnarToRow (9) - : : : +- CometColumnarExchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (21) - : : +- * Project (20) - : : +- Window (19) - : : +- * CometColumnarToRow (18) - : : +- CometSort (17) - : : +- ReusedExchange (16) - : +- CometSort (58) - : +- CometColumnarExchange (57) - : +- * HashAggregate (56) - : +- * CometColumnarToRow (55) - : +- CometColumnarExchange (54) - : +- * HashAggregate (53) - : +- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Project (44) - : : +- Window (43) - : : +- * CometColumnarToRow (42) - : : +- CometSort (41) - : : +- CometColumnarExchange (40) - : : +- * HashAggregate (39) - : : +- * CometColumnarToRow (38) - : : +- CometColumnarExchange (37) - : : +- * HashAggregate (36) - : : +- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * Filter (32) - : : : +- * ColumnarToRow (31) - : : : +- Scan parquet spark_catalog.default.store_sales (30) - : : +- ReusedExchange (33) - : +- BroadcastExchange (50) - : +- * Project (49) - : +- Window (48) - : +- * CometColumnarToRow (47) - : +- CometSort (46) - : +- ReusedExchange (45) - +- BroadcastExchange (71) - +- * Project (70) - +- Window (69) - +- * CometColumnarToRow (68) - +- CometSort (67) - +- ReusedExchange (66) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] - -(3) Filter [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(4) ReusedExchange [Reuses operator id: 82] -Output [2]: [d_date_sk#5, d_date#6] - -(5) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 2] -Output [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] - -(7) HashAggregate [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum#7] -Results [3]: [ws_item_sk#1, d_date#6, sum#8] - -(8) CometColumnarExchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(9) CometColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, d_date#6, sum#8] - -(10) HashAggregate [codegen id : 3] -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#9] -Results [4]: [ws_item_sk#1 AS item_sk#10, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#9,17,2) AS sumws#11, ws_item_sk#1] - -(11) CometColumnarExchange -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(12) CometSort -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(13) CometColumnarToRow [codegen id : 4] -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] - -(14) Window -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(15) Project [codegen id : 10] -Output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] -Input [5]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12] - -(16) ReusedExchange [Reuses operator id: 11] -Output [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] - -(17) CometSort -Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] -Arguments: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16], [ws_item_sk#16 ASC NULLS FIRST, d_date#14 ASC NULLS FIRST] - -(18) CometColumnarToRow [codegen id : 8] -Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] - -(19) Window -Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] -Arguments: [row_number() windowspecdefinition(ws_item_sk#16, d_date#14 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#17], [ws_item_sk#16], [d_date#14 ASC NULLS FIRST] - -(20) Project [codegen id : 9] -Output [3]: [item_sk#13, sumws#15, rk#17] -Input [5]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16, rk#17] - -(21) BroadcastExchange -Input [3]: [item_sk#13, sumws#15, rk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(22) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [item_sk#10] -Right keys [1]: [item_sk#13] -Join type: Inner -Join condition: (rk#12 >= rk#17) - -(23) Project [codegen id : 10] -Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] -Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#13, sumws#15, rk#17] - -(24) HashAggregate [codegen id : 10] -Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] -Keys [3]: [item_sk#10, d_date#6, sumws#11] -Functions [1]: [partial_sum(sumws#15)] -Aggregate Attributes [2]: [sum#18, isEmpty#19] -Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] - -(25) CometColumnarExchange -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(26) CometColumnarToRow [codegen id : 11] -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] - -(27) HashAggregate [codegen id : 11] -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -Keys [3]: [item_sk#10, d_date#6, sumws#11] -Functions [1]: [sum(sumws#15)] -Aggregate Attributes [1]: [sum(sumws#15)#22] -Results [3]: [item_sk#10, d_date#6, sum(sumws#15)#22 AS cume_sales#23] - -(28) CometColumnarExchange -Input [3]: [item_sk#10, d_date#6, cume_sales#23] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(29) CometSort -Input [3]: [item_sk#10, d_date#6, cume_sales#23] -Arguments: [item_sk#10, d_date#6, cume_sales#23], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(30) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 13] -Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] - -(32) Filter [codegen id : 13] -Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#24) - -(33) ReusedExchange [Reuses operator id: 82] -Output [2]: [d_date_sk#27, d_date#28] - -(34) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 13] -Output [3]: [ss_item_sk#24, ss_sales_price#25, d_date#28] -Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#27, d_date#28] - -(36) HashAggregate [codegen id : 13] -Input [3]: [ss_item_sk#24, ss_sales_price#25, d_date#28] -Keys [2]: [ss_item_sk#24, d_date#28] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum#29] -Results [3]: [ss_item_sk#24, d_date#28, sum#30] - -(37) CometColumnarExchange -Input [3]: [ss_item_sk#24, d_date#28, sum#30] -Arguments: hashpartitioning(ss_item_sk#24, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(38) CometColumnarToRow [codegen id : 14] -Input [3]: [ss_item_sk#24, d_date#28, sum#30] - -(39) HashAggregate [codegen id : 14] -Input [3]: [ss_item_sk#24, d_date#28, sum#30] -Keys [2]: [ss_item_sk#24, d_date#28] -Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#31] -Results [4]: [ss_item_sk#24 AS item_sk#32, d_date#28, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#31,17,2) AS sumss#33, ss_item_sk#24] - -(40) CometColumnarExchange -Input [4]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24] -Arguments: hashpartitioning(ss_item_sk#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(41) CometSort -Input [4]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24] -Arguments: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24], [ss_item_sk#24 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 15] -Input [4]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24] - -(43) Window -Input [4]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24] -Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#34], [ss_item_sk#24], [d_date#28 ASC NULLS FIRST] - -(44) Project [codegen id : 21] -Output [4]: [item_sk#32, d_date#28, sumss#33, rk#34] -Input [5]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24, rk#34] - -(45) ReusedExchange [Reuses operator id: 40] -Output [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] - -(46) CometSort -Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] -Arguments: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38], [ss_item_sk#38 ASC NULLS FIRST, d_date#36 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 19] -Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] - -(48) Window -Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] -Arguments: [row_number() windowspecdefinition(ss_item_sk#38, d_date#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#39], [ss_item_sk#38], [d_date#36 ASC NULLS FIRST] - -(49) Project [codegen id : 20] -Output [3]: [item_sk#35, sumss#37, rk#39] -Input [5]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38, rk#39] - -(50) BroadcastExchange -Input [3]: [item_sk#35, sumss#37, rk#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -(51) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [item_sk#32] -Right keys [1]: [item_sk#35] -Join type: Inner -Join condition: (rk#34 >= rk#39) - -(52) Project [codegen id : 21] -Output [4]: [item_sk#32, d_date#28, sumss#33, sumss#37] -Input [7]: [item_sk#32, d_date#28, sumss#33, rk#34, item_sk#35, sumss#37, rk#39] - -(53) HashAggregate [codegen id : 21] -Input [4]: [item_sk#32, d_date#28, sumss#33, sumss#37] -Keys [3]: [item_sk#32, d_date#28, sumss#33] -Functions [1]: [partial_sum(sumss#37)] -Aggregate Attributes [2]: [sum#40, isEmpty#41] -Results [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] - -(54) CometColumnarExchange -Input [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] -Arguments: hashpartitioning(item_sk#32, d_date#28, sumss#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(55) CometColumnarToRow [codegen id : 22] -Input [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] - -(56) HashAggregate [codegen id : 22] -Input [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] -Keys [3]: [item_sk#32, d_date#28, sumss#33] -Functions [1]: [sum(sumss#37)] -Aggregate Attributes [1]: [sum(sumss#37)#44] -Results [3]: [item_sk#32, d_date#28, sum(sumss#37)#44 AS cume_sales#45] - -(57) CometColumnarExchange -Input [3]: [item_sk#32, d_date#28, cume_sales#45] -Arguments: hashpartitioning(item_sk#32, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(58) CometSort -Input [3]: [item_sk#32, d_date#28, cume_sales#45] -Arguments: [item_sk#32, d_date#28, cume_sales#45], [item_sk#32 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] - -(59) CometSortMergeJoin -Left output [3]: [item_sk#10, d_date#6, cume_sales#23] -Right output [3]: [item_sk#32, d_date#28, cume_sales#45] -Arguments: [item_sk#10, d_date#6], [item_sk#32, d_date#28], FullOuter - -(60) CometFilter -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#32, d_date#28, cume_sales#45] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#32 END) - -(61) CometProject -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#32, d_date#28, cume_sales#45] -Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#32 END AS item_sk#46, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#28 END AS d_date#47, cume_sales#23 AS web_sales#48, cume_sales#45 AS store_sales#49] - -(62) CometExchange -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: hashpartitioning(item_sk#46, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(63) CometSort -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST] - -(64) CometColumnarToRow [codegen id : 23] -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] - -(65) Window -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: [row_number() windowspecdefinition(item_sk#46, d_date#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#50], [item_sk#46], [d_date#47 ASC NULLS FIRST] - -(66) ReusedExchange [Reuses operator id: 62] -Output [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] - -(67) CometSort -Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] -Arguments: [item_sk#51, d_date#52, web_sales#53, store_sales#54], [item_sk#51 ASC NULLS FIRST, d_date#52 ASC NULLS FIRST] - -(68) CometColumnarToRow [codegen id : 46] -Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] - -(69) Window -Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] -Arguments: [row_number() windowspecdefinition(item_sk#51, d_date#52 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#55], [item_sk#51], [d_date#52 ASC NULLS FIRST] - -(70) Project [codegen id : 47] -Output [4]: [item_sk#51, web_sales#53, store_sales#54, rk#55] -Input [5]: [item_sk#51, d_date#52, web_sales#53, store_sales#54, rk#55] - -(71) BroadcastExchange -Input [4]: [item_sk#51, web_sales#53, store_sales#54, rk#55] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -(72) BroadcastHashJoin [codegen id : 48] -Left keys [1]: [item_sk#46] -Right keys [1]: [item_sk#51] -Join type: Inner -Join condition: (rk#50 >= rk#55) - -(73) Project [codegen id : 48] -Output [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] -Input [9]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#50, item_sk#51, web_sales#53, store_sales#54, rk#55] - -(74) HashAggregate [codegen id : 48] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] -Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Functions [2]: [partial_max(web_sales#53), partial_max(store_sales#54)] -Aggregate Attributes [2]: [max#56, max#57] -Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#58, max#59] - -(75) HashAggregate [codegen id : 48] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#58, max#59] -Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Functions [2]: [max(web_sales#53), max(store_sales#54)] -Aggregate Attributes [2]: [max(web_sales#53)#60, max(store_sales#54)#61] -Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max(web_sales#53)#60 AS web_cumulative#62, max(store_sales#54)#61 AS store_cumulative#63] - -(76) Filter [codegen id : 48] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] -Condition : ((isnotnull(web_cumulative#62) AND isnotnull(store_cumulative#63)) AND (web_cumulative#62 > store_cumulative#63)) - -(77) TakeOrderedAndProject -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] -Arguments: 100, [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (82) -+- * CometColumnarToRow (81) - +- CometProject (80) - +- CometFilter (79) - +- CometNativeScan parquet spark_catalog.default.date_dim (78) - - -(78) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(79) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] -Condition : (((isnotnull(d_month_seq#64) AND (d_month_seq#64 >= 1212)) AND (d_month_seq#64 <= 1223)) AND isnotnull(d_date_sk#5)) - -(80) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#64] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(81) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] - -(82) BroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/simplified.txt deleted file mode 100644 index 00cd0a509c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/simplified.txt +++ /dev/null @@ -1,127 +0,0 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (48) - Filter [web_cumulative,store_cumulative] - HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] - HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] - Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (11) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (10) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometColumnarExchange [ws_item_sk] #4 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk,d_date] #5 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - Project [item_sk,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #8 - WholeStageCodegen (22) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumss] #9 - WholeStageCodegen (21) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (15) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometColumnarExchange [ss_item_sk] #10 - WholeStageCodegen (14) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk,d_date] #11 - WholeStageCodegen (13) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (20) - Project [item_sk,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (47) - Project [item_sk,web_sales,store_sales,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (46) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/explain.txt deleted file mode 100644 index 2cd07a9fb7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,448 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (76) -+- * Filter (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- Window (64) - : +- * CometColumnarToRow (63) - : +- CometSort (62) - : +- CometExchange (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometSortMergeJoin (58) - : :- CometSort (30) - : : +- CometColumnarExchange (29) - : : +- * HashAggregate (28) - : : +- * CometColumnarToRow (27) - : : +- CometColumnarExchange (26) - : : +- * HashAggregate (25) - : : +- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (16) - : : : +- Window (15) - : : : +- * CometColumnarToRow (14) - : : : +- CometSort (13) - : : : +- CometExchange (12) - : : : +- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- BroadcastExchange (22) - : : +- * Project (21) - : : +- Window (20) - : : +- * CometColumnarToRow (19) - : : +- CometSort (18) - : : +- ReusedExchange (17) - : +- CometSort (57) - : +- CometColumnarExchange (56) - : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) - : +- CometColumnarExchange (53) - : +- * HashAggregate (52) - : +- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Project (43) - : : +- Window (42) - : : +- * CometColumnarToRow (41) - : : +- CometSort (40) - : : +- CometExchange (39) - : : +- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometFilter (32) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (31) - : : +- ReusedExchange (33) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- Window (47) - : +- * CometColumnarToRow (46) - : +- CometSort (45) - : +- ReusedExchange (44) - +- BroadcastExchange (70) - +- * Project (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- ReusedExchange (65) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: [d_date_sk#5, d_date#6] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Right output [2]: [d_date_sk#5, d_date#6] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] -Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] - -(9) CometHashAggregate -Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] - -(10) CometExchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] - -(12) CometExchange -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(13) CometSort -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(14) CometColumnarToRow [codegen id : 1] -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] - -(15) Window -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(16) Project [codegen id : 4] -Output [4]: [item_sk#9, d_date#6, sumws#10, rk#11] -Input [5]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1, rk#11] - -(17) ReusedExchange [Reuses operator id: 12] -Output [4]: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15] - -(18) CometSort -Input [4]: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15] -Arguments: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15], [ws_item_sk#15 ASC NULLS FIRST, d_date#13 ASC NULLS FIRST] - -(19) CometColumnarToRow [codegen id : 2] -Input [4]: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15] - -(20) Window -Input [4]: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15] -Arguments: [row_number() windowspecdefinition(ws_item_sk#15, d_date#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#16], [ws_item_sk#15], [d_date#13 ASC NULLS FIRST] - -(21) Project [codegen id : 3] -Output [3]: [item_sk#12, sumws#14, rk#16] -Input [5]: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15, rk#16] - -(22) BroadcastExchange -Input [3]: [item_sk#12, sumws#14, rk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [item_sk#9] -Right keys [1]: [item_sk#12] -Join type: Inner -Join condition: (rk#11 >= rk#16) - -(24) Project [codegen id : 4] -Output [4]: [item_sk#9, d_date#6, sumws#10, sumws#14] -Input [7]: [item_sk#9, d_date#6, sumws#10, rk#11, item_sk#12, sumws#14, rk#16] - -(25) HashAggregate [codegen id : 4] -Input [4]: [item_sk#9, d_date#6, sumws#10, sumws#14] -Keys [3]: [item_sk#9, d_date#6, sumws#10] -Functions [1]: [partial_sum(sumws#14)] -Aggregate Attributes [2]: [sum#17, isEmpty#18] -Results [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] - -(26) CometColumnarExchange -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] -Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometColumnarToRow [codegen id : 5] -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] - -(28) HashAggregate [codegen id : 5] -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] -Keys [3]: [item_sk#9, d_date#6, sumws#10] -Functions [1]: [sum(sumws#14)] -Aggregate Attributes [1]: [sum(sumws#14)#21] -Results [3]: [item_sk#9, d_date#6, sum(sumws#14)#21 AS cume_sales#22] - -(29) CometColumnarExchange -Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(30) CometSort -Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: [item_sk#9, d_date#6, cume_sales#22], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_item_sk#23) - -(33) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#27, d_date#28] - -(34) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Right output [2]: [d_date_sk#27, d_date#28] -Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight - -(35) CometProject -Input [5]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_date#28] -Arguments: [ss_item_sk#23, ss_sales_price#24, d_date#28], [ss_item_sk#23, ss_sales_price#24, d_date#28] - -(36) CometHashAggregate -Input [3]: [ss_item_sk#23, ss_sales_price#24, d_date#28] -Keys [2]: [ss_item_sk#23, d_date#28] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#24))] - -(37) CometExchange -Input [3]: [ss_item_sk#23, d_date#28, sum#29] -Arguments: hashpartitioning(ss_item_sk#23, d_date#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(38) CometHashAggregate -Input [3]: [ss_item_sk#23, d_date#28, sum#29] -Keys [2]: [ss_item_sk#23, d_date#28] -Functions [1]: [sum(UnscaledValue(ss_sales_price#24))] - -(39) CometExchange -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: hashpartitioning(ss_item_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(40) CometSort -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23], [ss_item_sk#23 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] - -(41) CometColumnarToRow [codegen id : 6] -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] - -(42) Window -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: [row_number() windowspecdefinition(ss_item_sk#23, d_date#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [ss_item_sk#23], [d_date#28 ASC NULLS FIRST] - -(43) Project [codegen id : 9] -Output [4]: [item_sk#30, d_date#28, sumss#31, rk#32] -Input [5]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23, rk#32] - -(44) ReusedExchange [Reuses operator id: 39] -Output [4]: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36] - -(45) CometSort -Input [4]: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36] -Arguments: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36], [ss_item_sk#36 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST] - -(46) CometColumnarToRow [codegen id : 7] -Input [4]: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36] - -(47) Window -Input [4]: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36] -Arguments: [row_number() windowspecdefinition(ss_item_sk#36, d_date#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#37], [ss_item_sk#36], [d_date#34 ASC NULLS FIRST] - -(48) Project [codegen id : 8] -Output [3]: [item_sk#33, sumss#35, rk#37] -Input [5]: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36, rk#37] - -(49) BroadcastExchange -Input [3]: [item_sk#33, sumss#35, rk#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -(50) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [item_sk#30] -Right keys [1]: [item_sk#33] -Join type: Inner -Join condition: (rk#32 >= rk#37) - -(51) Project [codegen id : 9] -Output [4]: [item_sk#30, d_date#28, sumss#31, sumss#35] -Input [7]: [item_sk#30, d_date#28, sumss#31, rk#32, item_sk#33, sumss#35, rk#37] - -(52) HashAggregate [codegen id : 9] -Input [4]: [item_sk#30, d_date#28, sumss#31, sumss#35] -Keys [3]: [item_sk#30, d_date#28, sumss#31] -Functions [1]: [partial_sum(sumss#35)] -Aggregate Attributes [2]: [sum#38, isEmpty#39] -Results [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] - -(53) CometColumnarExchange -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -Arguments: hashpartitioning(item_sk#30, d_date#28, sumss#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(54) CometColumnarToRow [codegen id : 10] -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] - -(55) HashAggregate [codegen id : 10] -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -Keys [3]: [item_sk#30, d_date#28, sumss#31] -Functions [1]: [sum(sumss#35)] -Aggregate Attributes [1]: [sum(sumss#35)#42] -Results [3]: [item_sk#30, d_date#28, sum(sumss#35)#42 AS cume_sales#43] - -(56) CometColumnarExchange -Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: hashpartitioning(item_sk#30, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(57) CometSort -Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#30, d_date#28, cume_sales#43], [item_sk#30 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [3]: [item_sk#9, d_date#6, cume_sales#22] -Right output [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#9, d_date#6], [item_sk#30, d_date#28], FullOuter - -(59) CometFilter -Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END) - -(60) CometProject -Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END AS item_sk#44, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#28 END AS d_date#45, cume_sales#22 AS web_sales#46, cume_sales#43 AS store_sales#47] - -(61) CometExchange -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(62) CometSort -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] - -(63) CometColumnarToRow [codegen id : 11] -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] - -(64) Window -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [item_sk#44], [d_date#45 ASC NULLS FIRST] - -(65) ReusedExchange [Reuses operator id: 61] -Output [4]: [item_sk#49, d_date#50, web_sales#51, store_sales#52] - -(66) CometSort -Input [4]: [item_sk#49, d_date#50, web_sales#51, store_sales#52] -Arguments: [item_sk#49, d_date#50, web_sales#51, store_sales#52], [item_sk#49 ASC NULLS FIRST, d_date#50 ASC NULLS FIRST] - -(67) CometColumnarToRow [codegen id : 22] -Input [4]: [item_sk#49, d_date#50, web_sales#51, store_sales#52] - -(68) Window -Input [4]: [item_sk#49, d_date#50, web_sales#51, store_sales#52] -Arguments: [row_number() windowspecdefinition(item_sk#49, d_date#50 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#53], [item_sk#49], [d_date#50 ASC NULLS FIRST] - -(69) Project [codegen id : 23] -Output [4]: [item_sk#49, web_sales#51, store_sales#52, rk#53] -Input [5]: [item_sk#49, d_date#50, web_sales#51, store_sales#52, rk#53] - -(70) BroadcastExchange -Input [4]: [item_sk#49, web_sales#51, store_sales#52, rk#53] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -(71) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [item_sk#44] -Right keys [1]: [item_sk#49] -Join type: Inner -Join condition: (rk#48 >= rk#53) - -(72) Project [codegen id : 24] -Output [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] -Input [9]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#48, item_sk#49, web_sales#51, store_sales#52, rk#53] - -(73) HashAggregate [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] -Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Functions [2]: [partial_max(web_sales#51), partial_max(store_sales#52)] -Aggregate Attributes [2]: [max#54, max#55] -Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#56, max#57] - -(74) HashAggregate [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#56, max#57] -Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Functions [2]: [max(web_sales#51), max(store_sales#52)] -Aggregate Attributes [2]: [max(web_sales#51)#58, max(store_sales#52)#59] -Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max(web_sales#51)#58 AS web_cumulative#60, max(store_sales#52)#59 AS store_cumulative#61] - -(75) Filter [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#60, store_cumulative#61] -Condition : ((isnotnull(web_cumulative#60) AND isnotnull(store_cumulative#61)) AND (web_cumulative#60 > store_cumulative#61)) - -(76) TakeOrderedAndProject -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#60, store_cumulative#61] -Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#60, store_cumulative#61] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (81) -+- * CometColumnarToRow (80) - +- CometProject (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) - -(79) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(80) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] - -(81) BroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:2 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/simplified.txt deleted file mode 100644 index b3013059b0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,116 +0,0 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (24) - Filter [web_cumulative,store_cumulative] - HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] - HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] - Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (5) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (4) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometExchange [ws_item_sk] #4 - CometHashAggregate [sum] [item_sk,d_date,sumws,ws_item_sk,sum(UnscaledValue(ws_sales_price))] - CometExchange [ws_item_sk,d_date] #5 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [item_sk,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #9 - WholeStageCodegen (10) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumss] #10 - WholeStageCodegen (9) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sum] [item_sk,d_date,sumss,ss_item_sk,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_item_sk,d_date] #12 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - 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 [native_iceberg_compat] 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 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - Project [item_sk,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (23) - Project [item_sk,web_sales,store_sales,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (22) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt deleted file mode 100644 index 2cd07a9fb7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt +++ /dev/null @@ -1,448 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (76) -+- * Filter (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- Window (64) - : +- * CometColumnarToRow (63) - : +- CometSort (62) - : +- CometExchange (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometSortMergeJoin (58) - : :- CometSort (30) - : : +- CometColumnarExchange (29) - : : +- * HashAggregate (28) - : : +- * CometColumnarToRow (27) - : : +- CometColumnarExchange (26) - : : +- * HashAggregate (25) - : : +- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (16) - : : : +- Window (15) - : : : +- * CometColumnarToRow (14) - : : : +- CometSort (13) - : : : +- CometExchange (12) - : : : +- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- BroadcastExchange (22) - : : +- * Project (21) - : : +- Window (20) - : : +- * CometColumnarToRow (19) - : : +- CometSort (18) - : : +- ReusedExchange (17) - : +- CometSort (57) - : +- CometColumnarExchange (56) - : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) - : +- CometColumnarExchange (53) - : +- * HashAggregate (52) - : +- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Project (43) - : : +- Window (42) - : : +- * CometColumnarToRow (41) - : : +- CometSort (40) - : : +- CometExchange (39) - : : +- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometFilter (32) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (31) - : : +- ReusedExchange (33) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- Window (47) - : +- * CometColumnarToRow (46) - : +- CometSort (45) - : +- ReusedExchange (44) - +- BroadcastExchange (70) - +- * Project (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- ReusedExchange (65) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: [d_date_sk#5, d_date#6] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Right output [2]: [d_date_sk#5, d_date#6] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] -Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] - -(9) CometHashAggregate -Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] - -(10) CometExchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] - -(12) CometExchange -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(13) CometSort -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(14) CometColumnarToRow [codegen id : 1] -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] - -(15) Window -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(16) Project [codegen id : 4] -Output [4]: [item_sk#9, d_date#6, sumws#10, rk#11] -Input [5]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1, rk#11] - -(17) ReusedExchange [Reuses operator id: 12] -Output [4]: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15] - -(18) CometSort -Input [4]: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15] -Arguments: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15], [ws_item_sk#15 ASC NULLS FIRST, d_date#13 ASC NULLS FIRST] - -(19) CometColumnarToRow [codegen id : 2] -Input [4]: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15] - -(20) Window -Input [4]: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15] -Arguments: [row_number() windowspecdefinition(ws_item_sk#15, d_date#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#16], [ws_item_sk#15], [d_date#13 ASC NULLS FIRST] - -(21) Project [codegen id : 3] -Output [3]: [item_sk#12, sumws#14, rk#16] -Input [5]: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15, rk#16] - -(22) BroadcastExchange -Input [3]: [item_sk#12, sumws#14, rk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [item_sk#9] -Right keys [1]: [item_sk#12] -Join type: Inner -Join condition: (rk#11 >= rk#16) - -(24) Project [codegen id : 4] -Output [4]: [item_sk#9, d_date#6, sumws#10, sumws#14] -Input [7]: [item_sk#9, d_date#6, sumws#10, rk#11, item_sk#12, sumws#14, rk#16] - -(25) HashAggregate [codegen id : 4] -Input [4]: [item_sk#9, d_date#6, sumws#10, sumws#14] -Keys [3]: [item_sk#9, d_date#6, sumws#10] -Functions [1]: [partial_sum(sumws#14)] -Aggregate Attributes [2]: [sum#17, isEmpty#18] -Results [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] - -(26) CometColumnarExchange -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] -Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometColumnarToRow [codegen id : 5] -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] - -(28) HashAggregate [codegen id : 5] -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] -Keys [3]: [item_sk#9, d_date#6, sumws#10] -Functions [1]: [sum(sumws#14)] -Aggregate Attributes [1]: [sum(sumws#14)#21] -Results [3]: [item_sk#9, d_date#6, sum(sumws#14)#21 AS cume_sales#22] - -(29) CometColumnarExchange -Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(30) CometSort -Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: [item_sk#9, d_date#6, cume_sales#22], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_item_sk#23) - -(33) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#27, d_date#28] - -(34) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Right output [2]: [d_date_sk#27, d_date#28] -Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight - -(35) CometProject -Input [5]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_date#28] -Arguments: [ss_item_sk#23, ss_sales_price#24, d_date#28], [ss_item_sk#23, ss_sales_price#24, d_date#28] - -(36) CometHashAggregate -Input [3]: [ss_item_sk#23, ss_sales_price#24, d_date#28] -Keys [2]: [ss_item_sk#23, d_date#28] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#24))] - -(37) CometExchange -Input [3]: [ss_item_sk#23, d_date#28, sum#29] -Arguments: hashpartitioning(ss_item_sk#23, d_date#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(38) CometHashAggregate -Input [3]: [ss_item_sk#23, d_date#28, sum#29] -Keys [2]: [ss_item_sk#23, d_date#28] -Functions [1]: [sum(UnscaledValue(ss_sales_price#24))] - -(39) CometExchange -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: hashpartitioning(ss_item_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(40) CometSort -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23], [ss_item_sk#23 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] - -(41) CometColumnarToRow [codegen id : 6] -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] - -(42) Window -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: [row_number() windowspecdefinition(ss_item_sk#23, d_date#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [ss_item_sk#23], [d_date#28 ASC NULLS FIRST] - -(43) Project [codegen id : 9] -Output [4]: [item_sk#30, d_date#28, sumss#31, rk#32] -Input [5]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23, rk#32] - -(44) ReusedExchange [Reuses operator id: 39] -Output [4]: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36] - -(45) CometSort -Input [4]: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36] -Arguments: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36], [ss_item_sk#36 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST] - -(46) CometColumnarToRow [codegen id : 7] -Input [4]: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36] - -(47) Window -Input [4]: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36] -Arguments: [row_number() windowspecdefinition(ss_item_sk#36, d_date#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#37], [ss_item_sk#36], [d_date#34 ASC NULLS FIRST] - -(48) Project [codegen id : 8] -Output [3]: [item_sk#33, sumss#35, rk#37] -Input [5]: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36, rk#37] - -(49) BroadcastExchange -Input [3]: [item_sk#33, sumss#35, rk#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -(50) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [item_sk#30] -Right keys [1]: [item_sk#33] -Join type: Inner -Join condition: (rk#32 >= rk#37) - -(51) Project [codegen id : 9] -Output [4]: [item_sk#30, d_date#28, sumss#31, sumss#35] -Input [7]: [item_sk#30, d_date#28, sumss#31, rk#32, item_sk#33, sumss#35, rk#37] - -(52) HashAggregate [codegen id : 9] -Input [4]: [item_sk#30, d_date#28, sumss#31, sumss#35] -Keys [3]: [item_sk#30, d_date#28, sumss#31] -Functions [1]: [partial_sum(sumss#35)] -Aggregate Attributes [2]: [sum#38, isEmpty#39] -Results [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] - -(53) CometColumnarExchange -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -Arguments: hashpartitioning(item_sk#30, d_date#28, sumss#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(54) CometColumnarToRow [codegen id : 10] -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] - -(55) HashAggregate [codegen id : 10] -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -Keys [3]: [item_sk#30, d_date#28, sumss#31] -Functions [1]: [sum(sumss#35)] -Aggregate Attributes [1]: [sum(sumss#35)#42] -Results [3]: [item_sk#30, d_date#28, sum(sumss#35)#42 AS cume_sales#43] - -(56) CometColumnarExchange -Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: hashpartitioning(item_sk#30, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(57) CometSort -Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#30, d_date#28, cume_sales#43], [item_sk#30 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [3]: [item_sk#9, d_date#6, cume_sales#22] -Right output [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#9, d_date#6], [item_sk#30, d_date#28], FullOuter - -(59) CometFilter -Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END) - -(60) CometProject -Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END AS item_sk#44, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#28 END AS d_date#45, cume_sales#22 AS web_sales#46, cume_sales#43 AS store_sales#47] - -(61) CometExchange -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(62) CometSort -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] - -(63) CometColumnarToRow [codegen id : 11] -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] - -(64) Window -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [item_sk#44], [d_date#45 ASC NULLS FIRST] - -(65) ReusedExchange [Reuses operator id: 61] -Output [4]: [item_sk#49, d_date#50, web_sales#51, store_sales#52] - -(66) CometSort -Input [4]: [item_sk#49, d_date#50, web_sales#51, store_sales#52] -Arguments: [item_sk#49, d_date#50, web_sales#51, store_sales#52], [item_sk#49 ASC NULLS FIRST, d_date#50 ASC NULLS FIRST] - -(67) CometColumnarToRow [codegen id : 22] -Input [4]: [item_sk#49, d_date#50, web_sales#51, store_sales#52] - -(68) Window -Input [4]: [item_sk#49, d_date#50, web_sales#51, store_sales#52] -Arguments: [row_number() windowspecdefinition(item_sk#49, d_date#50 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#53], [item_sk#49], [d_date#50 ASC NULLS FIRST] - -(69) Project [codegen id : 23] -Output [4]: [item_sk#49, web_sales#51, store_sales#52, rk#53] -Input [5]: [item_sk#49, d_date#50, web_sales#51, store_sales#52, rk#53] - -(70) BroadcastExchange -Input [4]: [item_sk#49, web_sales#51, store_sales#52, rk#53] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -(71) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [item_sk#44] -Right keys [1]: [item_sk#49] -Join type: Inner -Join condition: (rk#48 >= rk#53) - -(72) Project [codegen id : 24] -Output [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] -Input [9]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#48, item_sk#49, web_sales#51, store_sales#52, rk#53] - -(73) HashAggregate [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] -Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Functions [2]: [partial_max(web_sales#51), partial_max(store_sales#52)] -Aggregate Attributes [2]: [max#54, max#55] -Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#56, max#57] - -(74) HashAggregate [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#56, max#57] -Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Functions [2]: [max(web_sales#51), max(store_sales#52)] -Aggregate Attributes [2]: [max(web_sales#51)#58, max(store_sales#52)#59] -Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max(web_sales#51)#58 AS web_cumulative#60, max(store_sales#52)#59 AS store_cumulative#61] - -(75) Filter [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#60, store_cumulative#61] -Condition : ((isnotnull(web_cumulative#60) AND isnotnull(store_cumulative#61)) AND (web_cumulative#60 > store_cumulative#61)) - -(76) TakeOrderedAndProject -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#60, store_cumulative#61] -Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#60, store_cumulative#61] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (81) -+- * CometColumnarToRow (80) - +- CometProject (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) - -(79) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(80) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] - -(81) BroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:2 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/extended.txt deleted file mode 100644 index f7d3371108..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/extended.txt +++ /dev/null @@ -1,216 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : : +- CometColumnarToRow - : : : +- CometSort - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometSortMergeJoin - :- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 138 out of 196 eligible operators (70%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index b3013059b0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt +++ /dev/null @@ -1,116 +0,0 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (24) - Filter [web_cumulative,store_cumulative] - HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] - HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] - Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (5) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (4) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometExchange [ws_item_sk] #4 - CometHashAggregate [sum] [item_sk,d_date,sumws,ws_item_sk,sum(UnscaledValue(ws_sales_price))] - CometExchange [ws_item_sk,d_date] #5 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [item_sk,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #9 - WholeStageCodegen (10) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumss] #10 - WholeStageCodegen (9) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sum] [item_sk,d_date,sumss,ss_item_sk,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_item_sk,d_date] #12 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - 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 [native_iceberg_compat] 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 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - Project [item_sk,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (23) - Project [item_sk,web_sales,store_sales,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (22) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/explain.txt deleted file mode 100644 index d2e466b254..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/explain.txt +++ /dev/null @@ -1,304 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (51) -+- * Project (50) - +- * BroadcastHashJoin Inner BuildRight (49) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * CometColumnarToRow (26) - : : +- CometSort (25) - : : +- CometColumnarExchange (24) - : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- * HashAggregate (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- Window (38) - : +- * CometColumnarToRow (37) - : +- CometSort (36) - : +- CometColumnarExchange (35) - : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) - : +- ReusedExchange (32) - +- BroadcastExchange (48) - +- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- ReusedExchange (43) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)))) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) AS i_brand#4, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)) AS i_category#5] - -(4) CometColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#4, i_category#5] - -(5) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(7) Filter [codegen id : 1] -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) - -(8) BroadcastExchange -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#7] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(11) ReusedExchange [Reuses operator id: 55] -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] - -(14) CometNativeScan parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#14, cc_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#14, cc_name#15] -Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) - -(16) CometColumnarToRow [codegen id : 3] -Input [2]: [cc_call_center_sk#14, cc_name#15] - -(17) BroadcastExchange -Input [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_call_center_sk#6] -Right keys [1]: [cc_call_center_sk#14] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] - -(20) HashAggregate [codegen id : 4] -Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] -Aggregate Attributes [1]: [sum#16] -Results [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] - -(21) CometColumnarExchange -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] - -(23) HashAggregate [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#8))#18] -Results [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#18,17,2) AS sum_sales#19, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#18,17,2) AS _w0#20] - -(24) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) CometColumnarToRow [codegen id : 6] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] - -(27) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(28) Filter [codegen id : 7] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(29) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] -Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(30) Filter [codegen id : 22] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] -Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) - -(31) Project [codegen id : 22] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] - -(32) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum#28] - -(33) CometColumnarToRow [codegen id : 12] -Input [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum#28] - -(34) HashAggregate [codegen id : 12] -Input [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum#28] -Keys [5]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27] -Functions [1]: [sum(UnscaledValue(cs_sales_price#29))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#29))#18] -Results [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(cs_sales_price#29))#18,17,2) AS sum_sales#30] - -(35) CometColumnarExchange -Input [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sales#30] -Arguments: hashpartitioning(i_category#23, i_brand#24, cc_name#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(36) CometSort -Input [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sales#30] -Arguments: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sales#30], [i_category#23 ASC NULLS FIRST, i_brand#24 ASC NULLS FIRST, cc_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] - -(37) CometColumnarToRow [codegen id : 13] -Input [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sales#30] - -(38) Window -Input [6]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sales#30] -Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#23, i_brand#24, cc_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#31], [i_category#23, i_brand#24, cc_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] - -(39) Project [codegen id : 14] -Output [5]: [i_category#23, i_brand#24, cc_name#25, sum_sales#30, rn#31] -Input [7]: [i_category#23, i_brand#24, cc_name#25, d_year#26, d_moy#27, sum_sales#30, rn#31] - -(40) BroadcastExchange -Input [5]: [i_category#23, i_brand#24, cc_name#25, sum_sales#30, rn#31] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] - -(41) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] -Right keys [4]: [i_category#23, i_brand#24, cc_name#25, (rn#31 + 1)] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 22] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#30] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#23, i_brand#24, cc_name#25, sum_sales#30, rn#31] - -(43) ReusedExchange [Reuses operator id: 35] -Output [6]: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sales#37] - -(44) CometSort -Input [6]: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sales#37] -Arguments: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sales#37], [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, cc_name#34 ASC NULLS FIRST, d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST] - -(45) CometColumnarToRow [codegen id : 20] -Input [6]: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sales#37] - -(46) Window -Input [6]: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sales#37] -Arguments: [rank(d_year#35, d_moy#36) windowspecdefinition(i_category#32, i_brand#33, cc_name#34, d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#32, i_brand#33, cc_name#34], [d_year#35 ASC NULLS FIRST, d_moy#36 ASC NULLS FIRST] - -(47) Project [codegen id : 21] -Output [5]: [i_category#32, i_brand#33, cc_name#34, sum_sales#37, rn#38] -Input [7]: [i_category#32, i_brand#33, cc_name#34, d_year#35, d_moy#36, sum_sales#37, rn#38] - -(48) BroadcastExchange -Input [5]: [i_category#32, i_brand#33, cc_name#34, sum_sales#37, rn#38] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] - -(49) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] -Right keys [4]: [i_category#32, i_brand#33, cc_name#34, (rn#38 - 1)] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 22] -Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#30 AS psum#39, sum_sales#37 AS nsum#40] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#30, i_category#32, i_brand#33, cc_name#34, sum_sales#37, rn#38] - -(51) TakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] -Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometFilter (53) - +- CometNativeScan parquet spark_catalog.default.date_dim (52) - - -(52) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(54) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(55) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/simplified.txt deleted file mode 100644 index e603f05322..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/simplified.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (22) - Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (7) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [cs_item_sk,cs_call_center_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk,cc_name] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/explain.txt deleted file mode 100644 index 9265c65475..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)))) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) AS i_brand#4, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) - -(6) CometBroadcastExchange -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_item_sk#1], [cs_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#14, cc_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#14, cc_name#15] -Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) - -(16) CometBroadcastExchange -Input [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cc_call_center_sk#14, cc_name#15] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] -Right output [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cs_call_center_sk#6], [cc_call_center_sk#14], Inner, BuildRight - -(18) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] -Arguments: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15], [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] - -(19) CometHashAggregate -Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] - -(20) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] - -(22) CometExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] - -(25) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(28) Filter [codegen id : 7] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] - -(30) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] - -(31) CometHashAggregate -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] -Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] - -(32) CometExchange -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] -Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] -Arguments: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28], [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] - -(35) Window -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#29], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] -Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28, rn#29] - -(37) BroadcastExchange -Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#29 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] - -(40) ReusedExchange [Reuses operator id: 32] -Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] - -(41) CometSort -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] -Arguments: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35], [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] - -(43) Window -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] -Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] -Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35, rn#36] - -(45) BroadcastExchange -Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#36 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#28 AS psum#37, sum_sales#35 AS nsum#38] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28, i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] - -(48) TakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/simplified.txt deleted file mode 100644 index d655789fe6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - 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 [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,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt deleted file mode 100644 index 9265c65475..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)))) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#2, 50)) AS i_brand#4, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#3, 50)) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) - -(6) CometBroadcastExchange -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_item_sk#1], [cs_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#14, cc_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#14, cc_name#15] -Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) - -(16) CometBroadcastExchange -Input [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cc_call_center_sk#14, cc_name#15] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] -Right output [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cs_call_center_sk#6], [cc_call_center_sk#14], Inner, BuildRight - -(18) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] -Arguments: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15], [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] - -(19) CometHashAggregate -Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] - -(20) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] - -(22) CometExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] - -(25) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(28) Filter [codegen id : 7] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] - -(30) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] - -(31) CometHashAggregate -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] -Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] - -(32) CometExchange -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] -Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] -Arguments: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28], [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] - -(35) Window -Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#29], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] -Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28, rn#29] - -(37) BroadcastExchange -Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#29 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] - -(40) ReusedExchange [Reuses operator id: 32] -Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] - -(41) CometSort -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] -Arguments: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35], [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] - -(43) Window -Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] -Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] -Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35, rn#36] - -(45) BroadcastExchange -Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#36 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#28 AS psum#37, sum_sales#35 AS nsum#38] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28, i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] - -(48) TakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/extended.txt deleted file mode 100644 index de7d26cd06..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d655789fe6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - 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 [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,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/explain.txt deleted file mode 100644 index 28d5613986..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/explain.txt +++ /dev/null @@ -1,602 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (99) -+- CometTakeOrderedAndProject (98) - +- CometHashAggregate (97) - +- CometColumnarExchange (96) - +- * HashAggregate (95) - +- Union (94) - :- * HashAggregate (79) - : +- * CometColumnarToRow (78) - : +- CometColumnarExchange (77) - : +- * HashAggregate (76) - : +- Union (75) - : :- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- * HashAggregate (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- Union (9) - : : : : :- * Project (4) - : : : : : +- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- * Project (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_returns (5) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometProject (15) - : : +- CometFilter (14) - : : +- CometNativeScan parquet spark_catalog.default.store (13) - : :- * HashAggregate (46) - : : +- * CometColumnarToRow (45) - : : +- CometColumnarExchange (44) - : : +- * HashAggregate (43) - : : +- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- Union (32) - : : : : :- * Project (27) - : : : : : +- * Filter (26) - : : : : : +- * ColumnarToRow (25) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (24) - : : : : +- * Project (31) - : : : : +- * Filter (30) - : : : : +- * ColumnarToRow (29) - : : : : +- Scan parquet spark_catalog.default.catalog_returns (28) - : : : +- ReusedExchange (33) - : : +- BroadcastExchange (40) - : : +- * CometColumnarToRow (39) - : : +- CometProject (38) - : : +- CometFilter (37) - : : +- CometNativeScan parquet spark_catalog.default.catalog_page (36) - : +- * HashAggregate (74) - : +- * CometColumnarToRow (73) - : +- CometColumnarExchange (72) - : +- * HashAggregate (71) - : +- * Project (70) - : +- * BroadcastHashJoin Inner BuildRight (69) - : :- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- Union (60) - : : : :- * Project (50) - : : : : +- * Filter (49) - : : : : +- * ColumnarToRow (48) - : : : : +- Scan parquet spark_catalog.default.web_sales (47) - : : : +- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildLeft (58) - : : : :- BroadcastExchange (53) - : : : : +- * ColumnarToRow (52) - : : : : +- Scan parquet spark_catalog.default.web_returns (51) - : : : +- * CometColumnarToRow (57) - : : : +- CometProject (56) - : : : +- CometFilter (55) - : : : +- CometNativeScan parquet spark_catalog.default.web_sales (54) - : : +- ReusedExchange (61) - : +- BroadcastExchange (68) - : +- * CometColumnarToRow (67) - : +- CometProject (66) - : +- CometFilter (65) - : +- CometNativeScan parquet spark_catalog.default.web_site (64) - :- * HashAggregate (86) - : +- * CometColumnarToRow (85) - : +- CometColumnarExchange (84) - : +- * HashAggregate (83) - : +- * HashAggregate (82) - : +- * CometColumnarToRow (81) - : +- ReusedExchange (80) - +- * HashAggregate (93) - +- * CometColumnarToRow (92) - +- CometColumnarExchange (91) - +- * HashAggregate (90) - +- * HashAggregate (89) - +- * CometColumnarToRow (88) - +- ReusedExchange (87) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 1] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(4) Project [codegen id : 1] -Output [6]: [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] - -(5) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 2] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] - -(7) Filter [codegen id : 2] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#12) - -(8) Project [codegen id : 2] -Output [6]: [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] - -(9) Union - -(10) ReusedExchange [Reuses operator id: 104] -Output [1]: [d_date_sk#22] - -(11) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [date_sk#7] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None - -(12) Project [codegen id : 5] -Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] - -(13) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#23, s_store_id#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [s_store_sk#23, s_store_id#24] -Condition : isnotnull(s_store_sk#23) - -(15) CometProject -Input [2]: [s_store_sk#23, s_store_id#24] -Arguments: [s_store_sk#23, s_store_id#25], [s_store_sk#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#24, 16)) AS s_store_id#25] - -(16) CometColumnarToRow [codegen id : 4] -Input [2]: [s_store_sk#23, s_store_id#25] - -(17) BroadcastExchange -Input [2]: [s_store_sk#23, s_store_id#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(18) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [store_sk#6] -Right keys [1]: [s_store_sk#23] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 5] -Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#25] - -(20) HashAggregate [codegen id : 5] -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -Keys [1]: [s_store_id#25] -Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum#26, sum#27, sum#28, sum#29] -Results [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] - -(21) CometColumnarExchange -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(22) CometColumnarToRow [codegen id : 6] -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] - -(23) HashAggregate [codegen id : 6] -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -Keys [1]: [s_store_id#25] -Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#34, sum(UnscaledValue(return_amt#10))#35, sum(UnscaledValue(profit#9))#36, sum(UnscaledValue(net_loss#11))#37] -Results [5]: [store channel AS channel#38, concat(store, s_store_id#25) AS id#39, MakeDecimal(sum(UnscaledValue(sales_price#8))#34,17,2) AS sales#40, MakeDecimal(sum(UnscaledValue(return_amt#10))#35,17,2) AS returns#41, (MakeDecimal(sum(UnscaledValue(profit#9))#36,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#37,17,2)) AS profit#42] - -(24) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_sold_date_sk#46 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cs_catalog_page_sk)] -ReadSchema: struct - -(25) ColumnarToRow [codegen id : 7] -Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] - -(26) Filter [codegen id : 7] -Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Condition : isnotnull(cs_catalog_page_sk#43) - -(27) Project [codegen id : 7] -Output [6]: [cs_catalog_page_sk#43 AS page_sk#47, cs_sold_date_sk#46 AS date_sk#48, cs_ext_sales_price#44 AS sales_price#49, cs_net_profit#45 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] -Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] - -(28) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cr_catalog_page_sk)] -ReadSchema: struct - -(29) ColumnarToRow [codegen id : 8] -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] - -(30) Filter [codegen id : 8] -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -Condition : isnotnull(cr_catalog_page_sk#53) - -(31) Project [codegen id : 8] -Output [6]: [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] - -(32) Union - -(33) ReusedExchange [Reuses operator id: 104] -Output [1]: [d_date_sk#63] - -(34) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [date_sk#48] -Right keys [1]: [d_date_sk#63] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 11] -Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] -Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] - -(36) CometNativeScan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Condition : isnotnull(cp_catalog_page_sk#64) - -(38) CometProject -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Arguments: [cp_catalog_page_sk#64, cp_catalog_page_id#66], [cp_catalog_page_sk#64, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#65, 16)) AS cp_catalog_page_id#66] - -(39) CometColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] - -(40) BroadcastExchange -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(41) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [page_sk#47] -Right keys [1]: [cp_catalog_page_sk#64] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 11] -Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] -Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#66] - -(43) HashAggregate [codegen id : 11] -Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] -Keys [1]: [cp_catalog_page_id#66] -Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] -Aggregate Attributes [4]: [sum#67, sum#68, sum#69, sum#70] -Results [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] - -(44) CometColumnarExchange -Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] -Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(45) CometColumnarToRow [codegen id : 12] -Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] - -(46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] -Keys [1]: [cp_catalog_page_id#66] -Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#75, sum(UnscaledValue(return_amt#51))#76, sum(UnscaledValue(profit#50))#77, sum(UnscaledValue(net_loss#52))#78] -Results [5]: [catalog channel AS channel#79, concat(catalog_page, cp_catalog_page_id#66) AS id#80, MakeDecimal(sum(UnscaledValue(sales_price#49))#75,17,2) AS sales#81, MakeDecimal(sum(UnscaledValue(return_amt#51))#76,17,2) AS returns#82, (MakeDecimal(sum(UnscaledValue(profit#50))#77,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#52))#78,17,2)) AS profit#83] - -(47) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_sold_date_sk#87 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] - -(49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] -Condition : isnotnull(ws_web_site_sk#84) - -(50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#84 AS wsr_web_site_sk#88, ws_sold_date_sk#87 AS date_sk#89, ws_ext_sales_price#85 AS sales_price#90, ws_net_profit#86 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] -Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] - -(51) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#5)] -ReadSchema: struct - -(52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] - -(53) BroadcastExchange -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [plan_id=5] - -(54) CometNativeScan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(55) CometFilter -Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) - -(56) CometProject -Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] - -(57) CometColumnarToRow -Input [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] - -(58) BroadcastHashJoin [codegen id : 15] -Left keys [2]: [wr_item_sk#94, wr_order_number#95] -Right keys [2]: [ws_item_sk#99, ws_order_number#101] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 15] -Output [6]: [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] -Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] - -(60) Union - -(61) ReusedExchange [Reuses operator id: 104] -Output [1]: [d_date_sk#109] - -(62) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [date_sk#89] -Right keys [1]: [d_date_sk#109] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 18] -Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] -Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] - -(64) CometNativeScan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#110, web_site_id#111] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(65) CometFilter -Input [2]: [web_site_sk#110, web_site_id#111] -Condition : isnotnull(web_site_sk#110) - -(66) CometProject -Input [2]: [web_site_sk#110, web_site_id#111] -Arguments: [web_site_sk#110, web_site_id#112], [web_site_sk#110, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#111, 16)) AS web_site_id#112] - -(67) CometColumnarToRow [codegen id : 17] -Input [2]: [web_site_sk#110, web_site_id#112] - -(68) BroadcastExchange -Input [2]: [web_site_sk#110, web_site_id#112] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(69) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [wsr_web_site_sk#88] -Right keys [1]: [web_site_sk#110] -Join type: Inner -Join condition: None - -(70) Project [codegen id : 18] -Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] -Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#112] - -(71) HashAggregate [codegen id : 18] -Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] -Keys [1]: [web_site_id#112] -Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum#113, sum#114, sum#115, sum#116] -Results [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] - -(72) CometColumnarExchange -Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] -Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(73) CometColumnarToRow [codegen id : 19] -Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] - -(74) HashAggregate [codegen id : 19] -Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] -Keys [1]: [web_site_id#112] -Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#121, sum(UnscaledValue(return_amt#92))#122, sum(UnscaledValue(profit#91))#123, sum(UnscaledValue(net_loss#93))#124] -Results [5]: [web channel AS channel#125, concat(web_site, web_site_id#112) AS id#126, MakeDecimal(sum(UnscaledValue(sales_price#90))#121,17,2) AS sales#127, MakeDecimal(sum(UnscaledValue(return_amt#92))#122,17,2) AS returns#128, (MakeDecimal(sum(UnscaledValue(profit#91))#123,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#124,17,2)) AS profit#129] - -(75) Union - -(76) HashAggregate [codegen id : 20] -Input [5]: [channel#38, id#39, sales#40, returns#41, profit#42] -Keys [2]: [channel#38, id#39] -Functions [3]: [partial_sum(sales#40), partial_sum(returns#41), partial_sum(profit#42)] -Aggregate Attributes [6]: [sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] -Results [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] - -(77) CometColumnarExchange -Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -Arguments: hashpartitioning(channel#38, id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(78) CometColumnarToRow [codegen id : 21] -Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] - -(79) HashAggregate [codegen id : 21] -Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -Keys [2]: [channel#38, id#39] -Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] -Aggregate Attributes [3]: [sum(sales#40)#142, sum(returns#41)#143, sum(profit#42)#144] -Results [5]: [channel#38, id#39, cast(sum(sales#40)#142 as decimal(37,2)) AS sales#145, cast(sum(returns#41)#143 as decimal(37,2)) AS returns#146, cast(sum(profit#42)#144 as decimal(38,2)) AS profit#147] - -(80) ReusedExchange [Reuses operator id: 77] -Output [8]: [channel#148, id#149, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] - -(81) CometColumnarToRow [codegen id : 42] -Input [8]: [channel#148, id#149, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] - -(82) HashAggregate [codegen id : 42] -Input [8]: [channel#148, id#149, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] -Keys [2]: [channel#148, id#149] -Functions [3]: [sum(sales#156), sum(returns#157), sum(profit#158)] -Aggregate Attributes [3]: [sum(sales#156)#142, sum(returns#157)#143, sum(profit#158)#144] -Results [4]: [channel#148, sum(sales#156)#142 AS sales#159, sum(returns#157)#143 AS returns#160, sum(profit#158)#144 AS profit#161] - -(83) HashAggregate [codegen id : 42] -Input [4]: [channel#148, sales#159, returns#160, profit#161] -Keys [1]: [channel#148] -Functions [3]: [partial_sum(sales#159), partial_sum(returns#160), partial_sum(profit#161)] -Aggregate Attributes [6]: [sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] -Results [7]: [channel#148, sum#168, isEmpty#169, sum#170, isEmpty#171, sum#172, isEmpty#173] - -(84) CometColumnarExchange -Input [7]: [channel#148, sum#168, isEmpty#169, sum#170, isEmpty#171, sum#172, isEmpty#173] -Arguments: hashpartitioning(channel#148, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(85) CometColumnarToRow [codegen id : 43] -Input [7]: [channel#148, sum#168, isEmpty#169, sum#170, isEmpty#171, sum#172, isEmpty#173] - -(86) HashAggregate [codegen id : 43] -Input [7]: [channel#148, sum#168, isEmpty#169, sum#170, isEmpty#171, sum#172, isEmpty#173] -Keys [1]: [channel#148] -Functions [3]: [sum(sales#159), sum(returns#160), sum(profit#161)] -Aggregate Attributes [3]: [sum(sales#159)#174, sum(returns#160)#175, sum(profit#161)#176] -Results [5]: [channel#148, null AS id#177, sum(sales#159)#174 AS sum(sales)#178, sum(returns#160)#175 AS sum(returns)#179, sum(profit#161)#176 AS sum(profit)#180] - -(87) ReusedExchange [Reuses operator id: 77] -Output [8]: [channel#181, id#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] - -(88) CometColumnarToRow [codegen id : 64] -Input [8]: [channel#181, id#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] - -(89) HashAggregate [codegen id : 64] -Input [8]: [channel#181, id#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188] -Keys [2]: [channel#181, id#182] -Functions [3]: [sum(sales#189), sum(returns#190), sum(profit#191)] -Aggregate Attributes [3]: [sum(sales#189)#142, sum(returns#190)#143, sum(profit#191)#144] -Results [3]: [sum(sales#189)#142 AS sales#192, sum(returns#190)#143 AS returns#193, sum(profit#191)#144 AS profit#194] - -(90) HashAggregate [codegen id : 64] -Input [3]: [sales#192, returns#193, profit#194] -Keys: [] -Functions [3]: [partial_sum(sales#192), partial_sum(returns#193), partial_sum(profit#194)] -Aggregate Attributes [6]: [sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200] -Results [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] - -(91) CometColumnarExchange -Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(92) CometColumnarToRow [codegen id : 65] -Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] - -(93) HashAggregate [codegen id : 65] -Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] -Keys: [] -Functions [3]: [sum(sales#192), sum(returns#193), sum(profit#194)] -Aggregate Attributes [3]: [sum(sales#192)#207, sum(returns#193)#208, sum(profit#194)#209] -Results [5]: [null AS channel#210, null AS id#211, sum(sales#192)#207 AS sum(sales)#212, sum(returns#193)#208 AS sum(returns)#213, sum(profit#194)#209 AS sum(profit)#214] - -(94) Union - -(95) HashAggregate [codegen id : 66] -Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] -Keys [5]: [channel#38, id#39, sales#145, returns#146, profit#147] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#38, id#39, sales#145, returns#146, profit#147] - -(96) CometColumnarExchange -Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] -Arguments: hashpartitioning(channel#38, id#39, sales#145, returns#146, profit#147, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(97) CometHashAggregate -Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] -Keys [5]: [channel#38, id#39, sales#145, returns#146, profit#147] -Functions: [] - -(98) CometTakeOrderedAndProject -Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#38 ASC NULLS FIRST,id#39 ASC NULLS FIRST], output=[channel#38,id#39,sales#145,returns#146,profit#147]), [channel#38, id#39, sales#145, returns#146, profit#147], 100, 0, [channel#38 ASC NULLS FIRST, id#39 ASC NULLS FIRST], [channel#38, id#39, sales#145, returns#146, profit#147] - -(99) CometColumnarToRow [codegen id : 67] -Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (104) -+- * CometColumnarToRow (103) - +- CometProject (102) - +- CometFilter (101) - +- CometNativeScan parquet spark_catalog.default.date_dim (100) - - -(100) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#215] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] -ReadSchema: struct - -(101) CometFilter -Input [2]: [d_date_sk#22, d_date#215] -Condition : (((isnotnull(d_date#215) AND (d_date#215 >= 1998-08-04)) AND (d_date#215 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) - -(102) CometProject -Input [2]: [d_date_sk#22, d_date#215] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(103) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(104) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 28 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#87 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/simplified.txt deleted file mode 100644 index dc9c9b1ce7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/simplified.txt +++ /dev/null @@ -1,167 +0,0 @@ -WholeStageCodegen (67) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (66) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (21) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (20) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [s_store_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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_id] #3 - WholeStageCodegen (5) - HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,s_store_id] - BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (1) - Project [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - WholeStageCodegen (2) - Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] - Filter [sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (12) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cp_catalog_page_id] #6 - WholeStageCodegen (11) - HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (7) - Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] - Filter [cs_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (8) - Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] - Filter [cr_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (19) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [web_site_id] #8 - WholeStageCodegen (18) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (13) - Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] - Filter [ws_web_site_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (15) - Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (14) - ColumnarToRow - InputAdapter - Scan 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 - CometColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (17) - CometColumnarToRow - InputAdapter - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - WholeStageCodegen (43) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #11 - WholeStageCodegen (42) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (65) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #12 - WholeStageCodegen (64) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/explain.txt deleted file mode 100644 index ca9ea23933..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,524 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (84) -+- CometTakeOrderedAndProject (83) - +- CometHashAggregate (82) - +- CometExchange (81) - +- CometHashAggregate (80) - +- CometUnion (79) - :- CometHashAggregate (68) - : +- CometExchange (67) - : +- CometHashAggregate (66) - : +- CometUnion (65) - : :- CometHashAggregate (22) - : : +- CometExchange (21) - : : +- CometHashAggregate (20) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometUnion (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : :- CometHashAggregate (41) - : : +- CometExchange (40) - : : +- CometHashAggregate (39) - : : +- CometProject (38) - : : +- CometBroadcastHashJoin (37) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometUnion (29) - : : : : :- CometProject (25) - : : : : : +- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) - : : : : +- CometProject (28) - : : : : +- CometFilter (27) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) - : : : +- ReusedExchange (30) - : : +- CometBroadcastExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) - : +- CometHashAggregate (64) - : +- CometExchange (63) - : +- CometHashAggregate (62) - : +- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (55) - : : +- CometBroadcastHashJoin (54) - : : :- CometUnion (52) - : : : :- CometProject (44) - : : : : +- CometFilter (43) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : : : +- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometBroadcastExchange (46) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) - : : : +- CometProject (49) - : : : +- CometFilter (48) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : : +- ReusedExchange (53) - : +- CometBroadcastExchange (59) - : +- CometProject (58) - : +- CometFilter (57) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) - :- CometHashAggregate (73) - : +- CometExchange (72) - : +- CometHashAggregate (71) - : +- CometHashAggregate (70) - : +- ReusedExchange (69) - +- CometHashAggregate (78) - +- CometExchange (77) - +- CometHashAggregate (76) - +- CometHashAggregate (75) - +- ReusedExchange (74) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Arguments: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11], [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#12) - -(6) CometProject -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Arguments: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21], [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] - -(7) CometUnion -Child 0 Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) - -(10) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(11) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(12) CometBroadcastHashJoin -Left output [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [1]: [d_date_sk#22] -Arguments: [date_sk#7], [d_date_sk#22], Inner, BuildRight - -(13) CometProject -Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] -Arguments: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11], [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_store_id#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#24, s_store_id#25] -Condition : isnotnull(s_store_sk#24) - -(16) CometProject -Input [2]: [s_store_sk#24, s_store_id#25] -Arguments: [s_store_sk#24, s_store_id#26], [s_store_sk#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#25, 16)) AS s_store_id#26] - -(17) CometBroadcastExchange -Input [2]: [s_store_sk#24, s_store_id#26] -Arguments: [s_store_sk#24, s_store_id#26] - -(18) CometBroadcastHashJoin -Left output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [2]: [s_store_sk#24, s_store_id#26] -Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight - -(19) CometProject -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#26] -Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] - -(20) CometHashAggregate -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] -Keys [1]: [s_store_id#26] -Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] - -(21) CometExchange -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Keys [1]: [s_store_id#26] -Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk)] -ReadSchema: struct - -(24) CometFilter -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : isnotnull(cs_catalog_page_sk#31) - -(25) CometProject -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cr_catalog_page_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Condition : isnotnull(cr_catalog_page_sk#42) - -(28) CometProject -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] - -(29) CometUnion -Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] - -(30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#52] - -(31) CometBroadcastHashJoin -Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [1]: [d_date_sk#52] -Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight - -(32) CometProject -Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] -Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Condition : isnotnull(cp_catalog_page_sk#53) - -(35) CometProject -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#54, 16)) AS cp_catalog_page_id#55] - -(36) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] - -(37) CometBroadcastHashJoin -Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight - -(38) CometProject -Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] - -(39) CometHashAggregate -Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] - -(40) CometExchange -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(41) CometHashAggregate -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] -PushedFilters: [IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(43) CometFilter -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_web_site_sk#60) - -(44) CometProject -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] -ReadSchema: struct - -(46) CometBroadcastExchange -Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(48) CometFilter -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) - -(49) CometProject -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] - -(50) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft - -(51) CometProject -Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] - -(52) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] - -(53) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#86] - -(54) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [1]: [d_date_sk#86] -Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight - -(55) CometProject -Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] -Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#87, web_site_id#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(57) CometFilter -Input [2]: [web_site_sk#87, web_site_id#88] -Condition : isnotnull(web_site_sk#87) - -(58) CometProject -Input [2]: [web_site_sk#87, web_site_id#88] -Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#88, 16)) AS web_site_id#89] - -(59) CometBroadcastExchange -Input [2]: [web_site_sk#87, web_site_id#89] -Arguments: [web_site_sk#87, web_site_id#89] - -(60) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [2]: [web_site_sk#87, web_site_id#89] -Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight - -(61) CometProject -Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] -Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] - -(62) CometHashAggregate -Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] -Keys [1]: [web_site_id#89] -Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] - -(63) CometExchange -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(64) CometHashAggregate -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Keys [1]: [web_site_id#89] -Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] - -(65) CometUnion -Child 0 Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] -Child 1 Input [5]: [channel#99, id#100, sales#101, returns#102, profit#103] -Child 2 Input [5]: [channel#104, id#105, sales#106, returns#107, profit#108] - -(66) CometHashAggregate -Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] -Keys [2]: [channel#94, id#95] -Functions [3]: [partial_sum(sales#96), partial_sum(returns#97), partial_sum(profit#98)] - -(67) CometExchange -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: hashpartitioning(channel#94, id#95, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(68) CometHashAggregate -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#94, id#95] -Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] - -(69) ReusedExchange [Reuses operator id: 67] -Output [8]: [channel#115, id#116, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] - -(70) CometHashAggregate -Input [8]: [channel#115, id#116, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Keys [2]: [channel#115, id#116] -Functions [3]: [sum(sales#123), sum(returns#124), sum(profit#125)] - -(71) CometHashAggregate -Input [4]: [channel#115, sales#126, returns#127, profit#128] -Keys [1]: [channel#115] -Functions [3]: [partial_sum(sales#126), partial_sum(returns#127), partial_sum(profit#128)] - -(72) CometExchange -Input [7]: [channel#115, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] -Arguments: hashpartitioning(channel#115, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(73) CometHashAggregate -Input [7]: [channel#115, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] -Keys [1]: [channel#115] -Functions [3]: [sum(sales#126), sum(returns#127), sum(profit#128)] - -(74) ReusedExchange [Reuses operator id: 67] -Output [8]: [channel#135, id#136, sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] - -(75) CometHashAggregate -Input [8]: [channel#135, id#136, sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] -Keys [2]: [channel#135, id#136] -Functions [3]: [sum(sales#143), sum(returns#144), sum(profit#145)] - -(76) CometHashAggregate -Input [3]: [sales#146, returns#147, profit#148] -Keys: [] -Functions [3]: [partial_sum(sales#146), partial_sum(returns#147), partial_sum(profit#148)] - -(77) CometExchange -Input [6]: [sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(78) CometHashAggregate -Input [6]: [sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] -Keys: [] -Functions [3]: [sum(sales#146), sum(returns#147), sum(profit#148)] - -(79) CometUnion -Child 0 Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] -Child 1 Input [5]: [channel#115, id#158, sum(sales)#159, sum(returns)#160, sum(profit)#161] -Child 2 Input [5]: [channel#162, id#163, sum(sales)#164, sum(returns)#165, sum(profit)#166] - -(80) CometHashAggregate -Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] -Keys [5]: [channel#94, id#95, sales#155, returns#156, profit#157] -Functions: [] - -(81) CometExchange -Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] -Arguments: hashpartitioning(channel#94, id#95, sales#155, returns#156, profit#157, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(82) CometHashAggregate -Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] -Keys [5]: [channel#94, id#95, sales#155, returns#156, profit#157] -Functions: [] - -(83) CometTakeOrderedAndProject -Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#94 ASC NULLS FIRST,id#95 ASC NULLS FIRST], output=[channel#94,id#95,sales#155,returns#156,profit#157]), [channel#94, id#95, sales#155, returns#156, profit#157], 100, 0, [channel#94 ASC NULLS FIRST, id#95 ASC NULLS FIRST], [channel#94, id#95, sales#155, returns#156, profit#157] - -(84) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (89) -+- * CometColumnarToRow (88) - +- CometProject (87) - +- CometFilter (86) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) - - -(85) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] -ReadSchema: struct - -(86) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) - -(87) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(88) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(89) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/simplified.txt deleted file mode 100644 index c0b236a57a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,99 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id] #2 - CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [s_store_id] #3 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - 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 [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,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #6 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [cp_catalog_page_id] #7 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - 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 [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,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [cp_catalog_page_sk,cp_catalog_page_id] #8 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [web_site_id] #9 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - 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 [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,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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,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 [native_iceberg_compat] 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_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 [web_site_sk,web_site_id] #11 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] - CometExchange [channel] #12 - CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] - CometExchange #13 - CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/explain.txt deleted file mode 100644 index ca9ea23933..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/explain.txt +++ /dev/null @@ -1,524 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (84) -+- CometTakeOrderedAndProject (83) - +- CometHashAggregate (82) - +- CometExchange (81) - +- CometHashAggregate (80) - +- CometUnion (79) - :- CometHashAggregate (68) - : +- CometExchange (67) - : +- CometHashAggregate (66) - : +- CometUnion (65) - : :- CometHashAggregate (22) - : : +- CometExchange (21) - : : +- CometHashAggregate (20) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometUnion (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : :- CometHashAggregate (41) - : : +- CometExchange (40) - : : +- CometHashAggregate (39) - : : +- CometProject (38) - : : +- CometBroadcastHashJoin (37) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometUnion (29) - : : : : :- CometProject (25) - : : : : : +- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) - : : : : +- CometProject (28) - : : : : +- CometFilter (27) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) - : : : +- ReusedExchange (30) - : : +- CometBroadcastExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) - : +- CometHashAggregate (64) - : +- CometExchange (63) - : +- CometHashAggregate (62) - : +- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (55) - : : +- CometBroadcastHashJoin (54) - : : :- CometUnion (52) - : : : :- CometProject (44) - : : : : +- CometFilter (43) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : : : +- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometBroadcastExchange (46) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) - : : : +- CometProject (49) - : : : +- CometFilter (48) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : : +- ReusedExchange (53) - : +- CometBroadcastExchange (59) - : +- CometProject (58) - : +- CometFilter (57) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) - :- CometHashAggregate (73) - : +- CometExchange (72) - : +- CometHashAggregate (71) - : +- CometHashAggregate (70) - : +- ReusedExchange (69) - +- CometHashAggregate (78) - +- CometExchange (77) - +- CometHashAggregate (76) - +- CometHashAggregate (75) - +- ReusedExchange (74) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Arguments: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11], [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#12) - -(6) CometProject -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Arguments: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21], [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] - -(7) CometUnion -Child 0 Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) - -(10) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(11) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(12) CometBroadcastHashJoin -Left output [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [1]: [d_date_sk#22] -Arguments: [date_sk#7], [d_date_sk#22], Inner, BuildRight - -(13) CometProject -Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] -Arguments: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11], [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_store_id#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#24, s_store_id#25] -Condition : isnotnull(s_store_sk#24) - -(16) CometProject -Input [2]: [s_store_sk#24, s_store_id#25] -Arguments: [s_store_sk#24, s_store_id#26], [s_store_sk#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#25, 16)) AS s_store_id#26] - -(17) CometBroadcastExchange -Input [2]: [s_store_sk#24, s_store_id#26] -Arguments: [s_store_sk#24, s_store_id#26] - -(18) CometBroadcastHashJoin -Left output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [2]: [s_store_sk#24, s_store_id#26] -Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight - -(19) CometProject -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#26] -Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] - -(20) CometHashAggregate -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] -Keys [1]: [s_store_id#26] -Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] - -(21) CometExchange -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Keys [1]: [s_store_id#26] -Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk)] -ReadSchema: struct - -(24) CometFilter -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : isnotnull(cs_catalog_page_sk#31) - -(25) CometProject -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cr_catalog_page_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Condition : isnotnull(cr_catalog_page_sk#42) - -(28) CometProject -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] - -(29) CometUnion -Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] - -(30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#52] - -(31) CometBroadcastHashJoin -Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [1]: [d_date_sk#52] -Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight - -(32) CometProject -Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] -Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Condition : isnotnull(cp_catalog_page_sk#53) - -(35) CometProject -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#54, 16)) AS cp_catalog_page_id#55] - -(36) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] - -(37) CometBroadcastHashJoin -Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight - -(38) CometProject -Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] - -(39) CometHashAggregate -Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] - -(40) CometExchange -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(41) CometHashAggregate -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] -PushedFilters: [IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(43) CometFilter -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_web_site_sk#60) - -(44) CometProject -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] -ReadSchema: struct - -(46) CometBroadcastExchange -Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(48) CometFilter -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) - -(49) CometProject -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] - -(50) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft - -(51) CometProject -Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] - -(52) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] - -(53) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#86] - -(54) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [1]: [d_date_sk#86] -Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight - -(55) CometProject -Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] -Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#87, web_site_id#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(57) CometFilter -Input [2]: [web_site_sk#87, web_site_id#88] -Condition : isnotnull(web_site_sk#87) - -(58) CometProject -Input [2]: [web_site_sk#87, web_site_id#88] -Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#88, 16)) AS web_site_id#89] - -(59) CometBroadcastExchange -Input [2]: [web_site_sk#87, web_site_id#89] -Arguments: [web_site_sk#87, web_site_id#89] - -(60) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [2]: [web_site_sk#87, web_site_id#89] -Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight - -(61) CometProject -Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] -Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] - -(62) CometHashAggregate -Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] -Keys [1]: [web_site_id#89] -Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] - -(63) CometExchange -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(64) CometHashAggregate -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Keys [1]: [web_site_id#89] -Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] - -(65) CometUnion -Child 0 Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] -Child 1 Input [5]: [channel#99, id#100, sales#101, returns#102, profit#103] -Child 2 Input [5]: [channel#104, id#105, sales#106, returns#107, profit#108] - -(66) CometHashAggregate -Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] -Keys [2]: [channel#94, id#95] -Functions [3]: [partial_sum(sales#96), partial_sum(returns#97), partial_sum(profit#98)] - -(67) CometExchange -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: hashpartitioning(channel#94, id#95, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(68) CometHashAggregate -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#94, id#95] -Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] - -(69) ReusedExchange [Reuses operator id: 67] -Output [8]: [channel#115, id#116, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] - -(70) CometHashAggregate -Input [8]: [channel#115, id#116, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Keys [2]: [channel#115, id#116] -Functions [3]: [sum(sales#123), sum(returns#124), sum(profit#125)] - -(71) CometHashAggregate -Input [4]: [channel#115, sales#126, returns#127, profit#128] -Keys [1]: [channel#115] -Functions [3]: [partial_sum(sales#126), partial_sum(returns#127), partial_sum(profit#128)] - -(72) CometExchange -Input [7]: [channel#115, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] -Arguments: hashpartitioning(channel#115, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(73) CometHashAggregate -Input [7]: [channel#115, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] -Keys [1]: [channel#115] -Functions [3]: [sum(sales#126), sum(returns#127), sum(profit#128)] - -(74) ReusedExchange [Reuses operator id: 67] -Output [8]: [channel#135, id#136, sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] - -(75) CometHashAggregate -Input [8]: [channel#135, id#136, sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] -Keys [2]: [channel#135, id#136] -Functions [3]: [sum(sales#143), sum(returns#144), sum(profit#145)] - -(76) CometHashAggregate -Input [3]: [sales#146, returns#147, profit#148] -Keys: [] -Functions [3]: [partial_sum(sales#146), partial_sum(returns#147), partial_sum(profit#148)] - -(77) CometExchange -Input [6]: [sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(78) CometHashAggregate -Input [6]: [sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] -Keys: [] -Functions [3]: [sum(sales#146), sum(returns#147), sum(profit#148)] - -(79) CometUnion -Child 0 Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] -Child 1 Input [5]: [channel#115, id#158, sum(sales)#159, sum(returns)#160, sum(profit)#161] -Child 2 Input [5]: [channel#162, id#163, sum(sales)#164, sum(returns)#165, sum(profit)#166] - -(80) CometHashAggregate -Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] -Keys [5]: [channel#94, id#95, sales#155, returns#156, profit#157] -Functions: [] - -(81) CometExchange -Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] -Arguments: hashpartitioning(channel#94, id#95, sales#155, returns#156, profit#157, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(82) CometHashAggregate -Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] -Keys [5]: [channel#94, id#95, sales#155, returns#156, profit#157] -Functions: [] - -(83) CometTakeOrderedAndProject -Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#94 ASC NULLS FIRST,id#95 ASC NULLS FIRST], output=[channel#94,id#95,sales#155,returns#156,profit#157]), [channel#94, id#95, sales#155, returns#156, profit#157], 100, 0, [channel#94 ASC NULLS FIRST, id#95 ASC NULLS FIRST], [channel#94, id#95, sales#155, returns#156, profit#157] - -(84) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (89) -+- * CometColumnarToRow (88) - +- CometProject (87) - +- CometFilter (86) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) - - -(85) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] -ReadSchema: struct - -(86) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) - -(87) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(88) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(89) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/extended.txt deleted file mode 100644 index 35b86dbff5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/extended.txt +++ /dev/null @@ -1,269 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastExchange - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastExchange - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - -Comet accelerated 242 out of 263 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index c0b236a57a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt +++ /dev/null @@ -1,99 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id] #2 - CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [s_store_id] #3 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - 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 [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,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #6 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [cp_catalog_page_id] #7 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - 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 [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,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [cp_catalog_page_sk,cp_catalog_page_id] #8 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [web_site_id] #9 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - 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 [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,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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,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 [native_iceberg_compat] 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_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 [web_site_sk,web_site_id] #11 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] - CometExchange [channel] #12 - CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] - CometExchange #13 - CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/explain.txt deleted file mode 100644 index 8c97a06a14..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (40) -+- * Filter (39) - +- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * CometColumnarToRow (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.customer (4) - : : +- BroadcastExchange (13) - : : +- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet spark_catalog.default.store_sales (10) - : +- ReusedExchange (16) - +- BroadcastExchange (32) - +- * CometColumnarToRow (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (20) - : +- CometNativeScan parquet spark_catalog.default.item (19) - +- CometBroadcastExchange (28) - +- CometFilter (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometFilter (22) - +- CometNativeScan parquet spark_catalog.default.item (21) - - -(1) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#1, ca_state#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ca_address_sk#1, ca_state#2] -Condition : isnotnull(ca_address_sk#1) - -(3) CometProject -Input [2]: [ca_address_sk#1, ca_state#2] -Arguments: [ca_address_sk#1, ca_state#3], [ca_address_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#2, 2)) AS ca_state#3] - -(4) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_customer_sk#4)) - -(6) CometBroadcastExchange -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [c_customer_sk#4, c_current_addr_sk#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#1, ca_state#3] -Right output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] - -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ca_state#3, c_customer_sk#4] - -(10) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 1] -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(12) Filter [codegen id : 1] -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) - -(13) BroadcastExchange -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [c_customer_sk#4] -Right keys [1]: [ss_customer_sk#7] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(16) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#10] - -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [2]: [ca_state#3, ss_item_sk#6] -Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] - -(19) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#11, i_current_price#12, i_category#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_category), IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [i_item_sk#11, i_current_price#12, i_category#13] -Condition : ((isnotnull(i_current_price#12) AND isnotnull(i_category#13)) AND isnotnull(i_item_sk#11)) - -(21) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_current_price#14, i_category#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_current_price#14, i_category#15] -Condition : isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#15, 50))) - -(23) CometProject -Input [2]: [i_current_price#14, i_category#15] -Arguments: [i_category#16, i_current_price#14], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#15, 50)) AS i_category#16, i_current_price#14] - -(24) CometHashAggregate -Input [2]: [i_category#16, i_current_price#14] -Keys [1]: [i_category#16] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#14))] - -(25) CometExchange -Input [3]: [i_category#16, sum#17, count#18] -Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(26) CometHashAggregate -Input [3]: [i_category#16, sum#17, count#18] -Keys [1]: [i_category#16] -Functions [1]: [avg(UnscaledValue(i_current_price#14))] - -(27) CometFilter -Input [2]: [avg(i_current_price)#19, i_category#16] -Condition : isnotnull(avg(i_current_price)#19) - -(28) CometBroadcastExchange -Input [2]: [avg(i_current_price)#19, i_category#16] -Arguments: [avg(i_current_price)#19, i_category#16] - -(29) CometBroadcastHashJoin -Left output [3]: [i_item_sk#11, i_current_price#12, i_category#13] -Right output [2]: [avg(i_current_price)#19, i_category#16] -Arguments: [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#13, 50))], [i_category#16], Inner, (cast(i_current_price#12 as decimal(14,7)) > (1.2 * avg(i_current_price)#19)), BuildRight - -(30) CometProject -Input [5]: [i_item_sk#11, i_current_price#12, i_category#13, avg(i_current_price)#19, i_category#16] -Arguments: [i_item_sk#11], [i_item_sk#11] - -(31) CometColumnarToRow [codegen id : 3] -Input [1]: [i_item_sk#11] - -(32) BroadcastExchange -Input [1]: [i_item_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(33) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#6] -Right keys [1]: [i_item_sk#11] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 4] -Output [1]: [ca_state#3] -Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#11] - -(35) HashAggregate [codegen id : 4] -Input [1]: [ca_state#3] -Keys [1]: [ca_state#3] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#20] -Results [2]: [ca_state#3, count#21] - -(36) CometColumnarExchange -Input [2]: [ca_state#3, count#21] -Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(37) CometColumnarToRow [codegen id : 5] -Input [2]: [ca_state#3, count#21] - -(38) HashAggregate [codegen id : 5] -Input [2]: [ca_state#3, count#21] -Keys [1]: [ca_state#3] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#22] -Results [2]: [ca_state#3 AS state#23, count(1)#22 AS cnt#24] - -(39) Filter [codegen id : 5] -Input [2]: [state#23, cnt#24] -Condition : (cnt#24 >= 10) - -(40) TakeOrderedAndProject -Input [2]: [state#23, cnt#24] -Arguments: 100, [cnt#24 ASC NULLS FIRST, state#23 ASC NULLS FIRST], [state#23, cnt#24] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometNativeScan parquet spark_catalog.default.date_dim (41) - - -(41) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), EqualTo(d_month_seq,ScalarSubquery#26), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#25] -Condition : ((isnotnull(d_month_seq#25) AND (d_month_seq#25 = ReusedSubquery Subquery scalar-subquery#26, [id=#27])) AND isnotnull(d_date_sk#10)) - -(43) CometProject -Input [2]: [d_date_sk#10, d_month_seq#25] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(45) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 42 Hosting Expression = ReusedSubquery Subquery scalar-subquery#26, [id=#27] - -Subquery:3 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#26, [id=#27] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometNativeScan parquet spark_catalog.default.date_dim (46) - - -(46) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#28, d_year#29, d_moy#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_month_seq#28, d_year#29, d_moy#30] -Condition : (((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 2000)) AND (d_moy#30 = 1)) - -(48) CometProject -Input [3]: [d_month_seq#28, d_year#29, d_moy#30] -Arguments: [d_month_seq#28], [d_month_seq#28] - -(49) CometHashAggregate -Input [1]: [d_month_seq#28] -Keys [1]: [d_month_seq#28] -Functions: [] - -(50) CometExchange -Input [1]: [d_month_seq#28] -Arguments: hashpartitioning(d_month_seq#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(51) CometHashAggregate -Input [1]: [d_month_seq#28] -Keys [1]: [d_month_seq#28] -Functions: [] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#28] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/simplified.txt deleted file mode 100644 index 6b96848c69..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/simplified.txt +++ /dev/null @@ -1,70 +0,0 @@ -TakeOrderedAndProject [cnt,state] - WholeStageCodegen (5) - Filter [cnt] - HashAggregate [ca_state,count] [count(1),state,cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state] #1 - WholeStageCodegen (4) - HashAggregate [ca_state] [count,count] - Project [ca_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ca_state,ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ca_state,ss_item_sk,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_customer_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #7 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #8 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_category,i_current_price] - CometFilter [i_current_price,i_category] - CometNativeScan 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/q6.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/explain.txt deleted file mode 100644 index a41d75f6da..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,310 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometFilter (38) - +- CometHashAggregate (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (4) - : : +- CometBroadcastExchange (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (9) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- CometBroadcastExchange (29) - +- CometFilter (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (22) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#1, ca_state#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ca_address_sk#1, ca_state#2] -Condition : isnotnull(ca_address_sk#1) - -(3) CometProject -Input [2]: [ca_address_sk#1, ca_state#2] -Arguments: [ca_address_sk#1, ca_state#3], [ca_address_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#2, 2)) AS ca_state#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_customer_sk#4)) - -(6) CometBroadcastExchange -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [c_customer_sk#4, c_current_addr_sk#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#1, ca_state#3] -Right output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) - -(11) CometBroadcastExchange -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(12) CometBroadcastHashJoin -Left output [2]: [ca_state#3, c_customer_sk#4] -Right output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [c_customer_sk#4], [ss_customer_sk#7], Inner, BuildRight - -(13) CometProject -Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8], [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), EqualTo(d_month_seq,ScalarSubquery#12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(16) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(18) CometBroadcastHashJoin -Left output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(19) CometProject -Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ca_state#3, ss_item_sk#6], [ca_state#3, ss_item_sk#6] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_category), IsNotNull(i_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Condition : ((isnotnull(i_current_price#15) AND isnotnull(i_category#16)) AND isnotnull(i_item_sk#14)) - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_current_price#17, i_category#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(23) CometFilter -Input [2]: [i_current_price#17, i_category#18] -Condition : isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#18, 50))) - -(24) CometProject -Input [2]: [i_current_price#17, i_category#18] -Arguments: [i_category#19, i_current_price#17], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#18, 50)) AS i_category#19, i_current_price#17] - -(25) CometHashAggregate -Input [2]: [i_category#19, i_current_price#17] -Keys [1]: [i_category#19] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#17))] - -(26) CometExchange -Input [3]: [i_category#19, sum#20, count#21] -Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [3]: [i_category#19, sum#20, count#21] -Keys [1]: [i_category#19] -Functions [1]: [avg(UnscaledValue(i_current_price#17))] - -(28) CometFilter -Input [2]: [avg(i_current_price)#22, i_category#19] -Condition : isnotnull(avg(i_current_price)#22) - -(29) CometBroadcastExchange -Input [2]: [avg(i_current_price)#22, i_category#19] -Arguments: [avg(i_current_price)#22, i_category#19] - -(30) CometBroadcastHashJoin -Left output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Right output [2]: [avg(i_current_price)#22, i_category#19] -Arguments: [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#16, 50))], [i_category#19], Inner, (cast(i_current_price#15 as decimal(14,7)) > (1.2 * avg(i_current_price)#22)), BuildRight - -(31) CometProject -Input [5]: [i_item_sk#14, i_current_price#15, i_category#16, avg(i_current_price)#22, i_category#19] -Arguments: [i_item_sk#14], [i_item_sk#14] - -(32) CometBroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: [i_item_sk#14] - -(33) CometBroadcastHashJoin -Left output [2]: [ca_state#3, ss_item_sk#6] -Right output [1]: [i_item_sk#14] -Arguments: [ss_item_sk#6], [i_item_sk#14], Inner, BuildRight - -(34) CometProject -Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#14] -Arguments: [ca_state#3], [ca_state#3] - -(35) CometHashAggregate -Input [1]: [ca_state#3] -Keys [1]: [ca_state#3] -Functions [1]: [partial_count(1)] - -(36) CometExchange -Input [2]: [ca_state#3, count#23] -Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(37) CometHashAggregate -Input [2]: [ca_state#3, count#23] -Keys [1]: [ca_state#3] -Functions [1]: [count(1)] - -(38) CometFilter -Input [2]: [state#24, cnt#25] -Condition : (cnt#25 >= 10) - -(39) CometTakeOrderedAndProject -Input [2]: [state#24, cnt#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#25 ASC NULLS FIRST,state#24 ASC NULLS FIRST], output=[state#24,cnt#25]), [state#24, cnt#25], 100, 0, [cnt#25 ASC NULLS FIRST, state#24 ASC NULLS FIRST], [state#24, cnt#25] - -(40) CometColumnarToRow [codegen id : 1] -Input [2]: [state#24, cnt#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), EqualTo(d_month_seq,ScalarSubquery#12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(43) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(45) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 42 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] - -Subquery:3 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) - - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#26, d_year#27, d_moy#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Condition : (((isnotnull(d_year#27) AND isnotnull(d_moy#28)) AND (d_year#27 = 2000)) AND (d_moy#28 = 1)) - -(48) CometProject -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Arguments: [d_month_seq#26], [d_month_seq#26] - -(49) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] -Functions: [] - -(50) CometExchange -Input [1]: [d_month_seq#26] -Arguments: hashpartitioning(d_month_seq#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(51) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] -Functions: [] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#26] - -Subquery:4 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] - -Subquery:5 Hosting operator id = 14 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/simplified.txt deleted file mode 100644 index 010720bc0e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,63 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [state,cnt] - CometFilter [state,cnt] - CometHashAggregate [count] [state,cnt,ca_state,count(1)] - CometExchange [ca_state] #1 - CometHashAggregate [ca_state,count] - CometProject [ca_state] - CometBroadcastHashJoin [ca_state,ss_item_sk,i_item_sk] - CometProject [ca_state,ss_item_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 [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,ca_state,c_customer_sk,c_current_addr_sk] - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometBroadcastExchange [i_item_sk] #7 - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #8 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #9 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_category,i_current_price] - CometFilter [i_current_price,i_category] - CometScan [native_iceberg_compat] 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/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/explain.txt deleted file mode 100644 index a41d75f6da..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/explain.txt +++ /dev/null @@ -1,310 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometFilter (38) - +- CometHashAggregate (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (4) - : : +- CometBroadcastExchange (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (9) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- CometBroadcastExchange (29) - +- CometFilter (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (22) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#1, ca_state#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ca_address_sk#1, ca_state#2] -Condition : isnotnull(ca_address_sk#1) - -(3) CometProject -Input [2]: [ca_address_sk#1, ca_state#2] -Arguments: [ca_address_sk#1, ca_state#3], [ca_address_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#2, 2)) AS ca_state#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_customer_sk#4)) - -(6) CometBroadcastExchange -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [c_customer_sk#4, c_current_addr_sk#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#1, ca_state#3] -Right output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) - -(11) CometBroadcastExchange -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(12) CometBroadcastHashJoin -Left output [2]: [ca_state#3, c_customer_sk#4] -Right output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [c_customer_sk#4], [ss_customer_sk#7], Inner, BuildRight - -(13) CometProject -Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8], [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), EqualTo(d_month_seq,ScalarSubquery#12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(16) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(18) CometBroadcastHashJoin -Left output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(19) CometProject -Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ca_state#3, ss_item_sk#6], [ca_state#3, ss_item_sk#6] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_category), IsNotNull(i_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Condition : ((isnotnull(i_current_price#15) AND isnotnull(i_category#16)) AND isnotnull(i_item_sk#14)) - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_current_price#17, i_category#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(23) CometFilter -Input [2]: [i_current_price#17, i_category#18] -Condition : isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#18, 50))) - -(24) CometProject -Input [2]: [i_current_price#17, i_category#18] -Arguments: [i_category#19, i_current_price#17], [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#18, 50)) AS i_category#19, i_current_price#17] - -(25) CometHashAggregate -Input [2]: [i_category#19, i_current_price#17] -Keys [1]: [i_category#19] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#17))] - -(26) CometExchange -Input [3]: [i_category#19, sum#20, count#21] -Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [3]: [i_category#19, sum#20, count#21] -Keys [1]: [i_category#19] -Functions [1]: [avg(UnscaledValue(i_current_price#17))] - -(28) CometFilter -Input [2]: [avg(i_current_price)#22, i_category#19] -Condition : isnotnull(avg(i_current_price)#22) - -(29) CometBroadcastExchange -Input [2]: [avg(i_current_price)#22, i_category#19] -Arguments: [avg(i_current_price)#22, i_category#19] - -(30) CometBroadcastHashJoin -Left output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Right output [2]: [avg(i_current_price)#22, i_category#19] -Arguments: [static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#16, 50))], [i_category#19], Inner, (cast(i_current_price#15 as decimal(14,7)) > (1.2 * avg(i_current_price)#22)), BuildRight - -(31) CometProject -Input [5]: [i_item_sk#14, i_current_price#15, i_category#16, avg(i_current_price)#22, i_category#19] -Arguments: [i_item_sk#14], [i_item_sk#14] - -(32) CometBroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: [i_item_sk#14] - -(33) CometBroadcastHashJoin -Left output [2]: [ca_state#3, ss_item_sk#6] -Right output [1]: [i_item_sk#14] -Arguments: [ss_item_sk#6], [i_item_sk#14], Inner, BuildRight - -(34) CometProject -Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#14] -Arguments: [ca_state#3], [ca_state#3] - -(35) CometHashAggregate -Input [1]: [ca_state#3] -Keys [1]: [ca_state#3] -Functions [1]: [partial_count(1)] - -(36) CometExchange -Input [2]: [ca_state#3, count#23] -Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(37) CometHashAggregate -Input [2]: [ca_state#3, count#23] -Keys [1]: [ca_state#3] -Functions [1]: [count(1)] - -(38) CometFilter -Input [2]: [state#24, cnt#25] -Condition : (cnt#25 >= 10) - -(39) CometTakeOrderedAndProject -Input [2]: [state#24, cnt#25] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#25 ASC NULLS FIRST,state#24 ASC NULLS FIRST], output=[state#24,cnt#25]), [state#24, cnt#25], 100, 0, [cnt#25 ASC NULLS FIRST, state#24 ASC NULLS FIRST], [state#24, cnt#25] - -(40) CometColumnarToRow [codegen id : 1] -Input [2]: [state#24, cnt#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), EqualTo(d_month_seq,ScalarSubquery#12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(43) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(45) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 42 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] - -Subquery:3 Hosting operator id = 41 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) - - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#26, d_year#27, d_moy#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Condition : (((isnotnull(d_year#27) AND isnotnull(d_moy#28)) AND (d_year#27 = 2000)) AND (d_moy#28 = 1)) - -(48) CometProject -Input [3]: [d_month_seq#26, d_year#27, d_moy#28] -Arguments: [d_month_seq#26], [d_month_seq#26] - -(49) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] -Functions: [] - -(50) CometExchange -Input [1]: [d_month_seq#26] -Arguments: hashpartitioning(d_month_seq#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(51) CometHashAggregate -Input [1]: [d_month_seq#26] -Keys [1]: [d_month_seq#26] -Functions: [] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#26] - -Subquery:4 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] - -Subquery:5 Hosting operator id = 14 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/extended.txt deleted file mode 100644 index 2d049f149f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/extended.txt +++ /dev/null @@ -1,59 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 48 out of 54 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 010720bc0e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt +++ /dev/null @@ -1,63 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [state,cnt] - CometFilter [state,cnt] - CometHashAggregate [count] [state,cnt,ca_state,count(1)] - CometExchange [ca_state] #1 - CometHashAggregate [ca_state,count] - CometProject [ca_state] - CometBroadcastHashJoin [ca_state,ss_item_sk,i_item_sk] - CometProject [ca_state,ss_item_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 [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,ca_state,c_customer_sk,c_current_addr_sk] - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometBroadcastExchange [i_item_sk] #7 - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #8 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #9 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_category,i_current_price] - CometFilter [i_current_price,i_category] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/explain.txt deleted file mode 100644 index 245bcb1bd2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/explain.txt +++ /dev/null @@ -1,1029 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (180) -+- CometSort (179) - +- CometExchange (178) - +- CometProject (177) - +- CometSortMergeJoin (176) - :- CometSort (105) - : +- CometExchange (104) - : +- CometHashAggregate (103) - : +- CometHashAggregate (102) - : +- CometProject (101) - : +- CometBroadcastHashJoin (100) - : :- CometProject (95) - : : +- CometBroadcastHashJoin (94) - : : :- CometProject (92) - : : : +- CometBroadcastHashJoin (91) - : : : :- CometProject (87) - : : : : +- CometBroadcastHashJoin (86) - : : : : :- CometProject (84) - : : : : : +- CometBroadcastHashJoin (83) - : : : : : :- CometProject (78) - : : : : : : +- CometBroadcastHashJoin (77) - : : : : : : :- CometProject (75) - : : : : : : : +- CometBroadcastHashJoin (74) - : : : : : : : :- CometProject (70) - : : : : : : : : +- CometBroadcastHashJoin (69) - : : : : : : : : :- CometProject (65) - : : : : : : : : : +- CometBroadcastHashJoin (64) - : : : : : : : : : :- CometProject (62) - : : : : : : : : : : +- CometBroadcastHashJoin (61) - : : : : : : : : : : :- CometProject (56) - : : : : : : : : : : : +- CometBroadcastHashJoin (55) - : : : : : : : : : : : :- CometProject (53) - : : : : : : : : : : : : +- CometBroadcastHashJoin (52) - : : : : : : : : : : : : :- CometProject (48) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (47) - : : : : : : : : : : : : : :- CometProject (43) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (42) - : : : : : : : : : : : : : : :- CometProject (37) - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (36) - : : : : : : : : : : : : : : : :- CometProject (32) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (31) - : : : : : : : : : : : : : : : : :- CometSort (12) - : : : : : : : : : : : : : : : : : +- CometColumnarExchange (11) - : : : : : : : : : : : : : : : : : +- * Project (10) - : : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : : : : : : : : : : : : : : :- BroadcastExchange (4) - : : : : : : : : : : : : : : : : : : +- * Filter (3) - : : : : : : : : : : : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (8) - : : : : : : : : : : : : : : : : : +- CometProject (7) - : : : : : : : : : : : : : : : : : +- CometFilter (6) - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (5) - : : : : : : : : : : : : : : : : +- CometSort (30) - : : : : : : : : : : : : : : : : +- CometProject (29) - : : : : : : : : : : : : : : : : +- CometFilter (28) - : : : : : : : : : : : : : : : : +- CometHashAggregate (27) - : : : : : : : : : : : : : : : : +- CometExchange (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometProject (24) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (23) - : : : : : : : : : : : : : : : : :- CometSort (17) - : : : : : : : : : : : : : : : : : +- CometExchange (16) - : : : : : : : : : : : : : : : : : +- CometProject (15) - : : : : : : : : : : : : : : : : : +- CometFilter (14) - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : : : : : : : : : : : +- CometSort (22) - : : : : : : : : : : : : : : : : +- CometExchange (21) - : : : : : : : : : : : : : : : : +- CometProject (20) - : : : : : : : : : : : : : : : : +- CometFilter (19) - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (18) - : : : : : : : : : : : : : : : +- CometBroadcastExchange (35) - : : : : : : : : : : : : : : : +- CometFilter (34) - : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (33) - : : : : : : : : : : : : : : +- CometBroadcastExchange (41) - : : : : : : : : : : : : : : +- CometProject (40) - : : : : : : : : : : : : : : +- CometFilter (39) - : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store (38) - : : : : : : : : : : : : : +- CometBroadcastExchange (46) - : : : : : : : : : : : : : +- CometFilter (45) - : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer (44) - : : : : : : : : : : : : +- CometBroadcastExchange (51) - : : : : : : : : : : : : +- CometFilter (50) - : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (49) - : : : : : : : : : : : +- ReusedExchange (54) - : : : : : : : : : : +- CometBroadcastExchange (60) - : : : : : : : : : : +- CometProject (59) - : : : : : : : : : : +- CometFilter (58) - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (57) - : : : : : : : : : +- ReusedExchange (63) - : : : : : : : : +- CometBroadcastExchange (68) - : : : : : : : : +- CometFilter (67) - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.promotion (66) - : : : : : : : +- CometBroadcastExchange (73) - : : : : : : : +- CometFilter (72) - : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (71) - : : : : : : +- ReusedExchange (76) - : : : : : +- CometBroadcastExchange (82) - : : : : : +- CometProject (81) - : : : : : +- CometFilter (80) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address (79) - : : : : +- ReusedExchange (85) - : : : +- CometBroadcastExchange (90) - : : : +- CometFilter (89) - : : : +- CometNativeScan parquet spark_catalog.default.income_band (88) - : : +- ReusedExchange (93) - : +- CometBroadcastExchange (99) - : +- CometProject (98) - : +- CometFilter (97) - : +- CometNativeScan parquet spark_catalog.default.item (96) - +- CometSort (175) - +- CometExchange (174) - +- CometHashAggregate (173) - +- CometHashAggregate (172) - +- CometProject (171) - +- CometBroadcastHashJoin (170) - :- CometProject (168) - : +- CometBroadcastHashJoin (167) - : :- CometProject (165) - : : +- CometBroadcastHashJoin (164) - : : :- CometProject (162) - : : : +- CometBroadcastHashJoin (161) - : : : :- CometProject (159) - : : : : +- CometBroadcastHashJoin (158) - : : : : :- CometProject (156) - : : : : : +- CometBroadcastHashJoin (155) - : : : : : :- CometProject (153) - : : : : : : +- CometBroadcastHashJoin (152) - : : : : : : :- CometProject (150) - : : : : : : : +- CometBroadcastHashJoin (149) - : : : : : : : :- CometProject (147) - : : : : : : : : +- CometBroadcastHashJoin (146) - : : : : : : : : :- CometProject (144) - : : : : : : : : : +- CometBroadcastHashJoin (143) - : : : : : : : : : :- CometProject (141) - : : : : : : : : : : +- CometBroadcastHashJoin (140) - : : : : : : : : : : :- CometProject (138) - : : : : : : : : : : : +- CometBroadcastHashJoin (137) - : : : : : : : : : : : :- CometProject (135) - : : : : : : : : : : : : +- CometBroadcastHashJoin (134) - : : : : : : : : : : : : :- CometProject (132) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (131) - : : : : : : : : : : : : : :- CometProject (129) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (128) - : : : : : : : : : : : : : : :- CometProject (124) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (123) - : : : : : : : : : : : : : : : :- CometSort (117) - : : : : : : : : : : : : : : : : +- CometColumnarExchange (116) - : : : : : : : : : : : : : : : : +- * Project (115) - : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (114) - : : : : : : : : : : : : : : : : :- BroadcastExchange (109) - : : : : : : : : : : : : : : : : : +- * Filter (108) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (107) - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (106) - : : : : : : : : : : : : : : : : +- * CometColumnarToRow (113) - : : : : : : : : : : : : : : : : +- CometProject (112) - : : : : : : : : : : : : : : : : +- CometFilter (111) - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (110) - : : : : : : : : : : : : : : : +- CometSort (122) - : : : : : : : : : : : : : : : +- CometProject (121) - : : : : : : : : : : : : : : : +- CometFilter (120) - : : : : : : : : : : : : : : : +- CometHashAggregate (119) - : : : : : : : : : : : : : : : +- ReusedExchange (118) - : : : : : : : : : : : : : : +- CometBroadcastExchange (127) - : : : : : : : : : : : : : : +- CometFilter (126) - : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (125) - : : : : : : : : : : : : : +- ReusedExchange (130) - : : : : : : : : : : : : +- ReusedExchange (133) - : : : : : : : : : : : +- ReusedExchange (136) - : : : : : : : : : : +- ReusedExchange (139) - : : : : : : : : : +- ReusedExchange (142) - : : : : : : : : +- ReusedExchange (145) - : : : : : : : +- ReusedExchange (148) - : : : : : : +- ReusedExchange (151) - : : : : : +- ReusedExchange (154) - : : : : +- ReusedExchange (157) - : : : +- ReusedExchange (160) - : : +- ReusedExchange (163) - : +- ReusedExchange (166) - +- ReusedExchange (169) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(3) Filter [codegen id : 1] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) - -(4) BroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=1] - -(5) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) - -(7) CometProject -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] - -(8) CometColumnarToRow -Input [2]: [sr_item_sk#14, sr_ticket_number#15] - -(9) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [ss_item_sk#1, ss_ticket_number#8] -Right keys [2]: [sr_item_sk#14, sr_ticket_number#15] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 2] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] - -(11) CometColumnarExchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(12) CometSort -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] - -(13) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) - -(15) CometProject -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] - -(16) CometExchange -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(17) CometSort -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] - -(18) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(19) CometFilter -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) - -(20) CometProject -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(21) CometExchange -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(22) CometSort -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] - -(23) CometSortMergeJoin -Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner - -(24) CometProject -Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(25) CometHashAggregate -Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Keys [1]: [cs_item_sk#17] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(26) CometExchange -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(27) CometHashAggregate -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Keys [1]: [cs_item_sk#17] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(28) CometFilter -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(29) CometProject -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Arguments: [cs_item_sk#17], [cs_item_sk#17] - -(30) CometSort -Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] - -(31) CometSortMergeJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [1]: [cs_item_sk#17] -Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner - -(32) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(33) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(35) CometBroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: [d_date_sk#32, d_year#33] - -(36) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight - -(37) CometProject -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] - -(38) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name)] -ReadSchema: struct - -(39) CometFilter -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#36, 10)))) - -(40) CometProject -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37], [s_store_sk#34, s_store_name#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#36, 10)) AS s_zip#37] - -(41) CometBroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37] - -(42) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight - -(43) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] - -(44) CometNativeScan parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(45) CometFilter -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Condition : (((((isnotnull(c_customer_sk#38) AND isnotnull(c_first_sales_date_sk#43)) AND isnotnull(c_first_shipto_date_sk#42)) AND isnotnull(c_current_cdemo_sk#39)) AND isnotnull(c_current_hdemo_sk#40)) AND isnotnull(c_current_addr_sk#41)) - -(46) CometBroadcastExchange -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(47) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -Right output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_customer_sk#2], [c_customer_sk#38], Inner, BuildRight - -(48) CometProject -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(49) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : isnotnull(d_date_sk#44) - -(51) CometBroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] - -(52) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [c_first_sales_date_sk#43], [d_date_sk#44], Inner, BuildRight - -(53) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43, d_date_sk#44, d_year#45] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] - -(54) ReusedExchange [Reuses operator id: 51] -Output [2]: [d_date_sk#46, d_year#47] - -(55) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [c_first_shipto_date_sk#42], [d_date_sk#46], Inner, BuildRight - -(56) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45, d_date_sk#46, d_year#47] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(57) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#48, cd_marital_status#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(58) CometFilter -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#49, 1)))) - -(59) CometProject -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Arguments: [cd_demo_sk#48, cd_marital_status#50], [cd_demo_sk#48, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#49, 1)) AS cd_marital_status#50] - -(60) CometBroadcastExchange -Input [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [cd_demo_sk#48, cd_marital_status#50] - -(61) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_cdemo_sk#3], [cd_demo_sk#48], Inner, BuildRight - -(62) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] - -(63) ReusedExchange [Reuses operator id: 60] -Output [2]: [cd_demo_sk#51, cd_marital_status#52] - -(64) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -Right output [2]: [cd_demo_sk#51, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#39], [cd_demo_sk#51], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(65) CometProject -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50, cd_demo_sk#51, cd_marital_status#52] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(66) CometNativeScan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(67) CometFilter -Input [1]: [p_promo_sk#53] -Condition : isnotnull(p_promo_sk#53) - -(68) CometBroadcastExchange -Input [1]: [p_promo_sk#53] -Arguments: [p_promo_sk#53] - -(69) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [1]: [p_promo_sk#53] -Arguments: [ss_promo_sk#7], [p_promo_sk#53], Inner, BuildRight - -(70) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, p_promo_sk#53] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(71) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(72) CometFilter -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Condition : (isnotnull(hd_demo_sk#54) AND isnotnull(hd_income_band_sk#55)) - -(73) CometBroadcastExchange -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [hd_demo_sk#54, hd_income_band_sk#55] - -(74) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_hdemo_sk#4], [hd_demo_sk#54], Inner, BuildRight - -(75) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] - -(76) ReusedExchange [Reuses operator id: 73] -Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] - -(77) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -Right output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [c_current_hdemo_sk#40], [hd_demo_sk#56], Inner, BuildRight - -(78) CometProject -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] - -(79) CometNativeScan parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(80) CometFilter -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Condition : isnotnull(ca_address_sk#58) - -(81) CometProject -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ca_address_sk#58, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_number#59, 10)) AS ca_street_number#63, ca_street_name#60, ca_city#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#62, 10)) AS ca_zip#64] - -(82) CometBroadcastExchange -Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(83) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -Right output [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_addr_sk#5], [ca_address_sk#58], Inner, BuildRight - -(84) CometProject -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(85) ReusedExchange [Reuses operator id: 82] -Output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(86) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Right output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [c_current_addr_sk#41], [ca_address_sk#65], Inner, BuildRight - -(87) CometProject -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(88) CometNativeScan parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(89) CometFilter -Input [1]: [ib_income_band_sk#70] -Condition : isnotnull(ib_income_band_sk#70) - -(90) CometBroadcastExchange -Input [1]: [ib_income_band_sk#70] -Arguments: [ib_income_band_sk#70] - -(91) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#70] -Arguments: [hd_income_band_sk#55], [ib_income_band_sk#70], Inner, BuildRight - -(92) CometProject -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#70] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(93) ReusedExchange [Reuses operator id: 90] -Output [1]: [ib_income_band_sk#71] - -(94) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#71] -Arguments: [hd_income_band_sk#57], [ib_income_band_sk#71], Inner, BuildRight - -(95) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#71] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(96) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(97) CometFilter -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Condition : ((((((isnotnull(i_current_price#73) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#74, 20)) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#73 >= 64.00)) AND (i_current_price#73 <= 74.00)) AND (i_current_price#73 >= 65.00)) AND (i_current_price#73 <= 79.00)) AND isnotnull(i_item_sk#72)) - -(98) CometProject -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Arguments: [i_item_sk#72, i_product_name#76], [i_item_sk#72, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#75, 50)) AS i_product_name#76] - -(99) CometBroadcastExchange -Input [2]: [i_item_sk#72, i_product_name#76] -Arguments: [i_item_sk#72, i_product_name#76] - -(100) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [2]: [i_item_sk#72, i_product_name#76] -Arguments: [ss_item_sk#1], [i_item_sk#72], Inner, BuildRight - -(101) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] - -(102) CometHashAggregate -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] - -(103) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#77, sum#78, sum#79, sum#80] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] - -(104) CometExchange -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: hashpartitioning(item_sk#82, store_name#83, store_zip#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(105) CometSort -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97], [item_sk#82 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, store_zip#84 ASC NULLS FIRST] - -(106) Scan parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#109), dynamicpruningexpression(ss_sold_date_sk#109 IN dynamicpruning#110)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(107) ColumnarToRow [codegen id : 3] -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(108) Filter [codegen id : 3] -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Condition : (((((((isnotnull(ss_item_sk#98) AND isnotnull(ss_ticket_number#105)) AND isnotnull(ss_store_sk#103)) AND isnotnull(ss_customer_sk#99)) AND isnotnull(ss_cdemo_sk#100)) AND isnotnull(ss_promo_sk#104)) AND isnotnull(ss_hdemo_sk#101)) AND isnotnull(ss_addr_sk#102)) - -(109) BroadcastExchange -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=7] - -(110) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(111) CometFilter -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) - -(112) CometProject -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] - -(113) CometColumnarToRow -Input [2]: [sr_item_sk#111, sr_ticket_number#112] - -(114) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [ss_item_sk#98, ss_ticket_number#105] -Right keys [2]: [sr_item_sk#111, sr_ticket_number#112] -Join type: Inner -Join condition: None - -(115) Project [codegen id : 4] -Output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, sr_item_sk#111, sr_ticket_number#112] - -(116) CometColumnarExchange -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: hashpartitioning(ss_item_sk#98, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(117) CometSort -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98 ASC NULLS FIRST] - -(118) ReusedExchange [Reuses operator id: 26] -Output [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] - -(119) CometHashAggregate -Input [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] -Keys [1]: [cs_item_sk#114] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#118)), sum(((cr_refunded_cash#119 + cr_reversed_charge#120) + cr_store_credit#121))] - -(120) CometFilter -Input [3]: [cs_item_sk#114, sale#122, refund#123] -Condition : ((isnotnull(sale#122) AND isnotnull(refund#123)) AND (cast(sale#122 as decimal(21,2)) > (2 * refund#123))) - -(121) CometProject -Input [3]: [cs_item_sk#114, sale#122, refund#123] -Arguments: [cs_item_sk#114], [cs_item_sk#114] - -(122) CometSort -Input [1]: [cs_item_sk#114] -Arguments: [cs_item_sk#114], [cs_item_sk#114 ASC NULLS FIRST] - -(123) CometSortMergeJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [1]: [cs_item_sk#114] -Arguments: [ss_item_sk#98], [cs_item_sk#114], Inner - -(124) CometProject -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, cs_item_sk#114] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(125) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#124, d_year#125] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#124, d_year#125] -Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) - -(127) CometBroadcastExchange -Input [2]: [d_date_sk#124, d_year#125] -Arguments: [d_date_sk#124, d_year#125] - -(128) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [d_date_sk#124, d_year#125] -Arguments: [ss_sold_date_sk#109], [d_date_sk#124], Inner, BuildRight - -(129) CometProject -Input [13]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, d_date_sk#124, d_year#125] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125] - -(130) ReusedExchange [Reuses operator id: 41] -Output [3]: [s_store_sk#126, s_store_name#127, s_zip#128] - -(131) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125] -Right output [3]: [s_store_sk#126, s_store_name#127, s_zip#128] -Arguments: [ss_store_sk#103], [s_store_sk#126], Inner, BuildRight - -(132) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_sk#126, s_store_name#127, s_zip#128] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128] - -(133) ReusedExchange [Reuses operator id: 46] -Output [6]: [c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] - -(134) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128] -Right output [6]: [c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] -Arguments: [ss_customer_sk#99], [c_customer_sk#129], Inner, BuildRight - -(135) CometProject -Input [18]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] - -(136) ReusedExchange [Reuses operator id: 51] -Output [2]: [d_date_sk#135, d_year#136] - -(137) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] -Right output [2]: [d_date_sk#135, d_year#136] -Arguments: [c_first_sales_date_sk#134], [d_date_sk#135], Inner, BuildRight - -(138) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134, d_date_sk#135, d_year#136] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136] - -(139) ReusedExchange [Reuses operator id: 51] -Output [2]: [d_date_sk#137, d_year#138] - -(140) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136] -Right output [2]: [d_date_sk#137, d_year#138] -Arguments: [c_first_shipto_date_sk#133], [d_date_sk#137], Inner, BuildRight - -(141) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136, d_date_sk#137, d_year#138] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] - -(142) ReusedExchange [Reuses operator id: 60] -Output [2]: [cd_demo_sk#139, cd_marital_status#140] - -(143) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] -Right output [2]: [cd_demo_sk#139, cd_marital_status#140] -Arguments: [ss_cdemo_sk#100], [cd_demo_sk#139], Inner, BuildRight - -(144) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_demo_sk#139, cd_marital_status#140] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140] - -(145) ReusedExchange [Reuses operator id: 60] -Output [2]: [cd_demo_sk#141, cd_marital_status#142] - -(146) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140] -Right output [2]: [cd_demo_sk#141, cd_marital_status#142] -Arguments: [c_current_cdemo_sk#130], [cd_demo_sk#141], Inner, NOT (cd_marital_status#140 = cd_marital_status#142), BuildRight - -(147) CometProject -Input [18]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140, cd_demo_sk#141, cd_marital_status#142] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] - -(148) ReusedExchange [Reuses operator id: 68] -Output [1]: [p_promo_sk#143] - -(149) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] -Right output [1]: [p_promo_sk#143] -Arguments: [ss_promo_sk#104], [p_promo_sk#143], Inner, BuildRight - -(150) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, p_promo_sk#143] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] - -(151) ReusedExchange [Reuses operator id: 73] -Output [2]: [hd_demo_sk#144, hd_income_band_sk#145] - -(152) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] -Right output [2]: [hd_demo_sk#144, hd_income_band_sk#145] -Arguments: [ss_hdemo_sk#101], [hd_demo_sk#144], Inner, BuildRight - -(153) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_demo_sk#144, hd_income_band_sk#145] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145] - -(154) ReusedExchange [Reuses operator id: 73] -Output [2]: [hd_demo_sk#146, hd_income_band_sk#147] - -(155) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145] -Right output [2]: [hd_demo_sk#146, hd_income_band_sk#147] -Arguments: [c_current_hdemo_sk#131], [hd_demo_sk#146], Inner, BuildRight - -(156) CometProject -Input [15]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_demo_sk#146, hd_income_band_sk#147] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147] - -(157) ReusedExchange [Reuses operator id: 82] -Output [5]: [ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] - -(158) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147] -Right output [5]: [ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] -Arguments: [ss_addr_sk#102], [ca_address_sk#148], Inner, BuildRight - -(159) CometProject -Input [18]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] - -(160) ReusedExchange [Reuses operator id: 82] -Output [5]: [ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(161) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] -Right output [5]: [ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Arguments: [c_current_addr_sk#132], [ca_address_sk#153], Inner, BuildRight - -(162) CometProject -Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(163) ReusedExchange [Reuses operator id: 90] -Output [1]: [ib_income_band_sk#158] - -(164) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Right output [1]: [ib_income_band_sk#158] -Arguments: [hd_income_band_sk#145], [ib_income_band_sk#158], Inner, BuildRight - -(165) CometProject -Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, ib_income_band_sk#158] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(166) ReusedExchange [Reuses operator id: 90] -Output [1]: [ib_income_band_sk#159] - -(167) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Right output [1]: [ib_income_band_sk#159] -Arguments: [hd_income_band_sk#147], [ib_income_band_sk#159], Inner, BuildRight - -(168) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, ib_income_band_sk#159] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(169) ReusedExchange [Reuses operator id: 99] -Output [2]: [i_item_sk#160, i_product_name#161] - -(170) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Right output [2]: [i_item_sk#160, i_product_name#161] -Arguments: [ss_item_sk#98], [i_item_sk#160], Inner, BuildRight - -(171) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] -Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] - -(172) CometHashAggregate -Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] -Keys [15]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#106)), partial_sum(UnscaledValue(ss_list_price#107)), partial_sum(UnscaledValue(ss_coupon_amt#108))] - -(173) CometHashAggregate -Input [19]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138, count#77, sum#162, sum#163, sum#164] -Keys [15]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#106)), sum(UnscaledValue(ss_list_price#107)), sum(UnscaledValue(ss_coupon_amt#108))] - -(174) CometExchange -Input [8]: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: hashpartitioning(item_sk#165, store_name#166, store_zip#167, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(175) CometSort -Input [8]: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172], [item_sk#165 ASC NULLS FIRST, store_name#166 ASC NULLS FIRST, store_zip#167 ASC NULLS FIRST] - -(176) CometSortMergeJoin -Left output [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Right output [8]: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#165, store_name#166, store_zip#167], Inner, (cnt#169 <= cnt#94) - -(177) CometProject -Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] - -(178) CometExchange -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#169 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#170 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(179) CometSort -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#169 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#170 ASC NULLS FIRST] - -(180) CometColumnarToRow [codegen id : 5] -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (184) -+- * CometColumnarToRow (183) - +- CometFilter (182) - +- CometNativeScan parquet spark_catalog.default.date_dim (181) - - -(181) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(182) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(183) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#33] - -(184) BroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] - -Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 -BroadcastExchange (188) -+- * CometColumnarToRow (187) - +- CometFilter (186) - +- CometNativeScan parquet spark_catalog.default.date_dim (185) - - -(185) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#124, d_year#125] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(186) CometFilter -Input [2]: [d_date_sk#124, d_year#125] -Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) - -(187) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#124, d_year#125] - -(188) BroadcastExchange -Input [2]: [d_date_sk#124, d_year#125] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/simplified.txt deleted file mode 100644 index d933995ffc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/simplified.txt +++ /dev/null @@ -1,206 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometExchange [product_name,store_name,cnt,s1,s1] #1 - CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometColumnarExchange [ss_item_sk] #3 - WholeStageCodegen (2) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometNativeScan 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] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [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] #11 - 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] - CometNativeScan 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] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk] - ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometColumnarExchange [ss_item_sk] #20 - WholeStageCodegen (4) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #21 - WholeStageCodegen (3) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet 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 #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - CometBroadcastExchange [d_date_sk,d_year] #23 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/explain.txt deleted file mode 100644 index 8764db8779..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,1011 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (176) -+- CometSort (175) - +- CometExchange (174) - +- CometProject (173) - +- CometSortMergeJoin (172) - :- CometSort (103) - : +- CometExchange (102) - : +- CometHashAggregate (101) - : +- CometHashAggregate (100) - : +- CometProject (99) - : +- CometBroadcastHashJoin (98) - : :- CometProject (93) - : : +- CometBroadcastHashJoin (92) - : : :- CometProject (90) - : : : +- CometBroadcastHashJoin (89) - : : : :- CometProject (85) - : : : : +- CometBroadcastHashJoin (84) - : : : : :- CometProject (82) - : : : : : +- CometBroadcastHashJoin (81) - : : : : : :- CometProject (76) - : : : : : : +- CometBroadcastHashJoin (75) - : : : : : : :- CometProject (73) - : : : : : : : +- CometBroadcastHashJoin (72) - : : : : : : : :- CometProject (68) - : : : : : : : : +- CometBroadcastHashJoin (67) - : : : : : : : : :- CometProject (63) - : : : : : : : : : +- CometBroadcastHashJoin (62) - : : : : : : : : : :- CometProject (60) - : : : : : : : : : : +- CometBroadcastHashJoin (59) - : : : : : : : : : : :- CometProject (54) - : : : : : : : : : : : +- CometBroadcastHashJoin (53) - : : : : : : : : : : : :- CometProject (51) - : : : : : : : : : : : : +- CometBroadcastHashJoin (50) - : : : : : : : : : : : : :- CometProject (46) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (45) - : : : : : : : : : : : : : :- CometProject (41) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (40) - : : : : : : : : : : : : : : :- CometProject (35) - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (34) - : : : : : : : : : : : : : : : :- CometProject (30) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) - : : : : : : : : : : : : : : : : :- CometSort (10) - : : : : : : : : : : : : : : : : : +- CometExchange (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- CometSort (28) - : : : : : : : : : : : : : : : : +- CometProject (27) - : : : : : : : : : : : : : : : : +- CometFilter (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometExchange (24) - : : : : : : : : : : : : : : : : +- CometHashAggregate (23) - : : : : : : : : : : : : : : : : +- CometProject (22) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) - : : : : : : : : : : : : : : : : :- CometSort (15) - : : : : : : : : : : : : : : : : : +- CometExchange (14) - : : : : : : : : : : : : : : : : : +- CometProject (13) - : : : : : : : : : : : : : : : : : +- CometFilter (12) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (11) - : : : : : : : : : : : : : : : : +- CometSort (20) - : : : : : : : : : : : : : : : : +- CometExchange (19) - : : : : : : : : : : : : : : : : +- CometProject (18) - : : : : : : : : : : : : : : : : +- CometFilter (17) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : : : : : : : : : : : : : : : +- CometBroadcastExchange (33) - : : : : : : : : : : : : : : : +- CometFilter (32) - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - : : : : : : : : : : : : : : +- CometBroadcastExchange (39) - : : : : : : : : : : : : : : +- CometProject (38) - : : : : : : : : : : : : : : +- CometFilter (37) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (36) - : : : : : : : : : : : : : +- CometBroadcastExchange (44) - : : : : : : : : : : : : : +- CometFilter (43) - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (42) - : : : : : : : : : : : : +- CometBroadcastExchange (49) - : : : : : : : : : : : : +- CometFilter (48) - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) - : : : : : : : : : : : +- ReusedExchange (52) - : : : : : : : : : : +- CometBroadcastExchange (58) - : : : : : : : : : : +- CometProject (57) - : : : : : : : : : : +- CometFilter (56) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (55) - : : : : : : : : : +- ReusedExchange (61) - : : : : : : : : +- CometBroadcastExchange (66) - : : : : : : : : +- CometFilter (65) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (64) - : : : : : : : +- CometBroadcastExchange (71) - : : : : : : : +- CometFilter (70) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (69) - : : : : : : +- ReusedExchange (74) - : : : : : +- CometBroadcastExchange (80) - : : : : : +- CometProject (79) - : : : : : +- CometFilter (78) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (77) - : : : : +- ReusedExchange (83) - : : : +- CometBroadcastExchange (88) - : : : +- CometFilter (87) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (86) - : : +- ReusedExchange (91) - : +- CometBroadcastExchange (97) - : +- CometProject (96) - : +- CometFilter (95) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (94) - +- CometSort (171) - +- CometExchange (170) - +- CometHashAggregate (169) - +- CometHashAggregate (168) - +- CometProject (167) - +- CometBroadcastHashJoin (166) - :- CometProject (164) - : +- CometBroadcastHashJoin (163) - : :- CometProject (161) - : : +- CometBroadcastHashJoin (160) - : : :- CometProject (158) - : : : +- CometBroadcastHashJoin (157) - : : : :- CometProject (155) - : : : : +- CometBroadcastHashJoin (154) - : : : : :- CometProject (152) - : : : : : +- CometBroadcastHashJoin (151) - : : : : : :- CometProject (149) - : : : : : : +- CometBroadcastHashJoin (148) - : : : : : : :- CometProject (146) - : : : : : : : +- CometBroadcastHashJoin (145) - : : : : : : : :- CometProject (143) - : : : : : : : : +- CometBroadcastHashJoin (142) - : : : : : : : : :- CometProject (140) - : : : : : : : : : +- CometBroadcastHashJoin (139) - : : : : : : : : : :- CometProject (137) - : : : : : : : : : : +- CometBroadcastHashJoin (136) - : : : : : : : : : : :- CometProject (134) - : : : : : : : : : : : +- CometBroadcastHashJoin (133) - : : : : : : : : : : : :- CometProject (131) - : : : : : : : : : : : : +- CometBroadcastHashJoin (130) - : : : : : : : : : : : : :- CometProject (128) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (127) - : : : : : : : : : : : : : :- CometProject (125) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (124) - : : : : : : : : : : : : : : :- CometProject (120) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (119) - : : : : : : : : : : : : : : : :- CometSort (113) - : : : : : : : : : : : : : : : : +- CometExchange (112) - : : : : : : : : : : : : : : : : +- CometProject (111) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (110) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (106) - : : : : : : : : : : : : : : : : : +- CometFilter (105) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (104) - : : : : : : : : : : : : : : : : +- CometProject (109) - : : : : : : : : : : : : : : : : +- CometFilter (108) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (107) - : : : : : : : : : : : : : : : +- CometSort (118) - : : : : : : : : : : : : : : : +- CometProject (117) - : : : : : : : : : : : : : : : +- CometFilter (116) - : : : : : : : : : : : : : : : +- CometHashAggregate (115) - : : : : : : : : : : : : : : : +- ReusedExchange (114) - : : : : : : : : : : : : : : +- CometBroadcastExchange (123) - : : : : : : : : : : : : : : +- CometFilter (122) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (121) - : : : : : : : : : : : : : +- ReusedExchange (126) - : : : : : : : : : : : : +- ReusedExchange (129) - : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : +- ReusedExchange (141) - : : : : : : : +- ReusedExchange (144) - : : : : : : +- ReusedExchange (147) - : : : : : +- ReusedExchange (150) - : : : : +- ReusedExchange (153) - : : : +- ReusedExchange (156) - : : +- ReusedExchange (159) - : +- ReusedExchange (162) - +- ReusedExchange (165) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) CometFilter -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) - -(3) CometBroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) - -(6) CometProject -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] - -(7) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner, BuildLeft - -(8) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(9) CometExchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometSort -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) - -(13) CometProject -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] - -(14) CometExchange -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(15) CometSort -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(17) CometFilter -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) - -(18) CometProject -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(19) CometExchange -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner - -(22) CometProject -Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(23) CometHashAggregate -Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Keys [1]: [cs_item_sk#17] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(24) CometExchange -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometHashAggregate -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Keys [1]: [cs_item_sk#17] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(26) CometFilter -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(27) CometProject -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Arguments: [cs_item_sk#17], [cs_item_sk#17] - -(28) CometSort -Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] - -(29) CometSortMergeJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [1]: [cs_item_sk#17] -Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner - -(30) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(33) CometBroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: [d_date_sk#32, d_year#33] - -(34) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight - -(35) CometProject -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#36, 10)))) - -(38) CometProject -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37], [s_store_sk#34, s_store_name#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#36, 10)) AS s_zip#37] - -(39) CometBroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37] - -(40) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight - -(41) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(43) CometFilter -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Condition : (((((isnotnull(c_customer_sk#38) AND isnotnull(c_first_sales_date_sk#43)) AND isnotnull(c_first_shipto_date_sk#42)) AND isnotnull(c_current_cdemo_sk#39)) AND isnotnull(c_current_hdemo_sk#40)) AND isnotnull(c_current_addr_sk#41)) - -(44) CometBroadcastExchange -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(45) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -Right output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_customer_sk#2], [c_customer_sk#38], Inner, BuildRight - -(46) CometProject -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(48) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : isnotnull(d_date_sk#44) - -(49) CometBroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] - -(50) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [c_first_sales_date_sk#43], [d_date_sk#44], Inner, BuildRight - -(51) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43, d_date_sk#44, d_year#45] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] - -(52) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#46, d_year#47] - -(53) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [c_first_shipto_date_sk#42], [d_date_sk#46], Inner, BuildRight - -(54) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45, d_date_sk#46, d_year#47] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#48, cd_marital_status#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#49, 1)))) - -(57) CometProject -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Arguments: [cd_demo_sk#48, cd_marital_status#50], [cd_demo_sk#48, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#49, 1)) AS cd_marital_status#50] - -(58) CometBroadcastExchange -Input [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [cd_demo_sk#48, cd_marital_status#50] - -(59) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_cdemo_sk#3], [cd_demo_sk#48], Inner, BuildRight - -(60) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] - -(61) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#51, cd_marital_status#52] - -(62) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -Right output [2]: [cd_demo_sk#51, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#39], [cd_demo_sk#51], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(63) CometProject -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50, cd_demo_sk#51, cd_marital_status#52] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(65) CometFilter -Input [1]: [p_promo_sk#53] -Condition : isnotnull(p_promo_sk#53) - -(66) CometBroadcastExchange -Input [1]: [p_promo_sk#53] -Arguments: [p_promo_sk#53] - -(67) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [1]: [p_promo_sk#53] -Arguments: [ss_promo_sk#7], [p_promo_sk#53], Inner, BuildRight - -(68) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, p_promo_sk#53] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(70) CometFilter -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Condition : (isnotnull(hd_demo_sk#54) AND isnotnull(hd_income_band_sk#55)) - -(71) CometBroadcastExchange -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [hd_demo_sk#54, hd_income_band_sk#55] - -(72) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_hdemo_sk#4], [hd_demo_sk#54], Inner, BuildRight - -(73) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] - -(74) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] - -(75) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -Right output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [c_current_hdemo_sk#40], [hd_demo_sk#56], Inner, BuildRight - -(76) CometProject -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(78) CometFilter -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Condition : isnotnull(ca_address_sk#58) - -(79) CometProject -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ca_address_sk#58, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_number#59, 10)) AS ca_street_number#63, ca_street_name#60, ca_city#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#62, 10)) AS ca_zip#64] - -(80) CometBroadcastExchange -Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(81) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -Right output [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_addr_sk#5], [ca_address_sk#58], Inner, BuildRight - -(82) CometProject -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(83) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(84) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Right output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [c_current_addr_sk#41], [ca_address_sk#65], Inner, BuildRight - -(85) CometProject -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(87) CometFilter -Input [1]: [ib_income_band_sk#70] -Condition : isnotnull(ib_income_band_sk#70) - -(88) CometBroadcastExchange -Input [1]: [ib_income_band_sk#70] -Arguments: [ib_income_band_sk#70] - -(89) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#70] -Arguments: [hd_income_band_sk#55], [ib_income_band_sk#70], Inner, BuildRight - -(90) CometProject -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#70] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(91) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#71] - -(92) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#71] -Arguments: [hd_income_band_sk#57], [ib_income_band_sk#71], Inner, BuildRight - -(93) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#71] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(95) CometFilter -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Condition : ((((((isnotnull(i_current_price#73) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#74, 20)) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#73 >= 64.00)) AND (i_current_price#73 <= 74.00)) AND (i_current_price#73 >= 65.00)) AND (i_current_price#73 <= 79.00)) AND isnotnull(i_item_sk#72)) - -(96) CometProject -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Arguments: [i_item_sk#72, i_product_name#76], [i_item_sk#72, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#75, 50)) AS i_product_name#76] - -(97) CometBroadcastExchange -Input [2]: [i_item_sk#72, i_product_name#76] -Arguments: [i_item_sk#72, i_product_name#76] - -(98) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [2]: [i_item_sk#72, i_product_name#76] -Arguments: [ss_item_sk#1], [i_item_sk#72], Inner, BuildRight - -(99) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] - -(100) CometHashAggregate -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] - -(101) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#77, sum#78, sum#79, sum#80] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] - -(102) CometExchange -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: hashpartitioning(item_sk#82, store_name#83, store_zip#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(103) CometSort -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97], [item_sk#82 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, store_zip#84 ASC NULLS FIRST] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#109), dynamicpruningexpression(ss_sold_date_sk#109 IN dynamicpruning#110)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(105) CometFilter -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Condition : (((((((isnotnull(ss_item_sk#98) AND isnotnull(ss_ticket_number#105)) AND isnotnull(ss_store_sk#103)) AND isnotnull(ss_customer_sk#99)) AND isnotnull(ss_cdemo_sk#100)) AND isnotnull(ss_promo_sk#104)) AND isnotnull(ss_hdemo_sk#101)) AND isnotnull(ss_addr_sk#102)) - -(106) CometBroadcastExchange -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(108) CometFilter -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) - -(109) CometProject -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] - -(110) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_ticket_number#105], [sr_item_sk#111, sr_ticket_number#112], Inner, BuildLeft - -(111) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(112) CometExchange -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: hashpartitioning(ss_item_sk#98, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(113) CometSort -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98 ASC NULLS FIRST] - -(114) ReusedExchange [Reuses operator id: 24] -Output [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] - -(115) CometHashAggregate -Input [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] -Keys [1]: [cs_item_sk#114] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#118)), sum(((cr_refunded_cash#119 + cr_reversed_charge#120) + cr_store_credit#121))] - -(116) CometFilter -Input [3]: [cs_item_sk#114, sale#122, refund#123] -Condition : ((isnotnull(sale#122) AND isnotnull(refund#123)) AND (cast(sale#122 as decimal(21,2)) > (2 * refund#123))) - -(117) CometProject -Input [3]: [cs_item_sk#114, sale#122, refund#123] -Arguments: [cs_item_sk#114], [cs_item_sk#114] - -(118) CometSort -Input [1]: [cs_item_sk#114] -Arguments: [cs_item_sk#114], [cs_item_sk#114 ASC NULLS FIRST] - -(119) CometSortMergeJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [1]: [cs_item_sk#114] -Arguments: [ss_item_sk#98], [cs_item_sk#114], Inner - -(120) CometProject -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, cs_item_sk#114] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(121) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#124, d_year#125] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(122) CometFilter -Input [2]: [d_date_sk#124, d_year#125] -Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) - -(123) CometBroadcastExchange -Input [2]: [d_date_sk#124, d_year#125] -Arguments: [d_date_sk#124, d_year#125] - -(124) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [d_date_sk#124, d_year#125] -Arguments: [ss_sold_date_sk#109], [d_date_sk#124], Inner, BuildRight - -(125) CometProject -Input [13]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, d_date_sk#124, d_year#125] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125] - -(126) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_store_sk#126, s_store_name#127, s_zip#128] - -(127) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125] -Right output [3]: [s_store_sk#126, s_store_name#127, s_zip#128] -Arguments: [ss_store_sk#103], [s_store_sk#126], Inner, BuildRight - -(128) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_sk#126, s_store_name#127, s_zip#128] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128] - -(129) ReusedExchange [Reuses operator id: 44] -Output [6]: [c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] - -(130) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128] -Right output [6]: [c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] -Arguments: [ss_customer_sk#99], [c_customer_sk#129], Inner, BuildRight - -(131) CometProject -Input [18]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] - -(132) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#135, d_year#136] - -(133) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] -Right output [2]: [d_date_sk#135, d_year#136] -Arguments: [c_first_sales_date_sk#134], [d_date_sk#135], Inner, BuildRight - -(134) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134, d_date_sk#135, d_year#136] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136] - -(135) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#137, d_year#138] - -(136) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136] -Right output [2]: [d_date_sk#137, d_year#138] -Arguments: [c_first_shipto_date_sk#133], [d_date_sk#137], Inner, BuildRight - -(137) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136, d_date_sk#137, d_year#138] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] - -(138) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#139, cd_marital_status#140] - -(139) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] -Right output [2]: [cd_demo_sk#139, cd_marital_status#140] -Arguments: [ss_cdemo_sk#100], [cd_demo_sk#139], Inner, BuildRight - -(140) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_demo_sk#139, cd_marital_status#140] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140] - -(141) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#141, cd_marital_status#142] - -(142) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140] -Right output [2]: [cd_demo_sk#141, cd_marital_status#142] -Arguments: [c_current_cdemo_sk#130], [cd_demo_sk#141], Inner, NOT (cd_marital_status#140 = cd_marital_status#142), BuildRight - -(143) CometProject -Input [18]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140, cd_demo_sk#141, cd_marital_status#142] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] - -(144) ReusedExchange [Reuses operator id: 66] -Output [1]: [p_promo_sk#143] - -(145) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] -Right output [1]: [p_promo_sk#143] -Arguments: [ss_promo_sk#104], [p_promo_sk#143], Inner, BuildRight - -(146) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, p_promo_sk#143] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] - -(147) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#144, hd_income_band_sk#145] - -(148) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] -Right output [2]: [hd_demo_sk#144, hd_income_band_sk#145] -Arguments: [ss_hdemo_sk#101], [hd_demo_sk#144], Inner, BuildRight - -(149) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_demo_sk#144, hd_income_band_sk#145] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145] - -(150) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#146, hd_income_band_sk#147] - -(151) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145] -Right output [2]: [hd_demo_sk#146, hd_income_band_sk#147] -Arguments: [c_current_hdemo_sk#131], [hd_demo_sk#146], Inner, BuildRight - -(152) CometProject -Input [15]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_demo_sk#146, hd_income_band_sk#147] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147] - -(153) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] - -(154) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147] -Right output [5]: [ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] -Arguments: [ss_addr_sk#102], [ca_address_sk#148], Inner, BuildRight - -(155) CometProject -Input [18]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] - -(156) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(157) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] -Right output [5]: [ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Arguments: [c_current_addr_sk#132], [ca_address_sk#153], Inner, BuildRight - -(158) CometProject -Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(159) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#158] - -(160) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Right output [1]: [ib_income_band_sk#158] -Arguments: [hd_income_band_sk#145], [ib_income_band_sk#158], Inner, BuildRight - -(161) CometProject -Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, ib_income_band_sk#158] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(162) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#159] - -(163) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Right output [1]: [ib_income_band_sk#159] -Arguments: [hd_income_band_sk#147], [ib_income_band_sk#159], Inner, BuildRight - -(164) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, ib_income_band_sk#159] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(165) ReusedExchange [Reuses operator id: 97] -Output [2]: [i_item_sk#160, i_product_name#161] - -(166) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Right output [2]: [i_item_sk#160, i_product_name#161] -Arguments: [ss_item_sk#98], [i_item_sk#160], Inner, BuildRight - -(167) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] -Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] - -(168) CometHashAggregate -Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] -Keys [15]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#106)), partial_sum(UnscaledValue(ss_list_price#107)), partial_sum(UnscaledValue(ss_coupon_amt#108))] - -(169) CometHashAggregate -Input [19]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138, count#77, sum#162, sum#163, sum#164] -Keys [15]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#106)), sum(UnscaledValue(ss_list_price#107)), sum(UnscaledValue(ss_coupon_amt#108))] - -(170) CometExchange -Input [8]: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: hashpartitioning(item_sk#165, store_name#166, store_zip#167, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(171) CometSort -Input [8]: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172], [item_sk#165 ASC NULLS FIRST, store_name#166 ASC NULLS FIRST, store_zip#167 ASC NULLS FIRST] - -(172) CometSortMergeJoin -Left output [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Right output [8]: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#165, store_name#166, store_zip#167], Inner, (cnt#169 <= cnt#94) - -(173) CometProject -Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] - -(174) CometExchange -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#169 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#170 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(175) CometSort -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#169 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#170 ASC NULLS FIRST] - -(176) CometColumnarToRow [codegen id : 1] -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (180) -+- * CometColumnarToRow (179) - +- CometFilter (178) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) - - -(177) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(178) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(179) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#33] - -(180) BroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 -BroadcastExchange (184) -+- * CometColumnarToRow (183) - +- CometFilter (182) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) - - -(181) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#124, d_year#125] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(182) CometFilter -Input [2]: [d_date_sk#124, d_year#125] -Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) - -(183) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#124, d_year#125] - -(184) BroadcastExchange -Input [2]: [d_date_sk#124, d_year#125] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/simplified.txt deleted file mode 100644 index 46fe063430..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,192 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometExchange [product_name,store_name,cnt,s1,s1] #1 - CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #3 - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [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] #11 - 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 [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk] - ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #20 - 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_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] #21 - 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 [native_iceberg_compat] 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 #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - CometBroadcastExchange [d_date_sk,d_year] #23 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt deleted file mode 100644 index 8764db8779..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt +++ /dev/null @@ -1,1011 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (176) -+- CometSort (175) - +- CometExchange (174) - +- CometProject (173) - +- CometSortMergeJoin (172) - :- CometSort (103) - : +- CometExchange (102) - : +- CometHashAggregate (101) - : +- CometHashAggregate (100) - : +- CometProject (99) - : +- CometBroadcastHashJoin (98) - : :- CometProject (93) - : : +- CometBroadcastHashJoin (92) - : : :- CometProject (90) - : : : +- CometBroadcastHashJoin (89) - : : : :- CometProject (85) - : : : : +- CometBroadcastHashJoin (84) - : : : : :- CometProject (82) - : : : : : +- CometBroadcastHashJoin (81) - : : : : : :- CometProject (76) - : : : : : : +- CometBroadcastHashJoin (75) - : : : : : : :- CometProject (73) - : : : : : : : +- CometBroadcastHashJoin (72) - : : : : : : : :- CometProject (68) - : : : : : : : : +- CometBroadcastHashJoin (67) - : : : : : : : : :- CometProject (63) - : : : : : : : : : +- CometBroadcastHashJoin (62) - : : : : : : : : : :- CometProject (60) - : : : : : : : : : : +- CometBroadcastHashJoin (59) - : : : : : : : : : : :- CometProject (54) - : : : : : : : : : : : +- CometBroadcastHashJoin (53) - : : : : : : : : : : : :- CometProject (51) - : : : : : : : : : : : : +- CometBroadcastHashJoin (50) - : : : : : : : : : : : : :- CometProject (46) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (45) - : : : : : : : : : : : : : :- CometProject (41) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (40) - : : : : : : : : : : : : : : :- CometProject (35) - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (34) - : : : : : : : : : : : : : : : :- CometProject (30) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) - : : : : : : : : : : : : : : : : :- CometSort (10) - : : : : : : : : : : : : : : : : : +- CometExchange (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- CometSort (28) - : : : : : : : : : : : : : : : : +- CometProject (27) - : : : : : : : : : : : : : : : : +- CometFilter (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometExchange (24) - : : : : : : : : : : : : : : : : +- CometHashAggregate (23) - : : : : : : : : : : : : : : : : +- CometProject (22) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) - : : : : : : : : : : : : : : : : :- CometSort (15) - : : : : : : : : : : : : : : : : : +- CometExchange (14) - : : : : : : : : : : : : : : : : : +- CometProject (13) - : : : : : : : : : : : : : : : : : +- CometFilter (12) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (11) - : : : : : : : : : : : : : : : : +- CometSort (20) - : : : : : : : : : : : : : : : : +- CometExchange (19) - : : : : : : : : : : : : : : : : +- CometProject (18) - : : : : : : : : : : : : : : : : +- CometFilter (17) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : : : : : : : : : : : : : : : +- CometBroadcastExchange (33) - : : : : : : : : : : : : : : : +- CometFilter (32) - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - : : : : : : : : : : : : : : +- CometBroadcastExchange (39) - : : : : : : : : : : : : : : +- CometProject (38) - : : : : : : : : : : : : : : +- CometFilter (37) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (36) - : : : : : : : : : : : : : +- CometBroadcastExchange (44) - : : : : : : : : : : : : : +- CometFilter (43) - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (42) - : : : : : : : : : : : : +- CometBroadcastExchange (49) - : : : : : : : : : : : : +- CometFilter (48) - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) - : : : : : : : : : : : +- ReusedExchange (52) - : : : : : : : : : : +- CometBroadcastExchange (58) - : : : : : : : : : : +- CometProject (57) - : : : : : : : : : : +- CometFilter (56) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (55) - : : : : : : : : : +- ReusedExchange (61) - : : : : : : : : +- CometBroadcastExchange (66) - : : : : : : : : +- CometFilter (65) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (64) - : : : : : : : +- CometBroadcastExchange (71) - : : : : : : : +- CometFilter (70) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (69) - : : : : : : +- ReusedExchange (74) - : : : : : +- CometBroadcastExchange (80) - : : : : : +- CometProject (79) - : : : : : +- CometFilter (78) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (77) - : : : : +- ReusedExchange (83) - : : : +- CometBroadcastExchange (88) - : : : +- CometFilter (87) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (86) - : : +- ReusedExchange (91) - : +- CometBroadcastExchange (97) - : +- CometProject (96) - : +- CometFilter (95) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (94) - +- CometSort (171) - +- CometExchange (170) - +- CometHashAggregate (169) - +- CometHashAggregate (168) - +- CometProject (167) - +- CometBroadcastHashJoin (166) - :- CometProject (164) - : +- CometBroadcastHashJoin (163) - : :- CometProject (161) - : : +- CometBroadcastHashJoin (160) - : : :- CometProject (158) - : : : +- CometBroadcastHashJoin (157) - : : : :- CometProject (155) - : : : : +- CometBroadcastHashJoin (154) - : : : : :- CometProject (152) - : : : : : +- CometBroadcastHashJoin (151) - : : : : : :- CometProject (149) - : : : : : : +- CometBroadcastHashJoin (148) - : : : : : : :- CometProject (146) - : : : : : : : +- CometBroadcastHashJoin (145) - : : : : : : : :- CometProject (143) - : : : : : : : : +- CometBroadcastHashJoin (142) - : : : : : : : : :- CometProject (140) - : : : : : : : : : +- CometBroadcastHashJoin (139) - : : : : : : : : : :- CometProject (137) - : : : : : : : : : : +- CometBroadcastHashJoin (136) - : : : : : : : : : : :- CometProject (134) - : : : : : : : : : : : +- CometBroadcastHashJoin (133) - : : : : : : : : : : : :- CometProject (131) - : : : : : : : : : : : : +- CometBroadcastHashJoin (130) - : : : : : : : : : : : : :- CometProject (128) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (127) - : : : : : : : : : : : : : :- CometProject (125) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (124) - : : : : : : : : : : : : : : :- CometProject (120) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (119) - : : : : : : : : : : : : : : : :- CometSort (113) - : : : : : : : : : : : : : : : : +- CometExchange (112) - : : : : : : : : : : : : : : : : +- CometProject (111) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (110) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (106) - : : : : : : : : : : : : : : : : : +- CometFilter (105) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (104) - : : : : : : : : : : : : : : : : +- CometProject (109) - : : : : : : : : : : : : : : : : +- CometFilter (108) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (107) - : : : : : : : : : : : : : : : +- CometSort (118) - : : : : : : : : : : : : : : : +- CometProject (117) - : : : : : : : : : : : : : : : +- CometFilter (116) - : : : : : : : : : : : : : : : +- CometHashAggregate (115) - : : : : : : : : : : : : : : : +- ReusedExchange (114) - : : : : : : : : : : : : : : +- CometBroadcastExchange (123) - : : : : : : : : : : : : : : +- CometFilter (122) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (121) - : : : : : : : : : : : : : +- ReusedExchange (126) - : : : : : : : : : : : : +- ReusedExchange (129) - : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : +- ReusedExchange (141) - : : : : : : : +- ReusedExchange (144) - : : : : : : +- ReusedExchange (147) - : : : : : +- ReusedExchange (150) - : : : : +- ReusedExchange (153) - : : : +- ReusedExchange (156) - : : +- ReusedExchange (159) - : +- ReusedExchange (162) - +- ReusedExchange (165) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) CometFilter -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) - -(3) CometBroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) - -(6) CometProject -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] - -(7) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner, BuildLeft - -(8) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(9) CometExchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometSort -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) - -(13) CometProject -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] - -(14) CometExchange -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(15) CometSort -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(17) CometFilter -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) - -(18) CometProject -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(19) CometExchange -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner - -(22) CometProject -Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(23) CometHashAggregate -Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Keys [1]: [cs_item_sk#17] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(24) CometExchange -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometHashAggregate -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Keys [1]: [cs_item_sk#17] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(26) CometFilter -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(27) CometProject -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Arguments: [cs_item_sk#17], [cs_item_sk#17] - -(28) CometSort -Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] - -(29) CometSortMergeJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [1]: [cs_item_sk#17] -Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner - -(30) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(33) CometBroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: [d_date_sk#32, d_year#33] - -(34) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight - -(35) CometProject -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#36, 10)))) - -(38) CometProject -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37], [s_store_sk#34, s_store_name#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#36, 10)) AS s_zip#37] - -(39) CometBroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37] - -(40) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight - -(41) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(43) CometFilter -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Condition : (((((isnotnull(c_customer_sk#38) AND isnotnull(c_first_sales_date_sk#43)) AND isnotnull(c_first_shipto_date_sk#42)) AND isnotnull(c_current_cdemo_sk#39)) AND isnotnull(c_current_hdemo_sk#40)) AND isnotnull(c_current_addr_sk#41)) - -(44) CometBroadcastExchange -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(45) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -Right output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_customer_sk#2], [c_customer_sk#38], Inner, BuildRight - -(46) CometProject -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(48) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : isnotnull(d_date_sk#44) - -(49) CometBroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] - -(50) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [c_first_sales_date_sk#43], [d_date_sk#44], Inner, BuildRight - -(51) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43, d_date_sk#44, d_year#45] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] - -(52) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#46, d_year#47] - -(53) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [c_first_shipto_date_sk#42], [d_date_sk#46], Inner, BuildRight - -(54) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45, d_date_sk#46, d_year#47] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#48, cd_marital_status#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#49, 1)))) - -(57) CometProject -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Arguments: [cd_demo_sk#48, cd_marital_status#50], [cd_demo_sk#48, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#49, 1)) AS cd_marital_status#50] - -(58) CometBroadcastExchange -Input [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [cd_demo_sk#48, cd_marital_status#50] - -(59) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_cdemo_sk#3], [cd_demo_sk#48], Inner, BuildRight - -(60) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] - -(61) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#51, cd_marital_status#52] - -(62) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -Right output [2]: [cd_demo_sk#51, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#39], [cd_demo_sk#51], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(63) CometProject -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50, cd_demo_sk#51, cd_marital_status#52] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(65) CometFilter -Input [1]: [p_promo_sk#53] -Condition : isnotnull(p_promo_sk#53) - -(66) CometBroadcastExchange -Input [1]: [p_promo_sk#53] -Arguments: [p_promo_sk#53] - -(67) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [1]: [p_promo_sk#53] -Arguments: [ss_promo_sk#7], [p_promo_sk#53], Inner, BuildRight - -(68) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, p_promo_sk#53] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(70) CometFilter -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Condition : (isnotnull(hd_demo_sk#54) AND isnotnull(hd_income_band_sk#55)) - -(71) CometBroadcastExchange -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [hd_demo_sk#54, hd_income_band_sk#55] - -(72) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_hdemo_sk#4], [hd_demo_sk#54], Inner, BuildRight - -(73) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] - -(74) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] - -(75) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -Right output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [c_current_hdemo_sk#40], [hd_demo_sk#56], Inner, BuildRight - -(76) CometProject -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(78) CometFilter -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Condition : isnotnull(ca_address_sk#58) - -(79) CometProject -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ca_address_sk#58, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_number#59, 10)) AS ca_street_number#63, ca_street_name#60, ca_city#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#62, 10)) AS ca_zip#64] - -(80) CometBroadcastExchange -Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(81) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -Right output [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_addr_sk#5], [ca_address_sk#58], Inner, BuildRight - -(82) CometProject -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(83) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(84) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Right output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [c_current_addr_sk#41], [ca_address_sk#65], Inner, BuildRight - -(85) CometProject -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(87) CometFilter -Input [1]: [ib_income_band_sk#70] -Condition : isnotnull(ib_income_band_sk#70) - -(88) CometBroadcastExchange -Input [1]: [ib_income_band_sk#70] -Arguments: [ib_income_band_sk#70] - -(89) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#70] -Arguments: [hd_income_band_sk#55], [ib_income_band_sk#70], Inner, BuildRight - -(90) CometProject -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#70] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(91) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#71] - -(92) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#71] -Arguments: [hd_income_band_sk#57], [ib_income_band_sk#71], Inner, BuildRight - -(93) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#71] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(95) CometFilter -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Condition : ((((((isnotnull(i_current_price#73) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#74, 20)) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#73 >= 64.00)) AND (i_current_price#73 <= 74.00)) AND (i_current_price#73 >= 65.00)) AND (i_current_price#73 <= 79.00)) AND isnotnull(i_item_sk#72)) - -(96) CometProject -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Arguments: [i_item_sk#72, i_product_name#76], [i_item_sk#72, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#75, 50)) AS i_product_name#76] - -(97) CometBroadcastExchange -Input [2]: [i_item_sk#72, i_product_name#76] -Arguments: [i_item_sk#72, i_product_name#76] - -(98) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [2]: [i_item_sk#72, i_product_name#76] -Arguments: [ss_item_sk#1], [i_item_sk#72], Inner, BuildRight - -(99) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] - -(100) CometHashAggregate -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] - -(101) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#77, sum#78, sum#79, sum#80] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] - -(102) CometExchange -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: hashpartitioning(item_sk#82, store_name#83, store_zip#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(103) CometSort -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97], [item_sk#82 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, store_zip#84 ASC NULLS FIRST] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#109), dynamicpruningexpression(ss_sold_date_sk#109 IN dynamicpruning#110)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(105) CometFilter -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Condition : (((((((isnotnull(ss_item_sk#98) AND isnotnull(ss_ticket_number#105)) AND isnotnull(ss_store_sk#103)) AND isnotnull(ss_customer_sk#99)) AND isnotnull(ss_cdemo_sk#100)) AND isnotnull(ss_promo_sk#104)) AND isnotnull(ss_hdemo_sk#101)) AND isnotnull(ss_addr_sk#102)) - -(106) CometBroadcastExchange -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(108) CometFilter -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) - -(109) CometProject -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] - -(110) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_ticket_number#105], [sr_item_sk#111, sr_ticket_number#112], Inner, BuildLeft - -(111) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(112) CometExchange -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: hashpartitioning(ss_item_sk#98, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(113) CometSort -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98 ASC NULLS FIRST] - -(114) ReusedExchange [Reuses operator id: 24] -Output [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] - -(115) CometHashAggregate -Input [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] -Keys [1]: [cs_item_sk#114] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#118)), sum(((cr_refunded_cash#119 + cr_reversed_charge#120) + cr_store_credit#121))] - -(116) CometFilter -Input [3]: [cs_item_sk#114, sale#122, refund#123] -Condition : ((isnotnull(sale#122) AND isnotnull(refund#123)) AND (cast(sale#122 as decimal(21,2)) > (2 * refund#123))) - -(117) CometProject -Input [3]: [cs_item_sk#114, sale#122, refund#123] -Arguments: [cs_item_sk#114], [cs_item_sk#114] - -(118) CometSort -Input [1]: [cs_item_sk#114] -Arguments: [cs_item_sk#114], [cs_item_sk#114 ASC NULLS FIRST] - -(119) CometSortMergeJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [1]: [cs_item_sk#114] -Arguments: [ss_item_sk#98], [cs_item_sk#114], Inner - -(120) CometProject -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, cs_item_sk#114] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(121) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#124, d_year#125] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(122) CometFilter -Input [2]: [d_date_sk#124, d_year#125] -Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) - -(123) CometBroadcastExchange -Input [2]: [d_date_sk#124, d_year#125] -Arguments: [d_date_sk#124, d_year#125] - -(124) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [d_date_sk#124, d_year#125] -Arguments: [ss_sold_date_sk#109], [d_date_sk#124], Inner, BuildRight - -(125) CometProject -Input [13]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, d_date_sk#124, d_year#125] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125] - -(126) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_store_sk#126, s_store_name#127, s_zip#128] - -(127) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125] -Right output [3]: [s_store_sk#126, s_store_name#127, s_zip#128] -Arguments: [ss_store_sk#103], [s_store_sk#126], Inner, BuildRight - -(128) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_sk#126, s_store_name#127, s_zip#128] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128] - -(129) ReusedExchange [Reuses operator id: 44] -Output [6]: [c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] - -(130) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128] -Right output [6]: [c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] -Arguments: [ss_customer_sk#99], [c_customer_sk#129], Inner, BuildRight - -(131) CometProject -Input [18]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] - -(132) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#135, d_year#136] - -(133) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] -Right output [2]: [d_date_sk#135, d_year#136] -Arguments: [c_first_sales_date_sk#134], [d_date_sk#135], Inner, BuildRight - -(134) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134, d_date_sk#135, d_year#136] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136] - -(135) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#137, d_year#138] - -(136) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136] -Right output [2]: [d_date_sk#137, d_year#138] -Arguments: [c_first_shipto_date_sk#133], [d_date_sk#137], Inner, BuildRight - -(137) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136, d_date_sk#137, d_year#138] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] - -(138) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#139, cd_marital_status#140] - -(139) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] -Right output [2]: [cd_demo_sk#139, cd_marital_status#140] -Arguments: [ss_cdemo_sk#100], [cd_demo_sk#139], Inner, BuildRight - -(140) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_demo_sk#139, cd_marital_status#140] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140] - -(141) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#141, cd_marital_status#142] - -(142) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140] -Right output [2]: [cd_demo_sk#141, cd_marital_status#142] -Arguments: [c_current_cdemo_sk#130], [cd_demo_sk#141], Inner, NOT (cd_marital_status#140 = cd_marital_status#142), BuildRight - -(143) CometProject -Input [18]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140, cd_demo_sk#141, cd_marital_status#142] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] - -(144) ReusedExchange [Reuses operator id: 66] -Output [1]: [p_promo_sk#143] - -(145) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] -Right output [1]: [p_promo_sk#143] -Arguments: [ss_promo_sk#104], [p_promo_sk#143], Inner, BuildRight - -(146) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, p_promo_sk#143] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] - -(147) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#144, hd_income_band_sk#145] - -(148) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] -Right output [2]: [hd_demo_sk#144, hd_income_band_sk#145] -Arguments: [ss_hdemo_sk#101], [hd_demo_sk#144], Inner, BuildRight - -(149) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_demo_sk#144, hd_income_band_sk#145] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145] - -(150) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#146, hd_income_band_sk#147] - -(151) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145] -Right output [2]: [hd_demo_sk#146, hd_income_band_sk#147] -Arguments: [c_current_hdemo_sk#131], [hd_demo_sk#146], Inner, BuildRight - -(152) CometProject -Input [15]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_demo_sk#146, hd_income_band_sk#147] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147] - -(153) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] - -(154) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147] -Right output [5]: [ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] -Arguments: [ss_addr_sk#102], [ca_address_sk#148], Inner, BuildRight - -(155) CometProject -Input [18]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] - -(156) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(157) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] -Right output [5]: [ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Arguments: [c_current_addr_sk#132], [ca_address_sk#153], Inner, BuildRight - -(158) CometProject -Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(159) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#158] - -(160) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Right output [1]: [ib_income_band_sk#158] -Arguments: [hd_income_band_sk#145], [ib_income_band_sk#158], Inner, BuildRight - -(161) CometProject -Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, ib_income_band_sk#158] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(162) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#159] - -(163) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Right output [1]: [ib_income_band_sk#159] -Arguments: [hd_income_band_sk#147], [ib_income_band_sk#159], Inner, BuildRight - -(164) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, ib_income_band_sk#159] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] - -(165) ReusedExchange [Reuses operator id: 97] -Output [2]: [i_item_sk#160, i_product_name#161] - -(166) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] -Right output [2]: [i_item_sk#160, i_product_name#161] -Arguments: [ss_item_sk#98], [i_item_sk#160], Inner, BuildRight - -(167) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] -Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] - -(168) CometHashAggregate -Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] -Keys [15]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#106)), partial_sum(UnscaledValue(ss_list_price#107)), partial_sum(UnscaledValue(ss_coupon_amt#108))] - -(169) CometHashAggregate -Input [19]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138, count#77, sum#162, sum#163, sum#164] -Keys [15]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#106)), sum(UnscaledValue(ss_list_price#107)), sum(UnscaledValue(ss_coupon_amt#108))] - -(170) CometExchange -Input [8]: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: hashpartitioning(item_sk#165, store_name#166, store_zip#167, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(171) CometSort -Input [8]: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172], [item_sk#165 ASC NULLS FIRST, store_name#166 ASC NULLS FIRST, store_zip#167 ASC NULLS FIRST] - -(172) CometSortMergeJoin -Left output [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Right output [8]: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#165, store_name#166, store_zip#167], Inner, (cnt#169 <= cnt#94) - -(173) CometProject -Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] - -(174) CometExchange -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#169 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#170 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(175) CometSort -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#169 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#170 ASC NULLS FIRST] - -(176) CometColumnarToRow [codegen id : 1] -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#170, s2#171, s3#172, syear#168, cnt#169] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (180) -+- * CometColumnarToRow (179) - +- CometFilter (178) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) - - -(177) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(178) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(179) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#33] - -(180) BroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 -BroadcastExchange (184) -+- * CometColumnarToRow (183) - +- CometFilter (182) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) - - -(181) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#124, d_year#125] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(182) CometFilter -Input [2]: [d_date_sk#124, d_year#125] -Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) - -(183) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#124, d_year#125] - -(184) BroadcastExchange -Input [2]: [d_date_sk#124, d_year#125] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/extended.txt deleted file mode 100644 index 059acab385..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/extended.txt +++ /dev/null @@ -1,247 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 46fe063430..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt +++ /dev/null @@ -1,192 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometExchange [product_name,store_name,cnt,s1,s1] #1 - CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #3 - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [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] #11 - 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 [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk] - ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #20 - 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_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] #21 - 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 [native_iceberg_compat] 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 #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - CometBroadcastExchange [d_date_sk,d_year] #23 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/explain.txt deleted file mode 100644 index 5af8806175..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/explain.txt +++ /dev/null @@ -1,548 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (90) -+- * Filter (89) - +- Window (88) - +- WindowGroupLimit (87) - +- * CometColumnarToRow (86) - +- CometSort (85) - +- CometColumnarExchange (84) - +- WindowGroupLimit (83) - +- * Sort (82) - +- Union (81) - :- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.item (14) - :- * HashAggregate (31) - : +- * CometColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- * HashAggregate (28) - : +- * HashAggregate (27) - : +- * CometColumnarToRow (26) - : +- ReusedExchange (25) - :- * HashAggregate (38) - : +- * CometColumnarToRow (37) - : +- CometColumnarExchange (36) - : +- * HashAggregate (35) - : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) - : +- ReusedExchange (32) - :- * HashAggregate (45) - : +- * CometColumnarToRow (44) - : +- CometColumnarExchange (43) - : +- * HashAggregate (42) - : +- * HashAggregate (41) - : +- * CometColumnarToRow (40) - : +- ReusedExchange (39) - :- * HashAggregate (52) - : +- * CometColumnarToRow (51) - : +- CometColumnarExchange (50) - : +- * HashAggregate (49) - : +- * HashAggregate (48) - : +- * CometColumnarToRow (47) - : +- ReusedExchange (46) - :- * HashAggregate (59) - : +- * CometColumnarToRow (58) - : +- CometColumnarExchange (57) - : +- * HashAggregate (56) - : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) - : +- ReusedExchange (53) - :- * HashAggregate (66) - : +- * CometColumnarToRow (65) - : +- CometColumnarExchange (64) - : +- * HashAggregate (63) - : +- * HashAggregate (62) - : +- * CometColumnarToRow (61) - : +- ReusedExchange (60) - :- * HashAggregate (73) - : +- * CometColumnarToRow (72) - : +- CometColumnarExchange (71) - : +- * HashAggregate (70) - : +- * HashAggregate (69) - : +- * CometColumnarToRow (68) - : +- ReusedExchange (67) - +- * HashAggregate (80) - +- * CometColumnarToRow (79) - +- CometColumnarExchange (78) - +- * HashAggregate (77) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- ReusedExchange (74) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(4) ReusedExchange [Reuses operator id: 95] -Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_store_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [s_store_sk#11, s_store_id#12] -Condition : isnotnull(s_store_sk#11) - -(9) CometProject -Input [2]: [s_store_sk#11, s_store_id#12] -Arguments: [s_store_sk#11, s_store_id#13], [s_store_sk#11, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#12, 16)) AS s_store_id#13] - -(10) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#11, s_store_id#13] - -(11) BroadcastExchange -Input [2]: [s_store_sk#11, s_store_id#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#13] - -(14) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -Condition : isnotnull(i_item_sk#14) - -(16) CometProject -Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -Arguments: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22], [i_item_sk#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#15, 50)) AS i_brand#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#16, 50)) AS i_class#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#17, 50)) AS i_category#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#18, 50)) AS i_product_name#22] - -(17) CometColumnarToRow [codegen id : 3] -Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] - -(18) BroadcastExchange -Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#14] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_brand#19, i_class#20, i_category#21, i_product_name#22] -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] - -(21) HashAggregate [codegen id : 4] -Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_brand#19, i_class#20, i_category#21, i_product_name#22] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] -Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [2]: [sum#23, isEmpty#24] -Results [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] - -(22) CometColumnarExchange -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 5] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] - -(24) HashAggregate [codegen id : 5] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#27] -Results [9]: [i_category#21 AS i_category#28, i_class#20 AS i_class#29, i_brand#19 AS i_brand#30, i_product_name#22 AS i_product_name#31, d_year#8 AS d_year#32, d_qoy#10 AS d_qoy#33, d_moy#9 AS d_moy#34, s_store_id#13 AS s_store_id#35, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#27 as decimal(38,2)) AS sumsales#36] - -(25) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43, s_store_id#44, sum#45, isEmpty#46] - -(26) CometColumnarToRow [codegen id : 10] -Input [10]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43, s_store_id#44, sum#45, isEmpty#46] - -(27) HashAggregate [codegen id : 10] -Input [10]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43, s_store_id#44, sum#45, isEmpty#46] -Keys [8]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43, s_store_id#44] -Functions [1]: [sum(coalesce((ss_sales_price#47 * cast(ss_quantity#48 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#47 * cast(ss_quantity#48 as decimal(10,0))), 0.00))#27] -Results [8]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43, sum(coalesce((ss_sales_price#47 * cast(ss_quantity#48 as decimal(10,0))), 0.00))#27 AS sumsales#49] - -(28) HashAggregate [codegen id : 10] -Input [8]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43, sumsales#49] -Keys [7]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43] -Functions [1]: [partial_sum(sumsales#49)] -Aggregate Attributes [2]: [sum#50, isEmpty#51] -Results [9]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43, sum#52, isEmpty#53] - -(29) CometColumnarExchange -Input [9]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43, sum#52, isEmpty#53] -Arguments: hashpartitioning(i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 11] -Input [9]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43, sum#52, isEmpty#53] - -(31) HashAggregate [codegen id : 11] -Input [9]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43, sum#52, isEmpty#53] -Keys [7]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43] -Functions [1]: [sum(sumsales#49)] -Aggregate Attributes [1]: [sum(sumsales#49)#54] -Results [9]: [i_category#37, i_class#38, i_brand#39, i_product_name#40, d_year#41, d_qoy#42, d_moy#43, null AS s_store_id#55, sum(sumsales#49)#54 AS sumsales#56] - -(32) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62, d_moy#63, s_store_id#64, sum#65, isEmpty#66] - -(33) CometColumnarToRow [codegen id : 16] -Input [10]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62, d_moy#63, s_store_id#64, sum#65, isEmpty#66] - -(34) HashAggregate [codegen id : 16] -Input [10]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62, d_moy#63, s_store_id#64, sum#65, isEmpty#66] -Keys [8]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62, d_moy#63, s_store_id#64] -Functions [1]: [sum(coalesce((ss_sales_price#67 * cast(ss_quantity#68 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#67 * cast(ss_quantity#68 as decimal(10,0))), 0.00))#27] -Results [7]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62, sum(coalesce((ss_sales_price#67 * cast(ss_quantity#68 as decimal(10,0))), 0.00))#27 AS sumsales#69] - -(35) HashAggregate [codegen id : 16] -Input [7]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62, sumsales#69] -Keys [6]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62] -Functions [1]: [partial_sum(sumsales#69)] -Aggregate Attributes [2]: [sum#70, isEmpty#71] -Results [8]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62, sum#72, isEmpty#73] - -(36) CometColumnarExchange -Input [8]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62, sum#72, isEmpty#73] -Arguments: hashpartitioning(i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(37) CometColumnarToRow [codegen id : 17] -Input [8]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62, sum#72, isEmpty#73] - -(38) HashAggregate [codegen id : 17] -Input [8]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62, sum#72, isEmpty#73] -Keys [6]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62] -Functions [1]: [sum(sumsales#69)] -Aggregate Attributes [1]: [sum(sumsales#69)#74] -Results [9]: [i_category#57, i_class#58, i_brand#59, i_product_name#60, d_year#61, d_qoy#62, null AS d_moy#75, null AS s_store_id#76, sum(sumsales#69)#74 AS sumsales#77] - -(39) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82, d_qoy#83, d_moy#84, s_store_id#85, sum#86, isEmpty#87] - -(40) CometColumnarToRow [codegen id : 22] -Input [10]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82, d_qoy#83, d_moy#84, s_store_id#85, sum#86, isEmpty#87] - -(41) HashAggregate [codegen id : 22] -Input [10]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82, d_qoy#83, d_moy#84, s_store_id#85, sum#86, isEmpty#87] -Keys [8]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82, d_qoy#83, d_moy#84, s_store_id#85] -Functions [1]: [sum(coalesce((ss_sales_price#88 * cast(ss_quantity#89 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#88 * cast(ss_quantity#89 as decimal(10,0))), 0.00))#27] -Results [6]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82, sum(coalesce((ss_sales_price#88 * cast(ss_quantity#89 as decimal(10,0))), 0.00))#27 AS sumsales#90] - -(42) HashAggregate [codegen id : 22] -Input [6]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82, sumsales#90] -Keys [5]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82] -Functions [1]: [partial_sum(sumsales#90)] -Aggregate Attributes [2]: [sum#91, isEmpty#92] -Results [7]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82, sum#93, isEmpty#94] - -(43) CometColumnarExchange -Input [7]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82, sum#93, isEmpty#94] -Arguments: hashpartitioning(i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(44) CometColumnarToRow [codegen id : 23] -Input [7]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82, sum#93, isEmpty#94] - -(45) HashAggregate [codegen id : 23] -Input [7]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82, sum#93, isEmpty#94] -Keys [5]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82] -Functions [1]: [sum(sumsales#90)] -Aggregate Attributes [1]: [sum(sumsales#90)#95] -Results [9]: [i_category#78, i_class#79, i_brand#80, i_product_name#81, d_year#82, null AS d_qoy#96, null AS d_moy#97, null AS s_store_id#98, sum(sumsales#90)#95 AS sumsales#99] - -(46) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#100, i_class#101, i_brand#102, i_product_name#103, d_year#104, d_qoy#105, d_moy#106, s_store_id#107, sum#108, isEmpty#109] - -(47) CometColumnarToRow [codegen id : 28] -Input [10]: [i_category#100, i_class#101, i_brand#102, i_product_name#103, d_year#104, d_qoy#105, d_moy#106, s_store_id#107, sum#108, isEmpty#109] - -(48) HashAggregate [codegen id : 28] -Input [10]: [i_category#100, i_class#101, i_brand#102, i_product_name#103, d_year#104, d_qoy#105, d_moy#106, s_store_id#107, sum#108, isEmpty#109] -Keys [8]: [i_category#100, i_class#101, i_brand#102, i_product_name#103, d_year#104, d_qoy#105, d_moy#106, s_store_id#107] -Functions [1]: [sum(coalesce((ss_sales_price#110 * cast(ss_quantity#111 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#110 * cast(ss_quantity#111 as decimal(10,0))), 0.00))#27] -Results [5]: [i_category#100, i_class#101, i_brand#102, i_product_name#103, sum(coalesce((ss_sales_price#110 * cast(ss_quantity#111 as decimal(10,0))), 0.00))#27 AS sumsales#112] - -(49) HashAggregate [codegen id : 28] -Input [5]: [i_category#100, i_class#101, i_brand#102, i_product_name#103, sumsales#112] -Keys [4]: [i_category#100, i_class#101, i_brand#102, i_product_name#103] -Functions [1]: [partial_sum(sumsales#112)] -Aggregate Attributes [2]: [sum#113, isEmpty#114] -Results [6]: [i_category#100, i_class#101, i_brand#102, i_product_name#103, sum#115, isEmpty#116] - -(50) CometColumnarExchange -Input [6]: [i_category#100, i_class#101, i_brand#102, i_product_name#103, sum#115, isEmpty#116] -Arguments: hashpartitioning(i_category#100, i_class#101, i_brand#102, i_product_name#103, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(51) CometColumnarToRow [codegen id : 29] -Input [6]: [i_category#100, i_class#101, i_brand#102, i_product_name#103, sum#115, isEmpty#116] - -(52) HashAggregate [codegen id : 29] -Input [6]: [i_category#100, i_class#101, i_brand#102, i_product_name#103, sum#115, isEmpty#116] -Keys [4]: [i_category#100, i_class#101, i_brand#102, i_product_name#103] -Functions [1]: [sum(sumsales#112)] -Aggregate Attributes [1]: [sum(sumsales#112)#117] -Results [9]: [i_category#100, i_class#101, i_brand#102, i_product_name#103, null AS d_year#118, null AS d_qoy#119, null AS d_moy#120, null AS s_store_id#121, sum(sumsales#112)#117 AS sumsales#122] - -(53) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#123, i_class#124, i_brand#125, i_product_name#126, d_year#127, d_qoy#128, d_moy#129, s_store_id#130, sum#131, isEmpty#132] - -(54) CometColumnarToRow [codegen id : 34] -Input [10]: [i_category#123, i_class#124, i_brand#125, i_product_name#126, d_year#127, d_qoy#128, d_moy#129, s_store_id#130, sum#131, isEmpty#132] - -(55) HashAggregate [codegen id : 34] -Input [10]: [i_category#123, i_class#124, i_brand#125, i_product_name#126, d_year#127, d_qoy#128, d_moy#129, s_store_id#130, sum#131, isEmpty#132] -Keys [8]: [i_category#123, i_class#124, i_brand#125, i_product_name#126, d_year#127, d_qoy#128, d_moy#129, s_store_id#130] -Functions [1]: [sum(coalesce((ss_sales_price#133 * cast(ss_quantity#134 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#133 * cast(ss_quantity#134 as decimal(10,0))), 0.00))#27] -Results [4]: [i_category#123, i_class#124, i_brand#125, sum(coalesce((ss_sales_price#133 * cast(ss_quantity#134 as decimal(10,0))), 0.00))#27 AS sumsales#135] - -(56) HashAggregate [codegen id : 34] -Input [4]: [i_category#123, i_class#124, i_brand#125, sumsales#135] -Keys [3]: [i_category#123, i_class#124, i_brand#125] -Functions [1]: [partial_sum(sumsales#135)] -Aggregate Attributes [2]: [sum#136, isEmpty#137] -Results [5]: [i_category#123, i_class#124, i_brand#125, sum#138, isEmpty#139] - -(57) CometColumnarExchange -Input [5]: [i_category#123, i_class#124, i_brand#125, sum#138, isEmpty#139] -Arguments: hashpartitioning(i_category#123, i_class#124, i_brand#125, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(58) CometColumnarToRow [codegen id : 35] -Input [5]: [i_category#123, i_class#124, i_brand#125, sum#138, isEmpty#139] - -(59) HashAggregate [codegen id : 35] -Input [5]: [i_category#123, i_class#124, i_brand#125, sum#138, isEmpty#139] -Keys [3]: [i_category#123, i_class#124, i_brand#125] -Functions [1]: [sum(sumsales#135)] -Aggregate Attributes [1]: [sum(sumsales#135)#140] -Results [9]: [i_category#123, i_class#124, i_brand#125, null AS i_product_name#141, null AS d_year#142, null AS d_qoy#143, null AS d_moy#144, null AS s_store_id#145, sum(sumsales#135)#140 AS sumsales#146] - -(60) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#147, i_class#148, i_brand#149, i_product_name#150, d_year#151, d_qoy#152, d_moy#153, s_store_id#154, sum#155, isEmpty#156] - -(61) CometColumnarToRow [codegen id : 40] -Input [10]: [i_category#147, i_class#148, i_brand#149, i_product_name#150, d_year#151, d_qoy#152, d_moy#153, s_store_id#154, sum#155, isEmpty#156] - -(62) HashAggregate [codegen id : 40] -Input [10]: [i_category#147, i_class#148, i_brand#149, i_product_name#150, d_year#151, d_qoy#152, d_moy#153, s_store_id#154, sum#155, isEmpty#156] -Keys [8]: [i_category#147, i_class#148, i_brand#149, i_product_name#150, d_year#151, d_qoy#152, d_moy#153, s_store_id#154] -Functions [1]: [sum(coalesce((ss_sales_price#157 * cast(ss_quantity#158 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#157 * cast(ss_quantity#158 as decimal(10,0))), 0.00))#27] -Results [3]: [i_category#147, i_class#148, sum(coalesce((ss_sales_price#157 * cast(ss_quantity#158 as decimal(10,0))), 0.00))#27 AS sumsales#159] - -(63) HashAggregate [codegen id : 40] -Input [3]: [i_category#147, i_class#148, sumsales#159] -Keys [2]: [i_category#147, i_class#148] -Functions [1]: [partial_sum(sumsales#159)] -Aggregate Attributes [2]: [sum#160, isEmpty#161] -Results [4]: [i_category#147, i_class#148, sum#162, isEmpty#163] - -(64) CometColumnarExchange -Input [4]: [i_category#147, i_class#148, sum#162, isEmpty#163] -Arguments: hashpartitioning(i_category#147, i_class#148, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(65) CometColumnarToRow [codegen id : 41] -Input [4]: [i_category#147, i_class#148, sum#162, isEmpty#163] - -(66) HashAggregate [codegen id : 41] -Input [4]: [i_category#147, i_class#148, sum#162, isEmpty#163] -Keys [2]: [i_category#147, i_class#148] -Functions [1]: [sum(sumsales#159)] -Aggregate Attributes [1]: [sum(sumsales#159)#164] -Results [9]: [i_category#147, i_class#148, null AS i_brand#165, null AS i_product_name#166, null AS d_year#167, null AS d_qoy#168, null AS d_moy#169, null AS s_store_id#170, sum(sumsales#159)#164 AS sumsales#171] - -(67) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#172, i_class#173, i_brand#174, i_product_name#175, d_year#176, d_qoy#177, d_moy#178, s_store_id#179, sum#180, isEmpty#181] - -(68) CometColumnarToRow [codegen id : 46] -Input [10]: [i_category#172, i_class#173, i_brand#174, i_product_name#175, d_year#176, d_qoy#177, d_moy#178, s_store_id#179, sum#180, isEmpty#181] - -(69) HashAggregate [codegen id : 46] -Input [10]: [i_category#172, i_class#173, i_brand#174, i_product_name#175, d_year#176, d_qoy#177, d_moy#178, s_store_id#179, sum#180, isEmpty#181] -Keys [8]: [i_category#172, i_class#173, i_brand#174, i_product_name#175, d_year#176, d_qoy#177, d_moy#178, s_store_id#179] -Functions [1]: [sum(coalesce((ss_sales_price#182 * cast(ss_quantity#183 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#182 * cast(ss_quantity#183 as decimal(10,0))), 0.00))#27] -Results [2]: [i_category#172, sum(coalesce((ss_sales_price#182 * cast(ss_quantity#183 as decimal(10,0))), 0.00))#27 AS sumsales#184] - -(70) HashAggregate [codegen id : 46] -Input [2]: [i_category#172, sumsales#184] -Keys [1]: [i_category#172] -Functions [1]: [partial_sum(sumsales#184)] -Aggregate Attributes [2]: [sum#185, isEmpty#186] -Results [3]: [i_category#172, sum#187, isEmpty#188] - -(71) CometColumnarExchange -Input [3]: [i_category#172, sum#187, isEmpty#188] -Arguments: hashpartitioning(i_category#172, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(72) CometColumnarToRow [codegen id : 47] -Input [3]: [i_category#172, sum#187, isEmpty#188] - -(73) HashAggregate [codegen id : 47] -Input [3]: [i_category#172, sum#187, isEmpty#188] -Keys [1]: [i_category#172] -Functions [1]: [sum(sumsales#184)] -Aggregate Attributes [1]: [sum(sumsales#184)#189] -Results [9]: [i_category#172, null AS i_class#190, null AS i_brand#191, null AS i_product_name#192, null AS d_year#193, null AS d_qoy#194, null AS d_moy#195, null AS s_store_id#196, sum(sumsales#184)#189 AS sumsales#197] - -(74) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#198, i_class#199, i_brand#200, i_product_name#201, d_year#202, d_qoy#203, d_moy#204, s_store_id#205, sum#206, isEmpty#207] - -(75) CometColumnarToRow [codegen id : 52] -Input [10]: [i_category#198, i_class#199, i_brand#200, i_product_name#201, d_year#202, d_qoy#203, d_moy#204, s_store_id#205, sum#206, isEmpty#207] - -(76) HashAggregate [codegen id : 52] -Input [10]: [i_category#198, i_class#199, i_brand#200, i_product_name#201, d_year#202, d_qoy#203, d_moy#204, s_store_id#205, sum#206, isEmpty#207] -Keys [8]: [i_category#198, i_class#199, i_brand#200, i_product_name#201, d_year#202, d_qoy#203, d_moy#204, s_store_id#205] -Functions [1]: [sum(coalesce((ss_sales_price#208 * cast(ss_quantity#209 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#208 * cast(ss_quantity#209 as decimal(10,0))), 0.00))#27] -Results [1]: [sum(coalesce((ss_sales_price#208 * cast(ss_quantity#209 as decimal(10,0))), 0.00))#27 AS sumsales#210] - -(77) HashAggregate [codegen id : 52] -Input [1]: [sumsales#210] -Keys: [] -Functions [1]: [partial_sum(sumsales#210)] -Aggregate Attributes [2]: [sum#211, isEmpty#212] -Results [2]: [sum#213, isEmpty#214] - -(78) CometColumnarExchange -Input [2]: [sum#213, isEmpty#214] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(79) CometColumnarToRow [codegen id : 53] -Input [2]: [sum#213, isEmpty#214] - -(80) HashAggregate [codegen id : 53] -Input [2]: [sum#213, isEmpty#214] -Keys: [] -Functions [1]: [sum(sumsales#210)] -Aggregate Attributes [1]: [sum(sumsales#210)#215] -Results [9]: [null AS i_category#216, null AS i_class#217, null AS i_brand#218, null AS i_product_name#219, null AS d_year#220, null AS d_qoy#221, null AS d_moy#222, null AS s_store_id#223, sum(sumsales#210)#215 AS sumsales#224] - -(81) Union - -(82) Sort [codegen id : 54] -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: [i_category#28 ASC NULLS FIRST, sumsales#36 DESC NULLS LAST], false, 0 - -(83) WindowGroupLimit -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: [i_category#28], [sumsales#36 DESC NULLS LAST], rank(sumsales#36), 100, Partial - -(84) CometColumnarExchange -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: hashpartitioning(i_category#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(85) CometSort -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36], [i_category#28 ASC NULLS FIRST, sumsales#36 DESC NULLS LAST] - -(86) CometColumnarToRow [codegen id : 55] -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] - -(87) WindowGroupLimit -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: [i_category#28], [sumsales#36 DESC NULLS LAST], rank(sumsales#36), 100, Final - -(88) Window -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: [rank(sumsales#36) windowspecdefinition(i_category#28, sumsales#36 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#225], [i_category#28], [sumsales#36 DESC NULLS LAST] - -(89) Filter [codegen id : 56] -Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#225] -Condition : (rk#225 <= 100) - -(90) TakeOrderedAndProject -Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#225] -Arguments: 100, [i_category#28 ASC NULLS FIRST, i_class#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, i_product_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_qoy#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, s_store_id#35 ASC NULLS FIRST, sumsales#36 ASC NULLS FIRST, rk#225 ASC NULLS FIRST], [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#225] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (95) -+- * CometColumnarToRow (94) - +- CometProject (93) - +- CometFilter (92) - +- CometNativeScan parquet spark_catalog.default.date_dim (91) - - -(91) CometNativeScan parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#226, d_year#8, d_moy#9, d_qoy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(92) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#226, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#226) AND (d_month_seq#226 >= 1212)) AND (d_month_seq#226 <= 1223)) AND isnotnull(d_date_sk#7)) - -(93) CometProject -Input [5]: [d_date_sk#7, d_month_seq#226, d_year#8, d_moy#9, d_qoy#10] -Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(94) CometColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(95) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/simplified.txt deleted file mode 100644 index 38fa0be072..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/simplified.txt +++ /dev/null @@ -1,147 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (56) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (55) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (54) - Sort [i_category,sumsales] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - WholeStageCodegen (11) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 - WholeStageCodegen (10) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (17) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 - WholeStageCodegen (16) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (23) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year] #8 - WholeStageCodegen (22) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (29) - HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name] #9 - WholeStageCodegen (28) - HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (35) - HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand] #10 - WholeStageCodegen (34) - HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (41) - HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #11 - WholeStageCodegen (40) - HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (47) - HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #12 - WholeStageCodegen (46) - HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (53) - HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #13 - WholeStageCodegen (52) - HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/explain.txt deleted file mode 100644 index 481b95c617..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,445 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (74) -+- * Filter (73) - +- Window (72) - +- WindowGroupLimit (71) - +- * CometColumnarToRow (70) - +- CometSort (69) - +- CometColumnarExchange (68) - +- WindowGroupLimit (67) - +- * CometColumnarToRow (66) - +- CometSort (65) - +- CometUnion (64) - :- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometHashAggregate (25) - : +- ReusedExchange (24) - :- CometHashAggregate (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- CometHashAggregate (30) - : +- ReusedExchange (29) - :- CometHashAggregate (38) - : +- CometExchange (37) - : +- CometHashAggregate (36) - : +- CometHashAggregate (35) - : +- ReusedExchange (34) - :- CometHashAggregate (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometHashAggregate (40) - : +- ReusedExchange (39) - :- CometHashAggregate (48) - : +- CometExchange (47) - : +- CometHashAggregate (46) - : +- CometHashAggregate (45) - : +- ReusedExchange (44) - :- CometHashAggregate (53) - : +- CometExchange (52) - : +- CometHashAggregate (51) - : +- CometHashAggregate (50) - : +- ReusedExchange (49) - :- CometHashAggregate (58) - : +- CometExchange (57) - : +- CometHashAggregate (56) - : +- CometHashAggregate (55) - : +- ReusedExchange (54) - +- CometHashAggregate (63) - +- CometExchange (62) - +- CometHashAggregate (61) - +- CometHashAggregate (60) - +- ReusedExchange (59) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(6) CometBroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Right output [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11], [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_store_id#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#12, s_store_id#13] -Condition : isnotnull(s_store_sk#12) - -(11) CometProject -Input [2]: [s_store_sk#12, s_store_id#13] -Arguments: [s_store_sk#12, s_store_id#14], [s_store_sk#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#13, 16)) AS s_store_id#14] - -(12) CometBroadcastExchange -Input [2]: [s_store_sk#12, s_store_id#14] -Arguments: [s_store_sk#12, s_store_id#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] -Right output [2]: [s_store_sk#12, s_store_id#14] -Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_sk#12, s_store_id#14] -Arguments: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14], [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Condition : isnotnull(i_item_sk#15) - -(17) CometProject -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23], [i_item_sk#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#16, 50)) AS i_brand#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#17, 50)) AS i_class#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#18, 50)) AS i_category#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#19, 50)) AS i_product_name#23] - -(18) CometBroadcastExchange -Input [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] -Right output [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight - -(20) CometProject -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23], [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23] - -(21) CometHashAggregate -Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(22) CometExchange -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#24, isEmpty#25] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#24, isEmpty#25] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(24) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#26, i_class#27, i_brand#28, i_product_name#29, d_year#30, d_qoy#31, d_moy#32, s_store_id#33, sum#34, isEmpty#35] - -(25) CometHashAggregate -Input [10]: [i_category#26, i_class#27, i_brand#28, i_product_name#29, d_year#30, d_qoy#31, d_moy#32, s_store_id#33, sum#34, isEmpty#35] -Keys [8]: [i_category#26, i_class#27, i_brand#28, i_product_name#29, d_year#30, d_qoy#31, d_moy#32, s_store_id#33] -Functions [1]: [sum(coalesce((ss_sales_price#36 * cast(ss_quantity#37 as decimal(10,0))), 0.00))] - -(26) CometHashAggregate -Input [8]: [i_category#26, i_class#27, i_brand#28, i_product_name#29, d_year#30, d_qoy#31, d_moy#32, sumsales#38] -Keys [7]: [i_category#26, i_class#27, i_brand#28, i_product_name#29, d_year#30, d_qoy#31, d_moy#32] -Functions [1]: [partial_sum(sumsales#38)] - -(27) CometExchange -Input [9]: [i_category#26, i_class#27, i_brand#28, i_product_name#29, d_year#30, d_qoy#31, d_moy#32, sum#39, isEmpty#40] -Arguments: hashpartitioning(i_category#26, i_class#27, i_brand#28, i_product_name#29, d_year#30, d_qoy#31, d_moy#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [9]: [i_category#26, i_class#27, i_brand#28, i_product_name#29, d_year#30, d_qoy#31, d_moy#32, sum#39, isEmpty#40] -Keys [7]: [i_category#26, i_class#27, i_brand#28, i_product_name#29, d_year#30, d_qoy#31, d_moy#32] -Functions [1]: [sum(sumsales#38)] - -(29) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#41, i_class#42, i_brand#43, i_product_name#44, d_year#45, d_qoy#46, d_moy#47, s_store_id#48, sum#49, isEmpty#50] - -(30) CometHashAggregate -Input [10]: [i_category#41, i_class#42, i_brand#43, i_product_name#44, d_year#45, d_qoy#46, d_moy#47, s_store_id#48, sum#49, isEmpty#50] -Keys [8]: [i_category#41, i_class#42, i_brand#43, i_product_name#44, d_year#45, d_qoy#46, d_moy#47, s_store_id#48] -Functions [1]: [sum(coalesce((ss_sales_price#51 * cast(ss_quantity#52 as decimal(10,0))), 0.00))] - -(31) CometHashAggregate -Input [7]: [i_category#41, i_class#42, i_brand#43, i_product_name#44, d_year#45, d_qoy#46, sumsales#53] -Keys [6]: [i_category#41, i_class#42, i_brand#43, i_product_name#44, d_year#45, d_qoy#46] -Functions [1]: [partial_sum(sumsales#53)] - -(32) CometExchange -Input [8]: [i_category#41, i_class#42, i_brand#43, i_product_name#44, d_year#45, d_qoy#46, sum#54, isEmpty#55] -Arguments: hashpartitioning(i_category#41, i_class#42, i_brand#43, i_product_name#44, d_year#45, d_qoy#46, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometHashAggregate -Input [8]: [i_category#41, i_class#42, i_brand#43, i_product_name#44, d_year#45, d_qoy#46, sum#54, isEmpty#55] -Keys [6]: [i_category#41, i_class#42, i_brand#43, i_product_name#44, d_year#45, d_qoy#46] -Functions [1]: [sum(sumsales#53)] - -(34) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#56, i_class#57, i_brand#58, i_product_name#59, d_year#60, d_qoy#61, d_moy#62, s_store_id#63, sum#64, isEmpty#65] - -(35) CometHashAggregate -Input [10]: [i_category#56, i_class#57, i_brand#58, i_product_name#59, d_year#60, d_qoy#61, d_moy#62, s_store_id#63, sum#64, isEmpty#65] -Keys [8]: [i_category#56, i_class#57, i_brand#58, i_product_name#59, d_year#60, d_qoy#61, d_moy#62, s_store_id#63] -Functions [1]: [sum(coalesce((ss_sales_price#66 * cast(ss_quantity#67 as decimal(10,0))), 0.00))] - -(36) CometHashAggregate -Input [6]: [i_category#56, i_class#57, i_brand#58, i_product_name#59, d_year#60, sumsales#68] -Keys [5]: [i_category#56, i_class#57, i_brand#58, i_product_name#59, d_year#60] -Functions [1]: [partial_sum(sumsales#68)] - -(37) CometExchange -Input [7]: [i_category#56, i_class#57, i_brand#58, i_product_name#59, d_year#60, sum#69, isEmpty#70] -Arguments: hashpartitioning(i_category#56, i_class#57, i_brand#58, i_product_name#59, d_year#60, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(38) CometHashAggregate -Input [7]: [i_category#56, i_class#57, i_brand#58, i_product_name#59, d_year#60, sum#69, isEmpty#70] -Keys [5]: [i_category#56, i_class#57, i_brand#58, i_product_name#59, d_year#60] -Functions [1]: [sum(sumsales#68)] - -(39) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, d_year#75, d_qoy#76, d_moy#77, s_store_id#78, sum#79, isEmpty#80] - -(40) CometHashAggregate -Input [10]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, d_year#75, d_qoy#76, d_moy#77, s_store_id#78, sum#79, isEmpty#80] -Keys [8]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, d_year#75, d_qoy#76, d_moy#77, s_store_id#78] -Functions [1]: [sum(coalesce((ss_sales_price#81 * cast(ss_quantity#82 as decimal(10,0))), 0.00))] - -(41) CometHashAggregate -Input [5]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, sumsales#83] -Keys [4]: [i_category#71, i_class#72, i_brand#73, i_product_name#74] -Functions [1]: [partial_sum(sumsales#83)] - -(42) CometExchange -Input [6]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, sum#84, isEmpty#85] -Arguments: hashpartitioning(i_category#71, i_class#72, i_brand#73, i_product_name#74, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(43) CometHashAggregate -Input [6]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, sum#84, isEmpty#85] -Keys [4]: [i_category#71, i_class#72, i_brand#73, i_product_name#74] -Functions [1]: [sum(sumsales#83)] - -(44) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#86, i_class#87, i_brand#88, i_product_name#89, d_year#90, d_qoy#91, d_moy#92, s_store_id#93, sum#94, isEmpty#95] - -(45) CometHashAggregate -Input [10]: [i_category#86, i_class#87, i_brand#88, i_product_name#89, d_year#90, d_qoy#91, d_moy#92, s_store_id#93, sum#94, isEmpty#95] -Keys [8]: [i_category#86, i_class#87, i_brand#88, i_product_name#89, d_year#90, d_qoy#91, d_moy#92, s_store_id#93] -Functions [1]: [sum(coalesce((ss_sales_price#96 * cast(ss_quantity#97 as decimal(10,0))), 0.00))] - -(46) CometHashAggregate -Input [4]: [i_category#86, i_class#87, i_brand#88, sumsales#98] -Keys [3]: [i_category#86, i_class#87, i_brand#88] -Functions [1]: [partial_sum(sumsales#98)] - -(47) CometExchange -Input [5]: [i_category#86, i_class#87, i_brand#88, sum#99, isEmpty#100] -Arguments: hashpartitioning(i_category#86, i_class#87, i_brand#88, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(48) CometHashAggregate -Input [5]: [i_category#86, i_class#87, i_brand#88, sum#99, isEmpty#100] -Keys [3]: [i_category#86, i_class#87, i_brand#88] -Functions [1]: [sum(sumsales#98)] - -(49) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#101, i_class#102, i_brand#103, i_product_name#104, d_year#105, d_qoy#106, d_moy#107, s_store_id#108, sum#109, isEmpty#110] - -(50) CometHashAggregate -Input [10]: [i_category#101, i_class#102, i_brand#103, i_product_name#104, d_year#105, d_qoy#106, d_moy#107, s_store_id#108, sum#109, isEmpty#110] -Keys [8]: [i_category#101, i_class#102, i_brand#103, i_product_name#104, d_year#105, d_qoy#106, d_moy#107, s_store_id#108] -Functions [1]: [sum(coalesce((ss_sales_price#111 * cast(ss_quantity#112 as decimal(10,0))), 0.00))] - -(51) CometHashAggregate -Input [3]: [i_category#101, i_class#102, sumsales#113] -Keys [2]: [i_category#101, i_class#102] -Functions [1]: [partial_sum(sumsales#113)] - -(52) CometExchange -Input [4]: [i_category#101, i_class#102, sum#114, isEmpty#115] -Arguments: hashpartitioning(i_category#101, i_class#102, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(53) CometHashAggregate -Input [4]: [i_category#101, i_class#102, sum#114, isEmpty#115] -Keys [2]: [i_category#101, i_class#102] -Functions [1]: [sum(sumsales#113)] - -(54) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#116, i_class#117, i_brand#118, i_product_name#119, d_year#120, d_qoy#121, d_moy#122, s_store_id#123, sum#124, isEmpty#125] - -(55) CometHashAggregate -Input [10]: [i_category#116, i_class#117, i_brand#118, i_product_name#119, d_year#120, d_qoy#121, d_moy#122, s_store_id#123, sum#124, isEmpty#125] -Keys [8]: [i_category#116, i_class#117, i_brand#118, i_product_name#119, d_year#120, d_qoy#121, d_moy#122, s_store_id#123] -Functions [1]: [sum(coalesce((ss_sales_price#126 * cast(ss_quantity#127 as decimal(10,0))), 0.00))] - -(56) CometHashAggregate -Input [2]: [i_category#116, sumsales#128] -Keys [1]: [i_category#116] -Functions [1]: [partial_sum(sumsales#128)] - -(57) CometExchange -Input [3]: [i_category#116, sum#129, isEmpty#130] -Arguments: hashpartitioning(i_category#116, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(58) CometHashAggregate -Input [3]: [i_category#116, sum#129, isEmpty#130] -Keys [1]: [i_category#116] -Functions [1]: [sum(sumsales#128)] - -(59) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#131, i_class#132, i_brand#133, i_product_name#134, d_year#135, d_qoy#136, d_moy#137, s_store_id#138, sum#139, isEmpty#140] - -(60) CometHashAggregate -Input [10]: [i_category#131, i_class#132, i_brand#133, i_product_name#134, d_year#135, d_qoy#136, d_moy#137, s_store_id#138, sum#139, isEmpty#140] -Keys [8]: [i_category#131, i_class#132, i_brand#133, i_product_name#134, d_year#135, d_qoy#136, d_moy#137, s_store_id#138] -Functions [1]: [sum(coalesce((ss_sales_price#141 * cast(ss_quantity#142 as decimal(10,0))), 0.00))] - -(61) CometHashAggregate -Input [1]: [sumsales#143] -Keys: [] -Functions [1]: [partial_sum(sumsales#143)] - -(62) CometExchange -Input [2]: [sum#144, isEmpty#145] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(63) CometHashAggregate -Input [2]: [sum#144, isEmpty#145] -Keys: [] -Functions [1]: [sum(sumsales#143)] - -(64) CometUnion -Child 0 Input [9]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154] -Child 1 Input [9]: [i_category#26, i_class#27, i_brand#28, i_product_name#29, d_year#30, d_qoy#31, d_moy#32, s_store_id#155, sumsales#156] -Child 2 Input [9]: [i_category#41, i_class#42, i_brand#43, i_product_name#44, d_year#45, d_qoy#46, d_moy#157, s_store_id#158, sumsales#159] -Child 3 Input [9]: [i_category#56, i_class#57, i_brand#58, i_product_name#59, d_year#60, d_qoy#160, d_moy#161, s_store_id#162, sumsales#163] -Child 4 Input [9]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, d_year#164, d_qoy#165, d_moy#166, s_store_id#167, sumsales#168] -Child 5 Input [9]: [i_category#86, i_class#87, i_brand#88, i_product_name#169, d_year#170, d_qoy#171, d_moy#172, s_store_id#173, sumsales#174] -Child 6 Input [9]: [i_category#101, i_class#102, i_brand#175, i_product_name#176, d_year#177, d_qoy#178, d_moy#179, s_store_id#180, sumsales#181] -Child 7 Input [9]: [i_category#116, i_class#182, i_brand#183, i_product_name#184, d_year#185, d_qoy#186, d_moy#187, s_store_id#188, sumsales#189] -Child 8 Input [9]: [i_category#190, i_class#191, i_brand#192, i_product_name#193, d_year#194, d_qoy#195, d_moy#196, s_store_id#197, sumsales#198] - -(65) CometSort -Input [9]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154] -Arguments: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154], [i_category#146 ASC NULLS FIRST, sumsales#154 DESC NULLS LAST] - -(66) CometColumnarToRow [codegen id : 1] -Input [9]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154] - -(67) WindowGroupLimit -Input [9]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154] -Arguments: [i_category#146], [sumsales#154 DESC NULLS LAST], rank(sumsales#154), 100, Partial - -(68) CometColumnarExchange -Input [9]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154] -Arguments: hashpartitioning(i_category#146, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(69) CometSort -Input [9]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154] -Arguments: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154], [i_category#146 ASC NULLS FIRST, sumsales#154 DESC NULLS LAST] - -(70) CometColumnarToRow [codegen id : 2] -Input [9]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154] - -(71) WindowGroupLimit -Input [9]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154] -Arguments: [i_category#146], [sumsales#154 DESC NULLS LAST], rank(sumsales#154), 100, Final - -(72) Window -Input [9]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154] -Arguments: [rank(sumsales#154) windowspecdefinition(i_category#146, sumsales#154 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#199], [i_category#146], [sumsales#154 DESC NULLS LAST] - -(73) Filter [codegen id : 3] -Input [10]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154, rk#199] -Condition : (rk#199 <= 100) - -(74) TakeOrderedAndProject -Input [10]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154, rk#199] -Arguments: 100, [i_category#146 ASC NULLS FIRST, i_class#147 ASC NULLS FIRST, i_brand#148 ASC NULLS FIRST, i_product_name#149 ASC NULLS FIRST, d_year#150 ASC NULLS FIRST, d_qoy#151 ASC NULLS FIRST, d_moy#152 ASC NULLS FIRST, s_store_id#153 ASC NULLS FIRST, sumsales#154 ASC NULLS FIRST, rk#199 ASC NULLS FIRST], [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154, rk#199] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (79) -+- * CometColumnarToRow (78) - +- CometProject (77) - +- CometFilter (76) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (75) - - -(75) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(76) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) - -(77) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(78) CometColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(79) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 6096ee1c25..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,88 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (3) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometUnion [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] - 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,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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] - CometHashAggregate [s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] - CometHashAggregate [d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year] #9 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] - CometHashAggregate [d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name] #10 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,sum,isEmpty] - CometHashAggregate [d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand] #11 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,sum,isEmpty] - CometHashAggregate [i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class] #12 - CometHashAggregate [sumsales] [i_category,i_class,sum,isEmpty] - CometHashAggregate [i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category] #13 - CometHashAggregate [sumsales] [i_category,sum,isEmpty] - CometHashAggregate [i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange #14 - CometHashAggregate [sumsales] [sum,isEmpty] - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/explain.txt deleted file mode 100644 index 481b95c617..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/explain.txt +++ /dev/null @@ -1,445 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (74) -+- * Filter (73) - +- Window (72) - +- WindowGroupLimit (71) - +- * CometColumnarToRow (70) - +- CometSort (69) - +- CometColumnarExchange (68) - +- WindowGroupLimit (67) - +- * CometColumnarToRow (66) - +- CometSort (65) - +- CometUnion (64) - :- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometHashAggregate (25) - : +- ReusedExchange (24) - :- CometHashAggregate (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- CometHashAggregate (30) - : +- ReusedExchange (29) - :- CometHashAggregate (38) - : +- CometExchange (37) - : +- CometHashAggregate (36) - : +- CometHashAggregate (35) - : +- ReusedExchange (34) - :- CometHashAggregate (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometHashAggregate (40) - : +- ReusedExchange (39) - :- CometHashAggregate (48) - : +- CometExchange (47) - : +- CometHashAggregate (46) - : +- CometHashAggregate (45) - : +- ReusedExchange (44) - :- CometHashAggregate (53) - : +- CometExchange (52) - : +- CometHashAggregate (51) - : +- CometHashAggregate (50) - : +- ReusedExchange (49) - :- CometHashAggregate (58) - : +- CometExchange (57) - : +- CometHashAggregate (56) - : +- CometHashAggregate (55) - : +- ReusedExchange (54) - +- CometHashAggregate (63) - +- CometExchange (62) - +- CometHashAggregate (61) - +- CometHashAggregate (60) - +- ReusedExchange (59) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(6) CometBroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Right output [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11], [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_store_id#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#12, s_store_id#13] -Condition : isnotnull(s_store_sk#12) - -(11) CometProject -Input [2]: [s_store_sk#12, s_store_id#13] -Arguments: [s_store_sk#12, s_store_id#14], [s_store_sk#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#13, 16)) AS s_store_id#14] - -(12) CometBroadcastExchange -Input [2]: [s_store_sk#12, s_store_id#14] -Arguments: [s_store_sk#12, s_store_id#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] -Right output [2]: [s_store_sk#12, s_store_id#14] -Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_sk#12, s_store_id#14] -Arguments: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14], [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Condition : isnotnull(i_item_sk#15) - -(17) CometProject -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23], [i_item_sk#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#16, 50)) AS i_brand#20, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#17, 50)) AS i_class#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#18, 50)) AS i_category#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#19, 50)) AS i_product_name#23] - -(18) CometBroadcastExchange -Input [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] -Right output [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight - -(20) CometProject -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23], [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23] - -(21) CometHashAggregate -Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(22) CometExchange -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#24, isEmpty#25] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#24, isEmpty#25] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(24) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#26, i_class#27, i_brand#28, i_product_name#29, d_year#30, d_qoy#31, d_moy#32, s_store_id#33, sum#34, isEmpty#35] - -(25) CometHashAggregate -Input [10]: [i_category#26, i_class#27, i_brand#28, i_product_name#29, d_year#30, d_qoy#31, d_moy#32, s_store_id#33, sum#34, isEmpty#35] -Keys [8]: [i_category#26, i_class#27, i_brand#28, i_product_name#29, d_year#30, d_qoy#31, d_moy#32, s_store_id#33] -Functions [1]: [sum(coalesce((ss_sales_price#36 * cast(ss_quantity#37 as decimal(10,0))), 0.00))] - -(26) CometHashAggregate -Input [8]: [i_category#26, i_class#27, i_brand#28, i_product_name#29, d_year#30, d_qoy#31, d_moy#32, sumsales#38] -Keys [7]: [i_category#26, i_class#27, i_brand#28, i_product_name#29, d_year#30, d_qoy#31, d_moy#32] -Functions [1]: [partial_sum(sumsales#38)] - -(27) CometExchange -Input [9]: [i_category#26, i_class#27, i_brand#28, i_product_name#29, d_year#30, d_qoy#31, d_moy#32, sum#39, isEmpty#40] -Arguments: hashpartitioning(i_category#26, i_class#27, i_brand#28, i_product_name#29, d_year#30, d_qoy#31, d_moy#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [9]: [i_category#26, i_class#27, i_brand#28, i_product_name#29, d_year#30, d_qoy#31, d_moy#32, sum#39, isEmpty#40] -Keys [7]: [i_category#26, i_class#27, i_brand#28, i_product_name#29, d_year#30, d_qoy#31, d_moy#32] -Functions [1]: [sum(sumsales#38)] - -(29) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#41, i_class#42, i_brand#43, i_product_name#44, d_year#45, d_qoy#46, d_moy#47, s_store_id#48, sum#49, isEmpty#50] - -(30) CometHashAggregate -Input [10]: [i_category#41, i_class#42, i_brand#43, i_product_name#44, d_year#45, d_qoy#46, d_moy#47, s_store_id#48, sum#49, isEmpty#50] -Keys [8]: [i_category#41, i_class#42, i_brand#43, i_product_name#44, d_year#45, d_qoy#46, d_moy#47, s_store_id#48] -Functions [1]: [sum(coalesce((ss_sales_price#51 * cast(ss_quantity#52 as decimal(10,0))), 0.00))] - -(31) CometHashAggregate -Input [7]: [i_category#41, i_class#42, i_brand#43, i_product_name#44, d_year#45, d_qoy#46, sumsales#53] -Keys [6]: [i_category#41, i_class#42, i_brand#43, i_product_name#44, d_year#45, d_qoy#46] -Functions [1]: [partial_sum(sumsales#53)] - -(32) CometExchange -Input [8]: [i_category#41, i_class#42, i_brand#43, i_product_name#44, d_year#45, d_qoy#46, sum#54, isEmpty#55] -Arguments: hashpartitioning(i_category#41, i_class#42, i_brand#43, i_product_name#44, d_year#45, d_qoy#46, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometHashAggregate -Input [8]: [i_category#41, i_class#42, i_brand#43, i_product_name#44, d_year#45, d_qoy#46, sum#54, isEmpty#55] -Keys [6]: [i_category#41, i_class#42, i_brand#43, i_product_name#44, d_year#45, d_qoy#46] -Functions [1]: [sum(sumsales#53)] - -(34) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#56, i_class#57, i_brand#58, i_product_name#59, d_year#60, d_qoy#61, d_moy#62, s_store_id#63, sum#64, isEmpty#65] - -(35) CometHashAggregate -Input [10]: [i_category#56, i_class#57, i_brand#58, i_product_name#59, d_year#60, d_qoy#61, d_moy#62, s_store_id#63, sum#64, isEmpty#65] -Keys [8]: [i_category#56, i_class#57, i_brand#58, i_product_name#59, d_year#60, d_qoy#61, d_moy#62, s_store_id#63] -Functions [1]: [sum(coalesce((ss_sales_price#66 * cast(ss_quantity#67 as decimal(10,0))), 0.00))] - -(36) CometHashAggregate -Input [6]: [i_category#56, i_class#57, i_brand#58, i_product_name#59, d_year#60, sumsales#68] -Keys [5]: [i_category#56, i_class#57, i_brand#58, i_product_name#59, d_year#60] -Functions [1]: [partial_sum(sumsales#68)] - -(37) CometExchange -Input [7]: [i_category#56, i_class#57, i_brand#58, i_product_name#59, d_year#60, sum#69, isEmpty#70] -Arguments: hashpartitioning(i_category#56, i_class#57, i_brand#58, i_product_name#59, d_year#60, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(38) CometHashAggregate -Input [7]: [i_category#56, i_class#57, i_brand#58, i_product_name#59, d_year#60, sum#69, isEmpty#70] -Keys [5]: [i_category#56, i_class#57, i_brand#58, i_product_name#59, d_year#60] -Functions [1]: [sum(sumsales#68)] - -(39) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, d_year#75, d_qoy#76, d_moy#77, s_store_id#78, sum#79, isEmpty#80] - -(40) CometHashAggregate -Input [10]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, d_year#75, d_qoy#76, d_moy#77, s_store_id#78, sum#79, isEmpty#80] -Keys [8]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, d_year#75, d_qoy#76, d_moy#77, s_store_id#78] -Functions [1]: [sum(coalesce((ss_sales_price#81 * cast(ss_quantity#82 as decimal(10,0))), 0.00))] - -(41) CometHashAggregate -Input [5]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, sumsales#83] -Keys [4]: [i_category#71, i_class#72, i_brand#73, i_product_name#74] -Functions [1]: [partial_sum(sumsales#83)] - -(42) CometExchange -Input [6]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, sum#84, isEmpty#85] -Arguments: hashpartitioning(i_category#71, i_class#72, i_brand#73, i_product_name#74, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(43) CometHashAggregate -Input [6]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, sum#84, isEmpty#85] -Keys [4]: [i_category#71, i_class#72, i_brand#73, i_product_name#74] -Functions [1]: [sum(sumsales#83)] - -(44) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#86, i_class#87, i_brand#88, i_product_name#89, d_year#90, d_qoy#91, d_moy#92, s_store_id#93, sum#94, isEmpty#95] - -(45) CometHashAggregate -Input [10]: [i_category#86, i_class#87, i_brand#88, i_product_name#89, d_year#90, d_qoy#91, d_moy#92, s_store_id#93, sum#94, isEmpty#95] -Keys [8]: [i_category#86, i_class#87, i_brand#88, i_product_name#89, d_year#90, d_qoy#91, d_moy#92, s_store_id#93] -Functions [1]: [sum(coalesce((ss_sales_price#96 * cast(ss_quantity#97 as decimal(10,0))), 0.00))] - -(46) CometHashAggregate -Input [4]: [i_category#86, i_class#87, i_brand#88, sumsales#98] -Keys [3]: [i_category#86, i_class#87, i_brand#88] -Functions [1]: [partial_sum(sumsales#98)] - -(47) CometExchange -Input [5]: [i_category#86, i_class#87, i_brand#88, sum#99, isEmpty#100] -Arguments: hashpartitioning(i_category#86, i_class#87, i_brand#88, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(48) CometHashAggregate -Input [5]: [i_category#86, i_class#87, i_brand#88, sum#99, isEmpty#100] -Keys [3]: [i_category#86, i_class#87, i_brand#88] -Functions [1]: [sum(sumsales#98)] - -(49) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#101, i_class#102, i_brand#103, i_product_name#104, d_year#105, d_qoy#106, d_moy#107, s_store_id#108, sum#109, isEmpty#110] - -(50) CometHashAggregate -Input [10]: [i_category#101, i_class#102, i_brand#103, i_product_name#104, d_year#105, d_qoy#106, d_moy#107, s_store_id#108, sum#109, isEmpty#110] -Keys [8]: [i_category#101, i_class#102, i_brand#103, i_product_name#104, d_year#105, d_qoy#106, d_moy#107, s_store_id#108] -Functions [1]: [sum(coalesce((ss_sales_price#111 * cast(ss_quantity#112 as decimal(10,0))), 0.00))] - -(51) CometHashAggregate -Input [3]: [i_category#101, i_class#102, sumsales#113] -Keys [2]: [i_category#101, i_class#102] -Functions [1]: [partial_sum(sumsales#113)] - -(52) CometExchange -Input [4]: [i_category#101, i_class#102, sum#114, isEmpty#115] -Arguments: hashpartitioning(i_category#101, i_class#102, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(53) CometHashAggregate -Input [4]: [i_category#101, i_class#102, sum#114, isEmpty#115] -Keys [2]: [i_category#101, i_class#102] -Functions [1]: [sum(sumsales#113)] - -(54) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#116, i_class#117, i_brand#118, i_product_name#119, d_year#120, d_qoy#121, d_moy#122, s_store_id#123, sum#124, isEmpty#125] - -(55) CometHashAggregate -Input [10]: [i_category#116, i_class#117, i_brand#118, i_product_name#119, d_year#120, d_qoy#121, d_moy#122, s_store_id#123, sum#124, isEmpty#125] -Keys [8]: [i_category#116, i_class#117, i_brand#118, i_product_name#119, d_year#120, d_qoy#121, d_moy#122, s_store_id#123] -Functions [1]: [sum(coalesce((ss_sales_price#126 * cast(ss_quantity#127 as decimal(10,0))), 0.00))] - -(56) CometHashAggregate -Input [2]: [i_category#116, sumsales#128] -Keys [1]: [i_category#116] -Functions [1]: [partial_sum(sumsales#128)] - -(57) CometExchange -Input [3]: [i_category#116, sum#129, isEmpty#130] -Arguments: hashpartitioning(i_category#116, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(58) CometHashAggregate -Input [3]: [i_category#116, sum#129, isEmpty#130] -Keys [1]: [i_category#116] -Functions [1]: [sum(sumsales#128)] - -(59) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#131, i_class#132, i_brand#133, i_product_name#134, d_year#135, d_qoy#136, d_moy#137, s_store_id#138, sum#139, isEmpty#140] - -(60) CometHashAggregate -Input [10]: [i_category#131, i_class#132, i_brand#133, i_product_name#134, d_year#135, d_qoy#136, d_moy#137, s_store_id#138, sum#139, isEmpty#140] -Keys [8]: [i_category#131, i_class#132, i_brand#133, i_product_name#134, d_year#135, d_qoy#136, d_moy#137, s_store_id#138] -Functions [1]: [sum(coalesce((ss_sales_price#141 * cast(ss_quantity#142 as decimal(10,0))), 0.00))] - -(61) CometHashAggregate -Input [1]: [sumsales#143] -Keys: [] -Functions [1]: [partial_sum(sumsales#143)] - -(62) CometExchange -Input [2]: [sum#144, isEmpty#145] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(63) CometHashAggregate -Input [2]: [sum#144, isEmpty#145] -Keys: [] -Functions [1]: [sum(sumsales#143)] - -(64) CometUnion -Child 0 Input [9]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154] -Child 1 Input [9]: [i_category#26, i_class#27, i_brand#28, i_product_name#29, d_year#30, d_qoy#31, d_moy#32, s_store_id#155, sumsales#156] -Child 2 Input [9]: [i_category#41, i_class#42, i_brand#43, i_product_name#44, d_year#45, d_qoy#46, d_moy#157, s_store_id#158, sumsales#159] -Child 3 Input [9]: [i_category#56, i_class#57, i_brand#58, i_product_name#59, d_year#60, d_qoy#160, d_moy#161, s_store_id#162, sumsales#163] -Child 4 Input [9]: [i_category#71, i_class#72, i_brand#73, i_product_name#74, d_year#164, d_qoy#165, d_moy#166, s_store_id#167, sumsales#168] -Child 5 Input [9]: [i_category#86, i_class#87, i_brand#88, i_product_name#169, d_year#170, d_qoy#171, d_moy#172, s_store_id#173, sumsales#174] -Child 6 Input [9]: [i_category#101, i_class#102, i_brand#175, i_product_name#176, d_year#177, d_qoy#178, d_moy#179, s_store_id#180, sumsales#181] -Child 7 Input [9]: [i_category#116, i_class#182, i_brand#183, i_product_name#184, d_year#185, d_qoy#186, d_moy#187, s_store_id#188, sumsales#189] -Child 8 Input [9]: [i_category#190, i_class#191, i_brand#192, i_product_name#193, d_year#194, d_qoy#195, d_moy#196, s_store_id#197, sumsales#198] - -(65) CometSort -Input [9]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154] -Arguments: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154], [i_category#146 ASC NULLS FIRST, sumsales#154 DESC NULLS LAST] - -(66) CometColumnarToRow [codegen id : 1] -Input [9]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154] - -(67) WindowGroupLimit -Input [9]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154] -Arguments: [i_category#146], [sumsales#154 DESC NULLS LAST], rank(sumsales#154), 100, Partial - -(68) CometColumnarExchange -Input [9]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154] -Arguments: hashpartitioning(i_category#146, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(69) CometSort -Input [9]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154] -Arguments: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154], [i_category#146 ASC NULLS FIRST, sumsales#154 DESC NULLS LAST] - -(70) CometColumnarToRow [codegen id : 2] -Input [9]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154] - -(71) WindowGroupLimit -Input [9]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154] -Arguments: [i_category#146], [sumsales#154 DESC NULLS LAST], rank(sumsales#154), 100, Final - -(72) Window -Input [9]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154] -Arguments: [rank(sumsales#154) windowspecdefinition(i_category#146, sumsales#154 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#199], [i_category#146], [sumsales#154 DESC NULLS LAST] - -(73) Filter [codegen id : 3] -Input [10]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154, rk#199] -Condition : (rk#199 <= 100) - -(74) TakeOrderedAndProject -Input [10]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154, rk#199] -Arguments: 100, [i_category#146 ASC NULLS FIRST, i_class#147 ASC NULLS FIRST, i_brand#148 ASC NULLS FIRST, i_product_name#149 ASC NULLS FIRST, d_year#150 ASC NULLS FIRST, d_qoy#151 ASC NULLS FIRST, d_moy#152 ASC NULLS FIRST, s_store_id#153 ASC NULLS FIRST, sumsales#154 ASC NULLS FIRST, rk#199 ASC NULLS FIRST], [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sumsales#154, rk#199] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (79) -+- * CometColumnarToRow (78) - +- CometProject (77) - +- CometFilter (76) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (75) - - -(75) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(76) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) - -(77) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(78) CometColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(79) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/extended.txt deleted file mode 100644 index b96ce25ae1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/extended.txt +++ /dev/null @@ -1,298 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 262 out of 285 eligible operators (91%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 6096ee1c25..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt +++ /dev/null @@ -1,88 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (3) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - WindowGroupLimit [i_category,sumsales] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometUnion [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] - 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,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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] - CometHashAggregate [s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] - CometHashAggregate [d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year] #9 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] - CometHashAggregate [d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name] #10 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,sum,isEmpty] - CometHashAggregate [d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand] #11 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,sum,isEmpty] - CometHashAggregate [i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class] #12 - CometHashAggregate [sumsales] [i_category,i_class,sum,isEmpty] - CometHashAggregate [i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category] #13 - CometHashAggregate [sumsales] [i_category,sum,isEmpty] - CometHashAggregate [i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange #14 - CometHashAggregate [sumsales] [sum,isEmpty] - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/explain.txt deleted file mode 100644 index b203fef099..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/explain.txt +++ /dev/null @@ -1,404 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (65) -+- * Project (64) - +- Window (63) - +- * CometColumnarToRow (62) - +- CometSort (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometColumnarExchange (58) - +- * HashAggregate (57) - +- Union (56) - :- * HashAggregate (41) - : +- * CometColumnarToRow (40) - : +- CometColumnarExchange (39) - : +- * HashAggregate (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- * BroadcastHashJoin LeftSemi BuildRight (33) - : :- * CometColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (32) - : +- * Project (31) - : +- * Filter (30) - : +- Window (29) - : +- WindowGroupLimit (28) - : +- * Sort (27) - : +- * HashAggregate (26) - : +- * CometColumnarToRow (25) - : +- CometColumnarExchange (24) - : +- * HashAggregate (23) - : +- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet spark_catalog.default.store_sales (10) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometProject (15) - : : +- CometFilter (14) - : : +- CometNativeScan parquet spark_catalog.default.store (13) - : +- ReusedExchange (20) - :- * HashAggregate (48) - : +- * CometColumnarToRow (47) - : +- CometColumnarExchange (46) - : +- * HashAggregate (45) - : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- ReusedExchange (42) - +- * HashAggregate (55) - +- * CometColumnarToRow (54) - +- CometColumnarExchange (53) - +- * HashAggregate (52) - +- * HashAggregate (51) - +- * CometColumnarToRow (50) - +- ReusedExchange (49) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 8] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] - -(3) Filter [codegen id : 8] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(4) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#5] - -(5) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 8] -Output [2]: [ss_store_sk#1, ss_net_profit#2] -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#6, s_county#7, s_state#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Condition : isnotnull(s_store_sk#6) - -(9) CometColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#6, s_county#7, s_state#8] - -(10) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] - -(12) Filter [codegen id : 4] -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_store_sk#9) - -(13) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_state#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [s_store_sk#12, s_state#13] -Condition : isnotnull(s_store_sk#12) - -(15) CometProject -Input [2]: [s_store_sk#12, s_state#13] -Arguments: [s_store_sk#12, s_state#14], [s_store_sk#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#13, 2)) AS s_state#14] - -(16) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#12, s_state#14] - -(17) BroadcastExchange -Input [2]: [s_store_sk#12, s_state#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#9] -Right keys [1]: [s_store_sk#12] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14] -Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#12, s_state#14] - -(20) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#15] - -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 4] -Output [2]: [ss_net_profit#10, s_state#14] -Input [4]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14, d_date_sk#15] - -(23) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#10, s_state#14] -Keys [1]: [s_state#14] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum#16] -Results [2]: [s_state#14, sum#17] - -(24) CometColumnarExchange -Input [2]: [s_state#14, sum#17] -Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(25) CometColumnarToRow [codegen id : 5] -Input [2]: [s_state#14, sum#17] - -(26) HashAggregate [codegen id : 5] -Input [2]: [s_state#14, sum#17] -Keys [1]: [s_state#14] -Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#18] -Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#18,17,2) AS _w0#19, s_state#14] - -(27) Sort [codegen id : 5] -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 - -(28) WindowGroupLimit -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [s_state#14], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final - -(29) Window -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#14, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#14], [_w0#19 DESC NULLS LAST] - -(30) Filter [codegen id : 6] -Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] -Condition : (ranking#20 <= 5) - -(31) Project [codegen id : 6] -Output [1]: [s_state#14] -Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] - -(32) BroadcastExchange -Input [1]: [s_state#14] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(33) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#8, 2))] -Right keys [1]: [s_state#14] -Join type: LeftSemi -Join condition: None - -(34) Project [codegen id : 7] -Output [3]: [s_store_sk#6, s_county#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#8, 2)) AS s_state#21] -Input [3]: [s_store_sk#6, s_county#7, s_state#8] - -(35) BroadcastExchange -Input [3]: [s_store_sk#6, s_county#7, s_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(36) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#6] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 8] -Output [3]: [ss_net_profit#2, s_county#7, s_state#21] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#21] - -(38) HashAggregate [codegen id : 8] -Input [3]: [ss_net_profit#2, s_county#7, s_state#21] -Keys [2]: [s_state#21, s_county#7] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [s_state#21, s_county#7, sum#23] - -(39) CometColumnarExchange -Input [3]: [s_state#21, s_county#7, sum#23] -Arguments: hashpartitioning(s_state#21, s_county#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(40) CometColumnarToRow [codegen id : 9] -Input [3]: [s_state#21, s_county#7, sum#23] - -(41) HashAggregate [codegen id : 9] -Input [3]: [s_state#21, s_county#7, sum#23] -Keys [2]: [s_state#21, s_county#7] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) as decimal(27,2)) AS total_sum#25, s_state#21 AS s_state#26, s_county#7 AS s_county#27, 0 AS g_state#28, 0 AS g_county#29, 0 AS lochierarchy#30] - -(42) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_state#31, s_county#32, sum#33] - -(43) CometColumnarToRow [codegen id : 18] -Input [3]: [s_state#31, s_county#32, sum#33] - -(44) HashAggregate [codegen id : 18] -Input [3]: [s_state#31, s_county#32, sum#33] -Keys [2]: [s_state#31, s_county#32] -Functions [1]: [sum(UnscaledValue(ss_net_profit#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#34))#24] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#34))#24,17,2) AS total_sum#35, s_state#31] - -(45) HashAggregate [codegen id : 18] -Input [2]: [total_sum#35, s_state#31] -Keys [1]: [s_state#31] -Functions [1]: [partial_sum(total_sum#35)] -Aggregate Attributes [2]: [sum#36, isEmpty#37] -Results [3]: [s_state#31, sum#38, isEmpty#39] - -(46) CometColumnarExchange -Input [3]: [s_state#31, sum#38, isEmpty#39] -Arguments: hashpartitioning(s_state#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(47) CometColumnarToRow [codegen id : 19] -Input [3]: [s_state#31, sum#38, isEmpty#39] - -(48) HashAggregate [codegen id : 19] -Input [3]: [s_state#31, sum#38, isEmpty#39] -Keys [1]: [s_state#31] -Functions [1]: [sum(total_sum#35)] -Aggregate Attributes [1]: [sum(total_sum#35)#40] -Results [6]: [sum(total_sum#35)#40 AS total_sum#41, s_state#31, null AS s_county#42, 0 AS g_state#43, 1 AS g_county#44, 1 AS lochierarchy#45] - -(49) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_state#46, s_county#47, sum#48] - -(50) CometColumnarToRow [codegen id : 28] -Input [3]: [s_state#46, s_county#47, sum#48] - -(51) HashAggregate [codegen id : 28] -Input [3]: [s_state#46, s_county#47, sum#48] -Keys [2]: [s_state#46, s_county#47] -Functions [1]: [sum(UnscaledValue(ss_net_profit#49))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#49))#24] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#49))#24,17,2) AS total_sum#50] - -(52) HashAggregate [codegen id : 28] -Input [1]: [total_sum#50] -Keys: [] -Functions [1]: [partial_sum(total_sum#50)] -Aggregate Attributes [2]: [sum#51, isEmpty#52] -Results [2]: [sum#53, isEmpty#54] - -(53) CometColumnarExchange -Input [2]: [sum#53, isEmpty#54] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(54) CometColumnarToRow [codegen id : 29] -Input [2]: [sum#53, isEmpty#54] - -(55) HashAggregate [codegen id : 29] -Input [2]: [sum#53, isEmpty#54] -Keys: [] -Functions [1]: [sum(total_sum#50)] -Aggregate Attributes [1]: [sum(total_sum#50)#55] -Results [6]: [sum(total_sum#50)#55 AS total_sum#56, null AS s_state#57, null AS s_county#58, 1 AS g_state#59, 1 AS g_county#60, 2 AS lochierarchy#61] - -(56) Union - -(57) HashAggregate [codegen id : 30] -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Functions: [] -Aggregate Attributes: [] -Results [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] - -(58) CometColumnarExchange -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Arguments: hashpartitioning(total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(59) CometHashAggregate -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Functions: [] - -(60) CometExchange -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] -Arguments: hashpartitioning(lochierarchy#30, _w0#62, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(61) CometSort -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] -Arguments: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62], [lochierarchy#30 ASC NULLS FIRST, _w0#62 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] - -(62) CometColumnarToRow [codegen id : 31] -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] - -(63) Window -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] -Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#30, _w0#62, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#63], [lochierarchy#30, _w0#62], [total_sum#25 DESC NULLS LAST] - -(64) Project [codegen id : 32] -Output [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#63] -Input [6]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62, rank_within_parent#63] - -(65) TakeOrderedAndProject -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#63] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#63 ASC NULLS FIRST], [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#63] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (70) -+- * CometColumnarToRow (69) - +- CometProject (68) - +- CometFilter (67) - +- CometNativeScan parquet spark_catalog.default.date_dim (66) - - -(66) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(67) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#64] -Condition : (((isnotnull(d_month_seq#64) AND (d_month_seq#64 >= 1212)) AND (d_month_seq#64 <= 1223)) AND isnotnull(d_date_sk#5)) - -(68) CometProject -Input [2]: [d_date_sk#5, d_month_seq#64] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(69) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(70) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/simplified.txt deleted file mode 100644 index ba40c1d591..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/simplified.txt +++ /dev/null @@ -1,107 +0,0 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (32) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (31) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] - CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (30) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (9) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - WholeStageCodegen (8) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (5) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #7 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (19) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #9 - WholeStageCodegen (18) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (29) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (28) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/explain.txt deleted file mode 100644 index a66acf75ca..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,408 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (66) -+- * Project (65) - +- Window (64) - +- * CometColumnarToRow (63) - +- CometSort (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometColumnarExchange (59) - +- * HashAggregate (58) - +- Union (57) - :- * HashAggregate (42) - : +- * CometColumnarToRow (41) - : +- CometColumnarExchange (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * CometColumnarToRow (9) - : : +- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- BroadcastExchange (36) - : +- * Project (35) - : +- * BroadcastHashJoin LeftSemi BuildRight (34) - : :- * CometColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) - : +- BroadcastExchange (33) - : +- * Project (32) - : +- * Filter (31) - : +- Window (30) - : +- WindowGroupLimit (29) - : +- * CometColumnarToRow (28) - : +- CometSort (27) - : +- CometHashAggregate (26) - : +- CometExchange (25) - : +- CometHashAggregate (24) - : +- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (20) - : : +- CometBroadcastHashJoin (19) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - : : +- CometBroadcastExchange (18) - : : +- CometProject (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - : +- ReusedExchange (21) - :- * HashAggregate (49) - : +- * CometColumnarToRow (48) - : +- CometColumnarExchange (47) - : +- * HashAggregate (46) - : +- * HashAggregate (45) - : +- * CometColumnarToRow (44) - : +- ReusedExchange (43) - +- * HashAggregate (56) - +- * CometColumnarToRow (55) - +- CometColumnarExchange (54) - +- * HashAggregate (53) - +- * HashAggregate (52) - +- * CometColumnarToRow (51) - +- ReusedExchange (50) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] - -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ss_store_sk#1, ss_net_profit#2] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#7, s_county#8, s_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Condition : isnotnull(s_store_sk#7) - -(12) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_store_sk#10) - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : isnotnull(s_store_sk#14) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14, s_state#16], [s_store_sk#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#15, 2)) AS s_state#16] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#14, s_state#16] -Arguments: [s_store_sk#14, s_state#16] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Right output [2]: [s_store_sk#14, s_state#16] -Arguments: [ss_store_sk#10], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#16] -Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16], [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#17] - -(22) CometBroadcastHashJoin -Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] -Right output [1]: [d_date_sk#17] -Arguments: [ss_sold_date_sk#12], [d_date_sk#17], Inner, BuildRight - -(23) CometProject -Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16, d_date_sk#17] -Arguments: [ss_net_profit#11, s_state#16], [ss_net_profit#11, s_state#16] - -(24) CometHashAggregate -Input [2]: [ss_net_profit#11, s_state#16] -Keys [1]: [s_state#16] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] - -(25) CometExchange -Input [2]: [s_state#16, sum#18] -Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(26) CometHashAggregate -Input [2]: [s_state#16, sum#18] -Keys [1]: [s_state#16] -Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] - -(27) CometSort -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16, _w0#19, s_state#16], [s_state#16 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] - -(28) CometColumnarToRow [codegen id : 1] -Input [3]: [s_state#16, _w0#19, s_state#16] - -(29) WindowGroupLimit -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final - -(30) Window -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#16, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#16], [_w0#19 DESC NULLS LAST] - -(31) Filter [codegen id : 2] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] -Condition : (ranking#20 <= 5) - -(32) Project [codegen id : 2] -Output [1]: [s_state#16] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] - -(33) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] - -(34) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#9, 2))] -Right keys [1]: [s_state#16] -Join type: LeftSemi -Join condition: None - -(35) Project [codegen id : 3] -Output [3]: [s_store_sk#7, s_county#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#9, 2)) AS s_state#21] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(36) BroadcastExchange -Input [3]: [s_store_sk#7, s_county#8, s_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_county#8, s_state#21] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#21] - -(39) HashAggregate [codegen id : 4] -Input [3]: [ss_net_profit#2, s_county#8, s_state#21] -Keys [2]: [s_state#21, s_county#8] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [s_state#21, s_county#8, sum#23] - -(40) CometColumnarExchange -Input [3]: [s_state#21, s_county#8, sum#23] -Arguments: hashpartitioning(s_state#21, s_county#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 5] -Input [3]: [s_state#21, s_county#8, sum#23] - -(42) HashAggregate [codegen id : 5] -Input [3]: [s_state#21, s_county#8, sum#23] -Keys [2]: [s_state#21, s_county#8] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) as decimal(27,2)) AS total_sum#25, s_state#21 AS s_state#26, s_county#8 AS s_county#27, 0 AS g_state#28, 0 AS g_county#29, 0 AS lochierarchy#30] - -(43) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_state#31, s_county#32, sum#33] - -(44) CometColumnarToRow [codegen id : 10] -Input [3]: [s_state#31, s_county#32, sum#33] - -(45) HashAggregate [codegen id : 10] -Input [3]: [s_state#31, s_county#32, sum#33] -Keys [2]: [s_state#31, s_county#32] -Functions [1]: [sum(UnscaledValue(ss_net_profit#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#34))#24] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#34))#24,17,2) AS total_sum#35, s_state#31] - -(46) HashAggregate [codegen id : 10] -Input [2]: [total_sum#35, s_state#31] -Keys [1]: [s_state#31] -Functions [1]: [partial_sum(total_sum#35)] -Aggregate Attributes [2]: [sum#36, isEmpty#37] -Results [3]: [s_state#31, sum#38, isEmpty#39] - -(47) CometColumnarExchange -Input [3]: [s_state#31, sum#38, isEmpty#39] -Arguments: hashpartitioning(s_state#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(48) CometColumnarToRow [codegen id : 11] -Input [3]: [s_state#31, sum#38, isEmpty#39] - -(49) HashAggregate [codegen id : 11] -Input [3]: [s_state#31, sum#38, isEmpty#39] -Keys [1]: [s_state#31] -Functions [1]: [sum(total_sum#35)] -Aggregate Attributes [1]: [sum(total_sum#35)#40] -Results [6]: [sum(total_sum#35)#40 AS total_sum#41, s_state#31, null AS s_county#42, 0 AS g_state#43, 1 AS g_county#44, 1 AS lochierarchy#45] - -(50) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_state#46, s_county#47, sum#48] - -(51) CometColumnarToRow [codegen id : 16] -Input [3]: [s_state#46, s_county#47, sum#48] - -(52) HashAggregate [codegen id : 16] -Input [3]: [s_state#46, s_county#47, sum#48] -Keys [2]: [s_state#46, s_county#47] -Functions [1]: [sum(UnscaledValue(ss_net_profit#49))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#49))#24] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#49))#24,17,2) AS total_sum#50] - -(53) HashAggregate [codegen id : 16] -Input [1]: [total_sum#50] -Keys: [] -Functions [1]: [partial_sum(total_sum#50)] -Aggregate Attributes [2]: [sum#51, isEmpty#52] -Results [2]: [sum#53, isEmpty#54] - -(54) CometColumnarExchange -Input [2]: [sum#53, isEmpty#54] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(55) CometColumnarToRow [codegen id : 17] -Input [2]: [sum#53, isEmpty#54] - -(56) HashAggregate [codegen id : 17] -Input [2]: [sum#53, isEmpty#54] -Keys: [] -Functions [1]: [sum(total_sum#50)] -Aggregate Attributes [1]: [sum(total_sum#50)#55] -Results [6]: [sum(total_sum#50)#55 AS total_sum#56, null AS s_state#57, null AS s_county#58, 1 AS g_state#59, 1 AS g_county#60, 2 AS lochierarchy#61] - -(57) Union - -(58) HashAggregate [codegen id : 18] -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Functions: [] -Aggregate Attributes: [] -Results [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] - -(59) CometColumnarExchange -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Arguments: hashpartitioning(total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(60) CometHashAggregate -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Functions: [] - -(61) CometExchange -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] -Arguments: hashpartitioning(lochierarchy#30, _w0#62, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(62) CometSort -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] -Arguments: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62], [lochierarchy#30 ASC NULLS FIRST, _w0#62 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] - -(63) CometColumnarToRow [codegen id : 19] -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] - -(64) Window -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] -Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#30, _w0#62, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#63], [lochierarchy#30, _w0#62], [total_sum#25 DESC NULLS LAST] - -(65) Project [codegen id : 20] -Output [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#63] -Input [6]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62, rank_within_parent#63] - -(66) TakeOrderedAndProject -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#63] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#63 ASC NULLS FIRST], [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#63] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometProject (69) - +- CometFilter (68) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (67) - - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(68) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(69) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(70) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(71) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 2832c5a348..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,101 +0,0 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (20) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] - CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (18) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [s_state,_w0] - CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] - CometExchange [s_state] #8 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - 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,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #9 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (11) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #10 - WholeStageCodegen (10) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #11 - WholeStageCodegen (16) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt deleted file mode 100644 index a66acf75ca..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt +++ /dev/null @@ -1,408 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (66) -+- * Project (65) - +- Window (64) - +- * CometColumnarToRow (63) - +- CometSort (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometColumnarExchange (59) - +- * HashAggregate (58) - +- Union (57) - :- * HashAggregate (42) - : +- * CometColumnarToRow (41) - : +- CometColumnarExchange (40) - : +- * HashAggregate (39) - : +- * Project (38) - : +- * BroadcastHashJoin Inner BuildRight (37) - : :- * CometColumnarToRow (9) - : : +- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- BroadcastExchange (36) - : +- * Project (35) - : +- * BroadcastHashJoin LeftSemi BuildRight (34) - : :- * CometColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) - : +- BroadcastExchange (33) - : +- * Project (32) - : +- * Filter (31) - : +- Window (30) - : +- WindowGroupLimit (29) - : +- * CometColumnarToRow (28) - : +- CometSort (27) - : +- CometHashAggregate (26) - : +- CometExchange (25) - : +- CometHashAggregate (24) - : +- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (20) - : : +- CometBroadcastHashJoin (19) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - : : +- CometBroadcastExchange (18) - : : +- CometProject (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - : +- ReusedExchange (21) - :- * HashAggregate (49) - : +- * CometColumnarToRow (48) - : +- CometColumnarExchange (47) - : +- * HashAggregate (46) - : +- * HashAggregate (45) - : +- * CometColumnarToRow (44) - : +- ReusedExchange (43) - +- * HashAggregate (56) - +- * CometColumnarToRow (55) - +- CometColumnarExchange (54) - +- * HashAggregate (53) - +- * HashAggregate (52) - +- * CometColumnarToRow (51) - +- ReusedExchange (50) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] - -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ss_store_sk#1, ss_net_profit#2] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#7, s_county#8, s_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Condition : isnotnull(s_store_sk#7) - -(12) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_store_sk#10) - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : isnotnull(s_store_sk#14) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14, s_state#16], [s_store_sk#14, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#15, 2)) AS s_state#16] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#14, s_state#16] -Arguments: [s_store_sk#14, s_state#16] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Right output [2]: [s_store_sk#14, s_state#16] -Arguments: [ss_store_sk#10], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#16] -Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16], [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#17] - -(22) CometBroadcastHashJoin -Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] -Right output [1]: [d_date_sk#17] -Arguments: [ss_sold_date_sk#12], [d_date_sk#17], Inner, BuildRight - -(23) CometProject -Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16, d_date_sk#17] -Arguments: [ss_net_profit#11, s_state#16], [ss_net_profit#11, s_state#16] - -(24) CometHashAggregate -Input [2]: [ss_net_profit#11, s_state#16] -Keys [1]: [s_state#16] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] - -(25) CometExchange -Input [2]: [s_state#16, sum#18] -Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(26) CometHashAggregate -Input [2]: [s_state#16, sum#18] -Keys [1]: [s_state#16] -Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] - -(27) CometSort -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16, _w0#19, s_state#16], [s_state#16 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] - -(28) CometColumnarToRow [codegen id : 1] -Input [3]: [s_state#16, _w0#19, s_state#16] - -(29) WindowGroupLimit -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final - -(30) Window -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#16, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#16], [_w0#19 DESC NULLS LAST] - -(31) Filter [codegen id : 2] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] -Condition : (ranking#20 <= 5) - -(32) Project [codegen id : 2] -Output [1]: [s_state#16] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] - -(33) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] - -(34) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#9, 2))] -Right keys [1]: [s_state#16] -Join type: LeftSemi -Join condition: None - -(35) Project [codegen id : 3] -Output [3]: [s_store_sk#7, s_county#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#9, 2)) AS s_state#21] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(36) BroadcastExchange -Input [3]: [s_store_sk#7, s_county#8, s_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_county#8, s_state#21] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#21] - -(39) HashAggregate [codegen id : 4] -Input [3]: [ss_net_profit#2, s_county#8, s_state#21] -Keys [2]: [s_state#21, s_county#8] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [s_state#21, s_county#8, sum#23] - -(40) CometColumnarExchange -Input [3]: [s_state#21, s_county#8, sum#23] -Arguments: hashpartitioning(s_state#21, s_county#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 5] -Input [3]: [s_state#21, s_county#8, sum#23] - -(42) HashAggregate [codegen id : 5] -Input [3]: [s_state#21, s_county#8, sum#23] -Keys [2]: [s_state#21, s_county#8] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) as decimal(27,2)) AS total_sum#25, s_state#21 AS s_state#26, s_county#8 AS s_county#27, 0 AS g_state#28, 0 AS g_county#29, 0 AS lochierarchy#30] - -(43) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_state#31, s_county#32, sum#33] - -(44) CometColumnarToRow [codegen id : 10] -Input [3]: [s_state#31, s_county#32, sum#33] - -(45) HashAggregate [codegen id : 10] -Input [3]: [s_state#31, s_county#32, sum#33] -Keys [2]: [s_state#31, s_county#32] -Functions [1]: [sum(UnscaledValue(ss_net_profit#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#34))#24] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#34))#24,17,2) AS total_sum#35, s_state#31] - -(46) HashAggregate [codegen id : 10] -Input [2]: [total_sum#35, s_state#31] -Keys [1]: [s_state#31] -Functions [1]: [partial_sum(total_sum#35)] -Aggregate Attributes [2]: [sum#36, isEmpty#37] -Results [3]: [s_state#31, sum#38, isEmpty#39] - -(47) CometColumnarExchange -Input [3]: [s_state#31, sum#38, isEmpty#39] -Arguments: hashpartitioning(s_state#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(48) CometColumnarToRow [codegen id : 11] -Input [3]: [s_state#31, sum#38, isEmpty#39] - -(49) HashAggregate [codegen id : 11] -Input [3]: [s_state#31, sum#38, isEmpty#39] -Keys [1]: [s_state#31] -Functions [1]: [sum(total_sum#35)] -Aggregate Attributes [1]: [sum(total_sum#35)#40] -Results [6]: [sum(total_sum#35)#40 AS total_sum#41, s_state#31, null AS s_county#42, 0 AS g_state#43, 1 AS g_county#44, 1 AS lochierarchy#45] - -(50) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_state#46, s_county#47, sum#48] - -(51) CometColumnarToRow [codegen id : 16] -Input [3]: [s_state#46, s_county#47, sum#48] - -(52) HashAggregate [codegen id : 16] -Input [3]: [s_state#46, s_county#47, sum#48] -Keys [2]: [s_state#46, s_county#47] -Functions [1]: [sum(UnscaledValue(ss_net_profit#49))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#49))#24] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#49))#24,17,2) AS total_sum#50] - -(53) HashAggregate [codegen id : 16] -Input [1]: [total_sum#50] -Keys: [] -Functions [1]: [partial_sum(total_sum#50)] -Aggregate Attributes [2]: [sum#51, isEmpty#52] -Results [2]: [sum#53, isEmpty#54] - -(54) CometColumnarExchange -Input [2]: [sum#53, isEmpty#54] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(55) CometColumnarToRow [codegen id : 17] -Input [2]: [sum#53, isEmpty#54] - -(56) HashAggregate [codegen id : 17] -Input [2]: [sum#53, isEmpty#54] -Keys: [] -Functions [1]: [sum(total_sum#50)] -Aggregate Attributes [1]: [sum(total_sum#50)#55] -Results [6]: [sum(total_sum#50)#55 AS total_sum#56, null AS s_state#57, null AS s_county#58, 1 AS g_state#59, 1 AS g_county#60, 2 AS lochierarchy#61] - -(57) Union - -(58) HashAggregate [codegen id : 18] -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Functions: [] -Aggregate Attributes: [] -Results [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] - -(59) CometColumnarExchange -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Arguments: hashpartitioning(total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(60) CometHashAggregate -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Functions: [] - -(61) CometExchange -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] -Arguments: hashpartitioning(lochierarchy#30, _w0#62, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(62) CometSort -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] -Arguments: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62], [lochierarchy#30 ASC NULLS FIRST, _w0#62 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] - -(63) CometColumnarToRow [codegen id : 19] -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] - -(64) Window -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] -Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#30, _w0#62, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#63], [lochierarchy#30, _w0#62], [total_sum#25 DESC NULLS LAST] - -(65) Project [codegen id : 20] -Output [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#63] -Input [6]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62, rank_within_parent#63] - -(66) TakeOrderedAndProject -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#63] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#63 ASC NULLS FIRST], [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#63] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometProject (69) - +- CometFilter (68) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (67) - - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(68) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(69) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(70) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(71) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/extended.txt deleted file mode 100644 index 191d0ef18a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/extended.txt +++ /dev/null @@ -1,176 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window - : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window - : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- CometColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 102 out of 156 eligible operators (65%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 2832c5a348..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt +++ /dev/null @@ -1,101 +0,0 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (20) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] - CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (18) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [s_state,_w0] - CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] - CometExchange [s_state] #8 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - 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,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #9 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (11) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #10 - WholeStageCodegen (10) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #11 - WholeStageCodegen (16) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/explain.txt deleted file mode 100644 index 24b9e03cc2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/explain.txt +++ /dev/null @@ -1,428 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (70) -+- CometTakeOrderedAndProject (69) - +- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometProject (65) - +- CometSortMergeJoin (64) - :- CometSort (58) - : +- CometColumnarExchange (57) - : +- * Project (56) - : +- * BroadcastHashJoin LeftOuter BuildRight (55) - : :- * Project (50) - : : +- * BroadcastHashJoin Inner BuildRight (49) - : : :- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (28) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Project (15) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : : :- * Project (9) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : : : : :- * Filter (3) - : : : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * CometColumnarToRow (6) - : : : : : : : : : +- CometFilter (5) - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory (4) - : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * CometColumnarToRow (12) - : : : : : : : : +- CometFilter (11) - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * CometColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) - : : : : : : +- BroadcastExchange (26) - : : : : : : +- * CometColumnarToRow (25) - : : : : : : +- CometProject (24) - : : : : : : +- CometFilter (23) - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (22) - : : : : : +- BroadcastExchange (33) - : : : : : +- * CometColumnarToRow (32) - : : : : : +- CometProject (31) - : : : : : +- CometFilter (30) - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (29) - : : : : +- ReusedExchange (36) - : : : +- BroadcastExchange (42) - : : : +- * CometColumnarToRow (41) - : : : +- CometFilter (40) - : : : +- CometNativeScan parquet spark_catalog.default.date_dim (39) - : : +- BroadcastExchange (48) - : : +- * CometColumnarToRow (47) - : : +- CometFilter (46) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (45) - : +- BroadcastExchange (54) - : +- * CometColumnarToRow (53) - : +- CometFilter (52) - : +- CometNativeScan parquet spark_catalog.default.promotion (51) - +- CometSort (63) - +- CometExchange (62) - +- CometProject (61) - +- CometFilter (60) - +- CometNativeScan parquet spark_catalog.default.catalog_returns (59) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 10] -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] - -(3) Filter [codegen id : 10] -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) - -(4) CometNativeScan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(inv_date_sk#13)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) - -(6) CometColumnarToRow [codegen id : 1] -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(7) BroadcastExchange -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [inv_item_sk#10] -Join type: Inner -Join condition: (inv_quantity_on_hand#12 < cs_quantity#7) - -(9) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(10) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Condition : isnotnull(w_warehouse_sk#14) - -(12) CometColumnarToRow [codegen id : 2] -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] - -(13) BroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_warehouse_sk#11] -Right keys [1]: [w_warehouse_sk#14] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] - -(16) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_desc#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [i_item_sk#16, i_item_desc#17] -Condition : isnotnull(i_item_sk#16) - -(18) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#16, i_item_desc#17] - -(19) BroadcastExchange -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(20) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#16] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 10] -Output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] - -(22) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#18, cd_marital_status#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_marital_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(23) CometFilter -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Condition : ((isnotnull(cd_marital_status#19) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#19, 1)) = M)) AND isnotnull(cd_demo_sk#18)) - -(24) CometProject -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Arguments: [cd_demo_sk#18], [cd_demo_sk#18] - -(25) CometColumnarToRow [codegen id : 4] -Input [1]: [cd_demo_sk#18] - -(26) BroadcastExchange -Input [1]: [cd_demo_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(27) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] - -(29) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_buy_potential#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Condition : ((isnotnull(hd_buy_potential#21) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#21, 15)) = 1001-5000 )) AND isnotnull(hd_demo_sk#20)) - -(31) CometProject -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Arguments: [hd_demo_sk#20], [hd_demo_sk#20] - -(32) CometColumnarToRow [codegen id : 5] -Input [1]: [hd_demo_sk#20] - -(33) BroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] - -(36) ReusedExchange [Reuses operator id: 75] -Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#8] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] - -(39) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_week_seq#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(40) CometFilter -Input [2]: [d_date_sk#25, d_week_seq#26] -Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) - -(41) CometColumnarToRow [codegen id : 7] -Input [2]: [d_date_sk#25, d_week_seq#26] - -(42) BroadcastExchange -Input [2]: [d_date_sk#25, d_week_seq#26] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] - -(43) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [d_week_seq#24, inv_date_sk#13] -Right keys [2]: [d_week_seq#26, d_date_sk#25] -Join type: Inner -Join condition: None - -(44) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#25, d_week_seq#26] - -(45) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#27, d_date#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [d_date_sk#27, d_date#28] -Condition : (isnotnull(d_date#28) AND isnotnull(d_date_sk#27)) - -(47) CometColumnarToRow [codegen id : 8] -Input [2]: [d_date_sk#27, d_date#28] - -(48) BroadcastExchange -Input [2]: [d_date_sk#27, d_date#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: (d_date#28 > date_add(d_date#23, 5)) - -(50) Project [codegen id : 10] -Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#27, d_date#28] - -(51) CometNativeScan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(52) CometFilter -Input [1]: [p_promo_sk#29] -Condition : isnotnull(p_promo_sk#29) - -(53) CometColumnarToRow [codegen id : 9] -Input [1]: [p_promo_sk#29] - -(54) BroadcastExchange -Input [1]: [p_promo_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(55) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_promo_sk#5] -Right keys [1]: [p_promo_sk#29] -Join type: LeftOuter -Join condition: None - -(56) Project [codegen id : 10] -Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] - -(57) CometColumnarExchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(58) CometSort -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] - -(59) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(60) CometFilter -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Condition : (isnotnull(cr_item_sk#30) AND isnotnull(cr_order_number#31)) - -(61) CometProject -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30, cr_order_number#31] - -(62) CometExchange -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(63) CometSort -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST] - -(64) CometSortMergeJoin -Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#30, cr_order_number#31], LeftOuter - -(65) CometProject -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#30, cr_order_number#31] -Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(66) CometHashAggregate -Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [partial_count(1)] - -(67) CometExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(68) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [count(1)] - -(69) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#36 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#34,promo#35,total_cnt#36]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36], 100, 0, [total_cnt#36 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] - -(70) CometColumnarToRow [codegen id : 11] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (75) -+- * CometColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometNativeScan parquet spark_catalog.default.date_dim (71) - - -(71) CometNativeScan parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(72) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] -Condition : ((((isnotnull(d_year#37) AND (d_year#37 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(73) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(74) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(75) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/simplified.txt deleted file mode 100644 index 025217a607..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/simplified.txt +++ /dev/null @@ -1,107 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] - CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] - CometProject [w_warehouse_name,i_item_desc,d_week_seq] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometColumnarExchange [cs_item_sk,cs_order_number] #2 - WholeStageCodegen (10) - Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [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] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometNativeScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometNativeScan 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/q72.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/explain.txt deleted file mode 100644 index 707d448aff..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,402 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (64) -+- CometTakeOrderedAndProject (63) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometSortMergeJoin (58) - :- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (45) - : : +- CometBroadcastHashJoin (44) - : : :- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (35) - : : : : +- CometBroadcastHashJoin (34) - : : : : :- CometProject (29) - : : : : : +- CometBroadcastHashJoin (28) - : : : : : :- CometProject (23) - : : : : : : +- CometBroadcastHashJoin (22) - : : : : : : :- CometProject (17) - : : : : : : : +- CometBroadcastHashJoin (16) - : : : : : : : :- CometProject (12) - : : : : : : : : +- CometBroadcastHashJoin (11) - : : : : : : : : :- CometProject (7) - : : : : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : : : : :- CometFilter (2) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- CometBroadcastExchange (5) - : : : : : : : : : +- CometFilter (4) - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (3) - : : : : : : : : +- CometBroadcastExchange (10) - : : : : : : : : +- CometFilter (9) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : : : : : : : +- CometBroadcastExchange (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (13) - : : : : : : +- CometBroadcastExchange (21) - : : : : : : +- CometProject (20) - : : : : : : +- CometFilter (19) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (18) - : : : : : +- CometBroadcastExchange (27) - : : : : : +- CometProject (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (24) - : : : : +- CometBroadcastExchange (33) - : : : : +- CometProject (32) - : : : : +- CometFilter (31) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - : : : +- CometBroadcastExchange (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (36) - : : +- CometBroadcastExchange (43) - : : +- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - : +- CometBroadcastExchange (48) - : +- CometFilter (47) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (46) - +- CometSort (57) - +- CometExchange (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (53) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#13)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) - -(5) CometBroadcastExchange -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(6) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Right output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_item_sk#4], [inv_item_sk#10], Inner, (inv_quantity_on_hand#12 < cs_quantity#7), BuildRight - -(7) CometProject -Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Condition : isnotnull(w_warehouse_sk#14) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [w_warehouse_sk#14, w_warehouse_name#15] - -(11) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -Right output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [inv_warehouse_sk#11], [w_warehouse_sk#14], Inner, BuildRight - -(12) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_desc#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [i_item_sk#16, i_item_desc#17] -Condition : isnotnull(i_item_sk#16) - -(15) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [i_item_sk#16, i_item_desc#17] - -(16) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -Right output [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [cs_item_sk#4], [i_item_sk#16], Inner, BuildRight - -(17) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#18, cd_marital_status#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_marital_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Condition : ((isnotnull(cd_marital_status#19) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#19, 1)) = M)) AND isnotnull(cd_demo_sk#18)) - -(20) CometProject -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Arguments: [cd_demo_sk#18], [cd_demo_sk#18] - -(21) CometBroadcastExchange -Input [1]: [cd_demo_sk#18] -Arguments: [cd_demo_sk#18] - -(22) CometBroadcastHashJoin -Left output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [cd_demo_sk#18] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#18], Inner, BuildRight - -(23) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] -Arguments: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_buy_potential#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Condition : ((isnotnull(hd_buy_potential#21) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#21, 15)) = 1001-5000 )) AND isnotnull(hd_demo_sk#20)) - -(26) CometProject -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Arguments: [hd_demo_sk#20], [hd_demo_sk#20] - -(27) CometBroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: [hd_demo_sk#20] - -(28) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [hd_demo_sk#20] -Arguments: [cs_bill_hdemo_sk#3], [hd_demo_sk#20], Inner, BuildRight - -(29) CometProject -Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(31) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(32) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(33) CometBroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24] - -(34) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_sold_date_sk#8], [d_date_sk#22], Inner, BuildRight - -(35) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_week_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [d_date_sk#26, d_week_seq#27] -Condition : (isnotnull(d_week_seq#27) AND isnotnull(d_date_sk#26)) - -(38) CometBroadcastExchange -Input [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_date_sk#26, d_week_seq#27] - -(39) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_week_seq#24, inv_date_sk#13], [d_week_seq#27, d_date_sk#26], Inner, BuildRight - -(40) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#26, d_week_seq#27] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_date#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#28, d_date#29] -Condition : (isnotnull(d_date#29) AND isnotnull(d_date_sk#28)) - -(43) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_date#29] -Arguments: [d_date_sk#28, d_date#29] - -(44) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#28, d_date#29] -Arguments: [cs_ship_date_sk#1], [d_date_sk#28], Inner, (d_date#29 > date_add(d_date#23, 5)), BuildRight - -(45) CometProject -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#28, d_date#29] -Arguments: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(47) CometFilter -Input [1]: [p_promo_sk#30] -Condition : isnotnull(p_promo_sk#30) - -(48) CometBroadcastExchange -Input [1]: [p_promo_sk#30] -Arguments: [p_promo_sk#30] - -(49) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [1]: [p_promo_sk#30] -Arguments: [cs_promo_sk#5], [p_promo_sk#30], LeftOuter, BuildRight - -(50) CometProject -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#30] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(51) CometExchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(52) CometSort -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Condition : (isnotnull(cr_item_sk#31) AND isnotnull(cr_order_number#32)) - -(55) CometProject -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31, cr_order_number#32] - -(56) CometExchange -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(57) CometSort -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#31, cr_order_number#32], LeftOuter - -(59) CometProject -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#31, cr_order_number#32] -Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(60) CometHashAggregate -Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [partial_count(1)] - -(61) CometExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(62) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [count(1)] - -(63) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, 0, [total_cnt#37 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -(64) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (69) -+- * CometColumnarToRow (68) - +- CometProject (67) - +- CometFilter (66) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) - - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(67) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(68) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(69) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/simplified.txt deleted file mode 100644 index 678f8c67df..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] - CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] - CometProject [w_warehouse_name,i_item_desc,d_week_seq] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometExchange [cs_item_sk,cs_order_number] #2 - CometProject [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,p_promo_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_date] - 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 [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_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_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_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_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 [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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_desc] #6 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange [cd_demo_sk] #7 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange [hd_demo_sk] #8 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [d_date_sk,d_week_seq] #10 - CometFilter [d_date_sk,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [d_date_sk,d_date] #11 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [p_promo_sk] #12 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/explain.txt deleted file mode 100644 index 707d448aff..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/explain.txt +++ /dev/null @@ -1,402 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (64) -+- CometTakeOrderedAndProject (63) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometSortMergeJoin (58) - :- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (45) - : : +- CometBroadcastHashJoin (44) - : : :- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (35) - : : : : +- CometBroadcastHashJoin (34) - : : : : :- CometProject (29) - : : : : : +- CometBroadcastHashJoin (28) - : : : : : :- CometProject (23) - : : : : : : +- CometBroadcastHashJoin (22) - : : : : : : :- CometProject (17) - : : : : : : : +- CometBroadcastHashJoin (16) - : : : : : : : :- CometProject (12) - : : : : : : : : +- CometBroadcastHashJoin (11) - : : : : : : : : :- CometProject (7) - : : : : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : : : : :- CometFilter (2) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- CometBroadcastExchange (5) - : : : : : : : : : +- CometFilter (4) - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (3) - : : : : : : : : +- CometBroadcastExchange (10) - : : : : : : : : +- CometFilter (9) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : : : : : : : +- CometBroadcastExchange (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (13) - : : : : : : +- CometBroadcastExchange (21) - : : : : : : +- CometProject (20) - : : : : : : +- CometFilter (19) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (18) - : : : : : +- CometBroadcastExchange (27) - : : : : : +- CometProject (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (24) - : : : : +- CometBroadcastExchange (33) - : : : : +- CometProject (32) - : : : : +- CometFilter (31) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - : : : +- CometBroadcastExchange (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (36) - : : +- CometBroadcastExchange (43) - : : +- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - : +- CometBroadcastExchange (48) - : +- CometFilter (47) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (46) - +- CometSort (57) - +- CometExchange (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (53) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#13)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) - -(5) CometBroadcastExchange -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(6) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Right output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_item_sk#4], [inv_item_sk#10], Inner, (inv_quantity_on_hand#12 < cs_quantity#7), BuildRight - -(7) CometProject -Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Condition : isnotnull(w_warehouse_sk#14) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [w_warehouse_sk#14, w_warehouse_name#15] - -(11) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -Right output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [inv_warehouse_sk#11], [w_warehouse_sk#14], Inner, BuildRight - -(12) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_desc#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [i_item_sk#16, i_item_desc#17] -Condition : isnotnull(i_item_sk#16) - -(15) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [i_item_sk#16, i_item_desc#17] - -(16) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -Right output [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [cs_item_sk#4], [i_item_sk#16], Inner, BuildRight - -(17) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#18, cd_marital_status#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_marital_status), IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Condition : ((isnotnull(cd_marital_status#19) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#19, 1)) = M)) AND isnotnull(cd_demo_sk#18)) - -(20) CometProject -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Arguments: [cd_demo_sk#18], [cd_demo_sk#18] - -(21) CometBroadcastExchange -Input [1]: [cd_demo_sk#18] -Arguments: [cd_demo_sk#18] - -(22) CometBroadcastHashJoin -Left output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [cd_demo_sk#18] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#18], Inner, BuildRight - -(23) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] -Arguments: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_buy_potential#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Condition : ((isnotnull(hd_buy_potential#21) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(hd_buy_potential#21, 15)) = 1001-5000 )) AND isnotnull(hd_demo_sk#20)) - -(26) CometProject -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Arguments: [hd_demo_sk#20], [hd_demo_sk#20] - -(27) CometBroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: [hd_demo_sk#20] - -(28) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [hd_demo_sk#20] -Arguments: [cs_bill_hdemo_sk#3], [hd_demo_sk#20], Inner, BuildRight - -(29) CometProject -Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(31) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(32) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(33) CometBroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24] - -(34) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_sold_date_sk#8], [d_date_sk#22], Inner, BuildRight - -(35) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_week_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [d_date_sk#26, d_week_seq#27] -Condition : (isnotnull(d_week_seq#27) AND isnotnull(d_date_sk#26)) - -(38) CometBroadcastExchange -Input [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_date_sk#26, d_week_seq#27] - -(39) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_week_seq#24, inv_date_sk#13], [d_week_seq#27, d_date_sk#26], Inner, BuildRight - -(40) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#26, d_week_seq#27] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_date#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#28, d_date#29] -Condition : (isnotnull(d_date#29) AND isnotnull(d_date_sk#28)) - -(43) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_date#29] -Arguments: [d_date_sk#28, d_date#29] - -(44) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#28, d_date#29] -Arguments: [cs_ship_date_sk#1], [d_date_sk#28], Inner, (d_date#29 > date_add(d_date#23, 5)), BuildRight - -(45) CometProject -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#28, d_date#29] -Arguments: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(47) CometFilter -Input [1]: [p_promo_sk#30] -Condition : isnotnull(p_promo_sk#30) - -(48) CometBroadcastExchange -Input [1]: [p_promo_sk#30] -Arguments: [p_promo_sk#30] - -(49) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [1]: [p_promo_sk#30] -Arguments: [cs_promo_sk#5], [p_promo_sk#30], LeftOuter, BuildRight - -(50) CometProject -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#30] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(51) CometExchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(52) CometSort -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Condition : (isnotnull(cr_item_sk#31) AND isnotnull(cr_order_number#32)) - -(55) CometProject -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31, cr_order_number#32] - -(56) CometExchange -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(57) CometSort -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#31, cr_order_number#32], LeftOuter - -(59) CometProject -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#31, cr_order_number#32] -Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(60) CometHashAggregate -Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [partial_count(1)] - -(61) CometExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(62) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [count(1)] - -(63) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, 0, [total_cnt#37 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -(64) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (69) -+- * CometColumnarToRow (68) - +- CometProject (67) - +- CometFilter (66) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) - - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(67) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(68) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(69) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/extended.txt deleted file mode 100644 index 9000d27d1b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/extended.txt +++ /dev/null @@ -1,72 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 678f8c67df..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] - CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] - CometProject [w_warehouse_name,i_item_desc,d_week_seq] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometExchange [cs_item_sk,cs_order_number] #2 - CometProject [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,p_promo_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_date] - 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 [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_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_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_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_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 [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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_desc] #6 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange [cd_demo_sk] #7 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange [hd_demo_sk] #8 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [d_date_sk,d_week_seq] #10 - CometFilter [d_date_sk,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [d_date_sk,d_date] #11 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [p_promo_sk] #12 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/explain.txt deleted file mode 100644 index 0ac606319d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/explain.txt +++ /dev/null @@ -1,513 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (79) -+- * Project (78) - +- * BroadcastHashJoin Inner BuildRight (77) - :- * Project (58) - : +- * BroadcastHashJoin Inner BuildRight (57) - : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- * HashAggregate (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * Project (28) - : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (26) - : : : +- * Filter (25) - : : : +- * ColumnarToRow (24) - : : : +- Scan parquet spark_catalog.default.store_sales (23) - : : +- ReusedExchange (29) - : +- BroadcastExchange (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- * CometColumnarToRow (53) - : +- CometColumnarExchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (41) - : : : +- CometProject (40) - : : : +- CometFilter (39) - : : : +- CometNativeScan parquet spark_catalog.default.customer (38) - : : +- BroadcastExchange (45) - : : +- * Filter (44) - : : +- * ColumnarToRow (43) - : : +- Scan parquet spark_catalog.default.web_sales (42) - : +- ReusedExchange (48) - +- BroadcastExchange (76) - +- * HashAggregate (75) - +- * CometColumnarToRow (74) - +- CometColumnarExchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * CometColumnarToRow (62) - : : +- CometProject (61) - : : +- CometFilter (60) - : : +- CometNativeScan parquet spark_catalog.default.customer (59) - : +- BroadcastExchange (66) - : +- * Filter (65) - : +- * ColumnarToRow (64) - : +- Scan parquet spark_catalog.default.web_sales (63) - +- ReusedExchange (69) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)))) - -(3) CometProject -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)) AS c_customer_id#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#3, 20)) AS c_first_name#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#4, 30)) AS c_last_name#7] - -(4) CometColumnarToRow [codegen id : 3] -Input [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] - -(5) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(7) Filter [codegen id : 1] -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Condition : isnotnull(ss_customer_sk#8) - -(8) BroadcastExchange -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#8] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(11) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#12, d_year#13] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#10] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] - -(14) HashAggregate [codegen id : 3] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] -Aggregate Attributes [1]: [sum#14] -Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] - -(15) CometColumnarExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 16] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] - -(17) HashAggregate [codegen id : 16] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#9))#16] -Results [2]: [c_customer_id#5 AS customer_id#17, MakeDecimal(sum(UnscaledValue(ss_net_paid#9))#16,17,2) AS year_total#18] - -(18) Filter [codegen id : 16] -Input [2]: [customer_id#17, year_total#18] -Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) - -(19) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(20) CometFilter -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Condition : (isnotnull(c_customer_sk#19) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#20, 16)))) - -(21) CometProject -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Arguments: [c_customer_sk#19, c_customer_id#23, c_first_name#24, c_last_name#25], [c_customer_sk#19, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#20, 16)) AS c_customer_id#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#21, 20)) AS c_first_name#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#22, 30)) AS c_last_name#25] - -(22) CometColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#19, c_customer_id#23, c_first_name#24, c_last_name#25] - -(23) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#26, ss_net_paid#27, ss_sold_date_sk#28] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#29)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#26, ss_net_paid#27, ss_sold_date_sk#28] - -(25) Filter [codegen id : 4] -Input [3]: [ss_customer_sk#26, ss_net_paid#27, ss_sold_date_sk#28] -Condition : isnotnull(ss_customer_sk#26) - -(26) BroadcastExchange -Input [3]: [ss_customer_sk#26, ss_net_paid#27, ss_sold_date_sk#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#19] -Right keys [1]: [ss_customer_sk#26] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, ss_net_paid#27, ss_sold_date_sk#28] -Input [7]: [c_customer_sk#19, c_customer_id#23, c_first_name#24, c_last_name#25, ss_customer_sk#26, ss_net_paid#27, ss_sold_date_sk#28] - -(29) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#30, d_year#31] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#28] -Right keys [1]: [d_date_sk#30] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, ss_net_paid#27, d_year#31] -Input [7]: [c_customer_id#23, c_first_name#24, c_last_name#25, ss_net_paid#27, ss_sold_date_sk#28, d_date_sk#30, d_year#31] - -(32) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, ss_net_paid#27, d_year#31] -Keys [4]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#27))] -Aggregate Attributes [1]: [sum#32] -Results [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31, sum#33] - -(33) CometColumnarExchange -Input [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31, sum#33] -Arguments: hashpartitioning(c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 7] -Input [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31, sum#33] - -(35) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31, sum#33] -Keys [4]: [c_customer_id#23, c_first_name#24, c_last_name#25, d_year#31] -Functions [1]: [sum(UnscaledValue(ss_net_paid#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#27))#16] -Results [4]: [c_customer_id#23 AS customer_id#34, c_first_name#24 AS customer_first_name#35, c_last_name#25 AS customer_last_name#36, MakeDecimal(sum(UnscaledValue(ss_net_paid#27))#16,17,2) AS year_total#37] - -(36) BroadcastExchange -Input [4]: [customer_id#34, customer_first_name#35, customer_last_name#36, year_total#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(37) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#34] -Join type: Inner -Join condition: None - -(38) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#38, c_customer_id#39, c_first_name#40, c_last_name#41] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(39) CometFilter -Input [4]: [c_customer_sk#38, c_customer_id#39, c_first_name#40, c_last_name#41] -Condition : (isnotnull(c_customer_sk#38) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#39, 16)))) - -(40) CometProject -Input [4]: [c_customer_sk#38, c_customer_id#39, c_first_name#40, c_last_name#41] -Arguments: [c_customer_sk#38, c_customer_id#42, c_first_name#43, c_last_name#44], [c_customer_sk#38, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#39, 16)) AS c_customer_id#42, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#40, 20)) AS c_first_name#43, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#41, 30)) AS c_last_name#44] - -(41) CometColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#38, c_customer_id#42, c_first_name#43, c_last_name#44] - -(42) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#45, ws_net_paid#46, ws_sold_date_sk#47] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#45, ws_net_paid#46, ws_sold_date_sk#47] - -(44) Filter [codegen id : 8] -Input [3]: [ws_bill_customer_sk#45, ws_net_paid#46, ws_sold_date_sk#47] -Condition : isnotnull(ws_bill_customer_sk#45) - -(45) BroadcastExchange -Input [3]: [ws_bill_customer_sk#45, ws_net_paid#46, ws_sold_date_sk#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -(46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#38] -Right keys [1]: [ws_bill_customer_sk#45] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 10] -Output [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, ws_net_paid#46, ws_sold_date_sk#47] -Input [7]: [c_customer_sk#38, c_customer_id#42, c_first_name#43, c_last_name#44, ws_bill_customer_sk#45, ws_net_paid#46, ws_sold_date_sk#47] - -(48) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#48, d_year#49] - -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#47] -Right keys [1]: [d_date_sk#48] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 10] -Output [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, ws_net_paid#46, d_year#49] -Input [7]: [c_customer_id#42, c_first_name#43, c_last_name#44, ws_net_paid#46, ws_sold_date_sk#47, d_date_sk#48, d_year#49] - -(51) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, ws_net_paid#46, d_year#49] -Keys [4]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#46))] -Aggregate Attributes [1]: [sum#50] -Results [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49, sum#51] - -(52) CometColumnarExchange -Input [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49, sum#51] -Arguments: hashpartitioning(c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(53) CometColumnarToRow [codegen id : 11] -Input [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49, sum#51] - -(54) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49, sum#51] -Keys [4]: [c_customer_id#42, c_first_name#43, c_last_name#44, d_year#49] -Functions [1]: [sum(UnscaledValue(ws_net_paid#46))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#46))#52] -Results [2]: [c_customer_id#42 AS customer_id#53, MakeDecimal(sum(UnscaledValue(ws_net_paid#46))#52,17,2) AS year_total#54] - -(55) Filter [codegen id : 11] -Input [2]: [customer_id#53, year_total#54] -Condition : (isnotnull(year_total#54) AND (year_total#54 > 0.00)) - -(56) BroadcastExchange -Input [2]: [customer_id#53, year_total#54] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(57) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#53] -Join type: Inner -Join condition: None - -(58) Project [codegen id : 16] -Output [7]: [customer_id#17, year_total#18, customer_id#34, customer_first_name#35, customer_last_name#36, year_total#37, year_total#54] -Input [8]: [customer_id#17, year_total#18, customer_id#34, customer_first_name#35, customer_last_name#36, year_total#37, customer_id#53, year_total#54] - -(59) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(60) CometFilter -Input [4]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58] -Condition : (isnotnull(c_customer_sk#55) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#56, 16)))) - -(61) CometProject -Input [4]: [c_customer_sk#55, c_customer_id#56, c_first_name#57, c_last_name#58] -Arguments: [c_customer_sk#55, c_customer_id#59, c_first_name#60, c_last_name#61], [c_customer_sk#55, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#56, 16)) AS c_customer_id#59, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#57, 20)) AS c_first_name#60, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#58, 30)) AS c_last_name#61] - -(62) CometColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#55, c_customer_id#59, c_first_name#60, c_last_name#61] - -(63) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#64), dynamicpruningexpression(ws_sold_date_sk#64 IN dynamicpruning#29)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(64) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] - -(65) Filter [codegen id : 12] -Input [3]: [ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] -Condition : isnotnull(ws_bill_customer_sk#62) - -(66) BroadcastExchange -Input [3]: [ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#55] -Right keys [1]: [ws_bill_customer_sk#62] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 14] -Output [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, ws_sold_date_sk#64] -Input [7]: [c_customer_sk#55, c_customer_id#59, c_first_name#60, c_last_name#61, ws_bill_customer_sk#62, ws_net_paid#63, ws_sold_date_sk#64] - -(69) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#65, d_year#66] - -(70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#64] -Right keys [1]: [d_date_sk#65] -Join type: Inner -Join condition: None - -(71) Project [codegen id : 14] -Output [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, d_year#66] -Input [7]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, ws_sold_date_sk#64, d_date_sk#65, d_year#66] - -(72) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, ws_net_paid#63, d_year#66] -Keys [4]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#63))] -Aggregate Attributes [1]: [sum#67] -Results [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66, sum#68] - -(73) CometColumnarExchange -Input [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66, sum#68] -Arguments: hashpartitioning(c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(74) CometColumnarToRow [codegen id : 15] -Input [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66, sum#68] - -(75) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66, sum#68] -Keys [4]: [c_customer_id#59, c_first_name#60, c_last_name#61, d_year#66] -Functions [1]: [sum(UnscaledValue(ws_net_paid#63))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#63))#52] -Results [2]: [c_customer_id#59 AS customer_id#69, MakeDecimal(sum(UnscaledValue(ws_net_paid#63))#52,17,2) AS year_total#70] - -(76) BroadcastExchange -Input [2]: [customer_id#69, year_total#70] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] - -(77) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#69] -Join type: Inner -Join condition: (CASE WHEN (year_total#54 > 0.00) THEN (year_total#70 / year_total#54) END > CASE WHEN (year_total#18 > 0.00) THEN (year_total#37 / year_total#18) END) - -(78) Project [codegen id : 16] -Output [3]: [customer_id#34, customer_first_name#35, customer_last_name#36] -Input [9]: [customer_id#17, year_total#18, customer_id#34, customer_first_name#35, customer_last_name#36, year_total#37, year_total#54, customer_id#69, year_total#70] - -(79) TakeOrderedAndProject -Input [3]: [customer_id#34, customer_first_name#35, customer_last_name#36] -Arguments: 100, [customer_first_name#35 ASC NULLS FIRST, customer_id#34 ASC NULLS FIRST, customer_last_name#36 ASC NULLS FIRST], [customer_id#34, customer_first_name#35, customer_last_name#36] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometFilter (81) - +- CometNativeScan parquet spark_catalog.default.date_dim (80) - - -(80) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(81) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(82) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(83) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 -BroadcastExchange (87) -+- * CometColumnarToRow (86) - +- CometFilter (85) - +- CometNativeScan parquet spark_catalog.default.date_dim (84) - - -(84) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#30, d_year#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(85) CometFilter -Input [2]: [d_date_sk#30, d_year#31] -Condition : (((isnotnull(d_year#31) AND (d_year#31 = 2002)) AND d_year#31 IN (2001,2002)) AND isnotnull(d_date_sk#30)) - -(86) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#30, d_year#31] - -(87) BroadcastExchange -Input [2]: [d_date_sk#30, d_year#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] - -Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#11 - -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#64 IN dynamicpruning#29 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/simplified.txt deleted file mode 100644 index 9a8a029e1b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/simplified.txt +++ /dev/null @@ -1,130 +0,0 @@ -TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] - WholeStageCodegen (16) - Project [customer_id,customer_first_name,customer_last_name] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/explain.txt deleted file mode 100644 index fefa1d47b1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,470 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometProject (53) - : +- CometBroadcastHashJoin (52) - : :- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (51) - : +- CometFilter (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : +- CometBroadcastExchange (41) - : : +- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) - : +- ReusedExchange (44) - +- CometBroadcastExchange (68) - +- CometHashAggregate (67) - +- CometExchange (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometFilter (55) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (62) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)))) - -(3) CometProject -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)) AS c_customer_id#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#3, 20)) AS c_first_name#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#4, 30)) AS c_last_name#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Condition : isnotnull(ss_customer_sk#8) - -(6) CometBroadcastExchange -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_sk#1], [ss_customer_sk#8], Inner, BuildRight - -(8) CometProject -Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(12) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#10], [d_date_sk#12], Inner, BuildRight - -(13) CometProject -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] - -(14) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] - -(15) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] - -(17) CometFilter -Input [2]: [customer_id#15, year_total#16] -Condition : (isnotnull(year_total#16) AND (year_total#16 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#18, 16)))) - -(20) CometProject -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Arguments: [c_customer_sk#17, c_customer_id#21, c_first_name#22, c_last_name#23], [c_customer_sk#17, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#18, 16)) AS c_customer_id#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#19, 20)) AS c_first_name#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#20, 30)) AS c_last_name#23] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_customer_sk#24) - -(23) CometBroadcastExchange -Input [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -Arguments: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] - -(24) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#17, c_customer_id#21, c_first_name#22, c_last_name#23] -Right output [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -Arguments: [c_customer_sk#17], [ss_customer_sk#24], Inner, BuildRight - -(25) CometProject -Input [7]: [c_customer_sk#17, c_customer_id#21, c_first_name#22, c_last_name#23, ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -Arguments: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26], [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_year#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#28, d_year#29] -Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_year#29] -Arguments: [d_date_sk#28, d_year#29] - -(29) CometBroadcastHashJoin -Left output [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26] -Right output [2]: [d_date_sk#28, d_year#29] -Arguments: [ss_sold_date_sk#26], [d_date_sk#28], Inner, BuildRight - -(30) CometProject -Input [7]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26, d_date_sk#28, d_year#29] -Arguments: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29], [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29] - -(31) CometHashAggregate -Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29] -Keys [4]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#25))] - -(32) CometExchange -Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, sum#30] -Arguments: hashpartitioning(c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, sum#30] -Keys [4]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29] -Functions [1]: [sum(UnscaledValue(ss_net_paid#25))] - -(34) CometBroadcastExchange -Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] -Arguments: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#15, year_total#16] -Right output [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] -Arguments: [customer_id#15], [customer_id#31], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Condition : (isnotnull(c_customer_sk#35) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#36, 16)))) - -(38) CometProject -Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Arguments: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#36, 16)) AS c_customer_id#39, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#37, 20)) AS c_first_name#40, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#38, 30)) AS c_last_name#41] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#45)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_bill_customer_sk#42) - -(41) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Arguments: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] - -(42) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41] -Right output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Arguments: [c_customer_sk#35], [ws_bill_customer_sk#42], Inner, BuildRight - -(43) CometProject -Input [7]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Arguments: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44], [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#46, d_year#47] - -(45) CometBroadcastHashJoin -Left output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [ws_sold_date_sk#44], [d_date_sk#46], Inner, BuildRight - -(46) CometProject -Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#46, d_year#47] -Arguments: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47], [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47] - -(47) CometHashAggregate -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] - -(48) CometExchange -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#48] -Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#48] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47] -Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] - -(50) CometFilter -Input [2]: [customer_id#49, year_total#50] -Condition : (isnotnull(year_total#50) AND (year_total#50 > 0.00)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#49, year_total#50] -Arguments: [customer_id#49, year_total#50] - -(52) CometBroadcastHashJoin -Left output [6]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] -Right output [2]: [customer_id#49, year_total#50] -Arguments: [customer_id#15], [customer_id#49], Inner, BuildRight - -(53) CometProject -Input [8]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, customer_id#49, year_total#50] -Arguments: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50], [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] -Condition : (isnotnull(c_customer_sk#51) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#52, 16)))) - -(56) CometProject -Input [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] -Arguments: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57], [c_customer_sk#51, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#52, 16)) AS c_customer_id#55, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#53, 20)) AS c_first_name#56, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#54, 30)) AS c_last_name#57] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#61)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] -Condition : isnotnull(ws_bill_customer_sk#58) - -(59) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] -Arguments: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] - -(60) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57] -Right output [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] -Arguments: [c_customer_sk#51], [ws_bill_customer_sk#58], Inner, BuildRight - -(61) CometProject -Input [7]: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57, ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] -Arguments: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60], [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#62, d_year#63] - -(63) CometBroadcastHashJoin -Left output [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60] -Right output [2]: [d_date_sk#62, d_year#63] -Arguments: [ws_sold_date_sk#60], [d_date_sk#62], Inner, BuildRight - -(64) CometProject -Input [7]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60, d_date_sk#62, d_year#63] -Arguments: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63], [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63] - -(65) CometHashAggregate -Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63] -Keys [4]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#59))] - -(66) CometExchange -Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, sum#64] -Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, sum#64] -Keys [4]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63] -Functions [1]: [sum(UnscaledValue(ws_net_paid#59))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#65, year_total#66] -Arguments: [customer_id#65, year_total#66] - -(69) CometBroadcastHashJoin -Left output [7]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50] -Right output [2]: [customer_id#65, year_total#66] -Arguments: [customer_id#15], [customer_id#65], Inner, (CASE WHEN (year_total#50 > 0.00) THEN (year_total#66 / year_total#50) END > CASE WHEN (year_total#16 > 0.00) THEN (year_total#34 / year_total#16) END), BuildRight - -(70) CometProject -Input [9]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50, customer_id#65, year_total#66] -Arguments: [customer_id#31, customer_first_name#32, customer_last_name#33], [customer_id#31, customer_first_name#32, customer_last_name#33] - -(71) CometTakeOrderedAndProject -Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_first_name#32 ASC NULLS FIRST,customer_id#31 ASC NULLS FIRST,customer_last_name#33 ASC NULLS FIRST], output=[customer_id#31,customer_first_name#32,customer_last_name#33]), [customer_id#31, customer_first_name#32, customer_last_name#33], 100, 0, [customer_first_name#32 ASC NULLS FIRST, customer_id#31 ASC NULLS FIRST, customer_last_name#33 ASC NULLS FIRST], [customer_id#31, customer_first_name#32, customer_last_name#33] - -(72) CometColumnarToRow [codegen id : 1] -Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(75) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(76) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#27 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_year#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [2]: [d_date_sk#28, d_year#29] -Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) - -(79) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#28, d_year#29] - -(80) BroadcastExchange -Input [2]: [d_date_sk#28, d_year#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#11 - -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#27 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/simplified.txt deleted file mode 100644 index 72e39422e0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - CometProject [customer_id,customer_first_name,customer_last_name] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 - CometHashAggregate [d_year,sum] [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/explain.txt deleted file mode 100644 index fefa1d47b1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/explain.txt +++ /dev/null @@ -1,470 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometProject (53) - : +- CometBroadcastHashJoin (52) - : :- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (51) - : +- CometFilter (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : +- CometBroadcastExchange (41) - : : +- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) - : +- ReusedExchange (44) - +- CometBroadcastExchange (68) - +- CometHashAggregate (67) - +- CometExchange (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometFilter (55) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (62) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)))) - -(3) CometProject -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#2, 16)) AS c_customer_id#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#3, 20)) AS c_first_name#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#4, 30)) AS c_last_name#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Condition : isnotnull(ss_customer_sk#8) - -(6) CometBroadcastExchange -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_sk#1], [ss_customer_sk#8], Inner, BuildRight - -(8) CometProject -Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(12) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#10], [d_date_sk#12], Inner, BuildRight - -(13) CometProject -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] - -(14) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] - -(15) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] - -(17) CometFilter -Input [2]: [customer_id#15, year_total#16] -Condition : (isnotnull(year_total#16) AND (year_total#16 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#18, 16)))) - -(20) CometProject -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Arguments: [c_customer_sk#17, c_customer_id#21, c_first_name#22, c_last_name#23], [c_customer_sk#17, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#18, 16)) AS c_customer_id#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#19, 20)) AS c_first_name#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#20, 30)) AS c_last_name#23] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_customer_sk#24) - -(23) CometBroadcastExchange -Input [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -Arguments: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] - -(24) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#17, c_customer_id#21, c_first_name#22, c_last_name#23] -Right output [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -Arguments: [c_customer_sk#17], [ss_customer_sk#24], Inner, BuildRight - -(25) CometProject -Input [7]: [c_customer_sk#17, c_customer_id#21, c_first_name#22, c_last_name#23, ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -Arguments: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26], [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_year#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#28, d_year#29] -Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_year#29] -Arguments: [d_date_sk#28, d_year#29] - -(29) CometBroadcastHashJoin -Left output [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26] -Right output [2]: [d_date_sk#28, d_year#29] -Arguments: [ss_sold_date_sk#26], [d_date_sk#28], Inner, BuildRight - -(30) CometProject -Input [7]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26, d_date_sk#28, d_year#29] -Arguments: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29], [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29] - -(31) CometHashAggregate -Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29] -Keys [4]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#25))] - -(32) CometExchange -Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, sum#30] -Arguments: hashpartitioning(c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, sum#30] -Keys [4]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29] -Functions [1]: [sum(UnscaledValue(ss_net_paid#25))] - -(34) CometBroadcastExchange -Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] -Arguments: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#15, year_total#16] -Right output [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] -Arguments: [customer_id#15], [customer_id#31], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Condition : (isnotnull(c_customer_sk#35) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#36, 16)))) - -(38) CometProject -Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Arguments: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#36, 16)) AS c_customer_id#39, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#37, 20)) AS c_first_name#40, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#38, 30)) AS c_last_name#41] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#45)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_bill_customer_sk#42) - -(41) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Arguments: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] - -(42) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41] -Right output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Arguments: [c_customer_sk#35], [ws_bill_customer_sk#42], Inner, BuildRight - -(43) CometProject -Input [7]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Arguments: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44], [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#46, d_year#47] - -(45) CometBroadcastHashJoin -Left output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [ws_sold_date_sk#44], [d_date_sk#46], Inner, BuildRight - -(46) CometProject -Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#46, d_year#47] -Arguments: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47], [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47] - -(47) CometHashAggregate -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] - -(48) CometExchange -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#48] -Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#48] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47] -Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] - -(50) CometFilter -Input [2]: [customer_id#49, year_total#50] -Condition : (isnotnull(year_total#50) AND (year_total#50 > 0.00)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#49, year_total#50] -Arguments: [customer_id#49, year_total#50] - -(52) CometBroadcastHashJoin -Left output [6]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] -Right output [2]: [customer_id#49, year_total#50] -Arguments: [customer_id#15], [customer_id#49], Inner, BuildRight - -(53) CometProject -Input [8]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, customer_id#49, year_total#50] -Arguments: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50], [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] -Condition : (isnotnull(c_customer_sk#51) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#52, 16)))) - -(56) CometProject -Input [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] -Arguments: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57], [c_customer_sk#51, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#52, 16)) AS c_customer_id#55, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#53, 20)) AS c_first_name#56, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#54, 30)) AS c_last_name#57] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#61)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] -Condition : isnotnull(ws_bill_customer_sk#58) - -(59) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] -Arguments: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] - -(60) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57] -Right output [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] -Arguments: [c_customer_sk#51], [ws_bill_customer_sk#58], Inner, BuildRight - -(61) CometProject -Input [7]: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57, ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] -Arguments: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60], [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#62, d_year#63] - -(63) CometBroadcastHashJoin -Left output [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60] -Right output [2]: [d_date_sk#62, d_year#63] -Arguments: [ws_sold_date_sk#60], [d_date_sk#62], Inner, BuildRight - -(64) CometProject -Input [7]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60, d_date_sk#62, d_year#63] -Arguments: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63], [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63] - -(65) CometHashAggregate -Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63] -Keys [4]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#59))] - -(66) CometExchange -Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, sum#64] -Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, sum#64] -Keys [4]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63] -Functions [1]: [sum(UnscaledValue(ws_net_paid#59))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#65, year_total#66] -Arguments: [customer_id#65, year_total#66] - -(69) CometBroadcastHashJoin -Left output [7]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50] -Right output [2]: [customer_id#65, year_total#66] -Arguments: [customer_id#15], [customer_id#65], Inner, (CASE WHEN (year_total#50 > 0.00) THEN (year_total#66 / year_total#50) END > CASE WHEN (year_total#16 > 0.00) THEN (year_total#34 / year_total#16) END), BuildRight - -(70) CometProject -Input [9]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50, customer_id#65, year_total#66] -Arguments: [customer_id#31, customer_first_name#32, customer_last_name#33], [customer_id#31, customer_first_name#32, customer_last_name#33] - -(71) CometTakeOrderedAndProject -Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_first_name#32 ASC NULLS FIRST,customer_id#31 ASC NULLS FIRST,customer_last_name#33 ASC NULLS FIRST], output=[customer_id#31,customer_first_name#32,customer_last_name#33]), [customer_id#31, customer_first_name#32, customer_last_name#33], 100, 0, [customer_first_name#32 ASC NULLS FIRST, customer_id#31 ASC NULLS FIRST, customer_last_name#33 ASC NULLS FIRST], [customer_id#31, customer_first_name#32, customer_last_name#33] - -(72) CometColumnarToRow [codegen id : 1] -Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(75) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(76) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#27 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_year#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [2]: [d_date_sk#28, d_year#29] -Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) - -(79) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#28, d_year#29] - -(80) BroadcastExchange -Input [2]: [d_date_sk#28, d_year#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#11 - -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#27 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/extended.txt deleted file mode 100644 index 476c7be954..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/extended.txt +++ /dev/null @@ -1,90 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 72e39422e0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - CometProject [customer_id,customer_first_name,customer_last_name] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 - CometHashAggregate [d_year,sum] [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/explain.txt deleted file mode 100644 index 9566cf6e23..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/explain.txt +++ /dev/null @@ -1,754 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (127) -+- CometTakeOrderedAndProject (126) - +- CometProject (125) - +- CometSortMergeJoin (124) - :- CometSort (68) - : +- CometExchange (67) - : +- CometFilter (66) - : +- CometHashAggregate (65) - : +- CometExchange (64) - : +- CometHashAggregate (63) - : +- CometHashAggregate (62) - : +- CometExchange (61) - : +- CometHashAggregate (60) - : +- CometUnion (59) - : :- CometProject (22) - : : +- CometSortMergeJoin (21) - : : :- CometSort (15) - : : : +- CometColumnarExchange (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : : +- ReusedExchange (11) - : : +- CometSort (20) - : : +- CometExchange (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (40) - : : +- CometSortMergeJoin (39) - : : :- CometSort (33) - : : : +- CometColumnarExchange (32) - : : : +- * Project (31) - : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : :- * Project (28) - : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : :- * Filter (25) - : : : : : +- * ColumnarToRow (24) - : : : : : +- Scan parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (26) - : : : +- ReusedExchange (29) - : : +- CometSort (38) - : : +- CometExchange (37) - : : +- CometProject (36) - : : +- CometFilter (35) - : : +- CometNativeScan parquet spark_catalog.default.store_returns (34) - : +- CometProject (58) - : +- CometSortMergeJoin (57) - : :- CometSort (51) - : : +- CometColumnarExchange (50) - : : +- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * Project (46) - : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : :- * Filter (43) - : : : : +- * ColumnarToRow (42) - : : : : +- Scan parquet spark_catalog.default.web_sales (41) - : : : +- ReusedExchange (44) - : : +- ReusedExchange (47) - : +- CometSort (56) - : +- CometExchange (55) - : +- CometProject (54) - : +- CometFilter (53) - : +- CometNativeScan parquet spark_catalog.default.web_returns (52) - +- CometSort (123) - +- CometExchange (122) - +- CometFilter (121) - +- CometHashAggregate (120) - +- CometExchange (119) - +- CometHashAggregate (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometUnion (114) - :- CometProject (83) - : +- CometSortMergeJoin (82) - : :- CometSort (79) - : : +- CometColumnarExchange (78) - : : +- * Project (77) - : : +- * BroadcastHashJoin Inner BuildRight (76) - : : :- * Project (74) - : : : +- * BroadcastHashJoin Inner BuildRight (73) - : : : :- * Filter (71) - : : : : +- * ColumnarToRow (70) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (69) - : : : +- ReusedExchange (72) - : : +- ReusedExchange (75) - : +- CometSort (81) - : +- ReusedExchange (80) - :- CometProject (98) - : +- CometSortMergeJoin (97) - : :- CometSort (94) - : : +- CometColumnarExchange (93) - : : +- * Project (92) - : : +- * BroadcastHashJoin Inner BuildRight (91) - : : :- * Project (89) - : : : +- * BroadcastHashJoin Inner BuildRight (88) - : : : :- * Filter (86) - : : : : +- * ColumnarToRow (85) - : : : : +- Scan parquet spark_catalog.default.store_sales (84) - : : : +- ReusedExchange (87) - : : +- ReusedExchange (90) - : +- CometSort (96) - : +- ReusedExchange (95) - +- CometProject (113) - +- CometSortMergeJoin (112) - :- CometSort (109) - : +- CometColumnarExchange (108) - : +- * Project (107) - : +- * BroadcastHashJoin Inner BuildRight (106) - : :- * Project (104) - : : +- * BroadcastHashJoin Inner BuildRight (103) - : : :- * Filter (101) - : : : +- * ColumnarToRow (100) - : : : +- Scan parquet spark_catalog.default.web_sales (99) - : : +- ReusedExchange (102) - : +- ReusedExchange (105) - +- CometSort (111) - +- ReusedExchange (110) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] - -(3) Filter [codegen id : 3] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((isnotnull(i_category#11) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#11, 50)) = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) - -(6) CometProject -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(7) CometColumnarToRow [codegen id : 1] -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(8) BroadcastExchange -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(11) ReusedExchange [Reuses operator id: 131] -Output [2]: [d_date_sk#13, d_year#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] - -(14) CometColumnarExchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(15) CometSort -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] - -(16) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) - -(18) CometProject -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(19) CometExchange -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter - -(22) CometProject -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21], [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] - -(23) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 6] -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] - -(25) Filter [codegen id : 6] -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#22) - -(26) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#22] -Right keys [1]: [i_item_sk#27] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] -Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] - -(29) ReusedExchange [Reuses operator id: 131] -Output [2]: [d_date_sk#32, d_year#33] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#32] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_date_sk#32, d_year#33] - -(32) CometColumnarExchange -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(33) CometSort -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33], [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] - -(34) CometNativeScan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37, sr_returned_date_sk#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [5]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37, sr_returned_date_sk#38] -Condition : (isnotnull(sr_ticket_number#35) AND isnotnull(sr_item_sk#34)) - -(36) CometProject -Input [5]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37, sr_returned_date_sk#38] -Arguments: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37], [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] - -(37) CometExchange -Input [4]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: hashpartitioning(sr_ticket_number#35, sr_item_sk#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(38) CometSort -Input [4]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37], [sr_ticket_number#35 ASC NULLS FIRST, sr_item_sk#34 ASC NULLS FIRST] - -(39) CometSortMergeJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Right output [4]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#35, sr_item_sk#34], LeftOuter - -(40) CometProject -Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33, sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: [d_year#33, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, sales_cnt#39, sales_amt#40], [d_year#33, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, (ss_quantity#24 - coalesce(sr_return_quantity#36, 0)) AS sales_cnt#39, (ss_ext_sales_price#25 - coalesce(sr_return_amt#37, 0.00)) AS sales_amt#40] - -(41) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(42) ColumnarToRow [codegen id : 9] -Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] - -(43) Filter [codegen id : 9] -Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -Condition : isnotnull(ws_item_sk#41) - -(44) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] - -(45) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#41] -Right keys [1]: [i_item_sk#46] -Join type: Inner -Join condition: None - -(46) Project [codegen id : 9] -Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] -Input [10]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] - -(47) ReusedExchange [Reuses operator id: 131] -Output [2]: [d_date_sk#51, d_year#52] - -(48) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#45] -Right keys [1]: [d_date_sk#51] -Join type: Inner -Join condition: None - -(49) Project [codegen id : 9] -Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Input [11]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_date_sk#51, d_year#52] - -(50) CometColumnarExchange -Input [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Arguments: hashpartitioning(ws_order_number#42, ws_item_sk#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(51) CometSort -Input [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Arguments: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52], [ws_order_number#42 ASC NULLS FIRST, ws_item_sk#41 ASC NULLS FIRST] - -(52) CometNativeScan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56, wr_returned_date_sk#57] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(53) CometFilter -Input [5]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56, wr_returned_date_sk#57] -Condition : (isnotnull(wr_order_number#54) AND isnotnull(wr_item_sk#53)) - -(54) CometProject -Input [5]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56, wr_returned_date_sk#57] -Arguments: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56], [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] - -(55) CometExchange -Input [4]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: hashpartitioning(wr_order_number#54, wr_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(56) CometSort -Input [4]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56], [wr_order_number#54 ASC NULLS FIRST, wr_item_sk#53 ASC NULLS FIRST] - -(57) CometSortMergeJoin -Left output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Right output [4]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: [ws_order_number#42, ws_item_sk#41], [wr_order_number#54, wr_item_sk#53], LeftOuter - -(58) CometProject -Input [13]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52, wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: [d_year#52, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, sales_cnt#58, sales_amt#59], [d_year#52, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, (ws_quantity#43 - coalesce(wr_return_quantity#55, 0)) AS sales_cnt#58, (ws_ext_sales_price#44 - coalesce(wr_return_amt#56, 0.00)) AS sales_amt#59] - -(59) CometUnion -Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#33, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, sales_cnt#39, sales_amt#40] -Child 2 Input [7]: [d_year#52, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, sales_cnt#58, sales_amt#59] - -(60) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(61) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(62) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(63) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(64) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#60, sum#61] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(65) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#60, sum#61] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(66) CometFilter -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Condition : isnotnull(sales_cnt#62) - -(67) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(68) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] - -(69) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#68), dynamicpruningexpression(cs_sold_date_sk#68 IN dynamicpruning#69)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(70) ColumnarToRow [codegen id : 12] -Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] - -(71) Filter [codegen id : 12] -Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] -Condition : isnotnull(cs_item_sk#64) - -(72) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] - -(73) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cs_item_sk#64] -Right keys [1]: [i_item_sk#70] -Join type: Inner -Join condition: None - -(74) Project [codegen id : 12] -Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -Input [10]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] - -(75) ReusedExchange [Reuses operator id: 135] -Output [2]: [d_date_sk#75, d_year#76] - -(76) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cs_sold_date_sk#68] -Right keys [1]: [d_date_sk#75] -Join type: Inner -Join condition: None - -(77) Project [codegen id : 12] -Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Input [11]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_date_sk#75, d_year#76] - -(78) CometColumnarExchange -Input [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Arguments: hashpartitioning(cs_order_number#65, cs_item_sk#64, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(79) CometSort -Input [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Arguments: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76], [cs_order_number#65 ASC NULLS FIRST, cs_item_sk#64 ASC NULLS FIRST] - -(80) ReusedExchange [Reuses operator id: 19] -Output [4]: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] - -(81) CometSort -Input [4]: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] -Arguments: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80], [cr_order_number#78 ASC NULLS FIRST, cr_item_sk#77 ASC NULLS FIRST] - -(82) CometSortMergeJoin -Left output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Right output [4]: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] -Arguments: [cs_order_number#65, cs_item_sk#64], [cr_order_number#78, cr_item_sk#77], LeftOuter - -(83) CometProject -Input [13]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76, cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] -Arguments: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#81, sales_amt#82], [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, (cs_quantity#66 - coalesce(cr_return_quantity#79, 0)) AS sales_cnt#81, (cs_ext_sales_price#67 - coalesce(cr_return_amount#80, 0.00)) AS sales_amt#82] - -(84) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#69)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(85) ColumnarToRow [codegen id : 15] -Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] - -(86) Filter [codegen id : 15] -Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Condition : isnotnull(ss_item_sk#83) - -(87) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#88, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92] - -(88) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ss_item_sk#83] -Right keys [1]: [i_item_sk#88] -Join type: Inner -Join condition: None - -(89) Project [codegen id : 15] -Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92] -Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#88, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92] - -(90) ReusedExchange [Reuses operator id: 135] -Output [2]: [d_date_sk#93, d_year#94] - -(91) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ss_sold_date_sk#87] -Right keys [1]: [d_date_sk#93] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 15] -Output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, d_year#94] -Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, d_date_sk#93, d_year#94] - -(93) CometColumnarExchange -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, d_year#94] -Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(94) CometSort -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, d_year#94] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, d_year#94], [ss_ticket_number#84 ASC NULLS FIRST, ss_item_sk#83 ASC NULLS FIRST] - -(95) ReusedExchange [Reuses operator id: 37] -Output [4]: [sr_item_sk#95, sr_ticket_number#96, sr_return_quantity#97, sr_return_amt#98] - -(96) CometSort -Input [4]: [sr_item_sk#95, sr_ticket_number#96, sr_return_quantity#97, sr_return_amt#98] -Arguments: [sr_item_sk#95, sr_ticket_number#96, sr_return_quantity#97, sr_return_amt#98], [sr_ticket_number#96 ASC NULLS FIRST, sr_item_sk#95 ASC NULLS FIRST] - -(97) CometSortMergeJoin -Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, d_year#94] -Right output [4]: [sr_item_sk#95, sr_ticket_number#96, sr_return_quantity#97, sr_return_amt#98] -Arguments: [ss_ticket_number#84, ss_item_sk#83], [sr_ticket_number#96, sr_item_sk#95], LeftOuter - -(98) CometProject -Input [13]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, d_year#94, sr_item_sk#95, sr_ticket_number#96, sr_return_quantity#97, sr_return_amt#98] -Arguments: [d_year#94, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, sales_cnt#99, sales_amt#100], [d_year#94, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, (ss_quantity#85 - coalesce(sr_return_quantity#97, 0)) AS sales_cnt#99, (ss_ext_sales_price#86 - coalesce(sr_return_amt#98, 0.00)) AS sales_amt#100] - -(99) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#105), dynamicpruningexpression(ws_sold_date_sk#105 IN dynamicpruning#69)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(100) ColumnarToRow [codegen id : 18] -Input [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105] - -(101) Filter [codegen id : 18] -Input [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105] -Condition : isnotnull(ws_item_sk#101) - -(102) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] - -(103) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_item_sk#101] -Right keys [1]: [i_item_sk#106] -Join type: Inner -Join condition: None - -(104) Project [codegen id : 18] -Output [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Input [10]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] - -(105) ReusedExchange [Reuses operator id: 135] -Output [2]: [d_date_sk#111, d_year#112] - -(106) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#105] -Right keys [1]: [d_date_sk#111] -Join type: Inner -Join condition: None - -(107) Project [codegen id : 18] -Output [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Input [11]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] - -(108) CometColumnarExchange -Input [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: hashpartitioning(ws_order_number#102, ws_item_sk#101, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(109) CometSort -Input [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_order_number#102 ASC NULLS FIRST, ws_item_sk#101 ASC NULLS FIRST] - -(110) ReusedExchange [Reuses operator id: 55] -Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] - -(111) CometSort -Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116], [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST] - -(112) CometSortMergeJoin -Left output [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Right output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [ws_order_number#102, ws_item_sk#101], [wr_order_number#114, wr_item_sk#113], LeftOuter - -(113) CometProject -Input [13]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#117, sales_amt#118], [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#103 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#117, (ws_ext_sales_price#104 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#118] - -(114) CometUnion -Child 0 Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#81, sales_amt#82] -Child 1 Input [7]: [d_year#94, i_brand_id#89, i_class_id#90, i_category_id#91, i_manufact_id#92, sales_cnt#99, sales_amt#100] -Child 2 Input [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#117, sales_amt#118] - -(115) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#81, sales_amt#82] -Keys [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#81, sales_amt#82] -Functions: [] - -(116) CometExchange -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#81, sales_amt#82] -Arguments: hashpartitioning(d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#81, sales_amt#82, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(117) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#81, sales_amt#82] -Keys [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#81, sales_amt#82] -Functions: [] - -(118) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#81, sales_amt#82] -Keys [5]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -Functions [2]: [partial_sum(sales_cnt#81), partial_sum(UnscaledValue(sales_amt#82))] - -(119) CometExchange -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sum#119, sum#120] -Arguments: hashpartitioning(d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(120) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sum#119, sum#120] -Keys [5]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -Functions [2]: [sum(sales_cnt#81), sum(UnscaledValue(sales_amt#82))] - -(121) CometFilter -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#121, sales_amt#122] -Condition : isnotnull(sales_cnt#121) - -(122) CometExchange -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#121, sales_amt#122] -Arguments: hashpartitioning(i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=16] - -(123) CometSort -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#121, sales_amt#122] -Arguments: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#121, sales_amt#122], [i_brand_id#71 ASC NULLS FIRST, i_class_id#72 ASC NULLS FIRST, i_category_id#73 ASC NULLS FIRST, i_manufact_id#74 ASC NULLS FIRST] - -(124) CometSortMergeJoin -Left output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Right output [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#121, sales_amt#122] -Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74], Inner, ((cast(sales_cnt#62 as decimal(17,2)) / cast(sales_cnt#121 as decimal(17,2))) < 0.90000000000000000000) - -(125) CometProject -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#121, sales_amt#122] -Arguments: [prev_year#123, year#124, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#125, curr_yr_cnt#126, sales_cnt_diff#127, sales_amt_diff#128], [d_year#76 AS prev_year#123, d_year#14 AS year#124, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#121 AS prev_yr_cnt#125, sales_cnt#62 AS curr_yr_cnt#126, (sales_cnt#62 - sales_cnt#121) AS sales_cnt_diff#127, (sales_amt#63 - sales_amt#122) AS sales_amt_diff#128] - -(126) CometTakeOrderedAndProject -Input [10]: [prev_year#123, year#124, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#125, curr_yr_cnt#126, sales_cnt_diff#127, sales_amt_diff#128] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#127 ASC NULLS FIRST,sales_amt_diff#128 ASC NULLS FIRST], output=[prev_year#123,year#124,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#125,curr_yr_cnt#126,sales_cnt_diff#127,sales_amt_diff#128]), [prev_year#123, year#124, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#125, curr_yr_cnt#126, sales_cnt_diff#127, sales_amt_diff#128], 100, 0, [sales_cnt_diff#127 ASC NULLS FIRST, sales_amt_diff#128 ASC NULLS FIRST], [prev_year#123, year#124, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#125, curr_yr_cnt#126, sales_cnt_diff#127, sales_amt_diff#128] - -(127) CometColumnarToRow [codegen id : 19] -Input [10]: [prev_year#123, year#124, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#125, curr_yr_cnt#126, sales_cnt_diff#127, sales_amt_diff#128] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (131) -+- * CometColumnarToRow (130) - +- CometFilter (129) - +- CometNativeScan parquet spark_catalog.default.date_dim (128) - - -(128) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(129) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(130) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_year#14] - -(131) BroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#6 - -Subquery:4 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#68 IN dynamicpruning#69 -BroadcastExchange (135) -+- * CometColumnarToRow (134) - +- CometFilter (133) - +- CometNativeScan parquet spark_catalog.default.date_dim (132) - - -(132) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#75, d_year#76] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(133) CometFilter -Input [2]: [d_date_sk#75, d_year#76] -Condition : ((isnotnull(d_year#76) AND (d_year#76 = 2001)) AND isnotnull(d_date_sk#75)) - -(134) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#75, d_year#76] - -(135) BroadcastExchange -Input [2]: [d_date_sk#75, d_year#76] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] - -Subquery:5 Hosting operator id = 84 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#69 - -Subquery:6 Hosting operator id = 99 Hosting Expression = ws_sold_date_sk#105 IN dynamicpruning#69 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/simplified.txt deleted file mode 100644 index aaf98f4b09..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/simplified.txt +++ /dev/null @@ -1,173 +0,0 @@ -WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometColumnarExchange [cs_order_number,cs_item_sk] #4 - WholeStageCodegen (3) - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [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] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #7 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 - WholeStageCodegen (6) - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #9 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometColumnarExchange [ws_order_number,ws_item_sk] #10 - WholeStageCodegen (9) - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #11 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometColumnarExchange [cs_order_number,cs_item_sk] #15 - WholeStageCodegen (12) - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 #16 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 - WholeStageCodegen (15) - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometColumnarExchange [ws_order_number,ws_item_sk] #18 - WholeStageCodegen (18) - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/explain.txt deleted file mode 100644 index ea5d8ded80..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,742 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (124) -+- CometTakeOrderedAndProject (123) - +- CometProject (122) - +- CometSortMergeJoin (121) - :- CometSort (66) - : +- CometExchange (65) - : +- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometHashAggregate (60) - : +- CometExchange (59) - : +- CometHashAggregate (58) - : +- CometUnion (57) - : :- CometProject (22) - : : +- CometSortMergeJoin (21) - : : :- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometSort (20) - : : +- CometExchange (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (39) - : : +- CometSortMergeJoin (38) - : : :- CometSort (32) - : : : +- CometExchange (31) - : : : +- CometProject (30) - : : : +- CometBroadcastHashJoin (29) - : : : :- CometProject (27) - : : : : +- CometBroadcastHashJoin (26) - : : : : :- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (25) - : : : +- ReusedExchange (28) - : : +- CometSort (37) - : : +- CometExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (33) - : +- CometProject (56) - : +- CometSortMergeJoin (55) - : :- CometSort (49) - : : +- CometExchange (48) - : : +- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometProject (44) - : : : +- CometBroadcastHashJoin (43) - : : : :- CometFilter (41) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) - : : : +- ReusedExchange (42) - : : +- ReusedExchange (45) - : +- CometSort (54) - : +- CometExchange (53) - : +- CometProject (52) - : +- CometFilter (51) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (50) - +- CometSort (120) - +- CometExchange (119) - +- CometFilter (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometHashAggregate (114) - +- CometExchange (113) - +- CometHashAggregate (112) - +- CometUnion (111) - :- CometProject (82) - : +- CometSortMergeJoin (81) - : :- CometSort (78) - : : +- CometExchange (77) - : : +- CometProject (76) - : : +- CometBroadcastHashJoin (75) - : : :- CometProject (71) - : : : +- CometBroadcastHashJoin (70) - : : : :- CometFilter (68) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) - : : : +- ReusedExchange (69) - : : +- CometBroadcastExchange (74) - : : +- CometFilter (73) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - : +- CometSort (80) - : +- ReusedExchange (79) - :- CometProject (96) - : +- CometSortMergeJoin (95) - : :- CometSort (92) - : : +- CometExchange (91) - : : +- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (87) - : : : +- CometBroadcastHashJoin (86) - : : : :- CometFilter (84) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (83) - : : : +- ReusedExchange (85) - : : +- ReusedExchange (88) - : +- CometSort (94) - : +- ReusedExchange (93) - +- CometProject (110) - +- CometSortMergeJoin (109) - :- CometSort (106) - : +- CometExchange (105) - : +- CometProject (104) - : +- CometBroadcastHashJoin (103) - : :- CometProject (101) - : : +- CometBroadcastHashJoin (100) - : : :- CometFilter (98) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (97) - : : +- ReusedExchange (99) - : +- ReusedExchange (102) - +- CometSort (108) - +- ReusedExchange (107) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((isnotnull(i_category#11) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#11, 50)) = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) - -(5) CometProject -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(6) CometBroadcastExchange -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(7) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Right output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: [d_date_sk#13, d_year#14] - -(12) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Right output [2]: [d_date_sk#13, d_year#14] -Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight - -(13) CometProject -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] - -(14) CometExchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(15) CometSort -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) - -(18) CometProject -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(19) CometExchange -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter - -(22) CometProject -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21], [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(24) CometFilter -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#22) - -(25) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(26) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Right output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22], [i_item_sk#28], Inner, BuildRight - -(27) CometProject -Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(28) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#33, d_year#34] - -(29) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Right output [2]: [d_date_sk#33, d_year#34] -Arguments: [ss_sold_date_sk#26], [d_date_sk#33], Inner, BuildRight - -(30) CometProject -Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_date_sk#33, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] - -(31) CometExchange -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometSort -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) - -(35) CometProject -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] - -(36) CometExchange -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(37) CometSort -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] - -(38) CometSortMergeJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter - -(39) CometProject -Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#47)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(41) CometFilter -Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Condition : isnotnull(ws_item_sk#42) - -(42) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(43) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Right output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42], [i_item_sk#48], Inner, BuildRight - -(44) CometProject -Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(45) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#53, d_year#54] - -(46) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [ws_sold_date_sk#46], [d_date_sk#53], Inner, BuildRight - -(47) CometProject -Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] - -(48) CometExchange -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(49) CometSort -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) - -(52) CometProject -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] - -(53) CometExchange -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(54) CometSort -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] - -(55) CometSortMergeJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter - -(56) CometProject -Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] - -(57) CometUnion -Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] - -(58) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(59) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(60) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(61) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(62) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(63) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(64) CometFilter -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Condition : isnotnull(sales_cnt#64) - -(65) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(66) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(68) CometFilter -Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Condition : isnotnull(cs_item_sk#66) - -(69) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(70) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Right output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66], [i_item_sk#72], Inner, BuildRight - -(71) CometProject -Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(74) CometBroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: [d_date_sk#77, d_year#78] - -(75) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Right output [2]: [d_date_sk#77, d_year#78] -Arguments: [cs_sold_date_sk#70], [d_date_sk#77], Inner, BuildRight - -(76) CometProject -Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] - -(77) CometExchange -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(78) CometSort -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] - -(79) ReusedExchange [Reuses operator id: 19] -Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] - -(80) CometSort -Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] - -(81) CometSortMergeJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter - -(82) CometProject -Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#83, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#84] - -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#89), dynamicpruningexpression(ss_sold_date_sk#89 IN dynamicpruning#90)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(84) CometFilter -Input [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] -Condition : isnotnull(ss_item_sk#85) - -(85) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] - -(86) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] -Right output [5]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] -Arguments: [ss_item_sk#85], [i_item_sk#91], Inner, BuildRight - -(87) CometProject -Input [10]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] -Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95], [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] - -(88) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#96, d_year#97] - -(89) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] -Right output [2]: [d_date_sk#96, d_year#97] -Arguments: [ss_sold_date_sk#89], [d_date_sk#96], Inner, BuildRight - -(90) CometProject -Input [11]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_date_sk#96, d_year#97] -Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97], [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] - -(91) CometExchange -Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] -Arguments: hashpartitioning(ss_ticket_number#86, ss_item_sk#85, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(92) CometSort -Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] -Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97], [ss_ticket_number#86 ASC NULLS FIRST, ss_item_sk#85 ASC NULLS FIRST] - -(93) ReusedExchange [Reuses operator id: 36] -Output [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] - -(94) CometSort -Input [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] -Arguments: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101], [sr_ticket_number#99 ASC NULLS FIRST, sr_item_sk#98 ASC NULLS FIRST] - -(95) CometSortMergeJoin -Left output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] -Right output [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] -Arguments: [ss_ticket_number#86, ss_item_sk#85], [sr_ticket_number#99, sr_item_sk#98], LeftOuter - -(96) CometProject -Input [13]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97, sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] -Arguments: [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, sales_cnt#102, sales_amt#103], [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, (ss_quantity#87 - coalesce(sr_return_quantity#100, 0)) AS sales_cnt#102, (ss_ext_sales_price#88 - coalesce(sr_return_amt#101, 0.00)) AS sales_amt#103] - -(97) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_sold_date_sk#108 IN dynamicpruning#109)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(98) CometFilter -Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] -Condition : isnotnull(ws_item_sk#104) - -(99) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] - -(100) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] -Right output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -Arguments: [ws_item_sk#104], [i_item_sk#110], Inner, BuildRight - -(101) CometProject -Input [10]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114], [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] - -(102) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#115, d_year#116] - -(103) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -Right output [2]: [d_date_sk#115, d_year#116] -Arguments: [ws_sold_date_sk#108], [d_date_sk#115], Inner, BuildRight - -(104) CometProject -Input [11]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_date_sk#115, d_year#116] -Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116], [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] - -(105) CometExchange -Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -Arguments: hashpartitioning(ws_order_number#105, ws_item_sk#104, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(106) CometSort -Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116], [ws_order_number#105 ASC NULLS FIRST, ws_item_sk#104 ASC NULLS FIRST] - -(107) ReusedExchange [Reuses operator id: 53] -Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] - -(108) CometSort -Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -Arguments: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120], [wr_order_number#118 ASC NULLS FIRST, wr_item_sk#117 ASC NULLS FIRST] - -(109) CometSortMergeJoin -Left output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -Right output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -Arguments: [ws_order_number#105, ws_item_sk#104], [wr_order_number#118, wr_item_sk#117], LeftOuter - -(110) CometProject -Input [13]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116, wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -Arguments: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, sales_cnt#121, sales_amt#122], [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, (ws_quantity#106 - coalesce(wr_return_quantity#119, 0)) AS sales_cnt#121, (ws_ext_sales_price#107 - coalesce(wr_return_amt#120, 0.00)) AS sales_amt#122] - -(111) CometUnion -Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Child 1 Input [7]: [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, sales_cnt#102, sales_amt#103] -Child 2 Input [7]: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, sales_cnt#121, sales_amt#122] - -(112) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Functions: [] - -(113) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] - -(114) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Functions: [] - -(115) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [partial_sum(sales_cnt#83), partial_sum(UnscaledValue(sales_amt#84))] - -(116) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#123, sum#124] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(117) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#123, sum#124] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [sum(sales_cnt#83), sum(UnscaledValue(sales_amt#84))] - -(118) CometFilter -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126] -Condition : isnotnull(sales_cnt#125) - -(119) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126] -Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(120) CometSort -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] - -(121) CometSortMergeJoin -Left output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Right output [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126] -Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], Inner, ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#125 as decimal(17,2))) < 0.90000000000000000000) - -(122) CometProject -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126] -Arguments: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132], [d_year#78 AS prev_year#127, d_year#14 AS year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#125 AS prev_yr_cnt#129, sales_cnt#64 AS curr_yr_cnt#130, (sales_cnt#64 - sales_cnt#125) AS sales_cnt_diff#131, (sales_amt#65 - sales_amt#126) AS sales_amt_diff#132] - -(123) CometTakeOrderedAndProject -Input [10]: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#131 ASC NULLS FIRST,sales_amt_diff#132 ASC NULLS FIRST], output=[prev_year#127,year#128,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#129,curr_yr_cnt#130,sales_cnt_diff#131,sales_amt_diff#132]), [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132], 100, 0, [sales_cnt_diff#131 ASC NULLS FIRST, sales_amt_diff#132 ASC NULLS FIRST], [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] - -(124) CometColumnarToRow [codegen id : 1] -Input [10]: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (128) -+- * CometColumnarToRow (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) - - -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(127) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_year#14] - -(128) BroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 - -Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 -BroadcastExchange (132) -+- * CometColumnarToRow (131) - +- CometFilter (130) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) - - -(129) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(130) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(131) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#77, d_year#78] - -(132) BroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] - -Subquery:5 Hosting operator id = 83 Hosting Expression = ss_sold_date_sk#89 IN dynamicpruning#71 - -Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#71 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/simplified.txt deleted file mode 100644 index d0520c8b63..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,144 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #4 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange [d_date_sk,d_year] #7 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #8 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #9 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #10 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #11 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #12 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #16 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_year] #18 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #19 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #20 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/explain.txt deleted file mode 100644 index ea5d8ded80..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/explain.txt +++ /dev/null @@ -1,742 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (124) -+- CometTakeOrderedAndProject (123) - +- CometProject (122) - +- CometSortMergeJoin (121) - :- CometSort (66) - : +- CometExchange (65) - : +- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometHashAggregate (60) - : +- CometExchange (59) - : +- CometHashAggregate (58) - : +- CometUnion (57) - : :- CometProject (22) - : : +- CometSortMergeJoin (21) - : : :- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometSort (20) - : : +- CometExchange (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (39) - : : +- CometSortMergeJoin (38) - : : :- CometSort (32) - : : : +- CometExchange (31) - : : : +- CometProject (30) - : : : +- CometBroadcastHashJoin (29) - : : : :- CometProject (27) - : : : : +- CometBroadcastHashJoin (26) - : : : : :- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (25) - : : : +- ReusedExchange (28) - : : +- CometSort (37) - : : +- CometExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (33) - : +- CometProject (56) - : +- CometSortMergeJoin (55) - : :- CometSort (49) - : : +- CometExchange (48) - : : +- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometProject (44) - : : : +- CometBroadcastHashJoin (43) - : : : :- CometFilter (41) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) - : : : +- ReusedExchange (42) - : : +- ReusedExchange (45) - : +- CometSort (54) - : +- CometExchange (53) - : +- CometProject (52) - : +- CometFilter (51) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (50) - +- CometSort (120) - +- CometExchange (119) - +- CometFilter (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometHashAggregate (114) - +- CometExchange (113) - +- CometHashAggregate (112) - +- CometUnion (111) - :- CometProject (82) - : +- CometSortMergeJoin (81) - : :- CometSort (78) - : : +- CometExchange (77) - : : +- CometProject (76) - : : +- CometBroadcastHashJoin (75) - : : :- CometProject (71) - : : : +- CometBroadcastHashJoin (70) - : : : :- CometFilter (68) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) - : : : +- ReusedExchange (69) - : : +- CometBroadcastExchange (74) - : : +- CometFilter (73) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - : +- CometSort (80) - : +- ReusedExchange (79) - :- CometProject (96) - : +- CometSortMergeJoin (95) - : :- CometSort (92) - : : +- CometExchange (91) - : : +- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (87) - : : : +- CometBroadcastHashJoin (86) - : : : :- CometFilter (84) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (83) - : : : +- ReusedExchange (85) - : : +- ReusedExchange (88) - : +- CometSort (94) - : +- ReusedExchange (93) - +- CometProject (110) - +- CometSortMergeJoin (109) - :- CometSort (106) - : +- CometExchange (105) - : +- CometProject (104) - : +- CometBroadcastHashJoin (103) - : :- CometProject (101) - : : +- CometBroadcastHashJoin (100) - : : :- CometFilter (98) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (97) - : : +- ReusedExchange (99) - : +- ReusedExchange (102) - +- CometSort (108) - +- ReusedExchange (107) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((isnotnull(i_category#11) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#11, 50)) = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) - -(5) CometProject -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(6) CometBroadcastExchange -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(7) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Right output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: [d_date_sk#13, d_year#14] - -(12) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Right output [2]: [d_date_sk#13, d_year#14] -Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight - -(13) CometProject -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] - -(14) CometExchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(15) CometSort -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) - -(18) CometProject -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(19) CometExchange -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter - -(22) CometProject -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21], [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(24) CometFilter -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#22) - -(25) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(26) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Right output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22], [i_item_sk#28], Inner, BuildRight - -(27) CometProject -Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(28) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#33, d_year#34] - -(29) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Right output [2]: [d_date_sk#33, d_year#34] -Arguments: [ss_sold_date_sk#26], [d_date_sk#33], Inner, BuildRight - -(30) CometProject -Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_date_sk#33, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] - -(31) CometExchange -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometSort -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) - -(35) CometProject -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] - -(36) CometExchange -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(37) CometSort -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] - -(38) CometSortMergeJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter - -(39) CometProject -Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#47)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(41) CometFilter -Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Condition : isnotnull(ws_item_sk#42) - -(42) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(43) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Right output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42], [i_item_sk#48], Inner, BuildRight - -(44) CometProject -Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(45) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#53, d_year#54] - -(46) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [ws_sold_date_sk#46], [d_date_sk#53], Inner, BuildRight - -(47) CometProject -Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] - -(48) CometExchange -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(49) CometSort -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) - -(52) CometProject -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] - -(53) CometExchange -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(54) CometSort -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] - -(55) CometSortMergeJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter - -(56) CometProject -Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] - -(57) CometUnion -Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] - -(58) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(59) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(60) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(61) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(62) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(63) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(64) CometFilter -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Condition : isnotnull(sales_cnt#64) - -(65) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(66) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(68) CometFilter -Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Condition : isnotnull(cs_item_sk#66) - -(69) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(70) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Right output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66], [i_item_sk#72], Inner, BuildRight - -(71) CometProject -Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(74) CometBroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: [d_date_sk#77, d_year#78] - -(75) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Right output [2]: [d_date_sk#77, d_year#78] -Arguments: [cs_sold_date_sk#70], [d_date_sk#77], Inner, BuildRight - -(76) CometProject -Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] - -(77) CometExchange -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(78) CometSort -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] - -(79) ReusedExchange [Reuses operator id: 19] -Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] - -(80) CometSort -Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] - -(81) CometSortMergeJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter - -(82) CometProject -Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#83, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#84] - -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#89), dynamicpruningexpression(ss_sold_date_sk#89 IN dynamicpruning#90)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(84) CometFilter -Input [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] -Condition : isnotnull(ss_item_sk#85) - -(85) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] - -(86) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] -Right output [5]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] -Arguments: [ss_item_sk#85], [i_item_sk#91], Inner, BuildRight - -(87) CometProject -Input [10]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] -Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95], [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] - -(88) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#96, d_year#97] - -(89) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] -Right output [2]: [d_date_sk#96, d_year#97] -Arguments: [ss_sold_date_sk#89], [d_date_sk#96], Inner, BuildRight - -(90) CometProject -Input [11]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_date_sk#96, d_year#97] -Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97], [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] - -(91) CometExchange -Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] -Arguments: hashpartitioning(ss_ticket_number#86, ss_item_sk#85, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(92) CometSort -Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] -Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97], [ss_ticket_number#86 ASC NULLS FIRST, ss_item_sk#85 ASC NULLS FIRST] - -(93) ReusedExchange [Reuses operator id: 36] -Output [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] - -(94) CometSort -Input [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] -Arguments: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101], [sr_ticket_number#99 ASC NULLS FIRST, sr_item_sk#98 ASC NULLS FIRST] - -(95) CometSortMergeJoin -Left output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] -Right output [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] -Arguments: [ss_ticket_number#86, ss_item_sk#85], [sr_ticket_number#99, sr_item_sk#98], LeftOuter - -(96) CometProject -Input [13]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97, sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] -Arguments: [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, sales_cnt#102, sales_amt#103], [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, (ss_quantity#87 - coalesce(sr_return_quantity#100, 0)) AS sales_cnt#102, (ss_ext_sales_price#88 - coalesce(sr_return_amt#101, 0.00)) AS sales_amt#103] - -(97) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_sold_date_sk#108 IN dynamicpruning#109)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(98) CometFilter -Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] -Condition : isnotnull(ws_item_sk#104) - -(99) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] - -(100) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] -Right output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -Arguments: [ws_item_sk#104], [i_item_sk#110], Inner, BuildRight - -(101) CometProject -Input [10]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114], [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] - -(102) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#115, d_year#116] - -(103) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -Right output [2]: [d_date_sk#115, d_year#116] -Arguments: [ws_sold_date_sk#108], [d_date_sk#115], Inner, BuildRight - -(104) CometProject -Input [11]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_date_sk#115, d_year#116] -Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116], [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] - -(105) CometExchange -Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -Arguments: hashpartitioning(ws_order_number#105, ws_item_sk#104, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(106) CometSort -Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116], [ws_order_number#105 ASC NULLS FIRST, ws_item_sk#104 ASC NULLS FIRST] - -(107) ReusedExchange [Reuses operator id: 53] -Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] - -(108) CometSort -Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -Arguments: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120], [wr_order_number#118 ASC NULLS FIRST, wr_item_sk#117 ASC NULLS FIRST] - -(109) CometSortMergeJoin -Left output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -Right output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -Arguments: [ws_order_number#105, ws_item_sk#104], [wr_order_number#118, wr_item_sk#117], LeftOuter - -(110) CometProject -Input [13]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116, wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -Arguments: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, sales_cnt#121, sales_amt#122], [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, (ws_quantity#106 - coalesce(wr_return_quantity#119, 0)) AS sales_cnt#121, (ws_ext_sales_price#107 - coalesce(wr_return_amt#120, 0.00)) AS sales_amt#122] - -(111) CometUnion -Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Child 1 Input [7]: [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, sales_cnt#102, sales_amt#103] -Child 2 Input [7]: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, sales_cnt#121, sales_amt#122] - -(112) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Functions: [] - -(113) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] - -(114) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Functions: [] - -(115) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [partial_sum(sales_cnt#83), partial_sum(UnscaledValue(sales_amt#84))] - -(116) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#123, sum#124] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(117) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#123, sum#124] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [sum(sales_cnt#83), sum(UnscaledValue(sales_amt#84))] - -(118) CometFilter -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126] -Condition : isnotnull(sales_cnt#125) - -(119) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126] -Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(120) CometSort -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] - -(121) CometSortMergeJoin -Left output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Right output [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126] -Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], Inner, ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#125 as decimal(17,2))) < 0.90000000000000000000) - -(122) CometProject -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126] -Arguments: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132], [d_year#78 AS prev_year#127, d_year#14 AS year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#125 AS prev_yr_cnt#129, sales_cnt#64 AS curr_yr_cnt#130, (sales_cnt#64 - sales_cnt#125) AS sales_cnt_diff#131, (sales_amt#65 - sales_amt#126) AS sales_amt_diff#132] - -(123) CometTakeOrderedAndProject -Input [10]: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#131 ASC NULLS FIRST,sales_amt_diff#132 ASC NULLS FIRST], output=[prev_year#127,year#128,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#129,curr_yr_cnt#130,sales_cnt_diff#131,sales_amt_diff#132]), [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132], 100, 0, [sales_cnt_diff#131 ASC NULLS FIRST, sales_amt_diff#132 ASC NULLS FIRST], [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] - -(124) CometColumnarToRow [codegen id : 1] -Input [10]: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (128) -+- * CometColumnarToRow (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) - - -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(127) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_year#14] - -(128) BroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 - -Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 -BroadcastExchange (132) -+- * CometColumnarToRow (131) - +- CometFilter (130) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) - - -(129) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(130) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(131) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#77, d_year#78] - -(132) BroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] - -Subquery:5 Hosting operator id = 83 Hosting Expression = ss_sold_date_sk#89 IN dynamicpruning#71 - -Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#71 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/extended.txt deleted file mode 100644 index 22ccc17641..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/extended.txt +++ /dev/null @@ -1,172 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : +- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - +- CometSort - +- CometExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - -Comet accelerated 159 out of 167 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d0520c8b63..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt +++ /dev/null @@ -1,144 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #4 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange [d_date_sk,d_year] #7 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #8 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #9 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #10 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #11 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #12 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #16 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_year] #18 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #19 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #20 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/explain.txt deleted file mode 100644 index 6df18600ea..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/explain.txt +++ /dev/null @@ -1,678 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (110) -+- CometTakeOrderedAndProject (109) - +- CometHashAggregate (108) - +- CometColumnarExchange (107) - +- * HashAggregate (106) - +- Union (105) - :- * HashAggregate (90) - : +- * CometColumnarToRow (89) - : +- CometColumnarExchange (88) - : +- * HashAggregate (87) - : +- Union (86) - : :- * Project (32) - : : +- * BroadcastHashJoin LeftOuter BuildRight (31) - : : :- * HashAggregate (16) - : : : +- * CometColumnarToRow (15) - : : : +- CometColumnarExchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * CometColumnarToRow (9) - : : : +- CometFilter (8) - : : : +- CometNativeScan parquet spark_catalog.default.store (7) - : : +- BroadcastExchange (30) - : : +- * HashAggregate (29) - : : +- * CometColumnarToRow (28) - : : +- CometColumnarExchange (27) - : : +- * HashAggregate (26) - : : +- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * Filter (19) - : : : : +- * ColumnarToRow (18) - : : : : +- Scan parquet spark_catalog.default.store_returns (17) - : : : +- ReusedExchange (20) - : : +- ReusedExchange (23) - : :- * Project (53) - : : +- * BroadcastNestedLoopJoin Inner BuildLeft (52) - : : :- BroadcastExchange (42) - : : : +- * HashAggregate (41) - : : : +- * CometColumnarToRow (40) - : : : +- CometColumnarExchange (39) - : : : +- * HashAggregate (38) - : : : +- * Project (37) - : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : :- * ColumnarToRow (34) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (33) - : : : +- ReusedExchange (35) - : : +- * HashAggregate (51) - : : +- * CometColumnarToRow (50) - : : +- CometColumnarExchange (49) - : : +- * HashAggregate (48) - : : +- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * ColumnarToRow (44) - : : : +- Scan parquet spark_catalog.default.catalog_returns (43) - : : +- ReusedExchange (45) - : +- * Project (85) - : +- * BroadcastHashJoin LeftOuter BuildRight (84) - : :- * HashAggregate (69) - : : +- * CometColumnarToRow (68) - : : +- CometColumnarExchange (67) - : : +- * HashAggregate (66) - : : +- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildRight (58) - : : : :- * Filter (56) - : : : : +- * ColumnarToRow (55) - : : : : +- Scan parquet spark_catalog.default.web_sales (54) - : : : +- ReusedExchange (57) - : : +- BroadcastExchange (63) - : : +- * CometColumnarToRow (62) - : : +- CometFilter (61) - : : +- CometNativeScan parquet spark_catalog.default.web_page (60) - : +- BroadcastExchange (83) - : +- * HashAggregate (82) - : +- * CometColumnarToRow (81) - : +- CometColumnarExchange (80) - : +- * HashAggregate (79) - : +- * Project (78) - : +- * BroadcastHashJoin Inner BuildRight (77) - : :- * Project (75) - : : +- * BroadcastHashJoin Inner BuildRight (74) - : : :- * Filter (72) - : : : +- * ColumnarToRow (71) - : : : +- Scan parquet spark_catalog.default.web_returns (70) - : : +- ReusedExchange (73) - : +- ReusedExchange (76) - :- * HashAggregate (97) - : +- * CometColumnarToRow (96) - : +- CometColumnarExchange (95) - : +- * HashAggregate (94) - : +- * HashAggregate (93) - : +- * CometColumnarToRow (92) - : +- ReusedExchange (91) - +- * HashAggregate (104) - +- * CometColumnarToRow (103) - +- CometColumnarExchange (102) - +- * HashAggregate (101) - +- * HashAggregate (100) - +- * CometColumnarToRow (99) - +- ReusedExchange (98) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 3] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(4) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [1]: [s_store_sk#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [1]: [s_store_sk#7] -Condition : isnotnull(s_store_sk#7) - -(9) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#7] - -(10) BroadcastExchange -Input [1]: [s_store_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None - -(12) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] - -(13) HashAggregate [codegen id : 3] -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Keys [1]: [s_store_sk#7] -Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum#8, sum#9] -Results [3]: [s_store_sk#7, sum#10, sum#11] - -(14) CometColumnarExchange -Input [3]: [s_store_sk#7, sum#10, sum#11] -Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(15) CometColumnarToRow [codegen id : 8] -Input [3]: [s_store_sk#7, sum#10, sum#11] - -(16) HashAggregate [codegen id : 8] -Input [3]: [s_store_sk#7, sum#10, sum#11] -Keys [1]: [s_store_sk#7] -Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#12, sum(UnscaledValue(ss_net_profit#3))#13] -Results [3]: [s_store_sk#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS sales#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#13,17,2) AS profit#15] - -(17) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 6] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] - -(19) Filter [codegen id : 6] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_store_sk#16) - -(20) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#20] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_returned_date_sk#19] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] -Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#20] - -(23) ReusedExchange [Reuses operator id: 10] -Output [1]: [s_store_sk#21] - -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_store_sk#16] -Right keys [1]: [s_store_sk#21] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#21] - -(26) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] -Keys [1]: [s_store_sk#21] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum#22, sum#23] -Results [3]: [s_store_sk#21, sum#24, sum#25] - -(27) CometColumnarExchange -Input [3]: [s_store_sk#21, sum#24, sum#25] -Arguments: hashpartitioning(s_store_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#21, sum#24, sum#25] - -(29) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#21, sum#24, sum#25] -Keys [1]: [s_store_sk#21] -Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#17))#26, sum(UnscaledValue(sr_net_loss#18))#27] -Results [3]: [s_store_sk#21, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#26,17,2) AS returns#28, MakeDecimal(sum(UnscaledValue(sr_net_loss#18))#27,17,2) AS profit_loss#29] - -(30) BroadcastExchange -Input [3]: [s_store_sk#21, returns#28, profit_loss#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [s_store_sk#7] -Right keys [1]: [s_store_sk#21] -Join type: LeftOuter -Join condition: None - -(32) Project [codegen id : 8] -Output [5]: [store channel AS channel#30, s_store_sk#7 AS id#31, sales#14, coalesce(returns#28, 0.00) AS returns#32, (profit#15 - coalesce(profit_loss#29, 0.00)) AS profit#33] -Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#21, returns#28, profit_loss#29] - -(33) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#5)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37] - -(35) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#38] - -(36) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#37] -Right keys [1]: [d_date_sk#38] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] -Input [5]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37, d_date_sk#38] - -(38) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] -Keys [1]: [cs_call_center_sk#34] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#35)), partial_sum(UnscaledValue(cs_net_profit#36))] -Aggregate Attributes [2]: [sum#39, sum#40] -Results [3]: [cs_call_center_sk#34, sum#41, sum#42] - -(39) CometColumnarExchange -Input [3]: [cs_call_center_sk#34, sum#41, sum#42] -Arguments: hashpartitioning(cs_call_center_sk#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(40) CometColumnarToRow [codegen id : 11] -Input [3]: [cs_call_center_sk#34, sum#41, sum#42] - -(41) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#34, sum#41, sum#42] -Keys [1]: [cs_call_center_sk#34] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#35)), sum(UnscaledValue(cs_net_profit#36))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#35))#43, sum(UnscaledValue(cs_net_profit#36))#44] -Results [3]: [cs_call_center_sk#34, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#35))#43,17,2) AS sales#45, MakeDecimal(sum(UnscaledValue(cs_net_profit#36))#44,17,2) AS profit#46] - -(42) BroadcastExchange -Input [3]: [cs_call_center_sk#34, sales#45, profit#46] -Arguments: IdentityBroadcastMode, [plan_id=6] - -(43) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#49), dynamicpruningexpression(cr_returned_date_sk#49 IN dynamicpruning#5)] -ReadSchema: struct - -(44) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49] - -(45) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#50] - -(46) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#49] -Right keys [1]: [d_date_sk#50] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 13] -Output [2]: [cr_return_amount#47, cr_net_loss#48] -Input [4]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49, d_date_sk#50] - -(48) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#47, cr_net_loss#48] -Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#47)), partial_sum(UnscaledValue(cr_net_loss#48))] -Aggregate Attributes [2]: [sum#51, sum#52] -Results [2]: [sum#53, sum#54] - -(49) CometColumnarExchange -Input [2]: [sum#53, sum#54] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(50) CometColumnarToRow -Input [2]: [sum#53, sum#54] - -(51) HashAggregate -Input [2]: [sum#53, sum#54] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#47)), sum(UnscaledValue(cr_net_loss#48))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#47))#55, sum(UnscaledValue(cr_net_loss#48))#56] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#47))#55,17,2) AS returns#57, MakeDecimal(sum(UnscaledValue(cr_net_loss#48))#56,17,2) AS profit_loss#58] - -(52) BroadcastNestedLoopJoin [codegen id : 14] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 14] -Output [5]: [catalog channel AS channel#59, cs_call_center_sk#34 AS id#60, sales#45, returns#57, (profit#46 - profit_loss#58) AS profit#61] -Input [5]: [cs_call_center_sk#34, sales#45, profit#46, returns#57, profit_loss#58] - -(54) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] - -(56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] -Condition : isnotnull(ws_web_page_sk#62) - -(57) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#66] - -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#65] -Right keys [1]: [d_date_sk#66] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64] -Input [5]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65, d_date_sk#66] - -(60) CometNativeScan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(61) CometFilter -Input [1]: [wp_web_page_sk#67] -Condition : isnotnull(wp_web_page_sk#67) - -(62) CometColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#67] - -(63) BroadcastExchange -Input [1]: [wp_web_page_sk#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#62] -Right keys [1]: [wp_web_page_sk#67] -Join type: Inner -Join condition: None - -(65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] -Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] - -(66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] -Keys [1]: [wp_web_page_sk#67] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#63)), partial_sum(UnscaledValue(ws_net_profit#64))] -Aggregate Attributes [2]: [sum#68, sum#69] -Results [3]: [wp_web_page_sk#67, sum#70, sum#71] - -(67) CometColumnarExchange -Input [3]: [wp_web_page_sk#67, sum#70, sum#71] -Arguments: hashpartitioning(wp_web_page_sk#67, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(68) CometColumnarToRow [codegen id : 22] -Input [3]: [wp_web_page_sk#67, sum#70, sum#71] - -(69) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#67, sum#70, sum#71] -Keys [1]: [wp_web_page_sk#67] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#63)), sum(UnscaledValue(ws_net_profit#64))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#63))#72, sum(UnscaledValue(ws_net_profit#64))#73] -Results [3]: [wp_web_page_sk#67, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#63))#72,17,2) AS sales#74, MakeDecimal(sum(UnscaledValue(ws_net_profit#64))#73,17,2) AS profit#75] - -(70) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#79), dynamicpruningexpression(wr_returned_date_sk#79 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_web_page_sk)] -ReadSchema: struct - -(71) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] - -(72) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] -Condition : isnotnull(wr_web_page_sk#76) - -(73) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#80] - -(74) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#79] -Right keys [1]: [d_date_sk#80] -Join type: Inner -Join condition: None - -(75) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78] -Input [5]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79, d_date_sk#80] - -(76) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#81] - -(77) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#76] -Right keys [1]: [wp_web_page_sk#81] -Join type: Inner -Join condition: None - -(78) Project [codegen id : 20] -Output [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] -Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] - -(79) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] -Keys [1]: [wp_web_page_sk#81] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#77)), partial_sum(UnscaledValue(wr_net_loss#78))] -Aggregate Attributes [2]: [sum#82, sum#83] -Results [3]: [wp_web_page_sk#81, sum#84, sum#85] - -(80) CometColumnarExchange -Input [3]: [wp_web_page_sk#81, sum#84, sum#85] -Arguments: hashpartitioning(wp_web_page_sk#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(81) CometColumnarToRow [codegen id : 21] -Input [3]: [wp_web_page_sk#81, sum#84, sum#85] - -(82) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#81, sum#84, sum#85] -Keys [1]: [wp_web_page_sk#81] -Functions [2]: [sum(UnscaledValue(wr_return_amt#77)), sum(UnscaledValue(wr_net_loss#78))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#77))#86, sum(UnscaledValue(wr_net_loss#78))#87] -Results [3]: [wp_web_page_sk#81, MakeDecimal(sum(UnscaledValue(wr_return_amt#77))#86,17,2) AS returns#88, MakeDecimal(sum(UnscaledValue(wr_net_loss#78))#87,17,2) AS profit_loss#89] - -(83) BroadcastExchange -Input [3]: [wp_web_page_sk#81, returns#88, profit_loss#89] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -(84) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#67] -Right keys [1]: [wp_web_page_sk#81] -Join type: LeftOuter -Join condition: None - -(85) Project [codegen id : 22] -Output [5]: [web channel AS channel#90, wp_web_page_sk#67 AS id#91, sales#74, coalesce(returns#88, 0.00) AS returns#92, (profit#75 - coalesce(profit_loss#89, 0.00)) AS profit#93] -Input [6]: [wp_web_page_sk#67, sales#74, profit#75, wp_web_page_sk#81, returns#88, profit_loss#89] - -(86) Union - -(87) HashAggregate [codegen id : 23] -Input [5]: [channel#30, id#31, sales#14, returns#32, profit#33] -Keys [2]: [channel#30, id#31] -Functions [3]: [partial_sum(sales#14), partial_sum(returns#32), partial_sum(profit#33)] -Aggregate Attributes [6]: [sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] -Results [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] - -(88) CometColumnarExchange -Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Arguments: hashpartitioning(channel#30, id#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(89) CometColumnarToRow [codegen id : 24] -Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] - -(90) HashAggregate [codegen id : 24] -Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Keys [2]: [channel#30, id#31] -Functions [3]: [sum(sales#14), sum(returns#32), sum(profit#33)] -Aggregate Attributes [3]: [sum(sales#14)#106, sum(returns#32)#107, sum(profit#33)#108] -Results [5]: [channel#30, id#31, cast(sum(sales#14)#106 as decimal(37,2)) AS sales#109, cast(sum(returns#32)#107 as decimal(37,2)) AS returns#110, cast(sum(profit#33)#108 as decimal(38,2)) AS profit#111] - -(91) ReusedExchange [Reuses operator id: 88] -Output [8]: [channel#112, id#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] - -(92) CometColumnarToRow [codegen id : 48] -Input [8]: [channel#112, id#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] - -(93) HashAggregate [codegen id : 48] -Input [8]: [channel#112, id#113, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Keys [2]: [channel#112, id#113] -Functions [3]: [sum(sales#120), sum(returns#121), sum(profit#122)] -Aggregate Attributes [3]: [sum(sales#120)#106, sum(returns#121)#107, sum(profit#122)#108] -Results [4]: [channel#112, sum(sales#120)#106 AS sales#123, sum(returns#121)#107 AS returns#124, sum(profit#122)#108 AS profit#125] - -(94) HashAggregate [codegen id : 48] -Input [4]: [channel#112, sales#123, returns#124, profit#125] -Keys [1]: [channel#112] -Functions [3]: [partial_sum(sales#123), partial_sum(returns#124), partial_sum(profit#125)] -Aggregate Attributes [6]: [sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] -Results [7]: [channel#112, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] - -(95) CometColumnarExchange -Input [7]: [channel#112, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] -Arguments: hashpartitioning(channel#112, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(96) CometColumnarToRow [codegen id : 49] -Input [7]: [channel#112, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] - -(97) HashAggregate [codegen id : 49] -Input [7]: [channel#112, sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] -Keys [1]: [channel#112] -Functions [3]: [sum(sales#123), sum(returns#124), sum(profit#125)] -Aggregate Attributes [3]: [sum(sales#123)#138, sum(returns#124)#139, sum(profit#125)#140] -Results [5]: [channel#112, null AS id#141, sum(sales#123)#138 AS sales#142, sum(returns#124)#139 AS returns#143, sum(profit#125)#140 AS profit#144] - -(98) ReusedExchange [Reuses operator id: 88] -Output [8]: [channel#145, id#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] - -(99) CometColumnarToRow [codegen id : 73] -Input [8]: [channel#145, id#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] - -(100) HashAggregate [codegen id : 73] -Input [8]: [channel#145, id#146, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] -Keys [2]: [channel#145, id#146] -Functions [3]: [sum(sales#153), sum(returns#154), sum(profit#155)] -Aggregate Attributes [3]: [sum(sales#153)#106, sum(returns#154)#107, sum(profit#155)#108] -Results [3]: [sum(sales#153)#106 AS sales#156, sum(returns#154)#107 AS returns#157, sum(profit#155)#108 AS profit#158] - -(101) HashAggregate [codegen id : 73] -Input [3]: [sales#156, returns#157, profit#158] -Keys: [] -Functions [3]: [partial_sum(sales#156), partial_sum(returns#157), partial_sum(profit#158)] -Aggregate Attributes [6]: [sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] -Results [6]: [sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] - -(102) CometColumnarExchange -Input [6]: [sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(103) CometColumnarToRow [codegen id : 74] -Input [6]: [sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] - -(104) HashAggregate [codegen id : 74] -Input [6]: [sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] -Keys: [] -Functions [3]: [sum(sales#156), sum(returns#157), sum(profit#158)] -Aggregate Attributes [3]: [sum(sales#156)#171, sum(returns#157)#172, sum(profit#158)#173] -Results [5]: [null AS channel#174, null AS id#175, sum(sales#156)#171 AS sales#176, sum(returns#157)#172 AS returns#177, sum(profit#158)#173 AS profit#178] - -(105) Union - -(106) HashAggregate [codegen id : 75] -Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] -Keys [5]: [channel#30, id#31, sales#109, returns#110, profit#111] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#30, id#31, sales#109, returns#110, profit#111] - -(107) CometColumnarExchange -Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] -Arguments: hashpartitioning(channel#30, id#31, sales#109, returns#110, profit#111, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] - -(108) CometHashAggregate -Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] -Keys [5]: [channel#30, id#31, sales#109, returns#110, profit#111] -Functions: [] - -(109) CometTakeOrderedAndProject -Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#30 ASC NULLS FIRST,id#31 ASC NULLS FIRST], output=[channel#30,id#31,sales#109,returns#110,profit#111]), [channel#30, id#31, sales#109, returns#110, profit#111], 100, 0, [channel#30 ASC NULLS FIRST, id#31 ASC NULLS FIRST], [channel#30, id#31, sales#109, returns#110, profit#111] - -(110) CometColumnarToRow [codegen id : 76] -Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (115) -+- * CometColumnarToRow (114) - +- CometProject (113) - +- CometFilter (112) - +- CometNativeScan parquet spark_catalog.default.date_dim (111) - - -(111) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#179] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(112) CometFilter -Input [2]: [d_date_sk#6, d_date#179] -Condition : (((isnotnull(d_date#179) AND (d_date#179 >= 1998-08-04)) AND (d_date#179 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) - -(113) CometProject -Input [2]: [d_date_sk#6, d_date#179] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(114) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(115) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] - -Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#49 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 70 Hosting Expression = wr_returned_date_sk#79 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/simplified.txt deleted file mode 100644 index 6f40fcdddf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/simplified.txt +++ /dev/null @@ -1,180 +0,0 @@ -WholeStageCodegen (76) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (75) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (24) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (23) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (8) - Project [s_store_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [s_store_sk,s_store_sk] - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #3 - WholeStageCodegen (3) - HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,s_store_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #7 - WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - Project [sr_return_amt,sr_net_loss,s_store_sk] - BroadcastHashJoin [sr_store_sk,s_store_sk] - Project [sr_store_sk,sr_return_amt,sr_net_loss] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [s_store_sk] #5 - WholeStageCodegen (14) - Project [cs_call_center_sk,sales,returns,profit,profit_loss] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_call_center_sk] #9 - WholeStageCodegen (10) - HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #4 - HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (13) - HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - Project [cr_return_amount,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (22) - Project [wp_web_page_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wp_web_page_sk] #11 - WholeStageCodegen (17) - HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_web_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (21) - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wp_web_page_sk] #14 - WholeStageCodegen (20) - HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - Project [wr_return_amt,wr_net_loss,wp_web_page_sk] - BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] - Project [wr_web_page_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Filter [wr_web_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [wp_web_page_sk] #12 - WholeStageCodegen (49) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #15 - WholeStageCodegen (48) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (74) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #16 - WholeStageCodegen (73) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/explain.txt deleted file mode 100644 index 9554c6732e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,621 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (103) -+- CometTakeOrderedAndProject (102) - +- CometHashAggregate (101) - +- CometColumnarExchange (100) - +- * HashAggregate (99) - +- Union (98) - :- * HashAggregate (83) - : +- * CometColumnarToRow (82) - : +- CometColumnarExchange (81) - : +- * HashAggregate (80) - : +- Union (79) - : :- * CometColumnarToRow (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : : +- CometBroadcastExchange (28) - : : +- CometHashAggregate (27) - : : +- CometExchange (26) - : : +- CometHashAggregate (25) - : : +- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (21) - : : : +- CometBroadcastHashJoin (20) - : : : :- CometFilter (18) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (17) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : :- * Project (50) - : : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) - : : :- BroadcastExchange (40) - : : : +- * CometColumnarToRow (39) - : : : +- CometHashAggregate (38) - : : : +- CometExchange (37) - : : : +- CometHashAggregate (36) - : : : +- CometProject (35) - : : : +- CometBroadcastHashJoin (34) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (32) - : : : +- ReusedExchange (33) - : : +- * CometColumnarToRow (48) - : : +- CometHashAggregate (47) - : : +- CometExchange (46) - : : +- CometHashAggregate (45) - : : +- CometProject (44) - : : +- CometBroadcastHashJoin (43) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (41) - : : +- ReusedExchange (42) - : +- * CometColumnarToRow (78) - : +- CometProject (77) - : +- CometBroadcastHashJoin (76) - : :- CometHashAggregate (63) - : : +- CometExchange (62) - : : +- CometHashAggregate (61) - : : +- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (55) - : : : +- CometBroadcastHashJoin (54) - : : : :- CometFilter (52) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (51) - : : : +- ReusedExchange (53) - : : +- CometBroadcastExchange (58) - : : +- CometFilter (57) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (56) - : +- CometBroadcastExchange (75) - : +- CometHashAggregate (74) - : +- CometExchange (73) - : +- CometHashAggregate (72) - : +- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometProject (68) - : : +- CometBroadcastHashJoin (67) - : : :- CometFilter (65) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (64) - : : +- ReusedExchange (66) - : +- ReusedExchange (69) - :- * HashAggregate (90) - : +- * CometColumnarToRow (89) - : +- CometColumnarExchange (88) - : +- * HashAggregate (87) - : +- * HashAggregate (86) - : +- * CometColumnarToRow (85) - : +- ReusedExchange (84) - +- * HashAggregate (97) - +- * CometColumnarToRow (96) - +- CometColumnarExchange (95) - +- * HashAggregate (94) - +- * HashAggregate (93) - +- * CometColumnarToRow (92) - +- ReusedExchange (91) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3], [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [1]: [s_store_sk#8] -Condition : isnotnull(s_store_sk#8) - -(11) CometBroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: [s_store_sk#8] - -(12) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -Right output [1]: [s_store_sk#8] -Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight - -(13) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] - -(14) CometHashAggregate -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Keys [1]: [s_store_sk#8] -Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] - -(15) CometExchange -Input [3]: [s_store_sk#8, sum#9, sum#10] -Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [3]: [s_store_sk#8, sum#9, sum#10] -Keys [1]: [s_store_sk#8] -Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(18) CometFilter -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Condition : isnotnull(sr_store_sk#11) - -(19) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#16] - -(20) CometBroadcastHashJoin -Left output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Right output [1]: [d_date_sk#16] -Arguments: [sr_returned_date_sk#14], [d_date_sk#16], Inner, BuildRight - -(21) CometProject -Input [5]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14, d_date_sk#16] -Arguments: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13], [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] - -(22) ReusedExchange [Reuses operator id: 11] -Output [1]: [s_store_sk#17] - -(23) CometBroadcastHashJoin -Left output [3]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] -Right output [1]: [s_store_sk#17] -Arguments: [sr_store_sk#11], [s_store_sk#17], Inner, BuildRight - -(24) CometProject -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Arguments: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17], [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] - -(25) CometHashAggregate -Input [3]: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Keys [1]: [s_store_sk#17] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#12)), partial_sum(UnscaledValue(sr_net_loss#13))] - -(26) CometExchange -Input [3]: [s_store_sk#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(27) CometHashAggregate -Input [3]: [s_store_sk#17, sum#18, sum#19] -Keys [1]: [s_store_sk#17] -Functions [2]: [sum(UnscaledValue(sr_return_amt#12)), sum(UnscaledValue(sr_net_loss#13))] - -(28) CometBroadcastExchange -Input [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#17, returns#20, profit_loss#21] - -(29) CometBroadcastHashJoin -Left output [3]: [s_store_sk#8, sales#22, profit#23] -Right output [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight - -(30) CometProject -Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] -Arguments: [channel#24, id#25, sales#22, returns#26, profit#27], [store channel AS channel#24, s_store_sk#8 AS id#25, sales#22, coalesce(returns#20, 0.00) AS returns#26, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#27] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] -ReadSchema: struct - -(33) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(34) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [cs_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(35) CometProject -Input [5]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31, d_date_sk#33] -Arguments: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30], [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] - -(36) CometHashAggregate -Input [3]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#29)), partial_sum(UnscaledValue(cs_net_profit#30))] - -(37) CometExchange -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Arguments: hashpartitioning(cs_call_center_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] - -(39) CometColumnarToRow [codegen id : 2] -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] - -(40) BroadcastExchange -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] -Arguments: IdentityBroadcastMode, [plan_id=4] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] -ReadSchema: struct - -(42) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#42] - -(43) CometBroadcastHashJoin -Left output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Right output [1]: [d_date_sk#42] -Arguments: [cr_returned_date_sk#40], [d_date_sk#42], Inner, BuildRight - -(44) CometProject -Input [4]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40, d_date_sk#42] -Arguments: [cr_return_amount#38, cr_net_loss#39], [cr_return_amount#38, cr_net_loss#39] - -(45) CometHashAggregate -Input [2]: [cr_return_amount#38, cr_net_loss#39] -Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] - -(46) CometExchange -Input [2]: [sum#43, sum#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometHashAggregate -Input [2]: [sum#43, sum#44] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] - -(48) CometColumnarToRow -Input [2]: [returns#45, profit_loss#46] - -(49) BroadcastNestedLoopJoin [codegen id : 3] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 3] -Output [5]: [catalog channel AS channel#47, cs_call_center_sk#28 AS id#48, sales#36, returns#45, (profit#37 - profit_loss#46) AS profit#49] -Input [5]: [cs_call_center_sk#28, sales#36, profit#37, returns#45, profit_loss#46] - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] -PushedFilters: [IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(52) CometFilter -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Condition : isnotnull(ws_web_page_sk#50) - -(53) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#55] - -(54) CometBroadcastHashJoin -Left output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Right output [1]: [d_date_sk#55] -Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight - -(55) CometProject -Input [5]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53, d_date_sk#55] -Arguments: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52], [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(57) CometFilter -Input [1]: [wp_web_page_sk#56] -Condition : isnotnull(wp_web_page_sk#56) - -(58) CometBroadcastExchange -Input [1]: [wp_web_page_sk#56] -Arguments: [wp_web_page_sk#56] - -(59) CometBroadcastHashJoin -Left output [3]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] -Right output [1]: [wp_web_page_sk#56] -Arguments: [ws_web_page_sk#50], [wp_web_page_sk#56], Inner, BuildRight - -(60) CometProject -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Arguments: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56], [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] - -(61) CometHashAggregate -Input [3]: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#51)), partial_sum(UnscaledValue(ws_net_profit#52))] - -(62) CometExchange -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Arguments: hashpartitioning(wp_web_page_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(63) CometHashAggregate -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#51)), sum(UnscaledValue(ws_net_profit#52))] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] -PushedFilters: [IsNotNull(wr_web_page_sk)] -ReadSchema: struct - -(65) CometFilter -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Condition : isnotnull(wr_web_page_sk#59) - -(66) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#64] - -(67) CometBroadcastHashJoin -Left output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Right output [1]: [d_date_sk#64] -Arguments: [wr_returned_date_sk#62], [d_date_sk#64], Inner, BuildRight - -(68) CometProject -Input [5]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62, d_date_sk#64] -Arguments: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61], [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] - -(69) ReusedExchange [Reuses operator id: 58] -Output [1]: [wp_web_page_sk#65] - -(70) CometBroadcastHashJoin -Left output [3]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] -Right output [1]: [wp_web_page_sk#65] -Arguments: [wr_web_page_sk#59], [wp_web_page_sk#65], Inner, BuildRight - -(71) CometProject -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Arguments: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65], [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] - -(72) CometHashAggregate -Input [3]: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#60)), partial_sum(UnscaledValue(wr_net_loss#61))] - -(73) CometExchange -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Arguments: hashpartitioning(wp_web_page_sk#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(74) CometHashAggregate -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [sum(UnscaledValue(wr_return_amt#60)), sum(UnscaledValue(wr_net_loss#61))] - -(75) CometBroadcastExchange -Input [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#65, returns#68, profit_loss#69] - -(76) CometBroadcastHashJoin -Left output [3]: [wp_web_page_sk#56, sales#70, profit#71] -Right output [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight - -(77) CometProject -Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [channel#72, id#73, sales#70, returns#74, profit#75], [web channel AS channel#72, wp_web_page_sk#56 AS id#73, sales#70, coalesce(returns#68, 0.00) AS returns#74, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#75] - -(78) CometColumnarToRow [codegen id : 4] -Input [5]: [channel#72, id#73, sales#70, returns#74, profit#75] - -(79) Union - -(80) HashAggregate [codegen id : 5] -Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] -Keys [2]: [channel#24, id#25] -Functions [3]: [partial_sum(sales#22), partial_sum(returns#26), partial_sum(profit#27)] -Aggregate Attributes [6]: [sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81] -Results [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(81) CometColumnarExchange -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Arguments: hashpartitioning(channel#24, id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(82) CometColumnarToRow [codegen id : 6] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(83) HashAggregate [codegen id : 6] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Keys [2]: [channel#24, id#25] -Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] -Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] -Results [5]: [channel#24, id#25, cast(sum(sales#22)#88 as decimal(37,2)) AS sales#91, cast(sum(returns#26)#89 as decimal(37,2)) AS returns#92, cast(sum(profit#27)#90 as decimal(38,2)) AS profit#93] - -(84) ReusedExchange [Reuses operator id: 81] -Output [8]: [channel#94, id#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101] - -(85) CometColumnarToRow [codegen id : 12] -Input [8]: [channel#94, id#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101] - -(86) HashAggregate [codegen id : 12] -Input [8]: [channel#94, id#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101] -Keys [2]: [channel#94, id#95] -Functions [3]: [sum(sales#102), sum(returns#103), sum(profit#104)] -Aggregate Attributes [3]: [sum(sales#102)#88, sum(returns#103)#89, sum(profit#104)#90] -Results [4]: [channel#94, sum(sales#102)#88 AS sales#105, sum(returns#103)#89 AS returns#106, sum(profit#104)#90 AS profit#107] - -(87) HashAggregate [codegen id : 12] -Input [4]: [channel#94, sales#105, returns#106, profit#107] -Keys [1]: [channel#94] -Functions [3]: [partial_sum(sales#105), partial_sum(returns#106), partial_sum(profit#107)] -Aggregate Attributes [6]: [sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] -Results [7]: [channel#94, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] - -(88) CometColumnarExchange -Input [7]: [channel#94, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Arguments: hashpartitioning(channel#94, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(89) CometColumnarToRow [codegen id : 13] -Input [7]: [channel#94, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] - -(90) HashAggregate [codegen id : 13] -Input [7]: [channel#94, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Keys [1]: [channel#94] -Functions [3]: [sum(sales#105), sum(returns#106), sum(profit#107)] -Aggregate Attributes [3]: [sum(sales#105)#120, sum(returns#106)#121, sum(profit#107)#122] -Results [5]: [channel#94, null AS id#123, sum(sales#105)#120 AS sales#124, sum(returns#106)#121 AS returns#125, sum(profit#107)#122 AS profit#126] - -(91) ReusedExchange [Reuses operator id: 81] -Output [8]: [channel#127, id#128, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] - -(92) CometColumnarToRow [codegen id : 19] -Input [8]: [channel#127, id#128, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] - -(93) HashAggregate [codegen id : 19] -Input [8]: [channel#127, id#128, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] -Keys [2]: [channel#127, id#128] -Functions [3]: [sum(sales#135), sum(returns#136), sum(profit#137)] -Aggregate Attributes [3]: [sum(sales#135)#88, sum(returns#136)#89, sum(profit#137)#90] -Results [3]: [sum(sales#135)#88 AS sales#138, sum(returns#136)#89 AS returns#139, sum(profit#137)#90 AS profit#140] - -(94) HashAggregate [codegen id : 19] -Input [3]: [sales#138, returns#139, profit#140] -Keys: [] -Functions [3]: [partial_sum(sales#138), partial_sum(returns#139), partial_sum(profit#140)] -Aggregate Attributes [6]: [sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Results [6]: [sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] - -(95) CometColumnarExchange -Input [6]: [sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(96) CometColumnarToRow [codegen id : 20] -Input [6]: [sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] - -(97) HashAggregate [codegen id : 20] -Input [6]: [sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] -Keys: [] -Functions [3]: [sum(sales#138), sum(returns#139), sum(profit#140)] -Aggregate Attributes [3]: [sum(sales#138)#153, sum(returns#139)#154, sum(profit#140)#155] -Results [5]: [null AS channel#156, null AS id#157, sum(sales#138)#153 AS sales#158, sum(returns#139)#154 AS returns#159, sum(profit#140)#155 AS profit#160] - -(98) Union - -(99) HashAggregate [codegen id : 21] -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#24, id#25, sales#91, returns#92, profit#93] - -(100) CometColumnarExchange -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Arguments: hashpartitioning(channel#24, id#25, sales#91, returns#92, profit#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(101) CometHashAggregate -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Functions: [] - -(102) CometTakeOrderedAndProject -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#24 ASC NULLS FIRST,id#25 ASC NULLS FIRST], output=[channel#24,id#25,sales#91,returns#92,profit#93]), [channel#24, id#25, sales#91, returns#92, profit#93], 100, 0, [channel#24 ASC NULLS FIRST, id#25 ASC NULLS FIRST], [channel#24, id#25, sales#91, returns#92, profit#93] - -(103) CometColumnarToRow [codegen id : 22] -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (108) -+- * CometColumnarToRow (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) - - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) - -(106) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(107) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(108) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#14 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 64 Hosting Expression = wr_returned_date_sk#62 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/simplified.txt deleted file mode 100644 index ac3d312ee8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,141 +0,0 @@ -WholeStageCodegen (22) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (21) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (5) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] - CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [s_store_sk,sales,profit,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] - CometExchange [s_store_sk] #3 - CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [s_store_sk,returns,profit_loss] #7 - CometHashAggregate [sum,sum] [s_store_sk,returns,profit_loss,sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss))] - CometExchange [s_store_sk] #8 - CometHashAggregate [sr_return_amt,sr_net_loss] [s_store_sk,sum,sum] - CometProject [sr_return_amt,sr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [s_store_sk] #6 - WholeStageCodegen (3) - Project [cs_call_center_sk,sales,returns,profit,profit_loss] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [cs_call_center_sk,sales,profit,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] - CometExchange [cs_call_center_sk] #10 - CometHashAggregate [cs_ext_sales_price,cs_net_profit] [cs_call_center_sk,sum,sum] - CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [returns,profit_loss,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] - CometExchange #11 - CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] - CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [wp_web_page_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] - CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [wp_web_page_sk,sales,profit,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] - CometExchange [wp_web_page_sk] #12 - CometHashAggregate [ws_ext_sales_price,ws_net_profit] [wp_web_page_sk,sum,sum] - CometProject [ws_ext_sales_price,ws_net_profit,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_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 [native_iceberg_compat] 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 [wp_web_page_sk] #13 - CometFilter [wp_web_page_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [wp_web_page_sk,returns,profit_loss] #14 - CometHashAggregate [sum,sum] [wp_web_page_sk,returns,profit_loss,sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss))] - CometExchange [wp_web_page_sk] #15 - CometHashAggregate [wr_return_amt,wr_net_loss] [wp_web_page_sk,sum,sum] - CometProject [wr_return_amt,wr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [wp_web_page_sk] #13 - WholeStageCodegen (13) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #16 - WholeStageCodegen (12) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (20) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #17 - WholeStageCodegen (19) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/explain.txt deleted file mode 100644 index 9554c6732e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/explain.txt +++ /dev/null @@ -1,621 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (103) -+- CometTakeOrderedAndProject (102) - +- CometHashAggregate (101) - +- CometColumnarExchange (100) - +- * HashAggregate (99) - +- Union (98) - :- * HashAggregate (83) - : +- * CometColumnarToRow (82) - : +- CometColumnarExchange (81) - : +- * HashAggregate (80) - : +- Union (79) - : :- * CometColumnarToRow (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : : +- CometBroadcastExchange (28) - : : +- CometHashAggregate (27) - : : +- CometExchange (26) - : : +- CometHashAggregate (25) - : : +- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (21) - : : : +- CometBroadcastHashJoin (20) - : : : :- CometFilter (18) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (17) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : :- * Project (50) - : : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) - : : :- BroadcastExchange (40) - : : : +- * CometColumnarToRow (39) - : : : +- CometHashAggregate (38) - : : : +- CometExchange (37) - : : : +- CometHashAggregate (36) - : : : +- CometProject (35) - : : : +- CometBroadcastHashJoin (34) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (32) - : : : +- ReusedExchange (33) - : : +- * CometColumnarToRow (48) - : : +- CometHashAggregate (47) - : : +- CometExchange (46) - : : +- CometHashAggregate (45) - : : +- CometProject (44) - : : +- CometBroadcastHashJoin (43) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (41) - : : +- ReusedExchange (42) - : +- * CometColumnarToRow (78) - : +- CometProject (77) - : +- CometBroadcastHashJoin (76) - : :- CometHashAggregate (63) - : : +- CometExchange (62) - : : +- CometHashAggregate (61) - : : +- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (55) - : : : +- CometBroadcastHashJoin (54) - : : : :- CometFilter (52) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (51) - : : : +- ReusedExchange (53) - : : +- CometBroadcastExchange (58) - : : +- CometFilter (57) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (56) - : +- CometBroadcastExchange (75) - : +- CometHashAggregate (74) - : +- CometExchange (73) - : +- CometHashAggregate (72) - : +- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometProject (68) - : : +- CometBroadcastHashJoin (67) - : : :- CometFilter (65) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (64) - : : +- ReusedExchange (66) - : +- ReusedExchange (69) - :- * HashAggregate (90) - : +- * CometColumnarToRow (89) - : +- CometColumnarExchange (88) - : +- * HashAggregate (87) - : +- * HashAggregate (86) - : +- * CometColumnarToRow (85) - : +- ReusedExchange (84) - +- * HashAggregate (97) - +- * CometColumnarToRow (96) - +- CometColumnarExchange (95) - +- * HashAggregate (94) - +- * HashAggregate (93) - +- * CometColumnarToRow (92) - +- ReusedExchange (91) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3], [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [1]: [s_store_sk#8] -Condition : isnotnull(s_store_sk#8) - -(11) CometBroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: [s_store_sk#8] - -(12) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -Right output [1]: [s_store_sk#8] -Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight - -(13) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] - -(14) CometHashAggregate -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Keys [1]: [s_store_sk#8] -Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] - -(15) CometExchange -Input [3]: [s_store_sk#8, sum#9, sum#10] -Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [3]: [s_store_sk#8, sum#9, sum#10] -Keys [1]: [s_store_sk#8] -Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(18) CometFilter -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Condition : isnotnull(sr_store_sk#11) - -(19) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#16] - -(20) CometBroadcastHashJoin -Left output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Right output [1]: [d_date_sk#16] -Arguments: [sr_returned_date_sk#14], [d_date_sk#16], Inner, BuildRight - -(21) CometProject -Input [5]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14, d_date_sk#16] -Arguments: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13], [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] - -(22) ReusedExchange [Reuses operator id: 11] -Output [1]: [s_store_sk#17] - -(23) CometBroadcastHashJoin -Left output [3]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] -Right output [1]: [s_store_sk#17] -Arguments: [sr_store_sk#11], [s_store_sk#17], Inner, BuildRight - -(24) CometProject -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Arguments: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17], [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] - -(25) CometHashAggregate -Input [3]: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Keys [1]: [s_store_sk#17] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#12)), partial_sum(UnscaledValue(sr_net_loss#13))] - -(26) CometExchange -Input [3]: [s_store_sk#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(27) CometHashAggregate -Input [3]: [s_store_sk#17, sum#18, sum#19] -Keys [1]: [s_store_sk#17] -Functions [2]: [sum(UnscaledValue(sr_return_amt#12)), sum(UnscaledValue(sr_net_loss#13))] - -(28) CometBroadcastExchange -Input [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#17, returns#20, profit_loss#21] - -(29) CometBroadcastHashJoin -Left output [3]: [s_store_sk#8, sales#22, profit#23] -Right output [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight - -(30) CometProject -Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] -Arguments: [channel#24, id#25, sales#22, returns#26, profit#27], [store channel AS channel#24, s_store_sk#8 AS id#25, sales#22, coalesce(returns#20, 0.00) AS returns#26, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#27] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] -ReadSchema: struct - -(33) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(34) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [cs_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(35) CometProject -Input [5]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31, d_date_sk#33] -Arguments: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30], [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] - -(36) CometHashAggregate -Input [3]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#29)), partial_sum(UnscaledValue(cs_net_profit#30))] - -(37) CometExchange -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Arguments: hashpartitioning(cs_call_center_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] - -(39) CometColumnarToRow [codegen id : 2] -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] - -(40) BroadcastExchange -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] -Arguments: IdentityBroadcastMode, [plan_id=4] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] -ReadSchema: struct - -(42) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#42] - -(43) CometBroadcastHashJoin -Left output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Right output [1]: [d_date_sk#42] -Arguments: [cr_returned_date_sk#40], [d_date_sk#42], Inner, BuildRight - -(44) CometProject -Input [4]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40, d_date_sk#42] -Arguments: [cr_return_amount#38, cr_net_loss#39], [cr_return_amount#38, cr_net_loss#39] - -(45) CometHashAggregate -Input [2]: [cr_return_amount#38, cr_net_loss#39] -Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] - -(46) CometExchange -Input [2]: [sum#43, sum#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometHashAggregate -Input [2]: [sum#43, sum#44] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] - -(48) CometColumnarToRow -Input [2]: [returns#45, profit_loss#46] - -(49) BroadcastNestedLoopJoin [codegen id : 3] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 3] -Output [5]: [catalog channel AS channel#47, cs_call_center_sk#28 AS id#48, sales#36, returns#45, (profit#37 - profit_loss#46) AS profit#49] -Input [5]: [cs_call_center_sk#28, sales#36, profit#37, returns#45, profit_loss#46] - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] -PushedFilters: [IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(52) CometFilter -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Condition : isnotnull(ws_web_page_sk#50) - -(53) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#55] - -(54) CometBroadcastHashJoin -Left output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Right output [1]: [d_date_sk#55] -Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight - -(55) CometProject -Input [5]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53, d_date_sk#55] -Arguments: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52], [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(57) CometFilter -Input [1]: [wp_web_page_sk#56] -Condition : isnotnull(wp_web_page_sk#56) - -(58) CometBroadcastExchange -Input [1]: [wp_web_page_sk#56] -Arguments: [wp_web_page_sk#56] - -(59) CometBroadcastHashJoin -Left output [3]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] -Right output [1]: [wp_web_page_sk#56] -Arguments: [ws_web_page_sk#50], [wp_web_page_sk#56], Inner, BuildRight - -(60) CometProject -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Arguments: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56], [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] - -(61) CometHashAggregate -Input [3]: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#51)), partial_sum(UnscaledValue(ws_net_profit#52))] - -(62) CometExchange -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Arguments: hashpartitioning(wp_web_page_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(63) CometHashAggregate -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#51)), sum(UnscaledValue(ws_net_profit#52))] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] -PushedFilters: [IsNotNull(wr_web_page_sk)] -ReadSchema: struct - -(65) CometFilter -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Condition : isnotnull(wr_web_page_sk#59) - -(66) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#64] - -(67) CometBroadcastHashJoin -Left output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Right output [1]: [d_date_sk#64] -Arguments: [wr_returned_date_sk#62], [d_date_sk#64], Inner, BuildRight - -(68) CometProject -Input [5]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62, d_date_sk#64] -Arguments: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61], [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] - -(69) ReusedExchange [Reuses operator id: 58] -Output [1]: [wp_web_page_sk#65] - -(70) CometBroadcastHashJoin -Left output [3]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] -Right output [1]: [wp_web_page_sk#65] -Arguments: [wr_web_page_sk#59], [wp_web_page_sk#65], Inner, BuildRight - -(71) CometProject -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Arguments: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65], [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] - -(72) CometHashAggregate -Input [3]: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#60)), partial_sum(UnscaledValue(wr_net_loss#61))] - -(73) CometExchange -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Arguments: hashpartitioning(wp_web_page_sk#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(74) CometHashAggregate -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [sum(UnscaledValue(wr_return_amt#60)), sum(UnscaledValue(wr_net_loss#61))] - -(75) CometBroadcastExchange -Input [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#65, returns#68, profit_loss#69] - -(76) CometBroadcastHashJoin -Left output [3]: [wp_web_page_sk#56, sales#70, profit#71] -Right output [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight - -(77) CometProject -Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [channel#72, id#73, sales#70, returns#74, profit#75], [web channel AS channel#72, wp_web_page_sk#56 AS id#73, sales#70, coalesce(returns#68, 0.00) AS returns#74, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#75] - -(78) CometColumnarToRow [codegen id : 4] -Input [5]: [channel#72, id#73, sales#70, returns#74, profit#75] - -(79) Union - -(80) HashAggregate [codegen id : 5] -Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] -Keys [2]: [channel#24, id#25] -Functions [3]: [partial_sum(sales#22), partial_sum(returns#26), partial_sum(profit#27)] -Aggregate Attributes [6]: [sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81] -Results [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(81) CometColumnarExchange -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Arguments: hashpartitioning(channel#24, id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(82) CometColumnarToRow [codegen id : 6] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(83) HashAggregate [codegen id : 6] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Keys [2]: [channel#24, id#25] -Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] -Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] -Results [5]: [channel#24, id#25, cast(sum(sales#22)#88 as decimal(37,2)) AS sales#91, cast(sum(returns#26)#89 as decimal(37,2)) AS returns#92, cast(sum(profit#27)#90 as decimal(38,2)) AS profit#93] - -(84) ReusedExchange [Reuses operator id: 81] -Output [8]: [channel#94, id#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101] - -(85) CometColumnarToRow [codegen id : 12] -Input [8]: [channel#94, id#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101] - -(86) HashAggregate [codegen id : 12] -Input [8]: [channel#94, id#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101] -Keys [2]: [channel#94, id#95] -Functions [3]: [sum(sales#102), sum(returns#103), sum(profit#104)] -Aggregate Attributes [3]: [sum(sales#102)#88, sum(returns#103)#89, sum(profit#104)#90] -Results [4]: [channel#94, sum(sales#102)#88 AS sales#105, sum(returns#103)#89 AS returns#106, sum(profit#104)#90 AS profit#107] - -(87) HashAggregate [codegen id : 12] -Input [4]: [channel#94, sales#105, returns#106, profit#107] -Keys [1]: [channel#94] -Functions [3]: [partial_sum(sales#105), partial_sum(returns#106), partial_sum(profit#107)] -Aggregate Attributes [6]: [sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] -Results [7]: [channel#94, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] - -(88) CometColumnarExchange -Input [7]: [channel#94, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Arguments: hashpartitioning(channel#94, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(89) CometColumnarToRow [codegen id : 13] -Input [7]: [channel#94, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] - -(90) HashAggregate [codegen id : 13] -Input [7]: [channel#94, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Keys [1]: [channel#94] -Functions [3]: [sum(sales#105), sum(returns#106), sum(profit#107)] -Aggregate Attributes [3]: [sum(sales#105)#120, sum(returns#106)#121, sum(profit#107)#122] -Results [5]: [channel#94, null AS id#123, sum(sales#105)#120 AS sales#124, sum(returns#106)#121 AS returns#125, sum(profit#107)#122 AS profit#126] - -(91) ReusedExchange [Reuses operator id: 81] -Output [8]: [channel#127, id#128, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] - -(92) CometColumnarToRow [codegen id : 19] -Input [8]: [channel#127, id#128, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] - -(93) HashAggregate [codegen id : 19] -Input [8]: [channel#127, id#128, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] -Keys [2]: [channel#127, id#128] -Functions [3]: [sum(sales#135), sum(returns#136), sum(profit#137)] -Aggregate Attributes [3]: [sum(sales#135)#88, sum(returns#136)#89, sum(profit#137)#90] -Results [3]: [sum(sales#135)#88 AS sales#138, sum(returns#136)#89 AS returns#139, sum(profit#137)#90 AS profit#140] - -(94) HashAggregate [codegen id : 19] -Input [3]: [sales#138, returns#139, profit#140] -Keys: [] -Functions [3]: [partial_sum(sales#138), partial_sum(returns#139), partial_sum(profit#140)] -Aggregate Attributes [6]: [sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Results [6]: [sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] - -(95) CometColumnarExchange -Input [6]: [sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(96) CometColumnarToRow [codegen id : 20] -Input [6]: [sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] - -(97) HashAggregate [codegen id : 20] -Input [6]: [sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] -Keys: [] -Functions [3]: [sum(sales#138), sum(returns#139), sum(profit#140)] -Aggregate Attributes [3]: [sum(sales#138)#153, sum(returns#139)#154, sum(profit#140)#155] -Results [5]: [null AS channel#156, null AS id#157, sum(sales#138)#153 AS sales#158, sum(returns#139)#154 AS returns#159, sum(profit#140)#155 AS profit#160] - -(98) Union - -(99) HashAggregate [codegen id : 21] -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#24, id#25, sales#91, returns#92, profit#93] - -(100) CometColumnarExchange -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Arguments: hashpartitioning(channel#24, id#25, sales#91, returns#92, profit#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(101) CometHashAggregate -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Functions: [] - -(102) CometTakeOrderedAndProject -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#24 ASC NULLS FIRST,id#25 ASC NULLS FIRST], output=[channel#24,id#25,sales#91,returns#92,profit#93]), [channel#24, id#25, sales#91, returns#92, profit#93], 100, 0, [channel#24 ASC NULLS FIRST, id#25 ASC NULLS FIRST], [channel#24, id#25, sales#91, returns#92, profit#93] - -(103) CometColumnarToRow [codegen id : 22] -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (108) -+- * CometColumnarToRow (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) - - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) - -(106) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(107) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(108) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#14 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 64 Hosting Expression = wr_returned_date_sk#62 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/extended.txt deleted file mode 100644 index b4318d03e2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/extended.txt +++ /dev/null @@ -1,355 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : : :- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : : :- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : :- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - -Comet accelerated 287 out of 332 eligible operators (86%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index ac3d312ee8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt +++ /dev/null @@ -1,141 +0,0 @@ -WholeStageCodegen (22) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (21) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (5) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] - CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [s_store_sk,sales,profit,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] - CometExchange [s_store_sk] #3 - CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [s_store_sk,returns,profit_loss] #7 - CometHashAggregate [sum,sum] [s_store_sk,returns,profit_loss,sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss))] - CometExchange [s_store_sk] #8 - CometHashAggregate [sr_return_amt,sr_net_loss] [s_store_sk,sum,sum] - CometProject [sr_return_amt,sr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [s_store_sk] #6 - WholeStageCodegen (3) - Project [cs_call_center_sk,sales,returns,profit,profit_loss] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [cs_call_center_sk,sales,profit,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] - CometExchange [cs_call_center_sk] #10 - CometHashAggregate [cs_ext_sales_price,cs_net_profit] [cs_call_center_sk,sum,sum] - CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [returns,profit_loss,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] - CometExchange #11 - CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] - CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [wp_web_page_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] - CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [wp_web_page_sk,sales,profit,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] - CometExchange [wp_web_page_sk] #12 - CometHashAggregate [ws_ext_sales_price,ws_net_profit] [wp_web_page_sk,sum,sum] - CometProject [ws_ext_sales_price,ws_net_profit,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_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 [native_iceberg_compat] 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 [wp_web_page_sk] #13 - CometFilter [wp_web_page_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [wp_web_page_sk,returns,profit_loss] #14 - CometHashAggregate [sum,sum] [wp_web_page_sk,returns,profit_loss,sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss))] - CometExchange [wp_web_page_sk] #15 - CometHashAggregate [wr_return_amt,wr_net_loss] [wp_web_page_sk,sum,sum] - CometProject [wr_return_amt,wr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [wp_web_page_sk] #13 - WholeStageCodegen (13) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #16 - WholeStageCodegen (12) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (20) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #17 - WholeStageCodegen (19) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/explain.txt deleted file mode 100644 index dbd053cfac..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/explain.txt +++ /dev/null @@ -1,417 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (70) -+- * Project (69) - +- * CometColumnarToRow (68) - +- CometSortMergeJoin (67) - :- CometProject (45) - : +- CometSortMergeJoin (44) - : :- CometSort (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometFilter (12) - : : : +- CometSortMergeJoin (11) - : : : :- CometSort (5) - : : : : +- CometColumnarExchange (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (10) - : : : +- CometExchange (9) - : : : +- CometProject (8) - : : : +- CometFilter (7) - : : : +- CometNativeScan parquet spark_catalog.default.store_returns (6) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (14) - : +- CometSort (43) - : +- CometFilter (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometFilter (34) - : : +- CometSortMergeJoin (33) - : : :- CometSort (27) - : : : +- CometColumnarExchange (26) - : : : +- * Filter (25) - : : : +- * ColumnarToRow (24) - : : : +- Scan parquet spark_catalog.default.web_sales (23) - : : +- CometSort (32) - : : +- CometExchange (31) - : : +- CometProject (30) - : : +- CometFilter (29) - : : +- CometNativeScan parquet spark_catalog.default.web_returns (28) - : +- ReusedExchange (36) - +- CometSort (66) - +- CometFilter (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometFilter (57) - : +- CometSortMergeJoin (56) - : :- CometSort (50) - : : +- CometColumnarExchange (49) - : : +- * Filter (48) - : : +- * ColumnarToRow (47) - : : +- Scan parquet spark_catalog.default.catalog_sales (46) - : +- CometSort (55) - : +- CometExchange (54) - : +- CometProject (53) - : +- CometFilter (52) - : +- CometNativeScan parquet spark_catalog.default.catalog_returns (51) - +- ReusedExchange (59) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(3) Filter [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_customer_sk#2)) - -(4) CometColumnarExchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_ticket_number#3, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_ticket_number#3 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) - -(8) CometProject -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] - -(9) CometExchange -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter - -(12) CometFilter -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Condition : isnull(sr_ticket_number#10) - -(13) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(14) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(17) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] - -(19) CometHashAggregate -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] - -(20) CometExchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(21) CometHashAggregate -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] - -(22) CometSort -Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] - -(23) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 2] -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(25) Filter [codegen id : 2] -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) - -(26) CometColumnarExchange -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometSort -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] - -(28) CometNativeScan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#28, wr_order_number#29, wr_returned_date_sk#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(29) CometFilter -Input [3]: [wr_item_sk#28, wr_order_number#29, wr_returned_date_sk#30] -Condition : (isnotnull(wr_order_number#29) AND isnotnull(wr_item_sk#28)) - -(30) CometProject -Input [3]: [wr_item_sk#28, wr_order_number#29, wr_returned_date_sk#30] -Arguments: [wr_item_sk#28, wr_order_number#29], [wr_item_sk#28, wr_order_number#29] - -(31) CometExchange -Input [2]: [wr_item_sk#28, wr_order_number#29] -Arguments: hashpartitioning(wr_order_number#29, wr_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(32) CometSort -Input [2]: [wr_item_sk#28, wr_order_number#29] -Arguments: [wr_item_sk#28, wr_order_number#29], [wr_order_number#29 ASC NULLS FIRST, wr_item_sk#28 ASC NULLS FIRST] - -(33) CometSortMergeJoin -Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [wr_item_sk#28, wr_order_number#29] -Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#29, wr_item_sk#28], LeftOuter - -(34) CometFilter -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#28, wr_order_number#29] -Condition : isnull(wr_order_number#29) - -(35) CometProject -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#28, wr_order_number#29] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(36) ReusedExchange [Reuses operator id: 16] -Output [2]: [d_date_sk#31, d_year#32] - -(37) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [d_date_sk#31, d_year#32] -Arguments: [ws_sold_date_sk#27], [d_date_sk#31], Inner, BuildRight - -(38) CometProject -Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#31, d_year#32] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#32], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#32] - -(39) CometHashAggregate -Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#32] -Keys [3]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] - -(40) CometExchange -Input [6]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22, sum#33, sum#34, sum#35] -Arguments: hashpartitioning(d_year#32, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(41) CometHashAggregate -Input [6]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22, sum#33, sum#34, sum#35] -Keys [3]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] - -(42) CometFilter -Input [6]: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Condition : (coalesce(ws_qty#38, 0) > 0) - -(43) CometSort -Input [6]: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Arguments: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40], [ws_sold_year#36 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#37 ASC NULLS FIRST] - -(44) CometSortMergeJoin -Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Right output [6]: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37], Inner - -(45) CometProject -Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40] - -(46) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#47), dynamicpruningexpression(cs_sold_date_sk#47 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(47) ColumnarToRow [codegen id : 3] -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] - -(48) Filter [codegen id : 3] -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Condition : (isnotnull(cs_item_sk#42) AND isnotnull(cs_bill_customer_sk#41)) - -(49) CometColumnarExchange -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Arguments: hashpartitioning(cs_order_number#43, cs_item_sk#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(50) CometSort -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Arguments: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47], [cs_order_number#43 ASC NULLS FIRST, cs_item_sk#42 ASC NULLS FIRST] - -(51) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#48, cr_order_number#49, cr_returned_date_sk#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(52) CometFilter -Input [3]: [cr_item_sk#48, cr_order_number#49, cr_returned_date_sk#50] -Condition : (isnotnull(cr_order_number#49) AND isnotnull(cr_item_sk#48)) - -(53) CometProject -Input [3]: [cr_item_sk#48, cr_order_number#49, cr_returned_date_sk#50] -Arguments: [cr_item_sk#48, cr_order_number#49], [cr_item_sk#48, cr_order_number#49] - -(54) CometExchange -Input [2]: [cr_item_sk#48, cr_order_number#49] -Arguments: hashpartitioning(cr_order_number#49, cr_item_sk#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(55) CometSort -Input [2]: [cr_item_sk#48, cr_order_number#49] -Arguments: [cr_item_sk#48, cr_order_number#49], [cr_order_number#49 ASC NULLS FIRST, cr_item_sk#48 ASC NULLS FIRST] - -(56) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Right output [2]: [cr_item_sk#48, cr_order_number#49] -Arguments: [cs_order_number#43, cs_item_sk#42], [cr_order_number#49, cr_item_sk#48], LeftOuter - -(57) CometFilter -Input [9]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47, cr_item_sk#48, cr_order_number#49] -Condition : isnull(cr_order_number#49) - -(58) CometProject -Input [9]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47, cr_item_sk#48, cr_order_number#49] -Arguments: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47], [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] - -(59) ReusedExchange [Reuses operator id: 16] -Output [2]: [d_date_sk#51, d_year#52] - -(60) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Right output [2]: [d_date_sk#51, d_year#52] -Arguments: [cs_sold_date_sk#47], [d_date_sk#51], Inner, BuildRight - -(61) CometProject -Input [8]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47, d_date_sk#51, d_year#52] -Arguments: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, d_year#52], [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, d_year#52] - -(62) CometHashAggregate -Input [6]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, d_year#52] -Keys [3]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41] -Functions [3]: [partial_sum(cs_quantity#44), partial_sum(UnscaledValue(cs_wholesale_cost#45)), partial_sum(UnscaledValue(cs_sales_price#46))] - -(63) CometExchange -Input [6]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41, sum#53, sum#54, sum#55] -Arguments: hashpartitioning(d_year#52, cs_item_sk#42, cs_bill_customer_sk#41, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(64) CometHashAggregate -Input [6]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41, sum#53, sum#54, sum#55] -Keys [3]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41] -Functions [3]: [sum(cs_quantity#44), sum(UnscaledValue(cs_wholesale_cost#45)), sum(UnscaledValue(cs_sales_price#46))] - -(65) CometFilter -Input [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] -Condition : (coalesce(cs_qty#58, 0) > 0) - -(66) CometSort -Input [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] -Arguments: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60], [cs_sold_year#56 ASC NULLS FIRST, cs_item_sk#42 ASC NULLS FIRST, cs_customer_sk#57 ASC NULLS FIRST] - -(67) CometSortMergeJoin -Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40] -Right output [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57], Inner - -(68) CometColumnarToRow [codegen id : 4] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40, cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] - -(69) Project [codegen id : 4] -Output [13]: [round((cast(ss_qty#18 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#38 + cs_qty#58), 1) as double)))), 2) AS ratio#61, ss_qty#18 AS store_qty#62, ss_wc#19 AS store_wholesale_cost#63, ss_sp#20 AS store_sales_price#64, (coalesce(ws_qty#38, 0) + coalesce(cs_qty#58, 0)) AS other_chan_qty#65, (coalesce(ws_wc#39, 0.00) + coalesce(cs_wc#59, 0.00)) AS other_chan_wholesale_cost#66, (coalesce(ws_sp#40, 0.00) + coalesce(cs_sp#60, 0.00)) AS other_chan_sales_price#67, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40, cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] - -(70) TakeOrderedAndProject -Input [13]: [ratio#61, store_qty#62, store_wholesale_cost#63, store_sales_price#64, other_chan_qty#65, other_chan_wholesale_cost#66, other_chan_sales_price#67, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#65 ASC NULLS FIRST, other_chan_wholesale_cost#66 ASC NULLS FIRST, other_chan_sales_price#67 ASC NULLS FIRST, ratio#61 ASC NULLS FIRST], [ratio#61, store_qty#62, store_wholesale_cost#63, store_sales_price#64, other_chan_qty#65, other_chan_wholesale_cost#66, other_chan_sales_price#67] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (74) -+- * CometColumnarToRow (73) - +- CometFilter (72) - +- CometNativeScan parquet spark_catalog.default.date_dim (71) - - -(71) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(72) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(73) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(74) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#47 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/simplified.txt deleted file mode 100644 index 4711669981..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/simplified.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (4) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] - CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #2 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #5 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] - CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometColumnarExchange [ws_order_number,ws_item_sk] #7 - WholeStageCodegen (2) - Filter [ws_item_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_order_number,wr_item_sk] #8 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] - CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometColumnarExchange [cs_order_number,cs_item_sk] #10 - WholeStageCodegen (3) - Filter [cs_item_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_order_number,cr_item_sk] #11 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/explain.txt deleted file mode 100644 index 82a5357f89..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,405 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (67) -+- * Project (66) - +- * CometColumnarToRow (65) - +- CometSortMergeJoin (64) - :- CometProject (43) - : +- CometSortMergeJoin (42) - : :- CometSort (21) - : : +- CometHashAggregate (20) - : : +- CometExchange (19) - : : +- CometHashAggregate (18) - : : +- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : +- CometBroadcastExchange (15) - : : +- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : +- CometSort (41) - : +- CometFilter (40) - : +- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (33) - : : +- CometFilter (32) - : : +- CometSortMergeJoin (31) - : : :- CometSort (25) - : : : +- CometExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (22) - : : +- CometSort (30) - : : +- CometExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (26) - : +- ReusedExchange (34) - +- CometSort (63) - +- CometFilter (62) - +- CometHashAggregate (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometProject (55) - : +- CometFilter (54) - : +- CometSortMergeJoin (53) - : :- CometSort (47) - : : +- CometExchange (46) - : : +- CometFilter (45) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (44) - : +- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometFilter (49) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (48) - +- ReusedExchange (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_customer_sk#2)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_ticket_number#3, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_ticket_number#3 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) - -(7) CometProject -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] - -(8) CometExchange -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter - -(11) CometFilter -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Condition : isnull(sr_ticket_number#10) - -(12) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(15) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(16) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight - -(17) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] - -(18) CometHashAggregate -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] - -(19) CometExchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometHashAggregate -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] - -(21) CometSort -Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(23) CometFilter -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) - -(24) CometExchange -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometSort -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Condition : (isnotnull(wr_order_number#30) AND isnotnull(wr_item_sk#29)) - -(28) CometProject -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_item_sk#29, wr_order_number#30] - -(29) CometExchange -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: hashpartitioning(wr_order_number#30, wr_item_sk#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(30) CometSort -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_order_number#30 ASC NULLS FIRST, wr_item_sk#29 ASC NULLS FIRST] - -(31) CometSortMergeJoin -Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#30, wr_item_sk#29], LeftOuter - -(32) CometFilter -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Condition : isnull(wr_order_number#30) - -(33) CometProject -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(34) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#32, d_year#33] - -(35) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ws_sold_date_sk#27], [d_date_sk#32], Inner, BuildRight - -(36) CometProject -Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#32, d_year#33] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] - -(37) CometHashAggregate -Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] - -(38) CometExchange -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Arguments: hashpartitioning(d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(39) CometHashAggregate -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] - -(40) CometFilter -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Condition : (coalesce(ws_qty#39, 0) > 0) - -(41) CometSort -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41], [ws_sold_year#37 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#38 ASC NULLS FIRST] - -(42) CometSortMergeJoin -Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Right output [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38], Inner - -(43) CometProject -Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#48), dynamicpruningexpression(cs_sold_date_sk#48 IN dynamicpruning#49)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(45) CometFilter -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Condition : (isnotnull(cs_item_sk#43) AND isnotnull(cs_bill_customer_sk#42)) - -(46) CometExchange -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(47) CometSort -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST] - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Condition : (isnotnull(cr_order_number#51) AND isnotnull(cr_item_sk#50)) - -(50) CometProject -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_item_sk#50, cr_order_number#51] - -(51) CometExchange -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(52) CometSort -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST] - -(53) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cs_order_number#44, cs_item_sk#43], [cr_order_number#51, cr_item_sk#50], LeftOuter - -(54) CometFilter -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Condition : isnull(cr_order_number#51) - -(55) CometProject -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] - -(56) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#53, d_year#54] - -(57) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [cs_sold_date_sk#48], [d_date_sk#53], Inner, BuildRight - -(58) CometProject -Input [8]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, d_date_sk#53, d_year#54] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] - -(59) CometHashAggregate -Input [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [partial_sum(cs_quantity#45), partial_sum(UnscaledValue(cs_wholesale_cost#46)), partial_sum(UnscaledValue(cs_sales_price#47))] - -(60) CometExchange -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Arguments: hashpartitioning(d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(61) CometHashAggregate -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [sum(cs_quantity#45), sum(UnscaledValue(cs_wholesale_cost#46)), sum(UnscaledValue(cs_sales_price#47))] - -(62) CometFilter -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Condition : (coalesce(cs_qty#60, 0) > 0) - -(63) CometSort -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62], [cs_sold_year#58 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST, cs_customer_sk#59 ASC NULLS FIRST] - -(64) CometSortMergeJoin -Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] -Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner - -(65) CometColumnarToRow [codegen id : 1] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(66) Project [codegen id : 1] -Output [13]: [round((cast(ss_qty#18 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)))), 2) AS ratio#63, ss_qty#18 AS store_qty#64, ss_wc#19 AS store_wholesale_cost#65, ss_sp#20 AS store_sales_price#66, (coalesce(ws_qty#39, 0) + coalesce(cs_qty#60, 0)) AS other_chan_qty#67, (coalesce(ws_wc#40, 0.00) + coalesce(cs_wc#61, 0.00)) AS other_chan_wholesale_cost#68, (coalesce(ws_sp#41, 0.00) + coalesce(cs_sp#62, 0.00)) AS other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(67) TakeOrderedAndProject -Input [13]: [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#67 ASC NULLS FIRST, other_chan_wholesale_cost#68 ASC NULLS FIRST, other_chan_sales_price#69 ASC NULLS FIRST, ratio#63 ASC NULLS FIRST], [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) - - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(70) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(71) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/simplified.txt deleted file mode 100644 index 273db28e77..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,78 +0,0 @@ -TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (1) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] - CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #5 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] - CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometExchange [ws_order_number,ws_item_sk] #7 - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_order_number,wr_item_sk] #8 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] - CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometExchange [cs_order_number,cs_item_sk] #10 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_order_number,cr_item_sk] #11 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/explain.txt deleted file mode 100644 index 82a5357f89..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/explain.txt +++ /dev/null @@ -1,405 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (67) -+- * Project (66) - +- * CometColumnarToRow (65) - +- CometSortMergeJoin (64) - :- CometProject (43) - : +- CometSortMergeJoin (42) - : :- CometSort (21) - : : +- CometHashAggregate (20) - : : +- CometExchange (19) - : : +- CometHashAggregate (18) - : : +- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : +- CometBroadcastExchange (15) - : : +- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : +- CometSort (41) - : +- CometFilter (40) - : +- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (33) - : : +- CometFilter (32) - : : +- CometSortMergeJoin (31) - : : :- CometSort (25) - : : : +- CometExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (22) - : : +- CometSort (30) - : : +- CometExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (26) - : +- ReusedExchange (34) - +- CometSort (63) - +- CometFilter (62) - +- CometHashAggregate (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometProject (55) - : +- CometFilter (54) - : +- CometSortMergeJoin (53) - : :- CometSort (47) - : : +- CometExchange (46) - : : +- CometFilter (45) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (44) - : +- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometFilter (49) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (48) - +- ReusedExchange (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_customer_sk#2)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_ticket_number#3, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_ticket_number#3 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) - -(7) CometProject -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] - -(8) CometExchange -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter - -(11) CometFilter -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Condition : isnull(sr_ticket_number#10) - -(12) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(15) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(16) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight - -(17) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] - -(18) CometHashAggregate -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] - -(19) CometExchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometHashAggregate -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] - -(21) CometSort -Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(23) CometFilter -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) - -(24) CometExchange -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometSort -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Condition : (isnotnull(wr_order_number#30) AND isnotnull(wr_item_sk#29)) - -(28) CometProject -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_item_sk#29, wr_order_number#30] - -(29) CometExchange -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: hashpartitioning(wr_order_number#30, wr_item_sk#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(30) CometSort -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_order_number#30 ASC NULLS FIRST, wr_item_sk#29 ASC NULLS FIRST] - -(31) CometSortMergeJoin -Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#30, wr_item_sk#29], LeftOuter - -(32) CometFilter -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Condition : isnull(wr_order_number#30) - -(33) CometProject -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(34) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#32, d_year#33] - -(35) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ws_sold_date_sk#27], [d_date_sk#32], Inner, BuildRight - -(36) CometProject -Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#32, d_year#33] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] - -(37) CometHashAggregate -Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] - -(38) CometExchange -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Arguments: hashpartitioning(d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(39) CometHashAggregate -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] - -(40) CometFilter -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Condition : (coalesce(ws_qty#39, 0) > 0) - -(41) CometSort -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41], [ws_sold_year#37 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#38 ASC NULLS FIRST] - -(42) CometSortMergeJoin -Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Right output [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38], Inner - -(43) CometProject -Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#48), dynamicpruningexpression(cs_sold_date_sk#48 IN dynamicpruning#49)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(45) CometFilter -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Condition : (isnotnull(cs_item_sk#43) AND isnotnull(cs_bill_customer_sk#42)) - -(46) CometExchange -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(47) CometSort -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST] - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Condition : (isnotnull(cr_order_number#51) AND isnotnull(cr_item_sk#50)) - -(50) CometProject -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_item_sk#50, cr_order_number#51] - -(51) CometExchange -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(52) CometSort -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST] - -(53) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cs_order_number#44, cs_item_sk#43], [cr_order_number#51, cr_item_sk#50], LeftOuter - -(54) CometFilter -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Condition : isnull(cr_order_number#51) - -(55) CometProject -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] - -(56) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#53, d_year#54] - -(57) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [cs_sold_date_sk#48], [d_date_sk#53], Inner, BuildRight - -(58) CometProject -Input [8]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, d_date_sk#53, d_year#54] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] - -(59) CometHashAggregate -Input [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [partial_sum(cs_quantity#45), partial_sum(UnscaledValue(cs_wholesale_cost#46)), partial_sum(UnscaledValue(cs_sales_price#47))] - -(60) CometExchange -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Arguments: hashpartitioning(d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(61) CometHashAggregate -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [sum(cs_quantity#45), sum(UnscaledValue(cs_wholesale_cost#46)), sum(UnscaledValue(cs_sales_price#47))] - -(62) CometFilter -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Condition : (coalesce(cs_qty#60, 0) > 0) - -(63) CometSort -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62], [cs_sold_year#58 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST, cs_customer_sk#59 ASC NULLS FIRST] - -(64) CometSortMergeJoin -Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] -Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner - -(65) CometColumnarToRow [codegen id : 1] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(66) Project [codegen id : 1] -Output [13]: [round((cast(ss_qty#18 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)))), 2) AS ratio#63, ss_qty#18 AS store_qty#64, ss_wc#19 AS store_wholesale_cost#65, ss_sp#20 AS store_sales_price#66, (coalesce(ws_qty#39, 0) + coalesce(cs_qty#60, 0)) AS other_chan_qty#67, (coalesce(ws_wc#40, 0.00) + coalesce(cs_wc#61, 0.00)) AS other_chan_wholesale_cost#68, (coalesce(ws_sp#41, 0.00) + coalesce(cs_sp#62, 0.00)) AS other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(67) TakeOrderedAndProject -Input [13]: [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#67 ASC NULLS FIRST, other_chan_wholesale_cost#68 ASC NULLS FIRST, other_chan_sales_price#69 ASC NULLS FIRST, ratio#63 ASC NULLS FIRST], [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) - - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(70) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(71) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/extended.txt deleted file mode 100644 index 9d5ba53be1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/extended.txt +++ /dev/null @@ -1,80 +0,0 @@ -TakeOrderedAndProject -+- Project [COMET: Comet does not support Spark's BigDecimal rounding] - +- CometColumnarToRow - +- CometSortMergeJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 273db28e77..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt +++ /dev/null @@ -1,78 +0,0 @@ -TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (1) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] - CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #5 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] - CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometExchange [ws_order_number,ws_item_sk] #7 - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_order_number,wr_item_sk] #8 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] - CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometExchange [cs_order_number,cs_item_sk] #10 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_order_number,cr_item_sk] #11 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/explain.txt deleted file mode 100644 index 0807f06c94..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/explain.txt +++ /dev/null @@ -1,695 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (119) -+- CometTakeOrderedAndProject (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometUnion (114) - :- CometHashAggregate (103) - : +- CometExchange (102) - : +- CometHashAggregate (101) - : +- CometUnion (100) - : :- CometHashAggregate (39) - : : +- CometExchange (38) - : : +- CometHashAggregate (37) - : : +- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (30) - : : : +- CometBroadcastHashJoin (29) - : : : :- CometProject (24) - : : : : +- CometBroadcastHashJoin (23) - : : : : :- CometProject (18) - : : : : : +- CometBroadcastHashJoin (17) - : : : : : :- CometProject (12) - : : : : : : +- CometSortMergeJoin (11) - : : : : : : :- CometSort (5) - : : : : : : : +- CometColumnarExchange (4) - : : : : : : : +- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (10) - : : : : : : +- CometExchange (9) - : : : : : : +- CometProject (8) - : : : : : : +- CometFilter (7) - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (6) - : : : : : +- CometBroadcastExchange (16) - : : : : : +- CometProject (15) - : : : : : +- CometFilter (14) - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (13) - : : : : +- CometBroadcastExchange (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometNativeScan parquet spark_catalog.default.store (19) - : : : +- CometBroadcastExchange (28) - : : : +- CometProject (27) - : : : +- CometFilter (26) - : : : +- CometNativeScan parquet spark_catalog.default.item (25) - : : +- CometBroadcastExchange (34) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometNativeScan parquet spark_catalog.default.promotion (31) - : :- CometHashAggregate (69) - : : +- CometExchange (68) - : : +- CometHashAggregate (67) - : : +- CometProject (66) - : : +- CometBroadcastHashJoin (65) - : : :- CometProject (63) - : : : +- CometBroadcastHashJoin (62) - : : : :- CometProject (60) - : : : : +- CometBroadcastHashJoin (59) - : : : : :- CometProject (54) - : : : : : +- CometBroadcastHashJoin (53) - : : : : : :- CometProject (51) - : : : : : : +- CometSortMergeJoin (50) - : : : : : : :- CometSort (44) - : : : : : : : +- CometColumnarExchange (43) - : : : : : : : +- * Filter (42) - : : : : : : : +- * ColumnarToRow (41) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (40) - : : : : : : +- CometSort (49) - : : : : : : +- CometExchange (48) - : : : : : : +- CometProject (47) - : : : : : : +- CometFilter (46) - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (45) - : : : : : +- ReusedExchange (52) - : : : : +- CometBroadcastExchange (58) - : : : : +- CometProject (57) - : : : : +- CometFilter (56) - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_page (55) - : : : +- ReusedExchange (61) - : : +- ReusedExchange (64) - : +- CometHashAggregate (99) - : +- CometExchange (98) - : +- CometHashAggregate (97) - : +- CometProject (96) - : +- CometBroadcastHashJoin (95) - : :- CometProject (93) - : : +- CometBroadcastHashJoin (92) - : : :- CometProject (90) - : : : +- CometBroadcastHashJoin (89) - : : : :- CometProject (84) - : : : : +- CometBroadcastHashJoin (83) - : : : : :- CometProject (81) - : : : : : +- CometSortMergeJoin (80) - : : : : : :- CometSort (74) - : : : : : : +- CometColumnarExchange (73) - : : : : : : +- * Filter (72) - : : : : : : +- * ColumnarToRow (71) - : : : : : : +- Scan parquet spark_catalog.default.web_sales (70) - : : : : : +- CometSort (79) - : : : : : +- CometExchange (78) - : : : : : +- CometProject (77) - : : : : : +- CometFilter (76) - : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns (75) - : : : : +- ReusedExchange (82) - : : : +- CometBroadcastExchange (88) - : : : +- CometProject (87) - : : : +- CometFilter (86) - : : : +- CometNativeScan parquet spark_catalog.default.web_site (85) - : : +- ReusedExchange (91) - : +- ReusedExchange (94) - :- CometHashAggregate (108) - : +- CometExchange (107) - : +- CometHashAggregate (106) - : +- CometHashAggregate (105) - : +- ReusedExchange (104) - +- CometHashAggregate (113) - +- CometExchange (112) - +- CometHashAggregate (111) - +- CometHashAggregate (110) - +- ReusedExchange (109) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] - -(3) Filter [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(4) CometColumnarExchange -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(7) CometFilter -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) - -(8) CometProject -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(9) CometExchange -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter - -(12) CometProject -Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] - -(13) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) - -(15) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] - -(19) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_store_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [s_store_sk#16, s_store_id#17] -Condition : isnotnull(s_store_sk#16) - -(21) CometProject -Input [2]: [s_store_sk#16, s_store_id#17] -Arguments: [s_store_sk#16, s_store_id#18], [s_store_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#17, 16)) AS s_store_id#18] - -(22) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_store_id#18] -Arguments: [s_store_sk#16, s_store_id#18] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -Right output [2]: [s_store_sk#16, s_store_id#18] -Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight - -(24) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#18] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(25) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_current_price#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [i_item_sk#19, i_current_price#20] -Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) - -(27) CometProject -Input [2]: [i_item_sk#19, i_current_price#20] -Arguments: [i_item_sk#19], [i_item_sk#19] - -(28) CometBroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: [i_item_sk#19] - -(29) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [i_item_sk#19] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, i_item_sk#19] -Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(31) CometNativeScan parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#21, p_channel_tv#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_channel_tv), IsNotNull(p_promo_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Condition : ((isnotnull(p_channel_tv#22) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_tv#22, 1)) = N)) AND isnotnull(p_promo_sk#21)) - -(33) CometProject -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Arguments: [p_promo_sk#21], [p_promo_sk#21] - -(34) CometBroadcastExchange -Input [1]: [p_promo_sk#21] -Arguments: [p_promo_sk#21] - -(35) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [p_promo_sk#21] -Arguments: [ss_promo_sk#3], [p_promo_sk#21], Inner, BuildRight - -(36) CometProject -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, p_promo_sk#21] -Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(37) CometHashAggregate -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Keys [1]: [s_store_id#18] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(38) CometExchange -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(39) CometHashAggregate -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [s_store_id#18] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(40) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(41) ColumnarToRow [codegen id : 2] -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] - -(42) Filter [codegen id : 2] -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) - -(43) CometColumnarExchange -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(44) CometSort -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] - -(45) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38, cr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(46) CometFilter -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38, cr_returned_date_sk#39] -Condition : (isnotnull(cr_item_sk#35) AND isnotnull(cr_order_number#36)) - -(47) CometProject -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38, cr_returned_date_sk#39] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38], [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] - -(48) CometExchange -Input [4]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: hashpartitioning(cr_item_sk#35, cr_order_number#36, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(49) CometSort -Input [4]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38], [cr_item_sk#35 ASC NULLS FIRST, cr_order_number#36 ASC NULLS FIRST] - -(50) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#35, cr_order_number#36], LeftOuter - -(51) CometProject -Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38] - -(52) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#40] - -(53) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38] -Right output [1]: [d_date_sk#40] -Arguments: [cs_sold_date_sk#34], [d_date_sk#40], Inner, BuildRight - -(54) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38, d_date_sk#40] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38] - -(55) CometNativeScan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#42] -Condition : isnotnull(cp_catalog_page_sk#41) - -(57) CometProject -Input [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#42] -Arguments: [cp_catalog_page_sk#41, cp_catalog_page_id#43], [cp_catalog_page_sk#41, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#42, 16)) AS cp_catalog_page_id#43] - -(58) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#43] -Arguments: [cp_catalog_page_sk#41, cp_catalog_page_id#43] - -(59) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38] -Right output [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#43] -Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#41], Inner, BuildRight - -(60) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_sk#41, cp_catalog_page_id#43] -Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] - -(61) ReusedExchange [Reuses operator id: 28] -Output [1]: [i_item_sk#44] - -(62) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] -Right output [1]: [i_item_sk#44] -Arguments: [cs_item_sk#29], [i_item_sk#44], Inner, BuildRight - -(63) CometProject -Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43, i_item_sk#44] -Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] - -(64) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#45] - -(65) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] -Right output [1]: [p_promo_sk#45] -Arguments: [cs_promo_sk#30], [p_promo_sk#45], Inner, BuildRight - -(66) CometProject -Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43, p_promo_sk#45] -Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] - -(67) CometHashAggregate -Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] -Keys [1]: [cp_catalog_page_id#43] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#37 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#38 as decimal(12,2)), 0.00)))] - -(68) CometExchange -Input [6]: [cp_catalog_page_id#43, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] -Arguments: hashpartitioning(cp_catalog_page_id#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(69) CometHashAggregate -Input [6]: [cp_catalog_page_id#43, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] -Keys [1]: [cp_catalog_page_id#43] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#37 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#38 as decimal(12,2)), 0.00)))] - -(70) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#57), dynamicpruningexpression(ws_sold_date_sk#57 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] -ReadSchema: struct - -(71) ColumnarToRow [codegen id : 3] -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] - -(72) Filter [codegen id : 3] -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Condition : ((isnotnull(ws_web_site_sk#52) AND isnotnull(ws_item_sk#51)) AND isnotnull(ws_promo_sk#53)) - -(73) CometColumnarExchange -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Arguments: hashpartitioning(ws_item_sk#51, ws_order_number#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(74) CometSort -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Arguments: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57], [ws_item_sk#51 ASC NULLS FIRST, ws_order_number#54 ASC NULLS FIRST] - -(75) CometNativeScan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] -ReadSchema: struct - -(76) CometFilter -Input [5]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Condition : (isnotnull(wr_item_sk#58) AND isnotnull(wr_order_number#59)) - -(77) CometProject -Input [5]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Arguments: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61], [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] - -(78) CometExchange -Input [4]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: hashpartitioning(wr_item_sk#58, wr_order_number#59, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(79) CometSort -Input [4]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61], [wr_item_sk#58 ASC NULLS FIRST, wr_order_number#59 ASC NULLS FIRST] - -(80) CometSortMergeJoin -Left output [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Right output [4]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: [ws_item_sk#51, ws_order_number#54], [wr_item_sk#58, wr_order_number#59], LeftOuter - -(81) CometProject -Input [11]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61], [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61] - -(82) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#63] - -(83) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61] -Right output [1]: [d_date_sk#63] -Arguments: [ws_sold_date_sk#57], [d_date_sk#63], Inner, BuildRight - -(84) CometProject -Input [9]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61, d_date_sk#63] -Arguments: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61], [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61] - -(85) CometNativeScan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#64, web_site_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(86) CometFilter -Input [2]: [web_site_sk#64, web_site_id#65] -Condition : isnotnull(web_site_sk#64) - -(87) CometProject -Input [2]: [web_site_sk#64, web_site_id#65] -Arguments: [web_site_sk#64, web_site_id#66], [web_site_sk#64, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#65, 16)) AS web_site_id#66] - -(88) CometBroadcastExchange -Input [2]: [web_site_sk#64, web_site_id#66] -Arguments: [web_site_sk#64, web_site_id#66] - -(89) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61] -Right output [2]: [web_site_sk#64, web_site_id#66] -Arguments: [ws_web_site_sk#52], [web_site_sk#64], Inner, BuildRight - -(90) CometProject -Input [9]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_sk#64, web_site_id#66] -Arguments: [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66], [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] - -(91) ReusedExchange [Reuses operator id: 28] -Output [1]: [i_item_sk#67] - -(92) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] -Right output [1]: [i_item_sk#67] -Arguments: [ws_item_sk#51], [i_item_sk#67], Inner, BuildRight - -(93) CometProject -Input [8]: [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66, i_item_sk#67] -Arguments: [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66], [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] - -(94) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#68] - -(95) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] -Right output [1]: [p_promo_sk#68] -Arguments: [ws_promo_sk#53], [p_promo_sk#68], Inner, BuildRight - -(96) CometProject -Input [7]: [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66, p_promo_sk#68] -Arguments: [ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66], [ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] - -(97) CometHashAggregate -Input [5]: [ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] -Keys [1]: [web_site_id#66] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#55)), partial_sum(coalesce(cast(wr_return_amt#60 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#56 - coalesce(cast(wr_net_loss#61 as decimal(12,2)), 0.00)))] - -(98) CometExchange -Input [6]: [web_site_id#66, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Arguments: hashpartitioning(web_site_id#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(99) CometHashAggregate -Input [6]: [web_site_id#66, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Keys [1]: [web_site_id#66] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#55)), sum(coalesce(cast(wr_return_amt#60 as decimal(12,2)), 0.00)), sum((ws_net_profit#56 - coalesce(cast(wr_net_loss#61 as decimal(12,2)), 0.00)))] - -(100) CometUnion -Child 0 Input [5]: [channel#74, id#75, sales#76, returns#77, profit#78] -Child 1 Input [5]: [channel#79, id#80, sales#81, returns#82, profit#83] -Child 2 Input [5]: [channel#84, id#85, sales#86, returns#87, profit#88] - -(101) CometHashAggregate -Input [5]: [channel#74, id#75, sales#76, returns#77, profit#78] -Keys [2]: [channel#74, id#75] -Functions [3]: [partial_sum(sales#76), partial_sum(returns#77), partial_sum(profit#78)] - -(102) CometExchange -Input [8]: [channel#74, id#75, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94] -Arguments: hashpartitioning(channel#74, id#75, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(103) CometHashAggregate -Input [8]: [channel#74, id#75, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94] -Keys [2]: [channel#74, id#75] -Functions [3]: [sum(sales#76), sum(returns#77), sum(profit#78)] - -(104) ReusedExchange [Reuses operator id: 102] -Output [8]: [channel#95, id#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] - -(105) CometHashAggregate -Input [8]: [channel#95, id#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Keys [2]: [channel#95, id#96] -Functions [3]: [sum(sales#103), sum(returns#104), sum(profit#105)] - -(106) CometHashAggregate -Input [4]: [channel#95, sales#106, returns#107, profit#108] -Keys [1]: [channel#95] -Functions [3]: [partial_sum(sales#106), partial_sum(returns#107), partial_sum(profit#108)] - -(107) CometExchange -Input [7]: [channel#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: hashpartitioning(channel#95, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(108) CometHashAggregate -Input [7]: [channel#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [1]: [channel#95] -Functions [3]: [sum(sales#106), sum(returns#107), sum(profit#108)] - -(109) ReusedExchange [Reuses operator id: 102] -Output [8]: [channel#115, id#116, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] - -(110) CometHashAggregate -Input [8]: [channel#115, id#116, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Keys [2]: [channel#115, id#116] -Functions [3]: [sum(sales#123), sum(returns#124), sum(profit#125)] - -(111) CometHashAggregate -Input [3]: [sales#126, returns#127, profit#128] -Keys: [] -Functions [3]: [partial_sum(sales#126), partial_sum(returns#127), partial_sum(profit#128)] - -(112) CometExchange -Input [6]: [sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(113) CometHashAggregate -Input [6]: [sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] -Keys: [] -Functions [3]: [sum(sales#126), sum(returns#127), sum(profit#128)] - -(114) CometUnion -Child 0 Input [5]: [channel#74, id#75, sales#135, returns#136, profit#137] -Child 1 Input [5]: [channel#95, id#138, sales#139, returns#140, profit#141] -Child 2 Input [5]: [channel#142, id#143, sales#144, returns#145, profit#146] - -(115) CometHashAggregate -Input [5]: [channel#74, id#75, sales#135, returns#136, profit#137] -Keys [5]: [channel#74, id#75, sales#135, returns#136, profit#137] -Functions: [] - -(116) CometExchange -Input [5]: [channel#74, id#75, sales#135, returns#136, profit#137] -Arguments: hashpartitioning(channel#74, id#75, sales#135, returns#136, profit#137, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] - -(117) CometHashAggregate -Input [5]: [channel#74, id#75, sales#135, returns#136, profit#137] -Keys [5]: [channel#74, id#75, sales#135, returns#136, profit#137] -Functions: [] - -(118) CometTakeOrderedAndProject -Input [5]: [channel#74, id#75, sales#135, returns#136, profit#137] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#74 ASC NULLS FIRST,id#75 ASC NULLS FIRST], output=[channel#74,id#75,sales#135,returns#136,profit#137]), [channel#74, id#75, sales#135, returns#136, profit#137], 100, 0, [channel#74 ASC NULLS FIRST, id#75 ASC NULLS FIRST], [channel#74, id#75, sales#135, returns#136, profit#137] - -(119) CometColumnarToRow [codegen id : 10] -Input [5]: [channel#74, id#75, sales#135, returns#136, profit#137] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (124) -+- * CometColumnarToRow (123) - +- CometProject (122) - +- CometFilter (121) - +- CometNativeScan parquet spark_catalog.default.date_dim (120) - - -(120) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(121) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) - -(122) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(123) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(124) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#57 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/simplified.txt deleted file mode 100644 index cfce9e20a1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/simplified.txt +++ /dev/null @@ -1,137 +0,0 @@ -WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id] #2 - CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [s_store_id] #3 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk,ss_ticket_number] #4 - WholeStageCodegen (1) - Filter [ss_store_sk,ss_item_sk,ss_promo_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #6 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #8 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #9 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #10 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,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)))] - CometExchange [cp_catalog_page_id] #11 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometColumnarExchange [cs_item_sk,cs_order_number] #12 - WholeStageCodegen (2) - Filter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [web_site_id] #15 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometColumnarExchange [ws_item_sk,ws_order_number] #16 - WholeStageCodegen (3) - Filter [ws_web_site_sk,ws_item_sk,ws_promo_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #17 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [web_site_sk,web_site_id] #18 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel] #19 - CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange #20 - CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/explain.txt deleted file mode 100644 index 224275b64b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,683 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (116) -+- CometTakeOrderedAndProject (115) - +- CometHashAggregate (114) - +- CometExchange (113) - +- CometHashAggregate (112) - +- CometUnion (111) - :- CometHashAggregate (100) - : +- CometExchange (99) - : +- CometHashAggregate (98) - : +- CometUnion (97) - : :- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometProject (29) - : : : +- CometBroadcastHashJoin (28) - : : : :- CometProject (23) - : : : : +- CometBroadcastHashJoin (22) - : : : : :- CometProject (17) - : : : : : +- CometBroadcastHashJoin (16) - : : : : : :- CometProject (11) - : : : : : : +- CometSortMergeJoin (10) - : : : : : : :- CometSort (4) - : : : : : : : +- CometExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (9) - : : : : : : +- CometExchange (8) - : : : : : : +- CometProject (7) - : : : : : : +- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : : +- CometBroadcastExchange (15) - : : : : : +- CometProject (14) - : : : : : +- CometFilter (13) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) - : : : : +- CometBroadcastExchange (21) - : : : : +- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) - : : : +- CometBroadcastExchange (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - : :- CometHashAggregate (67) - : : +- CometExchange (66) - : : +- CometHashAggregate (65) - : : +- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometProject (61) - : : : +- CometBroadcastHashJoin (60) - : : : :- CometProject (58) - : : : : +- CometBroadcastHashJoin (57) - : : : : :- CometProject (52) - : : : : : +- CometBroadcastHashJoin (51) - : : : : : :- CometProject (49) - : : : : : : +- CometSortMergeJoin (48) - : : : : : : :- CometSort (42) - : : : : : : : +- CometExchange (41) - : : : : : : : +- CometFilter (40) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) - : : : : : : +- CometSort (47) - : : : : : : +- CometExchange (46) - : : : : : : +- CometProject (45) - : : : : : : +- CometFilter (44) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) - : : : : : +- ReusedExchange (50) - : : : : +- CometBroadcastExchange (56) - : : : : +- CometProject (55) - : : : : +- CometFilter (54) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) - : : : +- ReusedExchange (59) - : : +- ReusedExchange (62) - : +- CometHashAggregate (96) - : +- CometExchange (95) - : +- CometHashAggregate (94) - : +- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (87) - : : : +- CometBroadcastHashJoin (86) - : : : :- CometProject (81) - : : : : +- CometBroadcastHashJoin (80) - : : : : :- CometProject (78) - : : : : : +- CometSortMergeJoin (77) - : : : : : :- CometSort (71) - : : : : : : +- CometExchange (70) - : : : : : : +- CometFilter (69) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) - : : : : : +- CometSort (76) - : : : : : +- CometExchange (75) - : : : : : +- CometProject (74) - : : : : : +- CometFilter (73) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) - : : : : +- ReusedExchange (79) - : : : +- CometBroadcastExchange (85) - : : : +- CometProject (84) - : : : +- CometFilter (83) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) - : : +- ReusedExchange (88) - : +- ReusedExchange (91) - :- CometHashAggregate (105) - : +- CometExchange (104) - : +- CometHashAggregate (103) - : +- CometHashAggregate (102) - : +- ReusedExchange (101) - +- CometHashAggregate (110) - +- CometExchange (109) - +- CometHashAggregate (108) - +- CometHashAggregate (107) - +- ReusedExchange (106) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) - -(7) CometProject -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(8) CometExchange -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter - -(11) CometProject -Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) - -(14) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(15) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(16) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight - -(17) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_store_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [s_store_sk#16, s_store_id#17] -Condition : isnotnull(s_store_sk#16) - -(20) CometProject -Input [2]: [s_store_sk#16, s_store_id#17] -Arguments: [s_store_sk#16, s_store_id#18], [s_store_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#17, 16)) AS s_store_id#18] - -(21) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_store_id#18] -Arguments: [s_store_sk#16, s_store_id#18] - -(22) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -Right output [2]: [s_store_sk#16, s_store_id#18] -Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight - -(23) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#18] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_current_price#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [i_item_sk#19, i_current_price#20] -Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) - -(26) CometProject -Input [2]: [i_item_sk#19, i_current_price#20] -Arguments: [i_item_sk#19], [i_item_sk#19] - -(27) CometBroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: [i_item_sk#19] - -(28) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [i_item_sk#19] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(29) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, i_item_sk#19] -Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#21, p_channel_tv#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_channel_tv), IsNotNull(p_promo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Condition : ((isnotnull(p_channel_tv#22) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_tv#22, 1)) = N)) AND isnotnull(p_promo_sk#21)) - -(32) CometProject -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Arguments: [p_promo_sk#21], [p_promo_sk#21] - -(33) CometBroadcastExchange -Input [1]: [p_promo_sk#21] -Arguments: [p_promo_sk#21] - -(34) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [p_promo_sk#21] -Arguments: [ss_promo_sk#3], [p_promo_sk#21], Inner, BuildRight - -(35) CometProject -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, p_promo_sk#21] -Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(36) CometHashAggregate -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Keys [1]: [s_store_id#18] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(37) CometExchange -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [s_store_id#18] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(40) CometFilter -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) - -(41) CometExchange -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(42) CometSort -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(44) CometFilter -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) - -(45) CometProject -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] - -(46) CometExchange -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometSort -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] - -(48) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter - -(49) CometProject -Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] - -(50) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#41] - -(51) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -Right output [1]: [d_date_sk#41] -Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight - -(52) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(54) CometFilter -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Condition : isnotnull(cp_catalog_page_sk#42) - -(55) CometProject -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#43, 16)) AS cp_catalog_page_id#44] - -(56) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] - -(57) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight - -(58) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(59) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#45] - -(60) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [i_item_sk#45] -Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight - -(61) CometProject -Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] -Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(62) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#46] - -(63) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [p_promo_sk#46] -Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight - -(64) CometProject -Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] -Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(65) CometHashAggregate -Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(66) CometExchange -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(67) CometHashAggregate -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] -ReadSchema: struct - -(69) CometFilter -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) - -(70) CometExchange -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(71) CometSort -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] -ReadSchema: struct - -(73) CometFilter -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) - -(74) CometProject -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] - -(75) CometExchange -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(76) CometSort -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] - -(77) CometSortMergeJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter - -(78) CometProject -Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] - -(79) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#65] - -(80) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -Right output [1]: [d_date_sk#65] -Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(81) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] - -(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#66, web_site_id#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(83) CometFilter -Input [2]: [web_site_sk#66, web_site_id#67] -Condition : isnotnull(web_site_sk#66) - -(84) CometProject -Input [2]: [web_site_sk#66, web_site_id#67] -Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#67, 16)) AS web_site_id#68] - -(85) CometBroadcastExchange -Input [2]: [web_site_sk#66, web_site_id#68] -Arguments: [web_site_sk#66, web_site_id#68] - -(86) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -Right output [2]: [web_site_sk#66, web_site_id#68] -Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight - -(87) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] -Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(88) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#69] - -(89) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [i_item_sk#69] -Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight - -(90) CometProject -Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] -Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(91) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#70] - -(92) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [p_promo_sk#70] -Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight - -(93) CometProject -Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] -Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(94) CometHashAggregate -Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Keys [1]: [web_site_id#68] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(95) CometExchange -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(96) CometHashAggregate -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Keys [1]: [web_site_id#68] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(97) CometUnion -Child 0 Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] -Child 1 Input [5]: [channel#81, id#82, sales#83, returns#84, profit#85] -Child 2 Input [5]: [channel#86, id#87, sales#88, returns#89, profit#90] - -(98) CometHashAggregate -Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] -Keys [2]: [channel#76, id#77] -Functions [3]: [partial_sum(sales#78), partial_sum(returns#79), partial_sum(profit#80)] - -(99) CometExchange -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Arguments: hashpartitioning(channel#76, id#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(100) CometHashAggregate -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Keys [2]: [channel#76, id#77] -Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] - -(101) ReusedExchange [Reuses operator id: 99] -Output [8]: [channel#97, id#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] - -(102) CometHashAggregate -Input [8]: [channel#97, id#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Keys [2]: [channel#97, id#98] -Functions [3]: [sum(sales#105), sum(returns#106), sum(profit#107)] - -(103) CometHashAggregate -Input [4]: [channel#97, sales#108, returns#109, profit#110] -Keys [1]: [channel#97] -Functions [3]: [partial_sum(sales#108), partial_sum(returns#109), partial_sum(profit#110)] - -(104) CometExchange -Input [7]: [channel#97, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] -Arguments: hashpartitioning(channel#97, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(105) CometHashAggregate -Input [7]: [channel#97, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] -Keys [1]: [channel#97] -Functions [3]: [sum(sales#108), sum(returns#109), sum(profit#110)] - -(106) ReusedExchange [Reuses operator id: 99] -Output [8]: [channel#117, id#118, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] - -(107) CometHashAggregate -Input [8]: [channel#117, id#118, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] -Keys [2]: [channel#117, id#118] -Functions [3]: [sum(sales#125), sum(returns#126), sum(profit#127)] - -(108) CometHashAggregate -Input [3]: [sales#128, returns#129, profit#130] -Keys: [] -Functions [3]: [partial_sum(sales#128), partial_sum(returns#129), partial_sum(profit#130)] - -(109) CometExchange -Input [6]: [sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(110) CometHashAggregate -Input [6]: [sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] -Keys: [] -Functions [3]: [sum(sales#128), sum(returns#129), sum(profit#130)] - -(111) CometUnion -Child 0 Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] -Child 1 Input [5]: [channel#97, id#140, sales#141, returns#142, profit#143] -Child 2 Input [5]: [channel#144, id#145, sales#146, returns#147, profit#148] - -(112) CometHashAggregate -Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] -Keys [5]: [channel#76, id#77, sales#137, returns#138, profit#139] -Functions: [] - -(113) CometExchange -Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] -Arguments: hashpartitioning(channel#76, id#77, sales#137, returns#138, profit#139, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] - -(114) CometHashAggregate -Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] -Keys [5]: [channel#76, id#77, sales#137, returns#138, profit#139] -Functions: [] - -(115) CometTakeOrderedAndProject -Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#76 ASC NULLS FIRST,id#77 ASC NULLS FIRST], output=[channel#76,id#77,sales#137,returns#138,profit#139]), [channel#76, id#77, sales#137, returns#138, profit#139], 100, 0, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#137, returns#138, profit#139] - -(116) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) - -(119) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(121) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/simplified.txt deleted file mode 100644 index d27ab827e1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,128 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id] #2 - CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [s_store_id] #3 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #4 - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #6 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #8 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #9 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #10 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,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)))] - CometExchange [cp_catalog_page_id] #11 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #12 - 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 [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [web_site_id] #15 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #16 - 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 [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #17 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [web_site_sk,web_site_id] #18 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel] #19 - CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange #20 - CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/explain.txt deleted file mode 100644 index 224275b64b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/explain.txt +++ /dev/null @@ -1,683 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (116) -+- CometTakeOrderedAndProject (115) - +- CometHashAggregate (114) - +- CometExchange (113) - +- CometHashAggregate (112) - +- CometUnion (111) - :- CometHashAggregate (100) - : +- CometExchange (99) - : +- CometHashAggregate (98) - : +- CometUnion (97) - : :- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometProject (29) - : : : +- CometBroadcastHashJoin (28) - : : : :- CometProject (23) - : : : : +- CometBroadcastHashJoin (22) - : : : : :- CometProject (17) - : : : : : +- CometBroadcastHashJoin (16) - : : : : : :- CometProject (11) - : : : : : : +- CometSortMergeJoin (10) - : : : : : : :- CometSort (4) - : : : : : : : +- CometExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (9) - : : : : : : +- CometExchange (8) - : : : : : : +- CometProject (7) - : : : : : : +- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : : +- CometBroadcastExchange (15) - : : : : : +- CometProject (14) - : : : : : +- CometFilter (13) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) - : : : : +- CometBroadcastExchange (21) - : : : : +- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) - : : : +- CometBroadcastExchange (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - : :- CometHashAggregate (67) - : : +- CometExchange (66) - : : +- CometHashAggregate (65) - : : +- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometProject (61) - : : : +- CometBroadcastHashJoin (60) - : : : :- CometProject (58) - : : : : +- CometBroadcastHashJoin (57) - : : : : :- CometProject (52) - : : : : : +- CometBroadcastHashJoin (51) - : : : : : :- CometProject (49) - : : : : : : +- CometSortMergeJoin (48) - : : : : : : :- CometSort (42) - : : : : : : : +- CometExchange (41) - : : : : : : : +- CometFilter (40) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) - : : : : : : +- CometSort (47) - : : : : : : +- CometExchange (46) - : : : : : : +- CometProject (45) - : : : : : : +- CometFilter (44) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) - : : : : : +- ReusedExchange (50) - : : : : +- CometBroadcastExchange (56) - : : : : +- CometProject (55) - : : : : +- CometFilter (54) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) - : : : +- ReusedExchange (59) - : : +- ReusedExchange (62) - : +- CometHashAggregate (96) - : +- CometExchange (95) - : +- CometHashAggregate (94) - : +- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (87) - : : : +- CometBroadcastHashJoin (86) - : : : :- CometProject (81) - : : : : +- CometBroadcastHashJoin (80) - : : : : :- CometProject (78) - : : : : : +- CometSortMergeJoin (77) - : : : : : :- CometSort (71) - : : : : : : +- CometExchange (70) - : : : : : : +- CometFilter (69) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) - : : : : : +- CometSort (76) - : : : : : +- CometExchange (75) - : : : : : +- CometProject (74) - : : : : : +- CometFilter (73) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) - : : : : +- ReusedExchange (79) - : : : +- CometBroadcastExchange (85) - : : : +- CometProject (84) - : : : +- CometFilter (83) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) - : : +- ReusedExchange (88) - : +- ReusedExchange (91) - :- CometHashAggregate (105) - : +- CometExchange (104) - : +- CometHashAggregate (103) - : +- CometHashAggregate (102) - : +- ReusedExchange (101) - +- CometHashAggregate (110) - +- CometExchange (109) - +- CometHashAggregate (108) - +- CometHashAggregate (107) - +- ReusedExchange (106) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) - -(7) CometProject -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(8) CometExchange -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter - -(11) CometProject -Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) - -(14) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(15) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(16) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight - -(17) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_store_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [s_store_sk#16, s_store_id#17] -Condition : isnotnull(s_store_sk#16) - -(20) CometProject -Input [2]: [s_store_sk#16, s_store_id#17] -Arguments: [s_store_sk#16, s_store_id#18], [s_store_sk#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#17, 16)) AS s_store_id#18] - -(21) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_store_id#18] -Arguments: [s_store_sk#16, s_store_id#18] - -(22) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -Right output [2]: [s_store_sk#16, s_store_id#18] -Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight - -(23) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#18] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_current_price#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [i_item_sk#19, i_current_price#20] -Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) - -(26) CometProject -Input [2]: [i_item_sk#19, i_current_price#20] -Arguments: [i_item_sk#19], [i_item_sk#19] - -(27) CometBroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: [i_item_sk#19] - -(28) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [i_item_sk#19] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(29) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, i_item_sk#19] -Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#21, p_channel_tv#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_channel_tv), IsNotNull(p_promo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Condition : ((isnotnull(p_channel_tv#22) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(p_channel_tv#22, 1)) = N)) AND isnotnull(p_promo_sk#21)) - -(32) CometProject -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Arguments: [p_promo_sk#21], [p_promo_sk#21] - -(33) CometBroadcastExchange -Input [1]: [p_promo_sk#21] -Arguments: [p_promo_sk#21] - -(34) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [p_promo_sk#21] -Arguments: [ss_promo_sk#3], [p_promo_sk#21], Inner, BuildRight - -(35) CometProject -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, p_promo_sk#21] -Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(36) CometHashAggregate -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Keys [1]: [s_store_id#18] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(37) CometExchange -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [s_store_id#18] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(40) CometFilter -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) - -(41) CometExchange -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(42) CometSort -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(44) CometFilter -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) - -(45) CometProject -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] - -(46) CometExchange -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometSort -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] - -(48) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter - -(49) CometProject -Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] - -(50) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#41] - -(51) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -Right output [1]: [d_date_sk#41] -Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight - -(52) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(54) CometFilter -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Condition : isnotnull(cp_catalog_page_sk#42) - -(55) CometProject -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#43, 16)) AS cp_catalog_page_id#44] - -(56) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] - -(57) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight - -(58) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(59) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#45] - -(60) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [i_item_sk#45] -Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight - -(61) CometProject -Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] -Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(62) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#46] - -(63) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [p_promo_sk#46] -Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight - -(64) CometProject -Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] -Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(65) CometHashAggregate -Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(66) CometExchange -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(67) CometHashAggregate -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] -ReadSchema: struct - -(69) CometFilter -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) - -(70) CometExchange -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(71) CometSort -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] -ReadSchema: struct - -(73) CometFilter -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) - -(74) CometProject -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] - -(75) CometExchange -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(76) CometSort -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] - -(77) CometSortMergeJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter - -(78) CometProject -Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] - -(79) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#65] - -(80) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -Right output [1]: [d_date_sk#65] -Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(81) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] - -(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#66, web_site_id#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(83) CometFilter -Input [2]: [web_site_sk#66, web_site_id#67] -Condition : isnotnull(web_site_sk#66) - -(84) CometProject -Input [2]: [web_site_sk#66, web_site_id#67] -Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#67, 16)) AS web_site_id#68] - -(85) CometBroadcastExchange -Input [2]: [web_site_sk#66, web_site_id#68] -Arguments: [web_site_sk#66, web_site_id#68] - -(86) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -Right output [2]: [web_site_sk#66, web_site_id#68] -Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight - -(87) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] -Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(88) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#69] - -(89) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [i_item_sk#69] -Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight - -(90) CometProject -Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] -Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(91) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#70] - -(92) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [p_promo_sk#70] -Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight - -(93) CometProject -Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] -Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(94) CometHashAggregate -Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Keys [1]: [web_site_id#68] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(95) CometExchange -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(96) CometHashAggregate -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Keys [1]: [web_site_id#68] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(97) CometUnion -Child 0 Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] -Child 1 Input [5]: [channel#81, id#82, sales#83, returns#84, profit#85] -Child 2 Input [5]: [channel#86, id#87, sales#88, returns#89, profit#90] - -(98) CometHashAggregate -Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] -Keys [2]: [channel#76, id#77] -Functions [3]: [partial_sum(sales#78), partial_sum(returns#79), partial_sum(profit#80)] - -(99) CometExchange -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Arguments: hashpartitioning(channel#76, id#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(100) CometHashAggregate -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Keys [2]: [channel#76, id#77] -Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] - -(101) ReusedExchange [Reuses operator id: 99] -Output [8]: [channel#97, id#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] - -(102) CometHashAggregate -Input [8]: [channel#97, id#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Keys [2]: [channel#97, id#98] -Functions [3]: [sum(sales#105), sum(returns#106), sum(profit#107)] - -(103) CometHashAggregate -Input [4]: [channel#97, sales#108, returns#109, profit#110] -Keys [1]: [channel#97] -Functions [3]: [partial_sum(sales#108), partial_sum(returns#109), partial_sum(profit#110)] - -(104) CometExchange -Input [7]: [channel#97, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] -Arguments: hashpartitioning(channel#97, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(105) CometHashAggregate -Input [7]: [channel#97, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] -Keys [1]: [channel#97] -Functions [3]: [sum(sales#108), sum(returns#109), sum(profit#110)] - -(106) ReusedExchange [Reuses operator id: 99] -Output [8]: [channel#117, id#118, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] - -(107) CometHashAggregate -Input [8]: [channel#117, id#118, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] -Keys [2]: [channel#117, id#118] -Functions [3]: [sum(sales#125), sum(returns#126), sum(profit#127)] - -(108) CometHashAggregate -Input [3]: [sales#128, returns#129, profit#130] -Keys: [] -Functions [3]: [partial_sum(sales#128), partial_sum(returns#129), partial_sum(profit#130)] - -(109) CometExchange -Input [6]: [sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(110) CometHashAggregate -Input [6]: [sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] -Keys: [] -Functions [3]: [sum(sales#128), sum(returns#129), sum(profit#130)] - -(111) CometUnion -Child 0 Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] -Child 1 Input [5]: [channel#97, id#140, sales#141, returns#142, profit#143] -Child 2 Input [5]: [channel#144, id#145, sales#146, returns#147, profit#148] - -(112) CometHashAggregate -Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] -Keys [5]: [channel#76, id#77, sales#137, returns#138, profit#139] -Functions: [] - -(113) CometExchange -Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] -Arguments: hashpartitioning(channel#76, id#77, sales#137, returns#138, profit#139, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] - -(114) CometHashAggregate -Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] -Keys [5]: [channel#76, id#77, sales#137, returns#138, profit#139] -Functions: [] - -(115) CometTakeOrderedAndProject -Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#76 ASC NULLS FIRST,id#77 ASC NULLS FIRST], output=[channel#76,id#77,sales#137,returns#138,profit#139]), [channel#76, id#77, sales#137, returns#138, profit#139], 100, 0, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#137, returns#138, profit#139] - -(116) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) - -(119) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(121) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/extended.txt deleted file mode 100644 index 0d6844d93c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/extended.txt +++ /dev/null @@ -1,392 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - -Comet accelerated 374 out of 386 eligible operators (96%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d27ab827e1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt +++ /dev/null @@ -1,128 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id] #2 - CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [s_store_id] #3 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #4 - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #6 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #8 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #9 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #10 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,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)))] - CometExchange [cp_catalog_page_id] #11 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #12 - 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 [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [web_site_id] #15 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #16 - 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 [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #17 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [web_site_sk,web_site_id] #18 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel] #19 - CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange #20 - CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/explain.txt deleted file mode 100644 index 639c731999..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/explain.txt +++ /dev/null @@ -1,267 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (41) -+- * Project (40) - +- Window (39) - +- * CometColumnarToRow (38) - +- CometSort (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometColumnarExchange (34) - +- * HashAggregate (33) - +- Union (32) - :- * HashAggregate (17) - : +- * CometColumnarToRow (16) - : +- CometColumnarExchange (15) - : +- * HashAggregate (14) - : +- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) - : +- CometProject (9) - : +- CometFilter (8) - : +- CometNativeScan parquet spark_catalog.default.item (7) - :- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * HashAggregate (20) - : +- * CometColumnarToRow (19) - : +- ReusedExchange (18) - +- * HashAggregate (31) - +- * CometColumnarToRow (30) - +- CometColumnarExchange (29) - +- * HashAggregate (28) - +- * HashAggregate (27) - +- * CometColumnarToRow (26) - +- ReusedExchange (25) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(4) ReusedExchange [Reuses operator id: 46] -Output [1]: [d_date_sk#5] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [2]: [ws_item_sk#1, ws_net_paid#2] -Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] - -(7) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#6, i_class#7, i_category#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Condition : isnotnull(i_item_sk#6) - -(9) CometProject -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Arguments: [i_item_sk#6, i_class#9, i_category#10], [i_item_sk#6, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#7, 50)) AS i_class#9, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#8, 50)) AS i_category#10] - -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#6, i_class#9, i_category#10] - -(11) BroadcastExchange -Input [3]: [i_item_sk#6, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [3]: [ws_net_paid#2, i_class#9, i_category#10] -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#6, i_class#9, i_category#10] - -(14) HashAggregate [codegen id : 3] -Input [3]: [ws_net_paid#2, i_class#9, i_category#10] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum#11] -Results [3]: [i_category#10, i_class#9, sum#12] - -(15) CometColumnarExchange -Input [3]: [i_category#10, i_class#9, sum#12] -Arguments: hashpartitioning(i_category#10, i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 4] -Input [3]: [i_category#10, i_class#9, sum#12] - -(17) HashAggregate [codegen id : 4] -Input [3]: [i_category#10, i_class#9, sum#12] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#13] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#13,17,2) as decimal(27,2)) AS total_sum#14, i_category#10 AS i_category#15, i_class#9 AS i_class#16, 0 AS g_category#17, 0 AS g_class#18, 0 AS lochierarchy#19] - -(18) ReusedExchange [Reuses operator id: 15] -Output [3]: [i_category#20, i_class#21, sum#22] - -(19) CometColumnarToRow [codegen id : 8] -Input [3]: [i_category#20, i_class#21, sum#22] - -(20) HashAggregate [codegen id : 8] -Input [3]: [i_category#20, i_class#21, sum#22] -Keys [2]: [i_category#20, i_class#21] -Functions [1]: [sum(UnscaledValue(ws_net_paid#23))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#23))#13] -Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#23))#13,17,2) AS total_sum#24, i_category#20] - -(21) HashAggregate [codegen id : 8] -Input [2]: [total_sum#24, i_category#20] -Keys [1]: [i_category#20] -Functions [1]: [partial_sum(total_sum#24)] -Aggregate Attributes [2]: [sum#25, isEmpty#26] -Results [3]: [i_category#20, sum#27, isEmpty#28] - -(22) CometColumnarExchange -Input [3]: [i_category#20, sum#27, isEmpty#28] -Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 9] -Input [3]: [i_category#20, sum#27, isEmpty#28] - -(24) HashAggregate [codegen id : 9] -Input [3]: [i_category#20, sum#27, isEmpty#28] -Keys [1]: [i_category#20] -Functions [1]: [sum(total_sum#24)] -Aggregate Attributes [1]: [sum(total_sum#24)#29] -Results [6]: [sum(total_sum#24)#29 AS total_sum#30, i_category#20, null AS i_class#31, 0 AS g_category#32, 1 AS g_class#33, 1 AS lochierarchy#34] - -(25) ReusedExchange [Reuses operator id: 15] -Output [3]: [i_category#35, i_class#36, sum#37] - -(26) CometColumnarToRow [codegen id : 13] -Input [3]: [i_category#35, i_class#36, sum#37] - -(27) HashAggregate [codegen id : 13] -Input [3]: [i_category#35, i_class#36, sum#37] -Keys [2]: [i_category#35, i_class#36] -Functions [1]: [sum(UnscaledValue(ws_net_paid#38))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#38))#13] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#38))#13,17,2) AS total_sum#39] - -(28) HashAggregate [codegen id : 13] -Input [1]: [total_sum#39] -Keys: [] -Functions [1]: [partial_sum(total_sum#39)] -Aggregate Attributes [2]: [sum#40, isEmpty#41] -Results [2]: [sum#42, isEmpty#43] - -(29) CometColumnarExchange -Input [2]: [sum#42, isEmpty#43] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 14] -Input [2]: [sum#42, isEmpty#43] - -(31) HashAggregate [codegen id : 14] -Input [2]: [sum#42, isEmpty#43] -Keys: [] -Functions [1]: [sum(total_sum#39)] -Aggregate Attributes [1]: [sum(total_sum#39)#44] -Results [6]: [sum(total_sum#39)#44 AS total_sum#45, null AS i_category#46, null AS i_class#47, 1 AS g_category#48, 1 AS g_class#49, 2 AS lochierarchy#50] - -(32) Union - -(33) HashAggregate [codegen id : 15] -Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Keys [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Functions: [] -Aggregate Attributes: [] -Results [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] - -(34) CometColumnarExchange -Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Arguments: hashpartitioning(total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(35) CometHashAggregate -Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Keys [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Functions: [] - -(36) CometExchange -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] -Arguments: hashpartitioning(lochierarchy#19, _w0#51, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(37) CometSort -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] -Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51], [lochierarchy#19 ASC NULLS FIRST, _w0#51 ASC NULLS FIRST, total_sum#14 DESC NULLS LAST] - -(38) CometColumnarToRow [codegen id : 16] -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] - -(39) Window -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] -Arguments: [rank(total_sum#14) windowspecdefinition(lochierarchy#19, _w0#51, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#52], [lochierarchy#19, _w0#51], [total_sum#14 DESC NULLS LAST] - -(40) Project [codegen id : 17] -Output [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] -Input [6]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51, rank_within_parent#52] - -(41) TakeOrderedAndProject -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] -Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#52 ASC NULLS FIRST], [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometNativeScan parquet spark_catalog.default.date_dim (42) - - -(42) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#53] -Condition : (((isnotnull(d_month_seq#53) AND (d_month_seq#53 >= 1212)) AND (d_month_seq#53 <= 1223)) AND isnotnull(d_date_sk#5)) - -(44) CometProject -Input [2]: [d_date_sk#5, d_month_seq#53] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(45) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(46) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/simplified.txt deleted file mode 100644 index 2ccc8c0c39..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (17) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] - CometColumnarExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - WholeStageCodegen (15) - HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (4) - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #3 - WholeStageCodegen (3) - HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] - Project [ws_net_paid,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - WholeStageCodegen (9) - HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #6 - WholeStageCodegen (8) - HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 - WholeStageCodegen (14) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #7 - WholeStageCodegen (13) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/explain.txt deleted file mode 100644 index 7f23c0ebbf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,241 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (37) -+- * Project (36) - +- Window (35) - +- * CometColumnarToRow (34) - +- CometSort (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometExchange (30) - +- CometHashAggregate (29) - +- CometUnion (28) - :- CometHashAggregate (17) - : +- CometExchange (16) - : +- CometHashAggregate (15) - : +- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - :- CometHashAggregate (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometHashAggregate (19) - : +- ReusedExchange (18) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometHashAggregate (24) - +- ReusedExchange (23) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] -Arguments: [ws_item_sk#1, ws_net_paid#2], [ws_item_sk#1, ws_net_paid#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#7, i_class#8, i_category#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Condition : isnotnull(i_item_sk#7) - -(11) CometProject -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Arguments: [i_item_sk#7, i_class#10, i_category#11], [i_item_sk#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#8, 50)) AS i_class#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#9, 50)) AS i_category#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [i_item_sk#7, i_class#10, i_category#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#1, ws_net_paid#2] -Right output [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_net_paid#2, i_class#10, i_category#11], [ws_net_paid#2, i_class#10, i_category#11] - -(15) CometHashAggregate -Input [3]: [ws_net_paid#2, i_class#10, i_category#11] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] - -(16) CometExchange -Input [3]: [i_category#11, i_class#10, sum#12] -Arguments: hashpartitioning(i_category#11, i_class#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [i_category#11, i_class#10, sum#12] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] - -(18) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#13, i_class#14, sum#15] - -(19) CometHashAggregate -Input [3]: [i_category#13, i_class#14, sum#15] -Keys [2]: [i_category#13, i_class#14] -Functions [1]: [sum(UnscaledValue(ws_net_paid#16))] - -(20) CometHashAggregate -Input [2]: [total_sum#17, i_category#13] -Keys [1]: [i_category#13] -Functions [1]: [partial_sum(total_sum#17)] - -(21) CometExchange -Input [3]: [i_category#13, sum#18, isEmpty#19] -Arguments: hashpartitioning(i_category#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(22) CometHashAggregate -Input [3]: [i_category#13, sum#18, isEmpty#19] -Keys [1]: [i_category#13] -Functions [1]: [sum(total_sum#17)] - -(23) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#20, i_class#21, sum#22] - -(24) CometHashAggregate -Input [3]: [i_category#20, i_class#21, sum#22] -Keys [2]: [i_category#20, i_class#21] -Functions [1]: [sum(UnscaledValue(ws_net_paid#23))] - -(25) CometHashAggregate -Input [1]: [total_sum#24] -Keys: [] -Functions [1]: [partial_sum(total_sum#24)] - -(26) CometExchange -Input [2]: [sum#25, isEmpty#26] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(27) CometHashAggregate -Input [2]: [sum#25, isEmpty#26] -Keys: [] -Functions [1]: [sum(total_sum#24)] - -(28) CometUnion -Child 0 Input [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] -Child 1 Input [6]: [total_sum#33, i_category#13, i_class#34, g_category#35, g_class#36, lochierarchy#37] -Child 2 Input [6]: [total_sum#38, i_category#39, i_class#40, g_category#41, g_class#42, lochierarchy#43] - -(29) CometHashAggregate -Input [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] -Keys [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] -Functions: [] - -(30) CometExchange -Input [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] -Arguments: hashpartitioning(total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(31) CometHashAggregate -Input [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] -Keys [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] -Functions: [] - -(32) CometExchange -Input [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44] -Arguments: hashpartitioning(lochierarchy#32, _w0#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(33) CometSort -Input [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44] -Arguments: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44], [lochierarchy#32 ASC NULLS FIRST, _w0#44 ASC NULLS FIRST, total_sum#27 DESC NULLS LAST] - -(34) CometColumnarToRow [codegen id : 1] -Input [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44] - -(35) Window -Input [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44] -Arguments: [rank(total_sum#27) windowspecdefinition(lochierarchy#32, _w0#44, total_sum#27 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#45], [lochierarchy#32, _w0#44], [total_sum#27 DESC NULLS LAST] - -(36) Project [codegen id : 2] -Output [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, rank_within_parent#45] -Input [6]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44, rank_within_parent#45] - -(37) TakeOrderedAndProject -Input [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, rank_within_parent#45] -Arguments: 100, [lochierarchy#32 DESC NULLS LAST, CASE WHEN (lochierarchy#32 = 0) THEN i_category#28 END ASC NULLS FIRST, rank_within_parent#45 ASC NULLS FIRST], [total_sum#27, i_category#28, i_class#29, lochierarchy#32, rank_within_parent#45] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (42) -+- * CometColumnarToRow (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) - - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(39) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(40) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(41) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(42) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 11e3c03d80..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (2) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] - CometExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometUnion [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometHashAggregate [sum] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(UnscaledValue(ws_net_paid)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ws_net_paid] [i_category,i_class,sum] - CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange [i_category] #7 - CometHashAggregate [total_sum] [i_category,sum,isEmpty] - CometHashAggregate [i_class,sum] [total_sum,i_category,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange #8 - CometHashAggregate [total_sum] [sum,isEmpty] - CometHashAggregate [i_category,i_class,sum] [total_sum,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt deleted file mode 100644 index 7f23c0ebbf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt +++ /dev/null @@ -1,241 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (37) -+- * Project (36) - +- Window (35) - +- * CometColumnarToRow (34) - +- CometSort (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometExchange (30) - +- CometHashAggregate (29) - +- CometUnion (28) - :- CometHashAggregate (17) - : +- CometExchange (16) - : +- CometHashAggregate (15) - : +- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - :- CometHashAggregate (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometHashAggregate (19) - : +- ReusedExchange (18) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometHashAggregate (24) - +- ReusedExchange (23) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] -Arguments: [ws_item_sk#1, ws_net_paid#2], [ws_item_sk#1, ws_net_paid#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#7, i_class#8, i_category#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Condition : isnotnull(i_item_sk#7) - -(11) CometProject -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Arguments: [i_item_sk#7, i_class#10, i_category#11], [i_item_sk#7, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#8, 50)) AS i_class#10, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#9, 50)) AS i_category#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [i_item_sk#7, i_class#10, i_category#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#1, ws_net_paid#2] -Right output [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_net_paid#2, i_class#10, i_category#11], [ws_net_paid#2, i_class#10, i_category#11] - -(15) CometHashAggregate -Input [3]: [ws_net_paid#2, i_class#10, i_category#11] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] - -(16) CometExchange -Input [3]: [i_category#11, i_class#10, sum#12] -Arguments: hashpartitioning(i_category#11, i_class#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [i_category#11, i_class#10, sum#12] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] - -(18) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#13, i_class#14, sum#15] - -(19) CometHashAggregate -Input [3]: [i_category#13, i_class#14, sum#15] -Keys [2]: [i_category#13, i_class#14] -Functions [1]: [sum(UnscaledValue(ws_net_paid#16))] - -(20) CometHashAggregate -Input [2]: [total_sum#17, i_category#13] -Keys [1]: [i_category#13] -Functions [1]: [partial_sum(total_sum#17)] - -(21) CometExchange -Input [3]: [i_category#13, sum#18, isEmpty#19] -Arguments: hashpartitioning(i_category#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(22) CometHashAggregate -Input [3]: [i_category#13, sum#18, isEmpty#19] -Keys [1]: [i_category#13] -Functions [1]: [sum(total_sum#17)] - -(23) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#20, i_class#21, sum#22] - -(24) CometHashAggregate -Input [3]: [i_category#20, i_class#21, sum#22] -Keys [2]: [i_category#20, i_class#21] -Functions [1]: [sum(UnscaledValue(ws_net_paid#23))] - -(25) CometHashAggregate -Input [1]: [total_sum#24] -Keys: [] -Functions [1]: [partial_sum(total_sum#24)] - -(26) CometExchange -Input [2]: [sum#25, isEmpty#26] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(27) CometHashAggregate -Input [2]: [sum#25, isEmpty#26] -Keys: [] -Functions [1]: [sum(total_sum#24)] - -(28) CometUnion -Child 0 Input [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] -Child 1 Input [6]: [total_sum#33, i_category#13, i_class#34, g_category#35, g_class#36, lochierarchy#37] -Child 2 Input [6]: [total_sum#38, i_category#39, i_class#40, g_category#41, g_class#42, lochierarchy#43] - -(29) CometHashAggregate -Input [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] -Keys [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] -Functions: [] - -(30) CometExchange -Input [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] -Arguments: hashpartitioning(total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(31) CometHashAggregate -Input [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] -Keys [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] -Functions: [] - -(32) CometExchange -Input [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44] -Arguments: hashpartitioning(lochierarchy#32, _w0#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(33) CometSort -Input [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44] -Arguments: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44], [lochierarchy#32 ASC NULLS FIRST, _w0#44 ASC NULLS FIRST, total_sum#27 DESC NULLS LAST] - -(34) CometColumnarToRow [codegen id : 1] -Input [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44] - -(35) Window -Input [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44] -Arguments: [rank(total_sum#27) windowspecdefinition(lochierarchy#32, _w0#44, total_sum#27 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#45], [lochierarchy#32, _w0#44], [total_sum#27 DESC NULLS LAST] - -(36) Project [codegen id : 2] -Output [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, rank_within_parent#45] -Input [6]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44, rank_within_parent#45] - -(37) TakeOrderedAndProject -Input [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, rank_within_parent#45] -Arguments: 100, [lochierarchy#32 DESC NULLS LAST, CASE WHEN (lochierarchy#32 = 0) THEN i_category#28 END ASC NULLS FIRST, rank_within_parent#45 ASC NULLS FIRST], [total_sum#27, i_category#28, i_class#29, lochierarchy#32, rank_within_parent#45] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (42) -+- * CometColumnarToRow (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) - - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(39) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(40) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(41) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(42) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/extended.txt deleted file mode 100644 index 46c47555a8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/extended.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 11e3c03d80..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (2) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] - CometExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometUnion [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometHashAggregate [sum] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(UnscaledValue(ws_net_paid)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ws_net_paid] [i_category,i_class,sum] - CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange [i_category] #7 - CometHashAggregate [total_sum] [i_category,sum,isEmpty] - CometHashAggregate [i_class,sum] [total_sum,i_category,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange #8 - CometHashAggregate [total_sum] [sum,isEmpty] - CometHashAggregate [i_category,i_class,sum] [total_sum,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/explain.txt deleted file mode 100644 index bb10f1112d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/explain.txt +++ /dev/null @@ -1,172 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometSort (24) - +- CometColumnarExchange (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.store_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.item (4) - +- ReusedExchange (11) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] - -(7) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(8) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(11) ReusedExchange [Reuses operator id: 30] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] - -(14) HashAggregate [codegen id : 3] -Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(15) CometColumnarExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(17) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#17] -Results [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#17,17,2) AS _w0#19] - -(18) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(22) Project [codegen id : 6] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] - -(23) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(24) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 7] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometNativeScan parquet spark_catalog.default.date_dim (26) - - -(26) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(28) CometProject -Input [2]: [d_date_sk#14, d_date#22] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(30) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/simplified.txt deleted file mode 100644 index c1ec019e57..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/simplified.txt +++ /dev/null @@ -1,47 +0,0 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (6) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/explain.txt deleted file mode 100644 index 7aac2f05bf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,173 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometSort (24) - +- CometColumnarExchange (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] - -(23) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 3] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(28) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(30) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/simplified.txt deleted file mode 100644 index 46191f59cd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,39 +0,0 @@ -WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #2 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt deleted file mode 100644 index 7aac2f05bf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt +++ /dev/null @@ -1,173 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometSort (24) - +- CometColumnarExchange (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_item_id#6, 16)) AS i_item_id#11, i_item_desc#7, i_current_price#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_class#9, 50)) AS i_class#12, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_category#10, 50)) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] - -(23) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 3] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(28) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(30) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/extended.txt deleted file mode 100644 index 3dbaf2e346..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/extended.txt +++ /dev/null @@ -1,33 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 46191f59cd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt +++ /dev/null @@ -1,39 +0,0 @@ -WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #2 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/explain.txt deleted file mode 100644 index 1f2e924269..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/explain.txt +++ /dev/null @@ -1,281 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : +- BroadcastExchange (9) - : : : +- * Project (8) - : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : :- * ColumnarToRow (5) - : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (6) - : : +- BroadcastExchange (22) - : : +- Union (21) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * ColumnarToRow (17) - : : : +- Scan parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (18) - : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.customer_address (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.customer_demographics (32) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(4) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] - -(6) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#7] - -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#4] -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] - -(9) BroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#4] -Join type: LeftSemi -Join condition: None - -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#6)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] - -(13) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#7] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#8 AS customer_sk#10] -Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#7] - -(16) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#11, cs_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#12), dynamicpruningexpression(cs_sold_date_sk#12 IN dynamicpruning#6)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#11, cs_sold_date_sk#12] - -(18) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#13] - -(19) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#12] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#11 AS customer_sk#14] -Input [3]: [cs_ship_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13] - -(21) Union - -(22) BroadcastExchange -Input [1]: [customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(23) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#10] -Join type: LeftSemi -Join condition: None - -(24) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(25) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#15, ca_county#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_county, [Dona Ana County,Douglas County,Gaines County,Richland County,Walker County]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [ca_address_sk#15, ca_county#16] -Condition : (ca_county#16 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#15)) - -(27) CometProject -Input [2]: [ca_address_sk#15, ca_county#16] -Arguments: [ca_address_sk#15], [ca_address_sk#15] - -(28) CometColumnarToRow [codegen id : 7] -Input [1]: [ca_address_sk#15] - -(29) BroadcastExchange -Input [1]: [ca_address_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(30) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#15] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 9] -Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#15] - -(32) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_education_status#20, cd_purchase_estimate#21, cd_credit_rating#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(33) CometFilter -Input [9]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_education_status#20, cd_purchase_estimate#21, cd_credit_rating#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Condition : isnotnull(cd_demo_sk#17) - -(34) CometProject -Input [9]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_education_status#20, cd_purchase_estimate#21, cd_credit_rating#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Arguments: [cd_demo_sk#17, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25], [cd_demo_sk#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#18, 1, true, false, true) AS cd_gender#26, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) AS cd_marital_status#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#20, 20, true, false, true) AS cd_education_status#28, cd_purchase_estimate#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#22, 10, true, false, true) AS cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] - -(35) CometColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#17, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] - -(36) BroadcastExchange -Input [9]: [cd_demo_sk#17, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#17] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 9] -Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#17, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] - -(39) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#30] -Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#31] - -(40) CometColumnarExchange -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#31] -Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(41) CometColumnarToRow [codegen id : 10] -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#31] - -(42) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#31] -Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#21, cd_credit_rating#29, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#32] -Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#32 AS cnt1#33, cd_purchase_estimate#21, count(1)#32 AS cnt2#34, cd_credit_rating#29, count(1)#32 AS cnt3#35, cd_dep_count#23, count(1)#32 AS cnt4#36, cd_dep_employed_count#24, count(1)#32 AS cnt5#37, cd_dep_college_count#25, count(1)#32 AS cnt6#38] - -(43) TakeOrderedAndProject -Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#33, cd_purchase_estimate#21, cnt2#34, cd_credit_rating#29, cnt3#35, cd_dep_count#23, cnt4#36, cd_dep_employed_count#24, cnt5#37, cd_dep_college_count#25, cnt6#38] -Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#21 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#33, cd_purchase_estimate#21, cnt2#34, cd_credit_rating#29, cnt3#35, cd_dep_count#23, cnt4#36, cd_dep_employed_count#24, cnt5#37, cd_dep_college_count#25, cnt6#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometNativeScan parquet spark_catalog.default.date_dim (44) - - -(44) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#39, d_moy#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [3]: [d_date_sk#7, d_year#39, d_moy#40] -Condition : (((((isnotnull(d_year#39) AND isnotnull(d_moy#40)) AND (d_year#39 = 2002)) AND (d_moy#40 >= 4)) AND (d_moy#40 <= 7)) AND isnotnull(d_date_sk#7)) - -(46) CometProject -Input [3]: [d_date_sk#7, d_year#39, d_moy#40] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(48) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#12 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/simplified.txt deleted file mode 100644 index 11bfb6f6c3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] - WholeStageCodegen (10) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] - Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - Union - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [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] - 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] - CometNativeScan 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/q10a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/explain.txt deleted file mode 100644 index ac9e39c190..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,267 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometHashAggregate (38) - +- CometExchange (37) - +- CometHashAggregate (36) - +- CometProject (35) - +- CometBroadcastHashJoin (34) - :- CometProject (29) - : +- CometBroadcastHashJoin (28) - : :- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometBroadcastHashJoin (11) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : +- CometBroadcastExchange (10) - : : : +- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : +- CometBroadcastExchange (21) - : : +- CometUnion (20) - : : :- CometProject (15) - : : : +- CometBroadcastHashJoin (14) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (12) - : : : +- ReusedExchange (13) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (17) - : +- CometBroadcastExchange (27) - : +- CometProject (26) - : +- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) - +- CometBroadcastExchange (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (30) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) - -(6) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#4], [ss_customer_sk#4] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: [ss_customer_sk#4] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ss_customer_sk#4] -Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(13) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#7] - -(14) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Right output [1]: [d_date_sk#7] -Arguments: [ws_sold_date_sk#11], [d_date_sk#7], Inner, BuildRight - -(15) CometProject -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#7] -Arguments: [customer_sk#13], [ws_bill_customer_sk#10 AS customer_sk#13] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] -ReadSchema: struct - -(17) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#17] - -(18) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -Right output [1]: [d_date_sk#17] -Arguments: [cs_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight - -(19) CometProject -Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] -Arguments: [customer_sk#18], [cs_ship_customer_sk#14 AS customer_sk#18] - -(20) CometUnion -Child 0 Input [1]: [customer_sk#13] -Child 1 Input [1]: [customer_sk#18] - -(21) CometBroadcastExchange -Input [1]: [customer_sk#13] -Arguments: [customer_sk#13] - -(22) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [customer_sk#13] -Arguments: [c_customer_sk#1], [customer_sk#13], LeftSemi, BuildRight - -(23) CometProject -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_county#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_county, [Dona Ana County,Douglas County,Gaines County,Richland County,Walker County]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [ca_address_sk#19, ca_county#20] -Condition : (ca_county#20 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#19)) - -(26) CometProject -Input [2]: [ca_address_sk#19, ca_county#20] -Arguments: [ca_address_sk#19], [ca_address_sk#19] - -(27) CometBroadcastExchange -Input [1]: [ca_address_sk#19] -Arguments: [ca_address_sk#19] - -(28) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ca_address_sk#19] -Arguments: [c_current_addr_sk#3], [ca_address_sk#19], Inner, BuildRight - -(29) CometProject -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19] -Arguments: [c_current_cdemo_sk#2], [c_current_cdemo_sk#2] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [9]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Condition : isnotnull(cd_demo_sk#21) - -(32) CometProject -Input [9]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Arguments: [cd_demo_sk#21, cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29], [cd_demo_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#22, 1, true, false, true) AS cd_gender#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#23, 1, true, false, true) AS cd_marital_status#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#24, 20, true, false, true) AS cd_education_status#32, cd_purchase_estimate#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#26, 10, true, false, true) AS cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] - -(33) CometBroadcastExchange -Input [9]: [cd_demo_sk#21, cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Arguments: [cd_demo_sk#21, cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] - -(34) CometBroadcastHashJoin -Left output [1]: [c_current_cdemo_sk#2] -Right output [9]: [cd_demo_sk#21, cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#21], Inner, BuildRight - -(35) CometProject -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#21, cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Arguments: [cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29], [cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] - -(36) CometHashAggregate -Input [8]: [cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Keys [8]: [cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Functions [1]: [partial_count(1)] - -(37) CometExchange -Input [9]: [cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, count#34] -Arguments: hashpartitioning(cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(38) CometHashAggregate -Input [9]: [cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, count#34] -Keys [8]: [cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Functions [1]: [count(1)] - -(39) CometTakeOrderedAndProject -Input [14]: [cd_gender#30, cd_marital_status#31, cd_education_status#32, cnt1#35, cd_purchase_estimate#25, cnt2#36, cd_credit_rating#33, cnt3#37, cd_dep_count#27, cnt4#38, cd_dep_employed_count#28, cnt5#39, cd_dep_college_count#29, cnt6#40] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#30 ASC NULLS FIRST,cd_marital_status#31 ASC NULLS FIRST,cd_education_status#32 ASC NULLS FIRST,cd_purchase_estimate#25 ASC NULLS FIRST,cd_credit_rating#33 ASC NULLS FIRST,cd_dep_count#27 ASC NULLS FIRST,cd_dep_employed_count#28 ASC NULLS FIRST,cd_dep_college_count#29 ASC NULLS FIRST], output=[cd_gender#30,cd_marital_status#31,cd_education_status#32,cnt1#35,cd_purchase_estimate#25,cnt2#36,cd_credit_rating#33,cnt3#37,cd_dep_count#27,cnt4#38,cd_dep_employed_count#28,cnt5#39,cd_dep_college_count#29,cnt6#40]), [cd_gender#30, cd_marital_status#31, cd_education_status#32, cnt1#35, cd_purchase_estimate#25, cnt2#36, cd_credit_rating#33, cnt3#37, cd_dep_count#27, cnt4#38, cd_dep_employed_count#28, cnt5#39, cd_dep_college_count#29, cnt6#40], 100, 0, [cd_gender#30 ASC NULLS FIRST, cd_marital_status#31 ASC NULLS FIRST, cd_education_status#32 ASC NULLS FIRST, cd_purchase_estimate#25 ASC NULLS FIRST, cd_credit_rating#33 ASC NULLS FIRST, cd_dep_count#27 ASC NULLS FIRST, cd_dep_employed_count#28 ASC NULLS FIRST, cd_dep_college_count#29 ASC NULLS FIRST], [cd_gender#30, cd_marital_status#31, cd_education_status#32, cnt1#35, cd_purchase_estimate#25, cnt2#36, cd_credit_rating#33, cnt3#37, cd_dep_count#27, cnt4#38, cd_dep_employed_count#28, cnt5#39, cd_dep_college_count#29, cnt6#40] - -(40) CometColumnarToRow [codegen id : 1] -Input [14]: [cd_gender#30, cd_marital_status#31, cd_education_status#32, cnt1#35, cd_purchase_estimate#25, cnt2#36, cd_credit_rating#33, cnt3#37, cd_dep_count#27, cnt4#38, cd_dep_employed_count#28, cnt5#39, cd_dep_college_count#29, cnt6#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) - -(43) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(45) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 38f41d10a5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6] - CometHashAggregate [count] [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6,count(1)] - CometExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - 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,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_cdemo_sk,c_current_addr_sk,ca_address_sk] - CometProject [c_current_cdemo_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [customer_sk] #5 - CometUnion [customer_sk] - CometProject [ws_bill_customer_sk] [customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - 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 - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [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] - 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 [native_iceberg_compat] 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/q10a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt deleted file mode 100644 index ac9e39c190..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt +++ /dev/null @@ -1,267 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometHashAggregate (38) - +- CometExchange (37) - +- CometHashAggregate (36) - +- CometProject (35) - +- CometBroadcastHashJoin (34) - :- CometProject (29) - : +- CometBroadcastHashJoin (28) - : :- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometBroadcastHashJoin (11) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : +- CometBroadcastExchange (10) - : : : +- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : +- CometBroadcastExchange (21) - : : +- CometUnion (20) - : : :- CometProject (15) - : : : +- CometBroadcastHashJoin (14) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (12) - : : : +- ReusedExchange (13) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (17) - : +- CometBroadcastExchange (27) - : +- CometProject (26) - : +- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) - +- CometBroadcastExchange (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (30) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) - -(6) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#4], [ss_customer_sk#4] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: [ss_customer_sk#4] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ss_customer_sk#4] -Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(13) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#7] - -(14) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Right output [1]: [d_date_sk#7] -Arguments: [ws_sold_date_sk#11], [d_date_sk#7], Inner, BuildRight - -(15) CometProject -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#7] -Arguments: [customer_sk#13], [ws_bill_customer_sk#10 AS customer_sk#13] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] -ReadSchema: struct - -(17) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#17] - -(18) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -Right output [1]: [d_date_sk#17] -Arguments: [cs_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight - -(19) CometProject -Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] -Arguments: [customer_sk#18], [cs_ship_customer_sk#14 AS customer_sk#18] - -(20) CometUnion -Child 0 Input [1]: [customer_sk#13] -Child 1 Input [1]: [customer_sk#18] - -(21) CometBroadcastExchange -Input [1]: [customer_sk#13] -Arguments: [customer_sk#13] - -(22) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [customer_sk#13] -Arguments: [c_customer_sk#1], [customer_sk#13], LeftSemi, BuildRight - -(23) CometProject -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_county#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_county, [Dona Ana County,Douglas County,Gaines County,Richland County,Walker County]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [ca_address_sk#19, ca_county#20] -Condition : (ca_county#20 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#19)) - -(26) CometProject -Input [2]: [ca_address_sk#19, ca_county#20] -Arguments: [ca_address_sk#19], [ca_address_sk#19] - -(27) CometBroadcastExchange -Input [1]: [ca_address_sk#19] -Arguments: [ca_address_sk#19] - -(28) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ca_address_sk#19] -Arguments: [c_current_addr_sk#3], [ca_address_sk#19], Inner, BuildRight - -(29) CometProject -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19] -Arguments: [c_current_cdemo_sk#2], [c_current_cdemo_sk#2] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [9]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [9]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Condition : isnotnull(cd_demo_sk#21) - -(32) CometProject -Input [9]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Arguments: [cd_demo_sk#21, cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29], [cd_demo_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#22, 1, true, false, true) AS cd_gender#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#23, 1, true, false, true) AS cd_marital_status#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#24, 20, true, false, true) AS cd_education_status#32, cd_purchase_estimate#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_credit_rating#26, 10, true, false, true) AS cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] - -(33) CometBroadcastExchange -Input [9]: [cd_demo_sk#21, cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Arguments: [cd_demo_sk#21, cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] - -(34) CometBroadcastHashJoin -Left output [1]: [c_current_cdemo_sk#2] -Right output [9]: [cd_demo_sk#21, cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#21], Inner, BuildRight - -(35) CometProject -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#21, cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Arguments: [cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29], [cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] - -(36) CometHashAggregate -Input [8]: [cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Keys [8]: [cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Functions [1]: [partial_count(1)] - -(37) CometExchange -Input [9]: [cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, count#34] -Arguments: hashpartitioning(cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(38) CometHashAggregate -Input [9]: [cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, count#34] -Keys [8]: [cd_gender#30, cd_marital_status#31, cd_education_status#32, cd_purchase_estimate#25, cd_credit_rating#33, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Functions [1]: [count(1)] - -(39) CometTakeOrderedAndProject -Input [14]: [cd_gender#30, cd_marital_status#31, cd_education_status#32, cnt1#35, cd_purchase_estimate#25, cnt2#36, cd_credit_rating#33, cnt3#37, cd_dep_count#27, cnt4#38, cd_dep_employed_count#28, cnt5#39, cd_dep_college_count#29, cnt6#40] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cd_gender#30 ASC NULLS FIRST,cd_marital_status#31 ASC NULLS FIRST,cd_education_status#32 ASC NULLS FIRST,cd_purchase_estimate#25 ASC NULLS FIRST,cd_credit_rating#33 ASC NULLS FIRST,cd_dep_count#27 ASC NULLS FIRST,cd_dep_employed_count#28 ASC NULLS FIRST,cd_dep_college_count#29 ASC NULLS FIRST], output=[cd_gender#30,cd_marital_status#31,cd_education_status#32,cnt1#35,cd_purchase_estimate#25,cnt2#36,cd_credit_rating#33,cnt3#37,cd_dep_count#27,cnt4#38,cd_dep_employed_count#28,cnt5#39,cd_dep_college_count#29,cnt6#40]), [cd_gender#30, cd_marital_status#31, cd_education_status#32, cnt1#35, cd_purchase_estimate#25, cnt2#36, cd_credit_rating#33, cnt3#37, cd_dep_count#27, cnt4#38, cd_dep_employed_count#28, cnt5#39, cd_dep_college_count#29, cnt6#40], 100, 0, [cd_gender#30 ASC NULLS FIRST, cd_marital_status#31 ASC NULLS FIRST, cd_education_status#32 ASC NULLS FIRST, cd_purchase_estimate#25 ASC NULLS FIRST, cd_credit_rating#33 ASC NULLS FIRST, cd_dep_count#27 ASC NULLS FIRST, cd_dep_employed_count#28 ASC NULLS FIRST, cd_dep_college_count#29 ASC NULLS FIRST], [cd_gender#30, cd_marital_status#31, cd_education_status#32, cnt1#35, cd_purchase_estimate#25, cnt2#36, cd_credit_rating#33, cnt3#37, cd_dep_count#27, cnt4#38, cd_dep_employed_count#28, cnt5#39, cd_dep_college_count#29, cnt6#40] - -(40) CometColumnarToRow [codegen id : 1] -Input [14]: [cd_gender#30, cd_marital_status#31, cd_education_status#32, cnt1#35, cd_purchase_estimate#25, cnt2#36, cd_credit_rating#33, cnt3#37, cd_dep_count#27, cnt4#38, cd_dep_employed_count#28, cnt5#39, cd_dep_college_count#29, cnt6#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_moy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Condition : (((((isnotnull(d_year#8) AND isnotnull(d_moy#9)) AND (d_year#8 = 2002)) AND (d_moy#9 >= 4)) AND (d_moy#9 <= 7)) AND isnotnull(d_date_sk#7)) - -(43) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_moy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(45) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/extended.txt deleted file mode 100644 index 2cdc75e15e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/extended.txt +++ /dev/null @@ -1,56 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 38f41d10a5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6] - CometHashAggregate [count] [cd_gender,cd_marital_status,cd_education_status,cnt1,cd_purchase_estimate,cnt2,cd_credit_rating,cnt3,cd_dep_count,cnt4,cd_dep_employed_count,cnt5,cd_dep_college_count,cnt6,count(1)] - CometExchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - 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,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_cdemo_sk,c_current_addr_sk,ca_address_sk] - CometProject [c_current_cdemo_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [customer_sk] #5 - CometUnion [customer_sk] - CometProject [ws_bill_customer_sk] [customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 [ca_address_sk] #6 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] - 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 - CometProject [cd_gender,cd_marital_status,cd_education_status,cd_credit_rating] [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] - 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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/explain.txt deleted file mode 100644 index 207cd830ed..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/explain.txt +++ /dev/null @@ -1,513 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (79) -+- * Project (78) - +- * BroadcastHashJoin Inner BuildRight (77) - :- * Project (58) - : +- * BroadcastHashJoin Inner BuildRight (57) - : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- * HashAggregate (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * Project (28) - : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (26) - : : : +- * Filter (25) - : : : +- * ColumnarToRow (24) - : : : +- Scan parquet spark_catalog.default.store_sales (23) - : : +- ReusedExchange (29) - : +- BroadcastExchange (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- * CometColumnarToRow (53) - : +- CometColumnarExchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (41) - : : : +- CometProject (40) - : : : +- CometFilter (39) - : : : +- CometNativeScan parquet spark_catalog.default.customer (38) - : : +- BroadcastExchange (45) - : : +- * Filter (44) - : : +- * ColumnarToRow (43) - : : +- Scan parquet spark_catalog.default.web_sales (42) - : +- ReusedExchange (48) - +- BroadcastExchange (76) - +- * HashAggregate (75) - +- * CometColumnarToRow (74) - +- CometColumnarExchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * CometColumnarToRow (62) - : : +- CometProject (61) - : : +- CometFilter (60) - : : +- CometNativeScan parquet spark_catalog.default.customer (59) - : +- BroadcastExchange (66) - : +- * Filter (65) - : +- * ColumnarToRow (64) - : +- Scan parquet spark_catalog.default.web_sales (63) - +- ReusedExchange (69) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] - -(4) CometColumnarToRow [codegen id : 3] -Input [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] - -(5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(7) Filter [codegen id : 1] -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#15) - -(8) BroadcastExchange -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#15] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(11) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#20, d_year#21] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#18] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] - -(14) HashAggregate [codegen id : 3] -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] -Aggregate Attributes [1]: [sum#22] -Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] - -(15) CometColumnarExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 16] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] - -(17) HashAggregate [codegen id : 16] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#23] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24] -Results [2]: [c_customer_id#9 AS customer_id#25, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#24,18,2) AS year_total#26] - -(18) Filter [codegen id : 16] -Input [2]: [customer_id#25, year_total#26] -Condition : (isnotnull(year_total#26) AND (year_total#26 > 0.00)) - -(19) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(20) CometFilter -Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] -Condition : (isnotnull(c_customer_sk#27) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#28, 16, true, false, true))) - -(21) CometProject -Input [8]: [c_customer_sk#27, c_customer_id#28, c_first_name#29, c_last_name#30, c_preferred_cust_flag#31, c_birth_country#32, c_login#33, c_email_address#34] -Arguments: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14], [c_customer_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#28, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#29, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#30, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#31, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#33, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#34, 50, true, false, true) AS c_email_address#14] - -(22) CometColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] - -(23) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_sold_date_sk#38 IN dynamicpruning#39)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] - -(25) Filter [codegen id : 4] -Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Condition : isnotnull(ss_customer_sk#35) - -(26) BroadcastExchange -Input [4]: [ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#27] -Right keys [1]: [ss_customer_sk#35] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] -Input [12]: [c_customer_sk#27, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_customer_sk#35, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38] - -(29) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#40, d_year#41] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, ss_sold_date_sk#38, d_date_sk#40, d_year#41] - -(32) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, ss_ext_discount_amt#36, ss_ext_list_price#37, d_year#41] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] -Aggregate Attributes [1]: [sum#42] -Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] - -(33) CometColumnarExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 7] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] - -(35) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14, sum#43] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#41, c_preferred_cust_flag#12, c_birth_country#32, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))#24] -Results [5]: [c_customer_id#9 AS customer_id#44, c_first_name#10 AS customer_first_name#45, c_last_name#11 AS customer_last_name#46, c_email_address#14 AS customer_email_address#47, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#37 - ss_ext_discount_amt#36)))#24,18,2) AS year_total#48] - -(36) BroadcastExchange -Input [5]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(37) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#44] -Join type: Inner -Join condition: None - -(38) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(39) CometFilter -Input [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] -Condition : (isnotnull(c_customer_sk#49) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#50, 16, true, false, true))) - -(40) CometProject -Input [8]: [c_customer_sk#49, c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56] -Arguments: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62], [c_customer_sk#49, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#50, 16, true, false, true) AS c_customer_id#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#51, 20, true, false, true) AS c_first_name#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#52, 30, true, false, true) AS c_last_name#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#53, 1, true, false, true) AS c_preferred_cust_flag#60, c_birth_country#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#55, 13, true, false, true) AS c_login#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#56, 50, true, false, true) AS c_email_address#62] - -(41) CometColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62] - -(42) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#66), dynamicpruningexpression(ws_sold_date_sk#66 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] - -(44) Filter [codegen id : 8] -Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] -Condition : isnotnull(ws_bill_customer_sk#63) - -(45) BroadcastExchange -Input [4]: [ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -(46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#49] -Right keys [1]: [ws_bill_customer_sk#63] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 10] -Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] -Input [12]: [c_customer_sk#49, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_bill_customer_sk#63, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66] - -(48) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#67, d_year#68] - -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#66] -Right keys [1]: [d_date_sk#67] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 10] -Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, d_year#68] -Input [12]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, ws_sold_date_sk#66, d_date_sk#67, d_year#68] - -(51) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, ws_ext_discount_amt#64, ws_ext_list_price#65, d_year#68] -Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))] -Aggregate Attributes [1]: [sum#69] -Results [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68, sum#70] - -(52) CometColumnarExchange -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68, sum#70] -Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(53) CometColumnarToRow [codegen id : 11] -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68, sum#70] - -(54) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68, sum#70] -Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#54, c_login#61, c_email_address#62, d_year#68] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))#71] -Results [2]: [c_customer_id#57 AS customer_id#72, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#65 - ws_ext_discount_amt#64)))#71,18,2) AS year_total#73] - -(55) Filter [codegen id : 11] -Input [2]: [customer_id#72, year_total#73] -Condition : (isnotnull(year_total#73) AND (year_total#73 > 0.00)) - -(56) BroadcastExchange -Input [2]: [customer_id#72, year_total#73] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(57) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#72] -Join type: Inner -Join condition: None - -(58) Project [codegen id : 16] -Output [8]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, year_total#73] -Input [9]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, customer_id#72, year_total#73] - -(59) CometNativeScan parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(60) CometFilter -Input [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] -Condition : (isnotnull(c_customer_sk#74) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#75, 16, true, false, true))) - -(61) CometProject -Input [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] -Arguments: [c_customer_sk#74, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62], [c_customer_sk#74, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#75, 16, true, false, true) AS c_customer_id#57, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#76, 20, true, false, true) AS c_first_name#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#77, 30, true, false, true) AS c_last_name#59, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#78, 1, true, false, true) AS c_preferred_cust_flag#60, c_birth_country#79, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#80, 13, true, false, true) AS c_login#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#81, 50, true, false, true) AS c_email_address#62] - -(62) CometColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#74, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62] - -(63) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#85), dynamicpruningexpression(ws_sold_date_sk#85 IN dynamicpruning#39)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(64) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] - -(65) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] -Condition : isnotnull(ws_bill_customer_sk#82) - -(66) BroadcastExchange -Input [4]: [ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#74] -Right keys [1]: [ws_bill_customer_sk#82] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 14] -Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] -Input [12]: [c_customer_sk#74, c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, ws_bill_customer_sk#82, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85] - -(69) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#86, d_year#87] - -(70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#85] -Right keys [1]: [d_date_sk#86] -Join type: Inner -Join condition: None - -(71) Project [codegen id : 14] -Output [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, ws_ext_discount_amt#83, ws_ext_list_price#84, d_year#87] -Input [12]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, ws_ext_discount_amt#83, ws_ext_list_price#84, ws_sold_date_sk#85, d_date_sk#86, d_year#87] - -(72) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, ws_ext_discount_amt#83, ws_ext_list_price#84, d_year#87] -Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#84 - ws_ext_discount_amt#83)))] -Aggregate Attributes [1]: [sum#88] -Results [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87, sum#89] - -(73) CometColumnarExchange -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87, sum#89] -Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(74) CometColumnarToRow [codegen id : 15] -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87, sum#89] - -(75) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87, sum#89] -Keys [8]: [c_customer_id#57, c_first_name#58, c_last_name#59, c_preferred_cust_flag#60, c_birth_country#79, c_login#61, c_email_address#62, d_year#87] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#84 - ws_ext_discount_amt#83)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#84 - ws_ext_discount_amt#83)))#71] -Results [2]: [c_customer_id#57 AS customer_id#90, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#84 - ws_ext_discount_amt#83)))#71,18,2) AS year_total#91] - -(76) BroadcastExchange -Input [2]: [customer_id#90, year_total#91] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] - -(77) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#25] -Right keys [1]: [customer_id#90] -Join type: Inner -Join condition: (CASE WHEN (year_total#73 > 0.00) THEN (year_total#91 / year_total#73) ELSE 0E-20 END > CASE WHEN (year_total#26 > 0.00) THEN (year_total#48 / year_total#26) ELSE 0E-20 END) - -(78) Project [codegen id : 16] -Output [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] -Input [10]: [customer_id#25, year_total#26, customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47, year_total#48, year_total#73, customer_id#90, year_total#91] - -(79) TakeOrderedAndProject -Input [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] -Arguments: 100, [customer_id#44 ASC NULLS FIRST, customer_first_name#45 ASC NULLS FIRST, customer_last_name#46 ASC NULLS FIRST, customer_email_address#47 ASC NULLS FIRST], [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometFilter (81) - +- CometNativeScan parquet spark_catalog.default.date_dim (80) - - -(80) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(81) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(82) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#20, d_year#21] - -(83) BroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 -BroadcastExchange (87) -+- * CometColumnarToRow (86) - +- CometFilter (85) - +- CometNativeScan parquet spark_catalog.default.date_dim (84) - - -(84) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#40, d_year#41] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(85) CometFilter -Input [2]: [d_date_sk#40, d_year#41] -Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#40)) - -(86) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#40, d_year#41] - -(87) BroadcastExchange -Input [2]: [d_date_sk#40, d_year#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] - -Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#66 IN dynamicpruning#19 - -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#85 IN dynamicpruning#39 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/simplified.txt deleted file mode 100644 index 00a3e659d4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/simplified.txt +++ /dev/null @@ -1,130 +0,0 @@ -TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - WholeStageCodegen (16) - Project [customer_id,customer_first_name,customer_last_name,customer_email_address] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 - WholeStageCodegen (6) - HashAggregate [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] [sum,sum] - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 - WholeStageCodegen (10) - HashAggregate [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] [sum,sum] - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 - WholeStageCodegen (14) - HashAggregate [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] [sum,sum] - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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] - CometNativeScan 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] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/explain.txt deleted file mode 100644 index 49c1c8e0d7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,470 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometProject (53) - : +- CometBroadcastHashJoin (52) - : :- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (51) - : +- CometFilter (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : +- CometBroadcastExchange (41) - : : +- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) - : +- ReusedExchange (44) - +- CometBroadcastExchange (68) - +- CometHashAggregate (67) - +- CometExchange (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometFilter (55) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (62) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#15) - -(6) CometBroadcastExchange -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(7) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_sk#1], [ss_customer_sk#15], Inner, BuildRight - -(8) CometProject -Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: [d_date_sk#20, d_year#21] - -(12) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Right output [2]: [d_date_sk#20, d_year#21] -Arguments: [ss_sold_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(13) CometProject -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] - -(14) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(15) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(17) CometFilter -Input [2]: [customer_id#23, year_total#24] -Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true))) - -(20) CometProject -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Arguments: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14], [c_customer_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#29, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#31, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#32, 50, true, false, true) AS c_email_address#14] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#36), dynamicpruningexpression(ss_sold_date_sk#36 IN dynamicpruning#37)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Condition : isnotnull(ss_customer_sk#33) - -(23) CometBroadcastExchange -Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] - -(24) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [c_customer_sk#25], [ss_customer_sk#33], Inner, BuildRight - -(25) CometProject -Input [12]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: [d_date_sk#38, d_year#39] - -(29) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Right output [2]: [d_date_sk#38, d_year#39] -Arguments: [ss_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight - -(30) CometProject -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36, d_date_sk#38, d_year#39] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] - -(31) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] - -(32) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] - -(34) CometBroadcastExchange -Input [5]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] -Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#23, year_total#24] -Right output [5]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] -Arguments: [customer_id#23], [customer_id#41], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] -Condition : (isnotnull(c_customer_sk#46) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#47, 16, true, false, true))) - -(38) CometProject -Input [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] -Arguments: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59], [c_customer_sk#46, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#47, 16, true, false, true) AS c_customer_id#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#48, 20, true, false, true) AS c_first_name#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#49, 30, true, false, true) AS c_last_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#50, 1, true, false, true) AS c_preferred_cust_flag#57, c_birth_country#51, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#52, 13, true, false, true) AS c_login#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#53, 50, true, false, true) AS c_email_address#59] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_bill_customer_sk#60) - -(41) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Arguments: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] - -(42) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59] -Right output [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Arguments: [c_customer_sk#46], [ws_bill_customer_sk#60], Inner, BuildRight - -(43) CometProject -Input [12]: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#65, d_year#66] - -(45) CometBroadcastHashJoin -Left output [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Right output [2]: [d_date_sk#65, d_year#66] -Arguments: [ws_sold_date_sk#63], [d_date_sk#65], Inner, BuildRight - -(46) CometProject -Input [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63, d_date_sk#65, d_year#66] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66] - -(47) CometHashAggregate -Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))] - -(48) CometExchange -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, sum#67] -Arguments: hashpartitioning(c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, sum#67] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))] - -(50) CometFilter -Input [2]: [customer_id#68, year_total#69] -Condition : (isnotnull(year_total#69) AND (year_total#69 > 0.00)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#68, year_total#69] -Arguments: [customer_id#68, year_total#69] - -(52) CometBroadcastHashJoin -Left output [7]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] -Right output [2]: [customer_id#68, year_total#69] -Arguments: [customer_id#23], [customer_id#68], Inner, BuildRight - -(53) CometProject -Input [9]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, customer_id#68, year_total#69] -Arguments: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69], [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] -Condition : (isnotnull(c_customer_sk#70) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#71, 16, true, false, true))) - -(56) CometProject -Input [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] -Arguments: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59], [c_customer_sk#70, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#71, 16, true, false, true) AS c_customer_id#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#72, 20, true, false, true) AS c_first_name#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#73, 30, true, false, true) AS c_last_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#74, 1, true, false, true) AS c_preferred_cust_flag#57, c_birth_country#75, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#76, 13, true, false, true) AS c_login#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#77, 50, true, false, true) AS c_email_address#59] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_bill_customer_sk#78) - -(59) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Arguments: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] - -(60) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59] -Right output [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Arguments: [c_customer_sk#70], [ws_bill_customer_sk#78], Inner, BuildRight - -(61) CometProject -Input [12]: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#83, d_year#84] - -(63) CometBroadcastHashJoin -Left output [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Right output [2]: [d_date_sk#83, d_year#84] -Arguments: [ws_sold_date_sk#81], [d_date_sk#83], Inner, BuildRight - -(64) CometProject -Input [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81, d_date_sk#83, d_year#84] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84] - -(65) CometHashAggregate -Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#80 - ws_ext_discount_amt#79)))] - -(66) CometExchange -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, sum#85] -Arguments: hashpartitioning(c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, sum#85] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#80 - ws_ext_discount_amt#79)))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#86, year_total#87] -Arguments: [customer_id#86, year_total#87] - -(69) CometBroadcastHashJoin -Left output [8]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69] -Right output [2]: [customer_id#86, year_total#87] -Arguments: [customer_id#23], [customer_id#86], Inner, (CASE WHEN (year_total#69 > 0.00) THEN (year_total#87 / year_total#69) ELSE 0E-20 END > CASE WHEN (year_total#24 > 0.00) THEN (year_total#45 / year_total#24) ELSE 0E-20 END), BuildRight - -(70) CometProject -Input [10]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69, customer_id#86, year_total#87] -Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44], [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] - -(71) CometTakeOrderedAndProject -Input [4]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#41 ASC NULLS FIRST,customer_first_name#42 ASC NULLS FIRST,customer_last_name#43 ASC NULLS FIRST,customer_email_address#44 ASC NULLS FIRST], output=[customer_id#41,customer_first_name#42,customer_last_name#43,customer_email_address#44]), [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44], 100, 0, [customer_id#41 ASC NULLS FIRST, customer_first_name#42 ASC NULLS FIRST, customer_last_name#43 ASC NULLS FIRST, customer_email_address#44 ASC NULLS FIRST], [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] - -(72) CometColumnarToRow [codegen id : 1] -Input [4]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(75) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#20, d_year#21] - -(76) BroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#36 IN dynamicpruning#37 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) - -(79) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#38, d_year#39] - -(80) BroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#19 - -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#37 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/simplified.txt deleted file mode 100644 index 157d1d587e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - CometProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] #5 - CometHashAggregate [d_year,c_preferred_cust_flag,c_birth_country,c_login,sum] [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_email_address,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt deleted file mode 100644 index 49c1c8e0d7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt +++ /dev/null @@ -1,470 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometProject (53) - : +- CometBroadcastHashJoin (52) - : :- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (51) - : +- CometFilter (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : +- CometBroadcastExchange (41) - : : +- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) - : +- ReusedExchange (44) - +- CometBroadcastExchange (68) - +- CometHashAggregate (67) - +- CometExchange (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometFilter (55) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (62) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Arguments: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#5, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#7, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#8, 50, true, false, true) AS c_email_address#14] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#15) - -(6) CometBroadcastExchange -Input [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(7) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_sk#1], [ss_customer_sk#15], Inner, BuildRight - -(8) CometProject -Input [12]: [c_customer_sk#1, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_customer_sk#15, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: [d_date_sk#20, d_year#21] - -(12) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18] -Right output [2]: [d_date_sk#20, d_year#21] -Arguments: [ss_sold_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(13) CometProject -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, ss_sold_date_sk#18, d_date_sk#20, d_year#21] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] - -(14) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, ss_ext_discount_amt#16, ss_ext_list_price#17, d_year#21] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(15) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14, sum#22] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#21, c_preferred_cust_flag#12, c_birth_country#6, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))] - -(17) CometFilter -Input [2]: [customer_id#23, year_total#24] -Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true))) - -(20) CometProject -Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] -Arguments: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14], [c_customer_sk#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#26, 16, true, false, true) AS c_customer_id#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#27, 20, true, false, true) AS c_first_name#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#28, 30, true, false, true) AS c_last_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#29, 1, true, false, true) AS c_preferred_cust_flag#12, c_birth_country#30, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#31, 13, true, false, true) AS c_login#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#32, 50, true, false, true) AS c_email_address#14] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#36), dynamicpruningexpression(ss_sold_date_sk#36 IN dynamicpruning#37)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Condition : isnotnull(ss_customer_sk#33) - -(23) CometBroadcastExchange -Input [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] - -(24) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Right output [4]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [c_customer_sk#25], [ss_customer_sk#33], Inner, BuildRight - -(25) CometProject -Input [12]: [c_customer_sk#25, c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: [d_date_sk#38, d_year#39] - -(29) CometBroadcastHashJoin -Left output [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36] -Right output [2]: [d_date_sk#38, d_year#39] -Arguments: [ss_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight - -(30) CometProject -Input [12]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, ss_sold_date_sk#36, d_date_sk#38, d_year#39] -Arguments: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39], [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] - -(31) CometHashAggregate -Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, ss_ext_discount_amt#34, ss_ext_list_price#35, d_year#39] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] - -(32) CometExchange -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] -Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14, sum#40] -Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#39, c_preferred_cust_flag#12, c_birth_country#30, c_login#13, c_email_address#14] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#35 - ss_ext_discount_amt#34)))] - -(34) CometBroadcastExchange -Input [5]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] -Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#23, year_total#24] -Right output [5]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] -Arguments: [customer_id#23], [customer_id#41], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] -Condition : (isnotnull(c_customer_sk#46) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#47, 16, true, false, true))) - -(38) CometProject -Input [8]: [c_customer_sk#46, c_customer_id#47, c_first_name#48, c_last_name#49, c_preferred_cust_flag#50, c_birth_country#51, c_login#52, c_email_address#53] -Arguments: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59], [c_customer_sk#46, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#47, 16, true, false, true) AS c_customer_id#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#48, 20, true, false, true) AS c_first_name#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#49, 30, true, false, true) AS c_last_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#50, 1, true, false, true) AS c_preferred_cust_flag#57, c_birth_country#51, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#52, 13, true, false, true) AS c_login#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#53, 50, true, false, true) AS c_email_address#59] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_bill_customer_sk#60) - -(41) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Arguments: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] - -(42) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59] -Right output [4]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Arguments: [c_customer_sk#46], [ws_bill_customer_sk#60], Inner, BuildRight - -(43) CometProject -Input [12]: [c_customer_sk#46, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#65, d_year#66] - -(45) CometBroadcastHashJoin -Left output [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63] -Right output [2]: [d_date_sk#65, d_year#66] -Arguments: [ws_sold_date_sk#63], [d_date_sk#65], Inner, BuildRight - -(46) CometProject -Input [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, ws_sold_date_sk#63, d_date_sk#65, d_year#66] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66] - -(47) CometHashAggregate -Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#66] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))] - -(48) CometExchange -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, sum#67] -Arguments: hashpartitioning(c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66, sum#67] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#51, c_login#58, c_email_address#59, d_year#66] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))] - -(50) CometFilter -Input [2]: [customer_id#68, year_total#69] -Condition : (isnotnull(year_total#69) AND (year_total#69 > 0.00)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#68, year_total#69] -Arguments: [customer_id#68, year_total#69] - -(52) CometBroadcastHashJoin -Left output [7]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45] -Right output [2]: [customer_id#68, year_total#69] -Arguments: [customer_id#23], [customer_id#68], Inner, BuildRight - -(53) CometProject -Input [9]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, customer_id#68, year_total#69] -Arguments: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69], [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] -Condition : (isnotnull(c_customer_sk#70) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#71, 16, true, false, true))) - -(56) CometProject -Input [8]: [c_customer_sk#70, c_customer_id#71, c_first_name#72, c_last_name#73, c_preferred_cust_flag#74, c_birth_country#75, c_login#76, c_email_address#77] -Arguments: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59], [c_customer_sk#70, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#71, 16, true, false, true) AS c_customer_id#54, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#72, 20, true, false, true) AS c_first_name#55, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#73, 30, true, false, true) AS c_last_name#56, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#74, 1, true, false, true) AS c_preferred_cust_flag#57, c_birth_country#75, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_login#76, 13, true, false, true) AS c_login#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_email_address#77, 50, true, false, true) AS c_email_address#59] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_bill_customer_sk#78) - -(59) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Arguments: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] - -(60) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59] -Right output [4]: [ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Arguments: [c_customer_sk#70], [ws_bill_customer_sk#78], Inner, BuildRight - -(61) CometProject -Input [12]: [c_customer_sk#70, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_bill_customer_sk#78, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#83, d_year#84] - -(63) CometBroadcastHashJoin -Left output [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Right output [2]: [d_date_sk#83, d_year#84] -Arguments: [ws_sold_date_sk#81], [d_date_sk#83], Inner, BuildRight - -(64) CometProject -Input [12]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, ws_sold_date_sk#81, d_date_sk#83, d_year#84] -Arguments: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84], [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84] - -(65) CometHashAggregate -Input [10]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, ws_ext_discount_amt#79, ws_ext_list_price#80, d_year#84] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#80 - ws_ext_discount_amt#79)))] - -(66) CometExchange -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, sum#85] -Arguments: hashpartitioning(c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [9]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84, sum#85] -Keys [8]: [c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#75, c_login#58, c_email_address#59, d_year#84] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#80 - ws_ext_discount_amt#79)))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#86, year_total#87] -Arguments: [customer_id#86, year_total#87] - -(69) CometBroadcastHashJoin -Left output [8]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69] -Right output [2]: [customer_id#86, year_total#87] -Arguments: [customer_id#23], [customer_id#86], Inner, (CASE WHEN (year_total#69 > 0.00) THEN (year_total#87 / year_total#69) ELSE 0E-20 END > CASE WHEN (year_total#24 > 0.00) THEN (year_total#45 / year_total#24) ELSE 0E-20 END), BuildRight - -(70) CometProject -Input [10]: [customer_id#23, year_total#24, customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44, year_total#45, year_total#69, customer_id#86, year_total#87] -Arguments: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44], [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] - -(71) CometTakeOrderedAndProject -Input [4]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#41 ASC NULLS FIRST,customer_first_name#42 ASC NULLS FIRST,customer_last_name#43 ASC NULLS FIRST,customer_email_address#44 ASC NULLS FIRST], output=[customer_id#41,customer_first_name#42,customer_last_name#43,customer_email_address#44]), [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44], 100, 0, [customer_id#41 ASC NULLS FIRST, customer_first_name#42 ASC NULLS FIRST, customer_last_name#43 ASC NULLS FIRST, customer_email_address#44 ASC NULLS FIRST], [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] - -(72) CometColumnarToRow [codegen id : 1] -Input [4]: [customer_id#41, customer_first_name#42, customer_last_name#43, customer_email_address#44] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#20, d_year#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [2]: [d_date_sk#20, d_year#21] -Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#20)) - -(75) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#20, d_year#21] - -(76) BroadcastExchange -Input [2]: [d_date_sk#20, d_year#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#36 IN dynamicpruning#37 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#38, d_year#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) - -(79) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#38, d_year#39] - -(80) BroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#19 - -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#37 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/extended.txt deleted file mode 100644 index 476c7be954..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/extended.txt +++ /dev/null @@ -1,90 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 157d1d587e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - CometProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total] #5 - CometHashAggregate [d_year,c_preferred_cust_flag,c_birth_country,c_login,sum] [customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,c_customer_id,c_first_name,c_last_name,c_email_address,sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #6 - CometHashAggregate [ss_ext_list_price,ss_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt)))] - CometExchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14 - CometHashAggregate [ws_ext_list_price,ws_ext_discount_amt] [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] - 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 [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,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] - CometProject [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] - 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 [native_iceberg_compat] 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 [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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/explain.txt deleted file mode 100644 index 2ebf9db593..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/explain.txt +++ /dev/null @@ -1,163 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.web_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.item (4) - +- ReusedExchange (11) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(7) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(8) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(11) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] - -(14) HashAggregate [codegen id : 3] -Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(15) CometColumnarExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(17) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#17] -Results [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#17,17,2) AS _w0#19] - -(18) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(22) Project [codegen id : 6] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] - -(23) TakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometNativeScan parquet spark_catalog.default.date_dim (24) - - -(24) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#22] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/simplified.txt deleted file mode 100644 index c129b42cdb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] - Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/explain.txt deleted file mode 100644 index 410aecff1b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ws_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/simplified.txt deleted file mode 100644 index 58ac81ba0d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt deleted file mode 100644 index 410aecff1b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ws_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ws_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/extended.txt deleted file mode 100644 index 3f41c97ff5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 58ac81ba0d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/explain.txt deleted file mode 100644 index 908ca52087..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/explain.txt +++ /dev/null @@ -1,769 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (87) -+- * BroadcastHashJoin Inner BuildRight (86) - :- * Filter (68) - : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) - : +- CometColumnarExchange (65) - : +- * HashAggregate (64) - : +- * Project (63) - : +- * BroadcastHashJoin Inner BuildRight (62) - : :- * Project (60) - : : +- * BroadcastHashJoin Inner BuildRight (59) - : : :- * BroadcastHashJoin LeftSemi BuildRight (52) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- BroadcastExchange (51) - : : : +- * Project (50) - : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : :- * CometColumnarToRow (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : : +- BroadcastExchange (48) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : :- * CometColumnarToRow (36) - : : : : +- CometHashAggregate (35) - : : : : +- CometColumnarExchange (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : +- BroadcastExchange (27) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : :- * CometColumnarToRow (12) - : : : : : : +- CometFilter (11) - : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) - : : : : : +- BroadcastExchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * CometColumnarToRow (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) - : : : : : +- ReusedExchange (22) - : : : : +- ReusedExchange (30) - : : : +- BroadcastExchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (42) - : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : :- * Filter (39) - : : : : : +- * ColumnarToRow (38) - : : : : : +- Scan parquet spark_catalog.default.web_sales (37) - : : : : +- ReusedExchange (40) - : : : +- ReusedExchange (43) - : : +- BroadcastExchange (58) - : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : :- * CometColumnarToRow (55) - : : : +- CometFilter (54) - : : : +- CometNativeScan parquet spark_catalog.default.item (53) - : : +- ReusedExchange (56) - : +- ReusedExchange (61) - +- BroadcastExchange (85) - +- * Filter (84) - +- * HashAggregate (83) - +- * CometColumnarToRow (82) - +- CometColumnarExchange (81) - +- * HashAggregate (80) - +- * Project (79) - +- * BroadcastHashJoin Inner BuildRight (78) - :- * Project (76) - : +- * BroadcastHashJoin Inner BuildRight (75) - : :- * BroadcastHashJoin LeftSemi BuildRight (73) - : : :- * Filter (71) - : : : +- * ColumnarToRow (70) - : : : +- Scan parquet spark_catalog.default.store_sales (69) - : : +- ReusedExchange (72) - : +- ReusedExchange (74) - +- ReusedExchange (77) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(6) CometColumnarToRow [codegen id : 11] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(10) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(11) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(12) CometColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(16) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) - -(18) CometColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(19) BroadcastExchange -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#19] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(22) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#23] - -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#23] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#23] - -(25) BroadcastExchange -Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] -Join type: LeftSemi -Join condition: None - -(27) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None - -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(30) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#24] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#25, i_class_id#15 AS class_id#26, i_category_id#16 AS category_id#27] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#24] - -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#25, class_id#26, category_id#27] - -(34) CometColumnarExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(35) CometHashAggregate -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] - -(36) CometColumnarToRow [codegen id : 10] -Input [3]: [brand_id#25, class_id#26, category_id#27] - -(37) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] - -(39) Filter [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Condition : isnotnull(ws_item_sk#28) - -(40) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#28] -Right keys [1]: [i_item_sk#30] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] -Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(43) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#34] - -(44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#29] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 9] -Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] - -(46) BroadcastExchange -Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] - -(47) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] -Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] -Join type: LeftSemi -Join condition: None - -(48) BroadcastExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] - -(49) BroadcastHashJoin [codegen id : 11] -Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#25, class_id#26, category_id#27] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#35] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] - -(51) BroadcastExchange -Input [1]: [ss_item_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(52) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(53) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(54) CometFilter -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Condition : (((isnotnull(i_item_sk#36) AND isnotnull(i_brand_id#37)) AND isnotnull(i_class_id#38)) AND isnotnull(i_category_id#39)) - -(55) CometColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(56) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#35] - -(57) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#36] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(58) BroadcastExchange -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(59) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#36] -Join type: Inner -Join condition: None - -(60) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(61) ReusedExchange [Reuses operator id: 112] -Output [1]: [d_date_sk#40] - -(62) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] - -(64) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] -Results [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(65) CometColumnarExchange -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Arguments: hashpartitioning(i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(66) CometColumnarToRow [codegen id : 52] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(67) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] -Results [6]: [store AS channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51] - -(68) Filter [codegen id : 52] -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51] -Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(69) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#57), dynamicpruningexpression(ss_sold_date_sk#57 IN dynamicpruning#58)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(70) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] - -(71) Filter [codegen id : 50] -Input [4]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57] -Condition : isnotnull(ss_item_sk#54) - -(72) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#35] - -(73) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(74) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] - -(75) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [i_item_sk#59] -Join type: Inner -Join condition: None - -(76) Project [codegen id : 50] -Output [6]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62] -Input [8]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] - -(77) ReusedExchange [Reuses operator id: 126] -Output [1]: [d_date_sk#63] - -(78) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#57] -Right keys [1]: [d_date_sk#63] -Join type: Inner -Join condition: None - -(79) Project [codegen id : 50] -Output [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] -Input [7]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62, d_date_sk#63] - -(80) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)] -Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] -Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] - -(81) CometColumnarExchange -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] -Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(82) CometColumnarToRow [codegen id : 51] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] - -(83) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70, count(1)#71] -Results [6]: [store AS channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70 AS sales#73, count(1)#71 AS number_sales#74] - -(84) Filter [codegen id : 51] -Input [6]: [channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] -Condition : (isnotnull(sales#73) AND (cast(sales#73 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(85) BroadcastExchange -Input [6]: [channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] - -(86) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Right keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Join type: Inner -Join condition: None - -(87) TakeOrderedAndProject -Input [12]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51, channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] -Arguments: 100, [i_brand_id#37 ASC NULLS FIRST, i_class_id#38 ASC NULLS FIRST, i_category_id#39 ASC NULLS FIRST], [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51, channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* HashAggregate (107) -+- * CometColumnarToRow (106) - +- CometColumnarExchange (105) - +- * HashAggregate (104) - +- Union (103) - :- * Project (92) - : +- * BroadcastHashJoin Inner BuildRight (91) - : :- * ColumnarToRow (89) - : : +- Scan parquet spark_catalog.default.store_sales (88) - : +- ReusedExchange (90) - :- * Project (97) - : +- * BroadcastHashJoin Inner BuildRight (96) - : :- * ColumnarToRow (94) - : : +- Scan parquet spark_catalog.default.catalog_sales (93) - : +- ReusedExchange (95) - +- * Project (102) - +- * BroadcastHashJoin Inner BuildRight (101) - :- * ColumnarToRow (99) - : +- Scan parquet spark_catalog.default.web_sales (98) - +- ReusedExchange (100) - - -(88) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#77), dynamicpruningexpression(ss_sold_date_sk#77 IN dynamicpruning#12)] -ReadSchema: struct - -(89) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77] - -(90) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#78] - -(91) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#77] -Right keys [1]: [d_date_sk#78] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 2] -Output [2]: [ss_quantity#75 AS quantity#79, ss_list_price#76 AS list_price#80] -Input [4]: [ss_quantity#75, ss_list_price#76, ss_sold_date_sk#77, d_date_sk#78] - -(93) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#12)] -ReadSchema: struct - -(94) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] - -(95) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#84] - -(96) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#83] -Right keys [1]: [d_date_sk#84] -Join type: Inner -Join condition: None - -(97) Project [codegen id : 4] -Output [2]: [cs_quantity#81 AS quantity#85, cs_list_price#82 AS list_price#86] -Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83, d_date_sk#84] - -(98) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#12)] -ReadSchema: struct - -(99) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89] - -(100) ReusedExchange [Reuses operator id: 121] -Output [1]: [d_date_sk#90] - -(101) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#89] -Right keys [1]: [d_date_sk#90] -Join type: Inner -Join condition: None - -(102) Project [codegen id : 6] -Output [2]: [ws_quantity#87 AS quantity#91, ws_list_price#88 AS list_price#92] -Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89, d_date_sk#90] - -(103) Union - -(104) HashAggregate [codegen id : 7] -Input [2]: [quantity#79, list_price#80] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] -Aggregate Attributes [2]: [sum#93, count#94] -Results [2]: [sum#95, count#96] - -(105) CometColumnarExchange -Input [2]: [sum#95, count#96] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(106) CometColumnarToRow [codegen id : 8] -Input [2]: [sum#95, count#96] - -(107) HashAggregate [codegen id : 8] -Input [2]: [sum#95, count#96] -Keys: [] -Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] -Aggregate Attributes [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97] -Results [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97 AS average_sales#98] - -Subquery:2 Hosting operator id = 88 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 93 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 98 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (112) -+- * CometColumnarToRow (111) - +- CometProject (110) - +- CometFilter (109) - +- CometNativeScan parquet spark_catalog.default.date_dim (108) - - -(108) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#40, d_week_seq#99] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(109) CometFilter -Input [2]: [d_date_sk#40, d_week_seq#99] -Condition : ((isnotnull(d_week_seq#99) AND (d_week_seq#99 = Subquery scalar-subquery#100, [id=#101])) AND isnotnull(d_date_sk#40)) - -(110) CometProject -Input [2]: [d_date_sk#40, d_week_seq#99] -Arguments: [d_date_sk#40], [d_date_sk#40] - -(111) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#40] - -(112) BroadcastExchange -Input [1]: [d_date_sk#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:6 Hosting operator id = 109 Hosting Expression = Subquery scalar-subquery#100, [id=#101] -* CometColumnarToRow (116) -+- CometProject (115) - +- CometFilter (114) - +- CometNativeScan parquet spark_catalog.default.date_dim (113) - - -(113) CometNativeScan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] -ReadSchema: struct - -(114) CometFilter -Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] -Condition : (((((isnotnull(d_year#103) AND isnotnull(d_moy#104)) AND isnotnull(d_dom#105)) AND (d_year#103 = 1999)) AND (d_moy#104 = 12)) AND (d_dom#105 = 16)) - -(115) CometProject -Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] -Arguments: [d_week_seq#102], [d_week_seq#102] - -(116) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#102] - -Subquery:7 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometNativeScan parquet spark_catalog.default.date_dim (117) - - -(117) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#103] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#24, d_year#103] -Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 <= 2000)) AND isnotnull(d_date_sk#24)) - -(119) CometProject -Input [2]: [d_date_sk#24, d_year#103] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#24] - -(121) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:8 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 - -Subquery:10 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:11 Hosting operator id = 69 Hosting Expression = ss_sold_date_sk#57 IN dynamicpruning#58 -BroadcastExchange (126) -+- * CometColumnarToRow (125) - +- CometProject (124) - +- CometFilter (123) - +- CometNativeScan parquet spark_catalog.default.date_dim (122) - - -(122) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#63, d_week_seq#106] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(123) CometFilter -Input [2]: [d_date_sk#63, d_week_seq#106] -Condition : ((isnotnull(d_week_seq#106) AND (d_week_seq#106 = Subquery scalar-subquery#107, [id=#108])) AND isnotnull(d_date_sk#63)) - -(124) CometProject -Input [2]: [d_date_sk#63, d_week_seq#106] -Arguments: [d_date_sk#63], [d_date_sk#63] - -(125) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#63] - -(126) BroadcastExchange -Input [1]: [d_date_sk#63] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] - -Subquery:12 Hosting operator id = 123 Hosting Expression = Subquery scalar-subquery#107, [id=#108] -* CometColumnarToRow (130) -+- CometProject (129) - +- CometFilter (128) - +- CometNativeScan parquet spark_catalog.default.date_dim (127) - - -(127) CometNativeScan parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] -ReadSchema: struct - -(128) CometFilter -Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] -Condition : (((((isnotnull(d_year#103) AND isnotnull(d_moy#104)) AND isnotnull(d_dom#105)) AND (d_year#103 = 1998)) AND (d_moy#104 = 12)) AND (d_dom#105 = 16)) - -(129) CometProject -Input [4]: [d_week_seq#102, d_year#103, d_moy#104, d_dom#105] -Arguments: [d_week_seq#102], [d_week_seq#102] - -(130) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#102] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/simplified.txt deleted file mode 100644 index b0eae963c3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/simplified.txt +++ /dev/null @@ -1,206 +0,0 @@ -TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (52) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - Filter [sales] - Subquery #4 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #12 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #6 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #5 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (51) - Filter [sales] - ReusedSubquery [average_sales] #4 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (50) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #15 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - ReusedExchange [ss_item_sk] #3 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #15 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/explain.txt deleted file mode 100644 index 33224acffb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,743 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (85) -+- CometTakeOrderedAndProject (84) - +- CometBroadcastHashJoin (83) - :- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometBroadcastHashJoin (47) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (46) - : : : +- CometProject (45) - : : : +- CometBroadcastHashJoin (44) - : : : :- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (43) - : : : +- CometBroadcastHashJoin (42) - : : : :- CometHashAggregate (32) - : : : : +- CometExchange (31) - : : : : +- CometHashAggregate (30) - : : : : +- CometProject (29) - : : : : +- CometBroadcastHashJoin (28) - : : : : :- CometProject (26) - : : : : : +- CometBroadcastHashJoin (25) - : : : : : :- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : +- CometBroadcastExchange (24) - : : : : : +- CometBroadcastHashJoin (23) - : : : : : :- CometFilter (8) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : +- CometBroadcastExchange (22) - : : : : : +- CometProject (21) - : : : : : +- CometBroadcastHashJoin (20) - : : : : : :- CometProject (15) - : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : :- CometFilter (10) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : +- CometBroadcastExchange (13) - : : : : : : +- CometFilter (12) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : +- CometBroadcastExchange (19) - : : : : : +- CometProject (18) - : : : : : +- CometFilter (17) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : +- ReusedExchange (27) - : : : +- CometBroadcastExchange (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (34) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometFilter (49) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : +- ReusedExchange (50) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometBroadcastExchange (82) - +- CometFilter (81) - +- CometHashAggregate (80) - +- CometExchange (79) - +- CometHashAggregate (78) - +- CometProject (77) - +- CometBroadcastHashJoin (76) - :- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometBroadcastHashJoin (68) - : : :- CometFilter (66) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (65) - : : +- ReusedExchange (67) - : +- ReusedExchange (69) - +- CometBroadcastExchange (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1998)) AND (d_year#25 <= 2000)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42)) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Condition : (isnotnull(sales#51) AND (cast(sales#51 as decimal(32,6)) > cast(Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Condition : isnotnull(ss_item_sk#55) - -(67) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(68) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#55], [ss_item_sk#38], LeftSemi, BuildRight - -(69) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] - -(70) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -Arguments: [ss_item_sk#55], [i_item_sk#60], Inner, BuildRight - -(71) CometProject -Input [8]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -Arguments: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63], [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#64, d_week_seq#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#64, d_week_seq#65] -Condition : ((isnotnull(d_week_seq#65) AND (d_week_seq#65 = ReusedSubquery Subquery scalar-subquery#66, [id=#67])) AND isnotnull(d_date_sk#64)) - -(74) CometProject -Input [2]: [d_date_sk#64, d_week_seq#65] -Arguments: [d_date_sk#64], [d_date_sk#64] - -(75) CometBroadcastExchange -Input [1]: [d_date_sk#64] -Arguments: [d_date_sk#64] - -(76) CometBroadcastHashJoin -Left output [6]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63] -Right output [1]: [d_date_sk#64] -Arguments: [ss_sold_date_sk#58], [d_date_sk#64], Inner, BuildRight - -(77) CometProject -Input [7]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] -Arguments: [ss_quantity#56, ss_list_price#57, i_brand_id#61, i_class_id#62, i_category_id#63], [ss_quantity#56, ss_list_price#57, i_brand_id#61, i_class_id#62, i_category_id#63] - -(78) CometHashAggregate -Input [5]: [ss_quantity#56, ss_list_price#57, i_brand_id#61, i_class_id#62, i_category_id#63] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [partial_sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), partial_count(1)] - -(79) CometExchange -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] -Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(80) CometHashAggregate -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), count(1)] - -(81) CometFilter -Input [6]: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] -Condition : (isnotnull(sales#72) AND (cast(sales#72 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(82) CometBroadcastExchange -Input [6]: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] -Arguments: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] - -(83) CometBroadcastHashJoin -Left output [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Right output [6]: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] -Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#61, i_class_id#62, i_category_id#63], Inner, BuildRight - -(84) CometTakeOrderedAndProject -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#71,i_brand_id#61,i_class_id#62,i_category_id#63,sales#72,number_sales#73]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73], 100, 0, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] - -(85) CometColumnarToRow [codegen id : 1] -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* CometColumnarToRow (102) -+- CometHashAggregate (101) - +- CometExchange (100) - +- CometHashAggregate (99) - +- CometUnion (98) - :- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (86) - : +- ReusedExchange (87) - :- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (90) - : +- ReusedExchange (91) - +- CometProject (97) - +- CometBroadcastHashJoin (96) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (94) - +- ReusedExchange (95) - - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#74, ss_list_price#75, ss_sold_date_sk#76] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#76), dynamicpruningexpression(ss_sold_date_sk#76 IN dynamicpruning#77)] -ReadSchema: struct - -(87) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#78] - -(88) CometBroadcastHashJoin -Left output [3]: [ss_quantity#74, ss_list_price#75, ss_sold_date_sk#76] -Right output [1]: [d_date_sk#78] -Arguments: [ss_sold_date_sk#76], [d_date_sk#78], Inner, BuildRight - -(89) CometProject -Input [4]: [ss_quantity#74, ss_list_price#75, ss_sold_date_sk#76, d_date_sk#78] -Arguments: [quantity#79, list_price#80], [ss_quantity#74 AS quantity#79, ss_list_price#75 AS list_price#80] - -(90) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] -ReadSchema: struct - -(91) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#85] - -(92) CometBroadcastHashJoin -Left output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] -Right output [1]: [d_date_sk#85] -Arguments: [cs_sold_date_sk#83], [d_date_sk#85], Inner, BuildRight - -(93) CometProject -Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83, d_date_sk#85] -Arguments: [quantity#86, list_price#87], [cs_quantity#81 AS quantity#86, cs_list_price#82 AS list_price#87] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#90), dynamicpruningexpression(ws_sold_date_sk#90 IN dynamicpruning#91)] -ReadSchema: struct - -(95) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#92] - -(96) CometBroadcastHashJoin -Left output [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] -Right output [1]: [d_date_sk#92] -Arguments: [ws_sold_date_sk#90], [d_date_sk#92], Inner, BuildRight - -(97) CometProject -Input [4]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90, d_date_sk#92] -Arguments: [quantity#93, list_price#94], [ws_quantity#88 AS quantity#93, ws_list_price#89 AS list_price#94] - -(98) CometUnion -Child 0 Input [2]: [quantity#79, list_price#80] -Child 1 Input [2]: [quantity#86, list_price#87] -Child 2 Input [2]: [quantity#93, list_price#94] - -(99) CometHashAggregate -Input [2]: [quantity#79, list_price#80] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] - -(100) CometExchange -Input [2]: [sum#95, count#96] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(101) CometHashAggregate -Input [2]: [sum#95, count#96] -Keys: [] -Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] - -(102) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#97] - -Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#90 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (107) -+- * CometColumnarToRow (106) - +- CometProject (105) - +- CometFilter (104) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) - - -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(104) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(105) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(106) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(107) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) -+- CometProject (110) - +- CometFilter (109) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) - - -(108) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] -ReadSchema: struct - -(109) CometFilter -Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Condition : (((((isnotnull(d_year#99) AND isnotnull(d_moy#100)) AND isnotnull(d_dom#101)) AND (d_year#99 = 1999)) AND (d_moy#100 = 12)) AND (d_dom#101 = 16)) - -(110) CometProject -Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Arguments: [d_week_seq#98], [d_week_seq#98] - -(111) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#98] - -Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (116) -+- * CometColumnarToRow (115) - +- CometProject (114) - +- CometFilter (113) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) - - -(112) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#99] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(113) CometFilter -Input [2]: [d_date_sk#26, d_year#99] -Condition : (((isnotnull(d_year#99) AND (d_year#99 >= 1998)) AND (d_year#99 <= 2000)) AND isnotnull(d_date_sk#26)) - -(114) CometProject -Input [2]: [d_date_sk#26, d_year#99] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(115) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(116) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:8 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:10 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] - -Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] - -Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#64, d_week_seq#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#64, d_week_seq#65] -Condition : ((isnotnull(d_week_seq#65) AND (d_week_seq#65 = Subquery scalar-subquery#66, [id=#67])) AND isnotnull(d_date_sk#64)) - -(119) CometProject -Input [2]: [d_date_sk#64, d_week_seq#65] -Arguments: [d_date_sk#64], [d_date_sk#64] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#64] - -(121) BroadcastExchange -Input [1]: [d_date_sk#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#66, [id=#67] -* CometColumnarToRow (125) -+- CometProject (124) - +- CometFilter (123) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) - - -(122) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] -ReadSchema: struct - -(123) CometFilter -Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Condition : (((((isnotnull(d_year#99) AND isnotnull(d_moy#100)) AND isnotnull(d_dom#101)) AND (d_year#99 = 1998)) AND (d_moy#100 = 12)) AND (d_dom#101 = 16)) - -(124) CometProject -Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Arguments: [d_week_seq#98], [d_week_seq#98] - -(125) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#98] - -Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/simplified.txt deleted file mode 100644 index fb9abae378..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,153 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #14 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #1 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [ss_item_sk] #3 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #4 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #5 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #10 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #11 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #3 - CometBroadcastExchange [d_date_sk] #13 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #4 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - ReusedExchange [ss_item_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastExchange [d_date_sk] #18 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt deleted file mode 100644 index 33224acffb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ /dev/null @@ -1,743 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (85) -+- CometTakeOrderedAndProject (84) - +- CometBroadcastHashJoin (83) - :- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (54) - : : +- CometBroadcastHashJoin (53) - : : :- CometBroadcastHashJoin (47) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (46) - : : : +- CometProject (45) - : : : +- CometBroadcastHashJoin (44) - : : : :- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (43) - : : : +- CometBroadcastHashJoin (42) - : : : :- CometHashAggregate (32) - : : : : +- CometExchange (31) - : : : : +- CometHashAggregate (30) - : : : : +- CometProject (29) - : : : : +- CometBroadcastHashJoin (28) - : : : : :- CometProject (26) - : : : : : +- CometBroadcastHashJoin (25) - : : : : : :- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : +- CometBroadcastExchange (24) - : : : : : +- CometBroadcastHashJoin (23) - : : : : : :- CometFilter (8) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : +- CometBroadcastExchange (22) - : : : : : +- CometProject (21) - : : : : : +- CometBroadcastHashJoin (20) - : : : : : :- CometProject (15) - : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : :- CometFilter (10) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : +- CometBroadcastExchange (13) - : : : : : : +- CometFilter (12) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : +- CometBroadcastExchange (19) - : : : : : +- CometProject (18) - : : : : : +- CometFilter (17) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : +- ReusedExchange (27) - : : : +- CometBroadcastExchange (41) - : : : +- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (37) - : : : : +- CometBroadcastHashJoin (36) - : : : : :- CometFilter (34) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- CometBroadcastExchange (52) - : : +- CometBroadcastHashJoin (51) - : : :- CometFilter (49) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : +- ReusedExchange (50) - : +- CometBroadcastExchange (58) - : +- CometProject (57) - : +- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometBroadcastExchange (82) - +- CometFilter (81) - +- CometHashAggregate (80) - +- CometExchange (79) - +- CometHashAggregate (78) - +- CometProject (77) - +- CometBroadcastHashJoin (76) - :- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometBroadcastHashJoin (68) - : : :- CometFilter (66) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (65) - : : +- ReusedExchange (67) - : +- ReusedExchange (69) - +- CometBroadcastExchange (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1998)) AND (d_year#25 <= 2000)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : (((isnotnull(i_item_sk#39) AND isnotnull(i_brand_id#40)) AND isnotnull(i_class_id#41)) AND isnotnull(i_category_id#42)) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = ReusedSubquery Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#47, isEmpty#48, count#49] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Condition : (isnotnull(sales#51) AND (cast(sales#51 as decimal(32,6)) > cast(Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Condition : isnotnull(ss_item_sk#55) - -(67) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(68) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#55], [ss_item_sk#38], LeftSemi, BuildRight - -(69) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] - -(70) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58] -Right output [4]: [i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -Arguments: [ss_item_sk#55], [i_item_sk#60], Inner, BuildRight - -(71) CometProject -Input [8]: [ss_item_sk#55, ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_item_sk#60, i_brand_id#61, i_class_id#62, i_category_id#63] -Arguments: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63], [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#64, d_week_seq#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#64, d_week_seq#65] -Condition : ((isnotnull(d_week_seq#65) AND (d_week_seq#65 = ReusedSubquery Subquery scalar-subquery#66, [id=#67])) AND isnotnull(d_date_sk#64)) - -(74) CometProject -Input [2]: [d_date_sk#64, d_week_seq#65] -Arguments: [d_date_sk#64], [d_date_sk#64] - -(75) CometBroadcastExchange -Input [1]: [d_date_sk#64] -Arguments: [d_date_sk#64] - -(76) CometBroadcastHashJoin -Left output [6]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63] -Right output [1]: [d_date_sk#64] -Arguments: [ss_sold_date_sk#58], [d_date_sk#64], Inner, BuildRight - -(77) CometProject -Input [7]: [ss_quantity#56, ss_list_price#57, ss_sold_date_sk#58, i_brand_id#61, i_class_id#62, i_category_id#63, d_date_sk#64] -Arguments: [ss_quantity#56, ss_list_price#57, i_brand_id#61, i_class_id#62, i_category_id#63], [ss_quantity#56, ss_list_price#57, i_brand_id#61, i_class_id#62, i_category_id#63] - -(78) CometHashAggregate -Input [5]: [ss_quantity#56, ss_list_price#57, i_brand_id#61, i_class_id#62, i_category_id#63] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [partial_sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), partial_count(1)] - -(79) CometExchange -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] -Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(80) CometHashAggregate -Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#68, isEmpty#69, count#70] -Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63] -Functions [2]: [sum((cast(ss_quantity#56 as decimal(10,0)) * ss_list_price#57)), count(1)] - -(81) CometFilter -Input [6]: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] -Condition : (isnotnull(sales#72) AND (cast(sales#72 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) - -(82) CometBroadcastExchange -Input [6]: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] -Arguments: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] - -(83) CometBroadcastHashJoin -Left output [6]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52] -Right output [6]: [channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] -Arguments: [i_brand_id#40, i_class_id#41, i_category_id#42], [i_brand_id#61, i_class_id#62, i_category_id#63], Inner, BuildRight - -(84) CometTakeOrderedAndProject -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#50,i_brand_id#40,i_class_id#41,i_category_id#42,sales#51,number_sales#52,channel#71,i_brand_id#61,i_class_id#62,i_category_id#63,sales#72,number_sales#73]), [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73], 100, 0, [i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] - -(85) CometColumnarToRow [codegen id : 1] -Input [12]: [channel#50, i_brand_id#40, i_class_id#41, i_category_id#42, sales#51, number_sales#52, channel#71, i_brand_id#61, i_class_id#62, i_category_id#63, sales#72, number_sales#73] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#53, [id=#54] -* CometColumnarToRow (102) -+- CometHashAggregate (101) - +- CometExchange (100) - +- CometHashAggregate (99) - +- CometUnion (98) - :- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (86) - : +- ReusedExchange (87) - :- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (90) - : +- ReusedExchange (91) - +- CometProject (97) - +- CometBroadcastHashJoin (96) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (94) - +- ReusedExchange (95) - - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#74, ss_list_price#75, ss_sold_date_sk#76] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#76), dynamicpruningexpression(ss_sold_date_sk#76 IN dynamicpruning#77)] -ReadSchema: struct - -(87) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#78] - -(88) CometBroadcastHashJoin -Left output [3]: [ss_quantity#74, ss_list_price#75, ss_sold_date_sk#76] -Right output [1]: [d_date_sk#78] -Arguments: [ss_sold_date_sk#76], [d_date_sk#78], Inner, BuildRight - -(89) CometProject -Input [4]: [ss_quantity#74, ss_list_price#75, ss_sold_date_sk#76, d_date_sk#78] -Arguments: [quantity#79, list_price#80], [ss_quantity#74 AS quantity#79, ss_list_price#75 AS list_price#80] - -(90) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] -ReadSchema: struct - -(91) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#85] - -(92) CometBroadcastHashJoin -Left output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83] -Right output [1]: [d_date_sk#85] -Arguments: [cs_sold_date_sk#83], [d_date_sk#85], Inner, BuildRight - -(93) CometProject -Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#83, d_date_sk#85] -Arguments: [quantity#86, list_price#87], [cs_quantity#81 AS quantity#86, cs_list_price#82 AS list_price#87] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#90), dynamicpruningexpression(ws_sold_date_sk#90 IN dynamicpruning#91)] -ReadSchema: struct - -(95) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#92] - -(96) CometBroadcastHashJoin -Left output [3]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90] -Right output [1]: [d_date_sk#92] -Arguments: [ws_sold_date_sk#90], [d_date_sk#92], Inner, BuildRight - -(97) CometProject -Input [4]: [ws_quantity#88, ws_list_price#89, ws_sold_date_sk#90, d_date_sk#92] -Arguments: [quantity#93, list_price#94], [ws_quantity#88 AS quantity#93, ws_list_price#89 AS list_price#94] - -(98) CometUnion -Child 0 Input [2]: [quantity#79, list_price#80] -Child 1 Input [2]: [quantity#86, list_price#87] -Child 2 Input [2]: [quantity#93, list_price#94] - -(99) CometHashAggregate -Input [2]: [quantity#79, list_price#80] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] - -(100) CometExchange -Input [2]: [sum#95, count#96] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(101) CometHashAggregate -Input [2]: [sum#95, count#96] -Keys: [] -Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))] - -(102) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#97] - -Subquery:2 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#12 - -Subquery:4 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#90 IN dynamicpruning#12 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (107) -+- * CometColumnarToRow (106) - +- CometProject (105) - +- CometFilter (104) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (103) - - -(103) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#43, d_week_seq#44] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(104) CometFilter -Input [2]: [d_date_sk#43, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#43)) - -(105) CometProject -Input [2]: [d_date_sk#43, d_week_seq#44] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(106) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(107) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:6 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#45, [id=#46] -* CometColumnarToRow (111) -+- CometProject (110) - +- CometFilter (109) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (108) - - -(108) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] -ReadSchema: struct - -(109) CometFilter -Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Condition : (((((isnotnull(d_year#99) AND isnotnull(d_moy#100)) AND isnotnull(d_dom#101)) AND (d_year#99 = 1999)) AND (d_moy#100 = 12)) AND (d_dom#101 = 16)) - -(110) CometProject -Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Arguments: [d_week_seq#98], [d_week_seq#98] - -(111) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#98] - -Subquery:7 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (116) -+- * CometColumnarToRow (115) - +- CometProject (114) - +- CometFilter (113) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (112) - - -(112) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#99] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(113) CometFilter -Input [2]: [d_date_sk#26, d_year#99] -Condition : (((isnotnull(d_year#99) AND (d_year#99 >= 1998)) AND (d_year#99 <= 2000)) AND isnotnull(d_date_sk#26)) - -(114) CometProject -Input [2]: [d_date_sk#26, d_year#99] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(115) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(116) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:8 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:10 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#45, [id=#46] - -Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] - -Subquery:12 Hosting operator id = 65 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#64, d_week_seq#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#64, d_week_seq#65] -Condition : ((isnotnull(d_week_seq#65) AND (d_week_seq#65 = Subquery scalar-subquery#66, [id=#67])) AND isnotnull(d_date_sk#64)) - -(119) CometProject -Input [2]: [d_date_sk#64, d_week_seq#65] -Arguments: [d_date_sk#64], [d_date_sk#64] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#64] - -(121) BroadcastExchange -Input [1]: [d_date_sk#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:13 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#66, [id=#67] -* CometColumnarToRow (125) -+- CometProject (124) - +- CometFilter (123) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (122) - - -(122) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] -ReadSchema: struct - -(123) CometFilter -Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Condition : (((((isnotnull(d_year#99) AND isnotnull(d_moy#100)) AND isnotnull(d_dom#101)) AND (d_year#99 = 1998)) AND (d_moy#100 = 12)) AND (d_dom#101 = 16)) - -(124) CometProject -Input [4]: [d_week_seq#98, d_year#99, d_moy#100, d_dom#101] -Arguments: [d_week_seq#98], [d_week_seq#98] - -(125) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#98] - -Subquery:14 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/extended.txt deleted file mode 100644 index f56d229b68..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/extended.txt +++ /dev/null @@ -1,339 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometBroadcastHashJoin - :- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 298 out of 327 eligible operators (91%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index fb9abae378..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ /dev/null @@ -1,153 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometBroadcastHashJoin [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #14 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #1 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [ss_item_sk] #3 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #4 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #5 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #10 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #11 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 - ReusedExchange [d_date_sk] #10 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #3 - CometBroadcastExchange [d_date_sk] #13 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] #15 - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #4 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #16 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #5 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - Subquery #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - ReusedExchange [ss_item_sk] #3 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 - CometBroadcastExchange [d_date_sk] #18 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_week_seq] - ReusedSubquery [d_week_seq] #6 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/explain.txt deleted file mode 100644 index 4c64b2cef3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/explain.txt +++ /dev/null @@ -1,1020 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (139) -+- CometTakeOrderedAndProject (138) - +- CometHashAggregate (137) - +- CometColumnarExchange (136) - +- * HashAggregate (135) - +- Union (134) - :- * HashAggregate (105) - : +- * CometColumnarToRow (104) - : +- CometColumnarExchange (103) - : +- * HashAggregate (102) - : +- Union (101) - : :- * Filter (68) - : : +- * HashAggregate (67) - : : +- * CometColumnarToRow (66) - : : +- CometColumnarExchange (65) - : : +- * HashAggregate (64) - : : +- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildRight (59) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (52) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (51) - : : : : +- * Project (50) - : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : :- * CometColumnarToRow (6) - : : : : : +- CometFilter (5) - : : : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : : : +- BroadcastExchange (48) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (47) - : : : : :- * CometColumnarToRow (36) - : : : : : +- CometHashAggregate (35) - : : : : : +- CometColumnarExchange (34) - : : : : : +- * HashAggregate (33) - : : : : : +- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Project (29) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (7) - : : : : : : +- BroadcastExchange (27) - : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) - : : : : : : :- * CometColumnarToRow (12) - : : : : : : : +- CometFilter (11) - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (10) - : : : : : : +- BroadcastExchange (25) - : : : : : : +- * Project (24) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Filter (15) - : : : : : : : : +- * ColumnarToRow (14) - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (13) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * CometColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) - : : : : : : +- ReusedExchange (22) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (46) - : : : : +- * Project (45) - : : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : : :- * Project (42) - : : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : : :- * Filter (39) - : : : : : : +- * ColumnarToRow (38) - : : : : : : +- Scan parquet spark_catalog.default.web_sales (37) - : : : : : +- ReusedExchange (40) - : : : : +- ReusedExchange (43) - : : : +- BroadcastExchange (58) - : : : +- * BroadcastHashJoin LeftSemi BuildRight (57) - : : : :- * CometColumnarToRow (55) - : : : : +- CometFilter (54) - : : : : +- CometNativeScan parquet spark_catalog.default.item (53) - : : : +- ReusedExchange (56) - : : +- ReusedExchange (61) - : :- * Filter (84) - : : +- * HashAggregate (83) - : : +- * CometColumnarToRow (82) - : : +- CometColumnarExchange (81) - : : +- * HashAggregate (80) - : : +- * Project (79) - : : +- * BroadcastHashJoin Inner BuildRight (78) - : : :- * Project (76) - : : : +- * BroadcastHashJoin Inner BuildRight (75) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (73) - : : : : :- * Filter (71) - : : : : : +- * ColumnarToRow (70) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (69) - : : : : +- ReusedExchange (72) - : : : +- ReusedExchange (74) - : : +- ReusedExchange (77) - : +- * Filter (100) - : +- * HashAggregate (99) - : +- * CometColumnarToRow (98) - : +- CometColumnarExchange (97) - : +- * HashAggregate (96) - : +- * Project (95) - : +- * BroadcastHashJoin Inner BuildRight (94) - : :- * Project (92) - : : +- * BroadcastHashJoin Inner BuildRight (91) - : : :- * BroadcastHashJoin LeftSemi BuildRight (89) - : : : :- * Filter (87) - : : : : +- * ColumnarToRow (86) - : : : : +- Scan parquet spark_catalog.default.web_sales (85) - : : : +- ReusedExchange (88) - : : +- ReusedExchange (90) - : +- ReusedExchange (93) - :- * HashAggregate (112) - : +- * CometColumnarToRow (111) - : +- CometColumnarExchange (110) - : +- * HashAggregate (109) - : +- * HashAggregate (108) - : +- * CometColumnarToRow (107) - : +- ReusedExchange (106) - :- * HashAggregate (119) - : +- * CometColumnarToRow (118) - : +- CometColumnarExchange (117) - : +- * HashAggregate (116) - : +- * HashAggregate (115) - : +- * CometColumnarToRow (114) - : +- ReusedExchange (113) - :- * HashAggregate (126) - : +- * CometColumnarToRow (125) - : +- CometColumnarExchange (124) - : +- * HashAggregate (123) - : +- * HashAggregate (122) - : +- * CometColumnarToRow (121) - : +- ReusedExchange (120) - +- * HashAggregate (133) - +- * CometColumnarToRow (132) - +- CometColumnarExchange (131) - +- * HashAggregate (130) - +- * HashAggregate (129) - +- * CometColumnarToRow (128) - +- ReusedExchange (127) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 25] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(6) CometColumnarToRow [codegen id : 11] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(7) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] - -(9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(10) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(11) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(12) CometColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(13) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] - -(15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(16) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) - -(18) CometColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(19) BroadcastExchange -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#19] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] - -(22) ReusedExchange [Reuses operator id: 174] -Output [1]: [d_date_sk#23] - -(23) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#23] -Join type: Inner -Join condition: None - -(24) Project [codegen id : 3] -Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#23] - -(25) BroadcastExchange -Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] -Join type: LeftSemi -Join condition: None - -(27) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None - -(29) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(30) ReusedExchange [Reuses operator id: 174] -Output [1]: [d_date_sk#24] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#24] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#25, i_class_id#15 AS class_id#26, i_category_id#16 AS category_id#27] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#24] - -(33) HashAggregate [codegen id : 6] -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] -Aggregate Attributes: [] -Results [3]: [brand_id#25, class_id#26, category_id#27] - -(34) CometColumnarExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: hashpartitioning(brand_id#25, class_id#26, category_id#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(35) CometHashAggregate -Input [3]: [brand_id#25, class_id#26, category_id#27] -Keys [3]: [brand_id#25, class_id#26, category_id#27] -Functions: [] - -(36) CometColumnarToRow [codegen id : 10] -Input [3]: [brand_id#25, class_id#26, category_id#27] - -(37) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] - -(39) Filter [codegen id : 9] -Input [2]: [ws_item_sk#28, ws_sold_date_sk#29] -Condition : isnotnull(ws_item_sk#28) - -(40) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#28] -Right keys [1]: [i_item_sk#30] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33] -Input [6]: [ws_item_sk#28, ws_sold_date_sk#29, i_item_sk#30, i_brand_id#31, i_class_id#32, i_category_id#33] - -(43) ReusedExchange [Reuses operator id: 174] -Output [1]: [d_date_sk#34] - -(44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#29] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 9] -Output [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Input [5]: [ws_sold_date_sk#29, i_brand_id#31, i_class_id#32, i_category_id#33, d_date_sk#34] - -(46) BroadcastExchange -Input [3]: [i_brand_id#31, i_class_id#32, i_category_id#33] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] - -(47) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#25, 0), isnull(brand_id#25), coalesce(class_id#26, 0), isnull(class_id#26), coalesce(category_id#27, 0), isnull(category_id#27)] -Right keys [6]: [coalesce(i_brand_id#31, 0), isnull(i_brand_id#31), coalesce(i_class_id#32, 0), isnull(i_class_id#32), coalesce(i_category_id#33, 0), isnull(i_category_id#33)] -Join type: LeftSemi -Join condition: None - -(48) BroadcastExchange -Input [3]: [brand_id#25, class_id#26, category_id#27] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] - -(49) BroadcastHashJoin [codegen id : 11] -Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#25, class_id#26, category_id#27] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#35] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#25, class_id#26, category_id#27] - -(51) BroadcastExchange -Input [1]: [ss_item_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(52) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(53) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(54) CometFilter -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Condition : isnotnull(i_item_sk#36) - -(55) CometColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(56) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#35] - -(57) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#36] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(58) BroadcastExchange -Input [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(59) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#36] -Join type: Inner -Join condition: None - -(60) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] - -(61) ReusedExchange [Reuses operator id: 169] -Output [1]: [d_date_sk#40] - -(62) BroadcastHashJoin [codegen id : 25] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#40] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] - -(64) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] -Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43] -Results [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(65) CometColumnarExchange -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Arguments: hashpartitioning(i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(66) CometColumnarToRow [codegen id : 26] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] - -(67) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46] -Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] -Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48] -Results [6]: [store AS channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51] - -(68) Filter [codegen id : 26] -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51] -Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(69) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_sold_date_sk#57 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(70) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] - -(71) Filter [codegen id : 51] -Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Condition : isnotnull(cs_item_sk#54) - -(72) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#35] - -(73) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#54] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(74) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61] - -(75) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#54] -Right keys [1]: [i_item_sk#58] -Join type: Inner -Join condition: None - -(76) Project [codegen id : 51] -Output [6]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#59, i_class_id#60, i_category_id#61] -Input [8]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_item_sk#58, i_brand_id#59, i_class_id#60, i_category_id#61] - -(77) ReusedExchange [Reuses operator id: 169] -Output [1]: [d_date_sk#62] - -(78) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#57] -Right keys [1]: [d_date_sk#62] -Join type: Inner -Join condition: None - -(79) Project [codegen id : 51] -Output [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#59, i_class_id#60, i_category_id#61] -Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#59, i_class_id#60, i_category_id#61, d_date_sk#62] - -(80) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#59, i_class_id#60, i_category_id#61] -Keys [3]: [i_brand_id#59, i_class_id#60, i_category_id#61] -Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] -Aggregate Attributes [3]: [sum#63, isEmpty#64, count#65] -Results [6]: [i_brand_id#59, i_class_id#60, i_category_id#61, sum#66, isEmpty#67, count#68] - -(81) CometColumnarExchange -Input [6]: [i_brand_id#59, i_class_id#60, i_category_id#61, sum#66, isEmpty#67, count#68] -Arguments: hashpartitioning(i_brand_id#59, i_class_id#60, i_category_id#61, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(82) CometColumnarToRow [codegen id : 52] -Input [6]: [i_brand_id#59, i_class_id#60, i_category_id#61, sum#66, isEmpty#67, count#68] - -(83) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#59, i_class_id#60, i_category_id#61, sum#66, isEmpty#67, count#68] -Keys [3]: [i_brand_id#59, i_class_id#60, i_category_id#61] -Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] -Aggregate Attributes [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#69, count(1)#70] -Results [6]: [catalog AS channel#71, i_brand_id#59, i_class_id#60, i_category_id#61, sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#69 AS sales#72, count(1)#70 AS number_sales#73] - -(84) Filter [codegen id : 52] -Input [6]: [channel#71, i_brand_id#59, i_class_id#60, i_category_id#61, sales#72, number_sales#73] -Condition : (isnotnull(sales#72) AND (cast(sales#72 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(85) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#77), dynamicpruningexpression(ws_sold_date_sk#77 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(86) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] - -(87) Filter [codegen id : 77] -Input [4]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77] -Condition : isnotnull(ws_item_sk#74) - -(88) ReusedExchange [Reuses operator id: 51] -Output [1]: [ss_item_sk#35] - -(89) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#74] -Right keys [1]: [ss_item_sk#35] -Join type: LeftSemi -Join condition: None - -(90) ReusedExchange [Reuses operator id: 58] -Output [4]: [i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81] - -(91) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#74] -Right keys [1]: [i_item_sk#78] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 77] -Output [6]: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#79, i_class_id#80, i_category_id#81] -Input [8]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_item_sk#78, i_brand_id#79, i_class_id#80, i_category_id#81] - -(93) ReusedExchange [Reuses operator id: 169] -Output [1]: [d_date_sk#82] - -(94) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#77] -Right keys [1]: [d_date_sk#82] -Join type: Inner -Join condition: None - -(95) Project [codegen id : 77] -Output [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#79, i_class_id#80, i_category_id#81] -Input [7]: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#79, i_class_id#80, i_category_id#81, d_date_sk#82] - -(96) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#79, i_class_id#80, i_category_id#81] -Keys [3]: [i_brand_id#79, i_class_id#80, i_category_id#81] -Functions [2]: [partial_sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), partial_count(1)] -Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85] -Results [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87, count#88] - -(97) CometColumnarExchange -Input [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87, count#88] -Arguments: hashpartitioning(i_brand_id#79, i_class_id#80, i_category_id#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(98) CometColumnarToRow [codegen id : 78] -Input [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87, count#88] - -(99) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87, count#88] -Keys [3]: [i_brand_id#79, i_class_id#80, i_category_id#81] -Functions [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), count(1)] -Aggregate Attributes [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76))#89, count(1)#90] -Results [6]: [web AS channel#91, i_brand_id#79, i_class_id#80, i_category_id#81, sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76))#89 AS sales#92, count(1)#90 AS number_sales#93] - -(100) Filter [codegen id : 78] -Input [6]: [channel#91, i_brand_id#79, i_class_id#80, i_category_id#81, sales#92, number_sales#93] -Condition : (isnotnull(sales#92) AND (cast(sales#92 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(101) Union - -(102) HashAggregate [codegen id : 79] -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51] -Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] -Aggregate Attributes [3]: [sum#94, isEmpty#95, sum#96] -Results [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] - -(103) CometColumnarExchange -Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] -Arguments: hashpartitioning(channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(104) CometColumnarToRow [codegen id : 80] -Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] - -(105) HashAggregate [codegen id : 80] -Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] -Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [sum(sales#50), sum(number_sales#51)] -Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] -Results [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] - -(106) ReusedExchange [Reuses operator id: 103] -Output [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] - -(107) CometColumnarToRow [codegen id : 160] -Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] - -(108) HashAggregate [codegen id : 160] -Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] -Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [sum(sales#50), sum(number_sales#51)] -Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] -Results [5]: [channel#49, i_brand_id#37, i_class_id#38, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] - -(109) HashAggregate [codegen id : 160] -Input [5]: [channel#49, i_brand_id#37, i_class_id#38, sum_sales#102, number_sales#103] -Keys [3]: [channel#49, i_brand_id#37, i_class_id#38] -Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] -Aggregate Attributes [3]: [sum#104, isEmpty#105, sum#106] -Results [6]: [channel#49, i_brand_id#37, i_class_id#38, sum#107, isEmpty#108, sum#109] - -(110) CometColumnarExchange -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, sum#107, isEmpty#108, sum#109] -Arguments: hashpartitioning(channel#49, i_brand_id#37, i_class_id#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(111) CometColumnarToRow [codegen id : 161] -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, sum#107, isEmpty#108, sum#109] - -(112) HashAggregate [codegen id : 161] -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, sum#107, isEmpty#108, sum#109] -Keys [3]: [channel#49, i_brand_id#37, i_class_id#38] -Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] -Aggregate Attributes [2]: [sum(sum_sales#102)#110, sum(number_sales#103)#111] -Results [6]: [channel#49, i_brand_id#37, i_class_id#38, null AS i_category_id#112, sum(sum_sales#102)#110 AS sum(sum_sales)#113, sum(number_sales#103)#111 AS sum(number_sales)#114] - -(113) ReusedExchange [Reuses operator id: 103] -Output [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] - -(114) CometColumnarToRow [codegen id : 241] -Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] - -(115) HashAggregate [codegen id : 241] -Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] -Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [sum(sales#50), sum(number_sales#51)] -Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] -Results [4]: [channel#49, i_brand_id#37, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] - -(116) HashAggregate [codegen id : 241] -Input [4]: [channel#49, i_brand_id#37, sum_sales#102, number_sales#103] -Keys [2]: [channel#49, i_brand_id#37] -Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] -Aggregate Attributes [3]: [sum#115, isEmpty#116, sum#117] -Results [5]: [channel#49, i_brand_id#37, sum#118, isEmpty#119, sum#120] - -(117) CometColumnarExchange -Input [5]: [channel#49, i_brand_id#37, sum#118, isEmpty#119, sum#120] -Arguments: hashpartitioning(channel#49, i_brand_id#37, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(118) CometColumnarToRow [codegen id : 242] -Input [5]: [channel#49, i_brand_id#37, sum#118, isEmpty#119, sum#120] - -(119) HashAggregate [codegen id : 242] -Input [5]: [channel#49, i_brand_id#37, sum#118, isEmpty#119, sum#120] -Keys [2]: [channel#49, i_brand_id#37] -Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] -Aggregate Attributes [2]: [sum(sum_sales#102)#121, sum(number_sales#103)#122] -Results [6]: [channel#49, i_brand_id#37, null AS i_class_id#123, null AS i_category_id#124, sum(sum_sales#102)#121 AS sum(sum_sales)#125, sum(number_sales#103)#122 AS sum(number_sales)#126] - -(120) ReusedExchange [Reuses operator id: 103] -Output [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] - -(121) CometColumnarToRow [codegen id : 322] -Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] - -(122) HashAggregate [codegen id : 322] -Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] -Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [sum(sales#50), sum(number_sales#51)] -Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] -Results [3]: [channel#49, sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] - -(123) HashAggregate [codegen id : 322] -Input [3]: [channel#49, sum_sales#102, number_sales#103] -Keys [1]: [channel#49] -Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] -Aggregate Attributes [3]: [sum#127, isEmpty#128, sum#129] -Results [4]: [channel#49, sum#130, isEmpty#131, sum#132] - -(124) CometColumnarExchange -Input [4]: [channel#49, sum#130, isEmpty#131, sum#132] -Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] - -(125) CometColumnarToRow [codegen id : 323] -Input [4]: [channel#49, sum#130, isEmpty#131, sum#132] - -(126) HashAggregate [codegen id : 323] -Input [4]: [channel#49, sum#130, isEmpty#131, sum#132] -Keys [1]: [channel#49] -Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] -Aggregate Attributes [2]: [sum(sum_sales#102)#133, sum(number_sales#103)#134] -Results [6]: [channel#49, null AS i_brand_id#135, null AS i_class_id#136, null AS i_category_id#137, sum(sum_sales#102)#133 AS sum(sum_sales)#138, sum(number_sales#103)#134 AS sum(number_sales)#139] - -(127) ReusedExchange [Reuses operator id: 103] -Output [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] - -(128) CometColumnarToRow [codegen id : 403] -Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] - -(129) HashAggregate [codegen id : 403] -Input [7]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum#97, isEmpty#98, sum#99] -Keys [4]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39] -Functions [2]: [sum(sales#50), sum(number_sales#51)] -Aggregate Attributes [2]: [sum(sales#50)#100, sum(number_sales#51)#101] -Results [2]: [sum(sales#50)#100 AS sum_sales#102, sum(number_sales#51)#101 AS number_sales#103] - -(130) HashAggregate [codegen id : 403] -Input [2]: [sum_sales#102, number_sales#103] -Keys: [] -Functions [2]: [partial_sum(sum_sales#102), partial_sum(number_sales#103)] -Aggregate Attributes [3]: [sum#140, isEmpty#141, sum#142] -Results [3]: [sum#143, isEmpty#144, sum#145] - -(131) CometColumnarExchange -Input [3]: [sum#143, isEmpty#144, sum#145] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(132) CometColumnarToRow [codegen id : 404] -Input [3]: [sum#143, isEmpty#144, sum#145] - -(133) HashAggregate [codegen id : 404] -Input [3]: [sum#143, isEmpty#144, sum#145] -Keys: [] -Functions [2]: [sum(sum_sales#102), sum(number_sales#103)] -Aggregate Attributes [2]: [sum(sum_sales#102)#146, sum(number_sales#103)#147] -Results [6]: [null AS channel#148, null AS i_brand_id#149, null AS i_class_id#150, null AS i_category_id#151, sum(sum_sales#102)#146 AS sum(sum_sales)#152, sum(number_sales#103)#147 AS sum(number_sales)#153] - -(134) Union - -(135) HashAggregate [codegen id : 405] -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] -Keys [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] -Functions: [] -Aggregate Attributes: [] -Results [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] - -(136) CometColumnarExchange -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] -Arguments: hashpartitioning(channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] - -(137) CometHashAggregate -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] -Keys [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] -Functions: [] - -(138) CometTakeOrderedAndProject -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#49 ASC NULLS FIRST,i_brand_id#37 ASC NULLS FIRST,i_class_id#38 ASC NULLS FIRST,i_category_id#39 ASC NULLS FIRST], output=[channel#49,i_brand_id#37,i_class_id#38,i_category_id#39,sum_sales#102,number_sales#103]), [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103], 100, 0, [channel#49 ASC NULLS FIRST, i_brand_id#37 ASC NULLS FIRST, i_class_id#38 ASC NULLS FIRST, i_category_id#39 ASC NULLS FIRST], [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] - -(139) CometColumnarToRow [codegen id : 406] -Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum_sales#102, number_sales#103] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* HashAggregate (159) -+- * CometColumnarToRow (158) - +- CometColumnarExchange (157) - +- * HashAggregate (156) - +- Union (155) - :- * Project (144) - : +- * BroadcastHashJoin Inner BuildRight (143) - : :- * ColumnarToRow (141) - : : +- Scan parquet spark_catalog.default.store_sales (140) - : +- ReusedExchange (142) - :- * Project (149) - : +- * BroadcastHashJoin Inner BuildRight (148) - : :- * ColumnarToRow (146) - : : +- Scan parquet spark_catalog.default.catalog_sales (145) - : +- ReusedExchange (147) - +- * Project (154) - +- * BroadcastHashJoin Inner BuildRight (153) - :- * ColumnarToRow (151) - : +- Scan parquet spark_catalog.default.web_sales (150) - +- ReusedExchange (152) - - -(140) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#154, ss_list_price#155, ss_sold_date_sk#156] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#156), dynamicpruningexpression(ss_sold_date_sk#156 IN dynamicpruning#12)] -ReadSchema: struct - -(141) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#154, ss_list_price#155, ss_sold_date_sk#156] - -(142) ReusedExchange [Reuses operator id: 174] -Output [1]: [d_date_sk#157] - -(143) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#156] -Right keys [1]: [d_date_sk#157] -Join type: Inner -Join condition: None - -(144) Project [codegen id : 2] -Output [2]: [ss_quantity#154 AS quantity#158, ss_list_price#155 AS list_price#159] -Input [4]: [ss_quantity#154, ss_list_price#155, ss_sold_date_sk#156, d_date_sk#157] - -(145) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#160, cs_list_price#161, cs_sold_date_sk#162] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#162), dynamicpruningexpression(cs_sold_date_sk#162 IN dynamicpruning#163)] -ReadSchema: struct - -(146) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#160, cs_list_price#161, cs_sold_date_sk#162] - -(147) ReusedExchange [Reuses operator id: 164] -Output [1]: [d_date_sk#164] - -(148) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#162] -Right keys [1]: [d_date_sk#164] -Join type: Inner -Join condition: None - -(149) Project [codegen id : 4] -Output [2]: [cs_quantity#160 AS quantity#165, cs_list_price#161 AS list_price#166] -Input [4]: [cs_quantity#160, cs_list_price#161, cs_sold_date_sk#162, d_date_sk#164] - -(150) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#167, ws_list_price#168, ws_sold_date_sk#169] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#169), dynamicpruningexpression(ws_sold_date_sk#169 IN dynamicpruning#163)] -ReadSchema: struct - -(151) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#167, ws_list_price#168, ws_sold_date_sk#169] - -(152) ReusedExchange [Reuses operator id: 164] -Output [1]: [d_date_sk#170] - -(153) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#169] -Right keys [1]: [d_date_sk#170] -Join type: Inner -Join condition: None - -(154) Project [codegen id : 6] -Output [2]: [ws_quantity#167 AS quantity#171, ws_list_price#168 AS list_price#172] -Input [4]: [ws_quantity#167, ws_list_price#168, ws_sold_date_sk#169, d_date_sk#170] - -(155) Union - -(156) HashAggregate [codegen id : 7] -Input [2]: [quantity#158, list_price#159] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#158 as decimal(10,0)) * list_price#159))] -Aggregate Attributes [2]: [sum#173, count#174] -Results [2]: [sum#175, count#176] - -(157) CometColumnarExchange -Input [2]: [sum#175, count#176] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=18] - -(158) CometColumnarToRow [codegen id : 8] -Input [2]: [sum#175, count#176] - -(159) HashAggregate [codegen id : 8] -Input [2]: [sum#175, count#176] -Keys: [] -Functions [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))] -Aggregate Attributes [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))#177] -Results [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))#177 AS average_sales#178] - -Subquery:2 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#156 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 145 Hosting Expression = cs_sold_date_sk#162 IN dynamicpruning#163 -BroadcastExchange (164) -+- * CometColumnarToRow (163) - +- CometProject (162) - +- CometFilter (161) - +- CometNativeScan parquet spark_catalog.default.date_dim (160) - - -(160) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#164, d_year#179] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(161) CometFilter -Input [2]: [d_date_sk#164, d_year#179] -Condition : (((isnotnull(d_year#179) AND (d_year#179 >= 1998)) AND (d_year#179 <= 2000)) AND isnotnull(d_date_sk#164)) - -(162) CometProject -Input [2]: [d_date_sk#164, d_year#179] -Arguments: [d_date_sk#164], [d_date_sk#164] - -(163) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#164] - -(164) BroadcastExchange -Input [1]: [d_date_sk#164] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] - -Subquery:4 Hosting operator id = 150 Hosting Expression = ws_sold_date_sk#169 IN dynamicpruning#163 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (169) -+- * CometColumnarToRow (168) - +- CometProject (167) - +- CometFilter (166) - +- CometNativeScan parquet spark_catalog.default.date_dim (165) - - -(165) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#40, d_year#180, d_moy#181] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(166) CometFilter -Input [3]: [d_date_sk#40, d_year#180, d_moy#181] -Condition : ((((isnotnull(d_year#180) AND isnotnull(d_moy#181)) AND (d_year#180 = 2000)) AND (d_moy#181 = 11)) AND isnotnull(d_date_sk#40)) - -(167) CometProject -Input [3]: [d_date_sk#40, d_year#180, d_moy#181] -Arguments: [d_date_sk#40], [d_date_sk#40] - -(168) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#40] - -(169) BroadcastExchange -Input [1]: [d_date_sk#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] - -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (174) -+- * CometColumnarToRow (173) - +- CometProject (172) - +- CometFilter (171) - +- CometNativeScan parquet spark_catalog.default.date_dim (170) - - -(170) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#182] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(171) CometFilter -Input [2]: [d_date_sk#24, d_year#182] -Condition : (((isnotnull(d_year#182) AND (d_year#182 >= 1999)) AND (d_year#182 <= 2001)) AND isnotnull(d_date_sk#24)) - -(172) CometProject -Input [2]: [d_date_sk#24, d_year#182] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(173) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#24] - -(174) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21] - -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:8 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 84 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:10 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 - -Subquery:11 Hosting operator id = 100 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:12 Hosting operator id = 85 Hosting Expression = ws_sold_date_sk#77 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/simplified.txt deleted file mode 100644 index 4458723393..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/simplified.txt +++ /dev/null @@ -1,276 +0,0 @@ -WholeStageCodegen (406) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - WholeStageCodegen (405) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - InputAdapter - Union - WholeStageCodegen (80) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (79) - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - InputAdapter - Union - WholeStageCodegen (26) - Filter [sales] - Subquery #3 - WholeStageCodegen (8) - HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #14 - WholeStageCodegen (7) - HashAggregate [quantity,list_price] [sum,count,sum,count] - InputAdapter - Union - WholeStageCodegen (2) - Project [ss_quantity,ss_list_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #8 - WholeStageCodegen (4) - Project [cs_quantity,cs_list_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #15 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #15 - WholeStageCodegen (6) - Project [ws_quantity,ws_list_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 - InputAdapter - ReusedExchange [d_date_sk] #15 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (25) - HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - BroadcastHashJoin [ss_item_sk,ss_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (11) - Project [i_item_sk] - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (10) - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometHashAggregate [brand_id,class_id,category_id] - CometColumnarExchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (6) - HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 - InputAdapter - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (23) - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (52) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (51) - HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - BroadcastHashJoin [cs_item_sk,ss_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (78) - Filter [sales] - ReusedSubquery [average_sales] #3 - HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (77) - HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] - Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] - BroadcastHashJoin [ws_item_sk,ss_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [ss_item_sk] #5 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (161) - HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id,i_class_id] #18 - WholeStageCodegen (160) - HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (242) - HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,i_brand_id] #19 - WholeStageCodegen (241) - HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (323) - HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #20 - WholeStageCodegen (322) - HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - WholeStageCodegen (404) - HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #21 - WholeStageCodegen (403) - HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] - HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/explain.txt deleted file mode 100644 index 4ac6c99894..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,914 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (122) -+- CometTakeOrderedAndProject (121) - +- CometHashAggregate (120) - +- CometExchange (119) - +- CometHashAggregate (118) - +- CometUnion (117) - :- CometHashAggregate (96) - : +- CometExchange (95) - : +- CometHashAggregate (94) - : +- CometUnion (93) - : :- CometFilter (64) - : : +- CometHashAggregate (63) - : : +- CometExchange (62) - : : +- CometHashAggregate (61) - : : +- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometBroadcastHashJoin (47) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (46) - : : : : +- CometProject (45) - : : : : +- CometBroadcastHashJoin (44) - : : : : :- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : : +- CometBroadcastExchange (43) - : : : : +- CometBroadcastHashJoin (42) - : : : : :- CometHashAggregate (32) - : : : : : +- CometExchange (31) - : : : : : +- CometHashAggregate (30) - : : : : : +- CometProject (29) - : : : : : +- CometBroadcastHashJoin (28) - : : : : : :- CometProject (26) - : : : : : : +- CometBroadcastHashJoin (25) - : : : : : : :- CometFilter (6) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : : +- CometBroadcastExchange (24) - : : : : : : +- CometBroadcastHashJoin (23) - : : : : : : :- CometFilter (8) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : : +- CometBroadcastExchange (22) - : : : : : : +- CometProject (21) - : : : : : : +- CometBroadcastHashJoin (20) - : : : : : : :- CometProject (15) - : : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : : :- CometFilter (10) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : : +- CometBroadcastExchange (13) - : : : : : : : +- CometFilter (12) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : : +- CometBroadcastExchange (19) - : : : : : : +- CometProject (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : : +- ReusedExchange (27) - : : : : +- CometBroadcastExchange (41) - : : : : +- CometProject (40) - : : : : +- CometBroadcastHashJoin (39) - : : : : :- CometProject (37) - : : : : : +- CometBroadcastHashJoin (36) - : : : : : :- CometFilter (34) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : : +- ReusedExchange (35) - : : : : +- ReusedExchange (38) - : : : +- CometBroadcastExchange (52) - : : : +- CometBroadcastHashJoin (51) - : : : :- CometFilter (49) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : : +- ReusedExchange (50) - : : +- CometBroadcastExchange (58) - : : +- CometProject (57) - : : +- CometFilter (56) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - : :- CometFilter (78) - : : +- CometHashAggregate (77) - : : +- CometExchange (76) - : : +- CometHashAggregate (75) - : : +- CometProject (74) - : : +- CometBroadcastHashJoin (73) - : : :- CometProject (71) - : : : +- CometBroadcastHashJoin (70) - : : : :- CometBroadcastHashJoin (68) - : : : : :- CometFilter (66) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (65) - : : : : +- ReusedExchange (67) - : : : +- ReusedExchange (69) - : : +- ReusedExchange (72) - : +- CometFilter (92) - : +- CometHashAggregate (91) - : +- CometExchange (90) - : +- CometHashAggregate (89) - : +- CometProject (88) - : +- CometBroadcastHashJoin (87) - : :- CometProject (85) - : : +- CometBroadcastHashJoin (84) - : : :- CometBroadcastHashJoin (82) - : : : :- CometFilter (80) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (79) - : : : +- ReusedExchange (81) - : : +- ReusedExchange (83) - : +- ReusedExchange (86) - :- CometHashAggregate (101) - : +- CometExchange (100) - : +- CometHashAggregate (99) - : +- CometHashAggregate (98) - : +- ReusedExchange (97) - :- CometHashAggregate (106) - : +- CometExchange (105) - : +- CometHashAggregate (104) - : +- CometHashAggregate (103) - : +- ReusedExchange (102) - :- CometHashAggregate (111) - : +- CometExchange (110) - : +- CometHashAggregate (109) - : +- CometHashAggregate (108) - : +- ReusedExchange (107) - +- CometHashAggregate (116) - +- CometExchange (115) - +- CometHashAggregate (114) - +- CometHashAggregate (113) - +- ReusedExchange (112) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : isnotnull(i_item_sk#39) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_sold_date_sk#57 IN dynamicpruning#58)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Condition : isnotnull(cs_item_sk#54) - -(67) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(68) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Right output [1]: [ss_item_sk#38] -Arguments: [cs_item_sk#54], [ss_item_sk#38], LeftSemi, BuildRight - -(69) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] - -(70) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Right output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -Arguments: [cs_item_sk#54], [i_item_sk#59], Inner, BuildRight - -(71) CometProject -Input [8]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -Arguments: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62], [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62] - -(72) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#63] - -(73) CometBroadcastHashJoin -Left output [6]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62] -Right output [1]: [d_date_sk#63] -Arguments: [cs_sold_date_sk#57], [d_date_sk#63], Inner, BuildRight - -(74) CometProject -Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62, d_date_sk#63] -Arguments: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62], [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] - -(75) CometHashAggregate -Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] - -(76) CometExchange -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#64, isEmpty#65, count#66] -Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(77) CometHashAggregate -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#64, isEmpty#65, count#66] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] - -(78) CometFilter -Input [6]: [channel#67, i_brand_id#60, i_class_id#61, i_category_id#62, sales#68, number_sales#69] -Condition : (isnotnull(sales#68) AND (cast(sales#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#70, ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#73), dynamicpruningexpression(ws_sold_date_sk#73 IN dynamicpruning#74)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(80) CometFilter -Input [4]: [ws_item_sk#70, ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73] -Condition : isnotnull(ws_item_sk#70) - -(81) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(82) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#70, ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73] -Right output [1]: [ss_item_sk#38] -Arguments: [ws_item_sk#70], [ss_item_sk#38], LeftSemi, BuildRight - -(83) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] - -(84) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#70, ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73] -Right output [4]: [i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] -Arguments: [ws_item_sk#70], [i_item_sk#75], Inner, BuildRight - -(85) CometProject -Input [8]: [ws_item_sk#70, ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73, i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] -Arguments: [ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78], [ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78] - -(86) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#79] - -(87) CometBroadcastHashJoin -Left output [6]: [ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78] -Right output [1]: [d_date_sk#79] -Arguments: [ws_sold_date_sk#73], [d_date_sk#79], Inner, BuildRight - -(88) CometProject -Input [7]: [ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78, d_date_sk#79] -Arguments: [ws_quantity#71, ws_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78], [ws_quantity#71, ws_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] - -(89) CometHashAggregate -Input [5]: [ws_quantity#71, ws_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] -Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] -Functions [2]: [partial_sum((cast(ws_quantity#71 as decimal(10,0)) * ws_list_price#72)), partial_count(1)] - -(90) CometExchange -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#80, isEmpty#81, count#82] -Arguments: hashpartitioning(i_brand_id#76, i_class_id#77, i_category_id#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(91) CometHashAggregate -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#80, isEmpty#81, count#82] -Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] -Functions [2]: [sum((cast(ws_quantity#71 as decimal(10,0)) * ws_list_price#72)), count(1)] - -(92) CometFilter -Input [6]: [channel#83, i_brand_id#76, i_class_id#77, i_category_id#78, sales#84, number_sales#85] -Condition : (isnotnull(sales#84) AND (cast(sales#84 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(93) CometUnion -Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Child 1 Input [6]: [channel#67, i_brand_id#60, i_class_id#61, i_category_id#62, sales#68, number_sales#69] -Child 2 Input [6]: [channel#83, i_brand_id#76, i_class_id#77, i_category_id#78, sales#84, number_sales#85] - -(94) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] - -(95) CometExchange -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] -Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(96) CometHashAggregate -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(97) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] - -(98) CometHashAggregate -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(99) CometHashAggregate -Input [5]: [channel#49, i_brand_id#40, i_class_id#41, sum_sales#89, number_sales#90] -Keys [3]: [channel#49, i_brand_id#40, i_class_id#41] -Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] - -(100) CometExchange -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, sum#91, isEmpty#92, sum#93] -Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(101) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, sum#91, isEmpty#92, sum#93] -Keys [3]: [channel#49, i_brand_id#40, i_class_id#41] -Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] - -(102) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] - -(103) CometHashAggregate -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(104) CometHashAggregate -Input [4]: [channel#49, i_brand_id#40, sum_sales#89, number_sales#90] -Keys [2]: [channel#49, i_brand_id#40] -Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] - -(105) CometExchange -Input [5]: [channel#49, i_brand_id#40, sum#94, isEmpty#95, sum#96] -Arguments: hashpartitioning(channel#49, i_brand_id#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(106) CometHashAggregate -Input [5]: [channel#49, i_brand_id#40, sum#94, isEmpty#95, sum#96] -Keys [2]: [channel#49, i_brand_id#40] -Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] - -(107) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] - -(108) CometHashAggregate -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(109) CometHashAggregate -Input [3]: [channel#49, sum_sales#89, number_sales#90] -Keys [1]: [channel#49] -Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] - -(110) CometExchange -Input [4]: [channel#49, sum#97, isEmpty#98, sum#99] -Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(111) CometHashAggregate -Input [4]: [channel#49, sum#97, isEmpty#98, sum#99] -Keys [1]: [channel#49] -Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] - -(112) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] - -(113) CometHashAggregate -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(114) CometHashAggregate -Input [2]: [sum_sales#89, number_sales#90] -Keys: [] -Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] - -(115) CometExchange -Input [3]: [sum#100, isEmpty#101, sum#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(116) CometHashAggregate -Input [3]: [sum#100, isEmpty#101, sum#102] -Keys: [] -Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] - -(117) CometUnion -Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] -Child 1 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#103, sum(sum_sales)#104, sum(number_sales)#105] -Child 2 Input [6]: [channel#49, i_brand_id#40, i_class_id#106, i_category_id#107, sum(sum_sales)#108, sum(number_sales)#109] -Child 3 Input [6]: [channel#49, i_brand_id#110, i_class_id#111, i_category_id#112, sum(sum_sales)#113, sum(number_sales)#114] -Child 4 Input [6]: [channel#115, i_brand_id#116, i_class_id#117, i_category_id#118, sum(sum_sales)#119, sum(number_sales)#120] - -(118) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] -Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] -Functions: [] - -(119) CometExchange -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] -Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(120) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] -Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] -Functions: [] - -(121) CometTakeOrderedAndProject -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#49 ASC NULLS FIRST,i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#49,i_brand_id#40,i_class_id#41,i_category_id#42,sum_sales#89,number_sales#90]), [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90], 100, 0, [channel#49 ASC NULLS FIRST, i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] - -(122) CometColumnarToRow [codegen id : 1] -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* CometColumnarToRow (142) -+- CometHashAggregate (141) - +- CometExchange (140) - +- CometHashAggregate (139) - +- CometUnion (138) - :- CometProject (126) - : +- CometBroadcastHashJoin (125) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (123) - : +- ReusedExchange (124) - :- CometProject (133) - : +- CometBroadcastHashJoin (132) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (127) - : +- CometBroadcastExchange (131) - : +- CometProject (130) - : +- CometFilter (129) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (128) - +- CometProject (137) - +- CometBroadcastHashJoin (136) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (134) - +- ReusedExchange (135) - - -(123) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#121, ss_list_price#122, ss_sold_date_sk#123] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#123), dynamicpruningexpression(ss_sold_date_sk#123 IN dynamicpruning#124)] -ReadSchema: struct - -(124) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#125] - -(125) CometBroadcastHashJoin -Left output [3]: [ss_quantity#121, ss_list_price#122, ss_sold_date_sk#123] -Right output [1]: [d_date_sk#125] -Arguments: [ss_sold_date_sk#123], [d_date_sk#125], Inner, BuildRight - -(126) CometProject -Input [4]: [ss_quantity#121, ss_list_price#122, ss_sold_date_sk#123, d_date_sk#125] -Arguments: [quantity#126, list_price#127], [ss_quantity#121 AS quantity#126, ss_list_price#122 AS list_price#127] - -(127) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#128, cs_list_price#129, cs_sold_date_sk#130] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#130), dynamicpruningexpression(cs_sold_date_sk#130 IN dynamicpruning#131)] -ReadSchema: struct - -(128) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#132, d_year#133] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(129) CometFilter -Input [2]: [d_date_sk#132, d_year#133] -Condition : (((isnotnull(d_year#133) AND (d_year#133 >= 1998)) AND (d_year#133 <= 2000)) AND isnotnull(d_date_sk#132)) - -(130) CometProject -Input [2]: [d_date_sk#132, d_year#133] -Arguments: [d_date_sk#132], [d_date_sk#132] - -(131) CometBroadcastExchange -Input [1]: [d_date_sk#132] -Arguments: [d_date_sk#132] - -(132) CometBroadcastHashJoin -Left output [3]: [cs_quantity#128, cs_list_price#129, cs_sold_date_sk#130] -Right output [1]: [d_date_sk#132] -Arguments: [cs_sold_date_sk#130], [d_date_sk#132], Inner, BuildRight - -(133) CometProject -Input [4]: [cs_quantity#128, cs_list_price#129, cs_sold_date_sk#130, d_date_sk#132] -Arguments: [quantity#134, list_price#135], [cs_quantity#128 AS quantity#134, cs_list_price#129 AS list_price#135] - -(134) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#136, ws_list_price#137, ws_sold_date_sk#138] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#138), dynamicpruningexpression(ws_sold_date_sk#138 IN dynamicpruning#139)] -ReadSchema: struct - -(135) ReusedExchange [Reuses operator id: 131] -Output [1]: [d_date_sk#140] - -(136) CometBroadcastHashJoin -Left output [3]: [ws_quantity#136, ws_list_price#137, ws_sold_date_sk#138] -Right output [1]: [d_date_sk#140] -Arguments: [ws_sold_date_sk#138], [d_date_sk#140], Inner, BuildRight - -(137) CometProject -Input [4]: [ws_quantity#136, ws_list_price#137, ws_sold_date_sk#138, d_date_sk#140] -Arguments: [quantity#141, list_price#142], [ws_quantity#136 AS quantity#141, ws_list_price#137 AS list_price#142] - -(138) CometUnion -Child 0 Input [2]: [quantity#126, list_price#127] -Child 1 Input [2]: [quantity#134, list_price#135] -Child 2 Input [2]: [quantity#141, list_price#142] - -(139) CometHashAggregate -Input [2]: [quantity#126, list_price#127] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#126 as decimal(10,0)) * list_price#127))] - -(140) CometExchange -Input [2]: [sum#143, count#144] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(141) CometHashAggregate -Input [2]: [sum#143, count#144] -Keys: [] -Functions [1]: [avg((cast(quantity#126 as decimal(10,0)) * list_price#127))] - -(142) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#145] - -Subquery:2 Hosting operator id = 123 Hosting Expression = ss_sold_date_sk#123 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 127 Hosting Expression = cs_sold_date_sk#130 IN dynamicpruning#131 -BroadcastExchange (147) -+- * CometColumnarToRow (146) - +- CometProject (145) - +- CometFilter (144) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (143) - - -(143) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#132, d_year#133] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(144) CometFilter -Input [2]: [d_date_sk#132, d_year#133] -Condition : (((isnotnull(d_year#133) AND (d_year#133 >= 1998)) AND (d_year#133 <= 2000)) AND isnotnull(d_date_sk#132)) - -(145) CometProject -Input [2]: [d_date_sk#132, d_year#133] -Arguments: [d_date_sk#132], [d_date_sk#132] - -(146) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#132] - -(147) BroadcastExchange -Input [1]: [d_date_sk#132] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#138 IN dynamicpruning#131 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (152) -+- * CometColumnarToRow (151) - +- CometProject (150) - +- CometFilter (149) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (148) - - -(148) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(149) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(150) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(151) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(152) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (157) -+- * CometColumnarToRow (156) - +- CometProject (155) - +- CometFilter (154) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (153) - - -(153) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#146] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(154) CometFilter -Input [2]: [d_date_sk#26, d_year#146] -Condition : (((isnotnull(d_year#146) AND (d_year#146 >= 1999)) AND (d_year#146 <= 2001)) AND isnotnull(d_date_sk#26)) - -(155) CometProject -Input [2]: [d_date_sk#26, d_year#146] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(156) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(157) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 78 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:10 Hosting operator id = 65 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 - -Subquery:11 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:12 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#73 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 758e917335..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,179 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometUnion [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - CometExchange [channel,i_brand_id,i_class_id,i_category_id] #2 - CometHashAggregate [sales,number_sales] [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] - CometUnion [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #16 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #18 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 - ReusedExchange [d_date_sk] #18 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #3 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk] #5 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #6 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #7 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #12 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #12 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #13 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #12 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #5 - CometBroadcastExchange [d_date_sk] #15 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #19 - CometHashAggregate [cs_quantity,cs_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk] - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #5 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - ReusedExchange [d_date_sk] #15 - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #20 - CometHashAggregate [ws_quantity,ws_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk] - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #5 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - ReusedExchange [d_date_sk] #15 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel,i_brand_id,i_class_id] #21 - CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,i_class_id,sum,isEmpty,sum] - CometHashAggregate [i_category_id,sum,isEmpty,sum] [channel,i_brand_id,i_class_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel,i_brand_id] #22 - CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,sum,isEmpty,sum] - CometHashAggregate [i_class_id,i_category_id,sum,isEmpty,sum] [channel,i_brand_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel] #23 - CometHashAggregate [sum_sales,number_sales] [channel,sum,isEmpty,sum] - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [channel,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange #24 - CometHashAggregate [sum_sales,number_sales] [sum,isEmpty,sum] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt deleted file mode 100644 index 4ac6c99894..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ /dev/null @@ -1,914 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (122) -+- CometTakeOrderedAndProject (121) - +- CometHashAggregate (120) - +- CometExchange (119) - +- CometHashAggregate (118) - +- CometUnion (117) - :- CometHashAggregate (96) - : +- CometExchange (95) - : +- CometHashAggregate (94) - : +- CometUnion (93) - : :- CometFilter (64) - : : +- CometHashAggregate (63) - : : +- CometExchange (62) - : : +- CometHashAggregate (61) - : : +- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometBroadcastHashJoin (47) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (46) - : : : : +- CometProject (45) - : : : : +- CometBroadcastHashJoin (44) - : : : : :- CometFilter (4) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : : +- CometBroadcastExchange (43) - : : : : +- CometBroadcastHashJoin (42) - : : : : :- CometHashAggregate (32) - : : : : : +- CometExchange (31) - : : : : : +- CometHashAggregate (30) - : : : : : +- CometProject (29) - : : : : : +- CometBroadcastHashJoin (28) - : : : : : :- CometProject (26) - : : : : : : +- CometBroadcastHashJoin (25) - : : : : : : :- CometFilter (6) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (5) - : : : : : : +- CometBroadcastExchange (24) - : : : : : : +- CometBroadcastHashJoin (23) - : : : : : : :- CometFilter (8) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (7) - : : : : : : +- CometBroadcastExchange (22) - : : : : : : +- CometProject (21) - : : : : : : +- CometBroadcastHashJoin (20) - : : : : : : :- CometProject (15) - : : : : : : : +- CometBroadcastHashJoin (14) - : : : : : : : :- CometFilter (10) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (9) - : : : : : : : +- CometBroadcastExchange (13) - : : : : : : : +- CometFilter (12) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (11) - : : : : : : +- CometBroadcastExchange (19) - : : : : : : +- CometProject (18) - : : : : : : +- CometFilter (17) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - : : : : : +- ReusedExchange (27) - : : : : +- CometBroadcastExchange (41) - : : : : +- CometProject (40) - : : : : +- CometBroadcastHashJoin (39) - : : : : :- CometProject (37) - : : : : : +- CometBroadcastHashJoin (36) - : : : : : :- CometFilter (34) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (33) - : : : : : +- ReusedExchange (35) - : : : : +- ReusedExchange (38) - : : : +- CometBroadcastExchange (52) - : : : +- CometBroadcastHashJoin (51) - : : : :- CometFilter (49) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (48) - : : : +- ReusedExchange (50) - : : +- CometBroadcastExchange (58) - : : +- CometProject (57) - : : +- CometFilter (56) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - : :- CometFilter (78) - : : +- CometHashAggregate (77) - : : +- CometExchange (76) - : : +- CometHashAggregate (75) - : : +- CometProject (74) - : : +- CometBroadcastHashJoin (73) - : : :- CometProject (71) - : : : +- CometBroadcastHashJoin (70) - : : : :- CometBroadcastHashJoin (68) - : : : : :- CometFilter (66) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (65) - : : : : +- ReusedExchange (67) - : : : +- ReusedExchange (69) - : : +- ReusedExchange (72) - : +- CometFilter (92) - : +- CometHashAggregate (91) - : +- CometExchange (90) - : +- CometHashAggregate (89) - : +- CometProject (88) - : +- CometBroadcastHashJoin (87) - : :- CometProject (85) - : : +- CometBroadcastHashJoin (84) - : : :- CometBroadcastHashJoin (82) - : : : :- CometFilter (80) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (79) - : : : +- ReusedExchange (81) - : : +- ReusedExchange (83) - : +- ReusedExchange (86) - :- CometHashAggregate (101) - : +- CometExchange (100) - : +- CometHashAggregate (99) - : +- CometHashAggregate (98) - : +- ReusedExchange (97) - :- CometHashAggregate (106) - : +- CometExchange (105) - : +- CometHashAggregate (104) - : +- CometHashAggregate (103) - : +- ReusedExchange (102) - :- CometHashAggregate (111) - : +- CometExchange (110) - : +- CometHashAggregate (109) - : +- CometHashAggregate (108) - : +- ReusedExchange (107) - +- CometHashAggregate (116) - +- CometExchange (115) - +- CometHashAggregate (114) - +- CometHashAggregate (113) - +- ReusedExchange (112) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) - -(7) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] -ReadSchema: struct - -(8) CometFilter -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Condition : isnotnull(i_item_sk#20) - -(13) CometBroadcastExchange -Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] - -(14) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Right output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_item_sk#17], [i_item_sk#20], Inner, BuildRight - -(15) CometProject -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23], [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) - -(18) CometProject -Input [2]: [d_date_sk#24, d_year#25] -Arguments: [d_date_sk#24], [d_date_sk#24] - -(19) CometBroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: [d_date_sk#24] - -(20) CometBroadcastHashJoin -Left output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] -Right output [1]: [d_date_sk#24] -Arguments: [cs_sold_date_sk#18], [d_date_sk#24], Inner, BuildRight - -(21) CometProject -Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23], [i_brand_id#21, i_class_id#22, i_category_id#23] - -(22) CometBroadcastExchange -Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [i_brand_id#21, i_class_id#22, i_category_id#23] - -(23) CometBroadcastHashJoin -Left output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] -Arguments: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)], [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)], LeftSemi, BuildRight - -(24) CometBroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] - -(25) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Right output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_item_sk#10], [i_item_sk#13], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16], [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] - -(27) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#26] - -(28) CometBroadcastHashJoin -Left output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Right output [1]: [d_date_sk#26] -Arguments: [ss_sold_date_sk#11], [d_date_sk#26], Inner, BuildRight - -(29) CometProject -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#26] -Arguments: [brand_id#27, class_id#28, category_id#29], [i_brand_id#14 AS brand_id#27, i_class_id#15 AS class_id#28, i_category_id#16 AS category_id#29] - -(30) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(31) CometExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: hashpartitioning(brand_id#27, class_id#28, category_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(32) CometHashAggregate -Input [3]: [brand_id#27, class_id#28, category_id#29] -Keys [3]: [brand_id#27, class_id#28, category_id#29] -Functions: [] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Condition : isnotnull(ws_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 13] -Output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] - -(36) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_sold_date_sk#31] -Right output [4]: [i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_item_sk#30], [i_item_sk#33], Inner, BuildRight - -(37) CometProject -Input [6]: [ws_item_sk#30, ws_sold_date_sk#31, i_item_sk#33, i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36], [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] - -(38) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#37] - -(39) CometBroadcastHashJoin -Left output [4]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36] -Right output [1]: [d_date_sk#37] -Arguments: [ws_sold_date_sk#31], [d_date_sk#37], Inner, BuildRight - -(40) CometProject -Input [5]: [ws_sold_date_sk#31, i_brand_id#34, i_class_id#35, i_category_id#36, d_date_sk#37] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36], [i_brand_id#34, i_class_id#35, i_category_id#36] - -(41) CometBroadcastExchange -Input [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [i_brand_id#34, i_class_id#35, i_category_id#36] - -(42) CometBroadcastHashJoin -Left output [3]: [brand_id#27, class_id#28, category_id#29] -Right output [3]: [i_brand_id#34, i_class_id#35, i_category_id#36] -Arguments: [coalesce(brand_id#27, 0), isnull(brand_id#27), coalesce(class_id#28, 0), isnull(class_id#28), coalesce(category_id#29, 0), isnull(category_id#29)], [coalesce(i_brand_id#34, 0), isnull(i_brand_id#34), coalesce(i_class_id#35, 0), isnull(i_class_id#35), coalesce(i_category_id#36, 0), isnull(i_category_id#36)], LeftSemi, BuildRight - -(43) CometBroadcastExchange -Input [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [brand_id#27, class_id#28, category_id#29] - -(44) CometBroadcastHashJoin -Left output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Right output [3]: [brand_id#27, class_id#28, category_id#29] -Arguments: [i_brand_id#7, i_class_id#8, i_category_id#9], [brand_id#27, class_id#28, category_id#29], Inner, BuildRight - -(45) CometProject -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#27, class_id#28, category_id#29] -Arguments: [ss_item_sk#38], [i_item_sk#6 AS ss_item_sk#38] - -(46) CometBroadcastExchange -Input [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#38] - -(47) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [1]: [ss_item_sk#38] -Arguments: [ss_item_sk#1], [ss_item_sk#38], LeftSemi, BuildRight - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Condition : isnotnull(i_item_sk#39) - -(50) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(51) CometBroadcastHashJoin -Left output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [ss_item_sk#38] -Arguments: [i_item_sk#39], [ss_item_sk#38], LeftSemi, BuildRight - -(52) CometBroadcastExchange -Input [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] - -(53) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Right output [4]: [i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_item_sk#1], [i_item_sk#39], Inner, BuildRight - -(54) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#39, i_brand_id#40, i_class_id#41, i_category_id#42] -Arguments: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(56) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(57) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(58) CometBroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: [d_date_sk#43] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42] -Right output [1]: [d_date_sk#43] -Arguments: [ss_sold_date_sk#4], [d_date_sk#43], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#40, i_class_id#41, i_category_id#42, d_date_sk#43] -Arguments: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42], [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] - -(61) CometHashAggregate -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#40, i_class_id#41, i_category_id#42] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] - -(62) CometExchange -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Arguments: hashpartitioning(i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(63) CometHashAggregate -Input [6]: [i_brand_id#40, i_class_id#41, i_category_id#42, sum#46, isEmpty#47, count#48] -Keys [3]: [i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] - -(64) CometFilter -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Condition : (isnotnull(sales#50) AND (cast(sales#50 as decimal(32,6)) > cast(Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#57), dynamicpruningexpression(cs_sold_date_sk#57 IN dynamicpruning#58)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Condition : isnotnull(cs_item_sk#54) - -(67) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(68) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Right output [1]: [ss_item_sk#38] -Arguments: [cs_item_sk#54], [ss_item_sk#38], LeftSemi, BuildRight - -(69) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] - -(70) CometBroadcastHashJoin -Left output [4]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57] -Right output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -Arguments: [cs_item_sk#54], [i_item_sk#59], Inner, BuildRight - -(71) CometProject -Input [8]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -Arguments: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62], [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62] - -(72) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#63] - -(73) CometBroadcastHashJoin -Left output [6]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62] -Right output [1]: [d_date_sk#63] -Arguments: [cs_sold_date_sk#57], [d_date_sk#63], Inner, BuildRight - -(74) CometProject -Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#60, i_class_id#61, i_category_id#62, d_date_sk#63] -Arguments: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62], [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] - -(75) CometHashAggregate -Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)] - -(76) CometExchange -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#64, isEmpty#65, count#66] -Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(77) CometHashAggregate -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#64, isEmpty#65, count#66] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)] - -(78) CometFilter -Input [6]: [channel#67, i_brand_id#60, i_class_id#61, i_category_id#62, sales#68, number_sales#69] -Condition : (isnotnull(sales#68) AND (cast(sales#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#70, ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#73), dynamicpruningexpression(ws_sold_date_sk#73 IN dynamicpruning#74)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(80) CometFilter -Input [4]: [ws_item_sk#70, ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73] -Condition : isnotnull(ws_item_sk#70) - -(81) ReusedExchange [Reuses operator id: 46] -Output [1]: [ss_item_sk#38] - -(82) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#70, ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73] -Right output [1]: [ss_item_sk#38] -Arguments: [ws_item_sk#70], [ss_item_sk#38], LeftSemi, BuildRight - -(83) ReusedExchange [Reuses operator id: 52] -Output [4]: [i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] - -(84) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#70, ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73] -Right output [4]: [i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] -Arguments: [ws_item_sk#70], [i_item_sk#75], Inner, BuildRight - -(85) CometProject -Input [8]: [ws_item_sk#70, ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73, i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] -Arguments: [ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78], [ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78] - -(86) ReusedExchange [Reuses operator id: 58] -Output [1]: [d_date_sk#79] - -(87) CometBroadcastHashJoin -Left output [6]: [ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78] -Right output [1]: [d_date_sk#79] -Arguments: [ws_sold_date_sk#73], [d_date_sk#79], Inner, BuildRight - -(88) CometProject -Input [7]: [ws_quantity#71, ws_list_price#72, ws_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78, d_date_sk#79] -Arguments: [ws_quantity#71, ws_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78], [ws_quantity#71, ws_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] - -(89) CometHashAggregate -Input [5]: [ws_quantity#71, ws_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] -Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] -Functions [2]: [partial_sum((cast(ws_quantity#71 as decimal(10,0)) * ws_list_price#72)), partial_count(1)] - -(90) CometExchange -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#80, isEmpty#81, count#82] -Arguments: hashpartitioning(i_brand_id#76, i_class_id#77, i_category_id#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(91) CometHashAggregate -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#80, isEmpty#81, count#82] -Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] -Functions [2]: [sum((cast(ws_quantity#71 as decimal(10,0)) * ws_list_price#72)), count(1)] - -(92) CometFilter -Input [6]: [channel#83, i_brand_id#76, i_class_id#77, i_category_id#78, sales#84, number_sales#85] -Condition : (isnotnull(sales#84) AND (cast(sales#84 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#52, [id=#53] as decimal(32,6)))) - -(93) CometUnion -Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Child 1 Input [6]: [channel#67, i_brand_id#60, i_class_id#61, i_category_id#62, sales#68, number_sales#69] -Child 2 Input [6]: [channel#83, i_brand_id#76, i_class_id#77, i_category_id#78, sales#84, number_sales#85] - -(94) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sales#50, number_sales#51] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [partial_sum(sales#50), partial_sum(number_sales#51)] - -(95) CometExchange -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] -Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(96) CometHashAggregate -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(97) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] - -(98) CometHashAggregate -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(99) CometHashAggregate -Input [5]: [channel#49, i_brand_id#40, i_class_id#41, sum_sales#89, number_sales#90] -Keys [3]: [channel#49, i_brand_id#40, i_class_id#41] -Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] - -(100) CometExchange -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, sum#91, isEmpty#92, sum#93] -Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(101) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, sum#91, isEmpty#92, sum#93] -Keys [3]: [channel#49, i_brand_id#40, i_class_id#41] -Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] - -(102) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] - -(103) CometHashAggregate -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(104) CometHashAggregate -Input [4]: [channel#49, i_brand_id#40, sum_sales#89, number_sales#90] -Keys [2]: [channel#49, i_brand_id#40] -Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] - -(105) CometExchange -Input [5]: [channel#49, i_brand_id#40, sum#94, isEmpty#95, sum#96] -Arguments: hashpartitioning(channel#49, i_brand_id#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(106) CometHashAggregate -Input [5]: [channel#49, i_brand_id#40, sum#94, isEmpty#95, sum#96] -Keys [2]: [channel#49, i_brand_id#40] -Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] - -(107) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] - -(108) CometHashAggregate -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(109) CometHashAggregate -Input [3]: [channel#49, sum_sales#89, number_sales#90] -Keys [1]: [channel#49] -Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] - -(110) CometExchange -Input [4]: [channel#49, sum#97, isEmpty#98, sum#99] -Arguments: hashpartitioning(channel#49, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(111) CometHashAggregate -Input [4]: [channel#49, sum#97, isEmpty#98, sum#99] -Keys [1]: [channel#49] -Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] - -(112) ReusedExchange [Reuses operator id: 95] -Output [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] - -(113) CometHashAggregate -Input [7]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum#86, isEmpty#87, sum#88] -Keys [4]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42] -Functions [2]: [sum(sales#50), sum(number_sales#51)] - -(114) CometHashAggregate -Input [2]: [sum_sales#89, number_sales#90] -Keys: [] -Functions [2]: [partial_sum(sum_sales#89), partial_sum(number_sales#90)] - -(115) CometExchange -Input [3]: [sum#100, isEmpty#101, sum#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(116) CometHashAggregate -Input [3]: [sum#100, isEmpty#101, sum#102] -Keys: [] -Functions [2]: [sum(sum_sales#89), sum(number_sales#90)] - -(117) CometUnion -Child 0 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] -Child 1 Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#103, sum(sum_sales)#104, sum(number_sales)#105] -Child 2 Input [6]: [channel#49, i_brand_id#40, i_class_id#106, i_category_id#107, sum(sum_sales)#108, sum(number_sales)#109] -Child 3 Input [6]: [channel#49, i_brand_id#110, i_class_id#111, i_category_id#112, sum(sum_sales)#113, sum(number_sales)#114] -Child 4 Input [6]: [channel#115, i_brand_id#116, i_class_id#117, i_category_id#118, sum(sum_sales)#119, sum(number_sales)#120] - -(118) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] -Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] -Functions: [] - -(119) CometExchange -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] -Arguments: hashpartitioning(channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(120) CometHashAggregate -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] -Keys [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] -Functions: [] - -(121) CometTakeOrderedAndProject -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#49 ASC NULLS FIRST,i_brand_id#40 ASC NULLS FIRST,i_class_id#41 ASC NULLS FIRST,i_category_id#42 ASC NULLS FIRST], output=[channel#49,i_brand_id#40,i_class_id#41,i_category_id#42,sum_sales#89,number_sales#90]), [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90], 100, 0, [channel#49 ASC NULLS FIRST, i_brand_id#40 ASC NULLS FIRST, i_class_id#41 ASC NULLS FIRST, i_category_id#42 ASC NULLS FIRST], [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] - -(122) CometColumnarToRow [codegen id : 1] -Input [6]: [channel#49, i_brand_id#40, i_class_id#41, i_category_id#42, sum_sales#89, number_sales#90] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#52, [id=#53] -* CometColumnarToRow (142) -+- CometHashAggregate (141) - +- CometExchange (140) - +- CometHashAggregate (139) - +- CometUnion (138) - :- CometProject (126) - : +- CometBroadcastHashJoin (125) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (123) - : +- ReusedExchange (124) - :- CometProject (133) - : +- CometBroadcastHashJoin (132) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (127) - : +- CometBroadcastExchange (131) - : +- CometProject (130) - : +- CometFilter (129) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (128) - +- CometProject (137) - +- CometBroadcastHashJoin (136) - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (134) - +- ReusedExchange (135) - - -(123) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_quantity#121, ss_list_price#122, ss_sold_date_sk#123] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#123), dynamicpruningexpression(ss_sold_date_sk#123 IN dynamicpruning#124)] -ReadSchema: struct - -(124) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#125] - -(125) CometBroadcastHashJoin -Left output [3]: [ss_quantity#121, ss_list_price#122, ss_sold_date_sk#123] -Right output [1]: [d_date_sk#125] -Arguments: [ss_sold_date_sk#123], [d_date_sk#125], Inner, BuildRight - -(126) CometProject -Input [4]: [ss_quantity#121, ss_list_price#122, ss_sold_date_sk#123, d_date_sk#125] -Arguments: [quantity#126, list_price#127], [ss_quantity#121 AS quantity#126, ss_list_price#122 AS list_price#127] - -(127) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_quantity#128, cs_list_price#129, cs_sold_date_sk#130] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#130), dynamicpruningexpression(cs_sold_date_sk#130 IN dynamicpruning#131)] -ReadSchema: struct - -(128) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#132, d_year#133] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(129) CometFilter -Input [2]: [d_date_sk#132, d_year#133] -Condition : (((isnotnull(d_year#133) AND (d_year#133 >= 1998)) AND (d_year#133 <= 2000)) AND isnotnull(d_date_sk#132)) - -(130) CometProject -Input [2]: [d_date_sk#132, d_year#133] -Arguments: [d_date_sk#132], [d_date_sk#132] - -(131) CometBroadcastExchange -Input [1]: [d_date_sk#132] -Arguments: [d_date_sk#132] - -(132) CometBroadcastHashJoin -Left output [3]: [cs_quantity#128, cs_list_price#129, cs_sold_date_sk#130] -Right output [1]: [d_date_sk#132] -Arguments: [cs_sold_date_sk#130], [d_date_sk#132], Inner, BuildRight - -(133) CometProject -Input [4]: [cs_quantity#128, cs_list_price#129, cs_sold_date_sk#130, d_date_sk#132] -Arguments: [quantity#134, list_price#135], [cs_quantity#128 AS quantity#134, cs_list_price#129 AS list_price#135] - -(134) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_quantity#136, ws_list_price#137, ws_sold_date_sk#138] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#138), dynamicpruningexpression(ws_sold_date_sk#138 IN dynamicpruning#139)] -ReadSchema: struct - -(135) ReusedExchange [Reuses operator id: 131] -Output [1]: [d_date_sk#140] - -(136) CometBroadcastHashJoin -Left output [3]: [ws_quantity#136, ws_list_price#137, ws_sold_date_sk#138] -Right output [1]: [d_date_sk#140] -Arguments: [ws_sold_date_sk#138], [d_date_sk#140], Inner, BuildRight - -(137) CometProject -Input [4]: [ws_quantity#136, ws_list_price#137, ws_sold_date_sk#138, d_date_sk#140] -Arguments: [quantity#141, list_price#142], [ws_quantity#136 AS quantity#141, ws_list_price#137 AS list_price#142] - -(138) CometUnion -Child 0 Input [2]: [quantity#126, list_price#127] -Child 1 Input [2]: [quantity#134, list_price#135] -Child 2 Input [2]: [quantity#141, list_price#142] - -(139) CometHashAggregate -Input [2]: [quantity#126, list_price#127] -Keys: [] -Functions [1]: [partial_avg((cast(quantity#126 as decimal(10,0)) * list_price#127))] - -(140) CometExchange -Input [2]: [sum#143, count#144] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(141) CometHashAggregate -Input [2]: [sum#143, count#144] -Keys: [] -Functions [1]: [avg((cast(quantity#126 as decimal(10,0)) * list_price#127))] - -(142) CometColumnarToRow [codegen id : 1] -Input [1]: [average_sales#145] - -Subquery:2 Hosting operator id = 123 Hosting Expression = ss_sold_date_sk#123 IN dynamicpruning#12 - -Subquery:3 Hosting operator id = 127 Hosting Expression = cs_sold_date_sk#130 IN dynamicpruning#131 -BroadcastExchange (147) -+- * CometColumnarToRow (146) - +- CometProject (145) - +- CometFilter (144) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (143) - - -(143) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#132, d_year#133] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(144) CometFilter -Input [2]: [d_date_sk#132, d_year#133] -Condition : (((isnotnull(d_year#133) AND (d_year#133 >= 1998)) AND (d_year#133 <= 2000)) AND isnotnull(d_date_sk#132)) - -(145) CometProject -Input [2]: [d_date_sk#132, d_year#133] -Arguments: [d_date_sk#132], [d_date_sk#132] - -(146) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#132] - -(147) BroadcastExchange -Input [1]: [d_date_sk#132] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -Subquery:4 Hosting operator id = 134 Hosting Expression = ws_sold_date_sk#138 IN dynamicpruning#131 - -Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (152) -+- * CometColumnarToRow (151) - +- CometProject (150) - +- CometFilter (149) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (148) - - -(148) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct - -(149) CometFilter -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#43)) - -(150) CometProject -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Arguments: [d_date_sk#43], [d_date_sk#43] - -(151) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#43] - -(152) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:6 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (157) -+- * CometColumnarToRow (156) - +- CometProject (155) - +- CometFilter (154) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (153) - - -(153) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#146] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(154) CometFilter -Input [2]: [d_date_sk#26, d_year#146] -Condition : (((isnotnull(d_year#146) AND (d_year#146 >= 1999)) AND (d_year#146 <= 2001)) AND isnotnull(d_date_sk#26)) - -(155) CometProject -Input [2]: [d_date_sk#26, d_year#146] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(156) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(157) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:7 Hosting operator id = 9 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 - -Subquery:8 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#12 - -Subquery:9 Hosting operator id = 78 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:10 Hosting operator id = 65 Hosting Expression = cs_sold_date_sk#57 IN dynamicpruning#5 - -Subquery:11 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#52, [id=#53] - -Subquery:12 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#73 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/extended.txt deleted file mode 100644 index 83deb45432..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/extended.txt +++ /dev/null @@ -1,2350 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometFilter - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 2127 out of 2302 eligible operators (92%). Final plan contains 46 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 758e917335..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ /dev/null @@ -1,179 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometExchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometUnion [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - CometExchange [channel,i_brand_id,i_class_id,i_category_id] #2 - CometHashAggregate [sales,number_sales] [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] - CometUnion [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - Subquery #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,count] [average_sales,avg((cast(quantity as decimal(10,0)) * list_price))] - CometExchange #16 - CometHashAggregate [quantity,list_price] [sum,count] - CometUnion [quantity,list_price] - CometProject [ss_quantity,ss_list_price] [quantity,list_price] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_quantity,cs_list_price,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #4 - BroadcastExchange #17 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #18 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometProject [ws_quantity,ws_list_price] [quantity,list_price] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #4 - ReusedExchange [d_date_sk] #18 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #3 - CometHashAggregate [ss_quantity,ss_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk,ss_item_sk] - CometFilter [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [ss_item_sk] #5 - CometProject [i_item_sk] [ss_item_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [brand_id,class_id,category_id] #6 - CometBroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - CometHashAggregate [brand_id,class_id,category_id] - CometExchange [brand_id,class_id,category_id] #7 - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - 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,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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - 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 [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastExchange [d_date_sk] #12 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk] #12 - CometBroadcastExchange [i_brand_id,i_class_id,i_category_id] #13 - CometProject [i_brand_id,i_class_id,i_category_id] - 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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk] #12 - CometBroadcastExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_class_id,i_category_id,ss_item_sk] - CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - ReusedExchange [ss_item_sk] #5 - CometBroadcastExchange [d_date_sk] #15 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #19 - CometHashAggregate [cs_quantity,cs_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk,ss_item_sk] - CometFilter [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #5 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - ReusedExchange [d_date_sk] #15 - CometFilter [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - ReusedSubquery [average_sales] #3 - CometHashAggregate [sum,isEmpty,count] [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales,sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1)] - CometExchange [i_brand_id,i_class_id,i_category_id] #20 - CometHashAggregate [ws_quantity,ws_list_price] [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] - CometProject [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,d_date_sk] - CometProject [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,i_item_sk,i_brand_id,i_class_id,i_category_id] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk,ss_item_sk] - CometFilter [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [ss_item_sk] #5 - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 - ReusedExchange [d_date_sk] #15 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel,i_brand_id,i_class_id] #21 - CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,i_class_id,sum,isEmpty,sum] - CometHashAggregate [i_category_id,sum,isEmpty,sum] [channel,i_brand_id,i_class_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel,i_brand_id] #22 - CometHashAggregate [sum_sales,number_sales] [channel,i_brand_id,sum,isEmpty,sum] - CometHashAggregate [i_class_id,i_category_id,sum,isEmpty,sum] [channel,i_brand_id,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange [channel] #23 - CometHashAggregate [sum_sales,number_sales] [channel,sum,isEmpty,sum] - CometHashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [channel,sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 - CometHashAggregate [sum,isEmpty,sum] [channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum(sum_sales),sum(number_salesL)] - CometExchange #24 - CometHashAggregate [sum_sales,number_sales] [sum,isEmpty,sum] - CometHashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum_sales,number_sales,sum(sales),sum(number_salesL)] - ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/explain.txt deleted file mode 100644 index 6f109523cf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/explain.txt +++ /dev/null @@ -1,944 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (161) -+- Union (160) - :- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- CometColumnarExchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (33) - : : +- * BroadcastHashJoin Inner BuildRight (32) - : : :- * Project (30) - : : : +- * BroadcastHashJoin Inner BuildRight (29) - : : : :- * Project (23) - : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : :- * Project (17) - : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : :- * Project (10) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : :- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : : : +- BroadcastExchange (8) - : : : : : : +- * CometColumnarToRow (7) - : : : : : : +- CometProject (6) - : : : : : : +- CometFilter (5) - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) - : : : : : +- BroadcastExchange (15) - : : : : : +- * CometColumnarToRow (14) - : : : : : +- CometProject (13) - : : : : : +- CometFilter (12) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (11) - : : : : +- BroadcastExchange (21) - : : : : +- * CometColumnarToRow (20) - : : : : +- CometFilter (19) - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (18) - : : : +- BroadcastExchange (28) - : : : +- * CometColumnarToRow (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (24) - : : +- ReusedExchange (31) - : +- BroadcastExchange (38) - : +- * CometColumnarToRow (37) - : +- CometProject (36) - : +- CometFilter (35) - : +- CometNativeScan parquet spark_catalog.default.item (34) - :- * HashAggregate (73) - : +- * CometColumnarToRow (72) - : +- CometColumnarExchange (71) - : +- * HashAggregate (70) - : +- * Project (69) - : +- * BroadcastHashJoin Inner BuildRight (68) - : :- * Project (66) - : : +- * BroadcastHashJoin Inner BuildRight (65) - : : :- * Project (63) - : : : +- * BroadcastHashJoin Inner BuildRight (62) - : : : :- * Project (56) - : : : : +- * BroadcastHashJoin Inner BuildRight (55) - : : : : :- * Project (53) - : : : : : +- * BroadcastHashJoin Inner BuildRight (52) - : : : : : :- * Project (50) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (49) - : : : : : : :- * Filter (47) - : : : : : : : +- * ColumnarToRow (46) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (45) - : : : : : : +- ReusedExchange (48) - : : : : : +- ReusedExchange (51) - : : : : +- ReusedExchange (54) - : : : +- BroadcastExchange (61) - : : : +- * CometColumnarToRow (60) - : : : +- CometProject (59) - : : : +- CometFilter (58) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (57) - : : +- ReusedExchange (64) - : +- ReusedExchange (67) - :- * HashAggregate (102) - : +- * CometColumnarToRow (101) - : +- CometColumnarExchange (100) - : +- * HashAggregate (99) - : +- * Project (98) - : +- * BroadcastHashJoin Inner BuildRight (97) - : :- * Project (95) - : : +- * BroadcastHashJoin Inner BuildRight (94) - : : :- * Project (92) - : : : +- * BroadcastHashJoin Inner BuildRight (91) - : : : :- * Project (85) - : : : : +- * BroadcastHashJoin Inner BuildRight (84) - : : : : :- * Project (82) - : : : : : +- * BroadcastHashJoin Inner BuildRight (81) - : : : : : :- * Project (79) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) - : : : : : : :- * Filter (76) - : : : : : : : +- * ColumnarToRow (75) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (74) - : : : : : : +- ReusedExchange (77) - : : : : : +- ReusedExchange (80) - : : : : +- ReusedExchange (83) - : : : +- BroadcastExchange (90) - : : : +- * CometColumnarToRow (89) - : : : +- CometProject (88) - : : : +- CometFilter (87) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (86) - : : +- ReusedExchange (93) - : +- ReusedExchange (96) - :- * HashAggregate (131) - : +- * CometColumnarToRow (130) - : +- CometColumnarExchange (129) - : +- * HashAggregate (128) - : +- * Project (127) - : +- * BroadcastHashJoin Inner BuildRight (126) - : :- * Project (124) - : : +- * BroadcastHashJoin Inner BuildRight (123) - : : :- * Project (121) - : : : +- * BroadcastHashJoin Inner BuildRight (120) - : : : :- * Project (114) - : : : : +- * BroadcastHashJoin Inner BuildRight (113) - : : : : :- * Project (111) - : : : : : +- * BroadcastHashJoin Inner BuildRight (110) - : : : : : :- * Project (108) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (107) - : : : : : : :- * Filter (105) - : : : : : : : +- * ColumnarToRow (104) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (103) - : : : : : : +- ReusedExchange (106) - : : : : : +- ReusedExchange (109) - : : : : +- ReusedExchange (112) - : : : +- BroadcastExchange (119) - : : : +- * CometColumnarToRow (118) - : : : +- CometProject (117) - : : : +- CometFilter (116) - : : : +- CometNativeScan parquet spark_catalog.default.customer_address (115) - : : +- ReusedExchange (122) - : +- ReusedExchange (125) - +- * HashAggregate (159) - +- * CometColumnarToRow (158) - +- CometColumnarExchange (157) - +- * HashAggregate (156) - +- * Project (155) - +- * BroadcastHashJoin Inner BuildRight (154) - :- * Project (149) - : +- * BroadcastHashJoin Inner BuildRight (148) - : :- * Project (146) - : : +- * BroadcastHashJoin Inner BuildRight (145) - : : :- * Project (143) - : : : +- * BroadcastHashJoin Inner BuildRight (142) - : : : :- * Project (140) - : : : : +- * BroadcastHashJoin Inner BuildRight (139) - : : : : :- * Project (137) - : : : : : +- * BroadcastHashJoin Inner BuildRight (136) - : : : : : :- * Filter (134) - : : : : : : +- * ColumnarToRow (133) - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (132) - : : : : : +- ReusedExchange (135) - : : : : +- ReusedExchange (138) - : : : +- ReusedExchange (141) - : : +- ReusedExchange (144) - : +- ReusedExchange (147) - +- BroadcastExchange (153) - +- * CometColumnarToRow (152) - +- CometFilter (151) - +- CometNativeScan parquet spark_catalog.default.item (150) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 7] -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] - -(3) Filter [codegen id : 7] -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) - -(4) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#12, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#11)) - -(6) CometProject -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] - -(7) CometColumnarToRow [codegen id : 1] -Input [2]: [cd_demo_sk#11, cd_dep_count#14] - -(8) BroadcastExchange -Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 7] -Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] - -(11) CometNativeScan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [In(c_birth_month, [1,10,12,4,5,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(12) CometFilter -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Condition : (((c_birth_month#18 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) - -(13) CometProject -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(14) CometColumnarToRow [codegen id : 2] -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(15) BroadcastExchange -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(16) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#15] -Join type: Inner -Join condition: None - -(17) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(18) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(19) CometFilter -Input [1]: [cd_demo_sk#20] -Condition : isnotnull(cd_demo_sk#20) - -(20) CometColumnarToRow [codegen id : 3] -Input [1]: [cd_demo_sk#20] - -(21) BroadcastExchange -Input [1]: [cd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#16] -Right keys [1]: [cd_demo_sk#20] -Join type: Inner -Join condition: None - -(23) Project [codegen id : 7] -Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] - -(24) CometNativeScan parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(25) CometFilter -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) - -(26) CometProject -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) AS ca_state#25, ca_country#24] - -(27) CometColumnarToRow [codegen id : 4] -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(28) BroadcastExchange -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(29) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#17] -Right keys [1]: [ca_address_sk#21] -Join type: Inner -Join condition: None - -(30) Project [codegen id : 7] -Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(31) ReusedExchange [Reuses operator id: 166] -Output [1]: [d_date_sk#26] - -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#26] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 7] -Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] - -(34) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#27, i_item_id#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [2]: [i_item_sk#27, i_item_id#28] -Condition : isnotnull(i_item_sk#27) - -(36) CometProject -Input [2]: [i_item_sk#27, i_item_id#28] -Arguments: [i_item_sk#27, i_item_id#29], [i_item_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#28, 16, true, false, true) AS i_item_id#29] - -(37) CometColumnarToRow [codegen id : 6] -Input [2]: [i_item_sk#27, i_item_id#29] - -(38) BroadcastExchange -Input [2]: [i_item_sk#27, i_item_id#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#27] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 7] -Output [11]: [i_item_id#29 AS i_item_id#30, ca_country#24 AS ca_country#31, ca_state#25 AS ca_state#32, ca_county#22 AS ca_county#33, cast(cs_quantity#4 as decimal(12,2)) AS agg1#34, cast(cs_list_price#5 as decimal(12,2)) AS agg2#35, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#36, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#37, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#38, cast(c_birth_year#19 as decimal(12,2)) AS agg6#39, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#40] -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#27, i_item_id#29] - -(41) HashAggregate [codegen id : 7] -Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#34, agg2#35, agg3#36, agg4#37, agg5#38, agg6#39, agg7#40] -Keys [4]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33] -Functions [7]: [partial_avg(agg1#34), partial_avg(agg2#35), partial_avg(agg3#36), partial_avg(agg4#37), partial_avg(agg5#38), partial_avg(agg6#39), partial_avg(agg7#40)] -Aggregate Attributes [14]: [sum#41, count#42, sum#43, count#44, sum#45, count#46, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54] -Results [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] - -(42) CometColumnarExchange -Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] -Arguments: hashpartitioning(i_item_id#30, ca_country#31, ca_state#32, ca_county#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) CometColumnarToRow [codegen id : 8] -Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] - -(44) HashAggregate [codegen id : 8] -Input [18]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] -Keys [4]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33] -Functions [7]: [avg(agg1#34), avg(agg2#35), avg(agg3#36), avg(agg4#37), avg(agg5#38), avg(agg6#39), avg(agg7#40)] -Aggregate Attributes [7]: [avg(agg1#34)#69, avg(agg2#35)#70, avg(agg3#36)#71, avg(agg4#37)#72, avg(agg5#38)#73, avg(agg6#39)#74, avg(agg7#40)#75] -Results [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, avg(agg1#34)#69 AS agg1#76, avg(agg2#35)#70 AS agg2#77, avg(agg3#36)#71 AS agg3#78, avg(agg4#37)#72 AS agg4#79, avg(agg5#38)#73 AS agg5#80, avg(agg6#39)#74 AS agg6#81, avg(agg7#40)#75 AS agg7#82] - -(45) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#91), dynamicpruningexpression(cs_sold_date_sk#91 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 15] -Input [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] - -(47) Filter [codegen id : 15] -Input [9]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91] -Condition : ((isnotnull(cs_bill_cdemo_sk#84) AND isnotnull(cs_bill_customer_sk#83)) AND isnotnull(cs_item_sk#85)) - -(48) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#92, cd_dep_count#93] - -(49) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_cdemo_sk#84] -Right keys [1]: [cd_demo_sk#92] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 15] -Output [9]: [cs_bill_customer_sk#83, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93] -Input [11]: [cs_bill_customer_sk#83, cs_bill_cdemo_sk#84, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_demo_sk#92, cd_dep_count#93] - -(51) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#94, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97] - -(52) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_customer_sk#83] -Right keys [1]: [c_customer_sk#94] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 15] -Output [11]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97] -Input [13]: [cs_bill_customer_sk#83, cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_customer_sk#94, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97] - -(54) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#98] - -(55) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_cdemo_sk#95] -Right keys [1]: [cd_demo_sk#98] -Join type: Inner -Join condition: None - -(56) Project [codegen id : 15] -Output [10]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_addr_sk#96, c_birth_year#97] -Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_cdemo_sk#95, c_current_addr_sk#96, c_birth_year#97, cd_demo_sk#98] - -(57) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#99, ca_state#100, ca_country#101] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [ca_address_sk#99, ca_state#100, ca_country#101] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#100, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#99)) - -(59) CometProject -Input [3]: [ca_address_sk#99, ca_state#100, ca_country#101] -Arguments: [ca_address_sk#99, ca_state#25, ca_country#101], [ca_address_sk#99, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#100, 2, true, false, true) AS ca_state#25, ca_country#101] - -(60) CometColumnarToRow [codegen id : 12] -Input [3]: [ca_address_sk#99, ca_state#25, ca_country#101] - -(61) BroadcastExchange -Input [3]: [ca_address_sk#99, ca_state#25, ca_country#101] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -(62) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_addr_sk#96] -Right keys [1]: [ca_address_sk#99] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 15] -Output [11]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_birth_year#97, ca_state#25, ca_country#101] -Input [13]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_current_addr_sk#96, c_birth_year#97, ca_address_sk#99, ca_state#25, ca_country#101] - -(64) ReusedExchange [Reuses operator id: 166] -Output [1]: [d_date_sk#102] - -(65) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_sold_date_sk#91] -Right keys [1]: [d_date_sk#102] -Join type: Inner -Join condition: None - -(66) Project [codegen id : 15] -Output [10]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#93, c_birth_year#97, ca_state#25, ca_country#101] -Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cs_sold_date_sk#91, cd_dep_count#93, c_birth_year#97, ca_state#25, ca_country#101, d_date_sk#102] - -(67) ReusedExchange [Reuses operator id: 38] -Output [2]: [i_item_sk#103, i_item_id#29] - -(68) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_item_sk#85] -Right keys [1]: [i_item_sk#103] -Join type: Inner -Join condition: None - -(69) Project [codegen id : 15] -Output [10]: [i_item_id#29, ca_country#101, ca_state#25, cast(cs_quantity#86 as decimal(12,2)) AS agg1#34, cast(cs_list_price#87 as decimal(12,2)) AS agg2#35, cast(cs_coupon_amt#89 as decimal(12,2)) AS agg3#36, cast(cs_sales_price#88 as decimal(12,2)) AS agg4#37, cast(cs_net_profit#90 as decimal(12,2)) AS agg5#38, cast(c_birth_year#97 as decimal(12,2)) AS agg6#39, cast(cd_dep_count#93 as decimal(12,2)) AS agg7#40] -Input [12]: [cs_item_sk#85, cs_quantity#86, cs_list_price#87, cs_sales_price#88, cs_coupon_amt#89, cs_net_profit#90, cd_dep_count#93, c_birth_year#97, ca_state#25, ca_country#101, i_item_sk#103, i_item_id#29] - -(70) HashAggregate [codegen id : 15] -Input [10]: [i_item_id#29, ca_country#101, ca_state#25, agg1#34, agg2#35, agg3#36, agg4#37, agg5#38, agg6#39, agg7#40] -Keys [3]: [i_item_id#29, ca_country#101, ca_state#25] -Functions [7]: [partial_avg(agg1#34), partial_avg(agg2#35), partial_avg(agg3#36), partial_avg(agg4#37), partial_avg(agg5#38), partial_avg(agg6#39), partial_avg(agg7#40)] -Aggregate Attributes [14]: [sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] -Results [17]: [i_item_id#29, ca_country#101, ca_state#25, sum#118, count#119, sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131] - -(71) CometColumnarExchange -Input [17]: [i_item_id#29, ca_country#101, ca_state#25, sum#118, count#119, sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131] -Arguments: hashpartitioning(i_item_id#29, ca_country#101, ca_state#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(72) CometColumnarToRow [codegen id : 16] -Input [17]: [i_item_id#29, ca_country#101, ca_state#25, sum#118, count#119, sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131] - -(73) HashAggregate [codegen id : 16] -Input [17]: [i_item_id#29, ca_country#101, ca_state#25, sum#118, count#119, sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131] -Keys [3]: [i_item_id#29, ca_country#101, ca_state#25] -Functions [7]: [avg(agg1#34), avg(agg2#35), avg(agg3#36), avg(agg4#37), avg(agg5#38), avg(agg6#39), avg(agg7#40)] -Aggregate Attributes [7]: [avg(agg1#34)#132, avg(agg2#35)#133, avg(agg3#36)#134, avg(agg4#37)#135, avg(agg5#38)#136, avg(agg6#39)#137, avg(agg7#40)#138] -Results [11]: [i_item_id#29, ca_country#101, ca_state#25, null AS county#139, avg(agg1#34)#132 AS agg1#140, avg(agg2#35)#133 AS agg2#141, avg(agg3#36)#134 AS agg3#142, avg(agg4#37)#135 AS agg4#143, avg(agg5#38)#136 AS agg5#144, avg(agg6#39)#137 AS agg6#145, avg(agg7#40)#138 AS agg7#146] - -(74) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#147, cs_bill_cdemo_sk#148, cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#155), dynamicpruningexpression(cs_sold_date_sk#155 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(75) ColumnarToRow [codegen id : 23] -Input [9]: [cs_bill_customer_sk#147, cs_bill_cdemo_sk#148, cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155] - -(76) Filter [codegen id : 23] -Input [9]: [cs_bill_customer_sk#147, cs_bill_cdemo_sk#148, cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155] -Condition : ((isnotnull(cs_bill_cdemo_sk#148) AND isnotnull(cs_bill_customer_sk#147)) AND isnotnull(cs_item_sk#149)) - -(77) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#156, cd_dep_count#157] - -(78) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_cdemo_sk#148] -Right keys [1]: [cd_demo_sk#156] -Join type: Inner -Join condition: None - -(79) Project [codegen id : 23] -Output [9]: [cs_bill_customer_sk#147, cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155, cd_dep_count#157] -Input [11]: [cs_bill_customer_sk#147, cs_bill_cdemo_sk#148, cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155, cd_demo_sk#156, cd_dep_count#157] - -(80) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#158, c_current_cdemo_sk#159, c_current_addr_sk#160, c_birth_year#161] - -(81) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_customer_sk#147] -Right keys [1]: [c_customer_sk#158] -Join type: Inner -Join condition: None - -(82) Project [codegen id : 23] -Output [11]: [cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155, cd_dep_count#157, c_current_cdemo_sk#159, c_current_addr_sk#160, c_birth_year#161] -Input [13]: [cs_bill_customer_sk#147, cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155, cd_dep_count#157, c_customer_sk#158, c_current_cdemo_sk#159, c_current_addr_sk#160, c_birth_year#161] - -(83) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#162] - -(84) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_cdemo_sk#159] -Right keys [1]: [cd_demo_sk#162] -Join type: Inner -Join condition: None - -(85) Project [codegen id : 23] -Output [10]: [cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155, cd_dep_count#157, c_current_addr_sk#160, c_birth_year#161] -Input [12]: [cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155, cd_dep_count#157, c_current_cdemo_sk#159, c_current_addr_sk#160, c_birth_year#161, cd_demo_sk#162] - -(86) CometNativeScan parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#163, ca_state#164, ca_country#165] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(87) CometFilter -Input [3]: [ca_address_sk#163, ca_state#164, ca_country#165] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#164, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#163)) - -(88) CometProject -Input [3]: [ca_address_sk#163, ca_state#164, ca_country#165] -Arguments: [ca_address_sk#163, ca_country#165], [ca_address_sk#163, ca_country#165] - -(89) CometColumnarToRow [codegen id : 20] -Input [2]: [ca_address_sk#163, ca_country#165] - -(90) BroadcastExchange -Input [2]: [ca_address_sk#163, ca_country#165] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -(91) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_addr_sk#160] -Right keys [1]: [ca_address_sk#163] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 23] -Output [10]: [cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155, cd_dep_count#157, c_birth_year#161, ca_country#165] -Input [12]: [cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155, cd_dep_count#157, c_current_addr_sk#160, c_birth_year#161, ca_address_sk#163, ca_country#165] - -(93) ReusedExchange [Reuses operator id: 166] -Output [1]: [d_date_sk#166] - -(94) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_sold_date_sk#155] -Right keys [1]: [d_date_sk#166] -Join type: Inner -Join condition: None - -(95) Project [codegen id : 23] -Output [9]: [cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cd_dep_count#157, c_birth_year#161, ca_country#165] -Input [11]: [cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cs_sold_date_sk#155, cd_dep_count#157, c_birth_year#161, ca_country#165, d_date_sk#166] - -(96) ReusedExchange [Reuses operator id: 38] -Output [2]: [i_item_sk#167, i_item_id#29] - -(97) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_item_sk#149] -Right keys [1]: [i_item_sk#167] -Join type: Inner -Join condition: None - -(98) Project [codegen id : 23] -Output [9]: [i_item_id#29, ca_country#165, cast(cs_quantity#150 as decimal(12,2)) AS agg1#34, cast(cs_list_price#151 as decimal(12,2)) AS agg2#35, cast(cs_coupon_amt#153 as decimal(12,2)) AS agg3#36, cast(cs_sales_price#152 as decimal(12,2)) AS agg4#37, cast(cs_net_profit#154 as decimal(12,2)) AS agg5#38, cast(c_birth_year#161 as decimal(12,2)) AS agg6#39, cast(cd_dep_count#157 as decimal(12,2)) AS agg7#40] -Input [11]: [cs_item_sk#149, cs_quantity#150, cs_list_price#151, cs_sales_price#152, cs_coupon_amt#153, cs_net_profit#154, cd_dep_count#157, c_birth_year#161, ca_country#165, i_item_sk#167, i_item_id#29] - -(99) HashAggregate [codegen id : 23] -Input [9]: [i_item_id#29, ca_country#165, agg1#34, agg2#35, agg3#36, agg4#37, agg5#38, agg6#39, agg7#40] -Keys [2]: [i_item_id#29, ca_country#165] -Functions [7]: [partial_avg(agg1#34), partial_avg(agg2#35), partial_avg(agg3#36), partial_avg(agg4#37), partial_avg(agg5#38), partial_avg(agg6#39), partial_avg(agg7#40)] -Aggregate Attributes [14]: [sum#168, count#169, sum#170, count#171, sum#172, count#173, sum#174, count#175, sum#176, count#177, sum#178, count#179, sum#180, count#181] -Results [16]: [i_item_id#29, ca_country#165, sum#182, count#183, sum#184, count#185, sum#186, count#187, sum#188, count#189, sum#190, count#191, sum#192, count#193, sum#194, count#195] - -(100) CometColumnarExchange -Input [16]: [i_item_id#29, ca_country#165, sum#182, count#183, sum#184, count#185, sum#186, count#187, sum#188, count#189, sum#190, count#191, sum#192, count#193, sum#194, count#195] -Arguments: hashpartitioning(i_item_id#29, ca_country#165, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(101) CometColumnarToRow [codegen id : 24] -Input [16]: [i_item_id#29, ca_country#165, sum#182, count#183, sum#184, count#185, sum#186, count#187, sum#188, count#189, sum#190, count#191, sum#192, count#193, sum#194, count#195] - -(102) HashAggregate [codegen id : 24] -Input [16]: [i_item_id#29, ca_country#165, sum#182, count#183, sum#184, count#185, sum#186, count#187, sum#188, count#189, sum#190, count#191, sum#192, count#193, sum#194, count#195] -Keys [2]: [i_item_id#29, ca_country#165] -Functions [7]: [avg(agg1#34), avg(agg2#35), avg(agg3#36), avg(agg4#37), avg(agg5#38), avg(agg6#39), avg(agg7#40)] -Aggregate Attributes [7]: [avg(agg1#34)#196, avg(agg2#35)#197, avg(agg3#36)#198, avg(agg4#37)#199, avg(agg5#38)#200, avg(agg6#39)#201, avg(agg7#40)#202] -Results [11]: [i_item_id#29, ca_country#165, null AS ca_state#203, null AS county#204, avg(agg1#34)#196 AS agg1#205, avg(agg2#35)#197 AS agg2#206, avg(agg3#36)#198 AS agg3#207, avg(agg4#37)#199 AS agg4#208, avg(agg5#38)#200 AS agg5#209, avg(agg6#39)#201 AS agg6#210, avg(agg7#40)#202 AS agg7#211] - -(103) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#212, cs_bill_cdemo_sk#213, cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#220), dynamicpruningexpression(cs_sold_date_sk#220 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(104) ColumnarToRow [codegen id : 31] -Input [9]: [cs_bill_customer_sk#212, cs_bill_cdemo_sk#213, cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220] - -(105) Filter [codegen id : 31] -Input [9]: [cs_bill_customer_sk#212, cs_bill_cdemo_sk#213, cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220] -Condition : ((isnotnull(cs_bill_cdemo_sk#213) AND isnotnull(cs_bill_customer_sk#212)) AND isnotnull(cs_item_sk#214)) - -(106) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#221, cd_dep_count#222] - -(107) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_cdemo_sk#213] -Right keys [1]: [cd_demo_sk#221] -Join type: Inner -Join condition: None - -(108) Project [codegen id : 31] -Output [9]: [cs_bill_customer_sk#212, cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220, cd_dep_count#222] -Input [11]: [cs_bill_customer_sk#212, cs_bill_cdemo_sk#213, cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220, cd_demo_sk#221, cd_dep_count#222] - -(109) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#223, c_current_cdemo_sk#224, c_current_addr_sk#225, c_birth_year#226] - -(110) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_customer_sk#212] -Right keys [1]: [c_customer_sk#223] -Join type: Inner -Join condition: None - -(111) Project [codegen id : 31] -Output [11]: [cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220, cd_dep_count#222, c_current_cdemo_sk#224, c_current_addr_sk#225, c_birth_year#226] -Input [13]: [cs_bill_customer_sk#212, cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220, cd_dep_count#222, c_customer_sk#223, c_current_cdemo_sk#224, c_current_addr_sk#225, c_birth_year#226] - -(112) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#227] - -(113) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_cdemo_sk#224] -Right keys [1]: [cd_demo_sk#227] -Join type: Inner -Join condition: None - -(114) Project [codegen id : 31] -Output [10]: [cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220, cd_dep_count#222, c_current_addr_sk#225, c_birth_year#226] -Input [12]: [cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220, cd_dep_count#222, c_current_cdemo_sk#224, c_current_addr_sk#225, c_birth_year#226, cd_demo_sk#227] - -(115) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#228, ca_state#229] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(116) CometFilter -Input [2]: [ca_address_sk#228, ca_state#229] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#229, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#228)) - -(117) CometProject -Input [2]: [ca_address_sk#228, ca_state#229] -Arguments: [ca_address_sk#228], [ca_address_sk#228] - -(118) CometColumnarToRow [codegen id : 28] -Input [1]: [ca_address_sk#228] - -(119) BroadcastExchange -Input [1]: [ca_address_sk#228] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -(120) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_addr_sk#225] -Right keys [1]: [ca_address_sk#228] -Join type: Inner -Join condition: None - -(121) Project [codegen id : 31] -Output [9]: [cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220, cd_dep_count#222, c_birth_year#226] -Input [11]: [cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220, cd_dep_count#222, c_current_addr_sk#225, c_birth_year#226, ca_address_sk#228] - -(122) ReusedExchange [Reuses operator id: 166] -Output [1]: [d_date_sk#230] - -(123) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_sold_date_sk#220] -Right keys [1]: [d_date_sk#230] -Join type: Inner -Join condition: None - -(124) Project [codegen id : 31] -Output [8]: [cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cd_dep_count#222, c_birth_year#226] -Input [10]: [cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cs_sold_date_sk#220, cd_dep_count#222, c_birth_year#226, d_date_sk#230] - -(125) ReusedExchange [Reuses operator id: 38] -Output [2]: [i_item_sk#231, i_item_id#29] - -(126) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_item_sk#214] -Right keys [1]: [i_item_sk#231] -Join type: Inner -Join condition: None - -(127) Project [codegen id : 31] -Output [8]: [i_item_id#29, cast(cs_quantity#215 as decimal(12,2)) AS agg1#34, cast(cs_list_price#216 as decimal(12,2)) AS agg2#35, cast(cs_coupon_amt#218 as decimal(12,2)) AS agg3#36, cast(cs_sales_price#217 as decimal(12,2)) AS agg4#37, cast(cs_net_profit#219 as decimal(12,2)) AS agg5#38, cast(c_birth_year#226 as decimal(12,2)) AS agg6#39, cast(cd_dep_count#222 as decimal(12,2)) AS agg7#40] -Input [10]: [cs_item_sk#214, cs_quantity#215, cs_list_price#216, cs_sales_price#217, cs_coupon_amt#218, cs_net_profit#219, cd_dep_count#222, c_birth_year#226, i_item_sk#231, i_item_id#29] - -(128) HashAggregate [codegen id : 31] -Input [8]: [i_item_id#29, agg1#34, agg2#35, agg3#36, agg4#37, agg5#38, agg6#39, agg7#40] -Keys [1]: [i_item_id#29] -Functions [7]: [partial_avg(agg1#34), partial_avg(agg2#35), partial_avg(agg3#36), partial_avg(agg4#37), partial_avg(agg5#38), partial_avg(agg6#39), partial_avg(agg7#40)] -Aggregate Attributes [14]: [sum#232, count#233, sum#234, count#235, sum#236, count#237, sum#238, count#239, sum#240, count#241, sum#242, count#243, sum#244, count#245] -Results [15]: [i_item_id#29, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255, sum#256, count#257, sum#258, count#259] - -(129) CometColumnarExchange -Input [15]: [i_item_id#29, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255, sum#256, count#257, sum#258, count#259] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(130) CometColumnarToRow [codegen id : 32] -Input [15]: [i_item_id#29, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255, sum#256, count#257, sum#258, count#259] - -(131) HashAggregate [codegen id : 32] -Input [15]: [i_item_id#29, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255, sum#256, count#257, sum#258, count#259] -Keys [1]: [i_item_id#29] -Functions [7]: [avg(agg1#34), avg(agg2#35), avg(agg3#36), avg(agg4#37), avg(agg5#38), avg(agg6#39), avg(agg7#40)] -Aggregate Attributes [7]: [avg(agg1#34)#260, avg(agg2#35)#261, avg(agg3#36)#262, avg(agg4#37)#263, avg(agg5#38)#264, avg(agg6#39)#265, avg(agg7#40)#266] -Results [11]: [i_item_id#29, null AS ca_country#267, null AS ca_state#268, null AS county#269, avg(agg1#34)#260 AS agg1#270, avg(agg2#35)#261 AS agg2#271, avg(agg3#36)#262 AS agg3#272, avg(agg4#37)#263 AS agg4#273, avg(agg5#38)#264 AS agg5#274, avg(agg6#39)#265 AS agg6#275, avg(agg7#40)#266 AS agg7#276] - -(132) Scan parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#277, cs_bill_cdemo_sk#278, cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#285), dynamicpruningexpression(cs_sold_date_sk#285 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(133) ColumnarToRow [codegen id : 39] -Input [9]: [cs_bill_customer_sk#277, cs_bill_cdemo_sk#278, cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285] - -(134) Filter [codegen id : 39] -Input [9]: [cs_bill_customer_sk#277, cs_bill_cdemo_sk#278, cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285] -Condition : ((isnotnull(cs_bill_cdemo_sk#278) AND isnotnull(cs_bill_customer_sk#277)) AND isnotnull(cs_item_sk#279)) - -(135) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#286, cd_dep_count#287] - -(136) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_cdemo_sk#278] -Right keys [1]: [cd_demo_sk#286] -Join type: Inner -Join condition: None - -(137) Project [codegen id : 39] -Output [9]: [cs_bill_customer_sk#277, cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285, cd_dep_count#287] -Input [11]: [cs_bill_customer_sk#277, cs_bill_cdemo_sk#278, cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285, cd_demo_sk#286, cd_dep_count#287] - -(138) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#288, c_current_cdemo_sk#289, c_current_addr_sk#290, c_birth_year#291] - -(139) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_customer_sk#277] -Right keys [1]: [c_customer_sk#288] -Join type: Inner -Join condition: None - -(140) Project [codegen id : 39] -Output [11]: [cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285, cd_dep_count#287, c_current_cdemo_sk#289, c_current_addr_sk#290, c_birth_year#291] -Input [13]: [cs_bill_customer_sk#277, cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285, cd_dep_count#287, c_customer_sk#288, c_current_cdemo_sk#289, c_current_addr_sk#290, c_birth_year#291] - -(141) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#292] - -(142) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_cdemo_sk#289] -Right keys [1]: [cd_demo_sk#292] -Join type: Inner -Join condition: None - -(143) Project [codegen id : 39] -Output [10]: [cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285, cd_dep_count#287, c_current_addr_sk#290, c_birth_year#291] -Input [12]: [cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285, cd_dep_count#287, c_current_cdemo_sk#289, c_current_addr_sk#290, c_birth_year#291, cd_demo_sk#292] - -(144) ReusedExchange [Reuses operator id: 119] -Output [1]: [ca_address_sk#293] - -(145) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_addr_sk#290] -Right keys [1]: [ca_address_sk#293] -Join type: Inner -Join condition: None - -(146) Project [codegen id : 39] -Output [9]: [cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285, cd_dep_count#287, c_birth_year#291] -Input [11]: [cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285, cd_dep_count#287, c_current_addr_sk#290, c_birth_year#291, ca_address_sk#293] - -(147) ReusedExchange [Reuses operator id: 166] -Output [1]: [d_date_sk#294] - -(148) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_sold_date_sk#285] -Right keys [1]: [d_date_sk#294] -Join type: Inner -Join condition: None - -(149) Project [codegen id : 39] -Output [8]: [cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cd_dep_count#287, c_birth_year#291] -Input [10]: [cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cs_sold_date_sk#285, cd_dep_count#287, c_birth_year#291, d_date_sk#294] - -(150) CometNativeScan parquet spark_catalog.default.item -Output [1]: [i_item_sk#295] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(151) CometFilter -Input [1]: [i_item_sk#295] -Condition : isnotnull(i_item_sk#295) - -(152) CometColumnarToRow [codegen id : 38] -Input [1]: [i_item_sk#295] - -(153) BroadcastExchange -Input [1]: [i_item_sk#295] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] - -(154) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_item_sk#279] -Right keys [1]: [i_item_sk#295] -Join type: Inner -Join condition: None - -(155) Project [codegen id : 39] -Output [7]: [cast(cs_quantity#280 as decimal(12,2)) AS agg1#34, cast(cs_list_price#281 as decimal(12,2)) AS agg2#35, cast(cs_coupon_amt#283 as decimal(12,2)) AS agg3#36, cast(cs_sales_price#282 as decimal(12,2)) AS agg4#37, cast(cs_net_profit#284 as decimal(12,2)) AS agg5#38, cast(c_birth_year#291 as decimal(12,2)) AS agg6#39, cast(cd_dep_count#287 as decimal(12,2)) AS agg7#40] -Input [9]: [cs_item_sk#279, cs_quantity#280, cs_list_price#281, cs_sales_price#282, cs_coupon_amt#283, cs_net_profit#284, cd_dep_count#287, c_birth_year#291, i_item_sk#295] - -(156) HashAggregate [codegen id : 39] -Input [7]: [agg1#34, agg2#35, agg3#36, agg4#37, agg5#38, agg6#39, agg7#40] -Keys: [] -Functions [7]: [partial_avg(agg1#34), partial_avg(agg2#35), partial_avg(agg3#36), partial_avg(agg4#37), partial_avg(agg5#38), partial_avg(agg6#39), partial_avg(agg7#40)] -Aggregate Attributes [14]: [sum#296, count#297, sum#298, count#299, sum#300, count#301, sum#302, count#303, sum#304, count#305, sum#306, count#307, sum#308, count#309] -Results [14]: [sum#310, count#311, sum#312, count#313, sum#314, count#315, sum#316, count#317, sum#318, count#319, sum#320, count#321, sum#322, count#323] - -(157) CometColumnarExchange -Input [14]: [sum#310, count#311, sum#312, count#313, sum#314, count#315, sum#316, count#317, sum#318, count#319, sum#320, count#321, sum#322, count#323] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(158) CometColumnarToRow [codegen id : 40] -Input [14]: [sum#310, count#311, sum#312, count#313, sum#314, count#315, sum#316, count#317, sum#318, count#319, sum#320, count#321, sum#322, count#323] - -(159) HashAggregate [codegen id : 40] -Input [14]: [sum#310, count#311, sum#312, count#313, sum#314, count#315, sum#316, count#317, sum#318, count#319, sum#320, count#321, sum#322, count#323] -Keys: [] -Functions [7]: [avg(agg1#34), avg(agg2#35), avg(agg3#36), avg(agg4#37), avg(agg5#38), avg(agg6#39), avg(agg7#40)] -Aggregate Attributes [7]: [avg(agg1#34)#324, avg(agg2#35)#325, avg(agg3#36)#326, avg(agg4#37)#327, avg(agg5#38)#328, avg(agg6#39)#329, avg(agg7#40)#330] -Results [11]: [null AS i_item_id#331, null AS ca_country#332, null AS ca_state#333, null AS county#334, avg(agg1#34)#324 AS agg1#335, avg(agg2#35)#325 AS agg2#336, avg(agg3#36)#326 AS agg3#337, avg(agg4#37)#327 AS agg4#338, avg(agg5#38)#328 AS agg5#339, avg(agg6#39)#329 AS agg6#340, avg(agg7#40)#330 AS agg7#341] - -(160) Union - -(161) TakeOrderedAndProject -Input [11]: [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] -Arguments: 100, [ca_country#31 ASC NULLS FIRST, ca_state#32 ASC NULLS FIRST, ca_county#33 ASC NULLS FIRST, i_item_id#30 ASC NULLS FIRST], [i_item_id#30, ca_country#31, ca_state#32, ca_county#33, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (166) -+- * CometColumnarToRow (165) - +- CometProject (164) - +- CometFilter (163) - +- CometNativeScan parquet spark_catalog.default.date_dim (162) - - -(162) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#342] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(163) CometFilter -Input [2]: [d_date_sk#26, d_year#342] -Condition : ((isnotnull(d_year#342) AND (d_year#342 = 2001)) AND isnotnull(d_date_sk#26)) - -(164) CometProject -Input [2]: [d_date_sk#26, d_year#342] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(165) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(166) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] - -Subquery:2 Hosting operator id = 45 Hosting Expression = cs_sold_date_sk#91 IN dynamicpruning#10 - -Subquery:3 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#155 IN dynamicpruning#10 - -Subquery:4 Hosting operator id = 103 Hosting Expression = cs_sold_date_sk#220 IN dynamicpruning#10 - -Subquery:5 Hosting operator id = 132 Hosting Expression = cs_sold_date_sk#285 IN dynamicpruning#10 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/simplified.txt deleted file mode 100644 index 71745258b5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/simplified.txt +++ /dev/null @@ -1,241 +0,0 @@ -TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - Union - WholeStageCodegen (8) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,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),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,ca_country,ca_state,ca_county] #1 - WholeStageCodegen (7) - HashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [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] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cd_demo_sk] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - WholeStageCodegen (16) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,ca_country,ca_state] #8 - WholeStageCodegen (15) - HashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,ca_state,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [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] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (24) - HashAggregate [i_item_id,ca_country,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),ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,ca_country] #10 - WholeStageCodegen (23) - HashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ca_country,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [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] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (32) - HashAggregate [i_item_id,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),ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #12 - WholeStageCodegen (31) - HashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [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] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (28) - CometColumnarToRow - InputAdapter - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #7 - WholeStageCodegen (40) - HashAggregate [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),i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #14 - WholeStageCodegen (39) - HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [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] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Filter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [cd_demo_sk,cd_dep_count] #3 - InputAdapter - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 - InputAdapter - ReusedExchange [cd_demo_sk] #5 - InputAdapter - ReusedExchange [ca_address_sk] #13 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (38) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/explain.txt deleted file mode 100644 index 86072fd830..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,846 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (146) -+- CometTakeOrderedAndProject (145) - +- CometUnion (144) - :- CometHashAggregate (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (19) - : : : : +- CometBroadcastHashJoin (18) - : : : : :- CometProject (14) - : : : : : +- CometBroadcastHashJoin (13) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : +- CometBroadcastExchange (6) - : : : : : : +- CometProject (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : : : : +- CometBroadcastExchange (12) - : : : : : +- CometProject (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : : : : +- CometBroadcastExchange (17) - : : : : +- CometFilter (16) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (15) - : : : +- CometBroadcastExchange (23) - : : : +- CometProject (22) - : : : +- CometFilter (21) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - : : +- CometBroadcastExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) - :- CometHashAggregate (66) - : +- CometExchange (65) - : +- CometHashAggregate (64) - : +- CometProject (63) - : +- CometBroadcastHashJoin (62) - : :- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (57) - : : : +- CometBroadcastHashJoin (56) - : : : :- CometProject (51) - : : : : +- CometBroadcastHashJoin (50) - : : : : :- CometProject (48) - : : : : : +- CometBroadcastHashJoin (47) - : : : : : :- CometProject (45) - : : : : : : +- CometBroadcastHashJoin (44) - : : : : : : :- CometFilter (42) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (41) - : : : : : : +- ReusedExchange (43) - : : : : : +- ReusedExchange (46) - : : : : +- ReusedExchange (49) - : : : +- CometBroadcastExchange (55) - : : : +- CometProject (54) - : : : +- CometFilter (53) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (52) - : : +- ReusedExchange (58) - : +- ReusedExchange (61) - :- CometHashAggregate (92) - : +- CometExchange (91) - : +- CometHashAggregate (90) - : +- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometProject (86) - : : +- CometBroadcastHashJoin (85) - : : :- CometProject (83) - : : : +- CometBroadcastHashJoin (82) - : : : :- CometProject (77) - : : : : +- CometBroadcastHashJoin (76) - : : : : :- CometProject (74) - : : : : : +- CometBroadcastHashJoin (73) - : : : : : :- CometProject (71) - : : : : : : +- CometBroadcastHashJoin (70) - : : : : : : :- CometFilter (68) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) - : : : : : : +- ReusedExchange (69) - : : : : : +- ReusedExchange (72) - : : : : +- ReusedExchange (75) - : : : +- CometBroadcastExchange (81) - : : : +- CometProject (80) - : : : +- CometFilter (79) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (78) - : : +- ReusedExchange (84) - : +- ReusedExchange (87) - :- CometHashAggregate (118) - : +- CometExchange (117) - : +- CometHashAggregate (116) - : +- CometProject (115) - : +- CometBroadcastHashJoin (114) - : :- CometProject (112) - : : +- CometBroadcastHashJoin (111) - : : :- CometProject (109) - : : : +- CometBroadcastHashJoin (108) - : : : :- CometProject (103) - : : : : +- CometBroadcastHashJoin (102) - : : : : :- CometProject (100) - : : : : : +- CometBroadcastHashJoin (99) - : : : : : :- CometProject (97) - : : : : : : +- CometBroadcastHashJoin (96) - : : : : : : :- CometFilter (94) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (93) - : : : : : : +- ReusedExchange (95) - : : : : : +- ReusedExchange (98) - : : : : +- ReusedExchange (101) - : : : +- CometBroadcastExchange (107) - : : : +- CometProject (106) - : : : +- CometFilter (105) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (104) - : : +- ReusedExchange (110) - : +- ReusedExchange (113) - +- CometHashAggregate (143) - +- CometExchange (142) - +- CometHashAggregate (141) - +- CometProject (140) - +- CometBroadcastHashJoin (139) - :- CometProject (135) - : +- CometBroadcastHashJoin (134) - : :- CometProject (132) - : : +- CometBroadcastHashJoin (131) - : : :- CometProject (129) - : : : +- CometBroadcastHashJoin (128) - : : : :- CometProject (126) - : : : : +- CometBroadcastHashJoin (125) - : : : : :- CometProject (123) - : : : : : +- CometBroadcastHashJoin (122) - : : : : : :- CometFilter (120) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (119) - : : : : : +- ReusedExchange (121) - : : : : +- ReusedExchange (124) - : : : +- ReusedExchange (127) - : : +- ReusedExchange (130) - : +- ReusedExchange (133) - +- CometBroadcastExchange (138) - +- CometFilter (137) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (136) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#12, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#11)) - -(5) CometProject -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] - -(6) CometBroadcastExchange -Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14] - -(7) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Right output [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight - -(8) CometProject -Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [In(c_birth_month, [1,10,12,4,5,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Condition : (((c_birth_month#18 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) - -(11) CometProject -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(12) CometBroadcastExchange -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(13) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight - -(14) CometProject -Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [cd_demo_sk#20] -Condition : isnotnull(cd_demo_sk#20) - -(17) CometBroadcastExchange -Input [1]: [cd_demo_sk#20] -Arguments: [cd_demo_sk#20] - -(18) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Right output [1]: [cd_demo_sk#20] -Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight - -(19) CometProject -Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) - -(22) CometProject -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) AS ca_state#25, ca_country#24] - -(23) CometBroadcastExchange -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(24) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -Right output [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight - -(25) CometProject -Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#26)) - -(28) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(29) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] - -(30) CometBroadcastHashJoin -Left output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [1]: [d_date_sk#26] -Arguments: [cs_sold_date_sk#9], [d_date_sk#26], Inner, BuildRight - -(31) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#28, i_item_id#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [i_item_sk#28, i_item_id#29] -Condition : isnotnull(i_item_sk#28) - -(34) CometProject -Input [2]: [i_item_sk#28, i_item_id#29] -Arguments: [i_item_sk#28, i_item_id#30], [i_item_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#29, 16, true, false, true) AS i_item_id#30] - -(35) CometBroadcastExchange -Input [2]: [i_item_sk#28, i_item_id#30] -Arguments: [i_item_sk#28, i_item_id#30] - -(36) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [2]: [i_item_sk#28, i_item_id#30] -Arguments: [cs_item_sk#3], [i_item_sk#28], Inner, BuildRight - -(37) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#28, i_item_id#30] -Arguments: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41], [i_item_id#30 AS i_item_id#31, ca_country#24 AS ca_country#32, ca_state#25 AS ca_state#33, ca_county#22 AS ca_county#34, cast(cs_quantity#4 as decimal(12,2)) AS agg1#35, cast(cs_list_price#5 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#39, cast(c_birth_year#19 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] - -(38) CometHashAggregate -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [4]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34] -Functions [7]: [partial_avg(agg1#35), partial_avg(agg2#36), partial_avg(agg3#37), partial_avg(agg4#38), partial_avg(agg5#39), partial_avg(agg6#40), partial_avg(agg7#41)] - -(39) CometExchange -Input [18]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] -Arguments: hashpartitioning(i_item_id#31, ca_country#32, ca_state#33, ca_county#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(40) CometHashAggregate -Input [18]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] -Keys [4]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34] -Functions [7]: [avg(agg1#35), avg(agg2#36), avg(agg3#37), avg(agg4#38), avg(agg5#39), avg(agg6#40), avg(agg7#41)] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#64), dynamicpruningexpression(cs_sold_date_sk#64 IN dynamicpruning#65)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(42) CometFilter -Input [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] -Condition : ((isnotnull(cs_bill_cdemo_sk#57) AND isnotnull(cs_bill_customer_sk#56)) AND isnotnull(cs_item_sk#58)) - -(43) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#66, cd_dep_count#67] - -(44) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] -Right output [2]: [cd_demo_sk#66, cd_dep_count#67] -Arguments: [cs_bill_cdemo_sk#57], [cd_demo_sk#66], Inner, BuildRight - -(45) CometProject -Input [11]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_demo_sk#66, cd_dep_count#67] -Arguments: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67], [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67] - -(46) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] - -(47) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67] -Right output [4]: [c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Arguments: [cs_bill_customer_sk#56], [c_customer_sk#68], Inner, BuildRight - -(48) CometProject -Input [13]: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] - -(49) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#72] - -(50) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Right output [1]: [cd_demo_sk#72] -Arguments: [c_current_cdemo_sk#69], [cd_demo_sk#72], Inner, BuildRight - -(51) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71, cd_demo_sk#72] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71] - -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#73, ca_state#74, ca_country#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [ca_address_sk#73, ca_state#74, ca_country#75] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#74, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#73)) - -(54) CometProject -Input [3]: [ca_address_sk#73, ca_state#74, ca_country#75] -Arguments: [ca_address_sk#73, ca_state#25, ca_country#75], [ca_address_sk#73, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#74, 2, true, false, true) AS ca_state#25, ca_country#75] - -(55) CometBroadcastExchange -Input [3]: [ca_address_sk#73, ca_state#25, ca_country#75] -Arguments: [ca_address_sk#73, ca_state#25, ca_country#75] - -(56) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71] -Right output [3]: [ca_address_sk#73, ca_state#25, ca_country#75] -Arguments: [c_current_addr_sk#70], [ca_address_sk#73], Inner, BuildRight - -(57) CometProject -Input [13]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71, ca_address_sk#73, ca_state#25, ca_country#75] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] - -(58) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#76] - -(59) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] -Right output [1]: [d_date_sk#76] -Arguments: [cs_sold_date_sk#64], [d_date_sk#76], Inner, BuildRight - -(60) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75, d_date_sk#76] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] - -(61) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#77, i_item_id#30] - -(62) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] -Right output [2]: [i_item_sk#77, i_item_id#30] -Arguments: [cs_item_sk#58], [i_item_sk#77], Inner, BuildRight - -(63) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75, i_item_sk#77, i_item_id#30] -Arguments: [i_item_id#30, ca_country#75, ca_state#25, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41], [i_item_id#30, ca_country#75, ca_state#25, cast(cs_quantity#59 as decimal(12,2)) AS agg1#35, cast(cs_list_price#60 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#62 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#61 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#63 as decimal(12,2)) AS agg5#39, cast(c_birth_year#71 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#67 as decimal(12,2)) AS agg7#41] - -(64) CometHashAggregate -Input [10]: [i_item_id#30, ca_country#75, ca_state#25, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [3]: [i_item_id#30, ca_country#75, ca_state#25] -Functions [7]: [partial_avg(agg1#35), partial_avg(agg2#36), partial_avg(agg3#37), partial_avg(agg4#38), partial_avg(agg5#39), partial_avg(agg6#40), partial_avg(agg7#41)] - -(65) CometExchange -Input [17]: [i_item_id#30, ca_country#75, ca_state#25, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] -Arguments: hashpartitioning(i_item_id#30, ca_country#75, ca_state#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(66) CometHashAggregate -Input [17]: [i_item_id#30, ca_country#75, ca_state#25, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] -Keys [3]: [i_item_id#30, ca_country#75, ca_state#25] -Functions [7]: [avg(agg1#35), avg(agg2#36), avg(agg3#37), avg(agg4#38), avg(agg5#39), avg(agg6#40), avg(agg7#41)] - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#92, cs_bill_cdemo_sk#93, cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#100), dynamicpruningexpression(cs_sold_date_sk#100 IN dynamicpruning#101)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(68) CometFilter -Input [9]: [cs_bill_customer_sk#92, cs_bill_cdemo_sk#93, cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100] -Condition : ((isnotnull(cs_bill_cdemo_sk#93) AND isnotnull(cs_bill_customer_sk#92)) AND isnotnull(cs_item_sk#94)) - -(69) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#102, cd_dep_count#103] - -(70) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#92, cs_bill_cdemo_sk#93, cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100] -Right output [2]: [cd_demo_sk#102, cd_dep_count#103] -Arguments: [cs_bill_cdemo_sk#93], [cd_demo_sk#102], Inner, BuildRight - -(71) CometProject -Input [11]: [cs_bill_customer_sk#92, cs_bill_cdemo_sk#93, cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_demo_sk#102, cd_dep_count#103] -Arguments: [cs_bill_customer_sk#92, cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103], [cs_bill_customer_sk#92, cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103] - -(72) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#104, c_current_cdemo_sk#105, c_current_addr_sk#106, c_birth_year#107] - -(73) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#92, cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103] -Right output [4]: [c_customer_sk#104, c_current_cdemo_sk#105, c_current_addr_sk#106, c_birth_year#107] -Arguments: [cs_bill_customer_sk#92], [c_customer_sk#104], Inner, BuildRight - -(74) CometProject -Input [13]: [cs_bill_customer_sk#92, cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_customer_sk#104, c_current_cdemo_sk#105, c_current_addr_sk#106, c_birth_year#107] -Arguments: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_current_cdemo_sk#105, c_current_addr_sk#106, c_birth_year#107], [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_current_cdemo_sk#105, c_current_addr_sk#106, c_birth_year#107] - -(75) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#108] - -(76) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_current_cdemo_sk#105, c_current_addr_sk#106, c_birth_year#107] -Right output [1]: [cd_demo_sk#108] -Arguments: [c_current_cdemo_sk#105], [cd_demo_sk#108], Inner, BuildRight - -(77) CometProject -Input [12]: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_current_cdemo_sk#105, c_current_addr_sk#106, c_birth_year#107, cd_demo_sk#108] -Arguments: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_current_addr_sk#106, c_birth_year#107], [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_current_addr_sk#106, c_birth_year#107] - -(78) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#109, ca_state#110, ca_country#111] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(79) CometFilter -Input [3]: [ca_address_sk#109, ca_state#110, ca_country#111] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#110, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#109)) - -(80) CometProject -Input [3]: [ca_address_sk#109, ca_state#110, ca_country#111] -Arguments: [ca_address_sk#109, ca_country#111], [ca_address_sk#109, ca_country#111] - -(81) CometBroadcastExchange -Input [2]: [ca_address_sk#109, ca_country#111] -Arguments: [ca_address_sk#109, ca_country#111] - -(82) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_current_addr_sk#106, c_birth_year#107] -Right output [2]: [ca_address_sk#109, ca_country#111] -Arguments: [c_current_addr_sk#106], [ca_address_sk#109], Inner, BuildRight - -(83) CometProject -Input [12]: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_current_addr_sk#106, c_birth_year#107, ca_address_sk#109, ca_country#111] -Arguments: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_birth_year#107, ca_country#111], [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_birth_year#107, ca_country#111] - -(84) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#112] - -(85) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_birth_year#107, ca_country#111] -Right output [1]: [d_date_sk#112] -Arguments: [cs_sold_date_sk#100], [d_date_sk#112], Inner, BuildRight - -(86) CometProject -Input [11]: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_birth_year#107, ca_country#111, d_date_sk#112] -Arguments: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cd_dep_count#103, c_birth_year#107, ca_country#111], [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cd_dep_count#103, c_birth_year#107, ca_country#111] - -(87) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#113, i_item_id#30] - -(88) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cd_dep_count#103, c_birth_year#107, ca_country#111] -Right output [2]: [i_item_sk#113, i_item_id#30] -Arguments: [cs_item_sk#94], [i_item_sk#113], Inner, BuildRight - -(89) CometProject -Input [11]: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cd_dep_count#103, c_birth_year#107, ca_country#111, i_item_sk#113, i_item_id#30] -Arguments: [i_item_id#30, ca_country#111, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41], [i_item_id#30, ca_country#111, cast(cs_quantity#95 as decimal(12,2)) AS agg1#35, cast(cs_list_price#96 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#98 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#97 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#99 as decimal(12,2)) AS agg5#39, cast(c_birth_year#107 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#103 as decimal(12,2)) AS agg7#41] - -(90) CometHashAggregate -Input [9]: [i_item_id#30, ca_country#111, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [2]: [i_item_id#30, ca_country#111] -Functions [7]: [partial_avg(agg1#35), partial_avg(agg2#36), partial_avg(agg3#37), partial_avg(agg4#38), partial_avg(agg5#39), partial_avg(agg6#40), partial_avg(agg7#41)] - -(91) CometExchange -Input [16]: [i_item_id#30, ca_country#111, sum#114, count#115, sum#116, count#117, sum#118, count#119, sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#126, count#127] -Arguments: hashpartitioning(i_item_id#30, ca_country#111, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(92) CometHashAggregate -Input [16]: [i_item_id#30, ca_country#111, sum#114, count#115, sum#116, count#117, sum#118, count#119, sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#126, count#127] -Keys [2]: [i_item_id#30, ca_country#111] -Functions [7]: [avg(agg1#35), avg(agg2#36), avg(agg3#37), avg(agg4#38), avg(agg5#39), avg(agg6#40), avg(agg7#41)] - -(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#128, cs_bill_cdemo_sk#129, cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#136), dynamicpruningexpression(cs_sold_date_sk#136 IN dynamicpruning#137)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(94) CometFilter -Input [9]: [cs_bill_customer_sk#128, cs_bill_cdemo_sk#129, cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136] -Condition : ((isnotnull(cs_bill_cdemo_sk#129) AND isnotnull(cs_bill_customer_sk#128)) AND isnotnull(cs_item_sk#130)) - -(95) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#138, cd_dep_count#139] - -(96) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#128, cs_bill_cdemo_sk#129, cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136] -Right output [2]: [cd_demo_sk#138, cd_dep_count#139] -Arguments: [cs_bill_cdemo_sk#129], [cd_demo_sk#138], Inner, BuildRight - -(97) CometProject -Input [11]: [cs_bill_customer_sk#128, cs_bill_cdemo_sk#129, cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_demo_sk#138, cd_dep_count#139] -Arguments: [cs_bill_customer_sk#128, cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139], [cs_bill_customer_sk#128, cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139] - -(98) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#140, c_current_cdemo_sk#141, c_current_addr_sk#142, c_birth_year#143] - -(99) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#128, cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139] -Right output [4]: [c_customer_sk#140, c_current_cdemo_sk#141, c_current_addr_sk#142, c_birth_year#143] -Arguments: [cs_bill_customer_sk#128], [c_customer_sk#140], Inner, BuildRight - -(100) CometProject -Input [13]: [cs_bill_customer_sk#128, cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_customer_sk#140, c_current_cdemo_sk#141, c_current_addr_sk#142, c_birth_year#143] -Arguments: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_current_cdemo_sk#141, c_current_addr_sk#142, c_birth_year#143], [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_current_cdemo_sk#141, c_current_addr_sk#142, c_birth_year#143] - -(101) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#144] - -(102) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_current_cdemo_sk#141, c_current_addr_sk#142, c_birth_year#143] -Right output [1]: [cd_demo_sk#144] -Arguments: [c_current_cdemo_sk#141], [cd_demo_sk#144], Inner, BuildRight - -(103) CometProject -Input [12]: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_current_cdemo_sk#141, c_current_addr_sk#142, c_birth_year#143, cd_demo_sk#144] -Arguments: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_current_addr_sk#142, c_birth_year#143], [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_current_addr_sk#142, c_birth_year#143] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#145, ca_state#146] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [ca_address_sk#145, ca_state#146] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#146, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#145)) - -(106) CometProject -Input [2]: [ca_address_sk#145, ca_state#146] -Arguments: [ca_address_sk#145], [ca_address_sk#145] - -(107) CometBroadcastExchange -Input [1]: [ca_address_sk#145] -Arguments: [ca_address_sk#145] - -(108) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_current_addr_sk#142, c_birth_year#143] -Right output [1]: [ca_address_sk#145] -Arguments: [c_current_addr_sk#142], [ca_address_sk#145], Inner, BuildRight - -(109) CometProject -Input [11]: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_current_addr_sk#142, c_birth_year#143, ca_address_sk#145] -Arguments: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_birth_year#143], [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_birth_year#143] - -(110) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#147] - -(111) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_birth_year#143] -Right output [1]: [d_date_sk#147] -Arguments: [cs_sold_date_sk#136], [d_date_sk#147], Inner, BuildRight - -(112) CometProject -Input [10]: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_birth_year#143, d_date_sk#147] -Arguments: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cd_dep_count#139, c_birth_year#143], [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cd_dep_count#139, c_birth_year#143] - -(113) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#148, i_item_id#30] - -(114) CometBroadcastHashJoin -Left output [8]: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cd_dep_count#139, c_birth_year#143] -Right output [2]: [i_item_sk#148, i_item_id#30] -Arguments: [cs_item_sk#130], [i_item_sk#148], Inner, BuildRight - -(115) CometProject -Input [10]: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cd_dep_count#139, c_birth_year#143, i_item_sk#148, i_item_id#30] -Arguments: [i_item_id#30, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41], [i_item_id#30, cast(cs_quantity#131 as decimal(12,2)) AS agg1#35, cast(cs_list_price#132 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#134 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#133 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#135 as decimal(12,2)) AS agg5#39, cast(c_birth_year#143 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#139 as decimal(12,2)) AS agg7#41] - -(116) CometHashAggregate -Input [8]: [i_item_id#30, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [1]: [i_item_id#30] -Functions [7]: [partial_avg(agg1#35), partial_avg(agg2#36), partial_avg(agg3#37), partial_avg(agg4#38), partial_avg(agg5#39), partial_avg(agg6#40), partial_avg(agg7#41)] - -(117) CometExchange -Input [15]: [i_item_id#30, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(118) CometHashAggregate -Input [15]: [i_item_id#30, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162] -Keys [1]: [i_item_id#30] -Functions [7]: [avg(agg1#35), avg(agg2#36), avg(agg3#37), avg(agg4#38), avg(agg5#39), avg(agg6#40), avg(agg7#41)] - -(119) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#163, cs_bill_cdemo_sk#164, cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#171), dynamicpruningexpression(cs_sold_date_sk#171 IN dynamicpruning#172)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(120) CometFilter -Input [9]: [cs_bill_customer_sk#163, cs_bill_cdemo_sk#164, cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171] -Condition : ((isnotnull(cs_bill_cdemo_sk#164) AND isnotnull(cs_bill_customer_sk#163)) AND isnotnull(cs_item_sk#165)) - -(121) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#173, cd_dep_count#174] - -(122) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#163, cs_bill_cdemo_sk#164, cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171] -Right output [2]: [cd_demo_sk#173, cd_dep_count#174] -Arguments: [cs_bill_cdemo_sk#164], [cd_demo_sk#173], Inner, BuildRight - -(123) CometProject -Input [11]: [cs_bill_customer_sk#163, cs_bill_cdemo_sk#164, cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_demo_sk#173, cd_dep_count#174] -Arguments: [cs_bill_customer_sk#163, cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174], [cs_bill_customer_sk#163, cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174] - -(124) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#175, c_current_cdemo_sk#176, c_current_addr_sk#177, c_birth_year#178] - -(125) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#163, cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174] -Right output [4]: [c_customer_sk#175, c_current_cdemo_sk#176, c_current_addr_sk#177, c_birth_year#178] -Arguments: [cs_bill_customer_sk#163], [c_customer_sk#175], Inner, BuildRight - -(126) CometProject -Input [13]: [cs_bill_customer_sk#163, cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_customer_sk#175, c_current_cdemo_sk#176, c_current_addr_sk#177, c_birth_year#178] -Arguments: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_current_cdemo_sk#176, c_current_addr_sk#177, c_birth_year#178], [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_current_cdemo_sk#176, c_current_addr_sk#177, c_birth_year#178] - -(127) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#179] - -(128) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_current_cdemo_sk#176, c_current_addr_sk#177, c_birth_year#178] -Right output [1]: [cd_demo_sk#179] -Arguments: [c_current_cdemo_sk#176], [cd_demo_sk#179], Inner, BuildRight - -(129) CometProject -Input [12]: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_current_cdemo_sk#176, c_current_addr_sk#177, c_birth_year#178, cd_demo_sk#179] -Arguments: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_current_addr_sk#177, c_birth_year#178], [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_current_addr_sk#177, c_birth_year#178] - -(130) ReusedExchange [Reuses operator id: 107] -Output [1]: [ca_address_sk#180] - -(131) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_current_addr_sk#177, c_birth_year#178] -Right output [1]: [ca_address_sk#180] -Arguments: [c_current_addr_sk#177], [ca_address_sk#180], Inner, BuildRight - -(132) CometProject -Input [11]: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_current_addr_sk#177, c_birth_year#178, ca_address_sk#180] -Arguments: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_birth_year#178], [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_birth_year#178] - -(133) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#181] - -(134) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_birth_year#178] -Right output [1]: [d_date_sk#181] -Arguments: [cs_sold_date_sk#171], [d_date_sk#181], Inner, BuildRight - -(135) CometProject -Input [10]: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_birth_year#178, d_date_sk#181] -Arguments: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cd_dep_count#174, c_birth_year#178], [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cd_dep_count#174, c_birth_year#178] - -(136) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#182] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(137) CometFilter -Input [1]: [i_item_sk#182] -Condition : isnotnull(i_item_sk#182) - -(138) CometBroadcastExchange -Input [1]: [i_item_sk#182] -Arguments: [i_item_sk#182] - -(139) CometBroadcastHashJoin -Left output [8]: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cd_dep_count#174, c_birth_year#178] -Right output [1]: [i_item_sk#182] -Arguments: [cs_item_sk#165], [i_item_sk#182], Inner, BuildRight - -(140) CometProject -Input [9]: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cd_dep_count#174, c_birth_year#178, i_item_sk#182] -Arguments: [agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41], [cast(cs_quantity#166 as decimal(12,2)) AS agg1#35, cast(cs_list_price#167 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#169 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#168 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#170 as decimal(12,2)) AS agg5#39, cast(c_birth_year#178 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#174 as decimal(12,2)) AS agg7#41] - -(141) CometHashAggregate -Input [7]: [agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys: [] -Functions [7]: [partial_avg(agg1#35), partial_avg(agg2#36), partial_avg(agg3#37), partial_avg(agg4#38), partial_avg(agg5#39), partial_avg(agg6#40), partial_avg(agg7#41)] - -(142) CometExchange -Input [14]: [sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(143) CometHashAggregate -Input [14]: [sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196] -Keys: [] -Functions [7]: [avg(agg1#35), avg(agg2#36), avg(agg3#37), avg(agg4#38), avg(agg5#39), avg(agg6#40), avg(agg7#41)] - -(144) CometUnion -Child 0 Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#197, agg2#198, agg3#199, agg4#200, agg5#201, agg6#202, agg7#203] -Child 1 Input [11]: [i_item_id#30, ca_country#75, ca_state#25, county#204, agg1#205, agg2#206, agg3#207, agg4#208, agg5#209, agg6#210, agg7#211] -Child 2 Input [11]: [i_item_id#30, ca_country#111, ca_state#212, county#213, agg1#214, agg2#215, agg3#216, agg4#217, agg5#218, agg6#219, agg7#220] -Child 3 Input [11]: [i_item_id#30, ca_country#221, ca_state#222, county#223, agg1#224, agg2#225, agg3#226, agg4#227, agg5#228, agg6#229, agg7#230] -Child 4 Input [11]: [i_item_id#231, ca_country#232, ca_state#233, county#234, agg1#235, agg2#236, agg3#237, agg4#238, agg5#239, agg6#240, agg7#241] - -(145) CometTakeOrderedAndProject -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#197, agg2#198, agg3#199, agg4#200, agg5#201, agg6#202, agg7#203] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#32 ASC NULLS FIRST,ca_state#33 ASC NULLS FIRST,ca_county#34 ASC NULLS FIRST,i_item_id#31 ASC NULLS FIRST], output=[i_item_id#31,ca_country#32,ca_state#33,ca_county#34,agg1#197,agg2#198,agg3#199,agg4#200,agg5#201,agg6#202,agg7#203]), [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#197, agg2#198, agg3#199, agg4#200, agg5#201, agg6#202, agg7#203], 100, 0, [ca_country#32 ASC NULLS FIRST, ca_state#33 ASC NULLS FIRST, ca_county#34 ASC NULLS FIRST, i_item_id#31 ASC NULLS FIRST], [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#197, agg2#198, agg3#199, agg4#200, agg5#201, agg6#202, agg7#203] - -(146) CometColumnarToRow [codegen id : 1] -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#197, agg2#198, agg3#199, agg4#200, agg5#201, agg6#202, agg7#203] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (151) -+- * CometColumnarToRow (150) - +- CometProject (149) - +- CometFilter (148) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (147) - - -(147) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(148) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#26)) - -(149) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(150) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(151) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#64 IN dynamicpruning#10 - -Subquery:3 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#100 IN dynamicpruning#10 - -Subquery:4 Hosting operator id = 93 Hosting Expression = cs_sold_date_sk#136 IN dynamicpruning#10 - -Subquery:5 Hosting operator id = 119 Hosting Expression = cs_sold_date_sk#171 IN dynamicpruning#10 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 1572a2a240..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,160 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometUnion [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country,ca_state,ca_county] #1 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometProject [i_item_id,ca_country,ca_state,ca_county,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,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_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 [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 [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,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_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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - 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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange [cd_demo_sk] #5 - CometFilter [cd_demo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #8 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country,ca_state] #9 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 [ca_address_sk,ca_state,ca_country] #10 - CometProject [ca_state] [ca_address_sk,ca_state,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country] #11 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 [ca_address_sk,ca_country] #12 - CometProject [ca_address_sk,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id] #13 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 [ca_address_sk] #14 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - ReusedExchange [d_date_sk] #7 - ReusedExchange [i_item_sk,i_item_id] #8 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange #15 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 - ReusedExchange [ca_address_sk] #14 - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [i_item_sk] #16 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt deleted file mode 100644 index 86072fd830..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt +++ /dev/null @@ -1,846 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (146) -+- CometTakeOrderedAndProject (145) - +- CometUnion (144) - :- CometHashAggregate (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (19) - : : : : +- CometBroadcastHashJoin (18) - : : : : :- CometProject (14) - : : : : : +- CometBroadcastHashJoin (13) - : : : : : :- CometProject (8) - : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : +- CometBroadcastExchange (6) - : : : : : : +- CometProject (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : : : : +- CometBroadcastExchange (12) - : : : : : +- CometProject (11) - : : : : : +- CometFilter (10) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (9) - : : : : +- CometBroadcastExchange (17) - : : : : +- CometFilter (16) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (15) - : : : +- CometBroadcastExchange (23) - : : : +- CometProject (22) - : : : +- CometFilter (21) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (20) - : : +- CometBroadcastExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometFilter (33) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) - :- CometHashAggregate (66) - : +- CometExchange (65) - : +- CometHashAggregate (64) - : +- CometProject (63) - : +- CometBroadcastHashJoin (62) - : :- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (57) - : : : +- CometBroadcastHashJoin (56) - : : : :- CometProject (51) - : : : : +- CometBroadcastHashJoin (50) - : : : : :- CometProject (48) - : : : : : +- CometBroadcastHashJoin (47) - : : : : : :- CometProject (45) - : : : : : : +- CometBroadcastHashJoin (44) - : : : : : : :- CometFilter (42) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (41) - : : : : : : +- ReusedExchange (43) - : : : : : +- ReusedExchange (46) - : : : : +- ReusedExchange (49) - : : : +- CometBroadcastExchange (55) - : : : +- CometProject (54) - : : : +- CometFilter (53) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (52) - : : +- ReusedExchange (58) - : +- ReusedExchange (61) - :- CometHashAggregate (92) - : +- CometExchange (91) - : +- CometHashAggregate (90) - : +- CometProject (89) - : +- CometBroadcastHashJoin (88) - : :- CometProject (86) - : : +- CometBroadcastHashJoin (85) - : : :- CometProject (83) - : : : +- CometBroadcastHashJoin (82) - : : : :- CometProject (77) - : : : : +- CometBroadcastHashJoin (76) - : : : : :- CometProject (74) - : : : : : +- CometBroadcastHashJoin (73) - : : : : : :- CometProject (71) - : : : : : : +- CometBroadcastHashJoin (70) - : : : : : : :- CometFilter (68) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) - : : : : : : +- ReusedExchange (69) - : : : : : +- ReusedExchange (72) - : : : : +- ReusedExchange (75) - : : : +- CometBroadcastExchange (81) - : : : +- CometProject (80) - : : : +- CometFilter (79) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (78) - : : +- ReusedExchange (84) - : +- ReusedExchange (87) - :- CometHashAggregate (118) - : +- CometExchange (117) - : +- CometHashAggregate (116) - : +- CometProject (115) - : +- CometBroadcastHashJoin (114) - : :- CometProject (112) - : : +- CometBroadcastHashJoin (111) - : : :- CometProject (109) - : : : +- CometBroadcastHashJoin (108) - : : : :- CometProject (103) - : : : : +- CometBroadcastHashJoin (102) - : : : : :- CometProject (100) - : : : : : +- CometBroadcastHashJoin (99) - : : : : : :- CometProject (97) - : : : : : : +- CometBroadcastHashJoin (96) - : : : : : : :- CometFilter (94) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (93) - : : : : : : +- ReusedExchange (95) - : : : : : +- ReusedExchange (98) - : : : : +- ReusedExchange (101) - : : : +- CometBroadcastExchange (107) - : : : +- CometProject (106) - : : : +- CometFilter (105) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (104) - : : +- ReusedExchange (110) - : +- ReusedExchange (113) - +- CometHashAggregate (143) - +- CometExchange (142) - +- CometHashAggregate (141) - +- CometProject (140) - +- CometBroadcastHashJoin (139) - :- CometProject (135) - : +- CometBroadcastHashJoin (134) - : :- CometProject (132) - : : +- CometBroadcastHashJoin (131) - : : :- CometProject (129) - : : : +- CometBroadcastHashJoin (128) - : : : :- CometProject (126) - : : : : +- CometBroadcastHashJoin (125) - : : : : :- CometProject (123) - : : : : : +- CometBroadcastHashJoin (122) - : : : : : :- CometFilter (120) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (119) - : : : : : +- ReusedExchange (121) - : : : : +- ReusedExchange (124) - : : : +- ReusedExchange (127) - : : +- ReusedExchange (130) - : +- ReusedExchange (133) - +- CometBroadcastExchange (138) - +- CometFilter (137) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (136) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : (((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#12, 1, true, false, true) = M) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = College )) AND isnotnull(cd_demo_sk#11)) - -(5) CometProject -Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] - -(6) CometBroadcastExchange -Input [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cd_demo_sk#11, cd_dep_count#14] - -(7) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] -Right output [2]: [cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#11], Inner, BuildRight - -(8) CometProject -Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] -Arguments: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14], [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [In(c_birth_month, [1,10,12,4,5,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Condition : (((c_birth_month#18 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) - -(11) CometProject -Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(12) CometBroadcastExchange -Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(13) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] -Right output [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_bill_customer_sk#1], [c_customer_sk#15], Inner, BuildRight - -(14) CometProject -Input [13]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [1]: [cd_demo_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [cd_demo_sk#20] -Condition : isnotnull(cd_demo_sk#20) - -(17) CometBroadcastExchange -Input [1]: [cd_demo_sk#20] -Arguments: [cd_demo_sk#20] - -(18) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] -Right output [1]: [cd_demo_sk#20] -Arguments: [c_current_cdemo_sk#16], [cd_demo_sk#20], Inner, BuildRight - -(19) CometProject -Input [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19, cd_demo_sk#20] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(21) CometFilter -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) - -(22) CometProject -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24], [ca_address_sk#21, ca_county#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#23, 2, true, false, true) AS ca_state#25, ca_country#24] - -(23) CometBroadcastExchange -Input [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] - -(24) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] -Right output [4]: [ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [c_current_addr_sk#17], [ca_address_sk#21], Inner, BuildRight - -(25) CometProject -Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#25, ca_country#24] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#26)) - -(28) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(29) CometBroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: [d_date_sk#26] - -(30) CometBroadcastHashJoin -Left output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [1]: [d_date_sk#26] -Arguments: [cs_sold_date_sk#9], [d_date_sk#26], Inner, BuildRight - -(31) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, d_date_sk#26] -Arguments: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24], [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#28, i_item_id#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [2]: [i_item_sk#28, i_item_id#29] -Condition : isnotnull(i_item_sk#28) - -(34) CometProject -Input [2]: [i_item_sk#28, i_item_id#29] -Arguments: [i_item_sk#28, i_item_id#30], [i_item_sk#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#29, 16, true, false, true) AS i_item_id#30] - -(35) CometBroadcastExchange -Input [2]: [i_item_sk#28, i_item_id#30] -Arguments: [i_item_sk#28, i_item_id#30] - -(36) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24] -Right output [2]: [i_item_sk#28, i_item_id#30] -Arguments: [cs_item_sk#3], [i_item_sk#28], Inner, BuildRight - -(37) CometProject -Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#25, ca_country#24, i_item_sk#28, i_item_id#30] -Arguments: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41], [i_item_id#30 AS i_item_id#31, ca_country#24 AS ca_country#32, ca_state#25 AS ca_state#33, ca_county#22 AS ca_county#34, cast(cs_quantity#4 as decimal(12,2)) AS agg1#35, cast(cs_list_price#5 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#39, cast(c_birth_year#19 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] - -(38) CometHashAggregate -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [4]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34] -Functions [7]: [partial_avg(agg1#35), partial_avg(agg2#36), partial_avg(agg3#37), partial_avg(agg4#38), partial_avg(agg5#39), partial_avg(agg6#40), partial_avg(agg7#41)] - -(39) CometExchange -Input [18]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] -Arguments: hashpartitioning(i_item_id#31, ca_country#32, ca_state#33, ca_county#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(40) CometHashAggregate -Input [18]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] -Keys [4]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34] -Functions [7]: [avg(agg1#35), avg(agg2#36), avg(agg3#37), avg(agg4#38), avg(agg5#39), avg(agg6#40), avg(agg7#41)] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#64), dynamicpruningexpression(cs_sold_date_sk#64 IN dynamicpruning#65)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(42) CometFilter -Input [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] -Condition : ((isnotnull(cs_bill_cdemo_sk#57) AND isnotnull(cs_bill_customer_sk#56)) AND isnotnull(cs_item_sk#58)) - -(43) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#66, cd_dep_count#67] - -(44) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64] -Right output [2]: [cd_demo_sk#66, cd_dep_count#67] -Arguments: [cs_bill_cdemo_sk#57], [cd_demo_sk#66], Inner, BuildRight - -(45) CometProject -Input [11]: [cs_bill_customer_sk#56, cs_bill_cdemo_sk#57, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_demo_sk#66, cd_dep_count#67] -Arguments: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67], [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67] - -(46) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] - -(47) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67] -Right output [4]: [c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Arguments: [cs_bill_customer_sk#56], [c_customer_sk#68], Inner, BuildRight - -(48) CometProject -Input [13]: [cs_bill_customer_sk#56, cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_customer_sk#68, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] - -(49) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#72] - -(50) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71] -Right output [1]: [cd_demo_sk#72] -Arguments: [c_current_cdemo_sk#69], [cd_demo_sk#72], Inner, BuildRight - -(51) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_cdemo_sk#69, c_current_addr_sk#70, c_birth_year#71, cd_demo_sk#72] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71] - -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#73, ca_state#74, ca_country#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [ca_address_sk#73, ca_state#74, ca_country#75] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#74, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#73)) - -(54) CometProject -Input [3]: [ca_address_sk#73, ca_state#74, ca_country#75] -Arguments: [ca_address_sk#73, ca_state#25, ca_country#75], [ca_address_sk#73, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#74, 2, true, false, true) AS ca_state#25, ca_country#75] - -(55) CometBroadcastExchange -Input [3]: [ca_address_sk#73, ca_state#25, ca_country#75] -Arguments: [ca_address_sk#73, ca_state#25, ca_country#75] - -(56) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71] -Right output [3]: [ca_address_sk#73, ca_state#25, ca_country#75] -Arguments: [c_current_addr_sk#70], [ca_address_sk#73], Inner, BuildRight - -(57) CometProject -Input [13]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_current_addr_sk#70, c_birth_year#71, ca_address_sk#73, ca_state#25, ca_country#75] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] - -(58) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#76] - -(59) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] -Right output [1]: [d_date_sk#76] -Arguments: [cs_sold_date_sk#64], [d_date_sk#76], Inner, BuildRight - -(60) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cs_sold_date_sk#64, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75, d_date_sk#76] -Arguments: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75], [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] - -(61) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#77, i_item_id#30] - -(62) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75] -Right output [2]: [i_item_sk#77, i_item_id#30] -Arguments: [cs_item_sk#58], [i_item_sk#77], Inner, BuildRight - -(63) CometProject -Input [12]: [cs_item_sk#58, cs_quantity#59, cs_list_price#60, cs_sales_price#61, cs_coupon_amt#62, cs_net_profit#63, cd_dep_count#67, c_birth_year#71, ca_state#25, ca_country#75, i_item_sk#77, i_item_id#30] -Arguments: [i_item_id#30, ca_country#75, ca_state#25, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41], [i_item_id#30, ca_country#75, ca_state#25, cast(cs_quantity#59 as decimal(12,2)) AS agg1#35, cast(cs_list_price#60 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#62 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#61 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#63 as decimal(12,2)) AS agg5#39, cast(c_birth_year#71 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#67 as decimal(12,2)) AS agg7#41] - -(64) CometHashAggregate -Input [10]: [i_item_id#30, ca_country#75, ca_state#25, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [3]: [i_item_id#30, ca_country#75, ca_state#25] -Functions [7]: [partial_avg(agg1#35), partial_avg(agg2#36), partial_avg(agg3#37), partial_avg(agg4#38), partial_avg(agg5#39), partial_avg(agg6#40), partial_avg(agg7#41)] - -(65) CometExchange -Input [17]: [i_item_id#30, ca_country#75, ca_state#25, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] -Arguments: hashpartitioning(i_item_id#30, ca_country#75, ca_state#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(66) CometHashAggregate -Input [17]: [i_item_id#30, ca_country#75, ca_state#25, sum#78, count#79, sum#80, count#81, sum#82, count#83, sum#84, count#85, sum#86, count#87, sum#88, count#89, sum#90, count#91] -Keys [3]: [i_item_id#30, ca_country#75, ca_state#25] -Functions [7]: [avg(agg1#35), avg(agg2#36), avg(agg3#37), avg(agg4#38), avg(agg5#39), avg(agg6#40), avg(agg7#41)] - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#92, cs_bill_cdemo_sk#93, cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#100), dynamicpruningexpression(cs_sold_date_sk#100 IN dynamicpruning#101)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(68) CometFilter -Input [9]: [cs_bill_customer_sk#92, cs_bill_cdemo_sk#93, cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100] -Condition : ((isnotnull(cs_bill_cdemo_sk#93) AND isnotnull(cs_bill_customer_sk#92)) AND isnotnull(cs_item_sk#94)) - -(69) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#102, cd_dep_count#103] - -(70) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#92, cs_bill_cdemo_sk#93, cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100] -Right output [2]: [cd_demo_sk#102, cd_dep_count#103] -Arguments: [cs_bill_cdemo_sk#93], [cd_demo_sk#102], Inner, BuildRight - -(71) CometProject -Input [11]: [cs_bill_customer_sk#92, cs_bill_cdemo_sk#93, cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_demo_sk#102, cd_dep_count#103] -Arguments: [cs_bill_customer_sk#92, cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103], [cs_bill_customer_sk#92, cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103] - -(72) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#104, c_current_cdemo_sk#105, c_current_addr_sk#106, c_birth_year#107] - -(73) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#92, cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103] -Right output [4]: [c_customer_sk#104, c_current_cdemo_sk#105, c_current_addr_sk#106, c_birth_year#107] -Arguments: [cs_bill_customer_sk#92], [c_customer_sk#104], Inner, BuildRight - -(74) CometProject -Input [13]: [cs_bill_customer_sk#92, cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_customer_sk#104, c_current_cdemo_sk#105, c_current_addr_sk#106, c_birth_year#107] -Arguments: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_current_cdemo_sk#105, c_current_addr_sk#106, c_birth_year#107], [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_current_cdemo_sk#105, c_current_addr_sk#106, c_birth_year#107] - -(75) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#108] - -(76) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_current_cdemo_sk#105, c_current_addr_sk#106, c_birth_year#107] -Right output [1]: [cd_demo_sk#108] -Arguments: [c_current_cdemo_sk#105], [cd_demo_sk#108], Inner, BuildRight - -(77) CometProject -Input [12]: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_current_cdemo_sk#105, c_current_addr_sk#106, c_birth_year#107, cd_demo_sk#108] -Arguments: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_current_addr_sk#106, c_birth_year#107], [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_current_addr_sk#106, c_birth_year#107] - -(78) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [3]: [ca_address_sk#109, ca_state#110, ca_country#111] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(79) CometFilter -Input [3]: [ca_address_sk#109, ca_state#110, ca_country#111] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#110, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#109)) - -(80) CometProject -Input [3]: [ca_address_sk#109, ca_state#110, ca_country#111] -Arguments: [ca_address_sk#109, ca_country#111], [ca_address_sk#109, ca_country#111] - -(81) CometBroadcastExchange -Input [2]: [ca_address_sk#109, ca_country#111] -Arguments: [ca_address_sk#109, ca_country#111] - -(82) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_current_addr_sk#106, c_birth_year#107] -Right output [2]: [ca_address_sk#109, ca_country#111] -Arguments: [c_current_addr_sk#106], [ca_address_sk#109], Inner, BuildRight - -(83) CometProject -Input [12]: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_current_addr_sk#106, c_birth_year#107, ca_address_sk#109, ca_country#111] -Arguments: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_birth_year#107, ca_country#111], [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_birth_year#107, ca_country#111] - -(84) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#112] - -(85) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_birth_year#107, ca_country#111] -Right output [1]: [d_date_sk#112] -Arguments: [cs_sold_date_sk#100], [d_date_sk#112], Inner, BuildRight - -(86) CometProject -Input [11]: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cs_sold_date_sk#100, cd_dep_count#103, c_birth_year#107, ca_country#111, d_date_sk#112] -Arguments: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cd_dep_count#103, c_birth_year#107, ca_country#111], [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cd_dep_count#103, c_birth_year#107, ca_country#111] - -(87) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#113, i_item_id#30] - -(88) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cd_dep_count#103, c_birth_year#107, ca_country#111] -Right output [2]: [i_item_sk#113, i_item_id#30] -Arguments: [cs_item_sk#94], [i_item_sk#113], Inner, BuildRight - -(89) CometProject -Input [11]: [cs_item_sk#94, cs_quantity#95, cs_list_price#96, cs_sales_price#97, cs_coupon_amt#98, cs_net_profit#99, cd_dep_count#103, c_birth_year#107, ca_country#111, i_item_sk#113, i_item_id#30] -Arguments: [i_item_id#30, ca_country#111, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41], [i_item_id#30, ca_country#111, cast(cs_quantity#95 as decimal(12,2)) AS agg1#35, cast(cs_list_price#96 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#98 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#97 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#99 as decimal(12,2)) AS agg5#39, cast(c_birth_year#107 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#103 as decimal(12,2)) AS agg7#41] - -(90) CometHashAggregate -Input [9]: [i_item_id#30, ca_country#111, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [2]: [i_item_id#30, ca_country#111] -Functions [7]: [partial_avg(agg1#35), partial_avg(agg2#36), partial_avg(agg3#37), partial_avg(agg4#38), partial_avg(agg5#39), partial_avg(agg6#40), partial_avg(agg7#41)] - -(91) CometExchange -Input [16]: [i_item_id#30, ca_country#111, sum#114, count#115, sum#116, count#117, sum#118, count#119, sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#126, count#127] -Arguments: hashpartitioning(i_item_id#30, ca_country#111, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(92) CometHashAggregate -Input [16]: [i_item_id#30, ca_country#111, sum#114, count#115, sum#116, count#117, sum#118, count#119, sum#120, count#121, sum#122, count#123, sum#124, count#125, sum#126, count#127] -Keys [2]: [i_item_id#30, ca_country#111] -Functions [7]: [avg(agg1#35), avg(agg2#36), avg(agg3#37), avg(agg4#38), avg(agg5#39), avg(agg6#40), avg(agg7#41)] - -(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#128, cs_bill_cdemo_sk#129, cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#136), dynamicpruningexpression(cs_sold_date_sk#136 IN dynamicpruning#137)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(94) CometFilter -Input [9]: [cs_bill_customer_sk#128, cs_bill_cdemo_sk#129, cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136] -Condition : ((isnotnull(cs_bill_cdemo_sk#129) AND isnotnull(cs_bill_customer_sk#128)) AND isnotnull(cs_item_sk#130)) - -(95) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#138, cd_dep_count#139] - -(96) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#128, cs_bill_cdemo_sk#129, cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136] -Right output [2]: [cd_demo_sk#138, cd_dep_count#139] -Arguments: [cs_bill_cdemo_sk#129], [cd_demo_sk#138], Inner, BuildRight - -(97) CometProject -Input [11]: [cs_bill_customer_sk#128, cs_bill_cdemo_sk#129, cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_demo_sk#138, cd_dep_count#139] -Arguments: [cs_bill_customer_sk#128, cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139], [cs_bill_customer_sk#128, cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139] - -(98) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#140, c_current_cdemo_sk#141, c_current_addr_sk#142, c_birth_year#143] - -(99) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#128, cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139] -Right output [4]: [c_customer_sk#140, c_current_cdemo_sk#141, c_current_addr_sk#142, c_birth_year#143] -Arguments: [cs_bill_customer_sk#128], [c_customer_sk#140], Inner, BuildRight - -(100) CometProject -Input [13]: [cs_bill_customer_sk#128, cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_customer_sk#140, c_current_cdemo_sk#141, c_current_addr_sk#142, c_birth_year#143] -Arguments: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_current_cdemo_sk#141, c_current_addr_sk#142, c_birth_year#143], [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_current_cdemo_sk#141, c_current_addr_sk#142, c_birth_year#143] - -(101) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#144] - -(102) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_current_cdemo_sk#141, c_current_addr_sk#142, c_birth_year#143] -Right output [1]: [cd_demo_sk#144] -Arguments: [c_current_cdemo_sk#141], [cd_demo_sk#144], Inner, BuildRight - -(103) CometProject -Input [12]: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_current_cdemo_sk#141, c_current_addr_sk#142, c_birth_year#143, cd_demo_sk#144] -Arguments: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_current_addr_sk#142, c_birth_year#143], [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_current_addr_sk#142, c_birth_year#143] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#145, ca_state#146] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [ca_address_sk#145, ca_state#146] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#146, 2, true, false, true) IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#145)) - -(106) CometProject -Input [2]: [ca_address_sk#145, ca_state#146] -Arguments: [ca_address_sk#145], [ca_address_sk#145] - -(107) CometBroadcastExchange -Input [1]: [ca_address_sk#145] -Arguments: [ca_address_sk#145] - -(108) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_current_addr_sk#142, c_birth_year#143] -Right output [1]: [ca_address_sk#145] -Arguments: [c_current_addr_sk#142], [ca_address_sk#145], Inner, BuildRight - -(109) CometProject -Input [11]: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_current_addr_sk#142, c_birth_year#143, ca_address_sk#145] -Arguments: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_birth_year#143], [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_birth_year#143] - -(110) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#147] - -(111) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_birth_year#143] -Right output [1]: [d_date_sk#147] -Arguments: [cs_sold_date_sk#136], [d_date_sk#147], Inner, BuildRight - -(112) CometProject -Input [10]: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cs_sold_date_sk#136, cd_dep_count#139, c_birth_year#143, d_date_sk#147] -Arguments: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cd_dep_count#139, c_birth_year#143], [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cd_dep_count#139, c_birth_year#143] - -(113) ReusedExchange [Reuses operator id: 35] -Output [2]: [i_item_sk#148, i_item_id#30] - -(114) CometBroadcastHashJoin -Left output [8]: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cd_dep_count#139, c_birth_year#143] -Right output [2]: [i_item_sk#148, i_item_id#30] -Arguments: [cs_item_sk#130], [i_item_sk#148], Inner, BuildRight - -(115) CometProject -Input [10]: [cs_item_sk#130, cs_quantity#131, cs_list_price#132, cs_sales_price#133, cs_coupon_amt#134, cs_net_profit#135, cd_dep_count#139, c_birth_year#143, i_item_sk#148, i_item_id#30] -Arguments: [i_item_id#30, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41], [i_item_id#30, cast(cs_quantity#131 as decimal(12,2)) AS agg1#35, cast(cs_list_price#132 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#134 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#133 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#135 as decimal(12,2)) AS agg5#39, cast(c_birth_year#143 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#139 as decimal(12,2)) AS agg7#41] - -(116) CometHashAggregate -Input [8]: [i_item_id#30, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [1]: [i_item_id#30] -Functions [7]: [partial_avg(agg1#35), partial_avg(agg2#36), partial_avg(agg3#37), partial_avg(agg4#38), partial_avg(agg5#39), partial_avg(agg6#40), partial_avg(agg7#41)] - -(117) CometExchange -Input [15]: [i_item_id#30, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(118) CometHashAggregate -Input [15]: [i_item_id#30, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162] -Keys [1]: [i_item_id#30] -Functions [7]: [avg(agg1#35), avg(agg2#36), avg(agg3#37), avg(agg4#38), avg(agg5#39), avg(agg6#40), avg(agg7#41)] - -(119) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [9]: [cs_bill_customer_sk#163, cs_bill_cdemo_sk#164, cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#171), dynamicpruningexpression(cs_sold_date_sk#171 IN dynamicpruning#172)] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(120) CometFilter -Input [9]: [cs_bill_customer_sk#163, cs_bill_cdemo_sk#164, cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171] -Condition : ((isnotnull(cs_bill_cdemo_sk#164) AND isnotnull(cs_bill_customer_sk#163)) AND isnotnull(cs_item_sk#165)) - -(121) ReusedExchange [Reuses operator id: 6] -Output [2]: [cd_demo_sk#173, cd_dep_count#174] - -(122) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#163, cs_bill_cdemo_sk#164, cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171] -Right output [2]: [cd_demo_sk#173, cd_dep_count#174] -Arguments: [cs_bill_cdemo_sk#164], [cd_demo_sk#173], Inner, BuildRight - -(123) CometProject -Input [11]: [cs_bill_customer_sk#163, cs_bill_cdemo_sk#164, cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_demo_sk#173, cd_dep_count#174] -Arguments: [cs_bill_customer_sk#163, cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174], [cs_bill_customer_sk#163, cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174] - -(124) ReusedExchange [Reuses operator id: 12] -Output [4]: [c_customer_sk#175, c_current_cdemo_sk#176, c_current_addr_sk#177, c_birth_year#178] - -(125) CometBroadcastHashJoin -Left output [9]: [cs_bill_customer_sk#163, cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174] -Right output [4]: [c_customer_sk#175, c_current_cdemo_sk#176, c_current_addr_sk#177, c_birth_year#178] -Arguments: [cs_bill_customer_sk#163], [c_customer_sk#175], Inner, BuildRight - -(126) CometProject -Input [13]: [cs_bill_customer_sk#163, cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_customer_sk#175, c_current_cdemo_sk#176, c_current_addr_sk#177, c_birth_year#178] -Arguments: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_current_cdemo_sk#176, c_current_addr_sk#177, c_birth_year#178], [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_current_cdemo_sk#176, c_current_addr_sk#177, c_birth_year#178] - -(127) ReusedExchange [Reuses operator id: 17] -Output [1]: [cd_demo_sk#179] - -(128) CometBroadcastHashJoin -Left output [11]: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_current_cdemo_sk#176, c_current_addr_sk#177, c_birth_year#178] -Right output [1]: [cd_demo_sk#179] -Arguments: [c_current_cdemo_sk#176], [cd_demo_sk#179], Inner, BuildRight - -(129) CometProject -Input [12]: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_current_cdemo_sk#176, c_current_addr_sk#177, c_birth_year#178, cd_demo_sk#179] -Arguments: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_current_addr_sk#177, c_birth_year#178], [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_current_addr_sk#177, c_birth_year#178] - -(130) ReusedExchange [Reuses operator id: 107] -Output [1]: [ca_address_sk#180] - -(131) CometBroadcastHashJoin -Left output [10]: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_current_addr_sk#177, c_birth_year#178] -Right output [1]: [ca_address_sk#180] -Arguments: [c_current_addr_sk#177], [ca_address_sk#180], Inner, BuildRight - -(132) CometProject -Input [11]: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_current_addr_sk#177, c_birth_year#178, ca_address_sk#180] -Arguments: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_birth_year#178], [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_birth_year#178] - -(133) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#181] - -(134) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_birth_year#178] -Right output [1]: [d_date_sk#181] -Arguments: [cs_sold_date_sk#171], [d_date_sk#181], Inner, BuildRight - -(135) CometProject -Input [10]: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cs_sold_date_sk#171, cd_dep_count#174, c_birth_year#178, d_date_sk#181] -Arguments: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cd_dep_count#174, c_birth_year#178], [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cd_dep_count#174, c_birth_year#178] - -(136) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#182] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(137) CometFilter -Input [1]: [i_item_sk#182] -Condition : isnotnull(i_item_sk#182) - -(138) CometBroadcastExchange -Input [1]: [i_item_sk#182] -Arguments: [i_item_sk#182] - -(139) CometBroadcastHashJoin -Left output [8]: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cd_dep_count#174, c_birth_year#178] -Right output [1]: [i_item_sk#182] -Arguments: [cs_item_sk#165], [i_item_sk#182], Inner, BuildRight - -(140) CometProject -Input [9]: [cs_item_sk#165, cs_quantity#166, cs_list_price#167, cs_sales_price#168, cs_coupon_amt#169, cs_net_profit#170, cd_dep_count#174, c_birth_year#178, i_item_sk#182] -Arguments: [agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41], [cast(cs_quantity#166 as decimal(12,2)) AS agg1#35, cast(cs_list_price#167 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#169 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#168 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#170 as decimal(12,2)) AS agg5#39, cast(c_birth_year#178 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#174 as decimal(12,2)) AS agg7#41] - -(141) CometHashAggregate -Input [7]: [agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys: [] -Functions [7]: [partial_avg(agg1#35), partial_avg(agg2#36), partial_avg(agg3#37), partial_avg(agg4#38), partial_avg(agg5#39), partial_avg(agg6#40), partial_avg(agg7#41)] - -(142) CometExchange -Input [14]: [sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(143) CometHashAggregate -Input [14]: [sum#183, count#184, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196] -Keys: [] -Functions [7]: [avg(agg1#35), avg(agg2#36), avg(agg3#37), avg(agg4#38), avg(agg5#39), avg(agg6#40), avg(agg7#41)] - -(144) CometUnion -Child 0 Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#197, agg2#198, agg3#199, agg4#200, agg5#201, agg6#202, agg7#203] -Child 1 Input [11]: [i_item_id#30, ca_country#75, ca_state#25, county#204, agg1#205, agg2#206, agg3#207, agg4#208, agg5#209, agg6#210, agg7#211] -Child 2 Input [11]: [i_item_id#30, ca_country#111, ca_state#212, county#213, agg1#214, agg2#215, agg3#216, agg4#217, agg5#218, agg6#219, agg7#220] -Child 3 Input [11]: [i_item_id#30, ca_country#221, ca_state#222, county#223, agg1#224, agg2#225, agg3#226, agg4#227, agg5#228, agg6#229, agg7#230] -Child 4 Input [11]: [i_item_id#231, ca_country#232, ca_state#233, county#234, agg1#235, agg2#236, agg3#237, agg4#238, agg5#239, agg6#240, agg7#241] - -(145) CometTakeOrderedAndProject -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#197, agg2#198, agg3#199, agg4#200, agg5#201, agg6#202, agg7#203] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_country#32 ASC NULLS FIRST,ca_state#33 ASC NULLS FIRST,ca_county#34 ASC NULLS FIRST,i_item_id#31 ASC NULLS FIRST], output=[i_item_id#31,ca_country#32,ca_state#33,ca_county#34,agg1#197,agg2#198,agg3#199,agg4#200,agg5#201,agg6#202,agg7#203]), [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#197, agg2#198, agg3#199, agg4#200, agg5#201, agg6#202, agg7#203], 100, 0, [ca_country#32 ASC NULLS FIRST, ca_state#33 ASC NULLS FIRST, ca_county#34 ASC NULLS FIRST, i_item_id#31 ASC NULLS FIRST], [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#197, agg2#198, agg3#199, agg4#200, agg5#201, agg6#202, agg7#203] - -(146) CometColumnarToRow [codegen id : 1] -Input [11]: [i_item_id#31, ca_country#32, ca_state#33, ca_county#34, agg1#197, agg2#198, agg3#199, agg4#200, agg5#201, agg6#202, agg7#203] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (151) -+- * CometColumnarToRow (150) - +- CometProject (149) - +- CometFilter (148) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (147) - - -(147) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_year#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(148) CometFilter -Input [2]: [d_date_sk#26, d_year#27] -Condition : ((isnotnull(d_year#27) AND (d_year#27 = 2001)) AND isnotnull(d_date_sk#26)) - -(149) CometProject -Input [2]: [d_date_sk#26, d_year#27] -Arguments: [d_date_sk#26], [d_date_sk#26] - -(150) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#26] - -(151) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 41 Hosting Expression = cs_sold_date_sk#64 IN dynamicpruning#10 - -Subquery:3 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#100 IN dynamicpruning#10 - -Subquery:4 Hosting operator id = 93 Hosting Expression = cs_sold_date_sk#136 IN dynamicpruning#10 - -Subquery:5 Hosting operator id = 119 Hosting Expression = cs_sold_date_sk#171 IN dynamicpruning#10 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/extended.txt deleted file mode 100644 index b18a444bb9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/extended.txt +++ /dev/null @@ -1,214 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 204 out of 210 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 1572a2a240..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/simplified.txt +++ /dev/null @@ -1,160 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometUnion [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country,ca_state,ca_county] #1 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - CometProject [i_item_id,ca_country,ca_state,ca_county,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,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_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 [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 [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,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_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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk,cd_dep_count] #3 - CometProject [cd_demo_sk,cd_dep_count] - CometFilter [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] - 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_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - CometBroadcastExchange [cd_demo_sk] #5 - CometFilter [cd_demo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state,ca_country] #6 - CometProject [ca_state] [ca_address_sk,ca_county,ca_state,ca_country] - CometFilter [ca_address_sk,ca_county,ca_state,ca_country] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_item_id] #8 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country,ca_state] #9 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 [ca_address_sk,ca_state,ca_country] #10 - CometProject [ca_state] [ca_address_sk,ca_state,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id,ca_country] #11 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 [ca_address_sk,ca_country] #12 - CometProject [ca_address_sk,ca_country] - CometFilter [ca_address_sk,ca_state,ca_country] - CometScan [native_iceberg_compat] 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 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange [i_item_id] #13 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 [ca_address_sk] #14 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - ReusedExchange [d_date_sk] #7 - ReusedExchange [i_item_sk,i_item_id] #8 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7)] - CometExchange #15 - CometHashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - 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,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_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 [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 [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,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_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 [native_iceberg_compat] 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 - ReusedExchange [ca_address_sk] #14 - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [i_item_sk] #16 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/explain.txt deleted file mode 100644 index 5a32d4019b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/explain.txt +++ /dev/null @@ -1,163 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.item (4) - +- ReusedExchange (11) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(7) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(8) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(11) ReusedExchange [Reuses operator id: 28] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] - -(14) HashAggregate [codegen id : 3] -Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(15) CometColumnarExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(17) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#17] -Results [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#17,17,2) AS _w0#19] - -(18) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(22) Project [codegen id : 6] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] - -(23) TakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometNativeScan parquet spark_catalog.default.date_dim (24) - - -(24) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#22] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/simplified.txt deleted file mode 100644 index 7bc0779e53..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (6) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] - Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/explain.txt deleted file mode 100644 index 322bbe63b3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [cs_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/simplified.txt deleted file mode 100644 index cf18e68a3d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt deleted file mode 100644 index 322bbe63b3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt +++ /dev/null @@ -1,164 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [cs_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [cs_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] - -(23) TakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (28) -+- * CometColumnarToRow (27) - +- CometProject (26) - +- CometFilter (25) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) - - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(26) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(27) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(28) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/extended.txt deleted file mode 100644 index cd52b2cd12..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index cf18e68a3d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt +++ /dev/null @@ -1,35 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/explain.txt deleted file mode 100644 index aa06ae2c5f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/explain.txt +++ /dev/null @@ -1,170 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (24) -+- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometColumnarExchange (21) - +- * HashAggregate (20) - +- * Expand (19) - +- * Project (18) - +- * BroadcastNestedLoopJoin Inner BuildRight (17) - :- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.inventory (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) - : +- CometProject (9) - : +- CometFilter (8) - : +- CometNativeScan parquet spark_catalog.default.item (7) - +- BroadcastExchange (16) - +- * CometColumnarToRow (15) - +- CometNativeScan parquet spark_catalog.default.warehouse (14) - - -(1) Scan parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#3), dynamicpruningexpression(inv_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(inv_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] - -(3) Filter [codegen id : 4] -Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Condition : isnotnull(inv_item_sk#1) - -(4) ReusedExchange [Reuses operator id: 29] -Output [1]: [d_date_sk#5] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] -Input [4]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3, d_date_sk#5] - -(7) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(8) CometFilter -Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] -Condition : isnotnull(i_item_sk#6) - -(9) CometProject -Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] -Arguments: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14], [i_item_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#7, 50, true, false, true) AS i_brand#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#8, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#10, 50, true, false, true) AS i_product_name#14] - -(10) CometColumnarToRow [codegen id : 2] -Input [5]: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] - -(11) BroadcastExchange -Input [5]: [i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#2, i_brand#11, i_class#12, i_category#13, i_product_name#14] -Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#6, i_brand#11, i_class#12, i_category#13, i_product_name#14] - -(14) CometNativeScan parquet spark_catalog.default.warehouse -Output: [] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -ReadSchema: struct<> - -(15) CometColumnarToRow [codegen id : 3] -Input: [] - -(16) BroadcastExchange -Input: [] -Arguments: IdentityBroadcastMode, [plan_id=2] - -(17) BroadcastNestedLoopJoin [codegen id : 4] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13] -Input [5]: [inv_quantity_on_hand#2, i_brand#11, i_class#12, i_category#13, i_product_name#14] - -(19) Expand [codegen id : 4] -Input [5]: [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13] -Arguments: [[inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, i_category#13, 0], [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, i_class#12, null, 1], [inv_quantity_on_hand#2, i_product_name#14, i_brand#11, null, null, 3], [inv_quantity_on_hand#2, i_product_name#14, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] - -(20) HashAggregate [codegen id : 4] -Input [6]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] -Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] -Functions [1]: [partial_avg(inv_quantity_on_hand#2)] -Aggregate Attributes [2]: [sum#20, count#21] -Results [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] - -(21) CometColumnarExchange -Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] -Arguments: hashpartitioning(i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] - -(23) HashAggregate [codegen id : 5] -Input [7]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19, sum#22, count#23] -Keys [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, spark_grouping_id#19] -Functions [1]: [avg(inv_quantity_on_hand#2)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#24] -Results [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, avg(inv_quantity_on_hand#2)#24 AS qoh#25] - -(24) TakeOrderedAndProject -Input [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25] -Arguments: 100, [qoh#25 ASC NULLS FIRST, i_product_name#15 ASC NULLS FIRST, i_brand#16 ASC NULLS FIRST, i_class#17 ASC NULLS FIRST, i_category#18 ASC NULLS FIRST], [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#25] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometNativeScan parquet spark_catalog.default.date_dim (25) - - -(25) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#26] -Condition : (((isnotnull(d_month_seq#26) AND (d_month_seq#26 >= 1200)) AND (d_month_seq#26 <= 1211)) AND isnotnull(d_date_sk#5)) - -(27) CometProject -Input [2]: [d_date_sk#5, d_month_seq#26] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(28) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(29) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/simplified.txt deleted file mode 100644 index 457dff46e2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (5) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastNestedLoopJoin - Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Filter [inv_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometNativeScan parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/explain.txt deleted file mode 100644 index f85c898208..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (26) -+- * HashAggregate (25) - +- * CometColumnarToRow (24) - +- CometColumnarExchange (23) - +- * HashAggregate (22) - +- * Expand (21) - +- * Project (20) - +- * BroadcastNestedLoopJoin Inner BuildRight (19) - :- * CometColumnarToRow (15) - : +- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - +- BroadcastExchange (18) - +- * CometColumnarToRow (17) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (16) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#3), dynamicpruningexpression(inv_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(inv_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Condition : isnotnull(inv_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [inv_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3, d_date_sk#5] -Arguments: [inv_item_sk#1, inv_quantity_on_hand#2], [inv_item_sk#1, inv_quantity_on_hand#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Condition : isnotnull(i_item_sk#7) - -(11) CometProject -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#8, 50, true, false, true) AS i_brand#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#11, 50, true, false, true) AS i_product_name#15] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(13) CometBroadcastHashJoin -Left output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] -Right output [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15], [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(15) CometColumnarToRow [codegen id : 2] -Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output: [] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -ReadSchema: struct<> - -(17) CometColumnarToRow [codegen id : 1] -Input: [] - -(18) BroadcastExchange -Input: [] -Arguments: IdentityBroadcastMode, [plan_id=1] - -(19) BroadcastNestedLoopJoin [codegen id : 2] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 2] -Output [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] -Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(21) Expand [codegen id : 2] -Input [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] -Arguments: [[inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14, 0], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, null, 1], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, null, null, 3], [inv_quantity_on_hand#2, i_product_name#15, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] - -(22) HashAggregate [codegen id : 2] -Input [6]: [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Functions [1]: [partial_avg(inv_quantity_on_hand#2)] -Aggregate Attributes [2]: [sum#21, count#22] -Results [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] - -(23) CometColumnarExchange -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -Arguments: hashpartitioning(i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(24) CometColumnarToRow [codegen id : 3] -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] - -(25) HashAggregate [codegen id : 3] -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Functions [1]: [avg(inv_quantity_on_hand#2)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#25] -Results [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, avg(inv_quantity_on_hand#2)#25 AS qoh#26] - -(26) TakeOrderedAndProject -Input [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#26] -Arguments: 100, [qoh#26 ASC NULLS FIRST, i_product_name#16 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_category#19 ASC NULLS FIRST], [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(29) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(31) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/simplified.txt deleted file mode 100644 index 221c6063ce..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (3) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (2) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - 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_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt deleted file mode 100644 index f85c898208..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt +++ /dev/null @@ -1,183 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (26) -+- * HashAggregate (25) - +- * CometColumnarToRow (24) - +- CometColumnarExchange (23) - +- * HashAggregate (22) - +- * Expand (21) - +- * Project (20) - +- * BroadcastNestedLoopJoin Inner BuildRight (19) - :- * CometColumnarToRow (15) - : +- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - +- BroadcastExchange (18) - +- * CometColumnarToRow (17) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (16) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#3), dynamicpruningexpression(inv_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(inv_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Condition : isnotnull(inv_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [inv_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3, d_date_sk#5] -Arguments: [inv_item_sk#1, inv_quantity_on_hand#2], [inv_item_sk#1, inv_quantity_on_hand#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Condition : isnotnull(i_item_sk#7) - -(11) CometProject -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#8, 50, true, false, true) AS i_brand#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#11, 50, true, false, true) AS i_product_name#15] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(13) CometBroadcastHashJoin -Left output [2]: [inv_item_sk#1, inv_quantity_on_hand#2] -Right output [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [inv_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15], [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(15) CometColumnarToRow [codegen id : 2] -Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output: [] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -ReadSchema: struct<> - -(17) CometColumnarToRow [codegen id : 1] -Input: [] - -(18) BroadcastExchange -Input: [] -Arguments: IdentityBroadcastMode, [plan_id=1] - -(19) BroadcastNestedLoopJoin [codegen id : 2] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 2] -Output [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] -Input [5]: [inv_quantity_on_hand#2, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(21) Expand [codegen id : 2] -Input [5]: [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14] -Arguments: [[inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, i_category#14, 0], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, i_class#13, null, 1], [inv_quantity_on_hand#2, i_product_name#15, i_brand#12, null, null, 3], [inv_quantity_on_hand#2, i_product_name#15, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] - -(22) HashAggregate [codegen id : 2] -Input [6]: [inv_quantity_on_hand#2, i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Functions [1]: [partial_avg(inv_quantity_on_hand#2)] -Aggregate Attributes [2]: [sum#21, count#22] -Results [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] - -(23) CometColumnarExchange -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -Arguments: hashpartitioning(i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(24) CometColumnarToRow [codegen id : 3] -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] - -(25) HashAggregate [codegen id : 3] -Input [7]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20, sum#23, count#24] -Keys [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, spark_grouping_id#20] -Functions [1]: [avg(inv_quantity_on_hand#2)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#25] -Results [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, avg(inv_quantity_on_hand#2)#25 AS qoh#26] - -(26) TakeOrderedAndProject -Input [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#26] -Arguments: 100, [qoh#26 ASC NULLS FIRST, i_product_name#16 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_category#19 ASC NULLS FIRST], [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) - -(29) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(30) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(31) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/extended.txt deleted file mode 100644 index 6cb89465d2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/extended.txt +++ /dev/null @@ -1,34 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - -Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 221c6063ce..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/simplified.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - WholeStageCodegen (3) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 - WholeStageCodegen (2) - HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] - Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] - BroadcastNestedLoopJoin - CometColumnarToRow - InputAdapter - CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - 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_item_sk,inv_quantity_on_hand,inv_date_sk,d_date_sk] - CometFilter [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/explain.txt deleted file mode 100644 index e53305a55b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/explain.txt +++ /dev/null @@ -1,356 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (55) -+- Union (54) - :- * HashAggregate (25) - : +- * HashAggregate (24) - : +- * HashAggregate (23) - : +- * CometColumnarToRow (22) - : +- CometColumnarExchange (21) - : +- * HashAggregate (20) - : +- * Project (19) - : +- * BroadcastHashJoin Inner BuildRight (18) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.inventory (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.item (7) - : +- BroadcastExchange (17) - : +- * CometColumnarToRow (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.warehouse (14) - :- * HashAggregate (32) - : +- * CometColumnarToRow (31) - : +- CometColumnarExchange (30) - : +- * HashAggregate (29) - : +- * HashAggregate (28) - : +- * CometColumnarToRow (27) - : +- ReusedExchange (26) - :- * HashAggregate (39) - : +- * CometColumnarToRow (38) - : +- CometColumnarExchange (37) - : +- * HashAggregate (36) - : +- * HashAggregate (35) - : +- * CometColumnarToRow (34) - : +- ReusedExchange (33) - :- * HashAggregate (46) - : +- * CometColumnarToRow (45) - : +- CometColumnarExchange (44) - : +- * HashAggregate (43) - : +- * HashAggregate (42) - : +- * CometColumnarToRow (41) - : +- ReusedExchange (40) - +- * HashAggregate (53) - +- * CometColumnarToRow (52) - +- CometColumnarExchange (51) - +- * HashAggregate (50) - +- * HashAggregate (49) - +- * CometColumnarToRow (48) - +- ReusedExchange (47) - - -(1) Scan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] - -(3) Filter [codegen id : 4] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(4) ReusedExchange [Reuses operator id: 60] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(8) CometFilter -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Condition : isnotnull(i_item_sk#7) - -(9) CometProject -Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] -Arguments: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#8, 50, true, false, true) AS i_brand#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#11, 50, true, false, true) AS i_product_name#15] - -(10) CometColumnarToRow [codegen id : 2] -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(11) BroadcastExchange -Input [5]: [i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#7, i_brand#12, i_class#13, i_category#14, i_product_name#15] - -(14) CometNativeScan parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(15) CometFilter -Input [1]: [w_warehouse_sk#16] -Condition : isnotnull(w_warehouse_sk#16) - -(16) CometColumnarToRow [codegen id : 3] -Input [1]: [w_warehouse_sk#16] - -(17) BroadcastExchange -Input [1]: [w_warehouse_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#16] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [5]: [inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15, w_warehouse_sk#16] - -(20) HashAggregate [codegen id : 4] -Input [5]: [inv_quantity_on_hand#3, i_brand#12, i_class#13, i_category#14, i_product_name#15] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [sum#17, count#18] -Results [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] - -(21) CometColumnarExchange -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] -Arguments: hashpartitioning(i_product_name#15, i_brand#12, i_class#13, i_category#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] - -(23) HashAggregate [codegen id : 5] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#19, count#20] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#21] -Results [5]: [i_product_name#15 AS i_product_name#22, i_brand#12 AS i_brand#23, i_class#13 AS i_class#24, i_category#14 AS i_category#25, avg(inv_quantity_on_hand#3)#21 AS qoh#26] - -(24) HashAggregate [codegen id : 5] -Input [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#26] -Keys [4]: [i_product_name#22, i_brand#23, i_class#24, i_category#25] -Functions [1]: [partial_avg(qoh#26)] -Aggregate Attributes [2]: [sum#27, count#28] -Results [6]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, sum#29, count#30] - -(25) HashAggregate [codegen id : 5] -Input [6]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, sum#29, count#30] -Keys [4]: [i_product_name#22, i_brand#23, i_class#24, i_category#25] -Functions [1]: [avg(qoh#26)] -Aggregate Attributes [1]: [avg(qoh#26)#31] -Results [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, avg(qoh#26)#31 AS qoh#32] - -(26) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] - -(27) CometColumnarToRow [codegen id : 10] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] - -(28) HashAggregate [codegen id : 10] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#33, count#34] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [avg(inv_quantity_on_hand#35)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#35)#21] -Results [4]: [i_product_name#15, i_brand#12, i_class#13, avg(inv_quantity_on_hand#35)#21 AS qoh#26] - -(29) HashAggregate [codegen id : 10] -Input [4]: [i_product_name#15, i_brand#12, i_class#13, qoh#26] -Keys [3]: [i_product_name#15, i_brand#12, i_class#13] -Functions [1]: [partial_avg(qoh#26)] -Aggregate Attributes [2]: [sum#36, count#37] -Results [5]: [i_product_name#15, i_brand#12, i_class#13, sum#38, count#39] - -(30) CometColumnarExchange -Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#38, count#39] -Arguments: hashpartitioning(i_product_name#15, i_brand#12, i_class#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(31) CometColumnarToRow [codegen id : 11] -Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#38, count#39] - -(32) HashAggregate [codegen id : 11] -Input [5]: [i_product_name#15, i_brand#12, i_class#13, sum#38, count#39] -Keys [3]: [i_product_name#15, i_brand#12, i_class#13] -Functions [1]: [avg(qoh#26)] -Aggregate Attributes [1]: [avg(qoh#26)#40] -Results [5]: [i_product_name#15, i_brand#12, i_class#13, null AS i_category#41, avg(qoh#26)#40 AS qoh#42] - -(33) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#43, count#44] - -(34) CometColumnarToRow [codegen id : 16] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#43, count#44] - -(35) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#43, count#44] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [avg(inv_quantity_on_hand#45)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#45)#21] -Results [3]: [i_product_name#15, i_brand#12, avg(inv_quantity_on_hand#45)#21 AS qoh#26] - -(36) HashAggregate [codegen id : 16] -Input [3]: [i_product_name#15, i_brand#12, qoh#26] -Keys [2]: [i_product_name#15, i_brand#12] -Functions [1]: [partial_avg(qoh#26)] -Aggregate Attributes [2]: [sum#46, count#47] -Results [4]: [i_product_name#15, i_brand#12, sum#48, count#49] - -(37) CometColumnarExchange -Input [4]: [i_product_name#15, i_brand#12, sum#48, count#49] -Arguments: hashpartitioning(i_product_name#15, i_brand#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(38) CometColumnarToRow [codegen id : 17] -Input [4]: [i_product_name#15, i_brand#12, sum#48, count#49] - -(39) HashAggregate [codegen id : 17] -Input [4]: [i_product_name#15, i_brand#12, sum#48, count#49] -Keys [2]: [i_product_name#15, i_brand#12] -Functions [1]: [avg(qoh#26)] -Aggregate Attributes [1]: [avg(qoh#26)#50] -Results [5]: [i_product_name#15, i_brand#12, null AS i_class#51, null AS i_category#52, avg(qoh#26)#50 AS qoh#53] - -(40) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#54, count#55] - -(41) CometColumnarToRow [codegen id : 22] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#54, count#55] - -(42) HashAggregate [codegen id : 22] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#54, count#55] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [avg(inv_quantity_on_hand#56)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#56)#21] -Results [2]: [i_product_name#15, avg(inv_quantity_on_hand#56)#21 AS qoh#26] - -(43) HashAggregate [codegen id : 22] -Input [2]: [i_product_name#15, qoh#26] -Keys [1]: [i_product_name#15] -Functions [1]: [partial_avg(qoh#26)] -Aggregate Attributes [2]: [sum#57, count#58] -Results [3]: [i_product_name#15, sum#59, count#60] - -(44) CometColumnarExchange -Input [3]: [i_product_name#15, sum#59, count#60] -Arguments: hashpartitioning(i_product_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(45) CometColumnarToRow [codegen id : 23] -Input [3]: [i_product_name#15, sum#59, count#60] - -(46) HashAggregate [codegen id : 23] -Input [3]: [i_product_name#15, sum#59, count#60] -Keys [1]: [i_product_name#15] -Functions [1]: [avg(qoh#26)] -Aggregate Attributes [1]: [avg(qoh#26)#61] -Results [5]: [i_product_name#15, null AS i_brand#62, null AS i_class#63, null AS i_category#64, avg(qoh#26)#61 AS qoh#65] - -(47) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#66, count#67] - -(48) CometColumnarToRow [codegen id : 28] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#66, count#67] - -(49) HashAggregate [codegen id : 28] -Input [6]: [i_product_name#15, i_brand#12, i_class#13, i_category#14, sum#66, count#67] -Keys [4]: [i_product_name#15, i_brand#12, i_class#13, i_category#14] -Functions [1]: [avg(inv_quantity_on_hand#68)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#68)#21] -Results [1]: [avg(inv_quantity_on_hand#68)#21 AS qoh#26] - -(50) HashAggregate [codegen id : 28] -Input [1]: [qoh#26] -Keys: [] -Functions [1]: [partial_avg(qoh#26)] -Aggregate Attributes [2]: [sum#69, count#70] -Results [2]: [sum#71, count#72] - -(51) CometColumnarExchange -Input [2]: [sum#71, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(52) CometColumnarToRow [codegen id : 29] -Input [2]: [sum#71, count#72] - -(53) HashAggregate [codegen id : 29] -Input [2]: [sum#71, count#72] -Keys: [] -Functions [1]: [avg(qoh#26)] -Aggregate Attributes [1]: [avg(qoh#26)#73] -Results [5]: [null AS i_product_name#74, null AS i_brand#75, null AS i_class#76, null AS i_category#77, avg(qoh#26)#73 AS qoh#78] - -(54) Union - -(55) TakeOrderedAndProject -Input [5]: [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32] -Arguments: 100, [qoh#32 ASC NULLS FIRST, i_product_name#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, i_class#24 ASC NULLS FIRST, i_category#25 ASC NULLS FIRST], [i_product_name#22, i_brand#23, i_class#24, i_category#25, qoh#32] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (60) -+- * CometColumnarToRow (59) - +- CometProject (58) - +- CometFilter (57) - +- CometNativeScan parquet spark_catalog.default.date_dim (56) - - -(56) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#79] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(57) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#79] -Condition : (((isnotnull(d_month_seq#79) AND (d_month_seq#79 >= 1212)) AND (d_month_seq#79 <= 1223)) AND isnotnull(d_date_sk#6)) - -(58) CometProject -Input [2]: [d_date_sk#6, d_month_seq#79] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(59) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(60) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/simplified.txt deleted file mode 100644 index a55feaac8c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] - Union - WholeStageCodegen (5) - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),i_product_name,i_brand,i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class,i_category] #1 - WholeStageCodegen (4) - HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] - Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [inv_item_sk,i_item_sk] - Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] - BroadcastHashJoin [inv_date_sk,d_date_sk] - Filter [inv_item_sk,inv_warehouse_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk] - WholeStageCodegen (11) - HashAggregate [i_product_name,i_brand,i_class,sum,count] [avg(qoh),i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand,i_class] #5 - WholeStageCodegen (10) - HashAggregate [i_product_name,i_brand,i_class,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (17) - HashAggregate [i_product_name,i_brand,sum,count] [avg(qoh),i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name,i_brand] #6 - WholeStageCodegen (16) - HashAggregate [i_product_name,i_brand,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (23) - HashAggregate [i_product_name,sum,count] [avg(qoh),i_brand,i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_product_name] #7 - WholeStageCodegen (22) - HashAggregate [i_product_name,qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - WholeStageCodegen (29) - HashAggregate [sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (28) - HashAggregate [qoh] [sum,count,sum,count] - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] - CometColumnarToRow - InputAdapter - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/explain.txt deleted file mode 100644 index 189a54b584..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,301 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometTakeOrderedAndProject (46) - +- CometUnion (45) - :- CometHashAggregate (24) - : +- CometHashAggregate (23) - : +- CometHashAggregate (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - : +- CometBroadcastExchange (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (15) - :- CometHashAggregate (29) - : +- CometExchange (28) - : +- CometHashAggregate (27) - : +- CometHashAggregate (26) - : +- ReusedExchange (25) - :- CometHashAggregate (34) - : +- CometExchange (33) - : +- CometHashAggregate (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - :- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometHashAggregate (36) - : +- ReusedExchange (35) - +- CometHashAggregate (44) - +- CometExchange (43) - +- CometHashAggregate (42) - +- CometHashAggregate (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [inv_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] -Arguments: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3], [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Condition : isnotnull(i_item_sk#8) - -(11) CometProject -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#12, 50, true, false, true) AS i_product_name#16] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(13) CometBroadcastHashJoin -Left output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -Right output [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight - -(14) CometProject -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [w_warehouse_sk#17] -Condition : isnotnull(w_warehouse_sk#17) - -(17) CometBroadcastExchange -Input [1]: [w_warehouse_sk#17] -Arguments: [w_warehouse_sk#17] - -(18) CometBroadcastHashJoin -Left output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Right output [1]: [w_warehouse_sk#17] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#17], Inner, BuildRight - -(19) CometProject -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16, w_warehouse_sk#17] -Arguments: [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(20) CometHashAggregate -Input [5]: [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [partial_avg(inv_quantity_on_hand#3)] - -(21) CometExchange -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#18, count#19] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, i_category#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#18, count#19] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#3)] - -(23) CometHashAggregate -Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#24] -Keys [4]: [i_product_name#20, i_brand#21, i_class#22, i_category#23] -Functions [1]: [partial_avg(qoh#24)] - -(24) CometHashAggregate -Input [6]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, sum#25, count#26] -Keys [4]: [i_product_name#20, i_brand#21, i_class#22, i_category#23] -Functions [1]: [avg(qoh#24)] - -(25) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#27, count#28] - -(26) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#27, count#28] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#29)] - -(27) CometHashAggregate -Input [4]: [i_product_name#16, i_brand#13, i_class#14, qoh#24] -Keys [3]: [i_product_name#16, i_brand#13, i_class#14] -Functions [1]: [partial_avg(qoh#24)] - -(28) CometExchange -Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#30, count#31] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#30, count#31] -Keys [3]: [i_product_name#16, i_brand#13, i_class#14] -Functions [1]: [avg(qoh#24)] - -(30) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#32, count#33] - -(31) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#32, count#33] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#34)] - -(32) CometHashAggregate -Input [3]: [i_product_name#16, i_brand#13, qoh#24] -Keys [2]: [i_product_name#16, i_brand#13] -Functions [1]: [partial_avg(qoh#24)] - -(33) CometExchange -Input [4]: [i_product_name#16, i_brand#13, sum#35, count#36] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(34) CometHashAggregate -Input [4]: [i_product_name#16, i_brand#13, sum#35, count#36] -Keys [2]: [i_product_name#16, i_brand#13] -Functions [1]: [avg(qoh#24)] - -(35) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#37, count#38] - -(36) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#37, count#38] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#39)] - -(37) CometHashAggregate -Input [2]: [i_product_name#16, qoh#24] -Keys [1]: [i_product_name#16] -Functions [1]: [partial_avg(qoh#24)] - -(38) CometExchange -Input [3]: [i_product_name#16, sum#40, count#41] -Arguments: hashpartitioning(i_product_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(39) CometHashAggregate -Input [3]: [i_product_name#16, sum#40, count#41] -Keys [1]: [i_product_name#16] -Functions [1]: [avg(qoh#24)] - -(40) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#42, count#43] - -(41) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#42, count#43] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#44)] - -(42) CometHashAggregate -Input [1]: [qoh#24] -Keys: [] -Functions [1]: [partial_avg(qoh#24)] - -(43) CometExchange -Input [2]: [sum#45, count#46] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(44) CometHashAggregate -Input [2]: [sum#45, count#46] -Keys: [] -Functions [1]: [avg(qoh#24)] - -(45) CometUnion -Child 0 Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#47] -Child 1 Input [5]: [i_product_name#16, i_brand#13, i_class#14, i_category#48, qoh#49] -Child 2 Input [5]: [i_product_name#16, i_brand#13, i_class#50, i_category#51, qoh#52] -Child 3 Input [5]: [i_product_name#16, i_brand#53, i_class#54, i_category#55, qoh#56] -Child 4 Input [5]: [i_product_name#57, i_brand#58, i_class#59, i_category#60, qoh#61] - -(46) CometTakeOrderedAndProject -Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#47] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#47 ASC NULLS FIRST,i_product_name#20 ASC NULLS FIRST,i_brand#21 ASC NULLS FIRST,i_class#22 ASC NULLS FIRST,i_category#23 ASC NULLS FIRST], output=[i_product_name#20,i_brand#21,i_class#22,i_category#23,qoh#47]), [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#47], 100, 0, [qoh#47 ASC NULLS FIRST, i_product_name#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, i_class#22 ASC NULLS FIRST, i_category#23 ASC NULLS FIRST], [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#47] - -(47) CometColumnarToRow [codegen id : 1] -Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#47] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) - -(50) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(52) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/simplified.txt deleted file mode 100644 index b59605103e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,57 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] - CometUnion [i_product_name,i_brand,i_class,i_category,qoh] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometHashAggregate [qoh] [i_product_name,i_brand,i_class,i_category,sum,count] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,i_product_name,i_brand,i_class,i_category,avg(inv_quantity_on_hand)] - CometExchange [i_product_name,i_brand,i_class,i_category] #1 - CometHashAggregate [inv_quantity_on_hand] [i_product_name,i_brand,i_class,i_category,sum,count] - CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - 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,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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange [w_warehouse_sk] #5 - CometFilter [w_warehouse_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name,i_brand,i_class] #6 - CometHashAggregate [qoh] [i_product_name,i_brand,i_class,sum,count] - CometHashAggregate [i_category,sum,count] [i_product_name,i_brand,i_class,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name,i_brand] #7 - CometHashAggregate [qoh] [i_product_name,i_brand,sum,count] - CometHashAggregate [i_class,i_category,sum,count] [i_product_name,i_brand,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name] #8 - CometHashAggregate [qoh] [i_product_name,sum,count] - CometHashAggregate [i_brand,i_class,i_category,sum,count] [i_product_name,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange #9 - CometHashAggregate [qoh] [sum,count] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt deleted file mode 100644 index 189a54b584..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt +++ /dev/null @@ -1,301 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (47) -+- CometTakeOrderedAndProject (46) - +- CometUnion (45) - :- CometHashAggregate (24) - : +- CometHashAggregate (23) - : +- CometHashAggregate (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - : +- CometBroadcastExchange (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (15) - :- CometHashAggregate (29) - : +- CometExchange (28) - : +- CometHashAggregate (27) - : +- CometHashAggregate (26) - : +- ReusedExchange (25) - :- CometHashAggregate (34) - : +- CometExchange (33) - : +- CometHashAggregate (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - :- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometHashAggregate (36) - : +- ReusedExchange (35) - +- CometHashAggregate (44) - +- CometExchange (43) - +- CometHashAggregate (42) - +- CometHashAggregate (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [inv_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#6] -Arguments: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3], [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Condition : isnotnull(i_item_sk#8) - -(11) CometProject -Input [5]: [i_item_sk#8, i_brand#9, i_class#10, i_category#11, i_product_name#12] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#9, 50, true, false, true) AS i_brand#13, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#12, 50, true, false, true) AS i_product_name#16] - -(12) CometBroadcastExchange -Input [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(13) CometBroadcastHashJoin -Left output [3]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3] -Right output [5]: [i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_item_sk#1], [i_item_sk#8], Inner, BuildRight - -(14) CometProject -Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#8, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Arguments: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [1]: [w_warehouse_sk#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(16) CometFilter -Input [1]: [w_warehouse_sk#17] -Condition : isnotnull(w_warehouse_sk#17) - -(17) CometBroadcastExchange -Input [1]: [w_warehouse_sk#17] -Arguments: [w_warehouse_sk#17] - -(18) CometBroadcastHashJoin -Left output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Right output [1]: [w_warehouse_sk#17] -Arguments: [inv_warehouse_sk#2], [w_warehouse_sk#17], Inner, BuildRight - -(19) CometProject -Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16, w_warehouse_sk#17] -Arguments: [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16], [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] - -(20) CometHashAggregate -Input [5]: [inv_quantity_on_hand#3, i_brand#13, i_class#14, i_category#15, i_product_name#16] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [partial_avg(inv_quantity_on_hand#3)] - -(21) CometExchange -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#18, count#19] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, i_category#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#18, count#19] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#3)] - -(23) CometHashAggregate -Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#24] -Keys [4]: [i_product_name#20, i_brand#21, i_class#22, i_category#23] -Functions [1]: [partial_avg(qoh#24)] - -(24) CometHashAggregate -Input [6]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, sum#25, count#26] -Keys [4]: [i_product_name#20, i_brand#21, i_class#22, i_category#23] -Functions [1]: [avg(qoh#24)] - -(25) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#27, count#28] - -(26) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#27, count#28] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#29)] - -(27) CometHashAggregate -Input [4]: [i_product_name#16, i_brand#13, i_class#14, qoh#24] -Keys [3]: [i_product_name#16, i_brand#13, i_class#14] -Functions [1]: [partial_avg(qoh#24)] - -(28) CometExchange -Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#30, count#31] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometHashAggregate -Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#30, count#31] -Keys [3]: [i_product_name#16, i_brand#13, i_class#14] -Functions [1]: [avg(qoh#24)] - -(30) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#32, count#33] - -(31) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#32, count#33] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#34)] - -(32) CometHashAggregate -Input [3]: [i_product_name#16, i_brand#13, qoh#24] -Keys [2]: [i_product_name#16, i_brand#13] -Functions [1]: [partial_avg(qoh#24)] - -(33) CometExchange -Input [4]: [i_product_name#16, i_brand#13, sum#35, count#36] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(34) CometHashAggregate -Input [4]: [i_product_name#16, i_brand#13, sum#35, count#36] -Keys [2]: [i_product_name#16, i_brand#13] -Functions [1]: [avg(qoh#24)] - -(35) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#37, count#38] - -(36) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#37, count#38] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#39)] - -(37) CometHashAggregate -Input [2]: [i_product_name#16, qoh#24] -Keys [1]: [i_product_name#16] -Functions [1]: [partial_avg(qoh#24)] - -(38) CometExchange -Input [3]: [i_product_name#16, sum#40, count#41] -Arguments: hashpartitioning(i_product_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(39) CometHashAggregate -Input [3]: [i_product_name#16, sum#40, count#41] -Keys [1]: [i_product_name#16] -Functions [1]: [avg(qoh#24)] - -(40) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#42, count#43] - -(41) CometHashAggregate -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#42, count#43] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#44)] - -(42) CometHashAggregate -Input [1]: [qoh#24] -Keys: [] -Functions [1]: [partial_avg(qoh#24)] - -(43) CometExchange -Input [2]: [sum#45, count#46] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(44) CometHashAggregate -Input [2]: [sum#45, count#46] -Keys: [] -Functions [1]: [avg(qoh#24)] - -(45) CometUnion -Child 0 Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#47] -Child 1 Input [5]: [i_product_name#16, i_brand#13, i_class#14, i_category#48, qoh#49] -Child 2 Input [5]: [i_product_name#16, i_brand#13, i_class#50, i_category#51, qoh#52] -Child 3 Input [5]: [i_product_name#16, i_brand#53, i_class#54, i_category#55, qoh#56] -Child 4 Input [5]: [i_product_name#57, i_brand#58, i_class#59, i_category#60, qoh#61] - -(46) CometTakeOrderedAndProject -Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#47] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[qoh#47 ASC NULLS FIRST,i_product_name#20 ASC NULLS FIRST,i_brand#21 ASC NULLS FIRST,i_class#22 ASC NULLS FIRST,i_category#23 ASC NULLS FIRST], output=[i_product_name#20,i_brand#21,i_class#22,i_category#23,qoh#47]), [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#47], 100, 0, [qoh#47 ASC NULLS FIRST, i_product_name#20 ASC NULLS FIRST, i_brand#21 ASC NULLS FIRST, i_class#22 ASC NULLS FIRST, i_category#23 ASC NULLS FIRST], [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#47] - -(47) CometColumnarToRow [codegen id : 1] -Input [5]: [i_product_name#20, i_brand#21, i_class#22, i_category#23, qoh#47] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [2]: [d_date_sk#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#6)) - -(50) CometProject -Input [2]: [d_date_sk#6, d_month_seq#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(52) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/extended.txt deleted file mode 100644 index c018673888..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/extended.txt +++ /dev/null @@ -1,159 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - -Comet accelerated 141 out of 151 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index b59605103e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt +++ /dev/null @@ -1,57 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_product_name,i_brand,i_class,i_category,qoh] - CometUnion [i_product_name,i_brand,i_class,i_category,qoh] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometHashAggregate [qoh] [i_product_name,i_brand,i_class,i_category,sum,count] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,i_product_name,i_brand,i_class,i_category,avg(inv_quantity_on_hand)] - CometExchange [i_product_name,i_brand,i_class,i_category] #1 - CometHashAggregate [inv_quantity_on_hand] [i_product_name,i_brand,i_class,i_category,sum,count] - CometProject [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] - 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,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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #3 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #4 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometBroadcastExchange [w_warehouse_sk] #5 - CometFilter [w_warehouse_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk] - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name,i_brand,i_class] #6 - CometHashAggregate [qoh] [i_product_name,i_brand,i_class,sum,count] - CometHashAggregate [i_category,sum,count] [i_product_name,i_brand,i_class,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name,i_brand] #7 - CometHashAggregate [qoh] [i_product_name,i_brand,sum,count] - CometHashAggregate [i_class,i_category,sum,count] [i_product_name,i_brand,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange [i_product_name] #8 - CometHashAggregate [qoh] [i_product_name,sum,count] - CometHashAggregate [i_brand,i_class,i_category,sum,count] [i_product_name,qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 - CometHashAggregate [sum,count] [i_product_name,i_brand,i_class,i_category,qoh,avg(qoh)] - CometExchange #9 - CometHashAggregate [qoh] [sum,count] - CometHashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [qoh,avg(inv_quantity_on_hand)] - ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/explain.txt deleted file mode 100644 index 6af787c781..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/explain.txt +++ /dev/null @@ -1,453 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (50) -+- CometSort (49) - +- CometColumnarExchange (48) - +- * Filter (47) - +- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometNativeScan parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometNativeScan parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.customer_address (32) - - -(1) CometNativeScan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometNativeScan parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) = pale ) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#19, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#21, 10, true, false, true) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometNativeScan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Condition : ((isnotnull(c_customer_sk#26) AND isnotnull(c_current_addr_sk#27)) AND isnotnull(c_birth_country#30)) - -(27) CometProject -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [c_customer_sk#26, c_current_addr_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#28, 20, true, false, true) AS c_first_name#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#29, 30, true, false, true) AS c_last_name#32, c_birth_country#30] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(31) CometColumnarToRow [codegen id : 2] -Input [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(32) CometNativeScan parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Condition : ((isnotnull(ca_address_sk#33) AND isnotnull(ca_country#36)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true))) - -(34) CometProject -Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Arguments: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36], [ca_address_sk#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#34, 2, true, false, true) AS ca_state#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true) AS ca_zip#38, ca_country#36] - -(35) CometColumnarToRow [codegen id : 1] -Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] - -(36) BroadcastExchange -Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [3]: [c_current_addr_sk#27, c_birth_country#30, s_zip#16] -Right keys [3]: [ca_address_sk#33, upper(ca_country#36), ca_zip#38] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] -Input [17]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30, ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#39] -Results [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] - -(40) CometColumnarExchange -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#41] -Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#41,17,2) AS netpaid#42] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, netpaid#42] -Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] -Functions [1]: [partial_sum(netpaid#42)] -Aggregate Attributes [2]: [sum#43, isEmpty#44] -Results [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] - -(44) CometColumnarExchange -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] -Functions [1]: [sum(netpaid#42)] -Aggregate Attributes [1]: [sum(netpaid#42)#47] -Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, sum(netpaid#42)#47 AS paid#48] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) - -(48) CometColumnarExchange -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(49) CometSort -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Arguments: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48], [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] - -(50) CometColumnarToRow [codegen id : 5] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#49, [id=#50] -* HashAggregate (80) -+- * CometColumnarToRow (79) - +- CometColumnarExchange (78) - +- * HashAggregate (77) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- CometColumnarExchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * CometColumnarToRow (69) - : +- CometProject (68) - : +- CometBroadcastHashJoin (67) - : :- CometProject (65) - : : +- CometBroadcastHashJoin (64) - : : :- CometProject (59) - : : : +- CometBroadcastHashJoin (58) - : : : :- CometProject (56) - : : : : +- CometSortMergeJoin (55) - : : : : :- CometSort (52) - : : : : : +- ReusedExchange (51) - : : : : +- CometSort (54) - : : : : +- ReusedExchange (53) - : : : +- ReusedExchange (57) - : : +- CometBroadcastExchange (63) - : : +- CometProject (62) - : : +- CometFilter (61) - : : +- CometNativeScan parquet spark_catalog.default.item (60) - : +- ReusedExchange (66) - +- ReusedExchange (70) - - -(51) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] - -(52) CometSort -Input [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55], [ss_ticket_number#54 ASC NULLS FIRST, ss_item_sk#51 ASC NULLS FIRST] - -(53) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#56, sr_ticket_number#57] - -(54) CometSort -Input [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [sr_item_sk#56, sr_ticket_number#57], [sr_ticket_number#57 ASC NULLS FIRST, sr_item_sk#56 ASC NULLS FIRST] - -(55) CometSortMergeJoin -Left output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -Right output [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [ss_ticket_number#54, ss_item_sk#51], [sr_ticket_number#57, sr_item_sk#56], Inner - -(56) CometProject -Input [7]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55, sr_item_sk#56, sr_ticket_number#57] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55], [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] - -(57) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] - -(58) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] -Right output [4]: [s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] -Arguments: [ss_store_sk#53], [s_store_sk#58], Inner, BuildRight - -(59) CometProject -Input [8]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55, s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16], [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] - -(60) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(61) CometFilter -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Condition : isnotnull(i_item_sk#60) - -(62) CometProject -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65], [i_item_sk#60, i_current_price#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#62, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#63, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#64, 10, true, false, true) AS i_units#25, i_manager_id#65] - -(63) CometBroadcastExchange -Input [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] - -(64) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [ss_item_sk#51], [i_item_sk#60], Inner, BuildRight - -(65) CometProject -Input [12]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65], [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] - -(66) ReusedExchange [Reuses operator id: 28] -Output [5]: [c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] - -(67) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Right output [5]: [c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -Arguments: [ss_customer_sk#52], [c_customer_sk#66], Inner, BuildRight - -(68) CometProject -Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -Arguments: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68], [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] - -(69) CometColumnarToRow [codegen id : 2] -Input [13]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] - -(70) ReusedExchange [Reuses operator id: 36] -Output [4]: [ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] - -(71) BroadcastHashJoin [codegen id : 2] -Left keys [3]: [c_current_addr_sk#67, c_birth_country#68, s_zip#16] -Right keys [3]: [ca_address_sk#69, upper(ca_country#70), ca_zip#38] -Join type: Inner -Join condition: None - -(72) Project [codegen id : 2] -Output [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] -Input [17]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68, ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] - -(73) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [sum#71] -Results [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] - -(74) CometColumnarExchange -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(75) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] - -(76) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#41] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#41,17,2) AS netpaid#42] - -(77) HashAggregate [codegen id : 3] -Input [1]: [netpaid#42] -Keys: [] -Functions [1]: [partial_avg(netpaid#42)] -Aggregate Attributes [2]: [sum#73, count#74] -Results [2]: [sum#75, count#76] - -(78) CometColumnarExchange -Input [2]: [sum#75, count#76] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(79) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#75, count#76] - -(80) HashAggregate [codegen id : 4] -Input [2]: [sum#75, count#76] -Keys: [] -Functions [1]: [avg(netpaid#42)] -Aggregate Attributes [1]: [avg(netpaid#42)#77] -Results [1]: [(0.05 * avg(netpaid#42)#77) AS (0.05 * avg(netpaid))#78] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/simplified.txt deleted file mode 100644 index b24e51723a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/simplified.txt +++ /dev/null @@ -1,99 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,s_store_name,paid] - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #5 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #12 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - InputAdapter - ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #4 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometNativeScan 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #5 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #7 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometNativeScan 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 (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] - CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] - CometNativeScan 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/q24.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/explain.txt deleted file mode 100644 index 02873e35d9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,453 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (50) -+- CometSort (49) - +- CometColumnarExchange (48) - +- * Filter (47) - +- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) = pale ) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#19, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#21, 10, true, false, true) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Condition : ((isnotnull(c_customer_sk#26) AND isnotnull(c_current_addr_sk#27)) AND isnotnull(c_birth_country#30)) - -(27) CometProject -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [c_customer_sk#26, c_current_addr_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#28, 20, true, false, true) AS c_first_name#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#29, 30, true, false, true) AS c_last_name#32, c_birth_country#30] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(31) CometColumnarToRow [codegen id : 2] -Input [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Condition : ((isnotnull(ca_address_sk#33) AND isnotnull(ca_country#36)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true))) - -(34) CometProject -Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Arguments: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36], [ca_address_sk#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#34, 2, true, false, true) AS ca_state#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true) AS ca_zip#38, ca_country#36] - -(35) CometColumnarToRow [codegen id : 1] -Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] - -(36) BroadcastExchange -Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [3]: [c_current_addr_sk#27, c_birth_country#30, s_zip#16] -Right keys [3]: [ca_address_sk#33, upper(ca_country#36), ca_zip#38] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] -Input [17]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30, ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#39] -Results [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] - -(40) CometColumnarExchange -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#41] -Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#41,17,2) AS netpaid#42] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, netpaid#42] -Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] -Functions [1]: [partial_sum(netpaid#42)] -Aggregate Attributes [2]: [sum#43, isEmpty#44] -Results [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] - -(44) CometColumnarExchange -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] -Functions [1]: [sum(netpaid#42)] -Aggregate Attributes [1]: [sum(netpaid#42)#47] -Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, sum(netpaid#42)#47 AS paid#48] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) - -(48) CometColumnarExchange -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(49) CometSort -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Arguments: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48], [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] - -(50) CometColumnarToRow [codegen id : 5] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#49, [id=#50] -* HashAggregate (80) -+- * CometColumnarToRow (79) - +- CometColumnarExchange (78) - +- * HashAggregate (77) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- CometColumnarExchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * CometColumnarToRow (69) - : +- CometProject (68) - : +- CometBroadcastHashJoin (67) - : :- CometProject (65) - : : +- CometBroadcastHashJoin (64) - : : :- CometProject (59) - : : : +- CometBroadcastHashJoin (58) - : : : :- CometProject (56) - : : : : +- CometSortMergeJoin (55) - : : : : :- CometSort (52) - : : : : : +- ReusedExchange (51) - : : : : +- CometSort (54) - : : : : +- ReusedExchange (53) - : : : +- ReusedExchange (57) - : : +- CometBroadcastExchange (63) - : : +- CometProject (62) - : : +- CometFilter (61) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (60) - : +- ReusedExchange (66) - +- ReusedExchange (70) - - -(51) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] - -(52) CometSort -Input [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55], [ss_ticket_number#54 ASC NULLS FIRST, ss_item_sk#51 ASC NULLS FIRST] - -(53) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#56, sr_ticket_number#57] - -(54) CometSort -Input [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [sr_item_sk#56, sr_ticket_number#57], [sr_ticket_number#57 ASC NULLS FIRST, sr_item_sk#56 ASC NULLS FIRST] - -(55) CometSortMergeJoin -Left output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -Right output [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [ss_ticket_number#54, ss_item_sk#51], [sr_ticket_number#57, sr_item_sk#56], Inner - -(56) CometProject -Input [7]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55, sr_item_sk#56, sr_ticket_number#57] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55], [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] - -(57) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] - -(58) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] -Right output [4]: [s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] -Arguments: [ss_store_sk#53], [s_store_sk#58], Inner, BuildRight - -(59) CometProject -Input [8]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55, s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16], [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] - -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(61) CometFilter -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Condition : isnotnull(i_item_sk#60) - -(62) CometProject -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65], [i_item_sk#60, i_current_price#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#62, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#63, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#64, 10, true, false, true) AS i_units#25, i_manager_id#65] - -(63) CometBroadcastExchange -Input [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] - -(64) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [ss_item_sk#51], [i_item_sk#60], Inner, BuildRight - -(65) CometProject -Input [12]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65], [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] - -(66) ReusedExchange [Reuses operator id: 28] -Output [5]: [c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] - -(67) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Right output [5]: [c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -Arguments: [ss_customer_sk#52], [c_customer_sk#66], Inner, BuildRight - -(68) CometProject -Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -Arguments: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68], [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] - -(69) CometColumnarToRow [codegen id : 2] -Input [13]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] - -(70) ReusedExchange [Reuses operator id: 36] -Output [4]: [ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] - -(71) BroadcastHashJoin [codegen id : 2] -Left keys [3]: [c_current_addr_sk#67, c_birth_country#68, s_zip#16] -Right keys [3]: [ca_address_sk#69, upper(ca_country#70), ca_zip#38] -Join type: Inner -Join condition: None - -(72) Project [codegen id : 2] -Output [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] -Input [17]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68, ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] - -(73) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [sum#71] -Results [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] - -(74) CometColumnarExchange -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(75) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] - -(76) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#41] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#41,17,2) AS netpaid#42] - -(77) HashAggregate [codegen id : 3] -Input [1]: [netpaid#42] -Keys: [] -Functions [1]: [partial_avg(netpaid#42)] -Aggregate Attributes [2]: [sum#73, count#74] -Results [2]: [sum#75, count#76] - -(78) CometColumnarExchange -Input [2]: [sum#75, count#76] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(79) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#75, count#76] - -(80) HashAggregate [codegen id : 4] -Input [2]: [sum#75, count#76] -Keys: [] -Functions [1]: [avg(netpaid#42)] -Aggregate Attributes [1]: [avg(netpaid#42)#77] -Results [1]: [(0.05 * avg(netpaid#42)#77) AS (0.05 * avg(netpaid))#78] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/simplified.txt deleted file mode 100644 index 62f492f632..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,99 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,s_store_name,paid] - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #5 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #12 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - InputAdapter - ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #4 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #5 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #7 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometScan [native_iceberg_compat] 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 (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] - CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] 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/q24/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt deleted file mode 100644 index 02873e35d9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt +++ /dev/null @@ -1,453 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (50) -+- CometSort (49) - +- CometColumnarExchange (48) - +- * Filter (47) - +- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * CometColumnarToRow (31) - : +- CometProject (30) - : +- CometBroadcastHashJoin (29) - : :- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (18) - : : : +- CometBroadcastHashJoin (17) - : : : :- CometProject (12) - : : : : +- CometSortMergeJoin (11) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometSort (10) - : : : : +- CometExchange (9) - : : : : +- CometProject (8) - : : : : +- CometFilter (7) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (6) - : : : +- CometBroadcastExchange (16) - : : : +- CometProject (15) - : : : +- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : : +- CometBroadcastExchange (22) - : : +- CometProject (21) - : : +- CometFilter (20) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (19) - : +- CometBroadcastExchange (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (32) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) - -(3) CometProject -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] - -(4) CometExchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(5) CometSort -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) - -(8) CometProject -Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] - -(9) CometExchange -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Right output [2]: [sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_ticket_number#4, ss_item_sk#1], [sr_ticket_number#8, sr_item_sk#7], Inner - -(12) CometProject -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true))) - -(15) CometProject -Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16], [s_store_sk#10, s_store_name#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#14, 10, true, false, true) AS s_zip#16] - -(16) CometBroadcastExchange -Input [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] - -(17) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] -Right output [4]: [s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#15, s_zip#16] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16], [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] - -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) = pale ) AND isnotnull(i_item_sk#17)) - -(21) CometProject -Input [6]: [i_item_sk#17, i_current_price#18, i_size#19, i_color#20, i_units#21, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [i_item_sk#17, i_current_price#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#19, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#20, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#21, 10, true, false, true) AS i_units#25, i_manager_id#22] - -(22) CometBroadcastExchange -Input [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(23) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_item_sk#1], [i_item_sk#17], Inner, BuildRight - -(24) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_item_sk#17, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Arguments: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22], [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Condition : ((isnotnull(c_customer_sk#26) AND isnotnull(c_current_addr_sk#27)) AND isnotnull(c_birth_country#30)) - -(27) CometProject -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#28, c_last_name#29, c_birth_country#30] -Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [c_customer_sk#26, c_current_addr_sk#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#28, 20, true, false, true) AS c_first_name#31, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#29, 30, true, false, true) AS c_last_name#32, c_birth_country#30] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(29) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22] -Right output [5]: [c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [ss_customer_sk#2], [c_customer_sk#26], Inner, BuildRight - -(30) CometProject -Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_customer_sk#26, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] -Arguments: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30], [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(31) CometColumnarToRow [codegen id : 2] -Input [13]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country)] -ReadSchema: struct - -(33) CometFilter -Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Condition : ((isnotnull(ca_address_sk#33) AND isnotnull(ca_country#36)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true))) - -(34) CometProject -Input [4]: [ca_address_sk#33, ca_state#34, ca_zip#35, ca_country#36] -Arguments: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36], [ca_address_sk#33, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#34, 2, true, false, true) AS ca_state#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#35, 10, true, false, true) AS ca_zip#38, ca_country#36] - -(35) CometColumnarToRow [codegen id : 1] -Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] - -(36) BroadcastExchange -Input [4]: [ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[3, string, true]), input[2, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 2] -Left keys [3]: [c_current_addr_sk#27, c_birth_country#30, s_zip#16] -Right keys [3]: [ca_address_sk#33, upper(ca_country#36), ca_zip#38] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 2] -Output [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] -Input [17]: [ss_net_paid#5, s_store_name#11, s_state#15, s_zip#16, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_current_addr_sk#27, c_first_name#31, c_last_name#32, c_birth_country#30, ca_address_sk#33, ca_state#37, ca_zip#38, ca_country#36] - -(39) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#5, s_store_name#11, s_state#15, i_current_price#18, i_size#23, i_color#24, i_units#25, i_manager_id#22, c_first_name#31, c_last_name#32, ca_state#37] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#39] -Results [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] - -(40) CometColumnarExchange -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(41) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] - -(42) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23, sum#40] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#11, ca_state#37, s_state#15, i_color#24, i_current_price#18, i_manager_id#22, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#41] -Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#41,17,2) AS netpaid#42] - -(43) HashAggregate [codegen id : 3] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, netpaid#42] -Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] -Functions [1]: [partial_sum(netpaid#42)] -Aggregate Attributes [2]: [sum#43, isEmpty#44] -Results [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] - -(44) CometColumnarExchange -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 4] -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] - -(46) HashAggregate [codegen id : 4] -Input [5]: [c_last_name#32, c_first_name#31, s_store_name#11, sum#45, isEmpty#46] -Keys [3]: [c_last_name#32, c_first_name#31, s_store_name#11] -Functions [1]: [sum(netpaid#42)] -Aggregate Attributes [1]: [sum(netpaid#42)#47] -Results [4]: [c_last_name#32, c_first_name#31, s_store_name#11, sum(netpaid#42)#47 AS paid#48] - -(47) Filter [codegen id : 4] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Condition : (isnotnull(paid#48) AND (cast(paid#48 as decimal(33,8)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(33,8)))) - -(48) CometColumnarExchange -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Arguments: rangepartitioning(c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(49) CometSort -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] -Arguments: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48], [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST] - -(50) CometColumnarToRow [codegen id : 5] -Input [4]: [c_last_name#32, c_first_name#31, s_store_name#11, paid#48] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#49, [id=#50] -* HashAggregate (80) -+- * CometColumnarToRow (79) - +- CometColumnarExchange (78) - +- * HashAggregate (77) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- CometColumnarExchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * CometColumnarToRow (69) - : +- CometProject (68) - : +- CometBroadcastHashJoin (67) - : :- CometProject (65) - : : +- CometBroadcastHashJoin (64) - : : :- CometProject (59) - : : : +- CometBroadcastHashJoin (58) - : : : :- CometProject (56) - : : : : +- CometSortMergeJoin (55) - : : : : :- CometSort (52) - : : : : : +- ReusedExchange (51) - : : : : +- CometSort (54) - : : : : +- ReusedExchange (53) - : : : +- ReusedExchange (57) - : : +- CometBroadcastExchange (63) - : : +- CometProject (62) - : : +- CometFilter (61) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (60) - : +- ReusedExchange (66) - +- ReusedExchange (70) - - -(51) ReusedExchange [Reuses operator id: 4] -Output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] - -(52) CometSort -Input [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55], [ss_ticket_number#54 ASC NULLS FIRST, ss_item_sk#51 ASC NULLS FIRST] - -(53) ReusedExchange [Reuses operator id: 9] -Output [2]: [sr_item_sk#56, sr_ticket_number#57] - -(54) CometSort -Input [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [sr_item_sk#56, sr_ticket_number#57], [sr_ticket_number#57 ASC NULLS FIRST, sr_item_sk#56 ASC NULLS FIRST] - -(55) CometSortMergeJoin -Left output [5]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55] -Right output [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [ss_ticket_number#54, ss_item_sk#51], [sr_ticket_number#57, sr_item_sk#56], Inner - -(56) CometProject -Input [7]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_ticket_number#54, ss_net_paid#55, sr_item_sk#56, sr_ticket_number#57] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55], [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] - -(57) ReusedExchange [Reuses operator id: 16] -Output [4]: [s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] - -(58) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55] -Right output [4]: [s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] -Arguments: [ss_store_sk#53], [s_store_sk#58], Inner, BuildRight - -(59) CometProject -Input [8]: [ss_item_sk#51, ss_customer_sk#52, ss_store_sk#53, ss_net_paid#55, s_store_sk#58, s_store_name#59, s_state#15, s_zip#16] -Arguments: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16], [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] - -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(61) CometFilter -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Condition : isnotnull(i_item_sk#60) - -(62) CometProject -Input [6]: [i_item_sk#60, i_current_price#61, i_size#62, i_color#63, i_units#64, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65], [i_item_sk#60, i_current_price#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_size#62, 20, true, false, true) AS i_size#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#63, 20, true, false, true) AS i_color#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_units#64, 10, true, false, true) AS i_units#25, i_manager_id#65] - -(63) CometBroadcastExchange -Input [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] - -(64) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16] -Right output [6]: [i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [ss_item_sk#51], [i_item_sk#60], Inner, BuildRight - -(65) CometProject -Input [12]: [ss_item_sk#51, ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_item_sk#60, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Arguments: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65], [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] - -(66) ReusedExchange [Reuses operator id: 28] -Output [5]: [c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] - -(67) CometBroadcastHashJoin -Left output [10]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65] -Right output [5]: [c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -Arguments: [ss_customer_sk#52], [c_customer_sk#66], Inner, BuildRight - -(68) CometProject -Input [15]: [ss_customer_sk#52, ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_customer_sk#66, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] -Arguments: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68], [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] - -(69) CometColumnarToRow [codegen id : 2] -Input [13]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68] - -(70) ReusedExchange [Reuses operator id: 36] -Output [4]: [ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] - -(71) BroadcastHashJoin [codegen id : 2] -Left keys [3]: [c_current_addr_sk#67, c_birth_country#68, s_zip#16] -Right keys [3]: [ca_address_sk#69, upper(ca_country#70), ca_zip#38] -Join type: Inner -Join condition: None - -(72) Project [codegen id : 2] -Output [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] -Input [17]: [ss_net_paid#55, s_store_name#59, s_state#15, s_zip#16, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_current_addr_sk#67, c_first_name#31, c_last_name#32, c_birth_country#68, ca_address_sk#69, ca_state#37, ca_zip#38, ca_country#70] - -(73) HashAggregate [codegen id : 2] -Input [11]: [ss_net_paid#55, s_store_name#59, s_state#15, i_current_price#61, i_size#23, i_color#24, i_units#25, i_manager_id#65, c_first_name#31, c_last_name#32, ca_state#37] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [sum#71] -Results [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] - -(74) CometColumnarExchange -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -Arguments: hashpartitioning(c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(75) CometColumnarToRow [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] - -(76) HashAggregate [codegen id : 3] -Input [11]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23, sum#72] -Keys [10]: [c_last_name#32, c_first_name#31, s_store_name#59, ca_state#37, s_state#15, i_color#24, i_current_price#61, i_manager_id#65, i_units#25, i_size#23] -Functions [1]: [sum(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#55))#41] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#55))#41,17,2) AS netpaid#42] - -(77) HashAggregate [codegen id : 3] -Input [1]: [netpaid#42] -Keys: [] -Functions [1]: [partial_avg(netpaid#42)] -Aggregate Attributes [2]: [sum#73, count#74] -Results [2]: [sum#75, count#76] - -(78) CometColumnarExchange -Input [2]: [sum#75, count#76] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(79) CometColumnarToRow [codegen id : 4] -Input [2]: [sum#75, count#76] - -(80) HashAggregate [codegen id : 4] -Input [2]: [sum#75, count#76] -Keys: [] -Functions [1]: [avg(netpaid#42)] -Aggregate Attributes [1]: [avg(netpaid#42)#77] -Results [1]: [(0.05 * avg(netpaid#42)#77) AS (0.05 * avg(netpaid))#78] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/extended.txt deleted file mode 100644 index b9384ca04f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/extended.txt +++ /dev/null @@ -1,99 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometColumnarExchange - +- Filter - : +- Subquery - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 72 out of 88 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 62f492f632..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/simplified.txt +++ /dev/null @@ -1,99 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,s_store_name,paid] - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #1 - WholeStageCodegen (4) - Filter [paid] - Subquery #1 - WholeStageCodegen (4) - HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (3) - HashAggregate [netpaid] [sum,count,sum,count] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 - CometSort [sr_item_sk,sr_ticket_number] - ReusedExchange [sr_item_sk,sr_ticket_number] #5 - ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #12 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - InputAdapter - ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 - HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name] #2 - WholeStageCodegen (3) - HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 - WholeStageCodegen (2) - HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] - Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] - BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] - CometColumnarToRow - InputAdapter - CometProject [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometBroadcastHashJoin [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometProject [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometProject [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid,s_store_sk,s_store_name,s_state,s_zip] - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometExchange [ss_ticket_number,ss_item_sk] #4 - CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] - CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #5 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [s_store_sk,s_store_name,s_state,s_zip] #6 - CometProject [s_state,s_zip] [s_store_sk,s_store_name,s_state,s_zip] - CometFilter [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] - CometBroadcastExchange [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] #7 - CometProject [i_size,i_color,i_units] [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometFilter [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] - CometScan [native_iceberg_compat] 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 (1) - CometColumnarToRow - InputAdapter - CometProject [ca_state,ca_zip] [ca_address_sk,ca_state,ca_zip,ca_country] - CometFilter [ca_address_sk,ca_state,ca_zip,ca_country] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/explain.txt deleted file mode 100644 index 0723f5c6b2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/explain.txt +++ /dev/null @@ -1,479 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (78) -+- Union (77) - :- * HashAggregate (31) - : +- * CometColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- * HashAggregate (28) - : +- * Project (27) - : +- * BroadcastHashJoin Inner BuildRight (26) - : :- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (4) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (18) - : : +- * CometColumnarToRow (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.store (14) - : +- BroadcastExchange (25) - : +- * CometColumnarToRow (24) - : +- CometProject (23) - : +- CometFilter (22) - : +- CometNativeScan parquet spark_catalog.default.item (21) - :- * HashAggregate (54) - : +- * CometColumnarToRow (53) - : +- CometColumnarExchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * Project (40) - : : : +- * BroadcastHashJoin Inner BuildRight (39) - : : : :- * Project (37) - : : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : : :- * Filter (34) - : : : : : +- * ColumnarToRow (33) - : : : : : +- Scan parquet spark_catalog.default.store_sales (32) - : : : : +- ReusedExchange (35) - : : : +- ReusedExchange (38) - : : +- BroadcastExchange (45) - : : +- * CometColumnarToRow (44) - : : +- CometProject (43) - : : +- CometFilter (42) - : : +- CometNativeScan parquet spark_catalog.default.store (41) - : +- ReusedExchange (48) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- CometColumnarExchange (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- * Project (66) - : +- * BroadcastHashJoin Inner BuildRight (65) - : :- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- * Project (60) - : : : +- * BroadcastHashJoin Inner BuildRight (59) - : : : :- * Filter (57) - : : : : +- * ColumnarToRow (56) - : : : : +- Scan parquet spark_catalog.default.store_sales (55) - : : : +- ReusedExchange (58) - : : +- ReusedExchange (61) - : +- ReusedExchange (64) - +- BroadcastExchange (70) - +- * CometColumnarToRow (69) - +- CometFilter (68) - +- CometNativeScan parquet spark_catalog.default.item (67) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(3) Filter [codegen id : 5] -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) - -(4) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = F) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = W)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = Primary )) AND isnotnull(cd_demo_sk#10)) - -(6) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(7) CometColumnarToRow [codegen id : 1] -Input [1]: [cd_demo_sk#10] - -(8) BroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 5] -Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] - -(11) ReusedExchange [Reuses operator id: 83] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#15, s_state#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#15, s_state#16] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#16, 2, true, false, true) = TN) AND isnotnull(s_store_sk#15)) - -(16) CometProject -Input [2]: [s_store_sk#15, s_state#16] -Arguments: [s_store_sk#15, s_state#17], [s_store_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#16, 2, true, false, true) AS s_state#17] - -(17) CometColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#15, s_state#17] - -(18) BroadcastExchange -Input [2]: [s_store_sk#15, s_state#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#15] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 5] -Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17] -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15, s_state#17] - -(21) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#18, i_item_id#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_item_sk#18, i_item_id#19] -Condition : isnotnull(i_item_sk#18) - -(23) CometProject -Input [2]: [i_item_sk#18, i_item_id#19] -Arguments: [i_item_sk#18, i_item_id#20], [i_item_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#19, 16, true, false, true) AS i_item_id#20] - -(24) CometColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#18, i_item_id#20] - -(25) BroadcastExchange -Input [2]: [i_item_sk#18, i_item_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#18] -Join type: Inner -Join condition: None - -(27) Project [codegen id : 5] -Output [6]: [i_item_id#20 AS i_item_id#21, s_state#17 AS s_state#22, ss_quantity#4 AS agg1#23, ss_list_price#5 AS agg2#24, ss_coupon_amt#7 AS agg3#25, ss_sales_price#6 AS agg4#26] -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#17, i_item_sk#18, i_item_id#20] - -(28) HashAggregate [codegen id : 5] -Input [6]: [i_item_id#21, s_state#22, agg1#23, agg2#24, agg3#25, agg4#26] -Keys [2]: [i_item_id#21, s_state#22] -Functions [4]: [partial_avg(agg1#23), partial_avg(UnscaledValue(agg2#24)), partial_avg(UnscaledValue(agg3#25)), partial_avg(UnscaledValue(agg4#26))] -Aggregate Attributes [8]: [sum#27, count#28, sum#29, count#30, sum#31, count#32, sum#33, count#34] -Results [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] - -(29) CometColumnarExchange -Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] -Arguments: hashpartitioning(i_item_id#21, s_state#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 6] -Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] - -(31) HashAggregate [codegen id : 6] -Input [10]: [i_item_id#21, s_state#22, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42] -Keys [2]: [i_item_id#21, s_state#22] -Functions [4]: [avg(agg1#23), avg(UnscaledValue(agg2#24)), avg(UnscaledValue(agg3#25)), avg(UnscaledValue(agg4#26))] -Aggregate Attributes [4]: [avg(agg1#23)#43, avg(UnscaledValue(agg2#24))#44, avg(UnscaledValue(agg3#25))#45, avg(UnscaledValue(agg4#26))#46] -Results [7]: [i_item_id#21, s_state#22, 0 AS g_state#47, avg(agg1#23)#43 AS agg1#48, cast((avg(UnscaledValue(agg2#24))#44 / 100.0) as decimal(11,6)) AS agg2#49, cast((avg(UnscaledValue(agg3#25))#45 / 100.0) as decimal(11,6)) AS agg3#50, cast((avg(UnscaledValue(agg4#26))#46 / 100.0) as decimal(11,6)) AS agg4#51] - -(32) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(33) ColumnarToRow [codegen id : 11] -Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] - -(34) Filter [codegen id : 11] -Input [8]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -Condition : ((isnotnull(ss_cdemo_sk#53) AND isnotnull(ss_store_sk#54)) AND isnotnull(ss_item_sk#52)) - -(35) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#60] - -(36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#53] -Right keys [1]: [cd_demo_sk#60] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 11] -Output [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59] -Input [9]: [ss_item_sk#52, ss_cdemo_sk#53, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, cd_demo_sk#60] - -(38) ReusedExchange [Reuses operator id: 83] -Output [1]: [d_date_sk#61] - -(39) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#59] -Right keys [1]: [d_date_sk#61] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 11] -Output [6]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] -Input [8]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, ss_sold_date_sk#59, d_date_sk#61] - -(41) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#62, s_state#63] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [s_store_sk#62, s_state#63] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#63, 2, true, false, true) = TN) AND isnotnull(s_store_sk#62)) - -(43) CometProject -Input [2]: [s_store_sk#62, s_state#63] -Arguments: [s_store_sk#62], [s_store_sk#62] - -(44) CometColumnarToRow [codegen id : 9] -Input [1]: [s_store_sk#62] - -(45) BroadcastExchange -Input [1]: [s_store_sk#62] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#54] -Right keys [1]: [s_store_sk#62] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 11] -Output [5]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58] -Input [7]: [ss_item_sk#52, ss_store_sk#54, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, s_store_sk#62] - -(48) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#64, i_item_id#20] - -(49) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#52] -Right keys [1]: [i_item_sk#64] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 11] -Output [5]: [i_item_id#20, ss_quantity#55 AS agg1#23, ss_list_price#56 AS agg2#24, ss_coupon_amt#58 AS agg3#25, ss_sales_price#57 AS agg4#26] -Input [7]: [ss_item_sk#52, ss_quantity#55, ss_list_price#56, ss_sales_price#57, ss_coupon_amt#58, i_item_sk#64, i_item_id#20] - -(51) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#20, agg1#23, agg2#24, agg3#25, agg4#26] -Keys [1]: [i_item_id#20] -Functions [4]: [partial_avg(agg1#23), partial_avg(UnscaledValue(agg2#24)), partial_avg(UnscaledValue(agg3#25)), partial_avg(UnscaledValue(agg4#26))] -Aggregate Attributes [8]: [sum#65, count#66, sum#67, count#68, sum#69, count#70, sum#71, count#72] -Results [9]: [i_item_id#20, sum#73, count#74, sum#75, count#76, sum#77, count#78, sum#79, count#80] - -(52) CometColumnarExchange -Input [9]: [i_item_id#20, sum#73, count#74, sum#75, count#76, sum#77, count#78, sum#79, count#80] -Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(53) CometColumnarToRow [codegen id : 12] -Input [9]: [i_item_id#20, sum#73, count#74, sum#75, count#76, sum#77, count#78, sum#79, count#80] - -(54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#20, sum#73, count#74, sum#75, count#76, sum#77, count#78, sum#79, count#80] -Keys [1]: [i_item_id#20] -Functions [4]: [avg(agg1#23), avg(UnscaledValue(agg2#24)), avg(UnscaledValue(agg3#25)), avg(UnscaledValue(agg4#26))] -Aggregate Attributes [4]: [avg(agg1#23)#81, avg(UnscaledValue(agg2#24))#82, avg(UnscaledValue(agg3#25))#83, avg(UnscaledValue(agg4#26))#84] -Results [7]: [i_item_id#20, null AS s_state#85, 1 AS g_state#86, avg(agg1#23)#81 AS agg1#87, cast((avg(UnscaledValue(agg2#24))#82 / 100.0) as decimal(11,6)) AS agg2#88, cast((avg(UnscaledValue(agg3#25))#83 / 100.0) as decimal(11,6)) AS agg3#89, cast((avg(UnscaledValue(agg4#26))#84 / 100.0) as decimal(11,6)) AS agg4#90] - -(55) Scan parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#91, ss_cdemo_sk#92, ss_store_sk#93, ss_quantity#94, ss_list_price#95, ss_sales_price#96, ss_coupon_amt#97, ss_sold_date_sk#98] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#98), dynamicpruningexpression(ss_sold_date_sk#98 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(56) ColumnarToRow [codegen id : 17] -Input [8]: [ss_item_sk#91, ss_cdemo_sk#92, ss_store_sk#93, ss_quantity#94, ss_list_price#95, ss_sales_price#96, ss_coupon_amt#97, ss_sold_date_sk#98] - -(57) Filter [codegen id : 17] -Input [8]: [ss_item_sk#91, ss_cdemo_sk#92, ss_store_sk#93, ss_quantity#94, ss_list_price#95, ss_sales_price#96, ss_coupon_amt#97, ss_sold_date_sk#98] -Condition : ((isnotnull(ss_cdemo_sk#92) AND isnotnull(ss_store_sk#93)) AND isnotnull(ss_item_sk#91)) - -(58) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#99] - -(59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#92] -Right keys [1]: [cd_demo_sk#99] -Join type: Inner -Join condition: None - -(60) Project [codegen id : 17] -Output [7]: [ss_item_sk#91, ss_store_sk#93, ss_quantity#94, ss_list_price#95, ss_sales_price#96, ss_coupon_amt#97, ss_sold_date_sk#98] -Input [9]: [ss_item_sk#91, ss_cdemo_sk#92, ss_store_sk#93, ss_quantity#94, ss_list_price#95, ss_sales_price#96, ss_coupon_amt#97, ss_sold_date_sk#98, cd_demo_sk#99] - -(61) ReusedExchange [Reuses operator id: 83] -Output [1]: [d_date_sk#100] - -(62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#98] -Right keys [1]: [d_date_sk#100] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 17] -Output [6]: [ss_item_sk#91, ss_store_sk#93, ss_quantity#94, ss_list_price#95, ss_sales_price#96, ss_coupon_amt#97] -Input [8]: [ss_item_sk#91, ss_store_sk#93, ss_quantity#94, ss_list_price#95, ss_sales_price#96, ss_coupon_amt#97, ss_sold_date_sk#98, d_date_sk#100] - -(64) ReusedExchange [Reuses operator id: 45] -Output [1]: [s_store_sk#101] - -(65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#93] -Right keys [1]: [s_store_sk#101] -Join type: Inner -Join condition: None - -(66) Project [codegen id : 17] -Output [5]: [ss_item_sk#91, ss_quantity#94, ss_list_price#95, ss_sales_price#96, ss_coupon_amt#97] -Input [7]: [ss_item_sk#91, ss_store_sk#93, ss_quantity#94, ss_list_price#95, ss_sales_price#96, ss_coupon_amt#97, s_store_sk#101] - -(67) CometNativeScan parquet spark_catalog.default.item -Output [1]: [i_item_sk#102] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(68) CometFilter -Input [1]: [i_item_sk#102] -Condition : isnotnull(i_item_sk#102) - -(69) CometColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#102] - -(70) BroadcastExchange -Input [1]: [i_item_sk#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -(71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#91] -Right keys [1]: [i_item_sk#102] -Join type: Inner -Join condition: None - -(72) Project [codegen id : 17] -Output [4]: [ss_quantity#94 AS agg1#23, ss_list_price#95 AS agg2#24, ss_coupon_amt#97 AS agg3#25, ss_sales_price#96 AS agg4#26] -Input [6]: [ss_item_sk#91, ss_quantity#94, ss_list_price#95, ss_sales_price#96, ss_coupon_amt#97, i_item_sk#102] - -(73) HashAggregate [codegen id : 17] -Input [4]: [agg1#23, agg2#24, agg3#25, agg4#26] -Keys: [] -Functions [4]: [partial_avg(agg1#23), partial_avg(UnscaledValue(agg2#24)), partial_avg(UnscaledValue(agg3#25)), partial_avg(UnscaledValue(agg4#26))] -Aggregate Attributes [8]: [sum#103, count#104, sum#105, count#106, sum#107, count#108, sum#109, count#110] -Results [8]: [sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118] - -(74) CometColumnarExchange -Input [8]: [sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(75) CometColumnarToRow [codegen id : 18] -Input [8]: [sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118] - -(76) HashAggregate [codegen id : 18] -Input [8]: [sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118] -Keys: [] -Functions [4]: [avg(agg1#23), avg(UnscaledValue(agg2#24)), avg(UnscaledValue(agg3#25)), avg(UnscaledValue(agg4#26))] -Aggregate Attributes [4]: [avg(agg1#23)#119, avg(UnscaledValue(agg2#24))#120, avg(UnscaledValue(agg3#25))#121, avg(UnscaledValue(agg4#26))#122] -Results [7]: [null AS i_item_id#123, null AS s_state#124, 1 AS g_state#125, avg(agg1#23)#119 AS agg1#126, cast((avg(UnscaledValue(agg2#24))#120 / 100.0) as decimal(11,6)) AS agg2#127, cast((avg(UnscaledValue(agg3#25))#121 / 100.0) as decimal(11,6)) AS agg3#128, cast((avg(UnscaledValue(agg4#26))#122 / 100.0) as decimal(11,6)) AS agg4#129] - -(77) Union - -(78) TakeOrderedAndProject -Input [7]: [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] -Arguments: 100, [i_item_id#21 ASC NULLS FIRST, s_state#22 ASC NULLS FIRST], [i_item_id#21, s_state#22, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometNativeScan parquet spark_catalog.default.date_dim (79) - - -(79) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#130] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(80) CometFilter -Input [2]: [d_date_sk#14, d_year#130] -Condition : ((isnotnull(d_year#130) AND (d_year#130 = 1998)) AND isnotnull(d_date_sk#14)) - -(81) CometProject -Input [2]: [d_date_sk#14, d_year#130] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(82) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(83) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#9 - -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#98 IN dynamicpruning#9 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/simplified.txt deleted file mode 100644 index c7093301af..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/simplified.txt +++ /dev/null @@ -1,122 +0,0 @@ -TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - Union - WholeStageCodegen (6) - HashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,s_state] #1 - WholeStageCodegen (5) - HashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Filter [ss_cdemo_sk,ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id] - WholeStageCodegen (12) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #6 - WholeStageCodegen (11) - HashAggregate [i_item_id,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Filter [ss_cdemo_sk,ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [cd_demo_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 - WholeStageCodegen (18) - HashAggregate [sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (17) - HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] - Project [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] - Filter [ss_cdemo_sk,ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [cd_demo_sk] #3 - InputAdapter - ReusedExchange [d_date_sk] #2 - InputAdapter - ReusedExchange [s_store_sk] #7 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk] - CometNativeScan parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/explain.txt deleted file mode 100644 index af8de10b38..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,437 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (71) -+- CometTakeOrderedAndProject (70) - +- CometUnion (69) - :- CometHashAggregate (29) - : +- CometExchange (28) - : +- CometHashAggregate (27) - : +- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometProject (20) - : : +- CometBroadcastHashJoin (19) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (18) - : : +- CometProject (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - : +- CometBroadcastExchange (24) - : +- CometProject (23) - : +- CometFilter (22) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) - :- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (37) - : : : +- CometBroadcastHashJoin (36) - : : : :- CometProject (34) - : : : : +- CometBroadcastHashJoin (33) - : : : : :- CometFilter (31) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (30) - : : : : +- ReusedExchange (32) - : : : +- ReusedExchange (35) - : : +- CometBroadcastExchange (41) - : : +- CometProject (40) - : : +- CometFilter (39) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (38) - : +- ReusedExchange (44) - +- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometProject (65) - +- CometBroadcastHashJoin (64) - :- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (57) - : : +- CometBroadcastHashJoin (56) - : : :- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometFilter (51) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : : : +- ReusedExchange (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (58) - +- CometBroadcastExchange (63) - +- CometFilter (62) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (61) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = F) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = W)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = Primary )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#16, s_state#17] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) = TN) AND isnotnull(s_store_sk#16)) - -(17) CometProject -Input [2]: [s_store_sk#16, s_state#17] -Arguments: [s_store_sk#16, s_state#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) AS s_state#18] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_state#18] -Arguments: [s_store_sk#16, s_state#18] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Right output [2]: [s_store_sk#16, s_state#18] -Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16, s_state#18] -Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_item_id#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_item_sk#19, i_item_id#20] -Condition : isnotnull(i_item_sk#19) - -(23) CometProject -Input [2]: [i_item_sk#19, i_item_id#20] -Arguments: [i_item_sk#19, i_item_id#21], [i_item_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#20, 16, true, false, true) AS i_item_id#21] - -(24) CometBroadcastExchange -Input [2]: [i_item_sk#19, i_item_id#21] -Arguments: [i_item_sk#19, i_item_id#21] - -(25) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] -Right output [2]: [i_item_sk#19, i_item_id#21] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(26) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18, i_item_sk#19, i_item_id#21] -Arguments: [i_item_id#22, s_state#23, agg1#24, agg2#25, agg3#26, agg4#27], [i_item_id#21 AS i_item_id#22, s_state#18 AS s_state#23, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] - -(27) CometHashAggregate -Input [6]: [i_item_id#22, s_state#23, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [2]: [i_item_id#22, s_state#23] -Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] - -(28) CometExchange -Input [10]: [i_item_id#22, s_state#23, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Arguments: hashpartitioning(i_item_id#22, s_state#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [10]: [i_item_id#22, s_state#23, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Keys [2]: [i_item_id#22, s_state#23] -Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#43), dynamicpruningexpression(ss_sold_date_sk#43 IN dynamicpruning#44)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(31) CometFilter -Input [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Condition : ((isnotnull(ss_cdemo_sk#37) AND isnotnull(ss_store_sk#38)) AND isnotnull(ss_item_sk#36)) - -(32) ReusedExchange [Reuses operator id: 6] -Output [1]: [cd_demo_sk#45] - -(33) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Right output [1]: [cd_demo_sk#45] -Arguments: [ss_cdemo_sk#37], [cd_demo_sk#45], Inner, BuildRight - -(34) CometProject -Input [9]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43, cd_demo_sk#45] -Arguments: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43], [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] - -(35) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#46] - -(36) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Right output [1]: [d_date_sk#46] -Arguments: [ss_sold_date_sk#43], [d_date_sk#46], Inner, BuildRight - -(37) CometProject -Input [8]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43, d_date_sk#46] -Arguments: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42], [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#47, s_state#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(39) CometFilter -Input [2]: [s_store_sk#47, s_state#48] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#48, 2, true, false, true) = TN) AND isnotnull(s_store_sk#47)) - -(40) CometProject -Input [2]: [s_store_sk#47, s_state#48] -Arguments: [s_store_sk#47], [s_store_sk#47] - -(41) CometBroadcastExchange -Input [1]: [s_store_sk#47] -Arguments: [s_store_sk#47] - -(42) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] -Right output [1]: [s_store_sk#47] -Arguments: [ss_store_sk#38], [s_store_sk#47], Inner, BuildRight - -(43) CometProject -Input [7]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, s_store_sk#47] -Arguments: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42], [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] - -(44) ReusedExchange [Reuses operator id: 24] -Output [2]: [i_item_sk#49, i_item_id#21] - -(45) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] -Right output [2]: [i_item_sk#49, i_item_id#21] -Arguments: [ss_item_sk#36], [i_item_sk#49], Inner, BuildRight - -(46) CometProject -Input [7]: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, i_item_sk#49, i_item_id#21] -Arguments: [i_item_id#21, agg1#24, agg2#25, agg3#26, agg4#27], [i_item_id#21, ss_quantity#39 AS agg1#24, ss_list_price#40 AS agg2#25, ss_coupon_amt#42 AS agg3#26, ss_sales_price#41 AS agg4#27] - -(47) CometHashAggregate -Input [5]: [i_item_id#21, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [1]: [i_item_id#21] -Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] - -(48) CometExchange -Input [9]: [i_item_id#21, sum#50, count#51, sum#52, count#53, sum#54, count#55, sum#56, count#57] -Arguments: hashpartitioning(i_item_id#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(49) CometHashAggregate -Input [9]: [i_item_id#21, sum#50, count#51, sum#52, count#53, sum#54, count#55, sum#56, count#57] -Keys [1]: [i_item_id#21] -Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#58, ss_cdemo_sk#59, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#65), dynamicpruningexpression(ss_sold_date_sk#65 IN dynamicpruning#66)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [8]: [ss_item_sk#58, ss_cdemo_sk#59, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Condition : ((isnotnull(ss_cdemo_sk#59) AND isnotnull(ss_store_sk#60)) AND isnotnull(ss_item_sk#58)) - -(52) ReusedExchange [Reuses operator id: 6] -Output [1]: [cd_demo_sk#67] - -(53) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#58, ss_cdemo_sk#59, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Right output [1]: [cd_demo_sk#67] -Arguments: [ss_cdemo_sk#59], [cd_demo_sk#67], Inner, BuildRight - -(54) CometProject -Input [9]: [ss_item_sk#58, ss_cdemo_sk#59, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64, ss_sold_date_sk#65, cd_demo_sk#67] -Arguments: [ss_item_sk#58, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64, ss_sold_date_sk#65], [ss_item_sk#58, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] - -(55) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#68] - -(56) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#58, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Right output [1]: [d_date_sk#68] -Arguments: [ss_sold_date_sk#65], [d_date_sk#68], Inner, BuildRight - -(57) CometProject -Input [8]: [ss_item_sk#58, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64, ss_sold_date_sk#65, d_date_sk#68] -Arguments: [ss_item_sk#58, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64], [ss_item_sk#58, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64] - -(58) ReusedExchange [Reuses operator id: 41] -Output [1]: [s_store_sk#69] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#58, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64] -Right output [1]: [s_store_sk#69] -Arguments: [ss_store_sk#60], [s_store_sk#69], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_item_sk#58, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64, s_store_sk#69] -Arguments: [ss_item_sk#58, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64], [ss_item_sk#58, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64] - -(61) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(62) CometFilter -Input [1]: [i_item_sk#70] -Condition : isnotnull(i_item_sk#70) - -(63) CometBroadcastExchange -Input [1]: [i_item_sk#70] -Arguments: [i_item_sk#70] - -(64) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#58, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64] -Right output [1]: [i_item_sk#70] -Arguments: [ss_item_sk#58], [i_item_sk#70], Inner, BuildRight - -(65) CometProject -Input [6]: [ss_item_sk#58, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64, i_item_sk#70] -Arguments: [agg1#24, agg2#25, agg3#26, agg4#27], [ss_quantity#61 AS agg1#24, ss_list_price#62 AS agg2#25, ss_coupon_amt#64 AS agg3#26, ss_sales_price#63 AS agg4#27] - -(66) CometHashAggregate -Input [4]: [agg1#24, agg2#25, agg3#26, agg4#27] -Keys: [] -Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] - -(67) CometExchange -Input [8]: [sum#71, count#72, sum#73, count#74, sum#75, count#76, sum#77, count#78] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(68) CometHashAggregate -Input [8]: [sum#71, count#72, sum#73, count#74, sum#75, count#76, sum#77, count#78] -Keys: [] -Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] - -(69) CometUnion -Child 0 Input [7]: [i_item_id#22, s_state#23, g_state#79, agg1#80, agg2#81, agg3#82, agg4#83] -Child 1 Input [7]: [i_item_id#21, s_state#84, g_state#85, agg1#86, agg2#87, agg3#88, agg4#89] -Child 2 Input [7]: [i_item_id#90, s_state#91, g_state#92, agg1#93, agg2#94, agg3#95, agg4#96] - -(70) CometTakeOrderedAndProject -Input [7]: [i_item_id#22, s_state#23, g_state#79, agg1#80, agg2#81, agg3#82, agg4#83] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#22 ASC NULLS FIRST,s_state#23 ASC NULLS FIRST], output=[i_item_id#22,s_state#23,g_state#79,agg1#80,agg2#81,agg3#82,agg4#83]), [i_item_id#22, s_state#23, g_state#79, agg1#80, agg2#81, agg3#82, agg4#83], 100, 0, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#79, agg1#80, agg2#81, agg3#82, agg4#83] - -(71) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#22, s_state#23, g_state#79, agg1#80, agg2#81, agg3#82, agg4#83] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) - -(74) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(75) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(76) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#43 IN dynamicpruning#9 - -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#65 IN dynamicpruning#9 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 0d18ca5626..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,83 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometUnion [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange [i_item_id,s_state] #1 - CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] - CometProject [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] - 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_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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_state] #5 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange [i_item_id] #7 - CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,sum,count,sum,count,sum,count,sum,count] - CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] - 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_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_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_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 [native_iceberg_compat] 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 [s_store_sk] #8 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange #9 - CometHashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count] - CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] - 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_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_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_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 [native_iceberg_compat] 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 [i_item_sk] #10 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt deleted file mode 100644 index af8de10b38..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt +++ /dev/null @@ -1,437 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (71) -+- CometTakeOrderedAndProject (70) - +- CometUnion (69) - :- CometHashAggregate (29) - : +- CometExchange (28) - : +- CometHashAggregate (27) - : +- CometProject (26) - : +- CometBroadcastHashJoin (25) - : :- CometProject (20) - : : +- CometBroadcastHashJoin (19) - : : :- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (3) - : : : +- CometBroadcastExchange (12) - : : : +- CometProject (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (18) - : : +- CometProject (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - : +- CometBroadcastExchange (24) - : +- CometProject (23) - : +- CometFilter (22) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (21) - :- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (37) - : : : +- CometBroadcastHashJoin (36) - : : : :- CometProject (34) - : : : : +- CometBroadcastHashJoin (33) - : : : : :- CometFilter (31) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (30) - : : : : +- ReusedExchange (32) - : : : +- ReusedExchange (35) - : : +- CometBroadcastExchange (41) - : : +- CometProject (40) - : : +- CometFilter (39) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (38) - : +- ReusedExchange (44) - +- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometProject (65) - +- CometBroadcastHashJoin (64) - :- CometProject (60) - : +- CometBroadcastHashJoin (59) - : :- CometProject (57) - : : +- CometBroadcastHashJoin (56) - : : :- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometFilter (51) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : : : +- ReusedExchange (52) - : : +- ReusedExchange (55) - : +- ReusedExchange (58) - +- CometBroadcastExchange (63) - +- CometFilter (62) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (61) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#11, 1, true, false, true) = F) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#12, 1, true, false, true) = W)) AND (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_education_status#13, 20, true, false, true) = Primary )) AND isnotnull(cd_demo_sk#10)) - -(5) CometProject -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Arguments: [cd_demo_sk#10], [cd_demo_sk#10] - -(6) CometBroadcastExchange -Input [1]: [cd_demo_sk#10] -Arguments: [cd_demo_sk#10] - -(7) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [cd_demo_sk#10] -Arguments: [ss_cdemo_sk#2], [cd_demo_sk#10], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, cd_demo_sk#10] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#8], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7], [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_state#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#16, s_state#17] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) = TN) AND isnotnull(s_store_sk#16)) - -(17) CometProject -Input [2]: [s_store_sk#16, s_state#17] -Arguments: [s_store_sk#16, s_state#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#17, 2, true, false, true) AS s_state#18] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_state#18] -Arguments: [s_store_sk#16, s_state#18] - -(19) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] -Right output [2]: [s_store_sk#16, s_state#18] -Arguments: [ss_store_sk#3], [s_store_sk#16], Inner, BuildRight - -(20) CometProject -Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#16, s_state#18] -Arguments: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18], [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_item_id#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_item_sk#19, i_item_id#20] -Condition : isnotnull(i_item_sk#19) - -(23) CometProject -Input [2]: [i_item_sk#19, i_item_id#20] -Arguments: [i_item_sk#19, i_item_id#21], [i_item_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#20, 16, true, false, true) AS i_item_id#21] - -(24) CometBroadcastExchange -Input [2]: [i_item_sk#19, i_item_id#21] -Arguments: [i_item_sk#19, i_item_id#21] - -(25) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18] -Right output [2]: [i_item_sk#19, i_item_id#21] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(26) CometProject -Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#18, i_item_sk#19, i_item_id#21] -Arguments: [i_item_id#22, s_state#23, agg1#24, agg2#25, agg3#26, agg4#27], [i_item_id#21 AS i_item_id#22, s_state#18 AS s_state#23, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] - -(27) CometHashAggregate -Input [6]: [i_item_id#22, s_state#23, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [2]: [i_item_id#22, s_state#23] -Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] - -(28) CometExchange -Input [10]: [i_item_id#22, s_state#23, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Arguments: hashpartitioning(i_item_id#22, s_state#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(29) CometHashAggregate -Input [10]: [i_item_id#22, s_state#23, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] -Keys [2]: [i_item_id#22, s_state#23] -Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#43), dynamicpruningexpression(ss_sold_date_sk#43 IN dynamicpruning#44)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(31) CometFilter -Input [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Condition : ((isnotnull(ss_cdemo_sk#37) AND isnotnull(ss_store_sk#38)) AND isnotnull(ss_item_sk#36)) - -(32) ReusedExchange [Reuses operator id: 6] -Output [1]: [cd_demo_sk#45] - -(33) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Right output [1]: [cd_demo_sk#45] -Arguments: [ss_cdemo_sk#37], [cd_demo_sk#45], Inner, BuildRight - -(34) CometProject -Input [9]: [ss_item_sk#36, ss_cdemo_sk#37, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43, cd_demo_sk#45] -Arguments: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43], [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] - -(35) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#46] - -(36) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43] -Right output [1]: [d_date_sk#46] -Arguments: [ss_sold_date_sk#43], [d_date_sk#46], Inner, BuildRight - -(37) CometProject -Input [8]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, ss_sold_date_sk#43, d_date_sk#46] -Arguments: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42], [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#47, s_state#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(39) CometFilter -Input [2]: [s_store_sk#47, s_state#48] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#48, 2, true, false, true) = TN) AND isnotnull(s_store_sk#47)) - -(40) CometProject -Input [2]: [s_store_sk#47, s_state#48] -Arguments: [s_store_sk#47], [s_store_sk#47] - -(41) CometBroadcastExchange -Input [1]: [s_store_sk#47] -Arguments: [s_store_sk#47] - -(42) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] -Right output [1]: [s_store_sk#47] -Arguments: [ss_store_sk#38], [s_store_sk#47], Inner, BuildRight - -(43) CometProject -Input [7]: [ss_item_sk#36, ss_store_sk#38, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, s_store_sk#47] -Arguments: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42], [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] - -(44) ReusedExchange [Reuses operator id: 24] -Output [2]: [i_item_sk#49, i_item_id#21] - -(45) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42] -Right output [2]: [i_item_sk#49, i_item_id#21] -Arguments: [ss_item_sk#36], [i_item_sk#49], Inner, BuildRight - -(46) CometProject -Input [7]: [ss_item_sk#36, ss_quantity#39, ss_list_price#40, ss_sales_price#41, ss_coupon_amt#42, i_item_sk#49, i_item_id#21] -Arguments: [i_item_id#21, agg1#24, agg2#25, agg3#26, agg4#27], [i_item_id#21, ss_quantity#39 AS agg1#24, ss_list_price#40 AS agg2#25, ss_coupon_amt#42 AS agg3#26, ss_sales_price#41 AS agg4#27] - -(47) CometHashAggregate -Input [5]: [i_item_id#21, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [1]: [i_item_id#21] -Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] - -(48) CometExchange -Input [9]: [i_item_id#21, sum#50, count#51, sum#52, count#53, sum#54, count#55, sum#56, count#57] -Arguments: hashpartitioning(i_item_id#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(49) CometHashAggregate -Input [9]: [i_item_id#21, sum#50, count#51, sum#52, count#53, sum#54, count#55, sum#56, count#57] -Keys [1]: [i_item_id#21] -Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [8]: [ss_item_sk#58, ss_cdemo_sk#59, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#65), dynamicpruningexpression(ss_sold_date_sk#65 IN dynamicpruning#66)] -PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [8]: [ss_item_sk#58, ss_cdemo_sk#59, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Condition : ((isnotnull(ss_cdemo_sk#59) AND isnotnull(ss_store_sk#60)) AND isnotnull(ss_item_sk#58)) - -(52) ReusedExchange [Reuses operator id: 6] -Output [1]: [cd_demo_sk#67] - -(53) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#58, ss_cdemo_sk#59, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Right output [1]: [cd_demo_sk#67] -Arguments: [ss_cdemo_sk#59], [cd_demo_sk#67], Inner, BuildRight - -(54) CometProject -Input [9]: [ss_item_sk#58, ss_cdemo_sk#59, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64, ss_sold_date_sk#65, cd_demo_sk#67] -Arguments: [ss_item_sk#58, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64, ss_sold_date_sk#65], [ss_item_sk#58, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] - -(55) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#68] - -(56) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#58, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] -Right output [1]: [d_date_sk#68] -Arguments: [ss_sold_date_sk#65], [d_date_sk#68], Inner, BuildRight - -(57) CometProject -Input [8]: [ss_item_sk#58, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64, ss_sold_date_sk#65, d_date_sk#68] -Arguments: [ss_item_sk#58, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64], [ss_item_sk#58, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64] - -(58) ReusedExchange [Reuses operator id: 41] -Output [1]: [s_store_sk#69] - -(59) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#58, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64] -Right output [1]: [s_store_sk#69] -Arguments: [ss_store_sk#60], [s_store_sk#69], Inner, BuildRight - -(60) CometProject -Input [7]: [ss_item_sk#58, ss_store_sk#60, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64, s_store_sk#69] -Arguments: [ss_item_sk#58, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64], [ss_item_sk#58, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64] - -(61) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [1]: [i_item_sk#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(62) CometFilter -Input [1]: [i_item_sk#70] -Condition : isnotnull(i_item_sk#70) - -(63) CometBroadcastExchange -Input [1]: [i_item_sk#70] -Arguments: [i_item_sk#70] - -(64) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#58, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64] -Right output [1]: [i_item_sk#70] -Arguments: [ss_item_sk#58], [i_item_sk#70], Inner, BuildRight - -(65) CometProject -Input [6]: [ss_item_sk#58, ss_quantity#61, ss_list_price#62, ss_sales_price#63, ss_coupon_amt#64, i_item_sk#70] -Arguments: [agg1#24, agg2#25, agg3#26, agg4#27], [ss_quantity#61 AS agg1#24, ss_list_price#62 AS agg2#25, ss_coupon_amt#64 AS agg3#26, ss_sales_price#63 AS agg4#27] - -(66) CometHashAggregate -Input [4]: [agg1#24, agg2#25, agg3#26, agg4#27] -Keys: [] -Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] - -(67) CometExchange -Input [8]: [sum#71, count#72, sum#73, count#74, sum#75, count#76, sum#77, count#78] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(68) CometHashAggregate -Input [8]: [sum#71, count#72, sum#73, count#74, sum#75, count#76, sum#77, count#78] -Keys: [] -Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] - -(69) CometUnion -Child 0 Input [7]: [i_item_id#22, s_state#23, g_state#79, agg1#80, agg2#81, agg3#82, agg4#83] -Child 1 Input [7]: [i_item_id#21, s_state#84, g_state#85, agg1#86, agg2#87, agg3#88, agg4#89] -Child 2 Input [7]: [i_item_id#90, s_state#91, g_state#92, agg1#93, agg2#94, agg3#95, agg4#96] - -(70) CometTakeOrderedAndProject -Input [7]: [i_item_id#22, s_state#23, g_state#79, agg1#80, agg2#81, agg3#82, agg4#83] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#22 ASC NULLS FIRST,s_state#23 ASC NULLS FIRST], output=[i_item_id#22,s_state#23,g_state#79,agg1#80,agg2#81,agg3#82,agg4#83]), [i_item_id#22, s_state#23, g_state#79, agg1#80, agg2#81, agg3#82, agg4#83], 100, 0, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#79, agg1#80, agg2#81, agg3#82, agg4#83] - -(71) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#22, s_state#23, g_state#79, agg1#80, agg2#81, agg3#82, agg4#83] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_year#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#14, d_year#15] -Condition : ((isnotnull(d_year#15) AND (d_year#15 = 1998)) AND isnotnull(d_date_sk#14)) - -(74) CometProject -Input [2]: [d_date_sk#14, d_year#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(75) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(76) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#43 IN dynamicpruning#9 - -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#65 IN dynamicpruning#9 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/extended.txt deleted file mode 100644 index affbc9ef37..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/extended.txt +++ /dev/null @@ -1,99 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 91 out of 95 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 0d18ca5626..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/simplified.txt +++ /dev/null @@ -1,83 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometUnion [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange [i_item_id,s_state] #1 - CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] - CometProject [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,s_state,agg1,agg2,agg3,agg4] - 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_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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [cd_demo_sk] #3 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_state] #5 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometFilter [i_item_sk,i_item_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id] - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange [i_item_id] #7 - CometHashAggregate [agg1,agg2,agg3,agg4] [i_item_id,sum,count,sum,count,sum,count,sum,count] - CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [i_item_id,agg1,agg2,agg3,agg4] - 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_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_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_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 [native_iceberg_compat] 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 [s_store_sk] #8 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [i_item_sk,i_item_id] #6 - CometHashAggregate [sum,count,sum,count,sum,count,sum,count] [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4))] - CometExchange #9 - CometHashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count] - CometProject [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [agg1,agg2,agg3,agg4] - 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_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_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_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 [native_iceberg_compat] 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 [i_item_sk] #10 - CometFilter [i_item_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/explain.txt deleted file mode 100644 index 587f232ea8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/explain.txt +++ /dev/null @@ -1,231 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (35) -+- CometSort (34) - +- CometColumnarExchange (33) - +- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * Filter (25) - : +- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.household_demographics (14) - +- BroadcastExchange (30) - +- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometNativeScan parquet spark_catalog.default.customer (26) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(4) ReusedExchange [Reuses operator id: 40] -Output [1]: [d_date_sk#7] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#8, s_county#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [s_store_sk#8, s_county#9] -Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND isnotnull(s_store_sk#8)) - -(9) CometProject -Input [2]: [s_store_sk#8, s_county#9] -Arguments: [s_store_sk#8], [s_store_sk#8] - -(10) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#8] - -(11) BroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] - -(14) CometNativeScan parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Condition : ((((isnotnull(hd_vehicle_count#13) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#11, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#11, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#12 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#13 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#10)) - -(16) CometProject -Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] -Arguments: [hd_demo_sk#10], [hd_demo_sk#10] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [hd_demo_sk#10] - -(18) BroadcastExchange -Input [1]: [hd_demo_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#10] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [2]: [ss_customer_sk#1, ss_ticket_number#4] -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#10] - -(21) HashAggregate [codegen id : 4] -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#14] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(22) CometColumnarExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] - -(24) HashAggregate [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#16] -Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count(1)#16 AS cnt#17] - -(25) Filter [codegen id : 6] -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) - -(26) CometNativeScan parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(27) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(28) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] - -(29) CometColumnarToRow [codegen id : 5] -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(30) BroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#18] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 6] -Output [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(33) CometColumnarExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(34) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST] - -(35) CometColumnarToRow [codegen id : 7] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (40) -+- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometNativeScan parquet spark_catalog.default.date_dim (36) - - -(36) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#27, d_dom#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [d_date_sk#7, d_year#27, d_dom#28] -Condition : (((((d_dom#28 >= 1) AND (d_dom#28 <= 3)) OR ((d_dom#28 >= 25) AND (d_dom#28 <= 28))) AND d_year#27 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(38) CometProject -Input [3]: [d_date_sk#7, d_year#27, d_dom#28] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(39) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(40) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/simplified.txt deleted file mode 100644 index 63c6ac3645..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/simplified.txt +++ /dev/null @@ -1,59 +0,0 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 - WholeStageCodegen (6) - Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - Filter [cnt] - HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_ticket_number,ss_customer_sk] #2 - WholeStageCodegen (4) - HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] - Project [ss_customer_sk,ss_ticket_number] - BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometNativeScan 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/q34.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/explain.txt deleted file mode 100644 index 50af5f4f75..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometSort (32) - +- CometExchange (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (24) - : +- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_county#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#10, s_county#11] -Condition : ((isnotnull(s_county#11) AND (s_county#11 = Williamson County)) AND isnotnull(s_store_sk#10)) - -(11) CometProject -Input [2]: [s_store_sk#10, s_county#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : ((((isnotnull(hd_vehicle_count#15) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#14 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#15 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#12)) - -(17) CometProject -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#12], [hd_demo_sk#12] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#12] -Arguments: [hd_demo_sk#12] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Right output [1]: [hd_demo_sk#12] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight - -(20) CometProject -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] -Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] - -(21) CometHashAggregate -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] - -(22) CometExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] - -(24) CometFilter -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(27) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(29) CometBroadcastHashJoin -Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Right output [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -(31) CometExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(32) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST] - -(33) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(36) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(37) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(38) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/simplified.txt deleted file mode 100644 index d57afd4fcf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 - CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] - CometExchange [ss_ticket_number,ss_customer_sk] #2 - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - 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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt deleted file mode 100644 index 50af5f4f75..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt +++ /dev/null @@ -1,222 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (33) -+- CometSort (32) - +- CometExchange (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (24) - : +- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (15) - +- CometBroadcastExchange (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_hdemo_sk#2)) AND isnotnull(ss_customer_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#10, s_county#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#10, s_county#11] -Condition : ((isnotnull(s_county#11) AND (s_county#11 = Williamson County)) AND isnotnull(s_store_sk#10)) - -(11) CometProject -Input [2]: [s_store_sk#10, s_county#11] -Arguments: [s_store_sk#10], [s_store_sk#10] - -(12) CometBroadcastExchange -Input [1]: [s_store_sk#10] -Arguments: [s_store_sk#10] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4] -Right output [1]: [s_store_sk#10] -Arguments: [ss_store_sk#3], [s_store_sk#10], Inner, BuildRight - -(14) CometProject -Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#10] -Arguments: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4], [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(16) CometFilter -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Condition : ((((isnotnull(hd_vehicle_count#15) AND ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = >10000 ) OR (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#13, 15, true, false, true) = unknown ))) AND (hd_vehicle_count#15 > 0)) AND CASE WHEN (hd_vehicle_count#15 > 0) THEN (knownfloatingpointnormalized(normalizenanandzero((cast(hd_dep_count#14 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(hd_vehicle_count#15 as double)))))) > 1.2) END) AND isnotnull(hd_demo_sk#12)) - -(17) CometProject -Input [4]: [hd_demo_sk#12, hd_buy_potential#13, hd_dep_count#14, hd_vehicle_count#15] -Arguments: [hd_demo_sk#12], [hd_demo_sk#12] - -(18) CometBroadcastExchange -Input [1]: [hd_demo_sk#12] -Arguments: [hd_demo_sk#12] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] -Right output [1]: [hd_demo_sk#12] -Arguments: [ss_hdemo_sk#2], [hd_demo_sk#12], Inner, BuildRight - -(20) CometProject -Input [4]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4, hd_demo_sk#12] -Arguments: [ss_customer_sk#1, ss_ticket_number#4], [ss_customer_sk#1, ss_ticket_number#4] - -(21) CometHashAggregate -Input [2]: [ss_customer_sk#1, ss_ticket_number#4] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [partial_count(1)] - -(22) CometExchange -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#16] -Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] -Functions [1]: [count(1)] - -(24) CometFilter -Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) - -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(26) CometFilter -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Condition : isnotnull(c_customer_sk#18) - -(27) CometProject -Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26], [c_customer_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_salutation#19, 10, true, false, true) AS c_salutation#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#20, 20, true, false, true) AS c_first_name#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#21, 30, true, false, true) AS c_last_name#25, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_preferred_cust_flag#22, 1, true, false, true) AS c_preferred_cust_flag#26] - -(28) CometBroadcastExchange -Input [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] - -(29) CometBroadcastHashJoin -Left output [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] -Right output [5]: [c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -(31) CometExchange -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(32) CometSort -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST] - -(33) CometColumnarToRow [codegen id : 1] -Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (38) -+- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (34) - - -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_dom#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Condition : (((((d_dom#9 >= 1) AND (d_dom#9 <= 3)) OR ((d_dom#9 >= 25) AND (d_dom#9 <= 28))) AND d_year#8 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) - -(36) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_dom#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(37) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(38) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/extended.txt deleted file mode 100644 index 3d9d23d2cc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/extended.txt +++ /dev/null @@ -1,41 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d57afd4fcf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt +++ /dev/null @@ -1,43 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 - CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [ss_ticket_number,ss_customer_sk,cnt] - CometHashAggregate [count] [ss_ticket_number,ss_customer_sk,cnt,count(1)] - CometExchange [ss_ticket_number,ss_customer_sk] #2 - CometHashAggregate [ss_ticket_number,ss_customer_sk,count] - CometProject [ss_customer_sk,ss_ticket_number] - 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_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_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_dom] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] - CometBroadcastExchange [s_store_sk] #5 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_county] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county] - CometBroadcastExchange [hd_demo_sk] #6 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] - CometBroadcastExchange [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] #7 - CometProject [c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometFilter [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/explain.txt deleted file mode 100644 index d3c5709bdd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/explain.txt +++ /dev/null @@ -1,295 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (45) -+- * HashAggregate (44) - +- * CometColumnarToRow (43) - +- CometColumnarExchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * BroadcastHashJoin Inner BuildRight (39) - :- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Project (26) - : : +- * Filter (25) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) - : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : : :- * CometColumnarToRow (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (9) - : : : : +- * Project (8) - : : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : : :- * ColumnarToRow (5) - : : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : : +- ReusedExchange (6) - : : : +- BroadcastExchange (16) - : : : +- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * ColumnarToRow (19) - : : : +- Scan parquet spark_catalog.default.catalog_sales (18) - : : +- ReusedExchange (20) - : +- BroadcastExchange (31) - : +- * CometColumnarToRow (30) - : +- CometProject (29) - : +- CometFilter (28) - : +- CometNativeScan parquet spark_catalog.default.customer_address (27) - +- BroadcastExchange (38) - +- * CometColumnarToRow (37) - +- CometProject (36) - +- CometFilter (35) - +- CometNativeScan parquet spark_catalog.default.customer_demographics (34) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(4) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] - -(6) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#9] - -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None - -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#6] -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] - -(9) BroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ss_customer_sk#6] -Join type: LeftSemi -Join condition: None - -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#8)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] - -(13) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#9] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#11] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#10] -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#9] - -(16) BroadcastExchange -Input [1]: [ws_bill_customer_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(17) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#10] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(18) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#8)] -ReadSchema: struct - -(19) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] - -(20) ReusedExchange [Reuses operator id: 50] -Output [1]: [d_date_sk#9] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#9] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#12] -Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#9] - -(23) BroadcastExchange -Input [1]: [cs_ship_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(24) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#12] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(25) Filter [codegen id : 9] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(26) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(27) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#14, ca_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(28) CometFilter -Input [2]: [ca_address_sk#14, ca_state#15] -Condition : isnotnull(ca_address_sk#14) - -(29) CometProject -Input [2]: [ca_address_sk#14, ca_state#15] -Arguments: [ca_address_sk#14, ca_state#16], [ca_address_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#15, 2, true, false, true) AS ca_state#16] - -(30) CometColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#14, ca_state#16] - -(31) BroadcastExchange -Input [2]: [ca_address_sk#14, ca_state#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#14] -Join type: Inner -Join condition: None - -(33) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#16] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#14, ca_state#16] - -(34) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(35) CometFilter -Input [6]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] -Condition : isnotnull(cd_demo_sk#17) - -(36) CometProject -Input [6]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] -Arguments: [cd_demo_sk#17, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22], [cd_demo_sk#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#18, 1, true, false, true) AS cd_gender#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) AS cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] - -(37) CometColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#17, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] - -(38) BroadcastExchange -Input [6]: [cd_demo_sk#17, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#17] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 9] -Output [6]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] -Input [8]: [c_current_cdemo_sk#4, ca_state#16, cd_demo_sk#17, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] - -(41) HashAggregate [codegen id : 9] -Input [6]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] -Keys [6]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#20), partial_max(cd_dep_count#20), partial_sum(cd_dep_count#20), partial_avg(cd_dep_employed_count#21), partial_max(cd_dep_employed_count#21), partial_sum(cd_dep_employed_count#21), partial_avg(cd_dep_college_count#22), partial_max(cd_dep_college_count#22), partial_sum(cd_dep_college_count#22)] -Aggregate Attributes [13]: [count#25, sum#26, count#27, max#28, sum#29, sum#30, count#31, max#32, sum#33, sum#34, count#35, max#36, sum#37] -Results [19]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, count#38, sum#39, count#40, max#41, sum#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50] - -(42) CometColumnarExchange -Input [19]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, count#38, sum#39, count#40, max#41, sum#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50] -Arguments: hashpartitioning(ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(43) CometColumnarToRow [codegen id : 10] -Input [19]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, count#38, sum#39, count#40, max#41, sum#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50] - -(44) HashAggregate [codegen id : 10] -Input [19]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22, count#38, sum#39, count#40, max#41, sum#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50] -Keys [6]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cd_dep_employed_count#21, cd_dep_college_count#22] -Functions [10]: [count(1), avg(cd_dep_count#20), max(cd_dep_count#20), sum(cd_dep_count#20), avg(cd_dep_employed_count#21), max(cd_dep_employed_count#21), sum(cd_dep_employed_count#21), avg(cd_dep_college_count#22), max(cd_dep_college_count#22), sum(cd_dep_college_count#22)] -Aggregate Attributes [10]: [count(1)#51, avg(cd_dep_count#20)#52, max(cd_dep_count#20)#53, sum(cd_dep_count#20)#54, avg(cd_dep_employed_count#21)#55, max(cd_dep_employed_count#21)#56, sum(cd_dep_employed_count#21)#57, avg(cd_dep_college_count#22)#58, max(cd_dep_college_count#22)#59, sum(cd_dep_college_count#22)#60] -Results [18]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, count(1)#51 AS cnt1#61, avg(cd_dep_count#20)#52 AS avg(cd_dep_count)#62, max(cd_dep_count#20)#53 AS max(cd_dep_count)#63, sum(cd_dep_count#20)#54 AS sum(cd_dep_count)#64, cd_dep_employed_count#21, count(1)#51 AS cnt2#65, avg(cd_dep_employed_count#21)#55 AS avg(cd_dep_employed_count)#66, max(cd_dep_employed_count#21)#56 AS max(cd_dep_employed_count)#67, sum(cd_dep_employed_count#21)#57 AS sum(cd_dep_employed_count)#68, cd_dep_college_count#22, count(1)#51 AS cnt3#69, avg(cd_dep_college_count#22)#58 AS avg(cd_dep_college_count)#70, max(cd_dep_college_count#22)#59 AS max(cd_dep_college_count)#71, sum(cd_dep_college_count#22)#60 AS sum(cd_dep_college_count)#72] - -(45) TakeOrderedAndProject -Input [18]: [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cnt1#61, avg(cd_dep_count)#62, max(cd_dep_count)#63, sum(cd_dep_count)#64, cd_dep_employed_count#21, cnt2#65, avg(cd_dep_employed_count)#66, max(cd_dep_employed_count)#67, sum(cd_dep_employed_count)#68, cd_dep_college_count#22, cnt3#69, avg(cd_dep_college_count)#70, max(cd_dep_college_count)#71, sum(cd_dep_college_count)#72] -Arguments: 100, [ca_state#16 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#20 ASC NULLS FIRST, cd_dep_employed_count#21 ASC NULLS FIRST, cd_dep_college_count#22 ASC NULLS FIRST], [ca_state#16, cd_gender#23, cd_marital_status#24, cd_dep_count#20, cnt1#61, avg(cd_dep_count)#62, max(cd_dep_count)#63, sum(cd_dep_count)#64, cd_dep_employed_count#21, cnt2#65, avg(cd_dep_employed_count)#66, max(cd_dep_employed_count)#67, sum(cd_dep_employed_count)#68, cd_dep_college_count#22, cnt3#69, avg(cd_dep_college_count)#70, max(cd_dep_college_count)#71, sum(cd_dep_college_count)#72] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (50) -+- * CometColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometNativeScan parquet spark_catalog.default.date_dim (46) - - -(46) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#73, d_qoy#74] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_date_sk#9, d_year#73, d_qoy#74] -Condition : ((((isnotnull(d_year#73) AND isnotnull(d_qoy#74)) AND (d_year#73 = 2002)) AND (d_qoy#74 < 4)) AND isnotnull(d_date_sk#9)) - -(48) CometProject -Input [3]: [d_date_sk#9, d_year#73, d_qoy#74] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(49) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(50) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/simplified.txt deleted file mode 100644 index 0fb95102c3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/simplified.txt +++ /dev/null @@ -1,77 +0,0 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (10) - HashAggregate [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] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [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,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan 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/q35.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/explain.txt deleted file mode 100644 index 5df7e6870f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * Filter (27) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(6) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: [d_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#9] -Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -Arguments: [ss_customer_sk#6], [ss_customer_sk#6] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: [ss_customer_sk#6] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Right output [1]: [ss_customer_sk#6] -Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#9] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Right output [1]: [d_date_sk#9] -Arguments: [ws_sold_date_sk#13], [d_date_sk#9], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#9] -Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#12] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#12] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#9] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Right output [1]: [d_date_sk#9] -Arguments: [cs_sold_date_sk#16], [d_date_sk#9], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#9] -Arguments: [cs_ship_customer_sk#15], [cs_ship_customer_sk#15] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#15] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#15] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(27) Filter [codegen id : 5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(28) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_state#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [ca_address_sk#18, ca_state#19] -Condition : isnotnull(ca_address_sk#18) - -(31) CometProject -Input [2]: [ca_address_sk#18, ca_state#19] -Arguments: [ca_address_sk#18, ca_state#20], [ca_address_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#19, 2, true, false, true) AS ca_state#20] - -(32) CometColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#18, ca_state#20] - -(33) BroadcastExchange -Input [2]: [ca_address_sk#18, ca_state#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(34) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#18] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, ca_state#20] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18, ca_state#20] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(37) CometFilter -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Condition : isnotnull(cd_demo_sk#21) - -(38) CometProject -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#22, 1, true, false, true) AS cd_gender#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#23, 1, true, false, true) AS cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(39) CometColumnarToRow [codegen id : 4] -Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(40) BroadcastExchange -Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(41) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#21] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 5] -Output [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Input [8]: [c_current_cdemo_sk#4, ca_state#20, cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(43) HashAggregate [codegen id : 5] -Input [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#24), partial_max(cd_dep_count#24), partial_sum(cd_dep_count#24), partial_avg(cd_dep_employed_count#25), partial_max(cd_dep_employed_count#25), partial_sum(cd_dep_employed_count#25), partial_avg(cd_dep_college_count#26), partial_max(cd_dep_college_count#26), partial_sum(cd_dep_college_count#26)] -Aggregate Attributes [13]: [count#29, sum#30, count#31, max#32, sum#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41] -Results [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] - -(44) CometColumnarExchange -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -Arguments: hashpartitioning(ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 6] -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] - -(46) HashAggregate [codegen id : 6] -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [10]: [count(1), avg(cd_dep_count#24), max(cd_dep_count#24), sum(cd_dep_count#24), avg(cd_dep_employed_count#25), max(cd_dep_employed_count#25), sum(cd_dep_employed_count#25), avg(cd_dep_college_count#26), max(cd_dep_college_count#26), sum(cd_dep_college_count#26)] -Aggregate Attributes [10]: [count(1)#55, avg(cd_dep_count#24)#56, max(cd_dep_count#24)#57, sum(cd_dep_count#24)#58, avg(cd_dep_employed_count#25)#59, max(cd_dep_employed_count#25)#60, sum(cd_dep_employed_count#25)#61, avg(cd_dep_college_count#26)#62, max(cd_dep_college_count#26)#63, sum(cd_dep_college_count#26)#64] -Results [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, count(1)#55 AS cnt1#65, avg(cd_dep_count#24)#56 AS avg(cd_dep_count)#66, max(cd_dep_count#24)#57 AS max(cd_dep_count)#67, sum(cd_dep_count#24)#58 AS sum(cd_dep_count)#68, cd_dep_employed_count#25, count(1)#55 AS cnt2#69, avg(cd_dep_employed_count#25)#59 AS avg(cd_dep_employed_count)#70, max(cd_dep_employed_count#25)#60 AS max(cd_dep_employed_count)#71, sum(cd_dep_employed_count#25)#61 AS sum(cd_dep_employed_count)#72, cd_dep_college_count#26, count(1)#55 AS cnt3#73, avg(cd_dep_college_count#26)#62 AS avg(cd_dep_college_count)#74, max(cd_dep_college_count#26)#63 AS max(cd_dep_college_count)#75, sum(cd_dep_college_count#26)#64 AS sum(cd_dep_college_count)#76] - -(47) TakeOrderedAndProject -Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] -Arguments: 100, [ca_state#20 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(52) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/simplified.txt deleted file mode 100644 index fea6a32741..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (6) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] 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/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt deleted file mode 100644 index 5df7e6870f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (47) -+- * HashAggregate (46) - +- * CometColumnarToRow (45) - +- CometColumnarExchange (44) - +- * HashAggregate (43) - +- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (28) - : : +- * Filter (27) - : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (26) - : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (19) - : : : :- * CometColumnarToRow (12) - : : : : +- CometBroadcastHashJoin (11) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (10) - : : : : +- CometProject (9) - : : : : +- CometBroadcastHashJoin (8) - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : : +- CometBroadcastExchange (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : : +- BroadcastExchange (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometProject (16) - : : : +- CometBroadcastHashJoin (15) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (13) - : : : +- ReusedExchange (14) - : : +- BroadcastExchange (25) - : : +- * CometColumnarToRow (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (20) - : : +- ReusedExchange (21) - : +- BroadcastExchange (33) - : +- * CometColumnarToRow (32) - : +- CometProject (31) - : +- CometFilter (30) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (29) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometProject (38) - +- CometFilter (37) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (36) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(6) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: [d_date_sk#9] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] -Right output [1]: [d_date_sk#9] -Arguments: [ss_sold_date_sk#7], [d_date_sk#9], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] -Arguments: [ss_customer_sk#6], [ss_customer_sk#6] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#6] -Arguments: [ss_customer_sk#6] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] -Right output [1]: [ss_customer_sk#6] -Arguments: [c_customer_sk#3], [ss_customer_sk#6], LeftSemi, BuildRight - -(12) CometColumnarToRow [codegen id : 5] -Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#13), dynamicpruningexpression(ws_sold_date_sk#13 IN dynamicpruning#14)] -ReadSchema: struct - -(14) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#9] - -(15) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] -Right output [1]: [d_date_sk#9] -Arguments: [ws_sold_date_sk#13], [d_date_sk#9], Inner, BuildRight - -(16) CometProject -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#9] -Arguments: [ws_bill_customer_sk#12], [ws_bill_customer_sk#12] - -(17) CometColumnarToRow [codegen id : 1] -Input [1]: [ws_bill_customer_sk#12] - -(18) BroadcastExchange -Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(19) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [ws_bill_customer_sk#12] -Join type: ExistenceJoin(exists#2) -Join condition: None - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] -ReadSchema: struct - -(21) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#9] - -(22) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] -Right output [1]: [d_date_sk#9] -Arguments: [cs_sold_date_sk#16], [d_date_sk#9], Inner, BuildRight - -(23) CometProject -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#9] -Arguments: [cs_ship_customer_sk#15], [cs_ship_customer_sk#15] - -(24) CometColumnarToRow [codegen id : 2] -Input [1]: [cs_ship_customer_sk#15] - -(25) BroadcastExchange -Input [1]: [cs_ship_customer_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(26) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#15] -Join type: ExistenceJoin(exists#1) -Join condition: None - -(27) Filter [codegen id : 5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] -Condition : (exists#2 OR exists#1) - -(28) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] - -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#18, ca_state#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [ca_address_sk#18, ca_state#19] -Condition : isnotnull(ca_address_sk#18) - -(31) CometProject -Input [2]: [ca_address_sk#18, ca_state#19] -Arguments: [ca_address_sk#18, ca_state#20], [ca_address_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#19, 2, true, false, true) AS ca_state#20] - -(32) CometColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#18, ca_state#20] - -(33) BroadcastExchange -Input [2]: [ca_address_sk#18, ca_state#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(34) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#18] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 5] -Output [2]: [c_current_cdemo_sk#4, ca_state#20] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18, ca_state#20] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(37) CometFilter -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Condition : isnotnull(cd_demo_sk#21) - -(38) CometProject -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26], [cd_demo_sk#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#22, 1, true, false, true) AS cd_gender#27, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#23, 1, true, false, true) AS cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(39) CometColumnarToRow [codegen id : 4] -Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(40) BroadcastExchange -Input [6]: [cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(41) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#21] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 5] -Output [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Input [8]: [c_current_cdemo_sk#4, ca_state#20, cd_demo_sk#21, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] - -(43) HashAggregate [codegen id : 5] -Input [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#24), partial_max(cd_dep_count#24), partial_sum(cd_dep_count#24), partial_avg(cd_dep_employed_count#25), partial_max(cd_dep_employed_count#25), partial_sum(cd_dep_employed_count#25), partial_avg(cd_dep_college_count#26), partial_max(cd_dep_college_count#26), partial_sum(cd_dep_college_count#26)] -Aggregate Attributes [13]: [count#29, sum#30, count#31, max#32, sum#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41] -Results [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] - -(44) CometColumnarExchange -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -Arguments: hashpartitioning(ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(45) CometColumnarToRow [codegen id : 6] -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] - -(46) HashAggregate [codegen id : 6] -Input [19]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -Keys [6]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cd_dep_employed_count#25, cd_dep_college_count#26] -Functions [10]: [count(1), avg(cd_dep_count#24), max(cd_dep_count#24), sum(cd_dep_count#24), avg(cd_dep_employed_count#25), max(cd_dep_employed_count#25), sum(cd_dep_employed_count#25), avg(cd_dep_college_count#26), max(cd_dep_college_count#26), sum(cd_dep_college_count#26)] -Aggregate Attributes [10]: [count(1)#55, avg(cd_dep_count#24)#56, max(cd_dep_count#24)#57, sum(cd_dep_count#24)#58, avg(cd_dep_employed_count#25)#59, max(cd_dep_employed_count#25)#60, sum(cd_dep_employed_count#25)#61, avg(cd_dep_college_count#26)#62, max(cd_dep_college_count#26)#63, sum(cd_dep_college_count#26)#64] -Results [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, count(1)#55 AS cnt1#65, avg(cd_dep_count#24)#56 AS avg(cd_dep_count)#66, max(cd_dep_count#24)#57 AS max(cd_dep_count)#67, sum(cd_dep_count#24)#58 AS sum(cd_dep_count)#68, cd_dep_employed_count#25, count(1)#55 AS cnt2#69, avg(cd_dep_employed_count#25)#59 AS avg(cd_dep_employed_count)#70, max(cd_dep_employed_count#25)#60 AS max(cd_dep_employed_count)#71, sum(cd_dep_employed_count#25)#61 AS sum(cd_dep_employed_count)#72, cd_dep_college_count#26, count(1)#55 AS cnt3#73, avg(cd_dep_college_count#26)#62 AS avg(cd_dep_college_count)#74, max(cd_dep_college_count#26)#63 AS max(cd_dep_college_count)#75, sum(cd_dep_college_count#26)#64 AS sum(cd_dep_college_count)#76] - -(47) TakeOrderedAndProject -Input [18]: [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] -Arguments: 100, [ca_state#20 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [ca_state#20, cd_gender#27, cd_marital_status#28, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometProject (50) - +- CometFilter (49) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (48) - - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Condition : ((((isnotnull(d_year#10) AND isnotnull(d_qoy#11)) AND (d_year#10 = 2002)) AND (d_qoy#11 < 4)) AND isnotnull(d_date_sk#9)) - -(50) CometProject -Input [3]: [d_date_sk#9, d_year#10, d_qoy#11] -Arguments: [d_date_sk#9], [d_date_sk#9] - -(51) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#9] - -(52) BroadcastExchange -Input [1]: [d_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/extended.txt deleted file mode 100644 index a6f33d6f7e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/extended.txt +++ /dev/null @@ -1,63 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index fea6a32741..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/simplified.txt +++ /dev/null @@ -1,73 +0,0 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (6) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (5) - HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - Filter [exists,exists] - BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - 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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [ws_bill_customer_sk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [cs_ship_customer_sk] - CometBroadcastHashJoin [cs_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/explain.txt deleted file mode 100644 index fbea598466..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/explain.txt +++ /dev/null @@ -1,281 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Project (38) - +- * BroadcastHashJoin Inner BuildRight (37) - :- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (24) - : : +- * BroadcastHashJoin LeftSemi BuildRight (23) - : : :- * BroadcastHashJoin LeftSemi BuildRight (10) - : : : :- * CometColumnarToRow (3) - : : : : +- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : +- BroadcastExchange (9) - : : : +- * Project (8) - : : : +- * BroadcastHashJoin Inner BuildRight (7) - : : : :- * ColumnarToRow (5) - : : : : +- Scan parquet spark_catalog.default.store_sales (4) - : : : +- ReusedExchange (6) - : : +- BroadcastExchange (22) - : : +- Union (21) - : : :- * Project (15) - : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : :- * ColumnarToRow (12) - : : : : +- Scan parquet spark_catalog.default.web_sales (11) - : : : +- ReusedExchange (13) - : : +- * Project (20) - : : +- * BroadcastHashJoin Inner BuildRight (19) - : : :- * ColumnarToRow (17) - : : : +- Scan parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (18) - : +- BroadcastExchange (29) - : +- * CometColumnarToRow (28) - : +- CometProject (27) - : +- CometFilter (26) - : +- CometNativeScan parquet spark_catalog.default.customer_address (25) - +- BroadcastExchange (36) - +- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometNativeScan parquet spark_catalog.default.customer_demographics (32) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometColumnarToRow [codegen id : 9] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(4) Scan parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 2] -Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] - -(6) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#7] - -(7) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(8) Project [codegen id : 2] -Output [1]: [ss_customer_sk#4] -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] - -(9) BroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(10) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#4] -Join type: LeftSemi -Join condition: None - -(11) Scan parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#6)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 4] -Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] - -(13) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#7] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ws_sold_date_sk#9] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#8 AS customsk#10] -Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#7] - -(16) Scan parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#11, cs_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#12), dynamicpruningexpression(cs_sold_date_sk#12 IN dynamicpruning#6)] -ReadSchema: struct - -(17) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#11, cs_sold_date_sk#12] - -(18) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#13] - -(19) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#12] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#11 AS customsk#14] -Input [3]: [cs_ship_customer_sk#11, cs_sold_date_sk#12, d_date_sk#13] - -(21) Union - -(22) BroadcastExchange -Input [1]: [customsk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(23) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customsk#10] -Join type: LeftSemi -Join condition: None - -(24) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] - -(25) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#15, ca_state#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [ca_address_sk#15, ca_state#16] -Condition : isnotnull(ca_address_sk#15) - -(27) CometProject -Input [2]: [ca_address_sk#15, ca_state#16] -Arguments: [ca_address_sk#15, ca_state#17], [ca_address_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#16, 2, true, false, true) AS ca_state#17] - -(28) CometColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#15, ca_state#17] - -(29) BroadcastExchange -Input [2]: [ca_address_sk#15, ca_state#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(30) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#15] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, ca_state#17] -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#15, ca_state#17] - -(32) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(33) CometFilter -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Condition : isnotnull(cd_demo_sk#18) - -(34) CometProject -Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Arguments: [cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23], [cd_demo_sk#18, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#19, 1, true, false, true) AS cd_gender#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#20, 1, true, false, true) AS cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] - -(35) CometColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] - -(36) BroadcastExchange -Input [6]: [cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 9] -Output [6]: [ca_state#17, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Input [8]: [c_current_cdemo_sk#2, ca_state#17, cd_demo_sk#18, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] - -(39) HashAggregate [codegen id : 9] -Input [6]: [ca_state#17, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Keys [6]: [ca_state#17, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#21), partial_max(cd_dep_count#21), partial_sum(cd_dep_count#21), partial_avg(cd_dep_employed_count#22), partial_max(cd_dep_employed_count#22), partial_sum(cd_dep_employed_count#22), partial_avg(cd_dep_college_count#23), partial_max(cd_dep_college_count#23), partial_sum(cd_dep_college_count#23)] -Aggregate Attributes [13]: [count#26, sum#27, count#28, max#29, sum#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38] -Results [19]: [ca_state#17, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] - -(40) CometColumnarExchange -Input [19]: [ca_state#17, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] -Arguments: hashpartitioning(ca_state#17, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(41) CometColumnarToRow [codegen id : 10] -Input [19]: [ca_state#17, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] - -(42) HashAggregate [codegen id : 10] -Input [19]: [ca_state#17, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] -Keys [6]: [ca_state#17, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cd_dep_employed_count#22, cd_dep_college_count#23] -Functions [10]: [count(1), avg(cd_dep_count#21), max(cd_dep_count#21), sum(cd_dep_count#21), avg(cd_dep_employed_count#22), max(cd_dep_employed_count#22), sum(cd_dep_employed_count#22), avg(cd_dep_college_count#23), max(cd_dep_college_count#23), sum(cd_dep_college_count#23)] -Aggregate Attributes [10]: [count(1)#52, avg(cd_dep_count#21)#53, max(cd_dep_count#21)#54, sum(cd_dep_count#21)#55, avg(cd_dep_employed_count#22)#56, max(cd_dep_employed_count#22)#57, sum(cd_dep_employed_count#22)#58, avg(cd_dep_college_count#23)#59, max(cd_dep_college_count#23)#60, sum(cd_dep_college_count#23)#61] -Results [18]: [ca_state#17, cd_gender#24, cd_marital_status#25, cd_dep_count#21, count(1)#52 AS cnt1#62, avg(cd_dep_count#21)#53 AS avg(cd_dep_count)#63, max(cd_dep_count#21)#54 AS max(cd_dep_count)#64, sum(cd_dep_count#21)#55 AS sum(cd_dep_count)#65, cd_dep_employed_count#22, count(1)#52 AS cnt2#66, avg(cd_dep_employed_count#22)#56 AS avg(cd_dep_employed_count)#67, max(cd_dep_employed_count#22)#57 AS max(cd_dep_employed_count)#68, sum(cd_dep_employed_count#22)#58 AS sum(cd_dep_employed_count)#69, cd_dep_college_count#23, count(1)#52 AS cnt3#70, avg(cd_dep_college_count#23)#59 AS avg(cd_dep_college_count)#71, max(cd_dep_college_count#23)#60 AS max(cd_dep_college_count)#72, sum(cd_dep_college_count#23)#61 AS sum(cd_dep_college_count)#73] - -(43) TakeOrderedAndProject -Input [18]: [ca_state#17, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#22, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#23, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] -Arguments: 100, [ca_state#17 ASC NULLS FIRST, cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_dep_count#21 ASC NULLS FIRST, cd_dep_employed_count#22 ASC NULLS FIRST, cd_dep_college_count#23 ASC NULLS FIRST], [ca_state#17, cd_gender#24, cd_marital_status#25, cd_dep_count#21, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#22, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#23, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometNativeScan parquet spark_catalog.default.date_dim (44) - - -(44) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#74, d_qoy#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [3]: [d_date_sk#7, d_year#74, d_qoy#75] -Condition : ((((isnotnull(d_year#74) AND isnotnull(d_qoy#75)) AND (d_year#74 = 1999)) AND (d_qoy#75 < 4)) AND isnotnull(d_date_sk#7)) - -(46) CometProject -Input [3]: [d_date_sk#7, d_year#74, d_qoy#75] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(48) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#12 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/simplified.txt deleted file mode 100644 index bed891a0a3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - WholeStageCodegen (10) - HashAggregate [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] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - WholeStageCodegen (9) - HashAggregate [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,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] - Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] - Project [c_current_cdemo_sk,ca_state] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_current_cdemo_sk,c_current_addr_sk] - BroadcastHashJoin [c_customer_sk,customsk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (2) - Project [ss_customer_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #4 - Union - WholeStageCodegen (4) - Project [ws_bill_customer_sk] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - WholeStageCodegen (6) - Project [cs_ship_customer_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometNativeScan 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.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/explain.txt deleted file mode 100644 index 97c7d7659b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,267 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometHashAggregate (38) - +- CometExchange (37) - +- CometHashAggregate (36) - +- CometProject (35) - +- CometBroadcastHashJoin (34) - :- CometProject (29) - : +- CometBroadcastHashJoin (28) - : :- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometBroadcastHashJoin (11) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : +- CometBroadcastExchange (10) - : : : +- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : +- CometBroadcastExchange (21) - : : +- CometUnion (20) - : : :- CometProject (15) - : : : +- CometBroadcastHashJoin (14) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (12) - : : : +- ReusedExchange (13) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (17) - : +- CometBroadcastExchange (27) - : +- CometProject (26) - : +- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) - +- CometBroadcastExchange (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (30) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) - -(6) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#4], [ss_customer_sk#4] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: [ss_customer_sk#4] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ss_customer_sk#4] -Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(13) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#7] - -(14) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Right output [1]: [d_date_sk#7] -Arguments: [ws_sold_date_sk#11], [d_date_sk#7], Inner, BuildRight - -(15) CometProject -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#7] -Arguments: [customsk#13], [ws_bill_customer_sk#10 AS customsk#13] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] -ReadSchema: struct - -(17) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#17] - -(18) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -Right output [1]: [d_date_sk#17] -Arguments: [cs_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight - -(19) CometProject -Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] -Arguments: [customsk#18], [cs_ship_customer_sk#14 AS customsk#18] - -(20) CometUnion -Child 0 Input [1]: [customsk#13] -Child 1 Input [1]: [customsk#18] - -(21) CometBroadcastExchange -Input [1]: [customsk#13] -Arguments: [customsk#13] - -(22) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [customsk#13] -Arguments: [c_customer_sk#1], [customsk#13], LeftSemi, BuildRight - -(23) CometProject -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_state#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [ca_address_sk#19, ca_state#20] -Condition : isnotnull(ca_address_sk#19) - -(26) CometProject -Input [2]: [ca_address_sk#19, ca_state#20] -Arguments: [ca_address_sk#19, ca_state#21], [ca_address_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#20, 2, true, false, true) AS ca_state#21] - -(27) CometBroadcastExchange -Input [2]: [ca_address_sk#19, ca_state#21] -Arguments: [ca_address_sk#19, ca_state#21] - -(28) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [2]: [ca_address_sk#19, ca_state#21] -Arguments: [c_current_addr_sk#3], [ca_address_sk#19], Inner, BuildRight - -(29) CometProject -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19, ca_state#21] -Arguments: [c_current_cdemo_sk#2, ca_state#21], [c_current_cdemo_sk#2, ca_state#21] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Condition : isnotnull(cd_demo_sk#22) - -(32) CometProject -Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Arguments: [cd_demo_sk#22, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27], [cd_demo_sk#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#23, 1, true, false, true) AS cd_gender#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#24, 1, true, false, true) AS cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] - -(33) CometBroadcastExchange -Input [6]: [cd_demo_sk#22, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Arguments: [cd_demo_sk#22, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] - -(34) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, ca_state#21] -Right output [6]: [cd_demo_sk#22, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#22], Inner, BuildRight - -(35) CometProject -Input [8]: [c_current_cdemo_sk#2, ca_state#21, cd_demo_sk#22, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Arguments: [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27], [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] - -(36) CometHashAggregate -Input [6]: [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Keys [6]: [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#25), partial_max(cd_dep_count#25), partial_sum(cd_dep_count#25), partial_avg(cd_dep_employed_count#26), partial_max(cd_dep_employed_count#26), partial_sum(cd_dep_employed_count#26), partial_avg(cd_dep_college_count#27), partial_max(cd_dep_college_count#27), partial_sum(cd_dep_college_count#27)] - -(37) CometExchange -Input [19]: [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38, sum#39, count#40, max#41, sum#42] -Arguments: hashpartitioning(ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(38) CometHashAggregate -Input [19]: [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38, sum#39, count#40, max#41, sum#42] -Keys [6]: [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Functions [10]: [count(1), avg(cd_dep_count#25), max(cd_dep_count#25), sum(cd_dep_count#25), avg(cd_dep_employed_count#26), max(cd_dep_employed_count#26), sum(cd_dep_employed_count#26), avg(cd_dep_college_count#27), max(cd_dep_college_count#27), sum(cd_dep_college_count#27)] - -(39) CometTakeOrderedAndProject -Input [18]: [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cnt1#43, avg(cd_dep_count)#44, max(cd_dep_count)#45, sum(cd_dep_count)#46, cd_dep_employed_count#26, cnt2#47, avg(cd_dep_employed_count)#48, max(cd_dep_employed_count)#49, sum(cd_dep_employed_count)#50, cd_dep_college_count#27, cnt3#51, avg(cd_dep_college_count)#52, max(cd_dep_college_count)#53, sum(cd_dep_college_count)#54] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#21 ASC NULLS FIRST,cd_gender#28 ASC NULLS FIRST,cd_marital_status#29 ASC NULLS FIRST,cd_dep_count#25 ASC NULLS FIRST,cd_dep_employed_count#26 ASC NULLS FIRST,cd_dep_college_count#27 ASC NULLS FIRST], output=[ca_state#21,cd_gender#28,cd_marital_status#29,cd_dep_count#25,cnt1#43,avg(cd_dep_count)#44,max(cd_dep_count)#45,sum(cd_dep_count)#46,cd_dep_employed_count#26,cnt2#47,avg(cd_dep_employed_count)#48,max(cd_dep_employed_count)#49,sum(cd_dep_employed_count)#50,cd_dep_college_count#27,cnt3#51,avg(cd_dep_college_count)#52,max(cd_dep_college_count)#53,sum(cd_dep_college_count)#54]), [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cnt1#43, avg(cd_dep_count)#44, max(cd_dep_count)#45, sum(cd_dep_count)#46, cd_dep_employed_count#26, cnt2#47, avg(cd_dep_employed_count)#48, max(cd_dep_employed_count)#49, sum(cd_dep_employed_count)#50, cd_dep_college_count#27, cnt3#51, avg(cd_dep_college_count)#52, max(cd_dep_college_count)#53, sum(cd_dep_college_count)#54], 100, 0, [ca_state#21 ASC NULLS FIRST, cd_gender#28 ASC NULLS FIRST, cd_marital_status#29 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cnt1#43, avg(cd_dep_count)#44, max(cd_dep_count)#45, sum(cd_dep_count)#46, cd_dep_employed_count#26, cnt2#47, avg(cd_dep_employed_count)#48, max(cd_dep_employed_count)#49, sum(cd_dep_employed_count)#50, cd_dep_college_count#27, cnt3#51, avg(cd_dep_college_count)#52, max(cd_dep_college_count)#53, sum(cd_dep_college_count)#54] - -(40) CometColumnarToRow [codegen id : 1] -Input [18]: [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cnt1#43, avg(cd_dep_count)#44, max(cd_dep_count)#45, sum(cd_dep_count)#46, cd_dep_employed_count#26, cnt2#47, avg(cd_dep_employed_count)#48, max(cd_dep_employed_count)#49, sum(cd_dep_employed_count)#50, cd_dep_college_count#27, cnt3#51, avg(cd_dep_college_count)#52, max(cd_dep_college_count)#53, sum(cd_dep_college_count)#54] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) - -(43) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(45) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 69e8d4868c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - CometHashAggregate [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - CometExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - 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,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_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] - CometProject [c_current_cdemo_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [customsk] #5 - CometUnion [customsk] - CometProject [ws_bill_customer_sk] [customsk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 [ca_address_sk,ca_state] #6 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] 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/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt deleted file mode 100644 index 97c7d7659b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt +++ /dev/null @@ -1,267 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometHashAggregate (38) - +- CometExchange (37) - +- CometHashAggregate (36) - +- CometProject (35) - +- CometBroadcastHashJoin (34) - :- CometProject (29) - : +- CometBroadcastHashJoin (28) - : :- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometBroadcastHashJoin (11) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : +- CometBroadcastExchange (10) - : : : +- CometProject (9) - : : : +- CometBroadcastHashJoin (8) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) - : : : +- CometBroadcastExchange (7) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (4) - : : +- CometBroadcastExchange (21) - : : +- CometUnion (20) - : : :- CometProject (15) - : : : +- CometBroadcastHashJoin (14) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (12) - : : : +- ReusedExchange (13) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (16) - : : +- ReusedExchange (17) - : +- CometBroadcastExchange (27) - : +- CometProject (26) - : +- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (24) - +- CometBroadcastExchange (33) - +- CometProject (32) - +- CometFilter (31) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (30) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -ReadSchema: struct - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) - -(6) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(7) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(8) CometBroadcastHashJoin -Left output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(9) CometProject -Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_customer_sk#4], [ss_customer_sk#4] - -(10) CometBroadcastExchange -Input [1]: [ss_customer_sk#4] -Arguments: [ss_customer_sk#4] - -(11) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [ss_customer_sk#4] -Arguments: [c_customer_sk#1], [ss_customer_sk#4], LeftSemi, BuildRight - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] -ReadSchema: struct - -(13) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#7] - -(14) CometBroadcastHashJoin -Left output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] -Right output [1]: [d_date_sk#7] -Arguments: [ws_sold_date_sk#11], [d_date_sk#7], Inner, BuildRight - -(15) CometProject -Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#7] -Arguments: [customsk#13], [ws_bill_customer_sk#10 AS customsk#13] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] -ReadSchema: struct - -(17) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#17] - -(18) CometBroadcastHashJoin -Left output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] -Right output [1]: [d_date_sk#17] -Arguments: [cs_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight - -(19) CometProject -Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] -Arguments: [customsk#18], [cs_ship_customer_sk#14 AS customsk#18] - -(20) CometUnion -Child 0 Input [1]: [customsk#13] -Child 1 Input [1]: [customsk#18] - -(21) CometBroadcastExchange -Input [1]: [customsk#13] -Arguments: [customsk#13] - -(22) CometBroadcastHashJoin -Left output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [1]: [customsk#13] -Arguments: [c_customer_sk#1], [customsk#13], LeftSemi, BuildRight - -(23) CometProject -Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] -Arguments: [c_current_cdemo_sk#2, c_current_addr_sk#3], [c_current_cdemo_sk#2, c_current_addr_sk#3] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#19, ca_state#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [ca_address_sk#19, ca_state#20] -Condition : isnotnull(ca_address_sk#19) - -(26) CometProject -Input [2]: [ca_address_sk#19, ca_state#20] -Arguments: [ca_address_sk#19, ca_state#21], [ca_address_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#20, 2, true, false, true) AS ca_state#21] - -(27) CometBroadcastExchange -Input [2]: [ca_address_sk#19, ca_state#21] -Arguments: [ca_address_sk#19, ca_state#21] - -(28) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Right output [2]: [ca_address_sk#19, ca_state#21] -Arguments: [c_current_addr_sk#3], [ca_address_sk#19], Inner, BuildRight - -(29) CometProject -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19, ca_state#21] -Arguments: [c_current_cdemo_sk#2, ca_state#21], [c_current_cdemo_sk#2, ca_state#21] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Condition : isnotnull(cd_demo_sk#22) - -(32) CometProject -Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Arguments: [cd_demo_sk#22, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27], [cd_demo_sk#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_gender#23, 1, true, false, true) AS cd_gender#28, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#24, 1, true, false, true) AS cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] - -(33) CometBroadcastExchange -Input [6]: [cd_demo_sk#22, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Arguments: [cd_demo_sk#22, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] - -(34) CometBroadcastHashJoin -Left output [2]: [c_current_cdemo_sk#2, ca_state#21] -Right output [6]: [cd_demo_sk#22, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Arguments: [c_current_cdemo_sk#2], [cd_demo_sk#22], Inner, BuildRight - -(35) CometProject -Input [8]: [c_current_cdemo_sk#2, ca_state#21, cd_demo_sk#22, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Arguments: [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27], [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] - -(36) CometHashAggregate -Input [6]: [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Keys [6]: [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#25), partial_max(cd_dep_count#25), partial_sum(cd_dep_count#25), partial_avg(cd_dep_employed_count#26), partial_max(cd_dep_employed_count#26), partial_sum(cd_dep_employed_count#26), partial_avg(cd_dep_college_count#27), partial_max(cd_dep_college_count#27), partial_sum(cd_dep_college_count#27)] - -(37) CometExchange -Input [19]: [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38, sum#39, count#40, max#41, sum#42] -Arguments: hashpartitioning(ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(38) CometHashAggregate -Input [19]: [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38, sum#39, count#40, max#41, sum#42] -Keys [6]: [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Functions [10]: [count(1), avg(cd_dep_count#25), max(cd_dep_count#25), sum(cd_dep_count#25), avg(cd_dep_employed_count#26), max(cd_dep_employed_count#26), sum(cd_dep_employed_count#26), avg(cd_dep_college_count#27), max(cd_dep_college_count#27), sum(cd_dep_college_count#27)] - -(39) CometTakeOrderedAndProject -Input [18]: [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cnt1#43, avg(cd_dep_count)#44, max(cd_dep_count)#45, sum(cd_dep_count)#46, cd_dep_employed_count#26, cnt2#47, avg(cd_dep_employed_count)#48, max(cd_dep_employed_count)#49, sum(cd_dep_employed_count)#50, cd_dep_college_count#27, cnt3#51, avg(cd_dep_college_count)#52, max(cd_dep_college_count)#53, sum(cd_dep_college_count)#54] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_state#21 ASC NULLS FIRST,cd_gender#28 ASC NULLS FIRST,cd_marital_status#29 ASC NULLS FIRST,cd_dep_count#25 ASC NULLS FIRST,cd_dep_employed_count#26 ASC NULLS FIRST,cd_dep_college_count#27 ASC NULLS FIRST], output=[ca_state#21,cd_gender#28,cd_marital_status#29,cd_dep_count#25,cnt1#43,avg(cd_dep_count)#44,max(cd_dep_count)#45,sum(cd_dep_count)#46,cd_dep_employed_count#26,cnt2#47,avg(cd_dep_employed_count)#48,max(cd_dep_employed_count)#49,sum(cd_dep_employed_count)#50,cd_dep_college_count#27,cnt3#51,avg(cd_dep_college_count)#52,max(cd_dep_college_count)#53,sum(cd_dep_college_count)#54]), [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cnt1#43, avg(cd_dep_count)#44, max(cd_dep_count)#45, sum(cd_dep_count)#46, cd_dep_employed_count#26, cnt2#47, avg(cd_dep_employed_count)#48, max(cd_dep_employed_count)#49, sum(cd_dep_employed_count)#50, cd_dep_college_count#27, cnt3#51, avg(cd_dep_college_count)#52, max(cd_dep_college_count)#53, sum(cd_dep_college_count)#54], 100, 0, [ca_state#21 ASC NULLS FIRST, cd_gender#28 ASC NULLS FIRST, cd_marital_status#29 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cnt1#43, avg(cd_dep_count)#44, max(cd_dep_count)#45, sum(cd_dep_count)#46, cd_dep_employed_count#26, cnt2#47, avg(cd_dep_employed_count)#48, max(cd_dep_employed_count)#49, sum(cd_dep_employed_count)#50, cd_dep_college_count#27, cnt3#51, avg(cd_dep_college_count)#52, max(cd_dep_college_count)#53, sum(cd_dep_college_count)#54] - -(40) CometColumnarToRow [codegen id : 1] -Input [18]: [ca_state#21, cd_gender#28, cd_marital_status#29, cd_dep_count#25, cnt1#43, avg(cd_dep_count)#44, max(cd_dep_count)#45, sum(cd_dep_count)#46, cd_dep_employed_count#26, cnt2#47, avg(cd_dep_employed_count)#48, max(cd_dep_employed_count)#49, sum(cd_dep_employed_count)#50, cd_dep_college_count#27, cnt3#51, avg(cd_dep_college_count)#52, max(cd_dep_college_count)#53, sum(cd_dep_college_count)#54] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Condition : ((((isnotnull(d_year#8) AND isnotnull(d_qoy#9)) AND (d_year#8 = 1999)) AND (d_qoy#9 < 4)) AND isnotnull(d_date_sk#7)) - -(43) CometProject -Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(45) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -Subquery:2 Hosting operator id = 12 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#6 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/extended.txt deleted file mode 100644 index 2cdc75e15e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/extended.txt +++ /dev/null @@ -1,56 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 69e8d4868c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/simplified.txt +++ /dev/null @@ -1,52 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - CometHashAggregate [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [ca_state,cd_gender,cd_marital_status,cd_dep_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cd_dep_employed_count,cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cd_dep_college_count,cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] - CometExchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 - 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,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_cdemo_sk,c_current_addr_sk,ca_address_sk,ca_state] - CometProject [c_current_cdemo_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] - CometBroadcastExchange [ss_customer_sk] #2 - CometProject [ss_customer_sk] - CometBroadcastHashJoin [ss_customer_sk,ss_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] - CometBroadcastExchange [customsk] #5 - CometUnion [customsk] - CometProject [ws_bill_customer_sk] [customsk] - CometBroadcastHashJoin [ws_bill_customer_sk,ws_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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_ship_customer_sk,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 [ca_address_sk,ca_state] #6 - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #7 - CometProject [cd_gender,cd_marital_status] [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometFilter [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/explain.txt deleted file mode 100644 index 0f517df9a4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- CometExchange (43) - +- CometHashAggregate (42) - +- CometColumnarExchange (41) - +- * HashAggregate (40) - +- Union (39) - :- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.item (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.store (14) - :- * HashAggregate (31) - : +- * CometColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- * HashAggregate (28) - : +- * HashAggregate (27) - : +- * CometColumnarToRow (26) - : +- ReusedExchange (25) - +- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * HashAggregate (34) - +- * CometColumnarToRow (33) - +- ReusedExchange (32) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) - -(4) ReusedExchange [Reuses operator id: 53] -Output [1]: [d_date_sk#7] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] - -(7) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Condition : isnotnull(i_item_sk#8) - -(9) CometProject -Input [3]: [i_item_sk#8, i_class#9, i_category#10] -Arguments: [i_item_sk#8, i_class#11, i_category#12], [i_item_sk#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#12] - -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#8, i_class#11, i_category#12] - -(11) BroadcastExchange -Input [3]: [i_item_sk#8, i_class#11, i_category#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#8] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#8, i_class#11, i_category#12] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#13, s_state#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#13, s_state#14] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#14, 2, true, false, true) = TN) AND isnotnull(s_store_sk#13)) - -(16) CometProject -Input [2]: [s_store_sk#13, s_state#14] -Arguments: [s_store_sk#13], [s_store_sk#13] - -(17) CometColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#13] - -(18) BroadcastExchange -Input [1]: [s_store_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#13] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12, s_store_sk#13] - -(21) HashAggregate [codegen id : 4] -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#11, i_category#12] -Keys [2]: [i_category#12, i_class#11] -Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum#15, sum#16] -Results [4]: [i_category#12, i_class#11, sum#17, sum#18] - -(22) CometColumnarExchange -Input [4]: [i_category#12, i_class#11, sum#17, sum#18] -Arguments: hashpartitioning(i_category#12, i_class#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 5] -Input [4]: [i_category#12, i_class#11, sum#17, sum#18] - -(24) HashAggregate [codegen id : 5] -Input [4]: [i_category#12, i_class#11, sum#17, sum#18] -Keys [2]: [i_category#12, i_class#11] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#19, sum(UnscaledValue(ss_ext_sales_price#3))#20] -Results [6]: [cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#19,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#20,17,2)) as decimal(38,20)) AS gross_margin#21, i_category#12 AS i_category#22, i_class#11 AS i_class#23, 0 AS t_category#24, 0 AS t_class#25, 0 AS lochierarchy#26] - -(25) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#12, i_class#11, sum#27, sum#28] - -(26) CometColumnarToRow [codegen id : 10] -Input [4]: [i_category#12, i_class#11, sum#27, sum#28] - -(27) HashAggregate [codegen id : 10] -Input [4]: [i_category#12, i_class#11, sum#27, sum#28] -Keys [2]: [i_category#12, i_class#11] -Functions [2]: [sum(UnscaledValue(ss_net_profit#29)), sum(UnscaledValue(ss_ext_sales_price#30))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#29))#31, sum(UnscaledValue(ss_ext_sales_price#30))#32] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#29))#31,17,2) AS ss_net_profit#33, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#30))#32,17,2) AS ss_ext_sales_price#34, i_category#12] - -(28) HashAggregate [codegen id : 10] -Input [3]: [ss_net_profit#33, ss_ext_sales_price#34, i_category#12] -Keys [1]: [i_category#12] -Functions [2]: [partial_sum(ss_net_profit#33), partial_sum(ss_ext_sales_price#34)] -Aggregate Attributes [4]: [sum#35, isEmpty#36, sum#37, isEmpty#38] -Results [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] - -(29) CometColumnarExchange -Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] -Arguments: hashpartitioning(i_category#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 11] -Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] - -(31) HashAggregate [codegen id : 11] -Input [5]: [i_category#12, sum#39, isEmpty#40, sum#41, isEmpty#42] -Keys [1]: [i_category#12] -Functions [2]: [sum(ss_net_profit#33), sum(ss_ext_sales_price#34)] -Aggregate Attributes [2]: [sum(ss_net_profit#33)#43, sum(ss_ext_sales_price#34)#44] -Results [6]: [cast((sum(ss_net_profit#33)#43 / sum(ss_ext_sales_price#34)#44) as decimal(38,20)) AS gross_margin#45, i_category#12, null AS i_class#46, 0 AS t_category#47, 1 AS t_class#48, 1 AS lochierarchy#49] - -(32) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#12, i_class#11, sum#50, sum#51] - -(33) CometColumnarToRow [codegen id : 16] -Input [4]: [i_category#12, i_class#11, sum#50, sum#51] - -(34) HashAggregate [codegen id : 16] -Input [4]: [i_category#12, i_class#11, sum#50, sum#51] -Keys [2]: [i_category#12, i_class#11] -Functions [2]: [sum(UnscaledValue(ss_net_profit#52)), sum(UnscaledValue(ss_ext_sales_price#53))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#52))#31, sum(UnscaledValue(ss_ext_sales_price#53))#32] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#52))#31,17,2) AS ss_net_profit#33, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#53))#32,17,2) AS ss_ext_sales_price#34] - -(35) HashAggregate [codegen id : 16] -Input [2]: [ss_net_profit#33, ss_ext_sales_price#34] -Keys: [] -Functions [2]: [partial_sum(ss_net_profit#33), partial_sum(ss_ext_sales_price#34)] -Aggregate Attributes [4]: [sum#54, isEmpty#55, sum#56, isEmpty#57] -Results [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] - -(36) CometColumnarExchange -Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(37) CometColumnarToRow [codegen id : 17] -Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] - -(38) HashAggregate [codegen id : 17] -Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] -Keys: [] -Functions [2]: [sum(ss_net_profit#33), sum(ss_ext_sales_price#34)] -Aggregate Attributes [2]: [sum(ss_net_profit#33)#62, sum(ss_ext_sales_price#34)#63] -Results [6]: [cast((sum(ss_net_profit#33)#62 / sum(ss_ext_sales_price#34)#63) as decimal(38,20)) AS gross_margin#64, null AS i_category#65, null AS i_class#66, 1 AS t_category#67, 1 AS t_class#68, 2 AS lochierarchy#69] - -(39) Union - -(40) HashAggregate [codegen id : 18] -Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Keys [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Functions: [] -Aggregate Attributes: [] -Results [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] - -(41) CometColumnarExchange -Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Arguments: hashpartitioning(gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(42) CometHashAggregate -Input [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Keys [6]: [gross_margin#21, i_category#22, i_class#23, t_category#24, t_class#25, lochierarchy#26] -Functions: [] - -(43) CometExchange -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#70] -Arguments: hashpartitioning(lochierarchy#26, _w0#70, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(44) CometSort -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#70] -Arguments: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#70], [lochierarchy#26 ASC NULLS FIRST, _w0#70 ASC NULLS FIRST, gross_margin#21 ASC NULLS FIRST] - -(45) CometColumnarToRow [codegen id : 19] -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#70] - -(46) Window -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#70] -Arguments: [rank(gross_margin#21) windowspecdefinition(lochierarchy#26, _w0#70, gross_margin#21 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#71], [lochierarchy#26, _w0#70], [gross_margin#21 ASC NULLS FIRST] - -(47) Project [codegen id : 20] -Output [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#71] -Input [6]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, _w0#70, rank_within_parent#71] - -(48) TakeOrderedAndProject -Input [5]: [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#71] -Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN i_category#22 END ASC NULLS FIRST, rank_within_parent#71 ASC NULLS FIRST], [gross_margin#21, i_category#22, i_class#23, lochierarchy#26, rank_within_parent#71] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometNativeScan parquet spark_catalog.default.date_dim (49) - - -(49) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#72] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [2]: [d_date_sk#7, d_year#72] -Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2001)) AND isnotnull(d_date_sk#7)) - -(51) CometProject -Input [2]: [d_date_sk#7, d_year#72] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(53) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/simplified.txt deleted file mode 100644 index 65728fc6f2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/simplified.txt +++ /dev/null @@ -1,81 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (20) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] - CometColumnarExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - WholeStageCodegen (18) - HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #3 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - WholeStageCodegen (11) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #7 - WholeStageCodegen (10) - HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (16) - HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/explain.txt deleted file mode 100644 index 0f966424f8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,275 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- Window (41) - +- * CometColumnarToRow (40) - +- CometSort (39) - +- CometExchange (38) - +- CometHashAggregate (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometUnion (34) - :- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometHashAggregate (25) - : +- ReusedExchange (24) - +- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometHashAggregate (30) - +- ReusedExchange (29) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4], [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#9, i_class#10, i_category#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Condition : isnotnull(i_item_sk#9) - -(11) CometProject -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Arguments: [i_item_sk#9, i_class#12, i_category#13], [i_item_sk#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#13] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [i_item_sk#9, i_class#12, i_category#13] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Right output [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#9], Inner, BuildRight - -(14) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) = TN) AND isnotnull(s_store_sk#14)) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14], [s_store_sk#14] - -(18) CometBroadcastExchange -Input [1]: [s_store_sk#14] -Arguments: [s_store_sk#14] - -(19) CometBroadcastHashJoin -Left output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -Right output [1]: [s_store_sk#14] -Arguments: [ss_store_sk#2], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13, s_store_sk#14] -Arguments: [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] - -(21) CometHashAggregate -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(22) CometExchange -Input [4]: [i_category#13, i_class#12, sum#16, sum#17] -Arguments: hashpartitioning(i_category#13, i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [4]: [i_category#13, i_class#12, sum#16, sum#17] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] - -(24) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#13, i_class#12, sum#18, sum#19] - -(25) CometHashAggregate -Input [4]: [i_category#13, i_class#12, sum#18, sum#19] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [sum(UnscaledValue(ss_net_profit#20)), sum(UnscaledValue(ss_ext_sales_price#21))] - -(26) CometHashAggregate -Input [3]: [ss_net_profit#22, ss_ext_sales_price#23, i_category#13] -Keys [1]: [i_category#13] -Functions [2]: [partial_sum(ss_net_profit#22), partial_sum(ss_ext_sales_price#23)] - -(27) CometExchange -Input [5]: [i_category#13, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(i_category#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [5]: [i_category#13, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [i_category#13] -Functions [2]: [sum(ss_net_profit#22), sum(ss_ext_sales_price#23)] - -(29) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#13, i_class#12, sum#28, sum#29] - -(30) CometHashAggregate -Input [4]: [i_category#13, i_class#12, sum#28, sum#29] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [sum(UnscaledValue(ss_net_profit#30)), sum(UnscaledValue(ss_ext_sales_price#31))] - -(31) CometHashAggregate -Input [2]: [ss_net_profit#22, ss_ext_sales_price#23] -Keys: [] -Functions [2]: [partial_sum(ss_net_profit#22), partial_sum(ss_ext_sales_price#23)] - -(32) CometExchange -Input [4]: [sum#32, isEmpty#33, sum#34, isEmpty#35] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometHashAggregate -Input [4]: [sum#32, isEmpty#33, sum#34, isEmpty#35] -Keys: [] -Functions [2]: [sum(ss_net_profit#22), sum(ss_ext_sales_price#23)] - -(34) CometUnion -Child 0 Input [6]: [gross_margin#36, i_category#37, i_class#38, t_category#39, t_class#40, lochierarchy#41] -Child 1 Input [6]: [gross_margin#42, i_category#13, i_class#43, t_category#44, t_class#45, lochierarchy#46] -Child 2 Input [6]: [gross_margin#47, i_category#48, i_class#49, t_category#50, t_class#51, lochierarchy#52] - -(35) CometHashAggregate -Input [6]: [gross_margin#36, i_category#37, i_class#38, t_category#39, t_class#40, lochierarchy#41] -Keys [6]: [gross_margin#36, i_category#37, i_class#38, t_category#39, t_class#40, lochierarchy#41] -Functions: [] - -(36) CometExchange -Input [6]: [gross_margin#36, i_category#37, i_class#38, t_category#39, t_class#40, lochierarchy#41] -Arguments: hashpartitioning(gross_margin#36, i_category#37, i_class#38, t_category#39, t_class#40, lochierarchy#41, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(37) CometHashAggregate -Input [6]: [gross_margin#36, i_category#37, i_class#38, t_category#39, t_class#40, lochierarchy#41] -Keys [6]: [gross_margin#36, i_category#37, i_class#38, t_category#39, t_class#40, lochierarchy#41] -Functions: [] - -(38) CometExchange -Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53] -Arguments: hashpartitioning(lochierarchy#41, _w0#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(39) CometSort -Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53] -Arguments: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53], [lochierarchy#41 ASC NULLS FIRST, _w0#53 ASC NULLS FIRST, gross_margin#36 ASC NULLS FIRST] - -(40) CometColumnarToRow [codegen id : 1] -Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53] - -(41) Window -Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53] -Arguments: [rank(gross_margin#36) windowspecdefinition(lochierarchy#41, _w0#53, gross_margin#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#54], [lochierarchy#41, _w0#53], [gross_margin#36 ASC NULLS FIRST] - -(42) Project [codegen id : 2] -Output [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] -Input [6]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53, rank_within_parent#54] - -(43) TakeOrderedAndProject -Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] -Arguments: 100, [lochierarchy#41 DESC NULLS LAST, CASE WHEN (lochierarchy#41 = 0) THEN i_category#37 END ASC NULLS FIRST, rank_within_parent#54 ASC NULLS FIRST], [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) - - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(46) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(48) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/simplified.txt deleted file mode 100644 index e91b278c14..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,55 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (2) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] - CometExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometUnion [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - 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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange [i_category] #8 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,i_category,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange #9 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_category,i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt deleted file mode 100644 index 0f966424f8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt +++ /dev/null @@ -1,275 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- Window (41) - +- * CometColumnarToRow (40) - +- CometSort (39) - +- CometExchange (38) - +- CometHashAggregate (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometUnion (34) - :- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometHashAggregate (25) - : +- ReusedExchange (24) - +- CometHashAggregate (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometHashAggregate (30) - +- ReusedExchange (29) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_store_sk#2)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: [d_date_sk#7] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] -Right output [1]: [d_date_sk#7] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5, d_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4], [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#9, i_class#10, i_category#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Condition : isnotnull(i_item_sk#9) - -(11) CometProject -Input [3]: [i_item_sk#9, i_class#10, i_category#11] -Arguments: [i_item_sk#9, i_class#12, i_category#13], [i_item_sk#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#10, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) AS i_category#13] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [i_item_sk#9, i_class#12, i_category#13] - -(13) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4] -Right output [3]: [i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#9], Inner, BuildRight - -(14) CometProject -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#9, i_class#12, i_category#13] -Arguments: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) = TN) AND isnotnull(s_store_sk#14)) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14], [s_store_sk#14] - -(18) CometBroadcastExchange -Input [1]: [s_store_sk#14] -Arguments: [s_store_sk#14] - -(19) CometBroadcastHashJoin -Left output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -Right output [1]: [s_store_sk#14] -Arguments: [ss_store_sk#2], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13, s_store_sk#14] -Arguments: [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13], [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] - -(21) CometHashAggregate -Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#12, i_category#13] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] - -(22) CometExchange -Input [4]: [i_category#13, i_class#12, sum#16, sum#17] -Arguments: hashpartitioning(i_category#13, i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [4]: [i_category#13, i_class#12, sum#16, sum#17] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] - -(24) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#13, i_class#12, sum#18, sum#19] - -(25) CometHashAggregate -Input [4]: [i_category#13, i_class#12, sum#18, sum#19] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [sum(UnscaledValue(ss_net_profit#20)), sum(UnscaledValue(ss_ext_sales_price#21))] - -(26) CometHashAggregate -Input [3]: [ss_net_profit#22, ss_ext_sales_price#23, i_category#13] -Keys [1]: [i_category#13] -Functions [2]: [partial_sum(ss_net_profit#22), partial_sum(ss_ext_sales_price#23)] - -(27) CometExchange -Input [5]: [i_category#13, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(i_category#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [5]: [i_category#13, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [i_category#13] -Functions [2]: [sum(ss_net_profit#22), sum(ss_ext_sales_price#23)] - -(29) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#13, i_class#12, sum#28, sum#29] - -(30) CometHashAggregate -Input [4]: [i_category#13, i_class#12, sum#28, sum#29] -Keys [2]: [i_category#13, i_class#12] -Functions [2]: [sum(UnscaledValue(ss_net_profit#30)), sum(UnscaledValue(ss_ext_sales_price#31))] - -(31) CometHashAggregate -Input [2]: [ss_net_profit#22, ss_ext_sales_price#23] -Keys: [] -Functions [2]: [partial_sum(ss_net_profit#22), partial_sum(ss_ext_sales_price#23)] - -(32) CometExchange -Input [4]: [sum#32, isEmpty#33, sum#34, isEmpty#35] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometHashAggregate -Input [4]: [sum#32, isEmpty#33, sum#34, isEmpty#35] -Keys: [] -Functions [2]: [sum(ss_net_profit#22), sum(ss_ext_sales_price#23)] - -(34) CometUnion -Child 0 Input [6]: [gross_margin#36, i_category#37, i_class#38, t_category#39, t_class#40, lochierarchy#41] -Child 1 Input [6]: [gross_margin#42, i_category#13, i_class#43, t_category#44, t_class#45, lochierarchy#46] -Child 2 Input [6]: [gross_margin#47, i_category#48, i_class#49, t_category#50, t_class#51, lochierarchy#52] - -(35) CometHashAggregate -Input [6]: [gross_margin#36, i_category#37, i_class#38, t_category#39, t_class#40, lochierarchy#41] -Keys [6]: [gross_margin#36, i_category#37, i_class#38, t_category#39, t_class#40, lochierarchy#41] -Functions: [] - -(36) CometExchange -Input [6]: [gross_margin#36, i_category#37, i_class#38, t_category#39, t_class#40, lochierarchy#41] -Arguments: hashpartitioning(gross_margin#36, i_category#37, i_class#38, t_category#39, t_class#40, lochierarchy#41, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(37) CometHashAggregate -Input [6]: [gross_margin#36, i_category#37, i_class#38, t_category#39, t_class#40, lochierarchy#41] -Keys [6]: [gross_margin#36, i_category#37, i_class#38, t_category#39, t_class#40, lochierarchy#41] -Functions: [] - -(38) CometExchange -Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53] -Arguments: hashpartitioning(lochierarchy#41, _w0#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(39) CometSort -Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53] -Arguments: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53], [lochierarchy#41 ASC NULLS FIRST, _w0#53 ASC NULLS FIRST, gross_margin#36 ASC NULLS FIRST] - -(40) CometColumnarToRow [codegen id : 1] -Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53] - -(41) Window -Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53] -Arguments: [rank(gross_margin#36) windowspecdefinition(lochierarchy#41, _w0#53, gross_margin#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#54], [lochierarchy#41, _w0#53], [gross_margin#36 ASC NULLS FIRST] - -(42) Project [codegen id : 2] -Output [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] -Input [6]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53, rank_within_parent#54] - -(43) TakeOrderedAndProject -Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] -Arguments: 100, [lochierarchy#41 DESC NULLS LAST, CASE WHEN (lochierarchy#41 = 0) THEN i_category#37 END ASC NULLS FIRST, rank_within_parent#54 ASC NULLS FIRST], [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) - - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_year#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(45) CometFilter -Input [2]: [d_date_sk#7, d_year#8] -Condition : ((isnotnull(d_year#8) AND (d_year#8 = 2001)) AND isnotnull(d_date_sk#7)) - -(46) CometProject -Input [2]: [d_date_sk#7, d_year#8] -Arguments: [d_date_sk#7], [d_date_sk#7] - -(47) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#7] - -(48) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/extended.txt deleted file mode 100644 index 1f6c984b22..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index e91b278c14..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt +++ /dev/null @@ -1,55 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (2) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] - CometExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometUnion [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - 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,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange [i_category] #8 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,i_category,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange #9 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_category,i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/explain.txt deleted file mode 100644 index 9613d03c41..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/explain.txt +++ /dev/null @@ -1,304 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (51) -+- * Project (50) - +- * BroadcastHashJoin Inner BuildRight (49) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * CometColumnarToRow (26) - : : +- CometSort (25) - : : +- CometColumnarExchange (24) - : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- * HashAggregate (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.store (14) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- Window (38) - : +- * CometColumnarToRow (37) - : +- CometSort (36) - : +- CometColumnarExchange (35) - : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) - : +- ReusedExchange (32) - +- BroadcastExchange (48) - +- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- ReusedExchange (43) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#4, i_category#5] - -(5) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(7) Filter [codegen id : 1] -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) - -(8) BroadcastExchange -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [ss_item_sk#6] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(11) ReusedExchange [Reuses operator id: 55] -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] -Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] - -(14) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) - -(16) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] - -(17) BroadcastExchange -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#7] -Right keys [1]: [s_store_sk#14] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] - -(20) HashAggregate [codegen id : 4] -Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] -Aggregate Attributes [1]: [sum#17] -Results [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] - -(21) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] - -(23) HashAggregate [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#18] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#8))#19] -Results [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#19,17,2) AS sum_sales#20, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#19,17,2) AS _w0#21] - -(24) CometColumnarExchange -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) CometColumnarToRow [codegen id : 6] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] - -(27) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(28) Filter [codegen id : 7] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(29) Window -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22] -Arguments: [avg(_w0#21) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(30) Filter [codegen id : 22] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] -Condition : ((isnotnull(avg_monthly_sales#23) AND (avg_monthly_sales#23 > 0.000000)) AND CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END) - -(31) Project [codegen id : 22] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, _w0#21, rn#22, avg_monthly_sales#23] - -(32) ReusedExchange [Reuses operator id: 21] -Output [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] - -(33) CometColumnarToRow [codegen id : 12] -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] - -(34) HashAggregate [codegen id : 12] -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -Keys [6]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] -Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#19] -Results [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#19,17,2) AS sum_sales#20] - -(35) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(36) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] -Arguments: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] - -(37) CometColumnarToRow [codegen id : 13] -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] - -(38) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20] -Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#5, i_brand#4, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] - -(39) Project [codegen id : 14] -Output [6]: [i_category#5 AS i_category#31, i_brand#4 AS i_brand#32, s_store_name#24, s_company_name#25, sum_sales#20 AS sum_sales#33, rn#30] -Input [8]: [i_category#5, i_brand#4, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#20, rn#30] - -(40) BroadcastExchange -Input [6]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] - -(41) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] -Right keys [5]: [i_category#31, i_brand#32, s_store_name#24, s_company_name#25, (rn#30 + 1)] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 22] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33] -Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, i_category#31, i_brand#32, s_store_name#24, s_company_name#25, sum_sales#33, rn#30] - -(43) ReusedExchange [Reuses operator id: 35] -Output [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] - -(44) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] -Arguments: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] - -(45) CometColumnarToRow [codegen id : 20] -Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] - -(46) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20] -Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#5, i_brand#4, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] - -(47) Project [codegen id : 21] -Output [6]: [i_category#5 AS i_category#39, i_brand#4 AS i_brand#40, s_store_name#34, s_company_name#35, sum_sales#20 AS sum_sales#41, rn#38] -Input [8]: [i_category#5, i_brand#4, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#20, rn#38] - -(48) BroadcastExchange -Input [6]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] - -(49) BroadcastHashJoin [codegen id : 22] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#22] -Right keys [5]: [i_category#39, i_brand#40, s_store_name#34, s_company_name#35, (rn#38 - 1)] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 22] -Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, sum_sales#33 AS psum#42, sum_sales#41 AS nsum#43] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#20, avg_monthly_sales#23, rn#22, sum_sales#33, i_category#39, i_brand#40, s_store_name#34, s_company_name#35, sum_sales#41, rn#38] - -(51) TakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] -Arguments: 100, [(sum_sales#20 - avg_monthly_sales#23) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#23, sum_sales#20, psum#42, nsum#43] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometFilter (53) - +- CometNativeScan parquet spark_catalog.default.date_dim (52) - - -(52) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(54) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(55) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/simplified.txt deleted file mode 100644 index ee8e4dd40c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/simplified.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (22) - Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (7) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] - Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - BroadcastHashJoin [i_item_sk,ss_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_store_name,s_company_name] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/explain.txt deleted file mode 100644 index 7b1f860f32..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_item_sk#1], [ss_item_sk#6], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) - -(16) CometBroadcastExchange -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [s_store_sk#14, s_store_name#15, s_company_name#16] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] -Right output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [ss_store_sk#7], [s_store_sk#14], Inner, BuildRight - -(18) CometProject -Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16], [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] - -(19) CometHashAggregate -Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] - -(20) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] - -(22) CometExchange -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] - -(25) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(28) Filter [codegen id : 7] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] - -(30) ReusedExchange [Reuses operator id: 20] -Output [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] - -(31) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] -Keys [6]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(ss_sales_price#27))] - -(32) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] - -(35) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [6]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] -Input [8]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28] - -(37) BroadcastExchange -Input [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] - -(40) ReusedExchange [Reuses operator id: 32] -Output [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(41) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(43) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#5, i_brand#4, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [6]: [i_category#5 AS i_category#37, i_brand#4 AS i_brand#38, s_store_name#32, s_company_name#33, sum_sales#18 AS sum_sales#39, rn#36] -Input [8]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18, rn#36] - -(45) BroadcastExchange -Input [6]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, (rn#36 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] - -(48) TakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/simplified.txt deleted file mode 100644 index a62c33ecc1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (7) - Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - 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 [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,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt deleted file mode 100644 index 7b1f860f32..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Condition : (isnotnull(ss_item_sk#6) AND isnotnull(ss_store_sk#7)) - -(6) CometBroadcastExchange -Input [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_item_sk#1], [ss_item_sk#6], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, ss_item_sk#6, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [ss_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, ss_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] -ReadSchema: struct - -(15) CometFilter -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Condition : ((isnotnull(s_store_sk#14) AND isnotnull(s_store_name#15)) AND isnotnull(s_company_name#16)) - -(16) CometBroadcastExchange -Input [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [s_store_sk#14, s_store_name#15, s_company_name#16] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13] -Right output [3]: [s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [ss_store_sk#7], [s_store_sk#14], Inner, BuildRight - -(18) CometProject -Input [9]: [i_brand#4, i_category#5, ss_store_sk#7, ss_sales_price#8, d_year#12, d_moy#13, s_store_sk#14, s_store_name#15, s_company_name#16] -Arguments: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16], [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] - -(19) CometHashAggregate -Input [7]: [i_brand#4, i_category#5, ss_sales_price#8, d_year#12, d_moy#13, s_store_name#15, s_company_name#16] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] - -(20) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] -Keys [6]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(ss_sales_price#8))] - -(22) CometExchange -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] - -(25) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(28) Filter [codegen id : 7] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] - -(30) ReusedExchange [Reuses operator id: 20] -Output [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] - -(31) CometHashAggregate -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] -Keys [6]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(ss_sales_price#27))] - -(32) CometExchange -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] - -(35) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [6]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] -Input [8]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28] - -(37) BroadcastExchange -Input [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] - -(40) ReusedExchange [Reuses operator id: 32] -Output [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(41) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(43) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#5, i_brand#4, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [6]: [i_category#5 AS i_category#37, i_brand#4 AS i_brand#38, s_store_name#32, s_company_name#33, sum_sales#18 AS sum_sales#39, rn#36] -Input [8]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18, rn#36] - -(45) BroadcastExchange -Input [6]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, (rn#36 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] - -(48) TakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/extended.txt deleted file mode 100644 index b50b570b4b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index a62c33ecc1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (7) - Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - 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 [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,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/explain.txt deleted file mode 100644 index 070113d8ea..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/explain.txt +++ /dev/null @@ -1,497 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (84) -+- CometTakeOrderedAndProject (83) - +- CometHashAggregate (82) - +- CometColumnarExchange (81) - +- * HashAggregate (80) - +- Union (79) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometColumnarExchange (19) - : +- * HashAggregate (18) - : +- * CometColumnarToRow (17) - : +- CometColumnarExchange (16) - : +- * HashAggregate (15) - : +- * Project (14) - : +- * BroadcastHashJoin Inner BuildRight (13) - : :- * Project (11) - : : +- * BroadcastHashJoin Inner BuildLeft (10) - : : :- BroadcastExchange (5) - : : : +- * Project (4) - : : : +- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : +- * CometColumnarToRow (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometNativeScan parquet spark_catalog.default.web_returns (6) - : +- ReusedExchange (12) - :- * Project (52) - : +- * Filter (51) - : +- Window (50) - : +- * Sort (49) - : +- Window (48) - : +- * CometColumnarToRow (47) - : +- CometSort (46) - : +- CometColumnarExchange (45) - : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- CometColumnarExchange (42) - : +- * HashAggregate (41) - : +- * Project (40) - : +- * BroadcastHashJoin Inner BuildRight (39) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildLeft (36) - : : :- BroadcastExchange (31) - : : : +- * Project (30) - : : : +- * Filter (29) - : : : +- * ColumnarToRow (28) - : : : +- Scan parquet spark_catalog.default.catalog_sales (27) - : : +- * CometColumnarToRow (35) - : : +- CometProject (34) - : : +- CometFilter (33) - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (32) - : +- ReusedExchange (38) - +- * Project (78) - +- * Filter (77) - +- Window (76) - +- * Sort (75) - +- Window (74) - +- * CometColumnarToRow (73) - +- CometSort (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * CometColumnarToRow (69) - +- CometColumnarExchange (68) - +- * HashAggregate (67) - +- * Project (66) - +- * BroadcastHashJoin Inner BuildRight (65) - :- * Project (63) - : +- * BroadcastHashJoin Inner BuildLeft (62) - : :- BroadcastExchange (57) - : : +- * Project (56) - : : +- * Filter (55) - : : +- * ColumnarToRow (54) - : : +- Scan parquet spark_catalog.default.store_sales (53) - : +- * CometColumnarToRow (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometNativeScan parquet spark_catalog.default.store_returns (58) - +- ReusedExchange (64) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] - -(3) Filter [codegen id : 1] -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND isnotnull(ws_quantity#3)) AND (ws_net_profit#5 > 1.00)) AND (ws_net_paid#4 > 0.00)) AND (ws_quantity#3 > 0)) AND isnotnull(ws_order_number#2)) AND isnotnull(ws_item_sk#1)) - -(4) Project [codegen id : 1] -Output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] - -(5) BroadcastExchange -Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=1] - -(6) CometNativeScan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) - -(8) CometProject -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(9) CometColumnarToRow -Input [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(10) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [ws_order_number#2, ws_item_sk#1] -Right keys [2]: [wr_order_number#9, wr_item_sk#8] -Join type: Inner -Join condition: None - -(11) Project [codegen id : 3] -Output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(12) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#13] - -(13) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#6] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(14) Project [codegen id : 3] -Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] - -(15) HashAggregate [codegen id : 3] -Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Keys [1]: [ws_item_sk#1] -Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#14, sum#15, sum#16, isEmpty#17, sum#18, isEmpty#19] -Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] - -(16) CometColumnarExchange -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(17) CometColumnarToRow [codegen id : 4] -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] - -(18) HashAggregate [codegen id : 4] -Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] -Keys [1]: [ws_item_sk#1] -Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] -Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] - -(19) CometColumnarExchange -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(20) CometSort -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [item#30, return_ratio#31, currency_ratio#32], [return_ratio#31 ASC NULLS FIRST] - -(21) CometColumnarToRow [codegen id : 5] -Input [3]: [item#30, return_ratio#31, currency_ratio#32] - -(22) Window -Input [3]: [item#30, return_ratio#31, currency_ratio#32] -Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] - -(23) Sort [codegen id : 6] -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 - -(24) Window -Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] -Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] - -(25) Filter [codegen id : 7] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] -Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) - -(26) Project [codegen id : 7] -Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] - -(27) Scan parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(28) ColumnarToRow [codegen id : 8] -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] - -(29) Filter [codegen id : 8] -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] -Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) - -(30) Project [codegen id : 8] -Output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] - -(31) BroadcastExchange -Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=4] - -(32) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(33) CometFilter -Input [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] -Condition : (((isnotnull(cr_return_amount#45) AND (cr_return_amount#45 > 10000.00)) AND isnotnull(cr_order_number#43)) AND isnotnull(cr_item_sk#42)) - -(34) CometProject -Input [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] -Arguments: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45], [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] - -(35) CometColumnarToRow -Input [4]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] - -(36) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [cs_order_number#37, cs_item_sk#36] -Right keys [2]: [cr_order_number#43, cr_item_sk#42] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 10] -Output [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45] -Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45] - -(38) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#47] - -(39) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#41] -Right keys [1]: [d_date_sk#47] -Join type: Inner -Join condition: None - -(40) Project [codegen id : 10] -Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] -Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#44, cr_return_amount#45, d_date_sk#47] - -(41) HashAggregate [codegen id : 10] -Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#44, cr_return_amount#45] -Keys [1]: [cs_item_sk#36] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#44, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#48, sum#49, sum#50, isEmpty#51, sum#52, isEmpty#53] -Results [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] - -(42) CometColumnarExchange -Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(43) CometColumnarToRow [codegen id : 11] -Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] - -(44) HashAggregate [codegen id : 11] -Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Keys [1]: [cs_item_sk#36] -Functions [4]: [sum(coalesce(cr_return_quantity#44, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#44, 0))#60, sum(coalesce(cs_quantity#38, 0))#61, sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#62, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63] -Results [3]: [cs_item_sk#36 AS item#64, (cast(sum(coalesce(cr_return_quantity#44, 0))#60 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#61 as decimal(15,4))) AS return_ratio#65, (cast(sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#62 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63 as decimal(15,4))) AS currency_ratio#66] - -(45) CometColumnarExchange -Input [3]: [item#64, return_ratio#65, currency_ratio#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(46) CometSort -Input [3]: [item#64, return_ratio#65, currency_ratio#66] -Arguments: [item#64, return_ratio#65, currency_ratio#66], [return_ratio#65 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 12] -Input [3]: [item#64, return_ratio#65, currency_ratio#66] - -(48) Window -Input [3]: [item#64, return_ratio#65, currency_ratio#66] -Arguments: [rank(return_ratio#65) windowspecdefinition(return_ratio#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#67], [return_ratio#65 ASC NULLS FIRST] - -(49) Sort [codegen id : 13] -Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] -Arguments: [currency_ratio#66 ASC NULLS FIRST], false, 0 - -(50) Window -Input [4]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67] -Arguments: [rank(currency_ratio#66) windowspecdefinition(currency_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#68], [currency_ratio#66 ASC NULLS FIRST] - -(51) Filter [codegen id : 14] -Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] -Condition : ((return_rank#67 <= 10) OR (currency_rank#68 <= 10)) - -(52) Project [codegen id : 14] -Output [5]: [catalog AS channel#69, item#64, return_ratio#65, return_rank#67, currency_rank#68] -Input [5]: [item#64, return_ratio#65, currency_ratio#66, return_rank#67, currency_rank#68] - -(53) Scan parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#75), dynamicpruningexpression(ss_sold_date_sk#75 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(54) ColumnarToRow [codegen id : 15] -Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] - -(55) Filter [codegen id : 15] -Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] -Condition : (((((((isnotnull(ss_net_profit#74) AND isnotnull(ss_net_paid#73)) AND isnotnull(ss_quantity#72)) AND (ss_net_profit#74 > 1.00)) AND (ss_net_paid#73 > 0.00)) AND (ss_quantity#72 > 0)) AND isnotnull(ss_ticket_number#71)) AND isnotnull(ss_item_sk#70)) - -(56) Project [codegen id : 15] -Output [5]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75] -Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_net_profit#74, ss_sold_date_sk#75] - -(57) BroadcastExchange -Input [5]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=7] - -(58) CometNativeScan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(59) CometFilter -Input [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] -Condition : (((isnotnull(sr_return_amt#79) AND (sr_return_amt#79 > 10000.00)) AND isnotnull(sr_ticket_number#77)) AND isnotnull(sr_item_sk#76)) - -(60) CometProject -Input [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] -Arguments: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79], [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] - -(61) CometColumnarToRow -Input [4]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] - -(62) BroadcastHashJoin [codegen id : 17] -Left keys [2]: [ss_ticket_number#71, ss_item_sk#70] -Right keys [2]: [sr_ticket_number#77, sr_item_sk#76] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 17] -Output [6]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79] -Input [9]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79] - -(64) ReusedExchange [Reuses operator id: 89] -Output [1]: [d_date_sk#81] - -(65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#75] -Right keys [1]: [d_date_sk#81] -Join type: Inner -Join condition: None - -(66) Project [codegen id : 17] -Output [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] -Input [7]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75, sr_return_quantity#78, sr_return_amt#79, d_date_sk#81] - -(67) HashAggregate [codegen id : 17] -Input [5]: [ss_item_sk#70, ss_quantity#72, ss_net_paid#73, sr_return_quantity#78, sr_return_amt#79] -Keys [1]: [ss_item_sk#70] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#78, 0)), partial_sum(coalesce(ss_quantity#72, 0)), partial_sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#82, sum#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Results [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] - -(68) CometColumnarExchange -Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] -Arguments: hashpartitioning(ss_item_sk#70, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(69) CometColumnarToRow [codegen id : 18] -Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] - -(70) HashAggregate [codegen id : 18] -Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#93] -Keys [1]: [ss_item_sk#70] -Functions [4]: [sum(coalesce(sr_return_quantity#78, 0)), sum(coalesce(ss_quantity#72, 0)), sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#78, 0))#94, sum(coalesce(ss_quantity#72, 0))#95, sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00))#96, sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97] -Results [3]: [ss_item_sk#70 AS item#98, (cast(sum(coalesce(sr_return_quantity#78, 0))#94 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#72, 0))#95 as decimal(15,4))) AS return_ratio#99, (cast(sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00))#96 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97 as decimal(15,4))) AS currency_ratio#100] - -(71) CometColumnarExchange -Input [3]: [item#98, return_ratio#99, currency_ratio#100] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(72) CometSort -Input [3]: [item#98, return_ratio#99, currency_ratio#100] -Arguments: [item#98, return_ratio#99, currency_ratio#100], [return_ratio#99 ASC NULLS FIRST] - -(73) CometColumnarToRow [codegen id : 19] -Input [3]: [item#98, return_ratio#99, currency_ratio#100] - -(74) Window -Input [3]: [item#98, return_ratio#99, currency_ratio#100] -Arguments: [rank(return_ratio#99) windowspecdefinition(return_ratio#99 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#101], [return_ratio#99 ASC NULLS FIRST] - -(75) Sort [codegen id : 20] -Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] -Arguments: [currency_ratio#100 ASC NULLS FIRST], false, 0 - -(76) Window -Input [4]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101] -Arguments: [rank(currency_ratio#100) windowspecdefinition(currency_ratio#100 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#102], [currency_ratio#100 ASC NULLS FIRST] - -(77) Filter [codegen id : 21] -Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] -Condition : ((return_rank#101 <= 10) OR (currency_rank#102 <= 10)) - -(78) Project [codegen id : 21] -Output [5]: [store AS channel#103, item#98, return_ratio#99, return_rank#101, currency_rank#102] -Input [5]: [item#98, return_ratio#99, currency_ratio#100, return_rank#101, currency_rank#102] - -(79) Union - -(80) HashAggregate [codegen id : 22] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] - -(81) CometColumnarExchange -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(82) CometHashAggregate -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Functions: [] - -(83) CometTakeOrderedAndProject -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,return_rank#33 ASC NULLS FIRST,currency_rank#34 ASC NULLS FIRST,item#30 ASC NULLS FIRST], output=[channel#35,item#30,return_ratio#31,return_rank#33,currency_rank#34]), [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34], 100, 0, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST, item#30 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] - -(84) CometColumnarToRow [codegen id : 23] -Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (89) -+- * CometColumnarToRow (88) - +- CometProject (87) - +- CometFilter (86) - +- CometNativeScan parquet spark_catalog.default.date_dim (85) - - -(85) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#104, d_moy#105] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(86) CometFilter -Input [3]: [d_date_sk#13, d_year#104, d_moy#105] -Condition : ((((isnotnull(d_year#104) AND isnotnull(d_moy#105)) AND (d_year#104 = 2001)) AND (d_moy#105 = 12)) AND isnotnull(d_date_sk#13)) - -(87) CometProject -Input [3]: [d_date_sk#13, d_year#104, d_moy#105] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(88) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(89) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 - -Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#75 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/simplified.txt deleted file mode 100644 index 047269434d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/simplified.txt +++ /dev/null @@ -1,140 +0,0 @@ -WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (22) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (7) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (6) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #2 - WholeStageCodegen (4) - HashAggregate [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - BroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - Filter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometColumnarToRow - InputAdapter - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (14) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (13) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (12) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #6 - WholeStageCodegen (11) - HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_item_sk] #7 - WholeStageCodegen (10) - HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - BroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - Project [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - Filter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometColumnarToRow - InputAdapter - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (21) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (20) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometColumnarExchange #9 - WholeStageCodegen (18) - HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk] #10 - WholeStageCodegen (17) - HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] - Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - BroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - Filter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/explain.txt deleted file mode 100644 index 74702d596d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,462 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- * Project (49) - : +- * Filter (48) - : +- Window (47) - : +- * Sort (46) - : +- Window (45) - : +- * CometColumnarToRow (44) - : +- CometSort (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometBroadcastExchange (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometBroadcastHashJoin (57) - : :- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (59) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND isnotnull(ws_quantity#3)) AND (ws_net_profit#5 > 1.00)) AND (ws_net_paid#4 > 0.00)) AND (ws_quantity#3 > 0)) AND isnotnull(ws_order_number#2)) AND isnotnull(ws_item_sk#1)) - -(3) CometProject -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6], [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(4) CometBroadcastExchange -Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) - -(7) CometProject -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(8) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner, BuildLeft - -(9) CometProject -Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(12) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] - -(16) CometHashAggregate -Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Keys [1]: [ws_item_sk#1] -Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(17) CometExchange -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Keys [1]: [ws_item_sk#1] -Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(19) CometExchange -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] - -(21) CometColumnarToRow [codegen id : 1] -Input [3]: [item#22, return_ratio#23, currency_ratio#24] - -(22) Window -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] - -(23) Sort [codegen id : 2] -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 - -(24) Window -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] - -(25) Filter [codegen id : 3] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] -Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) - -(26) Project [codegen id : 3] -Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_sold_date_sk#33 IN dynamicpruning#34)] -PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(28) CometFilter -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) - -(29) CometProject -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(30) CometBroadcastExchange -Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(32) CometFilter -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) - -(33) CometProject -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] - -(34) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft - -(35) CometProject -Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] - -(36) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#40] - -(37) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -Right output [1]: [d_date_sk#40] -Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight - -(38) CometProject -Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] - -(39) CometHashAggregate -Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -Keys [1]: [cs_item_sk#28] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(40) CometExchange -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(41) CometHashAggregate -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Keys [1]: [cs_item_sk#28] -Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(42) CometExchange -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(43) CometSort -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] - -(44) CometColumnarToRow [codegen id : 4] -Input [3]: [item#47, return_ratio#48, currency_ratio#49] - -(45) Window -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] - -(46) Sort [codegen id : 5] -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 - -(47) Window -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] - -(48) Filter [codegen id : 6] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] -Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) - -(49) Project [codegen id : 6] -Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) - -(52) CometProject -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(53) CometBroadcastExchange -Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(55) CometFilter -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) - -(56) CometProject -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] - -(57) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft - -(58) CometProject -Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] - -(59) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#65] - -(60) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(61) CometProject -Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] - -(62) CometHashAggregate -Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -Keys [1]: [ss_item_sk#53] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(63) CometExchange -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(64) CometHashAggregate -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Keys [1]: [ss_item_sk#53] -Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(65) CometExchange -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(66) CometSort -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] - -(67) CometColumnarToRow [codegen id : 7] -Input [3]: [item#72, return_ratio#73, currency_ratio#74] - -(68) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] - -(69) Sort [codegen id : 8] -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 - -(70) Window -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] - -(71) Filter [codegen id : 9] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] -Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) - -(72) Project [codegen id : 9] -Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] - -(73) Union - -(74) HashAggregate [codegen id : 10] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(75) CometColumnarExchange -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometHashAggregate -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] - -(77) CometTakeOrderedAndProject -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST,item#22 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST, item#22 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(78) CometColumnarToRow [codegen id : 11] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) - - -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(80) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(81) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(82) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(83) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 - -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/simplified.txt deleted file mode 100644 index ca80833ee5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,110 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (10) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (3) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (2) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - 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_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_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - 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_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_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 (9) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (8) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - 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_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_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt deleted file mode 100644 index 74702d596d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt +++ /dev/null @@ -1,462 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- * Project (49) - : +- * Filter (48) - : +- Window (47) - : +- * Sort (46) - : +- Window (45) - : +- * CometColumnarToRow (44) - : +- CometSort (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometBroadcastExchange (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometBroadcastHashJoin (57) - : :- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (59) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] -PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND isnotnull(ws_quantity#3)) AND (ws_net_profit#5 > 1.00)) AND (ws_net_paid#4 > 0.00)) AND (ws_quantity#3 > 0)) AND isnotnull(ws_order_number#2)) AND isnotnull(ws_item_sk#1)) - -(3) CometProject -Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6], [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(4) CometBroadcastExchange -Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) - -(7) CometProject -Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] -Arguments: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11], [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] - -(8) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner, BuildLeft - -(9) CometProject -Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(12) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(13) CometBroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: [d_date_sk#13] - -(14) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] -Right output [1]: [d_date_sk#13] -Arguments: [ws_sold_date_sk#6], [d_date_sk#13], Inner, BuildRight - -(15) CometProject -Input [7]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11, d_date_sk#13] -Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] - -(16) CometHashAggregate -Input [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] -Keys [1]: [ws_item_sk#1] -Functions [4]: [partial_sum(coalesce(wr_return_quantity#10, 0)), partial_sum(coalesce(ws_quantity#3, 0)), partial_sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(17) CometExchange -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(18) CometHashAggregate -Input [7]: [ws_item_sk#1, sum#16, sum#17, sum#18, isEmpty#19, sum#20, isEmpty#21] -Keys [1]: [ws_item_sk#1] -Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] - -(19) CometExchange -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] - -(21) CometColumnarToRow [codegen id : 1] -Input [3]: [item#22, return_ratio#23, currency_ratio#24] - -(22) Window -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] - -(23) Sort [codegen id : 2] -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 - -(24) Window -Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] - -(25) Filter [codegen id : 3] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] -Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) - -(26) Project [codegen id : 3] -Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_sold_date_sk#33 IN dynamicpruning#34)] -PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] -ReadSchema: struct - -(28) CometFilter -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) - -(29) CometProject -Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(30) CometBroadcastExchange -Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(32) CometFilter -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) - -(33) CometProject -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] - -(34) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft - -(35) CometProject -Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] - -(36) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#40] - -(37) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -Right output [1]: [d_date_sk#40] -Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight - -(38) CometProject -Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] -Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] - -(39) CometHashAggregate -Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -Keys [1]: [cs_item_sk#28] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(40) CometExchange -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(41) CometHashAggregate -Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] -Keys [1]: [cs_item_sk#28] -Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] - -(42) CometExchange -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(43) CometSort -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] - -(44) CometColumnarToRow [codegen id : 4] -Input [3]: [item#47, return_ratio#48, currency_ratio#49] - -(45) Window -Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] - -(46) Sort [codegen id : 5] -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 - -(47) Window -Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] - -(48) Filter [codegen id : 6] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] -Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) - -(49) Project [codegen id : 6] -Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) - -(52) CometProject -Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(53) CometBroadcastExchange -Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(55) CometFilter -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) - -(56) CometProject -Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] -Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] - -(57) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft - -(58) CometProject -Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] - -(59) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#65] - -(60) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -Right output [1]: [d_date_sk#65] -Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(61) CometProject -Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] -Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] - -(62) CometHashAggregate -Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -Keys [1]: [ss_item_sk#53] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(63) CometExchange -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(64) CometHashAggregate -Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Keys [1]: [ss_item_sk#53] -Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] - -(65) CometExchange -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(66) CometSort -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] - -(67) CometColumnarToRow [codegen id : 7] -Input [3]: [item#72, return_ratio#73, currency_ratio#74] - -(68) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] - -(69) Sort [codegen id : 8] -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 - -(70) Window -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] - -(71) Filter [codegen id : 9] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] -Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) - -(72) Project [codegen id : 9] -Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] - -(73) Union - -(74) HashAggregate [codegen id : 10] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(75) CometColumnarExchange -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(76) CometHashAggregate -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Functions: [] - -(77) CometTakeOrderedAndProject -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST,item#22 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST, item#22 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -(78) CometColumnarToRow [codegen id : 11] -Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) - - -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] -ReadSchema: struct - -(80) CometFilter -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) - -(81) CometProject -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Arguments: [d_date_sk#13], [d_date_sk#13] - -(82) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#13] - -(83) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 - -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/extended.txt deleted file mode 100644 index 75684a9669..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/extended.txt +++ /dev/null @@ -1,94 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index ca80833ee5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt +++ /dev/null @@ -1,110 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (10) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (3) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (2) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - 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_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_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] 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_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - 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_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_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 (9) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (8) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - 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_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_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/explain.txt deleted file mode 100644 index 23adfe692b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/explain.txt +++ /dev/null @@ -1,455 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (77) -+- * Filter (76) - +- * HashAggregate (75) - +- * HashAggregate (74) - +- * Project (73) - +- * BroadcastHashJoin Inner BuildRight (72) - :- Window (65) - : +- * CometColumnarToRow (64) - : +- CometSort (63) - : +- CometExchange (62) - : +- CometProject (61) - : +- CometFilter (60) - : +- CometSortMergeJoin (59) - : :- CometSort (29) - : : +- CometColumnarExchange (28) - : : +- * HashAggregate (27) - : : +- * CometColumnarToRow (26) - : : +- CometColumnarExchange (25) - : : +- * HashAggregate (24) - : : +- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * Project (15) - : : : +- Window (14) - : : : +- * CometColumnarToRow (13) - : : : +- CometSort (12) - : : : +- CometColumnarExchange (11) - : : : +- * HashAggregate (10) - : : : +- * CometColumnarToRow (9) - : : : +- CometColumnarExchange (8) - : : : +- * HashAggregate (7) - : : : +- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (21) - : : +- * Project (20) - : : +- Window (19) - : : +- * CometColumnarToRow (18) - : : +- CometSort (17) - : : +- ReusedExchange (16) - : +- CometSort (58) - : +- CometColumnarExchange (57) - : +- * HashAggregate (56) - : +- * CometColumnarToRow (55) - : +- CometColumnarExchange (54) - : +- * HashAggregate (53) - : +- * Project (52) - : +- * BroadcastHashJoin Inner BuildRight (51) - : :- * Project (44) - : : +- Window (43) - : : +- * CometColumnarToRow (42) - : : +- CometSort (41) - : : +- CometColumnarExchange (40) - : : +- * HashAggregate (39) - : : +- * CometColumnarToRow (38) - : : +- CometColumnarExchange (37) - : : +- * HashAggregate (36) - : : +- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * Filter (32) - : : : +- * ColumnarToRow (31) - : : : +- Scan parquet spark_catalog.default.store_sales (30) - : : +- ReusedExchange (33) - : +- BroadcastExchange (50) - : +- * Project (49) - : +- Window (48) - : +- * CometColumnarToRow (47) - : +- CometSort (46) - : +- ReusedExchange (45) - +- BroadcastExchange (71) - +- * Project (70) - +- Window (69) - +- * CometColumnarToRow (68) - +- CometSort (67) - +- ReusedExchange (66) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] - -(3) Filter [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(4) ReusedExchange [Reuses operator id: 82] -Output [2]: [d_date_sk#5, d_date#6] - -(5) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 2] -Output [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] - -(7) HashAggregate [codegen id : 2] -Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum#7] -Results [3]: [ws_item_sk#1, d_date#6, sum#8] - -(8) CometColumnarExchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(9) CometColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, d_date#6, sum#8] - -(10) HashAggregate [codegen id : 3] -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#9] -Results [4]: [ws_item_sk#1 AS item_sk#10, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#9,17,2) AS sumws#11, ws_item_sk#1] - -(11) CometColumnarExchange -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(12) CometSort -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(13) CometColumnarToRow [codegen id : 4] -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] - -(14) Window -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(15) Project [codegen id : 10] -Output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] -Input [5]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12] - -(16) ReusedExchange [Reuses operator id: 11] -Output [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] - -(17) CometSort -Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -Arguments: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14], [ws_item_sk#14 ASC NULLS FIRST, d_date#13 ASC NULLS FIRST] - -(18) CometColumnarToRow [codegen id : 8] -Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] - -(19) Window -Input [4]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14] -Arguments: [row_number() windowspecdefinition(ws_item_sk#14, d_date#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#15], [ws_item_sk#14], [d_date#13 ASC NULLS FIRST] - -(20) Project [codegen id : 9] -Output [3]: [item_sk#10 AS item_sk#16, sumws#11 AS sumws#17, rk#15] -Input [5]: [item_sk#10, d_date#13, sumws#11, ws_item_sk#14, rk#15] - -(21) BroadcastExchange -Input [3]: [item_sk#16, sumws#17, rk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(22) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [item_sk#10] -Right keys [1]: [item_sk#16] -Join type: Inner -Join condition: (rk#12 >= rk#15) - -(23) Project [codegen id : 10] -Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] -Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#16, sumws#17, rk#15] - -(24) HashAggregate [codegen id : 10] -Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#17] -Keys [3]: [item_sk#10, d_date#6, sumws#11] -Functions [1]: [partial_sum(sumws#17)] -Aggregate Attributes [2]: [sum#18, isEmpty#19] -Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] - -(25) CometColumnarExchange -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(26) CometColumnarToRow [codegen id : 11] -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] - -(27) HashAggregate [codegen id : 11] -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -Keys [3]: [item_sk#10, d_date#6, sumws#11] -Functions [1]: [sum(sumws#17)] -Aggregate Attributes [1]: [sum(sumws#17)#22] -Results [3]: [item_sk#10, d_date#6, sum(sumws#17)#22 AS cume_sales#23] - -(28) CometColumnarExchange -Input [3]: [item_sk#10, d_date#6, cume_sales#23] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(29) CometSort -Input [3]: [item_sk#10, d_date#6, cume_sales#23] -Arguments: [item_sk#10, d_date#6, cume_sales#23], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(30) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 13] -Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] - -(32) Filter [codegen id : 13] -Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#24) - -(33) ReusedExchange [Reuses operator id: 82] -Output [2]: [d_date_sk#27, d_date#28] - -(34) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 13] -Output [3]: [ss_item_sk#24, ss_sales_price#25, d_date#28] -Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#27, d_date#28] - -(36) HashAggregate [codegen id : 13] -Input [3]: [ss_item_sk#24, ss_sales_price#25, d_date#28] -Keys [2]: [ss_item_sk#24, d_date#28] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum#29] -Results [3]: [ss_item_sk#24, d_date#28, sum#30] - -(37) CometColumnarExchange -Input [3]: [ss_item_sk#24, d_date#28, sum#30] -Arguments: hashpartitioning(ss_item_sk#24, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(38) CometColumnarToRow [codegen id : 14] -Input [3]: [ss_item_sk#24, d_date#28, sum#30] - -(39) HashAggregate [codegen id : 14] -Input [3]: [ss_item_sk#24, d_date#28, sum#30] -Keys [2]: [ss_item_sk#24, d_date#28] -Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#31] -Results [4]: [ss_item_sk#24 AS item_sk#32, d_date#28, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#31,17,2) AS sumss#33, ss_item_sk#24] - -(40) CometColumnarExchange -Input [4]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24] -Arguments: hashpartitioning(ss_item_sk#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(41) CometSort -Input [4]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24] -Arguments: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24], [ss_item_sk#24 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 15] -Input [4]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24] - -(43) Window -Input [4]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24] -Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#34], [ss_item_sk#24], [d_date#28 ASC NULLS FIRST] - -(44) Project [codegen id : 21] -Output [4]: [item_sk#32, d_date#28, sumss#33, rk#34] -Input [5]: [item_sk#32, d_date#28, sumss#33, ss_item_sk#24, rk#34] - -(45) ReusedExchange [Reuses operator id: 40] -Output [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] - -(46) CometSort -Input [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] -Arguments: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36], [ss_item_sk#36 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST] - -(47) CometColumnarToRow [codegen id : 19] -Input [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] - -(48) Window -Input [4]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36] -Arguments: [row_number() windowspecdefinition(ss_item_sk#36, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#37], [ss_item_sk#36], [d_date#35 ASC NULLS FIRST] - -(49) Project [codegen id : 20] -Output [3]: [item_sk#32 AS item_sk#38, sumss#33 AS sumss#39, rk#37] -Input [5]: [item_sk#32, d_date#35, sumss#33, ss_item_sk#36, rk#37] - -(50) BroadcastExchange -Input [3]: [item_sk#38, sumss#39, rk#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -(51) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [item_sk#32] -Right keys [1]: [item_sk#38] -Join type: Inner -Join condition: (rk#34 >= rk#37) - -(52) Project [codegen id : 21] -Output [4]: [item_sk#32, d_date#28, sumss#33, sumss#39] -Input [7]: [item_sk#32, d_date#28, sumss#33, rk#34, item_sk#38, sumss#39, rk#37] - -(53) HashAggregate [codegen id : 21] -Input [4]: [item_sk#32, d_date#28, sumss#33, sumss#39] -Keys [3]: [item_sk#32, d_date#28, sumss#33] -Functions [1]: [partial_sum(sumss#39)] -Aggregate Attributes [2]: [sum#40, isEmpty#41] -Results [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] - -(54) CometColumnarExchange -Input [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] -Arguments: hashpartitioning(item_sk#32, d_date#28, sumss#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(55) CometColumnarToRow [codegen id : 22] -Input [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] - -(56) HashAggregate [codegen id : 22] -Input [5]: [item_sk#32, d_date#28, sumss#33, sum#42, isEmpty#43] -Keys [3]: [item_sk#32, d_date#28, sumss#33] -Functions [1]: [sum(sumss#39)] -Aggregate Attributes [1]: [sum(sumss#39)#44] -Results [3]: [item_sk#32, d_date#28, sum(sumss#39)#44 AS cume_sales#45] - -(57) CometColumnarExchange -Input [3]: [item_sk#32, d_date#28, cume_sales#45] -Arguments: hashpartitioning(item_sk#32, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(58) CometSort -Input [3]: [item_sk#32, d_date#28, cume_sales#45] -Arguments: [item_sk#32, d_date#28, cume_sales#45], [item_sk#32 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] - -(59) CometSortMergeJoin -Left output [3]: [item_sk#10, d_date#6, cume_sales#23] -Right output [3]: [item_sk#32, d_date#28, cume_sales#45] -Arguments: [item_sk#10, d_date#6], [item_sk#32, d_date#28], FullOuter - -(60) CometFilter -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#32, d_date#28, cume_sales#45] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#32 END) - -(61) CometProject -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#32, d_date#28, cume_sales#45] -Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#32 END AS item_sk#46, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#28 END AS d_date#47, cume_sales#23 AS web_sales#48, cume_sales#45 AS store_sales#49] - -(62) CometExchange -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: hashpartitioning(item_sk#46, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(63) CometSort -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST] - -(64) CometColumnarToRow [codegen id : 23] -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] - -(65) Window -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: [row_number() windowspecdefinition(item_sk#46, d_date#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#50], [item_sk#46], [d_date#47 ASC NULLS FIRST] - -(66) ReusedExchange [Reuses operator id: 62] -Output [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] - -(67) CometSort -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST] - -(68) CometColumnarToRow [codegen id : 46] -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] - -(69) Window -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: [row_number() windowspecdefinition(item_sk#46, d_date#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#51], [item_sk#46], [d_date#47 ASC NULLS FIRST] - -(70) Project [codegen id : 47] -Output [4]: [item_sk#46 AS item_sk#52, web_sales#48 AS web_sales#53, store_sales#49 AS store_sales#54, rk#51] -Input [5]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#51] - -(71) BroadcastExchange -Input [4]: [item_sk#52, web_sales#53, store_sales#54, rk#51] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -(72) BroadcastHashJoin [codegen id : 48] -Left keys [1]: [item_sk#46] -Right keys [1]: [item_sk#52] -Join type: Inner -Join condition: (rk#50 >= rk#51) - -(73) Project [codegen id : 48] -Output [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] -Input [9]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#50, item_sk#52, web_sales#53, store_sales#54, rk#51] - -(74) HashAggregate [codegen id : 48] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] -Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Functions [2]: [partial_max(web_sales#53), partial_max(store_sales#54)] -Aggregate Attributes [2]: [max#55, max#56] -Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#57, max#58] - -(75) HashAggregate [codegen id : 48] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#57, max#58] -Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Functions [2]: [max(web_sales#53), max(store_sales#54)] -Aggregate Attributes [2]: [max(web_sales#53)#59, max(store_sales#54)#60] -Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max(web_sales#53)#59 AS web_cumulative#61, max(store_sales#54)#60 AS store_cumulative#62] - -(76) Filter [codegen id : 48] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#61, store_cumulative#62] -Condition : ((isnotnull(web_cumulative#61) AND isnotnull(store_cumulative#62)) AND (web_cumulative#61 > store_cumulative#62)) - -(77) TakeOrderedAndProject -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#61, store_cumulative#62] -Arguments: 100, [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#61, store_cumulative#62] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (82) -+- * CometColumnarToRow (81) - +- CometProject (80) - +- CometFilter (79) - +- CometNativeScan parquet spark_catalog.default.date_dim (78) - - -(78) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#63] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(79) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#63] -Condition : (((isnotnull(d_month_seq#63) AND (d_month_seq#63 >= 1212)) AND (d_month_seq#63 <= 1223)) AND isnotnull(d_date_sk#5)) - -(80) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#63] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(81) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] - -(82) BroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/simplified.txt deleted file mode 100644 index 00cd0a509c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/simplified.txt +++ /dev/null @@ -1,127 +0,0 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (48) - Filter [web_cumulative,store_cumulative] - HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] - HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] - Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (23) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (11) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (10) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometColumnarExchange [ws_item_sk] #4 - WholeStageCodegen (3) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ws_item_sk,d_date] #5 - WholeStageCodegen (2) - HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] - Project [ws_item_sk,ws_sales_price,d_date] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (9) - Project [item_sk,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #8 - WholeStageCodegen (22) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumss] #9 - WholeStageCodegen (21) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (15) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometColumnarExchange [ss_item_sk] #10 - WholeStageCodegen (14) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_item_sk,d_date] #11 - WholeStageCodegen (13) - HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] - Project [ss_item_sk,ss_sales_price,d_date] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_date] #6 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (20) - Project [item_sk,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (47) - Project [item_sk,web_sales,store_sales,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (46) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/explain.txt deleted file mode 100644 index f8a95a6db8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,448 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (76) -+- * Filter (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- Window (64) - : +- * CometColumnarToRow (63) - : +- CometSort (62) - : +- CometExchange (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometSortMergeJoin (58) - : :- CometSort (30) - : : +- CometColumnarExchange (29) - : : +- * HashAggregate (28) - : : +- * CometColumnarToRow (27) - : : +- CometColumnarExchange (26) - : : +- * HashAggregate (25) - : : +- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (16) - : : : +- Window (15) - : : : +- * CometColumnarToRow (14) - : : : +- CometSort (13) - : : : +- CometExchange (12) - : : : +- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- BroadcastExchange (22) - : : +- * Project (21) - : : +- Window (20) - : : +- * CometColumnarToRow (19) - : : +- CometSort (18) - : : +- ReusedExchange (17) - : +- CometSort (57) - : +- CometColumnarExchange (56) - : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) - : +- CometColumnarExchange (53) - : +- * HashAggregate (52) - : +- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Project (43) - : : +- Window (42) - : : +- * CometColumnarToRow (41) - : : +- CometSort (40) - : : +- CometExchange (39) - : : +- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometFilter (32) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (31) - : : +- ReusedExchange (33) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- Window (47) - : +- * CometColumnarToRow (46) - : +- CometSort (45) - : +- ReusedExchange (44) - +- BroadcastExchange (70) - +- * Project (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- ReusedExchange (65) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: [d_date_sk#5, d_date#6] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Right output [2]: [d_date_sk#5, d_date#6] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] -Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] - -(9) CometHashAggregate -Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] - -(10) CometExchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] - -(12) CometExchange -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(13) CometSort -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(14) CometColumnarToRow [codegen id : 1] -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] - -(15) Window -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(16) Project [codegen id : 4] -Output [4]: [item_sk#9, d_date#6, sumws#10, rk#11] -Input [5]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1, rk#11] - -(17) ReusedExchange [Reuses operator id: 12] -Output [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] - -(18) CometSort -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -Arguments: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13], [ws_item_sk#13 ASC NULLS FIRST, d_date#12 ASC NULLS FIRST] - -(19) CometColumnarToRow [codegen id : 2] -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] - -(20) Window -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -Arguments: [row_number() windowspecdefinition(ws_item_sk#13, d_date#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#14], [ws_item_sk#13], [d_date#12 ASC NULLS FIRST] - -(21) Project [codegen id : 3] -Output [3]: [item_sk#9 AS item_sk#15, sumws#10 AS sumws#16, rk#14] -Input [5]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13, rk#14] - -(22) BroadcastExchange -Input [3]: [item_sk#15, sumws#16, rk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [item_sk#9] -Right keys [1]: [item_sk#15] -Join type: Inner -Join condition: (rk#11 >= rk#14) - -(24) Project [codegen id : 4] -Output [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] -Input [7]: [item_sk#9, d_date#6, sumws#10, rk#11, item_sk#15, sumws#16, rk#14] - -(25) HashAggregate [codegen id : 4] -Input [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] -Keys [3]: [item_sk#9, d_date#6, sumws#10] -Functions [1]: [partial_sum(sumws#16)] -Aggregate Attributes [2]: [sum#17, isEmpty#18] -Results [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] - -(26) CometColumnarExchange -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] -Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometColumnarToRow [codegen id : 5] -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] - -(28) HashAggregate [codegen id : 5] -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] -Keys [3]: [item_sk#9, d_date#6, sumws#10] -Functions [1]: [sum(sumws#16)] -Aggregate Attributes [1]: [sum(sumws#16)#21] -Results [3]: [item_sk#9, d_date#6, sum(sumws#16)#21 AS cume_sales#22] - -(29) CometColumnarExchange -Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(30) CometSort -Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: [item_sk#9, d_date#6, cume_sales#22], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_item_sk#23) - -(33) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#27, d_date#28] - -(34) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Right output [2]: [d_date_sk#27, d_date#28] -Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight - -(35) CometProject -Input [5]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_date#28] -Arguments: [ss_item_sk#23, ss_sales_price#24, d_date#28], [ss_item_sk#23, ss_sales_price#24, d_date#28] - -(36) CometHashAggregate -Input [3]: [ss_item_sk#23, ss_sales_price#24, d_date#28] -Keys [2]: [ss_item_sk#23, d_date#28] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#24))] - -(37) CometExchange -Input [3]: [ss_item_sk#23, d_date#28, sum#29] -Arguments: hashpartitioning(ss_item_sk#23, d_date#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(38) CometHashAggregate -Input [3]: [ss_item_sk#23, d_date#28, sum#29] -Keys [2]: [ss_item_sk#23, d_date#28] -Functions [1]: [sum(UnscaledValue(ss_sales_price#24))] - -(39) CometExchange -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: hashpartitioning(ss_item_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(40) CometSort -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23], [ss_item_sk#23 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] - -(41) CometColumnarToRow [codegen id : 6] -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] - -(42) Window -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: [row_number() windowspecdefinition(ss_item_sk#23, d_date#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [ss_item_sk#23], [d_date#28 ASC NULLS FIRST] - -(43) Project [codegen id : 9] -Output [4]: [item_sk#30, d_date#28, sumss#31, rk#32] -Input [5]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23, rk#32] - -(44) ReusedExchange [Reuses operator id: 39] -Output [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] - -(45) CometSort -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -Arguments: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34], [ss_item_sk#34 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST] - -(46) CometColumnarToRow [codegen id : 7] -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] - -(47) Window -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -Arguments: [row_number() windowspecdefinition(ss_item_sk#34, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#34], [d_date#33 ASC NULLS FIRST] - -(48) Project [codegen id : 8] -Output [3]: [item_sk#30 AS item_sk#36, sumss#31 AS sumss#37, rk#35] -Input [5]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34, rk#35] - -(49) BroadcastExchange -Input [3]: [item_sk#36, sumss#37, rk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -(50) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [item_sk#30] -Right keys [1]: [item_sk#36] -Join type: Inner -Join condition: (rk#32 >= rk#35) - -(51) Project [codegen id : 9] -Output [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] -Input [7]: [item_sk#30, d_date#28, sumss#31, rk#32, item_sk#36, sumss#37, rk#35] - -(52) HashAggregate [codegen id : 9] -Input [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] -Keys [3]: [item_sk#30, d_date#28, sumss#31] -Functions [1]: [partial_sum(sumss#37)] -Aggregate Attributes [2]: [sum#38, isEmpty#39] -Results [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] - -(53) CometColumnarExchange -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -Arguments: hashpartitioning(item_sk#30, d_date#28, sumss#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(54) CometColumnarToRow [codegen id : 10] -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] - -(55) HashAggregate [codegen id : 10] -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -Keys [3]: [item_sk#30, d_date#28, sumss#31] -Functions [1]: [sum(sumss#37)] -Aggregate Attributes [1]: [sum(sumss#37)#42] -Results [3]: [item_sk#30, d_date#28, sum(sumss#37)#42 AS cume_sales#43] - -(56) CometColumnarExchange -Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: hashpartitioning(item_sk#30, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(57) CometSort -Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#30, d_date#28, cume_sales#43], [item_sk#30 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [3]: [item_sk#9, d_date#6, cume_sales#22] -Right output [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#9, d_date#6], [item_sk#30, d_date#28], FullOuter - -(59) CometFilter -Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END) - -(60) CometProject -Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END AS item_sk#44, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#28 END AS d_date#45, cume_sales#22 AS web_sales#46, cume_sales#43 AS store_sales#47] - -(61) CometExchange -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(62) CometSort -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] - -(63) CometColumnarToRow [codegen id : 11] -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] - -(64) Window -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [item_sk#44], [d_date#45 ASC NULLS FIRST] - -(65) ReusedExchange [Reuses operator id: 61] -Output [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] - -(66) CometSort -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] - -(67) CometColumnarToRow [codegen id : 22] -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] - -(68) Window -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [item_sk#44], [d_date#45 ASC NULLS FIRST] - -(69) Project [codegen id : 23] -Output [4]: [item_sk#44 AS item_sk#50, web_sales#46 AS web_sales#51, store_sales#47 AS store_sales#52, rk#49] -Input [5]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#49] - -(70) BroadcastExchange -Input [4]: [item_sk#50, web_sales#51, store_sales#52, rk#49] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -(71) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [item_sk#44] -Right keys [1]: [item_sk#50] -Join type: Inner -Join condition: (rk#48 >= rk#49) - -(72) Project [codegen id : 24] -Output [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] -Input [9]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#48, item_sk#50, web_sales#51, store_sales#52, rk#49] - -(73) HashAggregate [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] -Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Functions [2]: [partial_max(web_sales#51), partial_max(store_sales#52)] -Aggregate Attributes [2]: [max#53, max#54] -Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] - -(74) HashAggregate [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] -Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Functions [2]: [max(web_sales#51), max(store_sales#52)] -Aggregate Attributes [2]: [max(web_sales#51)#57, max(store_sales#52)#58] -Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max(web_sales#51)#57 AS web_cumulative#59, max(store_sales#52)#58 AS store_cumulative#60] - -(75) Filter [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] -Condition : ((isnotnull(web_cumulative#59) AND isnotnull(store_cumulative#60)) AND (web_cumulative#59 > store_cumulative#60)) - -(76) TakeOrderedAndProject -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] -Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (81) -+- * CometColumnarToRow (80) - +- CometProject (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) - -(79) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(80) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] - -(81) BroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:2 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/simplified.txt deleted file mode 100644 index b3013059b0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,116 +0,0 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (24) - Filter [web_cumulative,store_cumulative] - HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] - HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] - Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (5) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (4) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometExchange [ws_item_sk] #4 - CometHashAggregate [sum] [item_sk,d_date,sumws,ws_item_sk,sum(UnscaledValue(ws_sales_price))] - CometExchange [ws_item_sk,d_date] #5 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [item_sk,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #9 - WholeStageCodegen (10) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumss] #10 - WholeStageCodegen (9) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sum] [item_sk,d_date,sumss,ss_item_sk,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_item_sk,d_date] #12 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - 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 [native_iceberg_compat] 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 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - Project [item_sk,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (23) - Project [item_sk,web_sales,store_sales,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (22) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt deleted file mode 100644 index f8a95a6db8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt +++ /dev/null @@ -1,448 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (76) -+- * Filter (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- Window (64) - : +- * CometColumnarToRow (63) - : +- CometSort (62) - : +- CometExchange (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometSortMergeJoin (58) - : :- CometSort (30) - : : +- CometColumnarExchange (29) - : : +- * HashAggregate (28) - : : +- * CometColumnarToRow (27) - : : +- CometColumnarExchange (26) - : : +- * HashAggregate (25) - : : +- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (16) - : : : +- Window (15) - : : : +- * CometColumnarToRow (14) - : : : +- CometSort (13) - : : : +- CometExchange (12) - : : : +- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- BroadcastExchange (22) - : : +- * Project (21) - : : +- Window (20) - : : +- * CometColumnarToRow (19) - : : +- CometSort (18) - : : +- ReusedExchange (17) - : +- CometSort (57) - : +- CometColumnarExchange (56) - : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) - : +- CometColumnarExchange (53) - : +- * HashAggregate (52) - : +- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Project (43) - : : +- Window (42) - : : +- * CometColumnarToRow (41) - : : +- CometSort (40) - : : +- CometExchange (39) - : : +- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometFilter (32) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (31) - : : +- ReusedExchange (33) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- Window (47) - : +- * CometColumnarToRow (46) - : +- CometSort (45) - : +- ReusedExchange (44) - +- BroadcastExchange (70) - +- * Project (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- ReusedExchange (65) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(6) CometBroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: [d_date_sk#5, d_date#6] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] -Right output [2]: [d_date_sk#5, d_date#6] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [5]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3, d_date_sk#5, d_date#6] -Arguments: [ws_item_sk#1, ws_sales_price#2, d_date#6], [ws_item_sk#1, ws_sales_price#2, d_date#6] - -(9) CometHashAggregate -Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] - -(10) CometExchange -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(11) CometHashAggregate -Input [3]: [ws_item_sk#1, d_date#6, sum#8] -Keys [2]: [ws_item_sk#1, d_date#6] -Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] - -(12) CometExchange -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(13) CometSort -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(14) CometColumnarToRow [codegen id : 1] -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] - -(15) Window -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(16) Project [codegen id : 4] -Output [4]: [item_sk#9, d_date#6, sumws#10, rk#11] -Input [5]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1, rk#11] - -(17) ReusedExchange [Reuses operator id: 12] -Output [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] - -(18) CometSort -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -Arguments: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13], [ws_item_sk#13 ASC NULLS FIRST, d_date#12 ASC NULLS FIRST] - -(19) CometColumnarToRow [codegen id : 2] -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] - -(20) Window -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -Arguments: [row_number() windowspecdefinition(ws_item_sk#13, d_date#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#14], [ws_item_sk#13], [d_date#12 ASC NULLS FIRST] - -(21) Project [codegen id : 3] -Output [3]: [item_sk#9 AS item_sk#15, sumws#10 AS sumws#16, rk#14] -Input [5]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13, rk#14] - -(22) BroadcastExchange -Input [3]: [item_sk#15, sumws#16, rk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [item_sk#9] -Right keys [1]: [item_sk#15] -Join type: Inner -Join condition: (rk#11 >= rk#14) - -(24) Project [codegen id : 4] -Output [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] -Input [7]: [item_sk#9, d_date#6, sumws#10, rk#11, item_sk#15, sumws#16, rk#14] - -(25) HashAggregate [codegen id : 4] -Input [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] -Keys [3]: [item_sk#9, d_date#6, sumws#10] -Functions [1]: [partial_sum(sumws#16)] -Aggregate Attributes [2]: [sum#17, isEmpty#18] -Results [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] - -(26) CometColumnarExchange -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] -Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometColumnarToRow [codegen id : 5] -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] - -(28) HashAggregate [codegen id : 5] -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] -Keys [3]: [item_sk#9, d_date#6, sumws#10] -Functions [1]: [sum(sumws#16)] -Aggregate Attributes [1]: [sum(sumws#16)#21] -Results [3]: [item_sk#9, d_date#6, sum(sumws#16)#21 AS cume_sales#22] - -(29) CometColumnarExchange -Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(30) CometSort -Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: [item_sk#9, d_date#6, cume_sales#22], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(32) CometFilter -Input [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_item_sk#23) - -(33) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#27, d_date#28] - -(34) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Right output [2]: [d_date_sk#27, d_date#28] -Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight - -(35) CometProject -Input [5]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_date#28] -Arguments: [ss_item_sk#23, ss_sales_price#24, d_date#28], [ss_item_sk#23, ss_sales_price#24, d_date#28] - -(36) CometHashAggregate -Input [3]: [ss_item_sk#23, ss_sales_price#24, d_date#28] -Keys [2]: [ss_item_sk#23, d_date#28] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#24))] - -(37) CometExchange -Input [3]: [ss_item_sk#23, d_date#28, sum#29] -Arguments: hashpartitioning(ss_item_sk#23, d_date#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(38) CometHashAggregate -Input [3]: [ss_item_sk#23, d_date#28, sum#29] -Keys [2]: [ss_item_sk#23, d_date#28] -Functions [1]: [sum(UnscaledValue(ss_sales_price#24))] - -(39) CometExchange -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: hashpartitioning(ss_item_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(40) CometSort -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23], [ss_item_sk#23 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] - -(41) CometColumnarToRow [codegen id : 6] -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] - -(42) Window -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: [row_number() windowspecdefinition(ss_item_sk#23, d_date#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [ss_item_sk#23], [d_date#28 ASC NULLS FIRST] - -(43) Project [codegen id : 9] -Output [4]: [item_sk#30, d_date#28, sumss#31, rk#32] -Input [5]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23, rk#32] - -(44) ReusedExchange [Reuses operator id: 39] -Output [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] - -(45) CometSort -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -Arguments: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34], [ss_item_sk#34 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST] - -(46) CometColumnarToRow [codegen id : 7] -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] - -(47) Window -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -Arguments: [row_number() windowspecdefinition(ss_item_sk#34, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#34], [d_date#33 ASC NULLS FIRST] - -(48) Project [codegen id : 8] -Output [3]: [item_sk#30 AS item_sk#36, sumss#31 AS sumss#37, rk#35] -Input [5]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34, rk#35] - -(49) BroadcastExchange -Input [3]: [item_sk#36, sumss#37, rk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -(50) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [item_sk#30] -Right keys [1]: [item_sk#36] -Join type: Inner -Join condition: (rk#32 >= rk#35) - -(51) Project [codegen id : 9] -Output [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] -Input [7]: [item_sk#30, d_date#28, sumss#31, rk#32, item_sk#36, sumss#37, rk#35] - -(52) HashAggregate [codegen id : 9] -Input [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] -Keys [3]: [item_sk#30, d_date#28, sumss#31] -Functions [1]: [partial_sum(sumss#37)] -Aggregate Attributes [2]: [sum#38, isEmpty#39] -Results [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] - -(53) CometColumnarExchange -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -Arguments: hashpartitioning(item_sk#30, d_date#28, sumss#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(54) CometColumnarToRow [codegen id : 10] -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] - -(55) HashAggregate [codegen id : 10] -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -Keys [3]: [item_sk#30, d_date#28, sumss#31] -Functions [1]: [sum(sumss#37)] -Aggregate Attributes [1]: [sum(sumss#37)#42] -Results [3]: [item_sk#30, d_date#28, sum(sumss#37)#42 AS cume_sales#43] - -(56) CometColumnarExchange -Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: hashpartitioning(item_sk#30, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(57) CometSort -Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#30, d_date#28, cume_sales#43], [item_sk#30 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [3]: [item_sk#9, d_date#6, cume_sales#22] -Right output [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#9, d_date#6], [item_sk#30, d_date#28], FullOuter - -(59) CometFilter -Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END) - -(60) CometProject -Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END AS item_sk#44, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#28 END AS d_date#45, cume_sales#22 AS web_sales#46, cume_sales#43 AS store_sales#47] - -(61) CometExchange -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(62) CometSort -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] - -(63) CometColumnarToRow [codegen id : 11] -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] - -(64) Window -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [item_sk#44], [d_date#45 ASC NULLS FIRST] - -(65) ReusedExchange [Reuses operator id: 61] -Output [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] - -(66) CometSort -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] - -(67) CometColumnarToRow [codegen id : 22] -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] - -(68) Window -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [item_sk#44], [d_date#45 ASC NULLS FIRST] - -(69) Project [codegen id : 23] -Output [4]: [item_sk#44 AS item_sk#50, web_sales#46 AS web_sales#51, store_sales#47 AS store_sales#52, rk#49] -Input [5]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#49] - -(70) BroadcastExchange -Input [4]: [item_sk#50, web_sales#51, store_sales#52, rk#49] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -(71) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [item_sk#44] -Right keys [1]: [item_sk#50] -Join type: Inner -Join condition: (rk#48 >= rk#49) - -(72) Project [codegen id : 24] -Output [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] -Input [9]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#48, item_sk#50, web_sales#51, store_sales#52, rk#49] - -(73) HashAggregate [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] -Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Functions [2]: [partial_max(web_sales#51), partial_max(store_sales#52)] -Aggregate Attributes [2]: [max#53, max#54] -Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] - -(74) HashAggregate [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] -Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Functions [2]: [max(web_sales#51), max(store_sales#52)] -Aggregate Attributes [2]: [max(web_sales#51)#57, max(store_sales#52)#58] -Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max(web_sales#51)#57 AS web_cumulative#59, max(store_sales#52)#58 AS store_cumulative#60] - -(75) Filter [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] -Condition : ((isnotnull(web_cumulative#59) AND isnotnull(store_cumulative#60)) AND (web_cumulative#59 > store_cumulative#60)) - -(76) TakeOrderedAndProject -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] -Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (81) -+- * CometColumnarToRow (80) - +- CometProject (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) - -(79) CometProject -Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] -Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] - -(80) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_date#6] - -(81) BroadcastExchange -Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - -Subquery:2 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/extended.txt deleted file mode 100644 index f7d3371108..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/extended.txt +++ /dev/null @@ -1,216 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : : +- CometColumnarToRow - : : : +- CometSort - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometSortMergeJoin - :- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 138 out of 196 eligible operators (70%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index b3013059b0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt +++ /dev/null @@ -1,116 +0,0 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (24) - Filter [web_cumulative,store_cumulative] - HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] - HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] - Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (5) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (4) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometExchange [ws_item_sk] #4 - CometHashAggregate [sum] [item_sk,d_date,sumws,ws_item_sk,sum(UnscaledValue(ws_sales_price))] - CometExchange [ws_item_sk,d_date] #5 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [item_sk,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #9 - WholeStageCodegen (10) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumss] #10 - WholeStageCodegen (9) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sum] [item_sk,d_date,sumss,ss_item_sk,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_item_sk,d_date] #12 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - 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 [native_iceberg_compat] 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 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - Project [item_sk,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (23) - Project [item_sk,web_sales,store_sales,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (22) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/explain.txt deleted file mode 100644 index be90722d2e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/explain.txt +++ /dev/null @@ -1,304 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (51) -+- * Project (50) - +- * BroadcastHashJoin Inner BuildRight (49) - :- * Project (42) - : +- * BroadcastHashJoin Inner BuildRight (41) - : :- * Project (31) - : : +- * Filter (30) - : : +- Window (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * CometColumnarToRow (26) - : : +- CometSort (25) - : : +- CometColumnarExchange (24) - : : +- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- * HashAggregate (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.item (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (40) - : +- * Project (39) - : +- Window (38) - : +- * CometColumnarToRow (37) - : +- CometSort (36) - : +- CometColumnarExchange (35) - : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) - : +- ReusedExchange (32) - +- BroadcastExchange (48) - +- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- ReusedExchange (43) - - -(1) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometColumnarToRow [codegen id : 4] -Input [3]: [i_item_sk#1, i_brand#4, i_category#5] - -(5) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(7) Filter [codegen id : 1] -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) - -(8) BroadcastExchange -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#7] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 4] -Output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(11) ReusedExchange [Reuses operator id: 55] -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] - -(14) CometNativeScan parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#14, cc_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#14, cc_name#15] -Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) - -(16) CometColumnarToRow [codegen id : 3] -Input [2]: [cc_call_center_sk#14, cc_name#15] - -(17) BroadcastExchange -Input [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_call_center_sk#6] -Right keys [1]: [cc_call_center_sk#14] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] - -(20) HashAggregate [codegen id : 4] -Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] -Aggregate Attributes [1]: [sum#16] -Results [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] - -(21) CometColumnarExchange -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(22) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] - -(23) HashAggregate [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#17] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#8))#18] -Results [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#18,17,2) AS sum_sales#19, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#18,17,2) AS _w0#20] - -(24) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(25) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) CometColumnarToRow [codegen id : 6] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] - -(27) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(28) Filter [codegen id : 7] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(29) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] -Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(30) Filter [codegen id : 22] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] -Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) - -(31) Project [codegen id : 22] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] - -(32) ReusedExchange [Reuses operator id: 21] -Output [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] - -(33) CometColumnarToRow [codegen id : 12] -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] - -(34) HashAggregate [codegen id : 12] -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum#26] -Keys [5]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25] -Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#27))#18] -Results [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#18,17,2) AS sum_sales#19] - -(35) CometColumnarExchange -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#23, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(36) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] -Arguments: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(37) CometColumnarToRow [codegen id : 13] -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] - -(38) Window -Input [6]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] - -(39) Project [codegen id : 14] -Output [5]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, cc_name#23, sum_sales#19 AS sum_sales#31, rn#28] -Input [7]: [i_category#5, i_brand#4, cc_name#23, d_year#24, d_moy#25, sum_sales#19, rn#28] - -(40) BroadcastExchange -Input [5]: [i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] - -(41) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] -Right keys [4]: [i_category#29, i_brand#30, cc_name#23, (rn#28 + 1)] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 22] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#29, i_brand#30, cc_name#23, sum_sales#31, rn#28] - -(43) ReusedExchange [Reuses operator id: 35] -Output [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] - -(44) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] -Arguments: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] - -(45) CometColumnarToRow [codegen id : 20] -Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] - -(46) Window -Input [6]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19] -Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#5, i_brand#4, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#35], [i_category#5, i_brand#4, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] - -(47) Project [codegen id : 21] -Output [5]: [i_category#5 AS i_category#36, i_brand#4 AS i_brand#37, cc_name#32, sum_sales#19 AS sum_sales#38, rn#35] -Input [7]: [i_category#5, i_brand#4, cc_name#32, d_year#33, d_moy#34, sum_sales#19, rn#35] - -(48) BroadcastExchange -Input [5]: [i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] - -(49) BroadcastHashJoin [codegen id : 22] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#21] -Right keys [4]: [i_category#36, i_brand#37, cc_name#32, (rn#35 - 1)] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 22] -Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#39, sum_sales#38 AS nsum#40] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#36, i_brand#37, cc_name#32, sum_sales#38, rn#35] - -(51) TakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] -Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (55) -+- * CometColumnarToRow (54) - +- CometFilter (53) - +- CometNativeScan parquet spark_catalog.default.date_dim (52) - - -(52) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(53) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(54) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(55) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/simplified.txt deleted file mode 100644 index e603f05322..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/simplified.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (22) - Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (7) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (5) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] - Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - BroadcastHashJoin [i_item_sk,cs_item_sk] - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [cs_item_sk,cs_call_center_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [cc_call_center_sk,cc_name] - CometNativeScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (14) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (13) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #7 - WholeStageCodegen (12) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (21) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (20) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/explain.txt deleted file mode 100644 index d1daee98c1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) - -(6) CometBroadcastExchange -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_item_sk#1], [cs_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#14, cc_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#14, cc_name#15] -Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) - -(16) CometBroadcastExchange -Input [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cc_call_center_sk#14, cc_name#15] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] -Right output [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cs_call_center_sk#6], [cc_call_center_sk#14], Inner, BuildRight - -(18) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] -Arguments: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15], [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] - -(19) CometHashAggregate -Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] - -(20) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] - -(22) CometExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] - -(25) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(28) Filter [codegen id : 7] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] - -(30) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] - -(31) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] -Keys [5]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23] -Functions [1]: [sum(UnscaledValue(cs_sales_price#25))] - -(32) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] - -(35) Window -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [5]: [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] -Input [7]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26] - -(37) BroadcastExchange -Input [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] - -(40) ReusedExchange [Reuses operator id: 32] -Output [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(41) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(43) Window -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#5, i_brand#4, cc_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#5, i_brand#4, cc_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [5]: [i_category#5 AS i_category#34, i_brand#4 AS i_brand#35, cc_name#30, sum_sales#17 AS sum_sales#36, rn#33] -Input [7]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17, rn#33] - -(45) BroadcastExchange -Input [5]: [i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#34, i_brand#35, cc_name#30, (rn#33 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] - -(48) TakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/simplified.txt deleted file mode 100644 index d655789fe6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - 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 [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,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt deleted file mode 100644 index d1daee98c1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt +++ /dev/null @@ -1,289 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#1, i_brand#2, i_category#3] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Condition : ((isnotnull(i_item_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true))) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true))) - -(3) CometProject -Input [3]: [i_item_sk#1, i_brand#2, i_category#3] -Arguments: [i_item_sk#1, i_brand#4, i_category#5], [i_item_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#2, 50, true, false, true) AS i_brand#4, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#3, 50, true, false, true) AS i_category#5] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Condition : (isnotnull(cs_item_sk#7) AND isnotnull(cs_call_center_sk#6)) - -(6) CometBroadcastExchange -Input [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] - -(7) CometBroadcastHashJoin -Left output [3]: [i_item_sk#1, i_brand#4, i_category#5] -Right output [4]: [cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_item_sk#1], [cs_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [7]: [i_item_sk#1, i_brand#4, i_category#5, cs_call_center_sk#6, cs_item_sk#7, cs_sales_price#8, cs_sold_date_sk#9] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(11) CometBroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [d_date_sk#11, d_year#12, d_moy#13] - -(12) CometBroadcastHashJoin -Left output [5]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9] -Right output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: [cs_sold_date_sk#9], [d_date_sk#11], Inner, BuildRight - -(13) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, cs_sold_date_sk#9, d_date_sk#11, d_year#12, d_moy#13] -Arguments: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13], [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Output [2]: [cc_call_center_sk#14, cc_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/call_center] -PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [cc_call_center_sk#14, cc_name#15] -Condition : (isnotnull(cc_call_center_sk#14) AND isnotnull(cc_name#15)) - -(16) CometBroadcastExchange -Input [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cc_call_center_sk#14, cc_name#15] - -(17) CometBroadcastHashJoin -Left output [6]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13] -Right output [2]: [cc_call_center_sk#14, cc_name#15] -Arguments: [cs_call_center_sk#6], [cc_call_center_sk#14], Inner, BuildRight - -(18) CometProject -Input [8]: [i_brand#4, i_category#5, cs_call_center_sk#6, cs_sales_price#8, d_year#12, d_moy#13, cc_call_center_sk#14, cc_name#15] -Arguments: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15], [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] - -(19) CometHashAggregate -Input [6]: [i_brand#4, i_category#5, cs_sales_price#8, d_year#12, d_moy#13, cc_name#15] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] - -(20) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(21) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] -Keys [5]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13] -Functions [1]: [sum(UnscaledValue(cs_sales_price#8))] - -(22) CometExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(23) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(24) CometColumnarToRow [codegen id : 1] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] - -(25) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) - -(27) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(28) Filter [codegen id : 7] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) - -(29) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] - -(30) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] - -(31) CometHashAggregate -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] -Keys [5]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23] -Functions [1]: [sum(UnscaledValue(cs_sales_price#25))] - -(32) CometExchange -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] - -(34) CometColumnarToRow [codegen id : 3] -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] - -(35) Window -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] - -(36) Project [codegen id : 4] -Output [5]: [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] -Input [7]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26] - -(37) BroadcastExchange -Input [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)] -Join type: Inner -Join condition: None - -(39) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] - -(40) ReusedExchange [Reuses operator id: 32] -Output [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(41) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(43) Window -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#5, i_brand#4, cc_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#5, i_brand#4, cc_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [5]: [i_category#5 AS i_category#34, i_brand#4 AS i_brand#35, cc_name#30, sum_sales#17 AS sum_sales#36, rn#33] -Input [7]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17, rn#33] - -(45) BroadcastExchange -Input [5]: [i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] - -(46) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#34, i_brand#35, cc_name#30, (rn#33 - 1)] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] - -(48) TakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - - -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#11, d_year#12, d_moy#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) - -(51) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] - -(52) BroadcastExchange -Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/extended.txt deleted file mode 100644 index de7d26cd06..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d655789fe6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - 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 [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,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - 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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/explain.txt deleted file mode 100644 index 6b57e4b9c6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/explain.txt +++ /dev/null @@ -1,602 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (99) -+- CometTakeOrderedAndProject (98) - +- CometHashAggregate (97) - +- CometColumnarExchange (96) - +- * HashAggregate (95) - +- Union (94) - :- * HashAggregate (79) - : +- * CometColumnarToRow (78) - : +- CometColumnarExchange (77) - : +- * HashAggregate (76) - : +- Union (75) - : :- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometColumnarExchange (21) - : : +- * HashAggregate (20) - : : +- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- Union (9) - : : : : :- * Project (4) - : : : : : +- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- * Project (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_returns (5) - : : : +- ReusedExchange (10) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometProject (15) - : : +- CometFilter (14) - : : +- CometNativeScan parquet spark_catalog.default.store (13) - : :- * HashAggregate (46) - : : +- * CometColumnarToRow (45) - : : +- CometColumnarExchange (44) - : : +- * HashAggregate (43) - : : +- * Project (42) - : : +- * BroadcastHashJoin Inner BuildRight (41) - : : :- * Project (35) - : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : :- Union (32) - : : : : :- * Project (27) - : : : : : +- * Filter (26) - : : : : : +- * ColumnarToRow (25) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (24) - : : : : +- * Project (31) - : : : : +- * Filter (30) - : : : : +- * ColumnarToRow (29) - : : : : +- Scan parquet spark_catalog.default.catalog_returns (28) - : : : +- ReusedExchange (33) - : : +- BroadcastExchange (40) - : : +- * CometColumnarToRow (39) - : : +- CometProject (38) - : : +- CometFilter (37) - : : +- CometNativeScan parquet spark_catalog.default.catalog_page (36) - : +- * HashAggregate (74) - : +- * CometColumnarToRow (73) - : +- CometColumnarExchange (72) - : +- * HashAggregate (71) - : +- * Project (70) - : +- * BroadcastHashJoin Inner BuildRight (69) - : :- * Project (63) - : : +- * BroadcastHashJoin Inner BuildRight (62) - : : :- Union (60) - : : : :- * Project (50) - : : : : +- * Filter (49) - : : : : +- * ColumnarToRow (48) - : : : : +- Scan parquet spark_catalog.default.web_sales (47) - : : : +- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildLeft (58) - : : : :- BroadcastExchange (53) - : : : : +- * ColumnarToRow (52) - : : : : +- Scan parquet spark_catalog.default.web_returns (51) - : : : +- * CometColumnarToRow (57) - : : : +- CometProject (56) - : : : +- CometFilter (55) - : : : +- CometNativeScan parquet spark_catalog.default.web_sales (54) - : : +- ReusedExchange (61) - : +- BroadcastExchange (68) - : +- * CometColumnarToRow (67) - : +- CometProject (66) - : +- CometFilter (65) - : +- CometNativeScan parquet spark_catalog.default.web_site (64) - :- * HashAggregate (86) - : +- * CometColumnarToRow (85) - : +- CometColumnarExchange (84) - : +- * HashAggregate (83) - : +- * HashAggregate (82) - : +- * CometColumnarToRow (81) - : +- ReusedExchange (80) - +- * HashAggregate (93) - +- * CometColumnarToRow (92) - +- CometColumnarExchange (91) - +- * HashAggregate (90) - +- * HashAggregate (89) - +- * CometColumnarToRow (88) - +- ReusedExchange (87) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 1] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(4) Project [codegen id : 1] -Output [6]: [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] - -(5) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 2] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] - -(7) Filter [codegen id : 2] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#12) - -(8) Project [codegen id : 2] -Output [6]: [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] - -(9) Union - -(10) ReusedExchange [Reuses operator id: 104] -Output [1]: [d_date_sk#22] - -(11) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [date_sk#7] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None - -(12) Project [codegen id : 5] -Output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] - -(13) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#23, s_store_id#24] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [s_store_sk#23, s_store_id#24] -Condition : isnotnull(s_store_sk#23) - -(15) CometProject -Input [2]: [s_store_sk#23, s_store_id#24] -Arguments: [s_store_sk#23, s_store_id#25], [s_store_sk#23, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#24, 16, true, false, true) AS s_store_id#25] - -(16) CometColumnarToRow [codegen id : 4] -Input [2]: [s_store_sk#23, s_store_id#25] - -(17) BroadcastExchange -Input [2]: [s_store_sk#23, s_store_id#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(18) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [store_sk#6] -Right keys [1]: [s_store_sk#23] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 5] -Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#25] - -(20) HashAggregate [codegen id : 5] -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#25] -Keys [1]: [s_store_id#25] -Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum#26, sum#27, sum#28, sum#29] -Results [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] - -(21) CometColumnarExchange -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(22) CometColumnarToRow [codegen id : 6] -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] - -(23) HashAggregate [codegen id : 6] -Input [5]: [s_store_id#25, sum#30, sum#31, sum#32, sum#33] -Keys [1]: [s_store_id#25] -Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#34, sum(UnscaledValue(return_amt#10))#35, sum(UnscaledValue(profit#9))#36, sum(UnscaledValue(net_loss#11))#37] -Results [5]: [store channel AS channel#38, concat(store, s_store_id#25) AS id#39, MakeDecimal(sum(UnscaledValue(sales_price#8))#34,17,2) AS sales#40, MakeDecimal(sum(UnscaledValue(return_amt#10))#35,17,2) AS returns#41, (MakeDecimal(sum(UnscaledValue(profit#9))#36,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#37,17,2)) AS profit#42] - -(24) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#46), dynamicpruningexpression(cs_sold_date_sk#46 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cs_catalog_page_sk)] -ReadSchema: struct - -(25) ColumnarToRow [codegen id : 7] -Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] - -(26) Filter [codegen id : 7] -Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] -Condition : isnotnull(cs_catalog_page_sk#43) - -(27) Project [codegen id : 7] -Output [6]: [cs_catalog_page_sk#43 AS page_sk#47, cs_sold_date_sk#46 AS date_sk#48, cs_ext_sales_price#44 AS sales_price#49, cs_net_profit#45 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] -Input [4]: [cs_catalog_page_sk#43, cs_ext_sales_price#44, cs_net_profit#45, cs_sold_date_sk#46] - -(28) Scan parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(cr_catalog_page_sk)] -ReadSchema: struct - -(29) ColumnarToRow [codegen id : 8] -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] - -(30) Filter [codegen id : 8] -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] -Condition : isnotnull(cr_catalog_page_sk#53) - -(31) Project [codegen id : 8] -Output [6]: [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] -Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] - -(32) Union - -(33) ReusedExchange [Reuses operator id: 104] -Output [1]: [d_date_sk#63] - -(34) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [date_sk#48] -Right keys [1]: [d_date_sk#63] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 11] -Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] -Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] - -(36) CometNativeScan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Condition : isnotnull(cp_catalog_page_sk#64) - -(38) CometProject -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Arguments: [cp_catalog_page_sk#64, cp_catalog_page_id#66], [cp_catalog_page_sk#64, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#65, 16, true, false, true) AS cp_catalog_page_id#66] - -(39) CometColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] - -(40) BroadcastExchange -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(41) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [page_sk#47] -Right keys [1]: [cp_catalog_page_sk#64] -Join type: Inner -Join condition: None - -(42) Project [codegen id : 11] -Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] -Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#66] - -(43) HashAggregate [codegen id : 11] -Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#66] -Keys [1]: [cp_catalog_page_id#66] -Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] -Aggregate Attributes [4]: [sum#67, sum#68, sum#69, sum#70] -Results [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] - -(44) CometColumnarExchange -Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] -Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(45) CometColumnarToRow [codegen id : 12] -Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] - -(46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#66, sum#71, sum#72, sum#73, sum#74] -Keys [1]: [cp_catalog_page_id#66] -Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#75, sum(UnscaledValue(return_amt#51))#76, sum(UnscaledValue(profit#50))#77, sum(UnscaledValue(net_loss#52))#78] -Results [5]: [catalog channel AS channel#79, concat(catalog_page, cp_catalog_page_id#66) AS id#80, MakeDecimal(sum(UnscaledValue(sales_price#49))#75,17,2) AS sales#81, MakeDecimal(sum(UnscaledValue(return_amt#51))#76,17,2) AS returns#82, (MakeDecimal(sum(UnscaledValue(profit#50))#77,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#52))#78,17,2)) AS profit#83] - -(47) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#87), dynamicpruningexpression(ws_sold_date_sk#87 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] - -(49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] -Condition : isnotnull(ws_web_site_sk#84) - -(50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#84 AS wsr_web_site_sk#88, ws_sold_date_sk#87 AS date_sk#89, ws_ext_sales_price#85 AS sales_price#90, ws_net_profit#86 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] -Input [4]: [ws_web_site_sk#84, ws_ext_sales_price#85, ws_net_profit#86, ws_sold_date_sk#87] - -(51) Scan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#5)] -ReadSchema: struct - -(52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] - -(53) BroadcastExchange -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [plan_id=5] - -(54) CometNativeScan parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(55) CometFilter -Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) - -(56) CometProject -Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] -Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] - -(57) CometColumnarToRow -Input [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] - -(58) BroadcastHashJoin [codegen id : 15] -Left keys [2]: [wr_item_sk#94, wr_order_number#95] -Right keys [2]: [ws_item_sk#99, ws_order_number#101] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 15] -Output [6]: [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] -Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] - -(60) Union - -(61) ReusedExchange [Reuses operator id: 104] -Output [1]: [d_date_sk#109] - -(62) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [date_sk#89] -Right keys [1]: [d_date_sk#109] -Join type: Inner -Join condition: None - -(63) Project [codegen id : 18] -Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] -Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] - -(64) CometNativeScan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#110, web_site_id#111] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(65) CometFilter -Input [2]: [web_site_sk#110, web_site_id#111] -Condition : isnotnull(web_site_sk#110) - -(66) CometProject -Input [2]: [web_site_sk#110, web_site_id#111] -Arguments: [web_site_sk#110, web_site_id#112], [web_site_sk#110, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#111, 16, true, false, true) AS web_site_id#112] - -(67) CometColumnarToRow [codegen id : 17] -Input [2]: [web_site_sk#110, web_site_id#112] - -(68) BroadcastExchange -Input [2]: [web_site_sk#110, web_site_id#112] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(69) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [wsr_web_site_sk#88] -Right keys [1]: [web_site_sk#110] -Join type: Inner -Join condition: None - -(70) Project [codegen id : 18] -Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] -Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#112] - -(71) HashAggregate [codegen id : 18] -Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#112] -Keys [1]: [web_site_id#112] -Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum#113, sum#114, sum#115, sum#116] -Results [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] - -(72) CometColumnarExchange -Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] -Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(73) CometColumnarToRow [codegen id : 19] -Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] - -(74) HashAggregate [codegen id : 19] -Input [5]: [web_site_id#112, sum#117, sum#118, sum#119, sum#120] -Keys [1]: [web_site_id#112] -Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#121, sum(UnscaledValue(return_amt#92))#122, sum(UnscaledValue(profit#91))#123, sum(UnscaledValue(net_loss#93))#124] -Results [5]: [web channel AS channel#125, concat(web_site, web_site_id#112) AS id#126, MakeDecimal(sum(UnscaledValue(sales_price#90))#121,17,2) AS sales#127, MakeDecimal(sum(UnscaledValue(return_amt#92))#122,17,2) AS returns#128, (MakeDecimal(sum(UnscaledValue(profit#91))#123,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#124,17,2)) AS profit#129] - -(75) Union - -(76) HashAggregate [codegen id : 20] -Input [5]: [channel#38, id#39, sales#40, returns#41, profit#42] -Keys [2]: [channel#38, id#39] -Functions [3]: [partial_sum(sales#40), partial_sum(returns#41), partial_sum(profit#42)] -Aggregate Attributes [6]: [sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] -Results [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] - -(77) CometColumnarExchange -Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -Arguments: hashpartitioning(channel#38, id#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(78) CometColumnarToRow [codegen id : 21] -Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] - -(79) HashAggregate [codegen id : 21] -Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -Keys [2]: [channel#38, id#39] -Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] -Aggregate Attributes [3]: [sum(sales#40)#142, sum(returns#41)#143, sum(profit#42)#144] -Results [5]: [channel#38, id#39, cast(sum(sales#40)#142 as decimal(37,2)) AS sales#145, cast(sum(returns#41)#143 as decimal(37,2)) AS returns#146, cast(sum(profit#42)#144 as decimal(38,2)) AS profit#147] - -(80) ReusedExchange [Reuses operator id: 77] -Output [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] - -(81) CometColumnarToRow [codegen id : 42] -Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] - -(82) HashAggregate [codegen id : 42] -Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -Keys [2]: [channel#38, id#39] -Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] -Aggregate Attributes [3]: [sum(sales#40)#142, sum(returns#41)#143, sum(profit#42)#144] -Results [4]: [channel#38, sum(sales#40)#142 AS sales#148, sum(returns#41)#143 AS returns#149, sum(profit#42)#144 AS profit#150] - -(83) HashAggregate [codegen id : 42] -Input [4]: [channel#38, sales#148, returns#149, profit#150] -Keys [1]: [channel#38] -Functions [3]: [partial_sum(sales#148), partial_sum(returns#149), partial_sum(profit#150)] -Aggregate Attributes [6]: [sum#151, isEmpty#152, sum#153, isEmpty#154, sum#155, isEmpty#156] -Results [7]: [channel#38, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] - -(84) CometColumnarExchange -Input [7]: [channel#38, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] -Arguments: hashpartitioning(channel#38, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(85) CometColumnarToRow [codegen id : 43] -Input [7]: [channel#38, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] - -(86) HashAggregate [codegen id : 43] -Input [7]: [channel#38, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] -Keys [1]: [channel#38] -Functions [3]: [sum(sales#148), sum(returns#149), sum(profit#150)] -Aggregate Attributes [3]: [sum(sales#148)#163, sum(returns#149)#164, sum(profit#150)#165] -Results [5]: [channel#38, null AS id#166, sum(sales#148)#163 AS sum(sales)#167, sum(returns#149)#164 AS sum(returns)#168, sum(profit#150)#165 AS sum(profit)#169] - -(87) ReusedExchange [Reuses operator id: 77] -Output [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] - -(88) CometColumnarToRow [codegen id : 64] -Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] - -(89) HashAggregate [codegen id : 64] -Input [8]: [channel#38, id#39, sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -Keys [2]: [channel#38, id#39] -Functions [3]: [sum(sales#40), sum(returns#41), sum(profit#42)] -Aggregate Attributes [3]: [sum(sales#40)#142, sum(returns#41)#143, sum(profit#42)#144] -Results [3]: [sum(sales#40)#142 AS sales#148, sum(returns#41)#143 AS returns#149, sum(profit#42)#144 AS profit#150] - -(90) HashAggregate [codegen id : 64] -Input [3]: [sales#148, returns#149, profit#150] -Keys: [] -Functions [3]: [partial_sum(sales#148), partial_sum(returns#149), partial_sum(profit#150)] -Aggregate Attributes [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] -Results [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] - -(91) CometColumnarExchange -Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(92) CometColumnarToRow [codegen id : 65] -Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] - -(93) HashAggregate [codegen id : 65] -Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] -Keys: [] -Functions [3]: [sum(sales#148), sum(returns#149), sum(profit#150)] -Aggregate Attributes [3]: [sum(sales#148)#182, sum(returns#149)#183, sum(profit#150)#184] -Results [5]: [null AS channel#185, null AS id#186, sum(sales#148)#182 AS sum(sales)#187, sum(returns#149)#183 AS sum(returns)#188, sum(profit#150)#184 AS sum(profit)#189] - -(94) Union - -(95) HashAggregate [codegen id : 66] -Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] -Keys [5]: [channel#38, id#39, sales#145, returns#146, profit#147] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#38, id#39, sales#145, returns#146, profit#147] - -(96) CometColumnarExchange -Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] -Arguments: hashpartitioning(channel#38, id#39, sales#145, returns#146, profit#147, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(97) CometHashAggregate -Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] -Keys [5]: [channel#38, id#39, sales#145, returns#146, profit#147] -Functions: [] - -(98) CometTakeOrderedAndProject -Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#38 ASC NULLS FIRST,id#39 ASC NULLS FIRST], output=[channel#38,id#39,sales#145,returns#146,profit#147]), [channel#38, id#39, sales#145, returns#146, profit#147], 100, 0, [channel#38 ASC NULLS FIRST, id#39 ASC NULLS FIRST], [channel#38, id#39, sales#145, returns#146, profit#147] - -(99) CometColumnarToRow [codegen id : 67] -Input [5]: [channel#38, id#39, sales#145, returns#146, profit#147] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (104) -+- * CometColumnarToRow (103) - +- CometProject (102) - +- CometFilter (101) - +- CometNativeScan parquet spark_catalog.default.date_dim (100) - - -(100) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#190] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] -ReadSchema: struct - -(101) CometFilter -Input [2]: [d_date_sk#22, d_date#190] -Condition : (((isnotnull(d_date#190) AND (d_date#190 >= 1998-08-04)) AND (d_date#190 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) - -(102) CometProject -Input [2]: [d_date_sk#22, d_date#190] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(103) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(104) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -Subquery:2 Hosting operator id = 5 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#46 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 28 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#87 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/simplified.txt deleted file mode 100644 index dc9c9b1ce7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/simplified.txt +++ /dev/null @@ -1,167 +0,0 @@ -WholeStageCodegen (67) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (66) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (21) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (20) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [s_store_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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_id] #3 - WholeStageCodegen (5) - HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,s_store_id] - BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (1) - Project [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - WholeStageCodegen (2) - Project [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] - Filter [sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - WholeStageCodegen (12) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cp_catalog_page_id] #6 - WholeStageCodegen (11) - HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (7) - Project [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] - Filter [cs_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (8) - Project [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] - Filter [cr_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (19) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [web_site_id] #8 - WholeStageCodegen (18) - HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (13) - Project [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] - Filter [ws_web_site_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (15) - Project [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (14) - ColumnarToRow - InputAdapter - Scan 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 - CometColumnarToRow - InputAdapter - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometNativeScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (17) - CometColumnarToRow - InputAdapter - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - WholeStageCodegen (43) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #11 - WholeStageCodegen (42) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (65) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #12 - WholeStageCodegen (64) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/explain.txt deleted file mode 100644 index 306485ca63..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,524 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (84) -+- CometTakeOrderedAndProject (83) - +- CometHashAggregate (82) - +- CometExchange (81) - +- CometHashAggregate (80) - +- CometUnion (79) - :- CometHashAggregate (68) - : +- CometExchange (67) - : +- CometHashAggregate (66) - : +- CometUnion (65) - : :- CometHashAggregate (22) - : : +- CometExchange (21) - : : +- CometHashAggregate (20) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometUnion (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : :- CometHashAggregate (41) - : : +- CometExchange (40) - : : +- CometHashAggregate (39) - : : +- CometProject (38) - : : +- CometBroadcastHashJoin (37) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometUnion (29) - : : : : :- CometProject (25) - : : : : : +- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) - : : : : +- CometProject (28) - : : : : +- CometFilter (27) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) - : : : +- ReusedExchange (30) - : : +- CometBroadcastExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) - : +- CometHashAggregate (64) - : +- CometExchange (63) - : +- CometHashAggregate (62) - : +- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (55) - : : +- CometBroadcastHashJoin (54) - : : :- CometUnion (52) - : : : :- CometProject (44) - : : : : +- CometFilter (43) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : : : +- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometBroadcastExchange (46) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) - : : : +- CometProject (49) - : : : +- CometFilter (48) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : : +- ReusedExchange (53) - : +- CometBroadcastExchange (59) - : +- CometProject (58) - : +- CometFilter (57) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) - :- CometHashAggregate (73) - : +- CometExchange (72) - : +- CometHashAggregate (71) - : +- CometHashAggregate (70) - : +- ReusedExchange (69) - +- CometHashAggregate (78) - +- CometExchange (77) - +- CometHashAggregate (76) - +- CometHashAggregate (75) - +- ReusedExchange (74) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Arguments: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11], [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#12) - -(6) CometProject -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Arguments: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21], [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] - -(7) CometUnion -Child 0 Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) - -(10) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(11) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(12) CometBroadcastHashJoin -Left output [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [1]: [d_date_sk#22] -Arguments: [date_sk#7], [d_date_sk#22], Inner, BuildRight - -(13) CometProject -Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] -Arguments: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11], [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_store_id#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#24, s_store_id#25] -Condition : isnotnull(s_store_sk#24) - -(16) CometProject -Input [2]: [s_store_sk#24, s_store_id#25] -Arguments: [s_store_sk#24, s_store_id#26], [s_store_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#25, 16, true, false, true) AS s_store_id#26] - -(17) CometBroadcastExchange -Input [2]: [s_store_sk#24, s_store_id#26] -Arguments: [s_store_sk#24, s_store_id#26] - -(18) CometBroadcastHashJoin -Left output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [2]: [s_store_sk#24, s_store_id#26] -Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight - -(19) CometProject -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#26] -Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] - -(20) CometHashAggregate -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] -Keys [1]: [s_store_id#26] -Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] - -(21) CometExchange -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Keys [1]: [s_store_id#26] -Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk)] -ReadSchema: struct - -(24) CometFilter -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : isnotnull(cs_catalog_page_sk#31) - -(25) CometProject -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cr_catalog_page_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Condition : isnotnull(cr_catalog_page_sk#42) - -(28) CometProject -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] - -(29) CometUnion -Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] - -(30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#52] - -(31) CometBroadcastHashJoin -Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [1]: [d_date_sk#52] -Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight - -(32) CometProject -Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] -Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Condition : isnotnull(cp_catalog_page_sk#53) - -(35) CometProject -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#54, 16, true, false, true) AS cp_catalog_page_id#55] - -(36) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] - -(37) CometBroadcastHashJoin -Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight - -(38) CometProject -Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] - -(39) CometHashAggregate -Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] - -(40) CometExchange -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(41) CometHashAggregate -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] -PushedFilters: [IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(43) CometFilter -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_web_site_sk#60) - -(44) CometProject -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] -ReadSchema: struct - -(46) CometBroadcastExchange -Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(48) CometFilter -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) - -(49) CometProject -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] - -(50) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft - -(51) CometProject -Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] - -(52) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] - -(53) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#86] - -(54) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [1]: [d_date_sk#86] -Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight - -(55) CometProject -Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] -Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#87, web_site_id#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(57) CometFilter -Input [2]: [web_site_sk#87, web_site_id#88] -Condition : isnotnull(web_site_sk#87) - -(58) CometProject -Input [2]: [web_site_sk#87, web_site_id#88] -Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#88, 16, true, false, true) AS web_site_id#89] - -(59) CometBroadcastExchange -Input [2]: [web_site_sk#87, web_site_id#89] -Arguments: [web_site_sk#87, web_site_id#89] - -(60) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [2]: [web_site_sk#87, web_site_id#89] -Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight - -(61) CometProject -Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] -Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] - -(62) CometHashAggregate -Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] -Keys [1]: [web_site_id#89] -Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] - -(63) CometExchange -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(64) CometHashAggregate -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Keys [1]: [web_site_id#89] -Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] - -(65) CometUnion -Child 0 Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] -Child 1 Input [5]: [channel#99, id#100, sales#101, returns#102, profit#103] -Child 2 Input [5]: [channel#104, id#105, sales#106, returns#107, profit#108] - -(66) CometHashAggregate -Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] -Keys [2]: [channel#94, id#95] -Functions [3]: [partial_sum(sales#96), partial_sum(returns#97), partial_sum(profit#98)] - -(67) CometExchange -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: hashpartitioning(channel#94, id#95, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(68) CometHashAggregate -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#94, id#95] -Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] - -(69) ReusedExchange [Reuses operator id: 67] -Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(70) CometHashAggregate -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#94, id#95] -Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] - -(71) CometHashAggregate -Input [4]: [channel#94, sales#115, returns#116, profit#117] -Keys [1]: [channel#94] -Functions [3]: [partial_sum(sales#115), partial_sum(returns#116), partial_sum(profit#117)] - -(72) CometExchange -Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] -Arguments: hashpartitioning(channel#94, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(73) CometHashAggregate -Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] -Keys [1]: [channel#94] -Functions [3]: [sum(sales#115), sum(returns#116), sum(profit#117)] - -(74) ReusedExchange [Reuses operator id: 67] -Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(75) CometHashAggregate -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#94, id#95] -Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] - -(76) CometHashAggregate -Input [3]: [sales#115, returns#116, profit#117] -Keys: [] -Functions [3]: [partial_sum(sales#115), partial_sum(returns#116), partial_sum(profit#117)] - -(77) CometExchange -Input [6]: [sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(78) CometHashAggregate -Input [6]: [sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -Keys: [] -Functions [3]: [sum(sales#115), sum(returns#116), sum(profit#117)] - -(79) CometUnion -Child 0 Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] -Child 1 Input [5]: [channel#94, id#133, sum(sales)#134, sum(returns)#135, sum(profit)#136] -Child 2 Input [5]: [channel#137, id#138, sum(sales)#139, sum(returns)#140, sum(profit)#141] - -(80) CometHashAggregate -Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] -Keys [5]: [channel#94, id#95, sales#130, returns#131, profit#132] -Functions: [] - -(81) CometExchange -Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] -Arguments: hashpartitioning(channel#94, id#95, sales#130, returns#131, profit#132, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(82) CometHashAggregate -Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] -Keys [5]: [channel#94, id#95, sales#130, returns#131, profit#132] -Functions: [] - -(83) CometTakeOrderedAndProject -Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#94 ASC NULLS FIRST,id#95 ASC NULLS FIRST], output=[channel#94,id#95,sales#130,returns#131,profit#132]), [channel#94, id#95, sales#130, returns#131, profit#132], 100, 0, [channel#94 ASC NULLS FIRST, id#95 ASC NULLS FIRST], [channel#94, id#95, sales#130, returns#131, profit#132] - -(84) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (89) -+- * CometColumnarToRow (88) - +- CometProject (87) - +- CometFilter (86) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) - - -(85) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] -ReadSchema: struct - -(86) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) - -(87) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(88) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(89) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/simplified.txt deleted file mode 100644 index c0b236a57a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,99 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id] #2 - CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [s_store_id] #3 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - 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 [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,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #6 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [cp_catalog_page_id] #7 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - 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 [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,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [cp_catalog_page_sk,cp_catalog_page_id] #8 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [web_site_id] #9 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - 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 [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,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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,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 [native_iceberg_compat] 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_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 [web_site_sk,web_site_id] #11 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] - CometExchange [channel] #12 - CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] - CometExchange #13 - CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt deleted file mode 100644 index 306485ca63..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt +++ /dev/null @@ -1,524 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (84) -+- CometTakeOrderedAndProject (83) - +- CometHashAggregate (82) - +- CometExchange (81) - +- CometHashAggregate (80) - +- CometUnion (79) - :- CometHashAggregate (68) - : +- CometExchange (67) - : +- CometHashAggregate (66) - : +- CometUnion (65) - : :- CometHashAggregate (22) - : : +- CometExchange (21) - : : +- CometHashAggregate (20) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometUnion (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : :- CometHashAggregate (41) - : : +- CometExchange (40) - : : +- CometHashAggregate (39) - : : +- CometProject (38) - : : +- CometBroadcastHashJoin (37) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometUnion (29) - : : : : :- CometProject (25) - : : : : : +- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) - : : : : +- CometProject (28) - : : : : +- CometFilter (27) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) - : : : +- ReusedExchange (30) - : : +- CometBroadcastExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) - : +- CometHashAggregate (64) - : +- CometExchange (63) - : +- CometHashAggregate (62) - : +- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (55) - : : +- CometBroadcastHashJoin (54) - : : :- CometUnion (52) - : : : :- CometProject (44) - : : : : +- CometFilter (43) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : : : +- CometProject (51) - : : : +- CometBroadcastHashJoin (50) - : : : :- CometBroadcastExchange (46) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) - : : : +- CometProject (49) - : : : +- CometFilter (48) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) - : : +- ReusedExchange (53) - : +- CometBroadcastExchange (59) - : +- CometProject (58) - : +- CometFilter (57) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) - :- CometHashAggregate (73) - : +- CometExchange (72) - : +- CometHashAggregate (71) - : +- CometHashAggregate (70) - : +- ReusedExchange (69) - +- CometHashAggregate (78) - +- CometExchange (77) - +- CometHashAggregate (76) - +- CometHashAggregate (75) - +- ReusedExchange (74) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Arguments: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11], [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Condition : isnotnull(sr_store_sk#12) - -(6) CometProject -Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] -Arguments: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21], [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] - -(7) CometUnion -Child 0 Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) - -(10) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(11) CometBroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: [d_date_sk#22] - -(12) CometBroadcastHashJoin -Left output [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [1]: [d_date_sk#22] -Arguments: [date_sk#7], [d_date_sk#22], Inner, BuildRight - -(13) CometProject -Input [7]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11, d_date_sk#22] -Arguments: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11], [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#24, s_store_id#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_store_sk#24, s_store_id#25] -Condition : isnotnull(s_store_sk#24) - -(16) CometProject -Input [2]: [s_store_sk#24, s_store_id#25] -Arguments: [s_store_sk#24, s_store_id#26], [s_store_sk#24, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#25, 16, true, false, true) AS s_store_id#26] - -(17) CometBroadcastExchange -Input [2]: [s_store_sk#24, s_store_id#26] -Arguments: [s_store_sk#24, s_store_id#26] - -(18) CometBroadcastHashJoin -Left output [5]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11] -Right output [2]: [s_store_sk#24, s_store_id#26] -Arguments: [store_sk#6], [s_store_sk#24], Inner, BuildRight - -(19) CometProject -Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#24, s_store_id#26] -Arguments: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26], [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] - -(20) CometHashAggregate -Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#26] -Keys [1]: [s_store_id#26] -Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] - -(21) CometExchange -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(22) CometHashAggregate -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] -Keys [1]: [s_store_id#26] -Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk)] -ReadSchema: struct - -(24) CometFilter -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : isnotnull(cs_catalog_page_sk#31) - -(25) CometProject -Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cr_catalog_page_sk)] -ReadSchema: struct - -(27) CometFilter -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Condition : isnotnull(cr_catalog_page_sk#42) - -(28) CometProject -Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] -Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] - -(29) CometUnion -Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] - -(30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#52] - -(31) CometBroadcastHashJoin -Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [1]: [d_date_sk#52] -Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight - -(32) CometProject -Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] -Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Condition : isnotnull(cp_catalog_page_sk#53) - -(35) CometProject -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#54, 16, true, false, true) AS cp_catalog_page_id#55] - -(36) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] - -(37) CometBroadcastHashJoin -Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight - -(38) CometProject -Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] -Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] - -(39) CometHashAggregate -Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] - -(40) CometExchange -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(41) CometHashAggregate -Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] -Keys [1]: [cp_catalog_page_id#55] -Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] -PushedFilters: [IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(43) CometFilter -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_web_site_sk#60) - -(44) CometProject -Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] -Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] - -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] -ReadSchema: struct - -(46) CometBroadcastExchange -Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] -ReadSchema: struct - -(48) CometFilter -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) - -(49) CometProject -Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] -Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] - -(50) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft - -(51) CometProject -Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] - -(52) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] - -(53) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#86] - -(54) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [1]: [d_date_sk#86] -Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight - -(55) CometProject -Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] -Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#87, web_site_id#88] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(57) CometFilter -Input [2]: [web_site_sk#87, web_site_id#88] -Condition : isnotnull(web_site_sk#87) - -(58) CometProject -Input [2]: [web_site_sk#87, web_site_id#88] -Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#88, 16, true, false, true) AS web_site_id#89] - -(59) CometBroadcastExchange -Input [2]: [web_site_sk#87, web_site_id#89] -Arguments: [web_site_sk#87, web_site_id#89] - -(60) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -Right output [2]: [web_site_sk#87, web_site_id#89] -Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight - -(61) CometProject -Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] -Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] - -(62) CometHashAggregate -Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] -Keys [1]: [web_site_id#89] -Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] - -(63) CometExchange -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(64) CometHashAggregate -Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] -Keys [1]: [web_site_id#89] -Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] - -(65) CometUnion -Child 0 Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] -Child 1 Input [5]: [channel#99, id#100, sales#101, returns#102, profit#103] -Child 2 Input [5]: [channel#104, id#105, sales#106, returns#107, profit#108] - -(66) CometHashAggregate -Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] -Keys [2]: [channel#94, id#95] -Functions [3]: [partial_sum(sales#96), partial_sum(returns#97), partial_sum(profit#98)] - -(67) CometExchange -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: hashpartitioning(channel#94, id#95, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(68) CometHashAggregate -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#94, id#95] -Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] - -(69) ReusedExchange [Reuses operator id: 67] -Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(70) CometHashAggregate -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#94, id#95] -Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] - -(71) CometHashAggregate -Input [4]: [channel#94, sales#115, returns#116, profit#117] -Keys [1]: [channel#94] -Functions [3]: [partial_sum(sales#115), partial_sum(returns#116), partial_sum(profit#117)] - -(72) CometExchange -Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] -Arguments: hashpartitioning(channel#94, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(73) CometHashAggregate -Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] -Keys [1]: [channel#94] -Functions [3]: [sum(sales#115), sum(returns#116), sum(profit#117)] - -(74) ReusedExchange [Reuses operator id: 67] -Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(75) CometHashAggregate -Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#94, id#95] -Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] - -(76) CometHashAggregate -Input [3]: [sales#115, returns#116, profit#117] -Keys: [] -Functions [3]: [partial_sum(sales#115), partial_sum(returns#116), partial_sum(profit#117)] - -(77) CometExchange -Input [6]: [sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(78) CometHashAggregate -Input [6]: [sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -Keys: [] -Functions [3]: [sum(sales#115), sum(returns#116), sum(profit#117)] - -(79) CometUnion -Child 0 Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] -Child 1 Input [5]: [channel#94, id#133, sum(sales)#134, sum(returns)#135, sum(profit)#136] -Child 2 Input [5]: [channel#137, id#138, sum(sales)#139, sum(returns)#140, sum(profit)#141] - -(80) CometHashAggregate -Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] -Keys [5]: [channel#94, id#95, sales#130, returns#131, profit#132] -Functions: [] - -(81) CometExchange -Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] -Arguments: hashpartitioning(channel#94, id#95, sales#130, returns#131, profit#132, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(82) CometHashAggregate -Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] -Keys [5]: [channel#94, id#95, sales#130, returns#131, profit#132] -Functions: [] - -(83) CometTakeOrderedAndProject -Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#94 ASC NULLS FIRST,id#95 ASC NULLS FIRST], output=[channel#94,id#95,sales#130,returns#131,profit#132]), [channel#94, id#95, sales#130, returns#131, profit#132], 100, 0, [channel#94 ASC NULLS FIRST, id#95 ASC NULLS FIRST], [channel#94, id#95, sales#130, returns#131, profit#132] - -(84) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (89) -+- * CometColumnarToRow (88) - +- CometProject (87) - +- CometFilter (86) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) - - -(85) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#22, d_date#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] -ReadSchema: struct - -(86) CometFilter -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) - -(87) CometProject -Input [2]: [d_date_sk#22, d_date#23] -Arguments: [d_date_sk#22], [d_date_sk#22] - -(88) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#22] - -(89) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/extended.txt deleted file mode 100644 index 35b86dbff5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/extended.txt +++ /dev/null @@ -1,269 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastExchange - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastExchange - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - -Comet accelerated 242 out of 263 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index c0b236a57a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt +++ /dev/null @@ -1,99 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id] #2 - CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [s_store_id] #3 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - 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 [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,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #6 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [cp_catalog_page_id] #7 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - 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 [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,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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 [cp_catalog_page_sk,cp_catalog_page_id] #8 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] - CometExchange [web_site_id] #9 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - 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 [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,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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,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 [native_iceberg_compat] 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_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 [web_site_sk,web_site_id] #11 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] - CometExchange [channel] #12 - CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] - CometExchange #13 - CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/explain.txt deleted file mode 100644 index e0b30c6833..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/explain.txt +++ /dev/null @@ -1,304 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (40) -+- * Filter (39) - +- * HashAggregate (38) - +- * CometColumnarToRow (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * Project (34) - +- * BroadcastHashJoin Inner BuildRight (33) - :- * Project (18) - : +- * BroadcastHashJoin Inner BuildRight (17) - : :- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- * CometColumnarToRow (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometFilter (5) - : : : +- CometNativeScan parquet spark_catalog.default.customer (4) - : : +- BroadcastExchange (13) - : : +- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet spark_catalog.default.store_sales (10) - : +- ReusedExchange (16) - +- BroadcastExchange (32) - +- * CometColumnarToRow (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometFilter (20) - : +- CometNativeScan parquet spark_catalog.default.item (19) - +- CometBroadcastExchange (28) - +- CometFilter (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometFilter (22) - +- CometNativeScan parquet spark_catalog.default.item (21) - - -(1) CometNativeScan parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#1, ca_state#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ca_address_sk#1, ca_state#2] -Condition : isnotnull(ca_address_sk#1) - -(3) CometProject -Input [2]: [ca_address_sk#1, ca_state#2] -Arguments: [ca_address_sk#1, ca_state#3], [ca_address_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#2, 2, true, false, true) AS ca_state#3] - -(4) CometNativeScan parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_customer_sk#4)) - -(6) CometBroadcastExchange -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [c_customer_sk#4, c_current_addr_sk#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#1, ca_state#3] -Right output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] - -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ca_state#3, c_customer_sk#4] - -(10) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 1] -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(12) Filter [codegen id : 1] -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) - -(13) BroadcastExchange -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] - -(14) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [c_customer_sk#4] -Right keys [1]: [ss_customer_sk#7] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 4] -Output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(16) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#10] - -(17) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] -Join type: Inner -Join condition: None - -(18) Project [codegen id : 4] -Output [2]: [ca_state#3, ss_item_sk#6] -Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] - -(19) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#11, i_current_price#12, i_category#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)] -ReadSchema: struct - -(20) CometFilter -Input [3]: [i_item_sk#11, i_current_price#12, i_category#13] -Condition : (isnotnull(i_current_price#12) AND isnotnull(i_item_sk#11)) - -(21) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_current_price#12, i_category#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(22) CometFilter -Input [2]: [i_current_price#12, i_category#13] -Condition : isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#13, 50, true, false, true)) - -(23) CometProject -Input [2]: [i_current_price#12, i_category#13] -Arguments: [i_current_price#12, i_category#14], [i_current_price#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#13, 50, true, false, true) AS i_category#14] - -(24) CometHashAggregate -Input [2]: [i_current_price#12, i_category#14] -Keys [1]: [i_category#14] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#12))] - -(25) CometExchange -Input [3]: [i_category#14, sum#15, count#16] -Arguments: hashpartitioning(i_category#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(26) CometHashAggregate -Input [3]: [i_category#14, sum#15, count#16] -Keys [1]: [i_category#14] -Functions [1]: [avg(UnscaledValue(i_current_price#12))] - -(27) CometFilter -Input [2]: [avg(i_current_price)#17, i_category#14] -Condition : isnotnull(avg(i_current_price)#17) - -(28) CometBroadcastExchange -Input [2]: [avg(i_current_price)#17, i_category#14] -Arguments: [avg(i_current_price)#17, i_category#14] - -(29) CometBroadcastHashJoin -Left output [3]: [i_item_sk#11, i_current_price#12, i_category#13] -Right output [2]: [avg(i_current_price)#17, i_category#14] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#13, 50, true, false, true)], [i_category#14], Inner, (cast(i_current_price#12 as decimal(14,7)) > (1.2 * avg(i_current_price)#17)), BuildRight - -(30) CometProject -Input [5]: [i_item_sk#11, i_current_price#12, i_category#13, avg(i_current_price)#17, i_category#14] -Arguments: [i_item_sk#11], [i_item_sk#11] - -(31) CometColumnarToRow [codegen id : 3] -Input [1]: [i_item_sk#11] - -(32) BroadcastExchange -Input [1]: [i_item_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(33) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#6] -Right keys [1]: [i_item_sk#11] -Join type: Inner -Join condition: None - -(34) Project [codegen id : 4] -Output [1]: [ca_state#3] -Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#11] - -(35) HashAggregate [codegen id : 4] -Input [1]: [ca_state#3] -Keys [1]: [ca_state#3] -Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#18] -Results [2]: [ca_state#3, count#19] - -(36) CometColumnarExchange -Input [2]: [ca_state#3, count#19] -Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(37) CometColumnarToRow [codegen id : 5] -Input [2]: [ca_state#3, count#19] - -(38) HashAggregate [codegen id : 5] -Input [2]: [ca_state#3, count#19] -Keys [1]: [ca_state#3] -Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#20] -Results [3]: [ca_state#3 AS state#21, count(1)#20 AS cnt#22, ca_state#3] - -(39) Filter [codegen id : 5] -Input [3]: [state#21, cnt#22, ca_state#3] -Condition : (cnt#22 >= 10) - -(40) TakeOrderedAndProject -Input [3]: [state#21, cnt#22, ca_state#3] -Arguments: 100, [cnt#22 ASC NULLS FIRST, ca_state#3 ASC NULLS FIRST], [state#21, cnt#22] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometNativeScan parquet spark_catalog.default.date_dim (41) - - -(41) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#23] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#23] -Condition : ((isnotnull(d_month_seq#23) AND (d_month_seq#23 = Subquery scalar-subquery#24, [id=#25])) AND isnotnull(d_date_sk#10)) - -(43) CometProject -Input [2]: [d_date_sk#10, d_month_seq#23] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(45) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#24, [id=#25] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometNativeScan parquet spark_catalog.default.date_dim (46) - - -(46) CometNativeScan parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#23, d_year#26, d_moy#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_month_seq#23, d_year#26, d_moy#27] -Condition : (((isnotnull(d_year#26) AND isnotnull(d_moy#27)) AND (d_year#26 = 2000)) AND (d_moy#27 = 1)) - -(48) CometProject -Input [3]: [d_month_seq#23, d_year#26, d_moy#27] -Arguments: [d_month_seq#23], [d_month_seq#23] - -(49) CometHashAggregate -Input [1]: [d_month_seq#23] -Keys [1]: [d_month_seq#23] -Functions: [] - -(50) CometExchange -Input [1]: [d_month_seq#23] -Arguments: hashpartitioning(d_month_seq#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(51) CometHashAggregate -Input [1]: [d_month_seq#23] -Keys [1]: [d_month_seq#23] -Functions: [] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#23] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/simplified.txt deleted file mode 100644 index 76dd43f9c7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/simplified.txt +++ /dev/null @@ -1,69 +0,0 @@ -TakeOrderedAndProject [cnt,ca_state,state] - WholeStageCodegen (5) - Filter [cnt] - HashAggregate [ca_state,count] [count(1),state,cnt,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ca_state] #1 - WholeStageCodegen (4) - HashAggregate [ca_state] [count,count] - Project [ca_state] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ca_state,ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ca_state,ss_item_sk,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [ca_state,c_customer_sk] - CometBroadcastHashJoin [ca_address_sk,ca_state,c_customer_sk,c_current_addr_sk] - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [ss_customer_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #7 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #8 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_current_price,i_category] - CometFilter [i_current_price,i_category] - CometNativeScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/explain.txt deleted file mode 100644 index f6395ec27c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometFilter (38) - +- CometHashAggregate (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (4) - : : +- CometBroadcastExchange (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (9) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- CometBroadcastExchange (29) - +- CometFilter (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (22) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#1, ca_state#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ca_address_sk#1, ca_state#2] -Condition : isnotnull(ca_address_sk#1) - -(3) CometProject -Input [2]: [ca_address_sk#1, ca_state#2] -Arguments: [ca_address_sk#1, ca_state#3], [ca_address_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#2, 2, true, false, true) AS ca_state#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_customer_sk#4)) - -(6) CometBroadcastExchange -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [c_customer_sk#4, c_current_addr_sk#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#1, ca_state#3] -Right output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) - -(11) CometBroadcastExchange -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(12) CometBroadcastHashJoin -Left output [2]: [ca_state#3, c_customer_sk#4] -Right output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [c_customer_sk#4], [ss_customer_sk#7], Inner, BuildRight - -(13) CometProject -Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8], [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(16) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(18) CometBroadcastHashJoin -Left output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(19) CometProject -Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ca_state#3, ss_item_sk#6], [ca_state#3, ss_item_sk#6] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Condition : (isnotnull(i_current_price#15) AND isnotnull(i_item_sk#14)) - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_current_price#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(23) CometFilter -Input [2]: [i_current_price#15, i_category#16] -Condition : isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true)) - -(24) CometProject -Input [2]: [i_current_price#15, i_category#16] -Arguments: [i_current_price#15, i_category#17], [i_current_price#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true) AS i_category#17] - -(25) CometHashAggregate -Input [2]: [i_current_price#15, i_category#17] -Keys [1]: [i_category#17] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#15))] - -(26) CometExchange -Input [3]: [i_category#17, sum#18, count#19] -Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [3]: [i_category#17, sum#18, count#19] -Keys [1]: [i_category#17] -Functions [1]: [avg(UnscaledValue(i_current_price#15))] - -(28) CometFilter -Input [2]: [avg(i_current_price)#20, i_category#17] -Condition : isnotnull(avg(i_current_price)#20) - -(29) CometBroadcastExchange -Input [2]: [avg(i_current_price)#20, i_category#17] -Arguments: [avg(i_current_price)#20, i_category#17] - -(30) CometBroadcastHashJoin -Left output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Right output [2]: [avg(i_current_price)#20, i_category#17] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true)], [i_category#17], Inner, (cast(i_current_price#15 as decimal(14,7)) > (1.2 * avg(i_current_price)#20)), BuildRight - -(31) CometProject -Input [5]: [i_item_sk#14, i_current_price#15, i_category#16, avg(i_current_price)#20, i_category#17] -Arguments: [i_item_sk#14], [i_item_sk#14] - -(32) CometBroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: [i_item_sk#14] - -(33) CometBroadcastHashJoin -Left output [2]: [ca_state#3, ss_item_sk#6] -Right output [1]: [i_item_sk#14] -Arguments: [ss_item_sk#6], [i_item_sk#14], Inner, BuildRight - -(34) CometProject -Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#14] -Arguments: [ca_state#3], [ca_state#3] - -(35) CometHashAggregate -Input [1]: [ca_state#3] -Keys [1]: [ca_state#3] -Functions [1]: [partial_count(1)] - -(36) CometExchange -Input [2]: [ca_state#3, count#21] -Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(37) CometHashAggregate -Input [2]: [ca_state#3, count#21] -Keys [1]: [ca_state#3] -Functions [1]: [count(1)] - -(38) CometFilter -Input [3]: [state#22, cnt#23, ca_state#3] -Condition : (cnt#23 >= 10) - -(39) CometTakeOrderedAndProject -Input [3]: [state#22, cnt#23, ca_state#3] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#23 ASC NULLS FIRST,ca_state#3 ASC NULLS FIRST], output=[state#22,cnt#23]), [state#22, cnt#23], 100, 0, [cnt#23 ASC NULLS FIRST, ca_state#3 ASC NULLS FIRST], [state#22, cnt#23] - -(40) CometColumnarToRow [codegen id : 1] -Input [2]: [state#22, cnt#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(43) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(45) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) - - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#11, d_year#24, d_moy#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_month_seq#11, d_year#24, d_moy#25] -Condition : (((isnotnull(d_year#24) AND isnotnull(d_moy#25)) AND (d_year#24 = 2000)) AND (d_moy#25 = 1)) - -(48) CometProject -Input [3]: [d_month_seq#11, d_year#24, d_moy#25] -Arguments: [d_month_seq#11], [d_month_seq#11] - -(49) CometHashAggregate -Input [1]: [d_month_seq#11] -Keys [1]: [d_month_seq#11] -Functions: [] - -(50) CometExchange -Input [1]: [d_month_seq#11] -Arguments: hashpartitioning(d_month_seq#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(51) CometHashAggregate -Input [1]: [d_month_seq#11] -Keys [1]: [d_month_seq#11] -Functions: [] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#11] - -Subquery:3 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/simplified.txt deleted file mode 100644 index 2978e30c1c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_state] [state,cnt] - CometFilter [state,cnt,ca_state] - CometHashAggregate [count] [state,cnt,ca_state,count(1)] - CometExchange [ca_state] #1 - CometHashAggregate [ca_state,count] - CometProject [ca_state] - CometBroadcastHashJoin [ca_state,ss_item_sk,i_item_sk] - CometProject [ca_state,ss_item_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 [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,ca_state,c_customer_sk,c_current_addr_sk] - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk] #7 - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #8 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #9 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_current_price,i_category] - CometFilter [i_current_price,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt deleted file mode 100644 index f6395ec27c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt +++ /dev/null @@ -1,306 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (40) -+- CometTakeOrderedAndProject (39) - +- CometFilter (38) - +- CometHashAggregate (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (4) - : : +- CometBroadcastExchange (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (9) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- CometBroadcastExchange (29) - +- CometFilter (28) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometFilter (23) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (22) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#1, ca_state#2] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [ca_address_sk#1, ca_state#2] -Condition : isnotnull(ca_address_sk#1) - -(3) CometProject -Input [2]: [ca_address_sk#1, ca_state#2] -Arguments: [ca_address_sk#1, ca_state#3], [ca_address_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_state#2, 2, true, false, true) AS ca_state#3] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_customer_sk#4)) - -(6) CometBroadcastExchange -Input [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [c_customer_sk#4, c_current_addr_sk#5] - -(7) CometBroadcastHashJoin -Left output [2]: [ca_address_sk#1, ca_state#3] -Right output [2]: [c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_address_sk#1], [c_current_addr_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ca_address_sk#1, ca_state#3, c_customer_sk#4, c_current_addr_sk#5] -Arguments: [ca_state#3, c_customer_sk#4], [ca_state#3, c_customer_sk#4] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Condition : (isnotnull(ss_customer_sk#7) AND isnotnull(ss_item_sk#6)) - -(11) CometBroadcastExchange -Input [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] - -(12) CometBroadcastHashJoin -Left output [2]: [ca_state#3, c_customer_sk#4] -Right output [3]: [ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [c_customer_sk#4], [ss_customer_sk#7], Inner, BuildRight - -(13) CometProject -Input [5]: [ca_state#3, c_customer_sk#4, ss_item_sk#6, ss_customer_sk#7, ss_sold_date_sk#8] -Arguments: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8], [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] - -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = ReusedSubquery Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(16) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(17) CometBroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: [d_date_sk#10] - -(18) CometBroadcastHashJoin -Left output [3]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8] -Right output [1]: [d_date_sk#10] -Arguments: [ss_sold_date_sk#8], [d_date_sk#10], Inner, BuildRight - -(19) CometProject -Input [4]: [ca_state#3, ss_item_sk#6, ss_sold_date_sk#8, d_date_sk#10] -Arguments: [ca_state#3, ss_item_sk#6], [ca_state#3, ss_item_sk#6] - -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)] -ReadSchema: struct - -(21) CometFilter -Input [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Condition : (isnotnull(i_current_price#15) AND isnotnull(i_item_sk#14)) - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_current_price#15, i_category#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -ReadSchema: struct - -(23) CometFilter -Input [2]: [i_current_price#15, i_category#16] -Condition : isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true)) - -(24) CometProject -Input [2]: [i_current_price#15, i_category#16] -Arguments: [i_current_price#15, i_category#17], [i_current_price#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true) AS i_category#17] - -(25) CometHashAggregate -Input [2]: [i_current_price#15, i_category#17] -Keys [1]: [i_category#17] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#15))] - -(26) CometExchange -Input [3]: [i_category#17, sum#18, count#19] -Arguments: hashpartitioning(i_category#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(27) CometHashAggregate -Input [3]: [i_category#17, sum#18, count#19] -Keys [1]: [i_category#17] -Functions [1]: [avg(UnscaledValue(i_current_price#15))] - -(28) CometFilter -Input [2]: [avg(i_current_price)#20, i_category#17] -Condition : isnotnull(avg(i_current_price)#20) - -(29) CometBroadcastExchange -Input [2]: [avg(i_current_price)#20, i_category#17] -Arguments: [avg(i_current_price)#20, i_category#17] - -(30) CometBroadcastHashJoin -Left output [3]: [i_item_sk#14, i_current_price#15, i_category#16] -Right output [2]: [avg(i_current_price)#20, i_category#17] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#16, 50, true, false, true)], [i_category#17], Inner, (cast(i_current_price#15 as decimal(14,7)) > (1.2 * avg(i_current_price)#20)), BuildRight - -(31) CometProject -Input [5]: [i_item_sk#14, i_current_price#15, i_category#16, avg(i_current_price)#20, i_category#17] -Arguments: [i_item_sk#14], [i_item_sk#14] - -(32) CometBroadcastExchange -Input [1]: [i_item_sk#14] -Arguments: [i_item_sk#14] - -(33) CometBroadcastHashJoin -Left output [2]: [ca_state#3, ss_item_sk#6] -Right output [1]: [i_item_sk#14] -Arguments: [ss_item_sk#6], [i_item_sk#14], Inner, BuildRight - -(34) CometProject -Input [3]: [ca_state#3, ss_item_sk#6, i_item_sk#14] -Arguments: [ca_state#3], [ca_state#3] - -(35) CometHashAggregate -Input [1]: [ca_state#3] -Keys [1]: [ca_state#3] -Functions [1]: [partial_count(1)] - -(36) CometExchange -Input [2]: [ca_state#3, count#21] -Arguments: hashpartitioning(ca_state#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(37) CometHashAggregate -Input [2]: [ca_state#3, count#21] -Keys [1]: [ca_state#3] -Functions [1]: [count(1)] - -(38) CometFilter -Input [3]: [state#22, cnt#23, ca_state#3] -Condition : (cnt#23 >= 10) - -(39) CometTakeOrderedAndProject -Input [3]: [state#22, cnt#23, ca_state#3] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[cnt#23 ASC NULLS FIRST,ca_state#3 ASC NULLS FIRST], output=[state#22,cnt#23]), [state#22, cnt#23], 100, 0, [cnt#23 ASC NULLS FIRST, ca_state#3 ASC NULLS FIRST], [state#22, cnt#23] - -(40) CometColumnarToRow [codegen id : 1] -Input [2]: [state#22, cnt#23] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (45) -+- * CometColumnarToRow (44) - +- CometProject (43) - +- CometFilter (42) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#10, d_month_seq#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#10, d_month_seq#11] -Condition : ((isnotnull(d_month_seq#11) AND (d_month_seq#11 = Subquery scalar-subquery#12, [id=#13])) AND isnotnull(d_date_sk#10)) - -(43) CometProject -Input [2]: [d_date_sk#10, d_month_seq#11] -Arguments: [d_date_sk#10], [d_date_sk#10] - -(44) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#10] - -(45) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -Subquery:2 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#12, [id=#13] -* CometColumnarToRow (52) -+- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) - - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#11, d_year#24, d_moy#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] -ReadSchema: struct - -(47) CometFilter -Input [3]: [d_month_seq#11, d_year#24, d_moy#25] -Condition : (((isnotnull(d_year#24) AND isnotnull(d_moy#25)) AND (d_year#24 = 2000)) AND (d_moy#25 = 1)) - -(48) CometProject -Input [3]: [d_month_seq#11, d_year#24, d_moy#25] -Arguments: [d_month_seq#11], [d_month_seq#11] - -(49) CometHashAggregate -Input [1]: [d_month_seq#11] -Keys [1]: [d_month_seq#11] -Functions: [] - -(50) CometExchange -Input [1]: [d_month_seq#11] -Arguments: hashpartitioning(d_month_seq#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(51) CometHashAggregate -Input [1]: [d_month_seq#11] -Keys [1]: [d_month_seq#11] -Functions: [] - -(52) CometColumnarToRow [codegen id : 1] -Input [1]: [d_month_seq#11] - -Subquery:3 Hosting operator id = 15 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/extended.txt deleted file mode 100644 index f5b69fc6cf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/extended.txt +++ /dev/null @@ -1,57 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 2978e30c1c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt +++ /dev/null @@ -1,61 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [ca_state] [state,cnt] - CometFilter [state,cnt,ca_state] - CometHashAggregate [count] [state,cnt,ca_state,count(1)] - CometExchange [ca_state] #1 - CometHashAggregate [ca_state,count] - CometProject [ca_state] - CometBroadcastHashJoin [ca_state,ss_item_sk,i_item_sk] - CometProject [ca_state,ss_item_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 [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,ca_state,c_customer_sk,c_current_addr_sk] - CometProject [ca_state] [ca_address_sk,ca_state] - CometFilter [ca_address_sk,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #2 - CometFilter [c_customer_sk,c_current_addr_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - Subquery #2 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [d_month_seq] - CometExchange [d_month_seq] #5 - CometHashAggregate [d_month_seq] - CometProject [d_month_seq] - CometFilter [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [d_month_seq] #2 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk] #7 - CometProject [i_item_sk] - CometBroadcastHashJoin [i_item_sk,i_current_price,i_category,avg(i_current_price),i_category] - CometFilter [i_item_sk,i_current_price,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] - CometBroadcastExchange [avg(i_current_price),i_category] #8 - CometFilter [avg(i_current_price),i_category] - CometHashAggregate [sum,count] [avg(i_current_price),i_category,avg(UnscaledValue(i_current_price))] - CometExchange [i_category] #9 - CometHashAggregate [i_current_price] [i_category,sum,count] - CometProject [i_category] [i_current_price,i_category] - CometFilter [i_current_price,i_category] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/explain.txt deleted file mode 100644 index 135549db3d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/explain.txt +++ /dev/null @@ -1,1029 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (180) -+- CometSort (179) - +- CometExchange (178) - +- CometProject (177) - +- CometSortMergeJoin (176) - :- CometSort (105) - : +- CometExchange (104) - : +- CometHashAggregate (103) - : +- CometHashAggregate (102) - : +- CometProject (101) - : +- CometBroadcastHashJoin (100) - : :- CometProject (95) - : : +- CometBroadcastHashJoin (94) - : : :- CometProject (92) - : : : +- CometBroadcastHashJoin (91) - : : : :- CometProject (87) - : : : : +- CometBroadcastHashJoin (86) - : : : : :- CometProject (84) - : : : : : +- CometBroadcastHashJoin (83) - : : : : : :- CometProject (78) - : : : : : : +- CometBroadcastHashJoin (77) - : : : : : : :- CometProject (75) - : : : : : : : +- CometBroadcastHashJoin (74) - : : : : : : : :- CometProject (70) - : : : : : : : : +- CometBroadcastHashJoin (69) - : : : : : : : : :- CometProject (65) - : : : : : : : : : +- CometBroadcastHashJoin (64) - : : : : : : : : : :- CometProject (62) - : : : : : : : : : : +- CometBroadcastHashJoin (61) - : : : : : : : : : : :- CometProject (56) - : : : : : : : : : : : +- CometBroadcastHashJoin (55) - : : : : : : : : : : : :- CometProject (53) - : : : : : : : : : : : : +- CometBroadcastHashJoin (52) - : : : : : : : : : : : : :- CometProject (48) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (47) - : : : : : : : : : : : : : :- CometProject (43) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (42) - : : : : : : : : : : : : : : :- CometProject (37) - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (36) - : : : : : : : : : : : : : : : :- CometProject (32) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (31) - : : : : : : : : : : : : : : : : :- CometSort (12) - : : : : : : : : : : : : : : : : : +- CometColumnarExchange (11) - : : : : : : : : : : : : : : : : : +- * Project (10) - : : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (9) - : : : : : : : : : : : : : : : : : :- BroadcastExchange (4) - : : : : : : : : : : : : : : : : : : +- * Filter (3) - : : : : : : : : : : : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- * CometColumnarToRow (8) - : : : : : : : : : : : : : : : : : +- CometProject (7) - : : : : : : : : : : : : : : : : : +- CometFilter (6) - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (5) - : : : : : : : : : : : : : : : : +- CometSort (30) - : : : : : : : : : : : : : : : : +- CometProject (29) - : : : : : : : : : : : : : : : : +- CometFilter (28) - : : : : : : : : : : : : : : : : +- CometHashAggregate (27) - : : : : : : : : : : : : : : : : +- CometExchange (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometProject (24) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (23) - : : : : : : : : : : : : : : : : :- CometSort (17) - : : : : : : : : : : : : : : : : : +- CometExchange (16) - : : : : : : : : : : : : : : : : : +- CometProject (15) - : : : : : : : : : : : : : : : : : +- CometFilter (14) - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales (13) - : : : : : : : : : : : : : : : : +- CometSort (22) - : : : : : : : : : : : : : : : : +- CometExchange (21) - : : : : : : : : : : : : : : : : +- CometProject (20) - : : : : : : : : : : : : : : : : +- CometFilter (19) - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (18) - : : : : : : : : : : : : : : : +- CometBroadcastExchange (35) - : : : : : : : : : : : : : : : +- CometFilter (34) - : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (33) - : : : : : : : : : : : : : : +- CometBroadcastExchange (41) - : : : : : : : : : : : : : : +- CometProject (40) - : : : : : : : : : : : : : : +- CometFilter (39) - : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store (38) - : : : : : : : : : : : : : +- CometBroadcastExchange (46) - : : : : : : : : : : : : : +- CometFilter (45) - : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer (44) - : : : : : : : : : : : : +- CometBroadcastExchange (51) - : : : : : : : : : : : : +- CometFilter (50) - : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (49) - : : : : : : : : : : : +- ReusedExchange (54) - : : : : : : : : : : +- CometBroadcastExchange (60) - : : : : : : : : : : +- CometProject (59) - : : : : : : : : : : +- CometFilter (58) - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (57) - : : : : : : : : : +- ReusedExchange (63) - : : : : : : : : +- CometBroadcastExchange (68) - : : : : : : : : +- CometFilter (67) - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.promotion (66) - : : : : : : : +- CometBroadcastExchange (73) - : : : : : : : +- CometFilter (72) - : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (71) - : : : : : : +- ReusedExchange (76) - : : : : : +- CometBroadcastExchange (82) - : : : : : +- CometProject (81) - : : : : : +- CometFilter (80) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address (79) - : : : : +- ReusedExchange (85) - : : : +- CometBroadcastExchange (90) - : : : +- CometFilter (89) - : : : +- CometNativeScan parquet spark_catalog.default.income_band (88) - : : +- ReusedExchange (93) - : +- CometBroadcastExchange (99) - : +- CometProject (98) - : +- CometFilter (97) - : +- CometNativeScan parquet spark_catalog.default.item (96) - +- CometSort (175) - +- CometExchange (174) - +- CometHashAggregate (173) - +- CometHashAggregate (172) - +- CometProject (171) - +- CometBroadcastHashJoin (170) - :- CometProject (168) - : +- CometBroadcastHashJoin (167) - : :- CometProject (165) - : : +- CometBroadcastHashJoin (164) - : : :- CometProject (162) - : : : +- CometBroadcastHashJoin (161) - : : : :- CometProject (159) - : : : : +- CometBroadcastHashJoin (158) - : : : : :- CometProject (156) - : : : : : +- CometBroadcastHashJoin (155) - : : : : : :- CometProject (153) - : : : : : : +- CometBroadcastHashJoin (152) - : : : : : : :- CometProject (150) - : : : : : : : +- CometBroadcastHashJoin (149) - : : : : : : : :- CometProject (147) - : : : : : : : : +- CometBroadcastHashJoin (146) - : : : : : : : : :- CometProject (144) - : : : : : : : : : +- CometBroadcastHashJoin (143) - : : : : : : : : : :- CometProject (141) - : : : : : : : : : : +- CometBroadcastHashJoin (140) - : : : : : : : : : : :- CometProject (138) - : : : : : : : : : : : +- CometBroadcastHashJoin (137) - : : : : : : : : : : : :- CometProject (135) - : : : : : : : : : : : : +- CometBroadcastHashJoin (134) - : : : : : : : : : : : : :- CometProject (132) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (131) - : : : : : : : : : : : : : :- CometProject (129) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (128) - : : : : : : : : : : : : : : :- CometProject (124) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (123) - : : : : : : : : : : : : : : : :- CometSort (117) - : : : : : : : : : : : : : : : : +- CometColumnarExchange (116) - : : : : : : : : : : : : : : : : +- * Project (115) - : : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildLeft (114) - : : : : : : : : : : : : : : : : :- BroadcastExchange (109) - : : : : : : : : : : : : : : : : : +- * Filter (108) - : : : : : : : : : : : : : : : : : +- * ColumnarToRow (107) - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales (106) - : : : : : : : : : : : : : : : : +- * CometColumnarToRow (113) - : : : : : : : : : : : : : : : : +- CometProject (112) - : : : : : : : : : : : : : : : : +- CometFilter (111) - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (110) - : : : : : : : : : : : : : : : +- CometSort (122) - : : : : : : : : : : : : : : : +- CometProject (121) - : : : : : : : : : : : : : : : +- CometFilter (120) - : : : : : : : : : : : : : : : +- CometHashAggregate (119) - : : : : : : : : : : : : : : : +- ReusedExchange (118) - : : : : : : : : : : : : : : +- CometBroadcastExchange (127) - : : : : : : : : : : : : : : +- CometFilter (126) - : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (125) - : : : : : : : : : : : : : +- ReusedExchange (130) - : : : : : : : : : : : : +- ReusedExchange (133) - : : : : : : : : : : : +- ReusedExchange (136) - : : : : : : : : : : +- ReusedExchange (139) - : : : : : : : : : +- ReusedExchange (142) - : : : : : : : : +- ReusedExchange (145) - : : : : : : : +- ReusedExchange (148) - : : : : : : +- ReusedExchange (151) - : : : : : +- ReusedExchange (154) - : : : : +- ReusedExchange (157) - : : : +- ReusedExchange (160) - : : +- ReusedExchange (163) - : +- ReusedExchange (166) - +- ReusedExchange (169) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(3) Filter [codegen id : 1] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) - -(4) BroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=1] - -(5) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) - -(7) CometProject -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] - -(8) CometColumnarToRow -Input [2]: [sr_item_sk#14, sr_ticket_number#15] - -(9) BroadcastHashJoin [codegen id : 2] -Left keys [2]: [ss_item_sk#1, ss_ticket_number#8] -Right keys [2]: [sr_item_sk#14, sr_ticket_number#15] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 2] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] - -(11) CometColumnarExchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(12) CometSort -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] - -(13) CometNativeScan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] -ReadSchema: struct - -(14) CometFilter -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) - -(15) CometProject -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] - -(16) CometExchange -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(17) CometSort -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] - -(18) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(19) CometFilter -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) - -(20) CometProject -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(21) CometExchange -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(22) CometSort -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] - -(23) CometSortMergeJoin -Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner - -(24) CometProject -Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(25) CometHashAggregate -Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Keys [1]: [cs_item_sk#17] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(26) CometExchange -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(27) CometHashAggregate -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Keys [1]: [cs_item_sk#17] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(28) CometFilter -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(29) CometProject -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Arguments: [cs_item_sk#17], [cs_item_sk#17] - -(30) CometSort -Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] - -(31) CometSortMergeJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [1]: [cs_item_sk#17] -Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner - -(32) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(33) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(35) CometBroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: [d_date_sk#32, d_year#33] - -(36) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight - -(37) CometProject -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] - -(38) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name)] -ReadSchema: struct - -(39) CometFilter -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true))) - -(40) CometProject -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37], [s_store_sk#34, s_store_name#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true) AS s_zip#37] - -(41) CometBroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37] - -(42) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight - -(43) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] - -(44) CometNativeScan parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(45) CometFilter -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Condition : (((((isnotnull(c_customer_sk#38) AND isnotnull(c_first_sales_date_sk#43)) AND isnotnull(c_first_shipto_date_sk#42)) AND isnotnull(c_current_cdemo_sk#39)) AND isnotnull(c_current_hdemo_sk#40)) AND isnotnull(c_current_addr_sk#41)) - -(46) CometBroadcastExchange -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(47) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -Right output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_customer_sk#2], [c_customer_sk#38], Inner, BuildRight - -(48) CometProject -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(49) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(50) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : isnotnull(d_date_sk#44) - -(51) CometBroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] - -(52) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [c_first_sales_date_sk#43], [d_date_sk#44], Inner, BuildRight - -(53) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43, d_date_sk#44, d_year#45] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] - -(54) ReusedExchange [Reuses operator id: 51] -Output [2]: [d_date_sk#46, d_year#47] - -(55) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [c_first_shipto_date_sk#42], [d_date_sk#46], Inner, BuildRight - -(56) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45, d_date_sk#46, d_year#47] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(57) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#48, cd_marital_status#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(58) CometFilter -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true))) - -(59) CometProject -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Arguments: [cd_demo_sk#48, cd_marital_status#50], [cd_demo_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true) AS cd_marital_status#50] - -(60) CometBroadcastExchange -Input [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [cd_demo_sk#48, cd_marital_status#50] - -(61) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_cdemo_sk#3], [cd_demo_sk#48], Inner, BuildRight - -(62) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] - -(63) ReusedExchange [Reuses operator id: 60] -Output [2]: [cd_demo_sk#51, cd_marital_status#52] - -(64) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -Right output [2]: [cd_demo_sk#51, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#39], [cd_demo_sk#51], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(65) CometProject -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50, cd_demo_sk#51, cd_marital_status#52] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(66) CometNativeScan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(67) CometFilter -Input [1]: [p_promo_sk#53] -Condition : isnotnull(p_promo_sk#53) - -(68) CometBroadcastExchange -Input [1]: [p_promo_sk#53] -Arguments: [p_promo_sk#53] - -(69) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [1]: [p_promo_sk#53] -Arguments: [ss_promo_sk#7], [p_promo_sk#53], Inner, BuildRight - -(70) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, p_promo_sk#53] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(71) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(72) CometFilter -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Condition : (isnotnull(hd_demo_sk#54) AND isnotnull(hd_income_band_sk#55)) - -(73) CometBroadcastExchange -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [hd_demo_sk#54, hd_income_band_sk#55] - -(74) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_hdemo_sk#4], [hd_demo_sk#54], Inner, BuildRight - -(75) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] - -(76) ReusedExchange [Reuses operator id: 73] -Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] - -(77) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -Right output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [c_current_hdemo_sk#40], [hd_demo_sk#56], Inner, BuildRight - -(78) CometProject -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] - -(79) CometNativeScan parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(80) CometFilter -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Condition : isnotnull(ca_address_sk#58) - -(81) CometProject -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ca_address_sk#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#59, 10, true, false, true) AS ca_street_number#63, ca_street_name#60, ca_city#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#62, 10, true, false, true) AS ca_zip#64] - -(82) CometBroadcastExchange -Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(83) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -Right output [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_addr_sk#5], [ca_address_sk#58], Inner, BuildRight - -(84) CometProject -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(85) ReusedExchange [Reuses operator id: 82] -Output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(86) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Right output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [c_current_addr_sk#41], [ca_address_sk#65], Inner, BuildRight - -(87) CometProject -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(88) CometNativeScan parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(89) CometFilter -Input [1]: [ib_income_band_sk#70] -Condition : isnotnull(ib_income_band_sk#70) - -(90) CometBroadcastExchange -Input [1]: [ib_income_band_sk#70] -Arguments: [ib_income_band_sk#70] - -(91) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#70] -Arguments: [hd_income_band_sk#55], [ib_income_band_sk#70], Inner, BuildRight - -(92) CometProject -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#70] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(93) ReusedExchange [Reuses operator id: 90] -Output [1]: [ib_income_band_sk#71] - -(94) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#71] -Arguments: [hd_income_band_sk#57], [ib_income_band_sk#71], Inner, BuildRight - -(95) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#71] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(96) CometNativeScan parquet spark_catalog.default.item -Output [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(97) CometFilter -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Condition : ((((((isnotnull(i_current_price#73) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#74, 20, true, false, true) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#73 >= 64.00)) AND (i_current_price#73 <= 74.00)) AND (i_current_price#73 >= 65.00)) AND (i_current_price#73 <= 79.00)) AND isnotnull(i_item_sk#72)) - -(98) CometProject -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Arguments: [i_item_sk#72, i_product_name#76], [i_item_sk#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#75, 50, true, false, true) AS i_product_name#76] - -(99) CometBroadcastExchange -Input [2]: [i_item_sk#72, i_product_name#76] -Arguments: [i_item_sk#72, i_product_name#76] - -(100) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [2]: [i_item_sk#72, i_product_name#76] -Arguments: [ss_item_sk#1], [i_item_sk#72], Inner, BuildRight - -(101) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] - -(102) CometHashAggregate -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] - -(103) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#77, sum#78, sum#79, sum#80] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] - -(104) CometExchange -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: hashpartitioning(item_sk#82, store_name#83, store_zip#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(105) CometSort -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97], [item_sk#82 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, store_zip#84 ASC NULLS FIRST] - -(106) Scan parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#109), dynamicpruningexpression(ss_sold_date_sk#109 IN dynamicpruning#110)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(107) ColumnarToRow [codegen id : 3] -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(108) Filter [codegen id : 3] -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Condition : (((((((isnotnull(ss_item_sk#98) AND isnotnull(ss_ticket_number#105)) AND isnotnull(ss_store_sk#103)) AND isnotnull(ss_customer_sk#99)) AND isnotnull(ss_cdemo_sk#100)) AND isnotnull(ss_promo_sk#104)) AND isnotnull(ss_hdemo_sk#101)) AND isnotnull(ss_addr_sk#102)) - -(109) BroadcastExchange -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=7] - -(110) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(111) CometFilter -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) - -(112) CometProject -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] - -(113) CometColumnarToRow -Input [2]: [sr_item_sk#111, sr_ticket_number#112] - -(114) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [ss_item_sk#98, ss_ticket_number#105] -Right keys [2]: [sr_item_sk#111, sr_ticket_number#112] -Join type: Inner -Join condition: None - -(115) Project [codegen id : 4] -Output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, sr_item_sk#111, sr_ticket_number#112] - -(116) CometColumnarExchange -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: hashpartitioning(ss_item_sk#98, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(117) CometSort -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98 ASC NULLS FIRST] - -(118) ReusedExchange [Reuses operator id: 26] -Output [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] - -(119) CometHashAggregate -Input [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] -Keys [1]: [cs_item_sk#114] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#118)), sum(((cr_refunded_cash#119 + cr_reversed_charge#120) + cr_store_credit#121))] - -(120) CometFilter -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(121) CometProject -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Arguments: [cs_item_sk#114], [cs_item_sk#114] - -(122) CometSort -Input [1]: [cs_item_sk#114] -Arguments: [cs_item_sk#114], [cs_item_sk#114 ASC NULLS FIRST] - -(123) CometSortMergeJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [1]: [cs_item_sk#114] -Arguments: [ss_item_sk#98], [cs_item_sk#114], Inner - -(124) CometProject -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, cs_item_sk#114] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(125) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(127) CometBroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: [d_date_sk#122, d_year#123] - -(128) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [d_date_sk#122, d_year#123] -Arguments: [ss_sold_date_sk#109], [d_date_sk#122], Inner, BuildRight - -(129) CometProject -Input [13]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, d_date_sk#122, d_year#123] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] - -(130) ReusedExchange [Reuses operator id: 41] -Output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] - -(131) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] -Right output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_store_sk#103], [s_store_sk#124], Inner, BuildRight - -(132) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] - -(133) ReusedExchange [Reuses operator id: 46] -Output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(134) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] -Right output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_customer_sk#99], [c_customer_sk#126], Inner, BuildRight - -(135) CometProject -Input [18]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(136) ReusedExchange [Reuses operator id: 51] -Output [2]: [d_date_sk#132, d_year#133] - -(137) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Right output [2]: [d_date_sk#132, d_year#133] -Arguments: [c_first_sales_date_sk#131], [d_date_sk#132], Inner, BuildRight - -(138) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131, d_date_sk#132, d_year#133] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] - -(139) ReusedExchange [Reuses operator id: 51] -Output [2]: [d_date_sk#134, d_year#135] - -(140) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] -Right output [2]: [d_date_sk#134, d_year#135] -Arguments: [c_first_shipto_date_sk#130], [d_date_sk#134], Inner, BuildRight - -(141) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133, d_date_sk#134, d_year#135] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(142) ReusedExchange [Reuses operator id: 60] -Output [2]: [cd_demo_sk#136, cd_marital_status#50] - -(143) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_cdemo_sk#100], [cd_demo_sk#136], Inner, BuildRight - -(144) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] - -(145) ReusedExchange [Reuses operator id: 60] -Output [2]: [cd_demo_sk#137, cd_marital_status#52] - -(146) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] -Right output [2]: [cd_demo_sk#137, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#127], [cd_demo_sk#137], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(147) CometProject -Input [18]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50, cd_demo_sk#137, cd_marital_status#52] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(148) ReusedExchange [Reuses operator id: 68] -Output [1]: [p_promo_sk#138] - -(149) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [1]: [p_promo_sk#138] -Arguments: [ss_promo_sk#104], [p_promo_sk#138], Inner, BuildRight - -(150) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, p_promo_sk#138] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(151) ReusedExchange [Reuses operator id: 73] -Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] - -(152) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_hdemo_sk#101], [hd_demo_sk#139], Inner, BuildRight - -(153) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] - -(154) ReusedExchange [Reuses operator id: 73] -Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] - -(155) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] -Right output [2]: [hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [c_current_hdemo_sk#128], [hd_demo_sk#141], Inner, BuildRight - -(156) CometProject -Input [15]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] - -(157) ReusedExchange [Reuses operator id: 82] -Output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(158) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] -Right output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_addr_sk#102], [ca_address_sk#143], Inner, BuildRight - -(159) CometProject -Input [18]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(160) ReusedExchange [Reuses operator id: 82] -Output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(161) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Right output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [c_current_addr_sk#129], [ca_address_sk#146], Inner, BuildRight - -(162) CometProject -Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(163) ReusedExchange [Reuses operator id: 90] -Output [1]: [ib_income_band_sk#149] - -(164) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#149] -Arguments: [hd_income_band_sk#140], [ib_income_band_sk#149], Inner, BuildRight - -(165) CometProject -Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#149] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(166) ReusedExchange [Reuses operator id: 90] -Output [1]: [ib_income_band_sk#150] - -(167) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#150] -Arguments: [hd_income_band_sk#142], [ib_income_band_sk#150], Inner, BuildRight - -(168) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#150] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(169) ReusedExchange [Reuses operator id: 99] -Output [2]: [i_item_sk#151, i_product_name#76] - -(170) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [2]: [i_item_sk#151, i_product_name#76] -Arguments: [ss_item_sk#98], [i_item_sk#151], Inner, BuildRight - -(171) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] - -(172) CometHashAggregate -Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#106)), partial_sum(UnscaledValue(ss_list_price#107)), partial_sum(UnscaledValue(ss_coupon_amt#108))] - -(173) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135, count#77, sum#152, sum#153, sum#154] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#106)), sum(UnscaledValue(ss_list_price#107)), sum(UnscaledValue(ss_coupon_amt#108))] - -(174) CometExchange -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: hashpartitioning(item_sk#155, store_name#156, store_zip#157, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(175) CometSort -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162], [item_sk#155 ASC NULLS FIRST, store_name#156 ASC NULLS FIRST, store_zip#157 ASC NULLS FIRST] - -(176) CometSortMergeJoin -Left output [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Right output [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#156, store_zip#157], Inner, (cnt#159 <= cnt#94) - -(177) CometProject -Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -(178) CometExchange -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(179) CometSort -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST] - -(180) CometColumnarToRow [codegen id : 5] -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (184) -+- * CometColumnarToRow (183) - +- CometFilter (182) - +- CometNativeScan parquet spark_catalog.default.date_dim (181) - - -(181) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(182) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(183) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#33] - -(184) BroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] - -Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 -BroadcastExchange (188) -+- * CometColumnarToRow (187) - +- CometFilter (186) - +- CometNativeScan parquet spark_catalog.default.date_dim (185) - - -(185) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(186) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(187) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#122, d_year#123] - -(188) BroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/simplified.txt deleted file mode 100644 index d933995ffc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/simplified.txt +++ /dev/null @@ -1,206 +0,0 @@ -WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometExchange [product_name,store_name,cnt,s1,s1] #1 - CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometColumnarExchange [ss_item_sk] #3 - WholeStageCodegen (2) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometNativeScan 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] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [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] #11 - 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] - CometNativeScan 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] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometNativeScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometNativeScan parquet spark_catalog.default.income_band [ib_income_band_sk] - ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometColumnarExchange [ss_item_sk] #20 - WholeStageCodegen (4) - Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] - BroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] - InputAdapter - BroadcastExchange #21 - WholeStageCodegen (3) - Filter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet 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 #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometColumnarToRow - InputAdapter - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - CometBroadcastExchange [d_date_sk,d_year] #23 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/explain.txt deleted file mode 100644 index 077daeca3a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,1011 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (176) -+- CometSort (175) - +- CometExchange (174) - +- CometProject (173) - +- CometSortMergeJoin (172) - :- CometSort (103) - : +- CometExchange (102) - : +- CometHashAggregate (101) - : +- CometHashAggregate (100) - : +- CometProject (99) - : +- CometBroadcastHashJoin (98) - : :- CometProject (93) - : : +- CometBroadcastHashJoin (92) - : : :- CometProject (90) - : : : +- CometBroadcastHashJoin (89) - : : : :- CometProject (85) - : : : : +- CometBroadcastHashJoin (84) - : : : : :- CometProject (82) - : : : : : +- CometBroadcastHashJoin (81) - : : : : : :- CometProject (76) - : : : : : : +- CometBroadcastHashJoin (75) - : : : : : : :- CometProject (73) - : : : : : : : +- CometBroadcastHashJoin (72) - : : : : : : : :- CometProject (68) - : : : : : : : : +- CometBroadcastHashJoin (67) - : : : : : : : : :- CometProject (63) - : : : : : : : : : +- CometBroadcastHashJoin (62) - : : : : : : : : : :- CometProject (60) - : : : : : : : : : : +- CometBroadcastHashJoin (59) - : : : : : : : : : : :- CometProject (54) - : : : : : : : : : : : +- CometBroadcastHashJoin (53) - : : : : : : : : : : : :- CometProject (51) - : : : : : : : : : : : : +- CometBroadcastHashJoin (50) - : : : : : : : : : : : : :- CometProject (46) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (45) - : : : : : : : : : : : : : :- CometProject (41) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (40) - : : : : : : : : : : : : : : :- CometProject (35) - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (34) - : : : : : : : : : : : : : : : :- CometProject (30) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) - : : : : : : : : : : : : : : : : :- CometSort (10) - : : : : : : : : : : : : : : : : : +- CometExchange (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- CometSort (28) - : : : : : : : : : : : : : : : : +- CometProject (27) - : : : : : : : : : : : : : : : : +- CometFilter (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometExchange (24) - : : : : : : : : : : : : : : : : +- CometHashAggregate (23) - : : : : : : : : : : : : : : : : +- CometProject (22) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) - : : : : : : : : : : : : : : : : :- CometSort (15) - : : : : : : : : : : : : : : : : : +- CometExchange (14) - : : : : : : : : : : : : : : : : : +- CometProject (13) - : : : : : : : : : : : : : : : : : +- CometFilter (12) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (11) - : : : : : : : : : : : : : : : : +- CometSort (20) - : : : : : : : : : : : : : : : : +- CometExchange (19) - : : : : : : : : : : : : : : : : +- CometProject (18) - : : : : : : : : : : : : : : : : +- CometFilter (17) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : : : : : : : : : : : : : : : +- CometBroadcastExchange (33) - : : : : : : : : : : : : : : : +- CometFilter (32) - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - : : : : : : : : : : : : : : +- CometBroadcastExchange (39) - : : : : : : : : : : : : : : +- CometProject (38) - : : : : : : : : : : : : : : +- CometFilter (37) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (36) - : : : : : : : : : : : : : +- CometBroadcastExchange (44) - : : : : : : : : : : : : : +- CometFilter (43) - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (42) - : : : : : : : : : : : : +- CometBroadcastExchange (49) - : : : : : : : : : : : : +- CometFilter (48) - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) - : : : : : : : : : : : +- ReusedExchange (52) - : : : : : : : : : : +- CometBroadcastExchange (58) - : : : : : : : : : : +- CometProject (57) - : : : : : : : : : : +- CometFilter (56) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (55) - : : : : : : : : : +- ReusedExchange (61) - : : : : : : : : +- CometBroadcastExchange (66) - : : : : : : : : +- CometFilter (65) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (64) - : : : : : : : +- CometBroadcastExchange (71) - : : : : : : : +- CometFilter (70) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (69) - : : : : : : +- ReusedExchange (74) - : : : : : +- CometBroadcastExchange (80) - : : : : : +- CometProject (79) - : : : : : +- CometFilter (78) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (77) - : : : : +- ReusedExchange (83) - : : : +- CometBroadcastExchange (88) - : : : +- CometFilter (87) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (86) - : : +- ReusedExchange (91) - : +- CometBroadcastExchange (97) - : +- CometProject (96) - : +- CometFilter (95) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (94) - +- CometSort (171) - +- CometExchange (170) - +- CometHashAggregate (169) - +- CometHashAggregate (168) - +- CometProject (167) - +- CometBroadcastHashJoin (166) - :- CometProject (164) - : +- CometBroadcastHashJoin (163) - : :- CometProject (161) - : : +- CometBroadcastHashJoin (160) - : : :- CometProject (158) - : : : +- CometBroadcastHashJoin (157) - : : : :- CometProject (155) - : : : : +- CometBroadcastHashJoin (154) - : : : : :- CometProject (152) - : : : : : +- CometBroadcastHashJoin (151) - : : : : : :- CometProject (149) - : : : : : : +- CometBroadcastHashJoin (148) - : : : : : : :- CometProject (146) - : : : : : : : +- CometBroadcastHashJoin (145) - : : : : : : : :- CometProject (143) - : : : : : : : : +- CometBroadcastHashJoin (142) - : : : : : : : : :- CometProject (140) - : : : : : : : : : +- CometBroadcastHashJoin (139) - : : : : : : : : : :- CometProject (137) - : : : : : : : : : : +- CometBroadcastHashJoin (136) - : : : : : : : : : : :- CometProject (134) - : : : : : : : : : : : +- CometBroadcastHashJoin (133) - : : : : : : : : : : : :- CometProject (131) - : : : : : : : : : : : : +- CometBroadcastHashJoin (130) - : : : : : : : : : : : : :- CometProject (128) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (127) - : : : : : : : : : : : : : :- CometProject (125) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (124) - : : : : : : : : : : : : : : :- CometProject (120) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (119) - : : : : : : : : : : : : : : : :- CometSort (113) - : : : : : : : : : : : : : : : : +- CometExchange (112) - : : : : : : : : : : : : : : : : +- CometProject (111) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (110) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (106) - : : : : : : : : : : : : : : : : : +- CometFilter (105) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (104) - : : : : : : : : : : : : : : : : +- CometProject (109) - : : : : : : : : : : : : : : : : +- CometFilter (108) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (107) - : : : : : : : : : : : : : : : +- CometSort (118) - : : : : : : : : : : : : : : : +- CometProject (117) - : : : : : : : : : : : : : : : +- CometFilter (116) - : : : : : : : : : : : : : : : +- CometHashAggregate (115) - : : : : : : : : : : : : : : : +- ReusedExchange (114) - : : : : : : : : : : : : : : +- CometBroadcastExchange (123) - : : : : : : : : : : : : : : +- CometFilter (122) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (121) - : : : : : : : : : : : : : +- ReusedExchange (126) - : : : : : : : : : : : : +- ReusedExchange (129) - : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : +- ReusedExchange (141) - : : : : : : : +- ReusedExchange (144) - : : : : : : +- ReusedExchange (147) - : : : : : +- ReusedExchange (150) - : : : : +- ReusedExchange (153) - : : : +- ReusedExchange (156) - : : +- ReusedExchange (159) - : +- ReusedExchange (162) - +- ReusedExchange (165) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) CometFilter -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) - -(3) CometBroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) - -(6) CometProject -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] - -(7) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner, BuildLeft - -(8) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(9) CometExchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometSort -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) - -(13) CometProject -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] - -(14) CometExchange -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(15) CometSort -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(17) CometFilter -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) - -(18) CometProject -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(19) CometExchange -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner - -(22) CometProject -Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(23) CometHashAggregate -Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Keys [1]: [cs_item_sk#17] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(24) CometExchange -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometHashAggregate -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Keys [1]: [cs_item_sk#17] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(26) CometFilter -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(27) CometProject -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Arguments: [cs_item_sk#17], [cs_item_sk#17] - -(28) CometSort -Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] - -(29) CometSortMergeJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [1]: [cs_item_sk#17] -Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner - -(30) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(33) CometBroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: [d_date_sk#32, d_year#33] - -(34) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight - -(35) CometProject -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true))) - -(38) CometProject -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37], [s_store_sk#34, s_store_name#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true) AS s_zip#37] - -(39) CometBroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37] - -(40) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight - -(41) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(43) CometFilter -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Condition : (((((isnotnull(c_customer_sk#38) AND isnotnull(c_first_sales_date_sk#43)) AND isnotnull(c_first_shipto_date_sk#42)) AND isnotnull(c_current_cdemo_sk#39)) AND isnotnull(c_current_hdemo_sk#40)) AND isnotnull(c_current_addr_sk#41)) - -(44) CometBroadcastExchange -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(45) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -Right output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_customer_sk#2], [c_customer_sk#38], Inner, BuildRight - -(46) CometProject -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(48) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : isnotnull(d_date_sk#44) - -(49) CometBroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] - -(50) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [c_first_sales_date_sk#43], [d_date_sk#44], Inner, BuildRight - -(51) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43, d_date_sk#44, d_year#45] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] - -(52) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#46, d_year#47] - -(53) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [c_first_shipto_date_sk#42], [d_date_sk#46], Inner, BuildRight - -(54) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45, d_date_sk#46, d_year#47] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#48, cd_marital_status#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true))) - -(57) CometProject -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Arguments: [cd_demo_sk#48, cd_marital_status#50], [cd_demo_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true) AS cd_marital_status#50] - -(58) CometBroadcastExchange -Input [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [cd_demo_sk#48, cd_marital_status#50] - -(59) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_cdemo_sk#3], [cd_demo_sk#48], Inner, BuildRight - -(60) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] - -(61) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#51, cd_marital_status#52] - -(62) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -Right output [2]: [cd_demo_sk#51, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#39], [cd_demo_sk#51], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(63) CometProject -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50, cd_demo_sk#51, cd_marital_status#52] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(65) CometFilter -Input [1]: [p_promo_sk#53] -Condition : isnotnull(p_promo_sk#53) - -(66) CometBroadcastExchange -Input [1]: [p_promo_sk#53] -Arguments: [p_promo_sk#53] - -(67) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [1]: [p_promo_sk#53] -Arguments: [ss_promo_sk#7], [p_promo_sk#53], Inner, BuildRight - -(68) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, p_promo_sk#53] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(70) CometFilter -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Condition : (isnotnull(hd_demo_sk#54) AND isnotnull(hd_income_band_sk#55)) - -(71) CometBroadcastExchange -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [hd_demo_sk#54, hd_income_band_sk#55] - -(72) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_hdemo_sk#4], [hd_demo_sk#54], Inner, BuildRight - -(73) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] - -(74) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] - -(75) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -Right output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [c_current_hdemo_sk#40], [hd_demo_sk#56], Inner, BuildRight - -(76) CometProject -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(78) CometFilter -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Condition : isnotnull(ca_address_sk#58) - -(79) CometProject -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ca_address_sk#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#59, 10, true, false, true) AS ca_street_number#63, ca_street_name#60, ca_city#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#62, 10, true, false, true) AS ca_zip#64] - -(80) CometBroadcastExchange -Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(81) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -Right output [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_addr_sk#5], [ca_address_sk#58], Inner, BuildRight - -(82) CometProject -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(83) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(84) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Right output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [c_current_addr_sk#41], [ca_address_sk#65], Inner, BuildRight - -(85) CometProject -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(87) CometFilter -Input [1]: [ib_income_band_sk#70] -Condition : isnotnull(ib_income_band_sk#70) - -(88) CometBroadcastExchange -Input [1]: [ib_income_band_sk#70] -Arguments: [ib_income_band_sk#70] - -(89) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#70] -Arguments: [hd_income_band_sk#55], [ib_income_band_sk#70], Inner, BuildRight - -(90) CometProject -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#70] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(91) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#71] - -(92) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#71] -Arguments: [hd_income_band_sk#57], [ib_income_band_sk#71], Inner, BuildRight - -(93) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#71] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(95) CometFilter -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Condition : ((((((isnotnull(i_current_price#73) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#74, 20, true, false, true) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#73 >= 64.00)) AND (i_current_price#73 <= 74.00)) AND (i_current_price#73 >= 65.00)) AND (i_current_price#73 <= 79.00)) AND isnotnull(i_item_sk#72)) - -(96) CometProject -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Arguments: [i_item_sk#72, i_product_name#76], [i_item_sk#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#75, 50, true, false, true) AS i_product_name#76] - -(97) CometBroadcastExchange -Input [2]: [i_item_sk#72, i_product_name#76] -Arguments: [i_item_sk#72, i_product_name#76] - -(98) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [2]: [i_item_sk#72, i_product_name#76] -Arguments: [ss_item_sk#1], [i_item_sk#72], Inner, BuildRight - -(99) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] - -(100) CometHashAggregate -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] - -(101) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#77, sum#78, sum#79, sum#80] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] - -(102) CometExchange -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: hashpartitioning(item_sk#82, store_name#83, store_zip#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(103) CometSort -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97], [item_sk#82 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, store_zip#84 ASC NULLS FIRST] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#109), dynamicpruningexpression(ss_sold_date_sk#109 IN dynamicpruning#110)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(105) CometFilter -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Condition : (((((((isnotnull(ss_item_sk#98) AND isnotnull(ss_ticket_number#105)) AND isnotnull(ss_store_sk#103)) AND isnotnull(ss_customer_sk#99)) AND isnotnull(ss_cdemo_sk#100)) AND isnotnull(ss_promo_sk#104)) AND isnotnull(ss_hdemo_sk#101)) AND isnotnull(ss_addr_sk#102)) - -(106) CometBroadcastExchange -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(108) CometFilter -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) - -(109) CometProject -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] - -(110) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_ticket_number#105], [sr_item_sk#111, sr_ticket_number#112], Inner, BuildLeft - -(111) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(112) CometExchange -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: hashpartitioning(ss_item_sk#98, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(113) CometSort -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98 ASC NULLS FIRST] - -(114) ReusedExchange [Reuses operator id: 24] -Output [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] - -(115) CometHashAggregate -Input [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] -Keys [1]: [cs_item_sk#114] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#118)), sum(((cr_refunded_cash#119 + cr_reversed_charge#120) + cr_store_credit#121))] - -(116) CometFilter -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(117) CometProject -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Arguments: [cs_item_sk#114], [cs_item_sk#114] - -(118) CometSort -Input [1]: [cs_item_sk#114] -Arguments: [cs_item_sk#114], [cs_item_sk#114 ASC NULLS FIRST] - -(119) CometSortMergeJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [1]: [cs_item_sk#114] -Arguments: [ss_item_sk#98], [cs_item_sk#114], Inner - -(120) CometProject -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, cs_item_sk#114] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(121) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(122) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(123) CometBroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: [d_date_sk#122, d_year#123] - -(124) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [d_date_sk#122, d_year#123] -Arguments: [ss_sold_date_sk#109], [d_date_sk#122], Inner, BuildRight - -(125) CometProject -Input [13]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, d_date_sk#122, d_year#123] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] - -(126) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] - -(127) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] -Right output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_store_sk#103], [s_store_sk#124], Inner, BuildRight - -(128) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] - -(129) ReusedExchange [Reuses operator id: 44] -Output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(130) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] -Right output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_customer_sk#99], [c_customer_sk#126], Inner, BuildRight - -(131) CometProject -Input [18]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(132) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#132, d_year#133] - -(133) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Right output [2]: [d_date_sk#132, d_year#133] -Arguments: [c_first_sales_date_sk#131], [d_date_sk#132], Inner, BuildRight - -(134) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131, d_date_sk#132, d_year#133] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] - -(135) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#134, d_year#135] - -(136) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] -Right output [2]: [d_date_sk#134, d_year#135] -Arguments: [c_first_shipto_date_sk#130], [d_date_sk#134], Inner, BuildRight - -(137) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133, d_date_sk#134, d_year#135] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(138) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#136, cd_marital_status#50] - -(139) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_cdemo_sk#100], [cd_demo_sk#136], Inner, BuildRight - -(140) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] - -(141) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#137, cd_marital_status#52] - -(142) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] -Right output [2]: [cd_demo_sk#137, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#127], [cd_demo_sk#137], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(143) CometProject -Input [18]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50, cd_demo_sk#137, cd_marital_status#52] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(144) ReusedExchange [Reuses operator id: 66] -Output [1]: [p_promo_sk#138] - -(145) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [1]: [p_promo_sk#138] -Arguments: [ss_promo_sk#104], [p_promo_sk#138], Inner, BuildRight - -(146) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, p_promo_sk#138] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(147) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] - -(148) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_hdemo_sk#101], [hd_demo_sk#139], Inner, BuildRight - -(149) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] - -(150) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] - -(151) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] -Right output [2]: [hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [c_current_hdemo_sk#128], [hd_demo_sk#141], Inner, BuildRight - -(152) CometProject -Input [15]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] - -(153) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(154) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] -Right output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_addr_sk#102], [ca_address_sk#143], Inner, BuildRight - -(155) CometProject -Input [18]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(156) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(157) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Right output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [c_current_addr_sk#129], [ca_address_sk#146], Inner, BuildRight - -(158) CometProject -Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(159) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#149] - -(160) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#149] -Arguments: [hd_income_band_sk#140], [ib_income_band_sk#149], Inner, BuildRight - -(161) CometProject -Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#149] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(162) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#150] - -(163) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#150] -Arguments: [hd_income_band_sk#142], [ib_income_band_sk#150], Inner, BuildRight - -(164) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#150] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(165) ReusedExchange [Reuses operator id: 97] -Output [2]: [i_item_sk#151, i_product_name#76] - -(166) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [2]: [i_item_sk#151, i_product_name#76] -Arguments: [ss_item_sk#98], [i_item_sk#151], Inner, BuildRight - -(167) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] - -(168) CometHashAggregate -Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#106)), partial_sum(UnscaledValue(ss_list_price#107)), partial_sum(UnscaledValue(ss_coupon_amt#108))] - -(169) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135, count#77, sum#152, sum#153, sum#154] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#106)), sum(UnscaledValue(ss_list_price#107)), sum(UnscaledValue(ss_coupon_amt#108))] - -(170) CometExchange -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: hashpartitioning(item_sk#155, store_name#156, store_zip#157, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(171) CometSort -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162], [item_sk#155 ASC NULLS FIRST, store_name#156 ASC NULLS FIRST, store_zip#157 ASC NULLS FIRST] - -(172) CometSortMergeJoin -Left output [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Right output [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#156, store_zip#157], Inner, (cnt#159 <= cnt#94) - -(173) CometProject -Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -(174) CometExchange -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(175) CometSort -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST] - -(176) CometColumnarToRow [codegen id : 1] -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (180) -+- * CometColumnarToRow (179) - +- CometFilter (178) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) - - -(177) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(178) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(179) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#33] - -(180) BroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 -BroadcastExchange (184) -+- * CometColumnarToRow (183) - +- CometFilter (182) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) - - -(181) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(182) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(183) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#122, d_year#123] - -(184) BroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/simplified.txt deleted file mode 100644 index 46fe063430..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,192 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometExchange [product_name,store_name,cnt,s1,s1] #1 - CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #3 - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [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] #11 - 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 [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk] - ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #20 - 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_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] #21 - 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 [native_iceberg_compat] 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 #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - CometBroadcastExchange [d_date_sk,d_year] #23 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt deleted file mode 100644 index 077daeca3a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt +++ /dev/null @@ -1,1011 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (176) -+- CometSort (175) - +- CometExchange (174) - +- CometProject (173) - +- CometSortMergeJoin (172) - :- CometSort (103) - : +- CometExchange (102) - : +- CometHashAggregate (101) - : +- CometHashAggregate (100) - : +- CometProject (99) - : +- CometBroadcastHashJoin (98) - : :- CometProject (93) - : : +- CometBroadcastHashJoin (92) - : : :- CometProject (90) - : : : +- CometBroadcastHashJoin (89) - : : : :- CometProject (85) - : : : : +- CometBroadcastHashJoin (84) - : : : : :- CometProject (82) - : : : : : +- CometBroadcastHashJoin (81) - : : : : : :- CometProject (76) - : : : : : : +- CometBroadcastHashJoin (75) - : : : : : : :- CometProject (73) - : : : : : : : +- CometBroadcastHashJoin (72) - : : : : : : : :- CometProject (68) - : : : : : : : : +- CometBroadcastHashJoin (67) - : : : : : : : : :- CometProject (63) - : : : : : : : : : +- CometBroadcastHashJoin (62) - : : : : : : : : : :- CometProject (60) - : : : : : : : : : : +- CometBroadcastHashJoin (59) - : : : : : : : : : : :- CometProject (54) - : : : : : : : : : : : +- CometBroadcastHashJoin (53) - : : : : : : : : : : : :- CometProject (51) - : : : : : : : : : : : : +- CometBroadcastHashJoin (50) - : : : : : : : : : : : : :- CometProject (46) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (45) - : : : : : : : : : : : : : :- CometProject (41) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (40) - : : : : : : : : : : : : : : :- CometProject (35) - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (34) - : : : : : : : : : : : : : : : :- CometProject (30) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (29) - : : : : : : : : : : : : : : : : :- CometSort (10) - : : : : : : : : : : : : : : : : : +- CometExchange (9) - : : : : : : : : : : : : : : : : : +- CometProject (8) - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) - : : : : : : : : : : : : : : : : : : +- CometFilter (2) - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : : : : : : : : : : : : +- CometProject (6) - : : : : : : : : : : : : : : : : : +- CometFilter (5) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : : : : : : : : : : : : : : +- CometSort (28) - : : : : : : : : : : : : : : : : +- CometProject (27) - : : : : : : : : : : : : : : : : +- CometFilter (26) - : : : : : : : : : : : : : : : : +- CometHashAggregate (25) - : : : : : : : : : : : : : : : : +- CometExchange (24) - : : : : : : : : : : : : : : : : +- CometHashAggregate (23) - : : : : : : : : : : : : : : : : +- CometProject (22) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (21) - : : : : : : : : : : : : : : : : :- CometSort (15) - : : : : : : : : : : : : : : : : : +- CometExchange (14) - : : : : : : : : : : : : : : : : : +- CometProject (13) - : : : : : : : : : : : : : : : : : +- CometFilter (12) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (11) - : : : : : : : : : : : : : : : : +- CometSort (20) - : : : : : : : : : : : : : : : : +- CometExchange (19) - : : : : : : : : : : : : : : : : +- CometProject (18) - : : : : : : : : : : : : : : : : +- CometFilter (17) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : : : : : : : : : : : : : : : +- CometBroadcastExchange (33) - : : : : : : : : : : : : : : : +- CometFilter (32) - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) - : : : : : : : : : : : : : : +- CometBroadcastExchange (39) - : : : : : : : : : : : : : : +- CometProject (38) - : : : : : : : : : : : : : : +- CometFilter (37) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (36) - : : : : : : : : : : : : : +- CometBroadcastExchange (44) - : : : : : : : : : : : : : +- CometFilter (43) - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (42) - : : : : : : : : : : : : +- CometBroadcastExchange (49) - : : : : : : : : : : : : +- CometFilter (48) - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (47) - : : : : : : : : : : : +- ReusedExchange (52) - : : : : : : : : : : +- CometBroadcastExchange (58) - : : : : : : : : : : +- CometProject (57) - : : : : : : : : : : +- CometFilter (56) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (55) - : : : : : : : : : +- ReusedExchange (61) - : : : : : : : : +- CometBroadcastExchange (66) - : : : : : : : : +- CometFilter (65) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (64) - : : : : : : : +- CometBroadcastExchange (71) - : : : : : : : +- CometFilter (70) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (69) - : : : : : : +- ReusedExchange (74) - : : : : : +- CometBroadcastExchange (80) - : : : : : +- CometProject (79) - : : : : : +- CometFilter (78) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (77) - : : : : +- ReusedExchange (83) - : : : +- CometBroadcastExchange (88) - : : : +- CometFilter (87) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (86) - : : +- ReusedExchange (91) - : +- CometBroadcastExchange (97) - : +- CometProject (96) - : +- CometFilter (95) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (94) - +- CometSort (171) - +- CometExchange (170) - +- CometHashAggregate (169) - +- CometHashAggregate (168) - +- CometProject (167) - +- CometBroadcastHashJoin (166) - :- CometProject (164) - : +- CometBroadcastHashJoin (163) - : :- CometProject (161) - : : +- CometBroadcastHashJoin (160) - : : :- CometProject (158) - : : : +- CometBroadcastHashJoin (157) - : : : :- CometProject (155) - : : : : +- CometBroadcastHashJoin (154) - : : : : :- CometProject (152) - : : : : : +- CometBroadcastHashJoin (151) - : : : : : :- CometProject (149) - : : : : : : +- CometBroadcastHashJoin (148) - : : : : : : :- CometProject (146) - : : : : : : : +- CometBroadcastHashJoin (145) - : : : : : : : :- CometProject (143) - : : : : : : : : +- CometBroadcastHashJoin (142) - : : : : : : : : :- CometProject (140) - : : : : : : : : : +- CometBroadcastHashJoin (139) - : : : : : : : : : :- CometProject (137) - : : : : : : : : : : +- CometBroadcastHashJoin (136) - : : : : : : : : : : :- CometProject (134) - : : : : : : : : : : : +- CometBroadcastHashJoin (133) - : : : : : : : : : : : :- CometProject (131) - : : : : : : : : : : : : +- CometBroadcastHashJoin (130) - : : : : : : : : : : : : :- CometProject (128) - : : : : : : : : : : : : : +- CometBroadcastHashJoin (127) - : : : : : : : : : : : : : :- CometProject (125) - : : : : : : : : : : : : : : +- CometBroadcastHashJoin (124) - : : : : : : : : : : : : : : :- CometProject (120) - : : : : : : : : : : : : : : : +- CometSortMergeJoin (119) - : : : : : : : : : : : : : : : :- CometSort (113) - : : : : : : : : : : : : : : : : +- CometExchange (112) - : : : : : : : : : : : : : : : : +- CometProject (111) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (110) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (106) - : : : : : : : : : : : : : : : : : +- CometFilter (105) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (104) - : : : : : : : : : : : : : : : : +- CometProject (109) - : : : : : : : : : : : : : : : : +- CometFilter (108) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (107) - : : : : : : : : : : : : : : : +- CometSort (118) - : : : : : : : : : : : : : : : +- CometProject (117) - : : : : : : : : : : : : : : : +- CometFilter (116) - : : : : : : : : : : : : : : : +- CometHashAggregate (115) - : : : : : : : : : : : : : : : +- ReusedExchange (114) - : : : : : : : : : : : : : : +- CometBroadcastExchange (123) - : : : : : : : : : : : : : : +- CometFilter (122) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (121) - : : : : : : : : : : : : : +- ReusedExchange (126) - : : : : : : : : : : : : +- ReusedExchange (129) - : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : +- ReusedExchange (141) - : : : : : : : +- ReusedExchange (144) - : : : : : : +- ReusedExchange (147) - : : : : : +- ReusedExchange (150) - : : : : +- ReusedExchange (153) - : : : +- ReusedExchange (156) - : : +- ReusedExchange (159) - : +- ReusedExchange (162) - +- ReusedExchange (165) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(2) CometFilter -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) - -(3) CometBroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) - -(6) CometProject -Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] -Arguments: [sr_item_sk#14, sr_ticket_number#15], [sr_item_sk#14, sr_ticket_number#15] - -(7) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner, BuildLeft - -(8) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(9) CometExchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(10) CometSort -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1 ASC NULLS FIRST] - -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] -ReadSchema: struct - -(12) CometFilter -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) - -(13) CometProject -Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] - -(14) CometExchange -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(15) CometSort -Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(17) CometFilter -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) - -(18) CometProject -Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(19) CometExchange -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] -Right output [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_order_number#18], [cr_item_sk#21, cr_order_number#22], Inner - -(22) CometProject -Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Arguments: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] - -(23) CometHashAggregate -Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] -Keys [1]: [cs_item_sk#17] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(24) CometExchange -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometHashAggregate -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] -Keys [1]: [cs_item_sk#17] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] - -(26) CometFilter -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(27) CometProject -Input [3]: [cs_item_sk#17, sale#30, refund#31] -Arguments: [cs_item_sk#17], [cs_item_sk#17] - -(28) CometSort -Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] - -(29) CometSortMergeJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [1]: [cs_item_sk#17] -Arguments: [ss_item_sk#1], [cs_item_sk#17], Inner - -(30) CometProject -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(33) CometBroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: [d_date_sk#32, d_year#33] - -(34) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ss_sold_date_sk#12], [d_date_sk#32], Inner, BuildRight - -(35) CometProject -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#32, d_year#33] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name)] -ReadSchema: struct - -(37) CometFilter -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Condition : ((isnotnull(s_store_sk#34) AND isnotnull(s_store_name#35)) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true))) - -(38) CometProject -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#36] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37], [s_store_sk#34, s_store_name#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_zip#36, 10, true, false, true) AS s_zip#37] - -(39) CometBroadcastExchange -Input [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [s_store_sk#34, s_store_name#35, s_zip#37] - -(40) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33] -Right output [3]: [s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_store_sk#6], [s_store_sk#34], Inner, BuildRight - -(41) CometProject -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_sk#34, s_store_name#35, s_zip#37] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] - -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct - -(43) CometFilter -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Condition : (((((isnotnull(c_customer_sk#38) AND isnotnull(c_first_sales_date_sk#43)) AND isnotnull(c_first_shipto_date_sk#42)) AND isnotnull(c_current_cdemo_sk#39)) AND isnotnull(c_current_hdemo_sk#40)) AND isnotnull(c_current_addr_sk#41)) - -(44) CometBroadcastExchange -Input [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(45) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37] -Right output [6]: [c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_customer_sk#2], [c_customer_sk#38], Inner, BuildRight - -(46) CometProject -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_customer_sk#38, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] - -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#44, d_year#45] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(48) CometFilter -Input [2]: [d_date_sk#44, d_year#45] -Condition : isnotnull(d_date_sk#44) - -(49) CometBroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: [d_date_sk#44, d_year#45] - -(50) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43] -Right output [2]: [d_date_sk#44, d_year#45] -Arguments: [c_first_sales_date_sk#43], [d_date_sk#44], Inner, BuildRight - -(51) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, c_first_sales_date_sk#43, d_date_sk#44, d_year#45] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] - -(52) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#46, d_year#47] - -(53) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45] -Right output [2]: [d_date_sk#46, d_year#47] -Arguments: [c_first_shipto_date_sk#42], [d_date_sk#46], Inner, BuildRight - -(54) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, c_first_shipto_date_sk#42, d_year#45, d_date_sk#46, d_year#47] -Arguments: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#48, cd_marital_status#49] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true))) - -(57) CometProject -Input [2]: [cd_demo_sk#48, cd_marital_status#49] -Arguments: [cd_demo_sk#48, cd_marital_status#50], [cd_demo_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#49, 1, true, false, true) AS cd_marital_status#50] - -(58) CometBroadcastExchange -Input [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [cd_demo_sk#48, cd_marital_status#50] - -(59) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_cdemo_sk#3], [cd_demo_sk#48], Inner, BuildRight - -(60) CometProject -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_demo_sk#48, cd_marital_status#50] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] - -(61) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#51, cd_marital_status#52] - -(62) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50] -Right output [2]: [cd_demo_sk#51, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#39], [cd_demo_sk#51], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(63) CometProject -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_cdemo_sk#39, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, cd_marital_status#50, cd_demo_sk#51, cd_marital_status#52] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#53] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(65) CometFilter -Input [1]: [p_promo_sk#53] -Condition : isnotnull(p_promo_sk#53) - -(66) CometBroadcastExchange -Input [1]: [p_promo_sk#53] -Arguments: [p_promo_sk#53] - -(67) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [1]: [p_promo_sk#53] -Arguments: [ss_promo_sk#7], [p_promo_sk#53], Inner, BuildRight - -(68) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, p_promo_sk#53] -Arguments: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47], [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] - -(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] -ReadSchema: struct - -(70) CometFilter -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Condition : (isnotnull(hd_demo_sk#54) AND isnotnull(hd_income_band_sk#55)) - -(71) CometBroadcastExchange -Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [hd_demo_sk#54, hd_income_band_sk#55] - -(72) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47] -Right output [2]: [hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_hdemo_sk#4], [hd_demo_sk#54], Inner, BuildRight - -(73) CometProject -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_demo_sk#54, hd_income_band_sk#55] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] - -(74) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] - -(75) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55] -Right output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [c_current_hdemo_sk#40], [hd_demo_sk#56], Inner, BuildRight - -(76) CometProject -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_hdemo_sk#40, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_demo_sk#56, hd_income_band_sk#57] -Arguments: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57], [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk)] -ReadSchema: struct - -(78) CometFilter -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Condition : isnotnull(ca_address_sk#58) - -(79) CometProject -Input [5]: [ca_address_sk#58, ca_street_number#59, ca_street_name#60, ca_city#61, ca_zip#62] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ca_address_sk#58, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_street_number#59, 10, true, false, true) AS ca_street_number#63, ca_street_name#60, ca_city#61, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, ca_zip#62, 10, true, false, true) AS ca_zip#64] - -(80) CometBroadcastExchange -Input [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(81) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57] -Right output [5]: [ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_addr_sk#5], [ca_address_sk#58], Inner, BuildRight - -(82) CometProject -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_address_sk#58, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] - -(83) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(84) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64] -Right output [5]: [ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [c_current_addr_sk#41], [ca_address_sk#65], Inner, BuildRight - -(85) CometProject -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, c_current_addr_sk#41, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_address_sk#65, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(86) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#70] -Batched: true -Location [not included in comparison]/{warehouse_dir}/income_band] -PushedFilters: [IsNotNull(ib_income_band_sk)] -ReadSchema: struct - -(87) CometFilter -Input [1]: [ib_income_band_sk#70] -Condition : isnotnull(ib_income_band_sk#70) - -(88) CometBroadcastExchange -Input [1]: [ib_income_band_sk#70] -Arguments: [ib_income_band_sk#70] - -(89) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#70] -Arguments: [hd_income_band_sk#55], [ib_income_band_sk#70], Inner, BuildRight - -(90) CometProject -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#55, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#70] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(91) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#71] - -(92) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [1]: [ib_income_band_sk#71] -Arguments: [hd_income_band_sk#57], [ib_income_band_sk#71], Inner, BuildRight - -(93) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, hd_income_band_sk#57, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, ib_income_band_sk#71] -Arguments: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69], [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] - -(94) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(95) CometFilter -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Condition : ((((((isnotnull(i_current_price#73) AND staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_color#74, 20, true, false, true) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#73 >= 64.00)) AND (i_current_price#73 <= 74.00)) AND (i_current_price#73 >= 65.00)) AND (i_current_price#73 <= 79.00)) AND isnotnull(i_item_sk#72)) - -(96) CometProject -Input [4]: [i_item_sk#72, i_current_price#73, i_color#74, i_product_name#75] -Arguments: [i_item_sk#72, i_product_name#76], [i_item_sk#72, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#75, 50, true, false, true) AS i_product_name#76] - -(97) CometBroadcastExchange -Input [2]: [i_item_sk#72, i_product_name#76] -Arguments: [i_item_sk#72, i_product_name#76] - -(98) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69] -Right output [2]: [i_item_sk#72, i_product_name#76] -Arguments: [ss_item_sk#1], [i_item_sk#72], Inner, BuildRight - -(99) CometProject -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, s_store_name#35, s_zip#37, d_year#45, d_year#47, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Arguments: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76], [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] - -(100) CometHashAggregate -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#33, d_year#45, d_year#47, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, i_item_sk#72, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] - -(101) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47, count#77, sum#78, sum#79, sum#80] -Keys [15]: [i_product_name#76, i_item_sk#72, s_store_name#35, s_zip#37, ca_street_number#63, ca_street_name#60, ca_city#61, ca_zip#64, ca_street_number#66, ca_street_name#67, ca_city#68, ca_zip#69, d_year#33, d_year#45, d_year#47] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] - -(102) CometExchange -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: hashpartitioning(item_sk#82, store_name#83, store_zip#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(103) CometSort -Input [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Arguments: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97], [item_sk#82 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, store_zip#84 ASC NULLS FIRST] - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#109), dynamicpruningexpression(ss_sold_date_sk#109 IN dynamicpruning#110)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] -ReadSchema: struct - -(105) CometFilter -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Condition : (((((((isnotnull(ss_item_sk#98) AND isnotnull(ss_ticket_number#105)) AND isnotnull(ss_store_sk#103)) AND isnotnull(ss_customer_sk#99)) AND isnotnull(ss_cdemo_sk#100)) AND isnotnull(ss_promo_sk#104)) AND isnotnull(ss_hdemo_sk#101)) AND isnotnull(ss_addr_sk#102)) - -(106) CometBroadcastExchange -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(108) CometFilter -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Condition : (isnotnull(sr_item_sk#111) AND isnotnull(sr_ticket_number#112)) - -(109) CometProject -Input [3]: [sr_item_sk#111, sr_ticket_number#112, sr_returned_date_sk#113] -Arguments: [sr_item_sk#111, sr_ticket_number#112], [sr_item_sk#111, sr_ticket_number#112] - -(110) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_ticket_number#105], [sr_item_sk#111, sr_ticket_number#112], Inner, BuildLeft - -(111) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_ticket_number#105, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, sr_item_sk#111, sr_ticket_number#112] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(112) CometExchange -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: hashpartitioning(ss_item_sk#98, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(113) CometSort -Input [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98 ASC NULLS FIRST] - -(114) ReusedExchange [Reuses operator id: 24] -Output [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] - -(115) CometHashAggregate -Input [4]: [cs_item_sk#114, sum#115, sum#116, isEmpty#117] -Keys [1]: [cs_item_sk#114] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#118)), sum(((cr_refunded_cash#119 + cr_reversed_charge#120) + cr_store_credit#121))] - -(116) CometFilter -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Condition : ((isnotnull(sale#30) AND isnotnull(refund#31)) AND (cast(sale#30 as decimal(21,2)) > (2 * refund#31))) - -(117) CometProject -Input [3]: [cs_item_sk#114, sale#30, refund#31] -Arguments: [cs_item_sk#114], [cs_item_sk#114] - -(118) CometSort -Input [1]: [cs_item_sk#114] -Arguments: [cs_item_sk#114], [cs_item_sk#114 ASC NULLS FIRST] - -(119) CometSortMergeJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [1]: [cs_item_sk#114] -Arguments: [ss_item_sk#98], [cs_item_sk#114], Inner - -(120) CometProject -Input [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, cs_item_sk#114] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] - -(121) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(122) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(123) CometBroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: [d_date_sk#122, d_year#123] - -(124) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109] -Right output [2]: [d_date_sk#122, d_year#123] -Arguments: [ss_sold_date_sk#109], [d_date_sk#122], Inner, BuildRight - -(125) CometProject -Input [13]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, ss_sold_date_sk#109, d_date_sk#122, d_year#123] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] - -(126) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] - -(127) CometBroadcastHashJoin -Left output [11]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123] -Right output [3]: [s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_store_sk#103], [s_store_sk#124], Inner, BuildRight - -(128) CometProject -Input [14]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_store_sk#103, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_sk#124, s_store_name#125, s_zip#37] -Arguments: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37], [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] - -(129) ReusedExchange [Reuses operator id: 44] -Output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(130) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37] -Right output [6]: [c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_customer_sk#99], [c_customer_sk#126], Inner, BuildRight - -(131) CometProject -Input [18]: [ss_item_sk#98, ss_customer_sk#99, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_customer_sk#126, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] - -(132) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#132, d_year#133] - -(133) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131] -Right output [2]: [d_date_sk#132, d_year#133] -Arguments: [c_first_sales_date_sk#131], [d_date_sk#132], Inner, BuildRight - -(134) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, c_first_sales_date_sk#131, d_date_sk#132, d_year#133] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] - -(135) ReusedExchange [Reuses operator id: 49] -Output [2]: [d_date_sk#134, d_year#135] - -(136) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133] -Right output [2]: [d_date_sk#134, d_year#135] -Arguments: [c_first_shipto_date_sk#130], [d_date_sk#134], Inner, BuildRight - -(137) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, c_first_shipto_date_sk#130, d_year#133, d_date_sk#134, d_year#135] -Arguments: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(138) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#136, cd_marital_status#50] - -(139) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_cdemo_sk#100], [cd_demo_sk#136], Inner, BuildRight - -(140) CometProject -Input [18]: [ss_item_sk#98, ss_cdemo_sk#100, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_demo_sk#136, cd_marital_status#50] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] - -(141) ReusedExchange [Reuses operator id: 58] -Output [2]: [cd_demo_sk#137, cd_marital_status#52] - -(142) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50] -Right output [2]: [cd_demo_sk#137, cd_marital_status#52] -Arguments: [c_current_cdemo_sk#127], [cd_demo_sk#137], Inner, NOT (cd_marital_status#50 = cd_marital_status#52), BuildRight - -(143) CometProject -Input [18]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_cdemo_sk#127, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, cd_marital_status#50, cd_demo_sk#137, cd_marital_status#52] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(144) ReusedExchange [Reuses operator id: 66] -Output [1]: [p_promo_sk#138] - -(145) CometBroadcastHashJoin -Left output [14]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [1]: [p_promo_sk#138] -Arguments: [ss_promo_sk#104], [p_promo_sk#138], Inner, BuildRight - -(146) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_promo_sk#104, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, p_promo_sk#138] -Arguments: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135], [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] - -(147) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] - -(148) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135] -Right output [2]: [hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_hdemo_sk#101], [hd_demo_sk#139], Inner, BuildRight - -(149) CometProject -Input [15]: [ss_item_sk#98, ss_hdemo_sk#101, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_demo_sk#139, hd_income_band_sk#140] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] - -(150) ReusedExchange [Reuses operator id: 71] -Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] - -(151) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140] -Right output [2]: [hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [c_current_hdemo_sk#128], [hd_demo_sk#141], Inner, BuildRight - -(152) CometProject -Input [15]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_hdemo_sk#128, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_demo_sk#141, hd_income_band_sk#142] -Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] - -(153) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(154) CometBroadcastHashJoin -Left output [13]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142] -Right output [5]: [ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_addr_sk#102], [ca_address_sk#143], Inner, BuildRight - -(155) CometProject -Input [18]: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_address_sk#143, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] - -(156) ReusedExchange [Reuses operator id: 80] -Output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(157) CometBroadcastHashJoin -Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64] -Right output [5]: [ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [c_current_addr_sk#129], [ca_address_sk#146], Inner, BuildRight - -(158) CometProject -Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, c_current_addr_sk#129, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_address_sk#146, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(159) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#149] - -(160) CometBroadcastHashJoin -Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#149] -Arguments: [hd_income_band_sk#140], [ib_income_band_sk#149], Inner, BuildRight - -(161) CometProject -Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#140, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#149] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(162) ReusedExchange [Reuses operator id: 88] -Output [1]: [ib_income_band_sk#150] - -(163) CometBroadcastHashJoin -Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [1]: [ib_income_band_sk#150] -Arguments: [hd_income_band_sk#142], [ib_income_band_sk#150], Inner, BuildRight - -(164) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, hd_income_band_sk#142, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, ib_income_band_sk#150] -Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] - -(165) ReusedExchange [Reuses operator id: 97] -Output [2]: [i_item_sk#151, i_product_name#76] - -(166) CometBroadcastHashJoin -Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69] -Right output [2]: [i_item_sk#151, i_product_name#76] -Arguments: [ss_item_sk#98], [i_item_sk#151], Inner, BuildRight - -(167) CometProject -Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, s_store_name#125, s_zip#37, d_year#133, d_year#135, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] - -(168) CometHashAggregate -Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#123, d_year#133, d_year#135, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, i_item_sk#151, i_product_name#76] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#106)), partial_sum(UnscaledValue(ss_list_price#107)), partial_sum(UnscaledValue(ss_coupon_amt#108))] - -(169) CometHashAggregate -Input [19]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135, count#77, sum#152, sum#153, sum#154] -Keys [15]: [i_product_name#76, i_item_sk#151, s_store_name#125, s_zip#37, ca_street_number#63, ca_street_name#144, ca_city#145, ca_zip#64, ca_street_number#66, ca_street_name#147, ca_city#148, ca_zip#69, d_year#123, d_year#133, d_year#135] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#106)), sum(UnscaledValue(ss_list_price#107)), sum(UnscaledValue(ss_coupon_amt#108))] - -(170) CometExchange -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: hashpartitioning(item_sk#155, store_name#156, store_zip#157, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(171) CometSort -Input [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162], [item_sk#155 ASC NULLS FIRST, store_name#156 ASC NULLS FIRST, store_zip#157 ASC NULLS FIRST] - -(172) CometSortMergeJoin -Left output [17]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97] -Right output [8]: [item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#156, store_zip#157], Inner, (cnt#159 <= cnt#94) - -(173) CometProject -Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -(174) CometExchange -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(175) CometSort -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST] - -(176) CometColumnarToRow [codegen id : 1] -Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (180) -+- * CometColumnarToRow (179) - +- CometFilter (178) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) - - -(177) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#32, d_year#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct - -(178) CometFilter -Input [2]: [d_date_sk#32, d_year#33] -Condition : ((isnotnull(d_year#33) AND (d_year#33 = 1999)) AND isnotnull(d_date_sk#32)) - -(179) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#32, d_year#33] - -(180) BroadcastExchange -Input [2]: [d_date_sk#32, d_year#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 104 Hosting Expression = ss_sold_date_sk#109 IN dynamicpruning#110 -BroadcastExchange (184) -+- * CometColumnarToRow (183) - +- CometFilter (182) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (181) - - -(181) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#122, d_year#123] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(182) CometFilter -Input [2]: [d_date_sk#122, d_year#123] -Condition : ((isnotnull(d_year#123) AND (d_year#123 = 2000)) AND isnotnull(d_date_sk#122)) - -(183) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#122, d_year#123] - -(184) BroadcastExchange -Input [2]: [d_date_sk#122, d_year#123] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/extended.txt deleted file mode 100644 index 059acab385..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/extended.txt +++ /dev/null @@ -1,247 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 46fe063430..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt +++ /dev/null @@ -1,192 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometExchange [product_name,store_name,cnt,s1,s1] #1 - CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #2 - CometHashAggregate [d_year,d_year,count,sum,sum,sum] [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #3 - 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_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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - CometExchange [cs_item_sk] #6 - CometHashAggregate [cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [cs_item_sk,sum,sum,isEmpty] - CometProject [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSortMergeJoin [cs_item_sk,cs_order_number,cs_ext_list_price,cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometSort [cs_item_sk,cs_order_number,cs_ext_list_price] - CometExchange [cs_item_sk,cs_order_number] #7 - CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] - CometFilter [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] - CometSort [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometExchange [cr_item_sk,cr_order_number] #8 - CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] - CometFilter [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [s_store_sk,s_store_name,s_zip] #10 - CometProject [s_zip] [s_store_sk,s_store_name,s_zip] - CometFilter [s_store_sk,s_store_name,s_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] - CometBroadcastExchange [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] #11 - 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 [native_iceberg_compat] 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] - CometBroadcastExchange [d_date_sk,d_year] #12 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [d_date_sk,d_year] #12 - CometBroadcastExchange [cd_demo_sk,cd_marital_status] #13 - CometProject [cd_marital_status] [cd_demo_sk,cd_marital_status] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - CometBroadcastExchange [p_promo_sk] #14 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometBroadcastExchange [hd_demo_sk,hd_income_band_sk] #15 - CometFilter [hd_demo_sk,hd_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometProject [ca_street_number,ca_zip] [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - CometBroadcastExchange [ib_income_band_sk] #17 - CometFilter [ib_income_band_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band [ib_income_band_sk] - ReusedExchange [ib_income_band_sk] #17 - CometBroadcastExchange [i_item_sk,i_product_name] #18 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_current_price,i_color,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] - CometSort [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] - CometExchange [item_sk,store_name,store_zip] #19 - CometHashAggregate [i_product_name,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,count,sum,sum,sum] [item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,i_item_sk,s_store_name,s_zip,d_year,count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt))] - CometHashAggregate [ss_wholesale_cost,ss_list_price,ss_coupon_amt] [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] - CometProject [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ib_income_band_sk] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_demo_sk,hd_income_band_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,p_promo_sk] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_demo_sk,cd_marital_status] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,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] - CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_sk,s_store_name,s_zip] - 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,d_year] - CometBroadcastHashJoin [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,d_date_sk,d_year] - 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] - CometSortMergeJoin [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,cs_item_sk] - CometSort [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] - CometExchange [ss_item_sk] #20 - 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_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] #21 - 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 [native_iceberg_compat] 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 #22 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometSort [cs_item_sk] - CometProject [cs_item_sk] - CometFilter [cs_item_sk,sale,refund] - CometHashAggregate [sum,sum,isEmpty] [cs_item_sk,sale,refund,sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit))] - ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 - CometBroadcastExchange [d_date_sk,d_year] #23 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - ReusedExchange [s_store_sk,s_store_name,s_zip] #10 - ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #11 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [d_date_sk,d_year] #12 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [cd_demo_sk,cd_marital_status] #13 - ReusedExchange [p_promo_sk] #14 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [hd_demo_sk,hd_income_band_sk] #15 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #16 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [ib_income_band_sk] #17 - ReusedExchange [i_item_sk,i_product_name] #18 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/explain.txt deleted file mode 100644 index 674c7c227b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/explain.txt +++ /dev/null @@ -1,533 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (87) -+- * Filter (86) - +- Window (85) - +- * CometColumnarToRow (84) - +- CometSort (83) - +- CometColumnarExchange (82) - +- Union (81) - :- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * Project (20) - : +- * BroadcastHashJoin Inner BuildRight (19) - : :- * Project (13) - : : +- * BroadcastHashJoin Inner BuildRight (12) - : : :- * Project (6) - : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- ReusedExchange (4) - : : +- BroadcastExchange (11) - : : +- * CometColumnarToRow (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (18) - : +- * CometColumnarToRow (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometNativeScan parquet spark_catalog.default.item (14) - :- * HashAggregate (31) - : +- * CometColumnarToRow (30) - : +- CometColumnarExchange (29) - : +- * HashAggregate (28) - : +- * HashAggregate (27) - : +- * CometColumnarToRow (26) - : +- ReusedExchange (25) - :- * HashAggregate (38) - : +- * CometColumnarToRow (37) - : +- CometColumnarExchange (36) - : +- * HashAggregate (35) - : +- * HashAggregate (34) - : +- * CometColumnarToRow (33) - : +- ReusedExchange (32) - :- * HashAggregate (45) - : +- * CometColumnarToRow (44) - : +- CometColumnarExchange (43) - : +- * HashAggregate (42) - : +- * HashAggregate (41) - : +- * CometColumnarToRow (40) - : +- ReusedExchange (39) - :- * HashAggregate (52) - : +- * CometColumnarToRow (51) - : +- CometColumnarExchange (50) - : +- * HashAggregate (49) - : +- * HashAggregate (48) - : +- * CometColumnarToRow (47) - : +- ReusedExchange (46) - :- * HashAggregate (59) - : +- * CometColumnarToRow (58) - : +- CometColumnarExchange (57) - : +- * HashAggregate (56) - : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) - : +- ReusedExchange (53) - :- * HashAggregate (66) - : +- * CometColumnarToRow (65) - : +- CometColumnarExchange (64) - : +- * HashAggregate (63) - : +- * HashAggregate (62) - : +- * CometColumnarToRow (61) - : +- ReusedExchange (60) - :- * HashAggregate (73) - : +- * CometColumnarToRow (72) - : +- CometColumnarExchange (71) - : +- * HashAggregate (70) - : +- * HashAggregate (69) - : +- * CometColumnarToRow (68) - : +- ReusedExchange (67) - +- * HashAggregate (80) - +- * CometColumnarToRow (79) - +- CometColumnarExchange (78) - +- * HashAggregate (77) - +- * HashAggregate (76) - +- * CometColumnarToRow (75) - +- ReusedExchange (74) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(4) ReusedExchange [Reuses operator id: 92] -Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(5) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#5] -Right keys [1]: [d_date_sk#7] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#11, s_store_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [2]: [s_store_sk#11, s_store_id#12] -Condition : isnotnull(s_store_sk#11) - -(9) CometProject -Input [2]: [s_store_sk#11, s_store_id#12] -Arguments: [s_store_sk#11, s_store_id#13], [s_store_sk#11, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#12, 16, true, false, true) AS s_store_id#13] - -(10) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#11, s_store_id#13] - -(11) BroadcastExchange -Input [2]: [s_store_sk#11, s_store_id#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#11] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 4] -Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13] -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#13] - -(14) CometNativeScan parquet spark_catalog.default.item -Output [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(15) CometFilter -Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -Condition : isnotnull(i_item_sk#14) - -(16) CometProject -Input [5]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17, i_product_name#18] -Arguments: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22], [i_item_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#15, 50, true, false, true) AS i_brand#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#16, 50, true, false, true) AS i_class#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#17, 50, true, false, true) AS i_category#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#18, 50, true, false, true) AS i_product_name#22] - -(17) CometColumnarToRow [codegen id : 3] -Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] - -(18) BroadcastExchange -Input [5]: [i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] - -(19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#14] -Join type: Inner -Join condition: None - -(20) Project [codegen id : 4] -Output [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_brand#19, i_class#20, i_category#21, i_product_name#22] -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_item_sk#14, i_brand#19, i_class#20, i_category#21, i_product_name#22] - -(21) HashAggregate [codegen id : 4] -Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#13, i_brand#19, i_class#20, i_category#21, i_product_name#22] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] -Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [2]: [sum#23, isEmpty#24] -Results [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] - -(22) CometColumnarExchange -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 5] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] - -(24) HashAggregate [codegen id : 5] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13, sum#25, isEmpty#26] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#8, d_qoy#10, d_moy#9, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#27] -Results [9]: [i_category#21 AS i_category#28, i_class#20 AS i_class#29, i_brand#19 AS i_brand#30, i_product_name#22 AS i_product_name#31, d_year#8 AS d_year#32, d_qoy#10 AS d_qoy#33, d_moy#9 AS d_moy#34, s_store_id#13 AS s_store_id#35, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#27 as decimal(38,2)) AS sumsales#36] - -(25) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] - -(26) CometColumnarToRow [codegen id : 10] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] - -(27) HashAggregate [codegen id : 10] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13, sum#40, isEmpty#41] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#27] -Results [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#27 AS sumsales#44] - -(28) HashAggregate [codegen id : 10] -Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sumsales#44] -Keys [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39] -Functions [1]: [partial_sum(sumsales#44)] -Aggregate Attributes [2]: [sum#45, isEmpty#46] -Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] - -(29) CometColumnarExchange -Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 11] -Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] - -(31) HashAggregate [codegen id : 11] -Input [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, sum#47, isEmpty#48] -Keys [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39] -Functions [1]: [sum(sumsales#44)] -Aggregate Attributes [1]: [sum(sumsales#44)#49] -Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#37, d_qoy#38, d_moy#39, null AS s_store_id#50, sum(sumsales#44)#49 AS sumsales#51] - -(32) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] - -(33) CometColumnarToRow [codegen id : 16] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] - -(34) HashAggregate [codegen id : 16] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13, sum#55, isEmpty#56] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, d_moy#54, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))#27] -Results [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum(coalesce((ss_sales_price#57 * cast(ss_quantity#58 as decimal(10,0))), 0.00))#27 AS sumsales#44] - -(35) HashAggregate [codegen id : 16] -Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sumsales#44] -Keys [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53] -Functions [1]: [partial_sum(sumsales#44)] -Aggregate Attributes [2]: [sum#59, isEmpty#60] -Results [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#61, isEmpty#62] - -(36) CometColumnarExchange -Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#61, isEmpty#62] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(37) CometColumnarToRow [codegen id : 17] -Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#61, isEmpty#62] - -(38) HashAggregate [codegen id : 17] -Input [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, sum#61, isEmpty#62] -Keys [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53] -Functions [1]: [sum(sumsales#44)] -Aggregate Attributes [1]: [sum(sumsales#44)#63] -Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#52, d_qoy#53, null AS d_moy#64, null AS s_store_id#65, sum(sumsales#44)#63 AS sumsales#66] - -(39) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67, d_qoy#68, d_moy#69, s_store_id#13, sum#70, isEmpty#71] - -(40) CometColumnarToRow [codegen id : 22] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67, d_qoy#68, d_moy#69, s_store_id#13, sum#70, isEmpty#71] - -(41) HashAggregate [codegen id : 22] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67, d_qoy#68, d_moy#69, s_store_id#13, sum#70, isEmpty#71] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67, d_qoy#68, d_moy#69, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#72 * cast(ss_quantity#73 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#72 * cast(ss_quantity#73 as decimal(10,0))), 0.00))#27] -Results [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67, sum(coalesce((ss_sales_price#72 * cast(ss_quantity#73 as decimal(10,0))), 0.00))#27 AS sumsales#44] - -(42) HashAggregate [codegen id : 22] -Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67, sumsales#44] -Keys [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67] -Functions [1]: [partial_sum(sumsales#44)] -Aggregate Attributes [2]: [sum#74, isEmpty#75] -Results [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67, sum#76, isEmpty#77] - -(43) CometColumnarExchange -Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67, sum#76, isEmpty#77] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(44) CometColumnarToRow [codegen id : 23] -Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67, sum#76, isEmpty#77] - -(45) HashAggregate [codegen id : 23] -Input [7]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67, sum#76, isEmpty#77] -Keys [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67] -Functions [1]: [sum(sumsales#44)] -Aggregate Attributes [1]: [sum(sumsales#44)#78] -Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#67, null AS d_qoy#79, null AS d_moy#80, null AS s_store_id#81, sum(sumsales#44)#78 AS sumsales#82] - -(46) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#83, d_qoy#84, d_moy#85, s_store_id#13, sum#86, isEmpty#87] - -(47) CometColumnarToRow [codegen id : 28] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#83, d_qoy#84, d_moy#85, s_store_id#13, sum#86, isEmpty#87] - -(48) HashAggregate [codegen id : 28] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#83, d_qoy#84, d_moy#85, s_store_id#13, sum#86, isEmpty#87] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#83, d_qoy#84, d_moy#85, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#88 * cast(ss_quantity#89 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#88 * cast(ss_quantity#89 as decimal(10,0))), 0.00))#27] -Results [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum(coalesce((ss_sales_price#88 * cast(ss_quantity#89 as decimal(10,0))), 0.00))#27 AS sumsales#44] - -(49) HashAggregate [codegen id : 28] -Input [5]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sumsales#44] -Keys [4]: [i_category#21, i_class#20, i_brand#19, i_product_name#22] -Functions [1]: [partial_sum(sumsales#44)] -Aggregate Attributes [2]: [sum#90, isEmpty#91] -Results [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#92, isEmpty#93] - -(50) CometColumnarExchange -Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#92, isEmpty#93] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, i_product_name#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(51) CometColumnarToRow [codegen id : 29] -Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#92, isEmpty#93] - -(52) HashAggregate [codegen id : 29] -Input [6]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, sum#92, isEmpty#93] -Keys [4]: [i_category#21, i_class#20, i_brand#19, i_product_name#22] -Functions [1]: [sum(sumsales#44)] -Aggregate Attributes [1]: [sum(sumsales#44)#94] -Results [9]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, null AS d_year#95, null AS d_qoy#96, null AS d_moy#97, null AS s_store_id#98, sum(sumsales#44)#94 AS sumsales#99] - -(53) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#100, d_qoy#101, d_moy#102, s_store_id#13, sum#103, isEmpty#104] - -(54) CometColumnarToRow [codegen id : 34] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#100, d_qoy#101, d_moy#102, s_store_id#13, sum#103, isEmpty#104] - -(55) HashAggregate [codegen id : 34] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#100, d_qoy#101, d_moy#102, s_store_id#13, sum#103, isEmpty#104] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#100, d_qoy#101, d_moy#102, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#27] -Results [4]: [i_category#21, i_class#20, i_brand#19, sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#27 AS sumsales#44] - -(56) HashAggregate [codegen id : 34] -Input [4]: [i_category#21, i_class#20, i_brand#19, sumsales#44] -Keys [3]: [i_category#21, i_class#20, i_brand#19] -Functions [1]: [partial_sum(sumsales#44)] -Aggregate Attributes [2]: [sum#107, isEmpty#108] -Results [5]: [i_category#21, i_class#20, i_brand#19, sum#109, isEmpty#110] - -(57) CometColumnarExchange -Input [5]: [i_category#21, i_class#20, i_brand#19, sum#109, isEmpty#110] -Arguments: hashpartitioning(i_category#21, i_class#20, i_brand#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(58) CometColumnarToRow [codegen id : 35] -Input [5]: [i_category#21, i_class#20, i_brand#19, sum#109, isEmpty#110] - -(59) HashAggregate [codegen id : 35] -Input [5]: [i_category#21, i_class#20, i_brand#19, sum#109, isEmpty#110] -Keys [3]: [i_category#21, i_class#20, i_brand#19] -Functions [1]: [sum(sumsales#44)] -Aggregate Attributes [1]: [sum(sumsales#44)#111] -Results [9]: [i_category#21, i_class#20, i_brand#19, null AS i_product_name#112, null AS d_year#113, null AS d_qoy#114, null AS d_moy#115, null AS s_store_id#116, sum(sumsales#44)#111 AS sumsales#117] - -(60) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#118, d_qoy#119, d_moy#120, s_store_id#13, sum#121, isEmpty#122] - -(61) CometColumnarToRow [codegen id : 40] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#118, d_qoy#119, d_moy#120, s_store_id#13, sum#121, isEmpty#122] - -(62) HashAggregate [codegen id : 40] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#118, d_qoy#119, d_moy#120, s_store_id#13, sum#121, isEmpty#122] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#118, d_qoy#119, d_moy#120, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#123 * cast(ss_quantity#124 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#123 * cast(ss_quantity#124 as decimal(10,0))), 0.00))#27] -Results [3]: [i_category#21, i_class#20, sum(coalesce((ss_sales_price#123 * cast(ss_quantity#124 as decimal(10,0))), 0.00))#27 AS sumsales#44] - -(63) HashAggregate [codegen id : 40] -Input [3]: [i_category#21, i_class#20, sumsales#44] -Keys [2]: [i_category#21, i_class#20] -Functions [1]: [partial_sum(sumsales#44)] -Aggregate Attributes [2]: [sum#125, isEmpty#126] -Results [4]: [i_category#21, i_class#20, sum#127, isEmpty#128] - -(64) CometColumnarExchange -Input [4]: [i_category#21, i_class#20, sum#127, isEmpty#128] -Arguments: hashpartitioning(i_category#21, i_class#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(65) CometColumnarToRow [codegen id : 41] -Input [4]: [i_category#21, i_class#20, sum#127, isEmpty#128] - -(66) HashAggregate [codegen id : 41] -Input [4]: [i_category#21, i_class#20, sum#127, isEmpty#128] -Keys [2]: [i_category#21, i_class#20] -Functions [1]: [sum(sumsales#44)] -Aggregate Attributes [1]: [sum(sumsales#44)#129] -Results [9]: [i_category#21, i_class#20, null AS i_brand#130, null AS i_product_name#131, null AS d_year#132, null AS d_qoy#133, null AS d_moy#134, null AS s_store_id#135, sum(sumsales#44)#129 AS sumsales#136] - -(67) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#137, d_qoy#138, d_moy#139, s_store_id#13, sum#140, isEmpty#141] - -(68) CometColumnarToRow [codegen id : 46] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#137, d_qoy#138, d_moy#139, s_store_id#13, sum#140, isEmpty#141] - -(69) HashAggregate [codegen id : 46] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#137, d_qoy#138, d_moy#139, s_store_id#13, sum#140, isEmpty#141] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#137, d_qoy#138, d_moy#139, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#142 * cast(ss_quantity#143 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#142 * cast(ss_quantity#143 as decimal(10,0))), 0.00))#27] -Results [2]: [i_category#21, sum(coalesce((ss_sales_price#142 * cast(ss_quantity#143 as decimal(10,0))), 0.00))#27 AS sumsales#44] - -(70) HashAggregate [codegen id : 46] -Input [2]: [i_category#21, sumsales#44] -Keys [1]: [i_category#21] -Functions [1]: [partial_sum(sumsales#44)] -Aggregate Attributes [2]: [sum#144, isEmpty#145] -Results [3]: [i_category#21, sum#146, isEmpty#147] - -(71) CometColumnarExchange -Input [3]: [i_category#21, sum#146, isEmpty#147] -Arguments: hashpartitioning(i_category#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(72) CometColumnarToRow [codegen id : 47] -Input [3]: [i_category#21, sum#146, isEmpty#147] - -(73) HashAggregate [codegen id : 47] -Input [3]: [i_category#21, sum#146, isEmpty#147] -Keys [1]: [i_category#21] -Functions [1]: [sum(sumsales#44)] -Aggregate Attributes [1]: [sum(sumsales#44)#148] -Results [9]: [i_category#21, null AS i_class#149, null AS i_brand#150, null AS i_product_name#151, null AS d_year#152, null AS d_qoy#153, null AS d_moy#154, null AS s_store_id#155, sum(sumsales#44)#148 AS sumsales#156] - -(74) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#157, d_qoy#158, d_moy#159, s_store_id#13, sum#160, isEmpty#161] - -(75) CometColumnarToRow [codegen id : 52] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#157, d_qoy#158, d_moy#159, s_store_id#13, sum#160, isEmpty#161] - -(76) HashAggregate [codegen id : 52] -Input [10]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#157, d_qoy#158, d_moy#159, s_store_id#13, sum#160, isEmpty#161] -Keys [8]: [i_category#21, i_class#20, i_brand#19, i_product_name#22, d_year#157, d_qoy#158, d_moy#159, s_store_id#13] -Functions [1]: [sum(coalesce((ss_sales_price#162 * cast(ss_quantity#163 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#162 * cast(ss_quantity#163 as decimal(10,0))), 0.00))#27] -Results [1]: [sum(coalesce((ss_sales_price#162 * cast(ss_quantity#163 as decimal(10,0))), 0.00))#27 AS sumsales#44] - -(77) HashAggregate [codegen id : 52] -Input [1]: [sumsales#44] -Keys: [] -Functions [1]: [partial_sum(sumsales#44)] -Aggregate Attributes [2]: [sum#164, isEmpty#165] -Results [2]: [sum#166, isEmpty#167] - -(78) CometColumnarExchange -Input [2]: [sum#166, isEmpty#167] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(79) CometColumnarToRow [codegen id : 53] -Input [2]: [sum#166, isEmpty#167] - -(80) HashAggregate [codegen id : 53] -Input [2]: [sum#166, isEmpty#167] -Keys: [] -Functions [1]: [sum(sumsales#44)] -Aggregate Attributes [1]: [sum(sumsales#44)#168] -Results [9]: [null AS i_category#169, null AS i_class#170, null AS i_brand#171, null AS i_product_name#172, null AS d_year#173, null AS d_qoy#174, null AS d_moy#175, null AS s_store_id#176, sum(sumsales#44)#168 AS sumsales#177] - -(81) Union - -(82) CometColumnarExchange -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: hashpartitioning(i_category#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(83) CometSort -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36], [i_category#28 ASC NULLS FIRST, sumsales#36 DESC NULLS LAST] - -(84) CometColumnarToRow [codegen id : 54] -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] - -(85) Window -Input [9]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36] -Arguments: [rank(sumsales#36) windowspecdefinition(i_category#28, sumsales#36 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#178], [i_category#28], [sumsales#36 DESC NULLS LAST] - -(86) Filter [codegen id : 55] -Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#178] -Condition : (rk#178 <= 100) - -(87) TakeOrderedAndProject -Input [10]: [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#178] -Arguments: 100, [i_category#28 ASC NULLS FIRST, i_class#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, i_product_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_qoy#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, s_store_id#35 ASC NULLS FIRST, sumsales#36 ASC NULLS FIRST, rk#178 ASC NULLS FIRST], [i_category#28, i_class#29, i_brand#30, i_product_name#31, d_year#32, d_qoy#33, d_moy#34, s_store_id#35, sumsales#36, rk#178] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (92) -+- * CometColumnarToRow (91) - +- CometProject (90) - +- CometFilter (89) - +- CometNativeScan parquet spark_catalog.default.date_dim (88) - - -(88) CometNativeScan parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#179, d_year#8, d_moy#9, d_qoy#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(89) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#179, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#179) AND (d_month_seq#179 >= 1212)) AND (d_month_seq#179 <= 1223)) AND isnotnull(d_date_sk#7)) - -(90) CometProject -Input [5]: [d_date_sk#7, d_month_seq#179, d_year#8, d_moy#9, d_qoy#10] -Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(91) CometColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] - -(92) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/simplified.txt deleted file mode 100644 index 4bc3f849c5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/simplified.txt +++ /dev/null @@ -1,142 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (55) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WholeStageCodegen (54) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometColumnarExchange [i_category] #1 - Union - WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - WholeStageCodegen (4) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] - Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - InputAdapter - ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - WholeStageCodegen (11) - 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] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 - WholeStageCodegen (10) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (17) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 - WholeStageCodegen (16) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (23) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name,d_year] #8 - WholeStageCodegen (22) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (29) - HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand,i_product_name] #9 - WholeStageCodegen (28) - HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (35) - HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class,i_brand] #10 - WholeStageCodegen (34) - HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (41) - HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #11 - WholeStageCodegen (40) - HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (47) - HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #12 - WholeStageCodegen (46) - HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - WholeStageCodegen (53) - HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #13 - WholeStageCodegen (52) - HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/explain.txt deleted file mode 100644 index ac04f86f96..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,426 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (70) -+- * Filter (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometUnion (64) - :- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometHashAggregate (25) - : +- ReusedExchange (24) - :- CometHashAggregate (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- CometHashAggregate (30) - : +- ReusedExchange (29) - :- CometHashAggregate (38) - : +- CometExchange (37) - : +- CometHashAggregate (36) - : +- CometHashAggregate (35) - : +- ReusedExchange (34) - :- CometHashAggregate (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometHashAggregate (40) - : +- ReusedExchange (39) - :- CometHashAggregate (48) - : +- CometExchange (47) - : +- CometHashAggregate (46) - : +- CometHashAggregate (45) - : +- ReusedExchange (44) - :- CometHashAggregate (53) - : +- CometExchange (52) - : +- CometHashAggregate (51) - : +- CometHashAggregate (50) - : +- ReusedExchange (49) - :- CometHashAggregate (58) - : +- CometExchange (57) - : +- CometHashAggregate (56) - : +- CometHashAggregate (55) - : +- ReusedExchange (54) - +- CometHashAggregate (63) - +- CometExchange (62) - +- CometHashAggregate (61) - +- CometHashAggregate (60) - +- ReusedExchange (59) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(6) CometBroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Right output [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11], [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_store_id#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#12, s_store_id#13] -Condition : isnotnull(s_store_sk#12) - -(11) CometProject -Input [2]: [s_store_sk#12, s_store_id#13] -Arguments: [s_store_sk#12, s_store_id#14], [s_store_sk#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#13, 16, true, false, true) AS s_store_id#14] - -(12) CometBroadcastExchange -Input [2]: [s_store_sk#12, s_store_id#14] -Arguments: [s_store_sk#12, s_store_id#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] -Right output [2]: [s_store_sk#12, s_store_id#14] -Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_sk#12, s_store_id#14] -Arguments: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14], [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Condition : isnotnull(i_item_sk#15) - -(17) CometProject -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#16, 50, true, false, true) AS i_brand#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#17, 50, true, false, true) AS i_class#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true) AS i_category#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#19, 50, true, false, true) AS i_product_name#23] - -(18) CometBroadcastExchange -Input [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] -Right output [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight - -(20) CometProject -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23], [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23] - -(21) CometHashAggregate -Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(22) CometExchange -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#24, isEmpty#25] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#24, isEmpty#25] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(24) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14, sum#29, isEmpty#30] - -(25) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14, sum#29, isEmpty#30] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#31 * cast(ss_quantity#32 as decimal(10,0))), 0.00))] - -(26) CometHashAggregate -Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sumsales#33] -Keys [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28] -Functions [1]: [partial_sum(sumsales#33)] - -(27) CometExchange -Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sum#34, isEmpty#35] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sum#34, isEmpty#35] -Keys [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28] -Functions [1]: [sum(sumsales#33)] - -(29) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14, sum#39, isEmpty#40] - -(30) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14, sum#39, isEmpty#40] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#41 * cast(ss_quantity#42 as decimal(10,0))), 0.00))] - -(31) CometHashAggregate -Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sumsales#33] -Keys [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37] -Functions [1]: [partial_sum(sumsales#33)] - -(32) CometExchange -Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sum#43, isEmpty#44] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometHashAggregate -Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sum#43, isEmpty#44] -Keys [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37] -Functions [1]: [sum(sumsales#33)] - -(34) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#45, d_qoy#46, d_moy#47, s_store_id#14, sum#48, isEmpty#49] - -(35) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#45, d_qoy#46, d_moy#47, s_store_id#14, sum#48, isEmpty#49] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#45, d_qoy#46, d_moy#47, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#50 * cast(ss_quantity#51 as decimal(10,0))), 0.00))] - -(36) CometHashAggregate -Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#45, sumsales#33] -Keys [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#45] -Functions [1]: [partial_sum(sumsales#33)] - -(37) CometExchange -Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#45, sum#52, isEmpty#53] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#45, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(38) CometHashAggregate -Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#45, sum#52, isEmpty#53] -Keys [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#45] -Functions [1]: [sum(sumsales#33)] - -(39) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#54, d_qoy#55, d_moy#56, s_store_id#14, sum#57, isEmpty#58] - -(40) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#54, d_qoy#55, d_moy#56, s_store_id#14, sum#57, isEmpty#58] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#54, d_qoy#55, d_moy#56, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#59 * cast(ss_quantity#60 as decimal(10,0))), 0.00))] - -(41) CometHashAggregate -Input [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sumsales#33] -Keys [4]: [i_category#22, i_class#21, i_brand#20, i_product_name#23] -Functions [1]: [partial_sum(sumsales#33)] - -(42) CometExchange -Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sum#61, isEmpty#62] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(43) CometHashAggregate -Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sum#61, isEmpty#62] -Keys [4]: [i_category#22, i_class#21, i_brand#20, i_product_name#23] -Functions [1]: [sum(sumsales#33)] - -(44) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#63, d_qoy#64, d_moy#65, s_store_id#14, sum#66, isEmpty#67] - -(45) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#63, d_qoy#64, d_moy#65, s_store_id#14, sum#66, isEmpty#67] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#63, d_qoy#64, d_moy#65, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#68 * cast(ss_quantity#69 as decimal(10,0))), 0.00))] - -(46) CometHashAggregate -Input [4]: [i_category#22, i_class#21, i_brand#20, sumsales#33] -Keys [3]: [i_category#22, i_class#21, i_brand#20] -Functions [1]: [partial_sum(sumsales#33)] - -(47) CometExchange -Input [5]: [i_category#22, i_class#21, i_brand#20, sum#70, isEmpty#71] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(48) CometHashAggregate -Input [5]: [i_category#22, i_class#21, i_brand#20, sum#70, isEmpty#71] -Keys [3]: [i_category#22, i_class#21, i_brand#20] -Functions [1]: [sum(sumsales#33)] - -(49) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#72, d_qoy#73, d_moy#74, s_store_id#14, sum#75, isEmpty#76] - -(50) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#72, d_qoy#73, d_moy#74, s_store_id#14, sum#75, isEmpty#76] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#72, d_qoy#73, d_moy#74, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#77 * cast(ss_quantity#78 as decimal(10,0))), 0.00))] - -(51) CometHashAggregate -Input [3]: [i_category#22, i_class#21, sumsales#33] -Keys [2]: [i_category#22, i_class#21] -Functions [1]: [partial_sum(sumsales#33)] - -(52) CometExchange -Input [4]: [i_category#22, i_class#21, sum#79, isEmpty#80] -Arguments: hashpartitioning(i_category#22, i_class#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(53) CometHashAggregate -Input [4]: [i_category#22, i_class#21, sum#79, isEmpty#80] -Keys [2]: [i_category#22, i_class#21] -Functions [1]: [sum(sumsales#33)] - -(54) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#81, d_qoy#82, d_moy#83, s_store_id#14, sum#84, isEmpty#85] - -(55) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#81, d_qoy#82, d_moy#83, s_store_id#14, sum#84, isEmpty#85] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#81, d_qoy#82, d_moy#83, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#86 * cast(ss_quantity#87 as decimal(10,0))), 0.00))] - -(56) CometHashAggregate -Input [2]: [i_category#22, sumsales#33] -Keys [1]: [i_category#22] -Functions [1]: [partial_sum(sumsales#33)] - -(57) CometExchange -Input [3]: [i_category#22, sum#88, isEmpty#89] -Arguments: hashpartitioning(i_category#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(58) CometHashAggregate -Input [3]: [i_category#22, sum#88, isEmpty#89] -Keys [1]: [i_category#22] -Functions [1]: [sum(sumsales#33)] - -(59) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#90, d_qoy#91, d_moy#92, s_store_id#14, sum#93, isEmpty#94] - -(60) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#90, d_qoy#91, d_moy#92, s_store_id#14, sum#93, isEmpty#94] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#90, d_qoy#91, d_moy#92, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#95 * cast(ss_quantity#96 as decimal(10,0))), 0.00))] - -(61) CometHashAggregate -Input [1]: [sumsales#33] -Keys: [] -Functions [1]: [partial_sum(sumsales#33)] - -(62) CometExchange -Input [2]: [sum#97, isEmpty#98] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(63) CometHashAggregate -Input [2]: [sum#97, isEmpty#98] -Keys: [] -Functions [1]: [sum(sumsales#33)] - -(64) CometUnion -Child 0 Input [9]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107] -Child 1 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#108, sumsales#109] -Child 2 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#110, s_store_id#111, sumsales#112] -Child 3 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#45, d_qoy#113, d_moy#114, s_store_id#115, sumsales#116] -Child 4 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#117, d_qoy#118, d_moy#119, s_store_id#120, sumsales#121] -Child 5 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#122, d_year#123, d_qoy#124, d_moy#125, s_store_id#126, sumsales#127] -Child 6 Input [9]: [i_category#22, i_class#21, i_brand#128, i_product_name#129, d_year#130, d_qoy#131, d_moy#132, s_store_id#133, sumsales#134] -Child 7 Input [9]: [i_category#22, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141, sumsales#142] -Child 8 Input [9]: [i_category#143, i_class#144, i_brand#145, i_product_name#146, d_year#147, d_qoy#148, d_moy#149, s_store_id#150, sumsales#151] - -(65) CometExchange -Input [9]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107] -Arguments: hashpartitioning(i_category#99, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(66) CometSort -Input [9]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107] -Arguments: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107], [i_category#99 ASC NULLS FIRST, sumsales#107 DESC NULLS LAST] - -(67) CometColumnarToRow [codegen id : 1] -Input [9]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107] - -(68) Window -Input [9]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107] -Arguments: [rank(sumsales#107) windowspecdefinition(i_category#99, sumsales#107 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#152], [i_category#99], [sumsales#107 DESC NULLS LAST] - -(69) Filter [codegen id : 2] -Input [10]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] -Condition : (rk#152 <= 100) - -(70) TakeOrderedAndProject -Input [10]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] -Arguments: 100, [i_category#99 ASC NULLS FIRST, i_class#100 ASC NULLS FIRST, i_brand#101 ASC NULLS FIRST, i_product_name#102 ASC NULLS FIRST, d_year#103 ASC NULLS FIRST, d_qoy#104 ASC NULLS FIRST, d_moy#105 ASC NULLS FIRST, s_store_id#106 ASC NULLS FIRST, sumsales#107 ASC NULLS FIRST, rk#152 ASC NULLS FIRST], [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (75) -+- * CometColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (71) - - -(71) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(72) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) - -(73) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(74) CometColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(75) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 5ca258aeca..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,82 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (2) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometExchange [i_category] #1 - CometUnion [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] - 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,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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] - CometHashAggregate [s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] - CometHashAggregate [d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year] #9 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] - CometHashAggregate [d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name] #10 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,sum,isEmpty] - CometHashAggregate [d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand] #11 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,sum,isEmpty] - CometHashAggregate [i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class] #12 - CometHashAggregate [sumsales] [i_category,i_class,sum,isEmpty] - CometHashAggregate [i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category] #13 - CometHashAggregate [sumsales] [i_category,sum,isEmpty] - CometHashAggregate [i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange #14 - CometHashAggregate [sumsales] [sum,isEmpty] - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt deleted file mode 100644 index ac04f86f96..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt +++ /dev/null @@ -1,426 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (70) -+- * Filter (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometUnion (64) - :- CometHashAggregate (23) - : +- CometExchange (22) - : +- CometHashAggregate (21) - : +- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometProject (14) - : : +- CometBroadcastHashJoin (13) - : : :- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (12) - : : +- CometProject (11) - : : +- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - :- CometHashAggregate (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometHashAggregate (25) - : +- ReusedExchange (24) - :- CometHashAggregate (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- CometHashAggregate (30) - : +- ReusedExchange (29) - :- CometHashAggregate (38) - : +- CometExchange (37) - : +- CometHashAggregate (36) - : +- CometHashAggregate (35) - : +- ReusedExchange (34) - :- CometHashAggregate (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometHashAggregate (40) - : +- ReusedExchange (39) - :- CometHashAggregate (48) - : +- CometExchange (47) - : +- CometHashAggregate (46) - : +- CometHashAggregate (45) - : +- ReusedExchange (44) - :- CometHashAggregate (53) - : +- CometExchange (52) - : +- CometHashAggregate (51) - : +- CometHashAggregate (50) - : +- ReusedExchange (49) - :- CometHashAggregate (58) - : +- CometExchange (57) - : +- CometHashAggregate (56) - : +- CometHashAggregate (55) - : +- ReusedExchange (54) - +- CometHashAggregate (63) - +- CometExchange (62) - +- CometHashAggregate (61) - +- CometHashAggregate (60) - +- ReusedExchange (59) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) - -(5) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(6) CometBroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(7) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] -Right output [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_sold_date_sk#5], [d_date_sk#7], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11], [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_store_id#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [s_store_sk#12, s_store_id#13] -Condition : isnotnull(s_store_sk#12) - -(11) CometProject -Input [2]: [s_store_sk#12, s_store_id#13] -Arguments: [s_store_sk#12, s_store_id#14], [s_store_sk#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#13, 16, true, false, true) AS s_store_id#14] - -(12) CometBroadcastExchange -Input [2]: [s_store_sk#12, s_store_id#14] -Arguments: [s_store_sk#12, s_store_id#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11] -Right output [2]: [s_store_sk#12, s_store_id#14] -Arguments: [ss_store_sk#2], [s_store_sk#12], Inner, BuildRight - -(14) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_sk#12, s_store_id#14] -Arguments: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14], [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(16) CometFilter -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Condition : isnotnull(i_item_sk#15) - -(17) CometProject -Input [5]: [i_item_sk#15, i_brand#16, i_class#17, i_category#18, i_product_name#19] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_brand#16, 50, true, false, true) AS i_brand#20, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#17, 50, true, false, true) AS i_class#21, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#18, 50, true, false, true) AS i_category#22, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#19, 50, true, false, true) AS i_product_name#23] - -(18) CometBroadcastExchange -Input [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] - -(19) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14] -Right output [5]: [i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_item_sk#1], [i_item_sk#15], Inner, BuildRight - -(20) CometProject -Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_item_sk#15, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Arguments: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23], [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23] - -(21) CometHashAggregate -Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#9, d_moy#10, d_qoy#11, s_store_id#14, i_brand#20, i_class#21, i_category#22, i_product_name#23] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(22) CometExchange -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#24, isEmpty#25] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(23) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#24, isEmpty#25] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] - -(24) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14, sum#29, isEmpty#30] - -(25) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14, sum#29, isEmpty#30] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#31 * cast(ss_quantity#32 as decimal(10,0))), 0.00))] - -(26) CometHashAggregate -Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sumsales#33] -Keys [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28] -Functions [1]: [partial_sum(sumsales#33)] - -(27) CometExchange -Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sum#34, isEmpty#35] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometHashAggregate -Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, sum#34, isEmpty#35] -Keys [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28] -Functions [1]: [sum(sumsales#33)] - -(29) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14, sum#39, isEmpty#40] - -(30) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14, sum#39, isEmpty#40] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#38, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#41 * cast(ss_quantity#42 as decimal(10,0))), 0.00))] - -(31) CometHashAggregate -Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sumsales#33] -Keys [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37] -Functions [1]: [partial_sum(sumsales#33)] - -(32) CometExchange -Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sum#43, isEmpty#44] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(33) CometHashAggregate -Input [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, sum#43, isEmpty#44] -Keys [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37] -Functions [1]: [sum(sumsales#33)] - -(34) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#45, d_qoy#46, d_moy#47, s_store_id#14, sum#48, isEmpty#49] - -(35) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#45, d_qoy#46, d_moy#47, s_store_id#14, sum#48, isEmpty#49] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#45, d_qoy#46, d_moy#47, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#50 * cast(ss_quantity#51 as decimal(10,0))), 0.00))] - -(36) CometHashAggregate -Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#45, sumsales#33] -Keys [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#45] -Functions [1]: [partial_sum(sumsales#33)] - -(37) CometExchange -Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#45, sum#52, isEmpty#53] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#45, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(38) CometHashAggregate -Input [7]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#45, sum#52, isEmpty#53] -Keys [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#45] -Functions [1]: [sum(sumsales#33)] - -(39) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#54, d_qoy#55, d_moy#56, s_store_id#14, sum#57, isEmpty#58] - -(40) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#54, d_qoy#55, d_moy#56, s_store_id#14, sum#57, isEmpty#58] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#54, d_qoy#55, d_moy#56, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#59 * cast(ss_quantity#60 as decimal(10,0))), 0.00))] - -(41) CometHashAggregate -Input [5]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sumsales#33] -Keys [4]: [i_category#22, i_class#21, i_brand#20, i_product_name#23] -Functions [1]: [partial_sum(sumsales#33)] - -(42) CometExchange -Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sum#61, isEmpty#62] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, i_product_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(43) CometHashAggregate -Input [6]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, sum#61, isEmpty#62] -Keys [4]: [i_category#22, i_class#21, i_brand#20, i_product_name#23] -Functions [1]: [sum(sumsales#33)] - -(44) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#63, d_qoy#64, d_moy#65, s_store_id#14, sum#66, isEmpty#67] - -(45) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#63, d_qoy#64, d_moy#65, s_store_id#14, sum#66, isEmpty#67] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#63, d_qoy#64, d_moy#65, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#68 * cast(ss_quantity#69 as decimal(10,0))), 0.00))] - -(46) CometHashAggregate -Input [4]: [i_category#22, i_class#21, i_brand#20, sumsales#33] -Keys [3]: [i_category#22, i_class#21, i_brand#20] -Functions [1]: [partial_sum(sumsales#33)] - -(47) CometExchange -Input [5]: [i_category#22, i_class#21, i_brand#20, sum#70, isEmpty#71] -Arguments: hashpartitioning(i_category#22, i_class#21, i_brand#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(48) CometHashAggregate -Input [5]: [i_category#22, i_class#21, i_brand#20, sum#70, isEmpty#71] -Keys [3]: [i_category#22, i_class#21, i_brand#20] -Functions [1]: [sum(sumsales#33)] - -(49) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#72, d_qoy#73, d_moy#74, s_store_id#14, sum#75, isEmpty#76] - -(50) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#72, d_qoy#73, d_moy#74, s_store_id#14, sum#75, isEmpty#76] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#72, d_qoy#73, d_moy#74, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#77 * cast(ss_quantity#78 as decimal(10,0))), 0.00))] - -(51) CometHashAggregate -Input [3]: [i_category#22, i_class#21, sumsales#33] -Keys [2]: [i_category#22, i_class#21] -Functions [1]: [partial_sum(sumsales#33)] - -(52) CometExchange -Input [4]: [i_category#22, i_class#21, sum#79, isEmpty#80] -Arguments: hashpartitioning(i_category#22, i_class#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(53) CometHashAggregate -Input [4]: [i_category#22, i_class#21, sum#79, isEmpty#80] -Keys [2]: [i_category#22, i_class#21] -Functions [1]: [sum(sumsales#33)] - -(54) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#81, d_qoy#82, d_moy#83, s_store_id#14, sum#84, isEmpty#85] - -(55) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#81, d_qoy#82, d_moy#83, s_store_id#14, sum#84, isEmpty#85] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#81, d_qoy#82, d_moy#83, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#86 * cast(ss_quantity#87 as decimal(10,0))), 0.00))] - -(56) CometHashAggregate -Input [2]: [i_category#22, sumsales#33] -Keys [1]: [i_category#22] -Functions [1]: [partial_sum(sumsales#33)] - -(57) CometExchange -Input [3]: [i_category#22, sum#88, isEmpty#89] -Arguments: hashpartitioning(i_category#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(58) CometHashAggregate -Input [3]: [i_category#22, sum#88, isEmpty#89] -Keys [1]: [i_category#22] -Functions [1]: [sum(sumsales#33)] - -(59) ReusedExchange [Reuses operator id: 22] -Output [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#90, d_qoy#91, d_moy#92, s_store_id#14, sum#93, isEmpty#94] - -(60) CometHashAggregate -Input [10]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#90, d_qoy#91, d_moy#92, s_store_id#14, sum#93, isEmpty#94] -Keys [8]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#90, d_qoy#91, d_moy#92, s_store_id#14] -Functions [1]: [sum(coalesce((ss_sales_price#95 * cast(ss_quantity#96 as decimal(10,0))), 0.00))] - -(61) CometHashAggregate -Input [1]: [sumsales#33] -Keys: [] -Functions [1]: [partial_sum(sumsales#33)] - -(62) CometExchange -Input [2]: [sum#97, isEmpty#98] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(63) CometHashAggregate -Input [2]: [sum#97, isEmpty#98] -Keys: [] -Functions [1]: [sum(sumsales#33)] - -(64) CometUnion -Child 0 Input [9]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107] -Child 1 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#26, d_qoy#27, d_moy#28, s_store_id#108, sumsales#109] -Child 2 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#36, d_qoy#37, d_moy#110, s_store_id#111, sumsales#112] -Child 3 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#45, d_qoy#113, d_moy#114, s_store_id#115, sumsales#116] -Child 4 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#23, d_year#117, d_qoy#118, d_moy#119, s_store_id#120, sumsales#121] -Child 5 Input [9]: [i_category#22, i_class#21, i_brand#20, i_product_name#122, d_year#123, d_qoy#124, d_moy#125, s_store_id#126, sumsales#127] -Child 6 Input [9]: [i_category#22, i_class#21, i_brand#128, i_product_name#129, d_year#130, d_qoy#131, d_moy#132, s_store_id#133, sumsales#134] -Child 7 Input [9]: [i_category#22, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141, sumsales#142] -Child 8 Input [9]: [i_category#143, i_class#144, i_brand#145, i_product_name#146, d_year#147, d_qoy#148, d_moy#149, s_store_id#150, sumsales#151] - -(65) CometExchange -Input [9]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107] -Arguments: hashpartitioning(i_category#99, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(66) CometSort -Input [9]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107] -Arguments: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107], [i_category#99 ASC NULLS FIRST, sumsales#107 DESC NULLS LAST] - -(67) CometColumnarToRow [codegen id : 1] -Input [9]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107] - -(68) Window -Input [9]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107] -Arguments: [rank(sumsales#107) windowspecdefinition(i_category#99, sumsales#107 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#152], [i_category#99], [sumsales#107 DESC NULLS LAST] - -(69) Filter [codegen id : 2] -Input [10]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] -Condition : (rk#152 <= 100) - -(70) TakeOrderedAndProject -Input [10]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] -Arguments: 100, [i_category#99 ASC NULLS FIRST, i_class#100 ASC NULLS FIRST, i_brand#101 ASC NULLS FIRST, i_product_name#102 ASC NULLS FIRST, d_year#103 ASC NULLS FIRST, d_qoy#104 ASC NULLS FIRST, d_moy#105 ASC NULLS FIRST, s_store_id#106 ASC NULLS FIRST, sumsales#107 ASC NULLS FIRST, rk#152 ASC NULLS FIRST], [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (75) -+- * CometColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (71) - - -(71) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(72) CometFilter -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Condition : (((isnotnull(d_month_seq#8) AND (d_month_seq#8 >= 1212)) AND (d_month_seq#8 <= 1223)) AND isnotnull(d_date_sk#7)) - -(73) CometProject -Input [5]: [d_date_sk#7, d_month_seq#8, d_year#9, d_moy#10, d_qoy#11] -Arguments: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11], [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(74) CometColumnarToRow [codegen id : 1] -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] - -(75) BroadcastExchange -Input [4]: [d_date_sk#7, d_year#9, d_moy#10, d_qoy#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/extended.txt deleted file mode 100644 index 1735511c2a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/extended.txt +++ /dev/null @@ -1,294 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 261 out of 282 eligible operators (92%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 5ca258aeca..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt +++ /dev/null @@ -1,82 +0,0 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (2) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometExchange [i_category] #1 - CometUnion [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] - 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,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_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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] - CometHashAggregate [s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] - CometHashAggregate [d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year] #9 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] - CometHashAggregate [d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name] #10 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,sum,isEmpty] - CometHashAggregate [d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand] #11 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,sum,isEmpty] - CometHashAggregate [i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class] #12 - CometHashAggregate [sumsales] [i_category,i_class,sum,isEmpty] - CometHashAggregate [i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category] #13 - CometHashAggregate [sumsales] [i_category,sum,isEmpty] - CometHashAggregate [i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange #14 - CometHashAggregate [sumsales] [sum,isEmpty] - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/explain.txt deleted file mode 100644 index 7afb402b0b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/explain.txt +++ /dev/null @@ -1,399 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (64) -+- * Project (63) - +- Window (62) - +- * CometColumnarToRow (61) - +- CometSort (60) - +- CometExchange (59) - +- CometHashAggregate (58) - +- CometColumnarExchange (57) - +- * HashAggregate (56) - +- Union (55) - :- * HashAggregate (40) - : +- * CometColumnarToRow (39) - : +- CometColumnarExchange (38) - : +- * HashAggregate (37) - : +- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (34) - : +- * Project (33) - : +- * BroadcastHashJoin LeftSemi BuildRight (32) - : :- * CometColumnarToRow (9) - : : +- CometFilter (8) - : : +- CometNativeScan parquet spark_catalog.default.store (7) - : +- BroadcastExchange (31) - : +- * Project (30) - : +- * Filter (29) - : +- Window (28) - : +- * Sort (27) - : +- * HashAggregate (26) - : +- * CometColumnarToRow (25) - : +- CometColumnarExchange (24) - : +- * HashAggregate (23) - : +- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet spark_catalog.default.store_sales (10) - : : +- BroadcastExchange (17) - : : +- * CometColumnarToRow (16) - : : +- CometProject (15) - : : +- CometFilter (14) - : : +- CometNativeScan parquet spark_catalog.default.store (13) - : +- ReusedExchange (20) - :- * HashAggregate (47) - : +- * CometColumnarToRow (46) - : +- CometColumnarExchange (45) - : +- * HashAggregate (44) - : +- * HashAggregate (43) - : +- * CometColumnarToRow (42) - : +- ReusedExchange (41) - +- * HashAggregate (54) - +- * CometColumnarToRow (53) - +- CometColumnarExchange (52) - +- * HashAggregate (51) - +- * HashAggregate (50) - +- * CometColumnarToRow (49) - +- ReusedExchange (48) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 8] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] - -(3) Filter [codegen id : 8] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(4) ReusedExchange [Reuses operator id: 69] -Output [1]: [d_date_sk#5] - -(5) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 8] -Output [2]: [ss_store_sk#1, ss_net_profit#2] -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [3]: [s_store_sk#6, s_county#7, s_state#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [s_store_sk#6, s_county#7, s_state#8] -Condition : isnotnull(s_store_sk#6) - -(9) CometColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#6, s_county#7, s_state#8] - -(10) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(11) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] - -(12) Filter [codegen id : 4] -Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_store_sk#9) - -(13) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#12, s_state#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [s_store_sk#12, s_state#13] -Condition : isnotnull(s_store_sk#12) - -(15) CometProject -Input [2]: [s_store_sk#12, s_state#13] -Arguments: [s_store_sk#12, s_state#14], [s_store_sk#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#13, 2, true, false, true) AS s_state#14] - -(16) CometColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#12, s_state#14] - -(17) BroadcastExchange -Input [2]: [s_store_sk#12, s_state#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#9] -Right keys [1]: [s_store_sk#12] -Join type: Inner -Join condition: None - -(19) Project [codegen id : 4] -Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14] -Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#12, s_state#14] - -(20) ReusedExchange [Reuses operator id: 69] -Output [1]: [d_date_sk#15] - -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#15] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 4] -Output [2]: [ss_net_profit#10, s_state#14] -Input [4]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14, d_date_sk#15] - -(23) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#10, s_state#14] -Keys [1]: [s_state#14] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum#16] -Results [2]: [s_state#14, sum#17] - -(24) CometColumnarExchange -Input [2]: [s_state#14, sum#17] -Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(25) CometColumnarToRow [codegen id : 5] -Input [2]: [s_state#14, sum#17] - -(26) HashAggregate [codegen id : 5] -Input [2]: [s_state#14, sum#17] -Keys [1]: [s_state#14] -Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#18] -Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#18,17,2) AS _w0#19, s_state#14] - -(27) Sort [codegen id : 5] -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 - -(28) Window -Input [3]: [s_state#14, _w0#19, s_state#14] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#14, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#14], [_w0#19 DESC NULLS LAST] - -(29) Filter [codegen id : 6] -Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] -Condition : (ranking#20 <= 5) - -(30) Project [codegen id : 6] -Output [1]: [s_state#14] -Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] - -(31) BroadcastExchange -Input [1]: [s_state#14] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true)] -Right keys [1]: [s_state#14] -Join type: LeftSemi -Join condition: None - -(33) Project [codegen id : 7] -Output [3]: [s_store_sk#6, s_county#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#8, 2, true, false, true) AS s_state#21] -Input [3]: [s_store_sk#6, s_county#7, s_state#8] - -(34) BroadcastExchange -Input [3]: [s_store_sk#6, s_county#7, s_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#6] -Join type: Inner -Join condition: None - -(36) Project [codegen id : 8] -Output [3]: [ss_net_profit#2, s_county#7, s_state#21] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#21] - -(37) HashAggregate [codegen id : 8] -Input [3]: [ss_net_profit#2, s_county#7, s_state#21] -Keys [2]: [s_state#21, s_county#7] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [s_state#21, s_county#7, sum#23] - -(38) CometColumnarExchange -Input [3]: [s_state#21, s_county#7, sum#23] -Arguments: hashpartitioning(s_state#21, s_county#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(39) CometColumnarToRow [codegen id : 9] -Input [3]: [s_state#21, s_county#7, sum#23] - -(40) HashAggregate [codegen id : 9] -Input [3]: [s_state#21, s_county#7, sum#23] -Keys [2]: [s_state#21, s_county#7] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) as decimal(27,2)) AS total_sum#25, s_state#21 AS s_state#26, s_county#7 AS s_county#27, 0 AS g_state#28, 0 AS g_county#29, 0 AS lochierarchy#30] - -(41) ReusedExchange [Reuses operator id: 38] -Output [3]: [s_state#21, s_county#31, sum#32] - -(42) CometColumnarToRow [codegen id : 18] -Input [3]: [s_state#21, s_county#31, sum#32] - -(43) HashAggregate [codegen id : 18] -Input [3]: [s_state#21, s_county#31, sum#32] -Keys [2]: [s_state#21, s_county#31] -Functions [1]: [sum(UnscaledValue(ss_net_profit#33))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#33))#24] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#33))#24,17,2) AS total_sum#34, s_state#21] - -(44) HashAggregate [codegen id : 18] -Input [2]: [total_sum#34, s_state#21] -Keys [1]: [s_state#21] -Functions [1]: [partial_sum(total_sum#34)] -Aggregate Attributes [2]: [sum#35, isEmpty#36] -Results [3]: [s_state#21, sum#37, isEmpty#38] - -(45) CometColumnarExchange -Input [3]: [s_state#21, sum#37, isEmpty#38] -Arguments: hashpartitioning(s_state#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(46) CometColumnarToRow [codegen id : 19] -Input [3]: [s_state#21, sum#37, isEmpty#38] - -(47) HashAggregate [codegen id : 19] -Input [3]: [s_state#21, sum#37, isEmpty#38] -Keys [1]: [s_state#21] -Functions [1]: [sum(total_sum#34)] -Aggregate Attributes [1]: [sum(total_sum#34)#39] -Results [6]: [sum(total_sum#34)#39 AS total_sum#40, s_state#21, null AS s_county#41, 0 AS g_state#42, 1 AS g_county#43, 1 AS lochierarchy#44] - -(48) ReusedExchange [Reuses operator id: 38] -Output [3]: [s_state#21, s_county#45, sum#46] - -(49) CometColumnarToRow [codegen id : 28] -Input [3]: [s_state#21, s_county#45, sum#46] - -(50) HashAggregate [codegen id : 28] -Input [3]: [s_state#21, s_county#45, sum#46] -Keys [2]: [s_state#21, s_county#45] -Functions [1]: [sum(UnscaledValue(ss_net_profit#47))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#47))#24] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#47))#24,17,2) AS total_sum#34] - -(51) HashAggregate [codegen id : 28] -Input [1]: [total_sum#34] -Keys: [] -Functions [1]: [partial_sum(total_sum#34)] -Aggregate Attributes [2]: [sum#48, isEmpty#49] -Results [2]: [sum#50, isEmpty#51] - -(52) CometColumnarExchange -Input [2]: [sum#50, isEmpty#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(53) CometColumnarToRow [codegen id : 29] -Input [2]: [sum#50, isEmpty#51] - -(54) HashAggregate [codegen id : 29] -Input [2]: [sum#50, isEmpty#51] -Keys: [] -Functions [1]: [sum(total_sum#34)] -Aggregate Attributes [1]: [sum(total_sum#34)#52] -Results [6]: [sum(total_sum#34)#52 AS total_sum#53, null AS s_state#54, null AS s_county#55, 1 AS g_state#56, 1 AS g_county#57, 2 AS lochierarchy#58] - -(55) Union - -(56) HashAggregate [codegen id : 30] -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Functions: [] -Aggregate Attributes: [] -Results [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] - -(57) CometColumnarExchange -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Arguments: hashpartitioning(total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(58) CometHashAggregate -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Functions: [] - -(59) CometExchange -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] -Arguments: hashpartitioning(lochierarchy#30, _w0#59, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(60) CometSort -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] -Arguments: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59], [lochierarchy#30 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] - -(61) CometColumnarToRow [codegen id : 31] -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] - -(62) Window -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] -Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#30, _w0#59, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#60], [lochierarchy#30, _w0#59], [total_sum#25 DESC NULLS LAST] - -(63) Project [codegen id : 32] -Output [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#60] -Input [6]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59, rank_within_parent#60] - -(64) TakeOrderedAndProject -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#60] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#60 ASC NULLS FIRST], [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#60] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (69) -+- * CometColumnarToRow (68) - +- CometProject (67) - +- CometFilter (66) - +- CometNativeScan parquet spark_catalog.default.date_dim (65) - - -(65) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#61] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(66) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#61] -Condition : (((isnotnull(d_month_seq#61) AND (d_month_seq#61 >= 1212)) AND (d_month_seq#61 <= 1223)) AND isnotnull(d_date_sk#5)) - -(67) CometProject -Input [2]: [d_date_sk#5, d_month_seq#61] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(68) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(69) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/simplified.txt deleted file mode 100644 index dcbf0ae4c8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/simplified.txt +++ /dev/null @@ -1,106 +0,0 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (32) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (31) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] - CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (30) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (9) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - WholeStageCodegen (8) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (7) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (6) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WholeStageCodegen (5) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #7 - WholeStageCodegen (4) - HashAggregate [s_state,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_state] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_net_profit,ss_sold_date_sk,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_state] - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (19) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #9 - WholeStageCodegen (18) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (29) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (28) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/explain.txt deleted file mode 100644 index 02cdbb3c66..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,403 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (65) -+- * Project (64) - +- Window (63) - +- * CometColumnarToRow (62) - +- CometSort (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometColumnarExchange (58) - +- * HashAggregate (57) - +- Union (56) - :- * HashAggregate (41) - : +- * CometColumnarToRow (40) - : +- CometColumnarExchange (39) - : +- * HashAggregate (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * CometColumnarToRow (9) - : : +- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- * BroadcastHashJoin LeftSemi BuildRight (33) - : :- * CometColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) - : +- BroadcastExchange (32) - : +- * Project (31) - : +- * Filter (30) - : +- Window (29) - : +- * CometColumnarToRow (28) - : +- CometSort (27) - : +- CometHashAggregate (26) - : +- CometExchange (25) - : +- CometHashAggregate (24) - : +- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (20) - : : +- CometBroadcastHashJoin (19) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - : : +- CometBroadcastExchange (18) - : : +- CometProject (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - : +- ReusedExchange (21) - :- * HashAggregate (48) - : +- * CometColumnarToRow (47) - : +- CometColumnarExchange (46) - : +- * HashAggregate (45) - : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- ReusedExchange (42) - +- * HashAggregate (55) - +- * CometColumnarToRow (54) - +- CometColumnarExchange (53) - +- * HashAggregate (52) - +- * HashAggregate (51) - +- * CometColumnarToRow (50) - +- ReusedExchange (49) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] - -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ss_store_sk#1, ss_net_profit#2] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#7, s_county#8, s_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Condition : isnotnull(s_store_sk#7) - -(12) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_store_sk#10) - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : isnotnull(s_store_sk#14) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14, s_state#16], [s_store_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) AS s_state#16] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#14, s_state#16] -Arguments: [s_store_sk#14, s_state#16] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Right output [2]: [s_store_sk#14, s_state#16] -Arguments: [ss_store_sk#10], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#16] -Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16], [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#17] - -(22) CometBroadcastHashJoin -Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] -Right output [1]: [d_date_sk#17] -Arguments: [ss_sold_date_sk#12], [d_date_sk#17], Inner, BuildRight - -(23) CometProject -Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16, d_date_sk#17] -Arguments: [ss_net_profit#11, s_state#16], [ss_net_profit#11, s_state#16] - -(24) CometHashAggregate -Input [2]: [ss_net_profit#11, s_state#16] -Keys [1]: [s_state#16] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] - -(25) CometExchange -Input [2]: [s_state#16, sum#18] -Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(26) CometHashAggregate -Input [2]: [s_state#16, sum#18] -Keys [1]: [s_state#16] -Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] - -(27) CometSort -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16, _w0#19, s_state#16], [s_state#16 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] - -(28) CometColumnarToRow [codegen id : 1] -Input [3]: [s_state#16, _w0#19, s_state#16] - -(29) Window -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#16, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#16], [_w0#19 DESC NULLS LAST] - -(30) Filter [codegen id : 2] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] -Condition : (ranking#20 <= 5) - -(31) Project [codegen id : 2] -Output [1]: [s_state#16] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] - -(32) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] - -(33) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)] -Right keys [1]: [s_state#16] -Join type: LeftSemi -Join condition: None - -(34) Project [codegen id : 3] -Output [3]: [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#21] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(35) BroadcastExchange -Input [3]: [s_store_sk#7, s_county#8, s_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(36) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_county#8, s_state#21] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#21] - -(38) HashAggregate [codegen id : 4] -Input [3]: [ss_net_profit#2, s_county#8, s_state#21] -Keys [2]: [s_state#21, s_county#8] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [s_state#21, s_county#8, sum#23] - -(39) CometColumnarExchange -Input [3]: [s_state#21, s_county#8, sum#23] -Arguments: hashpartitioning(s_state#21, s_county#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(40) CometColumnarToRow [codegen id : 5] -Input [3]: [s_state#21, s_county#8, sum#23] - -(41) HashAggregate [codegen id : 5] -Input [3]: [s_state#21, s_county#8, sum#23] -Keys [2]: [s_state#21, s_county#8] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) as decimal(27,2)) AS total_sum#25, s_state#21 AS s_state#26, s_county#8 AS s_county#27, 0 AS g_state#28, 0 AS g_county#29, 0 AS lochierarchy#30] - -(42) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_state#21, s_county#31, sum#32] - -(43) CometColumnarToRow [codegen id : 10] -Input [3]: [s_state#21, s_county#31, sum#32] - -(44) HashAggregate [codegen id : 10] -Input [3]: [s_state#21, s_county#31, sum#32] -Keys [2]: [s_state#21, s_county#31] -Functions [1]: [sum(UnscaledValue(ss_net_profit#33))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#33))#24] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#33))#24,17,2) AS total_sum#34, s_state#21] - -(45) HashAggregate [codegen id : 10] -Input [2]: [total_sum#34, s_state#21] -Keys [1]: [s_state#21] -Functions [1]: [partial_sum(total_sum#34)] -Aggregate Attributes [2]: [sum#35, isEmpty#36] -Results [3]: [s_state#21, sum#37, isEmpty#38] - -(46) CometColumnarExchange -Input [3]: [s_state#21, sum#37, isEmpty#38] -Arguments: hashpartitioning(s_state#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(47) CometColumnarToRow [codegen id : 11] -Input [3]: [s_state#21, sum#37, isEmpty#38] - -(48) HashAggregate [codegen id : 11] -Input [3]: [s_state#21, sum#37, isEmpty#38] -Keys [1]: [s_state#21] -Functions [1]: [sum(total_sum#34)] -Aggregate Attributes [1]: [sum(total_sum#34)#39] -Results [6]: [sum(total_sum#34)#39 AS total_sum#40, s_state#21, null AS s_county#41, 0 AS g_state#42, 1 AS g_county#43, 1 AS lochierarchy#44] - -(49) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_state#21, s_county#45, sum#46] - -(50) CometColumnarToRow [codegen id : 16] -Input [3]: [s_state#21, s_county#45, sum#46] - -(51) HashAggregate [codegen id : 16] -Input [3]: [s_state#21, s_county#45, sum#46] -Keys [2]: [s_state#21, s_county#45] -Functions [1]: [sum(UnscaledValue(ss_net_profit#47))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#47))#24] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#47))#24,17,2) AS total_sum#34] - -(52) HashAggregate [codegen id : 16] -Input [1]: [total_sum#34] -Keys: [] -Functions [1]: [partial_sum(total_sum#34)] -Aggregate Attributes [2]: [sum#48, isEmpty#49] -Results [2]: [sum#50, isEmpty#51] - -(53) CometColumnarExchange -Input [2]: [sum#50, isEmpty#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(54) CometColumnarToRow [codegen id : 17] -Input [2]: [sum#50, isEmpty#51] - -(55) HashAggregate [codegen id : 17] -Input [2]: [sum#50, isEmpty#51] -Keys: [] -Functions [1]: [sum(total_sum#34)] -Aggregate Attributes [1]: [sum(total_sum#34)#52] -Results [6]: [sum(total_sum#34)#52 AS total_sum#53, null AS s_state#54, null AS s_county#55, 1 AS g_state#56, 1 AS g_county#57, 2 AS lochierarchy#58] - -(56) Union - -(57) HashAggregate [codegen id : 18] -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Functions: [] -Aggregate Attributes: [] -Results [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] - -(58) CometColumnarExchange -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Arguments: hashpartitioning(total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(59) CometHashAggregate -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Functions: [] - -(60) CometExchange -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] -Arguments: hashpartitioning(lochierarchy#30, _w0#59, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(61) CometSort -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] -Arguments: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59], [lochierarchy#30 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] - -(62) CometColumnarToRow [codegen id : 19] -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] - -(63) Window -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] -Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#30, _w0#59, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#60], [lochierarchy#30, _w0#59], [total_sum#25 DESC NULLS LAST] - -(64) Project [codegen id : 20] -Output [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#60] -Input [6]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59, rank_within_parent#60] - -(65) TakeOrderedAndProject -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#60] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#60 ASC NULLS FIRST], [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#60] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (70) -+- * CometColumnarToRow (69) - +- CometProject (68) - +- CometFilter (67) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) - - -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(67) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(68) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(69) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(70) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 8de8b7e637..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,100 +0,0 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (20) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] - CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (18) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [s_state,_w0] - CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] - CometExchange [s_state] #8 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - 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,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #9 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (11) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #10 - WholeStageCodegen (10) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #11 - WholeStageCodegen (16) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt deleted file mode 100644 index 02cdbb3c66..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt +++ /dev/null @@ -1,403 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (65) -+- * Project (64) - +- Window (63) - +- * CometColumnarToRow (62) - +- CometSort (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometColumnarExchange (58) - +- * HashAggregate (57) - +- Union (56) - :- * HashAggregate (41) - : +- * CometColumnarToRow (40) - : +- CometColumnarExchange (39) - : +- * HashAggregate (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * CometColumnarToRow (9) - : : +- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- * BroadcastHashJoin LeftSemi BuildRight (33) - : :- * CometColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) - : +- BroadcastExchange (32) - : +- * Project (31) - : +- * Filter (30) - : +- Window (29) - : +- * CometColumnarToRow (28) - : +- CometSort (27) - : +- CometHashAggregate (26) - : +- CometExchange (25) - : +- CometHashAggregate (24) - : +- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (20) - : : +- CometBroadcastHashJoin (19) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - : : +- CometBroadcastExchange (18) - : : +- CometProject (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - : +- ReusedExchange (21) - :- * HashAggregate (48) - : +- * CometColumnarToRow (47) - : +- CometColumnarExchange (46) - : +- * HashAggregate (45) - : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- ReusedExchange (42) - +- * HashAggregate (55) - +- * CometColumnarToRow (54) - +- CometColumnarExchange (53) - +- * HashAggregate (52) - +- * HashAggregate (51) - +- * CometColumnarToRow (50) - +- ReusedExchange (49) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] -Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] - -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ss_store_sk#1, ss_net_profit#2] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#7, s_county#8, s_state#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(11) CometFilter -Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Condition : isnotnull(s_store_sk#7) - -(12) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(14) CometFilter -Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_store_sk#10) - -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#14, s_state#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(16) CometFilter -Input [2]: [s_store_sk#14, s_state#15] -Condition : isnotnull(s_store_sk#14) - -(17) CometProject -Input [2]: [s_store_sk#14, s_state#15] -Arguments: [s_store_sk#14, s_state#16], [s_store_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) AS s_state#16] - -(18) CometBroadcastExchange -Input [2]: [s_store_sk#14, s_state#16] -Arguments: [s_store_sk#14, s_state#16] - -(19) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] -Right output [2]: [s_store_sk#14, s_state#16] -Arguments: [ss_store_sk#10], [s_store_sk#14], Inner, BuildRight - -(20) CometProject -Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#16] -Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16], [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] - -(21) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#17] - -(22) CometBroadcastHashJoin -Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] -Right output [1]: [d_date_sk#17] -Arguments: [ss_sold_date_sk#12], [d_date_sk#17], Inner, BuildRight - -(23) CometProject -Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16, d_date_sk#17] -Arguments: [ss_net_profit#11, s_state#16], [ss_net_profit#11, s_state#16] - -(24) CometHashAggregate -Input [2]: [ss_net_profit#11, s_state#16] -Keys [1]: [s_state#16] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] - -(25) CometExchange -Input [2]: [s_state#16, sum#18] -Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(26) CometHashAggregate -Input [2]: [s_state#16, sum#18] -Keys [1]: [s_state#16] -Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] - -(27) CometSort -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16, _w0#19, s_state#16], [s_state#16 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] - -(28) CometColumnarToRow [codegen id : 1] -Input [3]: [s_state#16, _w0#19, s_state#16] - -(29) Window -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#16, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#16], [_w0#19 DESC NULLS LAST] - -(30) Filter [codegen id : 2] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] -Condition : (ranking#20 <= 5) - -(31) Project [codegen id : 2] -Output [1]: [s_state#16] -Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] - -(32) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] - -(33) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)] -Right keys [1]: [s_state#16] -Join type: LeftSemi -Join condition: None - -(34) Project [codegen id : 3] -Output [3]: [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#21] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(35) BroadcastExchange -Input [3]: [s_store_sk#7, s_county#8, s_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(36) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_county#8, s_state#21] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#21] - -(38) HashAggregate [codegen id : 4] -Input [3]: [ss_net_profit#2, s_county#8, s_state#21] -Keys [2]: [s_state#21, s_county#8] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [s_state#21, s_county#8, sum#23] - -(39) CometColumnarExchange -Input [3]: [s_state#21, s_county#8, sum#23] -Arguments: hashpartitioning(s_state#21, s_county#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(40) CometColumnarToRow [codegen id : 5] -Input [3]: [s_state#21, s_county#8, sum#23] - -(41) HashAggregate [codegen id : 5] -Input [3]: [s_state#21, s_county#8, sum#23] -Keys [2]: [s_state#21, s_county#8] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) as decimal(27,2)) AS total_sum#25, s_state#21 AS s_state#26, s_county#8 AS s_county#27, 0 AS g_state#28, 0 AS g_county#29, 0 AS lochierarchy#30] - -(42) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_state#21, s_county#31, sum#32] - -(43) CometColumnarToRow [codegen id : 10] -Input [3]: [s_state#21, s_county#31, sum#32] - -(44) HashAggregate [codegen id : 10] -Input [3]: [s_state#21, s_county#31, sum#32] -Keys [2]: [s_state#21, s_county#31] -Functions [1]: [sum(UnscaledValue(ss_net_profit#33))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#33))#24] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#33))#24,17,2) AS total_sum#34, s_state#21] - -(45) HashAggregate [codegen id : 10] -Input [2]: [total_sum#34, s_state#21] -Keys [1]: [s_state#21] -Functions [1]: [partial_sum(total_sum#34)] -Aggregate Attributes [2]: [sum#35, isEmpty#36] -Results [3]: [s_state#21, sum#37, isEmpty#38] - -(46) CometColumnarExchange -Input [3]: [s_state#21, sum#37, isEmpty#38] -Arguments: hashpartitioning(s_state#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(47) CometColumnarToRow [codegen id : 11] -Input [3]: [s_state#21, sum#37, isEmpty#38] - -(48) HashAggregate [codegen id : 11] -Input [3]: [s_state#21, sum#37, isEmpty#38] -Keys [1]: [s_state#21] -Functions [1]: [sum(total_sum#34)] -Aggregate Attributes [1]: [sum(total_sum#34)#39] -Results [6]: [sum(total_sum#34)#39 AS total_sum#40, s_state#21, null AS s_county#41, 0 AS g_state#42, 1 AS g_county#43, 1 AS lochierarchy#44] - -(49) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_state#21, s_county#45, sum#46] - -(50) CometColumnarToRow [codegen id : 16] -Input [3]: [s_state#21, s_county#45, sum#46] - -(51) HashAggregate [codegen id : 16] -Input [3]: [s_state#21, s_county#45, sum#46] -Keys [2]: [s_state#21, s_county#45] -Functions [1]: [sum(UnscaledValue(ss_net_profit#47))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#47))#24] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#47))#24,17,2) AS total_sum#34] - -(52) HashAggregate [codegen id : 16] -Input [1]: [total_sum#34] -Keys: [] -Functions [1]: [partial_sum(total_sum#34)] -Aggregate Attributes [2]: [sum#48, isEmpty#49] -Results [2]: [sum#50, isEmpty#51] - -(53) CometColumnarExchange -Input [2]: [sum#50, isEmpty#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(54) CometColumnarToRow [codegen id : 17] -Input [2]: [sum#50, isEmpty#51] - -(55) HashAggregate [codegen id : 17] -Input [2]: [sum#50, isEmpty#51] -Keys: [] -Functions [1]: [sum(total_sum#34)] -Aggregate Attributes [1]: [sum(total_sum#34)#52] -Results [6]: [sum(total_sum#34)#52 AS total_sum#53, null AS s_state#54, null AS s_county#55, 1 AS g_state#56, 1 AS g_county#57, 2 AS lochierarchy#58] - -(56) Union - -(57) HashAggregate [codegen id : 18] -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Functions: [] -Aggregate Attributes: [] -Results [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] - -(58) CometColumnarExchange -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Arguments: hashpartitioning(total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(59) CometHashAggregate -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Functions: [] - -(60) CometExchange -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] -Arguments: hashpartitioning(lochierarchy#30, _w0#59, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(61) CometSort -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] -Arguments: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59], [lochierarchy#30 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] - -(62) CometColumnarToRow [codegen id : 19] -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] - -(63) Window -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] -Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#30, _w0#59, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#60], [lochierarchy#30, _w0#59], [total_sum#25 DESC NULLS LAST] - -(64) Project [codegen id : 20] -Output [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#60] -Input [6]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59, rank_within_parent#60] - -(65) TakeOrderedAndProject -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#60] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#60 ASC NULLS FIRST], [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#60] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (70) -+- * CometColumnarToRow (69) - +- CometProject (68) - +- CometFilter (67) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) - - -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(67) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(68) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(69) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(70) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/extended.txt deleted file mode 100644 index a677fed73a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/extended.txt +++ /dev/null @@ -1,173 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 102 out of 153 eligible operators (66%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 8de8b7e637..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt +++ /dev/null @@ -1,100 +0,0 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (20) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] - CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (18) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - 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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [s_state,_w0] - CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] - CometExchange [s_state] #8 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - 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,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #9 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (11) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #10 - WholeStageCodegen (10) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #11 - WholeStageCodegen (16) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/explain.txt deleted file mode 100644 index 44a85d557d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/explain.txt +++ /dev/null @@ -1,428 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (70) -+- CometTakeOrderedAndProject (69) - +- CometHashAggregate (68) - +- CometExchange (67) - +- CometHashAggregate (66) - +- CometProject (65) - +- CometSortMergeJoin (64) - :- CometSort (58) - : +- CometColumnarExchange (57) - : +- * Project (56) - : +- * BroadcastHashJoin LeftOuter BuildRight (55) - : :- * Project (50) - : : +- * BroadcastHashJoin Inner BuildRight (49) - : : :- * Project (44) - : : : +- * BroadcastHashJoin Inner BuildRight (43) - : : : :- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (28) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Project (15) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : : :- * Project (9) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : : : : : : :- * Filter (3) - : : : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- BroadcastExchange (7) - : : : : : : : : : +- * CometColumnarToRow (6) - : : : : : : : : : +- CometFilter (5) - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory (4) - : : : : : : : : +- BroadcastExchange (13) - : : : : : : : : +- * CometColumnarToRow (12) - : : : : : : : : +- CometFilter (11) - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse (10) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * CometColumnarToRow (18) - : : : : : : : +- CometFilter (17) - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item (16) - : : : : : : +- BroadcastExchange (26) - : : : : : : +- * CometColumnarToRow (25) - : : : : : : +- CometProject (24) - : : : : : : +- CometFilter (23) - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics (22) - : : : : : +- BroadcastExchange (33) - : : : : : +- * CometColumnarToRow (32) - : : : : : +- CometProject (31) - : : : : : +- CometFilter (30) - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics (29) - : : : : +- ReusedExchange (36) - : : : +- BroadcastExchange (42) - : : : +- * CometColumnarToRow (41) - : : : +- CometFilter (40) - : : : +- CometNativeScan parquet spark_catalog.default.date_dim (39) - : : +- BroadcastExchange (48) - : : +- * CometColumnarToRow (47) - : : +- CometFilter (46) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (45) - : +- BroadcastExchange (54) - : +- * CometColumnarToRow (53) - : +- CometFilter (52) - : +- CometNativeScan parquet spark_catalog.default.promotion (51) - +- CometSort (63) - +- CometExchange (62) - +- CometProject (61) - +- CometFilter (60) - +- CometNativeScan parquet spark_catalog.default.catalog_returns (59) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 10] -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] - -(3) Filter [codegen id : 10] -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) - -(4) CometNativeScan parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Batched: true -Location: InMemoryFileIndex(0 paths)[] -PartitionFilters: [isnotnull(inv_date_sk#13)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(5) CometFilter -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) - -(6) CometColumnarToRow [codegen id : 1] -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(7) BroadcastExchange -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(8) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [inv_item_sk#10] -Join type: Inner -Join condition: (inv_quantity_on_hand#12 < cs_quantity#7) - -(9) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(10) CometNativeScan parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(11) CometFilter -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Condition : isnotnull(w_warehouse_sk#14) - -(12) CometColumnarToRow [codegen id : 2] -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] - -(13) BroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] - -(14) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_warehouse_sk#11] -Right keys [1]: [w_warehouse_sk#14] -Join type: Inner -Join condition: None - -(15) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] - -(16) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_desc#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [2]: [i_item_sk#16, i_item_desc#17] -Condition : isnotnull(i_item_sk#16) - -(18) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#16, i_item_desc#17] - -(19) BroadcastExchange -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(20) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#16] -Join type: Inner -Join condition: None - -(21) Project [codegen id : 10] -Output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] - -(22) CometNativeScan parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#18, cd_marital_status#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(23) CometFilter -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = M) AND isnotnull(cd_demo_sk#18)) - -(24) CometProject -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Arguments: [cd_demo_sk#18], [cd_demo_sk#18] - -(25) CometColumnarToRow [codegen id : 4] -Input [1]: [cd_demo_sk#18] - -(26) BroadcastExchange -Input [1]: [cd_demo_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(27) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#18] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] - -(29) CometNativeScan parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_buy_potential#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(30) CometFilter -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#21, 15, true, false, true) = 1001-5000 ) AND isnotnull(hd_demo_sk#20)) - -(31) CometProject -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Arguments: [hd_demo_sk#20], [hd_demo_sk#20] - -(32) CometColumnarToRow [codegen id : 5] -Input [1]: [hd_demo_sk#20] - -(33) BroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - -(34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_bill_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#20] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] - -(36) ReusedExchange [Reuses operator id: 75] -Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(37) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#8] -Right keys [1]: [d_date_sk#22] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 10] -Output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] - -(39) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_week_seq#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(40) CometFilter -Input [2]: [d_date_sk#25, d_week_seq#26] -Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) - -(41) CometColumnarToRow [codegen id : 7] -Input [2]: [d_date_sk#25, d_week_seq#26] - -(42) BroadcastExchange -Input [2]: [d_date_sk#25, d_week_seq#26] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] - -(43) BroadcastHashJoin [codegen id : 10] -Left keys [2]: [d_week_seq#24, inv_date_sk#13] -Right keys [2]: [d_week_seq#26, d_date_sk#25] -Join type: Inner -Join condition: None - -(44) Project [codegen id : 10] -Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#25, d_week_seq#26] - -(45) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#27, d_date#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [d_date_sk#27, d_date#28] -Condition : (isnotnull(d_date#28) AND isnotnull(d_date_sk#27)) - -(47) CometColumnarToRow [codegen id : 8] -Input [2]: [d_date_sk#27, d_date#28] - -(48) BroadcastExchange -Input [2]: [d_date_sk#27, d_date#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] - -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: (d_date#28 > date_add(d_date#23, 5)) - -(50) Project [codegen id : 10] -Output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#27, d_date#28] - -(51) CometNativeScan parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(52) CometFilter -Input [1]: [p_promo_sk#29] -Condition : isnotnull(p_promo_sk#29) - -(53) CometColumnarToRow [codegen id : 9] -Input [1]: [p_promo_sk#29] - -(54) BroadcastExchange -Input [1]: [p_promo_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(55) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_promo_sk#5] -Right keys [1]: [p_promo_sk#29] -Join type: LeftOuter -Join condition: None - -(56) Project [codegen id : 10] -Output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#29] - -(57) CometColumnarExchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(58) CometSort -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] - -(59) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(60) CometFilter -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Condition : (isnotnull(cr_item_sk#30) AND isnotnull(cr_order_number#31)) - -(61) CometProject -Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30, cr_order_number#31] - -(62) CometExchange -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(63) CometSort -Input [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST] - -(64) CometSortMergeJoin -Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#30, cr_order_number#31] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#30, cr_order_number#31], LeftOuter - -(65) CometProject -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#30, cr_order_number#31] -Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(66) CometHashAggregate -Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [partial_count(1)] - -(67) CometExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(68) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#33] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [count(1)] - -(69) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#36 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#34,promo#35,total_cnt#36]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36], 100, 0, [total_cnt#36 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] - -(70) CometColumnarToRow [codegen id : 11] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#34, promo#35, total_cnt#36] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (75) -+- * CometColumnarToRow (74) - +- CometProject (73) - +- CometFilter (72) - +- CometNativeScan parquet spark_catalog.default.date_dim (71) - - -(71) CometNativeScan parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(72) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] -Condition : ((((isnotnull(d_year#37) AND (d_year#37 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(73) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#37] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(74) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(75) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/simplified.txt deleted file mode 100644 index 025217a607..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/simplified.txt +++ /dev/null @@ -1,107 +0,0 @@ -WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] - CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] - CometProject [w_warehouse_name,i_item_desc,d_week_seq] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometColumnarExchange [cs_item_sk,cs_order_number] #2 - WholeStageCodegen (10) - Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_promo_sk,p_promo_sk] - Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk,d_date,d_date] - Project [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq] - BroadcastHashJoin [d_week_seq,inv_date_sk,d_week_seq,d_date_sk] - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Project [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] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [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] - BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,inv_item_sk,inv_quantity_on_hand,cs_quantity] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometNativeScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [w_warehouse_sk,w_warehouse_name] - CometNativeScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometFilter [i_item_sk,i_item_desc] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometNativeScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometNativeScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - InputAdapter - ReusedExchange [d_date_sk,d_date,d_week_seq] #3 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_week_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometFilter [p_promo_sk] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometNativeScan 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/q72.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/explain.txt deleted file mode 100644 index ee3242f2b1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,402 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (64) -+- CometTakeOrderedAndProject (63) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometSortMergeJoin (58) - :- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (45) - : : +- CometBroadcastHashJoin (44) - : : :- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (35) - : : : : +- CometBroadcastHashJoin (34) - : : : : :- CometProject (29) - : : : : : +- CometBroadcastHashJoin (28) - : : : : : :- CometProject (23) - : : : : : : +- CometBroadcastHashJoin (22) - : : : : : : :- CometProject (17) - : : : : : : : +- CometBroadcastHashJoin (16) - : : : : : : : :- CometProject (12) - : : : : : : : : +- CometBroadcastHashJoin (11) - : : : : : : : : :- CometProject (7) - : : : : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : : : : :- CometFilter (2) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- CometBroadcastExchange (5) - : : : : : : : : : +- CometFilter (4) - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (3) - : : : : : : : : +- CometBroadcastExchange (10) - : : : : : : : : +- CometFilter (9) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : : : : : : : +- CometBroadcastExchange (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (13) - : : : : : : +- CometBroadcastExchange (21) - : : : : : : +- CometProject (20) - : : : : : : +- CometFilter (19) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (18) - : : : : : +- CometBroadcastExchange (27) - : : : : : +- CometProject (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (24) - : : : : +- CometBroadcastExchange (33) - : : : : +- CometProject (32) - : : : : +- CometFilter (31) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - : : : +- CometBroadcastExchange (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (36) - : : +- CometBroadcastExchange (43) - : : +- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - : +- CometBroadcastExchange (48) - : +- CometFilter (47) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (46) - +- CometSort (57) - +- CometExchange (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (53) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#13)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) - -(5) CometBroadcastExchange -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(6) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Right output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_item_sk#4], [inv_item_sk#10], Inner, (inv_quantity_on_hand#12 < cs_quantity#7), BuildRight - -(7) CometProject -Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Condition : isnotnull(w_warehouse_sk#14) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [w_warehouse_sk#14, w_warehouse_name#15] - -(11) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -Right output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [inv_warehouse_sk#11], [w_warehouse_sk#14], Inner, BuildRight - -(12) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_desc#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [i_item_sk#16, i_item_desc#17] -Condition : isnotnull(i_item_sk#16) - -(15) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [i_item_sk#16, i_item_desc#17] - -(16) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -Right output [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [cs_item_sk#4], [i_item_sk#16], Inner, BuildRight - -(17) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#18, cd_marital_status#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = M) AND isnotnull(cd_demo_sk#18)) - -(20) CometProject -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Arguments: [cd_demo_sk#18], [cd_demo_sk#18] - -(21) CometBroadcastExchange -Input [1]: [cd_demo_sk#18] -Arguments: [cd_demo_sk#18] - -(22) CometBroadcastHashJoin -Left output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [cd_demo_sk#18] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#18], Inner, BuildRight - -(23) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] -Arguments: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_buy_potential#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#21, 15, true, false, true) = 1001-5000 ) AND isnotnull(hd_demo_sk#20)) - -(26) CometProject -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Arguments: [hd_demo_sk#20], [hd_demo_sk#20] - -(27) CometBroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: [hd_demo_sk#20] - -(28) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [hd_demo_sk#20] -Arguments: [cs_bill_hdemo_sk#3], [hd_demo_sk#20], Inner, BuildRight - -(29) CometProject -Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(31) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(32) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(33) CometBroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24] - -(34) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_sold_date_sk#8], [d_date_sk#22], Inner, BuildRight - -(35) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_week_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [d_date_sk#26, d_week_seq#27] -Condition : (isnotnull(d_week_seq#27) AND isnotnull(d_date_sk#26)) - -(38) CometBroadcastExchange -Input [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_date_sk#26, d_week_seq#27] - -(39) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_week_seq#24, inv_date_sk#13], [d_week_seq#27, d_date_sk#26], Inner, BuildRight - -(40) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#26, d_week_seq#27] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_date#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#28, d_date#29] -Condition : (isnotnull(d_date#29) AND isnotnull(d_date_sk#28)) - -(43) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_date#29] -Arguments: [d_date_sk#28, d_date#29] - -(44) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#28, d_date#29] -Arguments: [cs_ship_date_sk#1], [d_date_sk#28], Inner, (d_date#29 > date_add(d_date#23, 5)), BuildRight - -(45) CometProject -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#28, d_date#29] -Arguments: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(47) CometFilter -Input [1]: [p_promo_sk#30] -Condition : isnotnull(p_promo_sk#30) - -(48) CometBroadcastExchange -Input [1]: [p_promo_sk#30] -Arguments: [p_promo_sk#30] - -(49) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [1]: [p_promo_sk#30] -Arguments: [cs_promo_sk#5], [p_promo_sk#30], LeftOuter, BuildRight - -(50) CometProject -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#30] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(51) CometExchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(52) CometSort -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Condition : (isnotnull(cr_item_sk#31) AND isnotnull(cr_order_number#32)) - -(55) CometProject -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31, cr_order_number#32] - -(56) CometExchange -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(57) CometSort -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#31, cr_order_number#32], LeftOuter - -(59) CometProject -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#31, cr_order_number#32] -Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(60) CometHashAggregate -Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [partial_count(1)] - -(61) CometExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(62) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [count(1)] - -(63) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, 0, [total_cnt#37 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -(64) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (69) -+- * CometColumnarToRow (68) - +- CometProject (67) - +- CometFilter (66) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) - - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(67) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(68) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(69) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/simplified.txt deleted file mode 100644 index 678f8c67df..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] - CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] - CometProject [w_warehouse_name,i_item_desc,d_week_seq] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometExchange [cs_item_sk,cs_order_number] #2 - CometProject [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,p_promo_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_date] - 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 [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_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_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_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_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 [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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_desc] #6 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange [cd_demo_sk] #7 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange [hd_demo_sk] #8 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [d_date_sk,d_week_seq] #10 - CometFilter [d_date_sk,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [d_date_sk,d_date] #11 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [p_promo_sk] #12 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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/q72/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt deleted file mode 100644 index ee3242f2b1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt +++ /dev/null @@ -1,402 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (64) -+- CometTakeOrderedAndProject (63) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometSortMergeJoin (58) - :- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometBroadcastHashJoin (49) - : :- CometProject (45) - : : +- CometBroadcastHashJoin (44) - : : :- CometProject (40) - : : : +- CometBroadcastHashJoin (39) - : : : :- CometProject (35) - : : : : +- CometBroadcastHashJoin (34) - : : : : :- CometProject (29) - : : : : : +- CometBroadcastHashJoin (28) - : : : : : :- CometProject (23) - : : : : : : +- CometBroadcastHashJoin (22) - : : : : : : :- CometProject (17) - : : : : : : : +- CometBroadcastHashJoin (16) - : : : : : : : :- CometProject (12) - : : : : : : : : +- CometBroadcastHashJoin (11) - : : : : : : : : :- CometProject (7) - : : : : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : : : : :- CometFilter (2) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : : : +- CometBroadcastExchange (5) - : : : : : : : : : +- CometFilter (4) - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory (3) - : : : : : : : : +- CometBroadcastExchange (10) - : : : : : : : : +- CometFilter (9) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse (8) - : : : : : : : +- CometBroadcastExchange (15) - : : : : : : : +- CometFilter (14) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (13) - : : : : : : +- CometBroadcastExchange (21) - : : : : : : +- CometProject (20) - : : : : : : +- CometFilter (19) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (18) - : : : : : +- CometBroadcastExchange (27) - : : : : : +- CometProject (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (24) - : : : : +- CometBroadcastExchange (33) - : : : : +- CometProject (32) - : : : : +- CometFilter (31) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) - : : : +- CometBroadcastExchange (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (36) - : : +- CometBroadcastExchange (43) - : : +- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (41) - : +- CometBroadcastExchange (48) - : +- CometFilter (47) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (46) - +- CometSort (57) - +- CometExchange (56) - +- CometProject (55) - +- CometFilter (54) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (53) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] -PushedFilters: [IsNotNull(cs_quantity), IsNotNull(cs_item_sk), IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_hdemo_sk), IsNotNull(cs_ship_date_sk)] -ReadSchema: struct - -(2) CometFilter -Input [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotnull(cs_bill_cdemo_sk#2)) AND isnotnull(cs_bill_hdemo_sk#3)) AND isnotnull(cs_ship_date_sk#1)) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory -Output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#13)] -PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] -ReadSchema: struct - -(4) CometFilter -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) - -(5) CometBroadcastExchange -Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] - -(6) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] -Right output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_item_sk#4], [inv_item_sk#10], Inner, (inv_quantity_on_hand#12 < cs_quantity#7), BuildRight - -(7) CometProject -Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] - -(8) CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse -Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/warehouse] -PushedFilters: [IsNotNull(w_warehouse_sk)] -ReadSchema: struct - -(9) CometFilter -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Condition : isnotnull(w_warehouse_sk#14) - -(10) CometBroadcastExchange -Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [w_warehouse_sk#14, w_warehouse_name#15] - -(11) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] -Right output [2]: [w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [inv_warehouse_sk#11], [w_warehouse_sk#14], Inner, BuildRight - -(12) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13, w_warehouse_sk#14, w_warehouse_name#15] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#16, i_item_desc#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [i_item_sk#16, i_item_desc#17] -Condition : isnotnull(i_item_sk#16) - -(15) CometBroadcastExchange -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [i_item_sk#16, i_item_desc#17] - -(16) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] -Right output [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [cs_item_sk#4], [i_item_sk#16], Inner, BuildRight - -(17) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_sk#16, i_item_desc#17] -Arguments: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#18, cd_marital_status#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cd_marital_status#19, 1, true, false, true) = M) AND isnotnull(cd_demo_sk#18)) - -(20) CometProject -Input [2]: [cd_demo_sk#18, cd_marital_status#19] -Arguments: [cd_demo_sk#18], [cd_demo_sk#18] - -(21) CometBroadcastExchange -Input [1]: [cd_demo_sk#18] -Arguments: [cd_demo_sk#18] - -(22) CometBroadcastHashJoin -Left output [10]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [cd_demo_sk#18] -Arguments: [cs_bill_cdemo_sk#2], [cd_demo_sk#18], Inner, BuildRight - -(23) CometProject -Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, cd_demo_sk#18] -Arguments: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#20, hd_buy_potential#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_demo_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, hd_buy_potential#21, 15, true, false, true) = 1001-5000 ) AND isnotnull(hd_demo_sk#20)) - -(26) CometProject -Input [2]: [hd_demo_sk#20, hd_buy_potential#21] -Arguments: [hd_demo_sk#20], [hd_demo_sk#20] - -(27) CometBroadcastExchange -Input [1]: [hd_demo_sk#20] -Arguments: [hd_demo_sk#20] - -(28) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [1]: [hd_demo_sk#20] -Arguments: [cs_bill_hdemo_sk#3], [hd_demo_sk#20], Inner, BuildRight - -(29) CometProject -Input [10]: [cs_ship_date_sk#1, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, hd_demo_sk#20] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(31) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(32) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(33) CometBroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24] - -(34) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] -Right output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_sold_date_sk#8], [d_date_sk#22], Inner, BuildRight - -(35) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#26, d_week_seq#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [d_date_sk#26, d_week_seq#27] -Condition : (isnotnull(d_week_seq#27) AND isnotnull(d_date_sk#26)) - -(38) CometBroadcastExchange -Input [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_date_sk#26, d_week_seq#27] - -(39) CometBroadcastHashJoin -Left output [9]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#26, d_week_seq#27] -Arguments: [d_week_seq#24, inv_date_sk#13], [d_week_seq#27, d_date_sk#26], Inner, BuildRight - -(40) CometProject -Input [11]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#26, d_week_seq#27] -Arguments: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24], [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#28, d_date#29] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] -ReadSchema: struct - -(42) CometFilter -Input [2]: [d_date_sk#28, d_date#29] -Condition : (isnotnull(d_date#29) AND isnotnull(d_date_sk#28)) - -(43) CometBroadcastExchange -Input [2]: [d_date_sk#28, d_date#29] -Arguments: [d_date_sk#28, d_date#29] - -(44) CometBroadcastHashJoin -Left output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24] -Right output [2]: [d_date_sk#28, d_date#29] -Arguments: [cs_ship_date_sk#1], [d_date_sk#28], Inner, (d_date#29 > date_add(d_date#23, 5)), BuildRight - -(45) CometProject -Input [10]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_date#23, d_week_seq#24, d_date_sk#28, d_date#29] -Arguments: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(47) CometFilter -Input [1]: [p_promo_sk#30] -Condition : isnotnull(p_promo_sk#30) - -(48) CometBroadcastExchange -Input [1]: [p_promo_sk#30] -Arguments: [p_promo_sk#30] - -(49) CometBroadcastHashJoin -Left output [6]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [1]: [p_promo_sk#30] -Arguments: [cs_promo_sk#5], [p_promo_sk#30], LeftOuter, BuildRight - -(50) CometProject -Input [7]: [cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, p_promo_sk#30] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(51) CometExchange -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(52) CometSort -Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Arguments: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(54) CometFilter -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Condition : (isnotnull(cr_item_sk#31) AND isnotnull(cr_order_number#32)) - -(55) CometProject -Input [3]: [cr_item_sk#31, cr_order_number#32, cr_returned_date_sk#33] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31, cr_order_number#32] - -(56) CometExchange -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: hashpartitioning(cr_item_sk#31, cr_order_number#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(57) CometSort -Input [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cr_item_sk#31, cr_order_number#32], [cr_item_sk#31 ASC NULLS FIRST, cr_order_number#32 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Right output [2]: [cr_item_sk#31, cr_order_number#32] -Arguments: [cs_item_sk#4, cs_order_number#6], [cr_item_sk#31, cr_order_number#32], LeftOuter - -(59) CometProject -Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#31, cr_order_number#32] -Arguments: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24], [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] - -(60) CometHashAggregate -Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [partial_count(1)] - -(61) CometExchange -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(62) CometHashAggregate -Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] -Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] -Functions [1]: [count(1)] - -(63) CometTakeOrderedAndProject -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_cnt#37 DESC NULLS LAST,i_item_desc#17 ASC NULLS FIRST,w_warehouse_name#15 ASC NULLS FIRST,d_week_seq#24 ASC NULLS FIRST], output=[i_item_desc#17,w_warehouse_name#15,d_week_seq#24,no_promo#35,promo#36,total_cnt#37]), [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37], 100, 0, [total_cnt#37 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -(64) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#35, promo#36, total_cnt#37] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (69) -+- * CometColumnarToRow (68) - +- CometProject (67) - +- CometFilter (66) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) - - -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] -ReadSchema: struct - -(66) CometFilter -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) - -(67) CometProject -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] - -(68) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] - -(69) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/extended.txt deleted file mode 100644 index 9000d27d1b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/extended.txt +++ /dev/null @@ -1,72 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 678f8c67df..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/simplified.txt +++ /dev/null @@ -1,74 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt] - CometHashAggregate [count] [i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo,total_cnt,count(1)] - CometExchange [i_item_desc,w_warehouse_name,d_week_seq] #1 - CometHashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] - CometProject [w_warehouse_name,i_item_desc,d_week_seq] - CometSortMergeJoin [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,cr_item_sk,cr_order_number] - CometSort [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometExchange [cs_item_sk,cs_order_number] #2 - CometProject [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq,p_promo_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - CometBroadcastHashJoin [cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_date,d_week_seq,d_date_sk,d_date] - 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 [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_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_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_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_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 [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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - 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 [native_iceberg_compat] parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] - CometBroadcastExchange [w_warehouse_sk,w_warehouse_name] #5 - CometFilter [w_warehouse_sk,w_warehouse_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] - CometBroadcastExchange [i_item_sk,i_item_desc] #6 - CometFilter [i_item_sk,i_item_desc] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc] - CometBroadcastExchange [cd_demo_sk] #7 - CometProject [cd_demo_sk] - CometFilter [cd_demo_sk,cd_marital_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] - CometBroadcastExchange [hd_demo_sk] #8 - CometProject [hd_demo_sk] - CometFilter [hd_demo_sk,hd_buy_potential] - CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] - CometBroadcastExchange [d_date_sk,d_date,d_week_seq] #9 - CometProject [d_date_sk,d_date,d_week_seq] - CometFilter [d_date_sk,d_date,d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - CometBroadcastExchange [d_date_sk,d_week_seq] #10 - CometFilter [d_date_sk,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] - CometBroadcastExchange [d_date_sk,d_date] #11 - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [p_promo_sk] #12 - CometFilter [p_promo_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk] - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/explain.txt deleted file mode 100644 index 5d2bcfdc5c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/explain.txt +++ /dev/null @@ -1,513 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (79) -+- * Project (78) - +- * BroadcastHashJoin Inner BuildRight (77) - :- * Project (58) - : +- * BroadcastHashJoin Inner BuildRight (57) - : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (16) - : : : +- CometColumnarExchange (15) - : : : +- * HashAggregate (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * CometColumnarToRow (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet spark_catalog.default.customer (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Filter (7) - : : : : +- * ColumnarToRow (6) - : : : : +- Scan parquet spark_catalog.default.store_sales (5) - : : : +- ReusedExchange (11) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * Project (31) - : : +- * BroadcastHashJoin Inner BuildRight (30) - : : :- * Project (28) - : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : :- * CometColumnarToRow (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometNativeScan parquet spark_catalog.default.customer (19) - : : : +- BroadcastExchange (26) - : : : +- * Filter (25) - : : : +- * ColumnarToRow (24) - : : : +- Scan parquet spark_catalog.default.store_sales (23) - : : +- ReusedExchange (29) - : +- BroadcastExchange (56) - : +- * Filter (55) - : +- * HashAggregate (54) - : +- * CometColumnarToRow (53) - : +- CometColumnarExchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * CometColumnarToRow (41) - : : : +- CometProject (40) - : : : +- CometFilter (39) - : : : +- CometNativeScan parquet spark_catalog.default.customer (38) - : : +- BroadcastExchange (45) - : : +- * Filter (44) - : : +- * ColumnarToRow (43) - : : +- Scan parquet spark_catalog.default.web_sales (42) - : +- ReusedExchange (48) - +- BroadcastExchange (76) - +- * HashAggregate (75) - +- * CometColumnarToRow (74) - +- CometColumnarExchange (73) - +- * HashAggregate (72) - +- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (68) - : +- * BroadcastHashJoin Inner BuildRight (67) - : :- * CometColumnarToRow (62) - : : +- CometProject (61) - : : +- CometFilter (60) - : : +- CometNativeScan parquet spark_catalog.default.customer (59) - : +- BroadcastExchange (66) - : +- * Filter (65) - : +- * ColumnarToRow (64) - : +- Scan parquet spark_catalog.default.web_sales (63) - +- ReusedExchange (69) - - -(1) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#7] - -(4) CometColumnarToRow [codegen id : 3] -Input [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] - -(5) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(6) ColumnarToRow [codegen id : 1] -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(7) Filter [codegen id : 1] -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Condition : isnotnull(ss_customer_sk#8) - -(8) BroadcastExchange -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#8] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(11) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#12, d_year#13] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#10] -Right keys [1]: [d_date_sk#12] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] - -(14) HashAggregate [codegen id : 3] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] -Aggregate Attributes [1]: [sum#14] -Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] - -(15) CometColumnarExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 16] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] - -(17) HashAggregate [codegen id : 16] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#15] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#9))#16] -Results [2]: [c_customer_id#5 AS customer_id#17, MakeDecimal(sum(UnscaledValue(ss_net_paid#9))#16,17,2) AS year_total#18] - -(18) Filter [codegen id : 16] -Input [2]: [customer_id#17, year_total#18] -Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) - -(19) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(20) CometFilter -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Condition : (isnotnull(c_customer_sk#19) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#20, 16, true, false, true))) - -(21) CometProject -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Arguments: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#19, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#20, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#21, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#22, 30, true, false, true) AS c_last_name#7] - -(22) CometColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7] - -(23) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] - -(25) Filter [codegen id : 4] -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_customer_sk#23) - -(26) BroadcastExchange -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#19] -Right keys [1]: [ss_customer_sk#23] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25] -Input [7]: [c_customer_sk#19, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] - -(29) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#27, d_year#28] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#25] -Right keys [1]: [d_date_sk#27] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, ss_sold_date_sk#25, d_date_sk#27, d_year#28] - -(32) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#24, d_year#28] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#24))] -Aggregate Attributes [1]: [sum#29] -Results [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] - -(33) CometColumnarExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 7] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] - -(35) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28, sum#30] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#28] -Functions [1]: [sum(UnscaledValue(ss_net_paid#24))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#24))#16] -Results [4]: [c_customer_id#5 AS customer_id#31, c_first_name#6 AS customer_first_name#32, c_last_name#7 AS customer_last_name#33, MakeDecimal(sum(UnscaledValue(ss_net_paid#24))#16,17,2) AS year_total#34] - -(36) BroadcastExchange -Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(37) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#31] -Join type: Inner -Join condition: None - -(38) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(39) CometFilter -Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Condition : (isnotnull(c_customer_sk#35) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#36, 16, true, false, true))) - -(40) CometProject -Input [4]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38] -Arguments: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#35, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#36, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#37, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#38, 30, true, false, true) AS c_last_name#41] - -(41) CometColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41] - -(42) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] - -(44) Filter [codegen id : 8] -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_bill_customer_sk#42) - -(45) BroadcastExchange -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -(46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#35] -Right keys [1]: [ws_bill_customer_sk#42] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 10] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] -Input [7]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] - -(48) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#45, d_year#46] - -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#45] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 10] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#46] -Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#45, d_year#46] - -(51) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#46] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] -Aggregate Attributes [1]: [sum#47] -Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] - -(52) CometColumnarExchange -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] -Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(53) CometColumnarToRow [codegen id : 11] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] - -(54) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46, sum#48] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#46] -Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#49] -Results [2]: [c_customer_id#39 AS customer_id#50, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#49,17,2) AS year_total#51] - -(55) Filter [codegen id : 11] -Input [2]: [customer_id#50, year_total#51] -Condition : (isnotnull(year_total#51) AND (year_total#51 > 0.00)) - -(56) BroadcastExchange -Input [2]: [customer_id#50, year_total#51] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] - -(57) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#50] -Join type: Inner -Join condition: None - -(58) Project [codegen id : 16] -Output [7]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#51] -Input [8]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, customer_id#50, year_total#51] - -(59) CometNativeScan parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(60) CometFilter -Input [4]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55] -Condition : (isnotnull(c_customer_sk#52) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#53, 16, true, false, true))) - -(61) CometProject -Input [4]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55] -Arguments: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41], [c_customer_sk#52, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#53, 16, true, false, true) AS c_customer_id#39, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#54, 20, true, false, true) AS c_first_name#40, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#55, 30, true, false, true) AS c_last_name#41] - -(62) CometColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41] - -(63) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#26)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(64) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] - -(65) Filter [codegen id : 12] -Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] -Condition : isnotnull(ws_bill_customer_sk#56) - -(66) BroadcastExchange -Input [3]: [ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#52] -Right keys [1]: [ws_bill_customer_sk#56] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 14] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, ws_sold_date_sk#58] -Input [7]: [c_customer_sk#52, c_customer_id#39, c_first_name#40, c_last_name#41, ws_bill_customer_sk#56, ws_net_paid#57, ws_sold_date_sk#58] - -(69) ReusedExchange [Reuses operator id: 87] -Output [2]: [d_date_sk#59, d_year#60] - -(70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#58] -Right keys [1]: [d_date_sk#59] -Join type: Inner -Join condition: None - -(71) Project [codegen id : 14] -Output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, d_year#60] -Input [7]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, ws_sold_date_sk#58, d_date_sk#59, d_year#60] - -(72) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#57, d_year#60] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#57))] -Aggregate Attributes [1]: [sum#61] -Results [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] - -(73) CometColumnarExchange -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] -Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(74) CometColumnarToRow [codegen id : 15] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] - -(75) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60, sum#62] -Keys [4]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#60] -Functions [1]: [sum(UnscaledValue(ws_net_paid#57))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#57))#49] -Results [2]: [c_customer_id#39 AS customer_id#63, MakeDecimal(sum(UnscaledValue(ws_net_paid#57))#49,17,2) AS year_total#64] - -(76) BroadcastExchange -Input [2]: [customer_id#63, year_total#64] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] - -(77) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#63] -Join type: Inner -Join condition: (CASE WHEN (year_total#51 > 0.00) THEN (year_total#64 / year_total#51) END > CASE WHEN (year_total#18 > 0.00) THEN (year_total#34 / year_total#18) END) - -(78) Project [codegen id : 16] -Output [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] -Input [9]: [customer_id#17, year_total#18, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#51, customer_id#63, year_total#64] - -(79) TakeOrderedAndProject -Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] -Arguments: 100, [customer_first_name#32 ASC NULLS FIRST, customer_id#31 ASC NULLS FIRST, customer_last_name#33 ASC NULLS FIRST], [customer_id#31, customer_first_name#32, customer_last_name#33] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometFilter (81) - +- CometNativeScan parquet spark_catalog.default.date_dim (80) - - -(80) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(81) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(82) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(83) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 -BroadcastExchange (87) -+- * CometColumnarToRow (86) - +- CometFilter (85) - +- CometNativeScan parquet spark_catalog.default.date_dim (84) - - -(84) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#27, d_year#28] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(85) CometFilter -Input [2]: [d_date_sk#27, d_year#28] -Condition : (((isnotnull(d_year#28) AND (d_year#28 = 2002)) AND d_year#28 IN (2001,2002)) AND isnotnull(d_date_sk#27)) - -(86) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#28] - -(87) BroadcastExchange -Input [2]: [d_date_sk#27, d_year#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] - -Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#11 - -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#26 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/simplified.txt deleted file mode 100644 index 9a8a029e1b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/simplified.txt +++ /dev/null @@ -1,130 +0,0 @@ -TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] - WholeStageCodegen (16) - Project [customer_id,customer_first_name,customer_last_name] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - BroadcastHashJoin [customer_id,customer_id] - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - WholeStageCodegen (3) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (7) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #5 - WholeStageCodegen (6) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Filter [ss_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - Filter [year_total] - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #9 - WholeStageCodegen (10) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk,d_year] #3 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (15) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [c_customer_id,c_first_name,c_last_name,d_year] #12 - WholeStageCodegen (14) - HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] - BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometNativeScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - Filter [ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk,d_year] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/explain.txt deleted file mode 100644 index b0ae8bb695..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,470 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometProject (53) - : +- CometBroadcastHashJoin (52) - : :- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (51) - : +- CometFilter (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : +- CometBroadcastExchange (41) - : : +- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) - : +- ReusedExchange (44) - +- CometBroadcastExchange (68) - +- CometHashAggregate (67) - +- CometExchange (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometFilter (55) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (62) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Condition : isnotnull(ss_customer_sk#8) - -(6) CometBroadcastExchange -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_sk#1], [ss_customer_sk#8], Inner, BuildRight - -(8) CometProject -Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(12) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#10], [d_date_sk#12], Inner, BuildRight - -(13) CometProject -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] - -(14) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] - -(15) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] - -(17) CometFilter -Input [2]: [customer_id#15, year_total#16] -Condition : (isnotnull(year_total#16) AND (year_total#16 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true))) - -(20) CometProject -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Arguments: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#19, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#20, 30, true, false, true) AS c_last_name#7] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_customer_sk#21) - -(23) CometBroadcastExchange -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] - -(24) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [c_customer_sk#17], [ss_customer_sk#21], Inner, BuildRight - -(25) CometProject -Input [7]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: [d_date_sk#25, d_year#26] - -(29) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] -Right output [2]: [d_date_sk#25, d_year#26] -Arguments: [ss_sold_date_sk#23], [d_date_sk#25], Inner, BuildRight - -(30) CometProject -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23, d_date_sk#25, d_year#26] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] - -(31) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] - -(32) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] -Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] - -(34) CometBroadcastExchange -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#15, year_total#16] -Right output [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#15], [customer_id#28], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true))) - -(38) CometProject -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Arguments: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#34, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#35, 30, true, false, true) AS c_last_name#38] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(ws_sold_date_sk#41 IN dynamicpruning#42)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Condition : isnotnull(ws_bill_customer_sk#39) - -(41) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] - -(42) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38] -Right output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [c_customer_sk#32], [ws_bill_customer_sk#39], Inner, BuildRight - -(43) CometProject -Input [7]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#43, d_year#44] - -(45) CometBroadcastHashJoin -Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] -Right output [2]: [d_date_sk#43, d_year#44] -Arguments: [ws_sold_date_sk#41], [d_date_sk#43], Inner, BuildRight - -(46) CometProject -Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41, d_date_sk#43, d_year#44] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] - -(47) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#40))] - -(48) CometExchange -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] -Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] - -(50) CometFilter -Input [2]: [customer_id#46, year_total#47] -Condition : (isnotnull(year_total#47) AND (year_total#47 > 0.00)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#46, year_total#47] -Arguments: [customer_id#46, year_total#47] - -(52) CometBroadcastHashJoin -Left output [6]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Right output [2]: [customer_id#46, year_total#47] -Arguments: [customer_id#15], [customer_id#46], Inner, BuildRight - -(53) CometProject -Input [8]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#46, year_total#47] -Arguments: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47], [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Condition : (isnotnull(c_customer_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true))) - -(56) CometProject -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Arguments: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#50, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#51, 30, true, false, true) AS c_last_name#38] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#55)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Condition : isnotnull(ws_bill_customer_sk#52) - -(59) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] - -(60) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38] -Right output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [c_customer_sk#48], [ws_bill_customer_sk#52], Inner, BuildRight - -(61) CometProject -Input [7]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#56, d_year#57] - -(63) CometBroadcastHashJoin -Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] -Right output [2]: [d_date_sk#56, d_year#57] -Arguments: [ws_sold_date_sk#54], [d_date_sk#56], Inner, BuildRight - -(64) CometProject -Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54, d_date_sk#56, d_year#57] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] - -(65) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#53))] - -(66) CometExchange -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] -Functions [1]: [sum(UnscaledValue(ws_net_paid#53))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#59, year_total#60] -Arguments: [customer_id#59, year_total#60] - -(69) CometBroadcastHashJoin -Left output [7]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] -Right output [2]: [customer_id#59, year_total#60] -Arguments: [customer_id#15], [customer_id#59], Inner, (CASE WHEN (year_total#47 > 0.00) THEN (year_total#60 / year_total#47) END > CASE WHEN (year_total#16 > 0.00) THEN (year_total#31 / year_total#16) END), BuildRight - -(70) CometProject -Input [9]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47, customer_id#59, year_total#60] -Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30], [customer_id#28, customer_first_name#29, customer_last_name#30] - -(71) CometTakeOrderedAndProject -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_first_name#29 ASC NULLS FIRST,customer_id#28 ASC NULLS FIRST,customer_last_name#30 ASC NULLS FIRST], output=[customer_id#28,customer_first_name#29,customer_last_name#30]), [customer_id#28, customer_first_name#29, customer_last_name#30], 100, 0, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_last_name#30 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] - -(72) CometColumnarToRow [codegen id : 1] -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(75) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(76) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) - -(79) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#25, d_year#26] - -(80) BroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#41 IN dynamicpruning#11 - -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#24 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/simplified.txt deleted file mode 100644 index 72e39422e0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - CometProject [customer_id,customer_first_name,customer_last_name] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 - CometHashAggregate [d_year,sum] [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt deleted file mode 100644 index b0ae8bb695..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt +++ /dev/null @@ -1,470 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (72) -+- CometTakeOrderedAndProject (71) - +- CometProject (70) - +- CometBroadcastHashJoin (69) - :- CometProject (53) - : +- CometBroadcastHashJoin (52) - : :- CometBroadcastHashJoin (35) - : : :- CometFilter (17) - : : : +- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (34) - : : +- CometHashAggregate (33) - : : +- CometExchange (32) - : : +- CometHashAggregate (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometProject (25) - : : : +- CometBroadcastHashJoin (24) - : : : :- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (18) - : : : +- CometBroadcastExchange (23) - : : : +- CometFilter (22) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (21) - : : +- CometBroadcastExchange (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - : +- CometBroadcastExchange (51) - : +- CometFilter (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (43) - : : +- CometBroadcastHashJoin (42) - : : :- CometProject (38) - : : : +- CometFilter (37) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (36) - : : +- CometBroadcastExchange (41) - : : +- CometFilter (40) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (39) - : +- ReusedExchange (44) - +- CometBroadcastExchange (68) - +- CometHashAggregate (67) - +- CometExchange (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometFilter (55) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (57) - +- ReusedExchange (62) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true))) - -(3) CometProject -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Arguments: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#1, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#2, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#3, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#4, 30, true, false, true) AS c_last_name#7] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(5) CometFilter -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Condition : isnotnull(ss_customer_sk#8) - -(6) CometBroadcastExchange -Input [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] - -(7) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_sk#1], [ss_customer_sk#8], Inner, BuildRight - -(8) CometProject -Input [7]: [c_customer_sk#1, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#8, ss_net_paid#9, ss_sold_date_sk#10] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(12) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#10], [d_date_sk#12], Inner, BuildRight - -(13) CometProject -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, ss_sold_date_sk#10, d_date_sk#12, d_year#13] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] - -(14) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#9, d_year#13] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] - -(15) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13] -Functions [1]: [sum(UnscaledValue(ss_net_paid#9))] - -(17) CometFilter -Input [2]: [customer_id#15, year_total#16] -Condition : (isnotnull(year_total#16) AND (year_total#16 > 0.00)) - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(19) CometFilter -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Condition : (isnotnull(c_customer_sk#17) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true))) - -(20) CometProject -Input [4]: [c_customer_sk#17, c_customer_id#18, c_first_name#19, c_last_name#20] -Arguments: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7], [c_customer_sk#17, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#18, 16, true, false, true) AS c_customer_id#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#19, 20, true, false, true) AS c_first_name#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#20, 30, true, false, true) AS c_last_name#7] - -(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] -PushedFilters: [IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(22) CometFilter -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_customer_sk#21) - -(23) CometBroadcastExchange -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] - -(24) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7] -Right output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [c_customer_sk#17], [ss_customer_sk#21], Inner, BuildRight - -(25) CometProject -Input [7]: [c_customer_sk#17, c_customer_id#5, c_first_name#6, c_last_name#7, ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) - -(28) CometBroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: [d_date_sk#25, d_year#26] - -(29) CometBroadcastHashJoin -Left output [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23] -Right output [2]: [d_date_sk#25, d_year#26] -Arguments: [ss_sold_date_sk#23], [d_date_sk#25], Inner, BuildRight - -(30) CometProject -Input [7]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, ss_sold_date_sk#23, d_date_sk#25, d_year#26] -Arguments: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26], [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] - -(31) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, ss_net_paid#22, d_year#26] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] - -(32) CometExchange -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] -Arguments: hashpartitioning(c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(33) CometHashAggregate -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26, sum#27] -Keys [4]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#26] -Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] - -(34) CometBroadcastExchange -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] - -(35) CometBroadcastHashJoin -Left output [2]: [customer_id#15, year_total#16] -Right output [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#15], [customer_id#28], Inner, BuildRight - -(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(37) CometFilter -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true))) - -(38) CometProject -Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] -Arguments: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#32, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#33, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#34, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#35, 30, true, false, true) AS c_last_name#38] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(ws_sold_date_sk#41 IN dynamicpruning#42)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(40) CometFilter -Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Condition : isnotnull(ws_bill_customer_sk#39) - -(41) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] - -(42) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38] -Right output [3]: [ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [c_customer_sk#32], [ws_bill_customer_sk#39], Inner, BuildRight - -(43) CometProject -Input [7]: [c_customer_sk#32, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#39, ws_net_paid#40, ws_sold_date_sk#41] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] - -(44) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#43, d_year#44] - -(45) CometBroadcastHashJoin -Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41] -Right output [2]: [d_date_sk#43, d_year#44] -Arguments: [ws_sold_date_sk#41], [d_date_sk#43], Inner, BuildRight - -(46) CometProject -Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, ws_sold_date_sk#41, d_date_sk#43, d_year#44] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] - -(47) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#40, d_year#44] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#40))] - -(48) CometExchange -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(49) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44, sum#45] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#44] -Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] - -(50) CometFilter -Input [2]: [customer_id#46, year_total#47] -Condition : (isnotnull(year_total#47) AND (year_total#47 > 0.00)) - -(51) CometBroadcastExchange -Input [2]: [customer_id#46, year_total#47] -Arguments: [customer_id#46, year_total#47] - -(52) CometBroadcastHashJoin -Left output [6]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Right output [2]: [customer_id#46, year_total#47] -Arguments: [customer_id#15], [customer_id#46], Inner, BuildRight - -(53) CometProject -Input [8]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#46, year_total#47] -Arguments: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47], [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] - -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk)] -ReadSchema: struct - -(55) CometFilter -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Condition : (isnotnull(c_customer_sk#48) AND isnotnull(staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true))) - -(56) CometProject -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Arguments: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38], [c_customer_sk#48, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_customer_id#49, 16, true, false, true) AS c_customer_id#36, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_first_name#50, 20, true, false, true) AS c_first_name#37, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, c_last_name#51, 30, true, false, true) AS c_last_name#38] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#55)] -PushedFilters: [IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(58) CometFilter -Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Condition : isnotnull(ws_bill_customer_sk#52) - -(59) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] - -(60) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38] -Right output [3]: [ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [c_customer_sk#48], [ws_bill_customer_sk#52], Inner, BuildRight - -(61) CometProject -Input [7]: [c_customer_sk#48, c_customer_id#36, c_first_name#37, c_last_name#38, ws_bill_customer_sk#52, ws_net_paid#53, ws_sold_date_sk#54] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] - -(62) ReusedExchange [Reuses operator id: 28] -Output [2]: [d_date_sk#56, d_year#57] - -(63) CometBroadcastHashJoin -Left output [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54] -Right output [2]: [d_date_sk#56, d_year#57] -Arguments: [ws_sold_date_sk#54], [d_date_sk#56], Inner, BuildRight - -(64) CometProject -Input [7]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, ws_sold_date_sk#54, d_date_sk#56, d_year#57] -Arguments: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57], [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] - -(65) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, ws_net_paid#53, d_year#57] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#53))] - -(66) CometExchange -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(67) CometHashAggregate -Input [5]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57, sum#58] -Keys [4]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#57] -Functions [1]: [sum(UnscaledValue(ws_net_paid#53))] - -(68) CometBroadcastExchange -Input [2]: [customer_id#59, year_total#60] -Arguments: [customer_id#59, year_total#60] - -(69) CometBroadcastHashJoin -Left output [7]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47] -Right output [2]: [customer_id#59, year_total#60] -Arguments: [customer_id#15], [customer_id#59], Inner, (CASE WHEN (year_total#47 > 0.00) THEN (year_total#60 / year_total#47) END > CASE WHEN (year_total#16 > 0.00) THEN (year_total#31 / year_total#16) END), BuildRight - -(70) CometProject -Input [9]: [customer_id#15, year_total#16, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#47, customer_id#59, year_total#60] -Arguments: [customer_id#28, customer_first_name#29, customer_last_name#30], [customer_id#28, customer_first_name#29, customer_last_name#30] - -(71) CometTakeOrderedAndProject -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_first_name#29 ASC NULLS FIRST,customer_id#28 ASC NULLS FIRST,customer_last_name#30 ASC NULLS FIRST], output=[customer_id#28,customer_first_name#29,customer_last_name#30]), [customer_id#28, customer_first_name#29, customer_last_name#30], 100, 0, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_last_name#30 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] - -(72) CometColumnarToRow [codegen id : 1] -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometFilter (74) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - - -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(74) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 = 2001)) AND d_year#13 IN (2001,2002)) AND isnotnull(d_date_sk#12)) - -(75) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(76) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] - -Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) - - -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] -ReadSchema: struct - -(78) CometFilter -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) - -(79) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#25, d_year#26] - -(80) BroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] - -Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#41 IN dynamicpruning#11 - -Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#24 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/extended.txt deleted file mode 100644 index 476c7be954..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/extended.txt +++ /dev/null @@ -1,90 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 72e39422e0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/simplified.txt +++ /dev/null @@ -1,90 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - CometProject [customer_id,customer_first_name,customer_last_name] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total,customer_id,year_total] - CometProject [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,customer_id,year_total] - CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total] - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #1 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #2 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #4 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,customer_first_name,customer_last_name,year_total] #5 - CometHashAggregate [d_year,sum] [customer_id,customer_first_name,customer_last_name,year_total,c_customer_id,c_first_name,c_last_name,sum(UnscaledValue(ss_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #6 - CometHashAggregate [ss_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometBroadcastExchange [ss_customer_sk,ss_net_paid,ss_sold_date_sk] #7 - CometFilter [ss_customer_sk,ss_net_paid,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk,d_year] #9 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [customer_id,year_total] #10 - CometFilter [customer_id,year_total] - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #11 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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 - CometBroadcastExchange [customer_id,year_total] #13 - CometHashAggregate [c_first_name,c_last_name,d_year,sum] [customer_id,year_total,c_customer_id,sum(UnscaledValue(ws_net_paid))] - CometExchange [c_customer_id,c_first_name,c_last_name,d_year] #14 - CometHashAggregate [ws_net_paid] [c_customer_id,c_first_name,c_last_name,d_year,sum] - CometProject [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] - 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,c_customer_id,c_first_name,c_last_name,ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] - CometProject [c_customer_id,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] - 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 [native_iceberg_compat] 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.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/explain.txt deleted file mode 100644 index eddee4d8e5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/explain.txt +++ /dev/null @@ -1,754 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (127) -+- CometTakeOrderedAndProject (126) - +- CometProject (125) - +- CometSortMergeJoin (124) - :- CometSort (68) - : +- CometExchange (67) - : +- CometFilter (66) - : +- CometHashAggregate (65) - : +- CometExchange (64) - : +- CometHashAggregate (63) - : +- CometHashAggregate (62) - : +- CometExchange (61) - : +- CometHashAggregate (60) - : +- CometUnion (59) - : :- CometProject (22) - : : +- CometSortMergeJoin (21) - : : :- CometSort (15) - : : : +- CometColumnarExchange (14) - : : : +- * Project (13) - : : : +- * BroadcastHashJoin Inner BuildRight (12) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.catalog_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * CometColumnarToRow (7) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometNativeScan parquet spark_catalog.default.item (4) - : : : +- ReusedExchange (11) - : : +- CometSort (20) - : : +- CometExchange (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (40) - : : +- CometSortMergeJoin (39) - : : :- CometSort (33) - : : : +- CometColumnarExchange (32) - : : : +- * Project (31) - : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : :- * Project (28) - : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : :- * Filter (25) - : : : : : +- * ColumnarToRow (24) - : : : : : +- Scan parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (26) - : : : +- ReusedExchange (29) - : : +- CometSort (38) - : : +- CometExchange (37) - : : +- CometProject (36) - : : +- CometFilter (35) - : : +- CometNativeScan parquet spark_catalog.default.store_returns (34) - : +- CometProject (58) - : +- CometSortMergeJoin (57) - : :- CometSort (51) - : : +- CometColumnarExchange (50) - : : +- * Project (49) - : : +- * BroadcastHashJoin Inner BuildRight (48) - : : :- * Project (46) - : : : +- * BroadcastHashJoin Inner BuildRight (45) - : : : :- * Filter (43) - : : : : +- * ColumnarToRow (42) - : : : : +- Scan parquet spark_catalog.default.web_sales (41) - : : : +- ReusedExchange (44) - : : +- ReusedExchange (47) - : +- CometSort (56) - : +- CometExchange (55) - : +- CometProject (54) - : +- CometFilter (53) - : +- CometNativeScan parquet spark_catalog.default.web_returns (52) - +- CometSort (123) - +- CometExchange (122) - +- CometFilter (121) - +- CometHashAggregate (120) - +- CometExchange (119) - +- CometHashAggregate (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometUnion (114) - :- CometProject (83) - : +- CometSortMergeJoin (82) - : :- CometSort (79) - : : +- CometColumnarExchange (78) - : : +- * Project (77) - : : +- * BroadcastHashJoin Inner BuildRight (76) - : : :- * Project (74) - : : : +- * BroadcastHashJoin Inner BuildRight (73) - : : : :- * Filter (71) - : : : : +- * ColumnarToRow (70) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (69) - : : : +- ReusedExchange (72) - : : +- ReusedExchange (75) - : +- CometSort (81) - : +- ReusedExchange (80) - :- CometProject (98) - : +- CometSortMergeJoin (97) - : :- CometSort (94) - : : +- CometColumnarExchange (93) - : : +- * Project (92) - : : +- * BroadcastHashJoin Inner BuildRight (91) - : : :- * Project (89) - : : : +- * BroadcastHashJoin Inner BuildRight (88) - : : : :- * Filter (86) - : : : : +- * ColumnarToRow (85) - : : : : +- Scan parquet spark_catalog.default.store_sales (84) - : : : +- ReusedExchange (87) - : : +- ReusedExchange (90) - : +- CometSort (96) - : +- ReusedExchange (95) - +- CometProject (113) - +- CometSortMergeJoin (112) - :- CometSort (109) - : +- CometColumnarExchange (108) - : +- * Project (107) - : +- * BroadcastHashJoin Inner BuildRight (106) - : :- * Project (104) - : : +- * BroadcastHashJoin Inner BuildRight (103) - : : :- * Filter (101) - : : : +- * ColumnarToRow (100) - : : : +- Scan parquet spark_catalog.default.web_sales (99) - : : +- ReusedExchange (102) - : +- ReusedExchange (105) - +- CometSort (111) - +- ReusedExchange (110) - - -(1) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] - -(3) Filter [codegen id : 3] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) = Books ) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) - -(6) CometProject -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(7) CometColumnarToRow [codegen id : 1] -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(8) BroadcastExchange -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(11) ReusedExchange [Reuses operator id: 131] -Output [2]: [d_date_sk#13, d_year#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#13] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] - -(14) CometColumnarExchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(15) CometSort -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] - -(16) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) - -(18) CometProject -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(19) CometExchange -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometSort -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter - -(22) CometProject -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21], [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] - -(23) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 6] -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] - -(25) Filter [codegen id : 6] -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#22) - -(26) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] - -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#22] -Right keys [1]: [i_item_sk#27] -Join type: Inner -Join condition: None - -(28) Project [codegen id : 6] -Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] -Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31] - -(29) ReusedExchange [Reuses operator id: 131] -Output [2]: [d_date_sk#32, d_year#33] - -(30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#32] -Join type: Inner -Join condition: None - -(31) Project [codegen id : 6] -Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_date_sk#32, d_year#33] - -(32) CometColumnarExchange -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(33) CometSort -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33], [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] - -(34) CometNativeScan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37, sr_returned_date_sk#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(35) CometFilter -Input [5]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37, sr_returned_date_sk#38] -Condition : (isnotnull(sr_ticket_number#35) AND isnotnull(sr_item_sk#34)) - -(36) CometProject -Input [5]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37, sr_returned_date_sk#38] -Arguments: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37], [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] - -(37) CometExchange -Input [4]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: hashpartitioning(sr_ticket_number#35, sr_item_sk#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(38) CometSort -Input [4]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37], [sr_ticket_number#35 ASC NULLS FIRST, sr_item_sk#34 ASC NULLS FIRST] - -(39) CometSortMergeJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33] -Right output [4]: [sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#35, sr_item_sk#34], LeftOuter - -(40) CometProject -Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33, sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37] -Arguments: [d_year#33, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, sales_cnt#39, sales_amt#40], [d_year#33, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, (ss_quantity#24 - coalesce(sr_return_quantity#36, 0)) AS sales_cnt#39, (ss_ext_sales_price#25 - coalesce(sr_return_amt#37, 0.00)) AS sales_amt#40] - -(41) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(42) ColumnarToRow [codegen id : 9] -Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] - -(43) Filter [codegen id : 9] -Input [5]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -Condition : isnotnull(ws_item_sk#41) - -(44) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] - -(45) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#41] -Right keys [1]: [i_item_sk#46] -Join type: Inner -Join condition: None - -(46) Project [codegen id : 9] -Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] -Input [10]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50] - -(47) ReusedExchange [Reuses operator id: 131] -Output [2]: [d_date_sk#51, d_year#52] - -(48) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#45] -Right keys [1]: [d_date_sk#51] -Join type: Inner -Join condition: None - -(49) Project [codegen id : 9] -Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Input [11]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_date_sk#51, d_year#52] - -(50) CometColumnarExchange -Input [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Arguments: hashpartitioning(ws_order_number#42, ws_item_sk#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(51) CometSort -Input [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Arguments: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52], [ws_order_number#42 ASC NULLS FIRST, ws_item_sk#41 ASC NULLS FIRST] - -(52) CometNativeScan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56, wr_returned_date_sk#57] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(53) CometFilter -Input [5]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56, wr_returned_date_sk#57] -Condition : (isnotnull(wr_order_number#54) AND isnotnull(wr_item_sk#53)) - -(54) CometProject -Input [5]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56, wr_returned_date_sk#57] -Arguments: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56], [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] - -(55) CometExchange -Input [4]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: hashpartitioning(wr_order_number#54, wr_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(56) CometSort -Input [4]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56], [wr_order_number#54 ASC NULLS FIRST, wr_item_sk#53 ASC NULLS FIRST] - -(57) CometSortMergeJoin -Left output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52] -Right output [4]: [wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: [ws_order_number#42, ws_item_sk#41], [wr_order_number#54, wr_item_sk#53], LeftOuter - -(58) CometProject -Input [13]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52, wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56] -Arguments: [d_year#52, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, sales_cnt#58, sales_amt#59], [d_year#52, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, (ws_quantity#43 - coalesce(wr_return_quantity#55, 0)) AS sales_cnt#58, (ws_ext_sales_price#44 - coalesce(wr_return_amt#56, 0.00)) AS sales_amt#59] - -(59) CometUnion -Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#33, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, sales_cnt#39, sales_amt#40] -Child 2 Input [7]: [d_year#52, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, sales_cnt#58, sales_amt#59] - -(60) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(61) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(62) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(63) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(64) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#60, sum#61] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(65) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#60, sum#61] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(66) CometFilter -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Condition : isnotnull(sales_cnt#62) - -(67) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(68) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] - -(69) Scan parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#68), dynamicpruningexpression(cs_sold_date_sk#68 IN dynamicpruning#69)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(70) ColumnarToRow [codegen id : 12] -Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] - -(71) Filter [codegen id : 12] -Input [5]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68] -Condition : isnotnull(cs_item_sk#64) - -(72) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] - -(73) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cs_item_sk#64] -Right keys [1]: [i_item_sk#70] -Join type: Inner -Join condition: None - -(74) Project [codegen id : 12] -Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -Input [10]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] - -(75) ReusedExchange [Reuses operator id: 135] -Output [2]: [d_date_sk#75, d_year#76] - -(76) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cs_sold_date_sk#68] -Right keys [1]: [d_date_sk#75] -Join type: Inner -Join condition: None - -(77) Project [codegen id : 12] -Output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Input [11]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, cs_sold_date_sk#68, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_date_sk#75, d_year#76] - -(78) CometColumnarExchange -Input [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Arguments: hashpartitioning(cs_order_number#65, cs_item_sk#64, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(79) CometSort -Input [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Arguments: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76], [cs_order_number#65 ASC NULLS FIRST, cs_item_sk#64 ASC NULLS FIRST] - -(80) ReusedExchange [Reuses operator id: 19] -Output [4]: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] - -(81) CometSort -Input [4]: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] -Arguments: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80], [cr_order_number#78 ASC NULLS FIRST, cr_item_sk#77 ASC NULLS FIRST] - -(82) CometSortMergeJoin -Left output [9]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76] -Right output [4]: [cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] -Arguments: [cs_order_number#65, cs_item_sk#64], [cr_order_number#78, cr_item_sk#77], LeftOuter - -(83) CometProject -Input [13]: [cs_item_sk#64, cs_order_number#65, cs_quantity#66, cs_ext_sales_price#67, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, d_year#76, cr_item_sk#77, cr_order_number#78, cr_return_quantity#79, cr_return_amount#80] -Arguments: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21], [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, (cs_quantity#66 - coalesce(cr_return_quantity#79, 0)) AS sales_cnt#20, (cs_ext_sales_price#67 - coalesce(cr_return_amount#80, 0.00)) AS sales_amt#21] - -(84) Scan parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#85), dynamicpruningexpression(ss_sold_date_sk#85 IN dynamicpruning#69)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(85) ColumnarToRow [codegen id : 15] -Input [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85] - -(86) Filter [codegen id : 15] -Input [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85] -Condition : isnotnull(ss_item_sk#81) - -(87) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] - -(88) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ss_item_sk#81] -Right keys [1]: [i_item_sk#86] -Join type: Inner -Join condition: None - -(89) Project [codegen id : 15] -Output [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] -Input [10]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90] - -(90) ReusedExchange [Reuses operator id: 135] -Output [2]: [d_date_sk#91, d_year#92] - -(91) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ss_sold_date_sk#85] -Right keys [1]: [d_date_sk#91] -Join type: Inner -Join condition: None - -(92) Project [codegen id : 15] -Output [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92] -Input [11]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, ss_sold_date_sk#85, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_date_sk#91, d_year#92] - -(93) CometColumnarExchange -Input [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92] -Arguments: hashpartitioning(ss_ticket_number#82, ss_item_sk#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(94) CometSort -Input [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92] -Arguments: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92], [ss_ticket_number#82 ASC NULLS FIRST, ss_item_sk#81 ASC NULLS FIRST] - -(95) ReusedExchange [Reuses operator id: 37] -Output [4]: [sr_item_sk#93, sr_ticket_number#94, sr_return_quantity#95, sr_return_amt#96] - -(96) CometSort -Input [4]: [sr_item_sk#93, sr_ticket_number#94, sr_return_quantity#95, sr_return_amt#96] -Arguments: [sr_item_sk#93, sr_ticket_number#94, sr_return_quantity#95, sr_return_amt#96], [sr_ticket_number#94 ASC NULLS FIRST, sr_item_sk#93 ASC NULLS FIRST] - -(97) CometSortMergeJoin -Left output [9]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92] -Right output [4]: [sr_item_sk#93, sr_ticket_number#94, sr_return_quantity#95, sr_return_amt#96] -Arguments: [ss_ticket_number#82, ss_item_sk#81], [sr_ticket_number#94, sr_item_sk#93], LeftOuter - -(98) CometProject -Input [13]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_ext_sales_price#84, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, d_year#92, sr_item_sk#93, sr_ticket_number#94, sr_return_quantity#95, sr_return_amt#96] -Arguments: [d_year#92, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, sales_cnt#39, sales_amt#40], [d_year#92, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, (ss_quantity#83 - coalesce(sr_return_quantity#95, 0)) AS sales_cnt#39, (ss_ext_sales_price#84 - coalesce(sr_return_amt#96, 0.00)) AS sales_amt#40] - -(99) Scan parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#101), dynamicpruningexpression(ws_sold_date_sk#101 IN dynamicpruning#69)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(100) ColumnarToRow [codegen id : 18] -Input [5]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101] - -(101) Filter [codegen id : 18] -Input [5]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101] -Condition : isnotnull(ws_item_sk#97) - -(102) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#102, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] - -(103) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_item_sk#97] -Right keys [1]: [i_item_sk#102] -Join type: Inner -Join condition: None - -(104) Project [codegen id : 18] -Output [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] -Input [10]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_item_sk#102, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106] - -(105) ReusedExchange [Reuses operator id: 135] -Output [2]: [d_date_sk#107, d_year#108] - -(106) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#101] -Right keys [1]: [d_date_sk#107] -Join type: Inner -Join condition: None - -(107) Project [codegen id : 18] -Output [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108] -Input [11]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, ws_sold_date_sk#101, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_date_sk#107, d_year#108] - -(108) CometColumnarExchange -Input [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108] -Arguments: hashpartitioning(ws_order_number#98, ws_item_sk#97, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(109) CometSort -Input [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108] -Arguments: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108], [ws_order_number#98 ASC NULLS FIRST, ws_item_sk#97 ASC NULLS FIRST] - -(110) ReusedExchange [Reuses operator id: 55] -Output [4]: [wr_item_sk#109, wr_order_number#110, wr_return_quantity#111, wr_return_amt#112] - -(111) CometSort -Input [4]: [wr_item_sk#109, wr_order_number#110, wr_return_quantity#111, wr_return_amt#112] -Arguments: [wr_item_sk#109, wr_order_number#110, wr_return_quantity#111, wr_return_amt#112], [wr_order_number#110 ASC NULLS FIRST, wr_item_sk#109 ASC NULLS FIRST] - -(112) CometSortMergeJoin -Left output [9]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108] -Right output [4]: [wr_item_sk#109, wr_order_number#110, wr_return_quantity#111, wr_return_amt#112] -Arguments: [ws_order_number#98, ws_item_sk#97], [wr_order_number#110, wr_item_sk#109], LeftOuter - -(113) CometProject -Input [13]: [ws_item_sk#97, ws_order_number#98, ws_quantity#99, ws_ext_sales_price#100, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, d_year#108, wr_item_sk#109, wr_order_number#110, wr_return_quantity#111, wr_return_amt#112] -Arguments: [d_year#108, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, sales_cnt#58, sales_amt#59], [d_year#108, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, (ws_quantity#99 - coalesce(wr_return_quantity#111, 0)) AS sales_cnt#58, (ws_ext_sales_price#100 - coalesce(wr_return_amt#112, 0.00)) AS sales_amt#59] - -(114) CometUnion -Child 0 Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#92, i_brand_id#87, i_class_id#88, i_category_id#89, i_manufact_id#90, sales_cnt#39, sales_amt#40] -Child 2 Input [7]: [d_year#108, i_brand_id#103, i_class_id#104, i_category_id#105, i_manufact_id#106, sales_cnt#58, sales_amt#59] - -(115) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Functions: [] - -(116) CometExchange -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(117) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Functions: [] - -(118) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(119) CometExchange -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sum#60, sum#113] -Arguments: hashpartitioning(d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(120) CometHashAggregate -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sum#60, sum#113] -Keys [5]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(121) CometFilter -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115] -Condition : isnotnull(sales_cnt#114) - -(122) CometExchange -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115] -Arguments: hashpartitioning(i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=16] - -(123) CometSort -Input [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115] -Arguments: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115], [i_brand_id#71 ASC NULLS FIRST, i_class_id#72 ASC NULLS FIRST, i_category_id#73 ASC NULLS FIRST, i_manufact_id#74 ASC NULLS FIRST] - -(124) CometSortMergeJoin -Left output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Right output [7]: [d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115] -Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74], Inner, ((cast(sales_cnt#62 as decimal(17,2)) / cast(sales_cnt#114 as decimal(17,2))) < 0.90000000000000000000) - -(125) CometProject -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#76, i_brand_id#71, i_class_id#72, i_category_id#73, i_manufact_id#74, sales_cnt#114, sales_amt#115] -Arguments: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121], [d_year#76 AS prev_year#116, d_year#14 AS year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#114 AS prev_yr_cnt#118, sales_cnt#62 AS curr_yr_cnt#119, (sales_cnt#62 - sales_cnt#114) AS sales_cnt_diff#120, (sales_amt#63 - sales_amt#115) AS sales_amt_diff#121] - -(126) CometTakeOrderedAndProject -Input [10]: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#120 ASC NULLS FIRST,sales_amt_diff#121 ASC NULLS FIRST], output=[prev_year#116,year#117,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#118,curr_yr_cnt#119,sales_cnt_diff#120,sales_amt_diff#121]), [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121], 100, 0, [sales_cnt_diff#120 ASC NULLS FIRST, sales_amt_diff#121 ASC NULLS FIRST], [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] - -(127) CometColumnarToRow [codegen id : 19] -Input [10]: [prev_year#116, year#117, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#118, curr_yr_cnt#119, sales_cnt_diff#120, sales_amt_diff#121] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (131) -+- * CometColumnarToRow (130) - +- CometFilter (129) - +- CometNativeScan parquet spark_catalog.default.date_dim (128) - - -(128) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(129) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(130) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_year#14] - -(131) BroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#6 - -Subquery:4 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#68 IN dynamicpruning#69 -BroadcastExchange (135) -+- * CometColumnarToRow (134) - +- CometFilter (133) - +- CometNativeScan parquet spark_catalog.default.date_dim (132) - - -(132) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#75, d_year#76] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(133) CometFilter -Input [2]: [d_date_sk#75, d_year#76] -Condition : ((isnotnull(d_year#76) AND (d_year#76 = 2001)) AND isnotnull(d_date_sk#75)) - -(134) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#75, d_year#76] - -(135) BroadcastExchange -Input [2]: [d_date_sk#75, d_year#76] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] - -Subquery:5 Hosting operator id = 84 Hosting Expression = ss_sold_date_sk#85 IN dynamicpruning#69 - -Subquery:6 Hosting operator id = 99 Hosting Expression = ws_sold_date_sk#101 IN dynamicpruning#69 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/simplified.txt deleted file mode 100644 index aaf98f4b09..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/simplified.txt +++ /dev/null @@ -1,173 +0,0 @@ -WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometColumnarExchange [cs_order_number,cs_item_sk] #4 - WholeStageCodegen (3) - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [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] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #7 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #8 - WholeStageCodegen (6) - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #9 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometColumnarExchange [ws_order_number,ws_item_sk] #10 - WholeStageCodegen (9) - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #5 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #11 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometColumnarExchange [cs_order_number,cs_item_sk] #15 - WholeStageCodegen (12) - Project [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] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan 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 #16 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #17 - WholeStageCodegen (15) - Project [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] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometColumnarExchange [ws_order_number,ws_item_sk] #18 - WholeStageCodegen (18) - Project [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] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Project [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] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 - InputAdapter - ReusedExchange [d_date_sk,d_year] #16 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #11 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/explain.txt deleted file mode 100644 index 6925eb301e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,742 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (124) -+- CometTakeOrderedAndProject (123) - +- CometProject (122) - +- CometSortMergeJoin (121) - :- CometSort (66) - : +- CometExchange (65) - : +- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometHashAggregate (60) - : +- CometExchange (59) - : +- CometHashAggregate (58) - : +- CometUnion (57) - : :- CometProject (22) - : : +- CometSortMergeJoin (21) - : : :- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometSort (20) - : : +- CometExchange (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (39) - : : +- CometSortMergeJoin (38) - : : :- CometSort (32) - : : : +- CometExchange (31) - : : : +- CometProject (30) - : : : +- CometBroadcastHashJoin (29) - : : : :- CometProject (27) - : : : : +- CometBroadcastHashJoin (26) - : : : : :- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (25) - : : : +- ReusedExchange (28) - : : +- CometSort (37) - : : +- CometExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (33) - : +- CometProject (56) - : +- CometSortMergeJoin (55) - : :- CometSort (49) - : : +- CometExchange (48) - : : +- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometProject (44) - : : : +- CometBroadcastHashJoin (43) - : : : :- CometFilter (41) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) - : : : +- ReusedExchange (42) - : : +- ReusedExchange (45) - : +- CometSort (54) - : +- CometExchange (53) - : +- CometProject (52) - : +- CometFilter (51) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (50) - +- CometSort (120) - +- CometExchange (119) - +- CometFilter (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometHashAggregate (114) - +- CometExchange (113) - +- CometHashAggregate (112) - +- CometUnion (111) - :- CometProject (82) - : +- CometSortMergeJoin (81) - : :- CometSort (78) - : : +- CometExchange (77) - : : +- CometProject (76) - : : +- CometBroadcastHashJoin (75) - : : :- CometProject (71) - : : : +- CometBroadcastHashJoin (70) - : : : :- CometFilter (68) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) - : : : +- ReusedExchange (69) - : : +- CometBroadcastExchange (74) - : : +- CometFilter (73) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - : +- CometSort (80) - : +- ReusedExchange (79) - :- CometProject (96) - : +- CometSortMergeJoin (95) - : :- CometSort (92) - : : +- CometExchange (91) - : : +- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (87) - : : : +- CometBroadcastHashJoin (86) - : : : :- CometFilter (84) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (83) - : : : +- ReusedExchange (85) - : : +- ReusedExchange (88) - : +- CometSort (94) - : +- ReusedExchange (93) - +- CometProject (110) - +- CometSortMergeJoin (109) - :- CometSort (106) - : +- CometExchange (105) - : +- CometProject (104) - : +- CometBroadcastHashJoin (103) - : :- CometProject (101) - : : +- CometBroadcastHashJoin (100) - : : :- CometFilter (98) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (97) - : : +- ReusedExchange (99) - : +- ReusedExchange (102) - +- CometSort (108) - +- ReusedExchange (107) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) = Books ) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) - -(5) CometProject -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(6) CometBroadcastExchange -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(7) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Right output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: [d_date_sk#13, d_year#14] - -(12) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Right output [2]: [d_date_sk#13, d_year#14] -Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight - -(13) CometProject -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] - -(14) CometExchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(15) CometSort -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) - -(18) CometProject -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(19) CometExchange -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter - -(22) CometProject -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21], [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(24) CometFilter -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#22) - -(25) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(26) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Right output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22], [i_item_sk#28], Inner, BuildRight - -(27) CometProject -Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(28) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#33, d_year#34] - -(29) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Right output [2]: [d_date_sk#33, d_year#34] -Arguments: [ss_sold_date_sk#26], [d_date_sk#33], Inner, BuildRight - -(30) CometProject -Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_date_sk#33, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] - -(31) CometExchange -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometSort -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) - -(35) CometProject -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] - -(36) CometExchange -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(37) CometSort -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] - -(38) CometSortMergeJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter - -(39) CometProject -Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#47)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(41) CometFilter -Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Condition : isnotnull(ws_item_sk#42) - -(42) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(43) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Right output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42], [i_item_sk#48], Inner, BuildRight - -(44) CometProject -Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(45) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#53, d_year#54] - -(46) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [ws_sold_date_sk#46], [d_date_sk#53], Inner, BuildRight - -(47) CometProject -Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] - -(48) CometExchange -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(49) CometSort -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) - -(52) CometProject -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] - -(53) CometExchange -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(54) CometSort -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] - -(55) CometSortMergeJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter - -(56) CometProject -Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] - -(57) CometUnion -Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] - -(58) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(59) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(60) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(61) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(62) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(63) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(64) CometFilter -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Condition : isnotnull(sales_cnt#64) - -(65) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(66) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(68) CometFilter -Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Condition : isnotnull(cs_item_sk#66) - -(69) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(70) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Right output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66], [i_item_sk#72], Inner, BuildRight - -(71) CometProject -Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(74) CometBroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: [d_date_sk#77, d_year#78] - -(75) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Right output [2]: [d_date_sk#77, d_year#78] -Arguments: [cs_sold_date_sk#70], [d_date_sk#77], Inner, BuildRight - -(76) CometProject -Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] - -(77) CometExchange -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(78) CometSort -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] - -(79) ReusedExchange [Reuses operator id: 19] -Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] - -(80) CometSort -Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] - -(81) CometSortMergeJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter - -(82) CometProject -Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#20, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#21] - -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(84) CometFilter -Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Condition : isnotnull(ss_item_sk#83) - -(85) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] - -(86) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Right output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Arguments: [ss_item_sk#83], [i_item_sk#89], Inner, BuildRight - -(87) CometProject -Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] - -(88) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#94, d_year#95] - -(89) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Right output [2]: [d_date_sk#94, d_year#95] -Arguments: [ss_sold_date_sk#87], [d_date_sk#94], Inner, BuildRight - -(90) CometProject -Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_date_sk#94, d_year#95] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] - -(91) CometExchange -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(92) CometSort -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_ticket_number#84 ASC NULLS FIRST, ss_item_sk#83 ASC NULLS FIRST] - -(93) ReusedExchange [Reuses operator id: 36] -Output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] - -(94) CometSort -Input [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99], [sr_ticket_number#97 ASC NULLS FIRST, sr_item_sk#96 ASC NULLS FIRST] - -(95) CometSortMergeJoin -Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Right output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [ss_ticket_number#84, ss_item_sk#83], [sr_ticket_number#97, sr_item_sk#96], LeftOuter - -(96) CometProject -Input [13]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95, sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41], [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, (ss_quantity#85 - coalesce(sr_return_quantity#98, 0)) AS sales_cnt#40, (ss_ext_sales_price#86 - coalesce(sr_return_amt#99, 0.00)) AS sales_amt#41] - -(97) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_sold_date_sk#104 IN dynamicpruning#105)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(98) CometFilter -Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Condition : isnotnull(ws_item_sk#100) - -(99) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] - -(100) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Right output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Arguments: [ws_item_sk#100], [i_item_sk#106], Inner, BuildRight - -(101) CometProject -Input [10]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] - -(102) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#111, d_year#112] - -(103) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Right output [2]: [d_date_sk#111, d_year#112] -Arguments: [ws_sold_date_sk#104], [d_date_sk#111], Inner, BuildRight - -(104) CometProject -Input [11]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] - -(105) CometExchange -Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(106) CometSort -Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_order_number#101 ASC NULLS FIRST, ws_item_sk#100 ASC NULLS FIRST] - -(107) ReusedExchange [Reuses operator id: 53] -Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] - -(108) CometSort -Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116], [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST] - -(109) CometSortMergeJoin -Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Right output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [ws_order_number#101, ws_item_sk#100], [wr_order_number#114, wr_item_sk#113], LeftOuter - -(110) CometProject -Input [13]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61], [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#102 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#60, (ws_ext_sales_price#103 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#61] - -(111) CometUnion -Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61] - -(112) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Functions: [] - -(113) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] - -(114) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Functions: [] - -(115) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(116) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(117) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(118) CometFilter -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Condition : isnotnull(sales_cnt#118) - -(119) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(120) CometSort -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] - -(121) CometSortMergeJoin -Left output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Right output [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], Inner, ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#118 as decimal(17,2))) < 0.90000000000000000000) - -(122) CometProject -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], [d_year#78 AS prev_year#120, d_year#14 AS year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#118 AS prev_yr_cnt#122, sales_cnt#64 AS curr_yr_cnt#123, (sales_cnt#64 - sales_cnt#118) AS sales_cnt_diff#124, (sales_amt#65 - sales_amt#119) AS sales_amt_diff#125] - -(123) CometTakeOrderedAndProject -Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#124 ASC NULLS FIRST,sales_amt_diff#125 ASC NULLS FIRST], output=[prev_year#120,year#121,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#122,curr_yr_cnt#123,sales_cnt_diff#124,sales_amt_diff#125]), [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], 100, 0, [sales_cnt_diff#124 ASC NULLS FIRST, sales_amt_diff#125 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] - -(124) CometColumnarToRow [codegen id : 1] -Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (128) -+- * CometColumnarToRow (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) - - -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(127) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_year#14] - -(128) BroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 - -Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 -BroadcastExchange (132) -+- * CometColumnarToRow (131) - +- CometFilter (130) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) - - -(129) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(130) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(131) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#77, d_year#78] - -(132) BroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] - -Subquery:5 Hosting operator id = 83 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 - -Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/simplified.txt deleted file mode 100644 index d0520c8b63..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,144 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #4 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange [d_date_sk,d_year] #7 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #8 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #9 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #10 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #11 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #12 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #16 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_year] #18 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #19 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #20 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt deleted file mode 100644 index 6925eb301e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt +++ /dev/null @@ -1,742 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (124) -+- CometTakeOrderedAndProject (123) - +- CometProject (122) - +- CometSortMergeJoin (121) - :- CometSort (66) - : +- CometExchange (65) - : +- CometFilter (64) - : +- CometHashAggregate (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometHashAggregate (60) - : +- CometExchange (59) - : +- CometHashAggregate (58) - : +- CometUnion (57) - : :- CometProject (22) - : : +- CometSortMergeJoin (21) - : : :- CometSort (15) - : : : +- CometExchange (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometSort (20) - : : +- CometExchange (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (16) - : :- CometProject (39) - : : +- CometSortMergeJoin (38) - : : :- CometSort (32) - : : : +- CometExchange (31) - : : : +- CometProject (30) - : : : +- CometBroadcastHashJoin (29) - : : : :- CometProject (27) - : : : : +- CometBroadcastHashJoin (26) - : : : : :- CometFilter (24) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (23) - : : : : +- ReusedExchange (25) - : : : +- ReusedExchange (28) - : : +- CometSort (37) - : : +- CometExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (33) - : +- CometProject (56) - : +- CometSortMergeJoin (55) - : :- CometSort (49) - : : +- CometExchange (48) - : : +- CometProject (47) - : : +- CometBroadcastHashJoin (46) - : : :- CometProject (44) - : : : +- CometBroadcastHashJoin (43) - : : : :- CometFilter (41) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) - : : : +- ReusedExchange (42) - : : +- ReusedExchange (45) - : +- CometSort (54) - : +- CometExchange (53) - : +- CometProject (52) - : +- CometFilter (51) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (50) - +- CometSort (120) - +- CometExchange (119) - +- CometFilter (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometHashAggregate (114) - +- CometExchange (113) - +- CometHashAggregate (112) - +- CometUnion (111) - :- CometProject (82) - : +- CometSortMergeJoin (81) - : :- CometSort (78) - : : +- CometExchange (77) - : : +- CometProject (76) - : : +- CometBroadcastHashJoin (75) - : : :- CometProject (71) - : : : +- CometBroadcastHashJoin (70) - : : : :- CometFilter (68) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (67) - : : : +- ReusedExchange (69) - : : +- CometBroadcastExchange (74) - : : +- CometFilter (73) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) - : +- CometSort (80) - : +- ReusedExchange (79) - :- CometProject (96) - : +- CometSortMergeJoin (95) - : :- CometSort (92) - : : +- CometExchange (91) - : : +- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (87) - : : : +- CometBroadcastHashJoin (86) - : : : :- CometFilter (84) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (83) - : : : +- ReusedExchange (85) - : : +- ReusedExchange (88) - : +- CometSort (94) - : +- ReusedExchange (93) - +- CometProject (110) - +- CometSortMergeJoin (109) - :- CometSort (106) - : +- CometExchange (105) - : +- CometProject (104) - : +- CometBroadcastHashJoin (103) - : :- CometProject (101) - : : +- CometBroadcastHashJoin (100) - : : :- CometFilter (98) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (97) - : : +- ReusedExchange (99) - : +- ReusedExchange (102) - +- CometSort (108) - +- ReusedExchange (107) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#11, 50, true, false, true) = Books ) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) - -(5) CometProject -Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(6) CometBroadcastExchange -Input [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(7) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Right output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(8) CometProject -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(11) CometBroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: [d_date_sk#13, d_year#14] - -(12) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Right output [2]: [d_date_sk#13, d_year#14] -Arguments: [cs_sold_date_sk#5], [d_date_sk#13], Inner, BuildRight - -(13) CometProject -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#13, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] - -(14) CometExchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(15) CometSort -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Arguments: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14], [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST] - -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(17) CometFilter -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) - -(18) CometProject -Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] - -(19) CometExchange -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(20) CometSort -Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST] - -(21) CometSortMergeJoin -Left output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] -Right output [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [cs_order_number#2, cs_item_sk#1], [cr_order_number#16, cr_item_sk#15], LeftOuter - -(22) CometProject -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21], [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] - -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(24) CometFilter -Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#22) - -(25) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(26) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Right output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22], [i_item_sk#28], Inner, BuildRight - -(27) CometProject -Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] - -(28) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#33, d_year#34] - -(29) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] -Right output [2]: [d_date_sk#33, d_year#34] -Arguments: [ss_sold_date_sk#26], [d_date_sk#33], Inner, BuildRight - -(30) CometProject -Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_date_sk#33, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] - -(31) CometExchange -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(32) CometSort -Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Arguments: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34], [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST] - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(34) CometFilter -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) - -(35) CometProject -Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] - -(36) CometExchange -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(37) CometSort -Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST] - -(38) CometSortMergeJoin -Left output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] -Right output [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [ss_ticket_number#23, ss_item_sk#22], [sr_ticket_number#36, sr_item_sk#35], LeftOuter - -(39) CometProject -Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] -Arguments: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41], [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#47)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(41) CometFilter -Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Condition : isnotnull(ws_item_sk#42) - -(42) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(43) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] -Right output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42], [i_item_sk#48], Inner, BuildRight - -(44) CometProject -Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] - -(45) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#53, d_year#54] - -(46) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [ws_sold_date_sk#46], [d_date_sk#53], Inner, BuildRight - -(47) CometProject -Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] - -(48) CometExchange -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(49) CometSort -Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Arguments: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54], [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(51) CometFilter -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) - -(52) CometProject -Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] - -(53) CometExchange -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(54) CometSort -Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST] - -(55) CometSortMergeJoin -Left output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] -Right output [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [ws_order_number#43, ws_item_sk#42], [wr_order_number#56, wr_item_sk#55], LeftOuter - -(56) CometProject -Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] -Arguments: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61], [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] - -(57) CometUnion -Child 0 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, sales_cnt#60, sales_amt#61] - -(58) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(59) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(60) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Functions: [] - -(61) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(62) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(63) CometHashAggregate -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#62, sum#63] -Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(64) CometFilter -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Condition : isnotnull(sales_cnt#64) - -(65) CometExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(66) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65], [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST] - -(67) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct - -(68) CometFilter -Input [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Condition : isnotnull(cs_item_sk#66) - -(69) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(70) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70] -Right output [5]: [i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66], [i_item_sk#72], Inner, BuildRight - -(71) CometProject -Input [10]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_item_sk#72, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(73) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(74) CometBroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: [d_date_sk#77, d_year#78] - -(75) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Right output [2]: [d_date_sk#77, d_year#78] -Arguments: [cs_sold_date_sk#70], [d_date_sk#77], Inner, BuildRight - -(76) CometProject -Input [11]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, cs_sold_date_sk#70, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_date_sk#77, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] - -(77) CometExchange -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: hashpartitioning(cs_order_number#67, cs_item_sk#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(78) CometSort -Input [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Arguments: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78], [cs_order_number#67 ASC NULLS FIRST, cs_item_sk#66 ASC NULLS FIRST] - -(79) ReusedExchange [Reuses operator id: 19] -Output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] - -(80) CometSort -Input [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82], [cr_order_number#80 ASC NULLS FIRST, cr_item_sk#79 ASC NULLS FIRST] - -(81) CometSortMergeJoin -Left output [9]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78] -Right output [4]: [cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [cs_order_number#67, cs_item_sk#66], [cr_order_number#80, cr_item_sk#79], LeftOuter - -(82) CometProject -Input [13]: [cs_item_sk#66, cs_order_number#67, cs_quantity#68, cs_ext_sales_price#69, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, d_year#78, cr_item_sk#79, cr_order_number#80, cr_return_quantity#81, cr_return_amount#82] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21], [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, (cs_quantity#68 - coalesce(cr_return_quantity#81, 0)) AS sales_cnt#20, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#21] - -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(84) CometFilter -Input [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Condition : isnotnull(ss_item_sk#83) - -(85) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] - -(86) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87] -Right output [5]: [i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Arguments: [ss_item_sk#83], [i_item_sk#89], Inner, BuildRight - -(87) CometProject -Input [10]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_item_sk#89, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] - -(88) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#94, d_year#95] - -(89) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93] -Right output [2]: [d_date_sk#94, d_year#95] -Arguments: [ss_sold_date_sk#87], [d_date_sk#94], Inner, BuildRight - -(90) CometProject -Input [11]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, ss_sold_date_sk#87, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_date_sk#94, d_year#95] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] - -(91) CometExchange -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: hashpartitioning(ss_ticket_number#84, ss_item_sk#83, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(92) CometSort -Input [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Arguments: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95], [ss_ticket_number#84 ASC NULLS FIRST, ss_item_sk#83 ASC NULLS FIRST] - -(93) ReusedExchange [Reuses operator id: 36] -Output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] - -(94) CometSort -Input [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99], [sr_ticket_number#97 ASC NULLS FIRST, sr_item_sk#96 ASC NULLS FIRST] - -(95) CometSortMergeJoin -Left output [9]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95] -Right output [4]: [sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [ss_ticket_number#84, ss_item_sk#83], [sr_ticket_number#97, sr_item_sk#96], LeftOuter - -(96) CometProject -Input [13]: [ss_item_sk#83, ss_ticket_number#84, ss_quantity#85, ss_ext_sales_price#86, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, d_year#95, sr_item_sk#96, sr_ticket_number#97, sr_return_quantity#98, sr_return_amt#99] -Arguments: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41], [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, (ss_quantity#85 - coalesce(sr_return_quantity#98, 0)) AS sales_cnt#40, (ss_ext_sales_price#86 - coalesce(sr_return_amt#99, 0.00)) AS sales_amt#41] - -(97) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#104), dynamicpruningexpression(ws_sold_date_sk#104 IN dynamicpruning#105)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(98) CometFilter -Input [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Condition : isnotnull(ws_item_sk#100) - -(99) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] - -(100) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104] -Right output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Arguments: [ws_item_sk#100], [i_item_sk#106], Inner, BuildRight - -(101) CometProject -Input [10]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] - -(102) ReusedExchange [Reuses operator id: 74] -Output [2]: [d_date_sk#111, d_year#112] - -(103) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110] -Right output [2]: [d_date_sk#111, d_year#112] -Arguments: [ws_sold_date_sk#104], [d_date_sk#111], Inner, BuildRight - -(104) CometProject -Input [11]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, ws_sold_date_sk#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] - -(105) CometExchange -Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: hashpartitioning(ws_order_number#101, ws_item_sk#100, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(106) CometSort -Input [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Arguments: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112], [ws_order_number#101 ASC NULLS FIRST, ws_item_sk#100 ASC NULLS FIRST] - -(107) ReusedExchange [Reuses operator id: 53] -Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] - -(108) CometSort -Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116], [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST] - -(109) CometSortMergeJoin -Left output [9]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112] -Right output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [ws_order_number#101, ws_item_sk#100], [wr_order_number#114, wr_item_sk#113], LeftOuter - -(110) CometProject -Input [13]: [ws_item_sk#100, ws_order_number#101, ws_quantity#102, ws_ext_sales_price#103, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116] -Arguments: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61], [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#102 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#60, (ws_ext_sales_price#103 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#61] - -(111) CometUnion -Child 0 Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Child 1 Input [7]: [d_year#95, i_brand_id#90, i_class_id#91, i_category_id#92, i_manufact_id#93, sales_cnt#40, sales_amt#41] -Child 2 Input [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, sales_cnt#60, sales_amt#61] - -(112) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Functions: [] - -(113) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] - -(114) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Functions: [] - -(115) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#20, sales_amt#21] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] - -(116) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] - -(117) CometHashAggregate -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#62, sum#117] -Keys [5]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76] -Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] - -(118) CometFilter -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Condition : isnotnull(sales_cnt#118) - -(119) CometExchange -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: hashpartitioning(i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(120) CometSort -Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119], [i_brand_id#73 ASC NULLS FIRST, i_class_id#74 ASC NULLS FIRST, i_category_id#75 ASC NULLS FIRST, i_manufact_id#76 ASC NULLS FIRST] - -(121) CometSortMergeJoin -Left output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65] -Right output [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76], Inner, ((cast(sales_cnt#64 as decimal(17,2)) / cast(sales_cnt#118 as decimal(17,2))) < 0.90000000000000000000) - -(122) CometProject -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#64, sales_amt#65, d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#118, sales_amt#119] -Arguments: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], [d_year#78 AS prev_year#120, d_year#14 AS year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#118 AS prev_yr_cnt#122, sales_cnt#64 AS curr_yr_cnt#123, (sales_cnt#64 - sales_cnt#118) AS sales_cnt_diff#124, (sales_amt#65 - sales_amt#119) AS sales_amt_diff#125] - -(123) CometTakeOrderedAndProject -Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#124 ASC NULLS FIRST,sales_amt_diff#125 ASC NULLS FIRST], output=[prev_year#120,year#121,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#122,curr_yr_cnt#123,sales_cnt_diff#124,sales_amt_diff#125]), [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125], 100, 0, [sales_cnt_diff#124 ASC NULLS FIRST, sales_amt_diff#125 ASC NULLS FIRST], [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] - -(124) CometColumnarToRow [codegen id : 1] -Input [10]: [prev_year#120, year#121, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#122, curr_yr_cnt#123, sales_cnt_diff#124, sales_amt_diff#125] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (128) -+- * CometColumnarToRow (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) - - -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#13, d_year#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] -ReadSchema: struct - -(126) CometFilter -Input [2]: [d_date_sk#13, d_year#14] -Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) - -(127) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#13, d_year#14] - -(128) BroadcastExchange -Input [2]: [d_date_sk#13, d_year#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 - -Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 - -Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#70 IN dynamicpruning#71 -BroadcastExchange (132) -+- * CometColumnarToRow (131) - +- CometFilter (130) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (129) - - -(129) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#77, d_year#78] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct - -(130) CometFilter -Input [2]: [d_date_sk#77, d_year#78] -Condition : ((isnotnull(d_year#78) AND (d_year#78 = 2001)) AND isnotnull(d_date_sk#77)) - -(131) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#77, d_year#78] - -(132) BroadcastExchange -Input [2]: [d_date_sk#77, d_year#78] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] - -Subquery:5 Hosting operator id = 83 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#71 - -Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#104 IN dynamicpruning#71 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/extended.txt deleted file mode 100644 index 22ccc17641..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/extended.txt +++ /dev/null @@ -1,172 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : +- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - +- CometSort - +- CometExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - -Comet accelerated 159 out of 167 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d0520c8b63..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/simplified.txt +++ /dev/null @@ -1,144 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_cnt_diff,sales_amt_diff] - CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #4 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - 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_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] - CometBroadcastExchange [d_date_sk,d_year] #7 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometExchange [cr_order_number,cr_item_sk] #8 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #9 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometExchange [sr_ticket_number,sr_item_sk] #10 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #11 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #7 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometExchange [wr_order_number,wr_item_sk] #12 - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - CometFilter [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometHashAggregate [sum,sum] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,sum(sales_cnt),sum(UnscaledValue(sales_amt))] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #14 - CometHashAggregate [sales_cnt,sales_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #15 - CometHashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometUnion [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometProject [cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometSort [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] - CometExchange [cs_order_number,cs_item_sk] #16 - 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_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,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 [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] 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 [d_date_sk,d_year] #18 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #8 - CometProject [ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometSort [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] - CometExchange [ss_ticket_number,ss_item_sk] #19 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #10 - CometProject [ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometSortMergeJoin [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,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometSort [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] - CometExchange [ws_order_number,ws_item_sk] #20 - 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_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,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 [native_iceberg_compat] 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 - ReusedExchange [d_date_sk,d_year] #18 - CometSort [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - ReusedExchange [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/explain.txt deleted file mode 100644 index 8c28d58ce5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/explain.txt +++ /dev/null @@ -1,678 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (110) -+- CometTakeOrderedAndProject (109) - +- CometHashAggregate (108) - +- CometColumnarExchange (107) - +- * HashAggregate (106) - +- Union (105) - :- * HashAggregate (90) - : +- * CometColumnarToRow (89) - : +- CometColumnarExchange (88) - : +- * HashAggregate (87) - : +- Union (86) - : :- * Project (32) - : : +- * BroadcastHashJoin LeftOuter BuildRight (31) - : : :- * HashAggregate (16) - : : : +- * CometColumnarToRow (15) - : : : +- CometColumnarExchange (14) - : : : +- * HashAggregate (13) - : : : +- * Project (12) - : : : +- * BroadcastHashJoin Inner BuildRight (11) - : : : :- * Project (6) - : : : : +- * BroadcastHashJoin Inner BuildRight (5) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : +- ReusedExchange (4) - : : : +- BroadcastExchange (10) - : : : +- * CometColumnarToRow (9) - : : : +- CometFilter (8) - : : : +- CometNativeScan parquet spark_catalog.default.store (7) - : : +- BroadcastExchange (30) - : : +- * HashAggregate (29) - : : +- * CometColumnarToRow (28) - : : +- CometColumnarExchange (27) - : : +- * HashAggregate (26) - : : +- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (22) - : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : :- * Filter (19) - : : : : +- * ColumnarToRow (18) - : : : : +- Scan parquet spark_catalog.default.store_returns (17) - : : : +- ReusedExchange (20) - : : +- ReusedExchange (23) - : :- * Project (53) - : : +- * BroadcastNestedLoopJoin Inner BuildLeft (52) - : : :- BroadcastExchange (42) - : : : +- * HashAggregate (41) - : : : +- * CometColumnarToRow (40) - : : : +- CometColumnarExchange (39) - : : : +- * HashAggregate (38) - : : : +- * Project (37) - : : : +- * BroadcastHashJoin Inner BuildRight (36) - : : : :- * ColumnarToRow (34) - : : : : +- Scan parquet spark_catalog.default.catalog_sales (33) - : : : +- ReusedExchange (35) - : : +- * HashAggregate (51) - : : +- * CometColumnarToRow (50) - : : +- CometColumnarExchange (49) - : : +- * HashAggregate (48) - : : +- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) - : : :- * ColumnarToRow (44) - : : : +- Scan parquet spark_catalog.default.catalog_returns (43) - : : +- ReusedExchange (45) - : +- * Project (85) - : +- * BroadcastHashJoin LeftOuter BuildRight (84) - : :- * HashAggregate (69) - : : +- * CometColumnarToRow (68) - : : +- CometColumnarExchange (67) - : : +- * HashAggregate (66) - : : +- * Project (65) - : : +- * BroadcastHashJoin Inner BuildRight (64) - : : :- * Project (59) - : : : +- * BroadcastHashJoin Inner BuildRight (58) - : : : :- * Filter (56) - : : : : +- * ColumnarToRow (55) - : : : : +- Scan parquet spark_catalog.default.web_sales (54) - : : : +- ReusedExchange (57) - : : +- BroadcastExchange (63) - : : +- * CometColumnarToRow (62) - : : +- CometFilter (61) - : : +- CometNativeScan parquet spark_catalog.default.web_page (60) - : +- BroadcastExchange (83) - : +- * HashAggregate (82) - : +- * CometColumnarToRow (81) - : +- CometColumnarExchange (80) - : +- * HashAggregate (79) - : +- * Project (78) - : +- * BroadcastHashJoin Inner BuildRight (77) - : :- * Project (75) - : : +- * BroadcastHashJoin Inner BuildRight (74) - : : :- * Filter (72) - : : : +- * ColumnarToRow (71) - : : : +- Scan parquet spark_catalog.default.web_returns (70) - : : +- ReusedExchange (73) - : +- ReusedExchange (76) - :- * HashAggregate (97) - : +- * CometColumnarToRow (96) - : +- CometColumnarExchange (95) - : +- * HashAggregate (94) - : +- * HashAggregate (93) - : +- * CometColumnarToRow (92) - : +- ReusedExchange (91) - +- * HashAggregate (104) - +- * CometColumnarToRow (103) - +- CometColumnarExchange (102) - +- * HashAggregate (101) - +- * HashAggregate (100) - +- * CometColumnarToRow (99) - +- ReusedExchange (98) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] - -(3) Filter [codegen id : 3] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(4) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#6] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] - -(7) CometNativeScan parquet spark_catalog.default.store -Output [1]: [s_store_sk#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(8) CometFilter -Input [1]: [s_store_sk#7] -Condition : isnotnull(s_store_sk#7) - -(9) CometColumnarToRow [codegen id : 2] -Input [1]: [s_store_sk#7] - -(10) BroadcastExchange -Input [1]: [s_store_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] - -(11) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None - -(12) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] - -(13) HashAggregate [codegen id : 3] -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] -Keys [1]: [s_store_sk#7] -Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum#8, sum#9] -Results [3]: [s_store_sk#7, sum#10, sum#11] - -(14) CometColumnarExchange -Input [3]: [s_store_sk#7, sum#10, sum#11] -Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(15) CometColumnarToRow [codegen id : 8] -Input [3]: [s_store_sk#7, sum#10, sum#11] - -(16) HashAggregate [codegen id : 8] -Input [3]: [s_store_sk#7, sum#10, sum#11] -Keys [1]: [s_store_sk#7] -Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#12, sum(UnscaledValue(ss_net_profit#3))#13] -Results [3]: [s_store_sk#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS sales#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#13,17,2) AS profit#15] - -(17) Scan parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(18) ColumnarToRow [codegen id : 6] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] - -(19) Filter [codegen id : 6] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_store_sk#16) - -(20) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#20] - -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_returned_date_sk#19] -Right keys [1]: [d_date_sk#20] -Join type: Inner -Join condition: None - -(22) Project [codegen id : 6] -Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] -Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#20] - -(23) ReusedExchange [Reuses operator id: 10] -Output [1]: [s_store_sk#21] - -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [sr_store_sk#16] -Right keys [1]: [s_store_sk#21] -Join type: Inner -Join condition: None - -(25) Project [codegen id : 6] -Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] -Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#21] - -(26) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#21] -Keys [1]: [s_store_sk#21] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum#22, sum#23] -Results [3]: [s_store_sk#21, sum#24, sum#25] - -(27) CometColumnarExchange -Input [3]: [s_store_sk#21, sum#24, sum#25] -Arguments: hashpartitioning(s_store_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(28) CometColumnarToRow [codegen id : 7] -Input [3]: [s_store_sk#21, sum#24, sum#25] - -(29) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#21, sum#24, sum#25] -Keys [1]: [s_store_sk#21] -Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#17))#26, sum(UnscaledValue(sr_net_loss#18))#27] -Results [3]: [s_store_sk#21, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#26,17,2) AS returns#28, MakeDecimal(sum(UnscaledValue(sr_net_loss#18))#27,17,2) AS profit_loss#29] - -(30) BroadcastExchange -Input [3]: [s_store_sk#21, returns#28, profit_loss#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - -(31) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [s_store_sk#7] -Right keys [1]: [s_store_sk#21] -Join type: LeftOuter -Join condition: None - -(32) Project [codegen id : 8] -Output [5]: [store channel AS channel#30, s_store_sk#7 AS id#31, sales#14, coalesce(returns#28, 0.00) AS returns#32, (profit#15 - coalesce(profit_loss#29, 0.00)) AS profit#33] -Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#21, returns#28, profit_loss#29] - -(33) Scan parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#37), dynamicpruningexpression(cs_sold_date_sk#37 IN dynamicpruning#5)] -ReadSchema: struct - -(34) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37] - -(35) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#38] - -(36) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#37] -Right keys [1]: [d_date_sk#38] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] -Input [5]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36, cs_sold_date_sk#37, d_date_sk#38] - -(38) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#34, cs_ext_sales_price#35, cs_net_profit#36] -Keys [1]: [cs_call_center_sk#34] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#35)), partial_sum(UnscaledValue(cs_net_profit#36))] -Aggregate Attributes [2]: [sum#39, sum#40] -Results [3]: [cs_call_center_sk#34, sum#41, sum#42] - -(39) CometColumnarExchange -Input [3]: [cs_call_center_sk#34, sum#41, sum#42] -Arguments: hashpartitioning(cs_call_center_sk#34, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(40) CometColumnarToRow [codegen id : 11] -Input [3]: [cs_call_center_sk#34, sum#41, sum#42] - -(41) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#34, sum#41, sum#42] -Keys [1]: [cs_call_center_sk#34] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#35)), sum(UnscaledValue(cs_net_profit#36))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#35))#43, sum(UnscaledValue(cs_net_profit#36))#44] -Results [3]: [cs_call_center_sk#34, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#35))#43,17,2) AS sales#45, MakeDecimal(sum(UnscaledValue(cs_net_profit#36))#44,17,2) AS profit#46] - -(42) BroadcastExchange -Input [3]: [cs_call_center_sk#34, sales#45, profit#46] -Arguments: IdentityBroadcastMode, [plan_id=6] - -(43) Scan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#49), dynamicpruningexpression(cr_returned_date_sk#49 IN dynamicpruning#5)] -ReadSchema: struct - -(44) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49] - -(45) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#50] - -(46) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#49] -Right keys [1]: [d_date_sk#50] -Join type: Inner -Join condition: None - -(47) Project [codegen id : 13] -Output [2]: [cr_return_amount#47, cr_net_loss#48] -Input [4]: [cr_return_amount#47, cr_net_loss#48, cr_returned_date_sk#49, d_date_sk#50] - -(48) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#47, cr_net_loss#48] -Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#47)), partial_sum(UnscaledValue(cr_net_loss#48))] -Aggregate Attributes [2]: [sum#51, sum#52] -Results [2]: [sum#53, sum#54] - -(49) CometColumnarExchange -Input [2]: [sum#53, sum#54] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(50) CometColumnarToRow -Input [2]: [sum#53, sum#54] - -(51) HashAggregate -Input [2]: [sum#53, sum#54] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#47)), sum(UnscaledValue(cr_net_loss#48))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#47))#55, sum(UnscaledValue(cr_net_loss#48))#56] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#47))#55,17,2) AS returns#57, MakeDecimal(sum(UnscaledValue(cr_net_loss#48))#56,17,2) AS profit_loss#58] - -(52) BroadcastNestedLoopJoin [codegen id : 14] -Join type: Inner -Join condition: None - -(53) Project [codegen id : 14] -Output [5]: [catalog channel AS channel#59, cs_call_center_sk#34 AS id#60, sales#45, returns#57, (profit#46 - profit_loss#58) AS profit#61] -Input [5]: [cs_call_center_sk#34, sales#45, profit#46, returns#57, profit_loss#58] - -(54) Scan parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] - -(56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65] -Condition : isnotnull(ws_web_page_sk#62) - -(57) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#66] - -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#65] -Right keys [1]: [d_date_sk#66] -Join type: Inner -Join condition: None - -(59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64] -Input [5]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, ws_sold_date_sk#65, d_date_sk#66] - -(60) CometNativeScan parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(61) CometFilter -Input [1]: [wp_web_page_sk#67] -Condition : isnotnull(wp_web_page_sk#67) - -(62) CometColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#67] - -(63) BroadcastExchange -Input [1]: [wp_web_page_sk#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] - -(64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#62] -Right keys [1]: [wp_web_page_sk#67] -Join type: Inner -Join condition: None - -(65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] -Input [4]: [ws_web_page_sk#62, ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] - -(66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#63, ws_net_profit#64, wp_web_page_sk#67] -Keys [1]: [wp_web_page_sk#67] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#63)), partial_sum(UnscaledValue(ws_net_profit#64))] -Aggregate Attributes [2]: [sum#68, sum#69] -Results [3]: [wp_web_page_sk#67, sum#70, sum#71] - -(67) CometColumnarExchange -Input [3]: [wp_web_page_sk#67, sum#70, sum#71] -Arguments: hashpartitioning(wp_web_page_sk#67, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(68) CometColumnarToRow [codegen id : 22] -Input [3]: [wp_web_page_sk#67, sum#70, sum#71] - -(69) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#67, sum#70, sum#71] -Keys [1]: [wp_web_page_sk#67] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#63)), sum(UnscaledValue(ws_net_profit#64))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#63))#72, sum(UnscaledValue(ws_net_profit#64))#73] -Results [3]: [wp_web_page_sk#67, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#63))#72,17,2) AS sales#74, MakeDecimal(sum(UnscaledValue(ws_net_profit#64))#73,17,2) AS profit#75] - -(70) Scan parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#79), dynamicpruningexpression(wr_returned_date_sk#79 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(wr_web_page_sk)] -ReadSchema: struct - -(71) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] - -(72) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79] -Condition : isnotnull(wr_web_page_sk#76) - -(73) ReusedExchange [Reuses operator id: 115] -Output [1]: [d_date_sk#80] - -(74) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#79] -Right keys [1]: [d_date_sk#80] -Join type: Inner -Join condition: None - -(75) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78] -Input [5]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wr_returned_date_sk#79, d_date_sk#80] - -(76) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#81] - -(77) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#76] -Right keys [1]: [wp_web_page_sk#81] -Join type: Inner -Join condition: None - -(78) Project [codegen id : 20] -Output [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] -Input [4]: [wr_web_page_sk#76, wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] - -(79) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#77, wr_net_loss#78, wp_web_page_sk#81] -Keys [1]: [wp_web_page_sk#81] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#77)), partial_sum(UnscaledValue(wr_net_loss#78))] -Aggregate Attributes [2]: [sum#82, sum#83] -Results [3]: [wp_web_page_sk#81, sum#84, sum#85] - -(80) CometColumnarExchange -Input [3]: [wp_web_page_sk#81, sum#84, sum#85] -Arguments: hashpartitioning(wp_web_page_sk#81, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(81) CometColumnarToRow [codegen id : 21] -Input [3]: [wp_web_page_sk#81, sum#84, sum#85] - -(82) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#81, sum#84, sum#85] -Keys [1]: [wp_web_page_sk#81] -Functions [2]: [sum(UnscaledValue(wr_return_amt#77)), sum(UnscaledValue(wr_net_loss#78))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#77))#86, sum(UnscaledValue(wr_net_loss#78))#87] -Results [3]: [wp_web_page_sk#81, MakeDecimal(sum(UnscaledValue(wr_return_amt#77))#86,17,2) AS returns#88, MakeDecimal(sum(UnscaledValue(wr_net_loss#78))#87,17,2) AS profit_loss#89] - -(83) BroadcastExchange -Input [3]: [wp_web_page_sk#81, returns#88, profit_loss#89] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] - -(84) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#67] -Right keys [1]: [wp_web_page_sk#81] -Join type: LeftOuter -Join condition: None - -(85) Project [codegen id : 22] -Output [5]: [web channel AS channel#90, wp_web_page_sk#67 AS id#91, sales#74, coalesce(returns#88, 0.00) AS returns#92, (profit#75 - coalesce(profit_loss#89, 0.00)) AS profit#93] -Input [6]: [wp_web_page_sk#67, sales#74, profit#75, wp_web_page_sk#81, returns#88, profit_loss#89] - -(86) Union - -(87) HashAggregate [codegen id : 23] -Input [5]: [channel#30, id#31, sales#14, returns#32, profit#33] -Keys [2]: [channel#30, id#31] -Functions [3]: [partial_sum(sales#14), partial_sum(returns#32), partial_sum(profit#33)] -Aggregate Attributes [6]: [sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] -Results [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] - -(88) CometColumnarExchange -Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Arguments: hashpartitioning(channel#30, id#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] - -(89) CometColumnarToRow [codegen id : 24] -Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] - -(90) HashAggregate [codegen id : 24] -Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Keys [2]: [channel#30, id#31] -Functions [3]: [sum(sales#14), sum(returns#32), sum(profit#33)] -Aggregate Attributes [3]: [sum(sales#14)#106, sum(returns#32)#107, sum(profit#33)#108] -Results [5]: [channel#30, id#31, cast(sum(sales#14)#106 as decimal(37,2)) AS sales#109, cast(sum(returns#32)#107 as decimal(37,2)) AS returns#110, cast(sum(profit#33)#108 as decimal(38,2)) AS profit#111] - -(91) ReusedExchange [Reuses operator id: 88] -Output [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] - -(92) CometColumnarToRow [codegen id : 48] -Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] - -(93) HashAggregate [codegen id : 48] -Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Keys [2]: [channel#30, id#31] -Functions [3]: [sum(sales#14), sum(returns#32), sum(profit#33)] -Aggregate Attributes [3]: [sum(sales#14)#106, sum(returns#32)#107, sum(profit#33)#108] -Results [4]: [channel#30, sum(sales#14)#106 AS sales#112, sum(returns#32)#107 AS returns#113, sum(profit#33)#108 AS profit#114] - -(94) HashAggregate [codegen id : 48] -Input [4]: [channel#30, sales#112, returns#113, profit#114] -Keys [1]: [channel#30] -Functions [3]: [partial_sum(sales#112), partial_sum(returns#113), partial_sum(profit#114)] -Aggregate Attributes [6]: [sum#115, isEmpty#116, sum#117, isEmpty#118, sum#119, isEmpty#120] -Results [7]: [channel#30, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] - -(95) CometColumnarExchange -Input [7]: [channel#30, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] -Arguments: hashpartitioning(channel#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] - -(96) CometColumnarToRow [codegen id : 49] -Input [7]: [channel#30, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] - -(97) HashAggregate [codegen id : 49] -Input [7]: [channel#30, sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] -Keys [1]: [channel#30] -Functions [3]: [sum(sales#112), sum(returns#113), sum(profit#114)] -Aggregate Attributes [3]: [sum(sales#112)#127, sum(returns#113)#128, sum(profit#114)#129] -Results [5]: [channel#30, null AS id#130, sum(sales#112)#127 AS sales#131, sum(returns#113)#128 AS returns#132, sum(profit#114)#129 AS profit#133] - -(98) ReusedExchange [Reuses operator id: 88] -Output [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] - -(99) CometColumnarToRow [codegen id : 73] -Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] - -(100) HashAggregate [codegen id : 73] -Input [8]: [channel#30, id#31, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Keys [2]: [channel#30, id#31] -Functions [3]: [sum(sales#14), sum(returns#32), sum(profit#33)] -Aggregate Attributes [3]: [sum(sales#14)#106, sum(returns#32)#107, sum(profit#33)#108] -Results [3]: [sum(sales#14)#106 AS sales#112, sum(returns#32)#107 AS returns#113, sum(profit#33)#108 AS profit#114] - -(101) HashAggregate [codegen id : 73] -Input [3]: [sales#112, returns#113, profit#114] -Keys: [] -Functions [3]: [partial_sum(sales#112), partial_sum(returns#113), partial_sum(profit#114)] -Aggregate Attributes [6]: [sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Results [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] - -(102) CometColumnarExchange -Input [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] - -(103) CometColumnarToRow [codegen id : 74] -Input [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] - -(104) HashAggregate [codegen id : 74] -Input [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] -Keys: [] -Functions [3]: [sum(sales#112), sum(returns#113), sum(profit#114)] -Aggregate Attributes [3]: [sum(sales#112)#146, sum(returns#113)#147, sum(profit#114)#148] -Results [5]: [null AS channel#149, null AS id#150, sum(sales#112)#146 AS sales#151, sum(returns#113)#147 AS returns#152, sum(profit#114)#148 AS profit#153] - -(105) Union - -(106) HashAggregate [codegen id : 75] -Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] -Keys [5]: [channel#30, id#31, sales#109, returns#110, profit#111] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#30, id#31, sales#109, returns#110, profit#111] - -(107) CometColumnarExchange -Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] -Arguments: hashpartitioning(channel#30, id#31, sales#109, returns#110, profit#111, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] - -(108) CometHashAggregate -Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] -Keys [5]: [channel#30, id#31, sales#109, returns#110, profit#111] -Functions: [] - -(109) CometTakeOrderedAndProject -Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#30 ASC NULLS FIRST,id#31 ASC NULLS FIRST], output=[channel#30,id#31,sales#109,returns#110,profit#111]), [channel#30, id#31, sales#109, returns#110, profit#111], 100, 0, [channel#30 ASC NULLS FIRST, id#31 ASC NULLS FIRST], [channel#30, id#31, sales#109, returns#110, profit#111] - -(110) CometColumnarToRow [codegen id : 76] -Input [5]: [channel#30, id#31, sales#109, returns#110, profit#111] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (115) -+- * CometColumnarToRow (114) - +- CometProject (113) - +- CometFilter (112) - +- CometNativeScan parquet spark_catalog.default.date_dim (111) - - -(111) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#154] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(112) CometFilter -Input [2]: [d_date_sk#6, d_date#154] -Condition : (((isnotnull(d_date#154) AND (d_date#154 >= 1998-08-04)) AND (d_date#154 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) - -(113) CometProject -Input [2]: [d_date_sk#6, d_date#154] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(114) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(115) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] - -Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#49 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 70 Hosting Expression = wr_returned_date_sk#79 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/simplified.txt deleted file mode 100644 index 6f40fcdddf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/simplified.txt +++ /dev/null @@ -1,180 +0,0 @@ -WholeStageCodegen (76) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (75) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (24) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (23) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (8) - Project [s_store_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [s_store_sk,s_store_sk] - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #3 - WholeStageCodegen (3) - HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] - Project [ss_ext_sales_price,ss_net_profit,s_store_sk] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_store_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk] - CometNativeScan parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (7) - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_store_sk] #7 - WholeStageCodegen (6) - HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] - Project [sr_return_amt,sr_net_loss,s_store_sk] - BroadcastHashJoin [sr_store_sk,s_store_sk] - Project [sr_store_sk,sr_return_amt,sr_net_loss] - BroadcastHashJoin [sr_returned_date_sk,d_date_sk] - Filter [sr_store_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [s_store_sk] #5 - WholeStageCodegen (14) - Project [cs_call_center_sk,sales,returns,profit,profit_loss] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (11) - HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [cs_call_center_sk] #9 - WholeStageCodegen (10) - HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] - Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #4 - HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange #10 - WholeStageCodegen (13) - HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] - Project [cr_return_amount,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - InputAdapter - ReusedExchange [d_date_sk] #4 - WholeStageCodegen (22) - Project [wp_web_page_sk,sales,returns,profit,profit_loss] - BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wp_web_page_sk] #11 - WholeStageCodegen (17) - HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] - Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] - BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] - Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_web_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometFilter [wp_web_page_sk] - CometNativeScan parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (21) - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [wp_web_page_sk] #14 - WholeStageCodegen (20) - HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] - Project [wr_return_amt,wr_net_loss,wp_web_page_sk] - BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] - Project [wr_web_page_sk,wr_return_amt,wr_net_loss] - BroadcastHashJoin [wr_returned_date_sk,d_date_sk] - Filter [wr_web_page_sk] - ColumnarToRow - InputAdapter - Scan 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 - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - ReusedExchange [wp_web_page_sk] #12 - WholeStageCodegen (49) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #15 - WholeStageCodegen (48) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (74) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #16 - WholeStageCodegen (73) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/explain.txt deleted file mode 100644 index e36aaa9291..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,621 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (103) -+- CometTakeOrderedAndProject (102) - +- CometHashAggregate (101) - +- CometColumnarExchange (100) - +- * HashAggregate (99) - +- Union (98) - :- * HashAggregate (83) - : +- * CometColumnarToRow (82) - : +- CometColumnarExchange (81) - : +- * HashAggregate (80) - : +- Union (79) - : :- * CometColumnarToRow (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : : +- CometBroadcastExchange (28) - : : +- CometHashAggregate (27) - : : +- CometExchange (26) - : : +- CometHashAggregate (25) - : : +- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (21) - : : : +- CometBroadcastHashJoin (20) - : : : :- CometFilter (18) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (17) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : :- * Project (50) - : : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) - : : :- BroadcastExchange (40) - : : : +- * CometColumnarToRow (39) - : : : +- CometHashAggregate (38) - : : : +- CometExchange (37) - : : : +- CometHashAggregate (36) - : : : +- CometProject (35) - : : : +- CometBroadcastHashJoin (34) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (32) - : : : +- ReusedExchange (33) - : : +- * CometColumnarToRow (48) - : : +- CometHashAggregate (47) - : : +- CometExchange (46) - : : +- CometHashAggregate (45) - : : +- CometProject (44) - : : +- CometBroadcastHashJoin (43) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (41) - : : +- ReusedExchange (42) - : +- * CometColumnarToRow (78) - : +- CometProject (77) - : +- CometBroadcastHashJoin (76) - : :- CometHashAggregate (63) - : : +- CometExchange (62) - : : +- CometHashAggregate (61) - : : +- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (55) - : : : +- CometBroadcastHashJoin (54) - : : : :- CometFilter (52) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (51) - : : : +- ReusedExchange (53) - : : +- CometBroadcastExchange (58) - : : +- CometFilter (57) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (56) - : +- CometBroadcastExchange (75) - : +- CometHashAggregate (74) - : +- CometExchange (73) - : +- CometHashAggregate (72) - : +- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometProject (68) - : : +- CometBroadcastHashJoin (67) - : : :- CometFilter (65) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (64) - : : +- ReusedExchange (66) - : +- ReusedExchange (69) - :- * HashAggregate (90) - : +- * CometColumnarToRow (89) - : +- CometColumnarExchange (88) - : +- * HashAggregate (87) - : +- * HashAggregate (86) - : +- * CometColumnarToRow (85) - : +- ReusedExchange (84) - +- * HashAggregate (97) - +- * CometColumnarToRow (96) - +- CometColumnarExchange (95) - +- * HashAggregate (94) - +- * HashAggregate (93) - +- * CometColumnarToRow (92) - +- ReusedExchange (91) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3], [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [1]: [s_store_sk#8] -Condition : isnotnull(s_store_sk#8) - -(11) CometBroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: [s_store_sk#8] - -(12) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -Right output [1]: [s_store_sk#8] -Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight - -(13) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] - -(14) CometHashAggregate -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Keys [1]: [s_store_sk#8] -Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] - -(15) CometExchange -Input [3]: [s_store_sk#8, sum#9, sum#10] -Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [3]: [s_store_sk#8, sum#9, sum#10] -Keys [1]: [s_store_sk#8] -Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(18) CometFilter -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Condition : isnotnull(sr_store_sk#11) - -(19) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#16] - -(20) CometBroadcastHashJoin -Left output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Right output [1]: [d_date_sk#16] -Arguments: [sr_returned_date_sk#14], [d_date_sk#16], Inner, BuildRight - -(21) CometProject -Input [5]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14, d_date_sk#16] -Arguments: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13], [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] - -(22) ReusedExchange [Reuses operator id: 11] -Output [1]: [s_store_sk#17] - -(23) CometBroadcastHashJoin -Left output [3]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] -Right output [1]: [s_store_sk#17] -Arguments: [sr_store_sk#11], [s_store_sk#17], Inner, BuildRight - -(24) CometProject -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Arguments: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17], [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] - -(25) CometHashAggregate -Input [3]: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Keys [1]: [s_store_sk#17] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#12)), partial_sum(UnscaledValue(sr_net_loss#13))] - -(26) CometExchange -Input [3]: [s_store_sk#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(27) CometHashAggregate -Input [3]: [s_store_sk#17, sum#18, sum#19] -Keys [1]: [s_store_sk#17] -Functions [2]: [sum(UnscaledValue(sr_return_amt#12)), sum(UnscaledValue(sr_net_loss#13))] - -(28) CometBroadcastExchange -Input [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#17, returns#20, profit_loss#21] - -(29) CometBroadcastHashJoin -Left output [3]: [s_store_sk#8, sales#22, profit#23] -Right output [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight - -(30) CometProject -Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] -Arguments: [channel#24, id#25, sales#22, returns#26, profit#27], [store channel AS channel#24, s_store_sk#8 AS id#25, sales#22, coalesce(returns#20, 0.00) AS returns#26, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#27] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] -ReadSchema: struct - -(33) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(34) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [cs_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(35) CometProject -Input [5]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31, d_date_sk#33] -Arguments: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30], [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] - -(36) CometHashAggregate -Input [3]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#29)), partial_sum(UnscaledValue(cs_net_profit#30))] - -(37) CometExchange -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Arguments: hashpartitioning(cs_call_center_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] - -(39) CometColumnarToRow [codegen id : 2] -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] - -(40) BroadcastExchange -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] -Arguments: IdentityBroadcastMode, [plan_id=4] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] -ReadSchema: struct - -(42) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#42] - -(43) CometBroadcastHashJoin -Left output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Right output [1]: [d_date_sk#42] -Arguments: [cr_returned_date_sk#40], [d_date_sk#42], Inner, BuildRight - -(44) CometProject -Input [4]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40, d_date_sk#42] -Arguments: [cr_return_amount#38, cr_net_loss#39], [cr_return_amount#38, cr_net_loss#39] - -(45) CometHashAggregate -Input [2]: [cr_return_amount#38, cr_net_loss#39] -Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] - -(46) CometExchange -Input [2]: [sum#43, sum#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometHashAggregate -Input [2]: [sum#43, sum#44] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] - -(48) CometColumnarToRow -Input [2]: [returns#45, profit_loss#46] - -(49) BroadcastNestedLoopJoin [codegen id : 3] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 3] -Output [5]: [catalog channel AS channel#47, cs_call_center_sk#28 AS id#48, sales#36, returns#45, (profit#37 - profit_loss#46) AS profit#49] -Input [5]: [cs_call_center_sk#28, sales#36, profit#37, returns#45, profit_loss#46] - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] -PushedFilters: [IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(52) CometFilter -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Condition : isnotnull(ws_web_page_sk#50) - -(53) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#55] - -(54) CometBroadcastHashJoin -Left output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Right output [1]: [d_date_sk#55] -Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight - -(55) CometProject -Input [5]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53, d_date_sk#55] -Arguments: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52], [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(57) CometFilter -Input [1]: [wp_web_page_sk#56] -Condition : isnotnull(wp_web_page_sk#56) - -(58) CometBroadcastExchange -Input [1]: [wp_web_page_sk#56] -Arguments: [wp_web_page_sk#56] - -(59) CometBroadcastHashJoin -Left output [3]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] -Right output [1]: [wp_web_page_sk#56] -Arguments: [ws_web_page_sk#50], [wp_web_page_sk#56], Inner, BuildRight - -(60) CometProject -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Arguments: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56], [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] - -(61) CometHashAggregate -Input [3]: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#51)), partial_sum(UnscaledValue(ws_net_profit#52))] - -(62) CometExchange -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Arguments: hashpartitioning(wp_web_page_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(63) CometHashAggregate -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#51)), sum(UnscaledValue(ws_net_profit#52))] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] -PushedFilters: [IsNotNull(wr_web_page_sk)] -ReadSchema: struct - -(65) CometFilter -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Condition : isnotnull(wr_web_page_sk#59) - -(66) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#64] - -(67) CometBroadcastHashJoin -Left output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Right output [1]: [d_date_sk#64] -Arguments: [wr_returned_date_sk#62], [d_date_sk#64], Inner, BuildRight - -(68) CometProject -Input [5]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62, d_date_sk#64] -Arguments: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61], [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] - -(69) ReusedExchange [Reuses operator id: 58] -Output [1]: [wp_web_page_sk#65] - -(70) CometBroadcastHashJoin -Left output [3]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] -Right output [1]: [wp_web_page_sk#65] -Arguments: [wr_web_page_sk#59], [wp_web_page_sk#65], Inner, BuildRight - -(71) CometProject -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Arguments: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65], [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] - -(72) CometHashAggregate -Input [3]: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#60)), partial_sum(UnscaledValue(wr_net_loss#61))] - -(73) CometExchange -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Arguments: hashpartitioning(wp_web_page_sk#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(74) CometHashAggregate -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [sum(UnscaledValue(wr_return_amt#60)), sum(UnscaledValue(wr_net_loss#61))] - -(75) CometBroadcastExchange -Input [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#65, returns#68, profit_loss#69] - -(76) CometBroadcastHashJoin -Left output [3]: [wp_web_page_sk#56, sales#70, profit#71] -Right output [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight - -(77) CometProject -Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [channel#72, id#73, sales#70, returns#74, profit#75], [web channel AS channel#72, wp_web_page_sk#56 AS id#73, sales#70, coalesce(returns#68, 0.00) AS returns#74, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#75] - -(78) CometColumnarToRow [codegen id : 4] -Input [5]: [channel#72, id#73, sales#70, returns#74, profit#75] - -(79) Union - -(80) HashAggregate [codegen id : 5] -Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] -Keys [2]: [channel#24, id#25] -Functions [3]: [partial_sum(sales#22), partial_sum(returns#26), partial_sum(profit#27)] -Aggregate Attributes [6]: [sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81] -Results [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(81) CometColumnarExchange -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Arguments: hashpartitioning(channel#24, id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(82) CometColumnarToRow [codegen id : 6] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(83) HashAggregate [codegen id : 6] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Keys [2]: [channel#24, id#25] -Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] -Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] -Results [5]: [channel#24, id#25, cast(sum(sales#22)#88 as decimal(37,2)) AS sales#91, cast(sum(returns#26)#89 as decimal(37,2)) AS returns#92, cast(sum(profit#27)#90 as decimal(38,2)) AS profit#93] - -(84) ReusedExchange [Reuses operator id: 81] -Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(85) CometColumnarToRow [codegen id : 12] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(86) HashAggregate [codegen id : 12] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Keys [2]: [channel#24, id#25] -Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] -Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] -Results [4]: [channel#24, sum(sales#22)#88 AS sales#94, sum(returns#26)#89 AS returns#95, sum(profit#27)#90 AS profit#96] - -(87) HashAggregate [codegen id : 12] -Input [4]: [channel#24, sales#94, returns#95, profit#96] -Keys [1]: [channel#24] -Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] -Aggregate Attributes [6]: [sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Results [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] - -(88) CometColumnarExchange -Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Arguments: hashpartitioning(channel#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(89) CometColumnarToRow [codegen id : 13] -Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] - -(90) HashAggregate [codegen id : 13] -Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Keys [1]: [channel#24] -Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] -Aggregate Attributes [3]: [sum(sales#94)#109, sum(returns#95)#110, sum(profit#96)#111] -Results [5]: [channel#24, null AS id#112, sum(sales#94)#109 AS sales#113, sum(returns#95)#110 AS returns#114, sum(profit#96)#111 AS profit#115] - -(91) ReusedExchange [Reuses operator id: 81] -Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(92) CometColumnarToRow [codegen id : 19] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(93) HashAggregate [codegen id : 19] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Keys [2]: [channel#24, id#25] -Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] -Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] -Results [3]: [sum(sales#22)#88 AS sales#94, sum(returns#26)#89 AS returns#95, sum(profit#27)#90 AS profit#96] - -(94) HashAggregate [codegen id : 19] -Input [3]: [sales#94, returns#95, profit#96] -Keys: [] -Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] -Aggregate Attributes [6]: [sum#116, isEmpty#117, sum#118, isEmpty#119, sum#120, isEmpty#121] -Results [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] - -(95) CometColumnarExchange -Input [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(96) CometColumnarToRow [codegen id : 20] -Input [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] - -(97) HashAggregate [codegen id : 20] -Input [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] -Keys: [] -Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] -Aggregate Attributes [3]: [sum(sales#94)#128, sum(returns#95)#129, sum(profit#96)#130] -Results [5]: [null AS channel#131, null AS id#132, sum(sales#94)#128 AS sales#133, sum(returns#95)#129 AS returns#134, sum(profit#96)#130 AS profit#135] - -(98) Union - -(99) HashAggregate [codegen id : 21] -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#24, id#25, sales#91, returns#92, profit#93] - -(100) CometColumnarExchange -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Arguments: hashpartitioning(channel#24, id#25, sales#91, returns#92, profit#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(101) CometHashAggregate -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Functions: [] - -(102) CometTakeOrderedAndProject -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#24 ASC NULLS FIRST,id#25 ASC NULLS FIRST], output=[channel#24,id#25,sales#91,returns#92,profit#93]), [channel#24, id#25, sales#91, returns#92, profit#93], 100, 0, [channel#24 ASC NULLS FIRST, id#25 ASC NULLS FIRST], [channel#24, id#25, sales#91, returns#92, profit#93] - -(103) CometColumnarToRow [codegen id : 22] -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (108) -+- * CometColumnarToRow (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) - - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) - -(106) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(107) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(108) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#14 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 64 Hosting Expression = wr_returned_date_sk#62 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/simplified.txt deleted file mode 100644 index ac3d312ee8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,141 +0,0 @@ -WholeStageCodegen (22) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (21) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (5) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] - CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [s_store_sk,sales,profit,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] - CometExchange [s_store_sk] #3 - CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [s_store_sk,returns,profit_loss] #7 - CometHashAggregate [sum,sum] [s_store_sk,returns,profit_loss,sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss))] - CometExchange [s_store_sk] #8 - CometHashAggregate [sr_return_amt,sr_net_loss] [s_store_sk,sum,sum] - CometProject [sr_return_amt,sr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [s_store_sk] #6 - WholeStageCodegen (3) - Project [cs_call_center_sk,sales,returns,profit,profit_loss] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [cs_call_center_sk,sales,profit,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] - CometExchange [cs_call_center_sk] #10 - CometHashAggregate [cs_ext_sales_price,cs_net_profit] [cs_call_center_sk,sum,sum] - CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [returns,profit_loss,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] - CometExchange #11 - CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] - CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [wp_web_page_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] - CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [wp_web_page_sk,sales,profit,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] - CometExchange [wp_web_page_sk] #12 - CometHashAggregate [ws_ext_sales_price,ws_net_profit] [wp_web_page_sk,sum,sum] - CometProject [ws_ext_sales_price,ws_net_profit,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_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 [native_iceberg_compat] 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 [wp_web_page_sk] #13 - CometFilter [wp_web_page_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [wp_web_page_sk,returns,profit_loss] #14 - CometHashAggregate [sum,sum] [wp_web_page_sk,returns,profit_loss,sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss))] - CometExchange [wp_web_page_sk] #15 - CometHashAggregate [wr_return_amt,wr_net_loss] [wp_web_page_sk,sum,sum] - CometProject [wr_return_amt,wr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [wp_web_page_sk] #13 - WholeStageCodegen (13) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #16 - WholeStageCodegen (12) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (20) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #17 - WholeStageCodegen (19) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt deleted file mode 100644 index e36aaa9291..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt +++ /dev/null @@ -1,621 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (103) -+- CometTakeOrderedAndProject (102) - +- CometHashAggregate (101) - +- CometColumnarExchange (100) - +- * HashAggregate (99) - +- Union (98) - :- * HashAggregate (83) - : +- * CometColumnarToRow (82) - : +- CometColumnarExchange (81) - : +- * HashAggregate (80) - : +- Union (79) - : :- * CometColumnarToRow (31) - : : +- CometProject (30) - : : +- CometBroadcastHashJoin (29) - : : :- CometHashAggregate (16) - : : : +- CometExchange (15) - : : : +- CometHashAggregate (14) - : : : +- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometProject (5) - : : : : +- CometFilter (4) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : : +- CometBroadcastExchange (28) - : : +- CometHashAggregate (27) - : : +- CometExchange (26) - : : +- CometHashAggregate (25) - : : +- CometProject (24) - : : +- CometBroadcastHashJoin (23) - : : :- CometProject (21) - : : : +- CometBroadcastHashJoin (20) - : : : :- CometFilter (18) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (17) - : : : +- ReusedExchange (19) - : : +- ReusedExchange (22) - : :- * Project (50) - : : +- * BroadcastNestedLoopJoin Inner BuildLeft (49) - : : :- BroadcastExchange (40) - : : : +- * CometColumnarToRow (39) - : : : +- CometHashAggregate (38) - : : : +- CometExchange (37) - : : : +- CometHashAggregate (36) - : : : +- CometProject (35) - : : : +- CometBroadcastHashJoin (34) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (32) - : : : +- ReusedExchange (33) - : : +- * CometColumnarToRow (48) - : : +- CometHashAggregate (47) - : : +- CometExchange (46) - : : +- CometHashAggregate (45) - : : +- CometProject (44) - : : +- CometBroadcastHashJoin (43) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (41) - : : +- ReusedExchange (42) - : +- * CometColumnarToRow (78) - : +- CometProject (77) - : +- CometBroadcastHashJoin (76) - : :- CometHashAggregate (63) - : : +- CometExchange (62) - : : +- CometHashAggregate (61) - : : +- CometProject (60) - : : +- CometBroadcastHashJoin (59) - : : :- CometProject (55) - : : : +- CometBroadcastHashJoin (54) - : : : :- CometFilter (52) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (51) - : : : +- ReusedExchange (53) - : : +- CometBroadcastExchange (58) - : : +- CometFilter (57) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (56) - : +- CometBroadcastExchange (75) - : +- CometHashAggregate (74) - : +- CometExchange (73) - : +- CometHashAggregate (72) - : +- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometProject (68) - : : +- CometBroadcastHashJoin (67) - : : :- CometFilter (65) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (64) - : : +- ReusedExchange (66) - : +- ReusedExchange (69) - :- * HashAggregate (90) - : +- * CometColumnarToRow (89) - : +- CometColumnarExchange (88) - : +- * HashAggregate (87) - : +- * HashAggregate (86) - : +- * CometColumnarToRow (85) - : +- ReusedExchange (84) - +- * HashAggregate (97) - +- * CometColumnarToRow (96) - +- CometColumnarExchange (95) - +- * HashAggregate (94) - +- * HashAggregate (93) - +- * CometColumnarToRow (92) - +- ReusedExchange (91) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] -PushedFilters: [IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) CometFilter -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_store_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) - -(5) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: [d_date_sk#6] - -(7) CometBroadcastHashJoin -Left output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] -Right output [1]: [d_date_sk#6] -Arguments: [ss_sold_date_sk#4], [d_date_sk#6], Inner, BuildRight - -(8) CometProject -Input [5]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4, d_date_sk#6] -Arguments: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3], [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [1]: [s_store_sk#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(10) CometFilter -Input [1]: [s_store_sk#8] -Condition : isnotnull(s_store_sk#8) - -(11) CometBroadcastExchange -Input [1]: [s_store_sk#8] -Arguments: [s_store_sk#8] - -(12) CometBroadcastHashJoin -Left output [3]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3] -Right output [1]: [s_store_sk#8] -Arguments: [ss_store_sk#1], [s_store_sk#8], Inner, BuildRight - -(13) CometProject -Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Arguments: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8], [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] - -(14) CometHashAggregate -Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#8] -Keys [1]: [s_store_sk#8] -Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] - -(15) CometExchange -Input [3]: [s_store_sk#8, sum#9, sum#10] -Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(16) CometHashAggregate -Input [3]: [s_store_sk#8, sum#9, sum#10] -Keys [1]: [s_store_sk#8] -Functions [2]: [sum(UnscaledValue(ss_ext_sales_price#2)), sum(UnscaledValue(ss_net_profit#3))] - -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] -PushedFilters: [IsNotNull(sr_store_sk)] -ReadSchema: struct - -(18) CometFilter -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Condition : isnotnull(sr_store_sk#11) - -(19) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#16] - -(20) CometBroadcastHashJoin -Left output [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14] -Right output [1]: [d_date_sk#16] -Arguments: [sr_returned_date_sk#14], [d_date_sk#16], Inner, BuildRight - -(21) CometProject -Input [5]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, sr_returned_date_sk#14, d_date_sk#16] -Arguments: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13], [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] - -(22) ReusedExchange [Reuses operator id: 11] -Output [1]: [s_store_sk#17] - -(23) CometBroadcastHashJoin -Left output [3]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13] -Right output [1]: [s_store_sk#17] -Arguments: [sr_store_sk#11], [s_store_sk#17], Inner, BuildRight - -(24) CometProject -Input [4]: [sr_store_sk#11, sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Arguments: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17], [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] - -(25) CometHashAggregate -Input [3]: [sr_return_amt#12, sr_net_loss#13, s_store_sk#17] -Keys [1]: [s_store_sk#17] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#12)), partial_sum(UnscaledValue(sr_net_loss#13))] - -(26) CometExchange -Input [3]: [s_store_sk#17, sum#18, sum#19] -Arguments: hashpartitioning(s_store_sk#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(27) CometHashAggregate -Input [3]: [s_store_sk#17, sum#18, sum#19] -Keys [1]: [s_store_sk#17] -Functions [2]: [sum(UnscaledValue(sr_return_amt#12)), sum(UnscaledValue(sr_net_loss#13))] - -(28) CometBroadcastExchange -Input [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#17, returns#20, profit_loss#21] - -(29) CometBroadcastHashJoin -Left output [3]: [s_store_sk#8, sales#22, profit#23] -Right output [3]: [s_store_sk#17, returns#20, profit_loss#21] -Arguments: [s_store_sk#8], [s_store_sk#17], LeftOuter, BuildRight - -(30) CometProject -Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] -Arguments: [channel#24, id#25, sales#22, returns#26, profit#27], [store channel AS channel#24, s_store_sk#8 AS id#25, sales#22, coalesce(returns#20, 0.00) AS returns#26, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#27] - -(31) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] - -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] -ReadSchema: struct - -(33) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#33] - -(34) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31] -Right output [1]: [d_date_sk#33] -Arguments: [cs_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight - -(35) CometProject -Input [5]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30, cs_sold_date_sk#31, d_date_sk#33] -Arguments: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30], [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] - -(36) CometHashAggregate -Input [3]: [cs_call_center_sk#28, cs_ext_sales_price#29, cs_net_profit#30] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#29)), partial_sum(UnscaledValue(cs_net_profit#30))] - -(37) CometExchange -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Arguments: hashpartitioning(cs_call_center_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [3]: [cs_call_center_sk#28, sum#34, sum#35] -Keys [1]: [cs_call_center_sk#28] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#29)), sum(UnscaledValue(cs_net_profit#30))] - -(39) CometColumnarToRow [codegen id : 2] -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] - -(40) BroadcastExchange -Input [3]: [cs_call_center_sk#28, sales#36, profit#37] -Arguments: IdentityBroadcastMode, [plan_id=4] - -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] -ReadSchema: struct - -(42) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#42] - -(43) CometBroadcastHashJoin -Left output [3]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Right output [1]: [d_date_sk#42] -Arguments: [cr_returned_date_sk#40], [d_date_sk#42], Inner, BuildRight - -(44) CometProject -Input [4]: [cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40, d_date_sk#42] -Arguments: [cr_return_amount#38, cr_net_loss#39], [cr_return_amount#38, cr_net_loss#39] - -(45) CometHashAggregate -Input [2]: [cr_return_amount#38, cr_net_loss#39] -Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] - -(46) CometExchange -Input [2]: [sum#43, sum#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometHashAggregate -Input [2]: [sum#43, sum#44] -Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] - -(48) CometColumnarToRow -Input [2]: [returns#45, profit_loss#46] - -(49) BroadcastNestedLoopJoin [codegen id : 3] -Join type: Inner -Join condition: None - -(50) Project [codegen id : 3] -Output [5]: [catalog channel AS channel#47, cs_call_center_sk#28 AS id#48, sales#36, returns#45, (profit#37 - profit_loss#46) AS profit#49] -Input [5]: [cs_call_center_sk#28, sales#36, profit#37, returns#45, profit_loss#46] - -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] -PushedFilters: [IsNotNull(ws_web_page_sk)] -ReadSchema: struct - -(52) CometFilter -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Condition : isnotnull(ws_web_page_sk#50) - -(53) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#55] - -(54) CometBroadcastHashJoin -Left output [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53] -Right output [1]: [d_date_sk#55] -Arguments: [ws_sold_date_sk#53], [d_date_sk#55], Inner, BuildRight - -(55) CometProject -Input [5]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, ws_sold_date_sk#53, d_date_sk#55] -Arguments: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52], [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] - -(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#56] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_page] -PushedFilters: [IsNotNull(wp_web_page_sk)] -ReadSchema: struct - -(57) CometFilter -Input [1]: [wp_web_page_sk#56] -Condition : isnotnull(wp_web_page_sk#56) - -(58) CometBroadcastExchange -Input [1]: [wp_web_page_sk#56] -Arguments: [wp_web_page_sk#56] - -(59) CometBroadcastHashJoin -Left output [3]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52] -Right output [1]: [wp_web_page_sk#56] -Arguments: [ws_web_page_sk#50], [wp_web_page_sk#56], Inner, BuildRight - -(60) CometProject -Input [4]: [ws_web_page_sk#50, ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Arguments: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56], [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] - -(61) CometHashAggregate -Input [3]: [ws_ext_sales_price#51, ws_net_profit#52, wp_web_page_sk#56] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#51)), partial_sum(UnscaledValue(ws_net_profit#52))] - -(62) CometExchange -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Arguments: hashpartitioning(wp_web_page_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(63) CometHashAggregate -Input [3]: [wp_web_page_sk#56, sum#57, sum#58] -Keys [1]: [wp_web_page_sk#56] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#51)), sum(UnscaledValue(ws_net_profit#52))] - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] -PushedFilters: [IsNotNull(wr_web_page_sk)] -ReadSchema: struct - -(65) CometFilter -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Condition : isnotnull(wr_web_page_sk#59) - -(66) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#64] - -(67) CometBroadcastHashJoin -Left output [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Right output [1]: [d_date_sk#64] -Arguments: [wr_returned_date_sk#62], [d_date_sk#64], Inner, BuildRight - -(68) CometProject -Input [5]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62, d_date_sk#64] -Arguments: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61], [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] - -(69) ReusedExchange [Reuses operator id: 58] -Output [1]: [wp_web_page_sk#65] - -(70) CometBroadcastHashJoin -Left output [3]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61] -Right output [1]: [wp_web_page_sk#65] -Arguments: [wr_web_page_sk#59], [wp_web_page_sk#65], Inner, BuildRight - -(71) CometProject -Input [4]: [wr_web_page_sk#59, wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Arguments: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65], [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] - -(72) CometHashAggregate -Input [3]: [wr_return_amt#60, wr_net_loss#61, wp_web_page_sk#65] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#60)), partial_sum(UnscaledValue(wr_net_loss#61))] - -(73) CometExchange -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Arguments: hashpartitioning(wp_web_page_sk#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(74) CometHashAggregate -Input [3]: [wp_web_page_sk#65, sum#66, sum#67] -Keys [1]: [wp_web_page_sk#65] -Functions [2]: [sum(UnscaledValue(wr_return_amt#60)), sum(UnscaledValue(wr_net_loss#61))] - -(75) CometBroadcastExchange -Input [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#65, returns#68, profit_loss#69] - -(76) CometBroadcastHashJoin -Left output [3]: [wp_web_page_sk#56, sales#70, profit#71] -Right output [3]: [wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [wp_web_page_sk#56], [wp_web_page_sk#65], LeftOuter, BuildRight - -(77) CometProject -Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] -Arguments: [channel#72, id#73, sales#70, returns#74, profit#75], [web channel AS channel#72, wp_web_page_sk#56 AS id#73, sales#70, coalesce(returns#68, 0.00) AS returns#74, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#75] - -(78) CometColumnarToRow [codegen id : 4] -Input [5]: [channel#72, id#73, sales#70, returns#74, profit#75] - -(79) Union - -(80) HashAggregate [codegen id : 5] -Input [5]: [channel#24, id#25, sales#22, returns#26, profit#27] -Keys [2]: [channel#24, id#25] -Functions [3]: [partial_sum(sales#22), partial_sum(returns#26), partial_sum(profit#27)] -Aggregate Attributes [6]: [sum#76, isEmpty#77, sum#78, isEmpty#79, sum#80, isEmpty#81] -Results [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(81) CometColumnarExchange -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Arguments: hashpartitioning(channel#24, id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] - -(82) CometColumnarToRow [codegen id : 6] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(83) HashAggregate [codegen id : 6] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Keys [2]: [channel#24, id#25] -Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] -Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] -Results [5]: [channel#24, id#25, cast(sum(sales#22)#88 as decimal(37,2)) AS sales#91, cast(sum(returns#26)#89 as decimal(37,2)) AS returns#92, cast(sum(profit#27)#90 as decimal(38,2)) AS profit#93] - -(84) ReusedExchange [Reuses operator id: 81] -Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(85) CometColumnarToRow [codegen id : 12] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(86) HashAggregate [codegen id : 12] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Keys [2]: [channel#24, id#25] -Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] -Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] -Results [4]: [channel#24, sum(sales#22)#88 AS sales#94, sum(returns#26)#89 AS returns#95, sum(profit#27)#90 AS profit#96] - -(87) HashAggregate [codegen id : 12] -Input [4]: [channel#24, sales#94, returns#95, profit#96] -Keys [1]: [channel#24] -Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] -Aggregate Attributes [6]: [sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Results [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] - -(88) CometColumnarExchange -Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Arguments: hashpartitioning(channel#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(89) CometColumnarToRow [codegen id : 13] -Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] - -(90) HashAggregate [codegen id : 13] -Input [7]: [channel#24, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Keys [1]: [channel#24] -Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] -Aggregate Attributes [3]: [sum(sales#94)#109, sum(returns#95)#110, sum(profit#96)#111] -Results [5]: [channel#24, null AS id#112, sum(sales#94)#109 AS sales#113, sum(returns#95)#110 AS returns#114, sum(profit#96)#111 AS profit#115] - -(91) ReusedExchange [Reuses operator id: 81] -Output [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(92) CometColumnarToRow [codegen id : 19] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] - -(93) HashAggregate [codegen id : 19] -Input [8]: [channel#24, id#25, sum#82, isEmpty#83, sum#84, isEmpty#85, sum#86, isEmpty#87] -Keys [2]: [channel#24, id#25] -Functions [3]: [sum(sales#22), sum(returns#26), sum(profit#27)] -Aggregate Attributes [3]: [sum(sales#22)#88, sum(returns#26)#89, sum(profit#27)#90] -Results [3]: [sum(sales#22)#88 AS sales#94, sum(returns#26)#89 AS returns#95, sum(profit#27)#90 AS profit#96] - -(94) HashAggregate [codegen id : 19] -Input [3]: [sales#94, returns#95, profit#96] -Keys: [] -Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] -Aggregate Attributes [6]: [sum#116, isEmpty#117, sum#118, isEmpty#119, sum#120, isEmpty#121] -Results [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] - -(95) CometColumnarExchange -Input [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(96) CometColumnarToRow [codegen id : 20] -Input [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] - -(97) HashAggregate [codegen id : 20] -Input [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] -Keys: [] -Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] -Aggregate Attributes [3]: [sum(sales#94)#128, sum(returns#95)#129, sum(profit#96)#130] -Results [5]: [null AS channel#131, null AS id#132, sum(sales#94)#128 AS sales#133, sum(returns#95)#129 AS returns#134, sum(profit#96)#130 AS profit#135] - -(98) Union - -(99) HashAggregate [codegen id : 21] -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#24, id#25, sales#91, returns#92, profit#93] - -(100) CometColumnarExchange -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Arguments: hashpartitioning(channel#24, id#25, sales#91, returns#92, profit#93, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(101) CometHashAggregate -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Keys [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Functions: [] - -(102) CometTakeOrderedAndProject -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#24 ASC NULLS FIRST,id#25 ASC NULLS FIRST], output=[channel#24,id#25,sales#91,returns#92,profit#93]), [channel#24, id#25, sales#91, returns#92, profit#93], 100, 0, [channel#24 ASC NULLS FIRST, id#25 ASC NULLS FIRST], [channel#24, id#25, sales#91, returns#92, profit#93] - -(103) CometColumnarToRow [codegen id : 22] -Input [5]: [channel#24, id#25, sales#91, returns#92, profit#93] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (108) -+- * CometColumnarToRow (107) - +- CometProject (106) - +- CometFilter (105) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) - - -(104) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_date#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(105) CometFilter -Input [2]: [d_date_sk#6, d_date#7] -Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 1998-08-04)) AND (d_date#7 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) - -(106) CometProject -Input [2]: [d_date_sk#6, d_date#7] -Arguments: [d_date_sk#6], [d_date_sk#6] - -(107) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#6] - -(108) BroadcastExchange -Input [1]: [d_date_sk#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -Subquery:2 Hosting operator id = 17 Hosting Expression = sr_returned_date_sk#14 IN dynamicpruning#5 - -Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 - -Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 - -Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 - -Subquery:6 Hosting operator id = 64 Hosting Expression = wr_returned_date_sk#62 IN dynamicpruning#5 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/extended.txt deleted file mode 100644 index b4318d03e2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/extended.txt +++ /dev/null @@ -1,355 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : : :- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : : :- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : :- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - -Comet accelerated 287 out of 332 eligible operators (86%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index ac3d312ee8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/simplified.txt +++ /dev/null @@ -1,141 +0,0 @@ -WholeStageCodegen (22) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (21) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (6) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (5) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [s_store_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] - CometBroadcastHashJoin [s_store_sk,sales,profit,s_store_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [s_store_sk,sales,profit,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit))] - CometExchange [s_store_sk] #3 - CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,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_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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - CometBroadcastExchange [s_store_sk,returns,profit_loss] #7 - CometHashAggregate [sum,sum] [s_store_sk,returns,profit_loss,sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss))] - CometExchange [s_store_sk] #8 - CometHashAggregate [sr_return_amt,sr_net_loss] [s_store_sk,sum,sum] - CometProject [sr_return_amt,sr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [s_store_sk] #6 - WholeStageCodegen (3) - Project [cs_call_center_sk,sales,returns,profit,profit_loss] - BroadcastNestedLoopJoin - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [cs_call_center_sk,sales,profit,sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit))] - CometExchange [cs_call_center_sk] #10 - CometHashAggregate [cs_ext_sales_price,cs_net_profit] [cs_call_center_sk,sum,sum] - CometProject [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] - CometBroadcastHashJoin [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - CometColumnarToRow - InputAdapter - CometHashAggregate [sum,sum] [returns,profit_loss,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] - CometExchange #11 - CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] - CometProject [cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cr_return_amount,cr_net_loss,cr_returned_date_sk,d_date_sk] - CometScan [native_iceberg_compat] 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 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [wp_web_page_sk,returns,profit,profit_loss] [channel,id,sales,returns,profit] - CometBroadcastHashJoin [wp_web_page_sk,sales,profit,wp_web_page_sk,returns,profit_loss] - CometHashAggregate [sum,sum] [wp_web_page_sk,sales,profit,sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit))] - CometExchange [wp_web_page_sk] #12 - CometHashAggregate [ws_ext_sales_price,ws_net_profit] [wp_web_page_sk,sum,sum] - CometProject [ws_ext_sales_price,ws_net_profit,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_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 [native_iceberg_compat] 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 [wp_web_page_sk] #13 - CometFilter [wp_web_page_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - CometBroadcastExchange [wp_web_page_sk,returns,profit_loss] #14 - CometHashAggregate [sum,sum] [wp_web_page_sk,returns,profit_loss,sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss))] - CometExchange [wp_web_page_sk] #15 - CometHashAggregate [wr_return_amt,wr_net_loss] [wp_web_page_sk,sum,sum] - CometProject [wr_return_amt,wr_net_loss,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_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 [native_iceberg_compat] 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 - ReusedExchange [wp_web_page_sk] #13 - WholeStageCodegen (13) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #16 - WholeStageCodegen (12) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (20) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #17 - WholeStageCodegen (19) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/explain.txt deleted file mode 100644 index dbd053cfac..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/explain.txt +++ /dev/null @@ -1,417 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (70) -+- * Project (69) - +- * CometColumnarToRow (68) - +- CometSortMergeJoin (67) - :- CometProject (45) - : +- CometSortMergeJoin (44) - : :- CometSort (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometFilter (12) - : : : +- CometSortMergeJoin (11) - : : : :- CometSort (5) - : : : : +- CometColumnarExchange (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (10) - : : : +- CometExchange (9) - : : : +- CometProject (8) - : : : +- CometFilter (7) - : : : +- CometNativeScan parquet spark_catalog.default.store_returns (6) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometNativeScan parquet spark_catalog.default.date_dim (14) - : +- CometSort (43) - : +- CometFilter (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometFilter (34) - : : +- CometSortMergeJoin (33) - : : :- CometSort (27) - : : : +- CometColumnarExchange (26) - : : : +- * Filter (25) - : : : +- * ColumnarToRow (24) - : : : +- Scan parquet spark_catalog.default.web_sales (23) - : : +- CometSort (32) - : : +- CometExchange (31) - : : +- CometProject (30) - : : +- CometFilter (29) - : : +- CometNativeScan parquet spark_catalog.default.web_returns (28) - : +- ReusedExchange (36) - +- CometSort (66) - +- CometFilter (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometFilter (57) - : +- CometSortMergeJoin (56) - : :- CometSort (50) - : : +- CometColumnarExchange (49) - : : +- * Filter (48) - : : +- * ColumnarToRow (47) - : : +- Scan parquet spark_catalog.default.catalog_sales (46) - : +- CometSort (55) - : +- CometExchange (54) - : +- CometProject (53) - : +- CometFilter (52) - : +- CometNativeScan parquet spark_catalog.default.catalog_returns (51) - +- ReusedExchange (59) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(3) Filter [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_customer_sk#2)) - -(4) CometColumnarExchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_ticket_number#3, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_ticket_number#3 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(7) CometFilter -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) - -(8) CometProject -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] - -(9) CometExchange -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter - -(12) CometFilter -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Condition : isnull(sr_ticket_number#10) - -(13) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(14) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(16) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(17) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight - -(18) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] - -(19) CometHashAggregate -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] - -(20) CometExchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(21) CometHashAggregate -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] - -(22) CometSort -Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] - -(23) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 2] -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(25) Filter [codegen id : 2] -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) - -(26) CometColumnarExchange -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(27) CometSort -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] - -(28) CometNativeScan parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#28, wr_order_number#29, wr_returned_date_sk#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(29) CometFilter -Input [3]: [wr_item_sk#28, wr_order_number#29, wr_returned_date_sk#30] -Condition : (isnotnull(wr_order_number#29) AND isnotnull(wr_item_sk#28)) - -(30) CometProject -Input [3]: [wr_item_sk#28, wr_order_number#29, wr_returned_date_sk#30] -Arguments: [wr_item_sk#28, wr_order_number#29], [wr_item_sk#28, wr_order_number#29] - -(31) CometExchange -Input [2]: [wr_item_sk#28, wr_order_number#29] -Arguments: hashpartitioning(wr_order_number#29, wr_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(32) CometSort -Input [2]: [wr_item_sk#28, wr_order_number#29] -Arguments: [wr_item_sk#28, wr_order_number#29], [wr_order_number#29 ASC NULLS FIRST, wr_item_sk#28 ASC NULLS FIRST] - -(33) CometSortMergeJoin -Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [wr_item_sk#28, wr_order_number#29] -Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#29, wr_item_sk#28], LeftOuter - -(34) CometFilter -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#28, wr_order_number#29] -Condition : isnull(wr_order_number#29) - -(35) CometProject -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#28, wr_order_number#29] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(36) ReusedExchange [Reuses operator id: 16] -Output [2]: [d_date_sk#31, d_year#32] - -(37) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [d_date_sk#31, d_year#32] -Arguments: [ws_sold_date_sk#27], [d_date_sk#31], Inner, BuildRight - -(38) CometProject -Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#31, d_year#32] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#32], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#32] - -(39) CometHashAggregate -Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#32] -Keys [3]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] - -(40) CometExchange -Input [6]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22, sum#33, sum#34, sum#35] -Arguments: hashpartitioning(d_year#32, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(41) CometHashAggregate -Input [6]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22, sum#33, sum#34, sum#35] -Keys [3]: [d_year#32, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] - -(42) CometFilter -Input [6]: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Condition : (coalesce(ws_qty#38, 0) > 0) - -(43) CometSort -Input [6]: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Arguments: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40], [ws_sold_year#36 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#37 ASC NULLS FIRST] - -(44) CometSortMergeJoin -Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Right output [6]: [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37], Inner - -(45) CometProject -Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#36, ws_item_sk#21, ws_customer_sk#37, ws_qty#38, ws_wc#39, ws_sp#40] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40] - -(46) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#47), dynamicpruningexpression(cs_sold_date_sk#47 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(47) ColumnarToRow [codegen id : 3] -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] - -(48) Filter [codegen id : 3] -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Condition : (isnotnull(cs_item_sk#42) AND isnotnull(cs_bill_customer_sk#41)) - -(49) CometColumnarExchange -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Arguments: hashpartitioning(cs_order_number#43, cs_item_sk#42, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(50) CometSort -Input [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Arguments: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47], [cs_order_number#43 ASC NULLS FIRST, cs_item_sk#42 ASC NULLS FIRST] - -(51) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#48, cr_order_number#49, cr_returned_date_sk#50] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(52) CometFilter -Input [3]: [cr_item_sk#48, cr_order_number#49, cr_returned_date_sk#50] -Condition : (isnotnull(cr_order_number#49) AND isnotnull(cr_item_sk#48)) - -(53) CometProject -Input [3]: [cr_item_sk#48, cr_order_number#49, cr_returned_date_sk#50] -Arguments: [cr_item_sk#48, cr_order_number#49], [cr_item_sk#48, cr_order_number#49] - -(54) CometExchange -Input [2]: [cr_item_sk#48, cr_order_number#49] -Arguments: hashpartitioning(cr_order_number#49, cr_item_sk#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(55) CometSort -Input [2]: [cr_item_sk#48, cr_order_number#49] -Arguments: [cr_item_sk#48, cr_order_number#49], [cr_order_number#49 ASC NULLS FIRST, cr_item_sk#48 ASC NULLS FIRST] - -(56) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Right output [2]: [cr_item_sk#48, cr_order_number#49] -Arguments: [cs_order_number#43, cs_item_sk#42], [cr_order_number#49, cr_item_sk#48], LeftOuter - -(57) CometFilter -Input [9]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47, cr_item_sk#48, cr_order_number#49] -Condition : isnull(cr_order_number#49) - -(58) CometProject -Input [9]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47, cr_item_sk#48, cr_order_number#49] -Arguments: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47], [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] - -(59) ReusedExchange [Reuses operator id: 16] -Output [2]: [d_date_sk#51, d_year#52] - -(60) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47] -Right output [2]: [d_date_sk#51, d_year#52] -Arguments: [cs_sold_date_sk#47], [d_date_sk#51], Inner, BuildRight - -(61) CometProject -Input [8]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, cs_sold_date_sk#47, d_date_sk#51, d_year#52] -Arguments: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, d_year#52], [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, d_year#52] - -(62) CometHashAggregate -Input [6]: [cs_bill_customer_sk#41, cs_item_sk#42, cs_quantity#44, cs_wholesale_cost#45, cs_sales_price#46, d_year#52] -Keys [3]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41] -Functions [3]: [partial_sum(cs_quantity#44), partial_sum(UnscaledValue(cs_wholesale_cost#45)), partial_sum(UnscaledValue(cs_sales_price#46))] - -(63) CometExchange -Input [6]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41, sum#53, sum#54, sum#55] -Arguments: hashpartitioning(d_year#52, cs_item_sk#42, cs_bill_customer_sk#41, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(64) CometHashAggregate -Input [6]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41, sum#53, sum#54, sum#55] -Keys [3]: [d_year#52, cs_item_sk#42, cs_bill_customer_sk#41] -Functions [3]: [sum(cs_quantity#44), sum(UnscaledValue(cs_wholesale_cost#45)), sum(UnscaledValue(cs_sales_price#46))] - -(65) CometFilter -Input [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] -Condition : (coalesce(cs_qty#58, 0) > 0) - -(66) CometSort -Input [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] -Arguments: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60], [cs_sold_year#56 ASC NULLS FIRST, cs_item_sk#42 ASC NULLS FIRST, cs_customer_sk#57 ASC NULLS FIRST] - -(67) CometSortMergeJoin -Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40] -Right output [6]: [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57], Inner - -(68) CometColumnarToRow [codegen id : 4] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40, cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] - -(69) Project [codegen id : 4] -Output [13]: [round((cast(ss_qty#18 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#38 + cs_qty#58), 1) as double)))), 2) AS ratio#61, ss_qty#18 AS store_qty#62, ss_wc#19 AS store_wholesale_cost#63, ss_sp#20 AS store_sales_price#64, (coalesce(ws_qty#38, 0) + coalesce(cs_qty#58, 0)) AS other_chan_qty#65, (coalesce(ws_wc#39, 0.00) + coalesce(cs_wc#59, 0.00)) AS other_chan_wholesale_cost#66, (coalesce(ws_sp#40, 0.00) + coalesce(cs_sp#60, 0.00)) AS other_chan_sales_price#67, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#38, ws_wc#39, ws_sp#40, cs_sold_year#56, cs_item_sk#42, cs_customer_sk#57, cs_qty#58, cs_wc#59, cs_sp#60] - -(70) TakeOrderedAndProject -Input [13]: [ratio#61, store_qty#62, store_wholesale_cost#63, store_sales_price#64, other_chan_qty#65, other_chan_wholesale_cost#66, other_chan_sales_price#67, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#65 ASC NULLS FIRST, other_chan_wholesale_cost#66 ASC NULLS FIRST, other_chan_sales_price#67 ASC NULLS FIRST, ratio#61 ASC NULLS FIRST], [ratio#61, store_qty#62, store_wholesale_cost#63, store_sales_price#64, other_chan_qty#65, other_chan_wholesale_cost#66, other_chan_sales_price#67] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (74) -+- * CometColumnarToRow (73) - +- CometFilter (72) - +- CometNativeScan parquet spark_catalog.default.date_dim (71) - - -(71) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(72) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(73) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(74) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 23 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#47 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/simplified.txt deleted file mode 100644 index 4711669981..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/simplified.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (4) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] - CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometColumnarExchange [ss_ticket_number,ss_item_sk] #2 - WholeStageCodegen (1) - Filter [ss_item_sk,ss_customer_sk] - ColumnarToRow - InputAdapter - Scan 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #5 - CometFilter [d_date_sk,d_year] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] - CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometColumnarExchange [ws_order_number,ws_item_sk] #7 - WholeStageCodegen (2) - Filter [ws_item_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_order_number,wr_item_sk] #8 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] - CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometColumnarExchange [cs_order_number,cs_item_sk] #10 - WholeStageCodegen (3) - Filter [cs_item_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_order_number,cr_item_sk] #11 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/explain.txt deleted file mode 100644 index 82a5357f89..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,405 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (67) -+- * Project (66) - +- * CometColumnarToRow (65) - +- CometSortMergeJoin (64) - :- CometProject (43) - : +- CometSortMergeJoin (42) - : :- CometSort (21) - : : +- CometHashAggregate (20) - : : +- CometExchange (19) - : : +- CometHashAggregate (18) - : : +- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : +- CometBroadcastExchange (15) - : : +- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : +- CometSort (41) - : +- CometFilter (40) - : +- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (33) - : : +- CometFilter (32) - : : +- CometSortMergeJoin (31) - : : :- CometSort (25) - : : : +- CometExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (22) - : : +- CometSort (30) - : : +- CometExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (26) - : +- ReusedExchange (34) - +- CometSort (63) - +- CometFilter (62) - +- CometHashAggregate (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometProject (55) - : +- CometFilter (54) - : +- CometSortMergeJoin (53) - : :- CometSort (47) - : : +- CometExchange (46) - : : +- CometFilter (45) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (44) - : +- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometFilter (49) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (48) - +- ReusedExchange (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_customer_sk#2)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_ticket_number#3, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_ticket_number#3 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) - -(7) CometProject -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] - -(8) CometExchange -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter - -(11) CometFilter -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Condition : isnull(sr_ticket_number#10) - -(12) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(15) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(16) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight - -(17) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] - -(18) CometHashAggregate -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] - -(19) CometExchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometHashAggregate -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] - -(21) CometSort -Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(23) CometFilter -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) - -(24) CometExchange -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometSort -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Condition : (isnotnull(wr_order_number#30) AND isnotnull(wr_item_sk#29)) - -(28) CometProject -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_item_sk#29, wr_order_number#30] - -(29) CometExchange -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: hashpartitioning(wr_order_number#30, wr_item_sk#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(30) CometSort -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_order_number#30 ASC NULLS FIRST, wr_item_sk#29 ASC NULLS FIRST] - -(31) CometSortMergeJoin -Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#30, wr_item_sk#29], LeftOuter - -(32) CometFilter -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Condition : isnull(wr_order_number#30) - -(33) CometProject -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(34) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#32, d_year#33] - -(35) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ws_sold_date_sk#27], [d_date_sk#32], Inner, BuildRight - -(36) CometProject -Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#32, d_year#33] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] - -(37) CometHashAggregate -Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] - -(38) CometExchange -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Arguments: hashpartitioning(d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(39) CometHashAggregate -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] - -(40) CometFilter -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Condition : (coalesce(ws_qty#39, 0) > 0) - -(41) CometSort -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41], [ws_sold_year#37 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#38 ASC NULLS FIRST] - -(42) CometSortMergeJoin -Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Right output [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38], Inner - -(43) CometProject -Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#48), dynamicpruningexpression(cs_sold_date_sk#48 IN dynamicpruning#49)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(45) CometFilter -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Condition : (isnotnull(cs_item_sk#43) AND isnotnull(cs_bill_customer_sk#42)) - -(46) CometExchange -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(47) CometSort -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST] - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Condition : (isnotnull(cr_order_number#51) AND isnotnull(cr_item_sk#50)) - -(50) CometProject -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_item_sk#50, cr_order_number#51] - -(51) CometExchange -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(52) CometSort -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST] - -(53) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cs_order_number#44, cs_item_sk#43], [cr_order_number#51, cr_item_sk#50], LeftOuter - -(54) CometFilter -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Condition : isnull(cr_order_number#51) - -(55) CometProject -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] - -(56) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#53, d_year#54] - -(57) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [cs_sold_date_sk#48], [d_date_sk#53], Inner, BuildRight - -(58) CometProject -Input [8]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, d_date_sk#53, d_year#54] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] - -(59) CometHashAggregate -Input [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [partial_sum(cs_quantity#45), partial_sum(UnscaledValue(cs_wholesale_cost#46)), partial_sum(UnscaledValue(cs_sales_price#47))] - -(60) CometExchange -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Arguments: hashpartitioning(d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(61) CometHashAggregate -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [sum(cs_quantity#45), sum(UnscaledValue(cs_wholesale_cost#46)), sum(UnscaledValue(cs_sales_price#47))] - -(62) CometFilter -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Condition : (coalesce(cs_qty#60, 0) > 0) - -(63) CometSort -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62], [cs_sold_year#58 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST, cs_customer_sk#59 ASC NULLS FIRST] - -(64) CometSortMergeJoin -Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] -Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner - -(65) CometColumnarToRow [codegen id : 1] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(66) Project [codegen id : 1] -Output [13]: [round((cast(ss_qty#18 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)))), 2) AS ratio#63, ss_qty#18 AS store_qty#64, ss_wc#19 AS store_wholesale_cost#65, ss_sp#20 AS store_sales_price#66, (coalesce(ws_qty#39, 0) + coalesce(cs_qty#60, 0)) AS other_chan_qty#67, (coalesce(ws_wc#40, 0.00) + coalesce(cs_wc#61, 0.00)) AS other_chan_wholesale_cost#68, (coalesce(ws_sp#41, 0.00) + coalesce(cs_sp#62, 0.00)) AS other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(67) TakeOrderedAndProject -Input [13]: [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#67 ASC NULLS FIRST, other_chan_wholesale_cost#68 ASC NULLS FIRST, other_chan_sales_price#69 ASC NULLS FIRST, ratio#63 ASC NULLS FIRST], [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) - - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(70) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(71) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/simplified.txt deleted file mode 100644 index 273db28e77..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,78 +0,0 @@ -TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (1) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] - CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #5 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] - CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometExchange [ws_order_number,ws_item_sk] #7 - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_order_number,wr_item_sk] #8 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] - CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometExchange [cs_order_number,cs_item_sk] #10 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_order_number,cr_item_sk] #11 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt deleted file mode 100644 index 82a5357f89..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt +++ /dev/null @@ -1,405 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (67) -+- * Project (66) - +- * CometColumnarToRow (65) - +- CometSortMergeJoin (64) - :- CometProject (43) - : +- CometSortMergeJoin (42) - : :- CometSort (21) - : : +- CometHashAggregate (20) - : : +- CometExchange (19) - : : +- CometHashAggregate (18) - : : +- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometSortMergeJoin (10) - : : : :- CometSort (4) - : : : : +- CometExchange (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometProject (7) - : : : +- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : +- CometBroadcastExchange (15) - : : +- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (13) - : +- CometSort (41) - : +- CometFilter (40) - : +- CometHashAggregate (39) - : +- CometExchange (38) - : +- CometHashAggregate (37) - : +- CometProject (36) - : +- CometBroadcastHashJoin (35) - : :- CometProject (33) - : : +- CometFilter (32) - : : +- CometSortMergeJoin (31) - : : :- CometSort (25) - : : : +- CometExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (22) - : : +- CometSort (30) - : : +- CometExchange (29) - : : +- CometProject (28) - : : +- CometFilter (27) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (26) - : +- ReusedExchange (34) - +- CometSort (63) - +- CometFilter (62) - +- CometHashAggregate (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometProject (58) - +- CometBroadcastHashJoin (57) - :- CometProject (55) - : +- CometFilter (54) - : +- CometSortMergeJoin (53) - : :- CometSort (47) - : : +- CometExchange (46) - : : +- CometFilter (45) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (44) - : +- CometSort (52) - : +- CometExchange (51) - : +- CometProject (50) - : +- CometFilter (49) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (48) - +- ReusedExchange (56) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_customer_sk#2)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_ticket_number#3, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_ticket_number#3 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) - -(7) CometProject -Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] - -(8) CometExchange -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_ticket_number#3, ss_item_sk#1], [sr_ticket_number#10, sr_item_sk#9], LeftOuter - -(11) CometFilter -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Condition : isnull(sr_ticket_number#10) - -(12) CometProject -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(15) CometBroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: [d_date_sk#12, d_year#13] - -(16) CometBroadcastHashJoin -Left output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] -Right output [2]: [d_date_sk#12, d_year#13] -Arguments: [ss_sold_date_sk#7], [d_date_sk#12], Inner, BuildRight - -(17) CometProject -Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] -Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13], [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] - -(18) CometHashAggregate -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] - -(19) CometExchange -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(20) CometHashAggregate -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] -Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] -Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] - -(21) CometSort -Input [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20], [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST] - -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] -PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] -ReadSchema: struct - -(23) CometFilter -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Condition : (isnotnull(ws_item_sk#21) AND isnotnull(ws_bill_customer_sk#22)) - -(24) CometExchange -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_order_number#23, ws_item_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(25) CometSort -Input [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_order_number#23 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST] - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] -ReadSchema: struct - -(27) CometFilter -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Condition : (isnotnull(wr_order_number#30) AND isnotnull(wr_item_sk#29)) - -(28) CometProject -Input [3]: [wr_item_sk#29, wr_order_number#30, wr_returned_date_sk#31] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_item_sk#29, wr_order_number#30] - -(29) CometExchange -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: hashpartitioning(wr_order_number#30, wr_item_sk#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(30) CometSort -Input [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [wr_item_sk#29, wr_order_number#30], [wr_order_number#30 ASC NULLS FIRST, wr_item_sk#29 ASC NULLS FIRST] - -(31) CometSortMergeJoin -Left output [7]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [wr_item_sk#29, wr_order_number#30] -Arguments: [ws_order_number#23, ws_item_sk#21], [wr_order_number#30, wr_item_sk#29], LeftOuter - -(32) CometFilter -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Condition : isnull(wr_order_number#30) - -(33) CometProject -Input [9]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_order_number#23, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, wr_item_sk#29, wr_order_number#30] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] - -(34) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#32, d_year#33] - -(35) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27] -Right output [2]: [d_date_sk#32, d_year#33] -Arguments: [ws_sold_date_sk#27], [d_date_sk#32], Inner, BuildRight - -(36) CometProject -Input [8]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, ws_sold_date_sk#27, d_date_sk#32, d_year#33] -Arguments: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33], [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] - -(37) CometHashAggregate -Input [6]: [ws_item_sk#21, ws_bill_customer_sk#22, ws_quantity#24, ws_wholesale_cost#25, ws_sales_price#26, d_year#33] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [partial_sum(ws_quantity#24), partial_sum(UnscaledValue(ws_wholesale_cost#25)), partial_sum(UnscaledValue(ws_sales_price#26))] - -(38) CometExchange -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Arguments: hashpartitioning(d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(39) CometHashAggregate -Input [6]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22, sum#34, sum#35, sum#36] -Keys [3]: [d_year#33, ws_item_sk#21, ws_bill_customer_sk#22] -Functions [3]: [sum(ws_quantity#24), sum(UnscaledValue(ws_wholesale_cost#25)), sum(UnscaledValue(ws_sales_price#26))] - -(40) CometFilter -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Condition : (coalesce(ws_qty#39, 0) > 0) - -(41) CometSort -Input [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41], [ws_sold_year#37 ASC NULLS FIRST, ws_item_sk#21 ASC NULLS FIRST, ws_customer_sk#38 ASC NULLS FIRST] - -(42) CometSortMergeJoin -Left output [6]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Right output [6]: [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38], Inner - -(43) CometProject -Input [12]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_sold_year#37, ws_item_sk#21, ws_customer_sk#38, ws_qty#39, ws_wc#40, ws_sp#41] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41], [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#48), dynamicpruningexpression(cs_sold_date_sk#48 IN dynamicpruning#49)] -PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] -ReadSchema: struct - -(45) CometFilter -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Condition : (isnotnull(cs_item_sk#43) AND isnotnull(cs_bill_customer_sk#42)) - -(46) CometExchange -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(47) CometSort -Input [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST] - -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] -ReadSchema: struct - -(49) CometFilter -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Condition : (isnotnull(cr_order_number#51) AND isnotnull(cr_item_sk#50)) - -(50) CometProject -Input [3]: [cr_item_sk#50, cr_order_number#51, cr_returned_date_sk#52] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_item_sk#50, cr_order_number#51] - -(51) CometExchange -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(52) CometSort -Input [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cr_item_sk#50, cr_order_number#51], [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST] - -(53) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [cr_item_sk#50, cr_order_number#51] -Arguments: [cs_order_number#44, cs_item_sk#43], [cr_order_number#51, cr_item_sk#50], LeftOuter - -(54) CometFilter -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Condition : isnull(cr_order_number#51) - -(55) CometProject -Input [9]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, cr_item_sk#50, cr_order_number#51] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] - -(56) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#53, d_year#54] - -(57) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48] -Right output [2]: [d_date_sk#53, d_year#54] -Arguments: [cs_sold_date_sk#48], [d_date_sk#53], Inner, BuildRight - -(58) CometProject -Input [8]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, cs_sold_date_sk#48, d_date_sk#53, d_year#54] -Arguments: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54], [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] - -(59) CometHashAggregate -Input [6]: [cs_bill_customer_sk#42, cs_item_sk#43, cs_quantity#45, cs_wholesale_cost#46, cs_sales_price#47, d_year#54] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [partial_sum(cs_quantity#45), partial_sum(UnscaledValue(cs_wholesale_cost#46)), partial_sum(UnscaledValue(cs_sales_price#47))] - -(60) CometExchange -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Arguments: hashpartitioning(d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(61) CometHashAggregate -Input [6]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42, sum#55, sum#56, sum#57] -Keys [3]: [d_year#54, cs_item_sk#43, cs_bill_customer_sk#42] -Functions [3]: [sum(cs_quantity#45), sum(UnscaledValue(cs_wholesale_cost#46)), sum(UnscaledValue(cs_sales_price#47))] - -(62) CometFilter -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Condition : (coalesce(cs_qty#60, 0) > 0) - -(63) CometSort -Input [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62], [cs_sold_year#58 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST, cs_customer_sk#59 ASC NULLS FIRST] - -(64) CometSortMergeJoin -Left output [9]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41] -Right output [6]: [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] -Arguments: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2], [cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59], Inner - -(65) CometColumnarToRow [codegen id : 1] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(66) Project [codegen id : 1] -Output [13]: [round((cast(ss_qty#18 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)))), 2) AS ratio#63, ss_qty#18 AS store_qty#64, ss_wc#19 AS store_wholesale_cost#65, ss_sp#20 AS store_sales_price#66, (coalesce(ws_qty#39, 0) + coalesce(cs_qty#60, 0)) AS other_chan_qty#67, (coalesce(ws_wc#40, 0.00) + coalesce(cs_wc#61, 0.00)) AS other_chan_wholesale_cost#68, (coalesce(ws_sp#41, 0.00) + coalesce(cs_sp#62, 0.00)) AS other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Input [15]: [ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, ws_wc#40, ws_sp#41, cs_sold_year#58, cs_item_sk#43, cs_customer_sk#59, cs_qty#60, cs_wc#61, cs_sp#62] - -(67) TakeOrderedAndProject -Input [13]: [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69, ss_sold_year#17, ss_item_sk#1, ss_customer_sk#2, ss_qty#18, ss_wc#19, ss_sp#20] -Arguments: 100, [ss_sold_year#17 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#18 DESC NULLS LAST, ss_wc#19 DESC NULLS LAST, ss_sp#20 DESC NULLS LAST, other_chan_qty#67 ASC NULLS FIRST, other_chan_wholesale_cost#68 ASC NULLS FIRST, other_chan_sales_price#69 ASC NULLS FIRST, ratio#63 ASC NULLS FIRST], [ratio#63, store_qty#64, store_wholesale_cost#65, store_sales_price#66, other_chan_qty#67, other_chan_wholesale_cost#68, other_chan_sales_price#69] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (71) -+- * CometColumnarToRow (70) - +- CometFilter (69) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) - - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#12, d_year#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct - -(69) CometFilter -Input [2]: [d_date_sk#12, d_year#13] -Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) - -(70) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] - -(71) BroadcastExchange -Input [2]: [d_date_sk#12, d_year#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/extended.txt deleted file mode 100644 index 9d5ba53be1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/extended.txt +++ /dev/null @@ -1,80 +0,0 @@ -TakeOrderedAndProject -+- Project [COMET: Comet does not support Spark's BigDecimal rounding] - +- CometColumnarToRow - +- CometSortMergeJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 273db28e77..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/simplified.txt +++ /dev/null @@ -1,78 +0,0 @@ -TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (1) - Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] - CometColumnarToRow - InputAdapter - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp,cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] - CometSortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometSort [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp] - CometHashAggregate [sum,sum,sum] [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,d_year,sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price))] - CometExchange [d_year,ss_item_sk,ss_customer_sk] #1 - CometHashAggregate [ss_quantity,ss_wholesale_cost,ss_sales_price] [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] - CometBroadcastHashJoin [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year] - CometProject [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSortMergeJoin [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk,sr_item_sk,sr_ticket_number] - CometSort [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometExchange [ss_ticket_number,ss_item_sk] #2 - CometFilter [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] 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) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [sr_item_sk,sr_ticket_number] - CometExchange [sr_ticket_number,sr_item_sk] #4 - CometProject [sr_item_sk,sr_ticket_number] - CometFilter [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk,d_year] #5 - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometSort [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometFilter [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp] - CometHashAggregate [sum,sum,sum] [ws_sold_year,ws_item_sk,ws_customer_sk,ws_qty,ws_wc,ws_sp,d_year,ws_bill_customer_sk,sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price))] - CometExchange [d_year,ws_item_sk,ws_bill_customer_sk] #6 - CometHashAggregate [ws_quantity,ws_wholesale_cost,ws_sales_price] [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] - CometBroadcastHashJoin [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,d_date_sk,d_year] - CometProject [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSortMergeJoin [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk,wr_item_sk,wr_order_number] - CometSort [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometExchange [ws_order_number,ws_item_sk] #7 - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number] - CometExchange [wr_order_number,wr_item_sk] #8 - CometProject [wr_item_sk,wr_order_number] - CometFilter [wr_item_sk,wr_order_number,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 - CometSort [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometFilter [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp] - CometHashAggregate [sum,sum,sum] [cs_sold_year,cs_item_sk,cs_customer_sk,cs_qty,cs_wc,cs_sp,d_year,cs_bill_customer_sk,sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price))] - CometExchange [d_year,cs_item_sk,cs_bill_customer_sk] #9 - CometHashAggregate [cs_quantity,cs_wholesale_cost,cs_sales_price] [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] - CometBroadcastHashJoin [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year] - CometProject [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSortMergeJoin [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk,cr_item_sk,cr_order_number] - CometSort [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometExchange [cs_order_number,cs_item_sk] #10 - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number] - CometExchange [cr_order_number,cr_item_sk] #11 - CometProject [cr_item_sk,cr_order_number] - CometFilter [cr_item_sk,cr_order_number,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] - ReusedExchange [d_date_sk,d_year] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/explain.txt deleted file mode 100644 index 9f6f06d6f6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/explain.txt +++ /dev/null @@ -1,695 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (119) -+- CometTakeOrderedAndProject (118) - +- CometHashAggregate (117) - +- CometExchange (116) - +- CometHashAggregate (115) - +- CometUnion (114) - :- CometHashAggregate (103) - : +- CometExchange (102) - : +- CometHashAggregate (101) - : +- CometUnion (100) - : :- CometHashAggregate (39) - : : +- CometExchange (38) - : : +- CometHashAggregate (37) - : : +- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (30) - : : : +- CometBroadcastHashJoin (29) - : : : :- CometProject (24) - : : : : +- CometBroadcastHashJoin (23) - : : : : :- CometProject (18) - : : : : : +- CometBroadcastHashJoin (17) - : : : : : :- CometProject (12) - : : : : : : +- CometSortMergeJoin (11) - : : : : : : :- CometSort (5) - : : : : : : : +- CometColumnarExchange (4) - : : : : : : : +- * Filter (3) - : : : : : : : +- * ColumnarToRow (2) - : : : : : : : +- Scan parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (10) - : : : : : : +- CometExchange (9) - : : : : : : +- CometProject (8) - : : : : : : +- CometFilter (7) - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns (6) - : : : : : +- CometBroadcastExchange (16) - : : : : : +- CometProject (15) - : : : : : +- CometFilter (14) - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim (13) - : : : : +- CometBroadcastExchange (22) - : : : : +- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometNativeScan parquet spark_catalog.default.store (19) - : : : +- CometBroadcastExchange (28) - : : : +- CometProject (27) - : : : +- CometFilter (26) - : : : +- CometNativeScan parquet spark_catalog.default.item (25) - : : +- CometBroadcastExchange (34) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometNativeScan parquet spark_catalog.default.promotion (31) - : :- CometHashAggregate (69) - : : +- CometExchange (68) - : : +- CometHashAggregate (67) - : : +- CometProject (66) - : : +- CometBroadcastHashJoin (65) - : : :- CometProject (63) - : : : +- CometBroadcastHashJoin (62) - : : : :- CometProject (60) - : : : : +- CometBroadcastHashJoin (59) - : : : : :- CometProject (54) - : : : : : +- CometBroadcastHashJoin (53) - : : : : : :- CometProject (51) - : : : : : : +- CometSortMergeJoin (50) - : : : : : : :- CometSort (44) - : : : : : : : +- CometColumnarExchange (43) - : : : : : : : +- * Filter (42) - : : : : : : : +- * ColumnarToRow (41) - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales (40) - : : : : : : +- CometSort (49) - : : : : : : +- CometExchange (48) - : : : : : : +- CometProject (47) - : : : : : : +- CometFilter (46) - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns (45) - : : : : : +- ReusedExchange (52) - : : : : +- CometBroadcastExchange (58) - : : : : +- CometProject (57) - : : : : +- CometFilter (56) - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_page (55) - : : : +- ReusedExchange (61) - : : +- ReusedExchange (64) - : +- CometHashAggregate (99) - : +- CometExchange (98) - : +- CometHashAggregate (97) - : +- CometProject (96) - : +- CometBroadcastHashJoin (95) - : :- CometProject (93) - : : +- CometBroadcastHashJoin (92) - : : :- CometProject (90) - : : : +- CometBroadcastHashJoin (89) - : : : :- CometProject (84) - : : : : +- CometBroadcastHashJoin (83) - : : : : :- CometProject (81) - : : : : : +- CometSortMergeJoin (80) - : : : : : :- CometSort (74) - : : : : : : +- CometColumnarExchange (73) - : : : : : : +- * Filter (72) - : : : : : : +- * ColumnarToRow (71) - : : : : : : +- Scan parquet spark_catalog.default.web_sales (70) - : : : : : +- CometSort (79) - : : : : : +- CometExchange (78) - : : : : : +- CometProject (77) - : : : : : +- CometFilter (76) - : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns (75) - : : : : +- ReusedExchange (82) - : : : +- CometBroadcastExchange (88) - : : : +- CometProject (87) - : : : +- CometFilter (86) - : : : +- CometNativeScan parquet spark_catalog.default.web_site (85) - : : +- ReusedExchange (91) - : +- ReusedExchange (94) - :- CometHashAggregate (108) - : +- CometExchange (107) - : +- CometHashAggregate (106) - : +- CometHashAggregate (105) - : +- ReusedExchange (104) - +- CometHashAggregate (113) - +- CometExchange (112) - +- CometHashAggregate (111) - +- CometHashAggregate (110) - +- ReusedExchange (109) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] - -(3) Filter [codegen id : 1] -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(4) CometColumnarExchange -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=1] - -(5) CometSort -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST] - -(6) CometNativeScan parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(7) CometFilter -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) - -(8) CometProject -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(9) CometExchange -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(10) CometSort -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] - -(11) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter - -(12) CometProject -Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] - -(13) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(14) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) - -(15) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(16) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(17) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight - -(18) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] - -(19) CometNativeScan parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_store_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(20) CometFilter -Input [2]: [s_store_sk#16, s_store_id#17] -Condition : isnotnull(s_store_sk#16) - -(21) CometProject -Input [2]: [s_store_sk#16, s_store_id#17] -Arguments: [s_store_sk#16, s_store_id#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#17, 16, true, false, true) AS s_store_id#18] - -(22) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_store_id#18] -Arguments: [s_store_sk#16, s_store_id#18] - -(23) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -Right output [2]: [s_store_sk#16, s_store_id#18] -Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight - -(24) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#18] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(25) CometNativeScan parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_current_price#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(26) CometFilter -Input [2]: [i_item_sk#19, i_current_price#20] -Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) - -(27) CometProject -Input [2]: [i_item_sk#19, i_current_price#20] -Arguments: [i_item_sk#19], [i_item_sk#19] - -(28) CometBroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: [i_item_sk#19] - -(29) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [i_item_sk#19] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(30) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, i_item_sk#19] -Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(31) CometNativeScan parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#21, p_channel_tv#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(32) CometFilter -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#22, 1, true, false, true) = N) AND isnotnull(p_promo_sk#21)) - -(33) CometProject -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Arguments: [p_promo_sk#21], [p_promo_sk#21] - -(34) CometBroadcastExchange -Input [1]: [p_promo_sk#21] -Arguments: [p_promo_sk#21] - -(35) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [p_promo_sk#21] -Arguments: [ss_promo_sk#3], [p_promo_sk#21], Inner, BuildRight - -(36) CometProject -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, p_promo_sk#21] -Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(37) CometHashAggregate -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Keys [1]: [s_store_id#18] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(38) CometExchange -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(39) CometHashAggregate -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [s_store_id#18] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(40) Scan parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(41) ColumnarToRow [codegen id : 2] -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] - -(42) Filter [codegen id : 2] -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) - -(43) CometColumnarExchange -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(44) CometSort -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] - -(45) CometNativeScan parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38, cr_returned_date_sk#39] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(46) CometFilter -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38, cr_returned_date_sk#39] -Condition : (isnotnull(cr_item_sk#35) AND isnotnull(cr_order_number#36)) - -(47) CometProject -Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38, cr_returned_date_sk#39] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38], [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] - -(48) CometExchange -Input [4]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: hashpartitioning(cr_item_sk#35, cr_order_number#36, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(49) CometSort -Input [4]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38], [cr_item_sk#35 ASC NULLS FIRST, cr_order_number#36 ASC NULLS FIRST] - -(50) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#35, cr_order_number#36], LeftOuter - -(51) CometProject -Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#35, cr_order_number#36, cr_return_amount#37, cr_net_loss#38] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38] - -(52) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#40] - -(53) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38] -Right output [1]: [d_date_sk#40] -Arguments: [cs_sold_date_sk#34], [d_date_sk#40], Inner, BuildRight - -(54) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#37, cr_net_loss#38, d_date_sk#40] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38] - -(55) CometNativeScan parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#42] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(56) CometFilter -Input [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#42] -Condition : isnotnull(cp_catalog_page_sk#41) - -(57) CometProject -Input [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#42] -Arguments: [cp_catalog_page_sk#41, cp_catalog_page_id#43], [cp_catalog_page_sk#41, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#42, 16, true, false, true) AS cp_catalog_page_id#43] - -(58) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#43] -Arguments: [cp_catalog_page_sk#41, cp_catalog_page_id#43] - -(59) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38] -Right output [2]: [cp_catalog_page_sk#41, cp_catalog_page_id#43] -Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#41], Inner, BuildRight - -(60) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_sk#41, cp_catalog_page_id#43] -Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] - -(61) ReusedExchange [Reuses operator id: 28] -Output [1]: [i_item_sk#44] - -(62) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] -Right output [1]: [i_item_sk#44] -Arguments: [cs_item_sk#29], [i_item_sk#44], Inner, BuildRight - -(63) CometProject -Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43, i_item_sk#44] -Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] - -(64) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#45] - -(65) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] -Right output [1]: [p_promo_sk#45] -Arguments: [cs_promo_sk#30], [p_promo_sk#45], Inner, BuildRight - -(66) CometProject -Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43, p_promo_sk#45] -Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] - -(67) CometHashAggregate -Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#37, cr_net_loss#38, cp_catalog_page_id#43] -Keys [1]: [cp_catalog_page_id#43] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#37 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#38 as decimal(12,2)), 0.00)))] - -(68) CometExchange -Input [6]: [cp_catalog_page_id#43, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] -Arguments: hashpartitioning(cp_catalog_page_id#43, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(69) CometHashAggregate -Input [6]: [cp_catalog_page_id#43, sum#46, sum#47, isEmpty#48, sum#49, isEmpty#50] -Keys [1]: [cp_catalog_page_id#43] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#37 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#38 as decimal(12,2)), 0.00)))] - -(70) Scan parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#57), dynamicpruningexpression(ws_sold_date_sk#57 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] -ReadSchema: struct - -(71) ColumnarToRow [codegen id : 3] -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] - -(72) Filter [codegen id : 3] -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Condition : ((isnotnull(ws_web_site_sk#52) AND isnotnull(ws_item_sk#51)) AND isnotnull(ws_promo_sk#53)) - -(73) CometColumnarExchange -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Arguments: hashpartitioning(ws_item_sk#51, ws_order_number#54, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(74) CometSort -Input [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Arguments: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57], [ws_item_sk#51 ASC NULLS FIRST, ws_order_number#54 ASC NULLS FIRST] - -(75) CometNativeScan parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] -ReadSchema: struct - -(76) CometFilter -Input [5]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Condition : (isnotnull(wr_item_sk#58) AND isnotnull(wr_order_number#59)) - -(77) CometProject -Input [5]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61, wr_returned_date_sk#62] -Arguments: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61], [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] - -(78) CometExchange -Input [4]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: hashpartitioning(wr_item_sk#58, wr_order_number#59, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(79) CometSort -Input [4]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61], [wr_item_sk#58 ASC NULLS FIRST, wr_order_number#59 ASC NULLS FIRST] - -(80) CometSortMergeJoin -Left output [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57] -Right output [4]: [wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: [ws_item_sk#51, ws_order_number#54], [wr_item_sk#58, wr_order_number#59], LeftOuter - -(81) CometProject -Input [11]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_order_number#54, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_item_sk#58, wr_order_number#59, wr_return_amt#60, wr_net_loss#61] -Arguments: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61], [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61] - -(82) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#63] - -(83) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61] -Right output [1]: [d_date_sk#63] -Arguments: [ws_sold_date_sk#57], [d_date_sk#63], Inner, BuildRight - -(84) CometProject -Input [9]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, ws_sold_date_sk#57, wr_return_amt#60, wr_net_loss#61, d_date_sk#63] -Arguments: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61], [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61] - -(85) CometNativeScan parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#64, web_site_id#65] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(86) CometFilter -Input [2]: [web_site_sk#64, web_site_id#65] -Condition : isnotnull(web_site_sk#64) - -(87) CometProject -Input [2]: [web_site_sk#64, web_site_id#65] -Arguments: [web_site_sk#64, web_site_id#66], [web_site_sk#64, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#65, 16, true, false, true) AS web_site_id#66] - -(88) CometBroadcastExchange -Input [2]: [web_site_sk#64, web_site_id#66] -Arguments: [web_site_sk#64, web_site_id#66] - -(89) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61] -Right output [2]: [web_site_sk#64, web_site_id#66] -Arguments: [ws_web_site_sk#52], [web_site_sk#64], Inner, BuildRight - -(90) CometProject -Input [9]: [ws_item_sk#51, ws_web_site_sk#52, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_sk#64, web_site_id#66] -Arguments: [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66], [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] - -(91) ReusedExchange [Reuses operator id: 28] -Output [1]: [i_item_sk#67] - -(92) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] -Right output [1]: [i_item_sk#67] -Arguments: [ws_item_sk#51], [i_item_sk#67], Inner, BuildRight - -(93) CometProject -Input [8]: [ws_item_sk#51, ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66, i_item_sk#67] -Arguments: [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66], [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] - -(94) ReusedExchange [Reuses operator id: 34] -Output [1]: [p_promo_sk#68] - -(95) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] -Right output [1]: [p_promo_sk#68] -Arguments: [ws_promo_sk#53], [p_promo_sk#68], Inner, BuildRight - -(96) CometProject -Input [7]: [ws_promo_sk#53, ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66, p_promo_sk#68] -Arguments: [ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66], [ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] - -(97) CometHashAggregate -Input [5]: [ws_ext_sales_price#55, ws_net_profit#56, wr_return_amt#60, wr_net_loss#61, web_site_id#66] -Keys [1]: [web_site_id#66] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#55)), partial_sum(coalesce(cast(wr_return_amt#60 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#56 - coalesce(cast(wr_net_loss#61 as decimal(12,2)), 0.00)))] - -(98) CometExchange -Input [6]: [web_site_id#66, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Arguments: hashpartitioning(web_site_id#66, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(99) CometHashAggregate -Input [6]: [web_site_id#66, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Keys [1]: [web_site_id#66] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#55)), sum(coalesce(cast(wr_return_amt#60 as decimal(12,2)), 0.00)), sum((ws_net_profit#56 - coalesce(cast(wr_net_loss#61 as decimal(12,2)), 0.00)))] - -(100) CometUnion -Child 0 Input [5]: [channel#74, id#75, sales#76, returns#77, profit#78] -Child 1 Input [5]: [channel#79, id#80, sales#81, returns#82, profit#83] -Child 2 Input [5]: [channel#84, id#85, sales#86, returns#87, profit#88] - -(101) CometHashAggregate -Input [5]: [channel#74, id#75, sales#76, returns#77, profit#78] -Keys [2]: [channel#74, id#75] -Functions [3]: [partial_sum(sales#76), partial_sum(returns#77), partial_sum(profit#78)] - -(102) CometExchange -Input [8]: [channel#74, id#75, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94] -Arguments: hashpartitioning(channel#74, id#75, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(103) CometHashAggregate -Input [8]: [channel#74, id#75, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94] -Keys [2]: [channel#74, id#75] -Functions [3]: [sum(sales#76), sum(returns#77), sum(profit#78)] - -(104) ReusedExchange [Reuses operator id: 102] -Output [8]: [channel#74, id#75, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94] - -(105) CometHashAggregate -Input [8]: [channel#74, id#75, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94] -Keys [2]: [channel#74, id#75] -Functions [3]: [sum(sales#76), sum(returns#77), sum(profit#78)] - -(106) CometHashAggregate -Input [4]: [channel#74, sales#95, returns#96, profit#97] -Keys [1]: [channel#74] -Functions [3]: [partial_sum(sales#95), partial_sum(returns#96), partial_sum(profit#97)] - -(107) CometExchange -Input [7]: [channel#74, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103] -Arguments: hashpartitioning(channel#74, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(108) CometHashAggregate -Input [7]: [channel#74, sum#98, isEmpty#99, sum#100, isEmpty#101, sum#102, isEmpty#103] -Keys [1]: [channel#74] -Functions [3]: [sum(sales#95), sum(returns#96), sum(profit#97)] - -(109) ReusedExchange [Reuses operator id: 102] -Output [8]: [channel#74, id#75, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94] - -(110) CometHashAggregate -Input [8]: [channel#74, id#75, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94] -Keys [2]: [channel#74, id#75] -Functions [3]: [sum(sales#76), sum(returns#77), sum(profit#78)] - -(111) CometHashAggregate -Input [3]: [sales#95, returns#96, profit#97] -Keys: [] -Functions [3]: [partial_sum(sales#95), partial_sum(returns#96), partial_sum(profit#97)] - -(112) CometExchange -Input [6]: [sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(113) CometHashAggregate -Input [6]: [sum#104, isEmpty#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Keys: [] -Functions [3]: [sum(sales#95), sum(returns#96), sum(profit#97)] - -(114) CometUnion -Child 0 Input [5]: [channel#74, id#75, sales#110, returns#111, profit#112] -Child 1 Input [5]: [channel#74, id#113, sales#114, returns#115, profit#116] -Child 2 Input [5]: [channel#117, id#118, sales#119, returns#120, profit#121] - -(115) CometHashAggregate -Input [5]: [channel#74, id#75, sales#110, returns#111, profit#112] -Keys [5]: [channel#74, id#75, sales#110, returns#111, profit#112] -Functions: [] - -(116) CometExchange -Input [5]: [channel#74, id#75, sales#110, returns#111, profit#112] -Arguments: hashpartitioning(channel#74, id#75, sales#110, returns#111, profit#112, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] - -(117) CometHashAggregate -Input [5]: [channel#74, id#75, sales#110, returns#111, profit#112] -Keys [5]: [channel#74, id#75, sales#110, returns#111, profit#112] -Functions: [] - -(118) CometTakeOrderedAndProject -Input [5]: [channel#74, id#75, sales#110, returns#111, profit#112] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#74 ASC NULLS FIRST,id#75 ASC NULLS FIRST], output=[channel#74,id#75,sales#110,returns#111,profit#112]), [channel#74, id#75, sales#110, returns#111, profit#112], 100, 0, [channel#74 ASC NULLS FIRST, id#75 ASC NULLS FIRST], [channel#74, id#75, sales#110, returns#111, profit#112] - -(119) CometColumnarToRow [codegen id : 10] -Input [5]: [channel#74, id#75, sales#110, returns#111, profit#112] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (124) -+- * CometColumnarToRow (123) - +- CometProject (122) - +- CometFilter (121) - +- CometNativeScan parquet spark_catalog.default.date_dim (120) - - -(120) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(121) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) - -(122) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(123) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(124) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#57 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/simplified.txt deleted file mode 100644 index cfce9e20a1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/simplified.txt +++ /dev/null @@ -1,137 +0,0 @@ -WholeStageCodegen (10) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id] #2 - CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [s_store_id] #3 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometColumnarExchange [ss_item_sk,ss_ticket_number] #4 - WholeStageCodegen (1) - Filter [ss_store_sk,ss_item_sk,ss_promo_sk] - ColumnarToRow - InputAdapter - Scan 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #6 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #8 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometNativeScan parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #9 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #10 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometNativeScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,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)))] - CometExchange [cp_catalog_page_id] #11 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometColumnarExchange [cs_item_sk,cs_order_number] #12 - WholeStageCodegen (2) - Filter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometNativeScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [web_site_id] #15 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometColumnarExchange [ws_item_sk,ws_order_number] #16 - WholeStageCodegen (3) - Filter [ws_web_site_sk,ws_item_sk,ws_promo_sk] - ColumnarToRow - InputAdapter - Scan 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 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #17 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometNativeScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [web_site_sk,web_site_id] #18 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometNativeScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel] #19 - CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange #20 - CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/explain.txt deleted file mode 100644 index 6898d458e7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,683 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (116) -+- CometTakeOrderedAndProject (115) - +- CometHashAggregate (114) - +- CometExchange (113) - +- CometHashAggregate (112) - +- CometUnion (111) - :- CometHashAggregate (100) - : +- CometExchange (99) - : +- CometHashAggregate (98) - : +- CometUnion (97) - : :- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometProject (29) - : : : +- CometBroadcastHashJoin (28) - : : : :- CometProject (23) - : : : : +- CometBroadcastHashJoin (22) - : : : : :- CometProject (17) - : : : : : +- CometBroadcastHashJoin (16) - : : : : : :- CometProject (11) - : : : : : : +- CometSortMergeJoin (10) - : : : : : : :- CometSort (4) - : : : : : : : +- CometExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (9) - : : : : : : +- CometExchange (8) - : : : : : : +- CometProject (7) - : : : : : : +- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : : +- CometBroadcastExchange (15) - : : : : : +- CometProject (14) - : : : : : +- CometFilter (13) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) - : : : : +- CometBroadcastExchange (21) - : : : : +- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) - : : : +- CometBroadcastExchange (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - : :- CometHashAggregate (67) - : : +- CometExchange (66) - : : +- CometHashAggregate (65) - : : +- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometProject (61) - : : : +- CometBroadcastHashJoin (60) - : : : :- CometProject (58) - : : : : +- CometBroadcastHashJoin (57) - : : : : :- CometProject (52) - : : : : : +- CometBroadcastHashJoin (51) - : : : : : :- CometProject (49) - : : : : : : +- CometSortMergeJoin (48) - : : : : : : :- CometSort (42) - : : : : : : : +- CometExchange (41) - : : : : : : : +- CometFilter (40) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) - : : : : : : +- CometSort (47) - : : : : : : +- CometExchange (46) - : : : : : : +- CometProject (45) - : : : : : : +- CometFilter (44) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) - : : : : : +- ReusedExchange (50) - : : : : +- CometBroadcastExchange (56) - : : : : +- CometProject (55) - : : : : +- CometFilter (54) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) - : : : +- ReusedExchange (59) - : : +- ReusedExchange (62) - : +- CometHashAggregate (96) - : +- CometExchange (95) - : +- CometHashAggregate (94) - : +- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (87) - : : : +- CometBroadcastHashJoin (86) - : : : :- CometProject (81) - : : : : +- CometBroadcastHashJoin (80) - : : : : :- CometProject (78) - : : : : : +- CometSortMergeJoin (77) - : : : : : :- CometSort (71) - : : : : : : +- CometExchange (70) - : : : : : : +- CometFilter (69) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) - : : : : : +- CometSort (76) - : : : : : +- CometExchange (75) - : : : : : +- CometProject (74) - : : : : : +- CometFilter (73) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) - : : : : +- ReusedExchange (79) - : : : +- CometBroadcastExchange (85) - : : : +- CometProject (84) - : : : +- CometFilter (83) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) - : : +- ReusedExchange (88) - : +- ReusedExchange (91) - :- CometHashAggregate (105) - : +- CometExchange (104) - : +- CometHashAggregate (103) - : +- CometHashAggregate (102) - : +- ReusedExchange (101) - +- CometHashAggregate (110) - +- CometExchange (109) - +- CometHashAggregate (108) - +- CometHashAggregate (107) - +- ReusedExchange (106) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) - -(7) CometProject -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(8) CometExchange -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter - -(11) CometProject -Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) - -(14) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(15) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(16) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight - -(17) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_store_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [s_store_sk#16, s_store_id#17] -Condition : isnotnull(s_store_sk#16) - -(20) CometProject -Input [2]: [s_store_sk#16, s_store_id#17] -Arguments: [s_store_sk#16, s_store_id#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#17, 16, true, false, true) AS s_store_id#18] - -(21) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_store_id#18] -Arguments: [s_store_sk#16, s_store_id#18] - -(22) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -Right output [2]: [s_store_sk#16, s_store_id#18] -Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight - -(23) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#18] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_current_price#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [i_item_sk#19, i_current_price#20] -Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) - -(26) CometProject -Input [2]: [i_item_sk#19, i_current_price#20] -Arguments: [i_item_sk#19], [i_item_sk#19] - -(27) CometBroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: [i_item_sk#19] - -(28) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [i_item_sk#19] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(29) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, i_item_sk#19] -Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#21, p_channel_tv#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#22, 1, true, false, true) = N) AND isnotnull(p_promo_sk#21)) - -(32) CometProject -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Arguments: [p_promo_sk#21], [p_promo_sk#21] - -(33) CometBroadcastExchange -Input [1]: [p_promo_sk#21] -Arguments: [p_promo_sk#21] - -(34) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [p_promo_sk#21] -Arguments: [ss_promo_sk#3], [p_promo_sk#21], Inner, BuildRight - -(35) CometProject -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, p_promo_sk#21] -Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(36) CometHashAggregate -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Keys [1]: [s_store_id#18] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(37) CometExchange -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [s_store_id#18] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(40) CometFilter -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) - -(41) CometExchange -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(42) CometSort -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(44) CometFilter -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) - -(45) CometProject -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] - -(46) CometExchange -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometSort -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] - -(48) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter - -(49) CometProject -Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] - -(50) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#41] - -(51) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -Right output [1]: [d_date_sk#41] -Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight - -(52) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(54) CometFilter -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Condition : isnotnull(cp_catalog_page_sk#42) - -(55) CometProject -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#43, 16, true, false, true) AS cp_catalog_page_id#44] - -(56) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] - -(57) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight - -(58) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(59) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#45] - -(60) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [i_item_sk#45] -Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight - -(61) CometProject -Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] -Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(62) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#46] - -(63) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [p_promo_sk#46] -Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight - -(64) CometProject -Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] -Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(65) CometHashAggregate -Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(66) CometExchange -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(67) CometHashAggregate -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] -ReadSchema: struct - -(69) CometFilter -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) - -(70) CometExchange -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(71) CometSort -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] -ReadSchema: struct - -(73) CometFilter -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) - -(74) CometProject -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] - -(75) CometExchange -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(76) CometSort -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] - -(77) CometSortMergeJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter - -(78) CometProject -Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] - -(79) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#65] - -(80) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -Right output [1]: [d_date_sk#65] -Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(81) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] - -(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#66, web_site_id#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(83) CometFilter -Input [2]: [web_site_sk#66, web_site_id#67] -Condition : isnotnull(web_site_sk#66) - -(84) CometProject -Input [2]: [web_site_sk#66, web_site_id#67] -Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#67, 16, true, false, true) AS web_site_id#68] - -(85) CometBroadcastExchange -Input [2]: [web_site_sk#66, web_site_id#68] -Arguments: [web_site_sk#66, web_site_id#68] - -(86) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -Right output [2]: [web_site_sk#66, web_site_id#68] -Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight - -(87) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] -Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(88) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#69] - -(89) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [i_item_sk#69] -Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight - -(90) CometProject -Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] -Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(91) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#70] - -(92) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [p_promo_sk#70] -Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight - -(93) CometProject -Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] -Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(94) CometHashAggregate -Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Keys [1]: [web_site_id#68] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(95) CometExchange -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(96) CometHashAggregate -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Keys [1]: [web_site_id#68] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(97) CometUnion -Child 0 Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] -Child 1 Input [5]: [channel#81, id#82, sales#83, returns#84, profit#85] -Child 2 Input [5]: [channel#86, id#87, sales#88, returns#89, profit#90] - -(98) CometHashAggregate -Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] -Keys [2]: [channel#76, id#77] -Functions [3]: [partial_sum(sales#78), partial_sum(returns#79), partial_sum(profit#80)] - -(99) CometExchange -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Arguments: hashpartitioning(channel#76, id#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(100) CometHashAggregate -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Keys [2]: [channel#76, id#77] -Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] - -(101) ReusedExchange [Reuses operator id: 99] -Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] - -(102) CometHashAggregate -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Keys [2]: [channel#76, id#77] -Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] - -(103) CometHashAggregate -Input [4]: [channel#76, sales#97, returns#98, profit#99] -Keys [1]: [channel#76] -Functions [3]: [partial_sum(sales#97), partial_sum(returns#98), partial_sum(profit#99)] - -(104) CometExchange -Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Arguments: hashpartitioning(channel#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(105) CometHashAggregate -Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Keys [1]: [channel#76] -Functions [3]: [sum(sales#97), sum(returns#98), sum(profit#99)] - -(106) ReusedExchange [Reuses operator id: 99] -Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] - -(107) CometHashAggregate -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Keys [2]: [channel#76, id#77] -Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] - -(108) CometHashAggregate -Input [3]: [sales#97, returns#98, profit#99] -Keys: [] -Functions [3]: [partial_sum(sales#97), partial_sum(returns#98), partial_sum(profit#99)] - -(109) CometExchange -Input [6]: [sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(110) CometHashAggregate -Input [6]: [sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Keys: [] -Functions [3]: [sum(sales#97), sum(returns#98), sum(profit#99)] - -(111) CometUnion -Child 0 Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] -Child 1 Input [5]: [channel#76, id#115, sales#116, returns#117, profit#118] -Child 2 Input [5]: [channel#119, id#120, sales#121, returns#122, profit#123] - -(112) CometHashAggregate -Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] -Keys [5]: [channel#76, id#77, sales#112, returns#113, profit#114] -Functions: [] - -(113) CometExchange -Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] -Arguments: hashpartitioning(channel#76, id#77, sales#112, returns#113, profit#114, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] - -(114) CometHashAggregate -Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] -Keys [5]: [channel#76, id#77, sales#112, returns#113, profit#114] -Functions: [] - -(115) CometTakeOrderedAndProject -Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#76 ASC NULLS FIRST,id#77 ASC NULLS FIRST], output=[channel#76,id#77,sales#112,returns#113,profit#114]), [channel#76, id#77, sales#112, returns#113, profit#114], 100, 0, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#112, returns#113, profit#114] - -(116) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) - -(119) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(121) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/simplified.txt deleted file mode 100644 index d27ab827e1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,128 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id] #2 - CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [s_store_id] #3 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #4 - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #6 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #8 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #9 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #10 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,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)))] - CometExchange [cp_catalog_page_id] #11 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #12 - 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 [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [web_site_id] #15 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #16 - 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 [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #17 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [web_site_sk,web_site_id] #18 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel] #19 - CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange #20 - CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt deleted file mode 100644 index 6898d458e7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt +++ /dev/null @@ -1,683 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (116) -+- CometTakeOrderedAndProject (115) - +- CometHashAggregate (114) - +- CometExchange (113) - +- CometHashAggregate (112) - +- CometUnion (111) - :- CometHashAggregate (100) - : +- CometExchange (99) - : +- CometHashAggregate (98) - : +- CometUnion (97) - : :- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometProject (29) - : : : +- CometBroadcastHashJoin (28) - : : : :- CometProject (23) - : : : : +- CometBroadcastHashJoin (22) - : : : : :- CometProject (17) - : : : : : +- CometBroadcastHashJoin (16) - : : : : : :- CometProject (11) - : : : : : : +- CometSortMergeJoin (10) - : : : : : : :- CometSort (4) - : : : : : : : +- CometExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (9) - : : : : : : +- CometExchange (8) - : : : : : : +- CometProject (7) - : : : : : : +- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : : +- CometBroadcastExchange (15) - : : : : : +- CometProject (14) - : : : : : +- CometFilter (13) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) - : : : : +- CometBroadcastExchange (21) - : : : : +- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) - : : : +- CometBroadcastExchange (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - : :- CometHashAggregate (67) - : : +- CometExchange (66) - : : +- CometHashAggregate (65) - : : +- CometProject (64) - : : +- CometBroadcastHashJoin (63) - : : :- CometProject (61) - : : : +- CometBroadcastHashJoin (60) - : : : :- CometProject (58) - : : : : +- CometBroadcastHashJoin (57) - : : : : :- CometProject (52) - : : : : : +- CometBroadcastHashJoin (51) - : : : : : :- CometProject (49) - : : : : : : +- CometSortMergeJoin (48) - : : : : : : :- CometSort (42) - : : : : : : : +- CometExchange (41) - : : : : : : : +- CometFilter (40) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) - : : : : : : +- CometSort (47) - : : : : : : +- CometExchange (46) - : : : : : : +- CometProject (45) - : : : : : : +- CometFilter (44) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) - : : : : : +- ReusedExchange (50) - : : : : +- CometBroadcastExchange (56) - : : : : +- CometProject (55) - : : : : +- CometFilter (54) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) - : : : +- ReusedExchange (59) - : : +- ReusedExchange (62) - : +- CometHashAggregate (96) - : +- CometExchange (95) - : +- CometHashAggregate (94) - : +- CometProject (93) - : +- CometBroadcastHashJoin (92) - : :- CometProject (90) - : : +- CometBroadcastHashJoin (89) - : : :- CometProject (87) - : : : +- CometBroadcastHashJoin (86) - : : : :- CometProject (81) - : : : : +- CometBroadcastHashJoin (80) - : : : : :- CometProject (78) - : : : : : +- CometSortMergeJoin (77) - : : : : : :- CometSort (71) - : : : : : : +- CometExchange (70) - : : : : : : +- CometFilter (69) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) - : : : : : +- CometSort (76) - : : : : : +- CometExchange (75) - : : : : : +- CometProject (74) - : : : : : +- CometFilter (73) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) - : : : : +- ReusedExchange (79) - : : : +- CometBroadcastExchange (85) - : : : +- CometProject (84) - : : : +- CometFilter (83) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) - : : +- ReusedExchange (88) - : +- ReusedExchange (91) - :- CometHashAggregate (105) - : +- CometExchange (104) - : +- CometHashAggregate (103) - : +- CometHashAggregate (102) - : +- ReusedExchange (101) - +- CometHashAggregate (110) - +- CometExchange (109) - +- CometHashAggregate (108) - +- CometHashAggregate (107) - +- ReusedExchange (106) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] -PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Condition : ((isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) - -(3) CometExchange -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(4) CometSort -Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7], [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST] - -(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store_returns] -PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] -ReadSchema: struct - -(6) CometFilter -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) - -(7) CometProject -Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] - -(8) CometExchange -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(9) CometSort -Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] -Right output [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_ticket_number#4], [sr_item_sk#9, sr_ticket_number#10], LeftOuter - -(11) CometProject -Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] - -(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(13) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) - -(14) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(15) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(16) CometBroadcastHashJoin -Left output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#7], [d_date_sk#14], Inner, BuildRight - -(17) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] -Arguments: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12], [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] - -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#16, s_store_id#17] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct - -(19) CometFilter -Input [2]: [s_store_sk#16, s_store_id#17] -Condition : isnotnull(s_store_sk#16) - -(20) CometProject -Input [2]: [s_store_sk#16, s_store_id#17] -Arguments: [s_store_sk#16, s_store_id#18], [s_store_sk#16, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_store_id#17, 16, true, false, true) AS s_store_id#18] - -(21) CometBroadcastExchange -Input [2]: [s_store_sk#16, s_store_id#18] -Arguments: [s_store_sk#16, s_store_id#18] - -(22) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] -Right output [2]: [s_store_sk#16, s_store_id#18] -Arguments: [ss_store_sk#2], [s_store_sk#16], Inner, BuildRight - -(23) CometProject -Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#16, s_store_id#18] -Arguments: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [2]: [i_item_sk#19, i_current_price#20] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] -ReadSchema: struct - -(25) CometFilter -Input [2]: [i_item_sk#19, i_current_price#20] -Condition : ((isnotnull(i_current_price#20) AND (i_current_price#20 > 50.00)) AND isnotnull(i_item_sk#19)) - -(26) CometProject -Input [2]: [i_item_sk#19, i_current_price#20] -Arguments: [i_item_sk#19], [i_item_sk#19] - -(27) CometBroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: [i_item_sk#19] - -(28) CometBroadcastHashJoin -Left output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [i_item_sk#19] -Arguments: [ss_item_sk#1], [i_item_sk#19], Inner, BuildRight - -(29) CometProject -Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, i_item_sk#19] -Arguments: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [2]: [p_promo_sk#21, p_channel_tv#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/promotion] -PushedFilters: [IsNotNull(p_promo_sk)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Condition : ((staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, p_channel_tv#22, 1, true, false, true) = N) AND isnotnull(p_promo_sk#21)) - -(32) CometProject -Input [2]: [p_promo_sk#21, p_channel_tv#22] -Arguments: [p_promo_sk#21], [p_promo_sk#21] - -(33) CometBroadcastExchange -Input [1]: [p_promo_sk#21] -Arguments: [p_promo_sk#21] - -(34) CometBroadcastHashJoin -Left output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Right output [1]: [p_promo_sk#21] -Arguments: [ss_promo_sk#3], [p_promo_sk#21], Inner, BuildRight - -(35) CometProject -Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18, p_promo_sk#21] -Arguments: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18], [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] - -(36) CometHashAggregate -Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#18] -Keys [1]: [s_store_id#18] -Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(37) CometExchange -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(38) CometHashAggregate -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] -Keys [1]: [s_store_id#18] -Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] -PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] -ReadSchema: struct - -(40) CometFilter -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) - -(41) CometExchange -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(42) CometSort -Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] - -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] -ReadSchema: struct - -(44) CometFilter -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) - -(45) CometProject -Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] - -(46) CometExchange -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(47) CometSort -Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] - -(48) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] -Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter - -(49) CometProject -Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] - -(50) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#41] - -(51) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -Right output [1]: [d_date_sk#41] -Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight - -(52) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] -Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] - -(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_page] -PushedFilters: [IsNotNull(cp_catalog_page_sk)] -ReadSchema: struct - -(54) CometFilter -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Condition : isnotnull(cp_catalog_page_sk#42) - -(55) CometProject -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#43, 16, true, false, true) AS cp_catalog_page_id#44] - -(56) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] - -(57) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight - -(58) CometProject -Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] -Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(59) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#45] - -(60) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [i_item_sk#45] -Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight - -(61) CometProject -Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] -Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(62) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#46] - -(63) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Right output [1]: [p_promo_sk#46] -Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight - -(64) CometProject -Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] -Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] - -(65) CometHashAggregate -Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(66) CometExchange -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(67) CometHashAggregate -Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] -Keys [1]: [cp_catalog_page_id#44] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] - -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] -PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] -ReadSchema: struct - -(69) CometFilter -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) - -(70) CometExchange -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(71) CometSort -Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] - -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_returns] -PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] -ReadSchema: struct - -(73) CometFilter -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) - -(74) CometProject -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] - -(75) CometExchange -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(76) CometSort -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] - -(77) CometSortMergeJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] -Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter - -(78) CometProject -Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] - -(79) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#65] - -(80) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -Right output [1]: [d_date_sk#65] -Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight - -(81) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] -Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] - -(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#66, web_site_id#67] -Batched: true -Location [not included in comparison]/{warehouse_dir}/web_site] -PushedFilters: [IsNotNull(web_site_sk)] -ReadSchema: struct - -(83) CometFilter -Input [2]: [web_site_sk#66, web_site_id#67] -Condition : isnotnull(web_site_sk#66) - -(84) CometProject -Input [2]: [web_site_sk#66, web_site_id#67] -Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#67, 16, true, false, true) AS web_site_id#68] - -(85) CometBroadcastExchange -Input [2]: [web_site_sk#66, web_site_id#68] -Arguments: [web_site_sk#66, web_site_id#68] - -(86) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -Right output [2]: [web_site_sk#66, web_site_id#68] -Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight - -(87) CometProject -Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] -Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(88) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#69] - -(89) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [i_item_sk#69] -Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight - -(90) CometProject -Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] -Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(91) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#70] - -(92) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Right output [1]: [p_promo_sk#70] -Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight - -(93) CometProject -Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] -Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] - -(94) CometHashAggregate -Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -Keys [1]: [web_site_id#68] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(95) CometExchange -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(96) CometHashAggregate -Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] -Keys [1]: [web_site_id#68] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] - -(97) CometUnion -Child 0 Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] -Child 1 Input [5]: [channel#81, id#82, sales#83, returns#84, profit#85] -Child 2 Input [5]: [channel#86, id#87, sales#88, returns#89, profit#90] - -(98) CometHashAggregate -Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] -Keys [2]: [channel#76, id#77] -Functions [3]: [partial_sum(sales#78), partial_sum(returns#79), partial_sum(profit#80)] - -(99) CometExchange -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Arguments: hashpartitioning(channel#76, id#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] - -(100) CometHashAggregate -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Keys [2]: [channel#76, id#77] -Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] - -(101) ReusedExchange [Reuses operator id: 99] -Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] - -(102) CometHashAggregate -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Keys [2]: [channel#76, id#77] -Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] - -(103) CometHashAggregate -Input [4]: [channel#76, sales#97, returns#98, profit#99] -Keys [1]: [channel#76] -Functions [3]: [partial_sum(sales#97), partial_sum(returns#98), partial_sum(profit#99)] - -(104) CometExchange -Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Arguments: hashpartitioning(channel#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(105) CometHashAggregate -Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Keys [1]: [channel#76] -Functions [3]: [sum(sales#97), sum(returns#98), sum(profit#99)] - -(106) ReusedExchange [Reuses operator id: 99] -Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] - -(107) CometHashAggregate -Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Keys [2]: [channel#76, id#77] -Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] - -(108) CometHashAggregate -Input [3]: [sales#97, returns#98, profit#99] -Keys: [] -Functions [3]: [partial_sum(sales#97), partial_sum(returns#98), partial_sum(profit#99)] - -(109) CometExchange -Input [6]: [sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] - -(110) CometHashAggregate -Input [6]: [sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Keys: [] -Functions [3]: [sum(sales#97), sum(returns#98), sum(profit#99)] - -(111) CometUnion -Child 0 Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] -Child 1 Input [5]: [channel#76, id#115, sales#116, returns#117, profit#118] -Child 2 Input [5]: [channel#119, id#120, sales#121, returns#122, profit#123] - -(112) CometHashAggregate -Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] -Keys [5]: [channel#76, id#77, sales#112, returns#113, profit#114] -Functions: [] - -(113) CometExchange -Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] -Arguments: hashpartitioning(channel#76, id#77, sales#112, returns#113, profit#114, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] - -(114) CometHashAggregate -Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] -Keys [5]: [channel#76, id#77, sales#112, returns#113, profit#114] -Functions: [] - -(115) CometTakeOrderedAndProject -Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#76 ASC NULLS FIRST,id#77 ASC NULLS FIRST], output=[channel#76,id#77,sales#112,returns#113,profit#114]), [channel#76, id#77, sales#112, returns#113, profit#114], 100, 0, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#112, returns#113, profit#114] - -(116) CometColumnarToRow [codegen id : 1] -Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (121) -+- * CometColumnarToRow (120) - +- CometProject (119) - +- CometFilter (118) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) - - -(117) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] -ReadSchema: struct - -(118) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) - -(119) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(120) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(121) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] - -Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 - -Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/extended.txt deleted file mode 100644 index 0d6844d93c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/extended.txt +++ /dev/null @@ -1,392 +0,0 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - -Comet accelerated 374 out of 386 eligible operators (96%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index d27ab827e1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt +++ /dev/null @@ -1,128 +0,0 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [channel,id,sales,returns,profit] - CometHashAggregate [channel,id,sales,returns,profit] - CometExchange [channel,id,sales,returns,profit] #1 - CometHashAggregate [channel,id,sales,returns,profit] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel,id] #2 - CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometUnion [channel,id,sales,returns,profit] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [s_store_id] #3 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #4 - 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 [native_iceberg_compat] 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 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #6 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #7 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [s_store_sk,s_store_id] #8 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk] #9 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #10 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,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)))] - CometExchange [cp_catalog_page_id] #11 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #12 - 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 [native_iceberg_compat] 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 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] - CometExchange [web_site_id] #15 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #16 - 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 [native_iceberg_compat] 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 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #17 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [web_site_sk,web_site_id] #18 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange [channel] #19 - CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] - CometExchange #20 - CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/explain.txt deleted file mode 100644 index 358342baed..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/explain.txt +++ /dev/null @@ -1,267 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (41) -+- * Project (40) - +- Window (39) - +- * CometColumnarToRow (38) - +- CometSort (37) - +- CometExchange (36) - +- CometHashAggregate (35) - +- CometColumnarExchange (34) - +- * HashAggregate (33) - +- Union (32) - :- * HashAggregate (17) - : +- * CometColumnarToRow (16) - : +- CometColumnarExchange (15) - : +- * HashAggregate (14) - : +- * Project (13) - : +- * BroadcastHashJoin Inner BuildRight (12) - : :- * Project (6) - : : +- * BroadcastHashJoin Inner BuildRight (5) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet spark_catalog.default.web_sales (1) - : : +- ReusedExchange (4) - : +- BroadcastExchange (11) - : +- * CometColumnarToRow (10) - : +- CometProject (9) - : +- CometFilter (8) - : +- CometNativeScan parquet spark_catalog.default.item (7) - :- * HashAggregate (24) - : +- * CometColumnarToRow (23) - : +- CometColumnarExchange (22) - : +- * HashAggregate (21) - : +- * HashAggregate (20) - : +- * CometColumnarToRow (19) - : +- ReusedExchange (18) - +- * HashAggregate (31) - +- * CometColumnarToRow (30) - +- CometColumnarExchange (29) - +- * HashAggregate (28) - +- * HashAggregate (27) - +- * CometColumnarToRow (26) - +- ReusedExchange (25) - - -(1) Scan parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(4) ReusedExchange [Reuses operator id: 46] -Output [1]: [d_date_sk#5] - -(5) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] -Join type: Inner -Join condition: None - -(6) Project [codegen id : 3] -Output [2]: [ws_item_sk#1, ws_net_paid#2] -Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] - -(7) CometNativeScan parquet spark_catalog.default.item -Output [3]: [i_item_sk#6, i_class#7, i_category#8] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(8) CometFilter -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Condition : isnotnull(i_item_sk#6) - -(9) CometProject -Input [3]: [i_item_sk#6, i_class#7, i_category#8] -Arguments: [i_item_sk#6, i_class#9, i_category#10], [i_item_sk#6, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#7, 50, true, false, true) AS i_class#9, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#8, 50, true, false, true) AS i_category#10] - -(10) CometColumnarToRow [codegen id : 2] -Input [3]: [i_item_sk#6, i_class#9, i_category#10] - -(11) BroadcastExchange -Input [3]: [i_item_sk#6, i_class#9, i_category#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [3]: [ws_net_paid#2, i_class#9, i_category#10] -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#6, i_class#9, i_category#10] - -(14) HashAggregate [codegen id : 3] -Input [3]: [ws_net_paid#2, i_class#9, i_category#10] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum#11] -Results [3]: [i_category#10, i_class#9, sum#12] - -(15) CometColumnarExchange -Input [3]: [i_category#10, i_class#9, sum#12] -Arguments: hashpartitioning(i_category#10, i_class#9, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 4] -Input [3]: [i_category#10, i_class#9, sum#12] - -(17) HashAggregate [codegen id : 4] -Input [3]: [i_category#10, i_class#9, sum#12] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#13] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#13,17,2) as decimal(27,2)) AS total_sum#14, i_category#10 AS i_category#15, i_class#9 AS i_class#16, 0 AS g_category#17, 0 AS g_class#18, 0 AS lochierarchy#19] - -(18) ReusedExchange [Reuses operator id: 15] -Output [3]: [i_category#10, i_class#9, sum#20] - -(19) CometColumnarToRow [codegen id : 8] -Input [3]: [i_category#10, i_class#9, sum#20] - -(20) HashAggregate [codegen id : 8] -Input [3]: [i_category#10, i_class#9, sum#20] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#21))#13] -Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#21))#13,17,2) AS total_sum#22, i_category#10] - -(21) HashAggregate [codegen id : 8] -Input [2]: [total_sum#22, i_category#10] -Keys [1]: [i_category#10] -Functions [1]: [partial_sum(total_sum#22)] -Aggregate Attributes [2]: [sum#23, isEmpty#24] -Results [3]: [i_category#10, sum#25, isEmpty#26] - -(22) CometColumnarExchange -Input [3]: [i_category#10, sum#25, isEmpty#26] -Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(23) CometColumnarToRow [codegen id : 9] -Input [3]: [i_category#10, sum#25, isEmpty#26] - -(24) HashAggregate [codegen id : 9] -Input [3]: [i_category#10, sum#25, isEmpty#26] -Keys [1]: [i_category#10] -Functions [1]: [sum(total_sum#22)] -Aggregate Attributes [1]: [sum(total_sum#22)#27] -Results [6]: [sum(total_sum#22)#27 AS total_sum#28, i_category#10, null AS i_class#29, 0 AS g_category#30, 1 AS g_class#31, 1 AS lochierarchy#32] - -(25) ReusedExchange [Reuses operator id: 15] -Output [3]: [i_category#10, i_class#9, sum#33] - -(26) CometColumnarToRow [codegen id : 13] -Input [3]: [i_category#10, i_class#9, sum#33] - -(27) HashAggregate [codegen id : 13] -Input [3]: [i_category#10, i_class#9, sum#33] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#13] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#13,17,2) AS total_sum#22] - -(28) HashAggregate [codegen id : 13] -Input [1]: [total_sum#22] -Keys: [] -Functions [1]: [partial_sum(total_sum#22)] -Aggregate Attributes [2]: [sum#35, isEmpty#36] -Results [2]: [sum#37, isEmpty#38] - -(29) CometColumnarExchange -Input [2]: [sum#37, isEmpty#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(30) CometColumnarToRow [codegen id : 14] -Input [2]: [sum#37, isEmpty#38] - -(31) HashAggregate [codegen id : 14] -Input [2]: [sum#37, isEmpty#38] -Keys: [] -Functions [1]: [sum(total_sum#22)] -Aggregate Attributes [1]: [sum(total_sum#22)#39] -Results [6]: [sum(total_sum#22)#39 AS total_sum#40, null AS i_category#41, null AS i_class#42, 1 AS g_category#43, 1 AS g_class#44, 2 AS lochierarchy#45] - -(32) Union - -(33) HashAggregate [codegen id : 15] -Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Keys [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Functions: [] -Aggregate Attributes: [] -Results [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] - -(34) CometColumnarExchange -Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Arguments: hashpartitioning(total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(35) CometHashAggregate -Input [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Keys [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -Functions: [] - -(36) CometExchange -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#46] -Arguments: hashpartitioning(lochierarchy#19, _w0#46, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(37) CometSort -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#46] -Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#46], [lochierarchy#19 ASC NULLS FIRST, _w0#46 ASC NULLS FIRST, total_sum#14 DESC NULLS LAST] - -(38) CometColumnarToRow [codegen id : 16] -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#46] - -(39) Window -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#46] -Arguments: [rank(total_sum#14) windowspecdefinition(lochierarchy#19, _w0#46, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#47], [lochierarchy#19, _w0#46], [total_sum#14 DESC NULLS LAST] - -(40) Project [codegen id : 17] -Output [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#47] -Input [6]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#46, rank_within_parent#47] - -(41) TakeOrderedAndProject -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#47] -Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#47 ASC NULLS FIRST], [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#47] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometNativeScan parquet spark_catalog.default.date_dim (42) - - -(42) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#48] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(43) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#48] -Condition : (((isnotnull(d_month_seq#48) AND (d_month_seq#48 >= 1212)) AND (d_month_seq#48 <= 1223)) AND isnotnull(d_date_sk#5)) - -(44) CometProject -Input [2]: [d_date_sk#5, d_month_seq#48] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(45) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(46) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/simplified.txt deleted file mode 100644 index 2ccc8c0c39..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/simplified.txt +++ /dev/null @@ -1,71 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (17) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (16) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] - CometColumnarExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - WholeStageCodegen (15) - HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (4) - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category,i_class] #3 - WholeStageCodegen (3) - HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] - Project [ws_net_paid,i_class,i_category] - BroadcastHashJoin [ws_item_sk,i_item_sk] - Project [ws_item_sk,ws_net_paid] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] - Filter [ws_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - ReusedExchange [d_date_sk] #4 - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - WholeStageCodegen (9) - HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #6 - WholeStageCodegen (8) - HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 - WholeStageCodegen (14) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #7 - WholeStageCodegen (13) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/explain.txt deleted file mode 100644 index c05e23f926..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,241 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (37) -+- * Project (36) - +- Window (35) - +- * CometColumnarToRow (34) - +- CometSort (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometExchange (30) - +- CometHashAggregate (29) - +- CometUnion (28) - :- CometHashAggregate (17) - : +- CometExchange (16) - : +- CometHashAggregate (15) - : +- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - :- CometHashAggregate (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometHashAggregate (19) - : +- ReusedExchange (18) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometHashAggregate (24) - +- ReusedExchange (23) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] -Arguments: [ws_item_sk#1, ws_net_paid#2], [ws_item_sk#1, ws_net_paid#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#7, i_class#8, i_category#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Condition : isnotnull(i_item_sk#7) - -(11) CometProject -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Arguments: [i_item_sk#7, i_class#10, i_category#11], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#8, 50, true, false, true) AS i_class#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [i_item_sk#7, i_class#10, i_category#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#1, ws_net_paid#2] -Right output [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_net_paid#2, i_class#10, i_category#11], [ws_net_paid#2, i_class#10, i_category#11] - -(15) CometHashAggregate -Input [3]: [ws_net_paid#2, i_class#10, i_category#11] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] - -(16) CometExchange -Input [3]: [i_category#11, i_class#10, sum#12] -Arguments: hashpartitioning(i_category#11, i_class#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [i_category#11, i_class#10, sum#12] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] - -(18) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#11, i_class#10, sum#13] - -(19) CometHashAggregate -Input [3]: [i_category#11, i_class#10, sum#13] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [sum(UnscaledValue(ws_net_paid#14))] - -(20) CometHashAggregate -Input [2]: [total_sum#15, i_category#11] -Keys [1]: [i_category#11] -Functions [1]: [partial_sum(total_sum#15)] - -(21) CometExchange -Input [3]: [i_category#11, sum#16, isEmpty#17] -Arguments: hashpartitioning(i_category#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(22) CometHashAggregate -Input [3]: [i_category#11, sum#16, isEmpty#17] -Keys [1]: [i_category#11] -Functions [1]: [sum(total_sum#15)] - -(23) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#11, i_class#10, sum#18] - -(24) CometHashAggregate -Input [3]: [i_category#11, i_class#10, sum#18] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [sum(UnscaledValue(ws_net_paid#19))] - -(25) CometHashAggregate -Input [1]: [total_sum#15] -Keys: [] -Functions [1]: [partial_sum(total_sum#15)] - -(26) CometExchange -Input [2]: [sum#20, isEmpty#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(27) CometHashAggregate -Input [2]: [sum#20, isEmpty#21] -Keys: [] -Functions [1]: [sum(total_sum#15)] - -(28) CometUnion -Child 0 Input [6]: [total_sum#22, i_category#23, i_class#24, g_category#25, g_class#26, lochierarchy#27] -Child 1 Input [6]: [total_sum#28, i_category#11, i_class#29, g_category#30, g_class#31, lochierarchy#32] -Child 2 Input [6]: [total_sum#33, i_category#34, i_class#35, g_category#36, g_class#37, lochierarchy#38] - -(29) CometHashAggregate -Input [6]: [total_sum#22, i_category#23, i_class#24, g_category#25, g_class#26, lochierarchy#27] -Keys [6]: [total_sum#22, i_category#23, i_class#24, g_category#25, g_class#26, lochierarchy#27] -Functions: [] - -(30) CometExchange -Input [6]: [total_sum#22, i_category#23, i_class#24, g_category#25, g_class#26, lochierarchy#27] -Arguments: hashpartitioning(total_sum#22, i_category#23, i_class#24, g_category#25, g_class#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(31) CometHashAggregate -Input [6]: [total_sum#22, i_category#23, i_class#24, g_category#25, g_class#26, lochierarchy#27] -Keys [6]: [total_sum#22, i_category#23, i_class#24, g_category#25, g_class#26, lochierarchy#27] -Functions: [] - -(32) CometExchange -Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39] -Arguments: hashpartitioning(lochierarchy#27, _w0#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(33) CometSort -Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39] -Arguments: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39], [lochierarchy#27 ASC NULLS FIRST, _w0#39 ASC NULLS FIRST, total_sum#22 DESC NULLS LAST] - -(34) CometColumnarToRow [codegen id : 1] -Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39] - -(35) Window -Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39] -Arguments: [rank(total_sum#22) windowspecdefinition(lochierarchy#27, _w0#39, total_sum#22 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#40], [lochierarchy#27, _w0#39], [total_sum#22 DESC NULLS LAST] - -(36) Project [codegen id : 2] -Output [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] -Input [6]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39, rank_within_parent#40] - -(37) TakeOrderedAndProject -Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] -Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#23 END ASC NULLS FIRST, rank_within_parent#40 ASC NULLS FIRST], [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (42) -+- * CometColumnarToRow (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) - - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(39) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(40) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(41) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(42) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/simplified.txt deleted file mode 100644 index 11e3c03d80..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (2) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] - CometExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometUnion [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometHashAggregate [sum] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(UnscaledValue(ws_net_paid)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ws_net_paid] [i_category,i_class,sum] - CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange [i_category] #7 - CometHashAggregate [total_sum] [i_category,sum,isEmpty] - CometHashAggregate [i_class,sum] [total_sum,i_category,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange #8 - CometHashAggregate [total_sum] [sum,isEmpty] - CometHashAggregate [i_category,i_class,sum] [total_sum,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt deleted file mode 100644 index c05e23f926..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt +++ /dev/null @@ -1,241 +0,0 @@ -== Physical Plan == -TakeOrderedAndProject (37) -+- * Project (36) - +- Window (35) - +- * CometColumnarToRow (34) - +- CometSort (33) - +- CometExchange (32) - +- CometHashAggregate (31) - +- CometExchange (30) - +- CometHashAggregate (29) - +- CometUnion (28) - :- CometHashAggregate (17) - : +- CometExchange (16) - : +- CometHashAggregate (15) - : +- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (9) - :- CometHashAggregate (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometHashAggregate (19) - : +- ReusedExchange (18) - +- CometHashAggregate (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometHashAggregate (24) - +- ReusedExchange (23) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ws_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(4) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(5) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(6) CometBroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: [d_date_sk#5] - -(7) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] -Right output [1]: [d_date_sk#5] -Arguments: [ws_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight - -(8) CometProject -Input [4]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3, d_date_sk#5] -Arguments: [ws_item_sk#1, ws_net_paid#2], [ws_item_sk#1, ws_net_paid#2] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [3]: [i_item_sk#7, i_class#8, i_category#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(10) CometFilter -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Condition : isnotnull(i_item_sk#7) - -(11) CometProject -Input [3]: [i_item_sk#7, i_class#8, i_category#9] -Arguments: [i_item_sk#7, i_class#10, i_category#11], [i_item_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#8, 50, true, false, true) AS i_class#10, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#9, 50, true, false, true) AS i_category#11] - -(12) CometBroadcastExchange -Input [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [i_item_sk#7, i_class#10, i_category#11] - -(13) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#1, ws_net_paid#2] -Right output [3]: [i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_item_sk#1], [i_item_sk#7], Inner, BuildRight - -(14) CometProject -Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#7, i_class#10, i_category#11] -Arguments: [ws_net_paid#2, i_class#10, i_category#11], [ws_net_paid#2, i_class#10, i_category#11] - -(15) CometHashAggregate -Input [3]: [ws_net_paid#2, i_class#10, i_category#11] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] - -(16) CometExchange -Input [3]: [i_category#11, i_class#10, sum#12] -Arguments: hashpartitioning(i_category#11, i_class#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [3]: [i_category#11, i_class#10, sum#12] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] - -(18) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#11, i_class#10, sum#13] - -(19) CometHashAggregate -Input [3]: [i_category#11, i_class#10, sum#13] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [sum(UnscaledValue(ws_net_paid#14))] - -(20) CometHashAggregate -Input [2]: [total_sum#15, i_category#11] -Keys [1]: [i_category#11] -Functions [1]: [partial_sum(total_sum#15)] - -(21) CometExchange -Input [3]: [i_category#11, sum#16, isEmpty#17] -Arguments: hashpartitioning(i_category#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(22) CometHashAggregate -Input [3]: [i_category#11, sum#16, isEmpty#17] -Keys [1]: [i_category#11] -Functions [1]: [sum(total_sum#15)] - -(23) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#11, i_class#10, sum#18] - -(24) CometHashAggregate -Input [3]: [i_category#11, i_class#10, sum#18] -Keys [2]: [i_category#11, i_class#10] -Functions [1]: [sum(UnscaledValue(ws_net_paid#19))] - -(25) CometHashAggregate -Input [1]: [total_sum#15] -Keys: [] -Functions [1]: [partial_sum(total_sum#15)] - -(26) CometExchange -Input [2]: [sum#20, isEmpty#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(27) CometHashAggregate -Input [2]: [sum#20, isEmpty#21] -Keys: [] -Functions [1]: [sum(total_sum#15)] - -(28) CometUnion -Child 0 Input [6]: [total_sum#22, i_category#23, i_class#24, g_category#25, g_class#26, lochierarchy#27] -Child 1 Input [6]: [total_sum#28, i_category#11, i_class#29, g_category#30, g_class#31, lochierarchy#32] -Child 2 Input [6]: [total_sum#33, i_category#34, i_class#35, g_category#36, g_class#37, lochierarchy#38] - -(29) CometHashAggregate -Input [6]: [total_sum#22, i_category#23, i_class#24, g_category#25, g_class#26, lochierarchy#27] -Keys [6]: [total_sum#22, i_category#23, i_class#24, g_category#25, g_class#26, lochierarchy#27] -Functions: [] - -(30) CometExchange -Input [6]: [total_sum#22, i_category#23, i_class#24, g_category#25, g_class#26, lochierarchy#27] -Arguments: hashpartitioning(total_sum#22, i_category#23, i_class#24, g_category#25, g_class#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(31) CometHashAggregate -Input [6]: [total_sum#22, i_category#23, i_class#24, g_category#25, g_class#26, lochierarchy#27] -Keys [6]: [total_sum#22, i_category#23, i_class#24, g_category#25, g_class#26, lochierarchy#27] -Functions: [] - -(32) CometExchange -Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39] -Arguments: hashpartitioning(lochierarchy#27, _w0#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(33) CometSort -Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39] -Arguments: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39], [lochierarchy#27 ASC NULLS FIRST, _w0#39 ASC NULLS FIRST, total_sum#22 DESC NULLS LAST] - -(34) CometColumnarToRow [codegen id : 1] -Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39] - -(35) Window -Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39] -Arguments: [rank(total_sum#22) windowspecdefinition(lochierarchy#27, _w0#39, total_sum#22 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#40], [lochierarchy#27, _w0#39], [total_sum#22 DESC NULLS LAST] - -(36) Project [codegen id : 2] -Output [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] -Input [6]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39, rank_within_parent#40] - -(37) TakeOrderedAndProject -Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] -Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#23 END ASC NULLS FIRST, rank_within_parent#40 ASC NULLS FIRST], [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (42) -+- * CometColumnarToRow (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) - - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#5, d_month_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] -ReadSchema: struct - -(39) CometFilter -Input [2]: [d_date_sk#5, d_month_seq#6] -Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) - -(40) CometProject -Input [2]: [d_date_sk#5, d_month_seq#6] -Arguments: [d_date_sk#5], [d_date_sk#5] - -(41) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#5] - -(42) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/extended.txt deleted file mode 100644 index 46c47555a8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/extended.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 11e3c03d80..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt +++ /dev/null @@ -1,49 +0,0 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (2) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] - CometExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometUnion [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometHashAggregate [sum] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(UnscaledValue(ws_net_paid)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ws_net_paid] [i_category,i_class,sum] - CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - 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 [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange [i_category] #7 - CometHashAggregate [total_sum] [i_category,sum,isEmpty] - CometHashAggregate [i_class,sum] [total_sum,i_category,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange #8 - CometHashAggregate [total_sum] [sum,isEmpty] - CometHashAggregate [i_category,i_class,sum] [total_sum,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/explain.txt deleted file mode 100644 index 646b85d12e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/explain.txt +++ /dev/null @@ -1,172 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometSort (24) - +- CometColumnarExchange (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometColumnarExchange (18) - +- * HashAggregate (17) - +- * CometColumnarToRow (16) - +- CometColumnarExchange (15) - +- * HashAggregate (14) - +- * Project (13) - +- * BroadcastHashJoin Inner BuildRight (12) - :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet spark_catalog.default.store_sales (1) - : +- BroadcastExchange (8) - : +- * CometColumnarToRow (7) - : +- CometProject (6) - : +- CometFilter (5) - : +- CometNativeScan parquet spark_catalog.default.item (4) - +- ReusedExchange (11) - - -(1) Scan parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] - -(3) Filter [codegen id : 3] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(4) CometNativeScan parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(6) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(7) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(8) BroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] - -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#5] -Join type: Inner -Join condition: None - -(10) Project [codegen id : 3] -Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(11) ReusedExchange [Reuses operator id: 30] -Output [1]: [d_date_sk#14] - -(12) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#14] -Join type: Inner -Join condition: None - -(13) Project [codegen id : 3] -Output [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] - -(14) HashAggregate [codegen id : 3] -Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(15) CometColumnarExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(16) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(17) HashAggregate [codegen id : 4] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#17] -Results [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#17,17,2) AS itemrevenue#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#17,17,2) AS _w0#19] - -(18) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 5] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(22) Project [codegen id : 6] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] - -(23) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(24) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 7] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometNativeScan parquet spark_catalog.default.date_dim (26) - - -(26) CometNativeScan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#22] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#14, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-22)) AND (d_date#22 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(28) CometProject -Input [2]: [d_date_sk#14, d_date#22] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(30) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/simplified.txt deleted file mode 100644 index c1ec019e57..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/simplified.txt +++ /dev/null @@ -1,47 +0,0 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (6) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (4) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - WholeStageCodegen (3) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] - Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometNativeScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometNativeScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - InputAdapter - ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/explain.txt deleted file mode 100644 index 96a0ec392e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/explain.txt +++ /dev/null @@ -1,173 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometSort (24) - +- CometColumnarExchange (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] - -(23) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 3] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(28) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(30) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/simplified.txt deleted file mode 100644 index 46191f59cd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/simplified.txt +++ /dev/null @@ -1,39 +0,0 @@ -WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #2 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] 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/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt deleted file mode 100644 index 96a0ec392e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt +++ /dev/null @@ -1,173 +0,0 @@ -== Physical Plan == -* CometColumnarToRow (25) -+- CometSort (24) - +- CometColumnarExchange (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - - -(1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] -PushedFilters: [IsNotNull(ss_item_sk)] -ReadSchema: struct - -(2) CometFilter -Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_item_sk#1) - -(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(4) CometFilter -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) - -(5) CometProject -Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [i_item_sk#5, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_item_id#6, 16, true, false, true) AS i_item_id#11, i_item_desc#7, i_current_price#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_class#9, 50, true, false, true) AS i_class#12, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_category#10, 50, true, false, true) AS i_category#13] - -(6) CometBroadcastExchange -Input [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(7) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Right output [6]: [i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_item_sk#1], [i_item_sk#5], Inner, BuildRight - -(8) CometProject -Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Arguments: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(10) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(11) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(12) CometBroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: [d_date_sk#14] - -(13) CometBroadcastHashJoin -Left output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Right output [1]: [d_date_sk#14] -Arguments: [ss_sold_date_sk#3], [d_date_sk#14], Inner, BuildRight - -(14) CometProject -Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13, d_date_sk#14] -Arguments: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13], [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] - -(15) CometHashAggregate -Input [6]: [ss_ext_sales_price#2, i_item_id#11, i_item_desc#7, i_current_price#8, i_class#12, i_category#13] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] - -(16) CometExchange -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Arguments: hashpartitioning(i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] - -(17) CometHashAggregate -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] -Keys [5]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] - -(18) CometExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(19) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] - -(20) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] - -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] - -(23) CometColumnarExchange -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(24) CometSort -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] - -(25) CometColumnarToRow [codegen id : 3] -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) - - -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#14, d_date#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct - -(27) CometFilter -Input [2]: [d_date_sk#14, d_date#15] -Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) - -(28) CometProject -Input [2]: [d_date_sk#14, d_date#15] -Arguments: [d_date_sk#14], [d_date_sk#14] - -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [d_date_sk#14] - -(30) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] - - diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/extended.txt deleted file mode 100644 index 3dbaf2e346..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/extended.txt +++ /dev/null @@ -1,33 +0,0 @@ -CometColumnarToRow -+- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 deleted file mode 100644 index 46191f59cd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt +++ /dev/null @@ -1,39 +0,0 @@ -WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #2 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - 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,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 [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index 269925be46..bea701d490 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -989,9 +989,27 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { castTest(generateDates(), DataTypes.StringType) } - ignore("cast DateType to TimestampType") { - // Arrow error: Cast error: Casting from Date32 to Timestamp(Microsecond, Some("UTC")) not supported - castTest(generateDates(), DataTypes.TimestampType) + test("cast DateType to TimestampType") { + val compatibleTimezones = Seq( + "UTC", + "America/New_York", + "America/Chicago", + "America/Denver", + "America/Los_Angeles", + "Europe/London", + "Europe/Paris", + "Europe/Berlin", + "Asia/Tokyo", + "Asia/Shanghai", + "Asia/Singapore", + "Asia/Kolkata", + "Australia/Sydney", + "Pacific/Auckland") + compatibleTimezones.map { tz => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> tz) { + castTest(generateDates(), DataTypes.TimestampType) + } + } } // CAST from TimestampType @@ -1264,7 +1282,63 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { } private def generateDates(): DataFrame = { - val values = Seq("2024-01-01", "999-01-01", "12345-01-01") + // add 1st, 10th, 20th of each month from epoch to 2027 + val sampledDates = (1970 to 2027).flatMap { year => + (1 to 12).flatMap { month => + Seq(1, 10, 20).map(day => f"$year-$month%02d-$day%02d") + } + } + + // DST transition dates (1970-2099) for US, EU, Australia + val dstDates = (1970 to 2099).flatMap { year => + Seq( + // spring forward + s"$year-03-08", + s"$year-03-09", + s"$year-03-10", + s"$year-03-11", + s"$year-03-14", + s"$year-03-15", + s"$year-03-25", + s"$year-03-26", + s"$year-03-27", + s"$year-03-28", + s"$year-03-29", + s"$year-03-30", + s"$year-03-31", + // April (Australia fall back) + s"$year-04-01", + s"$year-04-02", + s"$year-04-03", + s"$year-04-04", + s"$year-04-05", + // October (EU fall back and Australia spring forward) + s"$year-10-01", + s"$year-10-02", + s"$year-10-03", + s"$year-10-04", + s"$year-10-05", + s"$year-10-25", + s"$year-10-26", + s"$year-10-27", + s"$year-10-28", + s"$year-10-29", + s"$year-10-30", + s"$year-10-31", + // US fall back + s"$year-11-01", + s"$year-11-02", + s"$year-11-03", + s"$year-11-04", + s"$year-11-05", + s"$year-11-06", + s"$year-11-07", + s"$year-11-08") + } + + // Edge cases + val edgeCases = Seq("1969-12-31", "2000-02-29", "999-01-01", "12345-01-01") + val values = (sampledDates ++ dstDates ++ edgeCases).distinct withNulls(values).toDF("b").withColumn("a", col("b").cast(DataTypes.DateType)).drop("b") } diff --git a/spark/src/test/scala/org/apache/comet/CometCsvExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometCsvExpressionSuite.scala new file mode 100644 index 0000000000..399462d5ea --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/CometCsvExpressionSuite.scala @@ -0,0 +1,198 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet + +import scala.jdk.CollectionConverters._ +import scala.util.Random + +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.CometTestBase +import org.apache.spark.sql.catalyst.expressions.StructsToCsv +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.StringType + +import org.apache.comet.testing.{DataGenOptions, ParquetGenerator, SchemaGenOptions} + +class CometCsvExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { + + test("to_csv - default options") { + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test.parquet") + val filename = path.toString + val random = new Random(42) + withSQLConf(CometConf.COMET_ENABLED.key -> "false") { + ParquetGenerator.makeParquetFile( + random, + spark, + filename, + 100, + SchemaGenOptions(generateArray = false, generateStruct = false, generateMap = false), + DataGenOptions(allowNull = true, generateNegativeZero = true)) + } + withSQLConf(CometConf.getExprAllowIncompatConfigKey(classOf[StructsToCsv]) -> "true") { + val df = spark.read + .parquet(filename) + .select( + to_csv( + struct( + col("c0"), + col("c1"), + col("c2"), + col("c3"), + col("c4"), + col("c5"), + col("c7"), + col("c8"), + col("c9"), + col("c12")))) + checkSparkAnswerAndOperator(df) + } + } + } + + test("to_csv - with configurable formatting options") { + val table = "t1" + withSQLConf( + CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_ICEBERG_COMPAT, + CometConf.getExprAllowIncompatConfigKey(classOf[StructsToCsv]) -> "true") { + withTable(table) { + val newLinesStr = + """ abc + | bcde""".stripMargin + sql(s"create table $table(col string) using parquet") + sql(s"insert into $table values('')") + sql(s"insert into $table values(cast(null as string))") + sql(s"insert into $table values(' abc')") + sql(s"insert into $table values('abc ')") + sql(s"insert into $table values(' abc ')") + sql(s"""insert into $table values('abc \"abc\"')""") + sql(s"""insert into $table values('$newLinesStr')""") + sql(s"""insert into $table values('abc,def')""") + sql(s"""insert into $table values('abc;def;ghi')""") + sql(s"""insert into $table values('abc\tdef')""") + sql(s"""insert into $table values('a"b"c')""") + sql(s"""insert into $table values('"quoted"')""") + sql(s"""insert into $table values('line1\nline2')""") + sql(s"""insert into $table values('line1\rline2')""") + sql(s"""insert into $table values('line1\r\nline2')""") + sql(s"""insert into $table values('a''b')""") + sql(s"""insert into $table values('a\\\\b')""") + + val df = sql(s"select * from $table order by col") + + // Default options + checkSparkAnswerAndOperator(df.select(to_csv(struct(col("col"), lit(1))))) + + // Custom delimiter + checkSparkAnswerAndOperator( + df.select(to_csv(struct(col("col"), lit(1)), Map("delimiter" -> ";").asJava))) + + checkSparkAnswerAndOperator( + df.select(to_csv(struct(col("col"), lit(1)), Map("delimiter" -> "|").asJava))) + + checkSparkAnswerAndOperator( + df.select(to_csv(struct(col("col"), lit(1)), Map("delimiter" -> "\t").asJava))) + + // Whitespace handling + checkSparkAnswerAndOperator( + df.select( + to_csv( + struct(col("col"), lit(1)), + Map( + "delimiter" -> ";", + "ignoreLeadingWhiteSpace" -> "false", + "ignoreTrailingWhiteSpace" -> "false").asJava))) + + checkSparkAnswerAndOperator( + df.select( + to_csv( + struct(col("col"), lit(1)), + Map( + "ignoreLeadingWhiteSpace" -> "true", + "ignoreTrailingWhiteSpace" -> "false").asJava))) + + checkSparkAnswerAndOperator( + df.select( + to_csv( + struct(col("col"), lit(1)), + Map( + "ignoreLeadingWhiteSpace" -> "false", + "ignoreTrailingWhiteSpace" -> "true").asJava))) + + checkSparkAnswerAndOperator(df.select(to_csv( + struct(col("col"), lit(1)), + Map("ignoreLeadingWhiteSpace" -> "true", "ignoreTrailingWhiteSpace" -> "true").asJava))) + + // Escape character + checkSparkAnswerAndOperator( + df.select(to_csv(struct(col("col"), lit(1)), Map("escape" -> "\\").asJava))) + + checkSparkAnswerAndOperator( + df.select(to_csv(struct(col("col"), lit(1)), Map("escape" -> "/").asJava))) + + // Quote options + checkSparkAnswerAndOperator( + df.select(to_csv(struct(col("col"), lit(1)), Map("quoteAll" -> "true").asJava))) + + checkSparkAnswerAndOperator( + df.select(to_csv(struct(col("col"), lit(1)), Map("quoteAll" -> "false").asJava))) + + // Null value representation + checkSparkAnswerAndOperator( + df.select(to_csv(struct(col("col"), lit(1)), Map("nullValue" -> "NULL").asJava))) + + checkSparkAnswerAndOperator( + df.select(to_csv(struct(col("col"), lit(1)), Map("nullValue" -> "N/A").asJava))) + + checkSparkAnswerAndOperator( + df.select(to_csv(struct(col("col"), lit(1)), Map("nullValue" -> "").asJava))) + + // Combined options + checkSparkAnswerAndOperator( + df.select( + to_csv( + struct(col("col"), lit(1)), + Map( + "delimiter" -> "|", + "quoteAll" -> "false", + "escape" -> "\\", + "nullValue" -> "NULL").asJava))) + + checkSparkAnswerAndOperator( + df.select(to_csv( + struct(col("col"), lit(1)), + Map( + "delimiter" -> ";", + "quoteAll" -> "false", + "ignoreLeadingWhiteSpace" -> "true", + "ignoreTrailingWhiteSpace" -> "true", + "nullValue" -> "N/A").asJava))) + + // Edge cases with multiple columns + checkSparkAnswerAndOperator( + df.select( + to_csv( + struct(col("col"), lit(1), lit("test"), lit(null).cast(StringType)), + Map("delimiter" -> ",", "quoteAll" -> "true").asJava))) + } + } + } +} diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 5a22583ae0..1126300452 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -185,7 +185,8 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } - test("basic data type support") { + // ignored: native_comet scan is no longer supported + ignore("basic data type support") { // this test requires native_comet scan due to unsigned u8/u16 issue withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_COMET) { Seq(true, false).foreach { dictionaryEnabled => @@ -216,7 +217,8 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } - test("uint data type support") { + // ignored: native_comet scan is no longer supported + ignore("uint data type support") { // this test requires native_comet scan due to unsigned u8/u16 issue withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_COMET) { Seq(true, false).foreach { dictionaryEnabled => @@ -567,6 +569,77 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } + test("RIGHT function") { + withParquetTable((0 until 10).map(i => (s"test$i", i)), "tbl") { + checkSparkAnswerAndOperator("SELECT _1, RIGHT(_1, 2) FROM tbl") + checkSparkAnswerAndOperator("SELECT _1, RIGHT(_1, 4) FROM tbl") + checkSparkAnswerAndOperator("SELECT _1, RIGHT(_1, 0) FROM tbl") + checkSparkAnswerAndOperator("SELECT _1, RIGHT(_1, -1) FROM tbl") + checkSparkAnswerAndOperator("SELECT _1, RIGHT(_1, 100) FROM tbl") + checkSparkAnswerAndOperator("SELECT RIGHT(CAST(NULL AS STRING), 2) FROM tbl LIMIT 1") + } + } + + test("RIGHT function with unicode") { + val data = Seq("café", "hello世界", "😀emoji", "తెలుగు") + withParquetTable(data.zipWithIndex, "unicode_tbl") { + checkSparkAnswerAndOperator("SELECT _1, RIGHT(_1, 2) FROM unicode_tbl") + checkSparkAnswerAndOperator("SELECT _1, RIGHT(_1, 3) FROM unicode_tbl") + checkSparkAnswerAndOperator("SELECT _1, RIGHT(_1, 0) FROM unicode_tbl") + } + } + + test("RIGHT function equivalence with SUBSTRING negative pos") { + withParquetTable((0 until 20).map(i => Tuple1(s"test$i")), "equiv_tbl") { + val df = spark.sql(""" + SELECT _1, + RIGHT(_1, 3) as right_result, + SUBSTRING(_1, -3, 3) as substring_result + FROM equiv_tbl + """) + checkAnswer( + df.filter( + "right_result != substring_result OR " + + "(right_result IS NULL AND substring_result IS NOT NULL) OR " + + "(right_result IS NOT NULL AND substring_result IS NULL)"), + Seq.empty) + } + } + + test("RIGHT function with dictionary") { + val data = (0 until 1000) + .map(_ % 5) + .map(i => s"value$i") + withParquetTable(data.zipWithIndex, "dict_tbl") { + checkSparkAnswerAndOperator("SELECT _1, RIGHT(_1, 3) FROM dict_tbl") + } + } + + test("RIGHT function NULL handling") { + // Test NULL propagation with len = 0 (critical edge case) + withParquetTable((0 until 5).map(i => (s"test$i", i)), "null_tbl") { + checkSparkAnswerAndOperator("SELECT RIGHT(CAST(NULL AS STRING), 0) FROM null_tbl LIMIT 1") + checkSparkAnswerAndOperator("SELECT RIGHT(CAST(NULL AS STRING), -1) FROM null_tbl LIMIT 1") + checkSparkAnswerAndOperator("SELECT RIGHT(CAST(NULL AS STRING), -5) FROM null_tbl LIMIT 1") + } + + // Test non-NULL strings with len <= 0 (should return empty string) + withParquetTable((0 until 5).map(i => (s"test$i", i)), "edge_tbl") { + checkSparkAnswerAndOperator("SELECT _1, RIGHT(_1, 0) FROM edge_tbl") + checkSparkAnswerAndOperator("SELECT _1, RIGHT(_1, -1) FROM edge_tbl") + } + + // Test mixed NULL and non-NULL values with a table + val table = "right_null_edge" + withTable(table) { + sql(s"create table $table(str string) using parquet") + sql(s"insert into $table values('hello'), (NULL), (''), ('world')") + checkSparkAnswerAndOperator(s"SELECT str, RIGHT(str, 0) FROM $table") + checkSparkAnswerAndOperator(s"SELECT str, RIGHT(str, -1) FROM $table") + checkSparkAnswerAndOperator(s"SELECT str, RIGHT(str, 2) FROM $table") + } + } + test("hour, minute, second") { Seq(true, false).foreach { dictionaryEnabled => withTempDir { dir => @@ -1161,7 +1234,39 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { // Filter rows that contains 'rose' in 'name' column val queryContains = sql(s"select id from $table where contains (name, 'rose')") - checkAnswer(queryContains, Row(5) :: Nil) + checkSparkAnswerAndOperator(queryContains) + + // Additional test cases for optimized contains implementation + // Test with empty pattern (should match all non-null rows) + val queryEmptyPattern = sql(s"select id from $table where contains (name, '')") + checkSparkAnswerAndOperator(queryEmptyPattern) + + // Test with pattern not found + val queryNotFound = sql(s"select id from $table where contains (name, 'xyz')") + checkSparkAnswerAndOperator(queryNotFound) + + // Test with pattern at start + val queryStart = sql(s"select id from $table where contains (name, 'James')") + checkSparkAnswerAndOperator(queryStart) + + // Test with pattern at end + val queryEnd = sql(s"select id from $table where contains (name, 'Smith')") + checkSparkAnswerAndOperator(queryEnd) + + // Test with null haystack + sql(s"insert into $table values(6, null)") + checkSparkAnswerAndOperator(sql(s"select id, contains(name, 'Rose') from $table")) + + // Test case sensitivity (should not match) + checkSparkAnswerAndOperator(sql(s"select id from $table where contains(name, 'james')")) + } + } + + test("contains with both columns") { + withParquetTable( + Seq(("hello world", "world"), ("foo bar", "baz"), ("abc", ""), (null, "x"), ("test", null)), + "tbl") { + checkSparkAnswerAndOperator(sql("select contains(_1, _2) from tbl")) } } @@ -1503,7 +1608,8 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } - test("round") { + // ignored: native_comet scan is no longer supported + ignore("round") { // https://github.com/apache/datafusion-comet/issues/1441 assume(usingLegacyNativeCometScan) Seq(true, false).foreach { dictionaryEnabled => @@ -1567,7 +1673,8 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } - test("hex") { + // ignored: native_comet scan is no longer supported + ignore("hex") { // https://github.com/apache/datafusion-comet/issues/1441 assume(usingLegacyNativeCometScan) Seq(true, false).foreach { dictionaryEnabled => @@ -2781,7 +2888,8 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } - test("test integral divide") { + // ignored: native_comet scan is no longer supported + ignore("test integral divide") { // this test requires native_comet scan due to unsigned u8/u16 issue withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_COMET) { Seq(true, false).foreach { dictionaryEnabled => diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala index f3c8a8b2a6..033b634e0f 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergNativeSuite.scala @@ -25,8 +25,10 @@ import java.nio.file.Files import org.apache.spark.sql.CometTestBase import org.apache.spark.sql.comet.CometIcebergNativeScanExec import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.types.{StringType, TimestampType} import org.apache.comet.iceberg.RESTCatalogHelper +import org.apache.comet.testing.{FuzzDataGenerator, SchemaGenOptions} /** * Test suite for native Iceberg scan using FileScanTasks and iceberg-rust. @@ -2291,10 +2293,89 @@ class CometIcebergNativeSuite extends CometTestBase with RESTCatalogHelper { } file.delete() } + deleteRecursively(dir) } } + test("runtime filtering - multiple DPP filters on two partition columns") { + assume(icebergAvailable, "Iceberg not available") + withTempIcebergDir { warehouseDir => + val dimDir = new File(warehouseDir, "dim_parquet") + withSQLConf( + "spark.sql.catalog.runtime_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.runtime_cat.type" -> "hadoop", + "spark.sql.catalog.runtime_cat.warehouse" -> warehouseDir.getAbsolutePath, + "spark.sql.autoBroadcastJoinThreshold" -> "1KB", + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // Create table partitioned by TWO columns: (data, bucket(8, id)) + // This mimics Iceberg's testMultipleRuntimeFilters + spark.sql(""" + CREATE TABLE runtime_cat.db.multi_dpp_fact ( + id BIGINT, + data STRING, + date DATE, + ts TIMESTAMP + ) USING iceberg + PARTITIONED BY (data, bucket(8, id)) + """) + + // Insert data - 99 rows with varying data and id values + val df = spark + .range(1, 100) + .selectExpr( + "id", + "CAST(DATE_ADD(DATE '1970-01-01', CAST(id % 4 AS INT)) AS STRING) as data", + "DATE_ADD(DATE '1970-01-01', CAST(id % 4 AS INT)) as date", + "CAST(DATE_ADD(DATE '1970-01-01', CAST(id % 4 AS INT)) AS TIMESTAMP) as ts") + df.coalesce(1) + .write + .format("iceberg") + .option("fanout-enabled", "true") + .mode("append") + .saveAsTable("runtime_cat.db.multi_dpp_fact") + + // Create dimension table with specific id=1, data='1970-01-02' + spark + .createDataFrame(Seq((1L, java.sql.Date.valueOf("1970-01-02"), "1970-01-02"))) + .toDF("id", "date", "data") + .write + .parquet(dimDir.getAbsolutePath) + spark.read.parquet(dimDir.getAbsolutePath).createOrReplaceTempView("dim") + + // Join on BOTH partition columns - this creates TWO DPP filters + val query = + """SELECT /*+ BROADCAST(d) */ f.* + |FROM runtime_cat.db.multi_dpp_fact f + |JOIN dim d ON f.id = d.id AND f.data = d.data + |WHERE d.date = DATE '1970-01-02'""".stripMargin + + // Verify plan has 2 dynamic pruning expressions + val df2 = spark.sql(query) + val planStr = df2.queryExecution.executedPlan.toString + // Count "dynamicpruningexpression(" to avoid matching "dynamicpruning#N" references + val dppCount = "dynamicpruningexpression\\(".r.findAllIn(planStr).length + assert(dppCount == 2, s"Expected 2 DPP expressions but found $dppCount in:\n$planStr") + + // Verify native Iceberg scan is used and DPP actually pruned partitions + val (_, cometPlan) = checkSparkAnswer(query) + val icebergScans = collectIcebergNativeScans(cometPlan) + assert( + icebergScans.nonEmpty, + s"Expected CometIcebergNativeScanExec but found none. Plan:\n$cometPlan") + // With 4 data values x 8 buckets = up to 32 partitions total + // DPP on (data='1970-01-02', bucket(id=1)) should prune to 1 + val numPartitions = icebergScans.head.numPartitions + assert(numPartitions == 1, s"Expected DPP to prune to 1 partition but got $numPartitions") + + spark.sql("DROP TABLE runtime_cat.db.multi_dpp_fact") + } + } + } + test("runtime filtering - join with dynamic partition pruning") { assume(icebergAvailable, "Iceberg not available") withTempIcebergDir { warehouseDir => @@ -2303,11 +2384,14 @@ class CometIcebergNativeSuite extends CometTestBase with RESTCatalogHelper { "spark.sql.catalog.runtime_cat" -> "org.apache.iceberg.spark.SparkCatalog", "spark.sql.catalog.runtime_cat.type" -> "hadoop", "spark.sql.catalog.runtime_cat.warehouse" -> warehouseDir.getAbsolutePath, + // Prevent fact table from being broadcast (force dimension to be broadcast) + "spark.sql.autoBroadcastJoinThreshold" -> "1KB", CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - // Create partitioned Iceberg table (fact table) + // Create partitioned Iceberg table (fact table) with 3 partitions + // Add enough data to prevent broadcast spark.sql(""" CREATE TABLE runtime_cat.db.fact_table ( id BIGINT, @@ -2323,7 +2407,11 @@ class CometIcebergNativeSuite extends CometTestBase with RESTCatalogHelper { (1, 'a', DATE '1970-01-01'), (2, 'b', DATE '1970-01-02'), (3, 'c', DATE '1970-01-02'), - (4, 'd', DATE '1970-01-03') + (4, 'd', DATE '1970-01-03'), + (5, 'e', DATE '1970-01-01'), + (6, 'f', DATE '1970-01-02'), + (7, 'g', DATE '1970-01-03'), + (8, 'h', DATE '1970-01-01') """) // Create dimension table (Parquet) in temp directory @@ -2335,8 +2423,9 @@ class CometIcebergNativeSuite extends CometTestBase with RESTCatalogHelper { spark.read.parquet(dimDir.getAbsolutePath).createOrReplaceTempView("dim") // This join should trigger dynamic partition pruning + // Use BROADCAST hint to force dimension table to be broadcast val query = - """SELECT f.* FROM runtime_cat.db.fact_table f + """SELECT /*+ BROADCAST(d) */ f.* FROM runtime_cat.db.fact_table f |JOIN dim d ON f.date = d.date AND d.id = 1 |ORDER BY f.id""".stripMargin @@ -2348,16 +2437,88 @@ class CometIcebergNativeSuite extends CometTestBase with RESTCatalogHelper { planStr.contains("dynamicpruning"), s"Expected dynamic pruning in plan but got:\n$planStr") - // Check results match Spark - // Note: AQE re-plans after subquery executes, converting dynamicpruningexpression(...) - // to dynamicpruningexpression(true), which allows native Iceberg scan to proceed. - // This is correct behavior - no actual subquery to wait for after AQE re-planning. - // However, the rest of the still contains non-native operators because CometExecRule - // doesn't run again. - checkSparkAnswer(df) + // Should now use native Iceberg scan with DPP + checkIcebergNativeScan(query) + + // Verify DPP actually pruned partitions (should only scan 1 of 3 partitions) + val (_, cometPlan) = checkSparkAnswer(query) + val icebergScans = collectIcebergNativeScans(cometPlan) + assert( + icebergScans.nonEmpty, + s"Expected CometIcebergNativeScanExec but found none. Plan:\n$cometPlan") + val numPartitions = icebergScans.head.numPartitions + assert(numPartitions == 1, s"Expected DPP to prune to 1 partition but got $numPartitions") spark.sql("DROP TABLE runtime_cat.db.fact_table") } } } + + // Regression test for a user reported issue + test("double partitioning with range filter on top-level partition") { + assume(icebergAvailable, "Iceberg not available") + + // Generate Iceberg table without Comet enabled + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + "spark.sql.files.maxRecordsPerFile" -> "50") { + + // timestamp + geohash with multi-column partitioning + spark.sql(""" + CREATE TABLE test_cat.db.geolocation_trips ( + outputTimestamp TIMESTAMP, + geohash7 STRING, + tripId STRING + ) USING iceberg + PARTITIONED BY (hours(outputTimestamp), truncate(3, geohash7)) + TBLPROPERTIES ( + 'format-version' = '2', + 'write.distribution-mode' = 'range', + 'write.target-file-size-bytes' = '1073741824' + ) + """) + val schema = FuzzDataGenerator.generateSchema( + SchemaGenOptions(primitiveTypes = Seq(TimestampType, StringType, StringType))) + + val random = new scala.util.Random(42) + // Set baseDate to match our filter range (around 2024-01-01) + val options = testing.DataGenOptions( + allowNull = false, + baseDate = 1704067200000L + ) // 2024-01-01 00:00:00 + + val df = FuzzDataGenerator + .generateDataFrame(random, spark, schema, 1000, options) + .toDF("outputTimestamp", "geohash7", "tripId") + + df.writeTo("test_cat.db.geolocation_trips").append() + } + + // Query using Comet native Iceberg scan + withSQLConf( + "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.test_cat.type" -> "hadoop", + "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // Filter for a range that does not align with hour boundaries + // Partitioning is hours(outputTimestamp), so filter in middle of hours forces residual filter + val startMs = 1704067200000L + 30 * 60 * 1000L // 2024-01-01 01:30:00 (30 min into hour) + val endMs = 1704078000000L - 15 * 60 * 1000L // 2024-01-01 03:45:00 (15 min before hour) + + checkIcebergNativeScan(s""" + SELECT COUNT(DISTINCT(tripId)) FROM test_cat.db.geolocation_trips + WHERE timestamp_millis($startMs) <= outputTimestamp + AND outputTimestamp < timestamp_millis($endMs) + """) + + spark.sql("DROP TABLE test_cat.db.geolocation_trips") + } + } + } } diff --git a/spark/src/test/scala/org/apache/comet/CometSqlFileTestSuite.scala b/spark/src/test/scala/org/apache/comet/CometSqlFileTestSuite.scala index 80ccf92557..4e3b9e0452 100644 --- a/spark/src/test/scala/org/apache/comet/CometSqlFileTestSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometSqlFileTestSuite.scala @@ -77,25 +77,45 @@ class CometSqlFileTestSuite extends CometTestBase with AdaptiveSparkPlanHelper { "spark.sql.optimizer.excludedRules" -> "org.apache.spark.sql.catalyst.optimizer.ConstantFolding") - private def runTestFile(file: SqlTestFile): Unit = { + private def runTestFile(relativePath: String, file: SqlTestFile): Unit = { val allConfigs = file.configs ++ constantFoldingExcluded withSQLConf(allConfigs: _*) { withTable(file.tables: _*) { file.records.foreach { - case SqlStatement(sql) => - spark.sql(sql) - case SqlQuery(sql, mode) => - mode match { - case CheckCoverageAndAnswer => - checkSparkAnswerAndOperator(sql) - case SparkAnswerOnly => - checkSparkAnswer(sql) - case WithTolerance(tol) => - checkSparkAnswerWithTolerance(sql, tol) - case ExpectFallback(reason) => - checkSparkAnswerAndFallbackReason(sql, reason) - case Ignore(reason) => - logInfo(s"IGNORED query (${reason}): $sql") + case SqlStatement(sql, line) => + val location = if (line > 0) s"$relativePath:$line" else relativePath + withClue(s"In SQL file $location, executing statement:\n$sql\n") { + spark.sql(sql) + } + case SqlQuery(sql, mode, line) => + val location = if (line > 0) s"$relativePath:$line" else relativePath + withClue(s"In SQL file $location, executing query:\n$sql\n") { + mode match { + case CheckCoverageAndAnswer => + checkSparkAnswerAndOperator(sql) + case SparkAnswerOnly => + checkSparkAnswer(sql) + case WithTolerance(tol) => + checkSparkAnswerWithTolerance(sql, tol) + case ExpectFallback(reason) => + checkSparkAnswerAndFallbackReason(sql, reason) + case Ignore(reason) => + logInfo(s"IGNORED query (${reason}): $sql") + case ExpectError(pattern) => + val (sparkError, cometError) = checkSparkAnswerMaybeThrows(spark.sql(sql)) + assert( + sparkError.isDefined, + s"Expected Spark to throw an error matching '$pattern' but query succeeded") + assert( + cometError.isDefined, + s"Expected Comet to throw an error matching '$pattern' but query succeeded") + assert( + sparkError.get.getMessage.contains(pattern), + s"Spark error '${sparkError.get.getMessage}' does not contain '$pattern'") + assert( + cometError.get.getMessage.contains(pattern), + s"Comet error '${cometError.get.getMessage}' does not contain '$pattern'") + } } } } @@ -118,7 +138,7 @@ class CometSqlFileTestSuite extends CometTestBase with AdaptiveSparkPlanHelper { logInfo(s"SKIPPED (requires Spark ${parsed.minSparkVersion.get}): $relativePath") } else { val effectiveConfigs = parsed.configs ++ combinations.headOption.getOrElse(Seq.empty) - runTestFile(parsed.copy(configs = effectiveConfigs)) + runTestFile(relativePath, parsed.copy(configs = effectiveConfigs)) } } } else { @@ -129,7 +149,7 @@ class CometSqlFileTestSuite extends CometTestBase with AdaptiveSparkPlanHelper { if (skip) { logInfo(s"SKIPPED (requires Spark ${parsed.minSparkVersion.get}): $relativePath") } else { - runTestFile(parsed.copy(configs = parsed.configs ++ matrixConfigs)) + runTestFile(relativePath, parsed.copy(configs = parsed.configs ++ matrixConfigs)) } } } diff --git a/spark/src/test/scala/org/apache/comet/SqlFileTestParser.scala b/spark/src/test/scala/org/apache/comet/SqlFileTestParser.scala index 62a349cdea..45198ed176 100644 --- a/spark/src/test/scala/org/apache/comet/SqlFileTestParser.scala +++ b/spark/src/test/scala/org/apache/comet/SqlFileTestParser.scala @@ -26,11 +26,27 @@ import scala.io.Source /** A record in a SQL test file: either a statement (DDL/DML) or a query (SELECT). */ sealed trait SqlTestRecord -/** A SQL statement to execute (CREATE TABLE, INSERT, etc.). */ -case class SqlStatement(sql: String) extends SqlTestRecord +/** + * A SQL statement to execute (CREATE TABLE, INSERT, etc.). + * + * @param sql + * The SQL text. + * @param line + * 1-based line number in the original .sql file where the statement starts. + */ +case class SqlStatement(sql: String, line: Int) extends SqlTestRecord -/** A SQL query whose results are compared between Spark and Comet. */ -case class SqlQuery(sql: String, mode: QueryAssertionMode = CheckCoverageAndAnswer) +/** + * A SQL query whose results are compared between Spark and Comet. + * + * @param sql + * The SQL text. + * @param mode + * How to validate the query. + * @param line + * 1-based line number in the original .sql file where the query starts. + */ +case class SqlQuery(sql: String, mode: QueryAssertionMode = CheckCoverageAndAnswer, line: Int) extends SqlTestRecord sealed trait QueryAssertionMode @@ -39,6 +55,7 @@ case object SparkAnswerOnly extends QueryAssertionMode case class WithTolerance(tol: Double) extends QueryAssertionMode case class ExpectFallback(reason: String) extends QueryAssertionMode case class Ignore(reason: String) extends QueryAssertionMode +case class ExpectError(pattern: String) extends QueryAssertionMode /** * Parsed representation of a .sql test file. @@ -103,17 +120,19 @@ object SqlFileTestParser { case "statement" => lineIdx += 1 + val startLine = lineIdx + 1 val (sql, nextIdx) = collectSql(lines, lineIdx) // Extract table names for cleanup CreateTablePattern.findFirstMatchIn(sql).foreach(m => tables += m.group(1)) - records += SqlStatement(sql) + records += SqlStatement(sql, startLine) lineIdx = nextIdx case s if s.startsWith("query") => val mode = parseQueryAssertionMode(s) lineIdx += 1 + val startLine = lineIdx + 1 val (sql, nextIdx) = collectSql(lines, lineIdx) - records += SqlQuery(sql, mode) + records += SqlQuery(sql, mode, startLine) lineIdx = nextIdx case _ => @@ -127,6 +146,7 @@ object SqlFileTestParser { private val FallbackPattern = """query\s+expect_fallback\((.+)\)""".r private val IgnorePattern = """query\s+ignore\((.+)\)""".r + private val ErrorPattern = """query\s+expect_error\((.+)\)""".r private def parseQueryAssertionMode(directive: String): QueryAssertionMode = { directive match { @@ -134,6 +154,8 @@ object SqlFileTestParser { ExpectFallback(reason.trim) case IgnorePattern(reason) => Ignore(reason.trim) + case ErrorPattern(pattern) => + ExpectError(pattern.trim) case _ => val parts = directive.split("\\s+") if (parts.length == 1) return CheckCoverageAndAnswer 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 2d07c03c1b..6c9bdf6eba 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -382,7 +382,8 @@ class CometExecSuite extends CometTestBase { } } - test("ReusedExchangeExec should work on CometBroadcastExchangeExec with V2 scan") { + // ignored: native_comet scan is no longer supported + ignore("ReusedExchangeExec should work on CometBroadcastExchangeExec with V2 scan") { withSQLConf( CometConf.COMET_EXEC_BROADCAST_FORCE_ENABLED.key -> "true", CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_COMET, diff --git a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala index 25008302f9..b691039f19 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala @@ -310,7 +310,8 @@ class CometParquetWriterSuite extends CometTestBase { } } - test("native write falls back when scan produces non-Arrow data") { + // ignored: native_comet scan is no longer supported + ignore("native write falls back when scan produces non-Arrow data") { // This test verifies that when a native scan (like native_comet) doesn't support // certain data types (complex types), the native write correctly falls back to Spark // instead of failing at runtime with "Comet execution only takes Arrow Arrays" error. diff --git a/spark/src/test/scala/org/apache/comet/parquet/ParquetReadSuite.scala b/spark/src/test/scala/org/apache/comet/parquet/ParquetReadSuite.scala index 3da00a2404..928e66b29b 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/ParquetReadSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/ParquetReadSuite.scala @@ -85,7 +85,8 @@ abstract class ParquetReadSuite extends CometTestBase { } } - test("unsupported Spark types") { + // ignored: native_comet scan is no longer supported + ignore("unsupported Spark types") { // TODO this test is not correctly implemented for scan implementations other than SCAN_NATIVE_COMET // https://github.com/apache/datafusion-comet/issues/2188 withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_COMET) { @@ -130,7 +131,8 @@ abstract class ParquetReadSuite extends CometTestBase { } } - test("unsupported Spark schema") { + // ignored: native_comet scan is no longer supported + ignore("unsupported Spark schema") { // TODO this test is not correctly implemented for scan implementations other than SCAN_NATIVE_COMET // https://github.com/apache/datafusion-comet/issues/2188 withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_COMET) { @@ -368,7 +370,8 @@ abstract class ParquetReadSuite extends CometTestBase { checkParquetFile(data) } - test("test multiple pages with different sizes and nulls") { + // ignored: native_comet scan is no longer supported + ignore("test multiple pages with different sizes and nulls") { def makeRawParquetFile( path: Path, dictionaryEnabled: Boolean, @@ -1344,7 +1347,8 @@ abstract class ParquetReadSuite extends CometTestBase { } } - test("scan metrics") { + // ignored: native_comet scan is no longer supported + ignore("scan metrics") { val cometScanMetricNames = Seq( "ParquetRowGroups", @@ -1866,8 +1870,7 @@ class ParquetReadV1Suite extends ParquetReadSuite with AdaptiveSparkPlanHelper { test("Test V1 parquet scan uses respective scanner") { Seq( - ("false", CometConf.SCAN_NATIVE_COMET, "FileScan parquet"), - ("true", CometConf.SCAN_NATIVE_COMET, "CometScan [native_comet] parquet"), + ("false", CometConf.SCAN_NATIVE_DATAFUSION, "FileScan parquet"), ("true", CometConf.SCAN_NATIVE_DATAFUSION, "CometNativeScan"), ("true", CometConf.SCAN_NATIVE_ICEBERG_COMPAT, "CometScan [native_iceberg_compat] parquet")) .foreach { case (cometEnabled, cometNativeScanImpl, expectedScanner) => @@ -2014,10 +2017,11 @@ class ParquetReadV1Suite extends ParquetReadSuite with AdaptiveSparkPlanHelper { } +// ignored: native_comet scan is no longer supported class ParquetReadV2Suite extends ParquetReadSuite with AdaptiveSparkPlanHelper { override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit pos: Position): Unit = { - super.test(testName, testTags: _*)( + super.ignore(testName, testTags: _*)( withSQLConf( SQLConf.USE_V1_SOURCE_LIST.key -> "", CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_COMET) { @@ -2040,7 +2044,8 @@ class ParquetReadV2Suite extends ParquetReadSuite with AdaptiveSparkPlanHelper { } } - test("Test V2 parquet scan uses respective scanner") { + // ignored: native_comet scan is no longer supported + ignore("Test V2 parquet scan uses respective scanner") { Seq(("false", "BatchScan"), ("true", "CometBatchScan")).foreach { case (cometEnabled, expectedScanner) => testScanner( diff --git a/spark/src/test/scala/org/apache/comet/rules/CometScanRuleSuite.scala b/spark/src/test/scala/org/apache/comet/rules/CometScanRuleSuite.scala index c7a07b3f1f..a349ab2b93 100644 --- a/spark/src/test/scala/org/apache/comet/rules/CometScanRuleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/rules/CometScanRuleSuite.scala @@ -101,7 +101,8 @@ class CometScanRuleSuite extends CometTestBase { } } - test("CometScanRule should replace V2 BatchScanExec, but only when Comet is enabled") { + // ignored: native_comet scan is no longer supported + ignore("CometScanRule should replace V2 BatchScanExec, but only when Comet is enabled") { withTempPath { path => createTestDataFrame.write.parquet(path.toString) withTempView("test_data") { diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBenchmarkBase.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBenchmarkBase.scala index 5d1d0c5718..2a81316c95 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBenchmarkBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBenchmarkBase.scala @@ -38,6 +38,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.DecimalType import org.apache.comet.CometConf +import org.apache.comet.CometConf.{SCAN_NATIVE_DATAFUSION, SCAN_NATIVE_ICEBERG_COMPAT} import org.apache.comet.CometSparkSessionExtensions trait CometBenchmarkBase @@ -164,6 +165,32 @@ trait CometBenchmarkBase benchmark.run() } + protected def addParquetScanCases( + benchmark: Benchmark, + query: String, + caseSuffix: String = "", + extraConf: Map[String, String] = Map.empty): Unit = { + val suffix = if (caseSuffix.nonEmpty) s" ($caseSuffix)" else "" + + benchmark.addCase(s"SQL Parquet - Spark$suffix") { _ => + withSQLConf(extraConf.toSeq: _*) { + spark.sql(query).noop() + } + } + + for (scanImpl <- Seq(SCAN_NATIVE_DATAFUSION, SCAN_NATIVE_ICEBERG_COMPAT)) { + benchmark.addCase(s"SQL Parquet - Comet ($scanImpl)$suffix") { _ => + withSQLConf( + (extraConf ++ Map( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_NATIVE_SCAN_IMPL.key -> scanImpl)).toSeq: _*) { + spark.sql(query).noop() + } + } + } + } + protected def prepareTable(dir: File, df: DataFrame, partition: Option[String] = None): Unit = { val testDf = if (partition.isDefined) { df.write.partitionBy(partition.get) diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometCsvExpressionBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometCsvExpressionBenchmark.scala new file mode 100644 index 0000000000..94288eb9cb --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometCsvExpressionBenchmark.scala @@ -0,0 +1,84 @@ +/* + * 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.benchmark + +import org.apache.spark.sql.catalyst.expressions.CsvToStructs + +import org.apache.comet.CometConf + +/** + * Configuration for a CSV expression benchmark. + * + * @param name + * Name for the benchmark + * @param query + * SQL query to benchmark + * @param extraCometConfigs + * Additional Comet configurations for the scan+exec case + */ +case class CsvExprConfig( + name: String, + query: String, + extraCometConfigs: Map[String, String] = Map.empty) + +// spotless:off +/** + * Benchmark to measure performance of Comet CSV expressions. To run this benchmark: + * `SPARK_GENERATE_BENCHMARK_FILES=1 make + * benchmark-org.apache.spark.sql.benchmark.CometCsvExpressionBenchmark` Results will be written + * to "spark/benchmarks/CometCsvExpressionBenchmark-**results.txt". + */ +// spotless:on +object CometCsvExpressionBenchmark extends CometBenchmarkBase { + + /** + * Generic method to run a CSV expression benchmark with the given configuration. + */ + def runCsvExprBenchmark(config: CsvExprConfig, values: Int): Unit = { + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable( + dir, + spark.sql( + s"SELECT CAST(value AS STRING) AS c1, CAST(value AS INT) AS c2, CAST(value AS LONG) AS c3 FROM $tbl")) + + val extraConfigs = Map( + CometConf.getExprAllowIncompatConfigKey( + classOf[CsvToStructs]) -> "true") ++ config.extraCometConfigs + + runExpressionBenchmark(config.name, values, config.query, extraConfigs) + } + } + } + + // Configuration for all CSV expression benchmarks + private val csvExpressions = List( + CsvExprConfig("to_csv", "SELECT to_csv(struct(c1, c2, c3)) FROM parquetV1Table")) + + override def runCometBenchmark(args: Array[String]): Unit = { + val values = 1024 * 1024 + + csvExpressions.foreach { config => + runBenchmarkWithTable(config.name, values) { value => + runCsvExprBenchmark(config, value) + } + } + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometIcebergReadBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometIcebergReadBenchmark.scala new file mode 100644 index 0000000000..b90b893712 --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometIcebergReadBenchmark.scala @@ -0,0 +1,81 @@ +/* + * 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.benchmark + +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.types._ + +import org.apache.comet.CometConf + +/** + * Benchmark to measure Comet Iceberg read performance. To run this benchmark: + * `SPARK_GENERATE_BENCHMARK_FILES=1 make + * benchmark-org.apache.spark.sql.benchmark.CometIcebergReadBenchmark` Results will be written to + * "spark/benchmarks/CometIcebergReadBenchmark-**results.txt". + */ +object CometIcebergReadBenchmark extends CometBenchmarkBase { + + def icebergScanBenchmark(values: Int, dataType: DataType): Unit = { + val sqlBenchmark = + new Benchmark(s"SQL Single ${dataType.sql} Iceberg Column Scan", values, output = output) + + withTempPath { dir => + withTempTable("icebergTable") { + prepareIcebergTable( + dir, + spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM $tbl"), + "icebergTable") + + val query = dataType match { + case BooleanType => "sum(cast(id as bigint))" + case _ => "sum(id)" + } + + sqlBenchmark.addCase("SQL Iceberg - Spark") { _ => + withSQLConf( + "spark.memory.offHeap.enabled" -> "true", + "spark.memory.offHeap.size" -> "10g") { + spark.sql(s"select $query from icebergTable").noop() + } + } + + sqlBenchmark.addCase("SQL Iceberg - Comet Iceberg-Rust") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + "spark.memory.offHeap.enabled" -> "true", + "spark.memory.offHeap.size" -> "10g", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + spark.sql(s"select $query from icebergTable").noop() + } + } + + sqlBenchmark.run() + } + } + } + + override def runCometBenchmark(mainArgs: Array[String]): Unit = { + runBenchmarkWithTable("SQL Single Numeric Iceberg Column Scan", 1024 * 1024 * 128) { v => + Seq(BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType) + .foreach(icebergScanBenchmark(v, _)) + } + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometOperatorSerdeBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometOperatorSerdeBenchmark.scala index 036d526a41..2f3904c5df 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometOperatorSerdeBenchmark.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometOperatorSerdeBenchmark.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.comet.CometConf +import org.apache.comet.rules.CometScanRule import org.apache.comet.serde.OperatorOuterClass import org.apache.comet.serde.operator.CometIcebergNativeScan @@ -301,9 +302,72 @@ object CometOperatorSerdeBenchmark extends CometBenchmarkBase { } } + /** + * Benchmarks CometScanRule.apply() on Iceberg BatchScanExec plans. + * + * This measures the validation overhead when converting Spark Iceberg scans to Comet scans. + */ + def icebergScanRuleBenchmark(numPartitions: Int): Unit = { + if (!icebergAvailable) { + // scalastyle:off println + println("Iceberg not available in classpath, skipping benchmark") + // scalastyle:on println + return + } + + withTempIcebergDir { warehouseDir => + withSQLConf( + "spark.sql.catalog.bench_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.bench_cat.type" -> "hadoop", + "spark.sql.catalog.bench_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // Create the partitioned table + createPartitionedIcebergTable(warehouseDir, numPartitions) + + val fullTableName = "bench_cat.db.serde_bench_table" + + // Get the sparkPlan (before post-hoc rules like CometScanRule) + val df = spark.sql(s"SELECT * FROM $fullTableName") + val sparkPlan = df.queryExecution.sparkPlan + + // scalastyle:off println + println(s"SparkPlan class: ${sparkPlan.getClass.getSimpleName}") + // scalastyle:on println + + val rule = CometScanRule(spark) + val iterations = 100 + + val benchmark = new Benchmark( + s"CometScanRule apply ($numPartitions partitions)", + iterations, + output = output) + + benchmark.addCase("CometScanRule.apply(sparkPlan)") { _ => + var i = 0 + while (i < iterations) { + rule.apply(sparkPlan) + i += 1 + } + } + + benchmark.run() + + // Cleanup + spark.sql(s"DROP TABLE IF EXISTS $fullTableName") + } + } + } + override def runCometBenchmark(args: Array[String]): Unit = { val numPartitions = if (args.nonEmpty) args(0).toInt else 30000 + runBenchmark("CometScanRule Benchmark") { + icebergScanRuleBenchmark(numPartitions) + } + runBenchmark("IcebergScan Operator Serde Benchmark") { icebergScanSerdeBenchmark(numPartitions) } diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometPartitionColumnBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometPartitionColumnBenchmark.scala new file mode 100644 index 0000000000..a7d170057f --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometPartitionColumnBenchmark.scala @@ -0,0 +1,82 @@ +/* + * 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.benchmark + +import org.apache.spark.benchmark.Benchmark + +/** + * Benchmark to measure partition column scan performance. This exercises the CometConstantVector + * path where constant columns are exported as 1-element Arrow arrays and expanded on the native + * side. + * + * To run this benchmark: + * {{{ + * SPARK_GENERATE_BENCHMARK_FILES=1 make \ + * benchmark-org.apache.spark.sql.benchmark.CometPartitionColumnBenchmark + * }}} + * + * Results will be written to "spark/benchmarks/CometPartitionColumnBenchmark-**results.txt". + */ +object CometPartitionColumnBenchmark extends CometBenchmarkBase { + + def partitionColumnScanBenchmark(values: Int, numPartitionCols: Int): Unit = { + val sqlBenchmark = new Benchmark( + s"Partitioned Scan with $numPartitionCols partition column(s)", + values, + output = output) + + withTempPath { dir => + withTempTable("parquetV1Table") { + val partCols = + (1 to numPartitionCols).map(i => s"'part$i' as p$i").mkString(", ") + val partNames = (1 to numPartitionCols).map(i => s"p$i") + val df = spark.sql(s"SELECT value as id, $partCols FROM $tbl") + val parquetDir = dir.getCanonicalPath + "/parquetV1" + df.write + .partitionBy(partNames: _*) + .mode("overwrite") + .option("compression", "snappy") + .parquet(parquetDir) + spark.read.parquet(parquetDir).createOrReplaceTempView("parquetV1Table") + + addParquetScanCases(sqlBenchmark, "select sum(id) from parquetV1Table") + + // Also benchmark reading partition columns themselves + val partSumExpr = + (1 to numPartitionCols).map(i => s"sum(length(p$i))").mkString(", ") + + addParquetScanCases( + sqlBenchmark, + s"select $partSumExpr from parquetV1Table", + caseSuffix = "partition cols") + + sqlBenchmark.run() + } + } + } + + override def runCometBenchmark(mainArgs: Array[String]): Unit = { + runBenchmarkWithTable("Partitioned Column Scan", 1024 * 1024 * 15) { v => + for (numPartCols <- List(1, 5)) { + partitionColumnScanBenchmark(v, numPartCols) + } + } + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometReadBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometReadBenchmark.scala index 9b2dd186dd..a2f196a4fc 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometReadBenchmark.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometReadBenchmark.scala @@ -38,7 +38,6 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnVector import org.apache.comet.{CometConf, WithHdfsCluster} -import org.apache.comet.CometConf.{SCAN_NATIVE_COMET, SCAN_NATIVE_DATAFUSION, SCAN_NATIVE_ICEBERG_COMPAT} import org.apache.comet.parquet.BatchReader /** @@ -50,7 +49,6 @@ import org.apache.comet.parquet.BatchReader class CometReadBaseBenchmark extends CometBenchmarkBase { def numericScanBenchmark(values: Int, dataType: DataType): Unit = { - // Benchmarks running through spark sql. val sqlBenchmark = new Benchmark(s"SQL Single ${dataType.sql} Column Scan", values, output = output) @@ -63,84 +61,13 @@ class CometReadBaseBenchmark extends CometBenchmarkBase { case _ => "sum(id)" } - sqlBenchmark.addCase("SQL Parquet - Spark") { _ => - spark.sql(s"select $query from parquetV1Table").noop() - } - - sqlBenchmark.addCase("SQL Parquet - Comet") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_COMET) { - spark.sql(s"select $query from parquetV1Table").noop() - } - } - - sqlBenchmark.addCase("SQL Parquet - Comet Native DataFusion") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_DATAFUSION) { - spark.sql(s"select $query from parquetV1Table").noop() - } - } - - sqlBenchmark.addCase("SQL Parquet - Comet Native Iceberg Compat") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_ICEBERG_COMPAT) { - spark.sql(s"select $query from parquetV1Table").noop() - } - } - - sqlBenchmark.run() - } - } - } - - def icebergScanBenchmark(values: Int, dataType: DataType): Unit = { - // Benchmarks running through spark sql. - val sqlBenchmark = - new Benchmark(s"SQL Single ${dataType.sql} Iceberg Column Scan", values, output = output) - - withTempPath { dir => - withTempTable("icebergTable") { - prepareIcebergTable( - dir, - spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM $tbl"), - "icebergTable") - - val query = dataType match { - case BooleanType => "sum(cast(id as bigint))" - case _ => "sum(id)" - } - - sqlBenchmark.addCase("SQL Iceberg - Spark") { _ => - withSQLConf( - "spark.memory.offHeap.enabled" -> "true", - "spark.memory.offHeap.size" -> "10g") { - spark.sql(s"select $query from icebergTable").noop() - } - } - - sqlBenchmark.addCase("SQL Iceberg - Comet Iceberg-Rust") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - "spark.memory.offHeap.enabled" -> "true", - "spark.memory.offHeap.size" -> "10g", - CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - spark.sql(s"select $query from icebergTable").noop() - } - } - + addParquetScanCases(sqlBenchmark, s"select $query from parquetV1Table") sqlBenchmark.run() } } } def encryptedScanBenchmark(values: Int, dataType: DataType): Unit = { - // Benchmarks running through spark sql. val sqlBenchmark = new Benchmark(s"SQL Single ${dataType.sql} Encrypted Column Scan", values, output = output) @@ -151,6 +78,15 @@ class CometReadBaseBenchmark extends CometBenchmarkBase { val cryptoFactoryClass = "org.apache.parquet.crypto.keytools.PropertiesDrivenCryptoFactory" + val cryptoConf = Map( + "spark.memory.offHeap.enabled" -> "true", + "spark.memory.offHeap.size" -> "10g", + DecryptionPropertiesFactory.CRYPTO_FACTORY_CLASS_PROPERTY_NAME -> cryptoFactoryClass, + KeyToolkit.KMS_CLIENT_CLASS_PROPERTY_NAME -> + "org.apache.parquet.crypto.keytools.mocks.InMemoryKMS", + InMemoryKMS.KEY_LIST_PROPERTY_NAME -> + s"footerKey: ${footerKey}, key1: ${key1}") + withTempPath { dir => withTempTable("parquetV1Table") { prepareEncryptedTable( @@ -162,66 +98,10 @@ class CometReadBaseBenchmark extends CometBenchmarkBase { case _ => "sum(id)" } - sqlBenchmark.addCase("SQL Parquet - Spark") { _ => - withSQLConf( - "spark.memory.offHeap.enabled" -> "true", - "spark.memory.offHeap.size" -> "10g", - DecryptionPropertiesFactory.CRYPTO_FACTORY_CLASS_PROPERTY_NAME -> cryptoFactoryClass, - KeyToolkit.KMS_CLIENT_CLASS_PROPERTY_NAME -> - "org.apache.parquet.crypto.keytools.mocks.InMemoryKMS", - InMemoryKMS.KEY_LIST_PROPERTY_NAME -> - s"footerKey: ${footerKey}, key1: ${key1}") { - spark.sql(s"select $query from parquetV1Table").noop() - } - } - - sqlBenchmark.addCase("SQL Parquet - Comet") { _ => - withSQLConf( - "spark.memory.offHeap.enabled" -> "true", - "spark.memory.offHeap.size" -> "10g", - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_COMET, - DecryptionPropertiesFactory.CRYPTO_FACTORY_CLASS_PROPERTY_NAME -> cryptoFactoryClass, - KeyToolkit.KMS_CLIENT_CLASS_PROPERTY_NAME -> - "org.apache.parquet.crypto.keytools.mocks.InMemoryKMS", - InMemoryKMS.KEY_LIST_PROPERTY_NAME -> - s"footerKey: ${footerKey}, key1: ${key1}") { - spark.sql(s"select $query from parquetV1Table").noop() - } - } - - sqlBenchmark.addCase("SQL Parquet - Comet Native DataFusion") { _ => - withSQLConf( - "spark.memory.offHeap.enabled" -> "true", - "spark.memory.offHeap.size" -> "10g", - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_DATAFUSION, - DecryptionPropertiesFactory.CRYPTO_FACTORY_CLASS_PROPERTY_NAME -> cryptoFactoryClass, - KeyToolkit.KMS_CLIENT_CLASS_PROPERTY_NAME -> - "org.apache.parquet.crypto.keytools.mocks.InMemoryKMS", - InMemoryKMS.KEY_LIST_PROPERTY_NAME -> - s"footerKey: ${footerKey}, key1: ${key1}") { - spark.sql(s"select $query from parquetV1Table").noop() - } - } - - sqlBenchmark.addCase("SQL Parquet - Comet Native Iceberg Compat") { _ => - withSQLConf( - "spark.memory.offHeap.enabled" -> "true", - "spark.memory.offHeap.size" -> "10g", - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_ICEBERG_COMPAT, - DecryptionPropertiesFactory.CRYPTO_FACTORY_CLASS_PROPERTY_NAME -> cryptoFactoryClass, - KeyToolkit.KMS_CLIENT_CLASS_PROPERTY_NAME -> - "org.apache.parquet.crypto.keytools.mocks.InMemoryKMS", - InMemoryKMS.KEY_LIST_PROPERTY_NAME -> - s"footerKey: ${footerKey}, key1: ${key1}") { - spark.sql(s"select $query from parquetV1Table").noop() - } - } - + addParquetScanCases( + sqlBenchmark, + s"select $query from parquetV1Table", + extraConf = cryptoConf) sqlBenchmark.run() } } @@ -241,36 +121,7 @@ class CometReadBaseBenchmark extends CometBenchmarkBase { s"SELECT CAST(value / 10000000.0 as DECIMAL($precision, $scale)) " + s"id FROM $tbl")) - sqlBenchmark.addCase("SQL Parquet - Spark") { _ => - spark.sql("select sum(id) from parquetV1Table").noop() - } - - sqlBenchmark.addCase("SQL Parquet - Comet") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_COMET) { - spark.sql("select sum(id) from parquetV1Table").noop() - } - } - - sqlBenchmark.addCase("SQL Parquet - Comet Native DataFusion") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_DATAFUSION) { - spark.sql("select sum(id) from parquetV1Table").noop() - } - } - - sqlBenchmark.addCase("SQL Parquet - Comet Native Iceberg Compat") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_ICEBERG_COMPAT) { - spark.sql("select sum(id) from parquetV1Table").noop() - } - } - + addParquetScanCases(sqlBenchmark, "select sum(id) from parquetV1Table") sqlBenchmark.run() } } @@ -369,36 +220,7 @@ class CometReadBaseBenchmark extends CometBenchmarkBase { s"SELECT IF(RAND(1) < $fractionOfZeros, -1, value) AS c1, value AS c2 FROM " + s"$tbl")) - benchmark.addCase("SQL Parquet - Spark") { _ => - spark.sql("select sum(c2) from parquetV1Table where c1 + 1 > 0").noop() - } - - benchmark.addCase("SQL Parquet - Comet") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_COMET) { - spark.sql("select sum(c2) from parquetV1Table where c1 + 1 > 0").noop() - } - } - - benchmark.addCase("SQL Parquet - Comet Native DataFusion") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_DATAFUSION) { - spark.sql("select sum(c2) from parquetV1Table where c1 + 1 > 0").noop() - } - } - - benchmark.addCase("SQL Parquet - Comet Native Iceberg Compat") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_ICEBERG_COMPAT) { - spark.sql("select sum(c2) from parquetV1Table where c1 + 1 > 0").noop() - } - } - + addParquetScanCases(benchmark, "select sum(c2) from parquetV1Table where c1 + 1 > 0") benchmark.run() } } @@ -427,36 +249,7 @@ class CometReadBaseBenchmark extends CometBenchmarkBase { |FROM tmp |""".stripMargin)) - sqlBenchmark.addCase("SQL Parquet - Spark") { _ => - spark.sql("select sum(length(id)) from parquetV1Table").noop() - } - - sqlBenchmark.addCase("SQL Parquet - Comet") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_COMET) { - spark.sql("select sum(length(id)) from parquetV1Table").noop() - } - } - - sqlBenchmark.addCase("SQL Parquet - Comet Native DataFusion") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_DATAFUSION) { - spark.sql("select sum(length(id)) from parquetV1Table").noop() - } - } - - sqlBenchmark.addCase("SQL Parquet - Comet Native Iceberg Compat") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_ICEBERG_COMPAT) { - spark.sql("select sum(length(id)) from parquetV1Table").noop() - } - } - + addParquetScanCases(sqlBenchmark, "select sum(length(id)) from parquetV1Table") sqlBenchmark.run() } } @@ -475,48 +268,10 @@ class CometReadBaseBenchmark extends CometBenchmarkBase { s"SELECT IF(RAND(1) < $fractionOfNulls, NULL, CAST(value as STRING)) AS c1, " + s"IF(RAND(2) < $fractionOfNulls, NULL, CAST(value as STRING)) AS c2 FROM $tbl")) - benchmark.addCase("SQL Parquet - Spark") { _ => - spark - .sql("select sum(length(c2)) from parquetV1Table where c1 is " + - "not NULL and c2 is not NULL") - .noop() - } - - benchmark.addCase("SQL Parquet - Comet") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_COMET) { - spark - .sql("select sum(length(c2)) from parquetV1Table where c1 is " + - "not NULL and c2 is not NULL") - .noop() - } - } - - benchmark.addCase("SQL Parquet - Comet Native DataFusion") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_DATAFUSION) { - spark - .sql("select sum(length(c2)) from parquetV1Table where c1 is " + - "not NULL and c2 is not NULL") - .noop() - } - } - - benchmark.addCase("SQL Parquet - Comet Native Iceberg Compat") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_ICEBERG_COMPAT) { - spark - .sql("select sum(length(c2)) from parquetV1Table where c1 is " + - "not NULL and c2 is not NULL") - .noop() - } - } - + addParquetScanCases( + benchmark, + "select sum(length(c2)) from parquetV1Table where c1 is " + + "not NULL and c2 is not NULL") benchmark.run() } } @@ -534,36 +289,7 @@ class CometReadBaseBenchmark extends CometBenchmarkBase { prepareTable(dir, spark.sql("SELECT * FROM t1")) - benchmark.addCase("SQL Parquet - Spark") { _ => - spark.sql(s"SELECT sum(c$middle) FROM parquetV1Table").noop() - } - - benchmark.addCase("SQL Parquet - Comet") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_COMET) { - spark.sql(s"SELECT sum(c$middle) FROM parquetV1Table").noop() - } - } - - benchmark.addCase("SQL Parquet - Comet Native DataFusion") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_DATAFUSION) { - spark.sql(s"SELECT sum(c$middle) FROM parquetV1Table").noop() - } - } - - benchmark.addCase("SQL Parquet - Comet Native Iceberg Compat") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_ICEBERG_COMPAT) { - spark.sql(s"SELECT sum(c$middle) FROM parquetV1Table").noop() - } - } - + addParquetScanCases(benchmark, s"SELECT sum(c$middle) FROM parquetV1Table") benchmark.run() } } @@ -585,36 +311,7 @@ class CometReadBaseBenchmark extends CometBenchmarkBase { s"SELECT IF(RAND(1) < $fractionOfZeros, -1, value) AS c1, " + s"REPEAT(CAST(value AS STRING), 100) AS c2 FROM $tbl")) - benchmark.addCase("SQL Parquet - Spark") { _ => - spark.sql("SELECT * FROM parquetV1Table WHERE c1 + 1 > 0").noop() - } - - benchmark.addCase("SQL Parquet - Comet") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_COMET) { - spark.sql("SELECT * FROM parquetV1Table WHERE c1 + 1 > 0").noop() - } - } - - benchmark.addCase("SQL Parquet - Comet Native DataFusion") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_DATAFUSION) { - spark.sql("SELECT * FROM parquetV1Table WHERE c1 + 1 > 0").noop() - } - } - - benchmark.addCase("SQL Parquet - Comet Native Iceberg Compat") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_ICEBERG_COMPAT) { - spark.sql("SELECT * FROM parquetV1Table WHERE c1 + 1 > 0").noop() - } - } - + addParquetScanCases(benchmark, "SELECT * FROM parquetV1Table WHERE c1 + 1 > 0") benchmark.run() } } @@ -636,36 +333,7 @@ class CometReadBaseBenchmark extends CometBenchmarkBase { s"SELECT IF(RAND(1) < $fractionOfZeros, -1, value) AS c1, " + s"REPEAT(CAST(value AS STRING), 100) AS c2 FROM $tbl ORDER BY c1, c2")) - benchmark.addCase("SQL Parquet - Spark") { _ => - spark.sql("SELECT * FROM parquetV1Table WHERE c1 + 1 > 0").noop() - } - - benchmark.addCase("SQL Parquet - Comet") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_COMET) { - spark.sql("SELECT * FROM parquetV1Table WHERE c1 + 1 > 0").noop() - } - } - - benchmark.addCase("SQL Parquet - Comet Native DataFusion") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_DATAFUSION) { - spark.sql("SELECT * FROM parquetV1Table WHERE c1 + 1 > 0").noop() - } - } - - benchmark.addCase("SQL Parquet - Comet Native Iceberg Compat") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_ICEBERG_COMPAT) { - spark.sql("SELECT * FROM parquetV1Table WHERE c1 + 1 > 0").noop() - } - } - + addParquetScanCases(benchmark, "SELECT * FROM parquetV1Table WHERE c1 + 1 > 0") benchmark.run() } } @@ -693,13 +361,6 @@ class CometReadBaseBenchmark extends CometBenchmarkBase { } } - runBenchmarkWithTable("SQL Single Numeric Iceberg Column Scan", 1024 * 1024 * 128) { v => - Seq(BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType) - .foreach { dataType => - icebergScanBenchmark(v, dataType) - } - } - runBenchmarkWithTable("SQL Single Numeric Encrypted Column Scan", 1024 * 1024 * 128) { v => Seq(BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType) .foreach { dataType => diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala index adf74ba549..69e52bd31a 100644 --- a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.internal.config.{MEMORY_OFFHEAP_ENABLED, MEMORY_OFFHEAP_ import org.apache.spark.sql.TPCDSBase import org.apache.spark.sql.catalyst.expressions.{AttributeSet, Cast} import org.apache.spark.sql.catalyst.util.resourceToString -import org.apache.spark.sql.execution.{FormattedMode, ReusedSubqueryExec, SparkPlan, SubqueryBroadcastExec, SubqueryExec} +import org.apache.spark.sql.execution.{ReusedSubqueryExec, SparkPlan, SubqueryBroadcastExec, SubqueryExec} import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec, ValidateRequirements} import org.apache.spark.sql.internal.SQLConf @@ -62,10 +62,8 @@ import org.apache.comet.CometSparkSessionExtensions.{isSpark35Plus, isSpark40Plu * }}} */ trait CometPlanStabilitySuite extends DisableAdaptiveExecutionSuite with TPCDSBase { - protected val scanImpls: Seq[String] = Seq( - CometConf.SCAN_AUTO, - CometConf.SCAN_NATIVE_ICEBERG_COMPAT, - CometConf.SCAN_NATIVE_DATAFUSION) + protected val scanImpls: Seq[String] = + Seq(CometConf.SCAN_NATIVE_ICEBERG_COMPAT, CometConf.SCAN_NATIVE_DATAFUSION) protected val baseResourcePath: File = { getWorkspaceFilePath("spark", "src", "test", "resources", "tpcds-plan-stability").toFile @@ -88,9 +86,7 @@ trait CometPlanStabilitySuite extends DisableAdaptiveExecutionSuite with TPCDSBa name } val nativeImpl = CometConf.COMET_NATIVE_SCAN_IMPL.get() - if (nativeImpl != CometConf.SCAN_AUTO) { - goldenFileName = s"$goldenFileName.$nativeImpl" - } + goldenFileName = s"$goldenFileName.$nativeImpl" new File(goldenFilePath, goldenFileName) } @@ -230,10 +226,8 @@ trait CometPlanStabilitySuite extends DisableAdaptiveExecutionSuite with TPCDSBa SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10MB") { val qe = sql(queryString).queryExecution val plan = qe.executedPlan - val explain = normalizeLocation(normalizeIds(qe.explainString(FormattedMode))) val extendedExplain = new ExtendedExplainInfo().generateExtendedInfo(qe.executedPlan) val extended = normalizeLocation(normalizeIds(extendedExplain)) - val simplified = getSimplifiedPlan(plan) assert(ValidateRequirements.validate(plan)) val name = query + suffix @@ -244,12 +238,8 @@ trait CometPlanStabilitySuite extends DisableAdaptiveExecutionSuite with TPCDSBa if (!dir.mkdirs()) { fail(s"Could not create dir: $dir") } - writeGoldenFile(dir, "simplified", simplified) - writeGoldenFile(dir, "explain", explain) writeGoldenFile(dir, "extended", extended) } else { - checkWithApproved(dir, name, "simplified", simplified) - checkWithApproved(dir, name, "explain", explain) checkWithApproved(dir, name, "extended", extended) } } diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/ParquetDatetimeRebaseSuite.scala b/spark/src/test/scala/org/apache/spark/sql/comet/ParquetDatetimeRebaseSuite.scala index 131423ddeb..c330bbe4c3 100644 --- a/spark/src/test/scala/org/apache/spark/sql/comet/ParquetDatetimeRebaseSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/comet/ParquetDatetimeRebaseSuite.scala @@ -37,7 +37,8 @@ abstract class ParquetDatetimeRebaseSuite extends CometTestBase { // visible under package `spark`. val SPARK_TESTING: String = "spark.testing" - test("reading ancient dates before 1582") { + // ignored: native_comet scan is no longer supported + ignore("reading ancient dates before 1582") { Seq(true, false).foreach { exceptionOnRebase => withSQLConf( CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_COMET, @@ -62,7 +63,8 @@ abstract class ParquetDatetimeRebaseSuite extends CometTestBase { } } - test("reading ancient timestamps before 1582") { + // ignored: native_comet scan is no longer supported + ignore("reading ancient timestamps before 1582") { assume(usingLegacyNativeCometScan(conf)) Seq(true, false).foreach { exceptionOnRebase => withSQLConf( @@ -89,7 +91,8 @@ abstract class ParquetDatetimeRebaseSuite extends CometTestBase { } } - test("reading ancient int96 timestamps before 1582") { + // ignored: native_comet scan is no longer supported + ignore("reading ancient int96 timestamps before 1582") { assume(usingLegacyNativeCometScan(conf)) Seq(true, false).foreach { exceptionOnRebase => withSQLConf( @@ -147,12 +150,11 @@ class ParquetDatetimeRebaseV1Suite extends ParquetDatetimeRebaseSuite { } } +// ignored: native_comet scan is no longer supported class ParquetDatetimeRebaseV2Suite extends ParquetDatetimeRebaseSuite { override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit pos: Position): Unit = { - // Datasource V2 is not supported by the native (datafusion based) readers so force - // the scan impl back to 'native_comet' - super.test(testName, testTags: _*)( + super.ignore(testName, testTags: _*)( withSQLConf( SQLConf.USE_V1_SOURCE_LIST.key -> "", CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_COMET) { diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/ParquetEncryptionITCase.scala b/spark/src/test/scala/org/apache/spark/sql/comet/ParquetEncryptionITCase.scala index b3e6a5a42a..db07b91e93 100644 --- a/spark/src/test/scala/org/apache/spark/sql/comet/ParquetEncryptionITCase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/comet/ParquetEncryptionITCase.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils import org.apache.comet.{CometConf, IntegrationTestSuite} -import org.apache.comet.CometConf.{SCAN_NATIVE_COMET, SCAN_NATIVE_DATAFUSION, SCAN_NATIVE_ICEBERG_COMPAT} +import org.apache.comet.CometConf.{SCAN_NATIVE_DATAFUSION, SCAN_NATIVE_ICEBERG_COMPAT} /** * A integration test suite that tests parquet modular encryption usage. @@ -238,12 +238,7 @@ class ParquetEncryptionITCase extends CometTestBase with SQLTestUtils { // native_datafusion and native_iceberg_compat fall back due to Arrow-rs // https://github.com/apache/arrow-rs/blob/da9829728e2a9dffb8d4f47ffe7b103793851724/parquet/src/file/metadata/parser.rs#L494 - if (CometConf.COMET_ENABLED.get(conf) && CometConf.COMET_NATIVE_SCAN_IMPL.get( - conf) == SCAN_NATIVE_COMET) { - checkSparkAnswerAndOperator(readDataset) - } else { - checkAnswer(readDataset, inputDF) - } + checkAnswer(readDataset, inputDF) } } } @@ -442,12 +437,7 @@ class ParquetEncryptionITCase extends CometTestBase with SQLTestUtils { // native_datafusion and native_iceberg_compat fall back due to Arrow-rs not // supporting other key lengths - if (CometConf.COMET_ENABLED.get(conf) && CometConf.COMET_NATIVE_SCAN_IMPL.get( - conf) == SCAN_NATIVE_COMET) { - checkSparkAnswerAndOperator(readDataset) - } else { - checkAnswer(readDataset, inputDF) - } + checkAnswer(readDataset, inputDF) } } } @@ -467,17 +457,16 @@ class ParquetEncryptionITCase extends CometTestBase with SQLTestUtils { Seq("true", "false").foreach { cometEnabled => if (cometEnabled == "true") { - Seq(SCAN_NATIVE_COMET, SCAN_NATIVE_DATAFUSION, SCAN_NATIVE_ICEBERG_COMPAT).foreach { - scanImpl => - super.test(testName + s" Comet($cometEnabled)" + s" Scan($scanImpl)", testTags: _*) { - withSQLConf( - CometConf.COMET_ENABLED.key -> cometEnabled, - CometConf.COMET_EXEC_ENABLED.key -> "true", - SQLConf.ANSI_ENABLED.key -> "false", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> scanImpl) { - testFun - } + Seq(SCAN_NATIVE_DATAFUSION, SCAN_NATIVE_ICEBERG_COMPAT).foreach { scanImpl => + super.test(testName + s" Comet($cometEnabled)" + s" Scan($scanImpl)", testTags: _*) { + withSQLConf( + CometConf.COMET_ENABLED.key -> cometEnabled, + CometConf.COMET_EXEC_ENABLED.key -> "true", + SQLConf.ANSI_ENABLED.key -> "false", + CometConf.COMET_NATIVE_SCAN_IMPL.key -> scanImpl) { + testFun } + } } } else { super.test(testName + s" Comet($cometEnabled)", testTags: _*) {